From fd38dbbba56120099bc063634b5d89e7932dee4e Mon Sep 17 00:00:00 2001 From: "Corey J. Nolet" Date: Wed, 6 May 2020 20:30:50 -0400 Subject: [PATCH 01/46] Initial commit of raft comms --- cpp/include/raft/comms.hpp | 0 cpp/include/raft/comms/comms.hpp | 87 ++++ cpp/include/raft/comms/std/std_comms.hpp | 583 ++++++++++++++++++++++ cpp/include/raft/comms/std/ucp_helper.hpp | 240 +++++++++ 4 files changed, 910 insertions(+) create mode 100644 cpp/include/raft/comms.hpp create mode 100644 cpp/include/raft/comms/comms.hpp create mode 100644 cpp/include/raft/comms/std/std_comms.hpp create mode 100644 cpp/include/raft/comms/std/ucp_helper.hpp diff --git a/cpp/include/raft/comms.hpp b/cpp/include/raft/comms.hpp new file mode 100644 index 0000000000..e69de29bb2 diff --git a/cpp/include/raft/comms/comms.hpp b/cpp/include/raft/comms/comms.hpp new file mode 100644 index 0000000000..0ddba4e7b8 --- /dev/null +++ b/cpp/include/raft/comms/comms.hpp @@ -0,0 +1,87 @@ +/* + * Copyright (c) 2019, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#pragma once + +#include + + +namespace raft { + +class comms { + public: + + typedef unsigned int request_t; + enum datatype_t { CHAR, UINT8, INT, UINT, INT64, UINT64, FLOAT, DOUBLE }; + enum op_t { SUM, PROD, MIN, MAX }; + + /** + * The resulting status of distributed stream synchronization + */ + enum status_t { + commStatusSuccess, // Synchronization successful + commStatusError, // An error occured querying sync status + commStatusAbort + }; // A failure occurred in sync, queued operations aborted + + virtual size_t getDatatypeSize(const comms::datatype_t datatype); + + template + virtual datatype_t getDataType() const; + + virtual ~comms(); + + virtual int getSize() const = 0; + virtual int getRank() const = 0; + + virtual std::unique_ptr commSplit(int color, int key) const = 0; + + virtual void barrier() const = 0; + + virtual status_t syncStream(cudaStream_t stream) const = 0; + + virtual void isend(const void* buf, int size, int dest, int tag, + request_t* request) const = 0; + + virtual void irecv(void* buf, int size, int source, int tag, + request_t* request) const = 0; + + virtual void waitall(int count, request_t array_of_requests[]) const = 0; + + virtual void allreduce(const void* sendbuff, void* recvbuff, int count, + datatype_t datatype, op_t op, + cudaStream_t stream) const = 0; + + virtual void bcast(void* buff, int count, datatype_t datatype, int root, + cudaStream_t stream) const = 0; + + virtual void reduce(const void* sendbuff, void* recvbuff, int count, + datatype_t datatype, op_t op, int root, + cudaStream_t stream) const = 0; + + virtual void allgather(const void* sendbuff, void* recvbuff, int sendcount, + datatype_t datatype, cudaStream_t stream) const = 0; + + virtual void allgatherv(const void* sendbuf, void* recvbuf, + const int recvcounts[], const int displs[], + datatype_t datatype, cudaStream_t stream) const = 0; + + virtual void reducescatter(const void* sendbuff, void* recvbuff, + int recvcount, datatype_t datatype, op_t op, + cudaStream_t stream) const = 0; +}; + +} // namespace raft diff --git a/cpp/include/raft/comms/std/std_comms.hpp b/cpp/include/raft/comms/std/std_comms.hpp new file mode 100644 index 0000000000..3ef08a8ce4 --- /dev/null +++ b/cpp/include/raft/comms/std/std_comms.hpp @@ -0,0 +1,583 @@ +/* + * Copyright (c) 2019-2020, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#pragma once + +#include +#include +#include + +#include + +#include + +#include +#include +#include "ucp_helper.hpp" + +#include + +constexpr bool UCX_ENABLED = true; + +#include +#include +#include +#include +#include +#include +#include +#include + +#include + +#include + +#include + + +#define NCCL_CHECK(call) \ + do { \ + ncclResult_t status = call; \ + ASSERT(ncclSuccess == status, "ERROR: NCCL call='%s'. Reason:%s\n", #call, \ + ncclGetErrorString(status)); \ + } while (0) + +#define NCCL_CHECK_NO_THROW(call) \ + do { \ + ncclResult_t status = call; \ + if (status != ncclSuccess) { \ + CUML_LOG_ERROR("NCCL call='%s' failed. Reason:%s\n", #call, \ + ncclGetErrorString(status)); \ + } \ + } while (0) + + + +namespace raft { + +namespace { + +size_t getDatatypeSize(const std_comms::datatype_t datatype) { + switch (datatype) { + case MLCommon::cumlCommunicator::CHAR: + return sizeof(char); + case MLCommon::cumlCommunicator::UINT8: + return sizeof(uint8_t); + case MLCommon::cumlCommunicator::INT: + return sizeof(int); + case MLCommon::cumlCommunicator::UINT: + return sizeof(unsigned int); + case MLCommon::cumlCommunicator::INT64: + return sizeof(int64_t); + case MLCommon::cumlCommunicator::UINT64: + return sizeof(uint64_t); + case MLCommon::cumlCommunicator::FLOAT: + return sizeof(float); + case MLCommon::cumlCommunicator::DOUBLE: + return sizeof(double); + } +} + +ncclDataType_t getNCCLDatatype( + const std_comms::datatype_t datatype) { + switch (datatype) { + case MLCommon::cumlCommunicator::CHAR: + return ncclChar; + case MLCommon::cumlCommunicator::UINT8: + return ncclUint8; + case MLCommon::cumlCommunicator::INT: + return ncclInt; + case MLCommon::cumlCommunicator::UINT: + return ncclUint32; + case MLCommon::cumlCommunicator::INT64: + return ncclInt64; + case MLCommon::cumlCommunicator::UINT64: + return ncclUint64; + case MLCommon::cumlCommunicator::FLOAT: + return ncclFloat; + case MLCommon::cumlCommunicator::DOUBLE: + return ncclDouble; + } +} + +ncclRedOp_t getNCCLOp(const std_comms::op_t op) { + switch (op) { + case MLCommon::cumlCommunicator::SUM: + return ncclSum; + case MLCommon::cumlCommunicator::PROD: + return ncclProd; + case MLCommon::cumlCommunicator::MIN: + return ncclMin; + case MLCommon::cumlCommunicator::MAX: + return ncclMax; + } +} +} // namespace + +bool ucx_enabled() { return UCX_ENABLED; } + +/** + * @brief Underlying comms, like NCCL and UCX, should be initialized and ready for use, + * and maintained, outside of the cuML Comms lifecycle. This allows us to decouple the + * ownership of the actual comms from cuml so that they can also be used outside of cuml. + * + * For instance, nccl-py can be used to bootstrap a ncclComm_t before it is + * used to construct a cuml comms instance. UCX endpoints can be bootstrapped + * in Python using ucx-py, before being used to construct a cuML comms instance. + */ +void inject_comms(cumlHandle &handle, ncclComm_t comm, ucp_worker_h ucp_worker, + std::shared_ptr eps, int size, int rank) { + auto communicator = std::make_shared( + std::unique_ptr( + new std_comms(comm, ucp_worker, eps, size, rank))); + handle.getImpl().setCommunicator(communicator); +} + +void inject_comms(cumlHandle &handle, ncclComm_t comm, int size, int rank) { + auto communicator = std::make_shared( + std::unique_ptr( + new std_comms(comm, size, rank))); + handle.getImpl().setCommunicator(communicator); +} + +void inject_comms_py_coll(cumlHandle *handle, ncclComm_t comm, int size, + int rank) { + inject_comms(*handle, comm, size, rank); +} + +void inject_comms_py(ML::cumlHandle *handle, ncclComm_t comm, void *ucp_worker, + void *eps, int size, int rank) { + std::shared_ptr eps_sp = + std::make_shared(new ucp_ep_h[size]); + + size_t *size_t_ep_arr = (size_t *)eps; + + for (int i = 0; i < size; i++) { + size_t ptr = size_t_ep_arr[i]; + ucp_ep_h *ucp_ep_v = (ucp_ep_h *)*eps_sp; + + if (ptr != 0) { + ucp_ep_h eps_ptr = (ucp_ep_h)size_t_ep_arr[i]; + ucp_ep_v[i] = eps_ptr; + } else { + ucp_ep_v[i] = nullptr; + } + } + + inject_comms(*handle, comm, (ucp_worker_h)ucp_worker, eps_sp, size, rank); +} + + +/** + * @brief A cumlCommunicator implementation capable of running collective communications + * with NCCL and point-to-point-communications with UCX. Note that the latter is optional. + * + * Underlying comms, like NCCL and UCX, should be initialized and ready for use, + * and maintained, outside of the cuML Comms lifecycle. This allows us to decouple the + * ownership of the actual comms from cuml so that they can also be used outside of cuml. + * + * For instance, nccl-py can be used to bootstrap a ncclComm_t before it is + * used to construct a cuml comms instance. UCX endpoints can be bootstrapped + * in Python using ucx-py, before being used to construct a cuML comms instance. + */ +class std_comms : public raft::comms { + public: + std_comms() = delete; + + /** + * @brief Constructor for collective + point-to-point operation. + * @param comm initialized nccl comm + * @param ucp_worker initialized ucp_worker instance + * @param eps shared pointer to array of ucp endpoints + * @param size size of the cluster + * @param rank rank of the current worker + */ + std_comms(ncclComm_t comm, ucp_worker_h ucp_worker, + std::shared_ptr eps, int size, int rank) : _nccl_comm(comm), + _ucp_worker(ucp_worker), + _ucp_eps(eps), + _size(size), + _rank(rank), + _next_request_id(0) { + initialize(); + p2p_enabled = true; + }; + + + /** + * @brief constructor for collective-only operation + * @param comm initilized nccl communicator + * @param size size of the cluster + * @param rank rank of the current worker + */ + std_comms(ncclComm_t comm, int size, int rank) + : _nccl_comm(comm), _size(size), _rank(rank) { + initialize(); + }; + + virtual ~std_comms(){ + CUDA_CHECK_NO_THROW(cudaStreamDestroy(_stream)); + + CUDA_CHECK_NO_THROW(cudaFree(_sendbuff)); + CUDA_CHECK_NO_THROW(cudaFree(_recvbuff)); + } + + size_t getDatatypeSize(const std_comms::datatype_t datatype) { + switch (datatype) { + case MLCommon::cumlCommunicator::CHAR: + return sizeof(char); + case MLCommon::cumlCommunicator::UINT8: + return sizeof(uint8_t); + case MLCommon::cumlCommunicator::INT: + return sizeof(int); + case MLCommon::cumlCommunicator::UINT: + return sizeof(unsigned int); + case MLCommon::cumlCommunicator::INT64: + return sizeof(int64_t); + case MLCommon::cumlCommunicator::UINT64: + return sizeof(uint64_t); + case MLCommon::cumlCommunicator::FLOAT: + return sizeof(float); + case MLCommon::cumlCommunicator::DOUBLE: + return sizeof(double); + } + } + + + template <> + cumlCommunicator::datatype_t getDataType() const { + return cumlCommunicator::CHAR; + } + + template <> + cumlCommunicator::datatype_t getDataType() const { + return cumlCommunicator::UINT8; + } + + template <> + cumlCommunicator::datatype_t getDataType() const { + return cumlCommunicator::INT; + } + + template <> + cumlCommunicator::datatype_t getDataType() const { + return cumlCommunicator::UINT; + } + + template <> + cumlCommunicator::datatype_t getDataType() const { + return cumlCommunicator::INT64; + } + + template <> + cumlCommunicator::datatype_t getDataType() const { + return cumlCommunicator::UINT64; + } + + template <> + cumlCommunicator::datatype_t getDataType() const { + return cumlCommunicator::FLOAT; + } + + template <> + cumlCommunicator::datatype_t getDataType() const { + return cumlCommunicator::DOUBLE; + } + + void initialize() { + CUDA_CHECK(cudaStreamCreate(&_stream)); + + CUDA_CHECK(cudaMalloc(&_sendbuff, sizeof(int))); + CUDA_CHECK(cudaMalloc(&_recvbuff, sizeof(int))); + } + + + int getSize() const { return _size; } + + int getRank() const { return _rank; } + + std::unique_ptr + commSplit(int color, int key) const { + // Not supported by NCCL + ASSERT(false, + "ERROR: commSplit called but not yet supported in this comms " + "implementation."); + } + + void barrier() const { + CUDA_CHECK(cudaMemsetAsync(_sendbuff, 1, sizeof(int), _stream)); + CUDA_CHECK(cudaMemsetAsync(_recvbuff, 1, sizeof(int), _stream)); + + allreduce(_sendbuff, _recvbuff, 1, MLCommon::cumlCommunicator::INT, + MLCommon::cumlCommunicator::SUM, _stream); + + ASSERT(syncStream(_stream) == status_t::commStatusSuccess, + "ERROR: syncStream failed. This can be caused by a failed rank."); + } + + void get_request_id(request_t *req) const { + + request_t req_id; + + if (this->_free_requests.empty()) + req_id = this->_next_request_id++; + else { + auto it = this->_free_requests.begin(); + req_id = *it; + this->_free_requests.erase(it); + } + *req = req_id; + } + + void isend(const void *buf, int size, int dest, + int tag, request_t *request) const { + ASSERT(UCX_ENABLED, "cuML Comms not built with UCX support"); + ASSERT(p2p_enabled, + "cuML Comms instance was not initialized for point-to-point"); + + ASSERT(_ucp_worker != nullptr, + "ERROR: UCX comms not initialized on communicator."); + + get_request_id(request); + ucp_ep_h ep_ptr = (*_ucp_eps)[dest]; + + ucp_request *ucp_req = (ucp_request *)malloc(sizeof(ucp_request)); + + this->_ucp_handler.ucp_isend(ucp_req, ep_ptr, buf, size, tag, + default_tag_mask, getRank()); + + CUML_LOG_DEBUG( + "%d: Created send request [id=%llu], ptr=%llu, to=%llu, ep=%llu", getRank(), + (unsigned long long)*request, (unsigned long long)ucp_req->req, + (unsigned long long)dest, (unsigned long long)ep_ptr); + + _requests_in_flight.insert(std::make_pair(*request, ucp_req)); + } + + void irecv(void *buf, int size, int source, int tag, + request_t *request) const { + ASSERT(UCX_ENABLED, "cuML Comms not built with UCX support"); + ASSERT(p2p_enabled, + "cuML Comms instance was not initialized for point-to-point"); + + ASSERT(_ucp_worker != nullptr, + "ERROR: UCX comms not initialized on communicator."); + + get_request_id(request); + + ucp_ep_h ep_ptr = (*_ucp_eps)[source]; + + ucp_tag_t tag_mask = default_tag_mask; + + if (source == CUML_ANY_SOURCE) { + tag_mask = any_rank_tag_mask; + } + + ucp_request *ucp_req = (ucp_request *)malloc(sizeof(ucp_request)); + _ucp_handler.ucp_irecv(ucp_req, _ucp_worker, ep_ptr, buf, size, tag, tag_mask, + source); + + CUML_LOG_DEBUG( + "%d: Created receive request [id=%llu], ptr=%llu, from=%llu, ep=%llu", + getRank(), (unsigned long long)*request, (unsigned long long)ucp_req->req, + (unsigned long long)source, (unsigned long long)ep_ptr); + + _requests_in_flight.insert(std::make_pair(*request, ucp_req)); + } + + void waitall(int count, + request_t array_of_requests[]) const { + ASSERT(UCX_ENABLED, "cuML Comms not built with UCX support"); + ASSERT(p2p_enabled, + "cuML Comms instance was not initialized for point-to-point"); + + ASSERT(_ucp_worker != nullptr, + "ERROR: UCX comms not initialized on communicator."); + + std::vector requests; + requests.reserve(count); + + time_t start = time(NULL); + + for (int i = 0; i < count; ++i) { + auto req_it = _requests_in_flight.find(array_of_requests[i]); + ASSERT(_requests_in_flight.end() != req_it, + "ERROR: waitall on invalid request: %d", array_of_requests[i]); + requests.push_back(req_it->second); + _free_requests.insert(req_it->first); + _requests_in_flight.erase(req_it); + } + + while (requests.size() > 0) { + time_t now = time(NULL); + + // Timeout if we have not gotten progress or completed any requests + // in 10 or more seconds. + ASSERT(now - start < 10, "Timed out waiting for requests."); + + for (std::vector::iterator it = requests.begin(); + it != requests.end();) { + bool restart = false; // resets the timeout when any progress was made + + // Causes UCP to progress through the send/recv message queue + while (_ucp_handler.ucp_progress(_ucp_worker) != 0) { + restart = true; + } + + auto req = *it; + + // If the message needs release, we know it will be sent/received + // asynchronously, so we will need to track and verify its state + if (req->needs_release) { + ASSERT(UCS_PTR_IS_PTR(req->req), + "UCX Request Error. Request is not valid UCX pointer"); + ASSERT(!UCS_PTR_IS_ERR(req->req), "UCX Request Error: %d\n", + UCS_PTR_STATUS(req->req)); + ASSERT(req->req->completed == 1 || req->req->completed == 0, + "request->completed not a valid value: %d\n", + req->req->completed); + } + + // If a message was sent synchronously (eg. completed before + // `isend`/`irecv` completed) or an asynchronous message + // is complete, we can go ahead and clean it up. + if (!req->needs_release || req->req->completed == 1) { + restart = true; + CUML_LOG_DEBUG( + "%d: request completed. [ptr=%llu, num_left=%lu," + " other_rank=%d, is_send=%d, completed_immediately=%d]", + getRank(), (unsigned long long)req->req, requests.size() - 1, + req->other_rank, req->is_send_request, !req->needs_release); + + // perform cleanup + _ucp_handler.free_ucp_request(req); + + // remove from pending requests + it = requests.erase(it); + } else { + ++it; + } + // if any progress was made, reset the timeout start time + if (restart) { + start = time(NULL); + } + } + } + } + + void allreduce(const void *sendbuff, void *recvbuff, + int count, datatype_t datatype, + op_t op, cudaStream_t stream) const { + NCCL_CHECK(ncclAllReduce(sendbuff, recvbuff, count, getNCCLDatatype(datatype), + getNCCLOp(op), _nccl_comm, stream)); + } + + void bcast(void *buff, int count, datatype_t datatype, + int root, cudaStream_t stream) const { + NCCL_CHECK(ncclBroadcast(buff, buff, count, getNCCLDatatype(datatype), root, + _nccl_comm, stream)); + } + + void reduce(const void *sendbuff, void *recvbuff, + int count, datatype_t datatype, op_t op, + int root, cudaStream_t stream) const { + NCCL_CHECK(ncclReduce(sendbuff, recvbuff, count, getNCCLDatatype(datatype), + getNCCLOp(op), root, _nccl_comm, stream)); + } + + void allgather(const void *sendbuff, void *recvbuff, + int sendcount, datatype_t datatype, + cudaStream_t stream) const { + NCCL_CHECK(ncclAllGather(sendbuff, recvbuff, sendcount, + getNCCLDatatype(datatype), _nccl_comm, stream)); + } + + void allgatherv(const void *sendbuf, void *recvbuf, + const int recvcounts[], + const int displs[], + datatype_t datatype, + cudaStream_t stream) const { + //From: "An Empirical Evaluation of Allgatherv on Multi-GPU Systems" - https://arxiv.org/pdf/1812.05964.pdf + //Listing 1 on page 4. + for (int root = 0; root < _size; ++root) + NCCL_CHECK(ncclBroadcast( + sendbuf, + static_cast(recvbuf) + displs[root] * getDatatypeSize(datatype), + recvcounts[root], getNCCLDatatype(datatype), root, _nccl_comm, stream)); + } + + void reducescatter(const void *sendbuff, + void *recvbuff, int recvcount, + datatype_t datatype, op_t op, + cudaStream_t stream) const { + NCCL_CHECK(ncclReduceScatter(sendbuff, recvbuff, recvcount, + getNCCLDatatype(datatype), getNCCLOp(op), + _nccl_comm, stream)); + } + + status_t std_comms::syncStream( + cudaStream_t stream) const { + cudaError_t cudaErr; + ncclResult_t ncclErr, ncclAsyncErr; + while (1) { + cudaErr = cudaStreamQuery(stream); + if (cudaErr == cudaSuccess) return status_t::commStatusSuccess; + + if (cudaErr != cudaErrorNotReady) { + // An error occurred querying the status of the stream + return status_t::commStatusError; + } + + ncclErr = ncclCommGetAsyncError(_nccl_comm, &ncclAsyncErr); + if (ncclErr != ncclSuccess) { + // An error occurred retrieving the asynchronous error + return status_t::commStatusError; + } + + if (ncclAsyncErr != ncclSuccess) { + // An asynchronous error happened. Stop the operation and destroy + // the communicator + ncclErr = ncclCommAbort(_nccl_comm); + if (ncclErr != ncclSuccess) + // Caller may abort with an exception or try to re-create a new communicator. + return status_t::commStatusAbort; + } + + // Let other threads (including NCCL threads) use the CPU. + pthread_yield(); + } + } + + private: + ncclComm_t _nccl_comm; + cudaStream_t _stream; + + int *_sendbuff, *_recvbuff; + + int _size; + int _rank; + + bool p2p_enabled = false; + comms_ucp_handler _ucp_handler; + ucp_worker_h _ucp_worker; + std::shared_ptr _ucp_eps; + mutable request_t _next_request_id; + mutable std::unordered_map + _requests_in_flight; + mutable std::unordered_set _free_requests; +}; + +} // end namespace ML diff --git a/cpp/include/raft/comms/std/ucp_helper.hpp b/cpp/include/raft/comms/std/ucp_helper.hpp new file mode 100644 index 0000000000..fbb8b3e110 --- /dev/null +++ b/cpp/include/raft/comms/std/ucp_helper.hpp @@ -0,0 +1,240 @@ +/* + * Copyright (c) 2019-2020, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include +#include +#include +#include +#include +#include +#include + +#pragma once + +typedef void (*dlsym_print_info)(ucp_ep_h, FILE *); +typedef void (*dlsym_rec_free)(void *); +typedef int (*dlsym_worker_progress)(ucp_worker_h); + +typedef ucs_status_ptr_t (*dlsym_send)(ucp_ep_h, const void *, size_t, + ucp_datatype_t, ucp_tag_t, + ucp_send_callback_t); +typedef ucs_status_ptr_t (*dlsym_recv)(ucp_worker_h, void *, size_t count, + ucp_datatype_t datatype, ucp_tag_t, + ucp_tag_t, ucp_tag_recv_callback_t); + +/** + * Standard UCX request object that will be passed + * around asynchronously. This object is really + * opaque and the comms layer only cares that it + * has been completed. Because cuml comms do not + * initialize the ucx application context, it doesn't + * own this object and thus it's important not to + * modify this struct. + */ +struct ucx_context { + int completed; +}; + +/** + * Wraps the `ucx_context` request and adds a few + * other fields for trace logging and cleanup. + */ +class ucp_request { + public: + struct ucx_context *req; + bool needs_release = true; + int other_rank = -1; + bool is_send_request = false; +}; + +// by default, match the whole tag +static const ucp_tag_t default_tag_mask = -1; + +// Only match the passed in tag, not the rank. This +// enables simulated multi-cast. +static const ucp_tag_t any_rank_tag_mask = 0xFFFF0000; + +// Per the MPI API, receiving from a rank of -1 denotes receiving +// from any rank that used the expected tag. +static const int UCP_ANY_RANK = -1; + +/** + * @brief Asynchronous send callback sets request to completed + */ +static void send_callback(void *request, ucs_status_t status) { + struct ucx_context *context = (struct ucx_context *)request; + context->completed = 1; +} + +/** + * @brief Asynchronous recv callback sets request to completed + */ +static void recv_callback(void *request, ucs_status_t status, + ucp_tag_recv_info_t *info) { + struct ucx_context *context = (struct ucx_context *)request; + context->completed = 1; +} + +/** + * Helper class for managing `dlopen` state and + * interacting with ucp. + */ +class comms_ucp_handler { + public: + comms_ucp_handler() { + load_ucp_handle(); + load_send_func(); + load_recv_func(); + load_free_req_func(); + load_print_info_func(); + load_worker_progress_func(); + } + + ~comms_ucp_handler() { dlclose(ucp_handle); } + + private: + void *ucp_handle; + + dlsym_print_info print_info_func; + dlsym_rec_free req_free_func; + dlsym_worker_progress worker_progress_func; + dlsym_send send_func; + dlsym_recv recv_func; + + void load_ucp_handle() { + ucp_handle = dlopen("libucp.so", RTLD_LAZY | RTLD_NOLOAD | RTLD_NODELETE); + if (!ucp_handle) { + ucp_handle = dlopen("libucp.so", RTLD_LAZY | RTLD_NODELETE); + ASSERT(ucp_handle, "Cannot open UCX library: %s\n", dlerror()); + } + // Reset any potential error + dlerror(); + } + + void assert_dlerror() { + char *error = dlerror(); + ASSERT(error == NULL, "Error loading function symbol: %s\n", error); + } + + void load_send_func() { + send_func = (dlsym_send)dlsym(ucp_handle, "ucp_tag_send_nb"); + assert_dlerror(); + } + + void load_free_req_func() { + req_free_func = (dlsym_rec_free)dlsym(ucp_handle, "ucp_request_free"); + assert_dlerror(); + } + + void load_print_info_func() { + print_info_func = (dlsym_print_info)dlsym(ucp_handle, "ucp_ep_print_info"); + assert_dlerror(); + } + + void load_worker_progress_func() { + worker_progress_func = + (dlsym_worker_progress)dlsym(ucp_handle, "ucp_worker_progress"); + assert_dlerror(); + } + + void load_recv_func() { + recv_func = (dlsym_recv)dlsym(ucp_handle, "ucp_tag_recv_nb"); + assert_dlerror(); + } + + ucp_tag_t build_message_tag(int rank, int tag) const { + // keeping the rank in the lower bits enables debugging. + return ((uint32_t)tag << 31) | (uint32_t)rank; + } + + public: + int ucp_progress(ucp_worker_h worker) const { + return (*(worker_progress_func))(worker); + } + + /** + * @brief Frees any memory underlying the given ucp request object + */ + void free_ucp_request(ucp_request *request) const { + if (request->needs_release) { + request->req->completed = 0; + (*(req_free_func))(request->req); + } + free(request); + } + + /** + * @brief Asynchronously send data to the given endpoint using the given tag + */ + void ucp_isend(ucp_request *req, ucp_ep_h ep_ptr, const void *buf, int size, + int tag, ucp_tag_t tag_mask, int rank) const { + ucp_tag_t ucp_tag = build_message_tag(rank, tag); + + CUML_LOG_DEBUG("Sending tag: %ld", ucp_tag); + + ucs_status_ptr_t send_result = (*(send_func))( + ep_ptr, buf, size, ucp_dt_make_contig(1), ucp_tag, send_callback); + struct ucx_context *ucp_req = (struct ucx_context *)send_result; + if (UCS_PTR_IS_ERR(send_result)) { + ASSERT(!UCS_PTR_IS_ERR(send_result), + "unable to send UCX data message (%d)\n", + UCS_PTR_STATUS(send_result)); + /** + * If the request didn't fail, but it's not OK, it is in flight. + * Expect the handler to be invoked + */ + } else if (UCS_PTR_STATUS(send_result) != UCS_OK) { + /** + * If the request is OK, it's already been completed and we don't need to wait on it. + * The request will be a nullptr, however, so we need to create a new request + * and set it to completed to make the "waitall()" function work properly. + */ + req->needs_release = true; + } else { + req->needs_release = false; + } + + req->other_rank = rank; + req->is_send_request = true; + req->req = ucp_req; + } + + /** + * @brief Asynchronously receive data from given endpoint with the given tag. + */ + void ucp_irecv(ucp_request *req, ucp_worker_h worker, ucp_ep_h ep_ptr, + void *buf, int size, int tag, ucp_tag_t tag_mask, + int sender_rank) const { + ucp_tag_t ucp_tag = build_message_tag(sender_rank, tag); + + CUML_LOG_DEBUG("%d: Receiving tag: %ld", ucp_tag); + + ucs_status_ptr_t recv_result = + (*(recv_func))(worker, buf, size, ucp_dt_make_contig(1), ucp_tag, + tag_mask, recv_callback); + + struct ucx_context *ucp_req = (struct ucx_context *)recv_result; + + req->req = ucp_req; + req->needs_release = true; + req->is_send_request = false; + req->other_rank = sender_rank; + + ASSERT(!UCS_PTR_IS_ERR(recv_result), + "unable to receive UCX data message (%d)\n", + UCS_PTR_STATUS(recv_result)); + } +}; From 8a6e05464db3f69d98b25b6f4ed1e866a30f7cbb Mon Sep 17 00:00:00 2001 From: "Corey J. Nolet" Date: Wed, 6 May 2020 20:34:35 -0400 Subject: [PATCH 02/46] Removing cuml verbiage --- cpp/include/raft/comms/std/std_comms.hpp | 112 +++++++++++------------ 1 file changed, 54 insertions(+), 58 deletions(-) diff --git a/cpp/include/raft/comms/std/std_comms.hpp b/cpp/include/raft/comms/std/std_comms.hpp index 3ef08a8ce4..d084274f18 100644 --- a/cpp/include/raft/comms/std/std_comms.hpp +++ b/cpp/include/raft/comms/std/std_comms.hpp @@ -70,58 +70,58 @@ namespace raft { namespace { -size_t getDatatypeSize(const std_comms::datatype_t datatype) { +size_t getDatatypeSize(const comms::datatype_t datatype) { switch (datatype) { - case MLCommon::cumlCommunicator::CHAR: + case comms::CHAR: return sizeof(char); - case MLCommon::cumlCommunicator::UINT8: + case comms::UINT8: return sizeof(uint8_t); - case MLCommon::cumlCommunicator::INT: + case comms::INT: return sizeof(int); - case MLCommon::cumlCommunicator::UINT: + case comms::UINT: return sizeof(unsigned int); - case MLCommon::cumlCommunicator::INT64: + case comms::INT64: return sizeof(int64_t); - case MLCommon::cumlCommunicator::UINT64: + case comms::UINT64: return sizeof(uint64_t); - case MLCommon::cumlCommunicator::FLOAT: + case comms::FLOAT: return sizeof(float); - case MLCommon::cumlCommunicator::DOUBLE: + case comms::DOUBLE: return sizeof(double); } } ncclDataType_t getNCCLDatatype( - const std_comms::datatype_t datatype) { + const comms::datatype_t datatype) { switch (datatype) { - case MLCommon::cumlCommunicator::CHAR: + case comms::CHAR: return ncclChar; - case MLCommon::cumlCommunicator::UINT8: + case comms::UINT8: return ncclUint8; - case MLCommon::cumlCommunicator::INT: + case comms::INT: return ncclInt; - case MLCommon::cumlCommunicator::UINT: + case comms::UINT: return ncclUint32; - case MLCommon::cumlCommunicator::INT64: + case comms::INT64: return ncclInt64; - case MLCommon::cumlCommunicator::UINT64: + case comms::UINT64: return ncclUint64; - case MLCommon::cumlCommunicator::FLOAT: + case comms::FLOAT: return ncclFloat; - case MLCommon::cumlCommunicator::DOUBLE: + case comms::DOUBLE: return ncclDouble; } } -ncclRedOp_t getNCCLOp(const std_comms::op_t op) { +ncclRedOp_t getNCCLOp(const comms::op_t op) { switch (op) { - case MLCommon::cumlCommunicator::SUM: + case comms::SUM: return ncclSum; - case MLCommon::cumlCommunicator::PROD: + case comms::PROD: return ncclProd; - case MLCommon::cumlCommunicator::MIN: + case comms::MIN: return ncclMin; - case MLCommon::cumlCommunicator::MAX: + case comms::MAX: return ncclMax; } } @@ -140,14 +140,14 @@ bool ucx_enabled() { return UCX_ENABLED; } */ void inject_comms(cumlHandle &handle, ncclComm_t comm, ucp_worker_h ucp_worker, std::shared_ptr eps, int size, int rank) { - auto communicator = std::make_shared( + auto communicator = std::make_shared( std::unique_ptr( new std_comms(comm, ucp_worker, eps, size, rank))); handle.getImpl().setCommunicator(communicator); } void inject_comms(cumlHandle &handle, ncclComm_t comm, int size, int rank) { - auto communicator = std::make_shared( + auto communicator = std::make_shared( std::unique_ptr( new std_comms(comm, size, rank))); handle.getImpl().setCommunicator(communicator); @@ -182,7 +182,7 @@ void inject_comms_py(ML::cumlHandle *handle, ncclComm_t comm, void *ucp_worker, /** - * @brief A cumlCommunicator implementation capable of running collective communications + * @brief A comms implementation capable of running collective communications * with NCCL and point-to-point-communications with UCX. Note that the latter is optional. * * Underlying comms, like NCCL and UCX, should be initialized and ready for use, @@ -235,66 +235,66 @@ class std_comms : public raft::comms { CUDA_CHECK_NO_THROW(cudaFree(_recvbuff)); } - size_t getDatatypeSize(const std_comms::datatype_t datatype) { + size_t getDatatypeSize(const comms::datatype_t datatype) { switch (datatype) { - case MLCommon::cumlCommunicator::CHAR: + case comms::CHAR: return sizeof(char); - case MLCommon::cumlCommunicator::UINT8: + case comms::UINT8: return sizeof(uint8_t); - case MLCommon::cumlCommunicator::INT: + case comms::INT: return sizeof(int); - case MLCommon::cumlCommunicator::UINT: + case comms::UINT: return sizeof(unsigned int); - case MLCommon::cumlCommunicator::INT64: + case comms::INT64: return sizeof(int64_t); - case MLCommon::cumlCommunicator::UINT64: + case comms::UINT64: return sizeof(uint64_t); - case MLCommon::cumlCommunicator::FLOAT: + case comms::FLOAT: return sizeof(float); - case MLCommon::cumlCommunicator::DOUBLE: + case comms::DOUBLE: return sizeof(double); } } template <> - cumlCommunicator::datatype_t getDataType() const { - return cumlCommunicator::CHAR; + comms::datatype_t getDataType() const { + return comms::CHAR; } template <> - cumlCommunicator::datatype_t getDataType() const { - return cumlCommunicator::UINT8; + comms::datatype_t getDataType() const { + return comms::UINT8; } template <> - cumlCommunicator::datatype_t getDataType() const { - return cumlCommunicator::INT; + comms::datatype_t getDataType() const { + return comms::INT; } template <> - cumlCommunicator::datatype_t getDataType() const { - return cumlCommunicator::UINT; + comms::datatype_t getDataType() const { + return comms::UINT; } template <> - cumlCommunicator::datatype_t getDataType() const { - return cumlCommunicator::INT64; + comms::datatype_t getDataType() const { + return comms::INT64; } template <> - cumlCommunicator::datatype_t getDataType() const { - return cumlCommunicator::UINT64; + comms::datatype_t getDataType() const { + return comms::UINT64; } template <> - cumlCommunicator::datatype_t getDataType() const { - return cumlCommunicator::FLOAT; + comms::datatype_t getDataType() const { + return comms::FLOAT; } template <> - cumlCommunicator::datatype_t getDataType() const { - return cumlCommunicator::DOUBLE; + comms::datatype_t getDataType() const { + return comms::DOUBLE; } void initialize() { @@ -309,7 +309,7 @@ class std_comms : public raft::comms { int getRank() const { return _rank; } - std::unique_ptr + std::unique_ptr commSplit(int color, int key) const { // Not supported by NCCL ASSERT(false, @@ -321,8 +321,8 @@ class std_comms : public raft::comms { CUDA_CHECK(cudaMemsetAsync(_sendbuff, 1, sizeof(int), _stream)); CUDA_CHECK(cudaMemsetAsync(_recvbuff, 1, sizeof(int), _stream)); - allreduce(_sendbuff, _recvbuff, 1, MLCommon::cumlCommunicator::INT, - MLCommon::cumlCommunicator::SUM, _stream); + allreduce(_sendbuff, _recvbuff, 1, comms::INT, + comms::SUM, _stream); ASSERT(syncStream(_stream) == status_t::commStatusSuccess, "ERROR: syncStream failed. This can be caused by a failed rank."); @@ -382,10 +382,6 @@ class std_comms : public raft::comms { ucp_tag_t tag_mask = default_tag_mask; - if (source == CUML_ANY_SOURCE) { - tag_mask = any_rank_tag_mask; - } - ucp_request *ucp_req = (ucp_request *)malloc(sizeof(ucp_request)); _ucp_handler.ucp_irecv(ucp_req, _ucp_worker, ep_ptr, buf, size, tag, tag_mask, source); From 2bfc0dc304e06c4e532fa2b7ee582b3ddfea8537 Mon Sep 17 00:00:00 2001 From: "Corey J. Nolet" Date: Thu, 7 May 2020 20:25:17 -0400 Subject: [PATCH 03/46] Renaming to comms_t, removing logs for now, adding to handle --- cpp/include/raft/comms.hpp | 0 cpp/include/raft/comms/comms.hpp | 8 +- cpp/include/raft/comms/std/std_comms.hpp | 162 +++++++++------------- cpp/include/raft/comms/std/ucp_helper.hpp | 21 +-- cpp/include/raft/handle.hpp | 24 ++-- 5 files changed, 88 insertions(+), 127 deletions(-) delete mode 100644 cpp/include/raft/comms.hpp diff --git a/cpp/include/raft/comms.hpp b/cpp/include/raft/comms.hpp deleted file mode 100644 index e69de29bb2..0000000000 diff --git a/cpp/include/raft/comms/comms.hpp b/cpp/include/raft/comms/comms.hpp index 0ddba4e7b8..b9d9c11d07 100644 --- a/cpp/include/raft/comms/comms.hpp +++ b/cpp/include/raft/comms/comms.hpp @@ -21,7 +21,7 @@ namespace raft { -class comms { +class comms_t { public: typedef unsigned int request_t; @@ -37,17 +37,17 @@ class comms { commStatusAbort }; // A failure occurred in sync, queued operations aborted - virtual size_t getDatatypeSize(const comms::datatype_t datatype); + virtual size_t getDatatypeSize(const comms_t::datatype_t datatype); template virtual datatype_t getDataType() const; - virtual ~comms(); + virtual ~comms_t(); virtual int getSize() const = 0; virtual int getRank() const = 0; - virtual std::unique_ptr commSplit(int color, int key) const = 0; + virtual std::unique_ptr commSplit(int color, int key) const = 0; virtual void barrier() const = 0; diff --git a/cpp/include/raft/comms/std/std_comms.hpp b/cpp/include/raft/comms/std/std_comms.hpp index d084274f18..13ad6773cb 100644 --- a/cpp/include/raft/comms/std/std_comms.hpp +++ b/cpp/include/raft/comms/std/std_comms.hpp @@ -59,7 +59,7 @@ constexpr bool UCX_ENABLED = true; do { \ ncclResult_t status = call; \ if (status != ncclSuccess) { \ - CUML_LOG_ERROR("NCCL call='%s' failed. Reason:%s\n", #call, \ + printf("NCCL call='%s' failed. Reason:%s\n", #call, \ ncclGetErrorString(status)); \ } \ } while (0) @@ -70,58 +70,58 @@ namespace raft { namespace { -size_t getDatatypeSize(const comms::datatype_t datatype) { +size_t getDatatypeSize(const comms_t::datatype_t datatype) { switch (datatype) { - case comms::CHAR: + case comms_t::CHAR: return sizeof(char); - case comms::UINT8: + case comms_t::UINT8: return sizeof(uint8_t); - case comms::INT: + case comms_t::INT: return sizeof(int); - case comms::UINT: + case comms_t::UINT: return sizeof(unsigned int); - case comms::INT64: + case comms_t::INT64: return sizeof(int64_t); - case comms::UINT64: + case comms_t::UINT64: return sizeof(uint64_t); - case comms::FLOAT: + case comms_t::FLOAT: return sizeof(float); - case comms::DOUBLE: + case comms_t::DOUBLE: return sizeof(double); } } ncclDataType_t getNCCLDatatype( - const comms::datatype_t datatype) { + const comms_t::datatype_t datatype) { switch (datatype) { - case comms::CHAR: + case comms_t::CHAR: return ncclChar; - case comms::UINT8: + case comms_t::UINT8: return ncclUint8; - case comms::INT: + case comms_t::INT: return ncclInt; - case comms::UINT: + case comms_t::UINT: return ncclUint32; - case comms::INT64: + case comms_t::INT64: return ncclInt64; - case comms::UINT64: + case comms_t::UINT64: return ncclUint64; - case comms::FLOAT: + case comms_t::FLOAT: return ncclFloat; - case comms::DOUBLE: + case comms_t::DOUBLE: return ncclDouble; } } -ncclRedOp_t getNCCLOp(const comms::op_t op) { +ncclRedOp_t getNCCLOp(const comms_t::op_t op) { switch (op) { - case comms::SUM: + case comms_t::SUM: return ncclSum; - case comms::PROD: + case comms_t::PROD: return ncclProd; - case comms::MIN: + case comms_t::MIN: return ncclMin; - case comms::MAX: + case comms_t::MAX: return ncclMax; } } @@ -129,26 +129,17 @@ ncclRedOp_t getNCCLOp(const comms::op_t op) { bool ucx_enabled() { return UCX_ENABLED; } -/** - * @brief Underlying comms, like NCCL and UCX, should be initialized and ready for use, - * and maintained, outside of the cuML Comms lifecycle. This allows us to decouple the - * ownership of the actual comms from cuml so that they can also be used outside of cuml. - * - * For instance, nccl-py can be used to bootstrap a ncclComm_t before it is - * used to construct a cuml comms instance. UCX endpoints can be bootstrapped - * in Python using ucx-py, before being used to construct a cuML comms instance. - */ void inject_comms(cumlHandle &handle, ncclComm_t comm, ucp_worker_h ucp_worker, std::shared_ptr eps, int size, int rank) { - auto communicator = std::make_shared( - std::unique_ptr( + auto communicator = std::make_shared( + std::unique_ptr( new std_comms(comm, ucp_worker, eps, size, rank))); handle.getImpl().setCommunicator(communicator); } void inject_comms(cumlHandle &handle, ncclComm_t comm, int size, int rank) { - auto communicator = std::make_shared( - std::unique_ptr( + auto communicator = std::make_shared( + std::unique_ptr( new std_comms(comm, size, rank))); handle.getImpl().setCommunicator(communicator); } @@ -181,19 +172,7 @@ void inject_comms_py(ML::cumlHandle *handle, ncclComm_t comm, void *ucp_worker, } -/** - * @brief A comms implementation capable of running collective communications - * with NCCL and point-to-point-communications with UCX. Note that the latter is optional. - * - * Underlying comms, like NCCL and UCX, should be initialized and ready for use, - * and maintained, outside of the cuML Comms lifecycle. This allows us to decouple the - * ownership of the actual comms from cuml so that they can also be used outside of cuml. - * - * For instance, nccl-py can be used to bootstrap a ncclComm_t before it is - * used to construct a cuml comms instance. UCX endpoints can be bootstrapped - * in Python using ucx-py, before being used to construct a cuML comms instance. - */ -class std_comms : public raft::comms { +class std_comms : public raft::comms_t { public: std_comms() = delete; @@ -235,66 +214,66 @@ class std_comms : public raft::comms { CUDA_CHECK_NO_THROW(cudaFree(_recvbuff)); } - size_t getDatatypeSize(const comms::datatype_t datatype) { + size_t getDatatypeSize(const c::datatype_t datatype) { switch (datatype) { - case comms::CHAR: + case comms_t::CHAR: return sizeof(char); - case comms::UINT8: + case comms_t::UINT8: return sizeof(uint8_t); - case comms::INT: + case comms_t::INT: return sizeof(int); - case comms::UINT: + case comms_t::UINT: return sizeof(unsigned int); - case comms::INT64: + case comms_t::INT64: return sizeof(int64_t); - case comms::UINT64: + case comms_t::UINT64: return sizeof(uint64_t); - case comms::FLOAT: + case comms_t::FLOAT: return sizeof(float); - case comms::DOUBLE: + case comms_t::DOUBLE: return sizeof(double); } } template <> - comms::datatype_t getDataType() const { - return comms::CHAR; + comms_t::datatype_t getDataType() const { + return comms_t::CHAR; } template <> - comms::datatype_t getDataType() const { - return comms::UINT8; + comms_t::datatype_t getDataType() const { + return comms_t::UINT8; } template <> - comms::datatype_t getDataType() const { - return comms::INT; + comms_t::datatype_t getDataType() const { + return comms_t::INT; } template <> - comms::datatype_t getDataType() const { - return comms::UINT; + comms_t::datatype_t getDataType() const { + return comms_t::UINT; } template <> - comms::datatype_t getDataType() const { - return comms::INT64; + comms_t::datatype_t getDataType() const { + return comms_t::INT64; } template <> - comms::datatype_t getDataType() const { - return comms::UINT64; + comms_t::datatype_t getDataType() const { + return comms_t::UINT64; } template <> - comms::datatype_t getDataType() const { - return comms::FLOAT; + comms_t::datatype_t getDataType() const { + return comms_t::FLOAT; } template <> - comms::datatype_t getDataType() const { - return comms::DOUBLE; + comms_t::datatype_t getDataType() const { + return comms_t::DOUBLE; } void initialize() { @@ -309,7 +288,7 @@ class std_comms : public raft::comms { int getRank() const { return _rank; } - std::unique_ptr + std::unique_ptr commSplit(int color, int key) const { // Not supported by NCCL ASSERT(false, @@ -321,8 +300,8 @@ class std_comms : public raft::comms { CUDA_CHECK(cudaMemsetAsync(_sendbuff, 1, sizeof(int), _stream)); CUDA_CHECK(cudaMemsetAsync(_recvbuff, 1, sizeof(int), _stream)); - allreduce(_sendbuff, _recvbuff, 1, comms::INT, - comms::SUM, _stream); + allreduce(_sendbuff, _recvbuff, 1, comms_t::INT, + comms_t::SUM, _stream); ASSERT(syncStream(_stream) == status_t::commStatusSuccess, "ERROR: syncStream failed. This can be caused by a failed rank."); @@ -344,9 +323,9 @@ class std_comms : public raft::comms { void isend(const void *buf, int size, int dest, int tag, request_t *request) const { - ASSERT(UCX_ENABLED, "cuML Comms not built with UCX support"); + ASSERT(UCX_ENABLED, "Comms not built with UCX support"); ASSERT(p2p_enabled, - "cuML Comms instance was not initialized for point-to-point"); + "Comms instance was not initialized for point-to-point"); ASSERT(_ucp_worker != nullptr, "ERROR: UCX comms not initialized on communicator."); @@ -359,19 +338,14 @@ class std_comms : public raft::comms { this->_ucp_handler.ucp_isend(ucp_req, ep_ptr, buf, size, tag, default_tag_mask, getRank()); - CUML_LOG_DEBUG( - "%d: Created send request [id=%llu], ptr=%llu, to=%llu, ep=%llu", getRank(), - (unsigned long long)*request, (unsigned long long)ucp_req->req, - (unsigned long long)dest, (unsigned long long)ep_ptr); - _requests_in_flight.insert(std::make_pair(*request, ucp_req)); } void irecv(void *buf, int size, int source, int tag, request_t *request) const { - ASSERT(UCX_ENABLED, "cuML Comms not built with UCX support"); + ASSERT(UCX_ENABLED, "Comms not built with UCX support"); ASSERT(p2p_enabled, - "cuML Comms instance was not initialized for point-to-point"); + "Comms instance was not initialized for point-to-point"); ASSERT(_ucp_worker != nullptr, "ERROR: UCX comms not initialized on communicator."); @@ -386,19 +360,14 @@ class std_comms : public raft::comms { _ucp_handler.ucp_irecv(ucp_req, _ucp_worker, ep_ptr, buf, size, tag, tag_mask, source); - CUML_LOG_DEBUG( - "%d: Created receive request [id=%llu], ptr=%llu, from=%llu, ep=%llu", - getRank(), (unsigned long long)*request, (unsigned long long)ucp_req->req, - (unsigned long long)source, (unsigned long long)ep_ptr); - _requests_in_flight.insert(std::make_pair(*request, ucp_req)); } void waitall(int count, request_t array_of_requests[]) const { - ASSERT(UCX_ENABLED, "cuML Comms not built with UCX support"); + ASSERT(UCX_ENABLED, "Comms not built with UCX support"); ASSERT(p2p_enabled, - "cuML Comms instance was not initialized for point-to-point"); + "Comms instance was not initialized for point-to-point"); ASSERT(_ucp_worker != nullptr, "ERROR: UCX comms not initialized on communicator."); @@ -452,11 +421,6 @@ class std_comms : public raft::comms { // is complete, we can go ahead and clean it up. if (!req->needs_release || req->req->completed == 1) { restart = true; - CUML_LOG_DEBUG( - "%d: request completed. [ptr=%llu, num_left=%lu," - " other_rank=%d, is_send=%d, completed_immediately=%d]", - getRank(), (unsigned long long)req->req, requests.size() - 1, - req->other_rank, req->is_send_request, !req->needs_release); // perform cleanup _ucp_handler.free_ucp_request(req); diff --git a/cpp/include/raft/comms/std/ucp_helper.hpp b/cpp/include/raft/comms/std/ucp_helper.hpp index fbb8b3e110..584bcc8a53 100644 --- a/cpp/include/raft/comms/std/ucp_helper.hpp +++ b/cpp/include/raft/comms/std/ucp_helper.hpp @@ -14,14 +14,11 @@ * limitations under the License. */ -#include +#include #include #include #include #include -#include -#include - #pragma once typedef void (*dlsym_print_info)(ucp_ep_h, FILE *); @@ -63,14 +60,6 @@ class ucp_request { // by default, match the whole tag static const ucp_tag_t default_tag_mask = -1; -// Only match the passed in tag, not the rank. This -// enables simulated multi-cast. -static const ucp_tag_t any_rank_tag_mask = 0xFFFF0000; - -// Per the MPI API, receiving from a rank of -1 denotes receiving -// from any rank that used the expected tag. -static const int UCP_ANY_RANK = -1; - /** * @brief Asynchronous send callback sets request to completed */ @@ -182,8 +171,8 @@ class comms_ucp_handler { void ucp_isend(ucp_request *req, ucp_ep_h ep_ptr, const void *buf, int size, int tag, ucp_tag_t tag_mask, int rank) const { ucp_tag_t ucp_tag = build_message_tag(rank, tag); - - CUML_LOG_DEBUG("Sending tag: %ld", ucp_tag); +// +// CUML_LOG_DEBUG("Sending tag: %ld", ucp_tag); ucs_status_ptr_t send_result = (*(send_func))( ep_ptr, buf, size, ucp_dt_make_contig(1), ucp_tag, send_callback); @@ -219,8 +208,8 @@ class comms_ucp_handler { void *buf, int size, int tag, ucp_tag_t tag_mask, int sender_rank) const { ucp_tag_t ucp_tag = build_message_tag(sender_rank, tag); - - CUML_LOG_DEBUG("%d: Receiving tag: %ld", ucp_tag); +// +// CUML_LOG_DEBUG("%d: Receiving tag: %ld", ucp_tag); ucs_status_ptr_t recv_result = (*(recv_func))(worker, buf, size, ucp_dt_make_contig(1), ucp_tag, diff --git a/cpp/include/raft/handle.hpp b/cpp/include/raft/handle.hpp index 7c8898fd96..ed4481364e 100644 --- a/cpp/include/raft/handle.hpp +++ b/cpp/include/raft/handle.hpp @@ -149,11 +149,21 @@ class handle_t { } } - ///@todo: enable this once we have cuml-comms migrated - // void setCommunicator( - // std::shared_ptr communicator); - // const MLCommon::cumlCommunicator& getCommunicator() const; - // bool commsInitialized() const; + void setCommunicator( + std::shared_ptr communicator) { + _communicator = communicator; + } + + const comms_t& getCommunicator() const { + ASSERT(nullptr != _communicator.get(), + "ERROR: Communicator was not initialized\n"); + return *_communicator; + } + + bool commsInitialized() const { + return (nullptr != _communicator.get()); + } + const cudaDeviceProp& getDeviceProperties() const { if (!_devicePropInitialized) { @@ -181,9 +191,7 @@ class handle_t { cudaEvent_t _event; mutable cudaDeviceProp _prop; mutable bool _devicePropInitialized; - - ///@todo: enable this once we have migrated cuml-comms - //std::shared_ptr _communicator; + std::shared_ptr _communicator; void createResources() { cudaStream_t stream; From f129710c8537572fd6d454e7433565acc1392fb7 Mon Sep 17 00:00:00 2001 From: "Corey J. Nolet" Date: Thu, 7 May 2020 20:27:33 -0400 Subject: [PATCH 04/46] Looking like it's building! --- cpp/include/raft/comms/comms.hpp | 2 +- cpp/include/raft/handle.hpp | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/cpp/include/raft/comms/comms.hpp b/cpp/include/raft/comms/comms.hpp index b9d9c11d07..c27ed430fd 100644 --- a/cpp/include/raft/comms/comms.hpp +++ b/cpp/include/raft/comms/comms.hpp @@ -40,7 +40,7 @@ class comms_t { virtual size_t getDatatypeSize(const comms_t::datatype_t datatype); template - virtual datatype_t getDataType() const; + datatype_t getDataType() const; virtual ~comms_t(); diff --git a/cpp/include/raft/handle.hpp b/cpp/include/raft/handle.hpp index ed4481364e..fab79372ff 100644 --- a/cpp/include/raft/handle.hpp +++ b/cpp/include/raft/handle.hpp @@ -34,6 +34,7 @@ #include #include #include +#include #include "allocator.hpp" #include "cudart_utils.h" From a0c13afdeb46c5cfd15c3555204b00b5bee1bb05 Mon Sep 17 00:00:00 2001 From: "Corey J. Nolet" Date: Thu, 7 May 2020 20:38:56 -0400 Subject: [PATCH 05/46] Adding NCCL and UCX build for tests --- cpp/CMakeLists.txt | 1 + cpp/include/raft/comms/CMakeLists.txt | 41 +++++++++++++++++++++++++++ 2 files changed, 42 insertions(+) create mode 100644 cpp/include/raft/comms/CMakeLists.txt diff --git a/cpp/CMakeLists.txt b/cpp/CMakeLists.txt index f5696cf121..b0783964a2 100644 --- a/cpp/CMakeLists.txt +++ b/cpp/CMakeLists.txt @@ -161,6 +161,7 @@ set(CMAKE_CUDA_FLAGS # - dependencies ------------------------------------------------------------- include(cmake/Dependencies.cmake) +add_subdirectory(include/raft/comms) ############################################################################## # - include paths ------------------------------------------------------------ diff --git a/cpp/include/raft/comms/CMakeLists.txt b/cpp/include/raft/comms/CMakeLists.txt new file mode 100644 index 0000000000..41b18926ae --- /dev/null +++ b/cpp/include/raft/comms/CMakeLists.txt @@ -0,0 +1,41 @@ +# +# Copyright (c) 2019-2020, NVIDIA CORPORATION. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +cmake_minimum_required(VERSION 3.14 FATAL_ERROR) +project(comms LANGUAGES CXX CUDA) + +set(CMAKE_MODULE_PATH ${CMAKE_MODULE_PATH} "${CMAKE_CURRENT_SOURCE_DIR}/cmake") + +option(WITH_UCX "Uses UCX for P2P comms" ON) + +if(NOT NCCL_PATH) + find_package(NCCL REQUIRED) +else() + message("-- Manually set NCCL PATH to ${NCCL_PATH}") + set(NCCL_INCLUDE_DIRS ${NCCL_PATH}/include) + set(NCCL_LIBRARIES ${NCCL_PATH}/lib/libnccl.so) +endif(NOT NCCL_PATH) + +set(CMAKE_CXX_STANDARD 14) +set(CMAKE_CXX_STANDARD_REQUIRED ON) + +find_package(UCX) +include_directories(${UCX_INCLUDE_DIRS}) +add_compile_definitions(WITH_UCX=1) + +add_definitions(-DHAVE_NCCL) +include_directories( ${NCCL_INCLUDE_DIRS} ) +list(APPEND RAFT_LINK_LIBRARIES ${NCCL_LIBRARIES}) \ No newline at end of file From cb194829f3573b80f712591f6b51b10c12df1a3d Mon Sep 17 00:00:00 2001 From: "Corey J. Nolet" Date: Thu, 7 May 2020 21:22:57 -0400 Subject: [PATCH 06/46] Consolidating injection functions --- cpp/include/raft/comms/std/std_comms.hpp | 32 +++++++++-------------- cpp/include/raft/comms/std/ucp_helper.hpp | 4 --- 2 files changed, 12 insertions(+), 24 deletions(-) diff --git a/cpp/include/raft/comms/std/std_comms.hpp b/cpp/include/raft/comms/std/std_comms.hpp index 13ad6773cb..7047914fde 100644 --- a/cpp/include/raft/comms/std/std_comms.hpp +++ b/cpp/include/raft/comms/std/std_comms.hpp @@ -129,28 +129,17 @@ ncclRedOp_t getNCCLOp(const comms_t::op_t op) { bool ucx_enabled() { return UCX_ENABLED; } -void inject_comms(cumlHandle &handle, ncclComm_t comm, ucp_worker_h ucp_worker, - std::shared_ptr eps, int size, int rank) { - auto communicator = std::make_shared( - std::unique_ptr( - new std_comms(comm, ucp_worker, eps, size, rank))); - handle.getImpl().setCommunicator(communicator); -} - -void inject_comms(cumlHandle &handle, ncclComm_t comm, int size, int rank) { - auto communicator = std::make_shared( - std::unique_ptr( - new std_comms(comm, size, rank))); - handle.getImpl().setCommunicator(communicator); -} - -void inject_comms_py_coll(cumlHandle *handle, ncclComm_t comm, int size, +void inject_comms(handle_t *handle, ncclComm_t comm, int size, int rank) { - inject_comms(*handle, comm, size, rank); + auto communicator = std::make_shared( + std::unique_ptr( + new std_comms(comm, size, rank))); + handle->setCommunicator(communicator); } -void inject_comms_py(ML::cumlHandle *handle, ncclComm_t comm, void *ucp_worker, +void inject_comms(handle_t *handle, ncclComm_t comm, void *ucp_worker, void *eps, int size, int rank) { + std::shared_ptr eps_sp = std::make_shared(new ucp_ep_h[size]); @@ -168,11 +157,14 @@ void inject_comms_py(ML::cumlHandle *handle, ncclComm_t comm, void *ucp_worker, } } - inject_comms(*handle, comm, (ucp_worker_h)ucp_worker, eps_sp, size, rank); + auto communicator = std::make_shared( + std::unique_ptr( + new std_comms(comm, ucp_worker, eps, size, rank))); + handle->setCommunicator(communicator); } -class std_comms : public raft::comms_t { +class std_comms : public comms_t { public: std_comms() = delete; diff --git a/cpp/include/raft/comms/std/ucp_helper.hpp b/cpp/include/raft/comms/std/ucp_helper.hpp index 584bcc8a53..2cf7bed6f8 100644 --- a/cpp/include/raft/comms/std/ucp_helper.hpp +++ b/cpp/include/raft/comms/std/ucp_helper.hpp @@ -171,8 +171,6 @@ class comms_ucp_handler { void ucp_isend(ucp_request *req, ucp_ep_h ep_ptr, const void *buf, int size, int tag, ucp_tag_t tag_mask, int rank) const { ucp_tag_t ucp_tag = build_message_tag(rank, tag); -// -// CUML_LOG_DEBUG("Sending tag: %ld", ucp_tag); ucs_status_ptr_t send_result = (*(send_func))( ep_ptr, buf, size, ucp_dt_make_contig(1), ucp_tag, send_callback); @@ -208,8 +206,6 @@ class comms_ucp_handler { void *buf, int size, int tag, ucp_tag_t tag_mask, int sender_rank) const { ucp_tag_t ucp_tag = build_message_tag(sender_rank, tag); -// -// CUML_LOG_DEBUG("%d: Receiving tag: %ld", ucp_tag); ucs_status_ptr_t recv_result = (*(recv_func))(worker, buf, size, ucp_dt_make_contig(1), ucp_tag, From 15481774f75250a7de8c8a627433f146533b6240 Mon Sep 17 00:00:00 2001 From: "Corey J. Nolet" Date: Tue, 12 May 2020 16:12:43 -0400 Subject: [PATCH 07/46] Making progress on Python. Need to refactor the comms interface a little bit --- cpp/include/raft/comms/CMakeLists.txt | 4 - cpp/include/raft/comms/comms.hpp | 50 ++- cpp/include/raft/comms/comms_helper.hpp | 146 ++++++++ cpp/include/raft/comms/nccl_helper.hpp | 18 + .../raft/comms/{std => }/std_comms.hpp | 134 +------ .../raft/comms/{std => }/ucp_helper.hpp | 2 +- cpp/include/raft/cudart_utils.h | 16 +- cpp/include/raft/handle.hpp | 10 +- python/raft/common/__init__.py | 0 python/raft/common/cuda.pxd | 36 ++ python/raft/common/cuda.pyx | 88 +++++ python/raft/common/handle.pxd | 38 ++ python/raft/common/handle.pyx | 108 ++++++ python/raft/dask/common/comms.py | 329 ++++++++++++++++++ python/raft/dask/common/comms_utils.pyx | 155 +++++++++ python/raft/dask/common/nccl.pyx | 233 +++++++++++++ python/raft/dask/common/ucx.py | 77 ++++ python/setup.py | 1 + 18 files changed, 1306 insertions(+), 139 deletions(-) create mode 100644 cpp/include/raft/comms/comms_helper.hpp create mode 100644 cpp/include/raft/comms/nccl_helper.hpp rename cpp/include/raft/comms/{std => }/std_comms.hpp (80%) rename cpp/include/raft/comms/{std => }/ucp_helper.hpp (99%) create mode 100644 python/raft/common/__init__.py create mode 100644 python/raft/common/cuda.pxd create mode 100644 python/raft/common/cuda.pyx create mode 100644 python/raft/common/handle.pxd create mode 100644 python/raft/common/handle.pyx create mode 100644 python/raft/dask/common/comms.py create mode 100644 python/raft/dask/common/comms_utils.pyx create mode 100644 python/raft/dask/common/nccl.pyx create mode 100644 python/raft/dask/common/ucx.py diff --git a/cpp/include/raft/comms/CMakeLists.txt b/cpp/include/raft/comms/CMakeLists.txt index 41b18926ae..734ce11812 100644 --- a/cpp/include/raft/comms/CMakeLists.txt +++ b/cpp/include/raft/comms/CMakeLists.txt @@ -19,8 +19,6 @@ project(comms LANGUAGES CXX CUDA) set(CMAKE_MODULE_PATH ${CMAKE_MODULE_PATH} "${CMAKE_CURRENT_SOURCE_DIR}/cmake") -option(WITH_UCX "Uses UCX for P2P comms" ON) - if(NOT NCCL_PATH) find_package(NCCL REQUIRED) else() @@ -34,8 +32,6 @@ set(CMAKE_CXX_STANDARD_REQUIRED ON) find_package(UCX) include_directories(${UCX_INCLUDE_DIRS}) -add_compile_definitions(WITH_UCX=1) -add_definitions(-DHAVE_NCCL) include_directories( ${NCCL_INCLUDE_DIRS} ) list(APPEND RAFT_LINK_LIBRARIES ${NCCL_LIBRARIES}) \ No newline at end of file diff --git a/cpp/include/raft/comms/comms.hpp b/cpp/include/raft/comms/comms.hpp index c27ed430fd..a5152be620 100644 --- a/cpp/include/raft/comms/comms.hpp +++ b/cpp/include/raft/comms/comms.hpp @@ -20,6 +20,7 @@ namespace raft { +namespace comms { class comms_t { public: @@ -37,11 +38,6 @@ class comms_t { commStatusAbort }; // A failure occurred in sync, queued operations aborted - virtual size_t getDatatypeSize(const comms_t::datatype_t datatype); - - template - datatype_t getDataType() const; - virtual ~comms_t(); virtual int getSize() const = 0; @@ -84,4 +80,48 @@ class comms_t { cudaStream_t stream) const = 0; }; + +template +comms_t::datatype_t getDataType(T a); + +template <> +comms_t::datatype_t getDataType(char a) { + return comms_t::CHAR; +} + +template <> +comms_t::datatype_t getDataType(uint8_t a) { + return comms_t::UINT8; +} + +template <> +comms_t::datatype_t getDataType(int a) { + return comms_t::INT; +} + +template <> +comms_t::datatype_t getDataType(uint32_t a) { + return comms_t::UINT; +} + +template <> +comms_t::datatype_t getDataType(int64_t a) { + return comms_t::INT64; +} + +template <> +comms_t::datatype_t getDataType(uint64_t a) { + return comms_t::UINT64; +} + +template <> +comms_t::datatype_t getDataType(float a) { + return comms_t::FLOAT; +} + +template <> +comms_t::datatype_t getDataType(double a) { + return comms_t::DOUBLE; +} +} /// namespace comms } // namespace raft diff --git a/cpp/include/raft/comms/comms_helper.hpp b/cpp/include/raft/comms/comms_helper.hpp new file mode 100644 index 0000000000..f1a5482334 --- /dev/null +++ b/cpp/include/raft/comms/comms_helper.hpp @@ -0,0 +1,146 @@ +/* + * Copyright (c) 2019, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#include +#include +#include +#include + +namespace raft { +namespace comms { + +void build_comms_nccl_only(handle_t *handle, ncclComm_t comm, int size, + int rank) { + + auto *raft_comm = new raft::comms::std_comms(comm, size, rank); + auto communicator = std::make_shared( + std::unique_ptr(raft_comm)); + handle->set_comms(communicator); +} + +void build_comms_nccl_ucx(handle_t *handle, ncclComm_t comm, void *ucp_worker, + void *eps, int size, int rank) { + + std::shared_ptr eps_sp = + std::make_shared(new ucp_ep_h[size]); + + size_t *size_t_ep_arr = (size_t *)eps; + + for (int i = 0; i < size; i++) { + size_t ptr = size_t_ep_arr[i]; + ucp_ep_h *ucp_ep_v = (ucp_ep_h *)*eps_sp; + + if (ptr != 0) { + ucp_ep_h eps_ptr = (ucp_ep_h)size_t_ep_arr[i]; + ucp_ep_v[i] = eps_ptr; + } else { + ucp_ep_v[i] = nullptr; + } + } + + auto communicator = std::make_shared( + std::unique_ptr( + new raft::comms::std_comms(comm, (ucp_worker_h)ucp_worker, eps_sp, size, rank))); + handle->set_comms(communicator); +} + + +bool test_collective_allreduce(const handle_t& handle) { + const comms_t& communicator = handle.get_comms(); + + const int send = 1; + + cudaStream_t stream = handle.get_stream(); + + raft::mr::device::buffer temp_d(handle.get_device_allocator(), stream); + temp_d.resize(1, stream); + CUDA_CHECK(cudaMemcpyAsync(temp_d.data(), &send, sizeof(int), + cudaMemcpyHostToDevice, stream)); + communicator.allreduce(temp_d.data(), temp_d.data(), 1, getDataType(temp_d.data()), + comms_t::SUM, stream); + int temp_h = 0; + CUDA_CHECK(cudaMemcpyAsync(&temp_h, temp_d.data(), sizeof(int), + cudaMemcpyDeviceToHost, stream)); + CUDA_CHECK(cudaStreamSynchronize(stream)); + communicator.barrier(); + + std::cout << "Clique size: " << communicator.getSize() << std::endl; + std::cout << "final_size: " << temp_h << std::endl; + + return temp_h == communicator.getSize(); +} + +bool test_pointToPoint_simple_send_recv(const handle_t& h, + int numTrials) { + const comms_t& communicator = h.get_comms(); + const int rank = communicator.getRank(); + + bool ret = true; + for (int i = 0; i < numTrials; i++) { + std::vector received_data((communicator.getSize() - 1), -1); + + std::vector requests; + requests.resize(2 * (communicator.getSize() - 1)); + int request_idx = 0; + //post receives + for (int r = 0; r < communicator.getSize(); ++r) { + if (r != rank) { + communicator.irecv(received_data.data() + request_idx, 1, r, 0, + requests.data() + request_idx); + ++request_idx; + } + } + + for (int r = 0; r < communicator.getSize(); ++r) { + if (r != rank) { + communicator.isend(&rank, 1, r, 0, requests.data() + request_idx); + ++request_idx; + } + } + + communicator.waitall(requests.size(), requests.data()); + communicator.barrier(); + + if (communicator.getRank() == 0) { + std::cout << "=========================" << std::endl; + std::cout << "Trial " << i << std::endl; + } + + for (int printrank = 0; printrank < communicator.getSize(); ++printrank) { + if (communicator.getRank() == printrank) { + std::cout << "Rank " << communicator.getRank() << " received: ["; + for (int i = 0; i < received_data.size(); i++) { + auto rec = received_data[i]; + std::cout << rec; + if (rec == -1) ret = false; + communicator.barrier(); + if (i < received_data.size() - 1) std::cout << ", "; + } + std::cout << "]" << std::endl; + } + + communicator.barrier(); + } + + if (communicator.getRank() == 0) + std::cout << "=========================" << std::endl; + } + + return ret; +} + +}; // namespace comms +}; // end namespace raft diff --git a/cpp/include/raft/comms/nccl_helper.hpp b/cpp/include/raft/comms/nccl_helper.hpp new file mode 100644 index 0000000000..5f367bafae --- /dev/null +++ b/cpp/include/raft/comms/nccl_helper.hpp @@ -0,0 +1,18 @@ +#include + + +namespace raft { +namespace comms { +inline void ncclUniqueIdFromChar(ncclUniqueId *id, char *uniqueId, int size) { + memcpy(id->internal, uniqueId, size); +} + +inline void get_unique_id(char *uid, int size) { + ncclUniqueId id; + ncclGetUniqueId(&id); + + memcpy(uid, id.internal, size); +} +} +} + diff --git a/cpp/include/raft/comms/std/std_comms.hpp b/cpp/include/raft/comms/std_comms.hpp similarity index 80% rename from cpp/include/raft/comms/std/std_comms.hpp rename to cpp/include/raft/comms/std_comms.hpp index 7047914fde..7906cf2027 100644 --- a/cpp/include/raft/comms/std/std_comms.hpp +++ b/cpp/include/raft/comms/std_comms.hpp @@ -22,7 +22,7 @@ #include -#include +#include #include #include @@ -30,8 +30,6 @@ #include -constexpr bool UCX_ENABLED = true; - #include #include #include @@ -68,7 +66,7 @@ constexpr bool UCX_ENABLED = true; namespace raft { -namespace { +namespace comms { size_t getDatatypeSize(const comms_t::datatype_t datatype) { switch (datatype) { @@ -91,6 +89,9 @@ size_t getDatatypeSize(const comms_t::datatype_t datatype) { } } + + + ncclDataType_t getNCCLDatatype( const comms_t::datatype_t datatype) { switch (datatype) { @@ -125,43 +126,6 @@ ncclRedOp_t getNCCLOp(const comms_t::op_t op) { return ncclMax; } } -} // namespace - -bool ucx_enabled() { return UCX_ENABLED; } - -void inject_comms(handle_t *handle, ncclComm_t comm, int size, - int rank) { - auto communicator = std::make_shared( - std::unique_ptr( - new std_comms(comm, size, rank))); - handle->setCommunicator(communicator); -} - -void inject_comms(handle_t *handle, ncclComm_t comm, void *ucp_worker, - void *eps, int size, int rank) { - - std::shared_ptr eps_sp = - std::make_shared(new ucp_ep_h[size]); - - size_t *size_t_ep_arr = (size_t *)eps; - - for (int i = 0; i < size; i++) { - size_t ptr = size_t_ep_arr[i]; - ucp_ep_h *ucp_ep_v = (ucp_ep_h *)*eps_sp; - - if (ptr != 0) { - ucp_ep_h eps_ptr = (ucp_ep_h)size_t_ep_arr[i]; - ucp_ep_v[i] = eps_ptr; - } else { - ucp_ep_v[i] = nullptr; - } - } - - auto communicator = std::make_shared( - std::unique_ptr( - new std_comms(comm, ucp_worker, eps, size, rank))); - handle->setCommunicator(communicator); -} class std_comms : public comms_t { @@ -206,67 +170,7 @@ class std_comms : public comms_t { CUDA_CHECK_NO_THROW(cudaFree(_recvbuff)); } - size_t getDatatypeSize(const c::datatype_t datatype) { - switch (datatype) { - case comms_t::CHAR: - return sizeof(char); - case comms_t::UINT8: - return sizeof(uint8_t); - case comms_t::INT: - return sizeof(int); - case comms_t::UINT: - return sizeof(unsigned int); - case comms_t::INT64: - return sizeof(int64_t); - case comms_t::UINT64: - return sizeof(uint64_t); - case comms_t::FLOAT: - return sizeof(float); - case comms_t::DOUBLE: - return sizeof(double); - } - } - - template <> - comms_t::datatype_t getDataType() const { - return comms_t::CHAR; - } - - template <> - comms_t::datatype_t getDataType() const { - return comms_t::UINT8; - } - - template <> - comms_t::datatype_t getDataType() const { - return comms_t::INT; - } - - template <> - comms_t::datatype_t getDataType() const { - return comms_t::UINT; - } - - template <> - comms_t::datatype_t getDataType() const { - return comms_t::INT64; - } - - template <> - comms_t::datatype_t getDataType() const { - return comms_t::UINT64; - } - - template <> - comms_t::datatype_t getDataType() const { - return comms_t::FLOAT; - } - - template <> - comms_t::datatype_t getDataType() const { - return comms_t::DOUBLE; - } void initialize() { CUDA_CHECK(cudaStreamCreate(&_stream)); @@ -315,9 +219,6 @@ class std_comms : public comms_t { void isend(const void *buf, int size, int dest, int tag, request_t *request) const { - ASSERT(UCX_ENABLED, "Comms not built with UCX support"); - ASSERT(p2p_enabled, - "Comms instance was not initialized for point-to-point"); ASSERT(_ucp_worker != nullptr, "ERROR: UCX comms not initialized on communicator."); @@ -335,10 +236,6 @@ class std_comms : public comms_t { void irecv(void *buf, int size, int source, int tag, request_t *request) const { - ASSERT(UCX_ENABLED, "Comms not built with UCX support"); - ASSERT(p2p_enabled, - "Comms instance was not initialized for point-to-point"); - ASSERT(_ucp_worker != nullptr, "ERROR: UCX comms not initialized on communicator."); @@ -357,9 +254,6 @@ class std_comms : public comms_t { void waitall(int count, request_t array_of_requests[]) const { - ASSERT(UCX_ENABLED, "Comms not built with UCX support"); - ASSERT(p2p_enabled, - "Comms instance was not initialized for point-to-point"); ASSERT(_ucp_worker != nullptr, "ERROR: UCX comms not initialized on communicator."); @@ -457,18 +351,25 @@ class std_comms : public comms_t { getNCCLDatatype(datatype), _nccl_comm, stream)); } +// const void* sendbuf, void* recvbuf, +// const int recvcounts[], const int displs[], +// datatype_t datatype, cudaStream_t stream +// + void allgatherv(const void *sendbuf, void *recvbuf, const int recvcounts[], const int displs[], - datatype_t datatype, + comms_t::datatype_t datatype, cudaStream_t stream) const { //From: "An Empirical Evaluation of Allgatherv on Multi-GPU Systems" - https://arxiv.org/pdf/1812.05964.pdf //Listing 1 on page 4. - for (int root = 0; root < _size; ++root) + for (int root = 0; root < _size; ++root) { + size_t dtype_size = getDatatypeSize(datatype); NCCL_CHECK(ncclBroadcast( sendbuf, - static_cast(recvbuf) + displs[root] * getDatatypeSize(datatype), + static_cast(recvbuf) + displs[root] * dtype_size, recvcounts[root], getNCCLDatatype(datatype), root, _nccl_comm, stream)); + } } void reducescatter(const void *sendbuff, @@ -480,7 +381,7 @@ class std_comms : public comms_t { _nccl_comm, stream)); } - status_t std_comms::syncStream( + comms_t::status_t syncStream( cudaStream_t stream) const { cudaError_t cudaErr; ncclResult_t ncclErr, ncclAsyncErr; @@ -531,5 +432,6 @@ class std_comms : public comms_t { _requests_in_flight; mutable std::unordered_set _free_requests; }; +} -} // end namespace ML +} // end namespace raft diff --git a/cpp/include/raft/comms/std/ucp_helper.hpp b/cpp/include/raft/comms/ucp_helper.hpp similarity index 99% rename from cpp/include/raft/comms/std/ucp_helper.hpp rename to cpp/include/raft/comms/ucp_helper.hpp index 2cf7bed6f8..ee22b59101 100644 --- a/cpp/include/raft/comms/std/ucp_helper.hpp +++ b/cpp/include/raft/comms/ucp_helper.hpp @@ -14,7 +14,7 @@ * limitations under the License. */ -#include +#include #include #include #include diff --git a/cpp/include/raft/cudart_utils.h b/cpp/include/raft/cudart_utils.h index 8bd4caf121..93543d09de 100644 --- a/cpp/include/raft/cudart_utils.h +++ b/cpp/include/raft/cudart_utils.h @@ -112,14 +112,14 @@ class exception : public std::exception { // * @brief check for cuda runtime API errors but log error instead of raising // * exception. // */ -// #define CUDA_CHECK_NO_THROW(call) \ -// do { \ -// cudaError_t status = call; \ -// if (status != cudaSuccess) { \ -// RAFT_LOG_ERROR("CUDA call='%s' at file=%s line=%d failed with %s ", \ -// #call, __FILE__, __LINE__, cudaGetErrorString(status)); \ -// } \ -// } while (0) + #define CUDA_CHECK_NO_THROW(call) \ + do { \ + cudaError_t status = call; \ + if (status != cudaSuccess) { \ + printf("CUDA call='%s' at file=%s line=%d failed with %s\n", \ + #call, __FILE__, __LINE__, cudaGetErrorString(status)); \ + } \ + } while (0) /** helper method to get max usable shared mem per block parameter */ inline int get_shared_memory_per_block() { diff --git a/cpp/include/raft/handle.hpp b/cpp/include/raft/handle.hpp index 297760c56b..055fdf9e3f 100644 --- a/cpp/include/raft/handle.hpp +++ b/cpp/include/raft/handle.hpp @@ -149,18 +149,18 @@ class handle_t { } } - void setCommunicator( - std::shared_ptr communicator) { + void set_comms( + std::shared_ptr communicator) { _communicator = communicator; } - const comms_t& getCommunicator() const { + const comms::comms_t& get_comms() const { ASSERT(nullptr != _communicator.get(), "ERROR: Communicator was not initialized\n"); return *_communicator; } - bool commsInitialized() const { + bool comms_initialized() const { return (nullptr != _communicator.get()); } @@ -175,7 +175,7 @@ class handle_t { } private: - std::shared_ptr _communicator; + std::shared_ptr _communicator; const int dev_id_; const int num_streams_; diff --git a/python/raft/common/__init__.py b/python/raft/common/__init__.py new file mode 100644 index 0000000000..e69de29bb2 diff --git a/python/raft/common/cuda.pxd b/python/raft/common/cuda.pxd new file mode 100644 index 0000000000..e407213f44 --- /dev/null +++ b/python/raft/common/cuda.pxd @@ -0,0 +1,36 @@ +# +# Copyright (c) 2019, NVIDIA CORPORATION. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# cython: profile=False +# distutils: language = c++ +# cython: embedsignature = True +# cython: language_level = 3 + + +# Populate this with more typedef's (eg: events) as and when needed +cdef extern from * nogil: + ctypedef void* _Stream "cudaStream_t" + ctypedef int _Error "cudaError_t" + + +# Populate this with more runtime api method declarations as and when needed +cdef extern from "cuda_runtime_api.h" nogil: + _Error cudaStreamCreate(_Stream* s) + _Error cudaStreamDestroy(_Stream s) + _Error cudaStreamSynchronize(_Stream s) + _Error cudaGetLastError() + const char* cudaGetErrorString(_Error e) + const char* cudaGetErrorName(_Error e) diff --git a/python/raft/common/cuda.pyx b/python/raft/common/cuda.pyx new file mode 100644 index 0000000000..09f347058f --- /dev/null +++ b/python/raft/common/cuda.pyx @@ -0,0 +1,88 @@ +# +# Copyright (c) 2019, NVIDIA CORPORATION. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# cython: profile=False +# distutils: language = c++ +# cython: embedsignature = True +# cython: language_level = 3 + +import functools +from libcpp.string cimport string + + +class CudaRuntimeError(RuntimeError): + def __init__(self, extraMsg=None): + cdef _Error e = cudaGetLastError() + cdef bytes errMsg = cudaGetErrorString(e) + cdef bytes errName = cudaGetErrorName(e) + msg = "Error! %s reason='%s'" % (errName.decode(), errMsg.decode()) + if extraMsg is not None: + msg += " extraMsg='%s'" % extraMsg + super(CudaRuntimeError, self).__init__(msg) + + +cdef class Stream: + """ + Stream represents a thin-wrapper around cudaStream_t and its operations. + + Examples + -------- + + .. code-block:: python + + import cuml + stream = cuml.cuda.Stream() + stream.sync() + del stream # optional! + """ + + # NOTE: + # If we store _Stream directly, this always leads to the following error: + # "Cannot convert Python object to '_Stream'" + # I was unable to find a good solution to this in reasonable time. Also, + # since cudaStream_t is a pointer anyways, storing it as an integer should + # be just fine (although, that certainly is ugly and hacky!). + cdef size_t s + + def __cinit__(self): + if self.s != 0: + return + cdef _Stream stream + cdef _Error e = cudaStreamCreate(&stream) + if e != 0: + raise CudaRuntimeError("Stream create") + self.s = stream + + def __dealloc__(self): + self.sync() + cdef _Stream stream = <_Stream>self.s + cdef _Error e = cudaStreamDestroy(stream) + if e != 0: + raise CudaRuntimeError("Stream destroy") + + def sync(self): + """ + Synchronize on the cudastream owned by this object. Note that this + could raise exception due to issues with previous asynchronous + launches + """ + cdef _Stream stream = <_Stream>self.s + cdef _Error e = cudaStreamSynchronize(stream) + if e != 0: + raise CudaRuntimeError("Stream sync") + + def getStream(self): + return self.s diff --git a/python/raft/common/handle.pxd b/python/raft/common/handle.pxd new file mode 100644 index 0000000000..ea40495a2a --- /dev/null +++ b/python/raft/common/handle.pxd @@ -0,0 +1,38 @@ +# +# Copyright (c) 2019, NVIDIA CORPORATION. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# cython: profile=False +# distutils: language = c++ +# cython: embedsignature = True +# cython: language_level = 3 + + +from libcpp.memory cimport shared_ptr +cimport raft.common.cuda + + +cdef extern from "raft/mr/device/allocator.hpp" namespace "raft::mr::device" nogil: + cdef cppclass allocator: + pass + +cdef extern from "raft/handle.hpp" namespace "raft" nogil: + cdef cppclass handle_t: + handle_t() except + + handle_t(int ns) except + + void set_stream(raft.common.cuda._Stream s) except + + void set_device_allocator(shared_ptr[allocator] a) except + + raft.common.cuda._Stream get_stream() except + + int get_num_internal_streams() except + \ No newline at end of file diff --git a/python/raft/common/handle.pyx b/python/raft/common/handle.pyx new file mode 100644 index 0000000000..75c10876ea --- /dev/null +++ b/python/raft/common/handle.pyx @@ -0,0 +1,108 @@ +# +# Copyright (c) 2019, NVIDIA CORPORATION. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# cython: profile=False +# distutils: language = c++ +# cython: embedsignature = True +# cython: language_level = 3 + +import raft +from libcpp.memory cimport shared_ptr +from raft.common.cuda cimport _Stream, _Error, cudaStreamSynchronize + +# +# cdef extern from ".cuml/common/rmmAllocatorAdapterhpp" namespace "ML" nogil: +# cdef cppclass rmmAllocatorAdapter(deviceAllocator): +# pass + +cdef class Handle: + """ + Handle is a lightweight python wrapper around the corresponding C++ class + of cumlHandle exposed by cuML's C++ interface. Refer to the header file + cuml/cuml.hpp for interface level details of this struct + + Examples + -------- + + .. code-block:: python + + import cuml + stream = cuml.cuda.Stream() + handle = cuml.Handle() + handle.setStream(stream) + handle.enableRMM() # Enable RMM as the device-side allocator + + # call ML algos here + + # final sync of all work launched in the stream of this handle + # this is same as `cuml.cuda.Stream.sync()` call, but safer in case + # the default stream inside the `cumlHandle` is being used + handle.sync() + del handle # optional! + """ + + # ML::cumlHandle doesn't have copy operator. So, use pointer for the object + # python world cannot access to this raw object directly, hence use + # 'size_t'! + cdef size_t h + + # not using __dict__ unless we need it to keep this Extension as lean as + # possible + cdef int n_streams + + def __cinit__(self, n_streams=0): + self.n_streams = n_streams + self.h = (new handle_t(n_streams)) + # cdef shared_ptr[deviceAllocator] rmmAlloc = ( + # shared_ptr[deviceAllocator](new rmmAllocatorAdapter())) + # cdef cumlHandle* h_ = self.h + # h_.setDeviceAllocator(rmmAlloc) + + def __dealloc__(self): + h_ = self.h + del h_ + + def setStream(self, stream): + cdef size_t s = stream.get_stream() + cdef handle_t* h_ = self.h + h_.set_stream(<_Stream>s) + + def sync(self): + """ + Issues a sync on the stream set for this handle. + + Once we make `cuml.cuda.Stream` as a mandatory option for creating + `cuml.Handle`, this should go away + """ + cdef handle_t* h_ = self.h + cdef _Stream stream = h_.get_stream() + cdef _Error e = cudaStreamSynchronize(stream) + if e != 0: + raise raft.cuda.CudaRuntimeError("Stream sync") + + def getHandle(self): + return self.h + + def getNumInternalStreams(self): + cdef handle_t* h_ = self.h + return h_.get_num_internal_streams() + + def __getstate__(self): + return self.n_streams + + def __setstate__(self, state): + self.n_streams = state + self.h = (new handle_t(self.n_streams)) diff --git a/python/raft/dask/common/comms.py b/python/raft/dask/common/comms.py new file mode 100644 index 0000000000..7273df3f16 --- /dev/null +++ b/python/raft/dask/common/comms.py @@ -0,0 +1,329 @@ +# Copyright (c) 2020, NVIDIA CORPORATION. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +from cuml.nccl import nccl +from cuml.dask.common.ucx import UCX + +from .comms_utils import inject_comms_on_handle +from .comms_utils import inject_comms_on_handle_coll_only + +from .utils import parse_host_port +from cuml.common.handle import Handle + +from dask.distributed import get_worker, default_client + +import warnings + +import time +import uuid + + +def worker_state(sessionId=None): + """ + Retrieves cuML comms state on local worker for the given + sessionId, creating a new session if it does not exist. + If no session id is given, returns the state dict for all + sessions. + :param sessionId: + :return: + """ + worker = get_worker() + if not hasattr(worker, "_cuml_comm_state"): + worker._cuml_comm_state = {} + if sessionId is not None and sessionId not in worker._cuml_comm_state: + # Build state for new session and mark session creation time + worker._cuml_comm_state[sessionId] = {"ts": time.time()} + + if sessionId is not None: + return worker._cuml_comm_state[sessionId] + return worker._cuml_comm_state + + +def get_ucx(): + """ + A simple convenience wrapper to make sure UCP listener and + endpoints are only ever assigned once per worker. + """ + if "ucx" not in worker_state("ucp"): + worker_state("ucp")["ucx"] = UCX.get() + return worker_state("ucp")["ucx"] + + +def _func_ucp_listener_port(): + return get_ucx().listener_port() + + +async def _func_init_all(sessionId, uniqueId, comms_p2p, + worker_info, verbose, streams_per_handle): + + session_state = worker_state(sessionId) + session_state["nccl_uid"] = uniqueId + session_state["wid"] = worker_info[get_worker().address]["rank"] + session_state["nworkers"] = len(worker_info) + + if verbose: + print("Initializing NCCL") + start = time.time() + + _func_init_nccl(sessionId, uniqueId) + + if verbose: + elapsed = time.time() - start + print("NCCL Initialization took: %f seconds." % elapsed) + + if comms_p2p: + if verbose: + print("Initializing UCX Endpoints") + + if verbose: + start = time.time() + await _func_ucp_create_endpoints(sessionId, worker_info) + + if verbose: + elapsed = time.time() - start + print("Done initializing UCX endpoints. Took: %f seconds." % + elapsed) + print("Building handle") + + _func_build_handle_p2p(sessionId, streams_per_handle, verbose) + + if verbose: + print("Done building handle.") + + else: + _func_build_handle(sessionId, streams_per_handle, verbose) + + +def _func_init_nccl(sessionId, uniqueId): + """ + Initialize ncclComm_t on worker + :param workerId: int ID of the current worker running the function + :param nWorkers: int Number of workers in the cluster + :param uniqueId: array[byte] The NCCL unique Id generated from the + client. + """ + + wid = worker_state(sessionId)["wid"] + nWorkers = worker_state(sessionId)["nworkers"] + + try: + n = nccl() + n.init(nWorkers, uniqueId, wid) + worker_state(sessionId)["nccl"] = n + except Exception: + print("An error occurred initializing NCCL!") + + +def _func_build_handle_p2p(sessionId, streams_per_handle, verbose): + """ + Builds a cumlHandle on the current worker given the initialized comms + :param nccl_comm: ncclComm_t Initialized NCCL comm + :param eps: size_t initialized endpoints + :param nWorkers: int number of workers in cluster + :param workerId: int Rank of current worker + :return: + """ + ucp_worker = get_ucx().get_worker() + session_state = worker_state(sessionId) + + handle = Handle(streams_per_handle) + nccl_comm = session_state["nccl"] + eps = session_state["ucp_eps"] + nWorkers = session_state["nworkers"] + workerId = session_state["wid"] + + inject_comms_on_handle(handle, nccl_comm, ucp_worker, eps, + nWorkers, workerId, verbose) + + worker_state(sessionId)["handle"] = handle + + +def _func_build_handle(sessionId, streams_per_handle, verbose): + """ + Builds a cumlHandle on the current worker given the initialized comms + :param nccl_comm: ncclComm_t Initialized NCCL comm + :param nWorkers: int number of workers in cluster + :param workerId: int Rank of current worker + :return: + """ + handle = Handle(streams_per_handle) + + session_state = worker_state(sessionId) + + workerId = session_state["wid"] + nWorkers = session_state["nworkers"] + + nccl_comm = session_state["nccl"] + inject_comms_on_handle_coll_only(handle, nccl_comm, nWorkers, + workerId, verbose) + session_state["handle"] = handle + + +def _func_store_initial_state(nworkers, sessionId, uniqueId, wid): + session_state = worker_state(sessionId) + session_state["nccl_uid"] = uniqueId + session_state["wid"] = wid + session_state["nworkers"] = nworkers + + +async def _func_ucp_create_endpoints(sessionId, worker_info): + """ + Runs on each worker to create ucp endpoints to all other workers + :param sessionId: uuid unique id for this instance + :param worker_info: dict Maps worker address to rank & UCX port + :param r: float a random number to stop the function from being cached + """ + dask_worker = get_worker() + local_address = dask_worker.address + + eps = [None] * len(worker_info) + count = 1 + + for k in worker_info: + if str(k) != str(local_address): + + ip, port = parse_host_port(k) + + ep = await get_ucx().get_endpoint(ip, worker_info[k]["port"]) + + eps[worker_info[k]["rank"]] = ep + count += 1 + + worker_state(sessionId)["ucp_eps"] = eps + + +async def _func_destroy_all(sessionId, comms_p2p, verbose=False): + worker_state(sessionId)["nccl"].destroy() + del worker_state(sessionId)["nccl"] + del worker_state(sessionId)["handle"] + + +def _func_ucp_ports(client, workers): + return client.run(_func_ucp_listener_port, + workers=workers) + + +def _func_worker_ranks(workers): + """ + Builds a dictionary of { (worker_address, worker_port) : worker_rank } + """ + return dict(list(zip(workers, range(len(workers))))) + + +class CommsContext: + + """ + A base class to initialize and manage underlying NCCL and UCX + comms handles across a Dask cluster. Classes extending CommsContext + are responsible for calling `self.init()` to initialize the comms. + Classes that extend or use the CommsContext are also responsible for + calling `destroy()` to clean up the underlying comms. + + This class is not meant to be thread-safe. + """ + + def __init__(self, comms_p2p=False, client=None, verbose=False, + streams_per_handle=0): + """ + Construct a new CommsContext instance + :param comms_p2p: bool Should p2p comms be initialized? + """ + self.client = client if client is not None else default_client() + self.comms_p2p = comms_p2p + + self.streams_per_handle = streams_per_handle + + self.sessionId = uuid.uuid4().bytes + + self.nccl_initialized = False + self.ucx_initialized = False + + self.verbose = verbose + + if verbose: + print("Initializing comms!") + + def __del__(self): + if self.nccl_initialized or self.ucx_initialized: + self.destroy() + + def worker_info(self, workers): + """ + Builds a dictionary of { (worker_address, worker_port) : + (worker_rank, worker_port ) } + """ + ranks = _func_worker_ranks(workers) + ports = _func_ucp_ports(self.client, workers) \ + if self.comms_p2p else None + + output = {} + for k in ranks.keys(): + output[k] = {"rank": ranks[k]} + if self.comms_p2p: + output[k]["port"] = ports[k] + return output + + def init(self, workers=None): + """ + Initializes the underlying comms. NCCL is required but + UCX is only initialized if `comms_p2p == True` + """ + + self.worker_addresses = list(set((self.client.has_what().keys() + if workers is None else workers))) + + if self.nccl_initialized: + warnings.warn("CommsContext has already been initialized.") + return + + worker_info = self.worker_info(self.worker_addresses) + worker_info = {w: worker_info[w] for w in self.worker_addresses} + + self.uniqueId = nccl.get_unique_id() + + self.client.run(_func_init_all, + self.sessionId, + self.uniqueId, + self.comms_p2p, + worker_info, + self.verbose, + self.streams_per_handle, + workers=self.worker_addresses, + wait=True) + + self.nccl_initialized = True + + if self.comms_p2p: + self.ucx_initialized = True + + if self.verbose: + print("Initialization complete.") + + def destroy(self): + """ + Shuts down initialized comms and cleans up resources. + """ + self.client.run(_func_destroy_all, + self.sessionId, + self.comms_p2p, + self.verbose, + wait=True, + workers=self.worker_addresses) + + if self.verbose: + print("Destroying comms.") + + self.nccl_initialized = False + self.ucx_initialized = False diff --git a/python/raft/dask/common/comms_utils.pyx b/python/raft/dask/common/comms_utils.pyx new file mode 100644 index 0000000000..099e38cf14 --- /dev/null +++ b/python/raft/dask/common/comms_utils.pyx @@ -0,0 +1,155 @@ +# Copyright (c) 2019-2020, NVIDIA CORPORATION. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# +# cython: profile=False +# distutils: language = c++ +# cython: embedsignature = True +# cython: language_level = 3 + +from libc.stdlib cimport malloc, free +from cython.operator cimport dereference as deref + +from cpython.long cimport PyLong_AsVoidPtr + +from libcpp cimport bool + + +from libc.stdint cimport uintptr_t + +cdef extern from "nccl.h": + + cdef struct ncclComm + ctypedef ncclComm *ncclComm_t + + +cdef extern from "raft/handle.hpp" namespace "raft": + cdef cppclass handle_t: + handle_t() except + + +cdef extern from "raft/comms/std_comms.hpp" namespace "raft::comms": + + cdef cppclass std_comms: + pass + + void build_comms_nccl_ucx(handle_t *handle, + ncclComm_t comm, + void *ucp_worker, + void *eps, + int size, + int rank) except + + + void build_comms_nccl_only(handle_t *handle, + ncclComm_t comm, + int size, + int rank) except + + + +cdef extern from "raft/comms/comms_helper.hpp" namespace "raft::comms": + + void build_comms_nccl_ucx(handle_t *handle, + ncclComm_t comm, + void *ucp_worker, + void *eps, + int size, + int rank) except + + + void build_comms_nccl_only(handle_t *handle, + ncclComm_t comm, + int size, + int rank) except + + + + bool test_collective_allreduce(const handle_t &h) except + + bool test_pointToPoint_simple_send_recv(const handle_t &h, + int numTrials) except + + + +def perform_test_comms_allreduce(handle): + """ + Performs an allreduce on the current worker + :param handle: Handle handle containing cumlCommunicator to use + """ + cdef const handle_t* h = handle.getHandle() + return test_collective_allreduce(deref(h)) + + +def perform_test_comms_send_recv(handle, n_trials): + """ + Performs a p2p send/recv on the current worker + :param handle: Handle handle containing cumlCommunicator to use + """ + cdef const handle_t *h = handle.getHandle() + return test_pointToPoint_simple_send_recv(deref(h), n_trials) + + + +def inject_comms_on_handle_coll_only(handle, nccl_inst, size, rank, verbose): + """ + Given a handle and initialized nccl comm, creates a cumlCommunicator + instance and injects it into the handle. + :param handle: Handle cumlHandle to inject comms into + :param nccl_inst: ncclComm_t initialized nccl comm + :param size: int number of workers in cluster + :param rank: int rank of current worker + """ + + cdef size_t handle_size_t = handle.getHandle() + handle_ = handle_size_t + + cdef size_t nccl_comm_size_t = nccl_inst.get_comm() + nccl_comm_ = nccl_comm_size_t + + build_comms_nccl_only(handle_, + deref(nccl_comm_), + size, + rank) + + +def inject_comms_on_handle(handle, nccl_inst, ucp_worker, eps, size, + rank, verbose): + """ + Given a handle and initialized comms, creates a cumlCommunicator instance + and injects it into the handle. + :param handle: Handle cumlHandle to inject comms into + :param nccl_inst: ncclComm_t initialized nccl comm + :param ucp_worker: size_t initialized ucp_worker_h instance + :param eps: size_t array of initialized ucp_ep_h instances + :param size: int number of workers in cluster + :param rank: int rank of current worker + """ + cdef size_t *ucp_eps = malloc(len(eps)*sizeof(size_t)) + + for i in range(len(eps)): + if eps[i] is not None: + ep_st = eps[i].get_ucp_endpoint() + ucp_eps[i] = ep_st + else: + ucp_eps[i] = 0 + + cdef void* ucp_worker_st = ucp_worker + + cdef size_t handle_size_t = handle.getHandle() + handle_ = handle_size_t + + cdef size_t nccl_comm_size_t = nccl_inst.get_comm() + nccl_comm_ = nccl_comm_size_t + + build_comms_nccl_ucx(handle_, + deref(nccl_comm_), + ucp_worker_st, + ucp_eps, + size, + rank) + + free(ucp_eps) diff --git a/python/raft/dask/common/nccl.pyx b/python/raft/dask/common/nccl.pyx new file mode 100644 index 0000000000..e6e2f8fcb1 --- /dev/null +++ b/python/raft/dask/common/nccl.pyx @@ -0,0 +1,233 @@ +# +# Copyright (c) 2019, NVIDIA CORPORATION. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# cython: profile=False +# distutils: language = c++ +# cython: embedsignature = True +# cython: language_level = 3 + +from libc.stdint cimport uintptr_t +from cython.operator cimport dereference as deref + +from libcpp cimport bool +from libc.stdlib cimport malloc, free + +cdef extern from "raft/include/comms/nccl_helper.hpp" namespace "raft::coms": + void get_unique_id(char *uid, int size) except + + void ncclUniqueIdFromChar(ncclUniqueId *id, + char *uniqueId, + int size) except + + +cdef extern from "nccl.h": + + cdef struct ncclComm + + ctypedef struct ncclUniqueId: + char *internal[128] + + ctypedef ncclComm *ncclComm_t + + ctypedef enum ncclResult_t: + ncclSuccess + ncclUnhandledCudaError + ncclSystemError + ncclInternalError + ncclInvalidArgument + ncclInvalidUsage + ncclNumResults + + ncclResult_t ncclCommInitRank(ncclComm_t *comm, + int nranks, + ncclUniqueId commId, + int rank) nogil + + ncclResult_t ncclGetUniqueId(ncclUniqueId *uniqueId) nogil + + ncclResult_t ncclCommUserRank(const ncclComm_t comm, int *rank) nogil + + ncclResult_t ncclCommCuDevice(const ncclComm_t comm, int *count) nogil + + const char *ncclGetErrorString(ncclResult_t result) nogil + + ncclResult_t ncclCommAbort(ncclComm_t comm) nogil + + ncclResult_t ncclCommDestroy(ncclComm_t comm) nogil + +NCCL_UNIQUE_ID_BYTES = 128 + + +def unique_id(): + """ + Returns a new ncclUniqueId converted to a + character array that can be safely serialized + and shared to a remote worker. + :return: string a 128-byte unique id string + """ + cdef char *uid = malloc(NCCL_UNIQUE_ID_BYTES * sizeof(char)) + get_unique_id(uid, NCCL_UNIQUE_ID_BYTES) + c_str = uid[:NCCL_UNIQUE_ID_BYTES-1] + free(uid) + return c_str + + +cdef class nccl: + """ + A NCCL wrapper for initializing and closing NCCL comms + in Python. + """ + cdef ncclComm_t *comm + + cdef int size + cdef int rank + + def __cinit__(self): + self.comm = malloc(sizeof(ncclComm_t)) + + def __dealloc__(self): + + comm_ = self.comm + + if comm_ != NULL: + free(self.comm) + self.comm = NULL + + @staticmethod + def get_unique_id(): + """ + Returns a new nccl unique id + :return: string nccl unique id + """ + return unique_id() + + def init(self, nranks, commId, rank): + """ + Construct a nccl-py object + :param nranks: int size of clique + :param commId: string unique id from client + :param rank: int rank of current worker + """ + self.size = nranks + self.rank = rank + + cdef ncclUniqueId *ident = malloc(sizeof(ncclUniqueId)) + ncclUniqueIdFromChar(ident, commId, NCCL_UNIQUE_ID_BYTES) + + comm_ = self.comm + + cdef int nr = nranks + cdef int r = rank + cdef ncclResult_t result + + import time + + start = time.time() + with nogil: + result = ncclCommInitRank(comm_, nr, + deref(ident), r) + + end = time.time() + if result != ncclSuccess: + with nogil: + err_str = ncclGetErrorString(result) + print("NCCL_ERROR: %s" % err_str) + + def destroy(self): + """ + Call destroy on the underlying NCCL comm + """ + comm_ = self.comm + + cdef ncclResult_t result + if comm_ != NULL: + with nogil: + result = ncclCommDestroy(deref(comm_)) + + if result != ncclSuccess: + with nogil: + err_str = ncclGetErrorString(result) + print("NCCL_ERROR: %s" % err_str) + + free(self.comm) + self.comm = NULL + + def abort(self): + """ + Call abort on the underlying nccl comm + """ + comm_ = self.comm + cdef ncclResult_t result + if comm_ != NULL: + with nogil: + result = ncclCommAbort(deref(comm_)) + + if result != ncclSuccess: + with nogil: + err_str = ncclGetErrorString(result) + print("NCCL_ERROR: %s" % err_str) + free(comm_) + self.comm = NULL + + def cu_device(self): + """ + Get the device backing the underlying comm + :returns int device id + """ + cdef int *dev = malloc(sizeof(int)) + + comm_ = self.comm + cdef ncclResult_t result + with nogil: + result = ncclCommCuDevice(deref(comm_), dev) + + if result != ncclSuccess: + with nogil: + err_str = ncclGetErrorString(result) + print("NCCL_ERROR: %s" % err_str) + + ret = dev[0] + free(dev) + return ret + + def user_rank(self): + """ + Get the rank id of the current comm + :return: int rank + """ + + cdef int *rank = malloc(sizeof(int)) + + comm_ = self.comm + + cdef ncclResult_t result + with nogil: + result = ncclCommUserRank(deref(comm_), rank) + + if result != ncclSuccess: + with nogil: + err_str = ncclGetErrorString(result) + print("NCCL_ERROR: %s" % err_str) + + ret = rank[0] + free(rank) + return ret + + def get_comm(self): + """ + Returns the underlying nccl comm in a size_t (similar to void*). + This can be safely typecasted from size_t into ncclComm_t* + :return: size_t ncclComm_t instance + """ + return self.comm diff --git a/python/raft/dask/common/ucx.py b/python/raft/dask/common/ucx.py new file mode 100644 index 0000000000..948e1433ee --- /dev/null +++ b/python/raft/dask/common/ucx.py @@ -0,0 +1,77 @@ +# Copyright (c) 2020, NVIDIA CORPORATION. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +import ucp + + +async def _connection_func(ep): + return 0 + + +class UCX: + """ + Singleton UCX context to encapsulate all interactions with the + UCX-py API and guarantee only a single listener & endpoints are + created by cuML on a single process. + """ + + __instance = None + + def __init__(self, listener_callback): + + self.listener_callback = listener_callback + + self._create_listener() + self._endpoints = {} + + assert UCX.__instance is None + + UCX.__instance = self + + @staticmethod + def get(listener_callback=_connection_func): + if UCX.__instance is None: + UCX(listener_callback) + return UCX.__instance + + def get_worker(self): + return ucp.get_ucp_worker() + + def _create_listener(self): + self._listener = ucp.create_listener(self.listener_callback) + + def listener_port(self): + return self._listener.port + + async def _create_endpoint(self, ip, port): + ep = await ucp.create_endpoint(ip, port) + self._endpoints[(ip, port)] = ep + return ep + + async def get_endpoint(self, ip, port): + if (ip, port) not in self._endpoints: + ep = await self._create_endpoint(ip, port) + else: + ep = self._endpoints[(ip, port)] + + return ep + + def __del__(self): + for ip_port, ep in self._endpoints.items(): + if not ep.closed(): + ep.abort() + del ep + + self._listener.close() diff --git a/python/setup.py b/python/setup.py index 4f47f41e0e..ccef769f42 100644 --- a/python/setup.py +++ b/python/setup.py @@ -96,6 +96,7 @@ include_dirs = [cuda_include_dir, numpy.get_include(), + "../cpp/include/", os.path.dirname(sysconfig.get_path("include"))] cmdclass = dict() From 5d37e9ef7b3b1f035e4de7e1dceea90b5d27a1b9 Mon Sep 17 00:00:00 2001 From: "Corey J. Nolet" Date: Wed, 13 May 2020 13:23:10 -0400 Subject: [PATCH 08/46] Cython is building! --- cpp/include/raft/comms/comms.hpp | 111 +++++++++++++++++++++--- cpp/include/raft/comms/comms_helper.hpp | 7 +- cpp/include/raft/comms/std_comms.hpp | 6 +- python/raft/dask/common/nccl.pyx | 2 +- 4 files changed, 104 insertions(+), 22 deletions(-) diff --git a/cpp/include/raft/comms/comms.hpp b/cpp/include/raft/comms/comms.hpp index a5152be620..e158af14e3 100644 --- a/cpp/include/raft/comms/comms.hpp +++ b/cpp/include/raft/comms/comms.hpp @@ -22,28 +22,32 @@ namespace raft { namespace comms { -class comms_t { + +class comms_iface { public: - typedef unsigned int request_t; - enum datatype_t { CHAR, UINT8, INT, UINT, INT64, UINT64, FLOAT, DOUBLE }; - enum op_t { SUM, PROD, MIN, MAX }; - /** - * The resulting status of distributed stream synchronization - */ - enum status_t { - commStatusSuccess, // Synchronization successful - commStatusError, // An error occured querying sync status - commStatusAbort - }; // A failure occurred in sync, queued operations aborted + typedef unsigned int request_t; + enum datatype_t { CHAR, UINT8, INT, UINT, INT64, UINT64, FLOAT, DOUBLE }; + enum op_t { SUM, PROD, MIN, MAX }; + + /** + * The resulting status of distributed stream synchronization + */ + enum status_t { + commStatusSuccess, // Synchronization successful + commStatusError, // An error occured querying sync status + commStatusAbort + }; // A failure occurred in sync, queued operations aborted + + - virtual ~comms_t(); + virtual ~comms_iface(); virtual int getSize() const = 0; virtual int getRank() const = 0; - virtual std::unique_ptr commSplit(int color, int key) const = 0; + virtual std::unique_ptr commSplit(int color, int key) const = 0; virtual void barrier() const = 0; @@ -80,6 +84,85 @@ class comms_t { cudaStream_t stream) const = 0; }; +class comms_t: public comms_iface { + public: + + + comms_t(std::unique_ptr impl) + : _impl(impl.release()) { + ASSERT(nullptr != _impl.get(), "ERROR: Invalid comms_iface used!"); + } + + int getSize() const { return _impl->getSize(); } + + int getRank() const { return _impl->getRank(); } + + std::unique_ptr commSplit(int color, int key) const { + return _impl->commSplit(color, key); + } + + void barrier() const { _impl->barrier(); } + + status_t syncStream( + cudaStream_t stream) const { + return _impl->syncStream(stream); + } + + void isend(const void* buf, int size, int dest, int tag, + request_t* request) const { + _impl->isend(buf, size, dest, tag, request); + } + + void irecv(void* buf, int size, int source, int tag, + request_t* request) const { + _impl->irecv(buf, size, source, tag, request); + } + + void waitall(int count, request_t array_of_requests[]) const { + _impl->waitall(count, array_of_requests); + } + + void allreduce(const void* sendbuff, void* recvbuff, + int count, datatype_t datatype, op_t op, + cudaStream_t stream) const { + _impl->allreduce(sendbuff, recvbuff, count, datatype, op, stream); + } + + void bcast(void* buff, int count, datatype_t datatype, + int root, cudaStream_t stream) const { + _impl->bcast(buff, count, datatype, root, stream); + } + + void reduce(const void* sendbuff, void* recvbuff, int count, + datatype_t datatype, op_t op, int root, + cudaStream_t stream) const { + _impl->reduce(sendbuff, recvbuff, count, datatype, op, root, stream); + } + + void allgather(const void* sendbuff, void* recvbuff, + int sendcount, datatype_t datatype, + cudaStream_t stream) const { + _impl->allgather(sendbuff, recvbuff, sendcount, datatype, stream); + } + + void allgatherv(const void* sendbuf, void* recvbuf, + const int recvcounts[], const int displs[], + datatype_t datatype, + cudaStream_t stream) const { + _impl->allgatherv(sendbuf, recvbuf, recvcounts, displs, datatype, stream); + } + + void reducescatter(const void* sendbuff, void* recvbuff, + int recvcount, datatype_t datatype, + op_t op, cudaStream_t stream) const { + _impl->reducescatter(sendbuff, recvbuff, recvcount, datatype, op, stream); + } + + private: + std::unique_ptr _impl; + +}; + template comms_t::datatype_t getDataType(T a); diff --git a/cpp/include/raft/comms/comms_helper.hpp b/cpp/include/raft/comms/comms_helper.hpp index f1a5482334..fed0710fbe 100644 --- a/cpp/include/raft/comms/comms_helper.hpp +++ b/cpp/include/raft/comms/comms_helper.hpp @@ -27,7 +27,7 @@ void build_comms_nccl_only(handle_t *handle, ncclComm_t comm, int size, auto *raft_comm = new raft::comms::std_comms(comm, size, rank); auto communicator = std::make_shared( - std::unique_ptr(raft_comm)); + std::unique_ptr(raft_comm)); handle->set_comms(communicator); } @@ -51,9 +51,8 @@ void build_comms_nccl_ucx(handle_t *handle, ncclComm_t comm, void *ucp_worker, } } - auto communicator = std::make_shared( - std::unique_ptr( - new raft::comms::std_comms(comm, (ucp_worker_h)ucp_worker, eps_sp, size, rank))); + auto *raft_comm = new raft::comms::std_comms(comm, (ucp_worker_h)ucp_worker, eps_sp, size, rank); + auto communicator = std::make_shared(std::unique_ptr(raft_comm)); handle->set_comms(communicator); } diff --git a/cpp/include/raft/comms/std_comms.hpp b/cpp/include/raft/comms/std_comms.hpp index 7906cf2027..6890e8d9f0 100644 --- a/cpp/include/raft/comms/std_comms.hpp +++ b/cpp/include/raft/comms/std_comms.hpp @@ -128,7 +128,7 @@ ncclRedOp_t getNCCLOp(const comms_t::op_t op) { } -class std_comms : public comms_t { +class std_comms : public comms_iface { public: std_comms() = delete; @@ -184,7 +184,7 @@ class std_comms : public comms_t { int getRank() const { return _rank; } - std::unique_ptr + std::unique_ptr commSplit(int color, int key) const { // Not supported by NCCL ASSERT(false, @@ -199,7 +199,7 @@ class std_comms : public comms_t { allreduce(_sendbuff, _recvbuff, 1, comms_t::INT, comms_t::SUM, _stream); - ASSERT(syncStream(_stream) == status_t::commStatusSuccess, + ASSERT(syncStream(_stream) == comms_t::status_t::commStatusSuccess, "ERROR: syncStream failed. This can be caused by a failed rank."); } diff --git a/python/raft/dask/common/nccl.pyx b/python/raft/dask/common/nccl.pyx index e6e2f8fcb1..c9d9fe0426 100644 --- a/python/raft/dask/common/nccl.pyx +++ b/python/raft/dask/common/nccl.pyx @@ -25,7 +25,7 @@ from cython.operator cimport dereference as deref from libcpp cimport bool from libc.stdlib cimport malloc, free -cdef extern from "raft/include/comms/nccl_helper.hpp" namespace "raft::coms": +cdef extern from "raft/comms/nccl_helper.hpp" namespace "raft::comms": void get_unique_id(char *uid, int size) except + void ncclUniqueIdFromChar(ncclUniqueId *id, char *uniqueId, From bd1b87632295987aafcbc6b48b5b68d8a7b64a58 Mon Sep 17 00:00:00 2001 From: "Corey J. Nolet" Date: Wed, 13 May 2020 14:17:43 -0400 Subject: [PATCH 09/46] The comms tests pass!!! --- cpp/include/raft/comms/comms.hpp | 117 +++++------ cpp/include/raft/comms/comms_helper.hpp | 6 +- cpp/include/raft/comms/std_comms.hpp | 56 +++--- python/raft/dask/common/__init__.py | 8 + python/raft/dask/common/comms.py | 6 +- python/raft/dask/common/comms_utils.pyx | 19 +- python/raft/dask/common/utils.py | 250 ++++++++++++++++++++++++ python/raft/test/__init__.py | 0 python/raft/test/conftest.py | 50 +++++ python/raft/test/test_comms.py | 104 ++++++++++ python/setup.py | 4 +- 11 files changed, 516 insertions(+), 104 deletions(-) create mode 100644 python/raft/dask/common/utils.py create mode 100644 python/raft/test/__init__.py create mode 100644 python/raft/test/conftest.py create mode 100644 python/raft/test/test_comms.py diff --git a/cpp/include/raft/comms/comms.hpp b/cpp/include/raft/comms/comms.hpp index e158af14e3..f0cad3a380 100644 --- a/cpp/include/raft/comms/comms.hpp +++ b/cpp/include/raft/comms/comms.hpp @@ -22,25 +22,23 @@ namespace raft { namespace comms { +typedef unsigned int request_t; +enum datatype_t { CHAR, UINT8, INT, UINT, INT64, UINT64, FLOAT, DOUBLE }; +enum op_t { SUM, PROD, MIN, MAX }; -class comms_iface { - public: - - - typedef unsigned int request_t; - enum datatype_t { CHAR, UINT8, INT, UINT, INT64, UINT64, FLOAT, DOUBLE }; - enum op_t { SUM, PROD, MIN, MAX }; +/** + * The resulting status of distributed stream synchronization + */ +enum status_t { + commStatusSuccess, // Synchronization successful + commStatusError, // An error occured querying sync status + commStatusAbort +}; // A failure occurred in sync, queued operations aborted - /** - * The resulting status of distributed stream synchronization - */ - enum status_t { - commStatusSuccess, // Synchronization successful - commStatusError, // An error occured querying sync status - commStatusAbort - }; // A failure occurred in sync, queued operations aborted +class comms_iface { + public: virtual ~comms_iface(); @@ -88,6 +86,7 @@ class comms_t: public comms_iface { public: + comms_t(std::unique_ptr impl) : _impl(impl.release()) { ASSERT(nullptr != _impl.get(), "ERROR: Invalid comms_iface used!"); @@ -163,48 +162,50 @@ class comms_t: public comms_iface { }; - -template -comms_t::datatype_t getDataType(T a); - -template <> -comms_t::datatype_t getDataType(char a) { - return comms_t::CHAR; -} - -template <> -comms_t::datatype_t getDataType(uint8_t a) { - return comms_t::UINT8; -} - -template <> -comms_t::datatype_t getDataType(int a) { - return comms_t::INT; -} - -template <> -comms_t::datatype_t getDataType(uint32_t a) { - return comms_t::UINT; -} - -template <> -comms_t::datatype_t getDataType(int64_t a) { - return comms_t::INT64; -} - -template <> -comms_t::datatype_t getDataType(uint64_t a) { - return comms_t::UINT64; -} - -template <> -comms_t::datatype_t getDataType(float a) { - return comms_t::FLOAT; -} - -template <> -comms_t::datatype_t getDataType(double a) { - return comms_t::DOUBLE; -} +comms_iface::~comms_iface() {} + + +//template +//inline datatype_t getDataType(T a); +// +//template <> +//inline datatype_t getDataType(char a) { +// return CHAR; +//} +// +//template <> +//inline datatype_t getDataType(uint8_t a) { +// return UINT8; +//} +// +//template <> +//inline datatype_t getDataType(int a) { +// return INT; +//} +// +//template <> +//inline datatype_t getDataType(uint32_t a) { +// return UINT; +//} +// +//template <> +//inline datatype_t getDataType(int64_t a) { +// return INT64; +//} +// +//template <> +//inline datatype_t getDataType(uint64_t a) { +// return UINT64; +//} +// +//template <> +//inline datatype_t getDataType(float a) { +// return FLOAT; +//} +// +//template <> +//inline datatype_t getDataType(double a) { +// return DOUBLE; +//} } /// namespace comms } // namespace raft diff --git a/cpp/include/raft/comms/comms_helper.hpp b/cpp/include/raft/comms/comms_helper.hpp index fed0710fbe..7a033d4205 100644 --- a/cpp/include/raft/comms/comms_helper.hpp +++ b/cpp/include/raft/comms/comms_helper.hpp @@ -68,8 +68,8 @@ bool test_collective_allreduce(const handle_t& handle) { temp_d.resize(1, stream); CUDA_CHECK(cudaMemcpyAsync(temp_d.data(), &send, sizeof(int), cudaMemcpyHostToDevice, stream)); - communicator.allreduce(temp_d.data(), temp_d.data(), 1, getDataType(temp_d.data()), - comms_t::SUM, stream); + communicator.allreduce(temp_d.data(), temp_d.data(), 1, datatype_t::INT, + SUM, stream); int temp_h = 0; CUDA_CHECK(cudaMemcpyAsync(&temp_h, temp_d.data(), sizeof(int), cudaMemcpyDeviceToHost, stream)); @@ -91,7 +91,7 @@ bool test_pointToPoint_simple_send_recv(const handle_t& h, for (int i = 0; i < numTrials; i++) { std::vector received_data((communicator.getSize() - 1), -1); - std::vector requests; + std::vector requests; requests.resize(2 * (communicator.getSize() - 1)); int request_idx = 0; //post receives diff --git a/cpp/include/raft/comms/std_comms.hpp b/cpp/include/raft/comms/std_comms.hpp index 6890e8d9f0..b185934cc6 100644 --- a/cpp/include/raft/comms/std_comms.hpp +++ b/cpp/include/raft/comms/std_comms.hpp @@ -68,23 +68,23 @@ namespace raft { namespace comms { -size_t getDatatypeSize(const comms_t::datatype_t datatype) { +size_t getDatatypeSize(const datatype_t datatype) { switch (datatype) { - case comms_t::CHAR: + case CHAR: return sizeof(char); - case comms_t::UINT8: + case UINT8: return sizeof(uint8_t); - case comms_t::INT: + case INT: return sizeof(int); - case comms_t::UINT: + case UINT: return sizeof(unsigned int); - case comms_t::INT64: + case INT64: return sizeof(int64_t); - case comms_t::UINT64: + case UINT64: return sizeof(uint64_t); - case comms_t::FLOAT: + case FLOAT: return sizeof(float); - case comms_t::DOUBLE: + case DOUBLE: return sizeof(double); } } @@ -93,36 +93,36 @@ size_t getDatatypeSize(const comms_t::datatype_t datatype) { ncclDataType_t getNCCLDatatype( - const comms_t::datatype_t datatype) { + const datatype_t datatype) { switch (datatype) { - case comms_t::CHAR: + case CHAR: return ncclChar; - case comms_t::UINT8: + case UINT8: return ncclUint8; - case comms_t::INT: + case INT: return ncclInt; - case comms_t::UINT: + case UINT: return ncclUint32; - case comms_t::INT64: + case INT64: return ncclInt64; - case comms_t::UINT64: + case UINT64: return ncclUint64; - case comms_t::FLOAT: + case FLOAT: return ncclFloat; - case comms_t::DOUBLE: + case DOUBLE: return ncclDouble; } } -ncclRedOp_t getNCCLOp(const comms_t::op_t op) { +ncclRedOp_t getNCCLOp(const op_t op) { switch (op) { - case comms_t::SUM: + case SUM: return ncclSum; - case comms_t::PROD: + case PROD: return ncclProd; - case comms_t::MIN: + case MIN: return ncclMin; - case comms_t::MAX: + case MAX: return ncclMax; } } @@ -196,10 +196,10 @@ class std_comms : public comms_iface { CUDA_CHECK(cudaMemsetAsync(_sendbuff, 1, sizeof(int), _stream)); CUDA_CHECK(cudaMemsetAsync(_recvbuff, 1, sizeof(int), _stream)); - allreduce(_sendbuff, _recvbuff, 1, comms_t::INT, - comms_t::SUM, _stream); + allreduce(_sendbuff, _recvbuff, 1, INT, + SUM, _stream); - ASSERT(syncStream(_stream) == comms_t::status_t::commStatusSuccess, + ASSERT(syncStream(_stream) == status_t::commStatusSuccess, "ERROR: syncStream failed. This can be caused by a failed rank."); } @@ -359,7 +359,7 @@ class std_comms : public comms_iface { void allgatherv(const void *sendbuf, void *recvbuf, const int recvcounts[], const int displs[], - comms_t::datatype_t datatype, + datatype_t datatype, cudaStream_t stream) const { //From: "An Empirical Evaluation of Allgatherv on Multi-GPU Systems" - https://arxiv.org/pdf/1812.05964.pdf //Listing 1 on page 4. @@ -381,7 +381,7 @@ class std_comms : public comms_iface { _nccl_comm, stream)); } - comms_t::status_t syncStream( + status_t syncStream( cudaStream_t stream) const { cudaError_t cudaErr; ncclResult_t ncclErr, ncclAsyncErr; diff --git a/python/raft/dask/common/__init__.py b/python/raft/dask/common/__init__.py index e69de29bb2..b8d7b28313 100644 --- a/python/raft/dask/common/__init__.py +++ b/python/raft/dask/common/__init__.py @@ -0,0 +1,8 @@ + + +from raft.dask.common.comms import CommsContext, worker_state + +from raft.dask.common.comms_utils import inject_comms_on_handle, \ + perform_test_comms_allreduce, perform_test_comms_send_recv, \ + inject_comms_on_handle_coll_only + diff --git a/python/raft/dask/common/comms.py b/python/raft/dask/common/comms.py index 7273df3f16..54e15c54c2 100644 --- a/python/raft/dask/common/comms.py +++ b/python/raft/dask/common/comms.py @@ -13,14 +13,14 @@ # limitations under the License. # -from cuml.nccl import nccl -from cuml.dask.common.ucx import UCX +from raft.dask.common.nccl import nccl +from raft.dask.common.ucx import UCX from .comms_utils import inject_comms_on_handle from .comms_utils import inject_comms_on_handle_coll_only from .utils import parse_host_port -from cuml.common.handle import Handle +from raft.common.handle import Handle from dask.distributed import get_worker, default_client diff --git a/python/raft/dask/common/comms_utils.pyx b/python/raft/dask/common/comms_utils.pyx index 099e38cf14..672c53aa28 100644 --- a/python/raft/dask/common/comms_utils.pyx +++ b/python/raft/dask/common/comms_utils.pyx @@ -42,17 +42,16 @@ cdef extern from "raft/comms/std_comms.hpp" namespace "raft::comms": cdef cppclass std_comms: pass - void build_comms_nccl_ucx(handle_t *handle, - ncclComm_t comm, - void *ucp_worker, - void *eps, - int size, - int rank) except + - void build_comms_nccl_only(handle_t *handle, - ncclComm_t comm, - int size, - int rank) except + +cdef extern from "raft/comms/comms.hpp" namespace "raft::comms": + + cdef cppclass comms_t: + pass + + cdef cppclass comms_iface: + pass + + cdef extern from "raft/comms/comms_helper.hpp" namespace "raft::comms": diff --git a/python/raft/dask/common/utils.py b/python/raft/dask/common/utils.py new file mode 100644 index 0000000000..10b048d28d --- /dev/null +++ b/python/raft/dask/common/utils.py @@ -0,0 +1,250 @@ +# Copyright (c) 2019, NVIDIA CORPORATION. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +import logging +import os +import numba.cuda +import random +import time + +from dask.distributed import default_client, wait + +from asyncio import InvalidStateError + +from threading import Lock + + +def get_visible_devices(): + """ + Return a list of the CUDA_VISIBLE_DEVICES + :return: list[int] visible devices + """ + # TODO: Shouldn't have to split on every call + return os.environ["CUDA_VISIBLE_DEVICES"].split(",") + + +def device_of_devicendarray(devicendarray): + """ + Returns the device that backs memory allocated on the given + deviceNDArray + :param devicendarray: devicendarray array to check + :return: int device id + """ + dev = device_of_gpu_matrix(devicendarray) + return get_visible_devices()[dev] + + +def get_device_id(canonical_name): + """ + Given a local device id, find the actual "global" id + :param canonical_name: the local device name in CUDA_VISIBLE_DEVICES + :return: the global device id for the system + """ + dev_order = get_visible_devices() + idx = 0 + for dev in dev_order: + if dev == canonical_name: + return idx + idx += 1 + + return -1 + + +def select_device(dev, close=True): + """ + Use numbas numba to select the given device, optionally + closing and opening up a new cuda context if it fails. + :param dev: int device to select + :param close: bool close the cuda context and create new one? + """ + if numba.cuda.get_current_device().id != dev: + logging.warn("Selecting device " + str(dev)) + if close: + numba.cuda.close() + numba.cuda.select_device(dev) + if dev != numba.cuda.get_current_device().id: + logging.warn("Current device " + + str(numba.cuda.get_current_device()) + + " does not match expected " + str(dev)) + + +def get_client(client=None): + return default_client() if client is None else client + + +def parse_host_port(address): + """ + Given a string address with host/port, build a tuple(host, port) + :param address: string address to parse + :return: tuple(host, port) + """ + if '://' in address: + address = address.rsplit('://', 1)[1] + host, port = address.split(':') + port = int(port) + return host, port + + +def build_host_dict(workers): + """ + Builds a dict to map the set of ports running on each host to + the hostname. + :param workers: list(tuple(host, port)) list of worker addresses + :return: dict(host, set(port)) + """ + hosts = set(map(lambda x: parse_host_port(x), workers)) + hosts_dict = {} + for host, port in hosts: + if host not in hosts_dict: + hosts_dict[host] = set([port]) + else: + hosts_dict[host].add(port) + + return hosts_dict + + +def persist_across_workers(client, objects, workers=None): + """ + Calls persist on the 'objects' ensuring they are spread + across the workers on 'workers'. + + Parameters + ---------- + client : dask.distributed.Client + objects : list + Dask distributed objects to be persisted + workers : list or None + List of workers across which to persist objects + If None, then all workers attached to 'client' will be used + """ + if workers is None: + workers = client.has_what().keys() # Default to all workers + return client.persist(objects, workers={o: workers for o in objects}) + + +def raise_exception_from_futures(futures): + """Raises a RuntimeError if any of the futures indicates an exception""" + errs = [f.exception() for f in futures if f.exception()] + if errs: + raise RuntimeError("%d of %d worker jobs failed: %s" % ( + len(errs), len(futures), ", ".join(map(str, errs)) + )) + + +def wait_and_raise_from_futures(futures): + """ + Returns the collected futures after all the futures + have finished and do not indicate any exceptions. + """ + wait(futures) + raise_exception_from_futures(futures) + return futures + + +def raise_mg_import_exception(): + raise Exception("cuML has not been built with multiGPU support " + "enabled. Build with the --multigpu flag to" + " enable multiGPU support.") + + +class MultiHolderLock: + """ + A per-process synchronization lock allowing multiple concurrent holders + at any one time. This is used in situations where resources might be + limited and it's important that the number of concurrent users of + the resources are constained. + + This lock is serializable, but relies on a Python threading.Lock + underneath to properly synchronize internal state across threads. + Note that this lock is only intended to be used per-process and + the underlying threading.Lock will not be serialized. + """ + + def __init__(self, n): + """ + Initialize the lock + :param n : integer the maximum number of concurrent holders + """ + self.n = n + self.current_tasks = 0 + self.lock = Lock() + + def _acquire(self, blocking=True, timeout=10): + lock_acquired = False + + inner_lock_acquired = self.lock.acquire(blocking, timeout) + + if inner_lock_acquired and self.current_tasks < self.n - 1: + self.current_tasks += 1 + lock_acquired = True + self.lock.release() + + return lock_acquired + + def acquire(self, blocking=True, timeout=10): + """ + Acquire the lock. + :param blocking : bool will block if True + :param timeout : a timeout (in seconds) to wait for the lock + before failing. + :return : True if lock was acquired successfully, False otherwise + """ + + t = time.time() + + lock_acquired = self._acquire(blocking, timeout) + + while blocking and not lock_acquired: + + if time.time() - t > timeout: + raise TimeoutError() + + lock_acquired = self.acquire(blocking, timeout) + time.sleep(random.uniform(0, 0.01)) + + return lock_acquired + + def __getstate__(self): + d = self.__dict__.copy() + if "lock" in d: + del d["lock"] + return d + + def __setstate__(self, d): + d["lock"] = Lock() + self.__dict__ = d + + def release(self, blocking=True, timeout=10): + """ + Release a hold on the lock to allow another holder. Note that + while Python's threading.Lock does not have options for blocking + or timeout in release(), this lock uses a threading.Lock + internally and so will need to acquire that lock in order + to properly synchronize the underlying state. + :param blocking : bool will bock if True + :param timeout : a timeout (in seconds) to wait for the lock + before failing. + :return : True if lock was released successfully, False otherwise. + """ + + if self.current_tasks == 0: + raise InvalidStateError("Cannot release lock when no " + "concurrent tasks are executing") + + lock_acquired = self.lock.acquire(blocking, timeout) + if lock_acquired: + self.current_tasks -= 1 + self.lock.release() + return lock_acquired diff --git a/python/raft/test/__init__.py b/python/raft/test/__init__.py new file mode 100644 index 0000000000..e69de29bb2 diff --git a/python/raft/test/conftest.py b/python/raft/test/conftest.py new file mode 100644 index 0000000000..83ed6b5d83 --- /dev/null +++ b/python/raft/test/conftest.py @@ -0,0 +1,50 @@ +import pytest + +from dask.distributed import Client + +from dask_cuda import initialize +from dask_cuda import LocalCUDACluster + +enable_tcp_over_ucx = True +enable_nvlink = False +enable_infiniband = False + + +@pytest.fixture(scope="module") +def cluster(): + + print("Starting cluster") + cluster = LocalCUDACluster(protocol="tcp", scheduler_port=0) + yield cluster + print("Closing cluster") + cluster.close() + print("Closed cluster") + + +@pytest.fixture(scope="module") +def ucx_cluster(): + initialize.initialize(create_cuda_context=True, + enable_tcp_over_ucx=enable_tcp_over_ucx, + enable_nvlink=enable_nvlink, + enable_infiniband=enable_infiniband) + cluster = LocalCUDACluster(protocol="ucx", + enable_tcp_over_ucx=enable_tcp_over_ucx, + enable_nvlink=enable_nvlink, + enable_infiniband=enable_infiniband, + ucx_net_devices="auto") + yield cluster + cluster.close() + + +@pytest.fixture() +def client(cluster): + client = Client(cluster) + yield client + client.close() + + +@pytest.fixture() +def ucx_client(ucx_cluster): + client = Client(cluster) + yield client + client.close() diff --git a/python/raft/test/test_comms.py b/python/raft/test/test_comms.py new file mode 100644 index 0000000000..b4b6d0e7cc --- /dev/null +++ b/python/raft/test/test_comms.py @@ -0,0 +1,104 @@ +# Copyright (c) 2019, NVIDIA CORPORATION. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +import pytest + +import random + +import raft + +from dask.distributed import Client, wait + +from raft.dask.common import CommsContext, worker_state +from raft.dask.common import perform_test_comms_send_recv +from raft.dask.common import perform_test_comms_allreduce + +pytestmark = pytest.mark.mg + + +def test_comms_init_no_p2p(cluster): + + client = Client(cluster) + + try: + cb = CommsContext(comms_p2p=False) + cb.init() + + assert cb.nccl_initialized is True + assert cb.ucx_initialized is False + + finally: + + cb.destroy() + client.close() + + +def func_test_allreduce(sessionId, r): + handle = worker_state(sessionId)["handle"] + return perform_test_comms_allreduce(handle) + + +def func_test_send_recv(sessionId, n_trials, r): + handle = worker_state(sessionId)["handle"] + return perform_test_comms_send_recv(handle, n_trials) + + + +@pytest.mark.nccl +def test_allreduce(cluster): + + client = Client(cluster) + + try: + cb = CommsContext() + cb.init() + + dfs = [client.submit(func_test_allreduce, cb.sessionId, + random.random(), workers=[w]) + for w in cb.worker_addresses] + wait(dfs, timeout=5) + + assert all([x.result() for x in dfs]) + + finally: + cb.destroy() + client.close() + + +@pytest.mark.ucx +@pytest.mark.parametrize("n_trials", [1, 5]) +def test_send_recv(n_trials, ucx_cluster): + + client = Client(ucx_cluster) + + try: + + cb = CommsContext(comms_p2p=True, verbose=True) + cb.init() + + dfs = [client.submit(func_test_send_recv, + cb.sessionId, + n_trials, + random.random(), + workers=[w]) + for w in cb.worker_addresses] + + wait(dfs, timeout=5) + + assert(list(map(lambda x: x.result(), dfs))) + + finally: + cb.destroy() + client.close() diff --git a/python/setup.py b/python/setup.py index ccef769f42..107a061bd9 100644 --- a/python/setup.py +++ b/python/setup.py @@ -40,7 +40,7 @@ # - Dependencies include and lib folder setup -------------------------------- install_requires = [ - 'cython' + 'cython', ] cuda_home = get_environment_option("CUDA_HOME") @@ -92,7 +92,7 @@ # - Cython extensions build and parameters ----------------------------------- -libs = [] +libs = ["nccl", "rmm", "cusolver", "cusparse", "cublas"] include_dirs = [cuda_include_dir, numpy.get_include(), From 98496d3990ef7807f4a0a09a1688e2b287797ed9 Mon Sep 17 00:00:00 2001 From: "Corey J. Nolet" Date: Wed, 13 May 2020 14:20:39 -0400 Subject: [PATCH 10/46] Fixing flake8 style --- python/raft/dask/common/utils.py | 47 -------------------------------- python/raft/test/test_comms.py | 3 -- 2 files changed, 50 deletions(-) diff --git a/python/raft/dask/common/utils.py b/python/raft/dask/common/utils.py index 10b048d28d..ff5a0b4538 100644 --- a/python/raft/dask/common/utils.py +++ b/python/raft/dask/common/utils.py @@ -13,9 +13,7 @@ # limitations under the License. # -import logging import os -import numba.cuda import random import time @@ -35,51 +33,6 @@ def get_visible_devices(): return os.environ["CUDA_VISIBLE_DEVICES"].split(",") -def device_of_devicendarray(devicendarray): - """ - Returns the device that backs memory allocated on the given - deviceNDArray - :param devicendarray: devicendarray array to check - :return: int device id - """ - dev = device_of_gpu_matrix(devicendarray) - return get_visible_devices()[dev] - - -def get_device_id(canonical_name): - """ - Given a local device id, find the actual "global" id - :param canonical_name: the local device name in CUDA_VISIBLE_DEVICES - :return: the global device id for the system - """ - dev_order = get_visible_devices() - idx = 0 - for dev in dev_order: - if dev == canonical_name: - return idx - idx += 1 - - return -1 - - -def select_device(dev, close=True): - """ - Use numbas numba to select the given device, optionally - closing and opening up a new cuda context if it fails. - :param dev: int device to select - :param close: bool close the cuda context and create new one? - """ - if numba.cuda.get_current_device().id != dev: - logging.warn("Selecting device " + str(dev)) - if close: - numba.cuda.close() - numba.cuda.select_device(dev) - if dev != numba.cuda.get_current_device().id: - logging.warn("Current device " + - str(numba.cuda.get_current_device()) + - " does not match expected " + str(dev)) - - def get_client(client=None): return default_client() if client is None else client diff --git a/python/raft/test/test_comms.py b/python/raft/test/test_comms.py index b4b6d0e7cc..226a471f3b 100644 --- a/python/raft/test/test_comms.py +++ b/python/raft/test/test_comms.py @@ -17,8 +17,6 @@ import random -import raft - from dask.distributed import Client, wait from raft.dask.common import CommsContext, worker_state @@ -55,7 +53,6 @@ def func_test_send_recv(sessionId, n_trials, r): return perform_test_comms_send_recv(handle, n_trials) - @pytest.mark.nccl def test_allreduce(cluster): From fb52f79dd065bca6718ada7881ebd8df53f43371 Mon Sep 17 00:00:00 2001 From: "Corey J. Nolet" Date: Wed, 13 May 2020 14:32:49 -0400 Subject: [PATCH 11/46] Running clang format and removing getDataType --- cpp/include/raft/comms/comms.hpp | 197 +++++++++--------------- cpp/include/raft/comms/comms_helper.hpp | 34 ++-- cpp/include/raft/comms/nccl_helper.hpp | 6 +- cpp/include/raft/comms/std_comms.hpp | 142 +++++++---------- cpp/include/raft/comms/ucp_helper.hpp | 2 +- 5 files changed, 148 insertions(+), 233 deletions(-) diff --git a/cpp/include/raft/comms/comms.hpp b/cpp/include/raft/comms/comms.hpp index f0cad3a380..fafcce03e1 100644 --- a/cpp/include/raft/comms/comms.hpp +++ b/cpp/include/raft/comms/comms.hpp @@ -18,7 +18,6 @@ #include - namespace raft { namespace comms { @@ -27,19 +26,16 @@ enum datatype_t { CHAR, UINT8, INT, UINT, INT64, UINT64, FLOAT, DOUBLE }; enum op_t { SUM, PROD, MIN, MAX }; /** - * The resulting status of distributed stream synchronization - */ + * The resulting status of distributed stream synchronization + */ enum status_t { - commStatusSuccess, // Synchronization successful - commStatusError, // An error occured querying sync status - commStatusAbort + commStatusSuccess, // Synchronization successful + commStatusError, // An error occured querying sync status + commStatusAbort }; // A failure occurred in sync, queued operations aborted - - class comms_iface { public: - virtual ~comms_iface(); virtual int getSize() const = 0; @@ -82,130 +78,77 @@ class comms_iface { cudaStream_t stream) const = 0; }; -class comms_t: public comms_iface { +class comms_t : public comms_iface { public: - - - - comms_t(std::unique_ptr impl) - : _impl(impl.release()) { - ASSERT(nullptr != _impl.get(), "ERROR: Invalid comms_iface used!"); - } - - int getSize() const { return _impl->getSize(); } - - int getRank() const { return _impl->getRank(); } - - std::unique_ptr commSplit(int color, int key) const { - return _impl->commSplit(color, key); - } - - void barrier() const { _impl->barrier(); } - - status_t syncStream( - cudaStream_t stream) const { - return _impl->syncStream(stream); - } - - void isend(const void* buf, int size, int dest, int tag, - request_t* request) const { - _impl->isend(buf, size, dest, tag, request); - } - - void irecv(void* buf, int size, int source, int tag, - request_t* request) const { - _impl->irecv(buf, size, source, tag, request); - } - - void waitall(int count, request_t array_of_requests[]) const { - _impl->waitall(count, array_of_requests); - } - - void allreduce(const void* sendbuff, void* recvbuff, - int count, datatype_t datatype, op_t op, - cudaStream_t stream) const { - _impl->allreduce(sendbuff, recvbuff, count, datatype, op, stream); - } - - void bcast(void* buff, int count, datatype_t datatype, - int root, cudaStream_t stream) const { - _impl->bcast(buff, count, datatype, root, stream); - } - - void reduce(const void* sendbuff, void* recvbuff, int count, - datatype_t datatype, op_t op, int root, - cudaStream_t stream) const { - _impl->reduce(sendbuff, recvbuff, count, datatype, op, root, stream); - } - - void allgather(const void* sendbuff, void* recvbuff, - int sendcount, datatype_t datatype, - cudaStream_t stream) const { - _impl->allgather(sendbuff, recvbuff, sendcount, datatype, stream); - } - - void allgatherv(const void* sendbuf, void* recvbuf, - const int recvcounts[], const int displs[], - datatype_t datatype, - cudaStream_t stream) const { - _impl->allgatherv(sendbuf, recvbuf, recvcounts, displs, datatype, stream); - } - - void reducescatter(const void* sendbuff, void* recvbuff, - int recvcount, datatype_t datatype, - op_t op, cudaStream_t stream) const { - _impl->reducescatter(sendbuff, recvbuff, recvcount, datatype, op, stream); - } + comms_t(std::unique_ptr impl) : _impl(impl.release()) { + ASSERT(nullptr != _impl.get(), "ERROR: Invalid comms_iface used!"); + } + + int getSize() const { return _impl->getSize(); } + + int getRank() const { return _impl->getRank(); } + + std::unique_ptr commSplit(int color, int key) const { + return _impl->commSplit(color, key); + } + + void barrier() const { _impl->barrier(); } + + status_t syncStream(cudaStream_t stream) const { + return _impl->syncStream(stream); + } + + void isend(const void* buf, int size, int dest, int tag, + request_t* request) const { + _impl->isend(buf, size, dest, tag, request); + } + + void irecv(void* buf, int size, int source, int tag, + request_t* request) const { + _impl->irecv(buf, size, source, tag, request); + } + + void waitall(int count, request_t array_of_requests[]) const { + _impl->waitall(count, array_of_requests); + } + + void allreduce(const void* sendbuff, void* recvbuff, int count, + datatype_t datatype, op_t op, cudaStream_t stream) const { + _impl->allreduce(sendbuff, recvbuff, count, datatype, op, stream); + } + + void bcast(void* buff, int count, datatype_t datatype, int root, + cudaStream_t stream) const { + _impl->bcast(buff, count, datatype, root, stream); + } + + void reduce(const void* sendbuff, void* recvbuff, int count, + datatype_t datatype, op_t op, int root, + cudaStream_t stream) const { + _impl->reduce(sendbuff, recvbuff, count, datatype, op, root, stream); + } + + void allgather(const void* sendbuff, void* recvbuff, int sendcount, + datatype_t datatype, cudaStream_t stream) const { + _impl->allgather(sendbuff, recvbuff, sendcount, datatype, stream); + } + + void allgatherv(const void* sendbuf, void* recvbuf, const int recvcounts[], + const int displs[], datatype_t datatype, + cudaStream_t stream) const { + _impl->allgatherv(sendbuf, recvbuf, recvcounts, displs, datatype, stream); + } + + void reducescatter(const void* sendbuff, void* recvbuff, int recvcount, + datatype_t datatype, op_t op, cudaStream_t stream) const { + _impl->reducescatter(sendbuff, recvbuff, recvcount, datatype, op, stream); + } private: std::unique_ptr _impl; - }; comms_iface::~comms_iface() {} - -//template -//inline datatype_t getDataType(T a); -// -//template <> -//inline datatype_t getDataType(char a) { -// return CHAR; -//} -// -//template <> -//inline datatype_t getDataType(uint8_t a) { -// return UINT8; -//} -// -//template <> -//inline datatype_t getDataType(int a) { -// return INT; -//} -// -//template <> -//inline datatype_t getDataType(uint32_t a) { -// return UINT; -//} -// -//template <> -//inline datatype_t getDataType(int64_t a) { -// return INT64; -//} -// -//template <> -//inline datatype_t getDataType(uint64_t a) { -// return UINT64; -//} -// -//template <> -//inline datatype_t getDataType(float a) { -// return FLOAT; -//} -// -//template <> -//inline datatype_t getDataType(double a) { -// return DOUBLE; -//} -} /// namespace comms +} // namespace comms } // namespace raft diff --git a/cpp/include/raft/comms/comms_helper.hpp b/cpp/include/raft/comms/comms_helper.hpp index 7a033d4205..33b45a761c 100644 --- a/cpp/include/raft/comms/comms_helper.hpp +++ b/cpp/include/raft/comms/comms_helper.hpp @@ -14,26 +14,24 @@ * limitations under the License. */ -#include +#include #include +#include #include -#include namespace raft { namespace comms { void build_comms_nccl_only(handle_t *handle, ncclComm_t comm, int size, - int rank) { - + int rank) { auto *raft_comm = new raft::comms::std_comms(comm, size, rank); - auto communicator = std::make_shared( - std::unique_ptr(raft_comm)); + auto communicator = + std::make_shared(std::unique_ptr(raft_comm)); handle->set_comms(communicator); } void build_comms_nccl_ucx(handle_t *handle, ncclComm_t comm, void *ucp_worker, - void *eps, int size, int rank) { - + void *eps, int size, int rank) { std::shared_ptr eps_sp = std::make_shared(new ucp_ep_h[size]); @@ -51,14 +49,15 @@ void build_comms_nccl_ucx(handle_t *handle, ncclComm_t comm, void *ucp_worker, } } - auto *raft_comm = new raft::comms::std_comms(comm, (ucp_worker_h)ucp_worker, eps_sp, size, rank); - auto communicator = std::make_shared(std::unique_ptr(raft_comm)); + auto *raft_comm = new raft::comms::std_comms(comm, (ucp_worker_h)ucp_worker, + eps_sp, size, rank); + auto communicator = + std::make_shared(std::unique_ptr(raft_comm)); handle->set_comms(communicator); } - -bool test_collective_allreduce(const handle_t& handle) { - const comms_t& communicator = handle.get_comms(); +bool test_collective_allreduce(const handle_t &handle) { + const comms_t &communicator = handle.get_comms(); const int send = 1; @@ -68,8 +67,8 @@ bool test_collective_allreduce(const handle_t& handle) { temp_d.resize(1, stream); CUDA_CHECK(cudaMemcpyAsync(temp_d.data(), &send, sizeof(int), cudaMemcpyHostToDevice, stream)); - communicator.allreduce(temp_d.data(), temp_d.data(), 1, datatype_t::INT, - SUM, stream); + communicator.allreduce(temp_d.data(), temp_d.data(), 1, datatype_t::INT, SUM, + stream); int temp_h = 0; CUDA_CHECK(cudaMemcpyAsync(&temp_h, temp_d.data(), sizeof(int), cudaMemcpyDeviceToHost, stream)); @@ -82,9 +81,8 @@ bool test_collective_allreduce(const handle_t& handle) { return temp_h == communicator.getSize(); } -bool test_pointToPoint_simple_send_recv(const handle_t& h, - int numTrials) { - const comms_t& communicator = h.get_comms(); +bool test_pointToPoint_simple_send_recv(const handle_t &h, int numTrials) { + const comms_t &communicator = h.get_comms(); const int rank = communicator.getRank(); bool ret = true; diff --git a/cpp/include/raft/comms/nccl_helper.hpp b/cpp/include/raft/comms/nccl_helper.hpp index 5f367bafae..ea70896e32 100644 --- a/cpp/include/raft/comms/nccl_helper.hpp +++ b/cpp/include/raft/comms/nccl_helper.hpp @@ -1,6 +1,5 @@ #include - namespace raft { namespace comms { inline void ncclUniqueIdFromChar(ncclUniqueId *id, char *uniqueId, int size) { @@ -13,6 +12,5 @@ inline void get_unique_id(char *uid, int size) { memcpy(uid, id.internal, size); } -} -} - +} // namespace comms +} // namespace raft diff --git a/cpp/include/raft/comms/std_comms.hpp b/cpp/include/raft/comms/std_comms.hpp index b185934cc6..5e2be6ce77 100644 --- a/cpp/include/raft/comms/std_comms.hpp +++ b/cpp/include/raft/comms/std_comms.hpp @@ -34,10 +34,10 @@ #include #include #include -#include #include #include #include +#include #include @@ -45,7 +45,6 @@ #include - #define NCCL_CHECK(call) \ do { \ ncclResult_t status = call; \ @@ -53,17 +52,15 @@ ncclGetErrorString(status)); \ } while (0) -#define NCCL_CHECK_NO_THROW(call) \ - do { \ - ncclResult_t status = call; \ - if (status != ncclSuccess) { \ +#define NCCL_CHECK_NO_THROW(call) \ + do { \ + ncclResult_t status = call; \ + if (status != ncclSuccess) { \ printf("NCCL call='%s' failed. Reason:%s\n", #call, \ - ncclGetErrorString(status)); \ - } \ + ncclGetErrorString(status)); \ + } \ } while (0) - - namespace raft { namespace comms { @@ -89,11 +86,7 @@ size_t getDatatypeSize(const datatype_t datatype) { } } - - - -ncclDataType_t getNCCLDatatype( - const datatype_t datatype) { +ncclDataType_t getNCCLDatatype(const datatype_t datatype) { switch (datatype) { case CHAR: return ncclChar; @@ -127,7 +120,6 @@ ncclRedOp_t getNCCLOp(const op_t op) { } } - class std_comms : public comms_iface { public: std_comms() = delete; @@ -141,16 +133,16 @@ class std_comms : public comms_iface { * @param rank rank of the current worker */ std_comms(ncclComm_t comm, ucp_worker_h ucp_worker, - std::shared_ptr eps, int size, int rank) : _nccl_comm(comm), - _ucp_worker(ucp_worker), - _ucp_eps(eps), - _size(size), - _rank(rank), - _next_request_id(0) { - initialize(); - p2p_enabled = true; - }; - + std::shared_ptr eps, int size, int rank) + : _nccl_comm(comm), + _ucp_worker(ucp_worker), + _ucp_eps(eps), + _size(size), + _rank(rank), + _next_request_id(0) { + initialize(); + p2p_enabled = true; + }; /** * @brief constructor for collective-only operation @@ -159,18 +151,16 @@ class std_comms : public comms_iface { * @param rank rank of the current worker */ std_comms(ncclComm_t comm, int size, int rank) - : _nccl_comm(comm), _size(size), _rank(rank) { - initialize(); + : _nccl_comm(comm), _size(size), _rank(rank) { + initialize(); }; - virtual ~std_comms(){ - CUDA_CHECK_NO_THROW(cudaStreamDestroy(_stream)); - - CUDA_CHECK_NO_THROW(cudaFree(_sendbuff)); - CUDA_CHECK_NO_THROW(cudaFree(_recvbuff)); - } - + virtual ~std_comms() { + CUDA_CHECK_NO_THROW(cudaStreamDestroy(_stream)); + CUDA_CHECK_NO_THROW(cudaFree(_sendbuff)); + CUDA_CHECK_NO_THROW(cudaFree(_recvbuff)); + } void initialize() { CUDA_CHECK(cudaStreamCreate(&_stream)); @@ -179,13 +169,11 @@ class std_comms : public comms_iface { CUDA_CHECK(cudaMalloc(&_recvbuff, sizeof(int))); } - int getSize() const { return _size; } int getRank() const { return _rank; } - std::unique_ptr - commSplit(int color, int key) const { + std::unique_ptr commSplit(int color, int key) const { // Not supported by NCCL ASSERT(false, "ERROR: commSplit called but not yet supported in this comms " @@ -196,15 +184,13 @@ class std_comms : public comms_iface { CUDA_CHECK(cudaMemsetAsync(_sendbuff, 1, sizeof(int), _stream)); CUDA_CHECK(cudaMemsetAsync(_recvbuff, 1, sizeof(int), _stream)); - allreduce(_sendbuff, _recvbuff, 1, INT, - SUM, _stream); + allreduce(_sendbuff, _recvbuff, 1, INT, SUM, _stream); ASSERT(syncStream(_stream) == status_t::commStatusSuccess, "ERROR: syncStream failed. This can be caused by a failed rank."); } void get_request_id(request_t *req) const { - request_t req_id; if (this->_free_requests.empty()) @@ -217,9 +203,8 @@ class std_comms : public comms_iface { *req = req_id; } - void isend(const void *buf, int size, int dest, - int tag, request_t *request) const { - + void isend(const void *buf, int size, int dest, int tag, + request_t *request) const { ASSERT(_ucp_worker != nullptr, "ERROR: UCX comms not initialized on communicator."); @@ -235,7 +220,7 @@ class std_comms : public comms_iface { } void irecv(void *buf, int size, int source, int tag, - request_t *request) const { + request_t *request) const { ASSERT(_ucp_worker != nullptr, "ERROR: UCX comms not initialized on communicator."); @@ -246,15 +231,13 @@ class std_comms : public comms_iface { ucp_tag_t tag_mask = default_tag_mask; ucp_request *ucp_req = (ucp_request *)malloc(sizeof(ucp_request)); - _ucp_handler.ucp_irecv(ucp_req, _ucp_worker, ep_ptr, buf, size, tag, tag_mask, - source); + _ucp_handler.ucp_irecv(ucp_req, _ucp_worker, ep_ptr, buf, size, tag, + tag_mask, source); _requests_in_flight.insert(std::make_pair(*request, ucp_req)); } - void waitall(int count, - request_t array_of_requests[]) const { - + void waitall(int count, request_t array_of_requests[]) const { ASSERT(_ucp_worker != nullptr, "ERROR: UCX comms not initialized on communicator."); @@ -324,65 +307,58 @@ class std_comms : public comms_iface { } } - void allreduce(const void *sendbuff, void *recvbuff, - int count, datatype_t datatype, - op_t op, cudaStream_t stream) const { - NCCL_CHECK(ncclAllReduce(sendbuff, recvbuff, count, getNCCLDatatype(datatype), - getNCCLOp(op), _nccl_comm, stream)); + void allreduce(const void *sendbuff, void *recvbuff, int count, + datatype_t datatype, op_t op, cudaStream_t stream) const { + NCCL_CHECK(ncclAllReduce(sendbuff, recvbuff, count, + getNCCLDatatype(datatype), getNCCLOp(op), + _nccl_comm, stream)); } - void bcast(void *buff, int count, datatype_t datatype, - int root, cudaStream_t stream) const { + void bcast(void *buff, int count, datatype_t datatype, int root, + cudaStream_t stream) const { NCCL_CHECK(ncclBroadcast(buff, buff, count, getNCCLDatatype(datatype), root, _nccl_comm, stream)); } - void reduce(const void *sendbuff, void *recvbuff, - int count, datatype_t datatype, op_t op, - int root, cudaStream_t stream) const { + void reduce(const void *sendbuff, void *recvbuff, int count, + datatype_t datatype, op_t op, int root, + cudaStream_t stream) const { NCCL_CHECK(ncclReduce(sendbuff, recvbuff, count, getNCCLDatatype(datatype), getNCCLOp(op), root, _nccl_comm, stream)); } - void allgather(const void *sendbuff, void *recvbuff, - int sendcount, datatype_t datatype, - cudaStream_t stream) const { + void allgather(const void *sendbuff, void *recvbuff, int sendcount, + datatype_t datatype, cudaStream_t stream) const { NCCL_CHECK(ncclAllGather(sendbuff, recvbuff, sendcount, getNCCLDatatype(datatype), _nccl_comm, stream)); } -// const void* sendbuf, void* recvbuf, -// const int recvcounts[], const int displs[], -// datatype_t datatype, cudaStream_t stream -// + // const void* sendbuf, void* recvbuf, + // const int recvcounts[], const int displs[], + // datatype_t datatype, cudaStream_t stream + // - void allgatherv(const void *sendbuf, void *recvbuf, - const int recvcounts[], - const int displs[], - datatype_t datatype, - cudaStream_t stream) const { + void allgatherv(const void *sendbuf, void *recvbuf, const int recvcounts[], + const int displs[], datatype_t datatype, + cudaStream_t stream) const { //From: "An Empirical Evaluation of Allgatherv on Multi-GPU Systems" - https://arxiv.org/pdf/1812.05964.pdf //Listing 1 on page 4. for (int root = 0; root < _size; ++root) { size_t dtype_size = getDatatypeSize(datatype); NCCL_CHECK(ncclBroadcast( - sendbuf, - static_cast(recvbuf) + displs[root] * dtype_size, + sendbuf, static_cast(recvbuf) + displs[root] * dtype_size, recvcounts[root], getNCCLDatatype(datatype), root, _nccl_comm, stream)); } } - void reducescatter(const void *sendbuff, - void *recvbuff, int recvcount, - datatype_t datatype, op_t op, - cudaStream_t stream) const { + void reducescatter(const void *sendbuff, void *recvbuff, int recvcount, + datatype_t datatype, op_t op, cudaStream_t stream) const { NCCL_CHECK(ncclReduceScatter(sendbuff, recvbuff, recvcount, getNCCLDatatype(datatype), getNCCLOp(op), _nccl_comm, stream)); } - status_t syncStream( - cudaStream_t stream) const { + status_t syncStream(cudaStream_t stream) const { cudaError_t cudaErr; ncclResult_t ncclErr, ncclAsyncErr; while (1) { @@ -426,12 +402,12 @@ class std_comms : public comms_iface { bool p2p_enabled = false; comms_ucp_handler _ucp_handler; ucp_worker_h _ucp_worker; - std::shared_ptr _ucp_eps; + std::shared_ptr _ucp_eps; mutable request_t _next_request_id; - mutable std::unordered_map + mutable std::unordered_map _requests_in_flight; mutable std::unordered_set _free_requests; }; -} +} // namespace comms } // end namespace raft diff --git a/cpp/include/raft/comms/ucp_helper.hpp b/cpp/include/raft/comms/ucp_helper.hpp index ee22b59101..6a52141545 100644 --- a/cpp/include/raft/comms/ucp_helper.hpp +++ b/cpp/include/raft/comms/ucp_helper.hpp @@ -14,8 +14,8 @@ * limitations under the License. */ -#include #include +#include #include #include #include From 46a27eb4fb7806d22ba543a73af9c1c93fdcb9f5 Mon Sep 17 00:00:00 2001 From: "Corey J. Nolet" Date: Wed, 13 May 2020 15:09:38 -0400 Subject: [PATCH 12/46] Cleaning up --- cpp/include/raft/comms/comms.hpp | 8 ++++---- cpp/include/raft/comms/comms_helper.hpp | 4 ++-- cpp/include/raft/comms/nccl_helper.hpp | 6 ++++-- cpp/include/raft/comms/std_comms.hpp | 6 ------ python/raft/common/handle.pyx | 11 +---------- python/raft/dask/common/comms_utils.pyx | 11 ----------- 6 files changed, 11 insertions(+), 35 deletions(-) diff --git a/cpp/include/raft/comms/comms.hpp b/cpp/include/raft/comms/comms.hpp index fafcce03e1..cd47322314 100644 --- a/cpp/include/raft/comms/comms.hpp +++ b/cpp/include/raft/comms/comms.hpp @@ -26,13 +26,13 @@ enum datatype_t { CHAR, UINT8, INT, UINT, INT64, UINT64, FLOAT, DOUBLE }; enum op_t { SUM, PROD, MIN, MAX }; /** - * The resulting status of distributed stream synchronization - */ + * The resulting status of distributed stream synchronization + */ enum status_t { commStatusSuccess, // Synchronization successful commStatusError, // An error occured querying sync status - commStatusAbort -}; // A failure occurred in sync, queued operations aborted + commStatusAbort // A failure occurred in sync, queued operations aborted +}; class comms_iface { public: diff --git a/cpp/include/raft/comms/comms_helper.hpp b/cpp/include/raft/comms/comms_helper.hpp index 33b45a761c..0d4cca48a4 100644 --- a/cpp/include/raft/comms/comms_helper.hpp +++ b/cpp/include/raft/comms/comms_helper.hpp @@ -95,7 +95,7 @@ bool test_pointToPoint_simple_send_recv(const handle_t &h, int numTrials) { //post receives for (int r = 0; r < communicator.getSize(); ++r) { if (r != rank) { - communicator.irecv(received_data.data() + request_idx, 1, r, 0, + communicator.irecv(received_data.data() + request_idx, sizeof(int), r, 0, requests.data() + request_idx); ++request_idx; } @@ -103,7 +103,7 @@ bool test_pointToPoint_simple_send_recv(const handle_t &h, int numTrials) { for (int r = 0; r < communicator.getSize(); ++r) { if (r != rank) { - communicator.isend(&rank, 1, r, 0, requests.data() + request_idx); + communicator.isend(&rank, sizeof(int), r, 0, requests.data() + request_idx); ++request_idx; } } diff --git a/cpp/include/raft/comms/nccl_helper.hpp b/cpp/include/raft/comms/nccl_helper.hpp index ea70896e32..5f367bafae 100644 --- a/cpp/include/raft/comms/nccl_helper.hpp +++ b/cpp/include/raft/comms/nccl_helper.hpp @@ -1,5 +1,6 @@ #include + namespace raft { namespace comms { inline void ncclUniqueIdFromChar(ncclUniqueId *id, char *uniqueId, int size) { @@ -12,5 +13,6 @@ inline void get_unique_id(char *uid, int size) { memcpy(uid, id.internal, size); } -} // namespace comms -} // namespace raft +} +} + diff --git a/cpp/include/raft/comms/std_comms.hpp b/cpp/include/raft/comms/std_comms.hpp index 5e2be6ce77..2644f46c7c 100644 --- a/cpp/include/raft/comms/std_comms.hpp +++ b/cpp/include/raft/comms/std_comms.hpp @@ -62,7 +62,6 @@ } while (0) namespace raft { - namespace comms { size_t getDatatypeSize(const datatype_t datatype) { @@ -333,11 +332,6 @@ class std_comms : public comms_iface { getNCCLDatatype(datatype), _nccl_comm, stream)); } - // const void* sendbuf, void* recvbuf, - // const int recvcounts[], const int displs[], - // datatype_t datatype, cudaStream_t stream - // - void allgatherv(const void *sendbuf, void *recvbuf, const int recvcounts[], const int displs[], datatype_t datatype, cudaStream_t stream) const { diff --git a/python/raft/common/handle.pyx b/python/raft/common/handle.pyx index 75c10876ea..3f01e3e34a 100644 --- a/python/raft/common/handle.pyx +++ b/python/raft/common/handle.pyx @@ -23,11 +23,6 @@ import raft from libcpp.memory cimport shared_ptr from raft.common.cuda cimport _Stream, _Error, cudaStreamSynchronize -# -# cdef extern from ".cuml/common/rmmAllocatorAdapterhpp" namespace "ML" nogil: -# cdef cppclass rmmAllocatorAdapter(deviceAllocator): -# pass - cdef class Handle: """ Handle is a lightweight python wrapper around the corresponding C++ class @@ -54,7 +49,7 @@ cdef class Handle: del handle # optional! """ - # ML::cumlHandle doesn't have copy operator. So, use pointer for the object + # handle_t doesn't have copy operator. So, use pointer for the object # python world cannot access to this raw object directly, hence use # 'size_t'! cdef size_t h @@ -66,10 +61,6 @@ cdef class Handle: def __cinit__(self, n_streams=0): self.n_streams = n_streams self.h = (new handle_t(n_streams)) - # cdef shared_ptr[deviceAllocator] rmmAlloc = ( - # shared_ptr[deviceAllocator](new rmmAllocatorAdapter())) - # cdef cumlHandle* h_ = self.h - # h_.setDeviceAllocator(rmmAlloc) def __dealloc__(self): h_ = self.h diff --git a/python/raft/dask/common/comms_utils.pyx b/python/raft/dask/common/comms_utils.pyx index 672c53aa28..be32be915a 100644 --- a/python/raft/dask/common/comms_utils.pyx +++ b/python/raft/dask/common/comms_utils.pyx @@ -43,17 +43,6 @@ cdef extern from "raft/comms/std_comms.hpp" namespace "raft::comms": pass -cdef extern from "raft/comms/comms.hpp" namespace "raft::comms": - - cdef cppclass comms_t: - pass - - cdef cppclass comms_iface: - pass - - - - cdef extern from "raft/comms/comms_helper.hpp" namespace "raft::comms": void build_comms_nccl_ucx(handle_t *handle, From fc6ba2ebd2d6739c3a8ab8ef5e8dbed200ceeb1f Mon Sep 17 00:00:00 2001 From: "Corey J. Nolet" Date: Wed, 13 May 2020 15:13:53 -0400 Subject: [PATCH 13/46] Fixing python style --- python/raft/common/handle.pxd | 5 ++-- python/raft/dask/common/comms_utils.pyx | 35 +++++++++++-------------- 2 files changed, 19 insertions(+), 21 deletions(-) diff --git a/python/raft/common/handle.pxd b/python/raft/common/handle.pxd index ea40495a2a..fd3dc787ac 100644 --- a/python/raft/common/handle.pxd +++ b/python/raft/common/handle.pxd @@ -24,7 +24,8 @@ from libcpp.memory cimport shared_ptr cimport raft.common.cuda -cdef extern from "raft/mr/device/allocator.hpp" namespace "raft::mr::device" nogil: +cdef extern from "raft/mr/device/allocator.hpp" \ + namespace "raft::mr::device" nogil: cdef cppclass allocator: pass @@ -35,4 +36,4 @@ cdef extern from "raft/handle.hpp" namespace "raft" nogil: void set_stream(raft.common.cuda._Stream s) except + void set_device_allocator(shared_ptr[allocator] a) except + raft.common.cuda._Stream get_stream() except + - int get_num_internal_streams() except + \ No newline at end of file + int get_num_internal_streams() except + diff --git a/python/raft/dask/common/comms_utils.pyx b/python/raft/dask/common/comms_utils.pyx index be32be915a..f5edf3158c 100644 --- a/python/raft/dask/common/comms_utils.pyx +++ b/python/raft/dask/common/comms_utils.pyx @@ -24,7 +24,6 @@ from cpython.long cimport PyLong_AsVoidPtr from libcpp cimport bool - from libc.stdint cimport uintptr_t cdef extern from "nccl.h": @@ -46,17 +45,16 @@ cdef extern from "raft/comms/std_comms.hpp" namespace "raft::comms": cdef extern from "raft/comms/comms_helper.hpp" namespace "raft::comms": void build_comms_nccl_ucx(handle_t *handle, - ncclComm_t comm, - void *ucp_worker, - void *eps, - int size, - int rank) except + + ncclComm_t comm, + void *ucp_worker, + void *eps, + int size, + int rank) except + void build_comms_nccl_only(handle_t *handle, - ncclComm_t comm, - int size, - int rank) except + - + ncclComm_t comm, + int size, + int rank) except + bool test_collective_allreduce(const handle_t &h) except + bool test_pointToPoint_simple_send_recv(const handle_t &h, @@ -81,7 +79,6 @@ def perform_test_comms_send_recv(handle, n_trials): return test_pointToPoint_simple_send_recv(deref(h), n_trials) - def inject_comms_on_handle_coll_only(handle, nccl_inst, size, rank, verbose): """ Given a handle and initialized nccl comm, creates a cumlCommunicator @@ -99,9 +96,9 @@ def inject_comms_on_handle_coll_only(handle, nccl_inst, size, rank, verbose): nccl_comm_ = nccl_comm_size_t build_comms_nccl_only(handle_, - deref(nccl_comm_), - size, - rank) + deref(nccl_comm_), + size, + rank) def inject_comms_on_handle(handle, nccl_inst, ucp_worker, eps, size, @@ -134,10 +131,10 @@ def inject_comms_on_handle(handle, nccl_inst, ucp_worker, eps, size, nccl_comm_ = nccl_comm_size_t build_comms_nccl_ucx(handle_, - deref(nccl_comm_), - ucp_worker_st, - ucp_eps, - size, - rank) + deref(nccl_comm_), + ucp_worker_st, + ucp_eps, + size, + rank) free(ucp_eps) From 842f5335592d78563df62da359f8718c56cda261 Mon Sep 17 00:00:00 2001 From: "Corey J. Nolet" Date: Wed, 13 May 2020 15:14:31 -0400 Subject: [PATCH 14/46] Fixing cpp style --- cpp/include/raft/comms/comms.hpp | 2 +- cpp/include/raft/comms/comms_helper.hpp | 7 ++++--- cpp/include/raft/comms/nccl_helper.hpp | 6 ++---- cpp/include/raft/cudart_utils.h | 16 ++++++++-------- cpp/include/raft/handle.hpp | 10 +++------- 5 files changed, 18 insertions(+), 23 deletions(-) diff --git a/cpp/include/raft/comms/comms.hpp b/cpp/include/raft/comms/comms.hpp index cd47322314..3ef25a9285 100644 --- a/cpp/include/raft/comms/comms.hpp +++ b/cpp/include/raft/comms/comms.hpp @@ -31,7 +31,7 @@ enum op_t { SUM, PROD, MIN, MAX }; enum status_t { commStatusSuccess, // Synchronization successful commStatusError, // An error occured querying sync status - commStatusAbort // A failure occurred in sync, queued operations aborted + commStatusAbort // A failure occurred in sync, queued operations aborted }; class comms_iface { diff --git a/cpp/include/raft/comms/comms_helper.hpp b/cpp/include/raft/comms/comms_helper.hpp index 0d4cca48a4..a1236233cc 100644 --- a/cpp/include/raft/comms/comms_helper.hpp +++ b/cpp/include/raft/comms/comms_helper.hpp @@ -95,15 +95,16 @@ bool test_pointToPoint_simple_send_recv(const handle_t &h, int numTrials) { //post receives for (int r = 0; r < communicator.getSize(); ++r) { if (r != rank) { - communicator.irecv(received_data.data() + request_idx, sizeof(int), r, 0, - requests.data() + request_idx); + communicator.irecv(received_data.data() + request_idx, sizeof(int), r, + 0, requests.data() + request_idx); ++request_idx; } } for (int r = 0; r < communicator.getSize(); ++r) { if (r != rank) { - communicator.isend(&rank, sizeof(int), r, 0, requests.data() + request_idx); + communicator.isend(&rank, sizeof(int), r, 0, + requests.data() + request_idx); ++request_idx; } } diff --git a/cpp/include/raft/comms/nccl_helper.hpp b/cpp/include/raft/comms/nccl_helper.hpp index 5f367bafae..ea70896e32 100644 --- a/cpp/include/raft/comms/nccl_helper.hpp +++ b/cpp/include/raft/comms/nccl_helper.hpp @@ -1,6 +1,5 @@ #include - namespace raft { namespace comms { inline void ncclUniqueIdFromChar(ncclUniqueId *id, char *uniqueId, int size) { @@ -13,6 +12,5 @@ inline void get_unique_id(char *uid, int size) { memcpy(uid, id.internal, size); } -} -} - +} // namespace comms +} // namespace raft diff --git a/cpp/include/raft/cudart_utils.h b/cpp/include/raft/cudart_utils.h index 93543d09de..47e76ab916 100644 --- a/cpp/include/raft/cudart_utils.h +++ b/cpp/include/raft/cudart_utils.h @@ -112,14 +112,14 @@ class exception : public std::exception { // * @brief check for cuda runtime API errors but log error instead of raising // * exception. // */ - #define CUDA_CHECK_NO_THROW(call) \ - do { \ - cudaError_t status = call; \ - if (status != cudaSuccess) { \ - printf("CUDA call='%s' at file=%s line=%d failed with %s\n", \ - #call, __FILE__, __LINE__, cudaGetErrorString(status)); \ - } \ - } while (0) +#define CUDA_CHECK_NO_THROW(call) \ + do { \ + cudaError_t status = call; \ + if (status != cudaSuccess) { \ + printf("CUDA call='%s' at file=%s line=%d failed with %s\n", #call, \ + __FILE__, __LINE__, cudaGetErrorString(status)); \ + } \ + } while (0) /** helper method to get max usable shared mem per block parameter */ inline int get_shared_memory_per_block() { diff --git a/cpp/include/raft/handle.hpp b/cpp/include/raft/handle.hpp index 055fdf9e3f..f8db324d90 100644 --- a/cpp/include/raft/handle.hpp +++ b/cpp/include/raft/handle.hpp @@ -149,8 +149,7 @@ class handle_t { } } - void set_comms( - std::shared_ptr communicator) { + void set_comms(std::shared_ptr communicator) { _communicator = communicator; } @@ -160,10 +159,7 @@ class handle_t { return *_communicator; } - bool comms_initialized() const { - return (nullptr != _communicator.get()); - } - + bool comms_initialized() const { return (nullptr != _communicator.get()); } const cudaDeviceProp& get_device_properties() const { std::lock_guard _(mutex_); @@ -176,7 +172,7 @@ class handle_t { private: std::shared_ptr _communicator; - + const int dev_id_; const int num_streams_; std::vector streams_; From 43f0d69f5275dd81d5f1794dcb5917133e1770cc Mon Sep 17 00:00:00 2001 From: "Corey J. Nolet" Date: Wed, 13 May 2020 15:20:56 -0400 Subject: [PATCH 15/46] Adding copyright headers --- python/raft.egg-info/PKG-INFO | 14 ++++++++++++++ python/raft.egg-info/SOURCES.txt | 13 +++++++++++++ python/raft.egg-info/dependency_links.txt | 1 + python/raft.egg-info/not-zip-safe | 1 + python/raft.egg-info/requires.txt | 1 + python/raft.egg-info/top_level.txt | 1 + python/raft/__init__.py | 15 +++++++++++++++ python/raft/common/__init__.py | 14 ++++++++++++++ python/raft/dask/__init__.py | 14 ++++++++++++++ python/raft/dask/common/__init__.py | 22 ++++++++++++++++++---- 10 files changed, 92 insertions(+), 4 deletions(-) create mode 100644 python/raft.egg-info/PKG-INFO create mode 100644 python/raft.egg-info/SOURCES.txt create mode 100644 python/raft.egg-info/dependency_links.txt create mode 100644 python/raft.egg-info/not-zip-safe create mode 100644 python/raft.egg-info/requires.txt create mode 100644 python/raft.egg-info/top_level.txt diff --git a/python/raft.egg-info/PKG-INFO b/python/raft.egg-info/PKG-INFO new file mode 100644 index 0000000000..6b926d9fe9 --- /dev/null +++ b/python/raft.egg-info/PKG-INFO @@ -0,0 +1,14 @@ +Metadata-Version: 1.1 +Name: raft +Version: 0+untagged.85.g5d37e9e.dirty +Summary: RAPIDS Analytics Frameworks Toolset +Home-page: UNKNOWN +Author: NVIDIA Corporation +Author-email: UNKNOWN +License: Apache +Description: UNKNOWN +Platform: UNKNOWN +Classifier: Intended Audience :: Developers +Classifier: Programming Language :: Python +Classifier: Programming Language :: Python :: 3.6 +Classifier: Programming Language :: Python :: 3.7 diff --git a/python/raft.egg-info/SOURCES.txt b/python/raft.egg-info/SOURCES.txt new file mode 100644 index 0000000000..18f16faa2b --- /dev/null +++ b/python/raft.egg-info/SOURCES.txt @@ -0,0 +1,13 @@ +setup.cfg +setup.py +raft.egg-info/PKG-INFO +raft.egg-info/SOURCES.txt +raft.egg-info/dependency_links.txt +raft.egg-info/not-zip-safe +raft.egg-info/requires.txt +raft.egg-info/top_level.txt +raft/common/cuda.cpp +raft/common/handle.cpp +raft/dask/common/comms_utils.cpp +raft/dask/common/nccl.cpp +raft/include_test/raft_include_test.cpp \ No newline at end of file diff --git a/python/raft.egg-info/dependency_links.txt b/python/raft.egg-info/dependency_links.txt new file mode 100644 index 0000000000..8b13789179 --- /dev/null +++ b/python/raft.egg-info/dependency_links.txt @@ -0,0 +1 @@ + diff --git a/python/raft.egg-info/not-zip-safe b/python/raft.egg-info/not-zip-safe new file mode 100644 index 0000000000..8b13789179 --- /dev/null +++ b/python/raft.egg-info/not-zip-safe @@ -0,0 +1 @@ + diff --git a/python/raft.egg-info/requires.txt b/python/raft.egg-info/requires.txt new file mode 100644 index 0000000000..f6629e0245 --- /dev/null +++ b/python/raft.egg-info/requires.txt @@ -0,0 +1 @@ +cython diff --git a/python/raft.egg-info/top_level.txt b/python/raft.egg-info/top_level.txt new file mode 100644 index 0000000000..72e8ffc0db --- /dev/null +++ b/python/raft.egg-info/top_level.txt @@ -0,0 +1 @@ +* diff --git a/python/raft/__init__.py b/python/raft/__init__.py index ff4479b422..b2431b4f6c 100644 --- a/python/raft/__init__.py +++ b/python/raft/__init__.py @@ -1 +1,16 @@ +# Copyright (c) 2020, NVIDIA CORPORATION. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + from .include_test import raft_include_test diff --git a/python/raft/common/__init__.py b/python/raft/common/__init__.py index e69de29bb2..df8a4ae3b9 100644 --- a/python/raft/common/__init__.py +++ b/python/raft/common/__init__.py @@ -0,0 +1,14 @@ +# Copyright (c) 2020, NVIDIA CORPORATION. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# \ No newline at end of file diff --git a/python/raft/dask/__init__.py b/python/raft/dask/__init__.py index e69de29bb2..df8a4ae3b9 100644 --- a/python/raft/dask/__init__.py +++ b/python/raft/dask/__init__.py @@ -0,0 +1,14 @@ +# Copyright (c) 2020, NVIDIA CORPORATION. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# \ No newline at end of file diff --git a/python/raft/dask/common/__init__.py b/python/raft/dask/common/__init__.py index b8d7b28313..2ab93d9996 100644 --- a/python/raft/dask/common/__init__.py +++ b/python/raft/dask/common/__init__.py @@ -1,8 +1,22 @@ - +# Copyright (c) 2020, NVIDIA CORPORATION. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# from raft.dask.common.comms import CommsContext, worker_state -from raft.dask.common.comms_utils import inject_comms_on_handle, \ - perform_test_comms_allreduce, perform_test_comms_send_recv, \ - inject_comms_on_handle_coll_only +from raft.dask.common.comms_utils import inject_comms_on_handle +from raft.dask.common.comms_utils import inject_comms_on_handle_coll_only +from raft.dask.common.comms_utils import perform_test_comms_allreduce +from raft.dask.common.comms_utils import perform_test_comms_send_recv From ac6e69920149374441811d35a7e115729117b7d8 Mon Sep 17 00:00:00 2001 From: "Corey J. Nolet" Date: Wed, 13 May 2020 15:21:31 -0400 Subject: [PATCH 16/46] Adding init py for tests --- python/raft/test/__init__.py | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/python/raft/test/__init__.py b/python/raft/test/__init__.py index e69de29bb2..df8a4ae3b9 100644 --- a/python/raft/test/__init__.py +++ b/python/raft/test/__init__.py @@ -0,0 +1,14 @@ +# Copyright (c) 2020, NVIDIA CORPORATION. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# \ No newline at end of file From c442d877558de56ef0099bf71c791f70a5f212ba Mon Sep 17 00:00:00 2001 From: "Corey J. Nolet" Date: Wed, 13 May 2020 15:24:52 -0400 Subject: [PATCH 17/46] Adding license headers and consistent namespacing --- cpp/include/raft/comms/comms.hpp | 2 +- cpp/include/raft/comms/comms_helper.hpp | 4 +++- cpp/include/raft/comms/nccl_helper.hpp | 18 ++++++++++++++++++ cpp/include/raft/comms/std_comms.hpp | 5 ++--- cpp/include/raft/comms/ucp_helper.hpp | 9 +++++++-- 5 files changed, 31 insertions(+), 7 deletions(-) diff --git a/cpp/include/raft/comms/comms.hpp b/cpp/include/raft/comms/comms.hpp index 3ef25a9285..3a61207214 100644 --- a/cpp/include/raft/comms/comms.hpp +++ b/cpp/include/raft/comms/comms.hpp @@ -1,5 +1,5 @@ /* - * Copyright (c) 2019, NVIDIA CORPORATION. + * Copyright (c) 2020, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. diff --git a/cpp/include/raft/comms/comms_helper.hpp b/cpp/include/raft/comms/comms_helper.hpp index a1236233cc..2e71009eaf 100644 --- a/cpp/include/raft/comms/comms_helper.hpp +++ b/cpp/include/raft/comms/comms_helper.hpp @@ -1,5 +1,5 @@ /* - * Copyright (c) 2019, NVIDIA CORPORATION. + * Copyright (c) 2020, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -14,6 +14,8 @@ * limitations under the License. */ +#pragma once + #include #include #include diff --git a/cpp/include/raft/comms/nccl_helper.hpp b/cpp/include/raft/comms/nccl_helper.hpp index ea70896e32..d7a14ba8ba 100644 --- a/cpp/include/raft/comms/nccl_helper.hpp +++ b/cpp/include/raft/comms/nccl_helper.hpp @@ -1,3 +1,21 @@ +/* + * Copyright (c) 2020, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#pragma once + #include namespace raft { diff --git a/cpp/include/raft/comms/std_comms.hpp b/cpp/include/raft/comms/std_comms.hpp index 2644f46c7c..2af64a262b 100644 --- a/cpp/include/raft/comms/std_comms.hpp +++ b/cpp/include/raft/comms/std_comms.hpp @@ -1,5 +1,5 @@ /* - * Copyright (c) 2019-2020, NVIDIA CORPORATION. + * Copyright (c) 2020, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -402,6 +402,5 @@ class std_comms : public comms_iface { _requests_in_flight; mutable std::unordered_set _free_requests; }; -} // namespace comms - +} // end namespace comms } // end namespace raft diff --git a/cpp/include/raft/comms/ucp_helper.hpp b/cpp/include/raft/comms/ucp_helper.hpp index 6a52141545..e092509fe2 100644 --- a/cpp/include/raft/comms/ucp_helper.hpp +++ b/cpp/include/raft/comms/ucp_helper.hpp @@ -1,5 +1,5 @@ /* - * Copyright (c) 2019-2020, NVIDIA CORPORATION. + * Copyright (c) 2020, NVIDIA CORPORATION. * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -14,13 +14,16 @@ * limitations under the License. */ +#pragma once + #include #include #include #include #include -#pragma once +namespace raft { +namespace comms { typedef void (*dlsym_print_info)(ucp_ep_h, FILE *); typedef void (*dlsym_rec_free)(void *); typedef int (*dlsym_worker_progress)(ucp_worker_h); @@ -223,3 +226,5 @@ class comms_ucp_handler { UCS_PTR_STATUS(recv_result)); } }; +} // end namespace comms +} // end namespace raft From 32e63c159b1c7f12560b1c32ec317a6739ac7544 Mon Sep 17 00:00:00 2001 From: "Corey J. Nolet" Date: Wed, 13 May 2020 15:26:46 -0400 Subject: [PATCH 18/46] More cleanup --- cpp/include/raft/comms/ucp_helper.hpp | 1 + 1 file changed, 1 insertion(+) diff --git a/cpp/include/raft/comms/ucp_helper.hpp b/cpp/include/raft/comms/ucp_helper.hpp index e092509fe2..06f687752f 100644 --- a/cpp/include/raft/comms/ucp_helper.hpp +++ b/cpp/include/raft/comms/ucp_helper.hpp @@ -24,6 +24,7 @@ namespace raft { namespace comms { + typedef void (*dlsym_print_info)(ucp_ep_h, FILE *); typedef void (*dlsym_rec_free)(void *); typedef int (*dlsym_worker_progress)(ucp_worker_h); From d377347c6a1fa83d8b9794b5b5b62b9f5341574b Mon Sep 17 00:00:00 2001 From: "Corey J. Nolet" Date: Wed, 13 May 2020 16:11:20 -0400 Subject: [PATCH 19/46] Cleaning up raft.dask.common.Comms --- python/raft/dask/common/__init__.py | 3 +- python/raft/dask/common/comms.py | 349 +++++++++++++++++----------- python/raft/dask/common/utils.py | 3 +- python/raft/test/test_comms.py | 55 +++-- 4 files changed, 261 insertions(+), 149 deletions(-) diff --git a/python/raft/dask/common/__init__.py b/python/raft/dask/common/__init__.py index 2ab93d9996..9b85a32800 100644 --- a/python/raft/dask/common/__init__.py +++ b/python/raft/dask/common/__init__.py @@ -13,7 +13,8 @@ # limitations under the License. # -from raft.dask.common.comms import CommsContext, worker_state +from raft.dask.common.comms import Comms +from raft.dask.common.comms import local_handle from raft.dask.common.comms_utils import inject_comms_on_handle from raft.dask.common.comms_utils import inject_comms_on_handle_coll_only diff --git a/python/raft/dask/common/comms.py b/python/raft/dask/common/comms.py index 54e15c54c2..4ac47e25a6 100644 --- a/python/raft/dask/common/comms.py +++ b/python/raft/dask/common/comms.py @@ -30,25 +30,204 @@ import uuid +class Comms: + + """ + Initializes and manages underlying NCCL and UCX comms handles across + the workers of a Dask cluster. It is expected that `init()` will be + called explicitly. It is recommended to also call `destroy()` when + the comms are no longer needed so the underlying resources can be + cleaned up. This class is not meant to be thread-safe. + + Examples + -------- + .. code-block:: python + + # The following code block assumes we have wrapped a C++ + # function in a Python function called `run_algorithm`, + # which takes a `raft::handle_t` as a single argument. + # Once the `Comms` instance is successfully initialized, + # the underlying `raft::handle_t` will contain an instance + # of `raft::comms::comms_t` + + from dask_cuda import LocalCUDACluster + from dask.distributed import Client + + from raft.dask.common import Comms, local_handle + + cluster = LocalCUDACluster() + client = Client(cluster) + + def _use_comms(sessionId): + return run_algorithm(local_handle(sessionId)) + + comms = Comms(client=client) + comms.init() + + futures = [client.submit(_use_comms, + comms.sessionId, + workers=[w], + pure=False) # Don't memoize + for w in cb.worker_addresses] + wait(dfs, timeout=5) + + comms.destroy() + client.close() + cluster.close() + """ + + def __init__(self, comms_p2p=False, client=None, verbose=False, + streams_per_handle=0): + """ + Construct a new CommsContext instance + + Parameters + ---------- + comms_p2p : bool + Initialize UCX endpoints? + client : dask.distributed.Client [optional] + Dask client to use + verbose : bool + Print verbose logging + """ + self.client = client if client is not None else default_client() + self.comms_p2p = comms_p2p + + self.streams_per_handle = streams_per_handle + + self.sessionId = uuid.uuid4().bytes + + self.nccl_initialized = False + self.ucx_initialized = False + + self.verbose = verbose + + if verbose: + print("Initializing comms!") + + def __del__(self): + if self.nccl_initialized or self.ucx_initialized: + self.destroy() + + def worker_info(self, workers): + """ + Builds a dictionary of { (worker_address, worker_port) : + (worker_rank, worker_port ) } + """ + ranks = _func_worker_ranks(workers) + ports = _func_ucp_ports(self.client, workers) \ + if self.comms_p2p else None + + output = {} + for k in ranks.keys(): + output[k] = {"rank": ranks[k]} + if self.comms_p2p: + output[k]["port"] = ports[k] + return output + + def init(self, workers=None): + """ + Initializes the underlying comms. NCCL is required but + UCX is only initialized if `comms_p2p == True` + + Parameters + ---------- + + workers : Sequence + Unique collection of workers for initializing comms. + """ + + self.worker_addresses = list(set( + self.client.scheduler_info()["workers"].keys() + if workers is None else workers)) + + if self.nccl_initialized or self.ucx_initialized: + warnings.warn("Comms have already been initialized.") + return + + worker_info = self.worker_info(self.worker_addresses) + worker_info = {w: worker_info[w] for w in self.worker_addresses} + + self.uniqueId = nccl.get_unique_id() + + self.client.run(_func_init_all, + self.sessionId, + self.uniqueId, + self.comms_p2p, + worker_info, + self.verbose, + self.streams_per_handle, + workers=self.worker_addresses, + wait=True) + + self.nccl_initialized = True + + if self.comms_p2p: + self.ucx_initialized = True + + if self.verbose: + print("Initialization complete.") + + def destroy(self): + """ + Shuts down initialized comms and cleans up resources. This will + be called automatically by the Comms destructor, but may be called + earlier to save resources. + """ + self.client.run(_func_destroy_all, + self.sessionId, + self.comms_p2p, + self.verbose, + wait=True, + workers=self.worker_addresses) + + if self.verbose: + print("Destroying comms.") + + self.nccl_initialized = False + self.ucx_initialized = False + + +def local_handle(sessionId): + """Simple helper function for retrieving the local handle_t instance + for a comms session on a worker. + + Parameters + ---------- + sessionId : str + session identifier from an initialized comms instance + + Returns + ------- + + handle : raft.Handle or None + """ + state = worker_state(sessionId) + return state["handle"] if "handle" in state else None + + def worker_state(sessionId=None): """ Retrieves cuML comms state on local worker for the given sessionId, creating a new session if it does not exist. If no session id is given, returns the state dict for all sessions. - :param sessionId: - :return: + + Parameters + ---------- + sessionId : str + session identifier from initialized comms instance """ worker = get_worker() - if not hasattr(worker, "_cuml_comm_state"): - worker._cuml_comm_state = {} - if sessionId is not None and sessionId not in worker._cuml_comm_state: + if not hasattr(worker, "_raft_comm_state"): + worker._raft_comm_state = {} + if sessionId is not None and sessionId not in worker._raft_comm_state: # Build state for new session and mark session creation time - worker._cuml_comm_state[sessionId] = {"ts": time.time()} + worker._raft_comm_state[sessionId] = {"ts": time.time()} if sessionId is not None: - return worker._cuml_comm_state[sessionId] - return worker._cuml_comm_state + return worker._raft_comm_state[sessionId] + return worker._raft_comm_state def get_ucx(): @@ -109,10 +288,14 @@ async def _func_init_all(sessionId, uniqueId, comms_p2p, def _func_init_nccl(sessionId, uniqueId): """ Initialize ncclComm_t on worker - :param workerId: int ID of the current worker running the function - :param nWorkers: int Number of workers in the cluster - :param uniqueId: array[byte] The NCCL unique Id generated from the - client. + + Parameters + ---------- + sessionId : str + session identifier from a comms instance + uniqueId : array[byte] + The NCCL unique Id generated from the + client. """ wid = worker_state(sessionId)["wid"] @@ -128,12 +311,13 @@ def _func_init_nccl(sessionId, uniqueId): def _func_build_handle_p2p(sessionId, streams_per_handle, verbose): """ - Builds a cumlHandle on the current worker given the initialized comms - :param nccl_comm: ncclComm_t Initialized NCCL comm - :param eps: size_t initialized endpoints - :param nWorkers: int number of workers in cluster - :param workerId: int Rank of current worker - :return: + Builds a handle_t on the current worker given the initialized comms + + Parameters + ---------- + sessionId : str id to reference state for current comms instance. + streams_per_handle : int number of internal streams to create + verbose : bool print verbose logging output """ ucp_worker = get_ucx().get_worker() session_state = worker_state(sessionId) @@ -152,11 +336,13 @@ def _func_build_handle_p2p(sessionId, streams_per_handle, verbose): def _func_build_handle(sessionId, streams_per_handle, verbose): """ - Builds a cumlHandle on the current worker given the initialized comms - :param nccl_comm: ncclComm_t Initialized NCCL comm - :param nWorkers: int number of workers in cluster - :param workerId: int Rank of current worker - :return: + Builds a handle_t on the current worker given the initialized comms + + Parameters + ---------- + sessionId : str id to reference state for current comms instance. + streams_per_handle : int number of internal streams to create + verbose : bool print verbose logging output """ handle = Handle(streams_per_handle) @@ -181,9 +367,13 @@ def _func_store_initial_state(nworkers, sessionId, uniqueId, wid): async def _func_ucp_create_endpoints(sessionId, worker_info): """ Runs on each worker to create ucp endpoints to all other workers - :param sessionId: uuid unique id for this instance - :param worker_info: dict Maps worker address to rank & UCX port - :param r: float a random number to stop the function from being cached + + Parameters + ---------- + sessionId : str + uuid unique id for this instance + worker_info : dict + Maps worker addresses to NCCL ranks & UCX ports """ dask_worker = get_worker() local_address = dask_worker.address @@ -220,110 +410,3 @@ def _func_worker_ranks(workers): Builds a dictionary of { (worker_address, worker_port) : worker_rank } """ return dict(list(zip(workers, range(len(workers))))) - - -class CommsContext: - - """ - A base class to initialize and manage underlying NCCL and UCX - comms handles across a Dask cluster. Classes extending CommsContext - are responsible for calling `self.init()` to initialize the comms. - Classes that extend or use the CommsContext are also responsible for - calling `destroy()` to clean up the underlying comms. - - This class is not meant to be thread-safe. - """ - - def __init__(self, comms_p2p=False, client=None, verbose=False, - streams_per_handle=0): - """ - Construct a new CommsContext instance - :param comms_p2p: bool Should p2p comms be initialized? - """ - self.client = client if client is not None else default_client() - self.comms_p2p = comms_p2p - - self.streams_per_handle = streams_per_handle - - self.sessionId = uuid.uuid4().bytes - - self.nccl_initialized = False - self.ucx_initialized = False - - self.verbose = verbose - - if verbose: - print("Initializing comms!") - - def __del__(self): - if self.nccl_initialized or self.ucx_initialized: - self.destroy() - - def worker_info(self, workers): - """ - Builds a dictionary of { (worker_address, worker_port) : - (worker_rank, worker_port ) } - """ - ranks = _func_worker_ranks(workers) - ports = _func_ucp_ports(self.client, workers) \ - if self.comms_p2p else None - - output = {} - for k in ranks.keys(): - output[k] = {"rank": ranks[k]} - if self.comms_p2p: - output[k]["port"] = ports[k] - return output - - def init(self, workers=None): - """ - Initializes the underlying comms. NCCL is required but - UCX is only initialized if `comms_p2p == True` - """ - - self.worker_addresses = list(set((self.client.has_what().keys() - if workers is None else workers))) - - if self.nccl_initialized: - warnings.warn("CommsContext has already been initialized.") - return - - worker_info = self.worker_info(self.worker_addresses) - worker_info = {w: worker_info[w] for w in self.worker_addresses} - - self.uniqueId = nccl.get_unique_id() - - self.client.run(_func_init_all, - self.sessionId, - self.uniqueId, - self.comms_p2p, - worker_info, - self.verbose, - self.streams_per_handle, - workers=self.worker_addresses, - wait=True) - - self.nccl_initialized = True - - if self.comms_p2p: - self.ucx_initialized = True - - if self.verbose: - print("Initialization complete.") - - def destroy(self): - """ - Shuts down initialized comms and cleans up resources. - """ - self.client.run(_func_destroy_all, - self.sessionId, - self.comms_p2p, - self.verbose, - wait=True, - workers=self.worker_addresses) - - if self.verbose: - print("Destroying comms.") - - self.nccl_initialized = False - self.ucx_initialized = False diff --git a/python/raft/dask/common/utils.py b/python/raft/dask/common/utils.py index ff5a0b4538..8bb17c5ed8 100644 --- a/python/raft/dask/common/utils.py +++ b/python/raft/dask/common/utils.py @@ -17,7 +17,8 @@ import random import time -from dask.distributed import default_client, wait +from dask.distributed import default_client +from dask.distributed import wait from asyncio import InvalidStateError diff --git a/python/raft/test/test_comms.py b/python/raft/test/test_comms.py index 226a471f3b..c95bb86b65 100644 --- a/python/raft/test/test_comms.py +++ b/python/raft/test/test_comms.py @@ -15,11 +15,11 @@ import pytest -import random +from dask.distributed import Client +from dask.distributed import wait -from dask.distributed import Client, wait - -from raft.dask.common import CommsContext, worker_state +from raft.dask.common import Comms +from raft.dask.common import local_handle from raft.dask.common import perform_test_comms_send_recv from raft.dask.common import perform_test_comms_allreduce @@ -31,7 +31,7 @@ def test_comms_init_no_p2p(cluster): client = Client(cluster) try: - cb = CommsContext(comms_p2p=False) + cb = Comms() cb.init() assert cb.nccl_initialized is True @@ -43,27 +43,54 @@ def test_comms_init_no_p2p(cluster): client.close() -def func_test_allreduce(sessionId, r): - handle = worker_state(sessionId)["handle"] +def func_test_allreduce(sessionId): + handle = local_handle(sessionId) return perform_test_comms_allreduce(handle) -def func_test_send_recv(sessionId, n_trials, r): - handle = worker_state(sessionId)["handle"] +def func_test_send_recv(sessionId, n_trials): + handle = local_handle(sessionId) return perform_test_comms_send_recv(handle, n_trials) +def test_handles(cluster): + + client = Client(cluster) + + def _has_handle(sessionId): + return local_handle(sessionId) is not None + + try: + cb = Comms() + cb.init() + + dfs = [client.submit(_has_handle, + cb.sessionId, + pure=False, + workers=[w]) + for w in cb.worker_addresses] + wait(dfs, timeout=5) + + assert all(client.compute(dfs, sync=True)) + + finally: + cb.destroy() + client.close() + + @pytest.mark.nccl def test_allreduce(cluster): client = Client(cluster) try: - cb = CommsContext() + cb = Comms() cb.init() - dfs = [client.submit(func_test_allreduce, cb.sessionId, - random.random(), workers=[w]) + dfs = [client.submit(func_test_allreduce, + cb.sessionId, + pure=False, + workers=[w]) for w in cb.worker_addresses] wait(dfs, timeout=5) @@ -82,13 +109,13 @@ def test_send_recv(n_trials, ucx_cluster): try: - cb = CommsContext(comms_p2p=True, verbose=True) + cb = Comms(comms_p2p=True, verbose=True) cb.init() dfs = [client.submit(func_test_send_recv, cb.sessionId, n_trials, - random.random(), + pure=False, workers=[w]) for w in cb.worker_addresses] From e1b4ea73b8fde13ca7edb7ed10dc4dff7393ff07 Mon Sep 17 00:00:00 2001 From: "Corey J. Nolet" Date: Wed, 13 May 2020 16:13:59 -0400 Subject: [PATCH 20/46] Ignoring raft egg artifacts --- .gitignore | 1 + python/raft.egg-info/PKG-INFO | 14 -------------- python/raft.egg-info/SOURCES.txt | 13 ------------- python/raft.egg-info/dependency_links.txt | 1 - python/raft.egg-info/not-zip-safe | 1 - python/raft.egg-info/requires.txt | 1 - python/raft.egg-info/top_level.txt | 1 - 7 files changed, 1 insertion(+), 31 deletions(-) delete mode 100644 python/raft.egg-info/PKG-INFO delete mode 100644 python/raft.egg-info/SOURCES.txt delete mode 100644 python/raft.egg-info/dependency_links.txt delete mode 100644 python/raft.egg-info/not-zip-safe delete mode 100644 python/raft.egg-info/requires.txt delete mode 100644 python/raft.egg-info/top_level.txt diff --git a/.gitignore b/.gitignore index ecc92d8dce..60a43f6b54 100644 --- a/.gitignore +++ b/.gitignore @@ -23,6 +23,7 @@ log .ipynb_checkpoints .DS_Store dask-worker-space/ +*.egg-info/ ## eclipse .project .cproject diff --git a/python/raft.egg-info/PKG-INFO b/python/raft.egg-info/PKG-INFO deleted file mode 100644 index 6b926d9fe9..0000000000 --- a/python/raft.egg-info/PKG-INFO +++ /dev/null @@ -1,14 +0,0 @@ -Metadata-Version: 1.1 -Name: raft -Version: 0+untagged.85.g5d37e9e.dirty -Summary: RAPIDS Analytics Frameworks Toolset -Home-page: UNKNOWN -Author: NVIDIA Corporation -Author-email: UNKNOWN -License: Apache -Description: UNKNOWN -Platform: UNKNOWN -Classifier: Intended Audience :: Developers -Classifier: Programming Language :: Python -Classifier: Programming Language :: Python :: 3.6 -Classifier: Programming Language :: Python :: 3.7 diff --git a/python/raft.egg-info/SOURCES.txt b/python/raft.egg-info/SOURCES.txt deleted file mode 100644 index 18f16faa2b..0000000000 --- a/python/raft.egg-info/SOURCES.txt +++ /dev/null @@ -1,13 +0,0 @@ -setup.cfg -setup.py -raft.egg-info/PKG-INFO -raft.egg-info/SOURCES.txt -raft.egg-info/dependency_links.txt -raft.egg-info/not-zip-safe -raft.egg-info/requires.txt -raft.egg-info/top_level.txt -raft/common/cuda.cpp -raft/common/handle.cpp -raft/dask/common/comms_utils.cpp -raft/dask/common/nccl.cpp -raft/include_test/raft_include_test.cpp \ No newline at end of file diff --git a/python/raft.egg-info/dependency_links.txt b/python/raft.egg-info/dependency_links.txt deleted file mode 100644 index 8b13789179..0000000000 --- a/python/raft.egg-info/dependency_links.txt +++ /dev/null @@ -1 +0,0 @@ - diff --git a/python/raft.egg-info/not-zip-safe b/python/raft.egg-info/not-zip-safe deleted file mode 100644 index 8b13789179..0000000000 --- a/python/raft.egg-info/not-zip-safe +++ /dev/null @@ -1 +0,0 @@ - diff --git a/python/raft.egg-info/requires.txt b/python/raft.egg-info/requires.txt deleted file mode 100644 index f6629e0245..0000000000 --- a/python/raft.egg-info/requires.txt +++ /dev/null @@ -1 +0,0 @@ -cython diff --git a/python/raft.egg-info/top_level.txt b/python/raft.egg-info/top_level.txt deleted file mode 100644 index 72e8ffc0db..0000000000 --- a/python/raft.egg-info/top_level.txt +++ /dev/null @@ -1 +0,0 @@ -* From 736696b602e0cf864ae4108de8461c7b0b2bc1d7 Mon Sep 17 00:00:00 2001 From: "Corey J. Nolet" Date: Wed, 13 May 2020 16:20:20 -0400 Subject: [PATCH 21/46] Cleaning up raft dask utils --- python/raft/dask/common/utils.py | 181 ++----------------------------- 1 file changed, 8 insertions(+), 173 deletions(-) diff --git a/python/raft/dask/common/utils.py b/python/raft/dask/common/utils.py index 8bb17c5ed8..c92fbf3e47 100644 --- a/python/raft/dask/common/utils.py +++ b/python/raft/dask/common/utils.py @@ -13,25 +13,7 @@ # limitations under the License. # -import os -import random -import time - from dask.distributed import default_client -from dask.distributed import wait - -from asyncio import InvalidStateError - -from threading import Lock - - -def get_visible_devices(): - """ - Return a list of the CUDA_VISIBLE_DEVICES - :return: list[int] visible devices - """ - # TODO: Shouldn't have to split on every call - return os.environ["CUDA_VISIBLE_DEVICES"].split(",") def get_client(client=None): @@ -41,164 +23,17 @@ def get_client(client=None): def parse_host_port(address): """ Given a string address with host/port, build a tuple(host, port) - :param address: string address to parse - :return: tuple(host, port) + + Parameters + ---------- + address: string address to parse + + Returns + ------- + tuple with host and port info : tuple(host, port) """ if '://' in address: address = address.rsplit('://', 1)[1] host, port = address.split(':') port = int(port) return host, port - - -def build_host_dict(workers): - """ - Builds a dict to map the set of ports running on each host to - the hostname. - :param workers: list(tuple(host, port)) list of worker addresses - :return: dict(host, set(port)) - """ - hosts = set(map(lambda x: parse_host_port(x), workers)) - hosts_dict = {} - for host, port in hosts: - if host not in hosts_dict: - hosts_dict[host] = set([port]) - else: - hosts_dict[host].add(port) - - return hosts_dict - - -def persist_across_workers(client, objects, workers=None): - """ - Calls persist on the 'objects' ensuring they are spread - across the workers on 'workers'. - - Parameters - ---------- - client : dask.distributed.Client - objects : list - Dask distributed objects to be persisted - workers : list or None - List of workers across which to persist objects - If None, then all workers attached to 'client' will be used - """ - if workers is None: - workers = client.has_what().keys() # Default to all workers - return client.persist(objects, workers={o: workers for o in objects}) - - -def raise_exception_from_futures(futures): - """Raises a RuntimeError if any of the futures indicates an exception""" - errs = [f.exception() for f in futures if f.exception()] - if errs: - raise RuntimeError("%d of %d worker jobs failed: %s" % ( - len(errs), len(futures), ", ".join(map(str, errs)) - )) - - -def wait_and_raise_from_futures(futures): - """ - Returns the collected futures after all the futures - have finished and do not indicate any exceptions. - """ - wait(futures) - raise_exception_from_futures(futures) - return futures - - -def raise_mg_import_exception(): - raise Exception("cuML has not been built with multiGPU support " - "enabled. Build with the --multigpu flag to" - " enable multiGPU support.") - - -class MultiHolderLock: - """ - A per-process synchronization lock allowing multiple concurrent holders - at any one time. This is used in situations where resources might be - limited and it's important that the number of concurrent users of - the resources are constained. - - This lock is serializable, but relies on a Python threading.Lock - underneath to properly synchronize internal state across threads. - Note that this lock is only intended to be used per-process and - the underlying threading.Lock will not be serialized. - """ - - def __init__(self, n): - """ - Initialize the lock - :param n : integer the maximum number of concurrent holders - """ - self.n = n - self.current_tasks = 0 - self.lock = Lock() - - def _acquire(self, blocking=True, timeout=10): - lock_acquired = False - - inner_lock_acquired = self.lock.acquire(blocking, timeout) - - if inner_lock_acquired and self.current_tasks < self.n - 1: - self.current_tasks += 1 - lock_acquired = True - self.lock.release() - - return lock_acquired - - def acquire(self, blocking=True, timeout=10): - """ - Acquire the lock. - :param blocking : bool will block if True - :param timeout : a timeout (in seconds) to wait for the lock - before failing. - :return : True if lock was acquired successfully, False otherwise - """ - - t = time.time() - - lock_acquired = self._acquire(blocking, timeout) - - while blocking and not lock_acquired: - - if time.time() - t > timeout: - raise TimeoutError() - - lock_acquired = self.acquire(blocking, timeout) - time.sleep(random.uniform(0, 0.01)) - - return lock_acquired - - def __getstate__(self): - d = self.__dict__.copy() - if "lock" in d: - del d["lock"] - return d - - def __setstate__(self, d): - d["lock"] = Lock() - self.__dict__ = d - - def release(self, blocking=True, timeout=10): - """ - Release a hold on the lock to allow another holder. Note that - while Python's threading.Lock does not have options for blocking - or timeout in release(), this lock uses a threading.Lock - internally and so will need to acquire that lock in order - to properly synchronize the underlying state. - :param blocking : bool will bock if True - :param timeout : a timeout (in seconds) to wait for the lock - before failing. - :return : True if lock was released successfully, False otherwise. - """ - - if self.current_tasks == 0: - raise InvalidStateError("Cannot release lock when no " - "concurrent tasks are executing") - - lock_acquired = self.lock.acquire(blocking, timeout) - if lock_acquired: - self.current_tasks -= 1 - self.lock.release() - return lock_acquired From 55d9dfdc191bb923cce9c1e4f44aa60dce85c5dc Mon Sep 17 00:00:00 2001 From: "Corey J. Nolet" Date: Wed, 13 May 2020 16:35:24 -0400 Subject: [PATCH 22/46] More cleanup, copyright headers, and docs --- python/raft/common/__init__.py | 5 ++- python/raft/common/cuda.pyx | 6 ++-- python/raft/common/handle.pxd | 2 +- python/raft/common/handle.pyx | 23 ++++++------ python/raft/dask/__init__.py | 4 ++- python/raft/dask/common/comms_utils.pyx | 48 ++++++++++++++++++------- python/raft/dask/common/nccl.pyx | 36 ++++++++++++++----- python/raft/dask/common/ucx.py | 2 +- python/raft/dask/common/utils.py | 2 +- 9 files changed, 86 insertions(+), 42 deletions(-) diff --git a/python/raft/common/__init__.py b/python/raft/common/__init__.py index df8a4ae3b9..ac84a7a93a 100644 --- a/python/raft/common/__init__.py +++ b/python/raft/common/__init__.py @@ -11,4 +11,7 @@ # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. # See the License for the specific language governing permissions and # limitations under the License. -# \ No newline at end of file +# + +from raft.common.cuda import Stream +from raft.common.handle import Handle \ No newline at end of file diff --git a/python/raft/common/cuda.pyx b/python/raft/common/cuda.pyx index 09f347058f..baa46bfef8 100644 --- a/python/raft/common/cuda.pyx +++ b/python/raft/common/cuda.pyx @@ -1,5 +1,5 @@ # -# Copyright (c) 2019, NVIDIA CORPORATION. +# Copyright (c) 2020, NVIDIA CORPORATION. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. @@ -43,8 +43,8 @@ cdef class Stream: .. code-block:: python - import cuml - stream = cuml.cuda.Stream() + from raft.common.cuda import Stream + stream = Stream() stream.sync() del stream # optional! """ diff --git a/python/raft/common/handle.pxd b/python/raft/common/handle.pxd index fd3dc787ac..dfdcdb929b 100644 --- a/python/raft/common/handle.pxd +++ b/python/raft/common/handle.pxd @@ -1,5 +1,5 @@ # -# Copyright (c) 2019, NVIDIA CORPORATION. +# Copyright (c) 2020, NVIDIA CORPORATION. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. diff --git a/python/raft/common/handle.pyx b/python/raft/common/handle.pyx index 3f01e3e34a..3659cb3669 100644 --- a/python/raft/common/handle.pyx +++ b/python/raft/common/handle.pyx @@ -1,5 +1,5 @@ # -# Copyright (c) 2019, NVIDIA CORPORATION. +# Copyright (c) 2020, NVIDIA CORPORATION. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. @@ -26,25 +26,24 @@ from raft.common.cuda cimport _Stream, _Error, cudaStreamSynchronize cdef class Handle: """ Handle is a lightweight python wrapper around the corresponding C++ class - of cumlHandle exposed by cuML's C++ interface. Refer to the header file - cuml/cuml.hpp for interface level details of this struct + of handle_t exposed by RAFT's C++ interface. Refer to the header file + raft/handle.hpp for interface level details of this struct Examples -------- .. code-block:: python - import cuml - stream = cuml.cuda.Stream() - handle = cuml.Handle() + from raft.common import Stream, Handle + stream = Stream() + handle = Handle() handle.setStream(stream) - handle.enableRMM() # Enable RMM as the device-side allocator - # call ML algos here + # call algos here # final sync of all work launched in the stream of this handle - # this is same as `cuml.cuda.Stream.sync()` call, but safer in case - # the default stream inside the `cumlHandle` is being used + # this is same as `raft.cuda.Stream.sync()` call, but safer in case + # the default stream inside the `handle_t` is being used handle.sync() del handle # optional! """ @@ -75,8 +74,8 @@ cdef class Handle: """ Issues a sync on the stream set for this handle. - Once we make `cuml.cuda.Stream` as a mandatory option for creating - `cuml.Handle`, this should go away + Once we make `raft.common.cuda.Stream` as a mandatory option for creating + `raft.common.Handle`, this should go away """ cdef handle_t* h_ = self.h cdef _Stream stream = h_.get_stream() diff --git a/python/raft/dask/__init__.py b/python/raft/dask/__init__.py index df8a4ae3b9..e58ac25f47 100644 --- a/python/raft/dask/__init__.py +++ b/python/raft/dask/__init__.py @@ -11,4 +11,6 @@ # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. # See the License for the specific language governing permissions and # limitations under the License. -# \ No newline at end of file +# + +from raft.dask.common.comms import Comms \ No newline at end of file diff --git a/python/raft/dask/common/comms_utils.pyx b/python/raft/dask/common/comms_utils.pyx index f5edf3158c..90fd97671f 100644 --- a/python/raft/dask/common/comms_utils.pyx +++ b/python/raft/dask/common/comms_utils.pyx @@ -41,7 +41,6 @@ cdef extern from "raft/comms/std_comms.hpp" namespace "raft::comms": cdef cppclass std_comms: pass - cdef extern from "raft/comms/comms_helper.hpp" namespace "raft::comms": void build_comms_nccl_ucx(handle_t *handle, @@ -64,7 +63,11 @@ cdef extern from "raft/comms/comms_helper.hpp" namespace "raft::comms": def perform_test_comms_allreduce(handle): """ Performs an allreduce on the current worker - :param handle: Handle handle containing cumlCommunicator to use + + Parameters + ---------- + handle : raft.common.Handle + handle containing comms_t to use """ cdef const handle_t* h = handle.getHandle() return test_collective_allreduce(deref(h)) @@ -73,7 +76,11 @@ def perform_test_comms_allreduce(handle): def perform_test_comms_send_recv(handle, n_trials): """ Performs a p2p send/recv on the current worker - :param handle: Handle handle containing cumlCommunicator to use + + Parameters + ---------- + handle : raft.common.Handle + handle containing comms_t to use """ cdef const handle_t *h = handle.getHandle() return test_pointToPoint_simple_send_recv(deref(h), n_trials) @@ -83,10 +90,18 @@ def inject_comms_on_handle_coll_only(handle, nccl_inst, size, rank, verbose): """ Given a handle and initialized nccl comm, creates a cumlCommunicator instance and injects it into the handle. - :param handle: Handle cumlHandle to inject comms into - :param nccl_inst: ncclComm_t initialized nccl comm - :param size: int number of workers in cluster - :param rank: int rank of current worker + + Parameters + ---------- + handle : raft.common.Handle + handle containing comms_t to use + nccl_inst : raft.dask.common.nccl + Initialized nccl comm to use + size : int + Number of workers in cluster + rank : int + Rank of current worker + """ cdef size_t handle_size_t = handle.getHandle() @@ -106,12 +121,19 @@ def inject_comms_on_handle(handle, nccl_inst, ucp_worker, eps, size, """ Given a handle and initialized comms, creates a cumlCommunicator instance and injects it into the handle. - :param handle: Handle cumlHandle to inject comms into - :param nccl_inst: ncclComm_t initialized nccl comm - :param ucp_worker: size_t initialized ucp_worker_h instance - :param eps: size_t array of initialized ucp_ep_h instances - :param size: int number of workers in cluster - :param rank: int rank of current worker + + Parameters + ---------- + handle : raft.common.Handle + handle containing comms_t to use + nccl_inst : raft.dask.common.nccl + Initialized nccl comm to use + ucp_worker : size_t pointer to initialized ucp_worker_h instance + eps: size_t pointer to array of initialized ucp_ep_h instances + size : int + Number of workers in cluster + rank : int + Rank of current worker """ cdef size_t *ucp_eps = malloc(len(eps)*sizeof(size_t)) diff --git a/python/raft/dask/common/nccl.pyx b/python/raft/dask/common/nccl.pyx index c9d9fe0426..b72bd3d80b 100644 --- a/python/raft/dask/common/nccl.pyx +++ b/python/raft/dask/common/nccl.pyx @@ -1,5 +1,5 @@ # -# Copyright (c) 2019, NVIDIA CORPORATION. +# Copyright (c) 2020, NVIDIA CORPORATION. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. @@ -74,7 +74,10 @@ def unique_id(): Returns a new ncclUniqueId converted to a character array that can be safely serialized and shared to a remote worker. - :return: string a 128-byte unique id string + + Returns + ------- + 128-byte unique id : str """ cdef char *uid = malloc(NCCL_UNIQUE_ID_BYTES * sizeof(char)) get_unique_id(uid, NCCL_UNIQUE_ID_BYTES) @@ -108,16 +111,22 @@ cdef class nccl: def get_unique_id(): """ Returns a new nccl unique id - :return: string nccl unique id + + Returns + ------- + nccl unique id : str """ return unique_id() def init(self, nranks, commId, rank): """ Construct a nccl-py object - :param nranks: int size of clique - :param commId: string unique id from client - :param rank: int rank of current worker + + Parameters + ---------- + nranks : int size of clique + commId : string unique id from client + rank : int rank of current worker """ self.size = nranks self.rank = rank @@ -183,7 +192,10 @@ cdef class nccl: def cu_device(self): """ Get the device backing the underlying comm - :returns int device id + + Returns + ------- + device id : int """ cdef int *dev = malloc(sizeof(int)) @@ -204,7 +216,10 @@ cdef class nccl: def user_rank(self): """ Get the rank id of the current comm - :return: int rank + + Returns + ------- + rank : int """ cdef int *rank = malloc(sizeof(int)) @@ -228,6 +243,9 @@ cdef class nccl: """ Returns the underlying nccl comm in a size_t (similar to void*). This can be safely typecasted from size_t into ncclComm_t* - :return: size_t ncclComm_t instance + + Returns + ------- + ncclComm_t instance pointer : size_t """ return self.comm diff --git a/python/raft/dask/common/ucx.py b/python/raft/dask/common/ucx.py index 948e1433ee..ed45c6ce8f 100644 --- a/python/raft/dask/common/ucx.py +++ b/python/raft/dask/common/ucx.py @@ -24,7 +24,7 @@ class UCX: """ Singleton UCX context to encapsulate all interactions with the UCX-py API and guarantee only a single listener & endpoints are - created by cuML on a single process. + created by RAFT Comms on a single process. """ __instance = None diff --git a/python/raft/dask/common/utils.py b/python/raft/dask/common/utils.py index c92fbf3e47..fdb5acfb5d 100644 --- a/python/raft/dask/common/utils.py +++ b/python/raft/dask/common/utils.py @@ -1,4 +1,4 @@ -# Copyright (c) 2019, NVIDIA CORPORATION. +# Copyright (c) 2020, NVIDIA CORPORATION. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. From 6eb0325c8cef11a7e74c5f0ff02e955403fd7fe5 Mon Sep 17 00:00:00 2001 From: "Corey J. Nolet" Date: Wed, 13 May 2020 16:37:16 -0400 Subject: [PATCH 23/46] Removing the last of references to cuml --- cpp/include/raft/comms/ucp_helper.hpp | 2 +- python/raft/dask/common/comms_utils.pyx | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/cpp/include/raft/comms/ucp_helper.hpp b/cpp/include/raft/comms/ucp_helper.hpp index 06f687752f..09fc897a79 100644 --- a/cpp/include/raft/comms/ucp_helper.hpp +++ b/cpp/include/raft/comms/ucp_helper.hpp @@ -40,7 +40,7 @@ typedef ucs_status_ptr_t (*dlsym_recv)(ucp_worker_h, void *, size_t count, * Standard UCX request object that will be passed * around asynchronously. This object is really * opaque and the comms layer only cares that it - * has been completed. Because cuml comms do not + * has been completed. Because raft comms do not * initialize the ucx application context, it doesn't * own this object and thus it's important not to * modify this struct. diff --git a/python/raft/dask/common/comms_utils.pyx b/python/raft/dask/common/comms_utils.pyx index 90fd97671f..1825d385a7 100644 --- a/python/raft/dask/common/comms_utils.pyx +++ b/python/raft/dask/common/comms_utils.pyx @@ -88,7 +88,7 @@ def perform_test_comms_send_recv(handle, n_trials): def inject_comms_on_handle_coll_only(handle, nccl_inst, size, rank, verbose): """ - Given a handle and initialized nccl comm, creates a cumlCommunicator + Given a handle and initialized nccl comm, creates a comms_t instance and injects it into the handle. Parameters @@ -119,7 +119,7 @@ def inject_comms_on_handle_coll_only(handle, nccl_inst, size, rank, verbose): def inject_comms_on_handle(handle, nccl_inst, ucp_worker, eps, size, rank, verbose): """ - Given a handle and initialized comms, creates a cumlCommunicator instance + Given a handle and initialized comms, creates a comms_t instance and injects it into the handle. Parameters From b65e20df6603a00626e15df74326cf17070130a3 Mon Sep 17 00:00:00 2001 From: "Corey J. Nolet" Date: Wed, 13 May 2020 16:38:08 -0400 Subject: [PATCH 24/46] Fixing python style --- python/raft/common/handle.pyx | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/python/raft/common/handle.pyx b/python/raft/common/handle.pyx index 3659cb3669..c19cbc8faa 100644 --- a/python/raft/common/handle.pyx +++ b/python/raft/common/handle.pyx @@ -74,8 +74,8 @@ cdef class Handle: """ Issues a sync on the stream set for this handle. - Once we make `raft.common.cuda.Stream` as a mandatory option for creating - `raft.common.Handle`, this should go away + Once we make `raft.common.cuda.Stream` as a mandatory option + for creating `raft.common.Handle`, this should go away """ cdef handle_t* h_ = self.h cdef _Stream stream = h_.get_stream() From 7a845cb2952ac8d1477f691ac9394f51ea67492d Mon Sep 17 00:00:00 2001 From: "Corey J. Nolet" Date: Wed, 13 May 2020 16:38:39 -0400 Subject: [PATCH 25/46] Fixing c++ style --- cpp/include/raft/comms/ucp_helper.hpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/cpp/include/raft/comms/ucp_helper.hpp b/cpp/include/raft/comms/ucp_helper.hpp index 09fc897a79..ca9965aed1 100644 --- a/cpp/include/raft/comms/ucp_helper.hpp +++ b/cpp/include/raft/comms/ucp_helper.hpp @@ -227,5 +227,5 @@ class comms_ucp_handler { UCS_PTR_STATUS(recv_result)); } }; -} // end namespace comms -} // end namespace raft +} // end namespace comms +} // end namespace raft From c26930074426e5337c8920da100c47c6035222fe Mon Sep 17 00:00:00 2001 From: "Corey J. Nolet" Date: Wed, 13 May 2020 16:55:40 -0400 Subject: [PATCH 26/46] Updating changelog --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 241d214f49..d72239e926 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -4,6 +4,7 @@ ## New Features - Initial RAFT version - PR #3: defining raft::handle_t, device_buffer, host_buffer, allocator classes +- PR #7: Migrating cuml comms -> raft comms_t ## Improvements From d4aa5c5cc58ba8754fa8d61bb95132b572c223cd Mon Sep 17 00:00:00 2001 From: "Corey J. Nolet" Date: Wed, 13 May 2020 17:06:53 -0400 Subject: [PATCH 27/46] Testing non-ucx cluster for pytests --- python/raft/test/test_comms.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/python/raft/test/test_comms.py b/python/raft/test/test_comms.py index c95bb86b65..9fb735c361 100644 --- a/python/raft/test/test_comms.py +++ b/python/raft/test/test_comms.py @@ -103,9 +103,9 @@ def test_allreduce(cluster): @pytest.mark.ucx @pytest.mark.parametrize("n_trials", [1, 5]) -def test_send_recv(n_trials, ucx_cluster): +def test_send_recv(n_trials, cluster): - client = Client(ucx_cluster) + client = Client(cluster) try: From 1ee1363b3740bcba1570812773580f9be575d483 Mon Sep 17 00:00:00 2001 From: "Corey J. Nolet" Date: Mon, 18 May 2020 19:54:09 -0400 Subject: [PATCH 28/46] Implementing review feedback --- cpp/include/raft/comms/comms.hpp | 74 ++++----- cpp/include/raft/comms/comms_helper.hpp | 25 +++- cpp/include/raft/comms/std_comms.hpp | 190 ++++++++++++------------ cpp/include/raft/comms/ucp_helper.hpp | 4 +- 4 files changed, 154 insertions(+), 139 deletions(-) diff --git a/cpp/include/raft/comms/comms.hpp b/cpp/include/raft/comms/comms.hpp index 3a61207214..4d97596170 100644 --- a/cpp/include/raft/comms/comms.hpp +++ b/cpp/include/raft/comms/comms.hpp @@ -22,13 +22,13 @@ namespace raft { namespace comms { typedef unsigned int request_t; -enum datatype_t { CHAR, UINT8, INT, UINT, INT64, UINT64, FLOAT, DOUBLE }; -enum op_t { SUM, PROD, MIN, MAX }; +enum class datatype_t { CHAR, UINT8, INT32, UINT32, INT64, UINT64, FLOAT32, FLOAT64 }; +enum class op_t { SUM, PROD, MIN, MAX }; /** * The resulting status of distributed stream synchronization */ -enum status_t { +enum class status_t { commStatusSuccess, // Synchronization successful commStatusError, // An error occured querying sync status commStatusAbort // A failure occurred in sync, queued operations aborted @@ -47,105 +47,105 @@ class comms_iface { virtual status_t syncStream(cudaStream_t stream) const = 0; - virtual void isend(const void* buf, int size, int dest, int tag, + virtual void isend(const void* buf, size_t size, int dest, int tag, request_t* request) const = 0; - virtual void irecv(void* buf, int size, int source, int tag, + virtual void irecv(void* buf, size_t size, int source, int tag, request_t* request) const = 0; virtual void waitall(int count, request_t array_of_requests[]) const = 0; - virtual void allreduce(const void* sendbuff, void* recvbuff, int count, + virtual void allreduce(const void* sendbuff, void* recvbuff, size_t count, datatype_t datatype, op_t op, cudaStream_t stream) const = 0; - virtual void bcast(void* buff, int count, datatype_t datatype, int root, + virtual void bcast(void* buff, size_t count, datatype_t datatype, int root, cudaStream_t stream) const = 0; - virtual void reduce(const void* sendbuff, void* recvbuff, int count, + virtual void reduce(const void* sendbuff, void* recvbuff, size_t count, datatype_t datatype, op_t op, int root, cudaStream_t stream) const = 0; - virtual void allgather(const void* sendbuff, void* recvbuff, int sendcount, + virtual void allgather(const void* sendbuff, void* recvbuff, size_t sendcount, datatype_t datatype, cudaStream_t stream) const = 0; virtual void allgatherv(const void* sendbuf, void* recvbuf, - const int recvcounts[], const int displs[], + const size_t recvcounts[], const int displs[], datatype_t datatype, cudaStream_t stream) const = 0; virtual void reducescatter(const void* sendbuff, void* recvbuff, - int recvcount, datatype_t datatype, op_t op, + size_t recvcount, datatype_t datatype, op_t op, cudaStream_t stream) const = 0; }; -class comms_t : public comms_iface { +class comms_t: comms_iface { public: - comms_t(std::unique_ptr impl) : _impl(impl.release()) { - ASSERT(nullptr != _impl.get(), "ERROR: Invalid comms_iface used!"); + comms_t(std::unique_ptr impl) : impl_(impl.release()) { + ASSERT(nullptr != impl.get(), "ERROR: Invalid comms_iface used!"); } - int getSize() const { return _impl->getSize(); } + int getSize() const { return impl_->getSize(); } - int getRank() const { return _impl->getRank(); } + int getRank() const { return impl_->getRank(); } std::unique_ptr commSplit(int color, int key) const { - return _impl->commSplit(color, key); + return impl_->commSplit(color, key); } - void barrier() const { _impl->barrier(); } + void barrier() const { impl_->barrier(); } status_t syncStream(cudaStream_t stream) const { - return _impl->syncStream(stream); + return impl_->syncStream(stream); } - void isend(const void* buf, int size, int dest, int tag, + void isend(const void* buf, size_t size, int dest, int tag, request_t* request) const { - _impl->isend(buf, size, dest, tag, request); + impl_->isend(buf, size, dest, tag, request); } - void irecv(void* buf, int size, int source, int tag, + void irecv(void* buf, size_t size, int source, int tag, request_t* request) const { - _impl->irecv(buf, size, source, tag, request); + impl_->irecv(buf, size, source, tag, request); } void waitall(int count, request_t array_of_requests[]) const { - _impl->waitall(count, array_of_requests); + impl_->waitall(count, array_of_requests); } - void allreduce(const void* sendbuff, void* recvbuff, int count, + void allreduce(const void* sendbuff, void* recvbuff, size_t count, datatype_t datatype, op_t op, cudaStream_t stream) const { - _impl->allreduce(sendbuff, recvbuff, count, datatype, op, stream); + impl_->allreduce(sendbuff, recvbuff, count, datatype, op, stream); } - void bcast(void* buff, int count, datatype_t datatype, int root, + void bcast(void* buff, size_t count, datatype_t datatype, int root, cudaStream_t stream) const { - _impl->bcast(buff, count, datatype, root, stream); + impl_->bcast(buff, count, datatype, root, stream); } - void reduce(const void* sendbuff, void* recvbuff, int count, + void reduce(const void* sendbuff, void* recvbuff, size_t count, datatype_t datatype, op_t op, int root, cudaStream_t stream) const { - _impl->reduce(sendbuff, recvbuff, count, datatype, op, root, stream); + impl_->reduce(sendbuff, recvbuff, count, datatype, op, root, stream); } - void allgather(const void* sendbuff, void* recvbuff, int sendcount, + void allgather(const void* sendbuff, void* recvbuff, size_t sendcount, datatype_t datatype, cudaStream_t stream) const { - _impl->allgather(sendbuff, recvbuff, sendcount, datatype, stream); + impl_->allgather(sendbuff, recvbuff, sendcount, datatype, stream); } - void allgatherv(const void* sendbuf, void* recvbuf, const int recvcounts[], + void allgatherv(const void* sendbuf, void* recvbuf, const size_t recvcounts[], const int displs[], datatype_t datatype, cudaStream_t stream) const { - _impl->allgatherv(sendbuf, recvbuf, recvcounts, displs, datatype, stream); + impl_->allgatherv(sendbuf, recvbuf, recvcounts, displs, datatype, stream); } - void reducescatter(const void* sendbuff, void* recvbuff, int recvcount, + void reducescatter(const void* sendbuff, void* recvbuff, size_t recvcount, datatype_t datatype, op_t op, cudaStream_t stream) const { - _impl->reducescatter(sendbuff, recvbuff, recvcount, datatype, op, stream); + impl_->reducescatter(sendbuff, recvbuff, recvcount, datatype, op, stream); } private: - std::unique_ptr _impl; + std::unique_ptr impl_; }; comms_iface::~comms_iface() {} diff --git a/cpp/include/raft/comms/comms_helper.hpp b/cpp/include/raft/comms/comms_helper.hpp index 2e71009eaf..1f0e772a5e 100644 --- a/cpp/include/raft/comms/comms_helper.hpp +++ b/cpp/include/raft/comms/comms_helper.hpp @@ -24,35 +24,46 @@ namespace raft { namespace comms { +/** + * Function to construct comms_t and inject it on a handle_t. This + * is used for convenience in the Python layer. + */ void build_comms_nccl_only(handle_t *handle, ncclComm_t comm, int size, int rank) { - auto *raft_comm = new raft::comms::std_comms(comm, size, rank); + + auto d_alloc = handle->get_device_allocator(); + auto *raft_comm = new raft::comms::std_comms(comm, size, rank, d_alloc); auto communicator = std::make_shared(std::unique_ptr(raft_comm)); handle->set_comms(communicator); } +/** + * Function to construct comms_t and inject it on a handle_t. This + * is used for convenience in the Python layer. + */ void build_comms_nccl_ucx(handle_t *handle, ncclComm_t comm, void *ucp_worker, void *eps, int size, int rank) { - std::shared_ptr eps_sp = + auto eps_sp = std::make_shared(new ucp_ep_h[size]); - size_t *size_t_ep_arr = (size_t *)eps; + auto size_t_ep_arr = reinterpret_cast(eps); for (int i = 0; i < size; i++) { size_t ptr = size_t_ep_arr[i]; - ucp_ep_h *ucp_ep_v = (ucp_ep_h *)*eps_sp; + auto ucp_ep_v = reinterpret_cast(*eps_sp); if (ptr != 0) { - ucp_ep_h eps_ptr = (ucp_ep_h)size_t_ep_arr[i]; + auto eps_ptr = reinterpret_cast(size_t_ep_arr[i]); ucp_ep_v[i] = eps_ptr; } else { ucp_ep_v[i] = nullptr; } } + auto d_alloc = handle->get_device_allocator(); auto *raft_comm = new raft::comms::std_comms(comm, (ucp_worker_h)ucp_worker, - eps_sp, size, rank); + eps_sp, size, rank, d_alloc); auto communicator = std::make_shared(std::unique_ptr(raft_comm)); handle->set_comms(communicator); @@ -69,7 +80,7 @@ bool test_collective_allreduce(const handle_t &handle) { temp_d.resize(1, stream); CUDA_CHECK(cudaMemcpyAsync(temp_d.data(), &send, sizeof(int), cudaMemcpyHostToDevice, stream)); - communicator.allreduce(temp_d.data(), temp_d.data(), 1, datatype_t::INT, SUM, + communicator.allreduce(temp_d.data(), temp_d.data(), 1, datatype_t::INT32, op_t::SUM, stream); int temp_h = 0; CUDA_CHECK(cudaMemcpyAsync(&temp_h, temp_d.data(), sizeof(int), diff --git a/cpp/include/raft/comms/std_comms.hpp b/cpp/include/raft/comms/std_comms.hpp index 2af64a262b..9b6dd91237 100644 --- a/cpp/include/raft/comms/std_comms.hpp +++ b/cpp/include/raft/comms/std_comms.hpp @@ -66,55 +66,55 @@ namespace comms { size_t getDatatypeSize(const datatype_t datatype) { switch (datatype) { - case CHAR: + case datatype_t::CHAR: return sizeof(char); - case UINT8: + case datatype_t::UINT8: return sizeof(uint8_t); - case INT: + case datatype_t::INT32: return sizeof(int); - case UINT: + case datatype_t::UINT32: return sizeof(unsigned int); - case INT64: + case datatype_t::INT64: return sizeof(int64_t); - case UINT64: + case datatype_t::UINT64: return sizeof(uint64_t); - case FLOAT: + case datatype_t::FLOAT32: return sizeof(float); - case DOUBLE: + case datatype_t::FLOAT64: return sizeof(double); } } ncclDataType_t getNCCLDatatype(const datatype_t datatype) { switch (datatype) { - case CHAR: + case datatype_t::CHAR: return ncclChar; - case UINT8: + case datatype_t::UINT8: return ncclUint8; - case INT: + case datatype_t::INT32: return ncclInt; - case UINT: + case datatype_t::UINT32: return ncclUint32; - case INT64: + case datatype_t::INT64: return ncclInt64; - case UINT64: + case datatype_t::UINT64: return ncclUint64; - case FLOAT: + case datatype_t::FLOAT32: return ncclFloat; - case DOUBLE: + case datatype_t::FLOAT64: return ncclDouble; } } ncclRedOp_t getNCCLOp(const op_t op) { switch (op) { - case SUM: + case op_t::SUM: return ncclSum; - case PROD: + case op_t::PROD: return ncclProd; - case MIN: + case op_t::MIN: return ncclMin; - case MAX: + case op_t::MAX: return ncclMax; } } @@ -131,16 +131,17 @@ class std_comms : public comms_iface { * @param size size of the cluster * @param rank rank of the current worker */ - std_comms(ncclComm_t comm, ucp_worker_h ucp_worker, - std::shared_ptr eps, int size, int rank) - : _nccl_comm(comm), - _ucp_worker(ucp_worker), - _ucp_eps(eps), - _size(size), - _rank(rank), - _next_request_id(0) { + std_comms(ncclComm_t nccl_comm, ucp_worker_h ucp_worker, + std::shared_ptr eps, int num_ranks, int rank, + const std::shared_ptr device_allocator) + : nccl_comm_(nccl_comm), + ucp_worker_(ucp_worker), + ucp_eps_(eps), + num_ranks_(num_ranks), + rank_(rank), + device_allocator_(device_allocator), + next_request_id_(0) { initialize(); - p2p_enabled = true; }; /** @@ -149,28 +150,30 @@ class std_comms : public comms_iface { * @param size size of the cluster * @param rank rank of the current worker */ - std_comms(ncclComm_t comm, int size, int rank) - : _nccl_comm(comm), _size(size), _rank(rank) { + std_comms(const ncclComm_t nccl_comm, int num_ranks, int rank, + const std::shared_ptr device_allocator) + : nccl_comm_(nccl_comm), num_ranks_(num_ranks), rank_(rank), + device_allocator_(device_allocator){ initialize(); }; virtual ~std_comms() { - CUDA_CHECK_NO_THROW(cudaStreamDestroy(_stream)); + CUDA_CHECK_NO_THROW(cudaStreamDestroy(stream_)); - CUDA_CHECK_NO_THROW(cudaFree(_sendbuff)); - CUDA_CHECK_NO_THROW(cudaFree(_recvbuff)); + device_allocator_->deallocate(sendbuff_, sizeof(int), stream_); + device_allocator_->deallocate(recvbuff_, sizeof(int), stream_); } void initialize() { - CUDA_CHECK(cudaStreamCreate(&_stream)); + CUDA_CHECK(cudaStreamCreate(&stream_)); - CUDA_CHECK(cudaMalloc(&_sendbuff, sizeof(int))); - CUDA_CHECK(cudaMalloc(&_recvbuff, sizeof(int))); + sendbuff_ = reinterpret_cast(device_allocator_->allocate(sizeof(int), stream_)); + recvbuff_ = reinterpret_cast(device_allocator_->allocate(sizeof(int), stream_)); } - int getSize() const { return _size; } + int getSize() const { return num_ranks_; } - int getRank() const { return _rank; } + int getRank() const { return rank_; } std::unique_ptr commSplit(int color, int key) const { // Not supported by NCCL @@ -180,64 +183,64 @@ class std_comms : public comms_iface { } void barrier() const { - CUDA_CHECK(cudaMemsetAsync(_sendbuff, 1, sizeof(int), _stream)); - CUDA_CHECK(cudaMemsetAsync(_recvbuff, 1, sizeof(int), _stream)); + CUDA_CHECK(cudaMemsetAsync(sendbuff_, 1, sizeof(int), stream_)); + CUDA_CHECK(cudaMemsetAsync(recvbuff_, 1, sizeof(int), stream_)); - allreduce(_sendbuff, _recvbuff, 1, INT, SUM, _stream); + allreduce(sendbuff_, recvbuff_, 1, datatype_t::INT32, op_t::SUM, stream_); - ASSERT(syncStream(_stream) == status_t::commStatusSuccess, - "ERROR: syncStream failed. This can be caused by a failed rank."); + ASSERT(syncStream(stream_) == status_t::commStatusSuccess, + "ERROR: syncStream failed. This can be caused by a failed rank_."); } void get_request_id(request_t *req) const { request_t req_id; - if (this->_free_requests.empty()) - req_id = this->_next_request_id++; + if (this->free_requests_.empty()) + req_id = this->next_request_id_++; else { - auto it = this->_free_requests.begin(); + auto it = this->free_requests_.begin(); req_id = *it; - this->_free_requests.erase(it); + this->free_requests_.erase(it); } *req = req_id; } - void isend(const void *buf, int size, int dest, int tag, + void isend(const void *buf, size_t size, int dest, int tag, request_t *request) const { - ASSERT(_ucp_worker != nullptr, + ASSERT(ucp_worker_ != nullptr, "ERROR: UCX comms not initialized on communicator."); get_request_id(request); - ucp_ep_h ep_ptr = (*_ucp_eps)[dest]; + ucp_ep_h ep_ptr = (*ucp_eps_)[dest]; ucp_request *ucp_req = (ucp_request *)malloc(sizeof(ucp_request)); - this->_ucp_handler.ucp_isend(ucp_req, ep_ptr, buf, size, tag, + this->ucp_handler_.ucp_isend(ucp_req, ep_ptr, buf, size, tag, default_tag_mask, getRank()); - _requests_in_flight.insert(std::make_pair(*request, ucp_req)); + requests_in_flight_.insert(std::make_pair(*request, ucp_req)); } - void irecv(void *buf, int size, int source, int tag, + void irecv(void *buf, size_t size, int source, int tag, request_t *request) const { - ASSERT(_ucp_worker != nullptr, + ASSERT(ucp_worker_ != nullptr, "ERROR: UCX comms not initialized on communicator."); get_request_id(request); - ucp_ep_h ep_ptr = (*_ucp_eps)[source]; + ucp_ep_h ep_ptr = (*ucp_eps_)[source]; ucp_tag_t tag_mask = default_tag_mask; ucp_request *ucp_req = (ucp_request *)malloc(sizeof(ucp_request)); - _ucp_handler.ucp_irecv(ucp_req, _ucp_worker, ep_ptr, buf, size, tag, + ucp_handler_.ucp_irecv(ucp_req, ucp_worker_, ep_ptr, buf, size, tag, tag_mask, source); - _requests_in_flight.insert(std::make_pair(*request, ucp_req)); + requests_in_flight_.insert(std::make_pair(*request, ucp_req)); } void waitall(int count, request_t array_of_requests[]) const { - ASSERT(_ucp_worker != nullptr, + ASSERT(ucp_worker_ != nullptr, "ERROR: UCX comms not initialized on communicator."); std::vector requests; @@ -246,12 +249,12 @@ class std_comms : public comms_iface { time_t start = time(NULL); for (int i = 0; i < count; ++i) { - auto req_it = _requests_in_flight.find(array_of_requests[i]); - ASSERT(_requests_in_flight.end() != req_it, + auto req_it = requests_in_flight_.find(array_of_requests[i]); + ASSERT(requests_in_flight_.end() != req_it, "ERROR: waitall on invalid request: %d", array_of_requests[i]); requests.push_back(req_it->second); - _free_requests.insert(req_it->first); - _requests_in_flight.erase(req_it); + free_requests_.insert(req_it->first); + requests_in_flight_.erase(req_it); } while (requests.size() > 0) { @@ -266,7 +269,7 @@ class std_comms : public comms_iface { bool restart = false; // resets the timeout when any progress was made // Causes UCP to progress through the send/recv message queue - while (_ucp_handler.ucp_progress(_ucp_worker) != 0) { + while (ucp_handler_.ucp_progress(ucp_worker_) != 0) { restart = true; } @@ -291,7 +294,7 @@ class std_comms : public comms_iface { restart = true; // perform cleanup - _ucp_handler.free_ucp_request(req); + ucp_handler_.free_ucp_request(req); // remove from pending requests it = requests.erase(it); @@ -306,50 +309,50 @@ class std_comms : public comms_iface { } } - void allreduce(const void *sendbuff, void *recvbuff, int count, + void allreduce(const void *sendbuff, void *recvbuff, size_t count, datatype_t datatype, op_t op, cudaStream_t stream) const { NCCL_CHECK(ncclAllReduce(sendbuff, recvbuff, count, getNCCLDatatype(datatype), getNCCLOp(op), - _nccl_comm, stream)); + nccl_comm_, stream)); } - void bcast(void *buff, int count, datatype_t datatype, int root, + void bcast(void *buff, size_t count, datatype_t datatype, int root, cudaStream_t stream) const { NCCL_CHECK(ncclBroadcast(buff, buff, count, getNCCLDatatype(datatype), root, - _nccl_comm, stream)); + nccl_comm_, stream)); } - void reduce(const void *sendbuff, void *recvbuff, int count, + void reduce(const void *sendbuff, void *recvbuff, size_t count, datatype_t datatype, op_t op, int root, cudaStream_t stream) const { NCCL_CHECK(ncclReduce(sendbuff, recvbuff, count, getNCCLDatatype(datatype), - getNCCLOp(op), root, _nccl_comm, stream)); + getNCCLOp(op), root, nccl_comm_, stream)); } - void allgather(const void *sendbuff, void *recvbuff, int sendcount, + void allgather(const void *sendbuff, void *recvbuff, size_t sendcount, datatype_t datatype, cudaStream_t stream) const { NCCL_CHECK(ncclAllGather(sendbuff, recvbuff, sendcount, - getNCCLDatatype(datatype), _nccl_comm, stream)); + getNCCLDatatype(datatype), nccl_comm_, stream)); } - void allgatherv(const void *sendbuf, void *recvbuf, const int recvcounts[], + void allgatherv(const void *sendbuf, void *recvbuf, const size_t recvcounts[], const int displs[], datatype_t datatype, cudaStream_t stream) const { //From: "An Empirical Evaluation of Allgatherv on Multi-GPU Systems" - https://arxiv.org/pdf/1812.05964.pdf //Listing 1 on page 4. - for (int root = 0; root < _size; ++root) { + for (int root = 0; root < num_ranks_; ++root) { size_t dtype_size = getDatatypeSize(datatype); NCCL_CHECK(ncclBroadcast( sendbuf, static_cast(recvbuf) + displs[root] * dtype_size, - recvcounts[root], getNCCLDatatype(datatype), root, _nccl_comm, stream)); + recvcounts[root], getNCCLDatatype(datatype), root, nccl_comm_, stream)); } } - void reducescatter(const void *sendbuff, void *recvbuff, int recvcount, + void reducescatter(const void *sendbuff, void *recvbuff, size_t recvcount, datatype_t datatype, op_t op, cudaStream_t stream) const { NCCL_CHECK(ncclReduceScatter(sendbuff, recvbuff, recvcount, getNCCLDatatype(datatype), getNCCLOp(op), - _nccl_comm, stream)); + nccl_comm_, stream)); } status_t syncStream(cudaStream_t stream) const { @@ -360,11 +363,11 @@ class std_comms : public comms_iface { if (cudaErr == cudaSuccess) return status_t::commStatusSuccess; if (cudaErr != cudaErrorNotReady) { - // An error occurred querying the status of the stream + // An error occurred querying the status of the stream_ return status_t::commStatusError; } - ncclErr = ncclCommGetAsyncError(_nccl_comm, &ncclAsyncErr); + ncclErr = ncclCommGetAsyncError(nccl_comm_, &ncclAsyncErr); if (ncclErr != ncclSuccess) { // An error occurred retrieving the asynchronous error return status_t::commStatusError; @@ -373,7 +376,7 @@ class std_comms : public comms_iface { if (ncclAsyncErr != ncclSuccess) { // An asynchronous error happened. Stop the operation and destroy // the communicator - ncclErr = ncclCommAbort(_nccl_comm); + ncclErr = ncclCommAbort(nccl_comm_); if (ncclErr != ncclSuccess) // Caller may abort with an exception or try to re-create a new communicator. return status_t::commStatusAbort; @@ -385,22 +388,23 @@ class std_comms : public comms_iface { } private: - ncclComm_t _nccl_comm; - cudaStream_t _stream; + ncclComm_t nccl_comm_; + cudaStream_t stream_; - int *_sendbuff, *_recvbuff; + int *sendbuff_, *recvbuff_; - int _size; - int _rank; + int num_ranks_; + int rank_; - bool p2p_enabled = false; - comms_ucp_handler _ucp_handler; - ucp_worker_h _ucp_worker; - std::shared_ptr _ucp_eps; - mutable request_t _next_request_id; + comms_ucp_handler ucp_handler_; + ucp_worker_h ucp_worker_; + std::shared_ptr ucp_eps_; + mutable request_t next_request_id_; mutable std::unordered_map - _requests_in_flight; - mutable std::unordered_set _free_requests; + requests_in_flight_; + mutable std::unordered_set free_requests_; + + std::shared_ptr device_allocator_; }; } // end namespace comms } // end namespace raft diff --git a/cpp/include/raft/comms/ucp_helper.hpp b/cpp/include/raft/comms/ucp_helper.hpp index ca9965aed1..47f3f4d61e 100644 --- a/cpp/include/raft/comms/ucp_helper.hpp +++ b/cpp/include/raft/comms/ucp_helper.hpp @@ -172,7 +172,7 @@ class comms_ucp_handler { /** * @brief Asynchronously send data to the given endpoint using the given tag */ - void ucp_isend(ucp_request *req, ucp_ep_h ep_ptr, const void *buf, int size, + void ucp_isend(ucp_request *req, ucp_ep_h ep_ptr, const void *buf, size_t size, int tag, ucp_tag_t tag_mask, int rank) const { ucp_tag_t ucp_tag = build_message_tag(rank, tag); @@ -207,7 +207,7 @@ class comms_ucp_handler { * @brief Asynchronously receive data from given endpoint with the given tag. */ void ucp_irecv(ucp_request *req, ucp_worker_h worker, ucp_ep_h ep_ptr, - void *buf, int size, int tag, ucp_tag_t tag_mask, + void *buf, size_t size, int tag, ucp_tag_t tag_mask, int sender_rank) const { ucp_tag_t ucp_tag = build_message_tag(sender_rank, tag); From c165a36bff343c1c9cd035e7663d5abb30eb145c Mon Sep 17 00:00:00 2001 From: "Corey J. Nolet" Date: Mon, 18 May 2020 20:08:45 -0400 Subject: [PATCH 29/46] More review feedback --- cpp/include/raft/comms/comms.hpp | 2 +- cpp/include/raft/comms/std_comms.hpp | 23 ++++++++++++----------- 2 files changed, 13 insertions(+), 12 deletions(-) diff --git a/cpp/include/raft/comms/comms.hpp b/cpp/include/raft/comms/comms.hpp index 4d97596170..54d11f38ba 100644 --- a/cpp/include/raft/comms/comms.hpp +++ b/cpp/include/raft/comms/comms.hpp @@ -78,7 +78,7 @@ class comms_iface { cudaStream_t stream) const = 0; }; -class comms_t: comms_iface { +class comms_t { public: comms_t(std::unique_ptr impl) : impl_(impl.release()) { ASSERT(nullptr != impl.get(), "ERROR: Invalid comms_iface used!"); diff --git a/cpp/include/raft/comms/std_comms.hpp b/cpp/include/raft/comms/std_comms.hpp index 9b6dd91237..7a49816242 100644 --- a/cpp/include/raft/comms/std_comms.hpp +++ b/cpp/include/raft/comms/std_comms.hpp @@ -64,7 +64,8 @@ namespace raft { namespace comms { -size_t getDatatypeSize(const datatype_t datatype) { + +constexpr size_t get_datatype_size(const datatype_t datatype) { switch (datatype) { case datatype_t::CHAR: return sizeof(char); @@ -85,7 +86,7 @@ size_t getDatatypeSize(const datatype_t datatype) { } } -ncclDataType_t getNCCLDatatype(const datatype_t datatype) { +ncclDataType_t get_nccl_datatype(const datatype_t datatype) { switch (datatype) { case datatype_t::CHAR: return ncclChar; @@ -106,7 +107,7 @@ ncclDataType_t getNCCLDatatype(const datatype_t datatype) { } } -ncclRedOp_t getNCCLOp(const op_t op) { +ncclRedOp_t get_nccl_op(const op_t op) { switch (op) { case op_t::SUM: return ncclSum; @@ -312,27 +313,27 @@ class std_comms : public comms_iface { void allreduce(const void *sendbuff, void *recvbuff, size_t count, datatype_t datatype, op_t op, cudaStream_t stream) const { NCCL_CHECK(ncclAllReduce(sendbuff, recvbuff, count, - getNCCLDatatype(datatype), getNCCLOp(op), + get_nccl_datatype(datatype), get_nccl_op(op), nccl_comm_, stream)); } void bcast(void *buff, size_t count, datatype_t datatype, int root, cudaStream_t stream) const { - NCCL_CHECK(ncclBroadcast(buff, buff, count, getNCCLDatatype(datatype), root, + NCCL_CHECK(ncclBroadcast(buff, buff, count, get_nccl_datatype(datatype), root, nccl_comm_, stream)); } void reduce(const void *sendbuff, void *recvbuff, size_t count, datatype_t datatype, op_t op, int root, cudaStream_t stream) const { - NCCL_CHECK(ncclReduce(sendbuff, recvbuff, count, getNCCLDatatype(datatype), - getNCCLOp(op), root, nccl_comm_, stream)); + NCCL_CHECK(ncclReduce(sendbuff, recvbuff, count, get_nccl_datatype(datatype), + get_nccl_op(op), root, nccl_comm_, stream)); } void allgather(const void *sendbuff, void *recvbuff, size_t sendcount, datatype_t datatype, cudaStream_t stream) const { NCCL_CHECK(ncclAllGather(sendbuff, recvbuff, sendcount, - getNCCLDatatype(datatype), nccl_comm_, stream)); + get_nccl_datatype(datatype), nccl_comm_, stream)); } void allgatherv(const void *sendbuf, void *recvbuf, const size_t recvcounts[], @@ -341,17 +342,17 @@ class std_comms : public comms_iface { //From: "An Empirical Evaluation of Allgatherv on Multi-GPU Systems" - https://arxiv.org/pdf/1812.05964.pdf //Listing 1 on page 4. for (int root = 0; root < num_ranks_; ++root) { - size_t dtype_size = getDatatypeSize(datatype); + size_t dtype_size = get_datatype_size(datatype); NCCL_CHECK(ncclBroadcast( sendbuf, static_cast(recvbuf) + displs[root] * dtype_size, - recvcounts[root], getNCCLDatatype(datatype), root, nccl_comm_, stream)); + recvcounts[root], get_nccl_datatype(datatype), root, nccl_comm_, stream)); } } void reducescatter(const void *sendbuff, void *recvbuff, size_t recvcount, datatype_t datatype, op_t op, cudaStream_t stream) const { NCCL_CHECK(ncclReduceScatter(sendbuff, recvbuff, recvcount, - getNCCLDatatype(datatype), getNCCLOp(op), + get_nccl_datatype(datatype), get_nccl_op(op), nccl_comm_, stream)); } From 12f3db7b5a75ef9bff3e9f8c19f864128ef7f59d Mon Sep 17 00:00:00 2001 From: "Corey J. Nolet" Date: Mon, 18 May 2020 20:14:43 -0400 Subject: [PATCH 30/46] Fixing style --- cpp/include/raft/comms/comms.hpp | 13 ++++++++++-- cpp/include/raft/comms/comms_helper.hpp | 10 ++++----- cpp/include/raft/comms/std_comms.hpp | 27 +++++++++++++++---------- cpp/include/raft/comms/ucp_helper.hpp | 4 ++-- 4 files changed, 33 insertions(+), 21 deletions(-) diff --git a/cpp/include/raft/comms/comms.hpp b/cpp/include/raft/comms/comms.hpp index 54d11f38ba..11ad32cfcc 100644 --- a/cpp/include/raft/comms/comms.hpp +++ b/cpp/include/raft/comms/comms.hpp @@ -22,7 +22,16 @@ namespace raft { namespace comms { typedef unsigned int request_t; -enum class datatype_t { CHAR, UINT8, INT32, UINT32, INT64, UINT64, FLOAT32, FLOAT64 }; +enum class datatype_t { + CHAR, + UINT8, + INT32, + UINT32, + INT64, + UINT64, + FLOAT32, + FLOAT64 +}; enum class op_t { SUM, PROD, MIN, MAX }; /** @@ -74,7 +83,7 @@ class comms_iface { datatype_t datatype, cudaStream_t stream) const = 0; virtual void reducescatter(const void* sendbuff, void* recvbuff, - size_t recvcount, datatype_t datatype, op_t op, + size_t recvcount, datatype_t datatype, op_t op, cudaStream_t stream) const = 0; }; diff --git a/cpp/include/raft/comms/comms_helper.hpp b/cpp/include/raft/comms/comms_helper.hpp index 1f0e772a5e..2238226383 100644 --- a/cpp/include/raft/comms/comms_helper.hpp +++ b/cpp/include/raft/comms/comms_helper.hpp @@ -30,7 +30,6 @@ namespace comms { */ void build_comms_nccl_only(handle_t *handle, ncclComm_t comm, int size, int rank) { - auto d_alloc = handle->get_device_allocator(); auto *raft_comm = new raft::comms::std_comms(comm, size, rank, d_alloc); auto communicator = @@ -44,10 +43,9 @@ void build_comms_nccl_only(handle_t *handle, ncclComm_t comm, int size, */ void build_comms_nccl_ucx(handle_t *handle, ncclComm_t comm, void *ucp_worker, void *eps, int size, int rank) { - auto eps_sp = - std::make_shared(new ucp_ep_h[size]); + auto eps_sp = std::make_shared(new ucp_ep_h[size]); - auto size_t_ep_arr = reinterpret_cast(eps); + auto size_t_ep_arr = reinterpret_cast(eps); for (int i = 0; i < size; i++) { size_t ptr = size_t_ep_arr[i]; @@ -80,8 +78,8 @@ bool test_collective_allreduce(const handle_t &handle) { temp_d.resize(1, stream); CUDA_CHECK(cudaMemcpyAsync(temp_d.data(), &send, sizeof(int), cudaMemcpyHostToDevice, stream)); - communicator.allreduce(temp_d.data(), temp_d.data(), 1, datatype_t::INT32, op_t::SUM, - stream); + communicator.allreduce(temp_d.data(), temp_d.data(), 1, datatype_t::INT32, + op_t::SUM, stream); int temp_h = 0; CUDA_CHECK(cudaMemcpyAsync(&temp_h, temp_d.data(), sizeof(int), cudaMemcpyDeviceToHost, stream)); diff --git a/cpp/include/raft/comms/std_comms.hpp b/cpp/include/raft/comms/std_comms.hpp index 7a49816242..e806740207 100644 --- a/cpp/include/raft/comms/std_comms.hpp +++ b/cpp/include/raft/comms/std_comms.hpp @@ -64,7 +64,6 @@ namespace raft { namespace comms { - constexpr size_t get_datatype_size(const datatype_t datatype) { switch (datatype) { case datatype_t::CHAR: @@ -152,9 +151,11 @@ class std_comms : public comms_iface { * @param rank rank of the current worker */ std_comms(const ncclComm_t nccl_comm, int num_ranks, int rank, - const std::shared_ptr device_allocator) - : nccl_comm_(nccl_comm), num_ranks_(num_ranks), rank_(rank), - device_allocator_(device_allocator){ + const std::shared_ptr device_allocator) + : nccl_comm_(nccl_comm), + num_ranks_(num_ranks), + rank_(rank), + device_allocator_(device_allocator) { initialize(); }; @@ -168,8 +169,10 @@ class std_comms : public comms_iface { void initialize() { CUDA_CHECK(cudaStreamCreate(&stream_)); - sendbuff_ = reinterpret_cast(device_allocator_->allocate(sizeof(int), stream_)); - recvbuff_ = reinterpret_cast(device_allocator_->allocate(sizeof(int), stream_)); + sendbuff_ = reinterpret_cast( + device_allocator_->allocate(sizeof(int), stream_)); + recvbuff_ = reinterpret_cast( + device_allocator_->allocate(sizeof(int), stream_)); } int getSize() const { return num_ranks_; } @@ -319,15 +322,16 @@ class std_comms : public comms_iface { void bcast(void *buff, size_t count, datatype_t datatype, int root, cudaStream_t stream) const { - NCCL_CHECK(ncclBroadcast(buff, buff, count, get_nccl_datatype(datatype), root, - nccl_comm_, stream)); + NCCL_CHECK(ncclBroadcast(buff, buff, count, get_nccl_datatype(datatype), + root, nccl_comm_, stream)); } void reduce(const void *sendbuff, void *recvbuff, size_t count, datatype_t datatype, op_t op, int root, cudaStream_t stream) const { - NCCL_CHECK(ncclReduce(sendbuff, recvbuff, count, get_nccl_datatype(datatype), - get_nccl_op(op), root, nccl_comm_, stream)); + NCCL_CHECK(ncclReduce(sendbuff, recvbuff, count, + get_nccl_datatype(datatype), get_nccl_op(op), root, + nccl_comm_, stream)); } void allgather(const void *sendbuff, void *recvbuff, size_t sendcount, @@ -345,7 +349,8 @@ class std_comms : public comms_iface { size_t dtype_size = get_datatype_size(datatype); NCCL_CHECK(ncclBroadcast( sendbuf, static_cast(recvbuf) + displs[root] * dtype_size, - recvcounts[root], get_nccl_datatype(datatype), root, nccl_comm_, stream)); + recvcounts[root], get_nccl_datatype(datatype), root, nccl_comm_, + stream)); } } diff --git a/cpp/include/raft/comms/ucp_helper.hpp b/cpp/include/raft/comms/ucp_helper.hpp index 47f3f4d61e..0cc1df1c36 100644 --- a/cpp/include/raft/comms/ucp_helper.hpp +++ b/cpp/include/raft/comms/ucp_helper.hpp @@ -172,8 +172,8 @@ class comms_ucp_handler { /** * @brief Asynchronously send data to the given endpoint using the given tag */ - void ucp_isend(ucp_request *req, ucp_ep_h ep_ptr, const void *buf, size_t size, - int tag, ucp_tag_t tag_mask, int rank) const { + void ucp_isend(ucp_request *req, ucp_ep_h ep_ptr, const void *buf, + size_t size, int tag, ucp_tag_t tag_mask, int rank) const { ucp_tag_t ucp_tag = build_message_tag(rank, tag); ucs_status_ptr_t send_result = (*(send_func))( From aae4625361664d0ae7ff2a554d72ffafa3825d70 Mon Sep 17 00:00:00 2001 From: Dante Gama Dessavre Date: Tue, 19 May 2020 10:23:24 -0500 Subject: [PATCH 31/46] FIX Use relative imports --- python/raft/common/handle.pxd | 6 +++--- python/raft/common/handle.pyx | 8 +++++--- 2 files changed, 8 insertions(+), 6 deletions(-) diff --git a/python/raft/common/handle.pxd b/python/raft/common/handle.pxd index dfdcdb929b..3b60e6708e 100644 --- a/python/raft/common/handle.pxd +++ b/python/raft/common/handle.pxd @@ -21,7 +21,7 @@ from libcpp.memory cimport shared_ptr -cimport raft.common.cuda +from .cuda cimport _Stream cdef extern from "raft/mr/device/allocator.hpp" \ @@ -33,7 +33,7 @@ cdef extern from "raft/handle.hpp" namespace "raft" nogil: cdef cppclass handle_t: handle_t() except + handle_t(int ns) except + - void set_stream(raft.common.cuda._Stream s) except + + void set_stream(_Stream s) except + void set_device_allocator(shared_ptr[allocator] a) except + - raft.common.cuda._Stream get_stream() except + + _Stream get_stream() except + int get_num_internal_streams() except + diff --git a/python/raft/common/handle.pyx b/python/raft/common/handle.pyx index c19cbc8faa..6658a825ce 100644 --- a/python/raft/common/handle.pyx +++ b/python/raft/common/handle.pyx @@ -19,9 +19,11 @@ # cython: embedsignature = True # cython: language_level = 3 -import raft +# import raft from libcpp.memory cimport shared_ptr -from raft.common.cuda cimport _Stream, _Error, cudaStreamSynchronize + +from .cuda cimport _Stream, _Error, cudaStreamSynchronize +from .cuda import CudaRuntimeError cdef class Handle: """ @@ -81,7 +83,7 @@ cdef class Handle: cdef _Stream stream = h_.get_stream() cdef _Error e = cudaStreamSynchronize(stream) if e != 0: - raise raft.cuda.CudaRuntimeError("Stream sync") + raise CudaRuntimeError("Stream sync") def getHandle(self): return self.h From bc3932161c6e58bb2054f46b7eacbf682501d246 Mon Sep 17 00:00:00 2001 From: "Corey J. Nolet" Date: Tue, 19 May 2020 11:33:36 -0400 Subject: [PATCH 32/46] Adding compile-time templates for comms_t to make interaction more straightforward --- cpp/include/raft/comms/comms.hpp | 149 ++++++++++++++++++++---- cpp/include/raft/comms/comms_helper.hpp | 17 +-- cpp/include/raft/comms/std_comms.hpp | 23 ++-- python/setup.py | 2 +- 4 files changed, 150 insertions(+), 41 deletions(-) diff --git a/cpp/include/raft/comms/comms.hpp b/cpp/include/raft/comms/comms.hpp index 11ad32cfcc..f76c456647 100644 --- a/cpp/include/raft/comms/comms.hpp +++ b/cpp/include/raft/comms/comms.hpp @@ -43,6 +43,7 @@ enum class status_t { commStatusAbort // A failure occurred in sync, queued operations aborted }; + class comms_iface { public: virtual ~comms_iface(); @@ -57,30 +58,30 @@ class comms_iface { virtual status_t syncStream(cudaStream_t stream) const = 0; virtual void isend(const void* buf, size_t size, int dest, int tag, - request_t* request) const = 0; + request_t* request) const=0; virtual void irecv(void* buf, size_t size, int source, int tag, - request_t* request) const = 0; + request_t* request) const = 0; virtual void waitall(int count, request_t array_of_requests[]) const = 0; virtual void allreduce(const void* sendbuff, void* recvbuff, size_t count, datatype_t datatype, op_t op, - cudaStream_t stream) const = 0; + cudaStream_t stream) const = 0; virtual void bcast(void* buff, size_t count, datatype_t datatype, int root, - cudaStream_t stream) const = 0; + cudaStream_t stream) const = 0; virtual void reduce(const void* sendbuff, void* recvbuff, size_t count, datatype_t datatype, op_t op, int root, - cudaStream_t stream) const = 0; + cudaStream_t stream) const = 0; virtual void allgather(const void* sendbuff, void* recvbuff, size_t sendcount, - datatype_t datatype, cudaStream_t stream) const = 0; + datatype_t datatype, cudaStream_t stream) const = 0; virtual void allgatherv(const void* sendbuf, void* recvbuf, const size_t recvcounts[], const int displs[], - datatype_t datatype, cudaStream_t stream) const = 0; + datatype_t datatype, cudaStream_t stream) const = 0; virtual void reducescatter(const void* sendbuff, void* recvbuff, size_t recvcount, datatype_t datatype, op_t op, @@ -90,9 +91,12 @@ class comms_iface { class comms_t { public: comms_t(std::unique_ptr impl) : impl_(impl.release()) { - ASSERT(nullptr != impl.get(), "ERROR: Invalid comms_iface used!"); + ASSERT(nullptr != impl_.get(), "ERROR: Invalid comms_iface used!"); } + template + datatype_t get_type() const; + int getSize() const { return impl_->getSize(); } int getRank() const { return impl_->getRank(); } @@ -108,49 +112,111 @@ class comms_t { } void isend(const void* buf, size_t size, int dest, int tag, + request_t* request) const { + impl_->isend(buf, size, dest, tag, request); + } + + template + void isend(const value_t* buf, size_t size, int dest, int tag, request_t* request) const { - impl_->isend(buf, size, dest, tag, request); + isend(static_cast(buf), size * sizeof(value_t), dest, tag, request); } void irecv(void* buf, size_t size, int source, int tag, + request_t* request) const { + impl_->irecv(buf, size, source, tag, request); + } + + template + void irecv(value_t* buf, size_t size, int source, int tag, request_t* request) const { - impl_->irecv(buf, size, source, tag, request); + irecv(static_cast(buf), size * sizeof(value_t), source, tag, request); } void waitall(int count, request_t array_of_requests[]) const { - impl_->waitall(count, array_of_requests); + impl_->waitall(count, array_of_requests); } void allreduce(const void* sendbuff, void* recvbuff, size_t count, - datatype_t datatype, op_t op, cudaStream_t stream) const { - impl_->allreduce(sendbuff, recvbuff, count, datatype, op, stream); + datatype_t datatype, op_t op, cudaStream_t stream) const { + + impl_->allreduce(sendbuff, recvbuff, count, datatype, op, stream); + } + + template + void allreduce(const value_t* sendbuff, value_t* recvbuff, size_t count, + op_t op, cudaStream_t stream) const { + allreduce(static_cast(sendbuff), static_cast(recvbuff) + , sizeof(value_t)*count, get_type(), op, stream); } void bcast(void* buff, size_t count, datatype_t datatype, int root, + cudaStream_t stream) const { + + impl_->bcast(buff, count, datatype , root, stream); + } + + template + void bcast(value_t* buff, size_t count, int root, cudaStream_t stream) const { - impl_->bcast(buff, count, datatype, root, stream); + bcast(static_cast(buff), count, get_type() , root, stream); } void reduce(const void* sendbuff, void* recvbuff, size_t count, - datatype_t datatype, op_t op, int root, + datatype_t datatype, op_t op, int root, + cudaStream_t stream) const { + + impl_->reduce(sendbuff, recvbuff, count, datatype, op, root, stream); + } + + template + void reduce(const value_t* sendbuff, value_t* recvbuff, size_t count, + op_t op, int root, cudaStream_t stream) const { - impl_->reduce(sendbuff, recvbuff, count, datatype, op, root, stream); + reduce(static_cast(sendbuff), static_cast(recvbuff), + count, get_type(), op, root, stream); } void allgather(const void* sendbuff, void* recvbuff, size_t sendcount, - datatype_t datatype, cudaStream_t stream) const { - impl_->allgather(sendbuff, recvbuff, sendcount, datatype, stream); + datatype_t datatype, cudaStream_t stream) const { + + impl_->allgather(sendbuff, recvbuff, + sendcount, datatype, stream); + } + + template + void allgather(const value_t* sendbuff, value_t* recvbuff, size_t sendcount, + cudaStream_t stream) const { + allgather(static_cast(sendbuff), static_cast(recvbuff), + sendcount, get_type(), stream); } void allgatherv(const void* sendbuf, void* recvbuf, const size_t recvcounts[], - const int displs[], datatype_t datatype, + const int displs[], datatype_t datatype, + cudaStream_t stream) const { + + impl_->allgatherv(sendbuf, recvbuf, recvcounts, displs, datatype, stream); + } + + template + void allgatherv(const value_t* sendbuf, value_t* recvbuf, const size_t recvcounts[], + const int displs[], cudaStream_t stream) const { - impl_->allgatherv(sendbuf, recvbuf, recvcounts, displs, datatype, stream); + allgatherv(static_cast(sendbuf), static_cast(recvbuf), recvcounts, displs, get_type(), stream); } void reducescatter(const void* sendbuff, void* recvbuff, size_t recvcount, - datatype_t datatype, op_t op, cudaStream_t stream) const { - impl_->reducescatter(sendbuff, recvbuff, recvcount, datatype, op, stream); + datatype_t datatype, op_t op, cudaStream_t stream) const { + + impl_->reducescatter(sendbuff, recvbuff, + recvcount, datatype , op, stream); + } + + template + void reducescatter(const value_t* sendbuff, value_t* recvbuff, size_t recvcount, + op_t op, cudaStream_t stream) const { + reducescatter(static_cast(sendbuff), static_cast(recvbuff), + recvcount, get_type() , op, stream); } private: @@ -159,5 +225,44 @@ class comms_t { comms_iface::~comms_iface() {} +template <> +constexpr datatype_t comms_t::get_type() const { + return datatype_t::CHAR; +} + +template <> +constexpr datatype_t comms_t::get_type() const { + return datatype_t::UINT8; +} + +template <> +constexpr datatype_t comms_t::get_type() const { + return datatype_t::INT32; +} + +template <> +constexpr datatype_t comms_t::get_type() const { + return datatype_t::UINT32; +} + +template <> +constexpr datatype_t comms_t::get_type() const { + return datatype_t::INT64; +} + +template <> +constexpr datatype_t comms_t::get_type() const { + return datatype_t::UINT64; +} + +template <> +constexpr datatype_t comms_t::get_type() const { + return datatype_t::FLOAT32; +} + +template <> +constexpr datatype_t comms_t::get_type() const { + return datatype_t::FLOAT64; +} } // namespace comms } // namespace raft diff --git a/cpp/include/raft/comms/comms_helper.hpp b/cpp/include/raft/comms/comms_helper.hpp index 2238226383..4edfc4b6c7 100644 --- a/cpp/include/raft/comms/comms_helper.hpp +++ b/cpp/include/raft/comms/comms_helper.hpp @@ -31,7 +31,9 @@ namespace comms { void build_comms_nccl_only(handle_t *handle, ncclComm_t comm, int size, int rank) { auto d_alloc = handle->get_device_allocator(); - auto *raft_comm = new raft::comms::std_comms(comm, size, rank, d_alloc); + raft::comms::comms_iface *raft_comm = new raft::comms::std_comms(comm, size, rank, d_alloc); + std::cout << "Comms: " << raft_comm->getSize() << std::endl; + auto communicator = std::make_shared(std::unique_ptr(raft_comm)); handle->set_comms(communicator); @@ -62,6 +64,7 @@ void build_comms_nccl_ucx(handle_t *handle, ncclComm_t comm, void *ucp_worker, auto d_alloc = handle->get_device_allocator(); auto *raft_comm = new raft::comms::std_comms(comm, (ucp_worker_h)ucp_worker, eps_sp, size, rank, d_alloc); + std::cout << "Comms: " << raft_comm << std::endl; auto communicator = std::make_shared(std::unique_ptr(raft_comm)); handle->set_comms(communicator); @@ -76,12 +79,12 @@ bool test_collective_allreduce(const handle_t &handle) { raft::mr::device::buffer temp_d(handle.get_device_allocator(), stream); temp_d.resize(1, stream); - CUDA_CHECK(cudaMemcpyAsync(temp_d.data(), &send, sizeof(int), + CUDA_CHECK(cudaMemcpyAsync(temp_d.data(), &send, 1, cudaMemcpyHostToDevice, stream)); - communicator.allreduce(temp_d.data(), temp_d.data(), 1, datatype_t::INT32, - op_t::SUM, stream); + communicator.allreduce(temp_d.data(), temp_d.data(), 1, + datatype_t::INT32, op_t::SUM, stream); int temp_h = 0; - CUDA_CHECK(cudaMemcpyAsync(&temp_h, temp_d.data(), sizeof(int), + CUDA_CHECK(cudaMemcpyAsync(&temp_h, temp_d.data(), 1, cudaMemcpyDeviceToHost, stream)); CUDA_CHECK(cudaStreamSynchronize(stream)); communicator.barrier(); @@ -106,7 +109,7 @@ bool test_pointToPoint_simple_send_recv(const handle_t &h, int numTrials) { //post receives for (int r = 0; r < communicator.getSize(); ++r) { if (r != rank) { - communicator.irecv(received_data.data() + request_idx, sizeof(int), r, + communicator.irecv(received_data.data() + request_idx, 1, r, 0, requests.data() + request_idx); ++request_idx; } @@ -114,7 +117,7 @@ bool test_pointToPoint_simple_send_recv(const handle_t &h, int numTrials) { for (int r = 0; r < communicator.getSize(); ++r) { if (r != rank) { - communicator.isend(&rank, sizeof(int), r, 0, + communicator.isend(&rank, 1, r, 0, requests.data() + request_idx); ++request_idx; } diff --git a/cpp/include/raft/comms/std_comms.hpp b/cpp/include/raft/comms/std_comms.hpp index e806740207..d885853f1d 100644 --- a/cpp/include/raft/comms/std_comms.hpp +++ b/cpp/include/raft/comms/std_comms.hpp @@ -64,7 +64,8 @@ namespace raft { namespace comms { -constexpr size_t get_datatype_size(const datatype_t datatype) { +constexpr size_t get_size(const datatype_t datatype) { + switch (datatype) { case datatype_t::CHAR: return sizeof(char); @@ -85,7 +86,7 @@ constexpr size_t get_datatype_size(const datatype_t datatype) { } } -ncclDataType_t get_nccl_datatype(const datatype_t datatype) { +constexpr ncclDataType_t get_nccl_type(const datatype_t datatype) { switch (datatype) { case datatype_t::CHAR: return ncclChar; @@ -106,7 +107,7 @@ ncclDataType_t get_nccl_datatype(const datatype_t datatype) { } } -ncclRedOp_t get_nccl_op(const op_t op) { +constexpr ncclRedOp_t get_nccl_op(const op_t op) { switch (op) { case op_t::SUM: return ncclSum; @@ -159,7 +160,7 @@ class std_comms : public comms_iface { initialize(); }; - virtual ~std_comms() { + ~std_comms() { CUDA_CHECK_NO_THROW(cudaStreamDestroy(stream_)); device_allocator_->deallocate(sendbuff_, sizeof(int), stream_); @@ -316,13 +317,13 @@ class std_comms : public comms_iface { void allreduce(const void *sendbuff, void *recvbuff, size_t count, datatype_t datatype, op_t op, cudaStream_t stream) const { NCCL_CHECK(ncclAllReduce(sendbuff, recvbuff, count, - get_nccl_datatype(datatype), get_nccl_op(op), + get_nccl_type(datatype), get_nccl_op(op), nccl_comm_, stream)); } void bcast(void *buff, size_t count, datatype_t datatype, int root, cudaStream_t stream) const { - NCCL_CHECK(ncclBroadcast(buff, buff, count, get_nccl_datatype(datatype), + NCCL_CHECK(ncclBroadcast(buff, buff, count, get_nccl_type(datatype), root, nccl_comm_, stream)); } @@ -330,14 +331,14 @@ class std_comms : public comms_iface { datatype_t datatype, op_t op, int root, cudaStream_t stream) const { NCCL_CHECK(ncclReduce(sendbuff, recvbuff, count, - get_nccl_datatype(datatype), get_nccl_op(op), root, + get_nccl_type(datatype), get_nccl_op(op), root, nccl_comm_, stream)); } void allgather(const void *sendbuff, void *recvbuff, size_t sendcount, datatype_t datatype, cudaStream_t stream) const { NCCL_CHECK(ncclAllGather(sendbuff, recvbuff, sendcount, - get_nccl_datatype(datatype), nccl_comm_, stream)); + get_nccl_type(datatype), nccl_comm_, stream)); } void allgatherv(const void *sendbuf, void *recvbuf, const size_t recvcounts[], @@ -346,10 +347,10 @@ class std_comms : public comms_iface { //From: "An Empirical Evaluation of Allgatherv on Multi-GPU Systems" - https://arxiv.org/pdf/1812.05964.pdf //Listing 1 on page 4. for (int root = 0; root < num_ranks_; ++root) { - size_t dtype_size = get_datatype_size(datatype); + size_t dtype_size = get_size(datatype); NCCL_CHECK(ncclBroadcast( sendbuf, static_cast(recvbuf) + displs[root] * dtype_size, - recvcounts[root], get_nccl_datatype(datatype), root, nccl_comm_, + recvcounts[root], get_nccl_type(datatype), root, nccl_comm_, stream)); } } @@ -357,7 +358,7 @@ class std_comms : public comms_iface { void reducescatter(const void *sendbuff, void *recvbuff, size_t recvcount, datatype_t datatype, op_t op, cudaStream_t stream) const { NCCL_CHECK(ncclReduceScatter(sendbuff, recvbuff, recvcount, - get_nccl_datatype(datatype), get_nccl_op(op), + get_nccl_type(datatype), get_nccl_op(op), nccl_comm_, stream)); } diff --git a/python/setup.py b/python/setup.py index 107a061bd9..4f4e6c8b96 100644 --- a/python/setup.py +++ b/python/setup.py @@ -112,7 +112,7 @@ os.path.join(os.sys.prefix, "lib")], libraries=libs, language='c++', - extra_compile_args=['-std=c++11']) + extra_compile_args=['-std=c++14']) ] From 3150fbd7a8a7803aef999993b5ad23562330aba0 Mon Sep 17 00:00:00 2001 From: "Corey J. Nolet" Date: Tue, 19 May 2020 11:40:41 -0400 Subject: [PATCH 33/46] Using std::this_thread::yield instead of pthread_yield() --- cpp/include/raft/comms/std_comms.hpp | 25 ++++++++++++------------- 1 file changed, 12 insertions(+), 13 deletions(-) diff --git a/cpp/include/raft/comms/std_comms.hpp b/cpp/include/raft/comms/std_comms.hpp index d885853f1d..e4e59115f1 100644 --- a/cpp/include/raft/comms/std_comms.hpp +++ b/cpp/include/raft/comms/std_comms.hpp @@ -64,8 +64,7 @@ namespace raft { namespace comms { -constexpr size_t get_size(const datatype_t datatype) { - +constexpr size_t get_datatype_size(const datatype_t datatype) { switch (datatype) { case datatype_t::CHAR: return sizeof(char); @@ -86,7 +85,7 @@ constexpr size_t get_size(const datatype_t datatype) { } } -constexpr ncclDataType_t get_nccl_type(const datatype_t datatype) { +ncclDataType_t get_nccl_datatype(const datatype_t datatype) { switch (datatype) { case datatype_t::CHAR: return ncclChar; @@ -107,7 +106,7 @@ constexpr ncclDataType_t get_nccl_type(const datatype_t datatype) { } } -constexpr ncclRedOp_t get_nccl_op(const op_t op) { +ncclRedOp_t get_nccl_op(const op_t op) { switch (op) { case op_t::SUM: return ncclSum; @@ -160,7 +159,7 @@ class std_comms : public comms_iface { initialize(); }; - ~std_comms() { + virtual ~std_comms() { CUDA_CHECK_NO_THROW(cudaStreamDestroy(stream_)); device_allocator_->deallocate(sendbuff_, sizeof(int), stream_); @@ -317,13 +316,13 @@ class std_comms : public comms_iface { void allreduce(const void *sendbuff, void *recvbuff, size_t count, datatype_t datatype, op_t op, cudaStream_t stream) const { NCCL_CHECK(ncclAllReduce(sendbuff, recvbuff, count, - get_nccl_type(datatype), get_nccl_op(op), + get_nccl_datatype(datatype), get_nccl_op(op), nccl_comm_, stream)); } void bcast(void *buff, size_t count, datatype_t datatype, int root, cudaStream_t stream) const { - NCCL_CHECK(ncclBroadcast(buff, buff, count, get_nccl_type(datatype), + NCCL_CHECK(ncclBroadcast(buff, buff, count, get_nccl_datatype(datatype), root, nccl_comm_, stream)); } @@ -331,14 +330,14 @@ class std_comms : public comms_iface { datatype_t datatype, op_t op, int root, cudaStream_t stream) const { NCCL_CHECK(ncclReduce(sendbuff, recvbuff, count, - get_nccl_type(datatype), get_nccl_op(op), root, + get_nccl_datatype(datatype), get_nccl_op(op), root, nccl_comm_, stream)); } void allgather(const void *sendbuff, void *recvbuff, size_t sendcount, datatype_t datatype, cudaStream_t stream) const { NCCL_CHECK(ncclAllGather(sendbuff, recvbuff, sendcount, - get_nccl_type(datatype), nccl_comm_, stream)); + get_nccl_datatype(datatype), nccl_comm_, stream)); } void allgatherv(const void *sendbuf, void *recvbuf, const size_t recvcounts[], @@ -347,10 +346,10 @@ class std_comms : public comms_iface { //From: "An Empirical Evaluation of Allgatherv on Multi-GPU Systems" - https://arxiv.org/pdf/1812.05964.pdf //Listing 1 on page 4. for (int root = 0; root < num_ranks_; ++root) { - size_t dtype_size = get_size(datatype); + size_t dtype_size = get_datatype_size(datatype); NCCL_CHECK(ncclBroadcast( sendbuf, static_cast(recvbuf) + displs[root] * dtype_size, - recvcounts[root], get_nccl_type(datatype), root, nccl_comm_, + recvcounts[root], get_nccl_datatype(datatype), root, nccl_comm_, stream)); } } @@ -358,7 +357,7 @@ class std_comms : public comms_iface { void reducescatter(const void *sendbuff, void *recvbuff, size_t recvcount, datatype_t datatype, op_t op, cudaStream_t stream) const { NCCL_CHECK(ncclReduceScatter(sendbuff, recvbuff, recvcount, - get_nccl_type(datatype), get_nccl_op(op), + get_nccl_datatype(datatype), get_nccl_op(op), nccl_comm_, stream)); } @@ -390,7 +389,7 @@ class std_comms : public comms_iface { } // Let other threads (including NCCL threads) use the CPU. - pthread_yield(); + std::this_thread::yield(); } } From 5628ad230d7c0035089666ec069e11a1232a4923 Mon Sep 17 00:00:00 2001 From: "Corey J. Nolet" Date: Tue, 19 May 2020 14:26:15 -0400 Subject: [PATCH 34/46] Adding python tests for collective functions --- cpp/include/raft/comms/comms.hpp | 174 +++++++++++++++-------- cpp/include/raft/comms/comms_helper.hpp | 176 ++++++++++++++++++++++-- cpp/include/raft/comms/std_comms.hpp | 1 + python/raft/dask/common/__init__.py | 5 +- python/raft/dask/common/comms_utils.pyx | 62 ++++++++- python/raft/test/test_comms.py | 31 +++-- 6 files changed, 365 insertions(+), 84 deletions(-) diff --git a/cpp/include/raft/comms/comms.hpp b/cpp/include/raft/comms/comms.hpp index f76c456647..5861e159ab 100644 --- a/cpp/include/raft/comms/comms.hpp +++ b/cpp/include/raft/comms/comms.hpp @@ -52,7 +52,6 @@ class comms_iface { virtual int getRank() const = 0; virtual std::unique_ptr commSplit(int color, int key) const = 0; - virtual void barrier() const = 0; virtual status_t syncStream(cudaStream_t stream) const = 0; @@ -97,125 +96,180 @@ class comms_t { template datatype_t get_type() const; + /** + * Returns the size of the communicator clique + */ + int getSize() const { return impl_->getSize(); } + + /** + * Returns the local rank + */ int getRank() const { return impl_->getRank(); } + /** + * Splits the current communicator clique into sub-cliques matching + * the given color and key + * + * @param color ranks w/ the same color are placed in the same communicator + * @param key controls rank assignment + */ std::unique_ptr commSplit(int color, int key) const { return impl_->commSplit(color, key); } + /** + * Performs a collective barrier synchronization + */ void barrier() const { impl_->barrier(); } + /** + * Some collective communications implementations (eg. NCCL) might use asynchronous + * collectives that are explicitly synchronized. It's important to always synchronize + * using this method to allow failures to propagate, rather than `cudaStreamSynchronize()`, + * to prevent the potential for deadlocks. + * + * @param stream the cuda stream to sync collective operations on + */ status_t syncStream(cudaStream_t stream) const { return impl_->syncStream(stream); } - void isend(const void* buf, size_t size, int dest, int tag, - request_t* request) const { - impl_->isend(buf, size, dest, tag, request); - } - + /** + * Performs an asynchronous point-to-point send + * @tparam value_t the type of data to send + * @param buf pointer to array of data to send + * @param size number of elements in buf + * @param dest destination rank + * @param tag a tag to use for the receiver to filter + * @param request pointer to hold returned request_t object. + * This will be used in `waitall()` to synchronize until the message is delivered (or fails). + */ template void isend(const value_t* buf, size_t size, int dest, int tag, request_t* request) const { - isend(static_cast(buf), size * sizeof(value_t), dest, tag, request); - } - - void irecv(void* buf, size_t size, int source, int tag, - request_t* request) const { - impl_->irecv(buf, size, source, tag, request); + impl_->isend(static_cast(buf), size * sizeof(value_t), dest, tag, request); } + /** + * Performs an asynchronous point-to-point receive + * @tparam value_t the type of data to be received + * @param buf pointer to (initialized) array that will hold received data + * @param size number of elements in buf + * @param source source rank + * @param tag a tag to use for message filtering + * @param request pointer to hold returned request_t object. + * This will be used in `waitall()` to synchronize until the message is delivered (or fails). + */ template void irecv(value_t* buf, size_t size, int source, int tag, request_t* request) const { - irecv(static_cast(buf), size * sizeof(value_t), source, tag, request); + impl_->irecv(static_cast(buf), size * sizeof(value_t), source, tag, request); } + /** + * Synchronize on an array of request_t objects returned from isend/irecv + * @param count number of requests to synchronize on + * @param array_of_requests an array of request_t objects returned from isend/irecv + */ void waitall(int count, request_t array_of_requests[]) const { impl_->waitall(count, array_of_requests); } - void allreduce(const void* sendbuff, void* recvbuff, size_t count, - datatype_t datatype, op_t op, cudaStream_t stream) const { - - impl_->allreduce(sendbuff, recvbuff, count, datatype, op, stream); - } - + /** + * Perform an allreduce collective + * @tparam value_t datatype of underlying buffers + * @param sendbuff data to reduce + * @param recvbuff buffer to hold the reduced result + * @param count number of elements in sendbuff + * @param op reduction operation to perform + * @param stream CUDA stream to synchronize operation + */ template void allreduce(const value_t* sendbuff, value_t* recvbuff, size_t count, op_t op, cudaStream_t stream) const { - allreduce(static_cast(sendbuff), static_cast(recvbuff) - , sizeof(value_t)*count, get_type(), op, stream); - } - - void bcast(void* buff, size_t count, datatype_t datatype, int root, - cudaStream_t stream) const { - - impl_->bcast(buff, count, datatype , root, stream); + impl_->allreduce(static_cast(sendbuff), static_cast(recvbuff) + , count, get_type(), op, stream); } + /** + * Broadcast data from one rank to the rest + * @tparam value_t datatype of underlying buffers + * @param buff buffer to send + * @param count number of elements if buff + * @param root the rank initiating the broadcast + * @param stream CUDA stream to synchronize operation + */ template void bcast(value_t* buff, size_t count, int root, cudaStream_t stream) const { - bcast(static_cast(buff), count, get_type() , root, stream); - } - - void reduce(const void* sendbuff, void* recvbuff, size_t count, - datatype_t datatype, op_t op, int root, - cudaStream_t stream) const { - - impl_->reduce(sendbuff, recvbuff, count, datatype, op, root, stream); + impl_->bcast(static_cast(buff), count, get_type(), root, stream); } + /** + * Reduce data from many ranks down to a single rank + * @tparam value_t datatype of underlying buffers + * @param sendbuff buffer containing data to reduce + * @param recvbuff buffer containing reduced data (only needs to be initialized on root) + * @param count number of elements in sendbuff + * @param op reduction operation to perform + * @param root rank to store the results + * @param stream CUDA stream to synchronize operation + */ template void reduce(const value_t* sendbuff, value_t* recvbuff, size_t count, op_t op, int root, cudaStream_t stream) const { - reduce(static_cast(sendbuff), static_cast(recvbuff), + impl_->reduce(static_cast(sendbuff), static_cast(recvbuff), count, get_type(), op, root, stream); } - void allgather(const void* sendbuff, void* recvbuff, size_t sendcount, - datatype_t datatype, cudaStream_t stream) const { - - impl_->allgather(sendbuff, recvbuff, - sendcount, datatype, stream); - } - + /** + * Gathers data from each rank onto all ranks + * @tparam value_t datatype of underlying buffers + * @param sendbuff buffer containing data to gather + * @param recvbuff buffer containing gathered data from all ranks + * @param sendcount number of elements in send buffer + * @param stream CUDA stream to synchronize operation + */ template void allgather(const value_t* sendbuff, value_t* recvbuff, size_t sendcount, cudaStream_t stream) const { - allgather(static_cast(sendbuff), static_cast(recvbuff), + impl_->allgather(static_cast(sendbuff), static_cast(recvbuff), sendcount, get_type(), stream); } - void allgatherv(const void* sendbuf, void* recvbuf, const size_t recvcounts[], - const int displs[], datatype_t datatype, - cudaStream_t stream) const { - - impl_->allgatherv(sendbuf, recvbuf, recvcounts, displs, datatype, stream); - } - + /** + * Gathers data from all ranks and delivers to combined data to all ranks + * @param value_t datatype of underlying buffers + * @param sendbuff buffer containing data to send + * @param recvbuff buffer containing data to receive + * @param recvcounts array (of length num_ranks size) containing the number of elements + * that are to be received from each rank + * @param displs array (of length num_ranks size) to specify the displacement (relative to recvbuf) + * at which to place the incoming data from each rank + * @param stream CUDA stream to synchronize operation + */ template void allgatherv(const value_t* sendbuf, value_t* recvbuf, const size_t recvcounts[], const int displs[], cudaStream_t stream) const { - allgatherv(static_cast(sendbuf), static_cast(recvbuf), recvcounts, displs, get_type(), stream); - } - - void reducescatter(const void* sendbuff, void* recvbuff, size_t recvcount, - datatype_t datatype, op_t op, cudaStream_t stream) const { - - impl_->reducescatter(sendbuff, recvbuff, - recvcount, datatype , op, stream); + impl_->allgatherv(static_cast(sendbuf), static_cast(recvbuf), recvcounts, displs, get_type(), stream); } + /** + * Reduces data from all ranks then scatters the result across ranks + * @tparam value_t datatype of underlying buffers + * @param sendbuff buffer containing data to send (size recvcount * num_ranks) + * @param recvbuff buffer containing received data + * @param op reduction operation to perform + * @param stream CUDA stream to synchronize operation + */ template void reducescatter(const value_t* sendbuff, value_t* recvbuff, size_t recvcount, op_t op, cudaStream_t stream) const { - reducescatter(static_cast(sendbuff), static_cast(recvbuff), + impl_->reducescatter(static_cast(sendbuff), static_cast(recvbuff), recvcount, get_type() , op, stream); } diff --git a/cpp/include/raft/comms/comms_helper.hpp b/cpp/include/raft/comms/comms_helper.hpp index 4edfc4b6c7..d762785fef 100644 --- a/cpp/include/raft/comms/comms_helper.hpp +++ b/cpp/include/raft/comms/comms_helper.hpp @@ -27,11 +27,16 @@ namespace comms { /** * Function to construct comms_t and inject it on a handle_t. This * is used for convenience in the Python layer. + * + * @param handle raft::handle_t for injecting the comms + * @param nccl_comm initialized NCCL communicator to use for collectives + * @param num_ranks number of ranks in communicator clique + * @param rank rank of local instance */ -void build_comms_nccl_only(handle_t *handle, ncclComm_t comm, int size, +void build_comms_nccl_only(handle_t *handle, ncclComm_t nccl_comm, int num_ranks, int rank) { auto d_alloc = handle->get_device_allocator(); - raft::comms::comms_iface *raft_comm = new raft::comms::std_comms(comm, size, rank, d_alloc); + raft::comms::comms_iface *raft_comm = new raft::comms::std_comms(nccl_comm, num_ranks, rank, d_alloc); std::cout << "Comms: " << raft_comm->getSize() << std::endl; auto communicator = @@ -42,14 +47,25 @@ void build_comms_nccl_only(handle_t *handle, ncclComm_t comm, int size, /** * Function to construct comms_t and inject it on a handle_t. This * is used for convenience in the Python layer. + * + * @param handle raft::handle_t for injecting the comms + * @param nccl_comm initialized NCCL communicator to use for collectives + * @param ucp_worker of local process + * Note: This is purposefully left as void* so that the ucp_worker_h + * doesn't need to be exposed through the cython layer + * @param eps array of ucp_ep_h instances. + * Note: This is purposefully left as void* so that + * the ucp_ep_h doesn't need to be exposed through the cython layer. + * @param num_ranks number of ranks in communicator clique + * @param rank rank of local instance */ -void build_comms_nccl_ucx(handle_t *handle, ncclComm_t comm, void *ucp_worker, - void *eps, int size, int rank) { - auto eps_sp = std::make_shared(new ucp_ep_h[size]); +void build_comms_nccl_ucx(handle_t *handle, ncclComm_t nccl_comm, void *ucp_worker, + void *eps, int num_ranks, int rank) { + auto eps_sp = std::make_shared(new ucp_ep_h[num_ranks]); auto size_t_ep_arr = reinterpret_cast(eps); - for (int i = 0; i < size; i++) { + for (int i = 0; i < num_ranks; i++) { size_t ptr = size_t_ep_arr[i]; auto ucp_ep_v = reinterpret_cast(*eps_sp); @@ -62,15 +78,20 @@ void build_comms_nccl_ucx(handle_t *handle, ncclComm_t comm, void *ucp_worker, } auto d_alloc = handle->get_device_allocator(); - auto *raft_comm = new raft::comms::std_comms(comm, (ucp_worker_h)ucp_worker, - eps_sp, size, rank, d_alloc); - std::cout << "Comms: " << raft_comm << std::endl; + auto *raft_comm = new raft::comms::std_comms(nccl_comm, (ucp_worker_h)ucp_worker, + eps_sp, num_ranks, rank, d_alloc); auto communicator = std::make_shared(std::unique_ptr(raft_comm)); handle->set_comms(communicator); } -bool test_collective_allreduce(const handle_t &handle) { +/** + * A simple sanity check that NCCL is able to perform a collective operation + * + * @param the raft handle to use. This is expected to already have an + * initialized comms instance. + */ +bool test_collective_allreduce(const handle_t &handle, int root) { const comms_t &communicator = handle.get_comms(); const int send = 1; @@ -82,7 +103,7 @@ bool test_collective_allreduce(const handle_t &handle) { CUDA_CHECK(cudaMemcpyAsync(temp_d.data(), &send, 1, cudaMemcpyHostToDevice, stream)); communicator.allreduce(temp_d.data(), temp_d.data(), 1, - datatype_t::INT32, op_t::SUM, stream); + op_t::SUM, stream); int temp_h = 0; CUDA_CHECK(cudaMemcpyAsync(&temp_h, temp_d.data(), 1, cudaMemcpyDeviceToHost, stream)); @@ -95,6 +116,139 @@ bool test_collective_allreduce(const handle_t &handle) { return temp_h == communicator.getSize(); } + +/** + * A simple sanity check that NCCL is able to perform a collective operation + * + * @param the raft handle to use. This is expected to already have an + * initialized comms instance. + */ +bool test_collective_broadcast(const handle_t &handle, int root) { + const comms_t &communicator = handle.get_comms(); + + const int send = root; + + cudaStream_t stream = handle.get_stream(); + + raft::mr::device::buffer temp_d(handle.get_device_allocator(), stream); + temp_d.resize(1, stream); + + if(communicator.getRank() == root) + CUDA_CHECK(cudaMemcpyAsync(temp_d.data(), &send, sizeof(int), + cudaMemcpyHostToDevice, stream)); + + communicator.bcast(temp_d.data(), 1, root, stream); + communicator.syncStream(stream); + int temp_h = -1; // Verify more than one byte is being sent + CUDA_CHECK(cudaMemcpyAsync(&temp_h, temp_d.data(), sizeof(int), + cudaMemcpyDeviceToHost, stream)); + CUDA_CHECK(cudaStreamSynchronize(stream)); + communicator.barrier(); + + std::cout << "Clique size: " << communicator.getSize() << std::endl; + std::cout << "final_size: " << temp_h << std::endl; + + return temp_h == root; +} + +bool test_collective_reduce(const handle_t &handle, int root) { + const comms_t &communicator = handle.get_comms(); + + const int send = root; + + cudaStream_t stream = handle.get_stream(); + + raft::mr::device::buffer temp_d(handle.get_device_allocator(), stream); + temp_d.resize(1, stream); + + CUDA_CHECK(cudaMemcpyAsync(temp_d.data(), &send, sizeof(int), + cudaMemcpyHostToDevice, stream)); + + communicator.reduce(temp_d.data(), temp_d.data(), 1, op_t::SUM, root, stream); + communicator.syncStream(stream); + int temp_h = -1; // Verify more than one byte is being sent + CUDA_CHECK(cudaMemcpyAsync(&temp_h, temp_d.data(), sizeof(int), + cudaMemcpyDeviceToHost, stream)); + CUDA_CHECK(cudaStreamSynchronize(stream)); + communicator.barrier(); + + std::cout << "Clique size: " << communicator.getSize() << std::endl; + std::cout << "final_size: " << temp_h << std::endl; + + if(communicator.getRank() == root) + return temp_h == root * communicator.getSize(); + else + return true; +} + + +bool test_collective_allgather(const handle_t &handle, int root) { + const comms_t &communicator = handle.get_comms(); + + const int send = root; + + cudaStream_t stream = handle.get_stream(); + + raft::mr::device::buffer temp_d(handle.get_device_allocator(), stream); + temp_d.resize(1, stream); + + raft::mr::device::buffer recv_d(handle.get_device_allocator(), stream, communicator.getSize()); + + CUDA_CHECK(cudaMemcpyAsync(temp_d.data(), &send, sizeof(int), + cudaMemcpyHostToDevice, stream)); + + communicator.allgather(temp_d.data(), recv_d.data(), 1, stream); + communicator.syncStream(stream); + int temp_h[communicator.getSize()]; // Verify more than one byte is being sent + CUDA_CHECK(cudaMemcpyAsync(&temp_h, temp_d.data(), sizeof(int)*communicator.getSize(), + cudaMemcpyDeviceToHost, stream)); + CUDA_CHECK(cudaStreamSynchronize(stream)); + communicator.barrier(); + + std::cout << "Clique size: " << communicator.getSize() << std::endl; + std::cout << "final_size: " << temp_h << std::endl; + + for(int i = 0; i < communicator.getSize(); i++) + if(temp_h[i] != i) + return false; + return true; +} + +bool test_collective_reducescatter(const handle_t &handle, int root) { + const comms_t &communicator = handle.get_comms(); + + const int send = 1; + + cudaStream_t stream = handle.get_stream(); + + raft::mr::device::buffer temp_d(handle.get_device_allocator(), stream, 1); + raft::mr::device::buffer recv_d(handle.get_device_allocator(), stream, 1); + + CUDA_CHECK(cudaMemcpyAsync(temp_d.data(), &send, sizeof(int), + cudaMemcpyHostToDevice, stream)); + + communicator.reducescatter(temp_d.data(), recv_d.data(), 1, op_t::SUM, stream); + communicator.syncStream(stream); + int temp_h = -1; // Verify more than one byte is being sent + CUDA_CHECK(cudaMemcpyAsync(&temp_h, temp_d.data(), sizeof(int), + cudaMemcpyDeviceToHost, stream)); + CUDA_CHECK(cudaStreamSynchronize(stream)); + communicator.barrier(); + + std::cout << "Clique size: " << communicator.getSize() << std::endl; + std::cout << "final_size: " << temp_h << std::endl; + + return temp_h = communicator.getSize(); +} + + +/** + * A simple sanity check that UCX is able to send messages between all ranks + * + * @param the raft handle to use. This is expected to already have an + * initialized comms instance. + * @param number of iterations of all-to-all messaging to perform + */ bool test_pointToPoint_simple_send_recv(const handle_t &h, int numTrials) { const comms_t &communicator = h.get_comms(); const int rank = communicator.getRank(); diff --git a/cpp/include/raft/comms/std_comms.hpp b/cpp/include/raft/comms/std_comms.hpp index e4e59115f1..ecd589a32c 100644 --- a/cpp/include/raft/comms/std_comms.hpp +++ b/cpp/include/raft/comms/std_comms.hpp @@ -92,6 +92,7 @@ ncclDataType_t get_nccl_datatype(const datatype_t datatype) { case datatype_t::UINT8: return ncclUint8; case datatype_t::INT32: + std::cout << "Returning int32" << std::endl; return ncclInt; case datatype_t::UINT32: return ncclUint32; diff --git a/python/raft/dask/common/__init__.py b/python/raft/dask/common/__init__.py index 9b85a32800..37981b70fb 100644 --- a/python/raft/dask/common/__init__.py +++ b/python/raft/dask/common/__init__.py @@ -20,4 +20,7 @@ from raft.dask.common.comms_utils import inject_comms_on_handle_coll_only from raft.dask.common.comms_utils import perform_test_comms_allreduce from raft.dask.common.comms_utils import perform_test_comms_send_recv - +from raft.dask.common.comms_utils import perform_test_comms_allgather +from raft.dask.common.comms_utils import perform_test_comms_bcast +from raft.dask.common.comms_utils import perform_test_comms_reduce +from raft.dask.common.comms_utils import perform_test_comms_reducescatter diff --git a/python/raft/dask/common/comms_utils.pyx b/python/raft/dask/common/comms_utils.pyx index 1825d385a7..8272446529 100644 --- a/python/raft/dask/common/comms_utils.pyx +++ b/python/raft/dask/common/comms_utils.pyx @@ -55,12 +55,16 @@ cdef extern from "raft/comms/comms_helper.hpp" namespace "raft::comms": int size, int rank) except + - bool test_collective_allreduce(const handle_t &h) except + + bool test_collective_allreduce(const handle_t &h, int root) except + + bool test_collective_broadcast(const handle_t &h, int root) except + + bool test_collective_reduce(const handle_t &h, int root) except + + bool test_collective_allgather(const handle_t &h, int root) except + + bool test_collective_reducescatter(const handle_t &h, int root) except + bool test_pointToPoint_simple_send_recv(const handle_t &h, int numTrials) except + -def perform_test_comms_allreduce(handle): +def perform_test_comms_allreduce(handle, root): """ Performs an allreduce on the current worker @@ -70,7 +74,59 @@ def perform_test_comms_allreduce(handle): handle containing comms_t to use """ cdef const handle_t* h = handle.getHandle() - return test_collective_allreduce(deref(h)) + return test_collective_allreduce(deref(h), root) + + +def perform_test_comms_reduce(handle, root): + """ + Performs an allreduce on the current worker + + Parameters + ---------- + handle : raft.common.Handle + handle containing comms_t to use + """ + cdef const handle_t* h = handle.getHandle() + return test_collective_reduce(deref(h), root) + + +def perform_test_comms_reducescatter(handle, root): + """ + Performs an allreduce on the current worker + + Parameters + ---------- + handle : raft.common.Handle + handle containing comms_t to use + """ + cdef const handle_t* h = handle.getHandle() + return test_collective_reducescatter(deref(h), root) + + +def perform_test_comms_bcast(handle, root): + """ + Performs an broadcast on the current worker + + Parameters + ---------- + handle : raft.common.Handle + handle containing comms_t to use + """ + cdef const handle_t* h = handle.getHandle() + return test_collective_broadcast(deref(h), root) + + +def perform_test_comms_allgather(handle, root): + """ + Performs an broadcast on the current worker + + Parameters + ---------- + handle : raft.common.Handle + handle containing comms_t to use + """ + cdef const handle_t* h = handle.getHandle() + return test_collective_allgather(deref(h), root) def perform_test_comms_send_recv(handle, n_trials): diff --git a/python/raft/test/test_comms.py b/python/raft/test/test_comms.py index 9fb735c361..d3a6c20ffc 100644 --- a/python/raft/test/test_comms.py +++ b/python/raft/test/test_comms.py @@ -22,6 +22,10 @@ from raft.dask.common import local_handle from raft.dask.common import perform_test_comms_send_recv from raft.dask.common import perform_test_comms_allreduce +from raft.dask.common import perform_test_comms_bcast +from raft.dask.common import perform_test_comms_reduce +from raft.dask.common import perform_test_comms_allgather +from raft.dask.common import perform_test_comms_reducescatter pytestmark = pytest.mark.mg @@ -43,9 +47,9 @@ def test_comms_init_no_p2p(cluster): client.close() -def func_test_allreduce(sessionId): +def func_test_collective(func, sessionId, root): handle = local_handle(sessionId) - return perform_test_comms_allreduce(handle) + return func(handle, root) def func_test_send_recv(sessionId, n_trials): @@ -79,7 +83,12 @@ def _has_handle(sessionId): @pytest.mark.nccl -def test_allreduce(cluster): +@pytest.mark.parametrize("func", [perform_test_comms_allgather, + perform_test_comms_allreduce, + perform_test_comms_bcast, + perform_test_comms_reduce, + perform_test_comms_reducescatter]) +def test_collectives(cluster, func): client = Client(cluster) @@ -87,12 +96,16 @@ def test_allreduce(cluster): cb = Comms() cb.init() - dfs = [client.submit(func_test_allreduce, - cb.sessionId, - pure=False, - workers=[w]) - for w in cb.worker_addresses] - wait(dfs, timeout=5) + for k, v in cb.worker_info(cb.worker_addresses).items(): + + dfs = [client.submit(func_test_collective, + perform_test_comms_allreduce, + cb.sessionId, + v["rank"], + pure=False, + workers=[w]) + for w in cb.worker_addresses] + wait(dfs, timeout=5) assert all([x.result() for x in dfs]) From 488d0d5866df9df1eb2845c359d8e223abd81cda Mon Sep 17 00:00:00 2001 From: "Corey J. Nolet" Date: Tue, 19 May 2020 14:27:30 -0400 Subject: [PATCH 35/46] Running cpp style --- cpp/include/raft/comms/comms.hpp | 83 ++++++++++++------------ cpp/include/raft/comms/comms_helper.hpp | 84 +++++++++++++------------ 2 files changed, 87 insertions(+), 80 deletions(-) diff --git a/cpp/include/raft/comms/comms.hpp b/cpp/include/raft/comms/comms.hpp index 5861e159ab..5c47316670 100644 --- a/cpp/include/raft/comms/comms.hpp +++ b/cpp/include/raft/comms/comms.hpp @@ -43,7 +43,6 @@ enum class status_t { commStatusAbort // A failure occurred in sync, queued operations aborted }; - class comms_iface { public: virtual ~comms_iface(); @@ -57,30 +56,30 @@ class comms_iface { virtual status_t syncStream(cudaStream_t stream) const = 0; virtual void isend(const void* buf, size_t size, int dest, int tag, - request_t* request) const=0; + request_t* request) const = 0; virtual void irecv(void* buf, size_t size, int source, int tag, - request_t* request) const = 0; + request_t* request) const = 0; virtual void waitall(int count, request_t array_of_requests[]) const = 0; virtual void allreduce(const void* sendbuff, void* recvbuff, size_t count, datatype_t datatype, op_t op, - cudaStream_t stream) const = 0; + cudaStream_t stream) const = 0; virtual void bcast(void* buff, size_t count, datatype_t datatype, int root, - cudaStream_t stream) const = 0; + cudaStream_t stream) const = 0; virtual void reduce(const void* sendbuff, void* recvbuff, size_t count, datatype_t datatype, op_t op, int root, - cudaStream_t stream) const = 0; + cudaStream_t stream) const = 0; virtual void allgather(const void* sendbuff, void* recvbuff, size_t sendcount, - datatype_t datatype, cudaStream_t stream) const = 0; + datatype_t datatype, cudaStream_t stream) const = 0; virtual void allgatherv(const void* sendbuf, void* recvbuf, const size_t recvcounts[], const int displs[], - datatype_t datatype, cudaStream_t stream) const = 0; + datatype_t datatype, cudaStream_t stream) const = 0; virtual void reducescatter(const void* sendbuff, void* recvbuff, size_t recvcount, datatype_t datatype, op_t op, @@ -102,7 +101,6 @@ class comms_t { int getSize() const { return impl_->getSize(); } - /** * Returns the local rank */ @@ -146,10 +144,11 @@ class comms_t { * @param request pointer to hold returned request_t object. * This will be used in `waitall()` to synchronize until the message is delivered (or fails). */ - template + template void isend(const value_t* buf, size_t size, int dest, int tag, request_t* request) const { - impl_->isend(static_cast(buf), size * sizeof(value_t), dest, tag, request); + impl_->isend(static_cast(buf), size * sizeof(value_t), dest, + tag, request); } /** @@ -162,10 +161,11 @@ class comms_t { * @param request pointer to hold returned request_t object. * This will be used in `waitall()` to synchronize until the message is delivered (or fails). */ - template + template void irecv(value_t* buf, size_t size, int source, int tag, request_t* request) const { - impl_->irecv(static_cast(buf), size * sizeof(value_t), source, tag, request); + impl_->irecv(static_cast(buf), size * sizeof(value_t), source, tag, + request); } /** @@ -174,7 +174,7 @@ class comms_t { * @param array_of_requests an array of request_t objects returned from isend/irecv */ void waitall(int count, request_t array_of_requests[]) const { - impl_->waitall(count, array_of_requests); + impl_->waitall(count, array_of_requests); } /** @@ -186,11 +186,12 @@ class comms_t { * @param op reduction operation to perform * @param stream CUDA stream to synchronize operation */ - template + template void allreduce(const value_t* sendbuff, value_t* recvbuff, size_t count, op_t op, cudaStream_t stream) const { - impl_->allreduce(static_cast(sendbuff), static_cast(recvbuff) - , count, get_type(), op, stream); + impl_->allreduce(static_cast(sendbuff), + static_cast(recvbuff), count, get_type(), + op, stream); } /** @@ -201,10 +202,10 @@ class comms_t { * @param root the rank initiating the broadcast * @param stream CUDA stream to synchronize operation */ - template - void bcast(value_t* buff, size_t count, int root, - cudaStream_t stream) const { - impl_->bcast(static_cast(buff), count, get_type(), root, stream); + template + void bcast(value_t* buff, size_t count, int root, cudaStream_t stream) const { + impl_->bcast(static_cast(buff), count, get_type(), root, + stream); } /** @@ -217,12 +218,12 @@ class comms_t { * @param root rank to store the results * @param stream CUDA stream to synchronize operation */ - template - void reduce(const value_t* sendbuff, value_t* recvbuff, size_t count, - op_t op, int root, - cudaStream_t stream) const { - impl_->reduce(static_cast(sendbuff), static_cast(recvbuff), - count, get_type(), op, root, stream); + template + void reduce(const value_t* sendbuff, value_t* recvbuff, size_t count, op_t op, + int root, cudaStream_t stream) const { + impl_->reduce(static_cast(sendbuff), + static_cast(recvbuff), count, get_type(), op, + root, stream); } /** @@ -233,11 +234,12 @@ class comms_t { * @param sendcount number of elements in send buffer * @param stream CUDA stream to synchronize operation */ - template + template void allgather(const value_t* sendbuff, value_t* recvbuff, size_t sendcount, cudaStream_t stream) const { - impl_->allgather(static_cast(sendbuff), static_cast(recvbuff), - sendcount, get_type(), stream); + impl_->allgather(static_cast(sendbuff), + static_cast(recvbuff), sendcount, + get_type(), stream); } /** @@ -251,11 +253,13 @@ class comms_t { * at which to place the incoming data from each rank * @param stream CUDA stream to synchronize operation */ - template - void allgatherv(const value_t* sendbuf, value_t* recvbuf, const size_t recvcounts[], - const int displs[], + template + void allgatherv(const value_t* sendbuf, value_t* recvbuf, + const size_t recvcounts[], const int displs[], cudaStream_t stream) const { - impl_->allgatherv(static_cast(sendbuf), static_cast(recvbuf), recvcounts, displs, get_type(), stream); + impl_->allgatherv(static_cast(sendbuf), + static_cast(recvbuf), recvcounts, displs, + get_type(), stream); } /** @@ -266,11 +270,12 @@ class comms_t { * @param op reduction operation to perform * @param stream CUDA stream to synchronize operation */ - template - void reducescatter(const value_t* sendbuff, value_t* recvbuff, size_t recvcount, - op_t op, cudaStream_t stream) const { - impl_->reducescatter(static_cast(sendbuff), static_cast(recvbuff), - recvcount, get_type() , op, stream); + template + void reducescatter(const value_t* sendbuff, value_t* recvbuff, + size_t recvcount, op_t op, cudaStream_t stream) const { + impl_->reducescatter(static_cast(sendbuff), + static_cast(recvbuff), recvcount, + get_type(), op, stream); } private: diff --git a/cpp/include/raft/comms/comms_helper.hpp b/cpp/include/raft/comms/comms_helper.hpp index d762785fef..2838929f8f 100644 --- a/cpp/include/raft/comms/comms_helper.hpp +++ b/cpp/include/raft/comms/comms_helper.hpp @@ -33,10 +33,11 @@ namespace comms { * @param num_ranks number of ranks in communicator clique * @param rank rank of local instance */ -void build_comms_nccl_only(handle_t *handle, ncclComm_t nccl_comm, int num_ranks, - int rank) { +void build_comms_nccl_only(handle_t *handle, ncclComm_t nccl_comm, + int num_ranks, int rank) { auto d_alloc = handle->get_device_allocator(); - raft::comms::comms_iface *raft_comm = new raft::comms::std_comms(nccl_comm, num_ranks, rank, d_alloc); + raft::comms::comms_iface *raft_comm = + new raft::comms::std_comms(nccl_comm, num_ranks, rank, d_alloc); std::cout << "Comms: " << raft_comm->getSize() << std::endl; auto communicator = @@ -59,8 +60,9 @@ void build_comms_nccl_only(handle_t *handle, ncclComm_t nccl_comm, int num_ranks * @param num_ranks number of ranks in communicator clique * @param rank rank of local instance */ -void build_comms_nccl_ucx(handle_t *handle, ncclComm_t nccl_comm, void *ucp_worker, - void *eps, int num_ranks, int rank) { +void build_comms_nccl_ucx(handle_t *handle, ncclComm_t nccl_comm, + void *ucp_worker, void *eps, int num_ranks, + int rank) { auto eps_sp = std::make_shared(new ucp_ep_h[num_ranks]); auto size_t_ep_arr = reinterpret_cast(eps); @@ -78,8 +80,8 @@ void build_comms_nccl_ucx(handle_t *handle, ncclComm_t nccl_comm, void *ucp_work } auto d_alloc = handle->get_device_allocator(); - auto *raft_comm = new raft::comms::std_comms(nccl_comm, (ucp_worker_h)ucp_worker, - eps_sp, num_ranks, rank, d_alloc); + auto *raft_comm = new raft::comms::std_comms( + nccl_comm, (ucp_worker_h)ucp_worker, eps_sp, num_ranks, rank, d_alloc); auto communicator = std::make_shared(std::unique_ptr(raft_comm)); handle->set_comms(communicator); @@ -100,13 +102,12 @@ bool test_collective_allreduce(const handle_t &handle, int root) { raft::mr::device::buffer temp_d(handle.get_device_allocator(), stream); temp_d.resize(1, stream); - CUDA_CHECK(cudaMemcpyAsync(temp_d.data(), &send, 1, - cudaMemcpyHostToDevice, stream)); - communicator.allreduce(temp_d.data(), temp_d.data(), 1, - op_t::SUM, stream); + CUDA_CHECK( + cudaMemcpyAsync(temp_d.data(), &send, 1, cudaMemcpyHostToDevice, stream)); + communicator.allreduce(temp_d.data(), temp_d.data(), 1, op_t::SUM, stream); int temp_h = 0; - CUDA_CHECK(cudaMemcpyAsync(&temp_h, temp_d.data(), 1, - cudaMemcpyDeviceToHost, stream)); + CUDA_CHECK( + cudaMemcpyAsync(&temp_h, temp_d.data(), 1, cudaMemcpyDeviceToHost, stream)); CUDA_CHECK(cudaStreamSynchronize(stream)); communicator.barrier(); @@ -116,7 +117,6 @@ bool test_collective_allreduce(const handle_t &handle, int root) { return temp_h == communicator.getSize(); } - /** * A simple sanity check that NCCL is able to perform a collective operation * @@ -133,13 +133,13 @@ bool test_collective_broadcast(const handle_t &handle, int root) { raft::mr::device::buffer temp_d(handle.get_device_allocator(), stream); temp_d.resize(1, stream); - if(communicator.getRank() == root) - CUDA_CHECK(cudaMemcpyAsync(temp_d.data(), &send, sizeof(int), - cudaMemcpyHostToDevice, stream)); + if (communicator.getRank() == root) + CUDA_CHECK(cudaMemcpyAsync(temp_d.data(), &send, sizeof(int), + cudaMemcpyHostToDevice, stream)); communicator.bcast(temp_d.data(), 1, root, stream); communicator.syncStream(stream); - int temp_h = -1; // Verify more than one byte is being sent + int temp_h = -1; // Verify more than one byte is being sent CUDA_CHECK(cudaMemcpyAsync(&temp_h, temp_d.data(), sizeof(int), cudaMemcpyDeviceToHost, stream)); CUDA_CHECK(cudaStreamSynchronize(stream)); @@ -162,11 +162,11 @@ bool test_collective_reduce(const handle_t &handle, int root) { temp_d.resize(1, stream); CUDA_CHECK(cudaMemcpyAsync(temp_d.data(), &send, sizeof(int), - cudaMemcpyHostToDevice, stream)); + cudaMemcpyHostToDevice, stream)); communicator.reduce(temp_d.data(), temp_d.data(), 1, op_t::SUM, root, stream); communicator.syncStream(stream); - int temp_h = -1; // Verify more than one byte is being sent + int temp_h = -1; // Verify more than one byte is being sent CUDA_CHECK(cudaMemcpyAsync(&temp_h, temp_d.data(), sizeof(int), cudaMemcpyDeviceToHost, stream)); CUDA_CHECK(cudaStreamSynchronize(stream)); @@ -175,13 +175,12 @@ bool test_collective_reduce(const handle_t &handle, int root) { std::cout << "Clique size: " << communicator.getSize() << std::endl; std::cout << "final_size: " << temp_h << std::endl; - if(communicator.getRank() == root) - return temp_h == root * communicator.getSize(); + if (communicator.getRank() == root) + return temp_h == root * communicator.getSize(); else - return true; + return true; } - bool test_collective_allgather(const handle_t &handle, int root) { const comms_t &communicator = handle.get_comms(); @@ -192,15 +191,18 @@ bool test_collective_allgather(const handle_t &handle, int root) { raft::mr::device::buffer temp_d(handle.get_device_allocator(), stream); temp_d.resize(1, stream); - raft::mr::device::buffer recv_d(handle.get_device_allocator(), stream, communicator.getSize()); + raft::mr::device::buffer recv_d(handle.get_device_allocator(), stream, + communicator.getSize()); CUDA_CHECK(cudaMemcpyAsync(temp_d.data(), &send, sizeof(int), - cudaMemcpyHostToDevice, stream)); + cudaMemcpyHostToDevice, stream)); communicator.allgather(temp_d.data(), recv_d.data(), 1, stream); communicator.syncStream(stream); - int temp_h[communicator.getSize()]; // Verify more than one byte is being sent - CUDA_CHECK(cudaMemcpyAsync(&temp_h, temp_d.data(), sizeof(int)*communicator.getSize(), + int + temp_h[communicator.getSize()]; // Verify more than one byte is being sent + CUDA_CHECK(cudaMemcpyAsync(&temp_h, temp_d.data(), + sizeof(int) * communicator.getSize(), cudaMemcpyDeviceToHost, stream)); CUDA_CHECK(cudaStreamSynchronize(stream)); communicator.barrier(); @@ -208,9 +210,8 @@ bool test_collective_allgather(const handle_t &handle, int root) { std::cout << "Clique size: " << communicator.getSize() << std::endl; std::cout << "final_size: " << temp_h << std::endl; - for(int i = 0; i < communicator.getSize(); i++) - if(temp_h[i] != i) - return false; + for (int i = 0; i < communicator.getSize(); i++) + if (temp_h[i] != i) return false; return true; } @@ -221,15 +222,18 @@ bool test_collective_reducescatter(const handle_t &handle, int root) { cudaStream_t stream = handle.get_stream(); - raft::mr::device::buffer temp_d(handle.get_device_allocator(), stream, 1); - raft::mr::device::buffer recv_d(handle.get_device_allocator(), stream, 1); + raft::mr::device::buffer temp_d(handle.get_device_allocator(), stream, + 1); + raft::mr::device::buffer recv_d(handle.get_device_allocator(), stream, + 1); CUDA_CHECK(cudaMemcpyAsync(temp_d.data(), &send, sizeof(int), - cudaMemcpyHostToDevice, stream)); + cudaMemcpyHostToDevice, stream)); - communicator.reducescatter(temp_d.data(), recv_d.data(), 1, op_t::SUM, stream); + communicator.reducescatter(temp_d.data(), recv_d.data(), 1, op_t::SUM, + stream); communicator.syncStream(stream); - int temp_h = -1; // Verify more than one byte is being sent + int temp_h = -1; // Verify more than one byte is being sent CUDA_CHECK(cudaMemcpyAsync(&temp_h, temp_d.data(), sizeof(int), cudaMemcpyDeviceToHost, stream)); CUDA_CHECK(cudaStreamSynchronize(stream)); @@ -241,7 +245,6 @@ bool test_collective_reducescatter(const handle_t &handle, int root) { return temp_h = communicator.getSize(); } - /** * A simple sanity check that UCX is able to send messages between all ranks * @@ -263,16 +266,15 @@ bool test_pointToPoint_simple_send_recv(const handle_t &h, int numTrials) { //post receives for (int r = 0; r < communicator.getSize(); ++r) { if (r != rank) { - communicator.irecv(received_data.data() + request_idx, 1, r, - 0, requests.data() + request_idx); + communicator.irecv(received_data.data() + request_idx, 1, r, 0, + requests.data() + request_idx); ++request_idx; } } for (int r = 0; r < communicator.getSize(); ++r) { if (r != rank) { - communicator.isend(&rank, 1, r, 0, - requests.data() + request_idx); + communicator.isend(&rank, 1, r, 0, requests.data() + request_idx); ++request_idx; } } From 3d362d00834eec17b4dfd9db3ec28287d4b3f269 Mon Sep 17 00:00:00 2001 From: "Corey J. Nolet" Date: Tue, 19 May 2020 14:37:29 -0400 Subject: [PATCH 36/46] Updating tabbing for pytests --- python/raft/test/test_comms.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/raft/test/test_comms.py b/python/raft/test/test_comms.py index d3a6c20ffc..b065d098af 100644 --- a/python/raft/test/test_comms.py +++ b/python/raft/test/test_comms.py @@ -107,7 +107,7 @@ def test_collectives(cluster, func): for w in cb.worker_addresses] wait(dfs, timeout=5) - assert all([x.result() for x in dfs]) + assert all([x.result() for x in dfs]) finally: cb.destroy() From 417a4bd43d92a71e74d0fc2b7b522c3c880d380e Mon Sep 17 00:00:00 2001 From: "Corey J. Nolet" Date: Tue, 19 May 2020 15:53:07 -0400 Subject: [PATCH 37/46] Following clang tidy standards --- cpp/include/raft/comms/comms.hpp | 20 ++++----- cpp/include/raft/comms/comms_helper.hpp | 58 ++++++++++++------------- cpp/include/raft/comms/std_comms.hpp | 12 ++--- cpp/include/raft/handle.hpp | 10 ++--- 4 files changed, 50 insertions(+), 50 deletions(-) diff --git a/cpp/include/raft/comms/comms.hpp b/cpp/include/raft/comms/comms.hpp index 5c47316670..93b4dab3dd 100644 --- a/cpp/include/raft/comms/comms.hpp +++ b/cpp/include/raft/comms/comms.hpp @@ -47,13 +47,13 @@ class comms_iface { public: virtual ~comms_iface(); - virtual int getSize() const = 0; - virtual int getRank() const = 0; + virtual int get_size() const = 0; + virtual int get_rank() const = 0; - virtual std::unique_ptr commSplit(int color, int key) const = 0; + virtual std::unique_ptr comm_split(int color, int key) const = 0; virtual void barrier() const = 0; - virtual status_t syncStream(cudaStream_t stream) const = 0; + virtual status_t sync_stream(cudaStream_t stream) const = 0; virtual void isend(const void* buf, size_t size, int dest, int tag, request_t* request) const = 0; @@ -99,12 +99,12 @@ class comms_t { * Returns the size of the communicator clique */ - int getSize() const { return impl_->getSize(); } + int get_size() const { return impl_->get_size(); } /** * Returns the local rank */ - int getRank() const { return impl_->getRank(); } + int get_rank() const { return impl_->get_rank(); } /** * Splits the current communicator clique into sub-cliques matching @@ -113,8 +113,8 @@ class comms_t { * @param color ranks w/ the same color are placed in the same communicator * @param key controls rank assignment */ - std::unique_ptr commSplit(int color, int key) const { - return impl_->commSplit(color, key); + std::unique_ptr comm_split(int color, int key) const { + return impl_->comm_split(color, key); } /** @@ -130,8 +130,8 @@ class comms_t { * * @param stream the cuda stream to sync collective operations on */ - status_t syncStream(cudaStream_t stream) const { - return impl_->syncStream(stream); + status_t sync_stream(cudaStream_t stream) const { + return impl_->sync_stream(stream); } /** diff --git a/cpp/include/raft/comms/comms_helper.hpp b/cpp/include/raft/comms/comms_helper.hpp index 2838929f8f..6e6ba942cc 100644 --- a/cpp/include/raft/comms/comms_helper.hpp +++ b/cpp/include/raft/comms/comms_helper.hpp @@ -38,7 +38,7 @@ void build_comms_nccl_only(handle_t *handle, ncclComm_t nccl_comm, auto d_alloc = handle->get_device_allocator(); raft::comms::comms_iface *raft_comm = new raft::comms::std_comms(nccl_comm, num_ranks, rank, d_alloc); - std::cout << "Comms: " << raft_comm->getSize() << std::endl; + std::cout << "Comms: " << raft_comm->get_size() << std::endl; auto communicator = std::make_shared(std::unique_ptr(raft_comm)); @@ -111,10 +111,10 @@ bool test_collective_allreduce(const handle_t &handle, int root) { CUDA_CHECK(cudaStreamSynchronize(stream)); communicator.barrier(); - std::cout << "Clique size: " << communicator.getSize() << std::endl; + std::cout << "Clique size: " << communicator.get_size() << std::endl; std::cout << "final_size: " << temp_h << std::endl; - return temp_h == communicator.getSize(); + return temp_h == communicator.get_size(); } /** @@ -133,19 +133,19 @@ bool test_collective_broadcast(const handle_t &handle, int root) { raft::mr::device::buffer temp_d(handle.get_device_allocator(), stream); temp_d.resize(1, stream); - if (communicator.getRank() == root) + if (communicator.get_rank() == root) CUDA_CHECK(cudaMemcpyAsync(temp_d.data(), &send, sizeof(int), cudaMemcpyHostToDevice, stream)); communicator.bcast(temp_d.data(), 1, root, stream); - communicator.syncStream(stream); + communicator.sync_stream(stream); int temp_h = -1; // Verify more than one byte is being sent CUDA_CHECK(cudaMemcpyAsync(&temp_h, temp_d.data(), sizeof(int), cudaMemcpyDeviceToHost, stream)); CUDA_CHECK(cudaStreamSynchronize(stream)); communicator.barrier(); - std::cout << "Clique size: " << communicator.getSize() << std::endl; + std::cout << "Clique size: " << communicator.get_size() << std::endl; std::cout << "final_size: " << temp_h << std::endl; return temp_h == root; @@ -165,18 +165,18 @@ bool test_collective_reduce(const handle_t &handle, int root) { cudaMemcpyHostToDevice, stream)); communicator.reduce(temp_d.data(), temp_d.data(), 1, op_t::SUM, root, stream); - communicator.syncStream(stream); + communicator.sync_stream(stream); int temp_h = -1; // Verify more than one byte is being sent CUDA_CHECK(cudaMemcpyAsync(&temp_h, temp_d.data(), sizeof(int), cudaMemcpyDeviceToHost, stream)); CUDA_CHECK(cudaStreamSynchronize(stream)); communicator.barrier(); - std::cout << "Clique size: " << communicator.getSize() << std::endl; + std::cout << "Clique size: " << communicator.get_size() << std::endl; std::cout << "final_size: " << temp_h << std::endl; - if (communicator.getRank() == root) - return temp_h == root * communicator.getSize(); + if (communicator.get_rank() == root) + return temp_h == root * communicator.get_size(); else return true; } @@ -192,25 +192,25 @@ bool test_collective_allgather(const handle_t &handle, int root) { temp_d.resize(1, stream); raft::mr::device::buffer recv_d(handle.get_device_allocator(), stream, - communicator.getSize()); + communicator.get_size()); CUDA_CHECK(cudaMemcpyAsync(temp_d.data(), &send, sizeof(int), cudaMemcpyHostToDevice, stream)); communicator.allgather(temp_d.data(), recv_d.data(), 1, stream); - communicator.syncStream(stream); + communicator.sync_stream(stream); int - temp_h[communicator.getSize()]; // Verify more than one byte is being sent + temp_h[communicator.get_size()]; // Verify more than one byte is being sent CUDA_CHECK(cudaMemcpyAsync(&temp_h, temp_d.data(), - sizeof(int) * communicator.getSize(), + sizeof(int) * communicator.get_size(), cudaMemcpyDeviceToHost, stream)); CUDA_CHECK(cudaStreamSynchronize(stream)); communicator.barrier(); - std::cout << "Clique size: " << communicator.getSize() << std::endl; + std::cout << "Clique size: " << communicator.get_size() << std::endl; std::cout << "final_size: " << temp_h << std::endl; - for (int i = 0; i < communicator.getSize(); i++) + for (int i = 0; i < communicator.get_size(); i++) if (temp_h[i] != i) return false; return true; } @@ -232,17 +232,17 @@ bool test_collective_reducescatter(const handle_t &handle, int root) { communicator.reducescatter(temp_d.data(), recv_d.data(), 1, op_t::SUM, stream); - communicator.syncStream(stream); + communicator.sync_stream(stream); int temp_h = -1; // Verify more than one byte is being sent CUDA_CHECK(cudaMemcpyAsync(&temp_h, temp_d.data(), sizeof(int), cudaMemcpyDeviceToHost, stream)); CUDA_CHECK(cudaStreamSynchronize(stream)); communicator.barrier(); - std::cout << "Clique size: " << communicator.getSize() << std::endl; + std::cout << "Clique size: " << communicator.get_size() << std::endl; std::cout << "final_size: " << temp_h << std::endl; - return temp_h = communicator.getSize(); + return temp_h = communicator.get_size(); } /** @@ -254,17 +254,17 @@ bool test_collective_reducescatter(const handle_t &handle, int root) { */ bool test_pointToPoint_simple_send_recv(const handle_t &h, int numTrials) { const comms_t &communicator = h.get_comms(); - const int rank = communicator.getRank(); + const int rank = communicator.get_rank(); bool ret = true; for (int i = 0; i < numTrials; i++) { - std::vector received_data((communicator.getSize() - 1), -1); + std::vector received_data((communicator.get_size() - 1), -1); std::vector requests; - requests.resize(2 * (communicator.getSize() - 1)); + requests.resize(2 * (communicator.get_size() - 1)); int request_idx = 0; //post receives - for (int r = 0; r < communicator.getSize(); ++r) { + for (int r = 0; r < communicator.get_size(); ++r) { if (r != rank) { communicator.irecv(received_data.data() + request_idx, 1, r, 0, requests.data() + request_idx); @@ -272,7 +272,7 @@ bool test_pointToPoint_simple_send_recv(const handle_t &h, int numTrials) { } } - for (int r = 0; r < communicator.getSize(); ++r) { + for (int r = 0; r < communicator.get_size(); ++r) { if (r != rank) { communicator.isend(&rank, 1, r, 0, requests.data() + request_idx); ++request_idx; @@ -282,14 +282,14 @@ bool test_pointToPoint_simple_send_recv(const handle_t &h, int numTrials) { communicator.waitall(requests.size(), requests.data()); communicator.barrier(); - if (communicator.getRank() == 0) { + if (communicator.get_rank() == 0) { std::cout << "=========================" << std::endl; std::cout << "Trial " << i << std::endl; } - for (int printrank = 0; printrank < communicator.getSize(); ++printrank) { - if (communicator.getRank() == printrank) { - std::cout << "Rank " << communicator.getRank() << " received: ["; + for (int printrank = 0; printrank < communicator.get_size(); ++printrank) { + if (communicator.get_rank() == printrank) { + std::cout << "Rank " << communicator.get_rank() << " received: ["; for (int i = 0; i < received_data.size(); i++) { auto rec = received_data[i]; std::cout << rec; @@ -303,7 +303,7 @@ bool test_pointToPoint_simple_send_recv(const handle_t &h, int numTrials) { communicator.barrier(); } - if (communicator.getRank() == 0) + if (communicator.get_rank() == 0) std::cout << "=========================" << std::endl; } diff --git a/cpp/include/raft/comms/std_comms.hpp b/cpp/include/raft/comms/std_comms.hpp index ecd589a32c..c20bc76bf6 100644 --- a/cpp/include/raft/comms/std_comms.hpp +++ b/cpp/include/raft/comms/std_comms.hpp @@ -176,11 +176,11 @@ class std_comms : public comms_iface { device_allocator_->allocate(sizeof(int), stream_)); } - int getSize() const { return num_ranks_; } + int get_size() const { return num_ranks_; } - int getRank() const { return rank_; } + int get_rank() const { return rank_; } - std::unique_ptr commSplit(int color, int key) const { + std::unique_ptr comm_split(int color, int key) const { // Not supported by NCCL ASSERT(false, "ERROR: commSplit called but not yet supported in this comms " @@ -193,7 +193,7 @@ class std_comms : public comms_iface { allreduce(sendbuff_, recvbuff_, 1, datatype_t::INT32, op_t::SUM, stream_); - ASSERT(syncStream(stream_) == status_t::commStatusSuccess, + ASSERT(sync_stream(stream_) == status_t::commStatusSuccess, "ERROR: syncStream failed. This can be caused by a failed rank_."); } @@ -221,7 +221,7 @@ class std_comms : public comms_iface { ucp_request *ucp_req = (ucp_request *)malloc(sizeof(ucp_request)); this->ucp_handler_.ucp_isend(ucp_req, ep_ptr, buf, size, tag, - default_tag_mask, getRank()); + default_tag_mask, get_rank()); requests_in_flight_.insert(std::make_pair(*request, ucp_req)); } @@ -362,7 +362,7 @@ class std_comms : public comms_iface { nccl_comm_, stream)); } - status_t syncStream(cudaStream_t stream) const { + status_t sync_stream(cudaStream_t stream) const { cudaError_t cudaErr; ncclResult_t ncclErr, ncclAsyncErr; while (1) { diff --git a/cpp/include/raft/handle.hpp b/cpp/include/raft/handle.hpp index f8db324d90..01d6d7a83f 100644 --- a/cpp/include/raft/handle.hpp +++ b/cpp/include/raft/handle.hpp @@ -150,16 +150,16 @@ class handle_t { } void set_comms(std::shared_ptr communicator) { - _communicator = communicator; + communicator_ = communicator; } const comms::comms_t& get_comms() const { - ASSERT(nullptr != _communicator.get(), + ASSERT(nullptr != communicator_.get(), "ERROR: Communicator was not initialized\n"); - return *_communicator; + return *communicator_; } - bool comms_initialized() const { return (nullptr != _communicator.get()); } + bool comms_initialized() const { return (nullptr != communicator_.get()); } const cudaDeviceProp& get_device_properties() const { std::lock_guard _(mutex_); @@ -171,7 +171,7 @@ class handle_t { } private: - std::shared_ptr _communicator; + std::shared_ptr communicator_; const int dev_id_; const int num_streams_; From cb92349f2cfb993b43c4884356b227b10493f04b Mon Sep 17 00:00:00 2001 From: "Corey J. Nolet" Date: Tue, 19 May 2020 18:51:38 -0400 Subject: [PATCH 38/46] Moving get_type out of comms_t --- cpp/include/raft/comms/comms.hpp | 86 +++++++++++++++++--------------- 1 file changed, 45 insertions(+), 41 deletions(-) diff --git a/cpp/include/raft/comms/comms.hpp b/cpp/include/raft/comms/comms.hpp index 93b4dab3dd..42827ef361 100644 --- a/cpp/include/raft/comms/comms.hpp +++ b/cpp/include/raft/comms/comms.hpp @@ -43,6 +43,51 @@ enum class status_t { commStatusAbort // A failure occurred in sync, queued operations aborted }; +template +constexpr datatype_t get_type(); + + +template <> +constexpr datatype_t get_type() { + return datatype_t::CHAR; +} + +template <> +constexpr datatype_t get_type() { + return datatype_t::UINT8; +} + +template <> +constexpr datatype_t get_type() { + return datatype_t::INT32; +} + +template <> +constexpr datatype_t get_type() { + return datatype_t::UINT32; +} + +template <> +constexpr datatype_t get_type() { + return datatype_t::INT64; +} + +template <> +constexpr datatype_t get_type() { + return datatype_t::UINT64; +} + +template <> +constexpr datatype_t get_type() { + return datatype_t::FLOAT32; +} + +template <> +constexpr datatype_t get_type() { + return datatype_t::FLOAT64; +} + + class comms_iface { public: virtual ~comms_iface(); @@ -92,9 +137,6 @@ class comms_t { ASSERT(nullptr != impl_.get(), "ERROR: Invalid comms_iface used!"); } - template - datatype_t get_type() const; - /** * Returns the size of the communicator clique */ @@ -284,44 +326,6 @@ class comms_t { comms_iface::~comms_iface() {} -template <> -constexpr datatype_t comms_t::get_type() const { - return datatype_t::CHAR; -} -template <> -constexpr datatype_t comms_t::get_type() const { - return datatype_t::UINT8; -} - -template <> -constexpr datatype_t comms_t::get_type() const { - return datatype_t::INT32; -} - -template <> -constexpr datatype_t comms_t::get_type() const { - return datatype_t::UINT32; -} - -template <> -constexpr datatype_t comms_t::get_type() const { - return datatype_t::INT64; -} - -template <> -constexpr datatype_t comms_t::get_type() const { - return datatype_t::UINT64; -} - -template <> -constexpr datatype_t comms_t::get_type() const { - return datatype_t::FLOAT32; -} - -template <> -constexpr datatype_t comms_t::get_type() const { - return datatype_t::FLOAT64; -} } // namespace comms } // namespace raft From 6e9025d03f67e96b80989f5e3b6c41cc1932f08b Mon Sep 17 00:00:00 2001 From: "Corey J. Nolet" Date: Tue, 26 May 2020 15:42:52 -0400 Subject: [PATCH 39/46] More review feedback --- cpp/CMakeLists.txt | 4 +- .../CMakeLists.txt => cmake/comms.cmake} | 2 +- cpp/include/raft/comms/comms.hpp | 8 +- cpp/include/raft/comms/helper.hpp | 104 ++++++++++++++++++ cpp/include/raft/comms/nccl_helper.hpp | 34 ------ cpp/include/raft/comms/std_comms.hpp | 28 ++--- .../raft/comms/{comms_helper.hpp => test.hpp} | 68 +----------- python/raft/common/__init__.py | 4 +- python/raft/dask/__init__.py | 2 +- python/raft/dask/common/__init__.py | 20 ++-- python/raft/dask/common/comms_utils.pyx | 4 +- python/raft/dask/common/nccl.pyx | 10 +- python/raft/test/conftest.py | 4 - 13 files changed, 149 insertions(+), 143 deletions(-) rename cpp/{include/raft/comms/CMakeLists.txt => cmake/comms.cmake} (95%) create mode 100644 cpp/include/raft/comms/helper.hpp delete mode 100644 cpp/include/raft/comms/nccl_helper.hpp rename cpp/include/raft/comms/{comms_helper.hpp => test.hpp} (77%) diff --git a/cpp/CMakeLists.txt b/cpp/CMakeLists.txt index f032e3a7e4..a45cf8c950 100644 --- a/cpp/CMakeLists.txt +++ b/cpp/CMakeLists.txt @@ -67,6 +67,8 @@ find_package(CUDA 10.0 REQUIRED) set(CMAKE_MODULE_PATH ${CMAKE_CURRENT_SOURCE_DIR}/cmake) +message("HELLO!") + ############################################################################## # - Compiler Options -------------------------------------------------------- @@ -161,7 +163,7 @@ set(CMAKE_CUDA_FLAGS # - dependencies ------------------------------------------------------------- include(cmake/Dependencies.cmake) -add_subdirectory(include/raft/comms) +include(cmake/comms.cmake) ############################################################################## # - include paths ------------------------------------------------------------ diff --git a/cpp/include/raft/comms/CMakeLists.txt b/cpp/cmake/comms.cmake similarity index 95% rename from cpp/include/raft/comms/CMakeLists.txt rename to cpp/cmake/comms.cmake index 734ce11812..c8496c7dc6 100644 --- a/cpp/include/raft/comms/CMakeLists.txt +++ b/cpp/cmake/comms.cmake @@ -34,4 +34,4 @@ find_package(UCX) include_directories(${UCX_INCLUDE_DIRS}) include_directories( ${NCCL_INCLUDE_DIRS} ) -list(APPEND RAFT_LINK_LIBRARIES ${NCCL_LIBRARIES}) \ No newline at end of file +list(APPEND RAFT_LINK_LIBRARIES ${NCCL_LIBRARIES}) diff --git a/cpp/include/raft/comms/comms.hpp b/cpp/include/raft/comms/comms.hpp index 42827ef361..8f675ddfe4 100644 --- a/cpp/include/raft/comms/comms.hpp +++ b/cpp/include/raft/comms/comms.hpp @@ -17,6 +17,7 @@ #pragma once #include +#include namespace raft { namespace comms { @@ -38,15 +39,14 @@ enum class op_t { SUM, PROD, MIN, MAX }; * The resulting status of distributed stream synchronization */ enum class status_t { - commStatusSuccess, // Synchronization successful - commStatusError, // An error occured querying sync status - commStatusAbort // A failure occurred in sync, queued operations aborted + SUCCESS, // Synchronization successful + ERROR, // An error occured querying sync status + ABORT // A failure occurred in sync, queued operations aborted }; template constexpr datatype_t get_type(); - template <> constexpr datatype_t get_type() { return datatype_t::CHAR; diff --git a/cpp/include/raft/comms/helper.hpp b/cpp/include/raft/comms/helper.hpp new file mode 100644 index 0000000000..67ba4cab0a --- /dev/null +++ b/cpp/include/raft/comms/helper.hpp @@ -0,0 +1,104 @@ +/* + * Copyright (c) 2020, NVIDIA CORPORATION. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#pragma once + +#include +#include +#include +#include +#include +#include + +namespace raft { +namespace comms { + +/** + * Function to construct comms_t and inject it on a handle_t. This + * is used for convenience in the Python layer. + * + * @param handle raft::handle_t for injecting the comms + * @param nccl_comm initialized NCCL communicator to use for collectives + * @param num_ranks number of ranks in communicator clique + * @param rank rank of local instance + */ +void build_comms_nccl_only(handle_t *handle, ncclComm_t nccl_comm, + int num_ranks, int rank) { + auto d_alloc = handle->get_device_allocator(); + cudaStream_t stream = handle->get_stream(); + comms_iface *raft_comm = new raft::comms::std_comms(nccl_comm, num_ranks, rank, d_alloc, stream); + + auto communicator = + std::make_shared(std::unique_ptr(raft_comm)); + handle->set_comms(communicator); +} + +/** + * Function to construct comms_t and inject it on a handle_t. This + * is used for convenience in the Python layer. + * + * @param handle raft::handle_t for injecting the comms + * @param nccl_comm initialized NCCL communicator to use for collectives + * @param ucp_worker of local process + * Note: This is purposefully left as void* so that the ucp_worker_h + * doesn't need to be exposed through the cython layer + * @param eps array of ucp_ep_h instances. + * Note: This is purposefully left as void* so that + * the ucp_ep_h doesn't need to be exposed through the cython layer. + * @param num_ranks number of ranks in communicator clique + * @param rank rank of local instance + */ +void build_comms_nccl_ucx(handle_t *handle, ncclComm_t nccl_comm, + void *ucp_worker, void *eps, int num_ranks, + int rank) { + auto eps_sp = std::make_shared(new ucp_ep_h[num_ranks]); + + auto size_t_ep_arr = reinterpret_cast(eps); + + for (int i = 0; i < num_ranks; i++) { + size_t ptr = size_t_ep_arr[i]; + auto ucp_ep_v = reinterpret_cast(*eps_sp); + + if (ptr != 0) { + auto eps_ptr = reinterpret_cast(size_t_ep_arr[i]); + ucp_ep_v[i] = eps_ptr; + } else { + ucp_ep_v[i] = nullptr; + } + } + + auto d_alloc = handle->get_device_allocator(); + cudaStream_t stream = handle->get_stream(); + + auto *raft_comm = new raft::comms::std_comms( + nccl_comm, (ucp_worker_h)ucp_worker, eps_sp, num_ranks, rank, d_alloc, stream); + auto communicator = + std::make_shared(std::unique_ptr(raft_comm)); + handle->set_comms(communicator); +} + +inline void nccl_unique_id_from_char(ncclUniqueId *id, char *uniqueId, int size) { + memcpy(id->internal, uniqueId, size); +} + +inline void get_unique_id(char *uid, int size) { + ncclUniqueId id; + ncclGetUniqueId(&id); + + memcpy(uid, id.internal, size); +} +}; // namespace comms +}; // end namespace raft diff --git a/cpp/include/raft/comms/nccl_helper.hpp b/cpp/include/raft/comms/nccl_helper.hpp deleted file mode 100644 index d7a14ba8ba..0000000000 --- a/cpp/include/raft/comms/nccl_helper.hpp +++ /dev/null @@ -1,34 +0,0 @@ -/* - * Copyright (c) 2020, NVIDIA CORPORATION. - * - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -#pragma once - -#include - -namespace raft { -namespace comms { -inline void ncclUniqueIdFromChar(ncclUniqueId *id, char *uniqueId, int size) { - memcpy(id->internal, uniqueId, size); -} - -inline void get_unique_id(char *uid, int size) { - ncclUniqueId id; - ncclGetUniqueId(&id); - - memcpy(uid, id.internal, size); -} -} // namespace comms -} // namespace raft diff --git a/cpp/include/raft/comms/std_comms.hpp b/cpp/include/raft/comms/std_comms.hpp index c20bc76bf6..ef83730749 100644 --- a/cpp/include/raft/comms/std_comms.hpp +++ b/cpp/include/raft/comms/std_comms.hpp @@ -85,7 +85,7 @@ constexpr size_t get_datatype_size(const datatype_t datatype) { } } -ncclDataType_t get_nccl_datatype(const datatype_t datatype) { +constexpr ncclDataType_t get_nccl_datatype(const datatype_t datatype) { switch (datatype) { case datatype_t::CHAR: return ncclChar; @@ -107,7 +107,7 @@ ncclDataType_t get_nccl_datatype(const datatype_t datatype) { } } -ncclRedOp_t get_nccl_op(const op_t op) { +constexpr ncclRedOp_t get_nccl_op(const op_t op) { switch (op) { case op_t::SUM: return ncclSum; @@ -134,13 +134,15 @@ class std_comms : public comms_iface { */ std_comms(ncclComm_t nccl_comm, ucp_worker_h ucp_worker, std::shared_ptr eps, int num_ranks, int rank, - const std::shared_ptr device_allocator) + const std::shared_ptr device_allocator, + cudaStream_t stream) : nccl_comm_(nccl_comm), ucp_worker_(ucp_worker), ucp_eps_(eps), num_ranks_(num_ranks), rank_(rank), device_allocator_(device_allocator), + stream_(stream), next_request_id_(0) { initialize(); }; @@ -152,24 +154,22 @@ class std_comms : public comms_iface { * @param rank rank of the current worker */ std_comms(const ncclComm_t nccl_comm, int num_ranks, int rank, - const std::shared_ptr device_allocator) + const std::shared_ptr device_allocator, + cudaStream_t stream) : nccl_comm_(nccl_comm), num_ranks_(num_ranks), rank_(rank), - device_allocator_(device_allocator) { + device_allocator_(device_allocator), + stream_(stream) { initialize(); }; virtual ~std_comms() { - CUDA_CHECK_NO_THROW(cudaStreamDestroy(stream_)); - device_allocator_->deallocate(sendbuff_, sizeof(int), stream_); device_allocator_->deallocate(recvbuff_, sizeof(int), stream_); } void initialize() { - CUDA_CHECK(cudaStreamCreate(&stream_)); - sendbuff_ = reinterpret_cast( device_allocator_->allocate(sizeof(int), stream_)); recvbuff_ = reinterpret_cast( @@ -193,7 +193,7 @@ class std_comms : public comms_iface { allreduce(sendbuff_, recvbuff_, 1, datatype_t::INT32, op_t::SUM, stream_); - ASSERT(sync_stream(stream_) == status_t::commStatusSuccess, + ASSERT(sync_stream(stream_) == status_t::SUCCESS, "ERROR: syncStream failed. This can be caused by a failed rank_."); } @@ -367,17 +367,17 @@ class std_comms : public comms_iface { ncclResult_t ncclErr, ncclAsyncErr; while (1) { cudaErr = cudaStreamQuery(stream); - if (cudaErr == cudaSuccess) return status_t::commStatusSuccess; + if (cudaErr == cudaSuccess) return status_t::SUCCESS; if (cudaErr != cudaErrorNotReady) { // An error occurred querying the status of the stream_ - return status_t::commStatusError; + return status_t::ERROR; } ncclErr = ncclCommGetAsyncError(nccl_comm_, &ncclAsyncErr); if (ncclErr != ncclSuccess) { // An error occurred retrieving the asynchronous error - return status_t::commStatusError; + return status_t::ERROR; } if (ncclAsyncErr != ncclSuccess) { @@ -386,7 +386,7 @@ class std_comms : public comms_iface { ncclErr = ncclCommAbort(nccl_comm_); if (ncclErr != ncclSuccess) // Caller may abort with an exception or try to re-create a new communicator. - return status_t::commStatusAbort; + return status_t::ABORT; } // Let other threads (including NCCL threads) use the CPU. diff --git a/cpp/include/raft/comms/comms_helper.hpp b/cpp/include/raft/comms/test.hpp similarity index 77% rename from cpp/include/raft/comms/comms_helper.hpp rename to cpp/include/raft/comms/test.hpp index 6e6ba942cc..a3bf81a007 100644 --- a/cpp/include/raft/comms/comms_helper.hpp +++ b/cpp/include/raft/comms/test.hpp @@ -24,69 +24,6 @@ namespace raft { namespace comms { -/** - * Function to construct comms_t and inject it on a handle_t. This - * is used for convenience in the Python layer. - * - * @param handle raft::handle_t for injecting the comms - * @param nccl_comm initialized NCCL communicator to use for collectives - * @param num_ranks number of ranks in communicator clique - * @param rank rank of local instance - */ -void build_comms_nccl_only(handle_t *handle, ncclComm_t nccl_comm, - int num_ranks, int rank) { - auto d_alloc = handle->get_device_allocator(); - raft::comms::comms_iface *raft_comm = - new raft::comms::std_comms(nccl_comm, num_ranks, rank, d_alloc); - std::cout << "Comms: " << raft_comm->get_size() << std::endl; - - auto communicator = - std::make_shared(std::unique_ptr(raft_comm)); - handle->set_comms(communicator); -} - -/** - * Function to construct comms_t and inject it on a handle_t. This - * is used for convenience in the Python layer. - * - * @param handle raft::handle_t for injecting the comms - * @param nccl_comm initialized NCCL communicator to use for collectives - * @param ucp_worker of local process - * Note: This is purposefully left as void* so that the ucp_worker_h - * doesn't need to be exposed through the cython layer - * @param eps array of ucp_ep_h instances. - * Note: This is purposefully left as void* so that - * the ucp_ep_h doesn't need to be exposed through the cython layer. - * @param num_ranks number of ranks in communicator clique - * @param rank rank of local instance - */ -void build_comms_nccl_ucx(handle_t *handle, ncclComm_t nccl_comm, - void *ucp_worker, void *eps, int num_ranks, - int rank) { - auto eps_sp = std::make_shared(new ucp_ep_h[num_ranks]); - - auto size_t_ep_arr = reinterpret_cast(eps); - - for (int i = 0; i < num_ranks; i++) { - size_t ptr = size_t_ep_arr[i]; - auto ucp_ep_v = reinterpret_cast(*eps_sp); - - if (ptr != 0) { - auto eps_ptr = reinterpret_cast(size_t_ep_arr[i]); - ucp_ep_v[i] = eps_ptr; - } else { - ucp_ep_v[i] = nullptr; - } - } - - auto d_alloc = handle->get_device_allocator(); - auto *raft_comm = new raft::comms::std_comms( - nccl_comm, (ucp_worker_h)ucp_worker, eps_sp, num_ranks, rank, d_alloc); - auto communicator = - std::make_shared(std::unique_ptr(raft_comm)); - handle->set_comms(communicator); -} - /** * A simple sanity check that NCCL is able to perform a collective operation * @@ -309,6 +246,5 @@ bool test_pointToPoint_simple_send_recv(const handle_t &h, int numTrials) { return ret; } - -}; // namespace comms -}; // end namespace raft +} +}; diff --git a/python/raft/common/__init__.py b/python/raft/common/__init__.py index ac84a7a93a..b5ef2b3079 100644 --- a/python/raft/common/__init__.py +++ b/python/raft/common/__init__.py @@ -13,5 +13,5 @@ # limitations under the License. # -from raft.common.cuda import Stream -from raft.common.handle import Handle \ No newline at end of file +from .cuda import Stream +from .handle import Handle \ No newline at end of file diff --git a/python/raft/dask/__init__.py b/python/raft/dask/__init__.py index e58ac25f47..74231d256f 100644 --- a/python/raft/dask/__init__.py +++ b/python/raft/dask/__init__.py @@ -13,4 +13,4 @@ # limitations under the License. # -from raft.dask.common.comms import Comms \ No newline at end of file +from .common.comms import Comms \ No newline at end of file diff --git a/python/raft/dask/common/__init__.py b/python/raft/dask/common/__init__.py index 37981b70fb..1aed648e4d 100644 --- a/python/raft/dask/common/__init__.py +++ b/python/raft/dask/common/__init__.py @@ -13,14 +13,14 @@ # limitations under the License. # -from raft.dask.common.comms import Comms -from raft.dask.common.comms import local_handle +from .comms import Comms +from .comms import local_handle -from raft.dask.common.comms_utils import inject_comms_on_handle -from raft.dask.common.comms_utils import inject_comms_on_handle_coll_only -from raft.dask.common.comms_utils import perform_test_comms_allreduce -from raft.dask.common.comms_utils import perform_test_comms_send_recv -from raft.dask.common.comms_utils import perform_test_comms_allgather -from raft.dask.common.comms_utils import perform_test_comms_bcast -from raft.dask.common.comms_utils import perform_test_comms_reduce -from raft.dask.common.comms_utils import perform_test_comms_reducescatter +from .comms_utils import inject_comms_on_handle +from .comms_utils import inject_comms_on_handle_coll_only +from .comms_utils import perform_test_comms_allreduce +from .comms_utils import perform_test_comms_send_recv +from .comms_utils import perform_test_comms_allgather +from .comms_utils import perform_test_comms_bcast +from .comms_utils import perform_test_comms_reduce +from .comms_utils import perform_test_comms_reducescatter diff --git a/python/raft/dask/common/comms_utils.pyx b/python/raft/dask/common/comms_utils.pyx index 8272446529..cc62fc66ca 100644 --- a/python/raft/dask/common/comms_utils.pyx +++ b/python/raft/dask/common/comms_utils.pyx @@ -41,7 +41,7 @@ cdef extern from "raft/comms/std_comms.hpp" namespace "raft::comms": cdef cppclass std_comms: pass -cdef extern from "raft/comms/comms_helper.hpp" namespace "raft::comms": +cdef extern from "raft/comms/helper.hpp" namespace "raft::comms": void build_comms_nccl_ucx(handle_t *handle, ncclComm_t comm, @@ -55,6 +55,8 @@ cdef extern from "raft/comms/comms_helper.hpp" namespace "raft::comms": int size, int rank) except + +cdef extern from "raft/comms/test.hpp" namespace "raft::comms": + bool test_collective_allreduce(const handle_t &h, int root) except + bool test_collective_broadcast(const handle_t &h, int root) except + bool test_collective_reduce(const handle_t &h, int root) except + diff --git a/python/raft/dask/common/nccl.pyx b/python/raft/dask/common/nccl.pyx index b72bd3d80b..d55a0e4c42 100644 --- a/python/raft/dask/common/nccl.pyx +++ b/python/raft/dask/common/nccl.pyx @@ -25,11 +25,11 @@ from cython.operator cimport dereference as deref from libcpp cimport bool from libc.stdlib cimport malloc, free -cdef extern from "raft/comms/nccl_helper.hpp" namespace "raft::comms": +cdef extern from "raft/comms/helper.hpp" namespace "raft::comms": void get_unique_id(char *uid, int size) except + - void ncclUniqueIdFromChar(ncclUniqueId *id, - char *uniqueId, - int size) except + + void nccl_unique_id_from_char(ncclUniqueId *id, + char *uniqueId, + int size) except + cdef extern from "nccl.h": @@ -132,7 +132,7 @@ cdef class nccl: self.rank = rank cdef ncclUniqueId *ident = malloc(sizeof(ncclUniqueId)) - ncclUniqueIdFromChar(ident, commId, NCCL_UNIQUE_ID_BYTES) + nccl_unique_id_from_char(ident, commId, NCCL_UNIQUE_ID_BYTES) comm_ = self.comm diff --git a/python/raft/test/conftest.py b/python/raft/test/conftest.py index 83ed6b5d83..68ad9b434f 100644 --- a/python/raft/test/conftest.py +++ b/python/raft/test/conftest.py @@ -12,13 +12,9 @@ @pytest.fixture(scope="module") def cluster(): - - print("Starting cluster") cluster = LocalCUDACluster(protocol="tcp", scheduler_port=0) yield cluster - print("Closing cluster") cluster.close() - print("Closed cluster") @pytest.fixture(scope="module") From 28f8101eddb083d017d4a11cae3d5f4d6a1a5f9a Mon Sep 17 00:00:00 2001 From: "Corey J. Nolet" Date: Tue, 26 May 2020 15:43:23 -0400 Subject: [PATCH 40/46] Running cpp style check --- cpp/include/raft/comms/comms.hpp | 4 +--- cpp/include/raft/comms/helper.hpp | 13 ++++++++----- cpp/include/raft/comms/test.hpp | 4 ++-- 3 files changed, 11 insertions(+), 10 deletions(-) diff --git a/cpp/include/raft/comms/comms.hpp b/cpp/include/raft/comms/comms.hpp index 8f675ddfe4..367b04f240 100644 --- a/cpp/include/raft/comms/comms.hpp +++ b/cpp/include/raft/comms/comms.hpp @@ -16,8 +16,8 @@ #pragma once -#include #include +#include namespace raft { namespace comms { @@ -87,7 +87,6 @@ constexpr datatype_t get_type() { return datatype_t::FLOAT64; } - class comms_iface { public: virtual ~comms_iface(); @@ -326,6 +325,5 @@ class comms_t { comms_iface::~comms_iface() {} - } // namespace comms } // namespace raft diff --git a/cpp/include/raft/comms/helper.hpp b/cpp/include/raft/comms/helper.hpp index 67ba4cab0a..4809c78825 100644 --- a/cpp/include/raft/comms/helper.hpp +++ b/cpp/include/raft/comms/helper.hpp @@ -16,9 +16,9 @@ #pragma once -#include #include #include +#include #include #include #include @@ -39,7 +39,8 @@ void build_comms_nccl_only(handle_t *handle, ncclComm_t nccl_comm, int num_ranks, int rank) { auto d_alloc = handle->get_device_allocator(); cudaStream_t stream = handle->get_stream(); - comms_iface *raft_comm = new raft::comms::std_comms(nccl_comm, num_ranks, rank, d_alloc, stream); + comms_iface *raft_comm = + new raft::comms::std_comms(nccl_comm, num_ranks, rank, d_alloc, stream); auto communicator = std::make_shared(std::unique_ptr(raft_comm)); @@ -83,14 +84,16 @@ void build_comms_nccl_ucx(handle_t *handle, ncclComm_t nccl_comm, auto d_alloc = handle->get_device_allocator(); cudaStream_t stream = handle->get_stream(); - auto *raft_comm = new raft::comms::std_comms( - nccl_comm, (ucp_worker_h)ucp_worker, eps_sp, num_ranks, rank, d_alloc, stream); + auto *raft_comm = + new raft::comms::std_comms(nccl_comm, (ucp_worker_h)ucp_worker, eps_sp, + num_ranks, rank, d_alloc, stream); auto communicator = std::make_shared(std::unique_ptr(raft_comm)); handle->set_comms(communicator); } -inline void nccl_unique_id_from_char(ncclUniqueId *id, char *uniqueId, int size) { +inline void nccl_unique_id_from_char(ncclUniqueId *id, char *uniqueId, + int size) { memcpy(id->internal, uniqueId, size); } diff --git a/cpp/include/raft/comms/test.hpp b/cpp/include/raft/comms/test.hpp index a3bf81a007..10d4b2c195 100644 --- a/cpp/include/raft/comms/test.hpp +++ b/cpp/include/raft/comms/test.hpp @@ -246,5 +246,5 @@ bool test_pointToPoint_simple_send_recv(const handle_t &h, int numTrials) { return ret; } -} -}; +} // namespace comms +}; // namespace raft From 1a18553a4cf06be14d05bd96c664a3e3ee71106d Mon Sep 17 00:00:00 2001 From: "Corey J. Nolet" Date: Wed, 27 May 2020 10:54:29 -0400 Subject: [PATCH 41/46] Nccl red op --- cpp/include/raft/comms/std_comms.hpp | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/cpp/include/raft/comms/std_comms.hpp b/cpp/include/raft/comms/std_comms.hpp index ef83730749..4a42276a8f 100644 --- a/cpp/include/raft/comms/std_comms.hpp +++ b/cpp/include/raft/comms/std_comms.hpp @@ -82,6 +82,8 @@ constexpr size_t get_datatype_size(const datatype_t datatype) { return sizeof(float); case datatype_t::FLOAT64: return sizeof(double); + default: + return -1; } } @@ -104,6 +106,8 @@ constexpr ncclDataType_t get_nccl_datatype(const datatype_t datatype) { return ncclFloat; case datatype_t::FLOAT64: return ncclDouble; + default: + return -1; } } @@ -117,6 +121,8 @@ constexpr ncclRedOp_t get_nccl_op(const op_t op) { return ncclMin; case op_t::MAX: return ncclMax; + default: + return -1; } } From a22dd6237299345153b870cd1febe360ed3a8cd5 Mon Sep 17 00:00:00 2001 From: "Corey J. Nolet" Date: Wed, 27 May 2020 11:57:19 -0400 Subject: [PATCH 42/46] Raising an exception to get around gcc issue --- cpp/include/raft/comms/std_comms.hpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/cpp/include/raft/comms/std_comms.hpp b/cpp/include/raft/comms/std_comms.hpp index 4a42276a8f..4fe55bff3c 100644 --- a/cpp/include/raft/comms/std_comms.hpp +++ b/cpp/include/raft/comms/std_comms.hpp @@ -83,7 +83,7 @@ constexpr size_t get_datatype_size(const datatype_t datatype) { case datatype_t::FLOAT64: return sizeof(double); default: - return -1; + throw "Unsupported"; } } @@ -107,7 +107,7 @@ constexpr ncclDataType_t get_nccl_datatype(const datatype_t datatype) { case datatype_t::FLOAT64: return ncclDouble; default: - return -1; + throw "Unsupported"; } } @@ -122,7 +122,7 @@ constexpr ncclRedOp_t get_nccl_op(const op_t op) { case op_t::MAX: return ncclMax; default: - return -1; + throw "Unsupported"; } } From fbd12aa835681fd18d1c8f0e0c2c085a03464c64 Mon Sep 17 00:00:00 2001 From: "Corey J. Nolet" Date: Wed, 27 May 2020 12:18:34 -0400 Subject: [PATCH 43/46] Using static for functions for now --- cpp/include/raft/comms/std_comms.hpp | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/cpp/include/raft/comms/std_comms.hpp b/cpp/include/raft/comms/std_comms.hpp index 4fe55bff3c..7b3a8bfbca 100644 --- a/cpp/include/raft/comms/std_comms.hpp +++ b/cpp/include/raft/comms/std_comms.hpp @@ -64,7 +64,11 @@ namespace raft { namespace comms { -constexpr size_t get_datatype_size(const datatype_t datatype) { + +static size_t get_datatype_size(const datatype_t datatype) { + + size_t ret = -1; + switch (datatype) { case datatype_t::CHAR: return sizeof(char); @@ -87,7 +91,7 @@ constexpr size_t get_datatype_size(const datatype_t datatype) { } } -constexpr ncclDataType_t get_nccl_datatype(const datatype_t datatype) { +static ncclDataType_t get_nccl_datatype(const datatype_t datatype) { switch (datatype) { case datatype_t::CHAR: return ncclChar; @@ -111,7 +115,7 @@ constexpr ncclDataType_t get_nccl_datatype(const datatype_t datatype) { } } -constexpr ncclRedOp_t get_nccl_op(const op_t op) { +static ncclRedOp_t get_nccl_op(const op_t op) { switch (op) { case op_t::SUM: return ncclSum; From cc71ccb660211378052f8e4ba446fbfbed62c90e Mon Sep 17 00:00:00 2001 From: "Corey J. Nolet" Date: Wed, 27 May 2020 12:22:35 -0400 Subject: [PATCH 44/46] Fixing style --- cpp/include/raft/comms/std_comms.hpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/cpp/include/raft/comms/std_comms.hpp b/cpp/include/raft/comms/std_comms.hpp index 7b3a8bfbca..1ba7552f9c 100644 --- a/cpp/include/raft/comms/std_comms.hpp +++ b/cpp/include/raft/comms/std_comms.hpp @@ -64,9 +64,7 @@ namespace raft { namespace comms { - static size_t get_datatype_size(const datatype_t datatype) { - size_t ret = -1; switch (datatype) { From dc123b2d59b938d33da86d245e1a4194bdba4fd7 Mon Sep 17 00:00:00 2001 From: "Corey J. Nolet" Date: Thu, 28 May 2020 12:23:21 -0400 Subject: [PATCH 45/46] Fixing more relative imports --- python/raft/dask/common/comms.py | 6 +++--- python/raft/test/test_comms.py | 2 +- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/python/raft/dask/common/comms.py b/python/raft/dask/common/comms.py index 4ac47e25a6..b49cb8d7b9 100644 --- a/python/raft/dask/common/comms.py +++ b/python/raft/dask/common/comms.py @@ -13,14 +13,14 @@ # limitations under the License. # -from raft.dask.common.nccl import nccl -from raft.dask.common.ucx import UCX +from .nccl import nccl +from .ucx import UCX from .comms_utils import inject_comms_on_handle from .comms_utils import inject_comms_on_handle_coll_only from .utils import parse_host_port -from raft.common.handle import Handle +from ...common.handle import Handle from dask.distributed import get_worker, default_client diff --git a/python/raft/test/test_comms.py b/python/raft/test/test_comms.py index b065d098af..56da5ad937 100644 --- a/python/raft/test/test_comms.py +++ b/python/raft/test/test_comms.py @@ -18,7 +18,7 @@ from dask.distributed import Client from dask.distributed import wait -from raft.dask.common import Comms +from raft.dask import Comms from raft.dask.common import local_handle from raft.dask.common import perform_test_comms_send_recv from raft.dask.common import perform_test_comms_allreduce From 60597f3acf0691536e6d817a01072bd573896f5e Mon Sep 17 00:00:00 2001 From: "Corey J. Nolet" Date: Wed, 3 Jun 2020 11:05:27 -0400 Subject: [PATCH 46/46] final updates based on feedback --- cpp/CMakeLists.txt | 2 -- cpp/cmake/comms.cmake | 2 -- 2 files changed, 4 deletions(-) diff --git a/cpp/CMakeLists.txt b/cpp/CMakeLists.txt index a45cf8c950..e6ae09cb35 100644 --- a/cpp/CMakeLists.txt +++ b/cpp/CMakeLists.txt @@ -67,8 +67,6 @@ find_package(CUDA 10.0 REQUIRED) set(CMAKE_MODULE_PATH ${CMAKE_CURRENT_SOURCE_DIR}/cmake) -message("HELLO!") - ############################################################################## # - Compiler Options -------------------------------------------------------- diff --git a/cpp/cmake/comms.cmake b/cpp/cmake/comms.cmake index c8496c7dc6..eab605dca3 100644 --- a/cpp/cmake/comms.cmake +++ b/cpp/cmake/comms.cmake @@ -17,8 +17,6 @@ cmake_minimum_required(VERSION 3.14 FATAL_ERROR) project(comms LANGUAGES CXX CUDA) -set(CMAKE_MODULE_PATH ${CMAKE_MODULE_PATH} "${CMAKE_CURRENT_SOURCE_DIR}/cmake") - if(NOT NCCL_PATH) find_package(NCCL REQUIRED) else()