Initial release.

This commit is contained in:
Nathan Luehr 2015-11-17 11:30:40 -08:00
commit 0673d5f44f
25 changed files with 6709 additions and 0 deletions

1
.gitignore vendored Normal file
View File

@ -0,0 +1 @@
/build

27
LICENSE.txt Normal file
View File

@ -0,0 +1,27 @@
Copyright (c) 2015, NVIDIA CORPORATION. All rights reserved.
Redistribution and use in source and binary forms, with or without
modification, are permitted provided that the following conditions
are met:
* Redistributions of source code must retain the above copyright
notice, this list of conditions and the following disclaimer.
* Redistributions in binary form must reproduce the above copyright
notice, this list of conditions and the following disclaimer in the
documentation and/or other materials provided with the distribution.
* Neither the name of NVIDIA CORPORATION nor the names of its
contributors may be used to endorse or promote products derived
from this software without specific prior written permission.
THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS ``AS IS'' AND ANY
EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR
PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR
CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL,
EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO,
PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR
PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY
OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.

133
Makefile Normal file
View File

@ -0,0 +1,133 @@
#
# Copyright (c) 2015, NVIDIA CORPORATION. All rights reserved.
#
# Redistribution and use in source and binary forms, with or without
# modification, are permitted provided that the following conditions
# are met:
# * Redistributions of source code must retain the above copyright
# notice, this list of conditions and the following disclaimer.
# * Redistributions in binary form must reproduce the above copyright
# notice, this list of conditions and the following disclaimer in the
# documentation and/or other materials provided with the distribution.
# * Neither the name of NVIDIA CORPORATION nor the names of its
# contributors may be used to endorse or promote products derived
# from this software without specific prior written permission.
#
# THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS ``AS IS'' AND ANY
# EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
# IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR
# PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR
# CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL,
# EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO,
# PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR
# PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY
# OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
# (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
# OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
#
CUDA_HOME ?= /usr/local/cuda
PREFIX ?= /usr/local
VERBOSE ?= 0
CUDACODE := -gencode=arch=compute_35,code=sm_35 \
-gencode=arch=compute_52,code=sm_52
BUILDDIR := build
NVCC := $(CUDA_HOME)/bin/nvcc
GPP := g++
CPPFLAGS := -I$(CUDA_HOME)/include
CXXFLAGS := -O3 -fPIC -fvisibility=hidden
NVCUFLAGS := $(CUDACODE) -O3 -lineinfo -std=c++11 -maxrregcount 96
ifneq ($(VERBOSE), 0)
NVCUFLAGS += -Xptxas -v -Xcompiler -Wall,-Wextra
CXXFLAGS += -Wall -Wextra
endif
LDFLAGS := -L$(CUDA_HOME)/lib64 -lcudart
MPIFLAGS := -I$(MPI_HOME)/include -L$(MPI_HOME)/lib -lmpi
.PHONY : lib clean test mpitest install
.DEFAULT : lib
INCEXPORTS := nccl.h
LIBSRCFILES := libwrap.cu core.cu all_gather.cu all_reduce.cu broadcast.cu reduce.cu reduce_scatter.cu
LIBNAME := libnccl.so
APIVER := 0
TESTS := all_gather_test all_reduce_test broadcast_test reduce_test reduce_scatter_test
MPITESTS := mpi_test
INCDIR := $(BUILDDIR)/include
LIBDIR := $(BUILDDIR)/lib
OBJDIR := $(BUILDDIR)/obj
TSTDIR := $(BUILDDIR)/test
MPITSTDIR := $(BUILDDIR)/mpitest
INCTARGETS := $(patsubst %, $(INCDIR)/%, $(INCEXPORTS))
LIBTARGET := $(patsubst %, $(LIBDIR)/%.$(APIVER), $(LIBNAME))
LIBLINK := $(patsubst lib%.so, -l%, $(LIBNAME))
LIBOBJ := $(patsubst %.cu, $(OBJDIR)/%.o, $(filter %.cu, $(LIBSRCFILES)))
TESTBINS := $(patsubst %, $(TSTDIR)/%, $(TESTS))
MPITESTBINS:= $(patsubst %, $(MPITSTDIR)/%, $(MPITESTS))
DEPFILES := $(patsubst %.o, %.d, $(LIBOBJ)) $(patsubst %, %.d, $(TESTBINS)) $(patsubst %, %.d, $(MPITESTBINS))
lib : $(INCTARGETS) $(LIBTARGET)
-include $(DEPFILES)
$(LIBTARGET) : $(LIBOBJ)
@printf "Linking %-25s\n" $@
@mkdir -p $(LIBDIR)
@$(GPP) $(CPPFLAGS) $(CXXFLAGS) -shared -Wl,-soname,$(LIBNAME).$(APIVER) -o $@ $(LDFLAGS) $(LIBOBJ)
@ln -sf $(LIBNAME).$(APIVER) $(LIBDIR)/$(LIBNAME)
$(INCDIR)/%.h : src/%.h
@printf "Grabbing %-25s > %-25s\n" $< $@
@mkdir -p $(INCDIR)
@cp -f $< $@
$(OBJDIR)/%.o : src/%.cu
@printf "Compiling %-25s > %-25s\n" $< $@
@mkdir -p $(OBJDIR)
@$(NVCC) -c $(CPPFLAGS) $(NVCUFLAGS) --compiler-options "$(CXXFLAGS)" $< -o $@
@$(NVCC) -M $(CPPFLAGS) $(NVCUFLAGS) --compiler-options "$(CXXFLAGS)" $< > $(@:%.o=%.d.tmp)
@sed "0,/^.*:/s//$(subst /,\/,$@):/" $(@:%.o=%.d.tmp) > $(@:%.o=%.d)
@sed -e 's/.*://' -e 's/\\$$//' < $(@:%.o=%.d.tmp) | fmt -1 | \
sed -e 's/^ *//' -e 's/$$/:/' >> $(@:%.o=%.d)
@rm -f $(@:%.o=%.d.tmp)
clean :
rm -rf build
test : lib $(TESTBINS)
$(TSTDIR)/% : src/%.cu lib
@printf "Building %-25s > %-24s\n" $< $@
@mkdir -p $(TSTDIR)
@$(NVCC) -Ibuild/include $(CPPFLAGS) $(NVCUFLAGS) --compiler-options "$(CXXFLAGS)" -o $@ $< -Lbuild/lib $(LIBLINK) $(LDFLAGS) -lcuda -lcurand -lnvToolsExt -lnvidia-ml
@$(NVCC) -M -Ibuild/include $(CPPFLAGS) $(NVCUFLAGS) --compiler-options "$(CXXFLAGS)" $< -Lbuild/lib $(LIBLINK) $(LDFLAGS) -lcuda -lcurand -lnvToolsExt -lnvidia-ml > $(@:%=%.d.tmp)
@sed "0,/^.*:/s//$(subst /,\/,$@):/" $(@:%=%.d.tmp) > $(@:%=%.d)
@sed -e 's/.*://' -e 's/\\$$//' < $(@:%=%.d.tmp) | fmt -1 | \
sed -e 's/^ *//' -e 's/$$/:/' >> $(@:%=%.d)
@rm -f $(@:%=%.d.tmp)
mpitest : lib $(MPITESTBINS)
$(MPITSTDIR)/% : src/%.cu lib
@printf "Building %-25s > %-24s\n" $< $@
@mkdir -p $(MPITSTDIR)
@$(NVCC) $(MPIFLAGS) -Ibuild/include $(CPPFLAGS) $(NVCUFLAGS) --compiler-options "$(CXXFLAGS)" -o $@ $< -Lbuild/lib $(LIBLINK) $(LDFLAGS)
@$(NVCC) $(MPIFLAGS) -M -Ibuild/include $(CPPFLAGS) $(NVCUFLAGS) --compiler-options "$(CXXFLAGS)" $< -Lbuild/lib $(LIBLINK) $(LDFLAGS) > $(@:%=%.d.tmp)
@sed "0,/^.*:/s//$(subst /,\/,$@):/" $(@:%=%.d.tmp) > $(@:%=%.d)
@sed -e 's/.*://' -e 's/\\$$//' < $(@:%=%.d.tmp) | fmt -1 | \
sed -e 's/^ *//' -e 's/$$/:/' >> $(@:%=%.d)
@rm -f $(@:%=%.d.tmp)
install : lib
@mkdir -p $(PREFIX)/lib
@mkdir -p $(PREFIX)/include
@cp -P -v build/lib/* $(PREFIX)/lib/
@cp -v build/include/* $(PREFIX)/include/

115
README.md Normal file
View File

@ -0,0 +1,115 @@
# NCCL
Optimized primitives for collective multi-GPU communication.
## Introduction
NCCL (pronounced "Nickel") is a stand-alone library of standard collective communication routines, such as all-gather, reduce, broadcast, etc., that have been optimized to achieve high bandwidth over PCIe. NCCL supports up to eight GPUs and can be used in either single- or multi-process (e.g., MPI) applications.
## What's inside
At present, the library implements the following collectives:
- all-reduce
- all-gather
- reduce-scatter
- reduce
- broadcast
These collectives are implemented using ring algorithms and have been optimized primarily for throughput. For best performance, small collectives should be batched into larger operations whenever possible. Small test binaries demonstrating how to use each of the above collectives are also provided.
## Requirements
NCCL requires at least CUDA 7.0 and Kepler or newer GPUs. Best performance is achieved when all GPUs are located on a common PCIe root complex, but multi-socket configurations are also supported.
Note: NCCL may also work with CUDA 6.5, but this is an untested configuration.
## Build & run
To build the library and tests.
```shell
$ cd nccl
$ make CUDA_HOME=<cuda install path> test
```
Test binaries are located in the subdirectories nccl/build/test and nccl/build/mpitest.
```shell
$ export LD_LIBRARY_PATH=$LD_LIBRARY_PATH:./build/lib
$ ./build/test/allreduce_test
Error: must specify at least data size in bytes!
Tests nccl AllReduce with user supplied arguments.
Usage: all_reduce_test <data size in bytes> [number of GPUs] [GPU 0] [GPU 1] ...
$ ./build/test/allreduce_test 10000000
# Using devices
# Device 0 -> 0 [0x0a] GeForce GTX TITAN X
# Device 1 -> 1 [0x09] GeForce GTX TITAN X
# Device 2 -> 2 [0x06] GeForce GTX TITAN X
# Device 3 -> 3 [0x05] GeForce GTX TITAN X
# out-of-place in-place
# bytes N type op time algbw busbw res time algbw busbw res
10000000 10000000 char sum 1.628 6.14 9.21 0e+00 1.932 5.18 7.77 0e+00
10000000 10000000 char prod 1.629 6.14 9.21 0e+00 1.643 6.09 9.13 0e+00
10000000 10000000 char max 1.621 6.17 9.25 0e+00 1.634 6.12 9.18 0e+00
10000000 10000000 char min 1.633 6.12 9.19 0e+00 1.637 6.11 9.17 0e+00
10000000 2500000 int sum 1.611 6.21 9.31 0e+00 1.626 6.15 9.23 0e+00
10000000 2500000 int prod 1.613 6.20 9.30 0e+00 1.629 6.14 9.21 0e+00
10000000 2500000 int max 1.619 6.18 9.26 0e+00 1.627 6.15 9.22 0e+00
10000000 2500000 int min 1.619 6.18 9.27 0e+00 1.624 6.16 9.24 0e+00
10000000 5000000 half sum 1.617 6.18 9.28 4e-03 1.636 6.11 9.17 4e-03
10000000 5000000 half prod 1.618 6.18 9.27 1e-03 1.657 6.03 9.05 1e-03
10000000 5000000 half max 1.608 6.22 9.33 0e+00 1.621 6.17 9.25 0e+00
10000000 5000000 half min 1.610 6.21 9.32 0e+00 1.627 6.15 9.22 0e+00
10000000 2500000 float sum 1.618 6.18 9.27 5e-07 1.622 6.17 9.25 5e-07
10000000 2500000 float prod 1.614 6.20 9.29 1e-07 1.628 6.14 9.21 1e-07
10000000 2500000 float max 1.616 6.19 9.28 0e+00 1.633 6.12 9.19 0e+00
10000000 2500000 float min 1.613 6.20 9.30 0e+00 1.628 6.14 9.21 0e+00
10000000 1250000 double sum 1.629 6.14 9.21 0e+00 1.628 6.14 9.21 0e+00
10000000 1250000 double prod 1.619 6.18 9.26 2e-16 1.628 6.14 9.21 2e-16
10000000 1250000 double max 1.613 6.20 9.30 0e+00 1.630 6.13 9.20 0e+00
10000000 1250000 double min 1.622 6.16 9.25 0e+00 1.623 6.16 9.24 0e+00
```
To install, run `make PREFIX=<install dir> install` and add `<instal dir>/lib` to your `LD_LIBRARY_PATH`.
## Usage
NCCL follows the MPI collectives API fairly closely. Before any collectives can be called, a communicator object must be initialized on each GPU. On a single-process machine, all GPUs can be conveniently initialized using `ncclCommInitAll`. For multi-process applications (e.g., with MPI), `ncclCommInitRank` must be called for each GPU. Internally `ncclCommInitRank` invokes a synchronization among all GPUs, so these calls must be invoked in different host threads (or processes) for each GPU. A brief single-process example follows, for an MPI example see src/mpi_test.cu. For details about the API see nccl.h.
```c
#include <nccl.h>
typedef struct {
double* sendBuff;
double* recvBuff;
int size;
cudaStream_t stream;
} PerThreadData;
int main(int argc, char* argv[])
{
int nGPUs;
cudaGetDeviceCount(&nGPUs);
ncclComm_t* comms = (ncclComm_t*)malloc(sizeof(ncclComm_t)*nGPUs);
ncclCommInitAll(comms, nGPUs); // initialize communicator
// One communicator per process
PerThreadData* data;
... // Allocate data and issue work to each GPU's
// perDevStream to populate the sendBuffs.
for(int i=0; i<nGPUs; ++i) {
cudaSetDevice(i); // Correct device must be set
// prior to each collective call.
ncclAllReduce(data[i].sendBuff, data[i].recvBuff, size,
ncclDouble, ncclSum, comms[i], data[i].stream);
}
... // Issue work into data[*].stream to consume buffers, etc.
}
```

490
src/all_gather.cu Normal file
View File

@ -0,0 +1,490 @@
/*************************************************************************
* Copyright (c) 2015, NVIDIA CORPORATION. All rights reserved.
*
* Redistribution and use in source and binary forms, with or without
* modification, are permitted provided that the following conditions
* are met:
* * Redistributions of source code must retain the above copyright
* notice, this list of conditions and the following disclaimer.
* * Redistributions in binary form must reproduce the above copyright
* notice, this list of conditions and the following disclaimer in the
* documentation and/or other materials provided with the distribution.
* * Neither the name of NVIDIA CORPORATION nor the names of its
* contributors may be used to endorse or promote products derived
* from this software without specific prior written permission.
*
* THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS ``AS IS'' AND ANY
* EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
* IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR
* PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR
* CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL,
* EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO,
* PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR
* PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY
* OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
* (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
* OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
************************************************************************/
#include <algorithm>
#include <cassert>
#include "core.h"
#include "common_kernel.h"
#include "copy_kernel.h"
#include "enqueue.h"
/* HIERARCHY
*
* The data is split into CHUNKS, and each CHUNK is split into NUM_SUBCHUNKS
* SUBCHUNKS, where each SUBCHUNK is processed independently. A SUBCHUNK is
* split into numUnroll UNROLLS and each thread performs UNROLL_COUNT
* single-data-element operations inside an UNROLL. As the name suggests, the
* UNROLL_COUNT operations within an UNROLL are unrolled.
*/
// Number of threads used to perform copies, etc. Must be multiple of 32.
// An additional thread is used to handle threadfences, so the CUDA blocks
// have dimension NUM_THREADS+1.
#define NUM_THREADS 256
// Each thread unrolls the innermost loop of the copy or reduction operations
// to this many single-data-element instructions
#define UNROLL_COUNT 8
#define UNROLL_SIZE (UNROLL_COUNT * NUM_THREADS)
// To hide the latency associated with the synchronization between different
// subchunks, we interleave the independent subchunks so that more data can be
// transferred while the sync is in progress. This is the number of subchunks
// that are active at the same time
#define NUM_SUBCHUNKS 1
// If this is called with STEP, it means that we just finished processing the
// data for step STEP on this GPU, which is the data required on the next GPU
// for step STEP + 1, so we signal the next GPU that its data for step STEP + 1
// is available. This is called by one particular consumer warp and so we select
// the first thread in the warp to set the flag.
#define SIGNAL_NEW_DATA_AVAILABLE(chunk, subchunk, step) \
do { \
__threadfence_system(); \
args.NextNewDataAvailableFlag[0] = \
NUM_SUBCHUNKS*((chunk) * (args.NumGPUs - 1) + (step)) + subchunk+1; \
} while (0)
// This is called by all producer threads, but only thread 0 spins on the flag,
#define WAIT_FOR_NEW_DATA(chunk, subchunk, step) \
do { \
if (tid == 0) { \
Wait([=] { \
return ((volatile int *)args.ThisNewDataAvailableFlag)[0] >= \
NUM_SUBCHUNKS*((chunk) * (args.NumGPUs - 1) + (step)) \
+ subchunk + 1 - NUM_SUBCHUNKS; \
}); \
} \
BAR(sync, 1, NUM_THREADS); \
} while (0)
#define SIGNAL_CHUNK_DONE(chunk, subchunk) \
do { \
__threadfence_system(); \
args.PrevChunkDoneFlag[0] = NUM_SUBCHUNKS*(chunk) + (subchunk) + 1; \
} while (0)
#define WAIT_FOR_PREV_CHUNK(chunk, subchunk) \
do { \
if (tid == 0) { \
Wait([=] { \
return ((volatile int*)args.ThisChunkDoneFlag)[0] >= \
NUM_SUBCHUNKS*(chunk) + subchunk + 1-NUM_SUBCHUNKS; \
}); \
} \
BAR(sync, 1, NUM_THREADS); \
} while (0)
__device__ inline void getSliceSizeAndChunkSize(int *sliceSize, int slice,
int numSlices, int numBigSlices, int numSmallSlices, int bigSliceN,
int smallSliceN, int lastSliceN) {
if (slice < numBigSlices) {
*sliceSize = bigSliceN;
} else {
*sliceSize = (slice < numBigSlices + numSmallSlices) ? smallSliceN
: ((slice == numSlices - 1) ? lastSliceN : 0);
}
}
template<typename T>
struct AllGatherKernelArgs {
// general parameters
int ThisId;
int NumGPUs;
int N;
int * UserFromRing;
// some pre-computed sizes
int SliceSize;
int ChunkSize;
int NumChunks;
int BufferSliceStride;
int BufferMisalignedN;
T ** ThisPtrToNextOutput;
T ** PrevPtrToThisOutput;
// local and remote input, output, and buffer
const T * __restrict__ ThisInput;
volatile T * __restrict__ ThisOutput;
volatile T * __restrict__ ThisBuffer;
volatile T * __restrict__ NextBuffer;
// local and remote flags
volatile int * __restrict__ ThisNewDataAvailableFlag;
volatile int * __restrict__ NextNewDataAvailableFlag;
volatile int * __restrict__ ThisChunkDoneFlag;
volatile int * __restrict__ PrevChunkDoneFlag;
};
__device__ inline int GetBlock(const int index, const int step,
const int * const userFromRing, const int numGPUs) {
return userFromRing[(numGPUs + index - step) % numGPUs];
}
__shared__ volatile void * nextOutput;
template<int THREADS, int UNROLL, bool PUSHRECV, typename T>
__global__ void AllGatherKernel(const AllGatherKernelArgs<T> args) {
if (args.N == 0) return;
int tid = threadIdx.x;
// First wait for args.PrevPtrToThisOutput to become nullptr to ensure that
// the previous GPU is done with a previous collective operation.
if (tid == 0) {
Wait([=] {
return *((T * volatile *)args.PrevPtrToThisOutput) == nullptr;
});
*((T * volatile *)args.PrevPtrToThisOutput) = (T*)args.ThisOutput;
Wait([=] {
return *((T * volatile *)args.ThisPtrToNextOutput) != nullptr;
});
if(PUSHRECV)
nextOutput = *((volatile void * volatile *)args.ThisPtrToNextOutput);
}
__syncthreads();
for (int chunk = 0; chunk < args.NumChunks; ++chunk) {
// calculate slice size. for all chunks except (possibly) the last one,
// this will just be args.SliceSize. For the last one, it may be smaller
int bigSliceN = args.SliceSize;
int smallSliceN = 0;
int lastSliceN = 0;
int numSlices = NUM_SUBCHUNKS;
int numBigSlices = numSlices;
int numSmallSlices = 0;
// last chunk
if ((chunk + 1 == args.NumChunks) && (args.N % args.ChunkSize > 0))
CalcLastChunk<THREADS, UNROLL, T>(&bigSliceN, &smallSliceN, &lastSliceN,
&numSlices, &numBigSlices, &numSmallSlices, args.N, args.NumChunks,
args.ChunkSize);
// this offset is only applied to Data pointers, not to Buffer pointers,
// since we only have one buffer per chunk
int chunkOffset = chunk * args.ChunkSize;
// step 0: copy the resident block from the ThisInput to ThisOutput and also
// to NextOutput
int step = 0;
int block = GetBlock(args.ThisId, step, args.UserFromRing, args.NumGPUs);
int outputOffset = chunkOffset + block * args.N;
int inputOffset = chunkOffset;
int bufferOffset;
int sliceSize;
if (!PUSHRECV) {
bufferOffset = block * NUM_SUBCHUNKS * args.BufferSliceStride +
block * args.BufferMisalignedN;
}
// Copy from ThisInput
if (tid < THREADS) {
for(int s=0; s<NUM_SUBCHUNKS; ++s) {
getSliceSizeAndChunkSize(&sliceSize, s, numSlices, numBigSlices,
numSmallSlices, bigSliceN, smallSliceN, lastSliceN);
if (!PUSHRECV)
WAIT_FOR_PREV_CHUNK(chunk, s);
if (PUSHRECV) {
DoubleCopy<UNROLL, THREADS>(
args.ThisOutput + outputOffset,
(volatile T *)nextOutput + outputOffset,
args.ThisInput + inputOffset,
sliceSize);
} else {
DoubleCopy<UNROLL, THREADS>(
args.ThisOutput + outputOffset,
args.NextBuffer + bufferOffset,
args.ThisInput + inputOffset,
sliceSize);
}
__syncthreads();
outputOffset += sliceSize;
inputOffset += sliceSize;
if (!PUSHRECV)
bufferOffset += sliceSize;
}
} else {
for(int s=0; s<NUM_SUBCHUNKS; ++s) {
__syncthreads();
SIGNAL_NEW_DATA_AVAILABLE(chunk, s, step);
}
}
// steps j with 0 < j < k - 1:
// copy a block that was pushed to this GPU to the next GPU
for (step = 1; step < args.NumGPUs - 1; ++step) {
block = GetBlock(args.ThisId, step, args.UserFromRing, args.NumGPUs);
outputOffset = chunkOffset + block * args.N;
if (!PUSHRECV) {
bufferOffset = block * NUM_SUBCHUNKS * args.BufferSliceStride +
block * args.BufferMisalignedN;
}
if (tid < THREADS) {
for(int s=0; s<NUM_SUBCHUNKS; ++s) {
getSliceSizeAndChunkSize(&sliceSize, s, numSlices, numBigSlices,
numSmallSlices, bigSliceN, smallSliceN, lastSliceN);
WAIT_FOR_NEW_DATA(chunk, s, step);
if (PUSHRECV) {
Copy<UNROLL, THREADS>(
(volatile T *)nextOutput + outputOffset,
args.ThisOutput + outputOffset,
sliceSize);
} else {
DoubleCopy<UNROLL, THREADS>(
args.NextBuffer + bufferOffset,
args.ThisOutput + outputOffset,
args.ThisBuffer + bufferOffset,
sliceSize);
}
__syncthreads();
outputOffset += sliceSize;
if (!PUSHRECV)
bufferOffset += sliceSize;
}
} else {
for(int s=0; s<NUM_SUBCHUNKS; ++s) {
__syncthreads();
SIGNAL_NEW_DATA_AVAILABLE(chunk, s, step);
}
}
}
if (!PUSHRECV) {
step = args.NumGPUs - 1;
block = GetBlock(args.ThisId, step, args.UserFromRing, args.NumGPUs);
outputOffset = chunkOffset + block * args.N;
bufferOffset = block * NUM_SUBCHUNKS * args.BufferSliceStride +
block * args.BufferMisalignedN;
// Make final copy from buffer to dest.
if (tid < THREADS) {
for(int s=0; s<NUM_SUBCHUNKS; ++s) {
getSliceSizeAndChunkSize(&sliceSize, s, numSlices, numBigSlices,
numSmallSlices, bigSliceN, smallSliceN, lastSliceN);
WAIT_FOR_NEW_DATA(chunk, s, step);
Copy<UNROLL, THREADS>(
args.ThisOutput + outputOffset,
args.ThisBuffer + bufferOffset,
sliceSize);
__syncthreads();
outputOffset += sliceSize;
bufferOffset += sliceSize;
}
} else {
for(int s=0; s<NUM_SUBCHUNKS; ++s) {
__syncthreads();
SIGNAL_CHUNK_DONE(chunk, s);
}
}
}
}
// wait for the last data to be pushed to us
if (tid < THREADS) {
if (PUSHRECV)
WAIT_FOR_NEW_DATA(args.NumChunks, NUM_SUBCHUNKS-1, 0);
else
WAIT_FOR_PREV_CHUNK(args.NumChunks, NUM_SUBCHUNKS-1);
if (tid == 0) {
args.ThisNewDataAvailableFlag[0] = 0;
args.ThisChunkDoneFlag[0] = 0;
*args.ThisPtrToNextOutput = nullptr;
}
}
}
template<typename T>
ncclResult_t ncclAllGatherWithType(const void* sendbuff, void* recvbuff,
int count, ncclComm* comm, int numUnroll, cudaStream_t stream) {
if (count == 0)
return ncclSuccess;
int index = comm->ncclId;
int blockSizeInBytes = count * sizeof(T);
int misalignedBytes = blockSizeInBytes % alignof(uint64_t);
assert((int)((misalignedBytes / sizeof(T)) * sizeof(T)) == misalignedBytes);
int misalignedN = misalignedBytes / sizeof(T);
assert(misalignedN < (int)(sizeof(uint64_t) / sizeof(T)));
int paddingN = (misalignedN > 0) ? sizeof(uint64_t) / sizeof(T) : 0;
// There is one slice per GPU, so a slice can be at most bufferN / numGPUs,
// where bufferN is the number of elements of type T that fit into the buffer.
int bufferN = comm->buffSize / sizeof(T);
// we only need buffer for k slices and k paddings
int bufferNPerSlice = (bufferN - comm->nDev * NUM_SUBCHUNKS * paddingN)
/ (comm->nDev * NUM_SUBCHUNKS);
// For efficiency, we want the slice size to be a multiple of UNROLL_SIZE
int maxSliceSize = (bufferNPerSlice / UNROLL_SIZE) * UNROLL_SIZE;
int nextId = (index + 1) % comm->nDev;
int prevId = (index + comm->nDev - 1) % comm->nDev;
AllGatherKernelArgs<T> args;
args.ThisId = index;
args.NumGPUs = comm->nDev;
args.N = count;
/* Block j is coming from sendbuff[j], which lives on device with logical
* index comm->ringFromUser[j]. But the block ordering does not necessarily
* follow the ring ordering. Hence the order in which a particular GPU
* processes the different blocks (the correspondence between the step in
* the reduction algorithm and the block on which a GPU operates in that
* particular step) is not the same as the ring order.
*
* Say we have 4 GPUs and comm->userFromRing = { 1, 2, 0, 3 }. Then there are 3
* step in the all-gather algorithm and block 0 comes from device 2, block 1
* from 0, block 2 from device 1, and block 3 comes from device 3. In the
* first step of the algorithm, each GPU must copy its own block from its
* sendbuff to the appropriate location in its recvbuff. The blocks that a
* GPU has to process in the next steps is determined by the previous step
* because each GPU only hands off data to the next GPU in the ring.
*
* In the above example, we get the following table of which block is
* processed by each GPU in a given step. The columns correspond to the
* different GPUs while the rows are the steps in the algorithm.
*
* GPU 0 1 2 3
* step
* 0 1 2 0 3
* 1 3 1 2 0
* 2 0 3 1 2
*
* We note the the rows in the above table are just comm->userFromRing in the
* first step and the list is cyclicly permuted to the right for each next
* step. The columns, which are what the individual GPUs need to know, are
* comm->userFromRing traversed backwards and starting at index k for GPU k.
* These columns are what we put into args.BlockVsStep to tell the GPU which
* block it needs to be processing at a particular step. */
args.UserFromRing = comm->devUserFromRing;
args.SliceSize = numUnroll * UNROLL_SIZE * sizeof(PackType) / sizeof(T);
args.SliceSize = std::min(maxSliceSize, args.SliceSize);
args.ChunkSize = NUM_SUBCHUNKS * args.SliceSize;
// don't reduce this if we cut the slice size in half below, because if that
// happens, the last chunk will be larger than the other chunks, and we will
// need the extra buffer space
args.BufferSliceStride = args.SliceSize + paddingN;
args.BufferMisalignedN = misalignedN;
// avoid a case where we have one or more big chunks and one tiny one
int remainder = args.N % args.ChunkSize;
if ((args.N > args.ChunkSize) && (remainder > 0) &&
(args.N < 5 * args.ChunkSize) && (2 * remainder < args.ChunkSize)) {
args.SliceSize /= 2;
args.ChunkSize = NUM_SUBCHUNKS * args.SliceSize;
// round down so we end up with a big last chunk
args.NumChunks = args.N / args.ChunkSize;
} else {
// round up
args.NumChunks = (args.N + args.ChunkSize - 1) / args.ChunkSize;
}
args.ThisPtrToNextOutput = (T**)&(comm->local[nextId]->recvPtrs[0]);
args.PrevPtrToThisOutput = (T**)&(comm->remote[prevId]->recvPtrs[0]);
args.ThisInput = (const T*)sendbuff;
args.ThisOutput = (volatile T*)recvbuff;
args.ThisBuffer = (volatile T*)comm->local[prevId]->buff;
args.NextBuffer = (volatile T*)comm->remote[nextId]->buff;
args.ThisNewDataAvailableFlag = comm->local[prevId]->flags;
args.NextNewDataAvailableFlag = comm->remote[nextId]->flags;
args.ThisChunkDoneFlag = comm->local[nextId]->flags + 1;
args.PrevChunkDoneFlag = comm->remote[prevId]->flags + 1;
if( comm->useRemoteRecv ) {
AllGatherKernel<NUM_THREADS, UNROLL_COUNT, true, T>
<<<1, NUM_THREADS + 1, 0, stream>>>(args);
} else {
AllGatherKernel<NUM_THREADS, UNROLL_COUNT, false, T>
<<<1, NUM_THREADS + 1, 0, stream>>>(args);
}
return ncclSuccess;
}
class AllGatherFunctor {
public:
ncclResult_t operator()(const void* sendbuff, void* recvbuff,
int count, ncclDataType_t datatype, ncclRedOp_t /*dummy operation*/,
int /*dummy root*/, ncclComm* comm, cudaStream_t stream) {
int numUnroll = 16; // this is optimal on dt07 with 4 GPUs
switch (datatype) {
case ncclChar:
return ncclAllGatherWithType<char>(sendbuff, recvbuff, count, comm,
numUnroll, stream);
case ncclInt:
return ncclAllGatherWithType<int>(sendbuff, recvbuff, count, comm,
numUnroll, stream);
#if CUDART_VERSION >= 7050
case ncclHalf:
return ncclAllGatherWithType<half>(sendbuff, recvbuff, count, comm,
numUnroll, stream);
#endif
case ncclFloat:
return ncclAllGatherWithType<float>(sendbuff, recvbuff, count, comm,
numUnroll, stream);
case ncclDouble:
return ncclAllGatherWithType<double>(sendbuff, recvbuff, count, comm,
numUnroll, stream);
}
return ncclInvalidType;
}
};
extern "C" DSOGLOBAL
ncclResult_t ncclAllGather(const void* sendbuff, int count, ncclDataType_t datatype,
void* recvbuff, ncclComm_t comm, cudaStream_t stream) {
return enqueue(AllGatherFunctor(), sendbuff, recvbuff, count, datatype,
ncclSum, 0, comm, stream);
}

236
src/all_gather_test.cu Normal file
View File

@ -0,0 +1,236 @@
/*************************************************************************
* Copyright (c) 2015, NVIDIA CORPORATION. All rights reserved.
*
* Redistribution and use in source and binary forms, with or without
* modification, are permitted provided that the following conditions
* are met:
* * Redistributions of source code must retain the above copyright
* notice, this list of conditions and the following disclaimer.
* * Redistributions in binary form must reproduce the above copyright
* notice, this list of conditions and the following disclaimer in the
* documentation and/or other materials provided with the distribution.
* * Neither the name of NVIDIA CORPORATION nor the names of its
* contributors may be used to endorse or promote products derived
* from this software without specific prior written permission.
*
* THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS ``AS IS'' AND ANY
* EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
* IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR
* PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR
* CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL,
* EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO,
* PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR
* PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY
* OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
* (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
* OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
************************************************************************/
#include <chrono>
#include <cstdio>
#include <cstdlib>
#include <string>
#include <vector>
#include "nccl.h"
#include "test_utilities.h"
template<typename T>
void RunTest(T** sendbuff, T** recvbuff, const int N, const ncclDataType_t type,
ncclComm_t* const comms, const std::vector<int>& dList) {
// initialize data
int nDev = 0;
ncclCommCount(comms[0], &nDev);
cudaStream_t* s = (cudaStream_t*)malloc(sizeof(cudaStream_t)*nDev);
T* buffer = (T*)malloc(nDev * N * sizeof(T));
T* result = (T*)malloc(nDev * N * sizeof(T));
memset(buffer, 0, nDev * N * sizeof(T));
memset(result, 0, nDev * N * sizeof(T));
for (int i = 0; i < nDev; ++i) {
CUDACHECK(cudaSetDevice(dList[i]));
CUDACHECK(cudaStreamCreate(s+i));
CUDACHECK(cudaMemset(recvbuff[i], 0, nDev * N * sizeof(T)));
Randomize(sendbuff[i], N, i);
CUDACHECK(cudaMemcpy(result + i * N, sendbuff[i], N * sizeof(T),
cudaMemcpyDeviceToHost));
}
// warm up GPU
for (int i = 0; i < nDev; ++i) {
CUDACHECK(cudaSetDevice(dList[i]));
ncclAllGather((const void*)sendbuff[i], std::min(32 * 1024, N), type,
(void*)recvbuff[i], comms[i], s[i]);
}
for (int i = 0; i < nDev; ++i) {
CUDACHECK(cudaSetDevice(dList[i]));
CUDACHECK(cudaStreamSynchronize(s[i]));
}
//for (int n = 1; n <= N; n = n << 1)
{
int n = N;
printf("%12i %12i %6s", (int)(n * sizeof(T)), n, TypeName(type).c_str());
auto start = std::chrono::high_resolution_clock::now();
for (int i = 0; i < nDev; ++i) {
CUDACHECK(cudaSetDevice(dList[i]));
ncclAllGather((const void*)sendbuff[i], n, type, (void*)recvbuff[i], comms[i],
s[i]);
}
for (int i = 0; i < nDev; ++i) {
CUDACHECK(cudaSetDevice(dList[i]));
CUDACHECK(cudaStreamSynchronize(s[i]));
}
auto stop = std::chrono::high_resolution_clock::now();
double elapsedSec =
std::chrono::duration_cast<std::chrono::duration<double>>(
stop - start).count();
double algbw = (double)(n * sizeof(T)) / 1.0E9 * (double)(nDev - 1)
/ elapsedSec;
double busbw = algbw;
double maxDelta = 0.0;
for (int i = 0; i < nDev; ++i) {
CUDACHECK(cudaSetDevice(dList[i]));
double tmpDelta = CheckDelta<T>(recvbuff[i], result, nDev*N);
maxDelta = std::max(tmpDelta, maxDelta);
}
printf(" %7.3f %5.2f %5.2f %7.0le\n", elapsedSec * 1.0E3, algbw, busbw,
maxDelta);
}
for (int i = 0; i < nDev; ++i) {
CUDACHECK(cudaSetDevice(dList[i]));
CUDACHECK(cudaStreamDestroy(s[i]));
}
free(s);
free(buffer);
free(result);
}
template<typename T>
void RunTests(const int N, const ncclDataType_t type, ncclComm_t* const comms,
const std::vector<int>& dList) {
int nDev = 0;
ncclCommCount(comms[0], &nDev);
T** sendbuff = (T**)malloc(nDev * sizeof(T*));
T** recvbuff = (T**)malloc(nDev * sizeof(T*));
for (int i = 0; i < nDev; ++i) {
CUDACHECK(cudaSetDevice(dList[i]));
CUDACHECK(cudaMalloc(sendbuff + i, N * sizeof(T)));
CUDACHECK(cudaMalloc(recvbuff + i, nDev * N * sizeof(T)));
}
RunTest<T>(sendbuff, recvbuff, N, type, comms, dList);
for (int i = 0; i < nDev; ++i) {
CUDACHECK(cudaSetDevice(dList[i]));
CUDACHECK(cudaFree(sendbuff[i]));
CUDACHECK(cudaFree(recvbuff[i]));
}
free(sendbuff);
free(recvbuff);
}
void usage() {
printf("Tests nccl AllGather with user supplied arguments.\n"
" Usage: all_reduce_test <data size in bytes> [number of GPUs] "
"[GPU 0] [GPU 1] ...\n\n");
}
int main(int argc, char* argv[]) {
int nVis = 0;
CUDACHECK(cudaGetDeviceCount(&nVis));
int N = 0;
if (argc > 1) {
int t = sscanf(argv[1], "%d", &N);
if (t == 0) {
printf("Error: %s is not an integer!\n\n", argv[1]);
usage();
exit(EXIT_FAILURE);
}
} else {
printf("Error: must specify at least data size in bytes!\n\n");
usage();
exit(EXIT_FAILURE);
}
int nDev = nVis;
if (argc > 2) {
int t = sscanf(argv[2], "%d", &nDev);
if (t == 0) {
printf("Error: %s is not an integer!\n\n", argv[1]);
usage();
exit(EXIT_FAILURE);
}
}
std::vector<int> dList(nDev);
for (int i = 0; i < nDev; ++i)
dList[i] = i % nVis;
if (argc > 3) {
if (argc - 3 != nDev) {
printf("Error: insufficient number of GPUs in list\n\n");
usage();
exit(EXIT_FAILURE);
}
for (int i = 0; i < nDev; ++i) {
int t = sscanf(argv[3 + i], "%d", dList.data() + i);
if (t == 0) {
printf("Error: %s is not an integer!\n\n", argv[2 + i]);
usage();
exit(EXIT_FAILURE);
}
}
}
ncclComm_t* comms = (ncclComm_t*)malloc(sizeof(ncclComm_t)*nDev);
ncclCommInitAll(comms, nDev, dList.data());
printf("# Using devices\n");
for (int g=0; g<nDev; ++g) {
int cudaDev;
int rank;
cudaDeviceProp prop;
ncclCommCuDevice(comms[g], &cudaDev);
ncclCommUserRank(comms[g], &rank);
CUDACHECK(cudaGetDeviceProperties(&prop, cudaDev));
printf("# Rank %2d uses device %2d [0x%02x] %s\n", rank, cudaDev,
prop.pciBusID, prop.name);
}
printf("\n");
printf("# %10s %12s %6s %7s %5s %5s %7s\n",
"bytes", "N", "type", "time", "algbw", "busbw", "delta");
RunTests<char>(N / sizeof(char), ncclChar, comms, dList);
RunTests<int>(N / sizeof(int), ncclInt, comms, dList);
#if CUDART_VERSION >= 7050
RunTests<half>(N / sizeof(half), ncclHalf, comms, dList);
#endif
RunTests<float>(N / sizeof(float), ncclFloat, comms, dList);
RunTests<double>(N / sizeof(double), ncclDouble, comms, dList);
printf("\n");
for(int i=0; i<nDev; ++i)
ncclCommDestroy(comms[i]);
free(comms);
exit(EXIT_SUCCESS);
}

504
src/all_reduce.cu Normal file
View File

@ -0,0 +1,504 @@
/*************************************************************************
* Copyright (c) 2015, NVIDIA CORPORATION. All rights reserved.
*
* Redistribution and use in source and binary forms, with or without
* modification, are permitted provided that the following conditions
* are met:
* * Redistributions of source code must retain the above copyright
* notice, this list of conditions and the following disclaimer.
* * Redistributions in binary form must reproduce the above copyright
* notice, this list of conditions and the following disclaimer in the
* documentation and/or other materials provided with the distribution.
* * Neither the name of NVIDIA CORPORATION nor the names of its
* contributors may be used to endorse or promote products derived
* from this software without specific prior written permission.
*
* THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS ``AS IS'' AND ANY
* EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
* IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR
* PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR
* CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL,
* EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO,
* PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR
* PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY
* OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
* (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
* OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
************************************************************************/
#include <nvToolsExt.h>
#include "core.h"
#include "common_kernel.h"
#include "copy_kernel.h"
#include "enqueue.h"
#include "reduce_kernel.h"
/* HIERARCHY
*
* The data is split into CHUNKS, and each CHUNK is split into NUM_SUBCHUNKS
* SUBCHUNKS, where each SUBCHUNK is an independent, complete reduction. Each
* GPU has a buffer that can fit an entire CHUNK, so that all SUBCHUNKS can be
* processed without checking that the buffer on the receiving GPU is empty. A
* SUBCHUNK is split into NUM_GPUS SLICES and each GPU works on a different
* SLICE at the same time. Before moving on the the next SLICE in the reduction
* algorithm, the GPU has to check whether it has received the data from the
* previous GPU it needs for this SLICE. To hide the latency of this
* communication, each GPU processes all the SLICES of all the SUBCHUNKS in
* sequence before moving on to the next SLICE. Each SLICE is split into a
* certain number of UNROLLS (determined by the buffer size) and each thread
* performs UNROLL_COUNT single-data-element operations inside an UNROLL. As the
* name suggests, the UNROLL_COUNT operations within an UNROLL are unrolled.
*/
// Number of threads used to perform copies, etc. Must be multiple of 32.
// An additional thread is used to handle threadfences, so the CUDA blocks
// have dimension NUM_THREADS+1.
#define NUM_THREADS 256
// Each thread unrolls the innermost loop of the copy or reduction operations
// to this many single-data-element instructions
#define UNROLL_COUNT 8
#define UNROLL_SIZE (UNROLL_COUNT * NUM_THREADS)
// To hide the latency associated with the synchronization between different
// subchunks, we interleave the independent subchunks so that more data can be
// transferred while the sync is in progress. This is the number of subchunks
// that are active at the same time
#define NUM_SUBCHUNKS 2
// If this is called with STEP, it means that we just finished processing the
// data for step STEP on this GPU, which is the data required on the next GPU
// for step STEP + 1, so we signal the next GPU that its data for step STEP + 1
// is available. This is called by one particular consumer warp and so we select
// the first thread in the warp to set the flag.
#define SIGNAL_NEW_DATA_AVAILABLE(chunk, subchunk, step) \
do { \
__threadfence_system(); \
args.NextNewDataAvailableFlag[0] = \
NUM_SUBCHUNKS*((chunk) * (2 * args.NumGPUs - 2) + (step) + 1)+subchunk; \
} while (0)
// This is called by all producer threads, but only thread 0 spins on the flag,
#define WAIT_FOR_NEW_DATA(chunk, subchunk, step) \
do { \
if (tid == 0) { \
Wait([=] { \
return ((volatile int *)args.ThisNewDataAvailableFlag)[0] >= \
2*((chunk) * (2 * args.NumGPUs - 2) + (step))+subchunk; \
}); \
} \
BAR(sync, 1, NUM_THREADS); \
} while (0)
#define SIGNAL_CHUNK_DONE(chunk, subchunk) \
do { \
args.PrevChunkDoneFlag[0] = NUM_SUBCHUNKS*(chunk) + subchunk + 1; \
} while (0)
#define WAIT_FOR_CHUNK(chunk, subchunk) \
do { \
if (tid == 0) { \
Wait([=] { \
return ((volatile int *)args.ThisChunkDoneFlag)[0] >= \
NUM_SUBCHUNKS*(chunk) + subchunk + 1 - NUM_SUBCHUNKS; \
}); \
} \
BAR(sync, 1, NUM_THREADS); \
} while (0)
__device__ inline void getSliceSizeAndOffset(int *size, int *offset, int slice,
int numSlices, int numBigSlices, int numSmallSlices, int bigSliceN,
int smallSliceN, int lastSliceN) {
if (slice < numBigSlices) {
*size = bigSliceN;
*offset = slice * bigSliceN;
} else {
*size = (slice < numBigSlices + numSmallSlices) ? smallSliceN
: ((slice == numSlices - 1) ? lastSliceN : 0);
*offset = numBigSlices * bigSliceN + (slice - numBigSlices) * smallSliceN;
}
}
template<typename T>
struct AllReduceKernelArgs {
// general parameters
int ThisId;
int NumGPUs;
int N;
// some pre-computed sizes
int SliceSize;
int ChunkSize;
int NumChunks;
T ** ThisPtrToNextOutput;
T ** PrevPtrToThisOutput;
// local and remote input, output, and buffer
const T * __restrict__ ThisInput;
volatile T * __restrict__ ThisOutput;
volatile T * __restrict__ ThisBuffer;
volatile T * __restrict__ NextBuffer;
// local and remote flags
volatile int * __restrict__ ThisNewDataAvailableFlag;
volatile int * __restrict__ NextNewDataAvailableFlag;
volatile int * __restrict__ ThisChunkDoneFlag;
volatile int * __restrict__ PrevChunkDoneFlag;
};
__shared__ volatile void * nextOutput;
template<int THREADS, int UNROLL, class FUNC, bool PUSHRECV, typename T>
__launch_bounds__(THREADS+WARP_SIZE, 1)
__global__ void AllReduceKernel(const AllReduceKernelArgs<T> args) {
if (args.N == 0) return;
const int tid = threadIdx.x;
// First wait for args.PrevPtrToThisOutput to become nullptr to ensure that
// the previous GPU is done with a previous collective operation.
if (tid == 0) {
Wait([=] {
return *((T * volatile *)args.PrevPtrToThisOutput) == nullptr;
});
*((T * volatile *)args.PrevPtrToThisOutput) = (T*)args.ThisOutput;
Wait([=] {
return *((T * volatile *)args.ThisPtrToNextOutput) != nullptr;
});
if (PUSHRECV)
nextOutput =
*((volatile void * volatile *)args.ThisPtrToNextOutput);
}
__syncthreads();
for (int chunk = 0; chunk < args.NumChunks; ++chunk) {
// calculate slice size. for all chunks except (possibly) the last one,
// this will just be args.SliceSize. For the last one, it may be smaller
int bigSliceN = args.SliceSize;
int smallSliceN = 0;
int lastSliceN = 0;
int numSlices = args.NumGPUs * NUM_SUBCHUNKS;
int numBigSlices = numSlices;
int numSmallSlices = 0;
// last chunk
if ((chunk + 1 == args.NumChunks) && (args.N % args.ChunkSize > 0))
CalcLastChunk<THREADS, UNROLL, T>(&bigSliceN, &smallSliceN, &lastSliceN,
&numSlices, &numBigSlices, &numSmallSlices, args.N, args.NumChunks,
args.ChunkSize);
// this offset is only applied to Data pointers, not to Buffer pointers,
// since we only have one buffer per chunk
int chunkOffset = chunk * args.ChunkSize;
/////////////// begin AllReduce steps ///////////////
// step 0: push data to next GPU
int step = 0;
int slice = args.ThisId;
int offset;
int sliceSize;
if (tid < THREADS) {
for(int s=0; s<NUM_SUBCHUNKS; ++s) {
if (s > 0) { slice += args.NumGPUs; }
getSliceSizeAndOffset(&sliceSize, &offset, slice, numSlices,
numBigSlices, numSmallSlices, bigSliceN, smallSliceN, lastSliceN);
if (!PUSHRECV && chunk > 0) {
WAIT_FOR_CHUNK(chunk, s);
}
Copy<UNROLL, THREADS>(
args.NextBuffer + offset,
args.ThisInput + chunkOffset + offset,
sliceSize);
__syncthreads();
}
} else { // is consumer thread
for(int s=0; s<NUM_SUBCHUNKS; ++s) {
__syncthreads();
SIGNAL_NEW_DATA_AVAILABLE(chunk, s, step);
}
}
// steps j with 1 <= j < k - 1, where k = number of GPUs:
// reduce and copy to next GPU
for (step = 1; step < args.NumGPUs - 1; ++step) {
if (tid < THREADS) {
slice = (args.NumGPUs + slice - 1) % args.NumGPUs;
for(int s=0; s<NUM_SUBCHUNKS; ++s) {
if (s > 0) { slice += args.NumGPUs; }
getSliceSizeAndOffset(&sliceSize, &offset, slice, numSlices,
numBigSlices, numSmallSlices, bigSliceN, smallSliceN, lastSliceN);
WAIT_FOR_NEW_DATA(chunk, s, step);
Reduce<UNROLL, THREADS, FUNC>(
args.NextBuffer + offset,
args.ThisBuffer + offset,
args.ThisInput + chunkOffset + offset,
sliceSize);
__syncthreads();
}
} else {
for(int s=0; s<NUM_SUBCHUNKS; ++s) {
__syncthreads();
SIGNAL_NEW_DATA_AVAILABLE(chunk, s, step);
}
}
}
// step k - 1: reduce this buffer and data, which will produce the final
// result that we store in this data and push to the next GPU
step = args.NumGPUs - 1;
if (tid < THREADS) {
slice = (args.NumGPUs + slice - 1) % args.NumGPUs;
for(int s=0; s<NUM_SUBCHUNKS; ++s) {
if (s > 0) { slice += args.NumGPUs; }
getSliceSizeAndOffset(&sliceSize, &offset, slice, numSlices,
numBigSlices, numSmallSlices, bigSliceN, smallSliceN, lastSliceN);
WAIT_FOR_NEW_DATA(chunk, s, step);
if (PUSHRECV) {
ReduceAndCopy<UNROLL, THREADS, FUNC>(
(volatile T *)nextOutput + chunkOffset + offset,
args.ThisOutput + chunkOffset + offset,
args.ThisBuffer + offset,
args.ThisInput + chunkOffset + offset,
sliceSize);
} else {
ReduceAndCopy<UNROLL, THREADS, FUNC>(
args.NextBuffer + offset,
args.ThisOutput + chunkOffset + offset,
args.ThisBuffer + offset,
args.ThisInput + chunkOffset + offset,
sliceSize);
}
__syncthreads();
}
} else {
for(int s=0; s<NUM_SUBCHUNKS; ++s) {
__syncthreads();
SIGNAL_NEW_DATA_AVAILABLE(chunk, s, step);
}
}
// steps j with k <= j < 2*k-2: copy result to next GPU
for (step = args.NumGPUs; step < 2 * args.NumGPUs - 2; ++step) {
if (tid < THREADS) {
slice = (args.NumGPUs + slice - 1) % args.NumGPUs;
for(int s=0; s<NUM_SUBCHUNKS; ++s) {
if (s > 0) { slice += args.NumGPUs; }
getSliceSizeAndOffset(&sliceSize, &offset, slice, numSlices,
numBigSlices, numSmallSlices, bigSliceN, smallSliceN, lastSliceN);
WAIT_FOR_NEW_DATA(chunk, s, step);
if( PUSHRECV ) {
Copy<UNROLL, THREADS>(
(volatile T *)nextOutput + chunkOffset + offset,
args.ThisOutput + chunkOffset + offset,
sliceSize);
} else {
DoubleCopy<UNROLL, THREADS>(
args.NextBuffer + offset,
args.ThisOutput + chunkOffset + offset,
args.ThisBuffer + offset,
sliceSize);
}
__syncthreads();
}
} else {
for(int s=0; s<NUM_SUBCHUNKS; ++s) {
__syncthreads();
SIGNAL_NEW_DATA_AVAILABLE(chunk, s, step);
}
}
}
if (!PUSHRECV) {
// Make final copy from buffer to dest.
if (tid < THREADS) {
slice = (args.NumGPUs + slice - 1) % args.NumGPUs;
for(int s=0; s<NUM_SUBCHUNKS; ++s) {
if (s > 0) { slice += args.NumGPUs; }
getSliceSizeAndOffset(&sliceSize, &offset, slice, numSlices,
numBigSlices, numSmallSlices, bigSliceN, smallSliceN, lastSliceN);
WAIT_FOR_NEW_DATA(chunk, s, step);
// Here we need to copy from buffer to this output.
Copy<UNROLL, THREADS>(
args.ThisOutput + chunkOffset + offset,
args.ThisBuffer + offset,
sliceSize);
__syncthreads();
}
} else {
for(int s=0; s<NUM_SUBCHUNKS; ++s) {
__syncthreads();
if(chunk+1 < args.NumChunks) {
SIGNAL_CHUNK_DONE(chunk, s);
}
}
}
}
}
// wait for the last data to be pushed to us
if (tid < THREADS) {
if(PUSHRECV) {
WAIT_FOR_NEW_DATA(args.NumChunks, NUM_SUBCHUNKS-1, 0);
}
if (tid == 0) {
args.ThisNewDataAvailableFlag[0] = 0;
if(!PUSHRECV) {
args.ThisChunkDoneFlag[0] = 0;
}
*args.ThisPtrToNextOutput = nullptr;
}
}
}
template<class FUNC, typename T>
ncclResult_t ncclAllReduceWithTypeAndFunc(const void* sendbuff, void* recvbuff,
const int count, ncclComm* comm, cudaStream_t stream) {
if (count == 0)
return ncclSuccess;
int index = comm->ncclId;
// There is one slice per GPU, so a slice can be at most bufferN / numGPUs,
// where bufferN is the number of elements of type T that fit into the buffer.
// For efficiency, we want the slice size to be a multiple of UNROLL_SIZE
int bufferN = comm->buffSize / sizeof(T);
int bufferNPerSlice = bufferN / (NUM_SUBCHUNKS * comm->nDev);
int sliceSize = (bufferNPerSlice / UNROLL_SIZE) * UNROLL_SIZE;
int nextId = (index + 1) % comm->nDev;
int prevId = (index + comm->nDev - 1) % comm->nDev;
AllReduceKernelArgs<T> args;
args.ThisId = index;
args.NumGPUs = comm->nDev;
args.N = count;
args.SliceSize = sliceSize;
int subchunkSize = comm->nDev * args.SliceSize;
args.ChunkSize = NUM_SUBCHUNKS * subchunkSize;
// avoid a case where we have one or more big chunks and one tiny one
int remainder = args.N % args.ChunkSize;
if ((args.N > args.ChunkSize) && (remainder > 0) &&
(args.N < 5 * args.ChunkSize) && (2 * remainder < args.ChunkSize)) {
args.SliceSize /= 2;
int subchunkSize = comm->nDev * args.SliceSize;
args.ChunkSize = NUM_SUBCHUNKS * subchunkSize;
// round down so we end up with a big last chunk
args.NumChunks = args.N / args.ChunkSize;
} else {
// round up
args.NumChunks = (args.N + args.ChunkSize - 1) / args.ChunkSize;
}
args.ThisPtrToNextOutput = (T**)&(comm->local[nextId]->recvPtrs[0]);
args.PrevPtrToThisOutput = (T**)&(comm->remote[prevId]->recvPtrs[0]);
args.ThisInput = (const T*)sendbuff;
args.ThisOutput = (volatile T*)recvbuff;
args.ThisBuffer = (volatile T*)comm->local[prevId]->buff;
args.NextBuffer = (volatile T*)comm->remote[nextId]->buff;
args.ThisNewDataAvailableFlag = comm->local[prevId]->flags;
args.NextNewDataAvailableFlag = comm->remote[nextId]->flags;
args.ThisChunkDoneFlag = comm->local[nextId]->flags + 1;
args.PrevChunkDoneFlag = comm->remote[prevId]->flags + 1;
if( comm->useRemoteRecv ) {
AllReduceKernel<NUM_THREADS, UNROLL_COUNT, FUNC, true, T>
<<<1, NUM_THREADS + 1, 0, stream>>>(args);
} else {
AllReduceKernel<NUM_THREADS, UNROLL_COUNT, FUNC, false, T>
<<<1, NUM_THREADS + 1, 0, stream>>>(args);
}
return ncclSuccess;
}
template<typename T>
ncclResult_t ncclAllReduceWithType(const void* sendbuff,
void* recvbuff, int count, ncclRedOp_t op, ncclComm* comm, cudaStream_t stream) {
switch (op) {
case ncclSum:
return ncclAllReduceWithTypeAndFunc<FuncSum<T>, T>(
sendbuff, recvbuff, count, comm, stream);
case ncclProd:
return ncclAllReduceWithTypeAndFunc<FuncProd<T>, T>(
sendbuff, recvbuff, count, comm, stream);
case ncclMax:
return ncclAllReduceWithTypeAndFunc<FuncMax<T>, T>(
sendbuff, recvbuff, count, comm, stream);
case ncclMin:
return ncclAllReduceWithTypeAndFunc<FuncMin<T>, T>(
sendbuff, recvbuff, count, comm, stream);
}
return ncclInvalidOperation;
}
class AllReduceFunctor {
public:
ncclResult_t operator()(const void* sendbuff, void* recvbuff,
int count, ncclDataType_t datatype, ncclRedOp_t op, int /*root*/,
ncclComm* comm, cudaStream_t stream) {
switch (datatype) {
case ncclChar:
return ncclAllReduceWithType<char>(sendbuff, recvbuff, count, op,
comm, stream);
case ncclInt:
return ncclAllReduceWithType<int>(sendbuff, recvbuff, count, op,
comm, stream);
#if CUDART_VERSION >= 7050
case ncclHalf:
return ncclAllReduceWithType<half>(sendbuff, recvbuff, count, op,
comm, stream);
#endif
case ncclFloat:
return ncclAllReduceWithType<float>(sendbuff, recvbuff, count, op,
comm, stream);
case ncclDouble:
return ncclAllReduceWithType<double>(sendbuff, recvbuff, count, op,
comm, stream);
}
return ncclInvalidType;
}
};
extern "C" DSOGLOBAL
ncclResult_t ncclAllReduce(const void* sendbuff, void* recvbuff, int count,
ncclDataType_t datatype, ncclRedOp_t op, ncclComm_t comm, cudaStream_t stream) {
return enqueue(AllReduceFunctor(), sendbuff, recvbuff, count, datatype, op, 0,
comm, stream);
}

299
src/all_reduce_test.cu Normal file
View File

@ -0,0 +1,299 @@
/*************************************************************************
* Copyright (c) 2015, NVIDIA CORPORATION. All rights reserved.
*
* Redistribution and use in source and binary forms, with or without
* modification, are permitted provided that the following conditions
* are met:
* * Redistributions of source code must retain the above copyright
* notice, this list of conditions and the following disclaimer.
* * Redistributions in binary form must reproduce the above copyright
* notice, this list of conditions and the following disclaimer in the
* documentation and/or other materials provided with the distribution.
* * Neither the name of NVIDIA CORPORATION nor the names of its
* contributors may be used to endorse or promote products derived
* from this software without specific prior written permission.
*
* THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS ``AS IS'' AND ANY
* EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
* IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR
* PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR
* CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL,
* EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO,
* PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR
* PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY
* OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
* (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
* OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
************************************************************************/
#include <chrono>
#include <cstdio>
#include <cstdlib>
#include <string>
#include <vector>
#include "nccl.h"
#include "test_utilities.h"
#include <nvToolsExt.h>
int csv = false;
template<typename T>
void RunTest(T** sendbuff, T** recvbuff, const int N, const ncclDataType_t type,
const ncclRedOp_t op, ncclComm_t* comms, const std::vector<int>& dList) {
// initialize data
T* buffer = (T*)malloc(N * sizeof(T));
T* result = (T*)malloc(N * sizeof(T));
memset(buffer, 0, N * sizeof(T));
memset(result, 0, N * sizeof(T));
int nDev = 0;
ncclCommCount(comms[0], &nDev);
cudaStream_t* s = (cudaStream_t*)malloc(sizeof(cudaStream_t)*nDev);
for (int i = 0; i < nDev; ++i) {
CUDACHECK(cudaSetDevice(dList[i]));
CUDACHECK(cudaStreamCreate(s+i));
CUDACHECK(cudaMemset(recvbuff[i], 0, N * sizeof(T)));
Randomize(sendbuff[i], N, i);
if(i == 0) {
CUDACHECK(cudaMemcpy(result, sendbuff[i], N*sizeof(T), cudaMemcpyDeviceToHost));
} else {
Accumulate<T>(result, sendbuff[i], N, op);
}
}
// warm up GPU
for (int i = 0; i < nDev; ++i) {
CUDACHECK(cudaSetDevice(dList[i]));
ncclAllReduce((const void*)sendbuff[i], (void*)recvbuff[i], std::min(N, 1024 * 1024), type, op, comms[i], s[i]);
}
for (int i = 0; i < nDev; ++i) {
CUDACHECK(cudaSetDevice(dList[i]));
CUDACHECK(cudaStreamSynchronize(s[i]));
}
// for (int n = 0; n <= N; n = (n > 0) ? n << 1 : 1)
{
int n = N;
printf((csv) ? "%i,%i,%s,%s," : "%12i %12i %6s %6s",
(int) (n * sizeof(T)), n, TypeName(type).c_str(),
OperationName(op).c_str());
// do out-of-place reduction first
nvtxRangePushA("out of place");
auto start = std::chrono::high_resolution_clock::now();
//for (int i=0; i<100; i++) {
for (int i = 0; i < nDev; ++i) {
CUDACHECK(cudaSetDevice(dList[i]));
ncclAllReduce((const void*)sendbuff[i], (void*)recvbuff[i], n, type, op,
comms[i], s[i]);
}
//}
for (int i = 0; i < nDev; ++i) {
CUDACHECK(cudaSetDevice(dList[i]));
CUDACHECK(cudaStreamSynchronize(s[i]));
}
auto stop = std::chrono::high_resolution_clock::now();
nvtxRangePop();
nvtxRangePushA("out of place bookkeeping");
double elapsedSec =
std::chrono::duration_cast<std::chrono::duration<double>>(
stop - start).count(); // / 100.0;
double algbw = (double)(n * sizeof(T)) / 1.0E9 / elapsedSec;
double busbw = algbw * (double)(2 * nDev - 2) / (double)nDev;
double maxDelta = 0.0;
for (int i = 0; i < nDev; ++i) {
CUDACHECK(cudaSetDevice(dList[i]));
double tmpDelta = CheckDelta<T>(recvbuff[i], result, N);
maxDelta = std::max(tmpDelta, maxDelta);
}
printf((csv)?"%f,%f,%f,%le,":" %7.3f %5.2f %5.2f %7.0le",
elapsedSec * 1.0E3, algbw, busbw, maxDelta);
nvtxRangePop();
}
// for (int n = 0; n <= N; n = (n > 0) ? n << 1 : 1)
{
int n = N;
// now do in-place reduction
nvtxRangePushA("in place");
auto start = std::chrono::high_resolution_clock::now();
//for (int i=0; i<100; i++) {
for (int i = 0; i < nDev; ++i) {
CUDACHECK(cudaSetDevice(dList[i]));
ncclAllReduce((const void*)sendbuff[i], (void*)sendbuff[i], n, type, op,
comms[i], s[i]);
}
//}
for (int i = 0; i < nDev; ++i) {
CUDACHECK(cudaSetDevice(dList[i]));
CUDACHECK(cudaStreamSynchronize(s[i]));
}
auto stop = std::chrono::high_resolution_clock::now();
nvtxRangePop();
nvtxRangePushA("in place bookkeeping");
double elapsedSec =
std::chrono::duration_cast<std::chrono::duration<double>>(
stop - start).count(); // / 100.0;
double algbw = (double)(n * sizeof(T)) / 1.0E9 / elapsedSec;
double busbw = algbw * (double)(2 * nDev - 2) / (double)nDev;
double maxDelta = 0.0;
for (int i = 0; i < nDev; ++i) {
CUDACHECK(cudaSetDevice(dList[i]));
double tmpDelta = CheckDelta<T>(sendbuff[i], result, N);
maxDelta = std::max(tmpDelta, maxDelta);
}
printf((csv)?"%f,%f,%f,%le,":" %7.3f %5.2f %5.2f %7.0le\n",
elapsedSec * 1.0E3, algbw, busbw, maxDelta);
nvtxRangePop();
}
for (int i = 0; i < nDev; ++i) {
CUDACHECK(cudaSetDevice(dList[i]));
CUDACHECK(cudaStreamDestroy(s[i]));
}
free(s);
free(buffer);
free(result);
}
template<typename T>
void RunTests(const int N, const ncclDataType_t type, ncclComm_t* comms,
const std::vector<int>& dList) {
int nDev = 0;
ncclCommCount(comms[0], &nDev);
T** sendbuff = (T**)malloc(nDev * sizeof(T*));
T** recvbuff = (T**)malloc(nDev * sizeof(T*));
for (int i = 0; i < nDev; ++i) {
CUDACHECK(cudaSetDevice(dList[i]));
CUDACHECK(cudaMalloc(sendbuff + i, N * sizeof(T)));
CUDACHECK(cudaMalloc(recvbuff + i, N * sizeof(T)));
}
for (ncclRedOp_t op : { ncclSum, ncclProd, ncclMax, ncclMin }) {
// for (ncclRedOp_t op : { ncclSum }) {
RunTest<T>(sendbuff, recvbuff, N, type, op, comms, dList);
}
for (int i = 0; i < nDev; ++i) {
CUDACHECK(cudaSetDevice(dList[i]));
CUDACHECK(cudaFree(sendbuff[i]));
CUDACHECK(cudaFree(recvbuff[i]));
}
free(sendbuff);
free(recvbuff);
}
void usage() {
printf("Tests nccl AllReduce with user supplied arguments.\n"
" Usage: all_reduce_test <data size in bytes> [number of GPUs] "
"[GPU 0] [GPU 1] ...\n\n");
}
int main(int argc, char* argv[]) {
int nVis = 0;
CUDACHECK(cudaGetDeviceCount(&nVis));
int N = 0;
if (argc > 1) {
int t = sscanf(argv[1], "%d", &N);
if (t == 0) {
printf("Error: %s is not an integer!\n\n", argv[1]);
usage();
exit(EXIT_FAILURE);
}
} else {
printf("Error: must specify at least data size in bytes!\n\n");
usage();
exit(EXIT_FAILURE);
}
int nDev = nVis;
if (argc > 2) {
int t = sscanf(argv[2], "%d", &nDev);
if (t == 0) {
printf("Error: %s is not an integer!\n\n", argv[1]);
usage();
exit(EXIT_FAILURE);
}
}
std::vector<int> dList(nDev);
for (int i = 0; i < nDev; ++i)
dList[i] = i % nVis;
if (argc > 3) {
if (argc - 3 != nDev) {
printf("Error: insufficient number of GPUs in list\n\n");
usage();
exit(EXIT_FAILURE);
}
for (int i = 0; i < nDev; ++i) {
int t = sscanf(argv[3 + i], "%d", dList.data() + i);
if (t == 0) {
printf("Error: %s is not an integer!\n\n", argv[2 + i]);
usage();
exit(EXIT_FAILURE);
}
}
}
ncclComm_t* comms = (ncclComm_t*)malloc(sizeof(ncclComm_t)*nDev);
ncclCommInitAll(comms, nDev, dList.data());
if (!csv) {
printf("# Using devices\n");
for (int g = 0; g < nDev; ++g) {
int cudaDev;
int rank;
cudaDeviceProp prop;
ncclCommCuDevice(comms[g], &cudaDev);
ncclCommUserRank(comms[g], &rank);
CUDACHECK(cudaGetDeviceProperties(&prop, cudaDev));
printf("# Rank %2d uses device %2d [0x%02x] %s\n", rank, cudaDev,
prop.pciBusID, prop.name);
}
printf("\n");
printf("# %10s %12s %6s %6s out-of-place in-place\n", "", "", "", "");
printf("# %10s %12s %6s %6s %7s %5s %5s %7s %7s %5s %5s %7s\n", "bytes", "N", "type", "op",
"time", "algbw", "busbw", "res", "time", "algbw", "busbw", "res");
}
else {
printf("B,N,type,op,oop_time,oop_algbw,oop_busbw,oop_res,ip_time,ip_algbw,ip_busbw,ip_res\n");
}
RunTests<char>(N / sizeof(char), ncclChar, comms, dList);
RunTests<int>(N / sizeof(int), ncclInt, comms, dList);
#if CUDART_VERSION >= 7050
RunTests<half>(N / sizeof(half), ncclHalf, comms, dList);
#endif
RunTests<float>(N / sizeof(float), ncclFloat, comms, dList);
RunTests<double>(N / sizeof(double), ncclDouble, comms, dList);
printf("\n");
for(int i=0; i<nDev; ++i)
ncclCommDestroy(comms[i]);
free(comms);
exit(EXIT_SUCCESS);
}

414
src/broadcast.cu Normal file
View File

@ -0,0 +1,414 @@
/*************************************************************************
* Copyright (c) 2015, NVIDIA CORPORATION. All rights reserved.
*
* Redistribution and use in source and binary forms, with or without
* modification, are permitted provided that the following conditions
* are met:
* * Redistributions of source code must retain the above copyright
* notice, this list of conditions and the following disclaimer.
* * Redistributions in binary form must reproduce the above copyright
* notice, this list of conditions and the following disclaimer in the
* documentation and/or other materials provided with the distribution.
* * Neither the name of NVIDIA CORPORATION nor the names of its
* contributors may be used to endorse or promote products derived
* from this software without specific prior written permission.
*
* THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS ``AS IS'' AND ANY
* EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
* IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR
* PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR
* CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL,
* EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO,
* PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR
* PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY
* OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
* (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
* OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
************************************************************************/
#include <algorithm>
#include <nvToolsExt.h>
#include "core.h"
#include "common_kernel.h"
#include "copy_kernel.h"
#include "enqueue.h"
/* HIERARCHY
*
* The data is split into CHUNKS, and each CHUNK is split into NUM_SUBCHUNKS
* SUBCHUNKS, where each SUBCHUNK is processed independently. A SUBCHUNK is
* split into numUnroll UNROLLS and each thread performs UNROLL_COUNT
* single-data-element operations inside an UNROLL. As the name suggests, the
* UNROLL_COUNT operations within an UNROLL are unrolled.
*/
// Number of threads used to perform copies, etc. Must be multiple of 32.
// An additional thread is used to handle threadfences, so the CUDA blocks
// have dimension NUM_THREADS+1.
#define NUM_THREADS 256
// Each thread unrolls the innermost loop of the copy or reduction operations
// to this many single-data-element instructions
#define UNROLL_COUNT 8
#define UNROLL_SIZE (UNROLL_COUNT * NUM_THREADS)
// To hide the latency associated with the synchronization between different
// subchunks, we interleave the independent subchunks so that more data can be
// transferred while the sync is in progress. This is the number of subchunks
// that are active at the same time
#define NUM_SUBCHUNKS 4
// if this is called with CHUNK, it means that we just finished pushing the data
// of chunk CHUNK to the next GPU, so it can proceed with CHUNK
// We add 1 to chunk so that the initial flag of 0 doesn't allow the non-root
// GPUs to proceed before the flag is incremented from the upstream GPU. This
// is called by one particular consumer warp and so we select the first thread
// in the warp to set the flag.
#define SIGNAL_NEW_DATA_AVAILABLE(chunk, subchunk) \
do { \
__threadfence_system(); \
args.NextNewDataAvailableFlag[0] = NUM_SUBCHUNKS*(chunk) + subchunk + 1; \
} while (0)
// This is called by all producer threads, but only thread 0 spins on the flag,
#define WAIT_FOR_NEW_DATA(chunk, subchunk) \
do { \
if (tid == 0) { \
Wait([=] { \
return ((volatile int *)args.ThisNewDataAvailableFlag)[0] >= \
NUM_SUBCHUNKS*(chunk) + subchunk + 1; \
}); \
} \
BAR(sync, 1, NUM_THREADS); \
} while (0)
// If this is called with CHUNK, it means that this GPU has just finished
// processing the chunk CHUNK and so the previous GPU can start with CHUNK + 1
#define SIGNAL_CHUNK_DONE(chunk, subchunk) \
do { \
args.PrevChunkDoneFlag[0] = NUM_SUBCHUNKS*(chunk) + subchunk + 1; \
} while (0)
// This is called by all producer threads, but only thread 0 spins on the flag,
// all threads synchronize after thread 0 is done spinning.
#define WAIT_FOR_CHUNK(chunk, subchunk) \
do { \
if (tid == 0) { \
Wait([=] { \
return ((volatile int *)args.ThisChunkDoneFlag)[0] >= \
NUM_SUBCHUNKS*(chunk) + subchunk + 1 - NUM_SUBCHUNKS; \
}); \
} \
BAR(sync, 1, NUM_THREADS); \
} while (0)
// This is called by all producer threads, but only thread 0 spins on the flag,
// all threads synchronize after thread 0 is done spinning.
#define WAIT_FOR_NEW_DATA_AND_CHUNK(chunk, subchunk) \
do { \
if (tid == 0) { \
Wait([=] { \
bool newDataAvailable = \
((volatile int *)args.ThisNewDataAvailableFlag)[0] >= \
NUM_SUBCHUNKS*(chunk) + subchunk + 1; \
bool chunkDone = \
((volatile int *)args.ThisChunkDoneFlag)[0] >= \
NUM_SUBCHUNKS*(chunk)+subchunk + 1 - NUM_SUBCHUNKS; \
return newDataAvailable && chunkDone; \
}); \
} \
BAR(sync, 1, NUM_THREADS); \
} while (0)
__device__ inline void getSliceSizeAndOffset(int *size, int *offset, int slice,
int numSlices, int numBigSlices, int numSmallSlices, int bigSliceN,
int smallSliceN, int lastSliceN) {
if (slice < numBigSlices) {
*size = bigSliceN;
*offset = slice * bigSliceN;
} else {
*size = (slice < numBigSlices + numSmallSlices) ? smallSliceN
: ((slice == numSlices - 1) ? lastSliceN : 0);
*offset = numBigSlices * bigSliceN + (slice - numBigSlices) * smallSliceN;
}
// if (threadIdx.x == 0)
// printf("[size=%d] [offset=%d] slice=%d numSlices=%d "
// "numBigSlices=%d numSmallSlices=%d bigSliceN=%d smallSliceN=%d "
// "lastSliceN=%d\n", *size, *offset, slice, numSlices, numBigSlices,
// numSmallSlices, bigSliceN, smallSliceN, lastSliceN);
}
template<typename T>
struct BroadcastKernelArgs {
// general parameters
int ThisId;
int N;
// some pre-computed sizes
int SliceSize;
int ChunkSize;
int NumChunks;
int BufferSliceStride;
T ** ThisPtrToNextData;
T ** PrevPtrToThisData;
// local and remote data
T * __restrict__ ThisData;
volatile T * __restrict__ ThisBuffer;
volatile T * __restrict__ NextBuffer;
// local and remote flags
volatile int * __restrict__ ThisNewDataAvailableFlag;
volatile int * __restrict__ NextNewDataAvailableFlag;
volatile int * __restrict__ ThisChunkDoneFlag;
volatile int * __restrict__ PrevChunkDoneFlag;
};
__shared__ volatile void * nextData;
enum BcastRole {ROOT=0, MIDDLE=1, END=2};
template<int THREADS, int UNROLL, bool PUSHRECV, int ROLE, typename T>
__global__ void BroadcastKernel(const BroadcastKernelArgs<T> args) {
if (args.N == 0) return;
int tid = threadIdx.x;
// First wait for args.PrevPtrToThisOutput to become nullptr to ensure that
// the previous GPU is done with a previous collective operation.
if (tid == 0) {
if (ROLE != ROOT) {
Wait([=] {
return *((T * volatile *)args.PrevPtrToThisData) == nullptr; // Wait for previous processor to be done
});
*((T * volatile *)args.PrevPtrToThisData) = (T*)args.ThisData; // Tell Previous I'm starting
}
if (ROLE != END) {
Wait([=] {
return *((T * volatile *)args.ThisPtrToNextData) != nullptr; // Wait till I've been told next started
});
if (PUSHRECV)
nextData = *((volatile void * volatile *)args.ThisPtrToNextData); // Grab next's pointer if needed.
}
}
__syncthreads();
for (int chunk = 0; chunk < args.NumChunks; ++chunk) {
// calculate slice size. for all chunks except (possibly) the last one,
// this will just be args.SliceSize. For the last one, it may be smaller
int bigSliceN = args.SliceSize;
int smallSliceN = 0;
int lastSliceN = 0;
int numSlices = NUM_SUBCHUNKS;
int numBigSlices = numSlices;
int numSmallSlices = 0;
// last chunk
if ((chunk + 1 == args.NumChunks) && (args.N % args.ChunkSize > 0))
CalcLastChunk<THREADS, UNROLL, T>(&bigSliceN, &smallSliceN, &lastSliceN,
&numSlices, &numBigSlices, &numSmallSlices, args.N, args.NumChunks,
args.ChunkSize);
// this offset is only applied to Data pointers, not to Buffer pointers,
// since we only have one buffer per chunk
int chunkOffset = chunk * args.ChunkSize;
int offset;
int sliceSize;
if (tid < THREADS) {
for(int s=0; s<NUM_SUBCHUNKS; ++s) {
getSliceSizeAndOffset(&sliceSize, &offset, s, numSlices,
numBigSlices, numSmallSlices, bigSliceN, smallSliceN, lastSliceN);
if (PUSHRECV) {
if (ROLE != ROOT)
WAIT_FOR_NEW_DATA(chunk, s);
if (ROLE != END)
Copy<UNROLL, THREADS>(
(volatile T *)nextData + chunkOffset + offset,
args.ThisData + chunkOffset + offset,
sliceSize);
} else { // PUSH2BUFF
if (ROLE == ROOT) {
WAIT_FOR_CHUNK(chunk, s);
Copy<UNROLL, THREADS>(
args.NextBuffer + (s * args.BufferSliceStride),
args.ThisData + chunkOffset + offset,
sliceSize);
} else if (ROLE == MIDDLE) {
WAIT_FOR_NEW_DATA_AND_CHUNK(chunk, s);
DoubleCopy<UNROLL, THREADS>(
args.NextBuffer + (s * args.BufferSliceStride),
args.ThisData + chunkOffset + offset,
args.ThisBuffer + (s * args.BufferSliceStride),
sliceSize);
} else { // ROLE == END
WAIT_FOR_NEW_DATA(chunk, s);
Copy<UNROLL, THREADS>(
args.ThisData + chunkOffset + offset,
args.ThisBuffer + (s * args.BufferSliceStride),
sliceSize);
}
}
__syncthreads();
}
} else { // Consumer thread
for(int s=0; s<NUM_SUBCHUNKS; ++s) {
__syncthreads();
if (ROLE != END)
SIGNAL_NEW_DATA_AVAILABLE(chunk, s);
// signal chunk done if we don't push into the receive buffer and this
// is no the last chunk and this is not root
if ((!PUSHRECV) && (ROLE != ROOT) && (chunk + 1 < args.NumChunks)) {
SIGNAL_CHUNK_DONE(chunk, s);
}
}
}
}
// reset flags
if (tid == 0) {
args.ThisNewDataAvailableFlag[0] = 0;
args.ThisChunkDoneFlag[0] = 0;
*args.ThisPtrToNextData = nullptr;
}
}
template<typename T>
ncclResult_t ncclBcastWithType(void* buff, const int count, const int root,
ncclComm* comm, int numUnroll, cudaStream_t stream) {
if (count == 0)
return ncclSuccess;
int index = comm->ncclId;
int rootId = comm->ringFromUser[root];
int nextId = (index + 1) % comm->nDev;
int prevId = (index + comm->nDev - 1) % comm->nDev;
// There is one slice per GPU, so a slice can be at most bufferN / numGPUs,
// where bufferN is the number of elements of type T that fit into the buffer.
// For efficiency, we want the slice size to be a multiple of UNROLL_SIZE
int bufferN = comm->buffSize / sizeof(T);
// we only need buffer for k slices and k paddings
int bufferNPerSlice = bufferN / NUM_SUBCHUNKS;
int maxSliceSize = (bufferNPerSlice / UNROLL_SIZE) * UNROLL_SIZE;
BroadcastKernelArgs<T> args;
args.ThisId = index;
args.N = count;
args.SliceSize = numUnroll * UNROLL_SIZE * sizeof(PackType) / sizeof(T);
// if we don't directly push into the remote receive buffer, make sure slice
// fits into the temporary buffer
if (!comm->useRemoteRecv) {
// Larger transfers help QPI more than tag updates hurt P2P.
args.SliceSize *= 8;
}
args.SliceSize = std::min(maxSliceSize, args.SliceSize);
args.BufferSliceStride = args.SliceSize;
args.ChunkSize = NUM_SUBCHUNKS * args.SliceSize;
// avoid a case where we have one or more big chunks and one tiny one
int remainder = args.N % args.ChunkSize;
if ((args.N > args.ChunkSize) && (remainder > 0) &&
(args.N < 5 * args.ChunkSize) && (2 * remainder < args.ChunkSize)) {
args.SliceSize /= 2;
args.ChunkSize = NUM_SUBCHUNKS * args.SliceSize;
// round down so we end up with a big last chunk
args.NumChunks = args.N / args.ChunkSize;
} else {
// round up
args.NumChunks = (args.N + args.ChunkSize - 1) / args.ChunkSize;
}
// printf("sliceSize = %i, chunkSize = %i, numChunks = %i\n", args.SliceSize, args.ChunkSize, args.NumChunks);
args.ThisPtrToNextData = (T**)&(comm->local[nextId]->recvPtrs[0]);
args.PrevPtrToThisData = (T**)&(comm->remote[prevId]->recvPtrs[0]);
args.ThisData = (T*)buff;
args.ThisBuffer = (volatile T*)comm->local[prevId]->buff;
args.NextBuffer = (volatile T*)comm->remote[nextId]->buff;
// we need 2 * NUM_SUBCHUNKS flags, so use the first NUM_SUBCHUNKS flags
// to signal the next GPU that new data is available and the following
// NUM_SUBCHUNKS to signal the previous GPU that a chunk is finished
args.ThisNewDataAvailableFlag = comm->local[prevId]->flags;
args.NextNewDataAvailableFlag = comm->remote[nextId]->flags;
args.ThisChunkDoneFlag = comm->local[nextId]->flags + 1;
args.PrevChunkDoneFlag = comm->remote[prevId]->flags + 1;
if (comm->useRemoteRecv) {
if (index == (rootId + comm->nDev - 1) % comm->nDev) {
BroadcastKernel<NUM_THREADS, UNROLL_COUNT, true, END, T>
<<<1, NUM_THREADS + 1, 0, stream>>>(args);
} else if (index == rootId) {
BroadcastKernel<NUM_THREADS, UNROLL_COUNT, true, ROOT, T>
<<<1, NUM_THREADS + 1, 0, stream>>>(args);
} else {
BroadcastKernel<NUM_THREADS, UNROLL_COUNT, true, MIDDLE, T>
<<<1, NUM_THREADS + 1, 0, stream>>>(args);
}
} else {
if (index == (rootId + comm->nDev - 1) % comm->nDev) {
BroadcastKernel<NUM_THREADS, UNROLL_COUNT, false, END, T>
<<<1, NUM_THREADS + 1, 0, stream>>>(args);
} else if (index == rootId) {
BroadcastKernel<NUM_THREADS, UNROLL_COUNT, false, ROOT, T>
<<<1, NUM_THREADS + 1, 0, stream>>>(args);
} else {
BroadcastKernel<NUM_THREADS, UNROLL_COUNT, false, MIDDLE, T>
<<<1, NUM_THREADS + 1, 0, stream>>>(args);
}
}
return ncclSuccess;
}
class BroadcastFunctor {
public:
ncclResult_t operator()(const void* /*dummy sendbuff*/,
void* buff, int count, ncclDataType_t datatype, ncclRedOp_t /*dummy operation*/,
int root, ncclComm* comm, cudaStream_t stream) {
int numUnroll = 4;
switch (datatype) {
case ncclChar:
return ncclBcastWithType<char>(buff, count, root, comm, numUnroll, stream);
case ncclInt:
return ncclBcastWithType<int>(buff, count, root, comm, numUnroll, stream);
#ifdef CUDA_HAS_HALF
case ncclHalf:
return ncclBcastWithType<half>(buff, count, root, comm, numUnroll, stream);
#endif
case ncclFloat:
return ncclBcastWithType<float>(buff, count, root, comm, numUnroll, stream);
case ncclDouble:
return ncclBcastWithType<double>(buff, count, root, comm, numUnroll, stream);
}
return ncclInvalidType;
}
};
extern "C" DSOGLOBAL
ncclResult_t ncclBcast(void* buff, int count, ncclDataType_t datatype, int root,
ncclComm_t comm, cudaStream_t stream) {
return enqueue(BroadcastFunctor(), nullptr, buff, count, datatype, ncclSum,
root, comm, stream);
}

236
src/broadcast_test.cu Normal file
View File

@ -0,0 +1,236 @@
/*************************************************************************
* Copyright (c) 2015, NVIDIA CORPORATION. All rights reserved.
*
* Redistribution and use in source and binary forms, with or without
* modification, are permitted provided that the following conditions
* are met:
* * Redistributions of source code must retain the above copyright
* notice, this list of conditions and the following disclaimer.
* * Redistributions in binary form must reproduce the above copyright
* notice, this list of conditions and the following disclaimer in the
* documentation and/or other materials provided with the distribution.
* * Neither the name of NVIDIA CORPORATION nor the names of its
* contributors may be used to endorse or promote products derived
* from this software without specific prior written permission.
*
* THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS ``AS IS'' AND ANY
* EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
* IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR
* PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR
* CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL,
* EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO,
* PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR
* PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY
* OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
* (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
* OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
************************************************************************/
#include <chrono>
#include <cstdio>
#include <cstdlib>
#include <string>
#include <vector>
#include "nccl.h"
#include "test_utilities.h"
template<typename T>
void RunTest(T** buff, const int N, const ncclDataType_t type, const int root,
ncclComm_t* const comms, const std::vector<int>& dList) {
// initialize data
int nDev = 0;
ncclCommCount(comms[0], &nDev);
cudaStream_t* s = (cudaStream_t*)malloc(sizeof(cudaStream_t)*nDev);
T* buffer = (T*)malloc(N * sizeof(T));
T* result = (T*)malloc(N * sizeof(T));
memset(result, 0, N * sizeof(T));
for (int i = 0; i < nDev; ++i) {
CUDACHECK(cudaSetDevice(dList[i]));
CUDACHECK(cudaStreamCreate(s+i));
if (i == root) {
Randomize(buff[root], N, root);
CUDACHECK(cudaMemcpy(result, buff[root], N * sizeof(T),
cudaMemcpyDeviceToHost));
} else {
CUDACHECK(cudaMemset(buff[i], 0, N * sizeof(T)));
}
CUDACHECK(cudaDeviceSynchronize());
}
// warm up GPU
for (int i = 0; i < nDev; ++i) {
CUDACHECK(cudaSetDevice(dList[i]));
ncclBcast((void*)buff[i], std::min(32 * 1024, N), type, root, comms[i], s[i]);
}
for (int i = 0; i < nDev; ++i) {
CUDACHECK(cudaSetDevice(dList[i]));
CUDACHECK(cudaStreamSynchronize(s[i]));
}
// for (int n = 1; n <= N; n = n << 1)
{
int n = N;
printf("%12i %12i %6s %4i", (int)(n * sizeof(T)), n,
TypeName(type).c_str(), root);
auto start = std::chrono::high_resolution_clock::now();
for (int i = 0; i < nDev; ++i) {
CUDACHECK(cudaSetDevice(dList[i]));
ncclBcast((void*)buff[i], n, type, root, comms[i], s[i]);
}
for (int i = 0; i < nDev; ++i) {
CUDACHECK(cudaSetDevice(dList[i]));
CUDACHECK(cudaStreamSynchronize(s[i]));
}
auto stop = std::chrono::high_resolution_clock::now();
double elapsedSec =
std::chrono::duration_cast<std::chrono::duration<double>>(
stop - start).count();
double algbw = (double)(n * sizeof(T)) / 1.0E9 / elapsedSec;
double busbw = algbw;
double maxDelta = 0.0;
for (int i = 0; i < nDev; ++i) {
CUDACHECK(cudaSetDevice(dList[i]));
double tmpDelta = CheckDelta<T>(buff[i], result, n);
maxDelta = std::max(tmpDelta, maxDelta);
}
printf(" %7.3f %5.2f %5.2f %7.0le\n", elapsedSec * 1.0E3, algbw, busbw,
maxDelta);
}
for(int i=0; i < nDev; ++i) {
CUDACHECK(cudaSetDevice(dList[i]));
CUDACHECK(cudaStreamDestroy(s[i]));
}
free(s);
free(buffer);
free(result);
}
template<typename T>
void RunTests(const int N, const ncclDataType_t type, ncclComm_t* const comms,
const std::vector<int>& dList) {
int nDev = 0;
ncclCommCount(comms[0], &nDev);
T** buff = (T**)malloc(nDev * sizeof(T*));
for (int i = 0; i < nDev; ++i) {
CUDACHECK(cudaSetDevice(dList[i]));
CUDACHECK(cudaMalloc(buff + i, N * sizeof(T)));
}
//for (int root = 1; root < 2; ++root) {
for (int root = 0; root < nDev; ++root) {
RunTest<T>(buff, N, type, root, comms, dList);
}
for (int i = 0; i < nDev; ++i) {
CUDACHECK(cudaSetDevice(dList[i]));
CUDACHECK(cudaFree(buff[i]));
}
free(buff);
}
void usage() {
printf("Tests nccl Broadcast with user supplied arguments.\n"
" Usage: broadcast_test <data size in bytes> [number of GPUs] "
"[GPU 0] [GPU 1] ...\n\n");
}
int main(int argc, char* argv[]) {
int nVis = 0;
CUDACHECK(cudaGetDeviceCount(&nVis));
unsigned long long N = 0;
if (argc > 1) {
int t = sscanf(argv[1], "%llu", &N);
if (t == 0) {
printf("Error: %s is not an integer!\n\n", argv[1]);
usage();
exit(EXIT_FAILURE);
}
} else {
printf("Error: must specify at least data size in bytes!\n\n");
usage();
exit(EXIT_FAILURE);
}
int nDev = nVis;
if (argc > 2) {
int t = sscanf(argv[2], "%d", &nDev);
if (t == 0) {
printf("Error: %s is not an integer!\n\n", argv[1]);
usage();
exit(EXIT_FAILURE);
}
}
std::vector<int> dList(nDev);
for (int i = 0; i < nDev; ++i)
dList[i] = i % nVis;
if (argc > 3) {
if (argc - 3 != nDev) {
printf("Error: insufficient number of GPUs in list\n\n");
usage();
exit(EXIT_FAILURE);
}
for (int i = 0; i < nDev; ++i) {
int t = sscanf(argv[3 + i], "%d", dList.data() + i);
if (t == 0) {
printf("Error: %s is not an integer!\n\n", argv[2 + i]);
usage();
exit(EXIT_FAILURE);
}
}
}
ncclComm_t* comms = (ncclComm_t*)malloc(sizeof(ncclComm_t)*nDev);;
ncclCommInitAll(comms, nDev, dList.data());
printf("# Using devices\n");
for (int g = 0; g < nDev; ++g) {
int cudaDev;
int rank;
cudaDeviceProp prop;
ncclCommCuDevice(comms[g], &cudaDev);
ncclCommUserRank(comms[g], &rank);
CUDACHECK(cudaGetDeviceProperties(&prop, cudaDev));
printf("# Rank %2d uses device %2d [0x%02x] %s\n", rank, cudaDev,
prop.pciBusID, prop.name);
}
printf("\n");
printf("# %10s %12s %6s %4s %7s %5s %5s %7s\n",
"bytes", "N", "type", "root", "time", "algbw", "busbw", "delta");
RunTests<char>(N / sizeof(char), ncclChar, comms, dList);
RunTests<int>(N / sizeof(int), ncclInt, comms, dList);
#ifdef CUDA_HAS_HALF
RunTests<half>(N / sizeof(half), ncclHalf, comms, dList);
#endif
RunTests<float>(N / sizeof(float), ncclFloat, comms, dList);
RunTests<double>(N / sizeof(double), ncclDouble, comms, dList);
printf("\n");
for(int i = 0; i < nDev; ++i)
ncclCommDestroy(comms[i]);
free(comms);
exit(EXIT_SUCCESS);
}

461
src/common_kernel.h Normal file
View File

@ -0,0 +1,461 @@
/*************************************************************************
* Copyright (c) 2015, NVIDIA CORPORATION. All rights reserved.
*
* Redistribution and use in source and binary forms, with or without
* modification, are permitted provided that the following conditions
* are met:
* * Redistributions of source code must retain the above copyright
* notice, this list of conditions and the following disclaimer.
* * Redistributions in binary form must reproduce the above copyright
* notice, this list of conditions and the following disclaimer in the
* documentation and/or other materials provided with the distribution.
* * Neither the name of NVIDIA CORPORATION nor the names of its
* contributors may be used to endorse or promote products derived
* from this software without specific prior written permission.
*
* THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS ``AS IS'' AND ANY
* EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
* IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR
* PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR
* CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL,
* EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO,
* PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR
* PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY
* OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
* (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
* OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
************************************************************************/
#ifndef COMMON_KERNEL_H_
#define COMMON_KERNEL_H_
#include <cstdio>
#include <cstdint>
#include <cuda_runtime.h>
// BAR macro and helpers
#define WARP_SIZE 32
#define ROUNDUP(x, y) \
(((((x) + (y) - 1) / (y))) * (y))
#define BAR_EXEC(type, barid, nthreads) \
asm("bar." #type " " #barid ", " #nthreads ";\n\t")
#define BAR_EXPAND(type, barid, nthreads) \
BAR_EXEC(type, barid, (nthreads))
// Named barrier macro.
// Expands to asm("bar.type barid, nthreads") where
// nthreads has been rounded up to WARP_SIZE.
#define BAR(type, barid, nthreads) \
BAR_EXPAND(type, barid, ROUNDUP(nthreads, WARP_SIZE))
__device__ unsigned int spinct;
// Spin wait until func evaluates to true
template<typename FUNC>
__device__ inline void Wait(const FUNC& func) {
while (!func()) {
// waste time
atomicInc(&spinct, 10);
}
}
typedef uint64_t PackType;
// unpack x and y to elements of type T and apply FUNC to each element
template<class FUNC, typename T>
struct MULTI {
__device__ PackType operator()(const PackType x, const PackType y) const;
};
template<class FUNC>
struct MULTI<FUNC, char> {
static_assert(sizeof(PackType) == 2 * sizeof(uint32_t),
"PackType must be twice the size of uint32_t.");
union converter {
PackType storage;
struct {
uint32_t a, b;
};
};
__device__ PackType operator()(const PackType x, const PackType y) const {
converter cx, cy, cr;
cx.storage = x;
cy.storage = y;
// for char, we do these as vector ops
cr.a = FUNC()(cx.a, cy.a);
cr.b = FUNC()(cx.b, cy.b);
return cr.storage;
}
};
template<class FUNC>
struct MULTI<FUNC, int> {
static_assert(sizeof(PackType) == 2 * sizeof(int),
"PackType must be twice the size of int.");
union converter {
PackType storage;
struct {
int a, b;
};
};
__device__ PackType operator()(const PackType x, const PackType y) const {
converter cx, cy, cr;
cx.storage = x;
cy.storage = y;
cr.a = FUNC()(cx.a, cy.a);
cr.b = FUNC()(cx.b, cy.b);
return cr.storage;
}
};
#ifdef CUDA_HAS_HALF
template<class FUNC>
struct MULTI<FUNC, half> {
static_assert(sizeof(PackType) == 2 * sizeof(float),
"PackType must be twice the size of float.");
union converter {
PackType storage;
struct {
half2 a, b;
};
};
__device__ PackType operator()(const PackType x, const PackType y) const {
converter cx, cy, cr;
cx.storage = x;
cy.storage = y;
cr.a = FUNC()(cx.a, cy.a);
cr.b = FUNC()(cx.b, cy.b);
return cr.storage;
}
};
#endif
template<class FUNC>
struct MULTI<FUNC, float> {
static_assert(sizeof(PackType) == 2 * sizeof(float),
"PackType must be twice the size of float.");
union converter {
PackType storage;
struct {
float a, b;
};
};
__device__ PackType operator()(const PackType x, const PackType y) const {
converter cx, cy, cr;
cx.storage = x;
cy.storage = y;
cr.a = FUNC()(cx.a, cy.a);
cr.b = FUNC()(cx.b, cy.b);
return cr.storage;
}
};
template<class FUNC>
struct MULTI<FUNC, double> {
static_assert(sizeof(PackType) == sizeof(double),
"PackType must be the same size as double.");
__device__ PackType operator()(const PackType x, const PackType y) const {
double rv = FUNC()(__longlong_as_double(x), __longlong_as_double(y));
return __double_as_longlong(rv);
}
};
template<typename T, bool FETCHTWO>
__device__ inline void FetchOneOrTwo64b(PackType& s0,
const volatile T * __restrict__ const src0, PackType& s1,
const volatile T * __restrict__ const src1, const int idx) {
s0 = (reinterpret_cast<const volatile PackType *>(src0))[idx];
if (FETCHTWO) {
s1 = (reinterpret_cast<const volatile PackType *>(src1))[idx];
}
}
template<typename T, bool STORETWO>
__device__ inline void StoreOneOrTwo64b(volatile T * __restrict__ const dest0,
volatile T * __restrict__ const dest1, PackType val, const int idx) {
(reinterpret_cast<volatile PackType *>(dest0))[idx] = val;
if (STORETWO) {
(reinterpret_cast<volatile PackType *>(dest1))[idx] = val;
}
}
template<class FUNC, typename T, bool ISREDUCE>
__device__ inline PackType ReduceOrCopy64b(const PackType s0,
const PackType s1) {
if (ISREDUCE) {
return MULTI<FUNC, T>()(s0, s1);
} else {
return s0;
}
}
#define ALIGNUP(x, a) ((((x)-1) & ~((a)-1)) + (a))
template<typename T>
__device__ inline volatile T* AlignUp(volatile T * ptr, size_t align) {
size_t ptrval = reinterpret_cast<size_t>(ptr);
return reinterpret_cast<volatile T*>(ALIGNUP(ptrval, align));
}
template<typename T> inline __device__
T vFetch(const volatile T* ptr) {
return *ptr;
}
#ifdef CUDA_HAS_HALF
template<> inline __device__
half vFetch<half>(const volatile half* ptr) {
half r;
r.x = ptr->x;
return r;
}
#endif
template<typename T> inline __device__
void vStore(volatile T* ptr, const T val) {
*ptr = val;
}
#ifdef CUDA_HAS_HALF
template<> inline __device__
void vStore<half>(volatile half* ptr, const half val) {
ptr->x = val.x;
}
#endif
// Assumptions:
// - there is exactly 1 block
// - THREADS is the number of producer threads
// - this function is called by all producer threads
template<int UNROLL, int THREADS, class FUNC, typename T, bool HAS_DEST1,
bool HAS_SRC1>
__device__ inline void ReduceOrCopy(const int tid,
volatile T * __restrict__ dest0, volatile T * __restrict__ dest1,
const volatile T * __restrict__ src0, const volatile T * __restrict__ src1,
int N) {
if (N==0) {
return;
}
const int UNROLL2 = (UNROLL >= 2) ? (UNROLL / 2) : 1;
const bool NOUNROLL2 = ((UNROLL / 2) == 0);
int Npreamble = AlignUp(dest0, alignof(PackType)) - dest0;
// stage 0: check if we'll be able to use the fast, 64-bit aligned path.
// If not, we'll just use the slow preamble path for the whole operation
bool alignable = (((AlignUp(src0, alignof(PackType)) == src0 + Npreamble)) &&
(!HAS_DEST1 || (AlignUp(dest1, alignof(PackType)) == dest1 + Npreamble)) &&
(!HAS_SRC1 || (AlignUp(src1, alignof(PackType)) == src1 + Npreamble)));
if (!alignable) {
Npreamble = N;
}
/*
if (threadIdx.x == 0) {
printf("** alignable: %s", (alignable ? "YES" : " NO"));
printf(", dest0 = 0x%08X", dest0);
printf(", src0 = 0x%08X", src0);
if (HAS_DEST1) printf(", dest1 = 0x%08X", dest1);
if (HAS_SRC1) printf(", src1 = 0x%08X", src1);
printf("\n");
}
*/
// stage 1: preamble: handle any elements up to the point of everything coming
// into alignment
for (int idx = tid; idx < Npreamble; idx += THREADS) {
// ought to be no way this is ever more than one iteration, except when
// alignable is false
T val = vFetch(src0+idx);
if (HAS_SRC1) {
val = FUNC()(val, vFetch(src1+idx));
}
vStore(dest0+idx, val);
if (HAS_DEST1) {
vStore(dest1+idx, val);
}
}
// reduce N by however many elements we've handled already
int Ndone = Npreamble;
int Nrem = N - Ndone;
// stage 2: fast path: use 64b loads/stores to do the bulk of the work,
// assuming the pointers we have are all 64-bit alignable.
if (alignable) {
if (Ndone > 0) {
// align up pointers
dest0 += Ndone; if (HAS_DEST1) { dest1 += Ndone; }
src0 += Ndone; if (HAS_SRC1) { src1 += Ndone; }
}
// stage 2a: main loop
int Nalign = (Nrem / (sizeof(PackType) / sizeof(T)) / (UNROLL * THREADS))
* (UNROLL * THREADS); // round down
#pragma unroll 1 // don't unroll this loop
for (int idx = tid; idx < Nalign; idx += UNROLL * THREADS) {
PackType t0[UNROLL2];
PackType t1[UNROLL2];
PackType t2[UNROLL2];
#pragma unroll
for (int j = 0; j < UNROLL2; ++j)
FetchOneOrTwo64b<T, HAS_SRC1>(t0[j], src0, t1[j], src1,
idx + j * THREADS);
#pragma unroll
for (int j = 0; j < UNROLL2; ++j)
t2[j] = ReduceOrCopy64b<FUNC, T, HAS_SRC1>(t0[j], t1[j]);
if (!NOUNROLL2) {
#pragma unroll
for (int j = 0; j < UNROLL2; ++j)
FetchOneOrTwo64b<T, HAS_SRC1>(t0[j], src0, t1[j], src1,
idx + (UNROLL2 + j) * THREADS);
}
#pragma unroll
for (int j = 0; j < UNROLL2; ++j)
StoreOneOrTwo64b<T, HAS_DEST1>(dest0, dest1, t2[j], idx + j * THREADS);
if (!NOUNROLL2) {
#pragma unroll
for (int j = 0; j < UNROLL2; ++j)
t2[j] = ReduceOrCopy64b<FUNC, T, HAS_SRC1>(t0[j], t1[j]);
#pragma unroll
for (int j = 0; j < UNROLL2; ++j)
StoreOneOrTwo64b<T, HAS_DEST1>(dest0, dest1, t2[j],
idx + (UNROLL2 + j) * THREADS);
}
}
// stage 2b: slightly less optimized for section when we don't have full
// UNROLLs
int Ndone2a = Nalign * (sizeof(PackType)/sizeof(T));
Ndone += Ndone2a;
Nrem = N - Ndone;
// TODO: This kind of pointer update arithmetic is expensive. Should
// probably find a better way.
if (Nrem > 0) {
dest0 += Ndone2a; if (HAS_DEST1) { dest1 += Ndone2a; }
src0 += Ndone2a; if (HAS_SRC1) { src1 += Ndone2a; }
}
Nalign = Nrem / (sizeof(PackType)/sizeof(T));
#pragma unroll 4
for (int idx = tid; idx < Nalign; idx += THREADS) {
PackType t0, t1, t2;
FetchOneOrTwo64b<T, HAS_SRC1>(t0, src0, t1, src1, idx);
t2 = ReduceOrCopy64b<FUNC, T, HAS_SRC1>(t0, t1);
StoreOneOrTwo64b<T, HAS_DEST1>(dest0, dest1, t2, idx);
}
// stage 2c: tail
int Ndone2b = Nalign * (sizeof(PackType)/sizeof(T));
Ndone += Nalign * (sizeof(PackType)/sizeof(T));
Nrem = N - Ndone;
if (Nrem > 0) {
dest0 += Ndone2b; if (HAS_DEST1) { dest1 += Ndone2b; }
src0 += Ndone2b; if (HAS_SRC1) { src1 += Ndone2b; }
}
for (int idx = tid; idx < Nrem; idx += THREADS) {
// never ought to make it more than one time through this loop. only a
// few threads should even participate
T val = vFetch(src0+idx);
if (HAS_SRC1) {
val = FUNC()(val, vFetch(src1+idx));
}
vStore(dest0+idx, val);
if (HAS_DEST1) {
vStore(dest1+idx, val);
}
}
} // done fast path
}
template<int THREADS, int UNROLL, typename T>
__device__ inline void CalcLastChunk(int * const bigSliceN,
int * const smallSliceN, int * const lastSliceN, int * const numSlices,
int * const numBigSlices, int * const numSmallSlices, const int N,
const int numChunks, const int chunkSize) {
int Nleft = N - ((numChunks - 1) * chunkSize);
// semi-equally split up the remaining work into numslices slices.
// it's "semi"-equal because we want the divisions to land as neatly as we
// can on alignable boundaries
int NperTile = UNROLL * THREADS * (sizeof(PackType)/sizeof(T));
int numTiles = (Nleft + NperTile - 1) / NperTile;
int numTilesPerBigSlice = (numTiles + *numSlices - 1)
/ *numSlices;
int numTilesPerSmallSlice = numTiles / *numSlices;
*bigSliceN = NperTile * numTilesPerBigSlice;
*smallSliceN = NperTile * numTilesPerSmallSlice;
*numBigSlices = numTiles % *numSlices;
*numSmallSlices = (*smallSliceN > 0) ?
*numSlices - *numBigSlices : 0;
// the lastSlice will take the place of one of the small slices unless
// there are no small slices (because this is a very small reduction), in
// which case we replace one of the big slices and leave the small slices
// as 0.
if (*numSmallSlices > 0) {
--*numSmallSlices;
if (*numSmallSlices == 0)
*smallSliceN = 0;
}
else {
--*numBigSlices;
if (*numBigSlices == 0)
*bigSliceN = 0;
}
*lastSliceN = Nleft -
(*numBigSlices * *bigSliceN
+ *numSmallSlices * *smallSliceN);
// in cases where args.N % numSlices is pretty small, we'd rather have one
// slightly big last slice than one big slice, a bunch of small slices,
// and one smaller last slice
if ((*numBigSlices == 1) &&
(*numSmallSlices == *numSlices - 2) &&
(*lastSliceN < *smallSliceN)) {
*numBigSlices += *numSmallSlices;
*numSmallSlices = 0;
*bigSliceN = *smallSliceN;
*smallSliceN = 0;
*lastSliceN = Nleft -
*numBigSlices * *bigSliceN;
}
// done recalculating
*numSlices = *numBigSlices +
*numSmallSlices + 1;
}
#endif // COMMON_KERNEL_H_

79
src/copy_kernel.h Normal file
View File

@ -0,0 +1,79 @@
/*************************************************************************
* Copyright (c) 2015, NVIDIA CORPORATION. All rights reserved.
*
* Redistribution and use in source and binary forms, with or without
* modification, are permitted provided that the following conditions
* are met:
* * Redistributions of source code must retain the above copyright
* notice, this list of conditions and the following disclaimer.
* * Redistributions in binary form must reproduce the above copyright
* notice, this list of conditions and the following disclaimer in the
* documentation and/or other materials provided with the distribution.
* * Neither the name of NVIDIA CORPORATION nor the names of its
* contributors may be used to endorse or promote products derived
* from this software without specific prior written permission.
*
* THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS ``AS IS'' AND ANY
* EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
* IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR
* PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR
* CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL,
* EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO,
* PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR
* PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY
* OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
* (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
* OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
************************************************************************/
#ifndef COPY_KERNEL_H_
#define COPY_KERNEL_H_
#include "common_kernel.h"
template<typename T>
struct FuncPassA {
__device__ T operator()(const T x, const T y) const {
return x;
}
};
#ifdef CUDA_HAS_HALF
template <>
struct FuncPassA<half> {
__device__ half2 operator()(const half2 x, const half2 y) const {
return x;
}
__device__ half operator()(const half x, const half y) const {
half r;
r.x = x.x;
return r;
}
};
#endif
// Assumptions:
// - there is exactly 1 block
// - THREADS is the number of producer threads
// - this function is called by all producer threads
template<int UNROLL, int THREADS, typename T>
__device__ void Copy(volatile T * __restrict__ const dest,
const volatile T * __restrict__ const src, const int N) {
ReduceOrCopy<UNROLL, THREADS, FuncPassA<T>, T, false, false>(threadIdx.x,
dest, nullptr, src, nullptr, N);
}
// Assumptions:
// - there is exactly 1 block
// - THREADS is the number of producer threads
// - this function is called by all producer threads
template<int UNROLL, int THREADS, typename T>
__device__ void DoubleCopy(volatile T * __restrict__ const dest0,
volatile T * __restrict__ const dest1,
const volatile T * __restrict__ const src, const int N) {
ReduceOrCopy<UNROLL, THREADS, FuncPassA<T>, T, true, false>(threadIdx.x,
dest0, dest1, src, nullptr, N);
}
#endif // COPY_KERNEL_H_

856
src/core.cu Normal file
View File

@ -0,0 +1,856 @@
/*************************************************************************
* Copyright (c) 2015, NVIDIA CORPORATION. All rights reserved.
*
* Redistribution and use in source and binary forms, with or without
* modification, are permitted provided that the following conditions
* are met:
* * Redistributions of source code must retain the above copyright
* notice, this list of conditions and the following disclaimer.
* * Redistributions in binary form must reproduce the above copyright
* notice, this list of conditions and the following disclaimer in the
* documentation and/or other materials provided with the distribution.
* * Neither the name of NVIDIA CORPORATION nor the names of its
* contributors may be used to endorse or promote products derived
* from this software without specific prior written permission.
*
* THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS ``AS IS'' AND ANY
* EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
* IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR
* PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR
* CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL,
* EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO,
* PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR
* PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY
* OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
* (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
* OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
************************************************************************/
#include <stdio.h>
#include <stdlib.h>
#include "core.h"
#include "libwrap.h"
#include <sys/mman.h>
#include <sys/stat.h>
#include <sys/types.h>
#include <sched.h>
#include <fcntl.h>
#include <unistd.h>
#include <cuda.h>
#include <cuda_runtime.h>
#include <string.h>
#include <errno.h>
DebugLevel ncclDebugLevel;
extern "C" DSOGLOBAL
ncclResult_t ncclGetUniqueId(ncclUniqueId* out) {
pid_t pid = getpid();
static int count = 0;
int commId = __sync_fetch_and_add(&count, 1);
int len = snprintf(out->internal, NCCL_UNIQUE_ID_BYTES, "nccl-%d-%d", pid, commId);
if(strlen(out->internal) < len) {
WARN("ncclUniqueId truncated");
return ncclInternalError;
}
return ncclSuccess;
}
static ncclResult_t shmOpen(const char* shmname, size_t bytes, void** ptr) {
int fd = shm_open(shmname, O_CREAT | O_RDWR, S_IRUSR | S_IWUSR);
if (fd == -1) {
WARN("shm_open failed to open %s", shmname);
return ncclSystemError;
}
if (ftruncate(fd, bytes) == -1) {
WARN("ftruncate failed to allocate %ld bytes", bytes);
shm_unlink(shmname);
close(fd);
return ncclSystemError;
}
*ptr = mmap(NULL, bytes, PROT_READ | PROT_WRITE, MAP_SHARED, fd, 0);
if (*ptr == MAP_FAILED) {
WARN("failure in mmap");
shm_unlink(shmname);
close(fd);
return ncclSystemError;
}
close(fd);
return ncclSuccess;
}
static ncclResult_t shmUnlink(const char* shmname) {
if(shm_unlink(shmname) == -1) {
WARN("smh_unlink failed");
return ncclSystemError;
} else {
return ncclSuccess;
}
}
static ncclResult_t shmUnmap(void* ptr, size_t bytes) {
if(munmap(ptr, bytes) == -1) {
WARN("munmap failed");
return ncclSystemError;
} else {
return ncclSuccess;
}
}
typedef struct {
int rank;
int ndev;
int cudaDev;
int ncclId;
pid_t pid;
ncclMem* hostptr;
ncclMem* devptr;
CUipcMemHandle devipc;
size_t buffSize;
} RankEntry;
static int compRanks(const void* a, const void* b) {
const RankEntry* A = (const RankEntry*)a;
const RankEntry* B = (const RankEntry*)b;
if (A->ncclId < B->ncclId) return -1;
if (A->ncclId > B->ncclId) return 1;
return 0;
}
static void orderRanks(RankEntry* ranks, int count) {
qsort(ranks, count, sizeof(RankEntry), compRanks);
for(int i=0; i<count; ++i)
ranks[i].ncclId = i;
}
typedef struct {
union {
struct {
volatile int bar;
};
char pad[16];
};
RankEntry ranks[1];
} RankGather;
static ncclResult_t initGather(RankGather** gather, ncclUniqueId commId,
int ndev, int rank, RankEntry myInfo) {
size_t bytes = offsetof(RankGather, ranks) + ndev*sizeof(RankEntry);
RankGather* tmp = NULL;
int bar_tmp;
ncclResult_t res = shmOpen(commId.internal, bytes, (void**)&tmp);
if (res != ncclSuccess) {
WARN("rank %d failed to open shm segment for gather", rank);
return res;
}
tmp->ranks[rank] = myInfo;
bar_tmp = tmp->bar - 1;
bool swapped;
do {
bar_tmp += 1;
if (bar_tmp == ndev-1) { // everyone is done
ncclResult_t res = shmUnlink(commId.internal);
if (res != ncclSuccess) {
WARN("rank %d failed to unlink shm segment for gather", rank);
shmUnmap(tmp, bytes);
return res;
}
orderRanks(tmp->ranks, ndev);
}
swapped = __sync_bool_compare_and_swap(&tmp->bar, bar_tmp, bar_tmp+1);
} while(!swapped);
while (tmp->bar < ndev)
sched_yield();
__sync_synchronize();
*gather = tmp;
return ncclSuccess;
}
static ncclResult_t closeGather(RankGather* gather, int ndev) {
int bar_tmp = gather->bar - 1;
bool swapped;
do {
bar_tmp += 1;
swapped = __sync_bool_compare_and_swap(&gather->bar, bar_tmp, bar_tmp+1);
} while(!swapped);
while (gather->bar != 2*ndev)
sched_yield();
__sync_synchronize();
size_t bytes = offsetof(RankGather, ranks) + ndev*sizeof(RankEntry);
ncclResult_t res = shmUnmap(gather, bytes);
if (res != ncclSuccess) {
WARN("failed to unmap %ld bytes of gather", bytes);
return res;
}
return ncclSuccess;
}
static ncclResult_t allocDevMem(ncclMem** ptr, size_t buffSize) {
size_t size = offsetof(struct ncclMem, buff) + buffSize;
cudaError_t res = cudaMalloc((void**)ptr, size);
if (res != cudaSuccess) {
*ptr = NULL;
WARN("failed to allocate %lu byte device buffer", size);
return ncclCudaMallocFailed;
}
if (cudaMemset(*ptr, 0, size) != cudaSuccess) {
WARN("failed to memset device buffer.");
cudaFree(*ptr);
*ptr = NULL;
return ncclUnhandledCudaError;
}
return ncclSuccess;
}
static const int ShmMapped = 1;
static const int ShmLinked = 2;
static ncclResult_t allocHostMem(ncclMem** ptr, size_t buffSize) {
size_t size = offsetof(struct ncclMem, buff) + buffSize;
cudaError_t res = cudaMallocHost((void**)ptr, size);
if (res != cudaSuccess) {
*ptr = NULL;
WARN("failed to allocate %lu byte host buffer", size);
return ncclSystemError;
}
memset(*ptr, 0, size);
return ncclSuccess;
}
static ncclResult_t openHostMemShm(const char* shmname, ncclMem** ptr, size_t buffSize) {
size_t size = offsetof(struct ncclMem, buff) + buffSize;
ncclResult_t res = shmOpen(shmname, size, (void**)ptr);
if (res != ncclSuccess) {
WARN("failed to allocate %lu byte shm buffer", size);
*ptr = NULL;
return res;
}
if(cudaHostRegister(*ptr, size, cudaHostRegisterMapped) != cudaSuccess) {
WARN("failed to register host buffer");
shmUnlink(shmname);
shmUnmap(*ptr, size);
*ptr = NULL;
return ncclUnhandledCudaError;
}
return ncclSuccess;
}
static ncclResult_t populateRankInfo(RankEntry* info, int rank, ncclComm_t comm) {
char busId[13];
nvmlDevice_t nvmlHandle;
cudaError_t res = cudaDeviceGetPCIBusId(busId, 13, comm->cudaDev);
if (res == cudaErrorInvalidDevice) {
WARN("rank %d attempted to access an invalid cuda device %d", rank, comm->cudaDev);
return ncclInvalidDeviceIndex;
} else if (res != cudaSuccess) {
WARN("rank %d failed to get PCI Bus Id for device %d", rank, comm->cudaDev);
return ncclUnhandledCudaError;
}
INFO("rank %d using device %d (%s)", rank, comm->cudaDev, busId);
if (wrapNvmlDeviceGetHandleByPciBusId(busId, &nvmlHandle) != ncclSuccess) {
WARN("rank %d failed to get nvml handle for device %s", rank, busId);
return ncclUnhandledCudaError;
}
// Order by nvml index
if (wrapNvmlDeviceGetIndex(nvmlHandle, (unsigned*)&info->ncclId) != ncclSuccess) {
WARN("rank %d failed to get nvml device index for device %d", rank, comm->cudaDev);
return ncclUnhandledCudaError;
}
info->rank = rank;
info->ndev = comm->nDev;
info->cudaDev = comm->cudaDev;
info->pid = getpid();
info->buffSize = comm->buffSize;
info->hostptr = comm->hostMem;
info->devptr = comm->devMem;
if (wrapCuIpcGetMemHandle(&info->devipc, (CUdeviceptr)comm->devMem) != ncclSuccess) {
WARN("rank %d failed to open CUDA IPC handle", rank);
return ncclUnhandledCudaError;
}
return ncclSuccess;
}
static const int CLEANUP_NONE = 0;
static const int CLEANUP_CUIPC = 1;
static const int CLEANUP_UNMAP = 2;
static ncclResult_t commClearMaps(ncclComm_t comm) {
ncclResult_t res, retval = ncclSuccess;
cudaError_t cures;
for(int d=0; d<comm->nDev; ++d) {
switch(comm->cleanup[d].type) {
case CLEANUP_NONE:
break;
case CLEANUP_CUIPC:
res = wrapCuIpcCloseMemHandle((CUdeviceptr)comm->cleanup[d].handle);
if (res != ncclSuccess) {
WARN("rank %d failed to close IPC handle to rank %d",
comm->userFromRing[comm->ncclId], comm->userFromRing[d]);
retval = (retval == ncclSuccess) ? res : retval;
}
break;
case CLEANUP_UNMAP:
cures = cudaHostUnregister(comm->cleanup[d].handle);
if (cures != cudaSuccess) {
WARN("rank %d failed to unregister handle to rank %d",
comm->userFromRing[comm->ncclId], comm->userFromRing[d]);
retval = (retval == ncclSuccess) ? ncclUnhandledCudaError : retval;
}
res = shmUnmap(comm->cleanup[d].handle, offsetof(ncclMem, buff) + comm->buffSize);
if (res != ncclSuccess) {
WARN("rank %d failed to unmap handle to rank %d",
comm->userFromRing[comm->ncclId], comm->userFromRing[d]);
retval = (retval == ncclSuccess) ? res : retval;
}
break;
default:
WARN("Unknown cleanup type %d", comm->cleanup[d].type);
}
comm->cleanup[d].type = 0;
comm->cleanup[d].handle = NULL;
}
memset(comm->userFromRing, 0, sizeof(int)*MAXPEERS);
memset(comm->ringFromUser, 0, sizeof(int)*MAXPEERS);
if (comm->devUserFromRing != NULL) {
cudaError_t err = cudaMemset(comm->devUserFromRing, 0, sizeof(int)*MAXPEERS);
if (err != cudaSuccess) {
WARN("Faild to clear dev map: %s", cudaGetErrorString(err));
retval = (retval == ncclSuccess) ? ncclUnhandledCudaError : retval;
}
}
return retval;
}
static ncclResult_t commBuildMaps(ncclComm_t comm, ncclUniqueId* commId, int rank, RankEntry* ranks) {
int ndev = comm->nDev;
for(int i=0; i<ndev; ++i) {
// Check for inconsistencies between ranks
// If two ranks use the same rank, then one slot of
// ranks[] will be left unset with zero ndev/buffSize.
if (ranks[i].buffSize != comm->buffSize
|| ranks[i].ndev != comm->nDev) {
commClearMaps(comm);
return ncclRankMismatch;
}
// Create rank<->nccl maps
int iRank = ranks[i].rank;
comm->userFromRing[i] = iRank;
comm->ringFromUser[iRank] = i;
}
if (cudaMemcpy(comm->devUserFromRing, comm->userFromRing, ndev*sizeof(int),
cudaMemcpyHostToDevice) != cudaSuccess) {
WARN("rank %d failed to copy maps to device", rank);
commClearMaps(comm);
return ncclUnhandledCudaError;
}
int myId = -1;
for (int i=0; i<ndev; ++i) {
if(ranks[i].rank == rank) {
myId = i;
break;
}
}
if (myId == -1) {
WARN("rank %d not found in communicator", rank);
return ncclInvalidRank;
}
comm->ncclId = myId;
int myDev = ranks[myId].cudaDev;
pid_t myPid = ranks[myId].pid;
comm->useRemoteRecv = 1; // Assume we directly write to result ptrs.
for (int i=0; i<ndev; ++i) {
int iRank = ranks[i].rank;
int iDev = ranks[i].cudaDev;
pid_t iPid = ranks[i].pid;
int canpeer = 0;
if (cudaDeviceCanAccessPeer(&canpeer, myDev, iDev) != cudaSuccess) {
INFO("peer query failed between rank %d (dev %d) and rank %d (dev %d)",
rank, myDev, iRank, iDev);
canpeer = 0;
}
if (canpeer) {
cudaError_t err;
err = cudaDeviceEnablePeerAccess(iDev, 0);
if (err == cudaErrorPeerAccessAlreadyEnabled) {
cudaGetLastError();
} else if (err != cudaSuccess) {
INFO("peer access failed between rank %d (dev %d) and rank %d (dev %d)\n",
rank, myDev, iRank, iDev);
canpeer = 0;
}
}
if (iPid == myPid && (canpeer || myDev == iDev)) {
INFO("rank access %d -> %d via P2P device mem", rank, iRank);
comm->local[i] = ranks[myId].devptr;
comm->remote[i] = ranks[i].devptr;
comm->cleanup[i].type = CLEANUP_NONE;
} else if (iPid == myPid) {
INFO("rank access %d -> %d via zero-copy host mem", rank, iRank);
comm->useRemoteRecv = 0;
if (cudaHostGetDevicePointer(comm->local+i, ranks[myId].hostptr, 0) != cudaSuccess) {
WARN("rank %d failed to map zero copy buffer to device", rank);
commClearMaps(comm);
return ncclUnhandledCudaError;
}
if (cudaHostGetDevicePointer(comm->remote+i, ranks[i].hostptr, 0) != cudaSuccess) {
WARN("rank %d failed to map %d's zero copy buffer to device", rank, iRank);
commClearMaps(comm);
return ncclUnhandledCudaError;
}
comm->cleanup[i].type = CLEANUP_NONE;
} else if (canpeer || myDev == iDev) {
INFO("rank access %d -> %d via Ipc P2P device mem", rank, iRank);
comm->useRemoteRecv = 0;
comm->local[i] = ranks[myId].devptr;
if (wrapCuIpcOpenMemHandle((CUdeviceptr*)(&comm->remote[i]),
ranks[i].devipc, CU_IPC_MEM_LAZY_ENABLE_PEER_ACCESS) != ncclSuccess) {
WARN("rank %d failed to open Ipc handle to rank %d", rank, iRank);
commClearMaps(comm);
return ncclUnhandledCudaError;
}
comm->cleanup[i].type = CLEANUP_CUIPC;
comm->cleanup[i].handle = comm->remote[i];
} else {
INFO("rank access %d -> %d via zero copy host shm", rank, iRank);
comm->useRemoteRecv = 0;
if (cudaHostGetDevicePointer(comm->local+i, ranks[myId].hostptr, 0) != cudaSuccess) {
WARN("rank %d failed to obtain dev ptr to sysmem buffer", rank);
commClearMaps(comm);
return ncclUnhandledCudaError;
}
char rankname[1024];
sprintf(rankname, "%s-%d", commId->internal, ranks[i].rank);
if (openHostMemShm(rankname, (ncclMem**)&comm->cleanup[i].handle, ranks[i].buffSize)
!= ncclSuccess) {
WARN("rank %d failed to open sysmem buffer of rank %d", rank, iRank);
commClearMaps(comm);
return ncclUnhandledCudaError;
}
if (shmUnlink(rankname) != ncclSuccess) {
INFO("rank %d failed to unlink sysmem beffer of rank %d, %s", rank, iRank, rankname);
}
if (cudaHostGetDevicePointer(comm->remote+i, comm->cleanup[i].handle, 0) != cudaSuccess) {
WARN("rank %d failed to obtain dev ptr for rank %d", rank, iRank);
commClearMaps(comm);
return ncclUnhandledCudaError;
}
comm->cleanup[i].type = CLEANUP_UNMAP;
}
}
INFO("PushToRecv algos are %s\n", (comm->useRemoteRecv) ? "enabled" : "disabled");
return ncclSuccess;
}
static void initDebug() {
const char* nccl_debug = getenv("NCCL_DEBUG");
if (nccl_debug == NULL) {
ncclDebugLevel = NONE;
} else if (strcmp(nccl_debug, "WARN") == 0) {
ncclDebugLevel = WARN;
} else if (strcmp(nccl_debug, "INFO") == 0) {
ncclDebugLevel = INFO;
INFO("NCCL debug level set to INFO");
} else if (strcmp(nccl_debug, "ABORT") == 0) {
ncclDebugLevel = ABORT;
INFO("NCCL debug level set to ABORT");
}
}
static void commFree(ncclComm_t comm) {
if (comm == NULL)
return;
for(int i=0; i<MAXQUEUE; ++i) {
if (comm->events.isDone[i] != NULL)
if (cudaEventDestroy(comm->events.isDone[i]) != cudaSuccess)
INFO("failed to destroy cuda event %d", i);
}
ncclResult_t res = commClearMaps(comm);
if (res != ncclSuccess)
INFO("failed to cleanup comm maps");
if (comm->devUserFromRing != NULL)
if (cudaFree(comm->devUserFromRing) != cudaSuccess)
INFO("commFree failed to free dev maps");
if (comm->devMem != NULL && cudaFree(comm->devMem) != cudaSuccess)
INFO("Failed to free devMap");
if (comm->hostMem != NULL) {
if (comm->hostMemState & ShmMapped) {
if (cudaHostUnregister(comm->hostMem) != cudaSuccess)
INFO("Failed to unregister hostMem");
size_t size = offsetof(ncclMem, buff) + comm->buffSize;
if (shmUnmap(comm->hostMem, size) != ncclSuccess)
INFO("Failed to unmap hostMem");
comm->hostMemState ^= ShmMapped;
} else {
cudaFreeHost(comm->hostMem);
}
}
free(comm);
}
static ncclResult_t commAlloc(ncclComm_t* comret, int ndev, const ncclUniqueId* commId, int rank) {
if (ndev < 1 || ndev > MAXPEERS) {
WARN("requested device count (%d) exceeds maximum of %d", ndev, MAXPEERS);
return ncclUnsupportedDeviceCount;
}
if (rank >= ndev || rank < 0) {
WARN("rank %d exceeds ndev=%d", rank, ndev);
return ncclInvalidRank;
}
struct ncclComm* comm = (struct ncclComm*)malloc(sizeof(struct ncclComm));
if (comm == NULL) {
WARN("comm allocation failed");
return ncclSystemError;
}
memset(comm, 0, sizeof(struct ncclComm));
comm->nDev = ndev;
cudaGetDevice(&comm->cudaDev);
const char* str = getenv("NCCL_BUFFSIZE");
if (str != NULL) {
errno = 0;
comm->buffSize = strtol(str, NULL, 10);
if (errno == ERANGE || comm->buffSize == 0) {
INFO("rank %d invalid NCCL_BUFFSIZE: %s, using default %lu",
rank, str, DEFAULT_BUFFER_SIZE_BYTES);
comm->buffSize = DEFAULT_BUFFER_SIZE_BYTES;
}
} else {
comm->buffSize = DEFAULT_BUFFER_SIZE_BYTES;
}
INFO("rank %d using buffSize = %lu", rank, comm->buffSize);
ncclResult_t res;
res = allocDevMem(&comm->devMem, comm->buffSize);
if (res != ncclSuccess) {
WARN("rank %d failed to allocate device buffer", rank);
commFree(comm);
return res;
}
if (cudaMalloc(&comm->devUserFromRing, MAXPEERS*sizeof(int)) != cudaSuccess) {
WARN("rank %d failed to allocated device maps", rank);
commFree(comm);
return ncclCudaMallocFailed;
}
EventQueue* eq = &comm->events;
for(int i=0; i<MAXQUEUE; ++i) {
if (cudaEventCreateWithFlags(eq->isDone+i, cudaEventDisableTiming) != cudaSuccess) {
WARN("rank %d failed to create nccl event %d", rank, i);
commFree(comm);
return ncclUnhandledCudaError;
}
}
if(commId == NULL) {
comm->hostMemState = 0;
res = allocHostMem(&comm->hostMem, comm->buffSize);
} else {
char rankname[1024];
sprintf(rankname, "%s-%d", commId->internal, rank);
res = openHostMemShm(rankname, &comm->hostMem, comm->buffSize);
if (res != ncclSuccess) {
WARN("rank %d failed to allocate host buffer", rank);
commFree(comm);
return res;
}
comm->hostMemState = ShmMapped | ShmLinked;
}
*comret = comm;
return ncclSuccess;
}
static ncclResult_t commUnlinkHostMem(ncclComm_t comm, ncclUniqueId commId, int rank) {
char rankname[1024];
sprintf(rankname, "%s-%d", commId.internal, rank);
if (comm->hostMemState & ShmLinked)
comm->hostMemState ^= ShmLinked;
return shmUnlink(rankname);
}
extern "C" DSOGLOBAL
ncclResult_t ncclCommInitRank(ncclComm_t* newcomm, int ndev, ncclUniqueId commId, int myrank) {
if (strlen(commId.internal) < 1 ||
strlen(commId.internal) >= NCCL_UNIQUE_ID_BYTES) {
WARN("rank %d invalid commId", myrank);
return ncclInvalidArgument;
}
initDebug();
ncclResult_t res;
RankEntry myStuff;
RankGather* gath = NULL;
res = wrapSymbols();
if (res != ncclSuccess) {
WARN("NCCL failed to initialize client libs");
return res;
}
res = wrapNvmlInit();
if (res != ncclSuccess) {
WARN("rank %d failed to initialize nvml", myrank);
return res;
}
res = commAlloc(newcomm, ndev, &commId, myrank);
if (res != ncclSuccess) {
WARN("rank %d failed to allocate communicator", myrank);
return res;
}
res = populateRankInfo(&myStuff, myrank, *newcomm);
if (res != ncclSuccess) {
WARN("rank %d failed to obtain rank info", myrank);
goto cleanup;
}
res = initGather(&gath, commId, ndev, myrank, myStuff);
if (res != ncclSuccess) {
WARN("rank %d failed to gather rank info", myrank);
goto cleanup;
}
res = commBuildMaps(*newcomm, &commId, myrank, gath->ranks);
if (res != ncclSuccess) {
WARN("rank %d failed to build comm maps", myrank);
goto cleanup;
}
res = closeGather(gath, ndev); // includes a barrier
gath = NULL;
if (res != ncclSuccess) {
WARN("rank %d failed to close gather", myrank);
goto cleanup;
}
goto final;
cleanup:
if (gath != NULL)
closeGather(gath, ndev);
commFree(*newcomm);
final:
if ((*newcomm)->hostMemState & ShmLinked) {
if (commUnlinkHostMem(*newcomm, commId, myrank) != ncclSuccess)
INFO("rank %d failed to unlink host mem shm segment", myrank);
}
if (wrapNvmlShutdown() != ncclSuccess)
INFO("rank %d did not shutdown nvml properly", myrank);
return res;
}
extern "C" DSOGLOBAL
ncclResult_t ncclCommInitAll(ncclComm_t* comms, int ndev, int* devlist) {
initDebug();
ncclResult_t res;
int savedDevice;
RankEntry* ranks = NULL;
int rank, cudaDev;
ncclComm_t comm = NULL;
char busId[13];
nvmlDevice_t nvmlHandle;
int affinity_set = 0;
res = wrapSymbols();
if (res != ncclSuccess) {
WARN("NCCL failed to initialize client libs");
return res;
}
cudaGetDevice(&savedDevice);
ranks = (RankEntry*)malloc(ndev*sizeof(RankEntry));
if (ranks == NULL) {
WARN("NCCL allocation failed");
return ncclSystemError;
}
memset(ranks, 0, ndev*sizeof(RankEntry));
res = wrapNvmlInit();
if (res != ncclSuccess) {
WARN("nccl failed to initialize nvml");
return res;
}
for(rank=0; rank<ndev; ++rank)
comms[rank] = NULL;
for (rank=0; rank<ndev; ++rank) {
cudaDev = (devlist == NULL) ? rank : devlist[rank];
if (cudaSetDevice(cudaDev) != cudaSuccess) {
WARN("rank %d failed to set cuda device %d", rank, cudaDev);
res = ncclInvalidDeviceIndex;
goto cleanup;
}
// Set CPU affinity
affinity_set = 0;
if (cudaDeviceGetPCIBusId(busId, 13, cudaDev) != cudaSuccess) {
INFO("rank %d failed to get PCI Bus Id for device %d", rank, cudaDev);
goto skipaffinity;
}
if (wrapNvmlDeviceGetHandleByPciBusId(busId, &nvmlHandle) != ncclSuccess) {
INFO("rank %d failed to get nvml handle for device %s", rank, busId);
goto skipaffinity;
}
if (wrapNvmlDeviceSetCpuAffinity(nvmlHandle) != ncclSuccess) {
INFO("rank %d failed to set affinity", rank);
goto skipaffinity;
}
affinity_set = 1;
skipaffinity:
res = commAlloc(&comm, ndev, NULL, rank);
if (res != ncclSuccess) {
WARN("rank %d failed to allocate communicator", rank);
goto cleanup;
}
comms[rank] = comm;
if (affinity_set && wrapNvmlDeviceClearCpuAffinity(nvmlHandle) != ncclSuccess) {
INFO("rank %d set but failed to clear cpu affinity", rank);
}
res = populateRankInfo(ranks+rank, rank, comm);
if (res != ncclSuccess) {
WARN("rank %d failed to obtain rank info", rank);
goto cleanup;
}
}
orderRanks(ranks, ndev);
for(rank=0; rank<ndev; ++rank) {
comm = comms[rank];
cudaSetDevice(comm->cudaDev);
res = commBuildMaps(comm, NULL, rank, ranks);
if (res != ncclSuccess) {
WARN("rank %d failed to build comm maps", rank);
goto cleanup;
}
}
free(ranks);
ranks = NULL;
res = ncclSuccess;
goto final;
cleanup:
if (ranks != NULL)
free(ranks);
for(rank=0; rank<ndev; ++rank) {
if(comms[rank] != NULL) {
commFree(comms[rank]);
}
}
final:
if(wrapNvmlShutdown() != ncclSuccess)
INFO("NCCL did not shutdown nvml properly");
cudaSetDevice(savedDevice);
return res;
}
extern "C" DSOGLOBAL
void ncclCommDestroy(ncclComm_t comm) {
if (comm == NULL)
return;
int savedDevice;
cudaGetDevice(&savedDevice);
int commDevice = comm->cudaDev;
if (savedDevice != commDevice) {
CUDACHECK(cudaSetDevice(commDevice));
}
commFree(comm);
if (savedDevice != commDevice)
cudaSetDevice(savedDevice);
}
extern "C" DSOGLOBAL
const char* ncclGetErrorString(ncclResult_t code) {
switch (code) {
case ncclSuccess : return "no error";
case ncclUnhandledCudaError : return "unhandled cuda error";
case ncclSystemError : return "system error";
case ncclInternalError : return "internal error";
case ncclInvalidDevicePointer : return "invalid device pointer";
case ncclInvalidRank : return "invalid rank";
case ncclUnsupportedDeviceCount : return "unsupported device count";
case ncclDeviceNotFound : return "device not found";
case ncclInvalidDeviceIndex : return "invalid device index";
case ncclLibWrapperNotSet : return "lib wrapper not initialized";
case ncclCudaMallocFailed : return "cuda malloc failed";
case ncclRankMismatch : return "parameter mismatch between ranks";
case ncclInvalidArgument : return "invalid argument";
case ncclInvalidType : return "invalid data type";
case ncclInvalidOperation : return "invalid reduction operations";
}
return "unknown result code";
}
extern "C" DSOGLOBAL
ncclResult_t ncclCommCount(const ncclComm_t comm, int* count) {
*count = comm->nDev;
return ncclSuccess;
}
extern "C" DSOGLOBAL
ncclResult_t ncclCommCuDevice(const ncclComm_t comm, int* devid) {
*devid = comm->cudaDev;
return ncclSuccess;
}
extern "C" DSOGLOBAL
ncclResult_t ncclCommUserRank(const ncclComm_t comm, int* rank) {
*rank = comm->userFromRing[comm->ncclId];
return ncclSuccess;
}

134
src/core.h Normal file
View File

@ -0,0 +1,134 @@
/*************************************************************************
* Copyright (c) 2015, NVIDIA CORPORATION. All rights reserved.
*
* Redistribution and use in source and binary forms, with or without
* modification, are permitted provided that the following conditions
* are met:
* * Redistributions of source code must retain the above copyright
* notice, this list of conditions and the following disclaimer.
* * Redistributions in binary form must reproduce the above copyright
* notice, this list of conditions and the following disclaimer in the
* documentation and/or other materials provided with the distribution.
* * Neither the name of NVIDIA CORPORATION nor the names of its
* contributors may be used to endorse or promote products derived
* from this software without specific prior written permission.
*
* THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS ``AS IS'' AND ANY
* EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
* IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR
* PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR
* CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL,
* EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO,
* PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR
* PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY
* OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
* (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
* OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
************************************************************************/
#ifndef CORE_H_
#define CORE_H_
#include "nccl.h"
#include <cstdio>
#include <cuda_runtime.h>
#define MAXPEERS 8 // Maximum number of devices.
#define MAXFLAGS 8
#define MAXQUEUE 4 // Maximum number of queued collectives per communicator.
#define DEFAULT_BUFFER_SIZE_BYTES (1UL << 21)
// DIE on error
#define CUDACHECK(cmd) do { \
cudaError_t e = cmd; \
if( e != cudaSuccess ) { \
printf("Cuda failure %s:%d '%s'\n", \
__FILE__,__LINE__,cudaGetErrorString(e)); \
exit(EXIT_FAILURE); \
} \
} while(false)
#define NCCL_MEM_PAD_ALIGN 4096
typedef struct {
cudaEvent_t isDone[MAXQUEUE];
int back; // Last event used
} EventQueue;
struct ncclMem {
union { // Pad this block so that devBuff is correctly aligned
struct {
int flags[MAXFLAGS];
void* recvPtrs[MAXFLAGS];
};
char pad[NCCL_MEM_PAD_ALIGN];
};
// devBuff will likely be bigger ; we only use its offset/address.
char buff[NCCL_MEM_PAD_ALIGN];
};
struct ncclComm {
int nDev; // number of devices in communicator
int cudaDev; // cuda device index
int ncclId; // nccl logical index
// Device and Host allocated chunks. Stored here to correctly free() memory.
ncclMem* devMem;
ncclMem* hostMem;
int hostMemState;
// Device-to-device communication structures to access remote or local device
// memory.
ncclMem* remote[MAXPEERS];
ncclMem* local[MAXPEERS];
struct {
int type;
void* handle;
} cleanup[MAXPEERS];
//int remoteCleanup[MAXPEERS]; // 0 is raw ptr, 1 is unregister/unmap, 2 is ipc close
// Placed between calling and internal device streams.
EventQueue events;
// Maps an internal nccl index to user-specified rank order. This is necessary
// since we need to know how the user expects data to be ordered across
// devices.
int userFromRing[MAXPEERS];
// copy of the above stored on each device
int* devUserFromRing;
// Inverse of userFromRing. Maps user specified index to internal nccl index.
int ringFromUser[MAXPEERS];
// Size of temp buffer in bytes.
size_t buffSize;
// Whether we have remote access to the recvbuff pointers passed from remote
// GPUs. In single process mode this can be used as long as QPI links are
// not present. In multi-process, we never push to a remote recvbuff.
int useRemoteRecv;
};
typedef enum {NONE=0, WARN=1, INFO=2, ABORT=3} DebugLevel;
extern DebugLevel ncclDebugLevel;
#define WARN(...) do { \
if (ncclDebugLevel >= WARN) { \
printf("WARN %s:%d ", __FILE__, __LINE__); \
printf(__VA_ARGS__); \
printf("\n"); \
if (ncclDebugLevel >= ABORT) abort(); \
} \
} while(0)
#define INFO(...) do { \
if (ncclDebugLevel >= INFO) { \
printf("INFO "); printf(__VA_ARGS__); printf("\n"); \
} \
} while(0)
#define DSOGLOBAL __attribute__((visibility("default")))
#endif // end include guard

80
src/enqueue.h Normal file
View File

@ -0,0 +1,80 @@
/*************************************************************************
* Copyright (c) 2015, NVIDIA CORPORATION. All rights reserved.
*
* Redistribution and use in source and binary forms, with or without
* modification, are permitted provided that the following conditions
* are met:
* * Redistributions of source code must retain the above copyright
* notice, this list of conditions and the following disclaimer.
* * Redistributions in binary form must reproduce the above copyright
* notice, this list of conditions and the following disclaimer in the
* documentation and/or other materials provided with the distribution.
* * Neither the name of NVIDIA CORPORATION nor the names of its
* contributors may be used to endorse or promote products derived
* from this software without specific prior written permission.
*
* THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS ``AS IS'' AND ANY
* EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
* IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR
* PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR
* CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL,
* EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO,
* PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR
* PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY
* OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
* (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
* OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
************************************************************************/
#ifndef enqueue_h_
#define enqueue_h_
#include "core.h"
int getRingIndex(const ncclComm_t comm, int device);
void lockEventQueue(EventQueue* eq);
void releaseEventQueue(EventQueue* eq);
void CUDART_CB freeEvent(cudaStream_t stream, cudaError_t status, void* eq_void);
/* Syncronize with user stream and launch the collective.
* All work is performed asynchronously with the host thread.
* The actual collective should be a functor with the
* folloaing signature.
* ncclResult_t collective(void* sendbuff, void* recvbuff,
* int count, ncclDataType_t type, ncclRedOp_t op,
* int root, ncclComm_t comm);
* Unneeded arguments should be ignored. The collective may
* assume that the appropriate cuda device has been set. */
template<typename ColFunc>
ncclResult_t enqueue(ColFunc colfunc,
const void* sendbuff,
void* recvbuff,
int count,
ncclDataType_t type,
ncclRedOp_t op,
int root,
ncclComm_t comm,
cudaStream_t stream)
{
int curDevice;
CUDACHECK( cudaGetDevice(&curDevice) );
// No need for a mutex here because we assume that all enqueue operations happen in a fixed
// order on all devices. Thus, thread race conditions SHOULD be impossible.
EventQueue* eq = &comm->events;
// Ensure that previous collective is complete
cudaError_t flag = cudaEventQuery(eq->isDone[eq->back]);
if( flag == cudaErrorNotReady )
CUDACHECK( cudaStreamWaitEvent(stream, eq->isDone[eq->back], 0) );
// Launch the collective here
ncclResult_t ret = colfunc(sendbuff, recvbuff, count, type, op, root, comm, stream);
eq->back = (eq->back + 1) % MAXQUEUE;
CUDACHECK( cudaEventRecord(eq->isDone[eq->back], stream) );
return ret;
}
#endif // End include guard

254
src/libwrap.cu Normal file
View File

@ -0,0 +1,254 @@
/*************************************************************************
* Copyright (c) 2015, NVIDIA CORPORATION. All rights reserved.
*
* Redistribution and use in source and binary forms, with or without
* modification, are permitted provided that the following conditions
* are met:
* * Redistributions of source code must retain the above copyright
* notice, this list of conditions and the following disclaimer.
* * Redistributions in binary form must reproduce the above copyright
* notice, this list of conditions and the following disclaimer in the
* documentation and/or other materials provided with the distribution.
* * Neither the name of NVIDIA CORPORATION nor the names of its
* contributors may be used to endorse or promote products derived
* from this software without specific prior written permission.
*
* THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS ``AS IS'' AND ANY
* EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
* IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR
* PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR
* CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL,
* EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO,
* PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR
* PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY
* OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
* (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
* OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
************************************************************************/
#include "libwrap.h"
#include <dlfcn.h>
#include "core.h"
typedef enum { SUCCESS = 0 } RetCode;
int symbolsLoaded = 0;
static RetCode (*nvmlInternalInit)(void);
static RetCode (*nvmlInternalShutdown)(void);
static RetCode (*nvmlInternalDeviceGetHandleByPciBusId)(const char* pciBusId, nvmlDevice_t* device);
static RetCode (*nvmlInternalDeviceGetIndex)(nvmlDevice_t device, unsigned* index);
static RetCode (*nvmlInternalDeviceSetCpuAffinity)(nvmlDevice_t device);
static RetCode (*nvmlInternalDeviceClearCpuAffinity)(nvmlDevice_t device);
static const char* (*nvmlInternalErrorString)(RetCode r);
static CUresult (*cuInternalGetErrorString)(CUresult error, const char** pStr);
static CUresult (*cuInternalIpcGetMemHandle)(CUipcMemHandle* pHandle, CUdeviceptr dptr);
static CUresult (*cuInternalIpcOpenMemHandle)(CUdeviceptr* pdptr, CUipcMemHandle handle, unsigned int Flags);
static CUresult (*cuInternalIpcCloseMemHandle)(CUdeviceptr dptr);
ncclResult_t wrapSymbols(void) {
if (symbolsLoaded)
return ncclSuccess;
static void* nvmlhandle = NULL;
static void* cuhandle = NULL;
void* tmp;
void** cast;
nvmlhandle=dlopen("libnvidia-ml.so", RTLD_NOW);
if (!nvmlhandle) {
WARN("Failed to open libnvidia-ml.so");
goto teardown;
}
cuhandle = dlopen("libcuda.so", RTLD_NOW);
if (!cuhandle) {
WARN("Failed to open libcuda.so");
goto teardown;
}
#define LOAD_SYM(handle, symbol, funcptr) do { \
cast = (void**)&funcptr; \
tmp = dlsym(handle, symbol); \
if (tmp == NULL) { \
WARN("dlsym failed on %s - %s", symbol, dlerror()); \
goto teardown; \
} \
*cast = tmp; \
} while (0)
LOAD_SYM(nvmlhandle, "nvmlInit", nvmlInternalInit);
LOAD_SYM(nvmlhandle, "nvmlShutdown", nvmlInternalShutdown);
LOAD_SYM(nvmlhandle, "nvmlDeviceGetHandleByPciBusId", nvmlInternalDeviceGetHandleByPciBusId);
LOAD_SYM(nvmlhandle, "nvmlDeviceGetIndex", nvmlInternalDeviceGetIndex);
LOAD_SYM(nvmlhandle, "nvmlDeviceSetCpuAffinity", nvmlInternalDeviceSetCpuAffinity);
LOAD_SYM(nvmlhandle, "nvmlDeviceClearCpuAffinity", nvmlInternalDeviceClearCpuAffinity);
LOAD_SYM(nvmlhandle, "nvmlErrorString", nvmlInternalErrorString);
LOAD_SYM(cuhandle, "cuGetErrorString", cuInternalGetErrorString);
LOAD_SYM(cuhandle, "cuIpcGetMemHandle", cuInternalIpcGetMemHandle);
LOAD_SYM(cuhandle, "cuIpcOpenMemHandle", cuInternalIpcOpenMemHandle);
LOAD_SYM(cuhandle, "cuIpcCloseMemHandle", cuInternalIpcCloseMemHandle);
symbolsLoaded = 1;
return ncclSuccess;
teardown:
nvmlInternalInit = NULL;
nvmlInternalShutdown = NULL;
nvmlInternalDeviceGetHandleByPciBusId = NULL;
nvmlInternalDeviceGetIndex = NULL;
nvmlInternalDeviceSetCpuAffinity = NULL;
nvmlInternalDeviceClearCpuAffinity = NULL;
cuInternalGetErrorString = NULL;
cuInternalIpcGetMemHandle = NULL;
cuInternalIpcOpenMemHandle = NULL;
cuInternalIpcCloseMemHandle = NULL;
if (cuhandle != NULL) dlclose(cuhandle);
if (nvmlhandle != NULL) dlclose(nvmlhandle);
return ncclSystemError;
}
ncclResult_t wrapNvmlInit(void) {
if (nvmlInternalInit == NULL) {
WARN("lib wrapper not initilaized.");
return ncclLibWrapperNotSet;
}
RetCode ret = nvmlInternalInit();
if (ret != SUCCESS) {
WARN("nvmlInit() failed: %s",
nvmlInternalErrorString(ret));
return ncclSystemError;
}
return ncclSuccess;
}
ncclResult_t wrapNvmlShutdown(void) {
if (nvmlInternalShutdown == NULL) {
WARN("lib wrapper not initilaized.");
return ncclLibWrapperNotSet;
}
RetCode ret = nvmlInternalShutdown();
if (ret != SUCCESS) {
WARN("nvmlShutdown() failed: %s ",
nvmlInternalErrorString(ret));
return ncclSystemError;
}
return ncclSuccess;
}
ncclResult_t wrapNvmlDeviceGetHandleByPciBusId(const char* pciBusId, nvmlDevice_t* device) {
if (nvmlInternalDeviceGetHandleByPciBusId == NULL) {
WARN("lib wrapper not initilaized.");
return ncclLibWrapperNotSet;
}
RetCode ret = nvmlInternalDeviceGetHandleByPciBusId(pciBusId, device);
if (ret != SUCCESS) {
WARN("nvmlDeviceGetHandleByPciBusId() failed: %s ",
nvmlInternalErrorString(ret));
return ncclSystemError;
}
return ncclSuccess;
}
ncclResult_t wrapNvmlDeviceGetIndex(nvmlDevice_t device, unsigned* index) {
if (nvmlInternalDeviceGetIndex == NULL) {
WARN("lib wrapper not initilaized.");
return ncclLibWrapperNotSet;
}
RetCode ret = nvmlInternalDeviceGetIndex(device, index);
if (ret != SUCCESS) {
WARN("nvmlDeviceGetIndex() failed: %s ",
nvmlInternalErrorString(ret));
return ncclSystemError;
}
return ncclSuccess;
}
ncclResult_t wrapNvmlDeviceSetCpuAffinity(nvmlDevice_t device) {
if (nvmlInternalDeviceSetCpuAffinity == NULL) {
WARN("lib wrapper not initilaized.");
return ncclLibWrapperNotSet;
}
RetCode ret = nvmlInternalDeviceSetCpuAffinity(device);
if (ret != SUCCESS) {
WARN("nvmlDeviceSetCpuAffinity() failed: %s ",
nvmlInternalErrorString(ret));
return ncclSystemError;
}
return ncclSuccess;
}
ncclResult_t wrapNvmlDeviceClearCpuAffinity(nvmlDevice_t device) {
if (nvmlInternalInit == NULL) {
WARN("lib wrapper not initilaized.");
return ncclLibWrapperNotSet;
}
RetCode ret = nvmlInternalDeviceClearCpuAffinity(device);
if (ret != SUCCESS) {
WARN("nvmlDeviceClearCpuAffinity() failed: %s ",
nvmlInternalErrorString(ret));
return ncclSystemError;
}
return ncclSuccess;
}
ncclResult_t wrapCuIpcGetMemHandle(CUipcMemHandle* pHandle, CUdeviceptr dptr) {
if (cuInternalIpcGetMemHandle == NULL) {
WARN("lib wrapper not initilaized.");
return ncclLibWrapperNotSet;
}
CUresult ret = cuInternalIpcGetMemHandle(pHandle, dptr);
if (ret != CUDA_SUCCESS) {
const char* reason = NULL;
cuInternalGetErrorString(ret, &reason);
if (reason != NULL)
WARN("cuInternalIpcGetMemHandle() failed: %s ", reason);
else
WARN("cuInternalIpcGetMemHandle() failed: %d ", ret);
return ncclSystemError;
}
return ncclSuccess;
}
ncclResult_t wrapCuIpcOpenMemHandle(CUdeviceptr* pdptr, CUipcMemHandle handle, unsigned int Flags) {
if (cuInternalIpcOpenMemHandle == NULL) {
WARN("lib wrapper not initilaized.");
return ncclLibWrapperNotSet;
}
CUresult ret = cuInternalIpcOpenMemHandle(pdptr, handle, Flags);
if (ret != CUDA_SUCCESS) {
const char* reason = NULL;
cuInternalGetErrorString(ret, &reason);
if (reason != NULL)
WARN("cuInternalIpcOpenMemHandle() failed: %s ", reason);
else
WARN("cuInternalIpcOpenMemHandle() failed: %d ", ret);
return ncclSystemError;
}
return ncclSuccess;
}
ncclResult_t wrapCuIpcCloseMemHandle(CUdeviceptr dptr) {
if (cuInternalIpcCloseMemHandle == NULL) {
WARN("lib wrapper not initilaized.");
return ncclLibWrapperNotSet;
}
CUresult ret = cuInternalIpcCloseMemHandle(dptr);
if (ret != CUDA_SUCCESS) {
const char* reason = NULL;
cuInternalGetErrorString(ret, &reason);
if (reason != NULL)
WARN("cuInternalIpcCloseMemHandle() failed: %s ", reason);
else
WARN("cuInternalIpcCloseMemHandle() failed: %d ", ret);
return ncclSystemError;
}
return ncclSuccess;
}

54
src/libwrap.h Normal file
View File

@ -0,0 +1,54 @@
/*************************************************************************
* Copyright (c) 2015, NVIDIA CORPORATION. All rights reserved.
*
* Redistribution and use in source and binary forms, with or without
* modification, are permitted provided that the following conditions
* are met:
* * Redistributions of source code must retain the above copyright
* notice, this list of conditions and the following disclaimer.
* * Redistributions in binary form must reproduce the above copyright
* notice, this list of conditions and the following disclaimer in the
* documentation and/or other materials provided with the distribution.
* * Neither the name of NVIDIA CORPORATION nor the names of its
* contributors may be used to endorse or promote products derived
* from this software without specific prior written permission.
*
* THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS ``AS IS'' AND ANY
* EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
* IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR
* PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR
* CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL,
* EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO,
* PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR
* PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY
* OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
* (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
* OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
************************************************************************/
// Dynamically handle dependencies on external libraries (other than cudart).
#ifndef SRC_LIBWRAP_H_
#define SRC_LIBWRAP_H_
#include "core.h"
#include "cuda.h"
typedef struct nvmlDevice_st* nvmlDevice_t;
ncclResult_t wrapSymbols(void);
ncclResult_t wrapNvmlInit(void);
ncclResult_t wrapNvmlShutdown(void);
ncclResult_t wrapNvmlDeviceGetHandleByPciBusId(const char* pciBusId, nvmlDevice_t* device);
ncclResult_t wrapNvmlDeviceGetIndex(nvmlDevice_t device, unsigned* index);
ncclResult_t wrapNvmlDeviceSetCpuAffinity(nvmlDevice_t device);
ncclResult_t wrapNvmlDeviceClearCpuAffinity(nvmlDevice_t device);
ncclResult_t wrapCuIpcGetMemHandle(CUipcMemHandle* pHandle, CUdeviceptr dptr);
ncclResult_t wrapCuIpcOpenMemHandle(CUdeviceptr* pdptr, CUipcMemHandle handle, unsigned int Flags);
ncclResult_t wrapCuIpcCloseMemHandle(CUdeviceptr dptr);
#endif // End include guard

83
src/mpi_test.cu Normal file
View File

@ -0,0 +1,83 @@
/*************************************************************************
* Copyright (c) 2015, NVIDIA CORPORATION. All rights reserved.
*
* Redistribution and use in source and binary forms, with or without
* modification, are permitted provided that the following conditions
* are met:
* * Redistributions of source code must retain the above copyright
* notice, this list of conditions and the following disclaimer.
* * Redistributions in binary form must reproduce the above copyright
* notice, this list of conditions and the following disclaimer in the
* documentation and/or other materials provided with the distribution.
* * Neither the name of NVIDIA CORPORATION nor the names of its
* contributors may be used to endorse or promote products derived
* from this software without specific prior written permission.
*
* THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS ``AS IS'' AND ANY
* EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
* IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR
* PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR
* CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL,
* EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO,
* PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR
* PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY
* OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
* (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
* OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
************************************************************************/
#include <sys/types.h>
#include <unistd.h>
#include "nccl.h"
#include "mpi.h"
#define CUDACHECK(cmd) do { \
cudaError_t e = cmd; \
if( e != cudaSuccess ) { \
printf("Cuda failure %s:%d '%s'\n", \
__FILE__,__LINE__,cudaGetErrorString(e)); \
exit(EXIT_FAILURE); \
} \
} while(false)
int main(int argc, char *argv[]) {
ncclUniqueId commId;
int size, rank;
int ret;
MPI_Init(&argc, &argv);
MPI_Comm_size(MPI_COMM_WORLD, &size);
MPI_Comm_rank(MPI_COMM_WORLD, &rank);
int gpu = atoi(argv[rank+1]);
printf("MPI Rank %d running on GPU %d\n", rank, gpu);
// We have to set our device before NCCL init
CUDACHECK(cudaSetDevice(gpu));
MPI_Barrier(MPI_COMM_WORLD);
ncclComm_t comm;
// Let's use rank 0 PID as job ID
ncclGetUniqueId(&commId);
MPI_Bcast(&commId, NCCL_UNIQUE_ID_BYTES, MPI_CHAR, 0, MPI_COMM_WORLD);
ret = ncclCommInitRank(&comm, size, commId, rank);
if (ret != ncclSuccess) {
printf("NCCL Init failed : %d\n", ret);
exit(1);
}
int *dptr;
CUDACHECK(cudaMalloc(&dptr, 1024*2*sizeof(int)));
int val = rank;
CUDACHECK(cudaMemcpy(dptr, &val, sizeof(int), cudaMemcpyHostToDevice));
ncclAllReduce((const void*)dptr, (void*)(dptr+1024), 1024, ncclInt, ncclSum, comm, cudaStreamDefault);
CUDACHECK(cudaMemcpy(&val, (dptr+1024), sizeof(int), cudaMemcpyDeviceToHost));
printf("Sum is %d\n", val);
CUDACHECK(cudaFree(dptr));
MPI_Finalize();
ncclCommDestroy(comm);
return 0;
}

198
src/nccl.h Normal file
View File

@ -0,0 +1,198 @@
/*************************************************************************
* Copyright (c) 2015, NVIDIA CORPORATION. All rights reserved.
*
* Redistribution and use in source and binary forms, with or without
* modification, are permitted provided that the following conditions
* are met:
* * Redistributions of source code must retain the above copyright
* notice, this list of conditions and the following disclaimer.
* * Redistributions in binary form must reproduce the above copyright
* notice, this list of conditions and the following disclaimer in the
* documentation and/or other materials provided with the distribution.
* * Neither the name of NVIDIA CORPORATION nor the names of its
* contributors may be used to endorse or promote products derived
* from this software without specific prior written permission.
*
* THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS ``AS IS'' AND ANY
* EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
* IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR
* PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR
* CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL,
* EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO,
* PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR
* PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY
* OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
* (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
* OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
************************************************************************/
#ifndef NCCL_H_
#define NCCL_H_
#include <cuda_runtime.h>
#if CUDART_VERSION >= 7050
#include <cuda_fp16.h>
#define CUDA_HAS_HALF 1
#else
#undef CUDA_HAS_HALF
#endif
#ifdef __cplusplus
extern "C" {
#endif
/* Opaque handle to communicator */
typedef struct ncclComm* ncclComm_t;
#define NCCL_UNIQUE_ID_BYTES 128
typedef struct { char internal[NCCL_UNIQUE_ID_BYTES]; } ncclUniqueId;
/* Error type */
typedef enum { ncclSuccess = 0,
ncclUnhandledCudaError = 1,
ncclSystemError = 2,
ncclInternalError = 3,
ncclInvalidDevicePointer = 4,
ncclInvalidRank = 5,
ncclUnsupportedDeviceCount = 6,
ncclDeviceNotFound = 7,
ncclInvalidDeviceIndex = 8,
ncclLibWrapperNotSet = 9,
ncclCudaMallocFailed = 10,
ncclRankMismatch = 11,
ncclInvalidArgument = 12,
ncclInvalidType = 13,
ncclInvalidOperation = 14,
nccl_NUM_RESULTS = 15 } ncclResult_t;
/* Generates a unique Id with each call. Used to generate commId for
* ncclCommInitAll. uniqueId will be created in such a way that it is
* guaranteed to be unique accross the host. */
ncclResult_t ncclGetUniqueId(ncclUniqueId* uniqueId);
/* Creates a new communicator (multi process version).
* rank must be between 0 and ndev-1 and unique within a communicator clique.
* ndev is number of logical devices
* The communicator is created on the current CUDA device.
* ncclCommInitRank implicitly syncronizes with other ranks, so INIT OF EACH RANK MUST
* BE CALLED IN A SEPARATE HOST THREADS to avoid deadlock. */
ncclResult_t ncclCommInitRank(ncclComm_t* comm, int ndev, ncclUniqueId commId, int rank);
/* Creates a clique of communicators.
* This is a convenience function to create a single-process communicator clique.
* Returns an array of ndev newly initialized communicators in comm.
* comm should be pre-allocated with size at least ndev*sizeof(ncclComm_t).
* If devlist is NULL, the first ndev CUDA devices are used.
* Order of devlist defines user-order of processors within the communicator. */
ncclResult_t ncclCommInitAll(ncclComm_t* comm, int ndev, int* devlist);
/* Frees resources associated with communicator object. */
void ncclCommDestroy(ncclComm_t comm);
/* Returns nice error message. */
const char* ncclGetErrorString(ncclResult_t result);
/* Sets count to number of devices in the communicator clique. */
ncclResult_t ncclCommCount(const ncclComm_t comm, int* count);
/* Returns cuda device number associated with communicator. */
ncclResult_t ncclCommCuDevice(const ncclComm_t comm, int* device);
/* Returns user-ordered "rank" assocaiated with communicator. */
ncclResult_t ncclCommUserRank(const ncclComm_t comm, int* rank);
/* Reduction opperation selector */
typedef enum { ncclSum = 0,
ncclProd = 1,
ncclMax = 2,
ncclMin = 3,
nccl_NUM_OPS = 4 } ncclRedOp_t;
/* Data types */
typedef enum { ncclChar = 0,
ncclInt = 1,
#ifdef CUDA_HAS_HALF
ncclHalf = 2,
#endif
ncclFloat = 3,
ncclDouble = 4,
nccl_NUM_TYPES = 5 } ncclDataType_t;
/* Reduces data arrays of length count in sendbuff into recvbuf using op operation.
* recvbuf may be NULL on all calls except for root device.
* On the root device, sendbuff and recvbuff are assumed to reside on
* the same device.
* Must be called separately for each communicator in communicator clique.
*/
ncclResult_t ncclReduce(const void* sendbuff, void* recvbuf, int count, ncclDataType_t datatype,
ncclRedOp_t op, int root, ncclComm_t comm, cudaStream_t stream);
/* Reduces data arrays of length count in sendbuff using op operation, and leaves
* identical copies of result on each GPUs recvbuff.
* Sendbuff and recvbuff are assumed to reside on the same device.
* Must be called separately for each communicator in communicator clique. */
ncclResult_t ncclAllReduce(const void* sendbuff, void* recvbuff, int count,
ncclDataType_t datatype, ncclRedOp_t op, ncclComm_t comm, cudaStream_t stream);
/* Reduces data in sendbuff using op operation and leaves reduced result scattered
* over the devices so that recvbuff on the i-th GPU will contain the i-th block of
* the result. Sendbuff and recvbuff are assumed to reside on same device. Assumes
* sendbuff has size at least ndev*recvcount elements, where ndev is number of
* communicators in communicator clique
* Must be called separately for each communicator in communicator clique.*/
ncclResult_t ncclReduceScatter(const void* sendbuff, void* recvbuff,
int recvcount, ncclDataType_t datatype, ncclRedOp_t op, ncclComm_t comm,
cudaStream_t stream);
/* Copies count values from root to all other devices.
* Root specifies the source device in user-order
* (see ncclCommInit).
* Must be called separately for each communicator in communicator clique. */
ncclResult_t ncclBcast(void* buff, int count, ncclDataType_t datatype, int root,
ncclComm_t comm, cudaStream_t stream);
/* Each device gathers count values from other GPUs.
* Result is ordered by comm's logical device order.
* Assumes recvbuff has size at least ndev*count, where ndev is number of communicators
* in communicator clique.
* Sendbuff and recvbuff are assumed to reside on same device.
* Must be called separately for each communicator in communicator clique. */
ncclResult_t ncclAllGather(const void* sendbuff, int count, ncclDataType_t datatype,
void* recvbuff, ncclComm_t comm, cudaStream_t stream);
/* The following collective operations are not implemented yet */
///* Gather count values from each device to recvbuff.
// * Result is ordered by comm's logical device order.
// * recvbuff may be NULL for all calls except for root device.
// * On the root device, sendbuff and recvbuff are assumed to reside on the same device.
// * Must be called separately for each communicator in communicator clique. */
// * All GPUs, including root, perform copies into recvbuff.
//ncclResult_t ncclGather(const void* sendbuff, int count, ncclDataType_t datatype,
// void* recvbuff, int root, ncclComm_t comm, cudaStream_t stream);
///* Root device scatters count values to each devices.
// * sendbuff may be NULL on all devices except a single root
// * device where it is assumed to have size at least nGPUs*count.
// * recvbuff allocated on each gpu, including root, size=count.
// * Result is ordered by comm's logical device order.
// * Called separately for each device in the ncclComm. */
//ncclResult_t ncclScatter(void* sendbuff, ncclDataType_t datatype, void* recvbuff,
// int count, int root, ncclComm_t comm, cudaStream_t stream);
//
///* All GPUs scatter blocks of count elements to other devices.
// * Must be called separately for each device in the ncclComm.
// * sendbuff and recvbuff assumed to reside on same device and
// * have size at least nGPUs*count.
// * Called separately for each device in the ncclComm. */
//ncclResult_t ncclAllToAll(void* sendbuff, int count, ncclDataType_t datatype,
// void* recvbuff, ncclComm_t comm, cudaStream_t stream);
#ifdef __cplusplus
} // end extern "C"
#endif
#endif // end include guard

412
src/reduce.cu Normal file
View File

@ -0,0 +1,412 @@
/*************************************************************************
* Copyright (c) 2015, NVIDIA CORPORATION. All rights reserved.
*
* Redistribution and use in source and binary forms, with or without
* modification, are permitted provided that the following conditions
* are met:
* * Redistributions of source code must retain the above copyright
* notice, this list of conditions and the following disclaimer.
* * Redistributions in binary form must reproduce the above copyright
* notice, this list of conditions and the following disclaimer in the
* documentation and/or other materials provided with the distribution.
* * Neither the name of NVIDIA CORPORATION nor the names of its
* contributors may be used to endorse or promote products derived
* from this software without specific prior written permission.
*
* THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS ``AS IS'' AND ANY
* EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
* IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR
* PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR
* CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL,
* EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO,
* PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR
* PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY
* OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
* (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
* OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
************************************************************************/
#include <algorithm>
#include <nvToolsExt.h>
#include "core.h"
#include "common_kernel.h"
#include "copy_kernel.h"
#include "enqueue.h"
#include "reduce_kernel.h"
/* HIERARCHY
*
* The data is split into CHUNKS, and each CHUNK is split into NUM_SUBCHUNKS
* SUBCHUNKS, where each SUBCHUNK is processed independently. A SUBCHUNK is
* split into numUnroll UNROLLS and each thread performs UNROLL_COUNT
* single-data-element operations inside an UNROLL. As the name suggests, the
* UNROLL_COUNT operations within an UNROLL are unrolled.
*/
// Number of threads used to perform copies, etc. Must be multiple of 32.
// An additional thread is used to handle threadfences, so the CUDA blocks
// have dimension NUM_THREADS+1.
#define NUM_THREADS 256
// Each thread unrolls the innermost loop of the copy or reduction operations
// to this many single-data-element instructions
#define UNROLL_COUNT 8
#define UNROLL_SIZE (UNROLL_COUNT * NUM_THREADS)
// To hide the latency associated with the synchronization between different
// subchunks, we interleave the independent subchunks so that more data can be
// transferred while the sync is in progress. This is the number of subchunks
// that are active at the same time
#define NUM_SUBCHUNKS 4
// if this is called with CHUNK, it means that we just finished pushing the data
// of chunk CHUNK to the next GPU, so it can proceed with CHUNK
// We add 1 to chunk so that the initial flag of 0 doesn't allow the non-root
// GPUs to proceed before the flag is incremented from the upstream GPU. This
// is called by one particular consumer warp and so we select the first thread
// in the warp to set the flag.
#define SIGNAL_NEW_DATA_AVAILABLE(chunk, subchunk) \
do { \
__threadfence_system(); \
args.NextNewDataAvailableFlag[0] = NUM_SUBCHUNKS*(chunk) + subchunk + 1; \
} while (0)
// This is called by all producer threads, but only thread 0 spins on the flag,
#define WAIT_FOR_NEW_DATA(chunk, subchunk) \
do { \
if (tid == 0) { \
Wait([=] { \
return ((volatile int *)args.ThisNewDataAvailableFlag)[0] >= \
NUM_SUBCHUNKS*(chunk) + subchunk + 1; \
}); \
} \
BAR(sync, 1, NUM_THREADS); \
} while (0)
// If this is called with CHUNK, it means that this GPU has just finished
// processing the chunk CHUNK and so the previous GPU can start with CHUNK + 1
#define SIGNAL_CHUNK_DONE(chunk, subchunk) \
do { \
args.PrevChunkDoneFlag[0] = NUM_SUBCHUNKS*(chunk) + subchunk + 1; \
} while (0)
// This is called by all producer threads, but only thread 0 spins on the flag,
// all threads synchronize after thread 0 is done spinning.
#define WAIT_FOR_CHUNK(chunk, subchunk) \
do { \
if (tid == 0) { \
Wait([=] { \
return ((volatile int *)args.ThisChunkDoneFlag)[0] >= \
NUM_SUBCHUNKS*(chunk) + subchunk + 1 - NUM_SUBCHUNKS; \
}); \
} \
BAR(sync, 1, NUM_THREADS); \
} while (0)
// This is called by all producer threads, but only thread 0 spins on the flag,
// all threads synchronize after thread 0 is done spinning.
#define WAIT_FOR_NEW_DATA_AND_CHUNK(chunk, subchunk) \
do { \
if (tid == 0) { \
Wait([=] { \
bool newDataAvailable = \
((volatile int *)args.ThisNewDataAvailableFlag)[0] >= \
NUM_SUBCHUNKS*(chunk) + subchunk + 1; \
bool chunkDone = \
((volatile int *)args.ThisChunkDoneFlag)[0] >= \
NUM_SUBCHUNKS*(chunk)+subchunk + 1 - NUM_SUBCHUNKS; \
return newDataAvailable && chunkDone; \
}); \
} \
BAR(sync, 1, NUM_THREADS); \
} while (0)
__device__ inline void getSliceSizeAndOffset(int *size, int *offset, int slice,
int numSlices, int numBigSlices, int numSmallSlices, int bigSliceN,
int smallSliceN, int lastSliceN) {
if (slice < numBigSlices) {
*size = bigSliceN;
*offset = slice * bigSliceN;
} else {
*size = (slice < numBigSlices + numSmallSlices) ? smallSliceN
: ((slice == numSlices - 1) ? lastSliceN : 0);
*offset = numBigSlices * bigSliceN + (slice - numBigSlices) * smallSliceN;
}
// if (threadIdx.x == 0)
// printf("[size=%d] [offset=%d] slice=%d numSlices=%d "
// "numBigSlices=%d numSmallSlices=%d bigSliceN=%d smallSliceN=%d "
// "lastSliceN=%d\n", *size, *offset, slice, numSlices, numBigSlices,
// numSmallSlices, bigSliceN, smallSliceN, lastSliceN);
}
template<typename T>
struct ReduceKernelArgs {
// general parameters
int ThisId;
int N;
// some pre-computed sizes
int SliceSize;
int ChunkSize;
int NumChunks;
int BufferSliceStride;
T ** ThisPtrToNextData;
T ** PrevPtrToThisData;
// local and remote data
T * __restrict__ Output;
const T * __restrict__ ThisData;
volatile T * __restrict__ ThisBuffer;
volatile T * __restrict__ NextBuffer;
// local and remote flags
volatile int * __restrict__ ThisNewDataAvailableFlag;
volatile int * __restrict__ NextNewDataAvailableFlag;
volatile int * __restrict__ ThisChunkDoneFlag;
volatile int * __restrict__ PrevChunkDoneFlag;
};
__shared__ volatile void * nextData;
enum ReduceRole {BEGIN=0, MIDDLE=1, END=2};
template<int THREADS, int UNROLL, class FUNC, int ROLE, typename T>
__global__ void ReduceKernel(const ReduceKernelArgs<T> args) {
if (args.N == 0) return;
int tid = threadIdx.x;
// First wait for args.PrevPtrToThisOutput to become nullptr to ensure that
// the previous GPU is done with a previous collective operation.
if (tid == 0) {
if (ROLE != BEGIN) {
Wait([=] {
return *((T * volatile *)args.PrevPtrToThisData) == nullptr; // Wait for previous processor to be done
});
*((T * volatile *)args.PrevPtrToThisData) = (T*)args.ThisData; // Tell Previous I'm starting
}
if (ROLE != END) {
Wait([=] {
return *((T * volatile *)args.ThisPtrToNextData) != nullptr; // Wait till I've been told next started
});
}
}
__syncthreads();
for (int chunk = 0; chunk < args.NumChunks; ++chunk) {
// calculate slice size. for all chunks except (possibly) the last one,
// this will just be args.SliceSize. For the last one, it may be smaller
int bigSliceN = args.SliceSize;
int smallSliceN = 0;
int lastSliceN = 0;
int numSlices = NUM_SUBCHUNKS;
int numBigSlices = numSlices;
int numSmallSlices = 0;
// last chunk
if ((chunk + 1 == args.NumChunks) && (args.N % args.ChunkSize > 0))
CalcLastChunk<THREADS, UNROLL, T>(&bigSliceN, &smallSliceN, &lastSliceN,
&numSlices, &numBigSlices, &numSmallSlices, args.N, args.NumChunks,
args.ChunkSize);
// this offset is only applied to Data pointers, not to Buffer pointers,
// since we only have one buffer per chunk
int chunkOffset = chunk * args.ChunkSize;
int offset;
int sliceSize;
if (tid < THREADS) {
for(int s=0; s<NUM_SUBCHUNKS; ++s) {
getSliceSizeAndOffset(&sliceSize, &offset, s, numSlices,
numBigSlices, numSmallSlices, bigSliceN, smallSliceN, lastSliceN);
if (ROLE == BEGIN) {
WAIT_FOR_CHUNK(chunk, s);
Copy<UNROLL, THREADS>(
args.NextBuffer + (s * args.BufferSliceStride),
args.ThisData + chunkOffset + offset,
sliceSize);
} else if (ROLE == MIDDLE) {
WAIT_FOR_NEW_DATA_AND_CHUNK(chunk, s);
Reduce<UNROLL, THREADS, FUNC>(
args.NextBuffer + (s * args.BufferSliceStride),
args.ThisData + chunkOffset + offset,
args.ThisBuffer + (s * args.BufferSliceStride),
sliceSize);
} else { // ROLE == END
WAIT_FOR_NEW_DATA(chunk, s);
Reduce<UNROLL, THREADS, FUNC>(
args.Output + chunkOffset + offset,
args.ThisData + chunkOffset + offset,
args.ThisBuffer + (s * args.BufferSliceStride),
sliceSize);
}
__syncthreads();
}
} else { // Consumer thread
for(int s=0; s<NUM_SUBCHUNKS; ++s) {
__syncthreads();
if (ROLE != END)
SIGNAL_NEW_DATA_AVAILABLE(chunk, s);
// signal chunk done if we don't push into the receive buffer and this
// is no the last chunk and this is not root
if ((ROLE != BEGIN) && (chunk + 1 < args.NumChunks)) {
SIGNAL_CHUNK_DONE(chunk, s);
}
}
}
}
// reset flags
if (tid == 0) {
args.ThisNewDataAvailableFlag[0] = 0;
args.ThisChunkDoneFlag[0] = 0;
*args.ThisPtrToNextData = nullptr;
}
}
template<class FUNC, typename T>
ncclResult_t ncclReduceWithTypeAndFunc(const void* sendbuff, void* recvbuff,
const int count, const int root, ncclComm* comm, cudaStream_t stream) {
if (count == 0)
return ncclSuccess;
int index = comm->ncclId;
const int numUnroll = 4;
int rootId = comm->ringFromUser[root];
int nextId = (index + 1) % comm->nDev;
int prevId = (index + comm->nDev - 1) % comm->nDev;
// There is one slice per GPU, so a slice can be at most bufferN / numGPUs,
// where bufferN is the number of elements of type T that fit into the buffer.
// For efficiency, we want the slice size to be a multiple of UNROLL_SIZE
int bufferN = comm->buffSize / sizeof(T);
// we only need buffer for k slices and k paddings
int bufferNPerSlice = bufferN / NUM_SUBCHUNKS;
int maxSliceSize = (bufferNPerSlice / UNROLL_SIZE) * UNROLL_SIZE;
ReduceKernelArgs<T> args;
args.ThisId = index;
args.N = count;
args.SliceSize = numUnroll * UNROLL_SIZE * sizeof(PackType) / sizeof(T);
if(!comm->useRemoteRecv) {
// Proxy for QPI. Reduce never pushes directly to recv.
// But larger transfers help QPI more than tag updates hurt P2P.
args.SliceSize *= 8;
}
// make sure slice fits into the temporary buffer
args.SliceSize = std::min(maxSliceSize, args.SliceSize);
args.BufferSliceStride = args.SliceSize;
args.ChunkSize = NUM_SUBCHUNKS * args.SliceSize;
// avoid a case where we have one or more big chunks and one tiny one
int remainder = args.N % args.ChunkSize;
if ((args.N > args.ChunkSize) && (remainder > 0) &&
(args.N < 5 * args.ChunkSize) && (2 * remainder < args.ChunkSize)) {
args.SliceSize /= 2;
args.ChunkSize = NUM_SUBCHUNKS * args.SliceSize;
// round down so we end up with a big last chunk
args.NumChunks = args.N / args.ChunkSize;
} else {
// round up
args.NumChunks = (args.N + args.ChunkSize - 1) / args.ChunkSize;
}
args.ThisPtrToNextData = (T**)&(comm->local[nextId]->recvPtrs[0]);
args.PrevPtrToThisData = (T**)&(comm->remote[prevId]->recvPtrs[0]);
args.Output = (T*)recvbuff;
args.ThisData = (const T*) sendbuff;
args.ThisBuffer = (volatile T*)comm->local[prevId]->buff;
args.NextBuffer = (volatile T*)comm->remote[nextId]->buff;
args.ThisNewDataAvailableFlag = comm->local[prevId]->flags;
args.NextNewDataAvailableFlag = comm->remote[nextId]->flags;
args.ThisChunkDoneFlag = comm->local[nextId]->flags + 1;
args.PrevChunkDoneFlag = comm->remote[prevId]->flags + 1;
if (index == (rootId + 1) % comm->nDev) {
ReduceKernel<NUM_THREADS, UNROLL_COUNT, FUNC, BEGIN, T>
<<<1, NUM_THREADS + 1, 0, stream>>>(args);
} else if (index == rootId) {
ReduceKernel<NUM_THREADS, UNROLL_COUNT, FUNC, END, T>
<<<1, NUM_THREADS + 1, 0, stream>>>(args);
} else {
ReduceKernel<NUM_THREADS, UNROLL_COUNT, FUNC, MIDDLE, T>
<<<1, NUM_THREADS + 1, 0, stream>>>(args);
}
return ncclSuccess;
}
template <typename T>
ncclResult_t ncclReduceWithType(const void* sendbuff,
void* recvbuff, int count, ncclRedOp_t op, int root,
ncclComm* comm, cudaStream_t stream) {
switch (op) {
case ncclSum:
return ncclReduceWithTypeAndFunc<FuncSum<T>, T>(
sendbuff, recvbuff, count, root, comm, stream);
case ncclProd:
return ncclReduceWithTypeAndFunc<FuncProd<T>, T>(
sendbuff, recvbuff, count, root, comm, stream);
case ncclMax:
return ncclReduceWithTypeAndFunc<FuncMax<T>, T>(
sendbuff, recvbuff, count, root, comm, stream);
case ncclMin:
return ncclReduceWithTypeAndFunc<FuncMin<T>, T>(
sendbuff, recvbuff, count, root, comm, stream);
}
return ncclInvalidOperation;
}
class ReduceFunctor {
public:
ncclResult_t operator()(const void* sendbuff,
void* recvbuff, int count, ncclDataType_t datatype, ncclRedOp_t op,
int root, ncclComm* comm, cudaStream_t stream) {
switch (datatype) {
case ncclChar:
return ncclReduceWithType<char>(sendbuff, recvbuff, count, op, root, comm, stream);
case ncclInt:
return ncclReduceWithType<int>(sendbuff, recvbuff, count, op, root, comm, stream);
#ifdef CUDA_HAS_HALF
case ncclHalf:
return ncclReduceWithType<half>(sendbuff, recvbuff, count, op, root, comm, stream);
#endif
case ncclFloat:
return ncclReduceWithType<float>(sendbuff, recvbuff, count, op, root, comm, stream);
case ncclDouble:
return ncclReduceWithType<double>(sendbuff, recvbuff, count, op, root, comm, stream);
}
return ncclInvalidType;
}
};
extern "C" DSOGLOBAL
ncclResult_t ncclReduce(const void* sendbuff, void* recvbuff, int count,
ncclDataType_t datatype, ncclRedOp_t op, int root, ncclComm_t comm,
cudaStream_t stream) {
return enqueue(ReduceFunctor(), sendbuff, recvbuff, count, datatype, op,
root, comm, stream);
}

289
src/reduce_kernel.h Normal file
View File

@ -0,0 +1,289 @@
/*************************************************************************
* Copyright (c) 2015, NVIDIA CORPORATION. All rights reserved.
*
* Redistribution and use in source and binary forms, with or without
* modification, are permitted provided that the following conditions
* are met:
* * Redistributions of source code must retain the above copyright
* notice, this list of conditions and the following disclaimer.
* * Redistributions in binary form must reproduce the above copyright
* notice, this list of conditions and the following disclaimer in the
* documentation and/or other materials provided with the distribution.
* * Neither the name of NVIDIA CORPORATION nor the names of its
* contributors may be used to endorse or promote products derived
* from this software without specific prior written permission.
*
* THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS ``AS IS'' AND ANY
* EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
* IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR
* PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR
* CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL,
* EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO,
* PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR
* PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY
* OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
* (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
* OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
************************************************************************/
#ifndef REDUCE_KERNEL_H_
#define REDUCE_KERNEL_H_
#include "common_kernel.h"
template<typename T>
struct FuncSum {
__device__ T operator()(const T x, const T y) const {
return x + y;
}
};
template<typename T>
struct FuncProd {
__device__ T operator()(const T x, const T y) const {
return x * y;
}
};
template<typename T>
struct FuncMax {
__device__ T operator()(const T x, const T y) const {
return (x < y) ? y : x;
}
};
template<typename T>
struct FuncMin {
__device__ T operator()(const T x, const T y) const {
return (x < y) ? x : y;
}
};
template<>
struct FuncSum<char> {
union converter {
uint32_t storage;
char4 a;
};
__device__ uint32_t operator()(const uint32_t x, const uint32_t y) const {
#if (__CUDA_ARCH__ >= 300) && (__CUDA_ARCH__ < 500)
int32_t rv, z=0;
asm("vadd4.s32.s32.s32 %0, %1, %2, %3;" : "=r"(rv) : "r"(x), "r"(y), "r"(z));
return rv;
#elif (__CUDA_ARCH__ >= 500)
int32_t rv;
asm("vadd.s32.s32.s32 %0, %1.b0, %2.b0; \n\t"
"vadd.s32.s32.s32 %0.b1, %1.b1, %2.b1, %0;\n\t"
"vadd.s32.s32.s32 %0.b2, %1.b2, %2.b2, %0;\n\t"
"vadd.s32.s32.s32 %0.b3, %1.b3, %2.b3, %0;" : "=r"(rv) : "r"(x), "r"(y));
return rv;
#else
converter cx, cy, cr;
cx.storage = x;
cy.storage = y;
cr.a.x = cx.a.x + cy.a.x;
cr.a.y = cx.a.y + cy.a.y;
cr.a.z = cx.a.z + cy.a.z;
cr.a.w = cx.a.w + cy.a.w;
return cr.storage;
#endif
}
__device__ char operator()(const char x, const char y) const {
return x+y;
}
};
template<>
struct FuncProd<char> {
union converter { uint32_t storage; char4 a; };
__device__ uint32_t operator()(const uint32_t x, const uint32_t y) const {
#if (__CUDA_ARCH__ >= 300)
int32_t rv, zero=0;
asm("{ .reg .u32 t0, t1, t2, t3;\n\t"
" vmad.u32.u32.u32 t3, %1.b3, %2.b3, %3;\n\t"
" vmad.u32.u32.u32 t2, %1.b2, %2.b2, %3;\n\t"
" shl.b32 t3, t3, 16;\n\t"
" shl.b32 t2, t2, 16;\n\t"
" vmad.u32.u32.u32 t1, %1.b1, %2.b1, t3;\n\t"
" shl.b32 t1, t1, 8;\n\t"
" vmad.u32.u32.u32 t0, %1.b0, %2.b0, t2;\n\t"
" and.b32 t1, t1, 0xff00ff00;\n\t"
" and.b32 t0, t0, 0x00ff00ff;\n\t"
" or.b32 %0, t0, t1;\n\t"
"}" : "=r"(rv) : "r"(x), "r"(y), "r"(zero));
return rv;
#else
converter cx, cy, cr;
cx.storage = x;
cy.storage = y;
cr.a.x = cx.a.x * cy.a.x;
cr.a.y = cx.a.y * cy.a.y;
cr.a.z = cx.a.z * cy.a.z;
cr.a.w = cx.a.w * cy.a.w;
return cr.storage;
#endif
}
__device__ char operator()(const char x, const char y) const {
return x*y;
}
};
template<>
struct FuncMax<char> {
union converter { uint32_t storage; char4 a; };
__device__ uint32_t operator()(const uint32_t x, const uint32_t y) const {
#if (__CUDA_ARCH__ >= 300) && (__CUDA_ARCH__ < 500)
int32_t rv, z=0;
asm("vmax4.s32.s32.s32 %0, %1, %2, %3;" : "=r"(rv) : "r"(x), "r"(y), "r"(z));
return rv;
#elif (__CUDA_ARCH__ >= 500)
int32_t rv;
asm("vmax.s32.s32.s32 %0, %1.b0, %2.b0; \n\t"
"vmax.s32.s32.s32 %0.b1, %1.b1, %2.b1, %0;\n\t"
"vmax.s32.s32.s32 %0.b2, %1.b2, %2.b2, %0;\n\t"
"vmax.s32.s32.s32 %0.b3, %1.b3, %2.b3, %0;" : "=r"(rv) : "r"(x), "r"(y));
return rv;
#else
converter cx, cy, cr;
cx.storage = x;
cy.storage = y;
cr.a.x = max(cx.a.x, cy.a.x);
cr.a.y = max(cx.a.y, cy.a.y);
cr.a.z = max(cx.a.z, cy.a.z);
cr.a.w = max(cx.a.w, cy.a.w);
return cr.storage;
#endif
}
__device__ char operator()(const char x, const char y) const {
return (x>y) ? x : y;
}
};
template<>
struct FuncMin<char> {
union converter { uint32_t storage; char4 a; };
__device__ uint32_t operator()(const uint32_t x, const uint32_t y) const {
#if (__CUDA_ARCH__ >= 300) && (__CUDA_ARCH__ < 500)
int32_t rv, z=0;
asm("vmin4.s32.s32.s32 %0, %1, %2, %3;" : "=r"(rv) : "r"(x), "r"(y), "r"(z));
return rv;
#elif (__CUDA_ARCH__ >= 500)
int32_t rv;
asm("vmin.s32.s32.s32 %0, %1.b0, %2.b0; \n\t"
"vmin.s32.s32.s32 %0.b1, %1.b1, %2.b1, %0;\n\t"
"vmin.s32.s32.s32 %0.b2, %1.b2, %2.b2, %0;\n\t"
"vmin.s32.s32.s32 %0.b3, %1.b3, %2.b3, %0;" : "=r"(rv) : "r"(x), "r"(y));
return rv;
#else
converter cx, cy, cr;
cx.storage = x;
cy.storage = y;
cr.a.x = min(cx.a.x, cy.a.x);
cr.a.y = min(cx.a.y, cy.a.y);
cr.a.z = min(cx.a.z, cy.a.z);
cr.a.w = min(cx.a.w, cy.a.w);
return cr.storage;
#endif
}
__device__ char operator()(const char x, const char y) const {
return (x<y) ? x : y;
}
};
#ifdef CUDA_HAS_HALF
template<>
struct FuncSum<half> {
__device__ half2 operator()(const half2 x, const half2 y) const {
float2 fx, fy, fr;
fx = __half22float2(x);
fy = __half22float2(y);
fr.x = fx.x + fy.x;
fr.y = fx.y + fy.y;
return __float22half2_rn(fr);
}
__device__ half operator()(const half x, const half y) const {
return __float2half( __half2float(x) + __half2float(y) );
}
};
template<>
struct FuncProd<half> {
__device__ half2 operator()(const half2 x, const half2 y) const {
float2 fx, fy, fr;
fx = __half22float2(x);
fy = __half22float2(y);
fr.x = fx.x * fy.x;
fr.y = fx.y * fy.y;
return __float22half2_rn(fr);
}
__device__ half operator()(const half x, const half y) const {
return __float2half( __half2float(x) * __half2float(y) );
}
};
template<>
struct FuncMax<half> {
__device__ half2 operator()(const half2 x, const half2 y) const {
float2 fx, fy, fr;
fx = __half22float2(x);
fy = __half22float2(y);
fr.x = fx.x > fy.x ? fx.x : fy.x;
fr.y = fx.y > fy.y ? fx.y : fy.y;
return __float22half2_rn(fr);
}
__device__ half operator()(const half x, const half y) const {
float fx, fy, fm;
fx = __half2float(x);
fy = __half2float(y);
fm = fx > fy ? fx : fy;
return __float2half(fm);
}
};
template<>
struct FuncMin<half> {
__device__ half2 operator()(const half2 x, const half2 y) const {
float2 fx, fy, fr;
fx = __half22float2(x);
fy = __half22float2(y);
fr.x = fx.x < fy.x ? fx.x : fy.x;
fr.y = fx.y < fy.y ? fx.y : fy.y;
return __float22half2_rn(fr);
}
__device__ half operator()(const half x, const half y) const {
float fx, fy, fm;
fx = __half2float(x);
fy = __half2float(y);
fm = fx < fy ? fx : fy;
return __float2half(fm);
}
};
#endif
// Assumptions:
// - there is exactly 1 block
// - THREADS is the number of threads in the CTA
// - this function is called by all producer threads
template<int UNROLL, int THREADS, class FUNC, typename T>
__device__ void Reduce(volatile T * __restrict__ const dest,
const volatile T * __restrict__ const src0,
const volatile T * __restrict__ const src1, const int N) {
ReduceOrCopy<UNROLL, THREADS, FUNC, T, false, true>(threadIdx.x, dest,
nullptr, src0, src1, N);
}
// Assumptions:
// - there is exactly 1 block
// - THREADS is the number of threads in the CTA
// - this function is called by all producer threads
template<int UNROLL, int THREADS, class FUNC, typename T>
__device__ void ReduceAndCopy(volatile T * __restrict__ const dest0,
volatile T * __restrict__ const dest1,
const volatile T * __restrict__ const src0,
const volatile T * __restrict__ const src1, const int N) {
ReduceOrCopy<UNROLL, THREADS, FUNC, T, true, true>(threadIdx.x, dest0, dest1,
src0, src1, N);
}
#endif // REDUCE_KERNEL_H_

488
src/reduce_scatter.cu Normal file
View File

@ -0,0 +1,488 @@
/*************************************************************************
* Copyright (c) 2015, NVIDIA CORPORATION. All rights reserved.
*
* Redistribution and use in source and binary forms, with or without
* modification, are permitted provided that the following conditions
* are met:
* * Redistributions of source code must retain the above copyright
* notice, this list of conditions and the following disclaimer.
* * Redistributions in binary form must reproduce the above copyright
* notice, this list of conditions and the following disclaimer in the
* documentation and/or other materials provided with the distribution.
* * Neither the name of NVIDIA CORPORATION nor the names of its
* contributors may be used to endorse or promote products derived
* from this software without specific prior written permission.
*
* THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS ``AS IS'' AND ANY
* EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
* IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR
* PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR
* CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL,
* EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO,
* PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR
* PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY
* OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
* (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
* OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
************************************************************************/
#include <cassert>
#include "core.h"
#include "common_kernel.h"
#include "copy_kernel.h"
#include "enqueue.h"
#include "reduce_kernel.h"
/* HIERARCHY
*
* The data is split into CHUNKS, and each CHUNK is split into NUM_SUBCHUNKS
* SUBCHUNKS, where each SUBCHUNK is an independent, complete reduction. Each
* GPU has a buffer that can fit an entire CHUNK, so that all SUBCHUNKS can be
* processed without checking that the buffer on the receiving GPU is empty. A
* SUBCHUNK is split into NUM_GPUS SLICES and each GPU works on a different
* SLICE at the same time. Before moving on the the next SLICE in the reduction
* algorithm, the GPU has to check whether it has received the data from the
* previous GPU it needs for this SLICE. To hide the latency of this
* communication, each GPU processes all the SLICES of all the SUBCHUNKS in
* sequence before moving on to the next SLICE. Each SLICE is split into a
* certain number of UNROLLS (determined by the buffer size) and each thread
* performs UNROLL_COUNT single-data-element operations inside an UNROLL. As the
* name suggests, the UNROLL_COUNT operations within an UNROLL are unrolled.
*/
// Number of threads used to perform copies, etc. Must be multiple of 32.
// An additional thread is used to handle threadfences, so the CUDA blocks
// have dimension NUM_THREADS+1.
#define NUM_THREADS 256
// Each thread unrolls the innermost loop of the copy or reduction operations
// to this many single-data-element instructions
#define UNROLL_COUNT 8
#define UNROLL_SIZE (UNROLL_COUNT * NUM_THREADS)
// To hide the latency associated with the synchronization between different
// subchunks, we interleave the independent subchunks so that more data can be
// transferred while the sync is in progress. This is the number of subchunks
// that are active at the same time
#define NUM_SUBCHUNKS 2
/*
* numGPUs BLOCKs consisting of recvcount words each
* BLOCK is split up into NumChunks CHUNKs
* CHUNK is split up into NUM_SUBCHUNKS SUBCHUNKs
* SUBCHUNK consists of exactly one SLICE
* SLICE is most efficiently processed in multiples of UNROLL_SIZE
*
* The algorithm has numGPUs steps and each step processes a SLICE (i.e.
* SUBCHUNK) of a different BLOCK. Only data of the BLOCKs not resident on the
* GPU need to be communicated, hence (numGPUs - 1) BLOCKs. So the buffer needs
* to have room for (numGPUs - 1) SLICEs.
*/
// do not encode the subchunk number into the flag, because there is a separate
// flag for each subchunk
// If this is called with STEP, it means that we just finished processing the
// data for step STEP on this GPU, which is the data required on the next GPU
// for step STEP + 1, so we signal the next GPU that its data for step STEP + 1
// is available. This is called by one particular consumer warp and so we select
// the first thread in the warp to set the flag.
#define SIGNAL_NEW_DATA_AVAILABLE(chunk, subchunk, step) \
do { \
args.NextNewDataAvailableFlag[0] = \
2*((chunk) * args.NumGPUs + (step)) + subchunk + 1; \
} while (0)
// This is called by all producer threads, but only thread 0 spins on the flag,
// all threads synchronize after thread 0 is done spinning.
#define WAIT_FOR_NEW_DATA(chunk, subchunk, step) \
do { \
if (tid == 0) { \
Wait([=] { \
return ((volatile int *)args.ThisNewDataAvailableFlag)[0] >= \
2*((chunk) * args.NumGPUs + (step)) + subchunk - 1; \
}); \
} \
BAR(sync, 1, NUM_THREADS); \
} while (0)
// If this is called with CHUNK, it means that this GPU has just finished
// processing the chunk CHUNK and so the previous GPU can start with CHUNK + 1
#define SIGNAL_CHUNK_DONE(chunk, subchunk) \
do { \
args.PrevChunkDoneFlag[0] = 2*(chunk) + subchunk + 1; \
} while (0)
// This is called by all producer threads, but only thread 0 spins on the flag,
// all threads synchronize after thread 0 is done spinning.
#define WAIT_FOR_CHUNK(chunk, subchunk) \
do { \
if (tid == 0) { \
Wait([=] { \
return ((volatile int *)args.ThisChunkDoneFlag)[0] >= \
2*(chunk) + subchunk - 1; \
}); \
} \
BAR(sync, 1, NUM_THREADS); \
} while (0)
__device__ inline void getSliceSizeAndChunkSize(int *sliceSize, int slice,
int numSlices, int numBigSlices, int numSmallSlices, int bigSliceN,
int smallSliceN, int lastSliceN) {
if (slice < numBigSlices) {
*sliceSize = bigSliceN;
} else {
*sliceSize = (slice < numBigSlices + numSmallSlices) ? smallSliceN
: ((slice == numSlices - 1) ? lastSliceN : 0);
}
/* if (threadIdx.x == 0)
printf("[sliceSize=%d] slice=%d numSlices=%d "
"numBigSlices=%d numSmallSlices=%d bigSliceN=%d smallSliceN=%d "
"lastSliceN=%d\n", *sliceSize, slice, numSlices, numBigSlices,
numSmallSlices, bigSliceN, smallSliceN, lastSliceN);
*/
}
template<typename T>
struct ReduceScatterKernelArgs {
// general parameters
int ThisId;
int NumGPUs;
int N;
int * UserFromRing;
// some pre-computed sizes
int SliceSize;
int ChunkSize;
int NumChunks;
int BufferSliceStride;
int BufferMisalignedN;
// local and remote input, output, and buffer
const T * __restrict__ ThisInput;
volatile T * __restrict__ ThisOutput;
volatile T * __restrict__ ThisBuffer;
volatile T * __restrict__ NextBuffer;
// local and remote flags
volatile int * __restrict__ ThisNewDataAvailableFlag;
volatile int * __restrict__ NextNewDataAvailableFlag;
volatile int * __restrict__ ThisChunkDoneFlag;
volatile int * __restrict__ PrevChunkDoneFlag;
};
__device__ inline int GetBlock(const int index, const int step,
const int * const userFromRing, const int numGPUs) {
return userFromRing[(numGPUs + index - 1 - step) % numGPUs];
}
template<int THREADS, int UNROLL, class FUNC, typename T>
__global__ void ReduceScatterKernel(const ReduceScatterKernelArgs<T> args) {
if (args.N == 0) return;
int tid = threadIdx.x;
for (int chunk = 0; chunk < args.NumChunks; ++chunk) {
// calculate slice size. for all chunks except (possibly) the last one,
// this will just be args.SliceSize. For the last one, it may be smaller
int bigSliceN = args.SliceSize;
int smallSliceN = 0;
int lastSliceN = 0;
int numSlices = NUM_SUBCHUNKS;
int numBigSlices = numSlices;
int numSmallSlices = 0;
// last chunk
if ((chunk + 1 == args.NumChunks) && (args.N % args.ChunkSize > 0))
CalcLastChunk<THREADS, UNROLL, T>(&bigSliceN, &smallSliceN, &lastSliceN,
&numSlices, &numBigSlices, &numSmallSlices, args.N, args.NumChunks,
args.ChunkSize);
// this offset is only applied to Data pointers, not to Buffer pointers,
// since we only have one buffer per chunk
int chunkOffset = chunk * args.ChunkSize;
// step 0: push data to next GPU
int step = 0;
int block = GetBlock(args.ThisId, step, args.UserFromRing, args.NumGPUs);
int blockOffset = chunkOffset + block * args.N;
int bufferOffset = block * NUM_SUBCHUNKS * args.BufferSliceStride +
((block * args.BufferMisalignedN) % alignof(PackType));
int sliceSize;
if (tid < NUM_THREADS) {
for(int s=0; s<NUM_SUBCHUNKS; ++s) {
getSliceSizeAndChunkSize(&sliceSize, s, numSlices, numBigSlices,
numSmallSlices, bigSliceN, smallSliceN, lastSliceN);
WAIT_FOR_CHUNK(chunk, s);
Copy<UNROLL, THREADS>(
args.NextBuffer + bufferOffset,
args.ThisInput + blockOffset,
sliceSize);
__syncthreads();
bufferOffset += sliceSize;
blockOffset += sliceSize;
}
} else { // Is consumer
for(int s=0; s<NUM_SUBCHUNKS; ++s) {
__syncthreads();
SIGNAL_NEW_DATA_AVAILABLE(chunk, s, step);
}
}
// steps j with 0 < j < k - 1, where k = number of GPUs: reduce and copy to
// next GPU
for (step = 1; step < args.NumGPUs - 1; ++step) {
int block = GetBlock(args.ThisId, step, args.UserFromRing, args.NumGPUs);
int blockOffset = chunkOffset + block * args.N;
int bufferOffset = block * NUM_SUBCHUNKS * args.BufferSliceStride +
((block * args.BufferMisalignedN) % alignof(PackType));
if (tid < NUM_THREADS) {
for(int s=0; s<NUM_SUBCHUNKS; ++s) {
getSliceSizeAndChunkSize(&sliceSize, s, numSlices, numBigSlices,
numSmallSlices, bigSliceN, smallSliceN, lastSliceN);
WAIT_FOR_NEW_DATA(chunk, s, step);
Reduce<UNROLL, THREADS, FUNC>(
args.NextBuffer + bufferOffset,
args.ThisBuffer + bufferOffset,
args.ThisInput + blockOffset,
sliceSize);
__syncthreads();
bufferOffset += sliceSize;
blockOffset += sliceSize;
}
} else {
for(int s=0; s<NUM_SUBCHUNKS; ++s) {
__syncthreads();
SIGNAL_NEW_DATA_AVAILABLE(chunk, s, step);
}
}
}
// step k - 1: reduce this buffer and data, which will produce the final
// result that we store in this data and push to the next GPU
step = args.NumGPUs - 1;
block = GetBlock(args.ThisId, step, args.UserFromRing, args.NumGPUs);
blockOffset = chunkOffset + block * args.N;
bufferOffset = block * NUM_SUBCHUNKS * args.BufferSliceStride +
((block * args.BufferMisalignedN) % alignof(PackType));
if (tid < NUM_THREADS) {
int outputOffset = 0;
for (int s=0; s<NUM_SUBCHUNKS; ++s) {
getSliceSizeAndChunkSize(&sliceSize, s, numSlices, numBigSlices,
numSmallSlices, bigSliceN, smallSliceN, lastSliceN);
WAIT_FOR_NEW_DATA(chunk, s, step);
Reduce<UNROLL, THREADS, FUNC>(
args.ThisOutput + (chunkOffset + outputOffset),
args.ThisBuffer + bufferOffset,
args.ThisInput + blockOffset,
sliceSize);
__syncthreads();
outputOffset += sliceSize;
bufferOffset += sliceSize;
blockOffset += sliceSize;
}
} else {
for (int s=0; s<NUM_SUBCHUNKS; ++s) {
__syncthreads();
SIGNAL_NEW_DATA_AVAILABLE(chunk, s, step);
// signal that chunk is done if this is not the last chunk
if (chunk + 1 < args.NumChunks) {
SIGNAL_CHUNK_DONE(chunk, s);
}
}
}
}
// wait for the last data to be pushed to us
if (tid < NUM_THREADS) {
WAIT_FOR_NEW_DATA(args.NumChunks, NUM_SUBCHUNKS-1, 0);
if (tid == 0) {
args.ThisNewDataAvailableFlag[tid] = 0;
args.ThisChunkDoneFlag[tid] = 0;
}
}
}
template<class FUNC, typename T>
ncclResult_t ncclReduceScatterWithTypeAndFunc(const void* sendbuff,
void* recvbuff, const int recvcount, ncclComm* comm, cudaStream_t stream) {
if (recvcount == 0) {
return ncclSuccess;
}
int index = comm->ncclId;
int blockSizeInBytes = recvcount * sizeof(T);
int misalignedBytes = blockSizeInBytes % alignof(uint64_t);
assert((int)((misalignedBytes / sizeof(T)) * sizeof(T)) == misalignedBytes);
int misalignedN = misalignedBytes / sizeof(T);
assert(misalignedN < (int)(sizeof(uint64_t) / sizeof(T)));
int paddingN = (misalignedN > 0) ? sizeof(uint64_t) / sizeof(T) : 0;
// There is one slice per GPU, so a slice can be at most bufferN / numGPUs,
// where bufferN is the number of elements of type T that fit into the buffer.
// For efficiency, we want the slice size to be a multiple of UNROLL_SIZE
int bufferN = comm->buffSize / sizeof(T);
// we only need buffer for k slices and k*k paddings (we need k paddings per
// block and we have k blocks)
int bufferNPerSlice = (bufferN - NUM_SUBCHUNKS * comm->nDev * paddingN) /
(NUM_SUBCHUNKS * comm->nDev);
int sliceSize = (bufferNPerSlice / UNROLL_SIZE) * UNROLL_SIZE;
int nextId = (index + 1) % comm->nDev;
int prevId = (index + comm->nDev - 1) % comm->nDev;
ReduceScatterKernelArgs<T> args;
args.ThisId = index;
args.NumGPUs = comm->nDev;
args.N = recvcount;
/* Block j must end up in recvbuff[j], which lives on device with logical
* index comm->ringFromUser[j]. But the block ordering does not necessarily
* follow the ring ordering. Hence the order in which a particular GPU
* processes the different blocks (the correspondence between the step in
* the reduction algorithm and the block on which a GPU operates in that
* particular step) is not the same as the ring order.
*
* Say we have 4 GPUs and comm->userFromRing = { 1, 2, 0, 3 }. Then there are 4
* step in the reduction algorithm and block 0 needs to end up device 2,
* block 1 on device 0, block 2 on device 1, and block 3 needs to end up on
* device 3. In the last step of the algorithm, each GPU must be processing
* the block that will end up on that GPU. The blocks that a GPU has to
* process in the previous steps is determined by the next step because each
* GPU only hands off data to the next GPU in the ring.
*
* In the above example, we get the following table of which block is
* processed by each GPU in a given step. The columns correspond to the
* different GPUs while the rows are the steps in the algorithm.
*
* GPU 0 1 2 3
* step
* 0 3 1 2 0
* 1 0 3 1 2
* 2 2 0 3 1
* 3 1 2 0 3
*
* We note the the rows in the above table are just comm->userFromRing in the last
* step and the list is cyclicly permuted to the left for each previous
* step. The columns, which are what the individual GPUs need to know, are
* comm->userFromRing traversed backwards and starting at index k-1 for GPU k.
* These columns are what we put into args.BlockVsStep to tell the GPU which
* block it needs to be processing at a particular step. */
args.UserFromRing = comm->devUserFromRing;
args.SliceSize = sliceSize;
args.ChunkSize = NUM_SUBCHUNKS * args.SliceSize;
// don't reduce this if we cut the slice size in half below, because if that
// happens, the last chunk will be larger than the other chunks, and we will
// need the extra buffer space
args.BufferSliceStride = args.SliceSize + paddingN;
args.BufferMisalignedN = misalignedN;
// avoid a case where we have one or more big chunks and one tiny one
int remainder = args.N % args.ChunkSize;
if ((args.N > args.ChunkSize) && (remainder > 0) &&
(args.N < 5 * args.ChunkSize) && (2 * remainder < args.ChunkSize)) {
args.SliceSize /= 2;
args.ChunkSize = NUM_SUBCHUNKS * args.SliceSize;
// round down so we end up with a big last chunk
args.NumChunks = args.N / args.ChunkSize;
} else {
// round up
args.NumChunks = (args.N + args.ChunkSize - 1) / args.ChunkSize;
}
// printf("sliceSize = %i, chunkSize = %i, numChunks = %i, sliceStride = %i, misalignedN = %i\n", args.SliceSize, args.ChunkSize, args.NumChunks, args.BufferSliceStride, args.BufferMisalignedN);
args.ThisInput = (const T*)sendbuff;
args.ThisOutput = (volatile T*)recvbuff;
args.ThisBuffer = (volatile T*)comm->local[prevId]->buff;
args.NextBuffer = (volatile T*)comm->remote[nextId]->buff;
// we need 2 * NUM_SUBCHUNKS flags, so use the first NUM_SUBCHUNKS flags
// to signal the next GPU that new data is available and the following
// NUM_SUBCHUNKS to signal the previous GPU that a chunk is finished
args.ThisNewDataAvailableFlag = comm->local[prevId]->flags;
args.NextNewDataAvailableFlag = comm->remote[nextId]->flags;
args.ThisChunkDoneFlag = comm->local[nextId]->flags + 1;
args.PrevChunkDoneFlag = comm->remote[prevId]->flags + 1;
ReduceScatterKernel<NUM_THREADS, UNROLL_COUNT, FUNC, T>
<<<1, NUM_THREADS + NUM_SUBCHUNKS * WARP_SIZE, 0, stream>>>(args);
return ncclSuccess;
}
template<typename T>
ncclResult_t ncclReduceScatterWithType(const void* sendbuff, void* recvbuff,
int recvcount, ncclRedOp_t op, ncclComm* comm, cudaStream_t stream) {
switch (op) {
case ncclSum:
return ncclReduceScatterWithTypeAndFunc<FuncSum<T>, T>(
sendbuff, recvbuff, recvcount, comm, stream);
case ncclProd:
return ncclReduceScatterWithTypeAndFunc<FuncProd<T>, T>(
sendbuff, recvbuff, recvcount, comm, stream);
case ncclMax:
return ncclReduceScatterWithTypeAndFunc<FuncMax<T>, T>(
sendbuff, recvbuff, recvcount, comm, stream);
case ncclMin:
return ncclReduceScatterWithTypeAndFunc<FuncMin<T>, T>(
sendbuff, recvbuff, recvcount, comm, stream);
}
return ncclInvalidOperation;
}
class ReduceScatterFunctor {
public:
ncclResult_t operator()(const void* sendbuff, void* recvbuff,
int recvcount, ncclDataType_t datatype, ncclRedOp_t op, int /*root*/,
ncclComm* comm, cudaStream_t stream) {
switch (datatype) {
case ncclChar:
return ncclReduceScatterWithType<char>(sendbuff, recvbuff, recvcount,
op, comm, stream);
case ncclInt:
return ncclReduceScatterWithType<int>(sendbuff, recvbuff, recvcount,
op, comm, stream);
#ifdef CUDA_HAS_HALF
case ncclHalf:
return ncclReduceScatterWithType<half>(sendbuff, recvbuff, recvcount,
op, comm, stream);
#endif
case ncclFloat:
return ncclReduceScatterWithType<float>(sendbuff, recvbuff, recvcount,
op, comm, stream);
case ncclDouble:
return ncclReduceScatterWithType<double>(sendbuff, recvbuff, recvcount,
op, comm, stream);
}
return ncclInvalidType;
}
};
extern "C" DSOGLOBAL
ncclResult_t ncclReduceScatter(const void* sendbuff, void* recvbuff,
int recvcount, ncclDataType_t datatype, ncclRedOp_t op, ncclComm* comm,
cudaStream_t stream) {
return enqueue(ReduceScatterFunctor(), sendbuff, recvbuff, recvcount,
datatype, op, 0, comm, stream);
}

283
src/reduce_scatter_test.cu Normal file
View File

@ -0,0 +1,283 @@
/*************************************************************************
* Copyright (c) 2015, NVIDIA CORPORATION. All rights reserved.
*
* Redistribution and use in source and binary forms, with or without
* modification, are permitted provided that the following conditions
* are met:
* * Redistributions of source code must retain the above copyright
* notice, this list of conditions and the following disclaimer.
* * Redistributions in binary form must reproduce the above copyright
* notice, this list of conditions and the following disclaimer in the
* documentation and/or other materials provided with the distribution.
* * Neither the name of NVIDIA CORPORATION nor the names of its
* contributors may be used to endorse or promote products derived
* from this software without specific prior written permission.
*
* THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS ``AS IS'' AND ANY
* EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
* IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR
* PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR
* CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL,
* EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO,
* PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR
* PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY
* OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
* (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
* OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
************************************************************************/
#include <chrono>
#include <cstdio>
#include <cstdlib>
#include <string>
#include <vector>
#include "nccl.h"
#include "test_utilities.h"
template<typename T>
void RunTest(T** sendbuff, T** recvbuff, const int N, const ncclDataType_t type,
const ncclRedOp_t op, ncclComm_t* const comms, const std::vector<int>& dList) {
// initialize data
int nDev = 0;
ncclCommCount(comms[0], &nDev);
cudaStream_t* s = (cudaStream_t*)malloc(sizeof(cudaStream_t)*nDev);
T* buffer = (T*)malloc(N * nDev * sizeof(T));
T* result = (T*)malloc(N * nDev * sizeof(T));
memset(buffer, 0, N * nDev * sizeof(T));
memset(result, 0, N * nDev * sizeof(T));
for (int i = 0; i < nDev; ++i) {
CUDACHECK(cudaSetDevice(dList[i]));
CUDACHECK(cudaStreamCreate(s+i));
CUDACHECK(cudaMemset(recvbuff[i], 0, N * sizeof(T)));
Randomize(sendbuff[i], N * nDev, i);
if (i == 0) {
CUDACHECK(cudaMemcpy(result, sendbuff[i], N * nDev * sizeof(T),
cudaMemcpyDeviceToHost));
} else {
Accumulate<T>(result, sendbuff[i], N * nDev, op);
}
}
// warm up GPU
for (int i = 0; i < nDev; ++i) {
CUDACHECK(cudaSetDevice(dList[i]));
ncclReduceScatter((const void*)sendbuff[i], (void*)recvbuff[i],
std::min(N, 1024 * 1024), type, op, comms[i], s[i]);
}
for (int i = 0; i < nDev; ++i) {
CUDACHECK(cudaSetDevice(dList[i]));
CUDACHECK(cudaStreamSynchronize(s[i]));
}
// for (int n = 0; n <= N; n = (n > 0) ? n << 1 : 1)
{
int n = N;
printf("%12i %12i %6s %6s", (int)(n * sizeof(T)), n,
TypeName(type).c_str(), OperationName(op).c_str());
// do out-of-place reduction first
auto start = std::chrono::high_resolution_clock::now();
for (int i = 0; i < nDev; ++i) {
CUDACHECK(cudaSetDevice(dList[i]));
ncclReduceScatter((const void*)sendbuff[i], (void*)recvbuff[i], n, type,
op, comms[i], s[i]);
}
for (int i = 0; i < nDev; ++i) {
CUDACHECK(cudaSetDevice(dList[i]));
CUDACHECK(cudaStreamSynchronize(s[i]));
}
auto stop = std::chrono::high_resolution_clock::now();
double elapsedSec =
std::chrono::duration_cast<std::chrono::duration<double>>(
stop - start).count();
double algbw = (double)(n * sizeof(T)) / 1.0E9 / elapsedSec;
double busbw = algbw * (double)(nDev - 1);
double maxDelta = 0.0;
for (int i = 0; i < nDev; ++i) {
CUDACHECK(cudaSetDevice(dList[i]));
double tmpDelta = CheckDelta<T>(recvbuff[i], result+i*n, n);
maxDelta = std::max(tmpDelta, maxDelta);
}
printf(" %7.3f %5.2f %5.2f %7.0le", elapsedSec * 1.0E3, algbw, busbw,
maxDelta);
}
{
// now do in-place reduction
int n = N;
auto start = std::chrono::high_resolution_clock::now();
for (int i = 0; i < nDev; ++i) {
CUDACHECK(cudaSetDevice(dList[i]));
ncclReduceScatter((const void*)sendbuff[i], (void*)sendbuff[i], n, type,
op, comms[i], s[i]);
}
for (int i = 0; i < nDev; ++i) {
CUDACHECK(cudaSetDevice(dList[i]));
CUDACHECK(cudaStreamSynchronize(s[i]));
}
auto stop = std::chrono::high_resolution_clock::now();
double elapsedSec =
std::chrono::duration_cast<std::chrono::duration<double>>(
stop - start).count();
double algbw = (double)(n * sizeof(T)) / 1.0E9 / elapsedSec;
double busbw = algbw * (double)(nDev - 1);
double maxDelta = 0.0;
for (int i = 0; i < nDev; ++i) {
CUDACHECK(cudaSetDevice(dList[i]));
double tmpDelta = CheckDelta<T>(sendbuff[i], result+i*n, n);
maxDelta = std::max(tmpDelta, maxDelta);
}
printf(" %7.3f %5.2f %5.2f %7.0le\n", elapsedSec * 1.0E3, algbw, busbw,
maxDelta);
}
for (int i = 0; i < nDev; ++i) {
CUDACHECK(cudaSetDevice(dList[i]));
CUDACHECK(cudaStreamDestroy(s[i]));
}
free(s);
free(buffer);
free(result);
}
template<typename T>
void RunTests(const int N, const ncclDataType_t type, ncclComm_t* const comms,
const std::vector<int>& dList) {
int nDev = 0;
ncclCommCount(comms[0], &nDev);
T** sendbuff = (T**)malloc(nDev * sizeof(T*));
T** recvbuff = (T**)malloc(nDev * sizeof(T*));
for (int i = 0; i < nDev; ++i) {
CUDACHECK(cudaSetDevice(dList[i]));
CUDACHECK(cudaMalloc(sendbuff + i, N * nDev * sizeof(T)));
CUDACHECK(cudaMalloc(recvbuff + i, N * sizeof(T)));
}
for (ncclRedOp_t op : { ncclSum, ncclProd, ncclMax, ncclMin }) {
// for (ncclRedOp_t op : { ncclSum }) {
RunTest<T>(sendbuff, recvbuff, N, type, op, comms, dList);
}
for (int i = 0; i < nDev; ++i) {
CUDACHECK(cudaSetDevice(dList[i]));
CUDACHECK(cudaFree(sendbuff[i]));
CUDACHECK(cudaFree(recvbuff[i]));
}
free(sendbuff);
free(recvbuff);
}
void usage() {
printf("Tests nccl ReduceScatter with user supplied arguments.\n"
" Usage: all_reduce_test <data size in bytes> [number of GPUs] "
"[GPU 0] [GPU 1] ...\n\n");
}
int main(int argc, char* argv[]) {
int nVis = 0;
CUDACHECK(cudaGetDeviceCount(&nVis));
int N = 0;
if (argc > 1) {
int t = sscanf(argv[1], "%d", &N);
if (t == 0) {
printf("Error: %s is not an integer!\n\n", argv[1]);
usage();
exit(EXIT_FAILURE);
}
} else {
printf("Error: must specify at least data size in bytes!\n\n");
usage();
exit(EXIT_FAILURE);
}
int nDev = nVis;
if (argc > 2) {
int t = sscanf(argv[2], "%d", &nDev);
if (t == 0) {
printf("Error: %s is not an integer!\n\n", argv[1]);
usage();
exit(EXIT_FAILURE);
}
}
std::vector<int> dList(nDev);
for (int i = 0; i < nDev; ++i)
dList[i] = i % nVis;
if (argc > 3) {
if (argc - 3 != nDev) {
printf("Error: insufficient number of GPUs in list\n\n");
usage();
exit(EXIT_FAILURE);
}
for (int i = 0; i < nDev; ++i) {
int t = sscanf(argv[3 + i], "%d", dList.data() + i);
if (t == 0) {
printf("Error: %s is not an integer!\n\n", argv[2 + i]);
usage();
exit(EXIT_FAILURE);
}
}
}
ncclComm_t* comms = (ncclComm_t*)malloc(sizeof(ncclComm_t)*nDev);
ncclCommInitAll(comms, nDev, dList.data());
printf("# Using devices\n");
for (int g = 0; g < nDev; ++g) {
int cudaDev;
int rank;
cudaDeviceProp prop;
ncclCommCuDevice(comms[g], &cudaDev);
ncclCommUserRank(comms[g], &rank);
CUDACHECK(cudaGetDeviceProperties(&prop, cudaDev));
printf("# Rank %2d uses device %2d [0x%02x] %s\n", rank, cudaDev,
prop.pciBusID, prop.name);
}
printf("\n");
printf("# %10s %12s %6s %6s out-of-place "
"in-place\n", "", "", "", "");
printf("# %10s %12s %6s %6s %7s %5s %5s %7s %7s %5s %5s %7s\n",
"bytes", "N", "type", "op", "time", "algbw", "busbw", "delta", "time",
"algbw", "busbw", "delta");
RunTests<char>(N / sizeof(char), ncclChar, comms, dList);
RunTests<int>(N / sizeof(int), ncclInt, comms, dList);
#ifdef CUDA_HAS_HALF
RunTests<half>(N / sizeof(half), ncclHalf, comms, dList);
#endif
RunTests<float>(N / sizeof(float), ncclFloat, comms, dList);
RunTests<double>(N / sizeof(double), ncclDouble, comms, dList);
printf("\n");
for(int i=0; i<nDev; ++i)
ncclCommDestroy(comms[i]);
free(comms);
exit(EXIT_SUCCESS);
}

297
src/reduce_test.cu Normal file
View File

@ -0,0 +1,297 @@
/*************************************************************************
* Copyright (c) 2015, NVIDIA CORPORATION. All rights reserved.
*
* Redistribution and use in source and binary forms, with or without
* modification, are permitted provided that the following conditions
* are met:
* * Redistributions of source code must retain the above copyright
* notice, this list of conditions and the following disclaimer.
* * Redistributions in binary form must reproduce the above copyright
* notice, this list of conditions and the following disclaimer in the
* documentation and/or other materials provided with the distribution.
* * Neither the name of NVIDIA CORPORATION nor the names of its
* contributors may be used to endorse or promote products derived
* from this software without specific prior written permission.
*
* THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS ``AS IS'' AND ANY
* EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
* IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR
* PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR
* CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL,
* EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO,
* PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR
* PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY
* OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
* (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
* OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
************************************************************************/
#include <chrono>
#include <cstdio>
#include <cstdlib>
#include <string>
#include <vector>
#include "nccl.h"
#include "test_utilities.h"
#include <nvToolsExt.h>
int csv = false;
template<typename T>
void RunTest(T** sendbuff, T** recvbuff, const int N, const ncclDataType_t type,
const ncclRedOp_t op, int root, ncclComm_t* const comms,
const std::vector<int>& dList) {
// initialize data
T* buffer = (T*)malloc(N * sizeof(T));
T* result = (T*)malloc(N * sizeof(T));
memset(buffer, 0, N * sizeof(T));
memset(result, 0, N * sizeof(T));
int nDev = 0;
ncclCommCount(comms[0], &nDev);
cudaStream_t* s = (cudaStream_t*)malloc(sizeof(cudaStream_t)*nDev);
for (int i = 0; i < nDev; ++i) {
CUDACHECK(cudaSetDevice(dList[i]));
CUDACHECK(cudaStreamCreate(s+i));
CUDACHECK(cudaMemset(recvbuff[i], 0, N * sizeof(T)));
Randomize(sendbuff[i], N, i);
if(i == 0) {
CUDACHECK(cudaMemcpy(result, sendbuff[i], N*sizeof(T), cudaMemcpyDeviceToHost));
} else {
Accumulate<T>(result, sendbuff[i], N, op);
}
}
// warm up GPU
for (int i = 0; i < nDev; ++i) {
CUDACHECK(cudaSetDevice(dList[i]));
ncclReduce((const void*)sendbuff[i], (void*)recvbuff[i], std::min(N, 1024 * 1024),
type, op, root, comms[i], s[i]);
}
for (int i = 0; i < nDev; ++i) {
CUDACHECK(cudaSetDevice(dList[i]));
CUDACHECK(cudaStreamSynchronize(s[i]));
}
// for (int n = 0; n <= N; n = (n > 0) ? n << 1 : 1)
{
int n = N;
printf((csv) ? "%i,%i,%s,%s,%d," : "%12i %12i %6s %6s %4d",
(int) (n * sizeof(T)), n, TypeName(type).c_str(),
OperationName(op).c_str(), root);
// do out-of-place reduction first
nvtxRangePushA("out of place");
auto start = std::chrono::high_resolution_clock::now();
//for (int i=0; i<100; i++) {
for (int i = 0; i < nDev; ++i) {
CUDACHECK(cudaSetDevice(dList[i]));
ncclReduce((const void*)sendbuff[i], (void*)recvbuff[i], n, type, op,
root, comms[i], s[i]);
}
//}
for (int i = 0; i < nDev; ++i) {
CUDACHECK(cudaSetDevice(dList[i]));
CUDACHECK(cudaStreamSynchronize(s[i]));
}
auto stop = std::chrono::high_resolution_clock::now();
nvtxRangePop();
nvtxRangePushA("out of place bookkeeping");
double elapsedSec =
std::chrono::duration_cast<std::chrono::duration<double>>(
stop - start).count(); // / 100.0;
double algbw = (double)(n * sizeof(T)) / 1.0E9 / elapsedSec;
double busbw = algbw;
CUDACHECK(cudaSetDevice(dList[root]));
double maxDelta = CheckDelta<T>(recvbuff[root], result, N);
printf((csv)?"%f,%f,%f,%le,":" %7.3f %5.2f %5.2f %7.0le",
elapsedSec * 1.0E3, algbw, busbw, maxDelta);
nvtxRangePop();
}
// for (int n = 0; n <= N; n = (n > 0) ? n << 1 : 1)
{
int n = N;
// now do in-place reduction
nvtxRangePushA("in place");
auto start = std::chrono::high_resolution_clock::now();
//for (int i=0; i<100; i++) {
for (int i = 0; i < nDev; ++i) {
CUDACHECK(cudaSetDevice(dList[i]));
ncclReduce((const void*)sendbuff[i], (void*)sendbuff[i], n, type, op,
root, comms[i], s[i]);
}
//}
for (int i = 0; i < nDev; ++i) {
CUDACHECK(cudaSetDevice(dList[i]));
CUDACHECK(cudaStreamSynchronize(s[i]));
}
auto stop = std::chrono::high_resolution_clock::now();
nvtxRangePop();
nvtxRangePushA("in place bookkeeping");
double elapsedSec =
std::chrono::duration_cast<std::chrono::duration<double>>(
stop - start).count(); // / 100.0;
double algbw = (double)(n * sizeof(T)) / 1.0E9 / elapsedSec;
double busbw = algbw;
CUDACHECK(cudaSetDevice(dList[root]));
double maxDelta = CheckDelta<T>(sendbuff[root], result, N);
printf((csv)?"%f,%f,%f,%le,":" %7.3f %5.2f %5.2f %7.0le\n",
elapsedSec * 1.0E3, algbw, busbw, maxDelta);
nvtxRangePop();
}
for (int i = 0; i < nDev; ++i) {
CUDACHECK(cudaSetDevice(dList[i]));
CUDACHECK(cudaStreamDestroy(s[i]));
}
free(s);
free(buffer);
free(result);
}
template<typename T>
void RunTests(const int N, const ncclDataType_t type, ncclComm_t* const comms,
const std::vector<int>& dList) {
int nDev = 0;
ncclCommCount(comms[0], &nDev);
T** sendbuff = (T**)malloc(nDev * sizeof(T*));
T** recvbuff = (T**)malloc(nDev * sizeof(T*));
for (int i = 0; i < nDev; ++i) {
CUDACHECK(cudaSetDevice(dList[i]));
CUDACHECK(cudaMalloc(sendbuff + i, N * sizeof(T)));
CUDACHECK(cudaMalloc(recvbuff + i, N * sizeof(T)));
}
for (ncclRedOp_t op : { ncclSum, ncclProd, ncclMax, ncclMin }) {
// for (ncclRedOp_t op : { ncclSum }) {
for(int root=0; root<nDev; ++root) {
RunTest<T>(sendbuff, recvbuff, N, type, op, root, comms, dList);
}
}
for (int i = 0; i < nDev; ++i) {
CUDACHECK(cudaSetDevice(dList[i]));
CUDACHECK(cudaFree(sendbuff[i]));
CUDACHECK(cudaFree(recvbuff[i]));
}
free(sendbuff);
free(recvbuff);
}
void usage() {
printf("Tests nccl Reduce with user supplied arguments.\n"
" Usage: reduce_test <data size in bytes> [number of GPUs] "
"[GPU 0] [GPU 1] ...\n\n");
}
int main(int argc, char* argv[]) {
int nVis = 0;
CUDACHECK(cudaGetDeviceCount(&nVis));
int N = 0;
if (argc > 1) {
int t = sscanf(argv[1], "%d", &N);
if (t == 0) {
printf("Error: %s is not an integer!\n\n", argv[1]);
usage();
exit(EXIT_FAILURE);
}
} else {
printf("Error: must specify at least data size in bytes!\n\n");
usage();
exit(EXIT_FAILURE);
}
int nDev = nVis;
if (argc > 2) {
int t = sscanf(argv[2], "%d", &nDev);
if (t == 0) {
printf("Error: %s is not an integer!\n\n", argv[1]);
usage();
exit(EXIT_FAILURE);
}
}
std::vector<int> dList(nDev);
for (int i = 0; i < nDev; ++i)
dList[i] = i % nVis;
if (argc > 3) {
if (argc - 3 != nDev) {
printf("Error: insufficient number of GPUs in list\n\n");
usage();
exit(EXIT_FAILURE);
}
for (int i = 0; i < nDev; ++i) {
int t = sscanf(argv[3 + i], "%d", dList.data() + i);
if (t == 0) {
printf("Error: %s is not an integer!\n\n", argv[2 + i]);
usage();
exit(EXIT_FAILURE);
}
}
}
ncclComm_t* comms = (ncclComm_t*)malloc(sizeof(ncclComm_t)*nDev);
ncclCommInitAll(comms, nDev, dList.data());
if (!csv) {
printf("# Using devices\n");
for (int g = 0; g < nDev; ++g) {
int cudaDev;
int rank;
cudaDeviceProp prop;
ncclCommCuDevice(comms[g], &cudaDev);
ncclCommUserRank(comms[g], &rank);
CUDACHECK(cudaGetDeviceProperties(&prop, cudaDev));
printf("# Rank %2d uses device %2d [0x%02x] %s\n", rank, cudaDev,
prop.pciBusID, prop.name);
}
printf("\n");
printf("# %10s %12s %6s %6s %4s out-of-place in-place\n", "", "", "", "", "");
printf("# %10s %12s %6s %6s %4s %7s %5s %5s %7s %7s %5s %5s %7s\n",
"bytes", "N", "type", "op", "root",
"time", "algbw", "busbw", "res", "time", "algbw", "busbw", "res");
}
else {
printf("B,N,type,op,root,oop_time,oop_algbw,oop_busbw,oop_res,ip_time,ip_algbw,ip_busbw,ip_res\n");
}
RunTests<char>(N / sizeof(char), ncclChar, comms, dList);
RunTests<int>(N / sizeof(int), ncclInt, comms, dList);
#ifdef CUDA_HAS_HALF
RunTests<half>(N / sizeof(half), ncclHalf, comms, dList);
#endif
RunTests<float>(N / sizeof(float), ncclFloat, comms, dList);
RunTests<double>(N / sizeof(double), ncclDouble, comms, dList);
printf("\n");
for(int i = 0; i < nDev; ++i)
ncclCommDestroy(comms[i]);
free(comms);
exit(EXIT_SUCCESS);
}

286
src/test_utilities.h Normal file
View File

@ -0,0 +1,286 @@
/*************************************************************************
* Copyright (c) 2015, NVIDIA CORPORATION. All rights reserved.
*
* Redistribution and use in source and binary forms, with or without
* modification, are permitted provided that the following conditions
* are met:
* * Redistributions of source code must retain the above copyright
* notice, this list of conditions and the following disclaimer.
* * Redistributions in binary form must reproduce the above copyright
* notice, this list of conditions and the following disclaimer in the
* documentation and/or other materials provided with the distribution.
* * Neither the name of NVIDIA CORPORATION nor the names of its
* contributors may be used to endorse or promote products derived
* from this software without specific prior written permission.
*
* THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS ``AS IS'' AND ANY
* EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
* IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR
* PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR
* CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL,
* EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO,
* PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR
* PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY
* OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
* (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
* OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
************************************************************************/
#ifndef SRC_TEST_UTILITIES_H_
#define SRC_TEST_UTILITIES_H_
#include <curand.h>
#define CUDACHECK(cmd) do { \
cudaError_t e = cmd; \
if( e != cudaSuccess ) { \
printf("Cuda failure %s:%d '%s'\n", \
__FILE__,__LINE__,cudaGetErrorString(e)); \
exit(EXIT_FAILURE); \
} \
} while(false)
template<typename T>
void Randomize(T* const dest, const int N, const int randomSeed);
template<typename T>
void Accumulate(T* dest, const T* contrib, int N, ncclRedOp_t op);
template<typename T>
double CheckDelta(const T* results, const T* expected, int N);
#define CURAND_CHK(cmd) \
do { \
curandStatus_t error = (cmd); \
if (error != CURAND_STATUS_SUCCESS) { \
printf("CuRAND error %i at %s:%i\n", error, __FILE__ , __LINE__); \
exit(EXIT_FAILURE); \
} \
} while (false)
template<typename T>
void GenerateRandom(curandGenerator_t generator, T * const dest,
const int N);
template<>
void GenerateRandom<char>(curandGenerator_t generator, char * const dest,
const int N) {
CURAND_CHK(curandGenerate(generator, (unsigned int*)dest,
N * sizeof(char) / sizeof(int)));
}
template<>
void GenerateRandom<int>(curandGenerator_t generator, int * const dest,
const int N) {
CURAND_CHK(curandGenerate(generator, (unsigned int*)dest, N));
}
template<>
void GenerateRandom<float>(curandGenerator_t generator, float * const dest,
const int N) {
CURAND_CHK(curandGenerateUniform(generator, dest, N));
}
template<>
void GenerateRandom<double>(curandGenerator_t generator, double * const dest,
const int N) {
CURAND_CHK(curandGenerateUniformDouble(generator, dest, N));
}
template<typename T>
void Randomize(T* const dest, const int N, const int randomSeed) {
curandGenerator_t gen;
CURAND_CHK(curandCreateGenerator(&gen, CURAND_RNG_PSEUDO_MT19937));
CURAND_CHK(curandSetPseudoRandomGeneratorSeed(gen, randomSeed));
GenerateRandom<T>(gen, dest, N);
CURAND_CHK(curandDestroyGenerator(gen));
CUDACHECK(cudaDeviceSynchronize());
}
#ifdef CUDA_HAS_HALF
__global__ void halve(const float * src, half* dest, int N) {
for(int tid = threadIdx.x + blockIdx.x*blockDim.x;
tid < N; tid += blockDim.x * gridDim.x)
dest[tid] = __float2half(src[tid]);
}
template<>
void Randomize<half>(half* const dest, const int N, const int randomSeed) {
curandGenerator_t gen;
CURAND_CHK(curandCreateGenerator(&gen, CURAND_RNG_PSEUDO_MT19937));
CURAND_CHK(curandSetPseudoRandomGeneratorSeed(gen, randomSeed));
float* temp;
CUDACHECK(cudaMalloc(&temp, N*sizeof(float)));
GenerateRandom<float>(gen, temp, N);
halve<<<128, 512>>>(temp, dest, N);
CURAND_CHK(curandDestroyGenerator(gen));
CUDACHECK(cudaFree(temp));
CUDACHECK(cudaDeviceSynchronize());
}
#endif
template<typename T, int OP> __global__ static
void accumKern(T* acum, const T* contrib, int N) {
int tid = threadIdx.x + blockIdx.x*blockDim.x;
int offset = blockDim.x*gridDim.x;
for(int i=tid; i<N; i+=offset) {
T c = contrib[i];
T a = acum[i];
if(OP == ncclSum) {
acum[i] = a+c;
} else if(OP == ncclProd) {
acum[i] = a*c;
} else if(OP == ncclMax) {
acum[i] = (a > c) ? a : c;
} else if(OP == ncclMin) {
acum[i] = (a < c) ? a : c;
}
}
}
#ifdef CUDA_HAS_HALF
template<> __global__
void accumKern<half, ncclSum>(half* acum, const half* contrib, int N) {
int tid = threadIdx.x + blockIdx.x*blockDim.x;
int offset = blockDim.x*gridDim.x;
for(int i=tid; i<N; i+=offset) {
float c = __half2float(contrib[i]);
float a = __half2float(acum[i]);
acum[i] = __float2half( a + c );
}
}
template<> __global__
void accumKern<half, ncclProd>(half* acum, const half* contrib, int N) {
int tid = threadIdx.x + blockIdx.x*blockDim.x;
int offset = blockDim.x*gridDim.x;
for(int i=tid; i<N; i+=offset) {
float c = __half2float(contrib[i]);
float a = __half2float(acum[i]);
acum[i] = __float2half( a * c );
}
}
template<> __global__
void accumKern<half, ncclMax>(half* acum, const half* contrib, int N) {
int tid = threadIdx.x + blockIdx.x*blockDim.x;
int offset = blockDim.x*gridDim.x;
for(int i=tid; i<N; i+=offset) {
float c = __half2float(contrib[i]);
float a = __half2float(acum[i]);
acum[i] = __float2half( (a>c) ? a : c );
}
}
template<> __global__
void accumKern<half, ncclMin>(half* acum, const half* contrib, int N) {
int tid = threadIdx.x + blockIdx.x*blockDim.x;
int offset = blockDim.x*gridDim.x;
for(int i=tid; i<N; i+=offset) {
float c = __half2float(contrib[i]);
float a = __half2float(acum[i]);
acum[i] = __float2half( (a<c) ? a : c );
}
}
#endif
template<typename T>
void Accumulate(T* dest, const T* contrib, int N, ncclRedOp_t op) {
T* devdest;
CUDACHECK(cudaHostRegister(dest, N*sizeof(T), 0));
CUDACHECK(cudaHostGetDevicePointer(&devdest, dest, 0));
switch(op) {
case ncclSum: accumKern<T, ncclSum> <<<256,256>>>(devdest, contrib, N); break;
case ncclProd: accumKern<T, ncclProd><<<256,256>>>(devdest, contrib, N); break;
case ncclMax: accumKern<T, ncclMax> <<<256,256>>>(devdest, contrib, N); break;
case ncclMin: accumKern<T, ncclMin> <<<256,256>>>(devdest, contrib, N); break;
default:
printf("Unknown reduction operation.\n");
exit(EXIT_FAILURE);
}
CUDACHECK(cudaHostUnregister(dest));
}
template<typename T> __device__
double absDiff(T a, T b) {
return fabs((double)(b - a));
}
#ifdef CUDA_HAS_HALF
template<> __device__
double absDiff<half>(half a, half b) {
float x = __half2float(a);
float y = __half2float(b);
return fabs((double)(y-x));
}
#endif
template<typename T, int BSIZE> __global__
void deltaKern(const T* A, const T* B, int N, double* max) {
__shared__ double temp[BSIZE];
int tid = threadIdx.x;
double locmax = 0.0;
for(int i=tid; i<N; i+=blockDim.x) {
double delta = absDiff(A[i], B[i]);
if( delta > locmax )
locmax = delta;
}
temp[tid] = locmax;
for(int stride = BSIZE/2; stride > 1; stride>>=1) {
__syncthreads();
if( tid < stride )
temp[tid] = temp[tid] > temp[tid+stride] ? temp[tid] : temp[tid+stride];
}
__syncthreads();
if( threadIdx.x == 0)
*max = temp[0] > temp[1] ? temp[0] : temp[1];
}
template<typename T>
double CheckDelta(const T* results, const T* expected, int N) {
T* devexp;
double maxerr;
double* devmax;
CUDACHECK(cudaHostRegister((void*)expected, N*sizeof(T), 0));
CUDACHECK(cudaHostGetDevicePointer((void**)&devexp, (void*)expected, 0));
CUDACHECK(cudaHostRegister((void*)&maxerr, sizeof(double), 0));
CUDACHECK(cudaHostGetDevicePointer(&devmax, &maxerr, 0));
deltaKern<T, 512><<<1, 512>>>(results, devexp, N, devmax);
CUDACHECK(cudaHostUnregister(&maxerr));
CUDACHECK(cudaHostUnregister((void*)devexp));
return maxerr;
}
std::string TypeName(const ncclDataType_t type) {
switch (type) {
case ncclChar: return "char";
case ncclInt: return "int";
#ifdef CUDA_HAS_HALF
case ncclHalf: return "half";
#endif
case ncclFloat: return "float";
case ncclDouble: return "double";
default: return "unknown";
}
}
std::string OperationName(const ncclRedOp_t op) {
switch (op) {
case ncclSum: return "sum";
case ncclProd: return "prod";
case ncclMax: return "max";
case ncclMin: return "min";
default: return "unknown";
}
}
#endif // SRC_TEST_UTILITIES_H_