From ab2b89c4c339bd7f816fbc114a4b05d386b66290 Mon Sep 17 00:00:00 2001 From: Sylvain Jeaugey Date: Tue, 26 Mar 2024 06:08:55 -0700 Subject: [PATCH] 2.21.5-1 Add support for IB SHARP 1PPN operation with user buffers. Improve support for MNNVL, add NVLS support and multi-clique support. * Detect the NVLS clique through NVML * Exchange XML between peers in the same NVLS clique and fuse XMLs before creating the topology graph. * Rework bootstrap allgather algorithms to allow for large allgather operations intra-node (XML exchange). Net/IB: add support for dynamic GID detection. * Automatically select RoCEv2/IPv4 interface by default. Allow to select IPv6 or even the network/mask. Reduce NVLS memory usage. * Add stepSize as property of a connection to allow for different sizes on different peers; set it to 128K for NVLink SHARP. Improve tuner loading * Look for more paths, be more consistent with the network device plugin. * Also search for tuner support inside the net plugin. Improve tuner API * Add context to support multi-device per process. Add magic number around comm object to detect comm corruption. * Add some basic check around communicators so that we can report a problem when a communicator gets corrupted or a wrong comm pointer is passed to NCCL. Fix net/IB error path. Github PR #1164 Fix collnet rail mapping with split comm. Fix packet reordering issue causing bootstrap mismatch * Use a different tag in ncclTransportP2pSetup for the connectInfo exchange and the following barrier. Fix hang when crossNic is inconsistent between ranks. Fix minCompCap/maxCompCap computation. Github issue #1184 --- ext-net/example/nccl/net_device.h | 2 - ext-tuner/example/nccl/tuner.h | 23 ++- ext-tuner/example/plugin.c | 8 +- makefiles/version.mk | 2 +- src/bootstrap.cc | 251 +++++++++++++++----------- src/channel.cc | 12 +- src/collectives.cc | 24 ++- src/debug.cc | 4 + src/device/all_gather.h | 60 ++++--- src/device/all_reduce.h | 101 ++++++++--- src/device/common.h | 14 +- src/device/prims_simple.h | 111 +++++++----- src/device/reduce_scatter.h | 60 ++++--- src/enqueue.cc | 71 ++++++-- src/graph/connect.cc | 26 ++- src/graph/paths.cc | 64 +++---- src/graph/search.cc | 111 +++++++----- src/graph/topo.cc | 140 ++++++++++----- src/graph/topo.h | 29 ++- src/graph/tuning.cc | 19 +- src/graph/xml.cc | 89 +++++++++- src/graph/xml.h | 91 +++++++++- src/include/argcheck.h | 1 + src/include/bootstrap.h | 4 +- src/include/comm.h | 30 +++- src/include/cudawrap.h | 70 ++++---- src/include/device.h | 21 ++- src/include/graph.h | 12 +- src/include/info.h | 10 +- src/include/nccl_common.h | 2 +- src/include/nccl_tuner.h | 23 ++- src/include/proxy.h | 20 ++- src/include/register.h | 6 +- src/include/socket.h | 1 + src/include/transport.h | 16 +- src/include/tuner.h | 4 +- src/init.cc | 282 +++++++++++++++++------------ src/misc/argcheck.cc | 9 + src/misc/cudawrap.cc | 215 +++++++++------------- src/misc/socket.cc | 18 ++ src/misc/tuner.cc | 195 +++++++++++++++----- src/nccl.h.in | 9 - src/net.cc | 95 ++++++++-- src/proxy.cc | 27 +-- src/register.cc | 9 +- src/transport.cc | 26 +-- src/transport/coll_net.cc | 286 ++++++++++++++++++++++++++---- src/transport/net.cc | 30 ++-- src/transport/net_ib.cc | 253 +++++++++++++++++++++++--- src/transport/nvls.cc | 47 +++-- src/transport/p2p.cc | 11 +- src/transport/shm.cc | 10 +- 52 files changed, 2117 insertions(+), 937 deletions(-) diff --git a/ext-net/example/nccl/net_device.h b/ext-net/example/nccl/net_device.h index a0b84c765..b430d9064 100644 --- a/ext-net/example/nccl/net_device.h +++ b/ext-net/example/nccl/net_device.h @@ -7,8 +7,6 @@ #ifndef NET_DEVICE_H_ #define NET_DEVICE_H_ -#include "net_device.h" - #define NCCL_NET_DEVICE_INVALID_VERSION 0x0 #define NCCL_NET_MTU_SIZE 4096 diff --git a/ext-tuner/example/nccl/tuner.h b/ext-tuner/example/nccl/tuner.h index 8076aa872..57825b99c 100644 --- a/ext-tuner/example/nccl/tuner.h +++ b/ext-tuner/example/nccl/tuner.h @@ -39,13 +39,17 @@ typedef struct { const char* name; // Initializes tuner states. - // nRanks: number of ranks in current communicator. Each communicator initialize its own tuner. - // nNodes: number of nodes in current communicator. - // logFunction: a logFunction can be useful to integrate logging together with NCCL core. - ncclResult_t (*init)(size_t nRanks, size_t nNodes, ncclDebugLogger_t logFunction); + // Inputs: + // - nRanks: number of ranks in current communicator. Each communicator initialize its own tuner. + // - nNodes: number of nodes in current communicator. + // - logFunction: a logFunction can be useful to integrate logging together with NCCL core. + // Outputs: + // - context: tuner context object + ncclResult_t (*init)(size_t nRanks, size_t nNodes, ncclDebugLogger_t logFunction, void **context); // Gets info (algo, protocol, number of ctas and threads) for a given collective. // Inputs: + // - context: tuner context object // - collType: collective type , e.g., allreduce, allgather… // - nBytes: collective size in bytes // - collNetSupport: whether collnet supports this type @@ -62,16 +66,17 @@ typedef struct { // Also, the plugin is allowed to not set any output, or set only the // algorithm and protocol, but not only the algorithm or only the protocol. // Unset fields will be set automatically by NCCL. - ncclResult_t (*getCollInfo)(ncclFunc_t collType, size_t nBytes, + ncclResult_t (*getCollInfo)(void* context, ncclFunc_t collType, size_t nBytes, int collNetSupport, int nvlsSupport, int numPipeOps, int *algorithm, int *protocol, int* nChannels); // Terminates the plugin and cleans up any resources that the plugin allocated. - ncclResult_t (*destroy)(); -} ncclTuner_v1_t; + // context: tuner context object + ncclResult_t (*destroy)(void* context); +} ncclTuner_v2_t; -typedef ncclTuner_v1_t ncclTuner_t; +typedef ncclTuner_v2_t ncclTuner_t; -#define NCCL_TUNER_PLUGIN_SYMBOL "ncclTunerPlugin_v1" +#define NCCL_TUNER_PLUGIN_SYMBOL "ncclTunerPlugin_v2" #endif diff --git a/ext-tuner/example/plugin.c b/ext-tuner/example/plugin.c index d972de3d3..3c669433a 100644 --- a/ext-tuner/example/plugin.c +++ b/ext-tuner/example/plugin.c @@ -8,17 +8,17 @@ #define __hidden __attribute__ ((visibility("hidden"))) -__hidden ncclResult_t pluginInit(size_t nRanks, size_t nNodes, ncclDebugLogger_t logFunction) { return ncclSuccess; } +__hidden ncclResult_t pluginInit(size_t nRanks, size_t nNodes, ncclDebugLogger_t logFunction, void **context) { return ncclSuccess; } -__hidden ncclResult_t pluginGetCollInfo(ncclFunc_t collType, size_t nBytes, +__hidden ncclResult_t pluginGetCollInfo(void* context, ncclFunc_t collType, size_t nBytes, int collNetSupport, int nvlsSupport, int numPipeOps, int *algorithm, int *protocol, int* nChannels) { *algorithm = NCCL_ALGO_RING; *protocol = NCCL_PROTO_SIMPLE; return ncclSuccess; } -__hidden ncclResult_t pluginDestroy() { return ncclSuccess; } +__hidden ncclResult_t pluginDestroy(void* context) { return ncclSuccess; } #define PLUGIN_NAME "Example" -const ncclTuner_v1_t ncclTunerPlugin_v1 = { +const ncclTuner_v2_t ncclTunerPlugin_v2 = { .name = PLUGIN_NAME, .init = pluginInit, .getCollInfo = pluginGetCollInfo, diff --git a/makefiles/version.mk b/makefiles/version.mk index 20f199141..d4da30daf 100644 --- a/makefiles/version.mk +++ b/makefiles/version.mk @@ -1,6 +1,6 @@ ##### version NCCL_MAJOR := 2 -NCCL_MINOR := 20 +NCCL_MINOR := 21 NCCL_PATCH := 5 NCCL_SUFFIX := PKG_REVISION := 1 diff --git a/src/bootstrap.cc b/src/bootstrap.cc index bbcabc877..cff2df50d 100644 --- a/src/bootstrap.cc +++ b/src/bootstrap.cc @@ -80,6 +80,16 @@ static ncclResult_t bootstrapNetRecv(struct ncclSocket* sock, void* data, int si NCCLCHECK(ncclSocketRecv(sock, data, std::min(recvSize, size))); return ncclSuccess; } +static ncclResult_t bootstrapNetSendRecv(struct ncclSocket* sendSock, void* sendData, int sendSize, struct ncclSocket* recvSock, void* recvData, int recvSize) { + int senderRecvSize; + NCCLCHECK(ncclSocketSendRecv(sendSock, &sendSize, sizeof(int), recvSock, &senderRecvSize, sizeof(int))); + if (senderRecvSize > recvSize) { + WARN("Message truncated : received %d bytes instead of %d", senderRecvSize, recvSize); + return ncclInternalError; + } + NCCLCHECK(ncclSocketSendRecv(sendSock, sendData, sendSize, recvSock, recvData, recvSize)); + return ncclSuccess; +} struct extInfo { int rank; @@ -390,103 +400,40 @@ ncclResult_t bootstrapSplit(struct ncclBootstrapHandle* handle, struct ncclComm* goto exit; } -ncclResult_t bootstrapAllGather(void* commState, void* allData, int size) { - struct bootstrapState* state = (struct bootstrapState*)commState; - char* data = (char*)allData; - int rank = state->rank; - int nranks = state->nranks; - - TRACE(NCCL_INIT, "rank %d nranks %d size %d", rank, nranks, size); +// Bootstrap send/receive functions +// +// We do not keep connections opened with all ranks at all times, and we have no guarantee +// that connections to our unique listen socket will arrive in the same order as we need +// them. Therefore, when establishing a connection, the sender sends a (peer, tag) tuple to +// allow the receiver to identify the flow, and keep it in an unexpected queue if needed. - /* Simple ring based AllGather - * At each step i receive data from (rank-i-1) from left - * and send previous step's data from (rank-i) to right - */ - for (int i=0; iringSendSocket, data+sslice*size, size)); - // Recv slice from the left - NCCLCHECK(bootstrapNetRecv(&state->ringRecvSocket, data+rslice*size, size)); - } +ncclResult_t bootstrapConnect(void* commState, int peer, int tag, struct ncclSocket* sock) { + ncclResult_t ret = ncclSuccess; + struct bootstrapState* state = (struct bootstrapState*)commState; - TRACE(NCCL_INIT, "rank %d nranks %d size %d - DONE", rank, nranks, size); + NCCLCHECKGOTO(ncclSocketInit(sock, state->peerCommAddresses+peer, state->magic, ncclSocketTypeBootstrap), ret, fail); + NCCLCHECKGOTO(ncclSocketConnect(sock), ret, fail); + NCCLCHECKGOTO(bootstrapNetSend(sock, &state->rank, sizeof(int)), ret, fail); + NCCLCHECKGOTO(bootstrapNetSend(sock, &tag, sizeof(int)), ret, fail); return ncclSuccess; +fail: + NCCLCHECK(ncclSocketClose(sock)); + return ret; } ncclResult_t bootstrapSend(void* commState, int peer, int tag, void* data, int size) { ncclResult_t ret = ncclSuccess; - struct bootstrapState* state = (struct bootstrapState*)commState; struct ncclSocket sock; - NCCLCHECKGOTO(ncclSocketInit(&sock, state->peerCommAddresses+peer, state->magic, ncclSocketTypeBootstrap), ret, fail); - NCCLCHECKGOTO(ncclSocketConnect(&sock), ret, fail); - NCCLCHECKGOTO(bootstrapNetSend(&sock, &state->rank, sizeof(int)), ret, fail); - NCCLCHECKGOTO(bootstrapNetSend(&sock, &tag, sizeof(int)), ret, fail); - NCCLCHECKGOTO(bootstrapNetSend(&sock, data, size), ret, fail); + TRACE(NCCL_BOOTSTRAP, "Sending to peer=%d tag=%d size=%d", peer, tag, size); + NCCLCHECK(bootstrapConnect(commState, peer, tag, &sock)); + NCCLCHECKGOTO(bootstrapNetSend(&sock, data, size), ret, exit); + + TRACE(NCCL_BOOTSTRAP, "Sent to peer=%d tag=%d size=%d", peer, tag, size); exit: NCCLCHECK(ncclSocketClose(&sock)); return ret; -fail: - goto exit; -} - -ncclResult_t bootstrapBarrier(void* commState, int *ranks, int rank, int nranks, int tag) { - if (nranks == 1) return ncclSuccess; - TRACE(NCCL_INIT, "rank %d nranks %d tag %x - ENTER", rank, nranks, tag); - - /* Simple intra process barrier - * - * Based on the dissemination algorithm by Debra Hensgen, Raphael Finkel, and Udi Manbet, - * "Two Algorithms for Barrier Synchronization," International Journal of Parallel Programming, 17(1):1-17, 1988" - */ - int data[1]; - for (int mask=1; masklistenSock), ret, fail); - NCCLCHECKGOTO(bootstrapNetRecv(&sock, &newPeer, sizeof(int)), ret, fail); - NCCLCHECKGOTO(bootstrapNetRecv(&sock, &newTag, sizeof(int)), ret, fail); - if (newPeer == peer && newTag == tag) { - NCCLCHECKGOTO(bootstrapNetRecv(&sock, ((char*)data), size), ret, fail); - goto exit; - } - // Unexpected connection. Save for later. - NCCLCHECKGOTO(unexpectedEnqueue(state, newPeer, newTag, &sock), ret, fail); + NCCLCHECKGOTO(ncclSocketInit(sock), ret, fail); + NCCLCHECKGOTO(ncclSocketAccept(sock, &state->listenSock), ret, fail); + NCCLCHECKGOTO(bootstrapNetRecv(sock, &newPeer, sizeof(int)), ret, fail); + NCCLCHECKGOTO(bootstrapNetRecv(sock, &newTag, sizeof(int)), ret, fail); + if (newPeer == peer && newTag == tag) return ncclSuccess; + NCCLCHECKGOTO(unexpectedEnqueue(state, newPeer, newTag, sock), ret, fail); } + return ncclSuccess; +fail: + NCCLCHECK(ncclSocketClose(sock)); + return ret; +} + +// We can't know who we'll receive from, so we need to receive everything at once +ncclResult_t bootstrapRecv(void* commState, int peer, int tag, void* data, int size) { + ncclResult_t ret; + struct ncclSocket sock; + NCCLCHECK(bootstrapAccept(commState, peer, tag, &sock)); + TRACE(NCCL_BOOTSTRAP, "Receiving tag=%d peer=%d size=%d", tag, peer, size); + NCCLCHECKGOTO(bootstrapNetRecv(&sock, ((char*)data), size), ret, exit); exit: NCCLCHECK(ncclSocketClose(&sock)); return ret; -fail: - goto exit; +} + +// Collective algorithms, based on bootstrapSend/Recv, and sometimes bootstrapConnect/Accept + +ncclResult_t bootstrapRingAllGather(struct ncclSocket* prevSocket, struct ncclSocket* nextSocket, int rank, int nranks, char* data, int size) { + /* Simple ring based AllGather + * At each step i receive data from (rank-i-1) from prev + * and send previous step's data from (rank-i) to next + */ + for (int i=0; irank; + int nranks = state->nranks; + + TRACE(NCCL_INIT, "rank %d nranks %d size %d", rank, nranks, size); + + NCCLCHECK(bootstrapRingAllGather(&state->ringRecvSocket, &state->ringSendSocket, rank, nranks, (char*)allData, size)); + + TRACE(NCCL_INIT, "rank %d nranks %d size %d - DONE", rank, nranks, size); + return ncclSuccess; +} + +ncclResult_t bootstrapIntraNodeBarrier(void* commState, int *ranks, int rank, int nranks, int tag) { + if (nranks == 1) return ncclSuccess; + TRACE(NCCL_INIT, "rank %d nranks %d tag %x - ENTER", rank, nranks, tag); + + /* Simple [intra] process barrier + * + * Based on the dissemination algorithm by Debra Hensgen, Raphael Finkel, and Udi Manbet, + * "Two Algorithms for Barrier Synchronization," International Journal of Parallel Programming, 17(1):1-17, 1988" + */ + int data[1]; + for (int mask=1; maskid != -1) return ncclSuccess; int nRanks = comm->nRanks; - int nPeers = nRanks + 1 /* Collnet */ + comm->localRanks /* NVLS */; + int nvlsRanks = comm->MNNVL ? comm->clique.size : comm->localRanks; + int nPeers = nRanks + 1 /* Collnet */ + nvlsRanks /* NVLS */; channel->id = channelId; channel->workFifoSent = 0; @@ -73,10 +74,11 @@ ncclResult_t initNvlsChannel(struct ncclComm* comm, int channelId, struct ncclCo NCCLCHECK(ncclStrongStreamAcquireUncaptured(&sharedRes->deviceStream)); + int nvlsRanks = comm->MNNVL ? comm->clique.size : comm->localRanks; if (share) { channel->nvlsPeers = parent->channels[channelId].nvlsPeers; channel->nvlsDevPeers = parent->channels[channelId].nvlsDevPeers; - for (int r = 0; r < comm->localRanks; ++r) { + for (int r = 0; r < nvlsRanks; ++r) { int tr = comm->topParentLocalRanks[r]; uintptr_t addr = (uintptr_t)(parent->channels[channelId].nvlsDevPeers + tr); channel->peers[comm->nRanks + 1 + r] = parent->channels[channelId].nvlsPeers + tr; @@ -85,9 +87,9 @@ ncclResult_t initNvlsChannel(struct ncclComm* comm, int channelId, struct ncclCo ncclAtomicRefCountIncrement(&parent->channels[channelId].nvlsPeers[tr].refCount); } } else { - NCCLCHECK(ncclCalloc(&channel->nvlsPeers, comm->localRanks)); - NCCLCHECK(ncclCudaCallocAsync(&channel->nvlsDevPeers, comm->localRanks, sharedRes->deviceStream.cudaStream)); - for (int r = 0; r < comm->localRanks; ++r) { + NCCLCHECK(ncclCalloc(&channel->nvlsPeers, nvlsRanks)); + NCCLCHECK(ncclCudaCallocAsync(&channel->nvlsDevPeers, nvlsRanks, sharedRes->deviceStream.cudaStream)); + for (int r = 0; r < nvlsRanks; ++r) { uintptr_t addr = (uintptr_t)(channel->nvlsDevPeers + r); channel->peers[comm->nRanks + 1 + r] = channel->nvlsPeers + r; NCCLCHECK(ncclCudaMemcpyAsync((uintptr_t*)(channel->devPeers + comm->nRanks + 1 + r), (uintptr_t*)&addr, 1, sharedRes->deviceStream.cudaStream)); diff --git a/src/collectives.cc b/src/collectives.cc index 89d8932db..571134fca 100644 --- a/src/collectives.cc +++ b/src/collectives.cc @@ -23,7 +23,8 @@ ncclResult_t ncclAllGather(const void* sendbuff, void* recvbuff, size_t sendcoun struct ncclInfo info = { ncclFuncAllGather, "AllGather", sendbuff, recvbuff, sendcount, datatype, ncclSum, 0, comm, stream, /* Args */ ALLGATHER_CHUNKSTEPS, ALLGATHER_SLICESTEPS }; - return ncclEnqueueCheck(&info); + NCCLCHECK(ncclEnqueueCheck(&info)); + return ncclSuccess; } NCCL_API(ncclResult_t, ncclAllReduce, const void* sendbuff, void* recvbuff, size_t count, @@ -46,7 +47,8 @@ ncclResult_t ncclAllReduce(const void* sendbuff, void* recvbuff, size_t count, struct ncclInfo info = { ncclFuncAllReduce, "AllReduce", sendbuff, recvbuff, count, datatype, op, 0, comm, stream, /* Args */ ALLREDUCE_CHUNKSTEPS, ALLREDUCE_SLICESTEPS }; - return ncclEnqueueCheck(&info); + NCCLCHECK(ncclEnqueueCheck(&info)); + return ncclSuccess; } NCCL_API(ncclResult_t, ncclBroadcast, const void* sendbuff, void* recvbuff, size_t count, ncclDataType_t datatype, int root, @@ -67,14 +69,16 @@ ncclResult_t ncclBroadcast(const void* sendbuff, void* recvbuff, size_t count, n struct ncclInfo info = { ncclFuncBroadcast, "Broadcast", sendbuff, recvbuff, count, datatype, ncclSum, root, comm, stream, /* Args */ BROADCAST_CHUNKSTEPS, BROADCAST_SLICESTEPS }; - return ncclEnqueueCheck(&info); + NCCLCHECK(ncclEnqueueCheck(&info)); + return ncclSuccess; } /* Deprecated original "in place" function, similar to MPI */ NCCL_API(ncclResult_t, ncclBcast, void* buff, size_t count, ncclDataType_t datatype, int root, ncclComm_t comm, cudaStream_t stream); ncclResult_t ncclBcast(void* buff, size_t count, ncclDataType_t datatype, int root, ncclComm_t comm, cudaStream_t stream) { - return ncclBroadcast(buff, buff, count, datatype, root, comm, stream); + NCCLCHECK(ncclBroadcast(buff, buff, count, datatype, root, comm, stream)); + return ncclSuccess; } NCCL_API(ncclResult_t, ncclReduce, const void* sendbuff, void* recvbuff, size_t count, @@ -98,7 +102,8 @@ ncclResult_t ncclReduce(const void* sendbuff, void* recvbuff, size_t count, struct ncclInfo info = { ncclFuncReduce, "Reduce", sendbuff, recvbuff, count, datatype, op, root, comm, stream, /* Args */ REDUCE_CHUNKSTEPS, REDUCE_SLICESTEPS }; - return ncclEnqueueCheck(&info); + NCCLCHECK(ncclEnqueueCheck(&info)); + return ncclSuccess; } NCCL_API(ncclResult_t, ncclReduceScatter, const void* sendbuff, void* recvbuff, size_t recvcount, @@ -120,7 +125,8 @@ ncclResult_t ncclReduceScatter(const void* sendbuff, void* recvbuff, size_t recv struct ncclInfo info = { ncclFuncReduceScatter, "ReduceScatter", sendbuff, recvbuff, recvcount, datatype, op, 0, comm, stream, /* Args */ REDUCESCATTER_CHUNKSTEPS, REDUCESCATTER_SLICESTEPS }; - return ncclEnqueueCheck(&info); + NCCLCHECK(ncclEnqueueCheck(&info)); + return ncclSuccess; } struct NvtxParamsSendRecv { @@ -144,7 +150,8 @@ ncclResult_t ncclSend(const void* sendbuff, size_t count, ncclDataType_t datatyp 1, 1 }; ncclResult_t ret; NCCLCHECK(ncclGroupStart()); - ret = ncclEnqueueCheck(&info); + NCCLCHECKGOTO(ncclEnqueueCheck(&info), ret, exit); +exit: NCCLCHECK(ncclGroupEnd()); return ret; } @@ -161,7 +168,8 @@ ncclResult_t ncclRecv(void* recvbuff, size_t count, ncclDataType_t datatype, int 1, 1 }; ncclResult_t ret; NCCLCHECK(ncclGroupStart()); - ret = ncclEnqueueCheck(&info); + NCCLCHECKGOTO(ncclEnqueueCheck(&info), ret, exit); +exit: NCCLCHECK(ncclGroupEnd()); return ret; } diff --git a/src/debug.cc b/src/debug.cc index 2771d1b70..522999b44 100644 --- a/src/debug.cc +++ b/src/debug.cc @@ -79,6 +79,10 @@ void ncclDebugInit() { mask = NCCL_PROXY; } else if (strcasecmp(subsys, "NVLS") == 0) { mask = NCCL_NVLS; + } else if (strcasecmp(subsys, "BOOTSTRAP") == 0) { + mask = NCCL_BOOTSTRAP; + } else if (strcasecmp(subsys, "REG") == 0) { + mask = NCCL_REG; } else if (strcasecmp(subsys, "ALL") == 0) { mask = NCCL_ALL; } diff --git a/src/device/all_gather.h b/src/device/all_gather.h index 702eb9764..809e8ae12 100644 --- a/src/device/all_gather.h +++ b/src/device/all_gather.h @@ -253,18 +253,26 @@ struct RunWorkElement, /*Direct=*/0, Proto, 0> - prims(tid, tn, nullptr, &direct->out, args->sendbuff, nullptr, - /*redOpArg=*/0, 0*Proto::MaxGroupWidth, 1, 1); - for (ssize_t railGridOffset=0; railGridOffset < nNodes*sizePerRank; railGridOffset += nChannels*chunkSize) { - ssize_t railAllBeg = railGridOffset + args->bid*chunkSize; - ssize_t railAllEnd = min(railAllBeg + chunkSize, nNodes*sizePerRank); - ssize_t railOneBeg = ncclShmem.comm.node*sizePerRank; - ssize_t railOneEnd = railOneBeg + sizePerRank; - ssize_t beg = max(railAllBeg, railOneBeg); - ssize_t end = min(railAllEnd, railOneEnd); - prims.send(beg-railOneBeg, max(ssize_t(0), end-beg)); + if (args->regUsed == NCCL_COLLNET_REG_BUFFER) { + if (tid == 0) { + int steps = (int)divUp(nNodes * sizePerRank * sizeof(T), NCCL_MAX_COLLNET_SIZE); + Primitives, /*Direct=*/0, Proto, 0>::sendPeerNotify(direct->out, 1, steps); + } + __syncwarp(); + } else { + // Phase 1: send to network + Primitives, /*Direct=*/0, Proto, 0> + prims(tid, tn, nullptr, &direct->out, args->sendbuff, nullptr, + /*redOpArg=*/0, 0 * Proto::MaxGroupWidth, 1, 1); + for (ssize_t railGridOffset = 0; railGridOffset < nNodes * sizePerRank; railGridOffset += nChannels * chunkSize) { + ssize_t railAllBeg = railGridOffset + args->bid * chunkSize; + ssize_t railAllEnd = min(railAllBeg + chunkSize, nNodes * sizePerRank); + ssize_t railOneBeg = ncclShmem.comm.node * sizePerRank; + ssize_t railOneEnd = railOneBeg + sizePerRank; + ssize_t beg = max(railAllBeg, railOneBeg); + ssize_t end = min(railAllEnd, railOneEnd); + prims.send(beg - railOneBeg, max(ssize_t(0), end - beg)); + } } return; } @@ -272,16 +280,24 @@ struct RunWorkElement deposit output + send to bcast - Primitives, /*Direct=*/0, Proto, 0> - prims(tid, tn, &direct->out, direct->heads+1, nullptr, nullptr, - /*redOpArg=*/0, 1*Proto::MaxGroupWidth, 0, 0); - for (ssize_t railGridOffset=0; railGridOffset < nNodes*sizePerRank; railGridOffset += nChannels*chunkSize) { - Scatterer scat; - scat.args = args; - scat.chunkSize = chunkSize; - scat.railGridOffset = railGridOffset; - prims.process(scat); + if (args->regUsed == NCCL_COLLNET_REG_BUFFER) { + if (tid == 0) { + int steps = (int)divUp(nNodes * sizePerRank * sizeof(T), NCCL_MAX_COLLNET_SIZE); + Primitives, /*Direct=*/0, Proto, 0>::recvPeerNotify(direct->out, 0, steps); + } + __syncwarp(); + } else { + // Phase 2: Recv network -> deposit output + send to bcast + Primitives, /*Direct=*/0, Proto, 0> + prims(tid, tn, &direct->out, direct->heads + 1, nullptr, nullptr, + /*redOpArg=*/0, 1 * Proto::MaxGroupWidth, 0, 0); + for (ssize_t railGridOffset = 0; railGridOffset < nNodes * sizePerRank; railGridOffset += nChannels * chunkSize) { + Scatterer scat; + scat.args = args; + scat.chunkSize = chunkSize; + scat.railGridOffset = railGridOffset; + prims.process(scat); + } } return; } diff --git a/src/device/all_reduce.h b/src/device/all_reduce.h index 75e2bed54..49f8dc65b 100644 --- a/src/device/all_reduce.h +++ b/src/device/all_reduce.h @@ -297,13 +297,21 @@ struct RunWorkElement, /*Direct=*/0, Proto, 0> + if (args->regUsed == NCCL_COLLNET_REG_BUFFER) { + if (tid == tidStartReduce) { + int steps = (int)divUp(size * sizeof(T), NCCL_MAX_COLLNET_SIZE); + Primitives, /*Direct=*/0, Proto, 0>::sendPeerNotify(direct->out, 1, steps); + } + __syncwarp(); + } else { + Primitives, /*Direct=*/0, Proto, 0> prims(tid-tidStartReduce, nThreadsReduce, nullptr, &direct->out, args->sendbuff, args->recvbuff, args->redOpArg, 3*Proto::MaxGroupWidth, 1, 1); - for (ssize_t gridOffset = 0; gridOffset < size; gridOffset += loopSize) { - ssize_t offset = gridOffset + (bid*direct->nHeads+direct->headRank)*chunkSize; - int nelem = min(chunkSize, size-offset); - prims.send(offset, nelem); + for (ssize_t gridOffset = 0; gridOffset < size; gridOffset += loopSize) { + ssize_t offset = gridOffset + (bid*direct->nHeads+direct->headRank)*chunkSize; + int nelem = min(chunkSize, size-offset); + prims.send(offset, nelem); + } } } } else if (tid < tidStartBcast && hasUp) { @@ -328,14 +336,22 @@ struct RunWorkElement, /*Direct=*/0, Proto, 0> - prims(tid-tidStartBcast, nThreadsBcast, &direct->out, nullptr, args->sendbuff, args->recvbuff, - args->redOpArg, 1*Proto::MaxGroupWidth, 0, 0); - for (ssize_t gridOffset = 0; gridOffset < size; gridOffset += loopSize) { - ssize_t offset = gridOffset + (bid*direct->nHeads+direct->headRank)*chunkSize; - int nelem = min(chunkSize, size-offset); - prims.recv(offset, nelem, /*postOp=*/true); + if (args->regUsed == NCCL_COLLNET_REG_BUFFER) { + if (tid == tidStartBcast) { + int steps = (int)divUp(size * sizeof(T), NCCL_MAX_COLLNET_SIZE); + Primitives, /*Direct=*/0, Proto, 0>::recvPeerNotify(direct->out, 0, steps); + } + __syncwarp(); + } else { + // Recv from network (no post thread needed) + Primitives, /*Direct=*/0, Proto, 0> + prims(tid - tidStartBcast, nThreadsBcast, &direct->out, nullptr, args->sendbuff, args->recvbuff, + args->redOpArg, 1 * Proto::MaxGroupWidth, 0, 0); + for (ssize_t gridOffset = 0; gridOffset < size; gridOffset += loopSize) { + ssize_t offset = gridOffset + (bid * direct->nHeads + direct->headRank) * chunkSize; + int nelem = min(chunkSize, size - offset); + prims.recv(offset, nelem, /*postOp=*/true); + } } } } @@ -616,21 +632,31 @@ struct RunWorkElement, /*Direct=*/1, Proto, 0> - prims(groupTid, groupNthreads, &recv, &send, args->sendbuff, args->recvbuff, - args->redOpArg, group*Proto::MaxGroupWidth, connIndex, connIndex); - if (tid < nthreadsSplit) { if (recv == -1) { - for (ssize_t gridOffset = 0; gridOffset < size; gridOffset += loopSize) { - ssize_t offset = gridOffset + bid*int(chunkSize); - int nelem = min(chunkSize, size-offset); - prims.send(offset, nelem); + if (args->regUsed == NCCL_COLLNET_REG_BUFFER) { + if (groupTid == 0) { + int steps = (int)divUp(size * sizeof(T), NCCL_MAX_COLLNET_SIZE); + Primitives, /*Direct=*/1, Proto, 0>::sendPeerNotify(send, connIndex, steps); + } + __syncwarp(); + } else { + Primitives, /*Direct=*/1, Proto, 0> + prims(groupTid, groupNthreads, &recv, &send, args->sendbuff, args->recvbuff, + args->redOpArg, group * Proto::MaxGroupWidth, connIndex, connIndex); + for (ssize_t gridOffset = 0; gridOffset < size; gridOffset += loopSize) { + ssize_t offset = gridOffset + bid * int(chunkSize); + int nelem = min(chunkSize, size - offset); + prims.send(offset, nelem); + } } } else { + Primitives, /*Direct=*/1, Proto, 0> + prims(groupTid, groupNthreads, &recv, &send, args->sendbuff, args->recvbuff, + args->redOpArg, group * Proto::MaxGroupWidth, connIndex, connIndex); for (ssize_t gridOffset = 0; gridOffset < size; gridOffset += loopSize) { - ssize_t offset = gridOffset + bid*int(chunkSize); - int nelem = min(chunkSize, size-offset); + ssize_t offset = gridOffset + bid * int(chunkSize); + int nelem = min(chunkSize, size - offset); prims.recvReduceSend(offset, nelem); } } @@ -639,19 +665,36 @@ struct RunWorkElementregUsed == NCCL_COLLNET_REG_BUFFER) { + if (groupTid == 0) { + int steps = (int)divUp(size * sizeof(T), NCCL_MAX_COLLNET_SIZE); + Primitives, /*Direct=*/1, Proto, 0>::recvPeerNotify(recv, connIndex, steps); + } + __syncwarp(); + } else { + Primitives, /*Direct=*/1, Proto, 0> + prims(groupTid, groupNthreads, &recv, &send, args->sendbuff, args->recvbuff, + args->redOpArg, group * Proto::MaxGroupWidth, connIndex, connIndex); + for (ssize_t gridOffset = 0; gridOffset < size; gridOffset += loopSize) { + ssize_t offset = gridOffset + bid * int(chunkSize); + int nelem = min(chunkSize, size - offset); + prims.recv(offset, nelem, /*postOp*/true); + } } } else { + Primitives, /*Direct=*/1, Proto, 0> + prims(groupTid, groupNthreads, &recv, &send, args->sendbuff, args->recvbuff, + args->redOpArg, group * Proto::MaxGroupWidth, connIndex, connIndex); for (ssize_t gridOffset = 0; gridOffset < size; gridOffset += loopSize) { - ssize_t offset = gridOffset + bid*int(chunkSize); - int nelem = min(chunkSize, size-offset); + ssize_t offset = gridOffset + bid * int(chunkSize); + int nelem = min(chunkSize, size - offset); prims.recvCopyDirectSend(offset, nelem, /*postOp*/true); } } } else { + Primitives, /*Direct=*/1, Proto, 0> + prims(groupTid, groupNthreads, &recv, &send, args->sendbuff, args->recvbuff, + args->redOpArg, group * Proto::MaxGroupWidth, connIndex, connIndex); if (send == -1) { for (ssize_t gridOffset = 0; gridOffset < size; gridOffset += loopSize) { ssize_t offset = gridOffset + bid*int(chunkSize); diff --git a/src/device/common.h b/src/device/common.h index 8f3cad328..d8581d3f4 100644 --- a/src/device/common.h +++ b/src/device/common.h @@ -18,19 +18,21 @@ typedef void(*ncclDevFuncPtr_t)(); extern __device__ ncclDevFuncPtr_t const ncclDevFuncTable[]; struct ncclShmemGroup { - ncclConnInfo *recvConns[NCCL_MAX_NVLS_ARITY]; - ncclConnInfo *sendConns[NCCL_MAX_NVLS_ARITY]; - void* srcs[NCCL_MAX_NVLS_ARITY+1]; - void* dsts[NCCL_MAX_NVLS_ARITY+1]; + ncclConnInfo *recvConns[NCCL_MAX_ARITY]; + ncclConnInfo *sendConns[NCCL_MAX_ARITY]; + void* userInput; + void* userOutput; + void* srcs[NCCL_MAX_ARITY+1]; + void* dsts[NCCL_MAX_ARITY+1]; union { unpackGroupShmem unpack; } devicePlugin; - int32_t dstSizes[NCCL_MAX_NVLS_ARITY+1]; + int32_t dstSizes[NCCL_MAX_ARITY+1]; }; struct ncclShmemData { struct ncclShmemGroup groups[NCCL_MAX_GROUPS]; - uint64_t redOpArgs[NCCL_MAX_NVLS_ARITY+1]; + uint64_t redOpArgs[NCCL_MAX_ARITY+1]; int channelId; int aborted; alignas(16) struct ncclDevComm comm; diff --git a/src/device/prims_simple.h b/src/device/prims_simple.h index 6bf8a1a8a..2431c2fdd 100644 --- a/src/device/prims_simple.h +++ b/src/device/prims_simple.h @@ -5,6 +5,7 @@ ************************************************************************/ #include "network/unpack/unpack.h" +#include template @@ -13,9 +14,7 @@ class Primitives< > { static constexpr int MaxRecv = Fan::MaxRecv, MaxSend = Fan::MaxSend; static constexpr int Input=0, Output=1; - static constexpr int RoleInput = 0x01, - RoleOutput = 0x02, - RoleWaitRecv = 0x04, + static constexpr int RoleWaitRecv = 0x04, // 0x1 0x2 are free to use RoleWaitSend = 0x08, RolePostSend = 0x10, RolePostRecv = 0x20, @@ -40,13 +39,11 @@ class Primitives< int group; uint64_t step; struct ncclConnFifo* connFifo = NULL; - union { - T *userBuff; // (flags & (RoleInput|RoleOutput)) - T *connEltsFifo; // !(flags & (RoleInput|RoleOutput)) - }; - T *directBuff; + T* connEltsFifo; + T* directBuff; uint64_t *connStepPtr; uint64_t connStepCache; // Cache last seen value of (*connStepPtr) + int connStepSize; // Connection step size void* mhandle; void* netDeviceHandle; @@ -153,7 +150,7 @@ class Primitives< } else if (flags & DirectRead) { // empty send ptrs[index] = nullptr; } else { - ptrs[index] = connEltsFifo + (step%NCCL_STEPS)*stepSize; + ptrs[index] = connEltsFifo + (step%NCCL_STEPS)*connStepSize; } } else if (!isSendNotRecv && DirectRecv) { if (flags & (DirectRead | NvlsDirectRead)) { @@ -161,11 +158,11 @@ class Primitives< } else if (flags & DirectWrite) { ptrs[index] = directBuff + dstIx + offset; // send to next from my output buffer } else { - ptrs[index] = connEltsFifo + (step%NCCL_STEPS)*stepSize; + ptrs[index] = connEltsFifo + (step%NCCL_STEPS)*connStepSize; } } else { - ptrs[index] = connEltsFifo + (step%NCCL_STEPS)*stepSize; + ptrs[index] = connEltsFifo + (step%NCCL_STEPS)*connStepSize; } if ((flags & (AnyNetDeviceUnpack)) && (flags & (Recv*RoleWaitRecv))) { ncclNetDeviceIncrementHead(group); @@ -232,10 +229,12 @@ class Primitives< #endif do { sliceSize = sliceSize < nelem-offset ? sliceSize : nelem-offset; - if (Src && (flags & (SrcBuf==Input ? RoleInput : RoleOutput))) - ncclShmem.groups[group].srcs[0] = userBuff + srcIx + offset; - if (Dst && (flags & (DstBuf==Input ? RoleInput : RoleOutput))) - ncclShmem.groups[group].dsts[0] = userBuff + dstIx + offset; + if (tid == 0) { + T* userInput = (T*)ncclShmem.groups[group].userInput; + T* userOutput = (T*)ncclShmem.groups[group].userOutput; + if (Src) ncclShmem.groups[group].srcs[0] = (SrcBuf==Input ? userInput : userOutput) + srcIx + offset; + if (Dst) ncclShmem.groups[group].dsts[0] = (DstBuf==Input ? userInput : userOutput) + dstIx + offset; + } waitPeer(srcIx, dstIx, offset, sliceSize); subBarrier(); /* if user abort the kernel, we don't need to actually perform copy/reduce; just set size @@ -303,6 +302,28 @@ class Primitives< } public: + static inline __device__ void sendPeerNotify(int peer, int connIndex, int steps) { + ncclDevChannelPeer* peerPtr = ncclShmem.channel.peers[peer]; + peerPtr->send[connIndex].step += steps; + st_relaxed_sys_global(peerPtr->send[connIndex].tail, peerPtr->send[connIndex].step); + } + + static inline __device__ void recvPeerNotify(int peer, int connIndex, int steps) { + int spins = 0; + ncclDevChannelPeer* peerPtr = ncclShmem.channel.peers[peer]; + peerPtr->recv[connIndex].step += steps; + st_relaxed_sys_global(peerPtr->recv[connIndex].head, peerPtr->recv[connIndex].step); + while (ld_volatile_global(peerPtr->recv[connIndex].tail) < peerPtr->recv[connIndex].step) { + if (spins++ == NCCL_SPINS_BEFORE_CHECK_ABORT) { + if (*ncclShmem.comm.abortFlag) { + ncclShmem.aborted = 1; + break; + } + spins = 0; + } + } + } + template __device__ __forceinline__ void process(Fn &&fn) { #pragma unroll 1 @@ -371,7 +392,7 @@ class Primitives< if (Send) { // Scatter pre-scales data of input buffer only in non-Direct case constexpr int PreOpSrcs = DirectSend ? 0 : 1; - if (flags & RoleInput) ncclShmem.groups[group].srcs[0] = userBuff + inpIx + offset; + if (tid==0) ncclShmem.groups[group].srcs[0] = (T*)ncclShmem.groups[group].userInput + inpIx + offset; // realSize is not accurate here; but intra-node does not rely on sizes FIFO waitPeer<0, DirectSend, 0, 1, 1, 0>(0, inpIx, offset, realSize); subBarrier(); @@ -391,7 +412,7 @@ class Primitives< } } } else if (Recv) { - if (flags & RoleOutput) ncclShmem.groups[group].dsts[0] = userBuff + outIx + offset; + if (tid==0) ncclShmem.groups[group].dsts[0] = (T*)ncclShmem.groups[group].userOutput + outIx + offset; ssize_t pOffset = index*peerOffset; if (skip >= 0 && index >= skip) pOffset += peerElem; // Adjust remote index with peer offset in case we are directly pulling from peer's output buffer @@ -436,6 +457,7 @@ class Primitives< flags |= (conn->flags & NCCL_NVLS_MIN_POLL) ? NvlsMinPolling : 0; connStepPtr = conn->tail; connStepCache = loadStepValue(connStepPtr); + connStepSize = conn->stepSize/sizeof(T); connEltsFifo = (T*)conn->buffs[NCCL_PROTO_SIMPLE]; if (conn->connFifo != nullptr) { flags |= ConnFifoEnabled; @@ -484,6 +506,7 @@ class Primitives< flags |= (conn->flags & NCCL_NVLS_MIN_POLL) ? NvlsMinPolling : 0; connStepPtr = conn->head; connStepCache = loadStepValue(connStepPtr); + connStepSize = conn->stepSize/sizeof(T); connEltsFifo = (T*)conn->buffs[NCCL_PROTO_SIMPLE]; if (connFifo == nullptr && Direct) { // User buffers have been registered @@ -528,24 +551,19 @@ class Primitives< while (nsend < MaxSend && sendPeers[nsend] != -1) nsend++; this->fan = Fan(nrecv, nsend); - constexpr int ThreadPerSync = 8; + constexpr int ThreadPerSync = + MaxSend >= 16 || MaxRecv >= 16 ? 32 : // NVLS may have an arity > 8. In that case increase the size of the groups + MaxSend >= 8 || MaxRecv >= 8 ? 16 : + 8; // Allows for all roles (WaitRecv/WaitSend/PostRecv/PostSend) within a single warp static_assert(MaxSend <= ThreadPerSync && MaxRecv <= ThreadPerSync, "Not enough threads to cover all peers"); - int g = tid / ThreadPerSync; - int ng = nthreads / ThreadPerSync; - index = tid % ThreadPerSync; + index = -1; flags = 0; - if (g == 0) { - if (index < nrecv) flags |= RoleWaitRecv; - if (index == nrecv) flags |= RoleInput; - } else if (g == 1) { - if (index < nsend) flags |= RoleWaitSend; - if (index == nsend) flags |= RoleOutput; - } else if (g == ng - 2) { - if (index < nrecv) flags |= RolePostRecv; - } else if (g == ng - 1) { - if (index < nsend) flags |= RolePostSend; - } + assert(2*(nrecv+nsend) <= nthreads); // Ensure no thread is assigned more than one role. + if (tid < nrecv) { flags |= RoleWaitRecv; index = tid; } + else if (tid < nrecv+nsend) { flags |= RoleWaitSend; index = tid-nrecv; } + else if (nthreads-nsend <= tid) { flags |= RolePostSend; index = tid-(nthreads-nsend); } + else if (nthreads-nrecv-nsend <= tid) { flags |= RolePostRecv; index = tid-(nthreads-nrecv-nsend); } int peer = 0; if (flags & (RoleWaitRecv|RolePostRecv)) peer = recvPeers[index]; @@ -558,15 +576,11 @@ class Primitives< if (barrierAny(flags & NetDeviceUnpack)) { flags |= AnyNetDeviceUnpack; - // g == 0 is the first ThreadPerSync # of threads of this warp - // g == 0 is also the RoleWaitRecv threads of this group, thus the thread ID will correlate to the peer index - if (g == 0) { - uint32_t mask = __ballot_sync((1U << ThreadPerSync) - 1, (flags & NetDeviceUnpack) ? 1 : 0); - - // We only want to update the shared memory variable with a single thread - if (tid == 0) { - ncclShmem.groups[this->group].devicePlugin.unpack.unpackNetDeviceIndexMask = mask; - } + // RoleWaitRecv starts at tid=0, so this creates the bitmask of which recv peers + // have NetDeviceUnpack. + uint32_t mask = __ballot_sync(~0u, ((flags & RoleWaitRecv) && (flags & NetDeviceUnpack)) ? 1 : 0); + if (tid == 0) { + ncclShmem.groups[this->group].devicePlugin.unpack.unpackNetDeviceIndexMask = mask; } } @@ -588,7 +602,8 @@ class Primitives< // was accessed directly. uint64_t prevStep = step - StepPerSlice; volatile ssize_t* ptr = &(connFifo[prevStep%NCCL_STEPS].size); - while (*ptr != -1); + int spins = 0; + while (*ptr != -1) if (checkAbort(spins)) break; } if ((flags & (AnyNetDeviceUnpack)) && (flags & (RoleWaitRecv))) { @@ -601,11 +616,11 @@ class Primitives< } __device__ void setDataPtrs(void const *inputBuf, void *outputBuf, uint64_t redOpArg, struct ncclWorkElemReg* e) { - if (flags & RoleInput) { - userBuff = (T*)inputBuf; + if (tid==0) { + ncclShmem.groups[group].userInput = (void*)inputBuf; + ncclShmem.groups[group].userOutput = (void*)outputBuf; ncclShmem.redOpArgs[0] = redOpArg; // scaler for local input } - if (flags & RoleOutput) userBuff = (T*)outputBuf; bool recvProvider = flags == (flags|RoleWaitRecv|DirectWrite); bool sendAcceptor = (flags == (flags|RoleWaitSend|DirectWrite)) || (flags == (flags|RoleWaitSend|NvlsDirectWrite)); bool sendProvider = flags == (flags|RoleWaitSend|DirectRead); // sender provides direct buffer (to be fetched) @@ -696,8 +711,10 @@ class Primitives< } __device__ void moveDataPtrs(intptr_t delta) { - if (flags & (RoleInput|RoleOutput)) - userBuff += delta; + if (tid==0) { + ncclShmem.groups[group].userInput = (T*)ncclShmem.groups[group].userInput + delta; + ncclShmem.groups[group].userOutput = (T*)ncclShmem.groups[group].userOutput + delta; + } } __device__ __forceinline__ void send(intptr_t inpIx, int eltN) { diff --git a/src/device/reduce_scatter.h b/src/device/reduce_scatter.h index 96a63caeb..d0b52494e 100644 --- a/src/device/reduce_scatter.h +++ b/src/device/reduce_scatter.h @@ -262,16 +262,24 @@ struct RunWorkElement send to network - Primitives, /*Direct=*/0, Proto, 0> - prims(tid, tn, direct->heads+1, &direct->out, nullptr, nullptr, - args->redOpArg, 1*Proto::MaxGroupWidth, 1, 1); - for (ssize_t railGridOffset=0; railGridOffset < nNodes*sizePerRank; railGridOffset += nChannels*chunkSize) { - Scatterer scat; - scat.args = args; - scat.chunkSize = chunkSize; - scat.railGridOffset = railGridOffset; - prims.process(scat); + if (args->regUsed == NCCL_COLLNET_REG_BUFFER) { + if (tid == 0) { + int steps = (int)divUp(nNodes * sizePerRank * sizeof(T), NCCL_MAX_COLLNET_SIZE); + Primitives, /*Direct=*/0, Proto, 0>::sendPeerNotify(direct->out, 1, steps); + } + __syncwarp(); + } else { + // Phase 2: Reduce from peers + local input -> send to network + Primitives, /*Direct=*/0, Proto, 0> + prims(tid, tn, direct->heads + 1, &direct->out, nullptr, nullptr, + args->redOpArg, 1 * Proto::MaxGroupWidth, 1, 1); + for (ssize_t railGridOffset = 0; railGridOffset < nNodes * sizePerRank; railGridOffset += nChannels * chunkSize) { + Scatterer scat; + scat.args = args; + scat.chunkSize = chunkSize; + scat.railGridOffset = railGridOffset; + prims.process(scat); + } } return; } @@ -279,18 +287,26 @@ struct RunWorkElement, /*Direct=*/0, Proto, 0> - prims(tid, tn, &direct->out, nullptr, nullptr, args->recvbuff, - args->redOpArg, 2*Proto::MaxGroupWidth, 0, 0); - for (ssize_t railGridOffset=0; railGridOffset < nNodes*sizePerRank; railGridOffset += nChannels*chunkSize) { - ssize_t railAllBeg = railGridOffset + args->bid*chunkSize; - ssize_t railAllEnd = min(railAllBeg + chunkSize, nNodes*sizePerRank); - ssize_t railOneBeg = ncclShmem.comm.node*sizePerRank; - ssize_t railOneEnd = railOneBeg + sizePerRank; - ssize_t beg = max(railAllBeg, railOneBeg); - ssize_t end = min(railAllEnd, railOneEnd); - prims.recv(beg-railOneBeg, max(ssize_t(0), end-beg), /*postOp=*/true); + if (args->regUsed == NCCL_COLLNET_REG_BUFFER) { + if (tid == 0) { + int steps = (int)divUp(nNodes * sizePerRank * sizeof(T), NCCL_MAX_COLLNET_SIZE); + Primitives, /*Direct=*/0, Proto, 0>::recvPeerNotify(direct->out, 0, steps); + } + __syncwarp(); + } else { + // Phase 3: recv from network + Primitives, /*Direct=*/0, Proto, 0> + prims(tid, tn, &direct->out, nullptr, nullptr, args->recvbuff, + args->redOpArg, 2 * Proto::MaxGroupWidth, 0, 0); + for (ssize_t railGridOffset = 0; railGridOffset < nNodes * sizePerRank; railGridOffset += nChannels * chunkSize) { + ssize_t railAllBeg = railGridOffset + args->bid * chunkSize; + ssize_t railAllEnd = min(railAllBeg + chunkSize, nNodes * sizePerRank); + ssize_t railOneBeg = ncclShmem.comm.node * sizePerRank; + ssize_t railOneEnd = railOneBeg + sizePerRank; + ssize_t beg = max(railAllBeg, railOneBeg); + ssize_t end = min(railAllEnd, railOneEnd); + prims.recv(beg - railOneBeg, max(ssize_t(0), end - beg), /*postOp=*/true); + } } return; } diff --git a/src/enqueue.cc b/src/enqueue.cc index 8ff4e0220..af57f1be4 100644 --- a/src/enqueue.cc +++ b/src/enqueue.cc @@ -680,6 +680,36 @@ static ncclResult_t registerIntraNodeBuffers( } } info->regBufType = NCCL_IPC_REG_BUFFER; + } else if ((info->algorithm == NCCL_ALGO_COLLNET_DIRECT || info->algorithm == NCCL_ALGO_COLLNET_CHAIN) && comm->collNetRegSupport && info->opFull.op != ncclDevPreMulSum && info->opFull.op != ncclDevSumPostDiv) { + int sendRegBufFlag = 0; + int recvRegBufFlag = 0; + void *sendHandle, *recvHandle; + + if (ncclParamLocalRegister()) { + ncclCollnetLocalRegisterBuffer(comm, info->sendbuff, info->sendbuffSize, collNetSend, &sendRegBufFlag, &sendHandle); + info->sendMhandle = sendHandle; + if (sendRegBufFlag) { + ncclCollnetLocalRegisterBuffer(comm, info->recvbuff, info->recvbuffSize, collNetRecv, &recvRegBufFlag, &recvHandle); + info->recvMhandle = recvHandle; + } + } + + if ((sendRegBufFlag == 0 || recvRegBufFlag == 0) && plan->persistent && ncclParamGraphRegister()) { + ncclCollnetGraphRegisterBuffer(comm, plan, info->sendbuff, info->sendbuffSize, collNetSend, &sendRegBufFlag, &sendHandle); + info->sendMhandle = sendHandle; + if (sendRegBufFlag) { + ncclCollnetGraphRegisterBuffer(comm, plan, info->recvbuff, info->recvbuffSize, collNetRecv, &recvRegBufFlag, &recvHandle); + info->recvMhandle = recvHandle; + } + } + + if (sendRegBufFlag && recvRegBufFlag) { + info->nChannels = std::max(comm->config.minCTAs, std::min(comm->config.maxCTAs, 1)); + info->regBufType = NCCL_COLLNET_REG_BUFFER; + if (sendRegBufFlag == 1 && recvRegBufFlag == 1) { + INFO(NCCL_REG, "rank %d successfully registered collNet sendbuff %p (handle %p), sendbuff size %ld, recvbuff %p (handle %p), recvbuff size %ld", comm->rank, info->sendbuff, sendHandle, info->sendbuffSize, info->recvbuff, recvHandle, info->recvbuffSize); + } + } } fallback: #endif @@ -806,7 +836,7 @@ static ncclResult_t scheduleCollTasksToPlan( while (!ncclIntruQueueEmpty(&tasks->collCBDQueue)) { // Get nChannels and peek whether the budget allows before we enqueue collInfo = ncclIntruQueueHead(&tasks->collCBDQueue); - collInfo->nChannels = DIVUP(collInfo->aggnBytes * tasks->usableChannels, totalCBDBytes); + collInfo->nChannels = DIVUP(collInfo->workBytes * tasks->usableChannels, totalCBDBytes); // Haven't got nChannels info yet, relax the budget boundary a bit. if (*nWorkBudget < collInfo->nChannels) return ncclSuccess; @@ -1173,6 +1203,12 @@ static ncclResult_t reclaimPlan(struct ncclComm* comm, struct ncclCommCallback* INFO(NCCL_NVLS, "rank %d - deregistered buffer %p on device %d, size %ld", comm->rank, (void*)obj->ptr, obj->dev, obj->size); ncclMemoryPoolFree(&comm->memPool_ncclNvlsHandleList, obj); } + while (!ncclIntruQueueEmpty(&plan->collnetHandleQueue)) { + struct ncclCollnetHandleList* obj = ncclIntruQueueDequeue(&plan->collnetHandleQueue); + NCCLCHECK(ncclCollnetDeregBuffer(comm, obj->proxyconn, obj->collnetHandle)); + INFO(NCCL_REG, "rank %d - deregistered collnet buffer handle %p, size %ld, buff %p", comm->rank, obj->collnetHandle, obj->size, obj->buffer); + ncclMemoryPoolFree(&comm->memPool_ncclCollnetHandleList, obj); + } } ncclMemoryPoolFree(&comm->memPool_ncclKernelPlan, plan); return ncclSuccess; @@ -1512,7 +1548,7 @@ static ncclResult_t getTunerInfo(struct ncclInfo* collInfo, int collNetSupport, collInfo->nChannels = 0; if (collInfo->comm->tuner != NULL) { NCCLCHECK(collInfo->comm->tuner->getCollInfo( - collInfo->coll, collInfo->nBytes, + collInfo->comm->tunerContext, collInfo->coll, collInfo->nBytes, collNetSupport, nvlsSupport, numPipeOps, &collInfo->algorithm, &collInfo->protocol, &collInfo->nChannels)); } @@ -1649,7 +1685,7 @@ static ncclResult_t setCollWorkElem(uint64_t workCount, uint64_t workOffset, siz static ncclResult_t initCollWorkElemReg(struct ncclComm* comm, struct ncclWorkElem* work, struct ncclChannel* channel, ncclRegBufferType regBufType, void* regBufSend[], void* regBufRecv[], struct ncclWorkElemReg* workElemReg) { if (regBufType == NCCL_IPC_REG_BUFFER) { workElemReg->elem = *work; - workElemReg->elem.regUsed = 1; + workElemReg->elem.regUsed = NCCL_IPC_REG_BUFFER; for (int i = 0; i < NCCL_MAX_DIRECT_ARITY; i++) { int peer = channel->collnetDirect.down[i]; if (peer == -1) break; @@ -1666,10 +1702,13 @@ static ncclResult_t initCollWorkElemReg(struct ncclComm* comm, struct ncclWorkEl } } else if (regBufType == NCCL_NVLS_REG_BUFFER) { workElemReg->elem = *work; - workElemReg->elem.regUsed = 1; + workElemReg->elem.regUsed = NCCL_NVLS_REG_BUFFER; /* NVLS only has one send and recv buffer registered */ workElemReg->dnInputs[0] = regBufSend[0]; workElemReg->dnOutputs[0] = regBufRecv[0]; + } else if (regBufType == NCCL_COLLNET_REG_BUFFER) { + workElemReg->elem = *work; + workElemReg->elem.regUsed = NCCL_COLLNET_REG_BUFFER; } else { /* impossible value */ WARN("Invalid regBufType %d\n", regBufType); @@ -1678,7 +1717,7 @@ static ncclResult_t initCollWorkElemReg(struct ncclComm* comm, struct ncclWorkEl return ncclSuccess; } -NCCL_PARAM(NvlsTreeChunkSize, "NVLSTREE_MAX_CHUNKSIZE", -2); +NCCL_PARAM(NvlsTreeMaxChunkSize, "NVLSTREE_MAX_CHUNKSIZE", -2); static ncclResult_t computeCollChunkInfo(struct ncclInfo* collInfo, size_t nBytes, int nChannels) { int stepSize = collInfo->comm->buffSizes[collInfo->protocol] / NCCL_STEPS; @@ -1701,7 +1740,7 @@ static ncclResult_t computeCollChunkInfo(struct ncclInfo* collInfo, size_t nByte while (nBytes / (nChannels * chunkSize) < collInfo->comm->channels[0].collnetChain.depth * 8 && chunkSize > 65536) chunkSize /= 2; while (nBytes / (nChannels * chunkSize) < collInfo->comm->channels[0].collnetChain.depth && chunkSize > 32768) chunkSize /= 2; } else if (collInfo->algorithm == NCCL_ALGO_NVLS) { - int maxChunkSize = 131072; + int maxChunkSize = collInfo->comm->nvlsChunkSize; if (collInfo->comm->nNodes > 1 && collInfo->comm->bandwidths[ncclFuncAllReduce][NCCL_ALGO_NVLS][NCCL_PROTO_SIMPLE] < 150) maxChunkSize = 32768; if (chunkSize > maxChunkSize) chunkSize = maxChunkSize; // Use uint64_t so that concurrentOps*chunkSize*X does not overflow @@ -1712,7 +1751,8 @@ static ncclResult_t computeCollChunkInfo(struct ncclInfo* collInfo, size_t nByte } else if (collInfo->algorithm == NCCL_ALGO_NVLS_TREE) { // Use uint64_t so that concurrentOps*chunkSize*X does not overflow uint64_t concurrentOps = nChannels * collInfo->comm->channels[0].nvls.nHeads; - int maxChunkSize = ncclParamNvlsTreeChunkSize(); + chunkSize = collInfo->comm->nvlsChunkSize; + int maxChunkSize = (int)ncclParamNvlsTreeMaxChunkSize(); if (maxChunkSize == -2) maxChunkSize = collInfo->comm->nNodes >= 4 ? 65536 : chunkSize; chunkSize = std::min(chunkSize, maxChunkSize); if ((nBytes < (32 * (concurrentOps * chunkSize))) && (chunkSize > 262144)) chunkSize = 262144; @@ -1747,11 +1787,22 @@ static ncclResult_t initCollProxyOp(struct ncclInfo* collInfo, int channelId, ui proxyOp->pattern = collInfo->pattern; proxyOp->coll = collInfo->coll; proxyOp->root = collInfo->root; - proxyOp->reg = 0; // This is used by P2P to reduce the receive buffer size. We don't use it in collectives // because some protocols need to transmit more than the total size, plus they sometimes // round up proxyOp->nbytes = collInfo->stepSize * proxyOp->sliceSteps; + if (collInfo->regBufType == NCCL_COLLNET_REG_BUFFER) { + proxyOp->reg = 1; + proxyOp->nsteps = DIVUP(collInfo->nBytes, NCCL_MAX_COLLNET_SIZE); + proxyOp->sendMhandle = collInfo->sendMhandle; + proxyOp->recvMhandle = collInfo->recvMhandle; + proxyOp->sendbuff = (uint8_t*)collInfo->sendbuff; + proxyOp->recvbuff = (uint8_t*)collInfo->recvbuff; + proxyOp->nbytes = collInfo->nBytes; + } else { + proxyOp->reg = 0; + } + proxyOp->channelId = channelId; proxyOp->opCount = opCount; @@ -1958,7 +2009,7 @@ ncclResult_t ncclEnqueueCheck(struct ncclInfo* info) { ncclResult_t ret = ncclSuccess; int devOld = -1; - NCCLCHECKGOTO(PtrCheck(info->comm, info->opName, "comm"), ret, fail); + NCCLCHECKGOTO(CommCheck(info->comm, info->opName, "comm"), ret, fail); // Check whether communicator is ready to communicate NCCLCHECKGOTO(ncclCommEnsureReady(info->comm), ret, fail); @@ -1990,7 +2041,7 @@ ncclResult_t ncclEnqueueCheck(struct ncclInfo* info) { NCCL_API(ncclResult_t, ncclRedOpCreatePreMulSum, ncclRedOp_t *op, void *scalar, ncclDataType_t datatype, ncclScalarResidence_t residence, ncclComm_t comm); ncclResult_t ncclRedOpCreatePreMulSum(ncclRedOp_t *op, void *scalar, ncclDataType_t datatype, ncclScalarResidence_t residence, ncclComm_t comm) { - NCCLCHECK(PtrCheck(comm, "ncclRedOpCreatePreMulSum", "comm")); + NCCLCHECK(CommCheck(comm, "ncclRedOpCreatePreMulSum", "comm")); /* join init thread before creating PreMulSum op. */ NCCLCHECK(ncclCommEnsureReady(comm)); diff --git a/src/graph/connect.cc b/src/graph/connect.cc index 86efcbaf4..90687bb6a 100644 --- a/src/graph/connect.cc +++ b/src/graph/connect.cc @@ -17,6 +17,7 @@ ncclResult_t ncclTopoPreset(struct ncclComm* comm, struct ncclTopoGraph** graphs, struct ncclTopoRanks* topoRanks) { int rank = comm->rank; int localRanks = comm->topo->nodes[GPU].count; + int nvlsRanks = comm->MNNVL ? comm->clique.size : localRanks; int nChannels = comm->nChannels; topoRanks->nvlsHeadNum = 0; @@ -71,7 +72,7 @@ ncclResult_t ncclTopoPreset(struct ncclComm* comm, struct ncclTopoGraph** graphs // Get nvls heads and the number of heads. Duplicate head is not allowed. for (int c = 0; c < graphs[NCCL_ALGO_NVLS]->nChannels; ++c) { bool addHead = true; - int* nvlsIntra = graphs[NCCL_ALGO_NVLS]->intra + c * localRanks; + int* nvlsIntra = graphs[NCCL_ALGO_NVLS]->intra + c * nvlsRanks; for (int dup = 0; dup < topoRanks->nvlsHeadNum; dup++) { if (topoRanks->nvlsHeads[dup] == nvlsIntra[0]) { @@ -257,8 +258,7 @@ static ncclResult_t connectNvls(struct ncclComm* comm, int* nvlsHeads, int nHead channel->nvls.nNodes = comm->nNodes; if (comm->collNetSupport && channel->nvls.headRank != -1) channel->nvls.out = comm->nRanks; } - // MNNVL: NVLS not yet supported - if (comm->nNodes == 1 || comm->MNNVL) return ncclSuccess; + if (comm->nNodes == 1) return ncclSuccess; // Connect Trees int tree0Parent, tree0Child0, tree0Child1, tree1Parent, tree1Child0, tree1Child1; @@ -310,9 +310,9 @@ static ncclResult_t connectNvls(struct ncclComm* comm, int* nvlsHeads, int nHead struct ncclNvls* nvls0 = &comm->channels[0].nvls; struct ncclNvls* nvls1 = &comm->channels[1].nvls; - INFO(NCCL_GRAPH, "NVLS Trees : %d/%d->%d->%d %d/%d->%d->%d", - nvls0->treeDown[0], nvls0->treeDown[1], comm->rank, nvls0->treeUp, - nvls1->treeDown[0], nvls1->treeDown[1], comm->rank, nvls1->treeUp); + INFO(NCCL_GRAPH, "NVLS Trees : %d/%d/%d->%d->%d %d/%d/%d->%d->%d", + nvls0->treeDown[0], nvls0->treeDown[1], nvls0->treeDown[2], comm->rank, nvls0->treeUp, + nvls1->treeDown[0], nvls1->treeDown[1], nvls1->treeDown[2], comm->rank, nvls1->treeUp); return ncclSuccess; } @@ -363,13 +363,14 @@ void exchangeValues(int* v0, int* v1) { *v0 = tmp; } -ncclResult_t ncclTopoPostset(struct ncclComm* comm, int* firstRanks, int* treePatterns, struct ncclTopoRanks** allTopoRanks, int* rings, struct ncclTopoGraph** graphs) { +ncclResult_t ncclTopoPostset(struct ncclComm* comm, int* firstRanks, int* treePatterns, struct ncclTopoRanks** allTopoRanks, int* rings, struct ncclTopoGraph** graphs, struct ncclComm* parent) { // Gather data from all ranks int *ringRecv, *ringSend, *ringPrev, *ringNext, *treeToParent, *treeToChild0, *treeToChild1, *nvlsHeads; int nranks = comm->nRanks; int nNodes = comm->nNodes; int nChannels = comm->nChannels; int minHeadNum = INT_MAX; + int shared = parent && parent->nvlsSupport && parent->config.splitShare; NCCLCHECK(ncclCalloc(&ringRecv, nNodes*MAXCHANNELS)); NCCLCHECK(ncclCalloc(&ringSend, nNodes*MAXCHANNELS)); NCCLCHECK(ncclCalloc(&ringPrev, nranks*MAXCHANNELS)); @@ -380,7 +381,7 @@ ncclResult_t ncclTopoPostset(struct ncclComm* comm, int* firstRanks, int* treePa NCCLCHECK(ncclCalloc(&nvlsHeads, nNodes*MAXCHANNELS)); // Alternate rings to avoid crossing rails - if (graphs[NCCL_ALGO_RING]->crossNic && (comm->nNodes % 2) == 0 && (nChannels % 2) == 0) { + if (graphs[NCCL_ALGO_RING]->crossNic && (nChannels % 2) == 0) { for (int r=0; rnRanks; r++) { if (comm->rankToNode[r] % 2 == 1) { // Exchange rings @@ -469,11 +470,20 @@ ncclResult_t ncclTopoPostset(struct ncclComm* comm, int* firstRanks, int* treePa } comm->collChannels = comm->nChannels; +#if CUDART_VERSION >= 12010 // Support maximal channel usage for aggregation + if (shared && comm->nvlsChannels > parent->nvlsResources->nChannels) { + comm->nvlsChannels = parent->nvlsResources->nChannels; + } if (comm->nChannels < comm->nvlsChannels) { nChannels = comm->nChannels = copyChannels(comm, comm->nChannels, comm->nvlsChannels, ringPrev, ringNext); } NCCLCHECK(connectNvls(comm, nvlsHeads, minHeadNum)); +#endif + if (shared && comm->nChannels > parent->sharedRes->tpNChannels) { + nChannels = comm->nChannels = parent->sharedRes->tpNChannels; + comm->collChannels = std::min(comm->collChannels, comm->nChannels); + } // Create rings array and check all is fine NCCLCHECK(ncclBuildRings(nChannels, rings, comm->rank, comm->nRanks, ringPrev, ringNext)); diff --git a/src/graph/paths.cc b/src/graph/paths.cc index dea2e7086..e033c5b45 100644 --- a/src/graph/paths.cc +++ b/src/graph/paths.cc @@ -58,6 +58,7 @@ static ncclResult_t ncclTopoSetPaths(struct ncclTopoNode* baseNode, struct ncclT struct ncclTopoNode* remNode = link->remNode; if (remNode->paths[baseNode->type] == NULL) { NCCLCHECK(ncclCalloc(remNode->paths+baseNode->type, system->nodes[baseNode->type].count)); + for (int i=0; inodes[baseNode->type].count; i++) remNode->paths[baseNode->type][i].type = PATH_DIS; } struct ncclTopoLinkList* remPath; NCCLCHECK(getPath(system, remNode, baseNode->type, baseNode->id, &remPath)); @@ -110,11 +111,12 @@ static ncclResult_t ncclTopoSetPaths(struct ncclTopoNode* baseNode, struct ncclT } static void printNodePaths(struct ncclTopoSystem* system, struct ncclTopoNode* node) { - char line[1024]; + const int linesize = 1024; + char line[linesize]; #ifdef ENABLE_TRACE INFO(NCCL_GRAPH, "Paths from %s/%lX :", topoNodeTypeStr[node->type], node->id); #else - sprintf(line, "%s/%lX :", topoNodeTypeStr[node->type], node->id); + snprintf(line, linesize, "%s/%lX :", topoNodeTypeStr[node->type], node->id); int offset = strlen(line); #endif for (int t=0; tpaths[t][n].count; i++) { struct ncclTopoLink* link = node->paths[t][n].list[i]; struct ncclTopoNode* remNode = link->remNode; - sprintf(line+offset, "--%s(%g)->%s/%lX", topoLinkTypeStr[link->type], link->bw, topoNodeTypeStr[remNode->type], remNode->id); + snprintf(line+offset, linesize-offset, "--%s(%g)->%s/%lx-%lx", topoLinkTypeStr[link->type], link->bw, topoNodeTypeStr[remNode->type], NCCL_TOPO_ID_SYSTEM_ID(remNode->id), NCCL_TOPO_ID_LOCAL_ID(remNode->id)); offset = strlen(line); } INFO(NCCL_GRAPH, "%s (%f)", line, node->paths[t][n].bw); #else - sprintf(line+offset, "%s/%lX (%d/%f/%s) ", topoNodeTypeStr[t], system->nodes[t].nodes[n].id, node->paths[t][n].count, node->paths[t][n].bw, topoPathTypeStr[node->paths[t][n].type]); + snprintf(line+offset, linesize-offset, "%s/%lx-%lx (%d/%.1f/%s) ", topoNodeTypeStr[t], NCCL_TOPO_ID_SYSTEM_ID(system->nodes[t].nodes[n].id), NCCL_TOPO_ID_LOCAL_ID(system->nodes[t].nodes[n].id), node->paths[t][n].count, node->paths[t][n].bw, topoPathTypeStr[node->paths[t][n].type]); offset = strlen(line); #endif } @@ -361,12 +363,12 @@ ncclResult_t ncclTopoCheckMNNVL(struct ncclTopoSystem* system, struct ncclPeerIn NCCL_PARAM(NetGdrRead, "NET_GDR_READ", -2); int ncclTopoUserGdrLevel = -1; -ncclResult_t ncclTopoCheckGdr(struct ncclTopoSystem* system, int64_t busId, int netDev, int read, int* useGdr) { +ncclResult_t ncclTopoCheckGdr(struct ncclTopoSystem* system, int64_t busId, int64_t netId, int read, int* useGdr) { *useGdr = 0; // Get GPU and NET int n, g; - NCCLCHECK(ncclTopoIdToIndex(system, NET, netDev, &n)); + NCCLCHECK(ncclTopoIdToIndex(system, NET, netId, &n)); struct ncclTopoNode* net = system->nodes[NET].nodes+n; NCCLCHECK(ncclTopoIdToIndex(system, GPU, busId, &g)); struct ncclTopoNode* gpu = system->nodes[GPU].nodes+g; @@ -403,18 +405,18 @@ ncclResult_t ncclTopoCheckGdr(struct ncclTopoSystem* system, int64_t busId, int if (distance == PATH_PXN) { // In case of PXN, use the intermediate GPU distance instead int proxyRank, g; - NCCLCHECK(ncclTopoGetIntermediateRank(system, gpu->gpu.rank, netDev, &proxyRank)); + NCCLCHECK(ncclTopoGetIntermediateRank(system, gpu->gpu.rank, netId, &proxyRank)); NCCLCHECK(ncclTopoRankToIndex(system, proxyRank, &g)); struct ncclTopoNode* proxyGpu = system->nodes[GPU].nodes+g; distance = proxyGpu->paths[NET][n].type; } if (distance > netGdrLevel) { - INFO(NCCL_NET,"GPU Direct RDMA Disabled for GPU %lx / HCA %d (distance %d > %d)", busId, netDev, distance, netGdrLevel); + INFO(NCCL_NET,"GPU Direct RDMA Disabled for GPU %lx / HCA %lx (distance %d > %d)", busId, netId, distance, netGdrLevel); return ncclSuccess; } *useGdr = 1; - INFO(NCCL_NET,"GPU Direct RDMA Enabled for GPU %lx / HCA %d (distance %d <= %d), read %d", busId, netDev, distance, netGdrLevel, read); + INFO(NCCL_NET,"GPU Direct RDMA Enabled for GPU %lx / HCA %lx (distance %d <= %d), read %d", busId, netId, distance, netGdrLevel, read); return ncclSuccess; } @@ -465,10 +467,10 @@ ncclResult_t ncclTopoCheckNet(struct ncclTopoSystem* system, int64_t id1, int64_ return ncclSuccess; } -ncclResult_t ncclTopoGetIntermediateRank(struct ncclTopoSystem* system, int rank, int netDev, int* intermediateRank) { +ncclResult_t ncclTopoGetIntermediateRank(struct ncclTopoSystem* system, int rank, int64_t netId, int* intermediateRank) { // Get GPU and NET int n, g; - NCCLCHECK(ncclTopoIdToIndex(system, NET, netDev, &n)); + NCCLCHECK(ncclTopoIdToIndex(system, NET, netId, &n)); NCCLCHECK(ncclTopoRankToIndex(system, rank, &g)); struct ncclTopoNode* gpu = system->nodes[GPU].nodes+g; struct ncclTopoLinkList* path = gpu->paths[NET]+n; @@ -480,7 +482,7 @@ ncclResult_t ncclTopoGetIntermediateRank(struct ncclTopoSystem* system, int rank type = node->type; } if (type != GPU) { - WARN("Could not find intermediate GPU between GPU rank %d and NIC %d", rank, netDev); + WARN("Could not find intermediate GPU between GPU rank %d and NIC %lx", rank, netId); return ncclInternalError; } *intermediateRank = node->gpu.rank; @@ -516,11 +518,12 @@ ncclResult_t ncclTopoGetPxnRanks(struct ncclComm* comm, int** intermediateRanks, int nr = 0; int* ranks = NULL; for (int rank=0; ranknRanks; rank++) { - int netDev, proxyRank; - NCCLCHECK(ncclTopoGetNetDev(comm, comm->rank, NULL, 0, rank, &netDev, &proxyRank)); + int64_t netId; + int proxyRank; + NCCLCHECK(ncclTopoGetNetDev(comm, comm->rank, NULL, 0, rank, &netId, NULL, &proxyRank)); if (proxyRank == comm->rank) continue; int useGdr; - NCCLCHECK(ncclTopoCheckGdr(comm->topo, comm->busId, netDev, 1, &useGdr)); + NCCLCHECK(ncclTopoCheckGdr(comm->topo, comm->busId, netId, 1, &useGdr)); if (useGdr == 0) continue; int found = 0; for (int r=0; rnodes[GPU].nodes+g; if (ncclPxnDisable(comm) != 1) { int localGpuIndex; - NCCLCHECK(ncclTopoGetLocalGpu(system, system->nodes[NET].nodes[n].id, &localGpuIndex)); + NCCLCHECK(ncclTopoGetLocalGpu(system, netNode->id, &localGpuIndex)); if (localGpuIndex != g && localGpuIndex != -1) { // PXN = PCI + NVLink. struct ncclTopoNode* peerNode = system->nodes[GPU].nodes+localGpuIndex; // Only use PXN for NIC n if remote GPU p ... if (peerNode->paths[NET][n].type <= PATH_PXB && // Is connected to the NIC through PCI peerNode->paths[GPU][g].type <= PATH_NVL && // Is connected to us through NVLink + NCCL_TOPO_ID_SYSTEM_ID(peerNode->id) == NCCL_TOPO_ID_SYSTEM_ID(gpu->id) && // Is on the same node as us (peerNode->paths[NET][n].bw > gpu->paths[NET][n].bw || // Has either higher BW to that NIC gpu->paths[NET][n].type > PATH_PXB)) // or avoids going through a CPU // We can use that GPU as relay to communicate with that NIC. @@ -618,15 +622,17 @@ ncclResult_t ncclTopoComputePaths(struct ncclTopoSystem* system, struct ncclComm NCCLCHECK(addInterStep(system, GPU, localGpuIndex, GPU, g, NET, n)); } } - // Update path when we dont want to / can't use GPU Direct RDMA. - int gdr; - NCCLCHECK(ncclTopoCheckGdr(system, system->nodes[GPU].nodes[g].id, netNode->id, 0, &gdr)); - if (gdr == 0) { - // We cannot use GPU Direct RDMA, divert all traffic through the CPU local to the GPU - int localCpu; - NCCLCHECK(getLocalCpu(system, g, &localCpu)); - NCCLCHECK(addInterStep(system, CPU, localCpu, NET, n, GPU, g)); - NCCLCHECK(addInterStep(system, CPU, localCpu, GPU, g, NET, n)); + if (gpu->paths[NET][n].type < PATH_PHB) { + // Update path when we dont want to / can't use GPU Direct RDMA. + int gdr; + NCCLCHECK(ncclTopoCheckGdr(system, system->nodes[GPU].nodes[g].id, netNode->id, 0, &gdr)); + if (gdr == 0) { + // We cannot use GPU Direct RDMA, divert all traffic through the CPU local to the GPU + int localCpu; + NCCLCHECK(getLocalCpu(system, g, &localCpu)); + NCCLCHECK(addInterStep(system, CPU, localCpu, NET, n, GPU, g)); + NCCLCHECK(addInterStep(system, CPU, localCpu, GPU, g, NET, n)); + } } } } @@ -669,8 +675,7 @@ ncclResult_t ncclTopoTrimSystem(struct ncclTopoSystem* system, struct ncclComm* NCCLCHECK(ncclTopoRemoveNode(system, GPU, g)); } - // MNNVL: Remove network nodes as they are connected via NVLink - if (system->nodes[GPU].count == comm->nRanks || comm->MNNVL) { + if (system->nodes[GPU].count == comm->nRanks) { for (int n=system->nodes[NET].count-1; n>=0; n--) NCCLCHECK(ncclTopoRemoveNode(system, NET, n)); } @@ -704,11 +709,6 @@ static ncclResult_t ncclTopoGetNchannels(struct ncclComm* comm, int g /*local gp } else { *nChannels = 2; } - } else if (comm->MNNVL) { - // MNNVL assume all GPUs are connected via NVLink - path = system->nodes[GPU].nodes[g].paths[GPU]+((g+1)%system->nodes[GPU].count); - float nvlBw = ncclTopoNVLinkBw(system->nodes[GPU].nodes[g].gpu.cudaCompCap); - *nChannels = 2*std::max(1, (int)(path->bw / nvlBw)); } else { // Remote rank, use network int nNetChannels = ncclParamNChannelsPerNetPeer(); diff --git a/src/graph/search.cc b/src/graph/search.cc index b189165eb..c7b4d96ae 100644 --- a/src/graph/search.cc +++ b/src/graph/search.cc @@ -4,6 +4,7 @@ * See LICENSE.txt for license information ************************************************************************/ +#include "comm.h" #include "core.h" #include "graph.h" #include "topo.h" @@ -39,6 +40,7 @@ ncclResult_t ncclTopoSearchInit(struct ncclTopoSystem* system) { int inter = system->nodes[NET].count; if (inter == 0 && system->nodes[GPU].count == 1) { system->maxBw = LOC_BW; + system->totalBw = LOC_BW; return ncclSuccess; } for (int g=0; gnodes[GPU].count; g++) { @@ -115,7 +117,6 @@ static ncclResult_t ncclTopoFollowPath(struct ncclTopoSystem* system, struct ncc WARN("No path computed to go from %s/%d to %s/%d", topoNodeTypeStr[type1], index1, topoNodeTypeStr[type2], index2); return ncclInternalError; } - if (path->count == 0 ) return ncclSuccess; // Now check link type *node = NULL; @@ -217,7 +218,7 @@ static ncclResult_t getNetIndex(struct ncclTopoSystem* system, int64_t id, int* } static ncclResult_t getNetPaths(struct ncclTopoSystem* system, struct ncclTopoGraph* graph, struct ncclTopoLinkList** netPaths) { - int netId = graph->inter[graph->nChannels*2]; + int64_t netId = graph->inter[graph->nChannels*2]; int n; NCCLCHECK(getNetIndex(system, netId, &n)); *netPaths=system->nodes[NET].nodes[n].paths[GPU]; @@ -261,6 +262,8 @@ ncclResult_t ncclTopoSearchNextGpuSort(struct ncclTopoSystem* system, struct ncc for (int i=0; inodes[NVS].count) { // NVSwitches prefer when we talk to a limited set of peers. Try to use neighbors first. int index = gpu-system->nodes[GPU].nodes; @@ -277,16 +280,18 @@ ncclResult_t ncclTopoSearchNextGpuSort(struct ncclTopoSystem* system, struct ncc } else { firstGpus[0] = nextGpu; firstGpuCount = 1; } + if (nextGpu == prevGpu && firstGpuCount == 2) firstGpuCount = 1; + int firstGpuRealCount = 0; for (int g=0; g0; i--) next[i] = next[i-1]; next[0] = firstGpus[g]; + firstGpuRealCount++; } } + *countPtr = firstGpuRealCount; } - - *countPtr = count; return ncclSuccess; } @@ -372,7 +377,6 @@ ncclResult_t ncclTopoCompareGraphs(struct ncclTopoSystem* system, struct ncclTop return ncclSuccess; } // 2. Try to get better bandwidth - // Give a 5% perf bonus to paths not crossing nics if (graph->nChannels*graph->bwIntra > refGraph->nChannels*refGraph->bwIntra) { *copy = 1; return ncclSuccess; @@ -405,8 +409,8 @@ ncclResult_t ncclTopoSelectNets(struct ncclTopoSystem* system, int typeInter, in localNetCount = 0; struct ncclTopoNode* gpu = system->nodes[GPU].nodes+g; for (int c = 0; cgpu.rank, c, &netId)); + int64_t netId; + NCCLCHECK(ncclTopoGetLocalNet(system, gpu->gpu.rank, c, &netId, NULL)); NCCLCHECK(ncclTopoIdToIndex(system, NET, netId, localNets+localNetCount)); if (localNetCount > 0 && localNets[localNetCount] == localNets[0]) break; localNetCount++; @@ -427,7 +431,7 @@ ncclResult_t ncclTopoSelectNets(struct ncclTopoSystem* system, int typeInter, in localNetCount = 0; struct ncclTopoNode* gpu = system->nodes[GPU].nodes+g; struct ncclTopoLinkList* paths = gpu->paths[NET]; - for (int n=0; nnodes[NET].count; n++) { + for (int n=0; nnodes[NET].count && nnodes[GPU].count; - int* inter = graph->inter+2*c; + int64_t* inter = graph->inter+2*c; int* intra = graph->intra+ngpus*c; int n=0, g=0; for (int s=0; snSubs; s++) { struct ncclXmlNode* sub = xmlChannel->subs[s]; - int dev; - NCCLCHECK(xmlGetAttrInt(sub, "dev", &dev)); + int64_t dev; + const char* str; + NCCLCHECK(xmlGetAttrStr(sub, "dev", &str)); + dev = strtol(str, NULL, 16); if (strcmp(sub->name, "net") == 0) { inter[n++] = dev; } else if (strcmp(sub->name, "gpu") == 0) { int rank = -1; for (int g=0; gnodes[GPU].nodes[g].gpu.dev == dev) rank = system->nodes[GPU].nodes[g].gpu.rank; + int systemId = NCCL_TOPO_ID_SYSTEM_ID(system->nodes[GPU].nodes[g].id); + if (NCCL_TOPO_ID(systemId, system->nodes[GPU].nodes[g].gpu.dev) == dev) rank = system->nodes[GPU].nodes[g].gpu.rank; } if (rank == -1) { - WARN("XML Import Channel : dev %d not found.", dev); + WARN("XML Import Channel : dev %ld not found.", dev); return ncclSystemError; } intra[g++] = rank; @@ -763,29 +770,33 @@ ncclResult_t ncclTopoGetGraphFromXml(struct ncclXmlNode *xmlGraphs, struct ncclT ncclResult_t ncclTopoGetXmlFromChannel(struct ncclTopoGraph* graph, int c, struct ncclTopoSystem* system, struct ncclXml *xml, struct ncclXmlNode* parent) { struct ncclXmlNode* xmlChannel; int ngpus = system->nodes[GPU].count; - int* inter = graph->inter+2*c; + int64_t* inter = graph->inter+2*c; int* intra = graph->intra+ngpus*c; NCCLCHECK(xmlAddNode(xml, parent, "channel", &xmlChannel)); struct ncclXmlNode* node; if (system->nodes[NET].count) { NCCLCHECK(xmlAddNode(xml, xmlChannel, "net", &node)); - NCCLCHECK(xmlSetAttrInt(node, "dev", inter[0])); + NCCLCHECK(xmlSetAttrLong(node, "dev", inter[0])); } for (int g=0; gnodes[GPU].nodes[i].gpu.rank == intra[g]) dev = system->nodes[GPU].nodes[i].gpu.dev; + if (system->nodes[GPU].nodes[i].gpu.rank == intra[g]) { + int systemId = NCCL_TOPO_ID_SYSTEM_ID(system->nodes[GPU].nodes[i].id); + dev = NCCL_TOPO_ID(systemId, system->nodes[GPU].nodes[i].gpu.dev); + } } if (dev == -1) { WARN("XML Export Channel : rank %d not found.", intra[g]); return ncclInternalError; } - NCCLCHECK(xmlSetAttrInt(node, "dev", dev)); + NCCLCHECK(xmlSetAttrLong(node, "dev", dev)); + if (graph->id == 3) break; // NVLS graphs only use the first GPU } if (system->nodes[NET].count) { NCCLCHECK(xmlAddNode(xml, xmlChannel, "net", &node)); - NCCLCHECK(xmlSetAttrInt(node, "dev", inter[1])); + NCCLCHECK(xmlSetAttrLong(node, "dev", inter[1])); } return ncclSuccess; } @@ -829,7 +840,7 @@ ncclResult_t ncclTopoDupChannels(struct ncclTopoGraph* graph, int ccMin, int ngp int dupChannels = std::min(graph->nChannels*2, graph->maxChannels); memcpy(graph->intra+graph->nChannels*ngpus, graph->intra, (dupChannels-graph->nChannels)*ngpus*sizeof(int)); - memcpy(graph->inter+graph->nChannels*2,graph->inter, (dupChannels-graph->nChannels)*2*sizeof(int)); + memcpy(graph->inter+graph->nChannels*2,graph->inter, (dupChannels-graph->nChannels)*2*sizeof(int64_t)); graph->bwIntra /= DIVUP(dupChannels, graph->nChannels); graph->bwInter /= DIVUP(dupChannels, graph->nChannels); graph->nChannels = dupChannels; @@ -841,7 +852,7 @@ float speedArrayInter[] = { 48.0, 30.0, 28.0, 24.0, 20.0, 18.0, 15.0, 12.0, 10.0 #define NSPEEDSINTRA (sizeof(speedArrayIntra)/sizeof(float)) #define NSPEEDSINTER (sizeof(speedArrayInter)/sizeof(float)) -float sm90SpeedArrayIntra[] = { 60.0, 50.0, 40.0, 30.0, 24.0, 20.0, 15.0, 12.0, 6.0, 3.0 }; +float sm90SpeedArrayIntra[] = { 60.0, 50.0, 40.0, 30.0, 24.0, 20.0, 15.0, 12.0, 11.0, 6.0, 3.0 }; float sm90SpeedArrayInter[] = { 48.0, 45.0, 42.0, 40.0, 30.0, 24.0, 22.0, 20.0, 17.5, 15.0, 12.0, 6.0, 3.0, 2.4, 1.2, 0.24, 0.12 }; #define NSPEEDSINTRA_SM90 (sizeof(sm90SpeedArrayIntra)/sizeof(float)) #define NSPEEDSINTER_SM90 (sizeof(sm90SpeedArrayInter)/sizeof(float)) @@ -868,7 +879,7 @@ ncclResult_t ncclTopoCompute(ncclTopoSystem* system, struct ncclTopoGraph* graph if (str) { INFO(NCCL_ENV, "NCCL_GRAPH_FILE set by environment to %s", str); struct ncclXml* xml; - NCCLCHECK(ncclCalloc(&xml, 1)); + NCCLCHECK(xmlAlloc(&xml, NCCL_GRAPH_XML_MAX_NODES)); NCCLCHECK(ncclTopoGetXmlGraphFromFile(str, xml)); int nChannels; NCCLCHECK(ncclTopoGetGraphFromXml(xml->nodes, system, graph, &nChannels)); @@ -907,7 +918,7 @@ ncclResult_t ncclTopoCompute(ncclTopoSystem* system, struct ncclTopoGraph* graph int speedIndex = 0; float maxBw = system->maxBw; float totalBw = system->totalBw; - if (ngpus == 1 || graph->pattern != NCCL_TOPO_PATTERN_RING) totalBw *= ngpus*1.0/(ngpus-1); + if (ngpus > 1 && graph->pattern != NCCL_TOPO_PATTERN_RING) totalBw *= ngpus*1.0/(ngpus-1); while ((speedArray[speedIndex] > maxBw || speedArray[speedIndex]*graph->minChannels > totalBw) && speedIndex < nspeeds-1) speedIndex++; tmpGraph.bwIntra = tmpGraph.bwInter = speedArray[speedIndex]; int64_t globalTimeout = NCCL_SEARCH_GLOBAL_TIMEOUT; @@ -926,7 +937,7 @@ ncclResult_t ncclTopoCompute(ncclTopoSystem* system, struct ncclTopoGraph* graph for (int g=0; gintra[c*ngpus+g]); } - printf("[%d %d]", graph->inter[c*2+0], graph->inter[c*2+1]); + printf("[%lx %lx]", graph->inter[c*2+0], graph->inter[c*2+1]); printf("\n"); } #endif @@ -1041,7 +1052,7 @@ ncclResult_t ncclTopoPrintGraph(struct ncclTopoSystem* system, struct ncclTopoGr sprintf(line, "%2d :", c); int offset = strlen(line); if (system->nodes[NET].count > 0) { - sprintf(line+offset, " %s/%d", topoNodeTypeStr[NET], graph->inter[2*c]); + sprintf(line+offset, " %s/%lx", topoNodeTypeStr[NET], graph->inter[2*c]); offset = strlen(line); } for (int i=0; inodes[NET].count > 0) { - sprintf(line+offset, " %s/%d", topoNodeTypeStr[NET], graph->inter[2*c+1]); + sprintf(line+offset, " %s/%lx", topoNodeTypeStr[NET], graph->inter[2*c+1]); offset = strlen(line); } INFO(NCCL_GRAPH, "%s", line); @@ -1062,7 +1073,7 @@ ncclResult_t ncclTopoDumpGraphs(struct ncclTopoSystem* system, int ngraphs, stru if (str) { INFO(NCCL_ENV, "NCCL_GRAPH_DUMP_FILE set by environment to %s", str); struct ncclXml* xml; - NCCLCHECK(ncclCalloc(&xml, 1)); + NCCLCHECK(xmlAlloc(&xml, NCCL_GRAPH_XML_MAX_NODES)); NCCLCHECK(ncclTopoGetXmlFromGraphs(ngraphs, graphs, system, xml)); NCCLCHECK(ncclTopoDumpXmlToFile(str, xml)); free(xml); @@ -1072,11 +1083,11 @@ ncclResult_t ncclTopoDumpGraphs(struct ncclTopoSystem* system, int ngraphs, stru #include "comm.h" // NVLS channels aren't compute channels. Find which NIC corresponds to our rank being the head -ncclResult_t getNvlsNetDev(struct ncclComm* comm, struct ncclTopoGraph* graph, int channelId, int* dev) { +ncclResult_t getNvlsNetDev(struct ncclComm* comm, struct ncclTopoGraph* graph, int channelId, int64_t* netId) { ncclResult_t ret = ncclSuccess; int localRanks = comm->topo->nodes[GPU].count; int netNum = 0; - int net[MAXCHANNELS]; + int64_t net[MAXCHANNELS]; for (int c = 0; c < graph->nChannels; c++) { if (graph->intra[c * localRanks] == comm->rank) { @@ -1084,7 +1095,7 @@ ncclResult_t getNvlsNetDev(struct ncclComm* comm, struct ncclTopoGraph* graph, i } } if (netNum) { - *dev = net[channelId % netNum]; + *netId = net[channelId % netNum]; } else { ret = ncclInternalError; goto fail; @@ -1100,23 +1111,30 @@ ncclResult_t getNvlsNetDev(struct ncclComm* comm, struct ncclTopoGraph* graph, i // 0: don't use PXN for P2P, 1: use PXN if needed, 2: use PXN as much as possible to maximize aggregation NCCL_PARAM(P2pPxnLevel, "P2P_PXN_LEVEL", 2); -ncclResult_t ncclTopoGetNetDev(struct ncclComm* comm, int rank, struct ncclTopoGraph* graph, int channelId, int peerRank, int* dev, int* proxyRank) { +ncclResult_t ncclTopoGetNetDev(struct ncclComm* comm, int rank, struct ncclTopoGraph* graph, int channelId, int peerRank, int64_t* id, int* dev, int* proxyRank) { + int64_t netId = -1; + int netDev = -1; if (graph) { // Honor the net device in the graph int channel = channelId%graph->nChannels; int ngpus = comm->topo->nodes[GPU].count; int index = graph->intra[channel*ngpus] == rank ? 0 : 1; if (graph->pattern != NCCL_TOPO_PATTERN_NVLS) { - *dev = graph->inter[channel*2+index]; + netId = graph->inter[channel*2+index]; } else { - NCCLCHECK(getNvlsNetDev(comm, graph, channelId, dev)); + NCCLCHECK(getNvlsNetDev(comm, graph, channelId, &netId)); } - NCCLCHECK(ncclTopoGetIntermediateRank(comm->topo, rank, *dev, proxyRank)); + NCCLCHECK(ncclTopoIdToNetDev(comm->topo, netId, &netDev)); + if (dev) *dev = netDev; + if (id) *id = netId; + NCCLCHECK(ncclTopoGetIntermediateRank(comm->topo, rank, netId, proxyRank)); } else if (peerRank == -1) { return ncclInternalError; } else { // Start with our local NIC and local Rank - NCCLCHECK(ncclTopoGetLocalNet(comm->topo, rank, channelId, dev)); + NCCLCHECK(ncclTopoGetLocalNet(comm->topo, rank, channelId, &netId, &netDev)); + if (dev) *dev = netDev; + if (id) *id = netId; *proxyRank = rank; int pxnLevel = ncclPxnDisable(comm) == 1 ? 0 : ncclParamP2pPxnLevel(); @@ -1126,38 +1144,35 @@ ncclResult_t ncclTopoGetNetDev(struct ncclComm* comm, int rank, struct ncclTopoG int nvmlDev = comm->peerInfo[peerRank].nvmlDev; int localRank; if (ncclTopoDevToRank(comm->topo, nvmlDev, &localRank) != ncclSuccess) return ncclSuccess; - int netDev; - NCCLCHECK(ncclTopoGetLocalNet(comm->topo, localRank, channelId, &netDev)); + NCCLCHECK(ncclTopoGetLocalNet(comm->topo, localRank, channelId, &netId, &netDev)); - int n; // Check that device exists on our node if (ncclParamCrossNic() == 0) { - if (ncclTopoIdToIndex(comm->topo, NET, netDev, &n) != ncclSuccess) { - WARN("Rank %d requires NIC %d but that NIC is not available for rank %d", peerRank, netDev, rank); - return ncclInvalidUsage; - } - *dev = netDev; + if (dev) *dev = netDev; + if (id) *id = netId; } if (pxnLevel == 1) { int g, n; NCCLCHECK(ncclTopoRankToIndex(comm->topo, rank, &g)); - NCCLCHECK(ncclTopoIdToIndex(comm->topo, NET, netDev, &n)); + NCCLCHECK(ncclTopoIdToIndex(comm->topo, NET, netId, &n)); struct ncclTopoNode* gpu = comm->topo->nodes[GPU].nodes+g; if (gpu->paths[NET][n].type <= PATH_PXN) { - *dev = netDev; + if (dev) *dev = netDev; + if (id) *id = netId; NCCLCHECK(ncclTopoGetIntermediateRank(comm->topo, rank, *dev, proxyRank)); } } else if (pxnLevel == 2) { // Check which local GPU corresponds to that NIC and see if we can use PXN. int n, g1, g2; - NCCLCHECK(ncclTopoIdToIndex(comm->topo, NET, netDev, &n)); + NCCLCHECK(ncclTopoIdToIndex(comm->topo, NET, netId, &n)); NCCLCHECK(ncclTopoRankToIndex(comm->topo, rank, &g1)); - NCCLCHECK(ncclTopoGetLocalGpu(comm->topo, netDev, &g2)); + NCCLCHECK(ncclTopoGetLocalGpu(comm->topo, netId, &g2)); if (g2 != -1) { struct ncclTopoNode* peerGpu = comm->topo->nodes[GPU].nodes+g2; if (peerGpu->paths[GPU][g1].type <= PATH_NVL && peerGpu->paths[NET][n].type <= PATH_PXB) { *proxyRank = peerGpu->gpu.rank; - *dev = netDev; + if (dev) *dev = netDev; + if (id) *id = netId; return ncclSuccess; } } diff --git a/src/graph/topo.cc b/src/graph/topo.cc index 402fa15ee..30304582f 100644 --- a/src/graph/topo.cc +++ b/src/graph/topo.cc @@ -15,13 +15,14 @@ #include #include "xml.h" #include "cpuset.h" +#include "bootstrap.h" #define BUSID_SIZE (sizeof("0000:00:00.0")) #define BUSID_REDUCED_SIZE (sizeof("0000:00")) const char* topoNodeTypeStr[] = { "GPU", "PCI", "NVS", "CPU", "NIC", "NET" }; const char* topoLinkTypeStr[] = { "LOC", "NVL", "", "PCI", "", "", "", "SYS", "NET" }; -const char* topoPathTypeStr[] = { "LOC", "NVL", "NVB", "PIX", "PXB", "PXN", "PHB", "SYS", "DIS" }; +const char* topoPathTypeStr[] = { "LOC", "NVL", "NVB", "PIX", "PXB", "PXN", "PHB", "SYS", "NET", "DIS" }; /******************************************************************/ /******************* Graph Creation Functions *********************/ @@ -156,9 +157,13 @@ ncclResult_t ncclTopoRemoveNode(struct ncclTopoSystem* system, int type, int ind ncclResult_t ncclTopoConnectNodes(struct ncclTopoNode* node, struct ncclTopoNode* remNode, int type, float bw) { // Aggregate links into higher bw for NVLink struct ncclTopoLink* link; - for (link = node->links; link->remNode; link++) { + for (link = node->links; link - node->links != NCCL_TOPO_MAX_LINKS && link->remNode; link++) { if (link->remNode == remNode && link->type == type) break; } + if (link - node->links == NCCL_TOPO_MAX_LINKS) { + WARN("Error : too many Topo links (max %d)", NCCL_TOPO_MAX_LINKS); + return ncclInternalError; + } if (link->remNode == NULL) node->nlinks++; link->type = type; link->remNode = remNode; @@ -218,6 +223,10 @@ ncclResult_t ncclTopoFlattenBcmSwitches(struct ncclTopoSystem* system) { struct ncclTopoNode* remNode = sub->links[l].remNode; if (remNode == pciSwitch) continue; // Add link from parent PCI switch -> PCI device + if (pciSwitch->nlinks == NCCL_TOPO_MAX_LINKS) { + WARN("Error : too many Topo links (max %d)", NCCL_TOPO_MAX_LINKS); + return ncclInternalError; + } memcpy(pciSwitch->links+pciSwitch->nlinks, sub->links+l, sizeof(struct ncclTopoLink)); pciSwitch->nlinks++; // Update link from PCI device -> parent PCI switch @@ -243,11 +252,13 @@ ncclResult_t ncclTopoFlattenBcmSwitches(struct ncclTopoSystem* system) { ncclResult_t ncclTopoConnectCpus(struct ncclTopoSystem* system) { // And connect all CPU nodes together for (int n=0; nnodes[CPU].count; n++) { + struct ncclTopoNode* cpu1 = system->nodes[CPU].nodes+n; for (int p=0; pnodes[CPU].count; p++) { - if (n == p) continue; + struct ncclTopoNode* cpu2 = system->nodes[CPU].nodes+p; + if (n == p || (NCCL_TOPO_ID_SYSTEM_ID(cpu1->id) != NCCL_TOPO_ID_SYSTEM_ID(cpu2->id))) continue; float bw; - NCCLCHECK(ncclTopoGetInterCpuBw(system->nodes[CPU].nodes+n, &bw)); - NCCLCHECK(ncclTopoConnectNodes(system->nodes[CPU].nodes+n, system->nodes[CPU].nodes+p, LINK_SYS, bw)); + NCCLCHECK(ncclTopoGetInterCpuBw(cpu1, &bw)); + NCCLCHECK(ncclTopoConnectNodes(cpu1, cpu2, LINK_SYS, bw)); } } return ncclSuccess; @@ -255,13 +266,13 @@ ncclResult_t ncclTopoConnectCpus(struct ncclTopoSystem* system) { static ncclResult_t ncclTopoPrintRec(struct ncclTopoNode* node, struct ncclTopoNode* prevNode, char* line, int offset) { if (node->type == GPU) { - sprintf(line+offset, "%s/%lX (%d)", topoNodeTypeStr[node->type], node->id, node->gpu.rank); + sprintf(line+offset, "%s/%lx-%lx (%d)", topoNodeTypeStr[node->type], NCCL_TOPO_ID_SYSTEM_ID(node->id), NCCL_TOPO_ID_LOCAL_ID(node->id), node->gpu.rank); } else if (node->type == CPU) { - sprintf(line+offset, "%s/%lX (%d/%d/%d)", topoNodeTypeStr[node->type], node->id, node->cpu.arch, node->cpu.vendor, node->cpu.model); + sprintf(line+offset, "%s/%lx-%lx (%d/%d/%d)", topoNodeTypeStr[node->type], NCCL_TOPO_ID_SYSTEM_ID(node->id), NCCL_TOPO_ID_LOCAL_ID(node->id), node->cpu.arch, node->cpu.vendor, node->cpu.model); } else if (node->type == PCI) { - sprintf(line+offset, "%s/%lX (%lx)", topoNodeTypeStr[node->type], node->id, node->pci.device); + sprintf(line+offset, "%s/%lx-%lx (%lx)", topoNodeTypeStr[node->type], NCCL_TOPO_ID_SYSTEM_ID(node->id), NCCL_TOPO_ID_LOCAL_ID(node->id), node->pci.device); } else { - sprintf(line+offset, "%s/%lX", topoNodeTypeStr[node->type], node->id); + sprintf(line+offset, "%s/%lx-%lx", topoNodeTypeStr[node->type], NCCL_TOPO_ID_SYSTEM_ID(node->id), NCCL_TOPO_ID_LOCAL_ID(node->id)); } INFO(NCCL_GRAPH, "%s", line); for (int i=0; inet.dev = dev; const char* str; NCCLCHECK(xmlGetAttr(xmlNet, "guid", &str)); if (str) sscanf(str, "0x%lx", &net->net.asic); @@ -356,14 +368,14 @@ ncclResult_t ncclTopoAddNet(struct ncclXmlNode* xmlNet, struct ncclTopoSystem* s return ncclSuccess; } -ncclResult_t ncclTopoAddNic(struct ncclXmlNode* xmlNic, struct ncclTopoSystem* system, struct ncclTopoNode* nic) { +ncclResult_t ncclTopoAddNic(struct ncclXmlNode* xmlNic, struct ncclTopoSystem* system, struct ncclTopoNode* nic, int systemId) { for (int s=0; snSubs; s++) { struct ncclXmlNode* xmlNet = xmlNic->subs[s]; if (strcmp(xmlNet->name, "net") != 0) continue; int index; NCCLCHECK(xmlGetAttrIndex(xmlNet, "dev", &index)); if (index == -1) continue; - NCCLCHECK(ncclTopoAddNet(xmlNet, system, nic)); + NCCLCHECK(ncclTopoAddNet(xmlNet, system, nic, systemId)); } return ncclSuccess; } @@ -382,7 +394,7 @@ struct kvDict kvDictPciGen[] = { { "2.5 GT/s", 15 }, { "5 GT/s", 30 }, { "8 GT/s", 60 }, { "16 GT/s", 120 }, { "32 GT/s", 240 }, /* Kernel 5.6 and earlier */ { "2.5 GT/s PCIe", 15 }, { "5.0 GT/s PCIe", 30 }, { "8.0 GT/s PCIe", 60 }, { "16.0 GT/s PCIe", 120 }, { "32.0 GT/s PCIe", 240 }, { "64.0 GT/s PCIe", 480 }, { NULL, 60 /* Default fallback */ } }; // x100 Mbps per lane -ncclResult_t ncclTopoAddPci(struct ncclXmlNode* xmlPci, struct ncclTopoSystem* system, struct ncclTopoNode* parent) { +ncclResult_t ncclTopoAddPci(struct ncclXmlNode* xmlPci, struct ncclTopoSystem* system, struct ncclTopoNode* parent, int systemId) { const char* str; int type; @@ -401,7 +413,7 @@ ncclResult_t ncclTopoAddPci(struct ncclXmlNode* xmlPci, struct ncclTopoSystem* s int index; NCCLCHECK(xmlGetAttrIndex(xmlGpu, "rank", &index)); if (index == -1) return ncclSuccess; - NCCLCHECK(ncclTopoCreateNode(system, &node, type, busId)); + NCCLCHECK(ncclTopoCreateNode(system, &node, type, NCCL_TOPO_ID(systemId, busId))); NCCLCHECK(ncclTopoAddGpu(xmlGpu, system, node)); } struct ncclXmlNode* xmlNic = NULL; @@ -411,14 +423,15 @@ ncclResult_t ncclTopoAddPci(struct ncclXmlNode* xmlPci, struct ncclTopoSystem* s // Ignore sub device ID and merge multi-port NICs into one PCI device. busId &= 0xfffffffffffffff0; struct ncclTopoNode* nicNode = NULL; - NCCLCHECK(ncclTopoGetNode(system, &nicNode, type, busId)); + int64_t id = NCCL_TOPO_ID(systemId, busId); + NCCLCHECK(ncclTopoGetNode(system, &nicNode, type, id)); if (nicNode == NULL) { - NCCLCHECK(ncclTopoCreateNode(system, &nicNode, type, busId)); + NCCLCHECK(ncclTopoCreateNode(system, &nicNode, type, id)); node = nicNode; // Connect it to parent later on } - NCCLCHECK(ncclTopoAddNic(xmlNic, system, nicNode)); + NCCLCHECK(ncclTopoAddNic(xmlNic, system, nicNode, systemId)); } else if (type == PCI) { - NCCLCHECK(ncclTopoCreateNode(system, &node, type, busId)); + NCCLCHECK(ncclTopoCreateNode(system, &node, type, NCCL_TOPO_ID(systemId, busId))); NCCLCHECK(xmlGetAttr(xmlPci, "vendor", &str)); if (str) node->pci.device += strtol(str, NULL, 0) << 48; NCCLCHECK(xmlGetAttr(xmlPci, "device", &str)); @@ -430,7 +443,7 @@ ncclResult_t ncclTopoAddPci(struct ncclXmlNode* xmlPci, struct ncclTopoSystem* s for (int s=0; snSubs; s++) { struct ncclXmlNode* xmlSubPci = xmlPci->subs[s]; - NCCLCHECK(ncclTopoAddPci(xmlSubPci, system, node)); + NCCLCHECK(ncclTopoAddPci(xmlSubPci, system, node, systemId)); } } @@ -452,11 +465,25 @@ ncclResult_t ncclTopoAddPci(struct ncclXmlNode* xmlPci, struct ncclTopoSystem* s struct kvDict kvDictCpuArch[] = { { "x86_64", NCCL_TOPO_CPU_ARCH_X86 }, { "arm64", NCCL_TOPO_CPU_ARCH_ARM }, { "ppc64", NCCL_TOPO_CPU_ARCH_POWER }, { NULL, 0 } }; struct kvDict kvDictCpuVendor[] = { { "GenuineIntel", NCCL_TOPO_CPU_VENDOR_INTEL }, { "AuthenticAMD", NCCL_TOPO_CPU_VENDOR_AMD }, { "CentaurHauls", NCCL_TOPO_CPU_VENDOR_ZHAOXIN }, { " Shanghai ", NCCL_TOPO_CPU_VENDOR_ZHAOXIN }, { NULL, 0 } }; +ncclResult_t ncclGetSystemId(struct ncclTopoSystem* system, struct ncclXmlNode* xmlCpu, int* systemIdPtr) { + const char* hostHashStr; + NCCLCHECK(xmlGetAttr(xmlCpu, "host_hash", &hostHashStr)); + uint64_t hostHash = hostHashStr ? strtoull(hostHashStr, NULL, 16) : 0; + int systemId; + for (systemId=0; systemIdnHosts; systemId++) if (system->hostHashes[systemId] == hostHash) break; + if (systemId == system->nHosts) system->hostHashes[system->nHosts++] = hostHash; + *systemIdPtr = systemId; + return ncclSuccess; +} + + ncclResult_t ncclTopoAddCpu(struct ncclXmlNode* xmlCpu, struct ncclTopoSystem* system) { int numaId; NCCLCHECK(xmlGetAttrInt(xmlCpu, "numaid", &numaId)); + int systemId; + NCCLCHECK(ncclGetSystemId(system, xmlCpu, &systemId)); struct ncclTopoNode* cpu; - NCCLCHECK(ncclTopoCreateNode(system, &cpu, CPU, numaId)); + NCCLCHECK(ncclTopoCreateNode(system, &cpu, CPU, NCCL_TOPO_ID(systemId, numaId))); const char* str; NCCLCHECK(xmlGetAttr(xmlCpu, "affinity", &str)); if (str != NULL) { @@ -482,26 +509,27 @@ ncclResult_t ncclTopoAddCpu(struct ncclXmlNode* xmlCpu, struct ncclTopoSystem* s } for (int s=0; snSubs; s++) { struct ncclXmlNode* node = xmlCpu->subs[s]; - if (strcmp(node->name, "pci") == 0) NCCLCHECK(ncclTopoAddPci(node, system, cpu)); + if (strcmp(node->name, "pci") == 0) NCCLCHECK(ncclTopoAddPci(node, system, cpu, systemId)); if (strcmp(node->name, "nic") == 0) { struct ncclTopoNode* nic = NULL; NCCLCHECK(ncclTopoGetNode(system, &nic, NIC, 0)); if (nic == NULL) { - NCCLCHECK(ncclTopoCreateNode(system, &nic, NIC, 0)); + NCCLCHECK(ncclTopoCreateNode(system, &nic, NIC, NCCL_TOPO_ID(systemId, 0))); NCCLCHECK(ncclTopoConnectNodes(cpu, nic, LINK_PCI, LOC_BW)); NCCLCHECK(ncclTopoConnectNodes(nic, cpu, LINK_PCI, LOC_BW)); } - NCCLCHECK(ncclTopoAddNic(node, system, nic)); + NCCLCHECK(ncclTopoAddNic(node, system, nic, systemId)); } } return ncclSuccess; } -ncclResult_t ncclTopoAddNvLinks(struct ncclXmlNode* node, struct ncclTopoSystem* system, const char* parentBusId) { +ncclResult_t ncclTopoAddNvLinks(struct ncclXmlNode* node, struct ncclTopoSystem* system, const char* parentBusId, int systemId) { if (strcmp(node->name, "nvlink") == 0) { struct ncclTopoNode* gpu = NULL; int64_t pBusId; NCCLCHECK(busIdToInt64(parentBusId, &pBusId)); + pBusId = NCCL_TOPO_ID(systemId, pBusId); NCCLCHECK(ncclTopoGetNode(system, &gpu, GPU, pBusId)); if (gpu == NULL) { WARN("Add NVLink error : could not find GPU %lx", pBusId); @@ -520,7 +548,7 @@ ncclResult_t ncclTopoAddNvLinks(struct ncclXmlNode* node, struct ncclTopoSystem* NCCLCHECK(xmlGetAttrStr(node, "target", &target)); int64_t busId; NCCLCHECK(busIdToInt64(target, &busId)); - NCCLCHECK(ncclTopoGetNode(system, &remote, GPU, busId)); + NCCLCHECK(ncclTopoGetNode(system, &remote, GPU, NCCL_TOPO_ID(systemId, busId))); } else if (targetType == CPU) { // NVL connection to the local CPU NCCLCHECK(findLocalCpu(gpu, &remote)); @@ -539,20 +567,24 @@ ncclResult_t ncclTopoAddNvLinks(struct ncclXmlNode* node, struct ncclTopoSystem* } } } else { + if (strcmp(node->name, "cpu") == 0) { + NCCLCHECK(ncclGetSystemId(system, node, &systemId)); + } const char* busId; NCCLCHECK(xmlGetAttr(node, "busid", &busId)); for (int s=0; snSubs; s++) { - NCCLCHECK(ncclTopoAddNvLinks(node->subs[s], system, busId ? busId : parentBusId)); + NCCLCHECK(ncclTopoAddNvLinks(node->subs[s], system, busId ? busId : parentBusId, systemId)); } } return ncclSuccess; } -ncclResult_t ncclTopoAddC2c(struct ncclXmlNode* node, struct ncclTopoSystem* system, const char* parentBusId) { +ncclResult_t ncclTopoAddC2c(struct ncclXmlNode* node, struct ncclTopoSystem* system, const char* parentBusId, int systemId) { if (strcmp(node->name, "c2c") == 0) { struct ncclTopoNode* gpu = NULL; int64_t pBusId; NCCLCHECK(busIdToInt64(parentBusId, &pBusId)); + pBusId = NCCL_TOPO_ID(systemId, pBusId); NCCLCHECK(ncclTopoGetNode(system, &gpu, GPU, pBusId)); if (gpu == NULL) { WARN("Add NVLink error : could not find GPU %lx", pBusId); @@ -569,25 +601,31 @@ ncclResult_t ncclTopoAddC2c(struct ncclXmlNode* node, struct ncclTopoSystem* sys NCCLCHECK(ncclTopoConnectNodes(gpu, cpu, LINK_NVL, c2cBw)); NCCLCHECK(ncclTopoConnectNodes(cpu, gpu, LINK_NVL, c2cBw)); } else { + if (strcmp(node->name, "cpu") == 0) { + NCCLCHECK(ncclGetSystemId(system, node, &systemId)); + } const char* busId; NCCLCHECK(xmlGetAttr(node, "busid", &busId)); for (int s=0; snSubs; s++) { - NCCLCHECK(ncclTopoAddC2c(node->subs[s], system, busId ? busId : parentBusId)); + NCCLCHECK(ncclTopoAddC2c(node->subs[s], system, busId ? busId : parentBusId, systemId)); } } return ncclSuccess; } -ncclResult_t ncclTopoGetSystemFromXml(struct ncclXml* xml, struct ncclTopoSystem** topoSystem) { +ncclResult_t ncclTopoGetSystemFromXml(struct ncclXml* xml, struct ncclTopoSystem** topoSystem, const uint64_t localHostHash) { NCCLCHECK(ncclCalloc(topoSystem, 1)); + struct ncclTopoSystem* system = *topoSystem; struct ncclXmlNode* topNode; NCCLCHECK(xmlFindTag(xml, "system", &topNode)); for (int s=0; snSubs; s++) { struct ncclXmlNode* node = topNode->subs[s]; if (strcmp(node->name, "cpu") == 0) NCCLCHECK(ncclTopoAddCpu(node, *topoSystem)); } - NCCLCHECK(ncclTopoAddNvLinks(topNode, *topoSystem, NULL)); - NCCLCHECK(ncclTopoAddC2c(topNode, *topoSystem, NULL)); + for (int systemId=0; systemIdnHosts; systemId++) if (system->hostHashes[systemId] == localHostHash) system->systemId = systemId; + + NCCLCHECK(ncclTopoAddNvLinks(topNode, *topoSystem, NULL, 0)); + NCCLCHECK(ncclTopoAddC2c(topNode, *topoSystem, NULL, 0)); NCCLCHECK(ncclTopoFlattenBcmSwitches(*topoSystem)); NCCLCHECK(ncclTopoConnectCpus(*topoSystem)); @@ -633,7 +671,7 @@ static ncclResult_t xmlInitAttrFloat(struct ncclXmlNode* node, const char* attrN ncclResult_t ncclTopoGetSystem(struct ncclComm* comm, struct ncclTopoSystem** system) { struct ncclXml* xml; - NCCLCHECK(ncclCalloc(&xml, 1)); + NCCLCHECK(xmlAlloc(&xml, NCCL_TOPO_XML_MAX_NODES)); const char* xmlTopoFile = ncclGetEnv("NCCL_TOPO_FILE"); if (xmlTopoFile) { INFO(NCCL_ENV, "NCCL_TOPO_FILE set by environment to %s", xmlTopoFile); @@ -707,13 +745,32 @@ ncclResult_t ncclTopoGetSystem(struct ncclComm* comm, struct ncclTopoSystem** sy // Remove XML branches which don't have a node with keep="1" (typically when importing a topology) NCCLCHECK(ncclTopoTrimXml(xml)); + if (comm->MNNVL) { + // MNNVL clique support + char* mem; + NCCLCHECK(ncclCalloc(&mem, comm->clique.size * xmlMemSize(NCCL_TOPO_XML_MAX_NODES))); + struct ncclXml* rankXml = (struct ncclXml*)(mem+xmlMemSize(NCCL_TOPO_XML_MAX_NODES)*comm->cliqueRank); + memcpy(rankXml, xml, xmlMemSize(NCCL_TOPO_XML_MAX_NODES)); + NCCLCHECK(ncclTopoConvertXml(rankXml, (uintptr_t)xml->nodes, 1)); + NCCLCHECK(bootstrapIntraNodeAllGather(comm->bootstrap, comm->clique.ranks, comm->cliqueRank, comm->clique.size, mem, xmlMemSize(NCCL_TOPO_XML_MAX_NODES))); + struct ncclXml* cliqueXml; + NCCLCHECK(xmlAlloc(&cliqueXml, comm->clique.size*NCCL_TOPO_XML_MAX_NODES)); + for (int i = 0; i < comm->clique.size; i++) { + struct ncclXml* peerXml = (struct ncclXml*)(mem+xmlMemSize(NCCL_TOPO_XML_MAX_NODES)*i); + NCCLCHECK(ncclTopoConvertXml(peerXml, (uintptr_t)peerXml->nodes, 0)); + NCCLCHECK(ncclTopoFuseXml(cliqueXml, peerXml)); + } + free(xml); + xml = cliqueXml; + } + xmlTopoFile = ncclGetEnv("NCCL_TOPO_DUMP_FILE"); if (xmlTopoFile && comm->rank == ncclParamTopoDumpFileRank()) { INFO(NCCL_ENV, "NCCL_TOPO_DUMP_FILE set by environment to %s", xmlTopoFile); NCCLCHECK(ncclTopoDumpXmlToFile(xmlTopoFile, xml)); } - NCCLCHECK(ncclTopoGetSystemFromXml(xml, system)); + NCCLCHECK(ncclTopoGetSystemFromXml(xml, system, comm->peerInfo[comm->rank].hostHash)); free(xml); return ncclSuccess; } @@ -761,7 +818,7 @@ ncclResult_t getLocalNetCountByBw(struct ncclTopoSystem* system, int gpu, int *c return ncclSuccess; } -ncclResult_t ncclTopoGetLocalNet(struct ncclTopoSystem* system, int rank, int channelId, int* id) { +ncclResult_t ncclTopoGetLocalNet(struct ncclTopoSystem* system, int rank, int channelId, int64_t* id, int* dev) { int gpu; NCCLCHECK(ncclTopoRankToIndex(system, rank, &gpu)); int* localNets; @@ -773,15 +830,16 @@ ncclResult_t ncclTopoGetLocalNet(struct ncclTopoSystem* system, int rank, int ch int net = system->nodes[GPU].nodes[gpu].gpu.dev; if (isPow2(localNetCount)) net = mirrorBits(net, localNetCount); net += channelId%(DIVUP(localNetCount,localGpuCount)); - *id = system->nodes[NET].nodes[localNets[net%localNetCount]].id; + if (id) *id = system->nodes[NET].nodes[localNets[net%localNetCount]].id; + if (dev) *dev = system->nodes[NET].nodes[localNets[net%localNetCount]].net.dev; free(localNets); free(localGpus); return ncclSuccess; } -ncclResult_t ncclTopoGetLocalGpu(struct ncclTopoSystem* system, int net, int* gpuIndex) { +ncclResult_t ncclTopoGetLocalGpu(struct ncclTopoSystem* system, int64_t netId, int* gpuIndex) { int netIndex; - NCCLCHECK(ncclTopoIdToIndex(system, NET, net, &netIndex)); + NCCLCHECK(ncclTopoIdToIndex(system, NET, netId, &netIndex)); int* localGpus = NULL; int localGpuCount; NCCLCHECK(ncclTopoGetLocal(system, NET, netIndex, GPU, &localGpus, &localGpuCount, NULL)); @@ -789,9 +847,9 @@ ncclResult_t ncclTopoGetLocalGpu(struct ncclTopoSystem* system, int net, int* gp for (int lg=0; lgnodes[GPU].nodes+g; - int id; - NCCLCHECK(ncclTopoGetLocalNet(system, gpu->gpu.rank, c, &id)); - if (net == id) { + int64_t id; + NCCLCHECK(ncclTopoGetLocalNet(system, gpu->gpu.rank, c, &id, NULL)); + if (netId == id) { *gpuIndex = g; free(localGpus); return ncclSuccess; diff --git a/src/graph/topo.h b/src/graph/topo.h index db1eb6e24..548747913 100644 --- a/src/graph/topo.h +++ b/src/graph/topo.h @@ -88,7 +88,7 @@ struct ncclTopoLink { float bw; struct ncclTopoNode* remNode; }; -#define NCCL_TOPO_MAX_LINKS 32 +#define NCCL_TOPO_MAX_LINKS 128 #define NCCL_TOPO_MAX_HOPS (NCCL_TOPO_MAX_NODES*NCCL_TOPO_NODE_TYPES) struct ncclTopoLinkList { @@ -103,6 +103,10 @@ struct ncclTopoLinkList { #define NCCL_TOPO_UNDEF (-1) +#define NCCL_TOPO_ID_SYSTEM_ID(id) (id >> 56) +#define NCCL_TOPO_ID_LOCAL_ID(id) (id & 0x00ffffffffffffff) +#define NCCL_TOPO_ID(systemid, localid) (((int64_t)systemid << 56) + localid) + struct ncclTopoNode { int type; int64_t id; @@ -115,6 +119,7 @@ struct ncclTopoNode { int gdrSupport; }gpu; struct { + int dev; // Plugin dev number uint64_t asic; int port; float bw; @@ -147,6 +152,9 @@ struct ncclTopoNodeSet { }; struct ncclTopoSystem { + int systemId; + uint64_t hostHashes[NCCL_TOPO_MAX_NODES]; + int nHosts; struct ncclTopoNodeSet nodes[NCCL_TOPO_NODE_TYPES]; float maxBw; float totalBw; @@ -158,9 +166,11 @@ ncclResult_t ncclTopoRemoveNode(struct ncclTopoSystem* system, int type, int id) ncclResult_t ncclTopoConnectNodes(struct ncclTopoNode* node, struct ncclTopoNode* remNode, int type, float bw); ncclResult_t ncclTopoPrintPaths(struct ncclTopoSystem* system); ncclResult_t ncclTopoLoadSystem(const char* xmlTopoFile, struct ncclTopoSystem* system); -ncclResult_t ncclTopoGetIntermediateRank(struct ncclTopoSystem* system, int rank, int netDev, int* intermediateRank); +ncclResult_t ncclTopoGetIntermediateRank(struct ncclTopoSystem* system, int rank, int64_t netId, int* intermediateRank); -ncclResult_t ncclTopoGetSystemFromXml(struct ncclXml* xml, struct ncclTopoSystem** topoSystem); +#define NCCL_TOPO_XML_MAX_NODES 256 +#define NCCL_GRAPH_XML_MAX_NODES 4096 +ncclResult_t ncclTopoGetSystemFromXml(struct ncclXml* xml, struct ncclTopoSystem** topoSystem, uint64_t localHostHash); ncclResult_t ncclTopoGetGraphFromXml(struct ncclXmlNode *xmlGraphs, struct ncclTopoSystem* system, struct ncclTopoGraph* graph, int* nChannels); ncclResult_t ncclTopoGetXmlFromGraphs(int ngraphs, struct ncclTopoGraph** graphs, struct ncclTopoSystem* system, struct ncclXml *xml); @@ -191,6 +201,7 @@ static ncclResult_t ncclTopoRankToIndex(struct ncclTopoSystem* system, int rank, static ncclResult_t ncclTopoDevToRank(struct ncclTopoSystem* system, int dev, int* rank) { *rank = -1; for (int i=0; inodes[GPU].count; i++) { + if (NCCL_TOPO_ID_SYSTEM_ID(system->nodes[GPU].nodes[i].id) != system->systemId) continue; // Only consider GPUs on our node if (system->nodes[GPU].nodes[i].gpu.dev == dev) { *rank = system->nodes[GPU].nodes[i].gpu.rank; return ncclSuccess; @@ -199,6 +210,18 @@ static ncclResult_t ncclTopoDevToRank(struct ncclTopoSystem* system, int dev, in return ncclInternalError; } +static ncclResult_t ncclTopoIdToNetDev(struct ncclTopoSystem* system, int64_t id, int* netDev) { + *netDev = -1; + for (int i=0; inodes[NET].count; i++) { + if (system->nodes[NET].nodes[i].id == id) { + *netDev = system->nodes[NET].nodes[i].net.dev; + return ncclSuccess; + } + } + WARN("Could not find NET with id %lx\n", id); + return ncclInternalError; +} + // Returns NVLink bw in GB/s static float ncclTopoNVLinkBw(int cudaCompCap) { return diff --git a/src/graph/tuning.cc b/src/graph/tuning.cc index 7ca592293..d8f0b6e44 100644 --- a/src/graph/tuning.cc +++ b/src/graph/tuning.cc @@ -66,7 +66,7 @@ static float hwLat [3][NCCL_NUM_ALGORITHMS][NCCL_NUM_PROTOCOLS] = { /* NVLINK */ { /* Tree (LL/LL128/Simple)*/ { .6, 1.25, 28 }, /* Ring (LL/LL128/Simple)*/ { .6, 1.9, 3.4 }, /* CollNetDirect (Simple)*/ { 0, 0, 3.7 }, /* CollNetChain (Simple)*/ { 0, 0, 2.8 }, - /* NVLS */ { 0, 0, 23 }, /* NVLSTree */ { 0, 0, 23 } }, + /* NVLS */ { 0, 0, 25 }, /* NVLSTree */ { 0, 0, 25 } }, /* PCI */ { /* Tree (LL/LL128/Simple)*/ { 1.0, 1.9, 28 }, /* Ring (LL/LL128/Simple)*/ { 1.0, 2.5, 5.7 }, /* CollNetDirect (Simple)*/ { 0, 0, 3.7 }, /* CollNetChain (Simple)*/ { 0, 0, 2.8 }, @@ -86,7 +86,7 @@ static float hwLat [3][NCCL_NUM_ALGORITHMS][NCCL_NUM_PROTOCOLS] = static const double llMaxBws[3][3] = { /* Volta-N1/Intel-N2/Intel-N4) */ {39.0, 39.0, 20.4}, /* Ampere-N1/AMD-N2/AMD-N4) */ {87.7, 22.5 /*avg of ring & tree*/, 19.0}, - /* Hopper-N1/AMD-N2/AMD-N4) */ {87.7, 22.5 /*avg of ring & tree*/, 19.0} + /* Hopper-N1/AMD-N2/AMD-N4) */ {141.0, 45.0 /*avg of ring & tree*/, 35.0} }; static const double perChMaxRingLL128Bws[3][3] = { @@ -132,8 +132,7 @@ ncclResult_t ncclTopoTuneModel(struct ncclComm* comm, int minCompCap, int maxCom comm->maxThreads[NCCL_ALGO_RING][NCCL_PROTO_LL128] = comm->maxThreads[NCCL_ALGO_TREE][NCCL_PROTO_LL128] = getNthreads("NCCL_LL128_NTHREADS", ncclParamLl128Nthreads(), NCCL_LL128_MAX_NTHREADS/4, NCCL_LL128_MAX_NTHREADS, NCCL_LL128_MAX_NTHREADS); - // MNNVL support - treat as a single NVLink connected node - int nNodes = comm->MNNVL ? 1 : comm->nNodes; + int nNodes = comm->nNodes; int nRanks = comm->nRanks; if (nRanks <= 1) return ncclSuccess; @@ -178,7 +177,7 @@ ncclResult_t ncclTopoTuneModel(struct ncclComm* comm, int minCompCap, int maxCom float busBw = graphs[a]->nChannels * bw; // Various model refinements - if (a == NCCL_ALGO_RING && p == NCCL_PROTO_LL) { busBw = std::min(llMaxBw, busBw * ((nNodes > 1 || coll == ncclFuncAllReduce || coll == ncclFuncReduce) ? 1.0/4.0 : 1.0/3.0)); } + if (a == NCCL_ALGO_RING && p == NCCL_PROTO_LL) { busBw = std::min(llMaxBw, busBw * .5); } if (a == NCCL_ALGO_RING && p == NCCL_PROTO_LL128) busBw = std::min(busBw * (ppn < 2 ? 0.7 : 0.92 /*120.0/128.0*/), graphs[a]->nChannels*perChMaxRingLL128Bw); if (a == NCCL_ALGO_TREE) busBw = std::min(busBw*.92, graphs[a]->nChannels*perChMaxTreeBw); if (a == NCCL_ALGO_TREE && p == NCCL_PROTO_LL) busBw = std::min(busBw*1.0/3.8, llMaxBw); @@ -190,7 +189,7 @@ ncclResult_t ncclTopoTuneModel(struct ncclComm* comm, int minCompCap, int maxCom if (coll == ncclFuncAllGather || coll == ncclFuncReduceScatter) { busBw = ppn * bw; // AllGather/ReduceScatter requires 1:1 GPU:NIC - int nicPerNode = comm->collNetHeadsUniqueNum; + int nicPerNode = comm->collNetHeadsNum; if (coll == ncclFuncAllGather && comm->nNodes > 1) { if (!comm->ncclCollNet || !comm->ncclCollNet->iallgather || ppn > nicPerNode) busBw = 0; } @@ -282,15 +281,13 @@ ncclResult_t ncclTopoTuneModel(struct ncclComm* comm, int minCompCap, int maxCom NCCLCHECK(parseList(algoStr, ncclAlgoStr, NCCL_NUM_ALGORITHMS, algoEnable)); } - // MNNVL: NVLS not yet supported - if (comm->nNodes == 1 || comm->MNNVL) algoEnable[NCCL_ALGO_NVLS_TREE] = 0; + if (comm->nNodes == 1) algoEnable[NCCL_ALGO_NVLS_TREE] = 0; // Disable CollNet if it is not supported if (comm->collNetSupport == 0) { algoEnable[NCCL_ALGO_COLLNET_DIRECT] = 0; algoEnable[NCCL_ALGO_COLLNET_CHAIN] = 0; - // MNNVL: NVLS not yet supported - if (comm->nNodes > 1 || comm->MNNVL) algoEnable[NCCL_ALGO_NVLS] = 0; + if (nNodes > 1) algoEnable[NCCL_ALGO_NVLS] = 0; // If user has hard set NCCL_ALGO=COLLNET, ignore it if (algoEnable[NCCL_ALGO_RING] == 0 && algoEnable[NCCL_ALGO_TREE] == 0 && algoEnable[NCCL_ALGO_NVLS] == 0 && algoEnable[NCCL_ALGO_NVLS_TREE] == 0) { @@ -437,7 +434,7 @@ ncclResult_t ncclTopoGetAlgoTime(struct ncclInfo* info, int algorithm, int proto int logSize = log2i(info->nBytes>>6); if (algorithm == NCCL_ALGO_TREE && logSize < 23) bw *= treeCorrectionFactor[protocol][logSize]; if (info->nChannels != 0) bw = bw / info->comm->nChannels * info->nChannels; - if (algorithm == NCCL_ALGO_RING && protocol == NCCL_PROTO_SIMPLE && (!info->comm->MNNVL && info->comm->nNodes > 1) + if (algorithm == NCCL_ALGO_RING && protocol == NCCL_PROTO_SIMPLE && info->comm->nNodes > 1 && info->coll == ncclFuncAllReduce && info->nBytes/(info->comm->nChannels*info->comm->nRanks) >= 64) { lat *= info->comm->minCompCap < 80 ? 1.9 : 1.4; // Plateau effect of ring } diff --git a/src/graph/xml.cc b/src/graph/xml.cc index 6f4d9ea61..b145d34ef 100644 --- a/src/graph/xml.cc +++ b/src/graph/xml.cc @@ -172,8 +172,8 @@ struct xmlHandler { ncclResult_t xmlLoadSub(FILE* file, struct ncclXml* xml, struct ncclXmlNode* head, struct xmlHandler handlers[], int nHandlers) { if (head && head->type == NODE_TYPE_SINGLE) return ncclSuccess; while (1) { - if (xml->maxIndex == MAX_NODES) { - WARN("Error : XML parser is limited to 1024 nodes"); + if (xml->maxIndex == xml->maxNodes) { + WARN("Error : XML parser is limited to %d nodes", xml->maxNodes); return ncclInternalError; } struct ncclXmlNode* node = xml->nodes+xml->maxIndex; @@ -198,7 +198,13 @@ ncclResult_t xmlLoadSub(FILE* file, struct ncclXml* xml, struct ncclXmlNode* hea int found = 0; for (int h=0; hname, handlers[h].name) == 0) { - if (head) head->subs[head->nSubs++] = node; + if (head) { + if (head->nSubs == MAX_SUBS) { + WARN("Error : XML parser is limited to %d subnodes", MAX_SUBS); + return ncclInternalError; + } + head->subs[head->nSubs++] = node; + } node->parent = head; node->nSubs = 0; xml->maxIndex++; @@ -218,6 +224,23 @@ ncclResult_t xmlLoadSub(FILE* file, struct ncclXml* xml, struct ncclXmlNode* hea /* XML Writer */ /**************/ +// exp == 1 -- serialize; exp == 0 -- deserialize +ncclResult_t ncclTopoConvertXml(struct ncclXml* xml, uintptr_t base, int exp) { + for (int n = 0; n < xml->maxIndex; n++) { + struct ncclXmlNode *node = &xml->nodes[n]; + + // For "parent", we shift the base by 1 so that we can distinguish actual + // NULL pointers from pointers pointing to the first node. + if (node->parent) + node->parent = (struct ncclXmlNode *) (exp ? ((uintptr_t)node->parent - base + 1) : (base - 1 + (uintptr_t)node->parent)); + + for (int s = 0; s < node->nSubs; s++) { + node->subs[s] = (struct ncclXmlNode *) (exp ? ((uintptr_t)node->subs[s] - base) : (base + (uintptr_t)node->subs[s])); + } + } + return ncclSuccess; +} + ncclResult_t ncclTopoDumpXmlRec(int indent, FILE* file, struct ncclXmlNode* node) { for (int i=0; iname); @@ -249,6 +272,60 @@ ncclResult_t ncclTopoDumpXmlToFile(const char* xmlTopoFile, struct ncclXml* xml) return ncclSuccess; } +ncclResult_t ncclTopoFuseXml(struct ncclXml* dst, struct ncclXml* src) { + struct ncclXmlNode* topNode; + NCCLCHECK(xmlFindTag(dst, "system", &topNode)); + + if (topNode == NULL) { + xmlAddTree(dst, NULL, src->nodes); + return ncclSuccess; + } + + // Fuse the CPUs with the first XML + struct ncclXmlNode* srcCpu; + NCCLCHECK(xmlFindTag(src, "cpu", &srcCpu)); + while (srcCpu) { + const char* srcNumaId; + const char* srcHostHash; + NCCLCHECK(xmlGetAttr(srcCpu, "numaid", &srcNumaId)); + if (srcNumaId == NULL) { + WARN("TopoFuseXmls : could not find CPU numa ID."); + return ncclInternalError; + } + xmlGetAttr(srcCpu, "host_hash", &srcHostHash); + if (srcHostHash == NULL) + srcHostHash = "0"; + + // Search through the destination for a duplicate. Note that + // this makes the complexity of this whole function O(n^2), but n + // is expected to be small. + struct ncclXmlNode* dstCpu; + NCCLCHECK(xmlFindTag(dst, "cpu", &dstCpu)); + while (dstCpu) { + const char* dstNumaId; + const char* dstHostHash; + NCCLCHECK(xmlGetAttr(dstCpu, "numaid", &dstNumaId)); + if (dstNumaId == NULL) { + WARN("TopoFuseXmls : could not find CPU numa ID."); + return ncclInternalError; + } + xmlGetAttr(dstCpu, "host_hash", &dstHostHash); + if (dstHostHash == NULL) + dstHostHash = "0"; + if (strcmp(srcNumaId, dstNumaId) == 0 && strcmp(srcHostHash, dstHostHash) == 0) + break; + + NCCLCHECK(xmlFindNextTag(dst, "cpu", dstCpu, &dstCpu)); + } + // Only add the CPU if no duplicate was found + if (dstCpu == NULL) + NCCLCHECK(xmlAddTree(dst, topNode, srcCpu)); + NCCLCHECK(xmlFindNextTag(src, "cpu", srcCpu, &srcCpu)); + } + return ncclSuccess; +} + + /****************************************/ /* Parser rules for our specific format */ /****************************************/ @@ -556,6 +633,7 @@ ncclResult_t ncclTopoGetXmlFromSys(struct ncclXmlNode* pciNode, struct ncclXml* NCCLCHECK(xmlGetSubKv(topNode, "cpu", &parent, "numaid", numaIdStr)); if (parent == NULL) { NCCLCHECK(xmlAddNode(xml, topNode, "cpu", &parent)); + NCCLCHECK(xmlSetAttrLong(parent, "host_hash", getHostHash())); NCCLCHECK(xmlSetAttr(parent, "numaid", numaIdStr)); } } else if (slashCount == 2) { @@ -581,6 +659,7 @@ ncclResult_t ncclTopoGetXmlFromSys(struct ncclXmlNode* pciNode, struct ncclXml* struct ncclXmlNode* topNode; NCCLCHECK(xmlFindTag(xml, "system", &topNode)); NCCLCHECK(xmlAddNode(xml, topNode, "cpu", &parent)); + NCCLCHECK(xmlSetAttrLong(parent, "host_hash", getHostHash())); NCCLCHECK(xmlSetAttr(parent, "numaid", "-1")); NCCLCHECK(ncclTopoGetXmlFromCpu(parent, xml)); } @@ -595,6 +674,10 @@ ncclResult_t ncclTopoGetXmlFromSys(struct ncclXmlNode* pciNode, struct ncclXml* NCCLCHECK(xmlGetAttr(parent->subs[s], "busid", &busId)); if (busId != NULL && strcmp(newBusId, busId) < 0) { subIndex = s; break; } } + if (parent->nSubs == MAX_SUBS) { + WARN("Error : XML parser is limited to %d subnodes", MAX_SUBS); + return ncclInternalError; + } for (int s = parent->nSubs; s > subIndex; s--) parent->subs[s] = parent->subs[s-1]; parent->subs[subIndex] = pciNode; parent->nSubs++; diff --git a/src/graph/xml.h b/src/graph/xml.h index 73f777d7d..9090ecc0f 100644 --- a/src/graph/xml.h +++ b/src/graph/xml.h @@ -10,13 +10,13 @@ #include "nccl.h" #include "debug.h" #include "checks.h" +#include "alloc.h" #include // A few constraints to make the implementation easy #define MAX_STR_LEN 255 #define MAX_ATTR_COUNT 16 -#define MAX_SUBS 32 -#define MAX_NODES 1024 +#define MAX_SUBS 128 #define NODE_TYPE_NONE 0 #define NODE_TYPE_OPEN 1 @@ -37,8 +37,8 @@ struct ncclXmlNode { }; struct ncclXml { - struct ncclXmlNode nodes[MAX_NODES]; - int maxIndex; + int maxIndex, maxNodes; + struct ncclXmlNode nodes[1]; }; /* File functions */ @@ -55,11 +55,27 @@ ncclResult_t ncclTopoFillNet(struct ncclXml* xml, const char* pciPath, const cha /* Remove unneeded parts */ ncclResult_t ncclTopoTrimXml(struct ncclXml* xml); +/* Fuse multiple system XMLs into one, skipping duplicate CPUs */ +ncclResult_t ncclTopoFuseXml(struct ncclXml* dst, struct ncclXml* src); +/* Relocate pointers in XML to (de-)serialize the structure */ +ncclResult_t ncclTopoConvertXml(struct ncclXml* xml, uintptr_t base, int exp); + /**************/ /* XML Struct */ /* Functions */ /**************/ +static size_t xmlMemSize(int maxNodes) { + return offsetof(struct ncclXml, nodes) + sizeof(struct ncclXmlNode)*maxNodes; +} +static ncclResult_t xmlAlloc(struct ncclXml** xml, int maxNodes) { + char* mem; + NCCLCHECK(ncclCalloc(&mem, xmlMemSize(maxNodes))); + *xml = (struct ncclXml*)mem; + (*xml)->maxNodes = maxNodes; + return ncclSuccess; +} + static ncclResult_t xmlGetAttrIndex(struct ncclXmlNode* node, const char* attrName, int* index) { *index = -1; const int nAttrs = node->nAttrs; @@ -101,6 +117,13 @@ static ncclResult_t xmlGetAttrIntDefault(struct ncclXmlNode* node, const char* a return ncclSuccess; } +static ncclResult_t xmlGetAttrLong(struct ncclXmlNode* node, const char* attrName, int64_t* value) { + const char* str; + NCCLCHECK(xmlGetAttrStr(node, attrName, &str)); + *value = strtol(str, NULL, 0); + return ncclSuccess; +} + static ncclResult_t xmlGetAttrFloat(struct ncclXmlNode* node, const char* attrName, float* value) { const char* str; @@ -121,6 +144,18 @@ static ncclResult_t xmlFindTag(struct ncclXml* xml, const char* tagName, struct return ncclSuccess; } +static ncclResult_t xmlFindNextTag(struct ncclXml* xml, const char* tagName, struct ncclXmlNode* prev, struct ncclXmlNode** node) { + *node = NULL; + for (int i=prev-xml->nodes+1; imaxIndex; i++) { + struct ncclXmlNode* n = xml->nodes+i; + if (strcmp(n->name, tagName) == 0) { + *node = n; + return ncclSuccess; + } + } + return ncclSuccess; +} + static ncclResult_t xmlFindTagKv(struct ncclXml* xml, const char* tagName, struct ncclXmlNode** node, const char* attrName, const char* attrValue) { *node = NULL; for (int i=0; imaxIndex; i++) { @@ -188,6 +223,19 @@ static ncclResult_t xmlSetAttrFloat(struct ncclXmlNode* node, const char* attrNa return ncclSuccess; } +static ncclResult_t xmlSetAttrLong(struct ncclXmlNode* node, const char* attrName, const int64_t value) { + int index; + NCCLCHECK(xmlGetAttrIndex(node, attrName, &index)); + if (index == -1) { + index = node->nAttrs++; + strncpy(node->attrs[index].key, attrName, MAX_STR_LEN); + node->attrs[index].key[MAX_STR_LEN] = '\0'; + } + snprintf(node->attrs[index].value, MAX_STR_LEN, "%#lx", value); + node->attrs[index].value[MAX_STR_LEN] = '\0'; + return ncclSuccess; +} + static ncclResult_t xmlUnsetAttr(struct ncclXmlNode* node, const char* attrName) { int index; NCCLCHECK(xmlGetAttrIndex(node, attrName, &index)); @@ -234,8 +282,8 @@ static ncclResult_t xmlGetSubKvInt(struct ncclXmlNode* node, const char* subName } static ncclResult_t xmlAddNode(struct ncclXml* xml, struct ncclXmlNode* parent, const char* subName, struct ncclXmlNode** sub) { - if (xml->maxIndex == MAX_NODES) { - WARN("Error : too many XML nodes (max %d)", MAX_NODES); + if (xml->maxIndex == xml->maxNodes) { + WARN("Error : too many XML nodes (max %d)", xml->maxNodes); return ncclInternalError; } struct ncclXmlNode* s = xml->nodes+xml->maxIndex++; @@ -243,7 +291,13 @@ static ncclResult_t xmlAddNode(struct ncclXml* xml, struct ncclXmlNode* parent, s->nAttrs = 0; *sub = s; s->parent = parent; - if (parent) parent->subs[parent->nSubs++] = s; + if (parent) { + if (parent->nSubs == MAX_SUBS) { + WARN("Error : too many XML subnodes (max %d)", MAX_SUBS); + return ncclInternalError; + } + parent->subs[parent->nSubs++] = s; + } strncpy(s->name, subName, MAX_STR_LEN); s->name[MAX_STR_LEN] = '\0'; return ncclSuccess; @@ -262,6 +316,29 @@ static ncclResult_t xmlRemoveNode(struct ncclXmlNode* node) { return ncclSuccess; } +static ncclResult_t xmlAddTree(struct ncclXml* dst, struct ncclXmlNode* parent, struct ncclXmlNode* srcNode) { + if (dst->maxIndex == dst->maxNodes) { + WARN("Error : too many XML nodes (max %d)", dst->maxNodes); + return ncclInternalError; + } + struct ncclXmlNode* dstNode = dst->nodes+dst->maxIndex++; + *dstNode = *srcNode; + dstNode->parent = parent; + if (parent) { + if (parent->nSubs == MAX_SUBS) { + WARN("Error : too many XML subnodes (max %d)", MAX_SUBS); + return ncclInternalError; + } + parent->subs[parent->nSubs++] = dstNode; + } + dstNode->nSubs = 0; + // Recursively copy the subtree(s) + for (int i=0; inSubs; i++) + NCCLCHECK(xmlAddTree(dst, dstNode, srcNode->subs[i])); + return ncclSuccess; +} + + // Dictionary for STR -> INT conversions. No dictionary size information, // there needs to be a last element with str == NULL. struct kvDict { diff --git a/src/include/argcheck.h b/src/include/argcheck.h index e4bdc32e2..ebc743224 100644 --- a/src/include/argcheck.h +++ b/src/include/argcheck.h @@ -11,6 +11,7 @@ #include "info.h" ncclResult_t PtrCheck(void* ptr, const char* opname, const char* ptrname); +ncclResult_t CommCheck(struct ncclComm* ptr, const char* opname, const char* ptrname); ncclResult_t ArgsCheck(struct ncclInfo* info); ncclResult_t CudaPtrCheck(const void* pointer, struct ncclComm* comm, const char* ptrname, const char* opname); diff --git a/src/include/bootstrap.h b/src/include/bootstrap.h index 400a479fb..8c5f081c4 100644 --- a/src/include/bootstrap.h +++ b/src/include/bootstrap.h @@ -24,7 +24,9 @@ ncclResult_t bootstrapSplit(struct ncclBootstrapHandle* handle, struct ncclComm* ncclResult_t bootstrapAllGather(void* commState, void* allData, int size); ncclResult_t bootstrapSend(void* commState, int peer, int tag, void* data, int size); ncclResult_t bootstrapRecv(void* commState, int peer, int tag, void* data, int size); -ncclResult_t bootstrapBarrier(void* commState, int *ranks, int rank, int nranks, int tag); +ncclResult_t bootstrapBarrier(void* commState, int rank, int nranks, int tag); +ncclResult_t bootstrapBroadcast(void* commState, int rank, int nranks, int root, void* bcastData, int size); +ncclResult_t bootstrapIntraNodeBarrier(void* commState, int *ranks, int rank, int nranks, int tag); ncclResult_t bootstrapIntraNodeAllGather(void* commState, int *ranks, int rank, int nranks, void* allData, int size); ncclResult_t bootstrapIntraNodeBroadcast(void* commState, int *ranks, int rank, int nranks, int root, void* bcastData, int size); ncclResult_t bootstrapClose(void* commState); diff --git a/src/include/comm.h b/src/include/comm.h index 81da1ad6a..0ba913ada 100644 --- a/src/include/comm.h +++ b/src/include/comm.h @@ -87,6 +87,12 @@ struct ncclNodeRanks { int* localRankToRank; }; +struct cliqueInfo { + int id; + int size; + int *ranks; +}; + struct ncclDestructor { struct ncclDestructor* next; void* obj; @@ -165,6 +171,14 @@ struct ncclNvlsMcHandleList { size_t size; }; +struct ncclCollnetHandleList { + struct ncclCollnetHandleList *next; + void* collnetHandle; + size_t size; + const void* buffer; + struct ncclProxyConnector* proxyconn; +}; + struct ncclKernelPlan { // A kernel plan is also a callback that reclaims itself. Hence this must // be the first member. @@ -188,6 +202,7 @@ struct ncclKernelPlan { struct ncclIntruQueue ipcMemQueue; struct ncclIntruQueue nvlsMcHandleQueue; + struct ncclIntruQueue collnetHandleQueue; struct Channel { int nWork; @@ -202,7 +217,10 @@ struct ncclKernelPlan { size_t maxBytesPerChannel; }; +#define NCCL_MAGIC 0x0280028002800280 // Nickel atomic number is 28. + struct ncclComm { + uint64_t startMagic; struct ncclMemoryStack memPermanent, memScoped; // List of destructors to run when comm is destructed struct ncclDestructor* destructorHead; @@ -245,7 +263,10 @@ struct ncclComm { int* localRankToRank; // localRanks and localRanktoRank for all nodes struct ncclNodeRanks* nodeRanks; - int MNNVL; // MNNVL: Multi-Node NVLink + // MNNVL: Multi-Node NVLink + int MNNVL; // true when MNNVL is available + struct cliqueInfo clique; // Our MNNVL clique information + int cliqueRank; // Our rank within the MNNVL clique bool checkPointers; bool dmaBufSupport; @@ -257,7 +278,6 @@ struct ncclComm { int nChannels; // connection nChannels int collChannels; // enqueue nChannels int nvlsChannels; // enqueue nChannels - int collNetChannels; // Channels (per peer) for p2p int p2pnChannels; int p2pnChannelsPerPeer; @@ -269,6 +289,7 @@ struct ncclComm { // Buffer sizes int buffSizes[NCCL_NUM_PROTOCOLS]; int p2pChunkSize; + int nvlsChunkSize; // Algorithm/Protocols thresholds ssize_t threadThresholds[NCCL_NUM_ALGORITHMS][NCCL_NUM_PROTOCOLS]; @@ -315,11 +336,11 @@ struct ncclComm { int proxyRefCountOld; /* store proxy post-atomic-sub refcount */ // Whether this communicator uses collNet int collNetSupport; + bool collNetRegSupport; uint8_t collNetSupportMatrix[4/*sum,prod,min,max*/][ncclNumTypes]; int intraHighestTransportType; int* collNetHeads; int collNetHeadsNum; - int collNetHeadsUniqueNum; int* collNetDenseToUserRank; int* collNetUserToDenseRank; /* sharable collNet proxy progress resource. */ @@ -336,6 +357,7 @@ struct ncclComm { struct ncclMemoryPool memPool_ncclKernelPlan; struct ncclMemoryPool memPool_ncclPointerList; struct ncclMemoryPool memPool_ncclNvlsHandleList; + struct ncclMemoryPool memPool_ncclCollnetHandleList; // Next comm in this thread's active ncclGroup[Start|End](). Holds "0x1" when // this comm is not yet in a group. struct ncclComm* groupNext; @@ -368,8 +390,10 @@ struct ncclComm { // Tuning plugin ncclTuner_t* tuner; + void *tunerContext; // buffer registration cache struct ncclRegCache regCache; + uint64_t endMagic; }; enum ncclLaunchMode { diff --git a/src/include/cudawrap.h b/src/include/cudawrap.h index 9350306d9..b4eb5c312 100644 --- a/src/include/cudawrap.h +++ b/src/include/cudawrap.h @@ -20,10 +20,6 @@ extern int ncclCuMemEnable(); // Handle type used for cuMemCreate() extern CUmemAllocationHandleType ncclCuMemHandleType; -#else -typedef CUresult (CUDAAPI *PFN_cuInit_v2000)(unsigned int Flags); -typedef CUresult (CUDAAPI *PFN_cuDriverGetVersion_v2020)(int *driverVersion); -typedef CUresult (CUDAAPI *PFN_cuGetProcAddress_v11030)(const char *symbol, void **pfn, int driverVersion, cuuint64_t flags); #endif #define CUPFN(symbol) pfn_##symbol @@ -69,53 +65,47 @@ typedef CUresult (CUDAAPI *PFN_cuGetProcAddress_v11030)(const char *symbol, void } \ } while(0) -#define DECLARE_CUDA_PFN_EXTERN(symbol,version) extern PFN_##symbol##_v##version pfn_##symbol +#define DECLARE_CUDA_PFN_EXTERN(symbol) extern PFN_##symbol pfn_##symbol #if CUDART_VERSION >= 11030 /* CUDA Driver functions loaded with cuGetProcAddress for versioning */ -DECLARE_CUDA_PFN_EXTERN(cuDeviceGet, 2000); -DECLARE_CUDA_PFN_EXTERN(cuDeviceGetAttribute, 2000); -DECLARE_CUDA_PFN_EXTERN(cuGetErrorString, 6000); -DECLARE_CUDA_PFN_EXTERN(cuGetErrorName, 6000); -DECLARE_CUDA_PFN_EXTERN(cuMemGetAddressRange, 3020); -DECLARE_CUDA_PFN_EXTERN(cuCtxCreate, 3020); -DECLARE_CUDA_PFN_EXTERN(cuCtxDestroy, 4000); -DECLARE_CUDA_PFN_EXTERN(cuCtxGetCurrent, 4000); -DECLARE_CUDA_PFN_EXTERN(cuCtxSetCurrent, 4000); -DECLARE_CUDA_PFN_EXTERN(cuCtxGetDevice, 2000); -DECLARE_CUDA_PFN_EXTERN(cuPointerGetAttribute, 4000); +DECLARE_CUDA_PFN_EXTERN(cuDeviceGet); +DECLARE_CUDA_PFN_EXTERN(cuDeviceGetAttribute); +DECLARE_CUDA_PFN_EXTERN(cuGetErrorString); +DECLARE_CUDA_PFN_EXTERN(cuGetErrorName); +DECLARE_CUDA_PFN_EXTERN(cuMemGetAddressRange); +DECLARE_CUDA_PFN_EXTERN(cuCtxCreate); +DECLARE_CUDA_PFN_EXTERN(cuCtxDestroy); +DECLARE_CUDA_PFN_EXTERN(cuCtxGetCurrent); +DECLARE_CUDA_PFN_EXTERN(cuCtxSetCurrent); +DECLARE_CUDA_PFN_EXTERN(cuCtxGetDevice); +DECLARE_CUDA_PFN_EXTERN(cuPointerGetAttribute); // cuMem API support -DECLARE_CUDA_PFN_EXTERN(cuMemAddressReserve, 10020); -DECLARE_CUDA_PFN_EXTERN(cuMemAddressFree, 10020); -DECLARE_CUDA_PFN_EXTERN(cuMemCreate, 10020); -DECLARE_CUDA_PFN_EXTERN(cuMemGetAllocationGranularity, 10020); -DECLARE_CUDA_PFN_EXTERN(cuMemExportToShareableHandle, 10020); -DECLARE_CUDA_PFN_EXTERN(cuMemImportFromShareableHandle, 10020); -DECLARE_CUDA_PFN_EXTERN(cuMemMap, 10020); -DECLARE_CUDA_PFN_EXTERN(cuMemRelease, 10020); -DECLARE_CUDA_PFN_EXTERN(cuMemRetainAllocationHandle, 11000); -DECLARE_CUDA_PFN_EXTERN(cuMemSetAccess, 10020); -DECLARE_CUDA_PFN_EXTERN(cuMemUnmap, 10020); +DECLARE_CUDA_PFN_EXTERN(cuMemAddressReserve); +DECLARE_CUDA_PFN_EXTERN(cuMemAddressFree); +DECLARE_CUDA_PFN_EXTERN(cuMemCreate); +DECLARE_CUDA_PFN_EXTERN(cuMemGetAllocationGranularity); +DECLARE_CUDA_PFN_EXTERN(cuMemExportToShareableHandle); +DECLARE_CUDA_PFN_EXTERN(cuMemImportFromShareableHandle); +DECLARE_CUDA_PFN_EXTERN(cuMemMap); +DECLARE_CUDA_PFN_EXTERN(cuMemRelease); +DECLARE_CUDA_PFN_EXTERN(cuMemRetainAllocationHandle); +DECLARE_CUDA_PFN_EXTERN(cuMemSetAccess); +DECLARE_CUDA_PFN_EXTERN(cuMemUnmap); #if CUDA_VERSION >= 11070 -DECLARE_CUDA_PFN_EXTERN(cuMemGetHandleForAddressRange, 11070); // DMA-BUF support +DECLARE_CUDA_PFN_EXTERN(cuMemGetHandleForAddressRange); // DMA-BUF support #endif #if CUDA_VERSION >= 12010 /* NVSwitch Multicast support */ -DECLARE_CUDA_PFN_EXTERN(cuMulticastAddDevice, 12010); -DECLARE_CUDA_PFN_EXTERN(cuMulticastBindMem, 12010); -DECLARE_CUDA_PFN_EXTERN(cuMulticastBindAddr, 12010); -DECLARE_CUDA_PFN_EXTERN(cuMulticastCreate, 12010); -DECLARE_CUDA_PFN_EXTERN(cuMulticastGetGranularity, 12010); -DECLARE_CUDA_PFN_EXTERN(cuMulticastUnbind, 12010); +DECLARE_CUDA_PFN_EXTERN(cuMulticastAddDevice); +DECLARE_CUDA_PFN_EXTERN(cuMulticastBindMem); +DECLARE_CUDA_PFN_EXTERN(cuMulticastBindAddr); +DECLARE_CUDA_PFN_EXTERN(cuMulticastCreate); +DECLARE_CUDA_PFN_EXTERN(cuMulticastGetGranularity); +DECLARE_CUDA_PFN_EXTERN(cuMulticastUnbind); #endif #endif -/* CUDA Driver functions loaded with dlsym() */ -DECLARE_CUDA_PFN_EXTERN(cuInit, 2000); -DECLARE_CUDA_PFN_EXTERN(cuDriverGetVersion, 2020); -DECLARE_CUDA_PFN_EXTERN(cuGetProcAddress, 11030); - - ncclResult_t ncclCudaLibraryInit(void); extern int ncclCudaDriverVersionCache; diff --git a/src/include/device.h b/src/include/device.h index 02ea883a9..50f841bfc 100644 --- a/src/include/device.h +++ b/src/include/device.h @@ -84,6 +84,15 @@ static_assert(NCCL_LL_CLEAN_MASK % NCCL_STEPS == 0, "Invalid NCCL_LL_CLEAN_MASK #define NCCL_IPC_READ 0x10 #define NCCL_NVLS_MIN_POLL 0x20 +#define NCCL_MAX_COLLNET_SIZE (1L << 29) + +enum ncclRegBufferType { + NCCL_REGULAR_BUFFER = 0, + NCCL_IPC_REG_BUFFER = 1, + NCCL_NVLS_REG_BUFFER = 2, + NCCL_COLLNET_REG_BUFFER = 3 +}; + struct ncclConnInfo { // Regular comm mechanism char *buffs[NCCL_NUM_PROTOCOLS]; // Local for recv, remote for send @@ -93,6 +102,7 @@ struct ncclConnInfo { int flags; // Direct communication / other flags int shared; // Buffers are shared + int stepSize; // Step size for the SIMPLE buffer void **ptrExchange; // Pointer exchange for direct communication uint64_t* redOpArgExchange; // PreOp scaler exchange for direct pull case @@ -157,7 +167,7 @@ struct ncclDirect { int down[NCCL_MAX_DIRECT_ARITY]; }; -#define NCCL_MAX_NVLS_ARITY 8 +#define NCCL_MAX_NVLS_ARITY 32 #define NCCL_MAX_NVLS_TREE_ARITY 3 struct ncclNvls { int out; @@ -171,6 +181,12 @@ struct ncclNvls { int nNodes; }; +#if __CUDA_ARCH__ >= 900 +#define NCCL_MAX_ARITY NCCL_MAX_NVLS_ARITY +#else +#define NCCL_MAX_ARITY NCCL_MAX_DIRECT_ARITY +#endif + #define NCCL_MAX_CONNS 2 struct ncclChannelPeer { struct ncclConnector send[NCCL_MAX_CONNS]; @@ -212,9 +228,10 @@ struct ncclWorkElem { union { uint8_t flagBits; struct { - uint8_t isUsed:1, redOpArgIsPtr:1, regUsed:1, oneNode:1; + uint8_t isUsed:1, redOpArgIsPtr:1, oneNode:1; }; }; + uint8_t regUsed; uint8_t nWarps; uint8_t direct; uint32_t root; diff --git a/src/include/graph.h b/src/include/graph.h index 2a455e9e2..08cfba4fd 100644 --- a/src/include/graph.h +++ b/src/include/graph.h @@ -31,10 +31,10 @@ ncclResult_t ncclTopoGetNvbGpus(struct ncclTopoSystem* system, int rank, int* nr int ncclTopoPathAllNVLink(struct ncclTopoSystem* system); // Query topology -ncclResult_t ncclTopoGetNetDev(struct ncclComm* comm, int rank, struct ncclTopoGraph* graph, int channelId, int peerRank, int* net, int* proxyRank); +ncclResult_t ncclTopoGetNetDev(struct ncclComm* comm, int rank, struct ncclTopoGraph* graph, int channelId, int peerRank, int64_t* id, int* dev, int* proxyRank); ncclResult_t ncclTopoCheckP2p(struct ncclTopoSystem* system, int64_t id1, int64_t id2, int* p2p, int *read, int* intermediateRank); ncclResult_t ncclTopoCheckMNNVL(struct ncclTopoSystem* system, struct ncclPeerInfo* info1, struct ncclPeerInfo* info2, int* ret); -ncclResult_t ncclTopoCheckGdr(struct ncclTopoSystem* topo, int64_t busId, int netDev, int read, int* useGdr); +ncclResult_t ncclTopoCheckGdr(struct ncclTopoSystem* topo, int64_t busId, int64_t netId, int read, int* useGdr); ncclResult_t ncclTopoNeedFlush(struct ncclTopoSystem* system, int64_t busId, int* flush); ncclResult_t ncclTopoCheckNet(struct ncclTopoSystem* system, int64_t id1, int64_t id2, int* net); int ncclPxnDisable(struct ncclComm* comm); @@ -56,8 +56,8 @@ ncclResult_t ncclTopoCpuType(struct ncclTopoSystem* system, int* arch, int* vend ncclResult_t ncclTopoGetGpuCount(struct ncclTopoSystem* system, int* count); ncclResult_t ncclTopoGetNetCount(struct ncclTopoSystem* system, int* count); ncclResult_t ncclTopoGetNvsCount(struct ncclTopoSystem* system, int* count); -ncclResult_t ncclTopoGetLocalNet(struct ncclTopoSystem* system, int rank, int channelId, int* id); -ncclResult_t ncclTopoGetLocalGpu(struct ncclTopoSystem* system, int net, int* gpuIndex); +ncclResult_t ncclTopoGetLocalNet(struct ncclTopoSystem* system, int rank, int channelId, int64_t* id, int* dev); +ncclResult_t ncclTopoGetLocalGpu(struct ncclTopoSystem* system, int64_t netId, int* gpuIndex); ncclResult_t getLocalNetCountByBw(struct ncclTopoSystem* system, int gpu, int *count); #define NCCL_TOPO_MAX_NODES 256 @@ -88,7 +88,7 @@ struct ncclTopoGraph { int sameChannels; int nHops; int intra[MAXCHANNELS*NCCL_TOPO_MAX_NODES]; - int inter[MAXCHANNELS*2]; + int64_t inter[MAXCHANNELS*2]; }; ncclResult_t ncclTopoCompute(struct ncclTopoSystem* system, struct ncclTopoGraph* graph); @@ -110,7 +110,7 @@ struct ncclTopoRanks { ncclResult_t ncclTopoPreset(struct ncclComm* comm, struct ncclTopoGraph** graphs, struct ncclTopoRanks* topoRanks); ncclResult_t ncclTopoPostset(struct ncclComm* comm, int* firstRanks, int* treePatterns, - struct ncclTopoRanks** allTopoRanks, int* rings, struct ncclTopoGraph** graphs); + struct ncclTopoRanks** allTopoRanks, int* rings, struct ncclTopoGraph** graphs, struct ncclComm* parent); ncclResult_t ncclTopoTuneModel(struct ncclComm* comm, int minCompCap, int maxCompCap, struct ncclTopoGraph** graphs); #include "info.h" diff --git a/src/include/info.h b/src/include/info.h index 3a3f4e5b1..0d53b9a21 100644 --- a/src/include/info.h +++ b/src/include/info.h @@ -31,13 +31,6 @@ typedef enum : uint8_t { ncclPatternRecv } ncclPattern_t; -enum ncclRegBufferType { - NCCL_REGULAR_BUFFER = 0, - NCCL_IPC_REG_BUFFER = 1, - NCCL_NVLS_REG_BUFFER = 2, - NCCL_REG_BUFFER_NUM = 3 -}; - // Used to pass NCCL call information between functions struct ncclInfo { ncclFunc_t coll; @@ -70,6 +63,9 @@ struct ncclInfo { ncclRegBufferType regBufType; void* regBufSend[NCCL_MAX_LOCAL_RANKS]; void* regBufRecv[NCCL_MAX_LOCAL_RANKS]; + // collnet buffer reg handles + void* sendMhandle; + void* recvMhandle; // Need to initialize int nThreads; int nChannels; diff --git a/src/include/nccl_common.h b/src/include/nccl_common.h index ded0dae9f..5796eb9fb 100644 --- a/src/include/nccl_common.h +++ b/src/include/nccl_common.h @@ -8,7 +8,7 @@ #define NCCL_DEBUG_H_ typedef enum {NCCL_LOG_NONE=0, NCCL_LOG_VERSION=1, NCCL_LOG_WARN=2, NCCL_LOG_INFO=3, NCCL_LOG_ABORT=4, NCCL_LOG_TRACE=5} ncclDebugLogLevel; -typedef enum {NCCL_INIT=1, NCCL_COLL=2, NCCL_P2P=4, NCCL_SHM=8, NCCL_NET=16, NCCL_GRAPH=32, NCCL_TUNING=64, NCCL_ENV=128, NCCL_ALLOC=256, NCCL_CALL=512, NCCL_PROXY=1024, NCCL_NVLS=2048, NCCL_ALL=~0} ncclDebugLogSubSys; +typedef enum {NCCL_INIT=1, NCCL_COLL=2, NCCL_P2P=4, NCCL_SHM=8, NCCL_NET=16, NCCL_GRAPH=32, NCCL_TUNING=64, NCCL_ENV=128, NCCL_ALLOC=256, NCCL_CALL=512, NCCL_PROXY=1024, NCCL_NVLS=2048, NCCL_BOOTSTRAP=4096, NCCL_REG=8192, NCCL_ALL=~0} ncclDebugLogSubSys; typedef void (*ncclDebugLogger_t)(ncclDebugLogLevel level, unsigned long flags, const char *file, int line, const char *fmt, ...); diff --git a/src/include/nccl_tuner.h b/src/include/nccl_tuner.h index b4a696e38..1917e2815 100644 --- a/src/include/nccl_tuner.h +++ b/src/include/nccl_tuner.h @@ -17,13 +17,17 @@ typedef struct { const char* name; // Initializes tuner states. - // nRanks: number of ranks in current communicator. Each communicator initialize its own tuner. - // nNodes: number of nodes in current communicator. - // logFunction: a logFunction can be useful to integrate logging together with NCCL core. - ncclResult_t (*init)(size_t nRanks, size_t nNodes, ncclDebugLogger_t logFunction); + // Inputs: + // - nRanks: number of ranks in current communicator. Each communicator initialize its own tuner. + // - nNodes: number of nodes in current communicator. + // - logFunction: a logFunction can be useful to integrate logging together with NCCL core. + // Outputs: + // - context: tuner context object + ncclResult_t (*init)(size_t nRanks, size_t nNodes, ncclDebugLogger_t logFunction, void **context); // Gets info (algo, protocol, number of ctas and threads) for a given collective. // Inputs: + // - context: tuner context object // - collType: collective type , e.g., allreduce, allgather… // - nBytes: collective size in bytes // - collNetTypeSupport: whether collnet supports this type @@ -40,16 +44,17 @@ typedef struct { // Also, the plugin is allowed to not set any output, or set only the // algorithm and protocol, but not only the algorithm or only the protocol. // Unset fields will be set automatically by NCCL. - ncclResult_t (*getCollInfo)(ncclFunc_t collType, size_t nBytes, + ncclResult_t (*getCollInfo)(void* context, ncclFunc_t collType, size_t nBytes, int collNetSupport, int nvlsSupport, int numPipeOps, int *algorithm, int *protocol, int* nChannels); // Terminates the plugin and cleans up any resources that the plugin allocated. - ncclResult_t (*destroy)(); -} ncclTuner_v1_t; + // context: tuner context object + ncclResult_t (*destroy)(void* context); +} ncclTuner_v2_t; -typedef ncclTuner_v1_t ncclTuner_t; +typedef ncclTuner_v2_t ncclTuner_t; -#define NCCL_TUNER_PLUGIN_SYMBOL "ncclTunerPlugin_v1" +#define NCCL_TUNER_PLUGIN_SYMBOL "ncclTunerPlugin_v2" #endif diff --git a/src/include/proxy.h b/src/include/proxy.h index 353426c1d..cb1c3b200 100644 --- a/src/include/proxy.h +++ b/src/include/proxy.h @@ -33,7 +33,6 @@ union ncclProxyOpSpecifics { struct ncclProxyOp { struct ncclProxyConnection* connection; - void* buffer; ssize_t nbytes; uint64_t opCount; int root; @@ -49,6 +48,11 @@ struct ncclProxyOp { uint8_t /*ncclPattern_t*/ pattern; uint8_t protocol; uint8_t reg; + // collnet buffer reg handles + void* sendMhandle; + void* recvMhandle; + uint8_t* sendbuff; + uint8_t* recvbuff; union ncclProxyOpSpecifics specifics; @@ -58,8 +62,14 @@ struct ncclProxyOp { struct ncclProxySubArgs { struct ncclProxyConnection* connection; int reg; - void* buffer; + // p2p mhandle void* mhandle; + // collnet handles + void* sendMhandle; + void* recvMhandle; + uint8_t* sendbuff; + uint8_t* recvbuff; + size_t offset; int channelId; int nsteps; ssize_t nbytes; @@ -88,6 +98,10 @@ struct ncclProxyArgs { int sliceSteps; int chunkSteps; int chunkSize; + size_t totalSendSize; + size_t totalRecvSize; + size_t sendSizePerRound; + size_t recvSizePerRound; uint8_t /*ncclDataType_t*/ dtype; uint8_t /*ncclDevRedOp_t*/ redOp; uint8_t /*ncclPattern_t*/ pattern; @@ -302,6 +316,8 @@ enum ncclProxyMsgType { ncclProxyMsgAbort = 7, ncclProxyMsgStop = 8, ncclProxyMsgGetFd = 9, // cuMem API support (UDS) + ncclProxyMsgRegister = 10, + ncclProxyMsgDeregister = 11 }; // This function is called by a client of the proxy that needs to invoke any of the non-progress proxyOp types diff --git a/src/include/register.h b/src/include/register.h index 2fb387f7c..d23e0da3e 100644 --- a/src/include/register.h +++ b/src/include/register.h @@ -5,7 +5,8 @@ enum { NET_REG_COMPLETE = 0x01, NVLS_REG_COMPLETE = 0x02, NVLS_REG_POSSIBLE = 0x04, - NVLS_REG_NO_SUPPORT = 0x08 + NVLS_REG_NO_SUPPORT = 0x08, + COLLNET_REG_COMPLETE = 0x10 }; struct ncclReg { @@ -26,6 +27,9 @@ struct ncclReg { int dev; CUmemGenericAllocationHandle mcHandle; uintptr_t caddrs[NCCL_MAX_LOCAL_RANKS]; /* use to check if NVLS buffers match among intra-node ranks */ + // collnet reg + void* collnetHandle; + struct ncclProxyConnector* proxyconn; }; struct ncclRegCache { diff --git a/src/include/socket.h b/src/include/socket.h index 9e5137289..60a413875 100644 --- a/src/include/socket.h +++ b/src/include/socket.h @@ -92,6 +92,7 @@ ncclResult_t ncclSocketProgress(int op, struct ncclSocket* sock, void* ptr, int ncclResult_t ncclSocketWait(int op, struct ncclSocket* sock, void* ptr, int size, int* offset); ncclResult_t ncclSocketSend(struct ncclSocket* sock, void* ptr, int size); ncclResult_t ncclSocketRecv(struct ncclSocket* sock, void* ptr, int size); +ncclResult_t ncclSocketSendRecv(struct ncclSocket* sendSock, void* sendPtr, int sendSize, struct ncclSocket* recvSock, void* recvPtr, int recvSize); ncclResult_t ncclSocketTryRecv(struct ncclSocket* sock, void* ptr, int size, int* closed, bool blocking); ncclResult_t ncclSocketClose(struct ncclSocket* sock); #endif diff --git a/src/include/transport.h b/src/include/transport.h index a21114807..1671db0e2 100644 --- a/src/include/transport.h +++ b/src/include/transport.h @@ -95,6 +95,8 @@ struct ncclTransportComm { ncclResult_t (*proxyConnect)(struct ncclProxyConnection* connection, struct ncclProxyState* proxyState, void* reqBuff, int reqSize, void* respBuff, int respSize, int* done); ncclResult_t (*proxyFree)(struct ncclProxyConnection* connection, struct ncclProxyState* proxyState); ncclResult_t (*proxyProgress)(struct ncclProxyState* proxyState, struct ncclProxyArgs*); + ncclResult_t (*proxyRegister)(struct ncclProxyConnection* connection, struct ncclProxyState* proxyState, void* reqBuff, int reqSize, void* respBuff, int respSize, int* done); + ncclResult_t (*proxyDeregister)(struct ncclProxyConnection* connection, struct ncclProxyState* proxyState, void* reqBuff, int reqSize, int* done); }; struct ncclTransport { @@ -107,15 +109,6 @@ struct ncclTransport { ncclResult_t ncclTransportP2pConnect(struct ncclComm* comm, int channelId, int nrecv, int* peerRecv, int nsend, int* peerSend, int connIndex); ncclResult_t ncclTransportP2pSetup(struct ncclComm* comm, struct ncclTopoGraph* graph, int connIndex, int* highestTransportType=NULL); -// Currently we only support POSIX_FILE_DESCRIPTOR handle exchange -#define USE_POSIX_FD 1 - -#if USE_POSIX_FD -#define NVLS_CU_MEM_HANDLE_TYPE CU_MEM_HANDLE_TYPE_POSIX_FILE_DESCRIPTOR -#else -#define NVLS_CU_MEM_HANDLE_TYPE CU_MEM_HANDLE_TYPE_NONE -#endif - ncclResult_t ncclNvlsInit(struct ncclComm* comm); ncclResult_t ncclNvlsSetup(struct ncclComm* comm, struct ncclComm* parent); ncclResult_t ncclNvlsGraphRegisterBuffer(struct ncclComm *comm, struct ncclKernelPlan *plan, const void *sendbuff, void *recvbuff, size_t sendbuffSize, size_t recvbuffSize, bool *outRegBufUsed, void **outRegBufSend, void **outRegBufRecv); @@ -124,7 +117,10 @@ ncclResult_t ncclNvlsDeregBuffer(CUmemGenericAllocationHandle *mcHandler, CUdevi ncclResult_t ncclNvlsFree(struct ncclComm* comm); enum { collNetRecv=0, collNetSend=1 }; -int ncclTransportCollNetSetup(struct ncclComm* comm, struct ncclTopoGraph* collNetGraph, struct ncclChannel* channel, int masterRank, int masterPeer, int collNetGraphChannelId, int type); +int ncclTransportCollNetSetup(struct ncclComm* comm, struct ncclTopoGraph* collNetGraph, struct ncclChannel* channel, int masterRank, int masterPeer, int collNetGraphChannelId, int type, ncclConnect* connect); ncclResult_t ncclTransportCollNetCheck(struct ncclComm* comm, int collNetSetupFail); ncclResult_t ncclTransportCollNetFree(struct ncclComm* comm); +ncclResult_t ncclCollnetLocalRegisterBuffer(struct ncclComm* comm, const void* userbuff, size_t buffSize, int type, int* outRegBufUsed, void** outHandle); +ncclResult_t ncclCollnetGraphRegisterBuffer(struct ncclComm* comm, struct ncclKernelPlan *plan, const void* userbuff, size_t buffSize, int type, int* outRegBufFlag, void** outHandle); +ncclResult_t ncclCollnetDeregBuffer(struct ncclComm* comm, struct ncclProxyConnector* proxyconn, void* handle); #endif diff --git a/src/include/tuner.h b/src/include/tuner.h index d8b275017..df8f5f21c 100644 --- a/src/include/tuner.h +++ b/src/include/tuner.h @@ -15,8 +15,8 @@ // Attempts to load NCCL tuner from environmental variable. // Returns ncclSuccess if the correct tuner symbol has been found and // successully loaded. Otherwise returns an error and also logs the error. -ncclResult_t ncclLoadTunerPlugin(ncclTuner_t** tuner); +ncclResult_t ncclTunerPluginLoad(ncclTuner_t** tuner); // Cleans up NCCL tuner plugin. -ncclResult_t ncclCloseTunerPlugin(ncclTuner_t** tuner); +ncclResult_t ncclTunerPluginUnload(ncclTuner_t** tuner); #endif diff --git a/src/init.cc b/src/init.cc index 060fa8b9d..cecb9bc05 100644 --- a/src/init.cc +++ b/src/init.cc @@ -117,6 +117,7 @@ ncclResult_t ncclGetUniqueId(ncclUniqueId* out) { void NCCL_NO_OPTIMIZE commPoison(ncclComm_t comm) { // Important that this does not trash intraComm0. comm->rank = comm->cudaDev = comm->busId = comm->nRanks = -1; + comm->startMagic = comm->endMagic = 0; } #undef NCCL_NO_OPTIMIZE @@ -280,7 +281,6 @@ static ncclResult_t dmaBufSupported(struct ncclComm* comm) { ncclResult_t ncclCommEnsureReady(ncclComm_t comm) { /* comm must be ready, or error will be reported */ ncclResult_t ret = ncclSuccess; - if (__atomic_load_n(comm->abortFlag, __ATOMIC_RELAXED)) { ncclGroupJobAbort(comm->groupJob); } else { @@ -351,6 +351,7 @@ static ncclResult_t commAlloc(struct ncclComm* comm, struct ncclComm* parent, in ncclMemoryPoolConstruct(&comm->memPool_ncclProxyOp); ncclMemoryPoolConstruct(&comm->memPool_ncclPointerList); ncclMemoryPoolConstruct(&comm->memPool_ncclNvlsHandleList); + ncclMemoryPoolConstruct(&comm->memPool_ncclCollnetHandleList); comm->groupNext = reinterpret_cast(0x1); comm->preconnectNext = reinterpret_cast(0x1); @@ -560,9 +561,8 @@ static ncclResult_t computeBuffSizes(struct ncclComm* comm) { comm->buffSizes[p] = envs[p] != -2 ? envs[p] : defaults[p]; } - // MNNVL support - if (!comm->MNNVL && comm->nNodes > 1) comm->p2pChunkSize = ncclParamP2pNetChunkSize(); - else if (comm->MNNVL || ncclTopoPathAllNVLink(comm->topo)) comm->p2pChunkSize = ncclParamP2pNvlChunkSize(); + if (comm->nNodes > 1) comm->p2pChunkSize = ncclParamP2pNetChunkSize(); + else if (ncclTopoPathAllNVLink(comm->topo)) comm->p2pChunkSize = ncclParamP2pNvlChunkSize(); else comm->p2pChunkSize = ncclParamP2pPciChunkSize(); // Make sure P2P chunksize is not larger than coll chunksize. @@ -584,16 +584,38 @@ NCCL_PARAM(CollNetNodeThreshold, "COLLNET_NODE_THRESHOLD", 2); NCCL_PARAM(NvbPreconnect, "NVB_PRECONNECT", 1); NCCL_PARAM(AllocP2pNetLLBuffers, "ALLOC_P2P_NET_LL_BUFFERS", 0); +static ncclResult_t collNetInitRailRankMap(ncclComm_t comm) { + int rank = comm->rank; + uint64_t nonHeadMask = (1ull << comm->localRanks) - 1; + + comm->collNetDenseToUserRank = ncclMemoryStackAlloc(&comm->memPermanent, comm->nRanks); + comm->collNetUserToDenseRank = ncclMemoryStackAlloc(&comm->memPermanent, comm->nRanks); + // initialize collNetUserToDenseRank[rank] + comm->collNetUserToDenseRank[rank] = -1; + for (int h = 0; h < comm->collNetHeadsNum; h++) { + nonHeadMask ^= 1ull << comm->rankToLocalRank[comm->collNetHeads[h]]; + if (comm->collNetHeads[h] == rank) { comm->collNetUserToDenseRank[rank] = h; break; } + } + if (comm->collNetUserToDenseRank[rank] == -1) { + comm->collNetUserToDenseRank[rank] = __builtin_popcountll(nonHeadMask & ((1ull << comm->localRank) - 1)); + } + comm->collNetUserToDenseRank[rank] += comm->node * comm->localRanks; + + NCCLCHECK(bootstrapAllGather(comm->bootstrap, comm->collNetUserToDenseRank, sizeof(int))); + for (int r = 0; r < comm->nRanks; r++) { + comm->collNetDenseToUserRank[comm->collNetUserToDenseRank[r]] = r; + } + return ncclSuccess; +} + static ncclResult_t collNetTrySetup(ncclComm_t comm, ncclComm_t parent, struct ncclTopoGraph* collNetGraph) { ncclResult_t ret = ncclSuccess; - int* heads = NULL; int rank = comm->rank; int collNetSetupFail = 0; int highestTypes[NCCL_MAX_LOCAL_RANKS] = { TRANSPORT_P2P }; // Find all head ranks - int nHeads = collNetGraph->nChannels; int nHeadsUnique = 0; - int headsUnique[NCCL_MAX_LOCAL_RANKS]; + int* headsUnique = NULL; int highestTransportType0, highestTransportType1; char line[1024]; bool share; @@ -604,27 +626,26 @@ static ncclResult_t collNetTrySetup(ncclComm_t comm, ncclComm_t parent, struct n }; struct collnetShareInfo* infos = NULL; - NCCLCHECKGOTO(ncclCalloc(&heads, nHeads), ret, fail); + NCCLCHECKGOTO(ncclCalloc(&headsUnique, collNetGraph->nChannels), ret, fail); { uint64_t mask = 0; // Head GPU index is always 0 - for (int c = 0; c < nHeads; c++) { - heads[c] = collNetGraph->intra[c * comm->localRanks + 0]; - assert(comm->rankToNode[heads[c]] == comm->node); + for (int c = 0; c < collNetGraph->nChannels; c++) { + int head = collNetGraph->intra[c * comm->localRanks + 0]; + assert(comm->rankToNode[head] == comm->node); uint64_t mask0 = mask; - mask |= 1ull<rankToLocalRank[heads[c]]; - if (mask != mask0) headsUnique[nHeadsUnique++] = heads[c]; + mask |= 1ull<rankToLocalRank[head]; + if (mask != mask0) headsUnique[nHeadsUnique++] = head; } } - comm->collNetHeads = heads; - comm->collNetHeadsNum = nHeads; - comm->collNetHeadsUniqueNum = nHeadsUnique; + comm->collNetHeads = headsUnique; + comm->collNetHeadsNum = nHeadsUnique; if (parent && parent->collNetSupport && parent->config.splitShare && parent->nNodes == comm->nNodes) { NCCLCHECKGOTO(ncclCalloc(&infos, comm->nRanks), ret, fail); /* check whether child can share collnet resources of parent. Since parent builds each collnet communicator * based on heads with the same head position in each node, as long as the collnet heads of child comm * can match parent's heads, we can let child communicator share parent's collnet resources. */ - for (int h = 0; h < nHeads; ++h) { + for (int h = 0; h < nHeadsUnique; ++h) { int prev = INT_MIN; struct collnetShareInfo* myinfo; @@ -632,7 +653,7 @@ static ncclResult_t collNetTrySetup(ncclComm_t comm, ncclComm_t parent, struct n myinfo = infos + comm->rank; memset(myinfo, 0, sizeof(struct collnetShareInfo)); /* find the child head position in parent collnet heads. */ - if (heads[h] == comm->rank) { + if (headsUnique[h] == comm->rank) { myinfo->headPosition = -1; myinfo->isMaster = 1; for (int th = 0; th < parent->collNetHeadsNum; ++th) @@ -658,10 +679,11 @@ static ncclResult_t collNetTrySetup(ncclComm_t comm, ncclComm_t parent, struct n if (share) { if (myinfo->isMaster) { comm->collNetSharedRes = parent->collNetSharedRes; - comm->collNetChannels = std::min(comm->nChannels, parent->collNetSharedRes->nChannels); - for (int c = 0; c < comm->collNetChannels; ++c) + for (int c = 0; c < comm->nChannels; ++c) NCCLCHECKGOTO(initCollnetChannel(comm, c, parent, true), ret, fail); } + + NCCLCHECKGOTO(collNetInitRailRankMap(comm), ret, fail); } else { /* TODO: CX-6 and CX-7 both do not support multiple sharp resources per process, if child comm cannot * share the sharp resource from parent, we cannot use sharp in this case. This restriction might be @@ -677,35 +699,19 @@ static ncclResult_t collNetTrySetup(ncclComm_t comm, ncclComm_t parent, struct n } else { /* this allocated buffer will be freed on proxy side */ NCCLCHECK(ncclCalloc(&comm->collNetSharedRes, 1)); - comm->collNetChannels = comm->collNetSharedRes->nChannels = comm->nChannels; + comm->collNetSharedRes->nChannels = comm->nChannels; comm->collNetSharedRes->buffSize = comm->buffSizes[NCCL_PROTO_SIMPLE]; - comm->collNetDenseToUserRank = ncclMemoryStackAlloc(&comm->memPermanent, comm->nRanks); - comm->collNetUserToDenseRank = ncclMemoryStackAlloc(&comm->memPermanent, comm->nRanks); - { // initialize collNetUserToDenseRank[rank] - uint64_t nonHeadMask = (1ull<localRanks)-1; - comm->collNetUserToDenseRank[rank] = -1; - for (int h=0; h < nHeadsUnique; h++) { - nonHeadMask ^= 1ull<rankToLocalRank[headsUnique[h]]; - if (headsUnique[h] == rank) { comm->collNetUserToDenseRank[rank] = h; break; } - } - if (comm->collNetUserToDenseRank[rank] == -1) { - comm->collNetUserToDenseRank[rank] = __builtin_popcountll(nonHeadMask & ((1ull<localRank)-1)); - } - comm->collNetUserToDenseRank[rank] += comm->node*comm->localRanks; - } - NCCLCHECK(bootstrapAllGather(comm->bootstrap, comm->collNetUserToDenseRank, sizeof(int))); - for (int r=0; r < comm->nRanks; r++) { - comm->collNetDenseToUserRank[comm->collNetUserToDenseRank[r]] = r; - } + NCCLCHECKGOTO(collNetInitRailRankMap(comm), ret, fail); - for (int c = 0; c < comm->collNetChannels; c++) { + for (int c = 0; c < comm->nChannels; c++) { struct ncclChannel* channel = comm->channels + c; NCCLCHECKGOTO(initCollnetChannel(comm, c, parent, false), ret, fail); - for (int h = 0; h < nHeads; h++) { - const int head = heads[h]; - collNetSetupFail |= ncclTransportCollNetSetup(comm, collNetGraph, channel, head, head, h, collNetRecv); - if (!collNetSetupFail) collNetSetupFail |= ncclTransportCollNetSetup(comm, collNetGraph, channel, head, head, h, collNetSend); + for (int h = 0; h < nHeadsUnique; h++) { + const int head = headsUnique[h]; + ncclConnect connect; + collNetSetupFail |= ncclTransportCollNetSetup(comm, collNetGraph, channel, head, head, h, collNetRecv, &connect); + if (!collNetSetupFail) collNetSetupFail |= ncclTransportCollNetSetup(comm, collNetGraph, channel, head, head, h, collNetSend, &connect); } // Verify CollNet setup across ranks after trying the first channel if (c == 0) { @@ -727,7 +733,7 @@ static ncclResult_t collNetTrySetup(ncclComm_t comm, ncclComm_t parent, struct n bool isHead = false; matrix = nullptr; NCCLCHECKGOTO(ncclCalloc(&matrix, comm->nRanks), ret, matrix_end); - for (int h = 0; h < nHeads; h++) isHead |= (heads[h] == comm->rank); + for (int h = 0; h < nHeadsUnique; h++) isHead |= (headsUnique[h] == comm->rank); if (isHead) { for (int ty=0; ty < ncclNumTypes; ty++) { for (int i=0; i < 4; i++) { @@ -817,7 +823,72 @@ static ncclResult_t collNetTrySetup(ncclComm_t comm, ncclComm_t parent, struct n } // MNNVL: Flag to indicate whether to enable Multi-Node NVLink -NCCL_PARAM(MNNVL, "MNNVL", -2); +NCCL_PARAM(MNNVLEnable, "MNNVL_ENABLE", 2); + +#if CUDART_VERSION >= 11030 + +#include +#include "cudawrap.h" + +// Determine if MNNVL support is available +static int checkMNNVL(struct ncclComm* comm) { + ncclResult_t ret = ncclSuccess; + + // MNNVL requires cuMem to be enabled + if (!ncclCuMemEnable()) return 0; + + // MNNVL also requires FABRIC handle support + int cudaDev; + int flag = 0; + CUdevice currentDev; + CUDACHECK(cudaGetDevice(&cudaDev)); + CUCHECK(cuDeviceGet(¤tDev, cudaDev)); + // Ignore error if CU_DEVICE_ATTRIBUTE_HANDLE_TYPE_FABRIC_SUPPORTED is not supported + (void) CUPFN(cuDeviceGetAttribute(&flag, CU_DEVICE_ATTRIBUTE_HANDLE_TYPE_FABRIC_SUPPORTED, currentDev));; + if (!flag) return 0; + // Check that all ranks have initialized the fabric fully + for (int i = 0; i < comm->nRanks; i++) { + if (comm->peerInfo[i].fabricInfo.state != NVML_GPU_FABRIC_STATE_COMPLETED) return 0; + } + + // Determine our MNNVL domain/clique + NCCLCHECKGOTO(ncclCalloc(&comm->clique.ranks, comm->nRanks), ret, fail); + comm->clique.id = comm->peerInfo[comm->rank].fabricInfo.cliqueId; + for (int i = 0; i < comm->nRanks; i++) { + nvmlGpuFabricInfoV_t *fabricInfo1 = &comm->peerInfo[comm->rank].fabricInfo; + nvmlGpuFabricInfoV_t *fabricInfo2 = &comm->peerInfo[i].fabricInfo; + // Check if the cluster UUID and cliqueId match + // A zero UUID means we don't have MNNVL fabric info - disable MNNVL + if ((((long *)&fabricInfo2->clusterUuid)[0]|((long *)fabricInfo2->clusterUuid)[1]) == 0) goto fail; + if ((memcmp(fabricInfo1->clusterUuid, fabricInfo2->clusterUuid, NVML_GPU_FABRIC_UUID_LEN) == 0) && + (fabricInfo1->cliqueId == fabricInfo2->cliqueId)) { + if (i == comm->rank) { + comm->cliqueRank = comm->clique.size; + } + comm->clique.ranks[comm->clique.size++] = i; + } + } + // Determine whether to enable MNNVL or not + comm->MNNVL = ncclParamMNNVLEnable() == 2 ? comm->clique.size > 1 : ncclParamMNNVLEnable(); + INFO(NCCL_INIT, "MNNVL %d cliqueId %x cliqueSize %d cliqueRank %d ", comm->MNNVL, comm->clique.id, comm->clique.size, comm->cliqueRank); + + if (comm->MNNVL) { + // Force the CUMEM handle type to be FABRIC for MNNVL + ncclCuMemHandleType = CU_MEM_HANDLE_TYPE_FABRIC; + } + + return comm->MNNVL; + +fail: + if (comm->clique.ranks) free(comm->clique.ranks); + return 0; +} + +#else +static int checkMNNVL(struct ncclComm* comm) { + return 0; +} +#endif static ncclResult_t initTransportsRank(struct ncclComm* comm, struct ncclComm* parent = NULL) { // We use 2 AllGathers @@ -842,6 +913,7 @@ static ncclResult_t initTransportsRank(struct ncclComm* comm, struct ncclComm* p float bwInter; int typeIntra; int typeInter; + int crossNic; }; struct allGatherInfo { @@ -875,61 +947,19 @@ static ncclResult_t initTransportsRank(struct ncclComm* comm, struct ncclComm* p } // AllGather1 - end -#if CUDART_VERSION >= 11030 - -#include -#include "cudawrap.h" - // MNNVL support - if (nNodes > 1) { - int cliqueSize = 0; - comm->MNNVL = 0; - // Determine the size of the MNNVL domain/clique - for (int i = 0; i < nranks; i++) { - nvmlGpuFabricInfoV_t *fabricInfo1 = &comm->peerInfo[rank].fabricInfo; - nvmlGpuFabricInfoV_t *fabricInfo2 = &comm->peerInfo[i].fabricInfo; - // Check that the Fabric state is fully initialized - if (fabricInfo2->state != NVML_GPU_FABRIC_STATE_COMPLETED) continue; - // Check that the cluster UUID and cliqueId match in each rank - // A zero UUID means we don't have MNNVL fabric info - disable MNNVL - if ((((long *)&fabricInfo2->clusterUuid)[0]|((long *)fabricInfo2->clusterUuid)[1]) == 0) continue; - if ((memcmp(fabricInfo1->clusterUuid, fabricInfo2->clusterUuid, NVML_GPU_FABRIC_UUID_LEN) == 0) && - (fabricInfo1->cliqueId == fabricInfo2->cliqueId)) { - cliqueSize++; - } - } - // Determine whether this is a MNNVL system - comm->MNNVL = ncclParamMNNVL() < 0 ? cliqueSize == comm->nRanks : ncclParamMNNVL(); - // MNNVL requires cuMem to be enabled - if (!ncclCuMemEnable()) comm->MNNVL = 0; - if (comm->MNNVL) { - // MNNVL also requires FABRIC handle support - int cudaDev; - int flag = 0; - CUdevice currentDev; - CUDACHECK(cudaGetDevice(&cudaDev)); - CUCHECK(cuDeviceGet(¤tDev, cudaDev)); - // Ignore error if CU_DEVICE_ATTRIBUTE_HANDLE_TYPE_FABRIC_SUPPORTED is not supported - (void) CUPFN(cuDeviceGetAttribute(&flag, CU_DEVICE_ATTRIBUTE_HANDLE_TYPE_FABRIC_SUPPORTED, currentDev));; - if (!flag) - comm->MNNVL = 0; - else - // Force the handle type to be FABRIC for MNNVL - ncclCuMemHandleType = CU_MEM_HANDLE_TYPE_FABRIC; - } - if (ncclParamMNNVL() == 1 && !comm->MNNVL) { - WARN("MNNVL is not supported on this system"); - ret = ncclSystemError; - goto fail; - } + if (nNodes > 1 && !checkMNNVL(comm) && ncclParamMNNVLEnable() == 1) { + // Return an error if the user specifically requested MNNVL support + WARN("MNNVL is not supported on this system"); + ret = ncclSystemError; + goto fail; } -#endif do { // Compute intra-process ranks int intraProcRank0 = -1, intraProcRank = -1, intraProcRanks = 0; - for (int i = 0; i < nranks; i++) comm->minCompCap = std::min(comm->minCompCap, comm->peerInfo[rank].cudaCompCap); - for (int i = 0; i < nranks; i++) comm->maxCompCap = std::max(comm->maxCompCap, comm->peerInfo[rank].cudaCompCap); + for (int i = 0; i < nranks; i++) comm->minCompCap = std::min(comm->minCompCap, comm->peerInfo[i].cudaCompCap); + for (int i = 0; i < nranks; i++) comm->maxCompCap = std::max(comm->maxCompCap, comm->peerInfo[i].cudaCompCap); comm->nvlsRegSupport = 1; for (int i = 0; i < nranks; i++) { @@ -955,6 +985,10 @@ static ncclResult_t initTransportsRank(struct ncclComm* comm, struct ncclComm* p } } } + + // Buffer Registration is not supported with MNNVL + if (comm->MNNVL) comm->nvlsRegSupport = 0; + TRACE(NCCL_INIT,"pidHash[%d] %lx intraProcRank %d intraProcRanks %d intraProcRank0 %d", rank, comm->peerInfo[rank].pidHash, intraProcRank, intraProcRanks, intraProcRank0); if (intraProcRank == -1 || intraProcRank0 == -1 || comm->peerInfo[intraProcRank0].comm == NULL) { @@ -1065,6 +1099,7 @@ static ncclResult_t initTransportsRank(struct ncclComm* comm, struct ncclComm* p allGather3Data[rank].graphInfo[a].bwInter = graphs[a]->bwInter; allGather3Data[rank].graphInfo[a].typeIntra = graphs[a]->typeIntra; allGather3Data[rank].graphInfo[a].typeInter = graphs[a]->typeInter; + allGather3Data[rank].graphInfo[a].crossNic = graphs[a]->crossNic; } comm->nChannels = std::min(treeGraph.nChannels, ringGraph.nChannels); @@ -1137,10 +1172,11 @@ static ncclResult_t initTransportsRank(struct ncclComm* comm, struct ncclComm* p graphs[a]->bwInter = std::min(allGather3Data[i].graphInfo[a].bwInter, graphs[a]->bwInter); graphs[a]->typeIntra = std::max(allGather3Data[i].graphInfo[a].typeIntra, graphs[a]->typeIntra); graphs[a]->typeInter = std::max(allGather3Data[i].graphInfo[a].typeInter, graphs[a]->typeInter); + graphs[a]->crossNic = std::max(allGather3Data[i].graphInfo[a].crossNic, graphs[a]->crossNic); } - if (graphs[NCCL_ALGO_COLLNET_CHAIN]->nChannels == 0) comm->collNetSupport = 0; - if (graphs[NCCL_ALGO_NVLS]->nChannels == 0) comm->nvlsSupport = 0; } + if (graphs[NCCL_ALGO_COLLNET_CHAIN]->nChannels == 0) comm->collNetSupport = 0; + if (graphs[NCCL_ALGO_NVLS]->nChannels == 0) comm->nvlsSupport = comm->nvlsChannels = 0; comm->nChannels = treeGraph.nChannels = ringGraph.nChannels = std::min(treeGraph.nChannels, ringGraph.nChannels); if (comm->nChannels < nChannelsOrig) { @@ -1156,17 +1192,22 @@ static ncclResult_t initTransportsRank(struct ncclComm* comm, struct ncclComm* p INFO(NCCL_INIT, "Communicator has %d nodes which is less than CollNet node threshold %d, disabling CollNet", comm->nNodes, collNetNodeThreshold); comm->collNetSupport = 0; } + comm->collNetRegSupport = true; for (int n=0; nnNodes; n++) { if (comm->nodeRanks[n].localRanks > NCCL_MAX_DIRECT_ARITY+1) { WARN("CollNet currently only supports up to %d GPUs per node, disabling CollNet", NCCL_MAX_DIRECT_ARITY+1); comm->collNetSupport = 0; break; } + if (comm->nodeRanks[n].localRanks > 1) { + // As long as there is more than 1 rank on any node, we need to disable collnet reg + comm->collNetRegSupport = false; + } } } NCCLCHECKGOTO(ncclCalloc(&rings, nranks*MAXCHANNELS), ret, fail); - NCCLCHECKGOTO(ncclTopoPostset(comm, nodesFirstRank, nodesTreePatterns, allTopoRanks, rings, graphs), ret, fail); + NCCLCHECKGOTO(ncclTopoPostset(comm, nodesFirstRank, nodesTreePatterns, allTopoRanks, rings, graphs, parent), ret, fail); // AllGather3 - end TRACE(NCCL_INIT, "rank %d nranks %d - BUILT %d TREES/RINGS", rank, nranks, comm->nChannels); @@ -1253,7 +1294,7 @@ static ncclResult_t initTransportsRank(struct ncclComm* comm, struct ncclComm* p // Compute time models for algorithm and protocol combinations NCCLCHECKGOTO(ncclTopoTuneModel(comm, comm->minCompCap, comm->maxCompCap, graphs), ret, fail); - INFO(NCCL_INIT, "%d coll channels, %d collnet channels, %d nvls channels, %d p2p channels, %d p2p channels per peer", comm->nChannels, comm->collNetChannels, comm->nvlsChannels, comm->p2pnChannels, comm->p2pnChannelsPerPeer); + INFO(NCCL_INIT, "%d coll channels, %d collnet channels, %d nvls channels, %d p2p channels, %d p2p channels per peer", comm->nChannels, comm->nChannels, comm->nvlsChannels, comm->p2pnChannels, comm->p2pnChannelsPerPeer); do { // Setup p2p structures in comm->tasks struct ncclTasks* tasks = &comm->tasks; @@ -1360,7 +1401,7 @@ static ncclResult_t initTransportsRank(struct ncclComm* comm, struct ncclComm* p NCCLCHECKGOTO(devCommSetup(comm), ret, fail); /* Local intra-node barrier */ - NCCLCHECKGOTO(bootstrapBarrier(comm->bootstrap, comm->localRankToRank, comm->localRank, comm->localRanks, comm->localRankToRank[0]), ret, fail); + NCCLCHECKGOTO(bootstrapIntraNodeBarrier(comm->bootstrap, comm->localRankToRank, comm->localRank, comm->localRanks, comm->localRankToRank[0]), ret, fail); // We should have allocated all buffers, collective fifos, ... we can // restore the affinity. @@ -1496,13 +1537,19 @@ static ncclResult_t ncclCommInitRankFunc(struct ncclAsyncJob* job_) { comm->cudaArch = cudaArch; comm->commHash = getHash(job->commId.internal, NCCL_UNIQUE_ID_BYTES); - INFO(NCCL_INIT,"comm %p rank %d nranks %d cudaDev %d nvmlDev %d busId %lx commId 0x%llx - Init START", comm, comm->rank, comm->nRanks, comm->cudaDev, comm->nvmlDev, comm->busId, (unsigned long long)hashUniqueId(job->commId)); + if (job->parent) { + INFO(NCCL_INIT,"ncclCommSplit comm %p rank %d nranks %d cudaDev %d nvmlDev %d busId %lx parent %p color %d key %d commId 0x%llx - Init START", + comm, comm->rank, comm->nRanks, comm->cudaDev, comm->nvmlDev, comm->busId, job->parent, job->color, job->key, (unsigned long long)hashUniqueId(job->commId)); + } else { + INFO(NCCL_INIT,"ncclCommInitRank comm %p rank %d nranks %d cudaDev %d nvmlDev %d busId %lx commId 0x%llx - Init START", + comm, comm->rank, comm->nRanks, comm->cudaDev, comm->nvmlDev, comm->busId, (unsigned long long)hashUniqueId(job->commId)); + } NCCLCHECKGOTO(initTransportsRank(comm, job->parent), res, fail); - NCCLCHECKGOTO(ncclLoadTunerPlugin(&comm->tuner), res, fail); + NCCLCHECKGOTO(ncclTunerPluginLoad(&comm->tuner), res, fail); if (comm->tuner) { - NCCLCHECK(comm->tuner->init(comm->nRanks, comm->nNodes, ncclDebugLog)); + NCCLCHECK(comm->tuner->init(comm->nRanks, comm->nNodes, ncclDebugLog, &comm->tunerContext)); } // update communicator state @@ -1519,8 +1566,13 @@ static ncclResult_t ncclCommInitRankFunc(struct ncclAsyncJob* job_) { comm, comm->nRanks, (unsigned long long)hashUniqueId(job->commId), comm->rank, comm->cudaDev); } - - INFO(NCCL_INIT,"comm %p rank %d nranks %d cudaDev %d nvmlDev %d busId %lx commId 0x%llx - Init COMPLETE", comm, comm->rank, comm->nRanks, comm->cudaDev, comm->nvmlDev, comm->busId, (unsigned long long)hashUniqueId(job->commId)); + if (job->parent) { + INFO(NCCL_INIT,"ncclCommSplit comm %p rank %d nranks %d cudaDev %d nvmlDev %d busId %lx parent %p color %d key %d commId 0x%llx - Init COMPLETE", + comm, comm->rank, comm->nRanks, comm->cudaDev, comm->nvmlDev, comm->busId, job->parent, job->color, job->key, (unsigned long long)hashUniqueId(job->commId)); + } else { + INFO(NCCL_INIT,"ncclCommInitRank comm %p rank %d nranks %d cudaDev %d nvmlDev %d busId %lx commId 0x%llx - Init COMPLETE", + comm, comm->rank, comm->nRanks, comm->cudaDev, comm->nvmlDev, comm->busId, (unsigned long long)hashUniqueId(job->commId)); + } exit: if (job->newcomm) { /* assign it to user pointer. */ @@ -1729,6 +1781,7 @@ static ncclResult_t ncclCommInitRankDev(ncclComm_t* newcomm, int nranks, ncclUni } NCCLCHECKGOTO(ncclCalloc(&comm, 1), res, fail); + comm->startMagic = comm->endMagic = NCCL_MAGIC; // Used to detect comm corruption. NCCLCHECKGOTO(ncclCudaHostCalloc((uint32_t**)&comm->abortFlag, 1), res, fail); NCCLCHECKGOTO(ncclCalloc((uint32_t**)&comm->abortFlagRefCount, 1), res, fail); *comm->abortFlagRefCount = 1; @@ -1926,8 +1979,8 @@ static ncclResult_t commCleanup(ncclComm_t comm) { } if (comm->tuner != NULL) { - NCCLCHECK(comm->tuner->destroy()); - NCCLCHECK(ncclCloseTunerPlugin(&comm->tuner)); + NCCLCHECK(comm->tuner->destroy(comm->tunerContext)); + NCCLCHECK(ncclTunerPluginUnload(&comm->tuner)); } NCCLCHECK(commFree(comm)); @@ -2142,7 +2195,7 @@ ncclResult_t ncclCommSplit(ncclComm_t comm, int color, int key, ncclComm_t *newc ncclResult_t res = ncclSuccess; NCCLCHECK(ncclGroupStartInternal()); - NCCLCHECKGOTO(PtrCheck(comm, "CommSplit", "comm"), res, fail); + NCCLCHECKGOTO(CommCheck(comm, "CommSplit", "comm"), res, fail); NCCLCHECKGOTO(PtrCheck(newcomm, "CommSplit", "newcomm"), res, fail); NCCLCHECKGOTO(ncclCommEnsureReady(comm), res, fail); @@ -2152,6 +2205,7 @@ ncclResult_t ncclCommSplit(ncclComm_t comm, int color, int key, ncclComm_t *newc INFO(NCCL_INIT, "Rank %d has color with NCCL_SPLIT_NOCOLOR, not creating a new communicator", comm->rank); } else { NCCLCHECKGOTO(ncclCalloc(&childComm, 1), res, fail); + childComm->startMagic = childComm->endMagic = NCCL_MAGIC; if (comm->config.splitShare) { childComm->abortFlag = comm->abortFlag; childComm->abortFlagRefCount = comm->abortFlagRefCount; @@ -2224,7 +2278,7 @@ const char* ncclGetLastError(ncclComm_t comm) { NCCL_API(ncclResult_t, ncclCommGetAsyncError, ncclComm_t comm, ncclResult_t *asyncError); ncclResult_t ncclCommGetAsyncError(ncclComm_t comm, ncclResult_t *asyncError) { - NCCLCHECK(PtrCheck(comm, "ncclGetAsyncError", "comm")); + NCCLCHECK(CommCheck(comm, "ncclGetAsyncError", "comm")); NCCLCHECK(PtrCheck(asyncError, "ncclGetAsyncError", "asyncError")); *asyncError = __atomic_load_n(&comm->asyncResult, __ATOMIC_ACQUIRE); @@ -2236,7 +2290,7 @@ NCCL_API(ncclResult_t, ncclCommCount, const ncclComm_t comm, int* count); ncclResult_t ncclCommCount(const ncclComm_t comm, int* count) { NVTX3_FUNC_RANGE_IN(nccl_domain); - NCCLCHECK(PtrCheck(comm, "CommCount", "comm")); + NCCLCHECK(CommCheck(comm, "CommCount", "comm")); NCCLCHECK(PtrCheck(count, "CommCount", "count")); /* init thread must be joined before we access the attributes of comm. */ @@ -2250,7 +2304,7 @@ NCCL_API(ncclResult_t, ncclCommCuDevice, const ncclComm_t comm, int* devid); ncclResult_t ncclCommCuDevice(const ncclComm_t comm, int* devid) { NVTX3_FUNC_RANGE_IN(nccl_domain); - NCCLCHECK(PtrCheck(comm, "CommCuDevice", "comm")); + NCCLCHECK(CommCheck(comm, "CommCuDevice", "comm")); NCCLCHECK(PtrCheck(devid, "CommCuDevice", "devid")); NCCLCHECK(ncclCommEnsureReady(comm)); @@ -2263,7 +2317,7 @@ NCCL_API(ncclResult_t, ncclCommUserRank, const ncclComm_t comm, int* rank); ncclResult_t ncclCommUserRank(const ncclComm_t comm, int* rank) { NVTX3_FUNC_RANGE_IN(nccl_domain); - NCCLCHECK(PtrCheck(comm, "CommUserRank", "comm")); + NCCLCHECK(CommCheck(comm, "CommUserRank", "comm")); NCCLCHECK(PtrCheck(rank, "CommUserRank", "rank")); NCCLCHECK(ncclCommEnsureReady(comm)); @@ -2302,7 +2356,7 @@ ncclResult_t ncclMemAlloc(void **ptr, size_t size) { if (mcSupport) { memprop.type = CU_MEM_ALLOCATION_TYPE_PINNED; memprop.location.type = CU_MEM_LOCATION_TYPE_DEVICE; - memprop.requestedHandleTypes = NVLS_CU_MEM_HANDLE_TYPE; + memprop.requestedHandleTypes = ncclCuMemHandleType; memprop.location.id = currentDev; // Query device to see if RDMA support is available CUCHECK(cuDeviceGetAttribute(&flag, CU_DEVICE_ATTRIBUTE_GPU_DIRECT_RDMA_SUPPORTED, currentDev)); @@ -2314,7 +2368,7 @@ ncclResult_t ncclMemAlloc(void **ptr, size_t size) { mcprop.size = size; /* device cnt is a dummy value right now, it might affect mc granularity in the future. */ mcprop.numDevices = dcnt; - mcprop.handleTypes = NVLS_CU_MEM_HANDLE_TYPE; + mcprop.handleTypes = ncclCuMemHandleType; mcprop.flags = 0; CUCHECK(cuMulticastGetGranularity(&mcGran, &mcprop, CU_MULTICAST_GRANULARITY_RECOMMENDED)); diff --git a/src/misc/argcheck.cc b/src/misc/argcheck.cc index c5909337d..59023ae79 100644 --- a/src/misc/argcheck.cc +++ b/src/misc/argcheck.cc @@ -33,6 +33,15 @@ ncclResult_t PtrCheck(void* ptr, const char* opname, const char* ptrname) { return ncclSuccess; } +ncclResult_t CommCheck(struct ncclComm* comm, const char* opname, const char* ptrname) { + NCCLCHECK(PtrCheck(comm, opname, ptrname)); + if (comm->startMagic != NCCL_MAGIC || comm->endMagic != NCCL_MAGIC) { + WARN("Error: corrupted comm object detected"); + return ncclInvalidArgument; + } + return ncclSuccess; +} + ncclResult_t ArgsCheck(struct ncclInfo* info) { // First, the easy ones if (info->root < 0 || info->root >= info->comm->nRanks) { diff --git a/src/misc/cudawrap.cc b/src/misc/cudawrap.cc index 8ccc7a876..6f5badfd8 100644 --- a/src/misc/cudawrap.cc +++ b/src/misc/cudawrap.cc @@ -9,8 +9,6 @@ #include "param.h" #include "cudawrap.h" -#include - // This env var (NCCL_CUMEM_ENABLE) toggles cuMem API usage NCCL_PARAM(CuMemEnable, "CUMEM_ENABLE", -2); @@ -51,112 +49,119 @@ int ncclCuMemEnable() { return param >= 0 ? param : (param == -2 && ncclCuMemSupported); } -#define DECLARE_CUDA_PFN(symbol,version) PFN_##symbol##_v##version pfn_##symbol = nullptr +#define DECLARE_CUDA_PFN(symbol) PFN_##symbol pfn_##symbol = nullptr #if CUDART_VERSION >= 11030 /* CUDA Driver functions loaded with cuGetProcAddress for versioning */ -DECLARE_CUDA_PFN(cuDeviceGet, 2000); -DECLARE_CUDA_PFN(cuDeviceGetAttribute, 2000); -DECLARE_CUDA_PFN(cuGetErrorString, 6000); -DECLARE_CUDA_PFN(cuGetErrorName, 6000); +DECLARE_CUDA_PFN(cuDeviceGet); +DECLARE_CUDA_PFN(cuDeviceGetAttribute); +DECLARE_CUDA_PFN(cuGetErrorString); +DECLARE_CUDA_PFN(cuGetErrorName); /* enqueue.cc */ -DECLARE_CUDA_PFN(cuMemGetAddressRange, 3020); +DECLARE_CUDA_PFN(cuMemGetAddressRange); /* proxy.cc */ -DECLARE_CUDA_PFN(cuCtxCreate, 3020); -DECLARE_CUDA_PFN(cuCtxDestroy, 4000); -DECLARE_CUDA_PFN(cuCtxGetCurrent, 4000); -DECLARE_CUDA_PFN(cuCtxSetCurrent, 4000); -DECLARE_CUDA_PFN(cuCtxGetDevice, 2000); +DECLARE_CUDA_PFN(cuCtxCreate); +DECLARE_CUDA_PFN(cuCtxDestroy); +DECLARE_CUDA_PFN(cuCtxGetCurrent); +DECLARE_CUDA_PFN(cuCtxSetCurrent); +DECLARE_CUDA_PFN(cuCtxGetDevice); /* cuMem API support */ -DECLARE_CUDA_PFN(cuMemAddressReserve, 10020); -DECLARE_CUDA_PFN(cuMemAddressFree, 10020); -DECLARE_CUDA_PFN(cuMemCreate, 10020); -DECLARE_CUDA_PFN(cuMemGetAllocationGranularity, 10020); -DECLARE_CUDA_PFN(cuMemExportToShareableHandle, 10020); -DECLARE_CUDA_PFN(cuMemImportFromShareableHandle, 10020); -DECLARE_CUDA_PFN(cuMemMap, 10020); -DECLARE_CUDA_PFN(cuMemRelease, 10020); -DECLARE_CUDA_PFN(cuMemRetainAllocationHandle, 11000); -DECLARE_CUDA_PFN(cuMemSetAccess, 10020); -DECLARE_CUDA_PFN(cuMemUnmap, 10020); +DECLARE_CUDA_PFN(cuMemAddressReserve); +DECLARE_CUDA_PFN(cuMemAddressFree); +DECLARE_CUDA_PFN(cuMemCreate); +DECLARE_CUDA_PFN(cuMemGetAllocationGranularity); +DECLARE_CUDA_PFN(cuMemExportToShareableHandle); +DECLARE_CUDA_PFN(cuMemImportFromShareableHandle); +DECLARE_CUDA_PFN(cuMemMap); +DECLARE_CUDA_PFN(cuMemRelease); +DECLARE_CUDA_PFN(cuMemRetainAllocationHandle); +DECLARE_CUDA_PFN(cuMemSetAccess); +DECLARE_CUDA_PFN(cuMemUnmap); /* ncclMemAlloc/Free */ -DECLARE_CUDA_PFN(cuPointerGetAttribute, 4000); +DECLARE_CUDA_PFN(cuPointerGetAttribute); #if CUDA_VERSION >= 11070 /* transport/collNet.cc/net.cc*/ -DECLARE_CUDA_PFN(cuMemGetHandleForAddressRange, 11070); // DMA-BUF support +DECLARE_CUDA_PFN(cuMemGetHandleForAddressRange); // DMA-BUF support #endif #if CUDA_VERSION >= 12010 /* NVSwitch Multicast support */ -DECLARE_CUDA_PFN(cuMulticastAddDevice, 12010); -DECLARE_CUDA_PFN(cuMulticastBindMem, 12010); -DECLARE_CUDA_PFN(cuMulticastBindAddr, 12010); -DECLARE_CUDA_PFN(cuMulticastCreate, 12010); -DECLARE_CUDA_PFN(cuMulticastGetGranularity, 12010); -DECLARE_CUDA_PFN(cuMulticastUnbind, 12010); +DECLARE_CUDA_PFN(cuMulticastAddDevice); +DECLARE_CUDA_PFN(cuMulticastBindMem); +DECLARE_CUDA_PFN(cuMulticastBindAddr); +DECLARE_CUDA_PFN(cuMulticastCreate); +DECLARE_CUDA_PFN(cuMulticastGetGranularity); +DECLARE_CUDA_PFN(cuMulticastUnbind); #endif #endif -/* CUDA Driver functions loaded with dlsym() */ -DECLARE_CUDA_PFN(cuInit, 2000); -DECLARE_CUDA_PFN(cuDriverGetVersion, 2020); -DECLARE_CUDA_PFN(cuGetProcAddress, 11030); - #define CUDA_DRIVER_MIN_VERSION 11030 -static void *cudaLib; int ncclCudaDriverVersionCache = -1; bool ncclCudaLaunchBlocking = false; #if CUDART_VERSION >= 11030 -/* - Load the CUDA symbols - */ -static ncclResult_t cudaPfnFuncLoader(void) { - CUresult res; -#define LOAD_SYM(symbol, version, ignore) do { \ - res = pfn_cuGetProcAddress(#symbol, (void **) (&pfn_##symbol), version, 0); \ - if (res != 0) { \ +#if CUDART_VERSION >= 12000 +#define LOAD_SYM(symbol, ignore) do { \ + cudaDriverEntryPointQueryResult driverStatus; \ + res = cudaGetDriverEntryPoint(#symbol, (void **) (&pfn_##symbol), cudaEnableDefault, &driverStatus); \ + if (res != cudaSuccess || driverStatus != cudaDriverEntryPointSuccess) { \ + if (!ignore) { \ + WARN("Retrieve %s failed with %d status %d", #symbol, res, driverStatus); \ + return ncclSystemError; } \ + } } while(0) +#else +#define LOAD_SYM(symbol, ignore) do { \ + res = cudaGetDriverEntryPoint(#symbol, (void **) (&pfn_##symbol), cudaEnableDefault); \ + if (res != cudaSuccess) { \ if (!ignore) { \ - WARN("Retrieve %s version %d failed with %d", #symbol, version, res); \ + WARN("Retrieve %s failed with %d", #symbol, res); \ return ncclSystemError; } \ } } while(0) +#endif - LOAD_SYM(cuGetErrorString, 6000, 0); - LOAD_SYM(cuGetErrorName, 6000, 0); - LOAD_SYM(cuDeviceGet, 2000, 0); - LOAD_SYM(cuDeviceGetAttribute, 2000, 0); - LOAD_SYM(cuMemGetAddressRange, 3020, 1); - LOAD_SYM(cuCtxCreate, 3020, 1); - LOAD_SYM(cuCtxDestroy, 4000, 1); - LOAD_SYM(cuCtxGetCurrent, 4000, 1); - LOAD_SYM(cuCtxSetCurrent, 4000, 1); - LOAD_SYM(cuCtxGetDevice, 2000, 1); +/* + Load the CUDA symbols + */ +static ncclResult_t cudaPfnFuncLoader(void) { + + cudaError_t res; + + LOAD_SYM(cuGetErrorString, 0); + LOAD_SYM(cuGetErrorName, 0); + LOAD_SYM(cuDeviceGet, 0); + LOAD_SYM(cuDeviceGetAttribute, 0); + LOAD_SYM(cuMemGetAddressRange, 1); + LOAD_SYM(cuCtxCreate, 1); + LOAD_SYM(cuCtxDestroy, 1); + LOAD_SYM(cuCtxGetCurrent, 1); + LOAD_SYM(cuCtxSetCurrent, 1); + LOAD_SYM(cuCtxGetDevice, 1); /* cuMem API support */ - LOAD_SYM(cuMemAddressReserve, 10020, 1); - LOAD_SYM(cuMemAddressFree, 10020, 1); - LOAD_SYM(cuMemCreate, 10020, 1); - LOAD_SYM(cuMemGetAllocationGranularity, 10020, 1); - LOAD_SYM(cuMemExportToShareableHandle, 10020, 1); - LOAD_SYM(cuMemImportFromShareableHandle, 10020, 1); - LOAD_SYM(cuMemMap, 10020, 1); - LOAD_SYM(cuMemRelease, 10020, 1); - LOAD_SYM(cuMemRetainAllocationHandle, 11000, 1); - LOAD_SYM(cuMemSetAccess, 10020, 1); - LOAD_SYM(cuMemUnmap, 10020, 1); + LOAD_SYM(cuMemAddressReserve, 1); + LOAD_SYM(cuMemAddressFree, 1); + LOAD_SYM(cuMemCreate, 1); + LOAD_SYM(cuMemGetAllocationGranularity, 1); + LOAD_SYM(cuMemExportToShareableHandle, 1); + LOAD_SYM(cuMemImportFromShareableHandle, 1); + LOAD_SYM(cuMemMap, 1); + LOAD_SYM(cuMemRelease, 1); + LOAD_SYM(cuMemRetainAllocationHandle, 1); + LOAD_SYM(cuMemSetAccess, 1); + LOAD_SYM(cuMemUnmap, 1); /* ncclMemAlloc/Free */ - LOAD_SYM(cuPointerGetAttribute, 4000, 1); + LOAD_SYM(cuPointerGetAttribute, 1); #if CUDA_VERSION >= 11070 - LOAD_SYM(cuMemGetHandleForAddressRange, 11070, 1); // DMA-BUF support + LOAD_SYM(cuMemGetHandleForAddressRange, 1); // DMA-BUF support #endif #if CUDA_VERSION >= 12010 /* NVSwitch Multicast support */ - LOAD_SYM(cuMulticastAddDevice, 12010, 1); - LOAD_SYM(cuMulticastBindMem, 12010, 1); - LOAD_SYM(cuMulticastBindAddr, 12010, 1); - LOAD_SYM(cuMulticastCreate, 12010, 1); - LOAD_SYM(cuMulticastGetGranularity, 12010, 1); - LOAD_SYM(cuMulticastUnbind, 12010, 1); + LOAD_SYM(cuMulticastAddDevice, 1); + LOAD_SYM(cuMulticastBindMem, 1); + LOAD_SYM(cuMulticastBindAddr, 1); + LOAD_SYM(cuMulticastCreate, 1); + LOAD_SYM(cuMulticastGetGranularity, 1); + LOAD_SYM(cuMulticastUnbind, 1); #endif return ncclSuccess; } @@ -171,47 +176,12 @@ static void initOnceFunc() { ncclCudaLaunchBlocking = val!=nullptr && val[0]!=0 && !(val[0]=='0' && val[1]==0); } while (0); - CUresult res; - /* - * Load CUDA driver library - */ - char path[1024]; - const char *ncclCudaPath = ncclGetEnv("NCCL_CUDA_PATH"); - if (ncclCudaPath == NULL) - snprintf(path, 1024, "%s", "libcuda.so"); - else - snprintf(path, 1024, "%s/%s", ncclCudaPath, "libcuda.so"); - - (void) dlerror(); // Clear any previous errors - cudaLib = dlopen(path, RTLD_LAZY); - if (cudaLib == NULL) { - WARN("Failed to find CUDA library %s (NCCL_CUDA_PATH='%s') : %s", path, ncclCudaPath ? ncclCudaPath : "", dlerror()); - goto error; - } - - /* - * Load initial CUDA functions - */ - - pfn_cuInit = (PFN_cuInit_v2000) dlsym(cudaLib, "cuInit"); - if (pfn_cuInit == NULL) { - WARN("Failed to load CUDA missing symbol cuInit"); - goto error; - } - - pfn_cuDriverGetVersion = (PFN_cuDriverGetVersion_v2020) dlsym(cudaLib, "cuDriverGetVersion"); - if (pfn_cuDriverGetVersion == NULL) { - WARN("Failed to load CUDA missing symbol cuDriverGetVersion"); - goto error; - } - + ncclResult_t ret = ncclSuccess; + int cudaDev; int driverVersion; - res = pfn_cuDriverGetVersion(&driverVersion); - if (res != 0) { - WARN("cuDriverGetVersion failed with %d", res); - goto error; - } + CUDACHECKGOTO(cudaGetDevice(&cudaDev), ret, error); // Initialize the driver + CUDACHECKGOTO(cudaDriverGetVersion(&driverVersion), ret, error); INFO(NCCL_INIT, "cudaDriverVersion %d", driverVersion); if (driverVersion < CUDA_DRIVER_MIN_VERSION) { @@ -220,19 +190,6 @@ static void initOnceFunc() { goto error; } - pfn_cuGetProcAddress = (PFN_cuGetProcAddress_v11030) dlsym(cudaLib, "cuGetProcAddress"); - if (pfn_cuGetProcAddress == NULL) { - WARN("Failed to load CUDA missing symbol cuGetProcAddress"); - goto error; - } - - /* - * Required to initialize the CUDA Driver. - * Multiple calls of cuInit() will return immediately - * without making any relevant change - */ - pfn_cuInit(0); - #if CUDART_VERSION >= 11030 if (cudaPfnFuncLoader()) { WARN("CUDA some PFN functions not found in the library"); @@ -243,7 +200,7 @@ static void initOnceFunc() { // Determine whether we support the cuMem APIs or not ncclCuMemSupported = ncclIsCuMemSupported(); - initResult = ncclSuccess; + initResult = ret; return; error: initResult = ncclSystemError; diff --git a/src/misc/socket.cc b/src/misc/socket.cc index 3aeed6c57..6e9fb0790 100644 --- a/src/misc/socket.cc +++ b/src/misc/socket.cc @@ -790,6 +790,24 @@ ncclResult_t ncclSocketRecv(struct ncclSocket* sock, void* ptr, int size) { return ncclSuccess; } +ncclResult_t ncclSocketSendRecv(struct ncclSocket* sendSock, void* sendPtr, int sendSize, struct ncclSocket* recvSock, void* recvPtr, int recvSize) { + int sendOffset = 0, recvOffset = 0; + if (sendSock == NULL || recvSock == NULL) { + WARN("ncclSocketSendRecv: invalid socket %p/%p", sendSock, recvSock); + return ncclInternalError; + } + if (sendSock->state != ncclSocketStateReady || recvSock->state != ncclSocketStateReady) { + WARN("ncclSocketSendRecv: socket state (%d/%d) is not ready", sendSock->state, recvSock->state); + return ncclInternalError; + } + while (sendOffset < sendSize || recvOffset < recvSize) { + if (sendOffset < sendSize) NCCLCHECK(socketProgress(NCCL_SOCKET_SEND, sendSock, sendPtr, sendSize, &sendOffset)); + if (recvOffset < recvSize) NCCLCHECK(socketProgress(NCCL_SOCKET_RECV, recvSock, recvPtr, recvSize, &recvOffset)); + } + return ncclSuccess; +} + + // Receive or detect connection closed ncclResult_t ncclSocketTryRecv(struct ncclSocket* sock, void* ptr, int size, int* closed, bool blocking) { int offset = 0; diff --git a/src/misc/tuner.cc b/src/misc/tuner.cc index 8f5b2ce34..ae6ade32f 100644 --- a/src/misc/tuner.cc +++ b/src/misc/tuner.cc @@ -13,69 +13,178 @@ #include "nccl_tuner.h" pthread_mutex_t tunerPluginLock = PTHREAD_MUTEX_INITIALIZER; -static int tunerPluginRefCount = -1; +static int tunerPluginRefCount; static void* tunerPluginLib = nullptr; ncclTuner_t* tunerSymbol = nullptr; -ncclResult_t ncclLoadTunerPlugin(ncclTuner_t** tuner) { +static void* tryOpenDynamicLib(const char* name) { + if (nullptr == name || strlen(name) == 0) { + return nullptr; + } + void *handle = dlopen(name, RTLD_LAZY | RTLD_LOCAL); + if (nullptr == handle) { + if (ENOENT == errno) { + INFO(NCCL_ENV|NCCL_TUNING, "TUNER/Plugin: No plugin found (%s)", name); + } + } + return handle; +} + +static void summarizeOpenTunerPluginLibErrors(char* pluginNames) { + const char *separator = " "; + int len = strlen(pluginNames); + // remove tail separator + pluginNames[len - 1] = '\0'; + + // remove last plugin name + while (len > 0 && pluginNames[--len] != *separator); + if (len > 0) { + pluginNames[len] = '\0'; + } + + // distinguish between one load attempt and multiple attempts + if (strstr(pluginNames, separator)) { + INFO(NCCL_ENV|NCCL_TUNING, "TUNER/Plugin: Most recent plugin load returned %d : %s. All attempts to load '%s' also failed.", errno, dlerror(), pluginNames); + } else { + INFO(NCCL_ENV|NCCL_TUNING, "TUNER/Plugin: Plugin load returned %d : %s : when loading %s", errno, dlerror(), pluginNames); + } +} + +static void* openTunerPluginLib(void) { + void *pluginLib; + +#define MAX_PLUGIN_LOAD 4 + + int len; + char tunerPluginLibNameTried[MAX_PLUGIN_LOAD * PATH_MAX] = { 0 }; + char *ptr = tunerPluginLibNameTried; + char tunerPluginLibName[PATH_MAX]; + const char *envTunerPluginName = getenv("NCCL_TUNER_PLUGIN"); + if (envTunerPluginName && strlen(envTunerPluginName)) { + INFO(NCCL_ENV|NCCL_TUNING, "TUNER/Plugin: NCCL_TUNER_PLUGIN set to %s", envTunerPluginName); + snprintf(tunerPluginLibName, PATH_MAX, "%s", envTunerPluginName); + pluginLib = tryOpenDynamicLib(tunerPluginLibName); + if (pluginLib) { + INFO(NCCL_ENV|NCCL_TUNING, "TUNER/Plugin: Plugin name set by env to %s", tunerPluginLibName); + return pluginLib; + } + len = PATH_MAX - strlen(ptr); + snprintf(ptr + strlen(ptr), len + 1, "%s ", tunerPluginLibName); + + snprintf(tunerPluginLibName, PATH_MAX, "libnccl-tuner-%s.so", envTunerPluginName); + pluginLib = tryOpenDynamicLib(tunerPluginLibName); + if (pluginLib) { + INFO(NCCL_ENV|NCCL_TUNING, "TUNER/Plugin: Plugin name set by env to %s", tunerPluginLibName); + return pluginLib; + } + len = PATH_MAX - strlen(ptr); + snprintf(ptr + strlen(ptr), len + 1, "%s ", tunerPluginLibName); + } else { + snprintf(tunerPluginLibName, PATH_MAX, "libnccl-tuner.so"); + pluginLib = tryOpenDynamicLib(tunerPluginLibName); + if (pluginLib) { + return pluginLib; + } + len = PATH_MAX - strlen(ptr); + snprintf(ptr + strlen(ptr), len + 1, "%s ", tunerPluginLibName); + } + + const char *envNetPluginName = getenv("NCCL_NET_PLUGIN"); + if (envNetPluginName && strlen(envNetPluginName)) { + // Users are allowed to pack tuner into the net plugin + snprintf(tunerPluginLibName, PATH_MAX, "%s", envNetPluginName); + pluginLib = tryOpenDynamicLib(tunerPluginLibName); + if (pluginLib) { + INFO(NCCL_ENV|NCCL_TUNING, "TUNER/Plugin: Plugin name set by env to %s", tunerPluginLibName); + return pluginLib; + } + len = PATH_MAX - strlen(ptr); + snprintf(ptr + strlen(ptr), len + 1, "%s ", tunerPluginLibName); + + snprintf(tunerPluginLibName, PATH_MAX, "libnccl-net-%s.so", envNetPluginName); + pluginLib = tryOpenDynamicLib(tunerPluginLibName); + if (pluginLib) { + INFO(NCCL_ENV|NCCL_TUNING, "TUNER/Plugin: Plugin name set by env to %s", tunerPluginLibName); + return pluginLib; + } + len = PATH_MAX - strlen(ptr); + snprintf(ptr + strlen(ptr), len + 1, "%s ", tunerPluginLibName); + } else { + snprintf(tunerPluginLibName, PATH_MAX, "libnccl-net.so"); + pluginLib = tryOpenDynamicLib(tunerPluginLibName); + if (pluginLib) { + return pluginLib; + } + len = PATH_MAX - strlen(ptr); + snprintf(ptr + strlen(ptr), len + 1, "%s ", tunerPluginLibName); + } + summarizeOpenTunerPluginLibErrors(ptr); + + tunerPluginLibName[0] = '\0'; + return nullptr; +} + +enum { + tunerPluginLoadFailed = -1, + tunerPluginLoadReady = 0, + tunerPluginLoadSuccess = 1, +}; + +ncclResult_t ncclTunerPluginLoad(ncclTuner_t** tuner) { // Initialize to nullptr by default if plugin tuner cannot be loaded. *tuner = nullptr; - if (tunerPluginRefCount == -2) return ncclSuccess; + static int status = tunerPluginLoadReady; + if (tunerPluginLoadFailed == status) { + return ncclSuccess; + } pthread_mutex_lock(&tunerPluginLock); - if (tunerPluginRefCount == -1) { - tunerPluginRefCount = -2; // Default: no plugin, don't try again later - - const char* name = getenv("NCCL_TUNER_PLUGIN"); - if (name) { - INFO(NCCL_TUNING, "NCCL_TUNER_PLUGIN set to %s", name); - tunerPluginLib = dlopen(name, RTLD_LAZY | RTLD_LOCAL); - if (tunerPluginLib == nullptr) { - // dlopen does not guarantee to set errno, but dlerror only gives us a - // string, so checking errno doesn't hurt to try to provide a better - // error message - if (errno == ENOENT) { - INFO(NCCL_TUNING, "Tuner: no plugin found '%s', using default tuner instead.", name); - } else { - INFO(NCCL_TUNING, "Tuner: plugin load '%s' returned error (%d : %s), using default tuner instead.", name, errno, dlerror()); - } - } else { - tunerSymbol = (ncclTuner_t*)dlsym(tunerPluginLib, NCCL_TUNER_PLUGIN_SYMBOL); - if (tunerSymbol == nullptr) { - INFO(NCCL_TUNING, "Tuner: failed to find " NCCL_TUNER_PLUGIN_SYMBOL " in plugin (%s), using default tuner instead.", name); - dlclose(tunerPluginLib); - tunerPluginLib = nullptr; - } else { - INFO(NCCL_TUNING, "Opened tuner: '%s'", tunerSymbol->name); - tunerPluginRefCount = 0; - } - } - } + if (tunerPluginLoadFailed == status) { + goto exit; } - if (tunerPluginRefCount >= 0) { + if (tunerPluginLoadSuccess == status) { *tuner = tunerSymbol; - INFO(NCCL_INIT, "Using tuner plugin: '%s'", tunerSymbol->name); - tunerPluginRefCount++; + ++tunerPluginRefCount; + goto exit; } + + tunerPluginLib = openTunerPluginLib(); + if (nullptr == tunerPluginLib) { + INFO(NCCL_ENV|NCCL_TUNING, "TUNER/Plugin: Using internal tuner plugin."); + goto fail; + } + + tunerSymbol = (ncclTuner_t*)dlsym(tunerPluginLib, NCCL_TUNER_PLUGIN_SYMBOL); + if (tunerSymbol == nullptr) { + INFO(NCCL_ENV|NCCL_TUNING, "TUNER/Plugin: Failed to find " NCCL_TUNER_PLUGIN_SYMBOL ", using internal tuner instead."); + dlclose(tunerPluginLib); + goto fail; + } + + INFO(NCCL_ENV|NCCL_TUNING, "TUNER/Plugin: Using tuner plugin %s", tunerSymbol->name); + *tuner = tunerSymbol; + ++tunerPluginRefCount; + status = tunerPluginLoadSuccess; + +exit: pthread_mutex_unlock(&tunerPluginLock); return ncclSuccess; +fail: + tunerPluginLib = nullptr; + status = tunerPluginLoadFailed; + goto exit; } -ncclResult_t ncclCloseTunerPlugin(ncclTuner_t** tuner) { +ncclResult_t ncclTunerPluginUnload(ncclTuner_t** tuner) { if (*tuner == nullptr) return ncclSuccess; pthread_mutex_lock(&tunerPluginLock); - if (--tunerPluginRefCount == 0) { - if (tunerPluginLib == nullptr) { - WARN("Tuner plugin refcount is 0, yet tunerPluginLib ptr is NULL\n"); - } else { - INFO(NCCL_TUNING, "Closing tuner: '%s'", tunerSymbol->name); - dlclose(tunerPluginLib); - } + if (0 == (--tunerPluginRefCount)) { + INFO(NCCL_TUNING, "TUNER/Plugin: Closing tuner: '%s'", tunerSymbol->name); + dlclose(tunerPluginLib); tunerPluginLib = nullptr; tunerSymbol = nullptr; *tuner = nullptr; - tunerPluginRefCount = -1; } pthread_mutex_unlock(&tunerPluginLock); return ncclSuccess; diff --git a/src/nccl.h.in b/src/nccl.h.in index 901d8c0d2..3cf619dcf 100644 --- a/src/nccl.h.in +++ b/src/nccl.h.in @@ -174,7 +174,6 @@ ncclResult_t pncclCommCuDevice(const ncclComm_t comm, int* device); ncclResult_t ncclCommUserRank(const ncclComm_t comm, int* rank); ncclResult_t pncclCommUserRank(const ncclComm_t comm, int* rank); - /* Register CUDA buffer for zero-copy operation */ ncclResult_t ncclCommRegister(const ncclComm_t comm, void* buff, size_t size, void** handle); ncclResult_t pncclCommRegister(const ncclComm_t comm, void* buff, size_t size, void** handle); @@ -433,14 +432,6 @@ ncclResult_t pncclGroupStart(); ncclResult_t ncclGroupEnd(); ncclResult_t pncclGroupEnd(); -/* Register CUDA buffer for zero-copy operation */ -ncclResult_t ncclCommRegister(const ncclComm_t comm, void* buff, size_t size, void** handle); -ncclResult_t pncclCommRegister(const ncclComm_t comm, void* buff, size_t size, void** handle); - -/* Deregister CUDA buffer */ -ncclResult_t ncclCommDeregister(const ncclComm_t comm, void* handle); -ncclResult_t pncclCommDeregister(const ncclComm_t comm, void* handle); - #ifdef __cplusplus } // end extern "C" #endif diff --git a/src/net.cc b/src/net.cc index ba3479282..e978a1854 100644 --- a/src/net.cc +++ b/src/net.cc @@ -339,26 +339,87 @@ enum ncclNetState { enum ncclNetState ncclNetStates[3] = { ncclNetStateInit, ncclNetStateInit, ncclNetStateInit }; enum ncclNetState ncclCollNetStates[3] = { ncclNetStateInit, ncclNetStateInit, ncclNetStateInit }; -ncclResult_t ncclNetPluginInit() { - char ncclNetPluginName[128]; - const char* envPluginName = ncclGetEnv("NCCL_NET_PLUGIN"); - if (envPluginName && strlen(envPluginName)) { - snprintf(ncclNetPluginName, 128, "libnccl-net-%s.so", envPluginName); - INFO(NCCL_INIT, "Plugin name set by env to %s", ncclNetPluginName); - } else { - sprintf(ncclNetPluginName, "libnccl-net.so"); +static void* tryOpenDynamicLib(char* name) { + if (nullptr == name || strlen(name) == 0) { + return nullptr; } - void* netPluginLib = dlopen(ncclNetPluginName, RTLD_NOW | RTLD_LOCAL); - if (netPluginLib == nullptr) { - // dlopen does not guarantee to set errno, but dlerror only gives us a - // string, so checking errno doesn't hurt to try to provide a better - // error message - if (errno == ENOENT) { - INFO(NCCL_INIT|NCCL_NET, "NET/Plugin : dlerror=%s No plugin found (%s), using internal implementation", dlerror(), ncclNetPluginName); - // exit(-1); + void *handle = dlopen(name, RTLD_NOW | RTLD_LOCAL); + if (nullptr == handle) { + if (ENOENT == errno) { + INFO(NCCL_INIT|NCCL_NET, "NET/Plugin: No plugin found (%s)", name); } else { - INFO(NCCL_INIT|NCCL_NET, "NET/Plugin : Plugin load returned %d : %s.", errno, dlerror()); + INFO(NCCL_INIT|NCCL_NET, "NET/Plugin: Plugin load returned %d : %s when loading %s", errno, dlerror(), name); } + } + return handle; +} + +static void summarizeOpenNetPluginErrors(char* pluginNames) { + const char *separator = " "; + int len = strlen(pluginNames); + // remove tail separator + pluginNames[len - 1] = '\0'; + + // remove last plugin name + while (len > 0 && pluginNames[--len] != *separator); + if (len > 0) { + pluginNames[len] = '\0'; + } + + // distinguish between one load attempt and multiple attempts + if (strstr(pluginNames, separator)) { + INFO(NCCL_ENV|NCCL_TUNING, "NET/Plugin: Most recent plugin load returned %d : %s. All attempts to load '%s' also failed.", errno, dlerror(), pluginNames); + } else { + INFO(NCCL_ENV|NCCL_TUNING, "NET/Plugin: Plugin load returned %d : %s : when loading %s", errno, dlerror(), pluginNames); + } +} + +static void* openNetPluginLib(void) { + void *pluginLib; + +#define MAX_PLUGIN_LOAD 2 + + int len; + char netPluginLibNameTried[MAX_PLUGIN_LOAD * PATH_MAX] = { 0 }; + char *ptr = netPluginLibNameTried; + char netPluginLibName[PATH_MAX]; + const char *envNetPluginName = getenv("NCCL_NET_PLUGIN"); + if (envNetPluginName && strlen(envNetPluginName)) { + snprintf(netPluginLibName, PATH_MAX, "%s", envNetPluginName); + pluginLib = tryOpenDynamicLib(netPluginLibName); + if (pluginLib) { + INFO(NCCL_INIT|NCCL_NET, "NET/Plugin: Plugin name set by env to %s", netPluginLibName); + return pluginLib; + } + len = PATH_MAX - strlen(ptr); + snprintf(ptr + strlen(ptr), len + 1, "%s ", netPluginLibName); + + snprintf(netPluginLibName, PATH_MAX, "libnccl-net-%s.so", envNetPluginName); + pluginLib = tryOpenDynamicLib(netPluginLibName); + if (pluginLib) { + INFO(NCCL_INIT|NCCL_NET, "NET/Plugin: Plugin name set by env to %s", netPluginLibName); + return pluginLib; + } + len = PATH_MAX - strlen(ptr); + snprintf(ptr + strlen(ptr), len + 1, "%s ", netPluginLibName); + } else { + snprintf(netPluginLibName, PATH_MAX, "libnccl-net.so"); + pluginLib = tryOpenDynamicLib(netPluginLibName); + if (pluginLib) { + return pluginLib; + } + len = PATH_MAX - strlen(ptr); + snprintf(ptr + strlen(ptr), len + 1, "%s ", netPluginLibName); + } + summarizeOpenNetPluginErrors(ptr); + + return nullptr; +} + +ncclResult_t ncclNetPluginInit() { + void* netPluginLib = openNetPluginLib(); + if (netPluginLib == nullptr) { + INFO(NCCL_INIT|NCCL_NET, "NET/Plugin: Using internal network plugin."); return ncclSuccess; } diff --git a/src/proxy.cc b/src/proxy.cc index 7e25e3525..955c415ec 100644 --- a/src/proxy.cc +++ b/src/proxy.cc @@ -358,9 +358,13 @@ static ncclResult_t ncclProxyOpToArgs(struct ncclProxyOp* op, struct ncclProxyAr sub->channelId = op->channelId; sub->nsteps = op->nsteps; sub->nbytes = op->nbytes; + sub->offset = 0; sub->peer = op->root; sub->reg = op->reg; - sub->buffer = op->buffer; + sub->sendMhandle = op->sendMhandle; + sub->recvMhandle = op->recvMhandle; + sub->sendbuff = op->sendbuff; + sub->recvbuff = op->recvbuff; args->nsubs = subIndex+1; if (subIndex) { if ((args->sliceSteps != op->sliceSteps) || @@ -634,7 +638,7 @@ ncclResult_t ncclProxyComputeP2p(struct ncclInfo* info, struct ncclProxyOp* op, if (ncclParamChunkSize() != 0) { info->chunkSize = ncclParamChunkSize(); } - op->buffer = op->reg ? info->recvbuff : NULL; + op->recvbuff = op->reg ? (uint8_t*)info->recvbuff : NULL; op->chunkSize = info->chunkSize; op->nbytes = info->count; @@ -820,7 +824,7 @@ static int setProxyThreadContext(struct ncclProxyState* proxyState) { if (createThreadContext) { if (proxyState->cudaCtx == NULL) { if (CUPFN(cuCtxCreate(&proxyState->cudaCtx, - CU_CTX_SCHED_SPIN|CU_CTX_MAP_HOST, proxyState->cudaDev)) != CUDA_SUCCESS) { + NULL, 0, CU_CTX_SCHED_SPIN|CU_CTX_MAP_HOST, proxyState->cudaDev)) != CUDA_SUCCESS) { WARN("Failed to create CUDA context on device %d", proxyState->cudaDev); createThreadContext = 0; } @@ -1083,7 +1087,8 @@ ncclResult_t ncclProxyConnect(struct ncclComm* comm, int transport, int send, in ncclResult_t ncclProxyCallBlockingUDS(struct ncclComm* comm, int tpRank, int type, void* reqBuff, int reqSize, void* respBuff, int respSize, int *respFd) { ncclResult_t res = ncclSuccess; struct ncclIpcSocket ipcSock = { 0 }; - void *opId = (void*)((((uintptr_t)random()) << 32) | random()); + void *opId; + NCCLCHECK(getRandomData(&opId, sizeof(opId))); int rank = comm->topParentLocalRanks[comm->localRank]; struct ncclProxyState* sharedProxyState = comm->proxyState; @@ -1365,6 +1370,12 @@ static ncclResult_t proxyProgressAsync(struct ncclProxyAsyncOp* op, struct ncclP else if (op->type == ncclProxyMsgInit) { TRACE(NCCL_PROXY, "proxyProgressAsync::ncclProxyMsgInit opId=%p op.reqBuff=%p", op->opId, op->reqBuff); res = proxyConnInit(peer, connectionPool, proxyState, (ncclProxyInitReq*) op->reqBuff, (ncclProxyInitResp*) op->respBuff, &op->connection); + } else if (op->type == ncclProxyMsgRegister) { + TRACE(NCCL_PROXY, "proxyProgressAsync::ncclProxyMsgRegister opId=%p op.reqBuff=%p, op->reqSize=%d, op->respSize=%d", op->opId, op->reqBuff, op->reqSize, op->respSize); + res = op->connection->tcomm->proxyRegister(op->connection, proxyState, op->reqBuff, op->reqSize, op->respBuff, op->respSize, &done); + } else if (op->type == ncclProxyMsgDeregister) { + TRACE(NCCL_PROXY, "proxyProgressAsync::ncclProxyMsgDeregister opId=%p op.reqBuff=%p, op->reqSize=%d, op->respSize=%d", op->opId, op->reqBuff, op->reqSize, op->respSize); + res = op->connection->tcomm->proxyDeregister(op->connection, proxyState, op->reqBuff, op->reqSize, &done); } else return ncclInternalError; if (done) { @@ -1435,6 +1446,8 @@ static bool proxyMatchOpType(int type) { case ncclProxyMsgSetup: case ncclProxyMsgConnect: case ncclProxyMsgGetFd: + case ncclProxyMsgRegister: + case ncclProxyMsgDeregister: return true; default: return false; @@ -1663,12 +1676,6 @@ ncclResult_t ncclProxyInit(struct ncclComm* comm, struct ncclSocket* sock, union // UDS support NCCLCHECK(ncclIpcSocketInit(&comm->proxyState->ipcSock, comm->rank, peerAddressesUDS[comm->rank], comm->abortFlag)); - // Seed the random number generator for UDS filename generation - struct timeval time; - gettimeofday(&time,NULL); - unsigned int seed = time.tv_sec*time.tv_usec; - seed ^= getpid(); - srandom(seed); return ncclSuccess; } diff --git a/src/register.cc b/src/register.cc index 0e252a2f2..c46899294 100644 --- a/src/register.cc +++ b/src/register.cc @@ -34,7 +34,7 @@ ncclResult_t ncclNetRegister(struct ncclComm* comm, void* addr, size_t size, str // Find local devices for p2p operations for (int c=0; cp2pnChannels; c++) { int dev; - if (ncclTopoGetLocalNet(comm->topo, comm->rank, c, &dev) != ncclSuccess) goto end; // No local net + if (ncclTopoGetLocalNet(comm->topo, comm->rank, c, NULL, &dev) != ncclSuccess) goto end; // No local net ncclNetProperties_t props; NCCLCHECKGOTO(comm->ncclNet->getProperties(dev, &props), ret, end); if (props.regIsGlobal == 0) { // We need to be sure all NICs support global registration. @@ -152,7 +152,7 @@ ncclResult_t ncclRegCleanup(struct ncclComm* comm) { NCCL_API(ncclResult_t, ncclCommRegister, const ncclComm_t comm, void* buff, size_t size, void** handle); ncclResult_t ncclCommRegister(const ncclComm_t comm, void* buff, size_t size, void** handle) { - NCCLCHECK(PtrCheck(comm, "ncclCommRegister", "comm")); + NCCLCHECK(CommCheck(comm, "ncclCommRegister", "comm")); if (comm->checkPointers) NCCLCHECK(CudaPtrCheck(buff, comm, "buff", "ncclCommRegister")); NCCLCHECK(ncclRegister(comm, buff, size, handle)); return ncclSuccess; @@ -160,7 +160,7 @@ ncclResult_t ncclCommRegister(const ncclComm_t comm, void* buff, size_t size, vo NCCL_API(ncclResult_t, ncclCommDeregister, const ncclComm_t comm, void* handle); ncclResult_t ncclCommDeregister(const ncclComm_t comm, void* handle) { - NCCLCHECK(PtrCheck(comm, "ncclCommRegister", "comm")); + NCCLCHECK(CommCheck(comm, "ncclCommRegister", "comm")); struct ncclReg* reg = (struct ncclReg*)handle; struct ncclRegCache* cache = &comm->regCache; int slot; @@ -175,6 +175,9 @@ ncclResult_t ncclCommDeregister(const ncclComm_t comm, void* handle) { NCCLCHECK(ncclNvlsDeregBuffer(®->mcHandle, reg->regAddr, reg->dev, reg->regSize)); reg->regAddr = (CUdeviceptr)NULL; } + if (reg->state & COLLNET_REG_COMPLETE) { + NCCLCHECK(ncclCollnetDeregBuffer(comm, reg->proxyconn, reg->collnetHandle)); + } free(reg); memmove(cache->slots+slot, cache->slots+slot+1, (cache->population-slot-1)*sizeof(struct ncclReg*)); cache->population -= 1; diff --git a/src/transport.cc b/src/transport.cc index e229d382c..710285680 100644 --- a/src/transport.cc +++ b/src/transport.cc @@ -229,7 +229,7 @@ ncclResult_t ncclTransportP2pSetup(struct ncclComm* comm, struct ncclTopoGraph* * others might still be trying to connect and import the buffer. No sync can lead to invalid * shmem/cuda buffer. In addition, we also clear all connect masks and free each connectInfo array */ for (int i = 1; i < comm->nRanks; i++) { - int bootstrapTag = (i << 8) + (graph ? graph->id + 1 : 0); + int bootstrapTag = (i << 8) + (1 << 7) + (graph ? graph->id + 1 : 0); int recvPeer = (comm->rank - i + comm->nRanks) % comm->nRanks; int sendPeer = (comm->rank + i) % comm->nRanks; int flag = 0; @@ -271,27 +271,19 @@ extern struct ncclTransport collNetTransport; // All ranks must participate in collNetSetup call // We do not NCCLCHECK this call because we would fall back to P2P network in case CollNet setup fails -int ncclTransportCollNetSetup(struct ncclComm* comm, struct ncclTopoGraph* collNetGraph, struct ncclChannel* channel, int masterRank, int masterPeer, int collNetGraphChannelId, int type) { +int ncclTransportCollNetSetup(struct ncclComm* comm, struct ncclTopoGraph* collNetGraph, struct ncclChannel* channel, int masterRank, int masterPeer, int collNetGraphChannelId, int type, ncclConnect* connect) { int fail = 1; int rank = comm->rank; int nranks = comm->nRanks; int nMasters = comm->nNodes; - int rankInCollNet = -1; int isMaster = (rank == masterRank) ? 1 : 0; - struct { - int collNetRank; - ncclConnect connect; - } sendrecvExchange; // check if we can connect to collnet, whose root is the nranks-th rank struct ncclPeerInfo *myInfo = comm->peerInfo+rank, *peerInfo = comm->peerInfo+nranks; peerInfo->rank = nranks; - // send master receives connect info from peer recv master if (isMaster && type == collNetSend) { - NCCLCHECK(bootstrapRecv(comm->bootstrap, masterPeer, collNetGraph->id, &sendrecvExchange, sizeof(sendrecvExchange))); - rankInCollNet = sendrecvExchange.collNetRank; - TRACE(NCCL_INIT, "CollNet [send] : rank %d collNetRank %d collNetNranks %d received connect from rank %d", rank, rankInCollNet, nMasters, masterPeer); + TRACE(NCCL_INIT, "CollNet [send] : rank %d collNetRank %d collNetNranks %d received connect from rank %d", rank, comm->node, nMasters, masterPeer); } // select @@ -327,24 +319,20 @@ int ncclTransportCollNetSetup(struct ncclComm* comm, struct ncclTopoGraph* collN c++; } } - if (isMaster) rankInCollNet = comm->node; } else { // send side : copy in connect info received from peer recv master - if (isMaster) memcpy(masterConnects+rankInCollNet, &(sendrecvExchange.connect), sizeof(struct ncclConnect)); + if (isMaster) memcpy(masterConnects+comm->node, connect, sizeof(struct ncclConnect)); } // connect if (isMaster) { - NCCLCHECKGOTO(transportComm->connect(comm, masterConnects, nMasters, rankInCollNet, conn), res, cleanup); + NCCLCHECKGOTO(transportComm->connect(comm, masterConnects, nMasters, comm->node, conn), res, cleanup); struct ncclDevChannelPeer* devRoot; CUDACHECKGOTO(cudaMemcpy(&devRoot, channel->devPeers + nranks, sizeof(struct ncclDevChannelPeer*), cudaMemcpyDeviceToHost), res, cleanup); struct ncclConnInfo* devConnInfo = (type == collNetRecv) ? devRoot->recv + type : devRoot->send + type; CUDACHECKGOTO(cudaMemcpy(devConnInfo, &conn->conn, sizeof(struct ncclConnInfo), cudaMemcpyHostToDevice), res, cleanup); } - // recv side sends connect info to send side if (isMaster && type == collNetRecv) { - sendrecvExchange.collNetRank = rankInCollNet; - memcpy(&sendrecvExchange.connect, masterConnects+rankInCollNet, sizeof(struct ncclConnect)); - NCCLCHECKGOTO(bootstrapSend(comm->bootstrap, masterPeer, collNetGraph->id, &sendrecvExchange, sizeof(sendrecvExchange)), res, cleanup); - TRACE(NCCL_INIT, "CollNet [recv] : rank %d collNetRank %d collNetNranks %d sent connect to rank %d", rank, rankInCollNet, nMasters, masterPeer); + memcpy(connect, masterConnects+comm->node, sizeof(struct ncclConnect)); + TRACE(NCCL_INIT, "CollNet [recv] : rank %d collNetRank %d collNetNranks %d sent connect to rank %d", rank, comm->node, nMasters, masterPeer); } fail = 0; cleanup: diff --git a/src/transport/coll_net.cc b/src/transport/coll_net.cc index 302e5263b..abe051822 100644 --- a/src/transport/coll_net.cc +++ b/src/transport/coll_net.cc @@ -9,6 +9,7 @@ #include "graph.h" #include "proxy.h" #include "gdrwrap.h" +#include "assert.h" int64_t ncclParamGdrCopySyncEnable(); int64_t ncclParamGdrCopyFlushEnable(); @@ -151,8 +152,9 @@ static ncclResult_t sendSetup(struct ncclComm* comm, struct ncclTopoGraph* graph struct setupReq req = { 0 }; int proxyRank, tpProxyRank; - NCCLCHECK(ncclTopoGetNetDev(comm, myInfo->rank, graph, channelId, -1, &req.netDev, &proxyRank)); - NCCLCHECK(ncclTopoCheckGdr(comm->topo, myInfo->busId, req.netDev, 1, &req.useGdr)); + int64_t netId; + NCCLCHECK(ncclTopoGetNetDev(comm, myInfo->rank, graph, channelId, -1, &netId, &req.netDev, &proxyRank)); + NCCLCHECK(ncclTopoCheckGdr(comm->topo, myInfo->busId, netId, 1, &req.useGdr)); send->conn.flags |= req.useGdr ? NCCL_DIRECT_NIC : 0; send->proxyConn.tpLocalRank = comm->topParentLocalRanks[comm->localRank]; @@ -171,8 +173,9 @@ static ncclResult_t recvSetup(struct ncclComm* comm, struct ncclTopoGraph* graph struct setupReq req = { 0 }; int proxyRank, tpProxyRank; - NCCLCHECK(ncclTopoGetNetDev(comm, myInfo->rank, graph, channelId, -1, &req.netDev, &proxyRank)); - NCCLCHECK(ncclTopoCheckGdr(comm->topo, myInfo->busId, req.netDev, 0, &req.useGdr)); + int64_t netId; + NCCLCHECK(ncclTopoGetNetDev(comm, myInfo->rank, graph, channelId, -1, &netId, &req.netDev, &proxyRank)); + NCCLCHECK(ncclTopoCheckGdr(comm->topo, myInfo->busId, netId, 0, &req.useGdr)); recv->conn.flags |= req.useGdr ? NCCL_DIRECT_NIC : 0; // Determine whether we need to flush the GDR buffer on recv or not if (req.useGdr) NCCLCHECK(ncclTopoNeedFlush(comm->topo, myInfo->busId, &req.needFlush)); @@ -696,8 +699,10 @@ static ncclResult_t sendProxyProgress(struct ncclProxyState* proxyState, struct if (sub->posted < sub->nsteps && sub->posted < sub->done + NCCL_STEPS) { int buffSlot = (sub->base+sub->posted)%NCCL_STEPS; - resources->recvMem->connFifo[buffSlot].offset = calcRegionOffset(args, 0, s, sub->posted, 0); - __sync_synchronize(); + if (sub->reg == 0) { + resources->recvMem->connFifo[buffSlot].offset = calcRegionOffset(args, 0, s, sub->posted, 0); + __sync_synchronize(); + } volatile uint64_t* sendHead = resources->gdcSync ? resources->gdcSync : &resources->sendMem->head; TRACE(NCCL_NET, "sendProxy [%ld/%d/%d] posted offset %d @ %p signal %ld->%ld", long(sub->posted), group, buffSlot, resources->recvMem->connFifo[buffSlot].offset, &resources->recvMem->connFifo[buffSlot].offset, long(*sendHead), long(sub->base + sub->posted + args->sliceSteps - NCCL_STEPS)); sub->posted += args->sliceSteps; @@ -708,8 +713,8 @@ static ncclResult_t sendProxyProgress(struct ncclProxyState* proxyState, struct int buffSlot = (sub->base+sub->received)%NCCL_STEPS; volatile struct ncclConnFifo* connFifo = (volatile struct ncclConnFifo*)resources->recvMem->connFifo; volatile uint64_t* recvTail = &resources->recvMem->tail; - if (connFifo[buffSlot].size != -1 && ((*recvTail > (sub->base+sub->received)))) { - if (args->coll != ncclFuncAllReduce) { + if ((connFifo[buffSlot].size != -1 || sub->reg) && ((*recvTail > (sub->base+sub->received)))) { + if (args->coll != ncclFuncAllReduce && sub->reg == 0) { int sendBeg = calcRegionOffset(args, 0, s, sub->received, 0); int sendEnd = calcRegionOffset(args, 0, s, sub->received, 1); if (sendEnd-sendBeg != connFifo[buffSlot].size) { @@ -740,33 +745,89 @@ static ncclResult_t sendProxyProgress(struct ncclProxyState* proxyState, struct reqFifo[group][buffSlot].size = recvEnd - recvBeg; size_t eltSize = ncclTypeSize((ncclDataType_t)args->dtype); - if (sendBeg==sendEnd && recvBeg==recvEnd) { + if (sendBeg==sendEnd && recvBeg==recvEnd && sub->reg == 0) { sub->requests[buffSlot] = nullptr; // trivally finished request } else { if (args->coll == ncclFuncAllReduce) { - int count = (sendEnd-sendBeg)/eltSize; - NCCLCHECK(proxyState->ncclCollNet->iallreduce(resources->collNetComm, region+sendBeg, region+recvBeg, count, (ncclDataType_t)args->dtype, (ncclRedOp_t)args->redOp, sendMhandle, recvMhandle, sub->requests+buffSlot)); + if (sub->reg) { + size_t nBytes = std::min(sub->nbytes, NCCL_MAX_COLLNET_SIZE); + int count = (int)(nBytes / eltSize); + NCCLCHECK(proxyState->ncclCollNet->iallreduce(resources->collNetComm, sub->sendbuff, sub->recvbuff, count, (ncclDataType_t)args->dtype, (ncclRedOp_t)args->redOp, sub->sendMhandle, sub->recvMhandle, sub->requests + buffSlot)); + if (sub->requests[buffSlot]) { + sub->nbytes -= nBytes; + sub->sendbuff += nBytes; + sub->recvbuff += nBytes; + } + } else { + int count = (sendEnd - sendBeg) / eltSize; + NCCLCHECK(proxyState->ncclCollNet->iallreduce(resources->collNetComm, region + sendBeg, region + recvBeg, count, (ncclDataType_t)args->dtype, (ncclRedOp_t)args->redOp, sendMhandle, recvMhandle, sub->requests + buffSlot)); + } } else { sizePerRank = args->specifics.collnetDirect.sizePerRank; if (args->coll == ncclFuncAllGather) { ncclNetSGE_v8_t recvParts; - recvParts.mhandle = recvMhandle; - recvParts.address = region + recvBeg; - recvParts.size = allEnd - allBeg; - NCCLCHECK(proxyState->ncclCollNet->iallgather( - resources->collNetComm, region+sendBeg, 1, &recvParts, - sizePerRank, allBeg, allEnd-allBeg, - sendMhandle, sub->requests+buffSlot)); + if (sub->reg) { + size_t nBytes = std::min(sub->nbytes, NCCL_MAX_COLLNET_SIZE); + void *sendbuff; + recvParts.mhandle = sub->recvMhandle; + recvParts.address = sub->recvbuff; + recvParts.size = nBytes; + if (sub->offset / sizePerRank == args->specifics.collnetDirect.node) { + sendbuff = sub->sendbuff + sub->offset % sizePerRank; + } else { + sendbuff = sub->sendbuff; + } + NCCLCHECK(proxyState->ncclCollNet->iallgather( + resources->collNetComm, sendbuff, 1, &recvParts, + sizePerRank, sub->offset, nBytes, + sub->sendMhandle, sub->requests + buffSlot)); + if (sub->requests[buffSlot]) { + sub->recvbuff += nBytes; + sub->nbytes -= nBytes; + sub->offset += nBytes; + } + } else { + recvParts.mhandle = recvMhandle; + recvParts.address = region + recvBeg; + recvParts.size = allEnd - allBeg; + NCCLCHECK(proxyState->ncclCollNet->iallgather( + resources->collNetComm, region + sendBeg, 1, &recvParts, + sizePerRank, allBeg, allEnd - allBeg, + sendMhandle, sub->requests + buffSlot)); + } } else { ncclNetSGE_v8_t sendParts; - sendParts.mhandle = sendMhandle; - sendParts.address = region + sendBeg; - sendParts.size = allEnd - allBeg; - NCCLCHECK(proxyState->ncclCollNet->ireducescatter( - resources->collNetComm, 1, &sendParts, region+recvBeg, - sizePerRank, allBeg, allEnd-allBeg, - (ncclDataType_t)args->dtype, (ncclRedOp_t)args->redOp, - recvMhandle, sub->requests+buffSlot)); + if (sub->reg) { + size_t nBytes = std::min(sub->nbytes, NCCL_MAX_COLLNET_SIZE); + void *recvbuff; + sendParts.mhandle = sub->sendMhandle; + sendParts.address = sub->sendbuff; + sendParts.size = nBytes; + if (sub->offset / sizePerRank == args->specifics.collnetDirect.node) { + recvbuff = sub->recvbuff + sub->offset % sizePerRank; + } else { + recvbuff = sub->recvbuff; + } + NCCLCHECK(proxyState->ncclCollNet->ireducescatter( + resources->collNetComm, 1, &sendParts, recvbuff, + sizePerRank, sub->offset, nBytes, + (ncclDataType_t)args->dtype, (ncclRedOp_t)args->redOp, + sub->recvMhandle, sub->requests + buffSlot)); + if (sub->requests[buffSlot]) { + sub->sendbuff += nBytes; + sub->nbytes -= nBytes; + sub->offset += nBytes; + } + } else { + sendParts.mhandle = sendMhandle; + sendParts.address = region + sendBeg; + sendParts.size = allEnd - allBeg; + NCCLCHECK(proxyState->ncclCollNet->ireducescatter( + resources->collNetComm, 1, &sendParts, region + recvBeg, + sizePerRank, allBeg, allEnd - allBeg, + (ncclDataType_t)args->dtype, (ncclRedOp_t)args->redOp, + recvMhandle, sub->requests + buffSlot)); + } } } if (sub->requests[buffSlot] == nullptr) continue; @@ -854,7 +915,7 @@ static ncclResult_t recvProxyProgress(struct ncclProxyState* proxyState, struct int totalSize = recvEnd - recvBeg; TRACE(NCCL_NET, "recvProxy [%ld/%d/%d] received, size %d chunkSize=%d", (long)sub->received, group, buffSlot, totalSize, args->chunkSize); sub->received += args->sliceSteps; - if (reqFifo[group][buffSlot].size > 0 && resources->useGdr && resources->needFlush) { + if ((reqFifo[group][buffSlot].size > 0 || sub->reg) && resources->useGdr && resources->needFlush) { // GDRCOPY support if (resources->gdcFlush) { #if defined (__x86_64__) @@ -865,7 +926,37 @@ static ncclResult_t recvProxyProgress(struct ncclProxyState* proxyState, struct return ncclInternalError; #endif } else { - NCCLCHECK(proxyState->ncclCollNet->iflush(resources->collNetComm, region+recvBeg, totalSize, mhandle, sub->requests+buffSlot)); + if (sub->reg) { + size_t nBytes = std::min(sub->nbytes, NCCL_MAX_COLLNET_SIZE); + size_t offset = 0; + if (args->coll == ncclFuncReduceScatter) { + size_t sizePerRank = args->specifics.collnetDirect.sizePerRank; + int node = args->specifics.collnetDirect.node; + int startNode = sub->offset / sizePerRank; + int lastNode = (sub->offset + nBytes) / sizePerRank; + if (startNode == node) { + offset = sub->offset % sizePerRank; + nBytes = std::min(sizePerRank - offset, nBytes); + } else if (startNode < node && node < lastNode) { + nBytes = sizePerRank; + } else if (node == lastNode) { + nBytes = (sub->offset + nBytes) % sizePerRank; + } else { + // no need to flush + nBytes = 0; + } + } + NCCLCHECK(proxyState->ncclCollNet->iflush(resources->collNetComm, sub->recvbuff + offset, nBytes, sub->recvMhandle, sub->requests+buffSlot)); + if (sub->requests[buffSlot]) { + sub->nbytes -= nBytes; + sub->offset += nBytes; + if (args->coll == ncclFuncAllGather || args->coll == ncclFuncAllReduce) { + sub->recvbuff += nBytes; + } + } + } else { + NCCLCHECK(proxyState->ncclCollNet->iflush(resources->collNetComm, region+recvBeg, totalSize, mhandle, sub->requests+buffSlot)); + } } } args->idle = 0; @@ -886,10 +977,12 @@ static ncclResult_t recvProxyProgress(struct ncclProxyState* proxyState, struct } } if (sub->transmitted < sub->flushed) { - int buffSlot = (sub->base + sub->transmitted)%NCCL_STEPS; - volatile struct ncclConnFifo* connFifo = (volatile struct ncclConnFifo*)resources->recvMem->connFifo; - connFifo[buffSlot].offset = calcRegionOffset(args, 1, s, sub->transmitted, 0); - __sync_synchronize(); + if (sub->reg == 0) { + int buffSlot = (sub->base + sub->transmitted)%NCCL_STEPS; + volatile struct ncclConnFifo* connFifo = (volatile struct ncclConnFifo*)resources->recvMem->connFifo; + connFifo[buffSlot].offset = calcRegionOffset(args, 1, s, sub->transmitted, 0); + __sync_synchronize(); + } volatile uint64_t* recvTail = resources->gdcSync ? resources->gdcSync : &resources->recvMem->tail; *recvTail = sub->base + sub->flushed; if (resources->gdcSync) wc_store_fence(); // Flush out WC write @@ -916,9 +1009,134 @@ static ncclResult_t recvProxyProgress(struct ncclProxyState* proxyState, struct return ncclSuccess; } +struct collnetRegInfo { + uintptr_t buffer; + size_t size; +}; + +ncclResult_t ncclCollnetLocalRegisterBuffer(struct ncclComm* comm, const void* userbuff, size_t buffSize, int type, int* outRegBufFlag, void** outHandle) { + ncclResult_t ret = ncclSuccess; + struct ncclReg *regRecord = NULL; + + *outRegBufFlag = 0; + *outHandle = NULL; + if (comm && userbuff && buffSize > 0) { + NCCLCHECKGOTO(ncclRegFind(comm, userbuff, buffSize, ®Record), ret, fail); + if (regRecord) { + if (regRecord->state & COLLNET_REG_COMPLETE) { + // reuse previous registration + *outRegBufFlag = 2; + *outHandle = regRecord->collnetHandle; + goto exit; + } else { + /* start register collnet buffer */ + struct collnetRegInfo info = {regRecord->addr, regRecord->pages * comm->regCache.pageSize}; + void* handle = NULL; + struct ncclProxyConnector* proxyconn = (type == collNetRecv) ? &comm->channels[0].peers[comm->nRanks]->recv[type].proxyConn : &comm->channels[0].peers[comm->nRanks]->send[type].proxyConn; + NCCLCHECKGOTO(ncclProxyCallBlocking(comm, proxyconn, ncclProxyMsgRegister, &info, sizeof(struct collnetRegInfo), &handle, sizeof(void*)), ret, fail); + if (handle) { + regRecord->state |= COLLNET_REG_COMPLETE; + regRecord->proxyconn = proxyconn; + *outHandle = regRecord->collnetHandle = handle; + *outRegBufFlag = 1; + } + } + } + } + +exit: + return ret; +fail: + *outRegBufFlag = 0; + *outHandle = NULL; + goto exit; +} + +ncclResult_t ncclCollnetGraphRegisterBuffer(struct ncclComm* comm, struct ncclKernelPlan *plan, const void* userbuff, size_t buffSize, int type, int* outRegBufFlag, void** outHandle) { + ncclResult_t ret = ncclSuccess; + void* handle = NULL; + struct ncclRegCache* cache = &comm->regCache; + uintptr_t pageSize = cache->pageSize; + uintptr_t addr = (uintptr_t)userbuff & -pageSize; + size_t size = DIVUP((uintptr_t)userbuff - addr + buffSize, pageSize) * pageSize; + collnetRegInfo info = {addr, size}; + struct ncclCollnetHandleList* record = NULL; + struct ncclProxyConnector* proxyConn = (type == collNetRecv) ? &comm->channels[0].peers[comm->nRanks]->recv[type].proxyConn : &comm->channels[0].peers[comm->nRanks]->send[type].proxyConn; + + *outRegBufFlag = 0; + NCCLCHECKGOTO(ncclProxyCallBlocking(comm, proxyConn, ncclProxyMsgRegister, &info, sizeof(struct collnetRegInfo), &handle, sizeof(void*)), ret, fail); + record = ncclMemoryPoolAlloc(&comm->memPool_ncclCollnetHandleList, &comm->memPermanent); + record->proxyconn = proxyConn; + record->buffer = userbuff; + record->size = buffSize; + *outHandle = record->collnetHandle = handle; + *outRegBufFlag = 1; + ncclIntruQueueEnqueue(&plan->collnetHandleQueue, record); + +exit: + return ret; +fail: + *outRegBufFlag = 0; + *outHandle = NULL; + goto exit; +} + +ncclResult_t ncclCollnetDeregBuffer(struct ncclComm* comm, struct ncclProxyConnector* proxyconn, void* handle) { + NCCLCHECK(ncclProxyCallBlocking(comm, proxyconn, ncclProxyMsgDeregister, &handle, sizeof(void*), NULL, 0)); + return ncclSuccess; +} + +static ncclResult_t sendProxyRegBuffer(struct ncclProxyConnection* connection, struct ncclProxyState* proxyState, void* reqBuff, int reqSize, void* respBuff, int respSize, int* done) { + void* handle; + struct collnetRegInfo* info = (struct collnetRegInfo*)reqBuff; + struct sendResources* resources = (struct sendResources*)(connection->transportResources); + + assert(reqSize == sizeof(struct collnetRegInfo)); + assert(respSize == sizeof(void*)); + if (proxyState->ncclCollNet->regMr(resources->collNetComm, (void*)info->buffer, info->size, NCCL_PTR_CUDA, &handle) != ncclSuccess) handle = NULL; + memcpy(respBuff, (void*)&handle, sizeof(void*)); + *done = 1; + return ncclSuccess; +} + +static ncclResult_t recvProxyRegBuffer(struct ncclProxyConnection* connection, struct ncclProxyState* proxyState, void* reqBuff, int reqSize, void* respBuff, int respSize, int* done) { + void* handle; + struct collnetRegInfo* info = (struct collnetRegInfo*)reqBuff; + struct recvResources* resources = (struct recvResources*)(connection->transportResources); + + assert(reqSize == sizeof(struct collnetRegInfo)); + assert(respSize == sizeof(void*)); + if (proxyState->ncclCollNet->regMr(resources->collNetComm, (void*)info->buffer, info->size, NCCL_PTR_CUDA, &handle) != ncclSuccess) handle = NULL; + memcpy(respBuff, (void*)&handle, sizeof(void*)); + *done = 1; + return ncclSuccess; +} + +static ncclResult_t sendProxyDeregBuffer(struct ncclProxyConnection* connection, struct ncclProxyState* proxyState, void* reqBuff, int reqSize, int* done) { + void* handle; + struct sendResources* resources = (struct sendResources*)(connection->transportResources); + + assert(reqSize == sizeof(void*)); + memcpy(&handle, reqBuff, sizeof(void*)); + NCCLCHECK(proxyState->ncclCollNet->deregMr(resources->collNetComm, handle)); + *done = 1; + return ncclSuccess; +} + +static ncclResult_t recvProxyDeregBuffer(struct ncclProxyConnection* connection, struct ncclProxyState* proxyState, void* reqBuff, int reqSize, int* done) { + void* handle; + struct recvResources* resources = (struct recvResources*)(connection->transportResources); + + assert(reqSize == sizeof(void*)); + memcpy(&handle, reqBuff, sizeof(void*)); + NCCLCHECK(proxyState->ncclCollNet->deregMr(resources->collNetComm, handle)); + *done = 1; + return ncclSuccess; +} + struct ncclTransport collNetTransport = { "COL", canConnect, - { sendSetup, sendConnect, sendFree, NULL, sendProxySetup, sendProxyConnect, sendProxyFree, sendProxyProgress }, - { recvSetup, recvConnect, recvFree, NULL, recvProxySetup, recvProxyConnect, recvProxyFree, recvProxyProgress } + { sendSetup, sendConnect, sendFree, NULL, sendProxySetup, sendProxyConnect, sendProxyFree, sendProxyProgress, sendProxyRegBuffer, sendProxyDeregBuffer }, + { recvSetup, recvConnect, recvFree, NULL, recvProxySetup, recvProxyConnect, recvProxyFree, recvProxyProgress, recvProxyRegBuffer, recvProxyDeregBuffer } }; diff --git a/src/transport/net.cc b/src/transport/net.cc index 58cb92144..cc388211c 100644 --- a/src/transport/net.cc +++ b/src/transport/net.cc @@ -179,8 +179,9 @@ static ncclResult_t sendSetup(struct ncclComm* comm, struct ncclTopoGraph* graph req.connIndex = connIndex; int proxyRank; - NCCLCHECK(ncclTopoGetNetDev(comm, myInfo->rank, graph, channelId, peerInfo->rank, &req.netDev, &proxyRank)); - NCCLCHECK(ncclTopoCheckGdr(comm->topo, myInfo->busId, req.netDev, 1, &req.useGdr)); + int64_t netId; + NCCLCHECK(ncclTopoGetNetDev(comm, myInfo->rank, graph, channelId, peerInfo->rank, &netId, &req.netDev, &proxyRank)); + NCCLCHECK(ncclTopoCheckGdr(comm->topo, myInfo->busId, netId, 1, &req.useGdr)); send->conn.flags |= req.useGdr ? NCCL_DIRECT_NIC : 0; tpProxyRank = comm->topParentRanks[proxyRank]; @@ -216,8 +217,9 @@ static ncclResult_t recvSetup(struct ncclComm* comm, struct ncclTopoGraph* graph // Use myInfo->rank as the receiver uses its own NIC int proxyRank, tpProxyRank; - NCCLCHECK(ncclTopoGetNetDev(comm, myInfo->rank, graph, channelId, myInfo->rank, &req.netDev, &proxyRank)); - NCCLCHECK(ncclTopoCheckGdr(comm->topo, myInfo->busId, req.netDev, 0, &req.useGdr)); + int64_t netId; + NCCLCHECK(ncclTopoGetNetDev(comm, myInfo->rank, graph, channelId, myInfo->rank, &netId, &req.netDev, &proxyRank)); + NCCLCHECK(ncclTopoCheckGdr(comm->topo, myInfo->busId, netId, 0, &req.useGdr)); // Determine whether we need to flush the GDR buffer on recv or not if (req.useGdr) NCCLCHECK(ncclTopoNeedFlush(comm->topo, myInfo->busId, &req.needFlush)); @@ -347,6 +349,7 @@ static ncclResult_t sendConnect(struct ncclComm* comm, struct ncclConnect* conne struct ncclRecvMem *recvMem = (struct ncclRecvMem*) NCCL_NET_MAP_GET_POINTER(map, gpu, recvMem); send->conn.tail = &recvMem->tail; + send->conn.stepSize = comm->buffSizes[NCCL_PROTO_SIMPLE]/NCCL_STEPS; send->conn.connFifo = recvMem->connFifo; // Only fuse P2P buffers, continue to allocate dedicated buffers for ring/tree for (int i=0; imems[NCCL_NET_MAP_GDCMEM].gpuPtr; recv->conn.tail = gdcMem ? (uint64_t*)gdcMem : &recvMem->tail; + recv->conn.stepSize = comm->buffSizes[NCCL_PROTO_SIMPLE]/NCCL_STEPS; recv->conn.connFifo = recvMem->connFifo; // Only fuse P2P buffers, continue to allocate dedicated buffers for ring/tree for (int i=0; iposted = sub->transmitted = sub->done = 0; for (uint64_t step=0; stepnsteps; step++) ncclProfilingRecord(args, s, step, ncclProxyProfileBegin); if (sub->reg && sub->nbytes > 0) { - NCCLCHECK(proxyState->ncclNet->regMr(resources->netSendComm, sub->buffer, sub->nbytes, NCCL_PTR_CUDA, &sub->mhandle)); + NCCLCHECK(proxyState->ncclNet->regMr(resources->netSendComm, sub->recvbuff, sub->nbytes, NCCL_PTR_CUDA, &sub->mhandle)); } else { sub->mhandle = resources->mhandles[args->protocol]; } @@ -1110,7 +1114,7 @@ static ncclResult_t sendProxyProgress(struct ncclProxyState* proxyState, struct if (f1[0] != flag || f2[0] != flag) { ready = 0; break; } } } else if (p == NCCL_PROTO_SIMPLE && resources->shared) { - buff = sub->reg ? (char*)sub->buffer : localBuff+resources->recvMem->connFifo[buffSlot].offset; + buff = sub->reg ? (char*)sub->recvbuff : localBuff+resources->recvMem->connFifo[buffSlot].offset; } if (ready) { // Data is ready, try to send. @@ -1134,7 +1138,7 @@ static ncclResult_t sendProxyProgress(struct ncclProxyState* proxyState, struct if (done) { if (sub->reg) { if (size < sub->nbytes) { - sub->buffer = ((char*)sub->buffer)+size; + sub->recvbuff += size; sub->nbytes -= size; // Do one more step (at least) sub->nsteps++; @@ -1215,7 +1219,7 @@ static ncclResult_t recvProxyProgress(struct ncclProxyState* proxyState, struct for (uint64_t step=0; stepnsteps; step++) ncclProfilingRecord(args, s, step, ncclProxyProfileBegin); if (sub->reg && sub->nbytes > 0) { // Register buffer - NCCLCHECK(proxyState->ncclNet->regMr(resources->netRecvComm, sub->buffer, sub->nbytes, NCCL_PTR_CUDA, &sub->mhandle)); + NCCLCHECK(proxyState->ncclNet->regMr(resources->netRecvComm, sub->recvbuff, sub->nbytes, NCCL_PTR_CUDA, &sub->mhandle)); } else { sub->mhandle = resources->mhandles[args->protocol]; } @@ -1247,7 +1251,7 @@ static ncclResult_t recvProxyProgress(struct ncclProxyState* proxyState, struct if (sub->reg) { // Wait until CUDA kernel has started before we access the user buffer directly. if (connFifo[sub->base%NCCL_STEPS].size == -1) continue; - ptrs[subCount] = sub->buffer; + ptrs[subCount] = sub->recvbuff; sizes[subCount] = std::min(MAX_NET_SIZE, sub->nbytes); } else { int sharedBuffSlot = sub->posted%maxDepth; @@ -1307,7 +1311,7 @@ static ncclResult_t recvProxyProgress(struct ncclProxyState* proxyState, struct int size = sizes[subIndex++]; if (sub->reg) { if (size < sub->nbytes) { - sub->buffer = ((char*)sub->buffer) + size; + sub->recvbuff += size; sub->nbytes -= size; // Do one more step (at least) sub->nsteps++; @@ -1349,7 +1353,7 @@ static ncclResult_t recvProxyProgress(struct ncclProxyState* proxyState, struct char* localBuff = NCCL_NET_MAP_GET_POINTER(&resources->map, cpu, buffs[p]); int buffSlot = (sub->base+sub->received-args->sliceSteps)%NCCL_STEPS; ptrs[subCount] = resources->shared ? - (sub->reg ? sub->buffer : localBuff+resources->recvMem->connFifo[buffSlot].offset) : + (sub->reg ? (char*)sub->recvbuff : localBuff+resources->recvMem->connFifo[buffSlot].offset) : localBuff+buffSlot*stepSize; mhandles[subCount] = sub->mhandle; subCount++; @@ -1439,6 +1443,6 @@ static ncclResult_t recvProxyProgress(struct ncclProxyState* proxyState, struct struct ncclTransport netTransport = { "NET", canConnect, - { sendSetup, sendConnect, sendFree, proxySharedInit, sendProxySetup, sendProxyConnect, sendProxyFree, sendProxyProgress }, - { recvSetup, recvConnect, recvFree, proxySharedInit, recvProxySetup, recvProxyConnect, recvProxyFree, recvProxyProgress } + { sendSetup, sendConnect, sendFree, proxySharedInit, sendProxySetup, sendProxyConnect, sendProxyFree, sendProxyProgress, NULL }, + { recvSetup, recvConnect, recvFree, proxySharedInit, recvProxySetup, recvProxyConnect, recvProxyFree, recvProxyProgress, NULL } }; diff --git a/src/transport/net_ib.cc b/src/transport/net_ib.cc index 97dc60c49..551ca61fd 100644 --- a/src/transport/net_ib.cc +++ b/src/transport/net_ib.cc @@ -77,7 +77,8 @@ struct ncclIbDev ncclIbDevs[MAX_IB_DEVS]; pthread_mutex_t ncclIbLock = PTHREAD_MUTEX_INITIALIZER; static int ncclIbRelaxedOrderingEnabled = 0; -NCCL_PARAM(IbGidIndex, "IB_GID_INDEX", 0); +NCCL_PARAM(IbGidIndex, "IB_GID_INDEX", -1); +NCCL_PARAM(IbRoceVersionNum, "IB_ROCE_VERSION_NUM", 2); NCCL_PARAM(IbTimeout, "IB_TIMEOUT", 18); NCCL_PARAM(IbRetryCnt, "IB_RETRY_CNT", 7); NCCL_PARAM(IbPkey, "IB_PKEY", 0); @@ -103,6 +104,210 @@ static void* ncclIbAsyncThreadMain(void* args) { return NULL; } +static sa_family_t envIbAddrFamily(void) { + sa_family_t family = AF_INET; + const char* env = ncclGetEnv("NCCL_IB_ADDR_FAMILY"); + if (env == NULL || strlen(env) == 0) { + return family; + } + + INFO(NCCL_ENV, "NCCL_IB_ADDR_FAMILY set by environment to %s", env); + + if (strcmp(env, "AF_INET") == 0) { + family = AF_INET; + } else if (strcmp(env, "AF_INET6") == 0) { + family = AF_INET6; + } + + return family; +} + +static void* envIbAddrRange(sa_family_t af, int* mask) { + *mask = 0; + static struct in_addr addr; + static struct in6_addr addr6; + void *ret = (af == AF_INET) ? (void *)&addr : (void *)&addr6; + + const char* env = ncclGetEnv("NCCL_IB_ADDR_RANGE"); + if (NULL == env || strlen(env) == 0) { + return NULL; + } + + INFO(NCCL_ENV, "NCCL_IB_ADDR_RANGE set by environment to %s", env); + + char addrString[128] = { 0 }; + snprintf(addrString, 128, "%s", env); + char *addrStrPtr = addrString; + char *maskStrPtr = strstr(addrString, "/") + 1; + if (NULL == maskStrPtr) { + return NULL; + } + *(maskStrPtr - 1) = '\0'; + + if (inet_pton(af, addrStrPtr, ret) == 0) { + WARN("NET/IB: Ip address '%s' is invalid for family %s, ignoring address", addrStrPtr, (af == AF_INET) ? "AF_INET" : "AF_INET6"); + return NULL; + } + + *mask = (int)strtol(maskStrPtr, NULL, 10); + if (af == AF_INET && *mask > 32) { + WARN("NET/IB: Ip address mask '%d' is invalid for family %s, ignoring mask", *mask, (af == AF_INET) ? "AF_INET" : "AF_INET6"); + *mask = 0; + ret = NULL; + } else if (af == AF_INET6 && *mask > 128) { + WARN("NET/IB: Ip address mask '%d' is invalid for family %s, ignoring mask", *mask, (af == AF_INET) ? "AF_INET" : "AF_INET6"); + *mask = 0; + ret = NULL; + } + + return ret; +} + +static sa_family_t getGidAddrFamily(union ibv_gid* gid) { + const struct in6_addr *a = (struct in6_addr *)gid->raw; + bool isIpV4Mapped = ((a->s6_addr32[0] | a->s6_addr32[1]) | (a->s6_addr32[2] ^ htonl(0x0000ffff))) == 0UL; + bool isIpV4MappedMulticast = (a->s6_addr32[0] == htonl(0xff0e0000) && ((a->s6_addr32[1] | (a->s6_addr32[2] ^ htonl(0x0000ffff))) == 0UL)); + return (isIpV4Mapped || isIpV4MappedMulticast) ? AF_INET : AF_INET6; +} + +static bool matchGidAddrPrefix(sa_family_t af, void* prefix, int prefixlen, union ibv_gid* gid) { + struct in_addr *base = NULL; + struct in6_addr *base6 = NULL; + struct in6_addr *addr6 = NULL;; + if (af == AF_INET) { + base = (struct in_addr *)prefix; + } else { + base6 = (struct in6_addr *)prefix; + } + addr6 = (struct in6_addr *)gid->raw; + +#define NETMASK(bits) (htonl(0xffffffff ^ ((1 << (32 - bits)) - 1))) + + int i = 0; + while (prefixlen > 0 && i < 4) { + if (af == AF_INET) { + int mask = NETMASK(prefixlen); + if ((base->s_addr & mask) ^ (addr6->s6_addr32[3] & mask)) { + break; + } + prefixlen = 0; + break; + } else { + if (prefixlen >= 32) { + if (base6->s6_addr32[i] ^ addr6->s6_addr32[i]) { + break; + } + prefixlen -= 32; + ++i; + } else { + int mask = NETMASK(prefixlen); + if ((base6->s6_addr32[i] & mask) ^ (addr6->s6_addr32[i] & mask)) { + break; + } + prefixlen = 0; + } + } + } + + return (prefixlen == 0) ? true : false; +} + +static bool configuredGid(union ibv_gid* gid) { + const struct in6_addr *a = (struct in6_addr *)gid->raw; + int trailer = (a->s6_addr32[1] | a->s6_addr32[2] | a->s6_addr32[3]); + if (((a->s6_addr32[0] | trailer) == 0UL) || ((a->s6_addr32[0] == htonl(0xfe800000)) && (trailer == 0UL))) { + return false; + } + return true; +} + +static bool linkLocalGid(union ibv_gid* gid) { + const struct in6_addr *a = (struct in6_addr *)gid->raw; + if (a->s6_addr32[0] == htonl(0xfe800000) && a->s6_addr32[1] == 0UL) { + return true; + } + return false; +} + +static bool validGid(union ibv_gid* gid) { + return (configuredGid(gid) && !linkLocalGid(gid)); +} + +static ncclResult_t ncclIbRoceGetVersionNum(const char* deviceName, int portNum, int gidIndex, int* version) { + char gidRoceVerStr[16] = { 0 }; + char roceTypePath[PATH_MAX] = { 0 }; + sprintf(roceTypePath, "/sys/class/infiniband/%s/ports/%d/gid_attrs/types/%d", deviceName, portNum, gidIndex); + + int fd = open(roceTypePath, O_RDONLY); + if (fd == -1) { + return ncclSystemError; + } + int ret = read(fd, gidRoceVerStr, 15); + close(fd); + + if (ret == -1) { + return ncclSystemError; + } + + if (strlen(gidRoceVerStr)) { + if (strncmp(gidRoceVerStr, "IB/RoCE v1", strlen("IB/RoCE v1")) == 0 || strncmp(gidRoceVerStr, "RoCE v1", strlen("RoCE v1")) == 0) { + *version = 1; + } else if (strncmp(gidRoceVerStr, "RoCE v2", strlen("RoCE v2")) == 0) { + *version = 2; + } + } + + return ncclSuccess; +} + +static ncclResult_t ncclUpdateGidIndex(struct ibv_context* context, uint8_t portNum, sa_family_t af, void* prefix, int prefixlen, int roceVer, int gidIndexCandidate, int* gidIndex) { + union ibv_gid gid, gidCandidate; + NCCLCHECK(wrap_ibv_query_gid(context, portNum, *gidIndex, &gid)); + NCCLCHECK(wrap_ibv_query_gid(context, portNum, gidIndexCandidate, &gidCandidate)); + + sa_family_t usrFam = af; + sa_family_t gidFam = getGidAddrFamily(&gid); + sa_family_t gidCandidateFam = getGidAddrFamily(&gidCandidate); + bool gidCandidateMatchSubnet = matchGidAddrPrefix(usrFam, prefix, prefixlen, &gidCandidate); + + if (gidCandidateFam != gidFam && gidCandidateFam == usrFam && gidCandidateMatchSubnet) { + *gidIndex = gidIndexCandidate; + } else { + if (gidCandidateFam != usrFam || !validGid(&gidCandidate) || !gidCandidateMatchSubnet) { + return ncclSuccess; + } + int usrRoceVer = roceVer; + int gidRoceVerNum, gidRoceVerNumCandidate; + const char* deviceName = wrap_ibv_get_device_name(context->device); + NCCLCHECK(ncclIbRoceGetVersionNum(deviceName, portNum, *gidIndex, &gidRoceVerNum)); + NCCLCHECK(ncclIbRoceGetVersionNum(deviceName, portNum, gidIndexCandidate, &gidRoceVerNumCandidate)); + if ((gidRoceVerNum != gidRoceVerNumCandidate || !validGid(&gid)) && gidRoceVerNumCandidate == usrRoceVer) { + *gidIndex = gidIndexCandidate; + } + } + + return ncclSuccess; +} + +static ncclResult_t ncclIbGetGidIndex(struct ibv_context *context, uint8_t portNum, int gidTblLen, int *gidIndex) { + *gidIndex = ncclParamIbGidIndex(); + if (*gidIndex >= 0) { + return ncclSuccess; + } + + sa_family_t userAddrFamily = envIbAddrFamily(); + int userRoceVersion = ncclParamIbRoceVersionNum(); + int prefixlen; + void *prefix = envIbAddrRange(userAddrFamily, &prefixlen); + + *gidIndex = 0; + for (int gidIndexNext = 1; gidIndexNext < gidTblLen; ++gidIndexNext) { + NCCLCHECK(ncclUpdateGidIndex(context, portNum, userAddrFamily, prefix, prefixlen, userRoceVersion, gidIndexNext, gidIndex)); + } + + return ncclSuccess; +} + NCCL_PARAM(IbDisable, "IB_DISABLE", 0); NCCL_PARAM(IbMergeVfs, "IB_MERGE_VFS", 1); NCCL_PARAM(IbMergeNics, "IB_MERGE_NICS", 1); @@ -182,6 +387,7 @@ int ncclIbFindMatchingDev(int dev) { } ncclResult_t ncclIbInit(ncclDebugLogger_t logFunction) { + ncclResult_t ret; if (ncclParamIbDisable()) return ncclInternalError; static int shownIbHcaEnv = 0; if(wrap_ibv_symbols() != ncclSuccess) { return ncclInternalError; } @@ -194,7 +400,8 @@ ncclResult_t ncclIbInit(ncclDebugLogger_t logFunction) { ncclNMergedIbDevs = 0; if (ncclFindInterfaces(ncclIbIfName, &ncclIbIfAddr, MAX_IF_NAME_SIZE, 1) != 1) { WARN("NET/IB : No IP interface found."); - return ncclInternalError; + ret = ncclInternalError; + goto fail; } // Detect IB cards @@ -211,7 +418,7 @@ ncclResult_t ncclIbInit(ncclDebugLogger_t logFunction) { if (searchExact) userIbEnv++; int nUserIfs = parseStringList(userIbEnv, userIfs, MAX_IB_DEVS); - if (ncclSuccess != wrap_ibv_get_device_list(&devices, &nIbDevs)) return ncclInternalError; + if (ncclSuccess != wrap_ibv_get_device_list(&devices, &nIbDevs)) { ret = ncclInternalError; goto fail; } for (int d=0; dname); - if (ncclSuccess != wrap_ibv_close_device(context)) { return ncclInternalError; } + if (ncclSuccess != wrap_ibv_close_device(context)) { ret = ncclInternalError; goto fail; } continue; } for (int port_num = 1; port_num <= devAttr.phys_port_cnt; port_num++) { @@ -244,6 +451,7 @@ ncclResult_t ncclIbInit(ncclDebugLogger_t logFunction) { pthread_mutex_init(&ncclIbDevs[ncclNIbDevs].lock, NULL); ncclIbDevs[ncclNIbDevs].device = d; ncclIbDevs[ncclNIbDevs].guid = devAttr.sys_image_guid; + ncclIbDevs[ncclNIbDevs].portAttr = portAttr; ncclIbDevs[ncclNIbDevs].portNum = port_num; ncclIbDevs[ncclNIbDevs].link = portAttr.link_layer; ncclIbDevs[ncclNIbDevs].speed = ncclIbSpeed(portAttr.active_speed) * ncclIbWidth(portAttr.active_width); @@ -295,9 +503,9 @@ ncclResult_t ncclIbInit(ncclDebugLogger_t logFunction) { ncclNIbDevs++; nPorts++; } - if (nPorts == 0 && ncclSuccess != wrap_ibv_close_device(context)) { return ncclInternalError; } + if (nPorts == 0 && ncclSuccess != wrap_ibv_close_device(context)) { ret = ncclInternalError; goto fail; } } - if (nIbDevs && (ncclSuccess != wrap_ibv_free_device_list(devices))) { return ncclInternalError; }; + if (nIbDevs && (ncclSuccess != wrap_ibv_free_device_list(devices))) { ret = ncclInternalError; goto fail; }; } if (ncclNIbDevs == 0) { INFO(NCCL_INIT|NCCL_NET, "NET/IB : No device found."); @@ -333,6 +541,9 @@ ncclResult_t ncclIbInit(ncclDebugLogger_t logFunction) { pthread_mutex_unlock(&ncclIbLock); } return ncclSuccess; +fail: + pthread_mutex_unlock(&ncclIbLock); + return ret; } ncclResult_t ncclIbDevices(int* ndev) { @@ -484,6 +695,7 @@ struct ncclIbHandle { struct ncclIbGidInfo { uint8_t link_layer; union ibv_gid localGid; + int32_t localGidIndex; }; #define NCCL_NET_IB_REQ_UNUSED 0 @@ -516,7 +728,7 @@ struct ncclIbNetCommDevBase { int ibDevN; struct ibv_pd* pd; struct ibv_cq* cq; - uint64_t pad[1]; + uint64_t pad[2]; struct ncclIbGidInfo gidInfo; }; @@ -698,7 +910,7 @@ ncclResult_t ncclIbCreateQp(uint8_t ib_port, struct ncclIbNetCommDevBase* base, return ncclSuccess; } -ncclResult_t ncclIbRtrQp(struct ibv_qp* qp, uint32_t dest_qp_num, struct ncclIbDevInfo* info) { +ncclResult_t ncclIbRtrQp(struct ibv_qp* qp, uint8_t sGidIndex, uint32_t dest_qp_num, struct ncclIbDevInfo* info) { struct ibv_qp_attr qpAttr; memset(&qpAttr, 0, sizeof(struct ibv_qp_attr)); qpAttr.qp_state = IBV_QPS_RTR; @@ -712,7 +924,7 @@ ncclResult_t ncclIbRtrQp(struct ibv_qp* qp, uint32_t dest_qp_num, struct ncclIbD qpAttr.ah_attr.grh.dgid.global.subnet_prefix = info->spn; qpAttr.ah_attr.grh.dgid.global.interface_id = info->iid; qpAttr.ah_attr.grh.flow_label = 0; - qpAttr.ah_attr.grh.sgid_index = ncclParamIbGidIndex(); + qpAttr.ah_attr.grh.sgid_index = sGidIndex; qpAttr.ah_attr.grh.hop_limit = 255; qpAttr.ah_attr.grh.traffic_class = ncclParamIbTc(); } else { @@ -818,9 +1030,6 @@ ncclResult_t ncclIbConnect(int dev, void* opaqueHandle, void** sendComm, ncclNet for (int i = 0; i < comm->base.ndevs; i++) { ncclIbSendCommDev* commDev = comm->devs + i; ncclIbDev* ibDev = ncclIbDevs + commDev->base.ibDevN; - // Send my QP Info to receiver through the socket. Hope this won't block. - // TODO - I thought I queried this in init? - NCCLCHECK(wrap_ibv_query_port(ibDev->context, ibDev->portNum, &ibDev->portAttr)); // Write to the metadata struct via this pointer ncclIbDevInfo* devInfo = meta.devs + i; @@ -835,7 +1044,8 @@ ncclResult_t ncclIbConnect(int dev, void* opaqueHandle, void** sendComm, ncclNet // RoCE support devInfo->link_layer = commDev->base.gidInfo.link_layer = ibDev->portAttr.link_layer; if (devInfo->link_layer == IBV_LINK_LAYER_ETHERNET) { - NCCLCHECK(wrap_ibv_query_gid(ibDev->context, ibDev->portNum, ncclParamIbGidIndex(), &commDev->base.gidInfo.localGid)); + NCCLCHECK(ncclIbGetGidIndex(ibDev->context, ibDev->portNum, ibDev->portAttr.gid_tbl_len, &commDev->base.gidInfo.localGidIndex)); + NCCLCHECK(wrap_ibv_query_gid(ibDev->context, ibDev->portNum, commDev->base.gidInfo.localGidIndex, &commDev->base.gidInfo.localGid)); devInfo->spn = commDev->base.gidInfo.localGid.global.subnet_prefix; devInfo->iid = commDev->base.gidInfo.localGid.global.interface_id; } @@ -854,7 +1064,7 @@ ncclResult_t ncclIbConnect(int dev, void* opaqueHandle, void** sendComm, ncclNet if (comm->base.qps[q].devIndex == i) INFO(NCCL_NET,"NET/IB: %s %d IbDev %d Port %d qpn %d mtu %d query_ece={supported=%d, vendor_id=0x%x, options=0x%x, comp_mask=0x%x} GID %ld (%lX/%lX) fifoRkey=0x%x fifoLkey=0x%x", comm->base.ndevs > 2 ? "NCCL MergedDev" : "NCCL Dev", dev, - commDev->base.ibDevN, ibDev->portNum, meta.qpInfo[q].qpn, devInfo->mtu, meta.qpInfo[q].ece_supported, meta.qpInfo[q].ece.vendor_id, meta.qpInfo[q].ece.options, meta.qpInfo[q].ece.comp_mask, ncclParamIbGidIndex(), + commDev->base.ibDevN, ibDev->portNum, meta.qpInfo[q].qpn, devInfo->mtu, meta.qpInfo[q].ece_supported, meta.qpInfo[q].ece.vendor_id, meta.qpInfo[q].ece.options, meta.qpInfo[q].ece.comp_mask, (int64_t)commDev->base.gidInfo.localGidIndex, devInfo->spn, devInfo->iid, devInfo->fifoRkey, commDev->fifoMr->lkey); } } @@ -923,12 +1133,15 @@ ncclResult_t ncclIbConnect(int dev, void* opaqueHandle, void** sendComm, ncclNet // Assign per-QP remDev comm->base.qps[q].remDevIdx = remQpInfo->devIndex; + int devIndex = comm->base.qps[q].devIndex; + ncclIbSendCommDev* commDev = comm->devs + devIndex; + uint8_t gidIndex = commDev->base.gidInfo.localGidIndex; struct ibv_qp* qp = comm->base.qps[q].qp; if (remQpInfo->ece_supported && remQpInfo->ece_supported) NCCLCHECK(wrap_ibv_set_ece(qp, &remQpInfo->ece, &remQpInfo->ece_supported)); - NCCLCHECK(ncclIbRtrQp(qp, remQpInfo->qpn, remDevInfo)); + NCCLCHECK(ncclIbRtrQp(qp, gidIndex, remQpInfo->qpn, remDevInfo)); NCCLCHECK(ncclIbRtsQp(qp)); } @@ -1024,8 +1237,8 @@ ncclResult_t ncclIbAccept(void* listenComm, void** recvComm, ncclNetDeviceHandle ibDevN = mergedDev->devs[i]; NCCLCHECK(ncclIbInitCommDevBase(ibDevN, &rCommDev->base)); ibDev = ncclIbDevs + ibDevN; - NCCLCHECK(wrap_ibv_query_port(ibDev->context, ibDev->portNum, &ibDev->portAttr)); - NCCLCHECK(wrap_ibv_query_gid(ibDev->context, ibDev->portNum, ncclParamIbGidIndex(), &rCommDev->base.gidInfo.localGid)); + NCCLCHECK(ncclIbGetGidIndex(ibDev->context, ibDev->portNum, ibDev->portAttr.gid_tbl_len, &rCommDev->base.gidInfo.localGidIndex)); + NCCLCHECK(wrap_ibv_query_gid(ibDev->context, ibDev->portNum, rCommDev->base.gidInfo.localGidIndex, &rCommDev->base.gidInfo.localGid)); } // Copy remDevInfo for things like remGidInfo, remFifoAddr, etc. @@ -1064,7 +1277,7 @@ ncclResult_t ncclIbAccept(void* listenComm, void** recvComm, ncclNetDeviceHandle NCCLCHECK(wrap_ibv_query_ece(qp->qp, &meta.qpInfo[q].ece, &meta.qpInfo[q].ece_supported)); } - NCCLCHECK(ncclIbRtrQp(qp->qp, remMeta.qpInfo[q].qpn, remDevInfo)); + NCCLCHECK(ncclIbRtrQp(qp->qp, rCommDev->base.gidInfo.localGidIndex, remMeta.qpInfo[q].qpn, remDevInfo)); NCCLCHECK(ncclIbRtsQp(qp->qp)); } @@ -1097,7 +1310,7 @@ ncclResult_t ncclIbAccept(void* listenComm, void** recvComm, ncclNetDeviceHandle devInfo.spn = rCommDev->base.gidInfo.localGid.global.subnet_prefix; devInfo.iid = rCommDev->base.gidInfo.localGid.global.interface_id; devInfo.mtu = ibDev->portAttr.active_mtu; - NCCLCHECK(ncclIbRtrQp(rCommDev->gpuFlush.qp.qp, rCommDev->gpuFlush.qp.qp->qp_num, &devInfo)); + NCCLCHECK(ncclIbRtrQp(rCommDev->gpuFlush.qp.qp, rCommDev->base.gidInfo.localGidIndex, rCommDev->gpuFlush.qp.qp->qp_num, &devInfo)); NCCLCHECK(ncclIbRtsQp(rCommDev->gpuFlush.qp.qp)); } @@ -1724,7 +1937,7 @@ ncclResult_t ncclIbTest(void* request, int* done, int* sizes) { return ncclInternalError; } if (req->nreqs == 1) { - req->recv.sizes[0] += wc->imm_data; + req->recv.sizes[0] = wc->imm_data; } } req->events[i]--; diff --git a/src/transport/nvls.cc b/src/transport/nvls.cc index a03aab387..0dd7c52ff 100644 --- a/src/transport/nvls.cc +++ b/src/transport/nvls.cc @@ -46,12 +46,12 @@ struct ncclTransport nvlsTransport = { { NULL, NULL, nvlsRecvFree, NULL, NULL, NULL, NULL, NULL } }; -ncclResult_t nvlsGetProperties(struct ncclComm *comm, struct ncclNvlsSharedRes* resources, int dev, int nranks, size_t size) { +ncclResult_t nvlsGetProperties(struct ncclComm *comm, struct ncclNvlsSharedRes* resources, int dev, size_t size) { CUmulticastObjectProp* prop = &resources->properties; memset(prop, 0, sizeof(*prop)); prop->size = size; - prop->numDevices = nranks; - prop->handleTypes = NVLS_CU_MEM_HANDLE_TYPE; + prop->numDevices = comm->MNNVL ? comm->clique.size : comm->localRanks; + prop->handleTypes = ncclCuMemHandleType; prop->flags = 0; // Could be changed to CU_MULTICAST_GRANULARITY_MINIMUM when 3418538 resolved @@ -70,6 +70,7 @@ ncclResult_t nvlsGetProperties(struct ncclComm *comm, struct ncclNvlsSharedRes* } ncclResult_t nvlsGroupCreate(struct ncclComm *comm, CUmulticastObjectProp *prop, int rank, unsigned int nranks, CUmemGenericAllocationHandle *mcHandle, char *shareableHandle) { + CUmemAllocationHandleType type = ncclCuMemHandleType; size_t size = prop->size; // Create a Multicast group @@ -77,9 +78,9 @@ ncclResult_t nvlsGroupCreate(struct ncclComm *comm, CUmulticastObjectProp *prop, INFO(NCCL_NVLS, "NVLS Creating Multicast group nranks %d size %zi on rank %d", nranks, size, rank); CUCHECK(cuMulticastCreate(mcHandle, prop)); - if ((NVLS_CU_MEM_HANDLE_TYPE != CU_MEM_HANDLE_TYPE_NONE) && (NVLS_CU_MEM_HANDLE_TYPE != CU_MEM_HANDLE_TYPE_POSIX_FILE_DESCRIPTOR)) { + if (type == CU_MEM_HANDLE_TYPE_FABRIC) { // Get a handle to pass to other ranks - CUCHECK(cuMemExportToShareableHandle(shareableHandle, *mcHandle, NVLS_CU_MEM_HANDLE_TYPE, 0)); + CUCHECK(cuMemExportToShareableHandle(shareableHandle, *mcHandle, ncclCuMemHandleType, 0)); } else { memcpy(shareableHandle, mcHandle, sizeof(CUmemGenericAllocationHandle)); @@ -97,7 +98,7 @@ ncclResult_t nvlsGroupAddDevice(struct ncclComm *comm, struct ncclNvlsSharedRes* } ncclResult_t nvlsGroupConnect(struct ncclComm *comm, char *shareableHandle, int rank, CUmemGenericAllocationHandle *mcHandle) { - CUmemAllocationHandleType type = NVLS_CU_MEM_HANDLE_TYPE; + CUmemAllocationHandleType type = ncclCuMemHandleType; INFO(NCCL_NVLS, "NVLS importing shareableHandle %p from rank %d", shareableHandle, rank); @@ -113,7 +114,7 @@ ncclResult_t nvlsGroupConnect(struct ncclComm *comm, char *shareableHandle, int CUCHECK(cuMemImportFromShareableHandle(mcHandle, (void *)(uintptr_t)fd, type)); (void) close(fd); } else { - if (NVLS_CU_MEM_HANDLE_TYPE != CU_MEM_HANDLE_TYPE_NONE) { + if (type == CU_MEM_HANDLE_TYPE_FABRIC) { CUCHECK(cuMemImportFromShareableHandle(mcHandle, (void *)shareableHandle, type)); } else { memcpy(mcHandle, shareableHandle, sizeof(CUmemGenericAllocationHandle)); @@ -136,7 +137,7 @@ ncclResult_t nvlsGroupBindMem(struct ncclComm *comm, struct ncclNvlsSharedRes* r prop.type = CU_MEM_ALLOCATION_TYPE_PINNED; prop.location.type = CU_MEM_LOCATION_TYPE_DEVICE; prop.location.id = resources->dev; - prop.requestedHandleTypes = NVLS_CU_MEM_HANDLE_TYPE; + prop.requestedHandleTypes = ncclCuMemHandleType; CUCHECK(cuMemGetAllocationGranularity(&granularity, &prop, CU_MEM_ALLOC_GRANULARITY_RECOMMENDED)); resources->ucGran = granularity; @@ -229,6 +230,7 @@ ncclResult_t nvlsGroupUnmapMem(struct ncclComm *comm, struct ncclNvlsSharedRes* NCCL_PARAM(NvlsEnable, "NVLS_ENABLE", 2); NCCL_PARAM(NvlsChannels, "NVLS_NCHANNELS", 16); +NCCL_PARAM(NvlsChunkSize, "NVLS_CHUNKSIZE", 128*1024); ncclResult_t ncclNvlsInit(struct ncclComm* comm) { comm->nvlsSupport = 0; @@ -236,8 +238,7 @@ ncclResult_t ncclNvlsInit(struct ncclComm* comm) { int gpuCount; NCCLCHECK(ncclTopoGetGpuCount(comm->topo, &gpuCount)); - // NVLS is not supported on MNNVL yet - if (!ncclParamNvlsEnable() || gpuCount <= 2 || comm->nNodes > 1 || comm->MNNVL) return ncclSuccess; + if (!ncclParamNvlsEnable() || ((!comm->MNNVL && gpuCount <= 2) || (comm->MNNVL && comm->clique.size <= 2))) return ncclSuccess; CUdevice dev; int driverVersion; @@ -306,7 +307,8 @@ ncclResult_t ncclNvlsSetup(struct ncclComm* comm, struct ncclComm* parent) { NCCLCHECK(initNvlsChannel(comm, c, parent, false)); } - size_t buffSize = comm->buffSizes[NCCL_PROTO_SIMPLE]; + int nvlsStepSize = comm->nvlsChunkSize = ncclParamNvlsChunkSize(); + size_t buffSize = nvlsStepSize * NCCL_STEPS; size_t memSize = NVLS_MEM_ALIGN_SIZE; size_t nvlsPerRankSize = nChannels * 2 * (buffSize + memSize); size_t nvlsTotalSize = nvlsPerRankSize * nHeads; @@ -315,7 +317,7 @@ ncclResult_t ncclNvlsSetup(struct ncclComm* comm, struct ncclComm* parent) { comm, headRank, nHeads, buffSize, memSize, nvlsPerRankSize, nvlsTotalSize); char* shareableHandle = resources->shareableHandle; - NCCLCHECKGOTO(nvlsGetProperties(comm, resources, dev, comm->localRanks, nvlsTotalSize), res, cleanup); + NCCLCHECKGOTO(nvlsGetProperties(comm, resources, dev, nvlsTotalSize), res, cleanup); if (comm->localRank == 0) { NCCLCHECKGOTO(nvlsGroupCreate(comm, &resources->properties, comm->localRank, comm->localRanks, &resources->mcHandle, shareableHandle), res, cleanup); NCCLCHECKGOTO(bootstrapIntraNodeBroadcast(comm->bootstrap, comm->localRankToRank, comm->localRank, comm->localRanks, 0, shareableHandle, NVLS_HANDLE_SIZE), res, cleanup); @@ -326,8 +328,14 @@ ncclResult_t ncclNvlsSetup(struct ncclComm* comm, struct ncclComm* parent) { NCCLCHECKGOTO(nvlsGroupAddDevice(comm, resources), res, cleanup); NCCLCHECKGOTO(nvlsGroupBindMem(comm, resources), res, cleanup); - // Local intra-node barrier to ensure everyone has bound their memory to the group - NCCLCHECKGOTO(bootstrapBarrier(comm->bootstrap, comm->localRankToRank, comm->localRank, comm->localRanks, comm->localRankToRank[0]), res, cleanup); + if (comm->localRanks > 1) { + // Local intra-node barrier to ensure everyone has bound their memory to the group + NCCLCHECKGOTO(bootstrapIntraNodeBarrier(comm->bootstrap, comm->localRankToRank, comm->localRank, comm->localRanks, comm->localRankToRank[0]), res, cleanup); + } + if (comm->MNNVL) { + // MNNVL: Clique wide barrier to ensure everyone has bound their memory to the group + NCCLCHECKGOTO(bootstrapIntraNodeBarrier(comm->bootstrap, comm->clique.ranks, comm->cliqueRank, comm->clique.size, comm->clique.ranks[0]), res, cleanup); + } NCCLCHECKGOTO(nvlsGroupMapMem(comm, resources), res, cleanup); for (int h = 0; h < nHeads; h++) { @@ -343,11 +351,13 @@ ncclResult_t ncclNvlsSetup(struct ncclComm* comm, struct ncclComm* parent) { peer->send[1].conn.buffs[NCCL_PROTO_SIMPLE] = mem; peer->send[1].conn.head = (uint64_t*)(mem + buffSize); peer->send[1].conn.tail = (uint64_t*)(mem + buffSize + memSize / 2); + peer->send[1].conn.stepSize = nvlsStepSize; mem = resources->mcBuff + (h * 2 * nChannels + c) * (buffSize + memSize); peer->recv[0].transportComm = &nvlsTransport.recv; peer->recv[0].conn.buffs[NCCL_PROTO_SIMPLE] = mem; peer->recv[0].conn.head = (uint64_t*)(mem + buffSize); peer->recv[0].conn.tail = (uint64_t*)(mem + buffSize + memSize / 2); + peer->recv[0].conn.stepSize = nvlsStepSize; peer->recv[0].conn.flags |= NCCL_NVLS_MIN_POLL; // Broadcast MC -> UC @@ -356,11 +366,13 @@ ncclResult_t ncclNvlsSetup(struct ncclComm* comm, struct ncclComm* parent) { peer->recv[1].conn.buffs[NCCL_PROTO_SIMPLE] = mem; peer->recv[1].conn.head = (uint64_t*)(mem + buffSize); peer->recv[1].conn.tail = (uint64_t*)(mem + buffSize + memSize / 2); + peer->recv[1].conn.stepSize = nvlsStepSize; mem = resources->mcBuff + ((h * 2 + 1) * nChannels + c) * (buffSize + memSize); peer->send[0].transportComm = &nvlsTransport.send; peer->send[0].conn.buffs[NCCL_PROTO_SIMPLE] = mem; peer->send[0].conn.head = (uint64_t*)(mem + buffSize); peer->send[0].conn.tail = (uint64_t*)(mem + buffSize + memSize / 2); + peer->send[0].conn.stepSize = nvlsStepSize; peer->send[0].conn.flags |= NCCL_NVLS_MIN_POLL; CUDACHECKGOTO(cudaMemcpyAsync(&comm->channels[c].devPeersHostPtr[nvlsPeer]->send[0], &peer->send[0].conn, sizeof(struct ncclConnInfo), cudaMemcpyHostToDevice, comm->sharedRes->hostStream.cudaStream), res, cleanup); @@ -378,6 +390,9 @@ ncclResult_t ncclNvlsSetup(struct ncclComm* comm, struct ncclComm* parent) { } } + // MNNVL does not support NVLS buffer registration + if (comm->MNNVL) return res; + /* create shared memory for fast NVLS buffer registration */ typeSize = sizeof(struct localRegData) << 1; @@ -595,7 +610,7 @@ ncclResult_t ncclNvlsLocalRegisterBuffer(struct ncclComm *comm, const void *send if ((!sendNeedReg || sendbuff == NULL) && (!recvNeedReg || recvbuff == NULL)) { localRegBufUsed = true; - INFO(NCCL_NVLS, "rank %d reuse local-registered sendbuff %p, recvbuff %p, sendbuff size %ld, recvbuff size %ld, reg sendbuff %p, reg recvbuff %p", comm->rank, sendbuff, recvbuff, sendbuffSize, recvbuffSize, (void*)regSendPtr, (void*)regRecvPtr); + INFO(NCCL_NVLS, "rank %d reuse local-registered NVLS sendbuff %p, recvbuff %p, sendbuff size %ld, recvbuff size %ld, reg sendbuff %p, reg recvbuff %p", comm->rank, sendbuff, recvbuff, sendbuffSize, recvbuffSize, (void*)regSendPtr, (void*)regRecvPtr); goto exit; } @@ -611,7 +626,7 @@ ncclResult_t ncclNvlsLocalRegisterBuffer(struct ncclComm *comm, const void *send if (localRegBufUsed == false) goto fail; } - INFO(NCCL_NVLS, "rank %d successfully local-registered sendbuff %p, recvbuff %p, sendbuff size %ld, recvbuff size %ld, reg sendbuff %p, reg recvbuff %p", comm->rank, sendbuff, recvbuff, sendbuffSize, recvbuffSize, (void*)regSendPtr, (void*)regRecvPtr); + INFO(NCCL_NVLS, "rank %d successfully local-registered NVLS sendbuff %p, recvbuff %p, sendbuff size %ld, recvbuff size %ld, reg sendbuff %p, reg recvbuff %p", comm->rank, sendbuff, recvbuff, sendbuffSize, recvbuffSize, (void*)regSendPtr, (void*)regRecvPtr); exit: *outRegBufSend = (void*)regSendPtr; diff --git a/src/transport/p2p.cc b/src/transport/p2p.cc index b29224e13..17a5d69ee 100644 --- a/src/transport/p2p.cc +++ b/src/transport/p2p.cc @@ -99,12 +99,15 @@ NCCL_PARAM(P2pUseCudaMemcpy, "P2P_USE_CUDA_MEMCPY", 0); static int useMemcpy = 0; static void initCeOperation(); + +extern int64_t ncclParamMNNVLEnable(); + /* Determine if two peers can communicate through p2p */ ncclResult_t p2pCanConnect(int* ret, struct ncclTopoSystem* topo, struct ncclTopoGraph* graph, struct ncclPeerInfo* info1, struct ncclPeerInfo* info2) { initCeOperation(); // MNNVL support - if (info1->hostHash != info2->hostHash) { + if (ncclParamMNNVLEnable() != 0 && info1->hostHash != info2->hostHash) { NCCLCHECK(ncclTopoCheckMNNVL(topo, info1, info2, ret)); if (*ret) return ncclSuccess; } @@ -467,6 +470,7 @@ static ncclResult_t p2pSendConnect(struct ncclComm* comm, struct ncclConnect* co buff += comm->buffSizes[p]; } } + send->conn.stepSize = comm->buffSizes[NCCL_PROTO_SIMPLE]/NCCL_STEPS; if (useMemcpy) { send->conn.tail = &resources->proxyInfo.ceRecvMem->tail; @@ -512,6 +516,7 @@ ncclResult_t p2pRecvConnect(struct ncclComm* comm, struct ncclConnect* connectIn recv->conn.ptrExchange = &remDevMem->ptrExchange; recv->conn.redOpArgExchange = remDevMem->redOpArgExchange; } + recv->conn.stepSize = comm->buffSizes[NCCL_PROTO_SIMPLE]/NCCL_STEPS; char* buff = (char*)(resources->recvDevMem+1); for (int p=0; pconn.tail = &resources->devRemHostMem->tail; send->conn.head = &resources->devHostMem->head; + send->conn.stepSize = comm->buffSizes[NCCL_PROTO_SIMPLE]/NCCL_STEPS; if (useMemcpyRecv) { send->conn.connFifo = resources->devRemHostMem->connFifo; @@ -189,6 +190,7 @@ static ncclResult_t shmRecvConnect(struct ncclComm* comm, struct ncclConnect* co } recv->conn.head = &resources->devRemHostMem->head; recv->conn.tail = &resources->devHostMem->tail; + recv->conn.stepSize = comm->buffSizes[NCCL_PROTO_SIMPLE]/NCCL_STEPS; if (useMemcpyRecv) { NCCLCHECK(ncclProxyConnect(comm, TRANSPORT_SHM, 0, comm->rank, &recv->proxyConn)); @@ -210,6 +212,7 @@ static ncclResult_t shmSendFree(struct ncclConnector* send) { NCCLCHECK(ncclShmClose(resources->hostHandle)); NCCLCHECK(ncclShmClose(resources->remHandle)); free(resources); + send->transportResources = NULL; } return ncclSuccess; } @@ -220,6 +223,7 @@ static ncclResult_t shmRecvFree(struct ncclConnector* recv) { NCCLCHECK(ncclShmClose(resources->hostHandle)); NCCLCHECK(ncclShmClose(resources->remHandle)); free(resources); + recv->transportResources = NULL; } return ncclSuccess; } @@ -271,6 +275,7 @@ static ncclResult_t shmSendProxyFree(struct ncclProxyConnection* connection, str CUDACHECK(cudaEventDestroy(resources->events[i])); } free(connection->transportResources); + connection->transportResources = NULL; } return ncclSuccess; } @@ -286,6 +291,7 @@ static ncclResult_t shmRecvProxyFree(struct ncclProxyConnection* connection, str CUDACHECK(cudaEventDestroy(resources->events[i])); } free(connection->transportResources); + connection->transportResources = NULL; } return ncclSuccess; } @@ -409,8 +415,8 @@ static ncclResult_t shmRecvProxyProgress(struct ncclProxyState* proxyState, stru struct ncclTransport shmTransport = { "SHM", shmCanConnect, - { shmSendSetup, shmSendConnect, shmSendFree, NULL, NULL, NULL, NULL, NULL }, - { shmRecvSetup, shmRecvConnect, shmRecvFree, NULL, NULL, NULL, NULL, NULL } + { shmSendSetup, shmSendConnect, shmSendFree, NULL, NULL, NULL, NULL, NULL, NULL }, + { shmRecvSetup, shmRecvConnect, shmRecvFree, NULL, NULL, NULL, NULL, NULL, NULL } }; static void initCeOperation() {