From ac5ed0f7e2aa27f300b69adff11901fd574c33b6 Mon Sep 17 00:00:00 2001 From: Thejaswi Rao Date: Mon, 27 Apr 2020 20:14:04 -0700 Subject: [PATCH 01/56] FEA initial migration of device and host buffers into raft --- cpp/include/raft/buffer_base.hpp | 185 +++++++++++++++++++++++++++++ cpp/include/raft/device_buffer.hpp | 46 +++++++ cpp/include/raft/host_buffer.hpp | 75 ++++++++++++ 3 files changed, 306 insertions(+) create mode 100644 cpp/include/raft/buffer_base.hpp create mode 100644 cpp/include/raft/device_buffer.hpp create mode 100644 cpp/include/raft/host_buffer.hpp diff --git a/cpp/include/raft/buffer_base.hpp b/cpp/include/raft/buffer_base.hpp new file mode 100644 index 0000000000..fa06711dd3 --- /dev/null +++ b/cpp/include/raft/buffer_base.hpp @@ -0,0 +1,185 @@ +/* + * 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 + +//TODO: include utils.h +#include "../utils.h" + +namespace raft { + +/** + * @brief Base for all RAII-based owning of temporary memory allocations. This + * class should ideally not be used by users directly, but instead via + * the child classes `device_buffer` and `host_buffer`. + */ +template +class buffer_base { + public: + using size_type = std::size_t; + using value_type = T; + using iterator = value_type*; + using const_iterator = const value_type*; + using reference = T&; + using const_reference = const T&; + + buffer_base() = delete; + + buffer_base(const buffer_base& other) = delete; + + buffer_base& operator=(const buffer_base& other) = delete; + + /** + * @brief Main ctor + * + * @param[in] allocator asynchronous allocator used for managing buffer life + * @param[in] stream cuda stream where this allocation operations are async + * @param[in] n size of the buffer (in number of elements) + */ + buffer_base(std::shared_ptr allocator, cudaStream_t stream, + size_type n = 0) + : _size(n), + _capacity(n), + _data(nullptr), + _stream(stream), + _allocator(allocator) { + if (_capacity > 0) { + _data = static_cast( + _allocator->allocate(_capacity * sizeof(value_type), _stream)); + CUDA_CHECK(cudaStreamSynchronize(_stream)); + } + } + + ~buffer_base() { + if (nullptr != _data) { + _allocator->deallocate(_data, _capacity * sizeof(value_type), _stream); + } + } + + value_type* data() { return _data; } + + const value_type* data() const { return _data; } + + size_type size() const { return _size; } + + void clear() { _size = 0; } + + iterator begin() { return _data; } + + const_iterator begin() const { return _data; } + + iterator end() { return _data + _size; } + + const_iterator end() const { return _data + _size; } + + /** + * @brief Reserve new memory size for this buffer. + * + * It re-allocates a fresh buffer if the new requested capacity is more than + * the current one, copies the old buffer contents to this new buffer and + * removes the old one. + * + * @param[in] new_capacity new capacity (in number of elements) + * @param[in] stream cuda stream where allocation operations are queued + */ + void reserve(const size_type new_capacity, cudaStream_t stream) { + set_stream(stream); + if (new_capacity > _capacity) { + value_type* new_data = static_cast( + _allocator->allocate(new_capacity * sizeof(value_type), _stream)); + if (_size > 0) { + CUDA_CHECK(cudaMemcpyAsync(new_data, _data, _size * sizeof(value_type), + cudaMemcpyDefault, _stream)); + } + if (nullptr != _data) { + _allocator->deallocate(_data, _capacity * sizeof(value_type), _stream); + } + _data = new_data; + _capacity = new_capacity; + } + } + + /** + * @brief Resize the underlying buffer (uses `reserve` method internally) + * + * @param[in] new_size new buffer size + * @param[in] stream cuda stream where the work will be queued + */ + void resize(const size_type new_size, cudaStream_t stream) { + reserve(new_size, stream); + _size = new_size; + } + + /** + * @brief Deletes the underlying buffer + * + * If this method is not explicitly called, it will be during the destructor + * + * @param[in] stream cuda stream where the work will be queued + */ + void release(cudaStream_t stream) { + set_stream(stream); + if (nullptr != _data) { + _allocator->deallocate(_data, _capacity * sizeof(value_type), _stream); + } + _data = nullptr; + _capacity = 0; + _size = 0; + } + + /** + * @brief returns the underlying allocator used + * + * @return the allocator pointer + */ + std::shared_ptr get_allocator() const { return _allocator; } + + protected: + value_type* _data; + + private: + size_type _size; + size_type _capacity; + cudaStream_t _stream; + std::shared_ptr _allocator; + + /** + * @brief Sets a new cuda stream where the future operations will be queued + * + * This method makes sure that the inter-stream dependencies are met and taken + * care of, before setting the input stream as a new stream for this buffer. + * Ideally, the same cuda stream passed during constructor is expected to be + * used throughout this buffer's lifetime, for performance. + * + * @param[in] stream new cuda stream to be set. If it is the same as the + * current one, then this method will be a no-op. + */ + void set_stream(cudaStream_t stream) { + if (_stream != stream) { + cudaEvent_t event; + CUDA_CHECK(cudaEventCreateWithFlags(&event, cudaEventDisableTiming)); + CUDA_CHECK(cudaEventRecord(event, _stream)); + CUDA_CHECK(cudaStreamWaitEvent(stream, event, 0)); + _stream = stream; + CUDA_CHECK(cudaEventDestroy(event)); + } + } +}; // class buffer_base + +} // namespace raft diff --git a/cpp/include/raft/device_buffer.hpp b/cpp/include/raft/device_buffer.hpp new file mode 100644 index 0000000000..359c69ef4a --- /dev/null +++ b/cpp/include/raft/device_buffer.hpp @@ -0,0 +1,46 @@ +/* + * 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 "buffer_base.hpp" +//TODO: update allocator header +#include + +namespace raft { + +/** + * @brief RAII object owning a contiguous typed device buffer. The passed in + * allocator supports asynchronous allocation and deallocation so this + * can also be used for temporary memory + * + * @code{.cpp} + * template + * void foo(const raft_handle& h, ..., cudaStream_t stream) { + * ... + * device_buffer temp( h.getDeviceAllocator(), stream, 0 ); + * + * temp.resize(n, stream); + * kernelA<<>>(...,temp.data(),...); + * kernelB<<>>(...,temp.data(),...); + * temp.release(stream); + * } + * @endcode + */ +template +using device_buffer = buffer_base; + +} // namespace raft diff --git a/cpp/include/raft/host_buffer.hpp b/cpp/include/raft/host_buffer.hpp new file mode 100644 index 0000000000..2df3dbd71a --- /dev/null +++ b/cpp/include/raft/host_buffer.hpp @@ -0,0 +1,75 @@ +/* + * 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 "buffer_base.hpp" +//TODO: update allocator header +#include + +namespace raft { + +/** + * @brief RAII object owning a contigous typed host buffer (aka pinned memory). + * The passed in allocator supports asynchronus allocation and + * deallocation so this can also be used for temporary memory + * + * @code{.cpp} + * template + * void foo(const raft_handle& h, const T* in_d , T* out_d, ..., cudaStream_t stream ) { + * ... + * host_buffer temp( handle->getHostAllocator(), stream, 0 ) + * + * temp.resize(n, stream); + * cudaMemcpyAsync( temp.data(), in_d, temp.size()*sizeof(T), cudaMemcpyDeviceToHost ); + * ... + * cudaMemcpyAsync( out_d, temp.data(), temp.size()*sizeof(T), cudaMemcpyHostToDevice ); + * temp.release(stream); + * } + * @endcode + */ +template +class host_buffer : public buffer_base { + public: + using size_type = typename buffer_base::size_type; + using value_type = typename buffer_base::value_type; + using iterator = typename buffer_base::iterator; + using const_iterator = typename buffer_base::const_iterator; + using reference = typename buffer_base::reference; + using const_reference = + typename buffer_base::const_reference; + + host_buffer() = delete; + + host_buffer(const host_buffer& other) = delete; + + host_buffer& operator=(const host_buffer& other) = delete; + + host_buffer(std::shared_ptr allocator, cudaStream_t stream, + size_type n = 0) + : buffer_base(allocator, stream, n) {} + + ~host_buffer() {} + + reference operator[](size_type pos) { return _data[pos]; } + + const_reference operator[](size_type pos) const { return _data[pos]; } + + private: + using buffer_base::_data; +}; + +} // namespace raft From 2f81d836401408a929bdf5bfe6584fa2f570f8e8 Mon Sep 17 00:00:00 2001 From: Thejaswi Rao Date: Mon, 27 Apr 2020 20:29:20 -0700 Subject: [PATCH 02/56] FEA added cudaruntime utils which will be used in raft repo --- cpp/include/raft/buffer_base.hpp | 4 +- cpp/include/raft/cudart_utils.h | 224 +++++++++++++++++++++++++++++++ 2 files changed, 225 insertions(+), 3 deletions(-) create mode 100644 cpp/include/raft/cudart_utils.h diff --git a/cpp/include/raft/buffer_base.hpp b/cpp/include/raft/buffer_base.hpp index fa06711dd3..4af4f86ccd 100644 --- a/cpp/include/raft/buffer_base.hpp +++ b/cpp/include/raft/buffer_base.hpp @@ -18,9 +18,7 @@ #include #include - -//TODO: include utils.h -#include "../utils.h" +#include "cudart_utils.h" namespace raft { diff --git a/cpp/include/raft/cudart_utils.h b/cpp/include/raft/cudart_utils.h new file mode 100644 index 0000000000..fd23295b68 --- /dev/null +++ b/cpp/include/raft/cudart_utils.h @@ -0,0 +1,224 @@ +/* + * 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 +///@todo: enable once logging has been enabled in raft +//#include "logger.hpp" + +namespace raft { + +/** base exception class for the whole of raft */ +class Exception : public std::exception { + public: + /** default ctor */ + Exception() throw() : std::exception(), msg() {} + + /** copy ctor */ + Exception(const Exception& src) throw() : std::exception(), msg(src.what()) { + collectCallStack(); + } + + /** ctor from an input message */ + Exception(const std::string& _msg) throw() : std::exception(), msg(_msg) { + collectCallStack(); + } + + /** dtor */ + virtual ~Exception() throw() {} + + /** get the message associated with this exception */ + virtual const char* what() const throw() { return msg.c_str(); } + + private: + /** message associated with this exception */ + std::string msg; + + /** append call stack info to this exception's message for ease of debug */ + // Courtesy: https://www.gnu.org/software/libc/manual/html_node/Backtraces.html + void collectCallStack() throw() { +#ifdef __GNUC__ + const int MaxStackDepth = 64; + void* stack[MaxStackDepth]; + auto depth = backtrace(stack, MaxStackDepth); + std::ostringstream oss; + oss << std::endl << "Obtained " << depth << " stack frames" << std::endl; + char** strings = backtrace_symbols(stack, depth); + if (strings == nullptr) { + oss << "But no stack trace could be found!" << std::endl; + msg += oss.str(); + return; + } + ///@todo: support for demangling of C++ symbol names + for (int i = 0; i < depth; ++i) { + oss << "#" << i << " in " << strings[i] << std::endl; + } + free(strings); + msg += oss.str(); +#endif // __GNUC__ + } +}; + +/** macro to throw a runtime error */ +#define THROW(fmt, ...) \ + do { \ + std::string msg; \ + char errMsg[2048]; \ + std::snprintf(errMsg, sizeof(errMsg), \ + "Exception occured! file=%s line=%d: ", __FILE__, __LINE__); \ + msg += errMsg; \ + std::snprintf(errMsg, sizeof(errMsg), fmt, ##__VA_ARGS__); \ + msg += errMsg; \ + throw raft::Exception(msg); \ + } while (0) + +/** macro to check for a conditional and assert on failure */ +#define ASSERT(check, fmt, ...) \ + do { \ + if (!(check)) THROW(fmt, ##__VA_ARGS__); \ + } while (0) + +/** check for cuda runtime API errors and assert accordingly */ +#define CUDA_CHECK(call) \ + do { \ + cudaError_t status = call; \ + ASSERT(status == cudaSuccess, "FAIL: call='%s'. Reason:%s", #call, \ + cudaGetErrorString(status)); \ + } while (0) + +///@todo: enable this only after we have added logging support in raft +// /** +// * @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) + +/** helper method to get max usable shared mem per block parameter */ +inline int getSharedMemPerBlock() { + int devId; + CUDA_CHECK(cudaGetDevice(&devId)); + int smemPerBlk; + CUDA_CHECK(cudaDeviceGetAttribute(&smemPerBlk, + cudaDevAttrMaxSharedMemoryPerBlock, devId)); + return smemPerBlk; +} +/** helper method to get multi-processor count parameter */ +inline int getMultiProcessorCount() { + int devId; + CUDA_CHECK(cudaGetDevice(&devId)); + int mpCount; + CUDA_CHECK( + cudaDeviceGetAttribute(&mpCount, cudaDevAttrMultiProcessorCount, devId)); + return mpCount; +} + +/** + * @brief Generic copy method for all kinds of transfers + * @tparam Type data type + * @param dst destination pointer + * @param src source pointer + * @param len lenth of the src/dst buffers in terms of number of elements + * @param stream cuda stream + */ +template +void copy(Type* dst, const Type* src, size_t len, cudaStream_t stream) { + CUDA_CHECK( + cudaMemcpyAsync(dst, src, len * sizeof(Type), cudaMemcpyDefault, stream)); +} + +/** + * @defgroup Copy Copy methods + * These are here along with the generic 'copy' method in order to improve + * code readability using explicitly specified function names + * @{ + */ +/** performs a host to device copy */ +template +void updateDevice(Type* dPtr, const Type* hPtr, size_t len, + cudaStream_t stream) { + copy(dPtr, hPtr, len, stream); +} + +/** performs a device to host copy */ +template +void updateHost(Type* hPtr, const Type* dPtr, size_t len, cudaStream_t stream) { + copy(hPtr, dPtr, len, stream); +} + +template +void copyAsync(Type* dPtr1, const Type* dPtr2, size_t len, + cudaStream_t stream) { + CUDA_CHECK(cudaMemcpyAsync(dPtr1, dPtr2, len * sizeof(Type), + cudaMemcpyDeviceToDevice, stream)); +} +/** @} */ + +/** + * @defgroup Debug Utils for debugging device buffers + * @{ + */ +template +void printHostVector(const char* variableName, const T* hostMem, + size_t componentsCount, OutStream& out) { + out << variableName << "=["; + for (size_t i = 0; i < componentsCount; ++i) { + if (i != 0) out << ","; + out << hostMem[i]; + } + out << "];\n"; +} + +template +void printHostVector(const char* variableName, const T* hostMem, + size_t componentsCount) { + printHostVector(variableName, hostMem, componentsCount, std::cout); + std::cout.flush(); +} + +template +void printDevVector(const char* variableName, const T* devMem, + size_t componentsCount, OutStream& out) { + T* hostMem = new T[componentsCount]; + CUDA_CHECK(cudaMemcpy(hostMem, devMem, componentsCount * sizeof(T), + cudaMemcpyDeviceToHost)); + printHostVector(variableName, hostMem, componentsCount, out); + delete[] hostMem; +} + +template +void printDevVector(const char* variableName, const T* devMem, + size_t componentsCount) { + printDevVector(variableName, devMem, componentsCount, std::cout); + std::cout.flush(); +} +/** @} */ + +}; // namespace raft From 92aed07ef80c0497a4c78016546cb4ee5e42dae1 Mon Sep 17 00:00:00 2001 From: Thejaswi Rao Date: Mon, 27 Apr 2020 22:56:57 -0700 Subject: [PATCH 03/56] FEA added allocator interface --- cpp/include/raft/allocator.hpp | 118 +++++++++++++++++++++++++++++++++ 1 file changed, 118 insertions(+) create mode 100644 cpp/include/raft/allocator.hpp diff --git a/cpp/include/raft/allocator.hpp b/cpp/include/raft/allocator.hpp new file mode 100644 index 0000000000..6853c5c7ea --- /dev/null +++ b/cpp/include/raft/allocator.hpp @@ -0,0 +1,118 @@ +/* + * 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 "cudart_utils.h" + +namespace raft { + +/** + * @brief Interface for an asynchronous device/host allocator. + * + * An implementation of this interface can make the following assumptions: + * - It does not need to be but it can allow async allocate and deallocate. + */ +class Allocator { + public: + /** + * @brief Asynchronously allocates a memory region. + * + * An implementation of this need to return a allocation of n bytes properly + * align bytes on the configured device. The allocation can optionally be + * asynchronous in the sense that it is only save to use after all work + * submitted to the passed in stream prior to the call to allocate has + * completed. If the allocation is used before, e.g. in another stream the + * behaviour may be undefined. + * @todo: Add alignment requirments. + * + * @param[in] n number of bytes to allocate + * @param[in] stream stream to issue the possible asynchronous allocation in + */ + virtual void* allocate(std::size_t n, cudaStream_t stream) = 0; + + /** + * @brief Asynchronously deallocates device memory + * + * An implementation of this need to ensure that the allocation that the + * passed in pointer points to remains usable until all work sheduled in + * stream prior to the call to deallocate has completed. + * + * @param[inout] p pointer to the buffer to deallocte + * @param[in] n size of the buffer to deallocte in bytes + * @param[in] stream stream in which the allocation might be still in use + */ + virtual void deallocate(void* p, std::size_t n, cudaStream_t stream) = 0; + + virtual ~Allocator() {} +}; // class Allocator + +/** + * @brief An explicit interface for an asynchronous device allocator. + + * This is mostly done in order to reduce work needed in cuML codebase. + * An implementation of this interface can make the following assumptions, + * further to the ones listed in `Allocator`: + * - Allocations may be always on the device that was specified on construction. + */ +class deviceAllocator : public Allocator { +}; // class deviceAllocator + +/** + * @brief An explicit interface for an asynchronous host allocations. + * + * This is mostly done in order to reduce work needed in cuML codebase. + * An implementation of this interface can make the following assumptions, + * further to the ones listed in `Allocator`: + * - Allocations don't need to be zero copy accessible form a device. + */ +class hostAllocator : public Allocator { +}; // class hostAllocator + +/** Default cudaMalloc/cudaFree based device allocator */ +class defaultDeviceAllocator : public deviceAllocator { + public: + void* allocate(std::size_t n, cudaStream_t stream) override { + void* ptr = 0; + CUDA_CHECK(cudaMalloc(&ptr, n)); + return ptr; + } + + void deallocate(void* p, std::size_t n, cudaStream_t stream) override { + ///@todo: enable this once logging is enabled in raft + //CUDA_CHECK_NO_THROW(cudaFree(p)); + CUDA_CHECK(cudaFree(p)); + } +}; // class defaultDeviceAllocator + +/** Default cudaMallocHost/cudaFreeHost based host allocator */ +class defaultHostAllocator : public hostAllocator { + public: + void* allocate(std::size_t n, cudaStream_t stream) override { + void* ptr = 0; + CUDA_CHECK(cudaMallocHost(&ptr, n)); + return ptr; + } + + void deallocate(void* p, std::size_t n, cudaStream_t stream) override { + ///@todo: enable this once logging is enabled in raft + //CUDA_CHECK_NO_THROW(cudaFreeHost(p)); + CUDA_CHECK(cudaFreeHost(p)); + } +}; // class defaultHostAllocator + +}; // end namespace raft From dea96768fa5727c0e30c544b6fae178d860b9144 Mon Sep 17 00:00:00 2001 From: Thejaswi Rao Date: Mon, 27 Apr 2020 22:57:57 -0700 Subject: [PATCH 04/56] DOC doxygen updates --- cpp/include/raft/allocator.hpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cpp/include/raft/allocator.hpp b/cpp/include/raft/allocator.hpp index 6853c5c7ea..6751393013 100644 --- a/cpp/include/raft/allocator.hpp +++ b/cpp/include/raft/allocator.hpp @@ -63,7 +63,7 @@ class Allocator { /** * @brief An explicit interface for an asynchronous device allocator. - + * * This is mostly done in order to reduce work needed in cuML codebase. * An implementation of this interface can make the following assumptions, * further to the ones listed in `Allocator`: From 1921918265429a72b2111d809e50668d0dd8ad81 Mon Sep 17 00:00:00 2001 From: Thejaswi Rao Date: Mon, 27 Apr 2020 23:19:34 -0700 Subject: [PATCH 05/56] FEA added raft::handle_t --- cpp/include/raft/handle.hpp | 240 ++++++++++++++++++++++++++++++++++++ 1 file changed, 240 insertions(+) create mode 100644 cpp/include/raft/handle.hpp diff --git a/cpp/include/raft/handle.hpp b/cpp/include/raft/handle.hpp new file mode 100644 index 0000000000..ee7c3a8a95 --- /dev/null +++ b/cpp/include/raft/handle.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. + */ + +#pragma once + +#include +#include +#include +#include + +#include +#include +#include +#include +#include + +///@todo: enable once we have migrated cuml-comms layer too +//#include + +#include "allocator.hpp" +#include "cudart_utils.h" + +namespace raft { + +/** + * @brief Main handle object that stores all necessary context used for calling + * necessary cuda kernels and/or libraries + */ +class handle_t { + private: + static constexpr int NumDefaultWorkerStreams = 0; + + public: + /** + * @brief Construct a handle with the specified number of worker streams + * + * @param[in] n_streams number worker streams to be created + */ + handle_t(int n_streams = NumDefaultWorkerStreams) : + : _dev_id([]() -> int { + int cur_dev = -1; + CUDA_CHECK(cudaGetDevice(&cur_dev)); + return cur_dev; + }()), + _num_streams(n_streams), + _cublasInitialized(false), + _cusolverDnInitialized(false), + _cusolverSpInitialized(false), + _cusparseInitialized(false), + _deviceAllocator(std::make_shared()), + _hostAllocator(std::make_shared()), + _userStream(NULL), + _devicePropInitialized(false) { + createResources(); + } + + /** Destroys all held-up resources */ + ~handle_t() { destroyResources(); } + + int getDevice() const { return _dev_id; } + + void setStream(cudaStream_t stream) { _userStream = stream; } + cudaStream_t getStream() const { return _userStream; } + + void setDeviceAllocator(std::shared_ptr allocator) { + _deviceAllocator = allocator; + } + std::shared_ptr getDeviceAllocator() const { + return _deviceAllocator; + } + + void setHostAllocator(std::shared_ptr allocator) { + _hostAllocator = allocator; + } + std::shared_ptr getHostAllocator() const { + return _hostAllocator; + } + + cublasHandle_t getCublasHandle() const { + if (!_cublasInitialized) { + CUBLAS_CHECK(cublasCreate(&_cublas_handle)); + _cublasInitialized = true; + } + return _cublas_handle; + } + + cusolverDnHandle_t getcusolverDnHandle() const { + if (!_cusolverDnInitialized) { + CUSOLVER_CHECK(cusolverDnCreate(&_cusolverDn_handle)); + _cusolverDnInitialized = true; + } + return _cusolverDn_handle; + } + + cusolverSpHandle_t getcusolverSpHandle() const { + if (!_cusolverSpInitialized) { + CUSOLVER_CHECK(cusolverSpCreate(&_cusolverSp_handle)); + _cusolverSpInitialized = true; + } + return _cusolverSp_handle; + } + + cusparseHandle_t getcusparseHandle() const { + if (!_cusparseInitialized) { + CUSPARSE_CHECK(cusparseCreate(&_cusparse_handle)); + _cusparseInitialized = true; + } + return _cusparse_handle; + } + + cudaStream_t getInternalStream(int sid) const { return _streams[sid]; } + int getNumInternalStreams() const { return _num_streams; } + std::vector getInternalStreams() const { + std::vector int_streams_vec(_num_streams); + for (auto s : _streams) { + int_streams_vec.push_back(s); + } + return int_streams_vec; + } + + void waitOnUserStream() const { + CUDA_CHECK(cudaEventRecord(_event, _userStream)); + for (auto s : _streams) { + CUDA_CHECK(cudaStreamWaitEvent(s, _event, 0)); + } + } + + void waitOnInternalStreams() const { + for (auto s : _streams) { + CUDA_CHECK(cudaEventRecord(_event, s)); + CUDA_CHECK(cudaStreamWaitEvent(_userStream, _event, 0)); + } + } + + ///@todo: enable this once we have cuml-comms migrated + // void setCommunicator( + // std::shared_ptr communicator); + // const MLCommon::cumlCommunicator& getCommunicator() const; + // bool commsInitialized() const; + + const cudaDeviceProp& getDeviceProperties() const { + if (!_devicePropInitialized) { + CUDA_CHECK(cudaGetDeviceProperties(&_prop, _dev_id)); + _devicePropInitialized = true; + } + return _prop; + } + + private: + const int _dev_id; + const int _num_streams; + std::vector _streams; + mutable cublasHandle_t _cublas_handle; + mutable bool _cublasInitialized; + mutable cusolverDnHandle_t _cusolverDn_handle; + mutable bool _cusolverDnInitialized; + mutable cusolverSpHandle_t _cusolverSp_handle; + mutable bool _cusolverSpInitialized; + mutable cusparseHandle_t _cusparse_handle; + mutable bool _cusparseInitialized; + std::shared_ptr _deviceAllocator; + std::shared_ptr _hostAllocator; + cudaStream_t _userStream; + cudaEvent_t _event; + mutable cudaDeviceProp _prop; + mutable bool _devicePropInitialized; + + ///@todo: enable this once we have migrated cuml-comms + //std::shared_ptr _communicator; + + void createResources() { + cudaStream_t stream; + CUDA_CHECK(cudaStreamCreateWithFlags(&stream, cudaStreamNonBlocking)); + _streams.push_back(stream); + for (int i = 1; i < _num_streams; ++i) { + cudaStream_t stream; + CUDA_CHECK(cudaStreamCreateWithFlags(&stream, cudaStreamNonBlocking)); + _streams.push_back(stream); + } + CUDA_CHECK(cudaEventCreateWithFlags(&_event, cudaEventDisableTiming)); + } + + void destroyResources() { + ///@todo: enable *_NO_THROW variants once we have enabled logging + if (_cusparseInitialized) { + //CUSPARSE_CHECK_NO_THROW(cusparseDestroy(_cusparse_handle)); + CUSPARSE_CHECK(cusparseDestroy(_cusparse_handle)); + } + if (_cusolverDnInitialized) { + //CUSOLVER_CHECK_NO_THROW(cusolverDnDestroy(_cusolverDn_handle)); + } + if (_cusolverSpInitialized) { + //CUSOLVER_CHECK_NO_THROW(cusolverSpDestroy(_cusolverSp_handle)); + CUSOLVER_CHECK(cusolverSpDestroy(_cusolverSp_handle)); + } + if (_cublasInitialized) { + //CUBLAS_CHECK_NO_THROW(cublasDestroy(_cublas_handle)); + CUBLAS_CHECK(cublasDestroy(_cublas_handle)); + } + while (!_streams.empty()) { + //CUDA_CHECK_NO_THROW(cudaStreamDestroy(_streams.back())); + CUDA_CHECK(cudaStreamDestroy(_streams.back())); + _streams.pop_back(); + } + //CUDA_CHECK_NO_THROW(cudaEventDestroy(_event)); + CUDA_CHECK(cudaEventDestroy(_event)); + } +}; // class handle_t + +/** + * @brief RAII approach to synchronizing across all streams in the handle + */ +class streamSyncer { + public: + streamSyncer(const handle_t& handle) : _handle(handle) { + _handle.waitOnUserStream(); + } + ~streamSyncer() { _handle.waitOnInternalStreams(); } + + streamSyncer(const streamSyncer& other) = delete; + streamSyncer& operator=(const streamSyncer& other) = delete; + + private: + const handle_t& _handle; +}; // class streamSyncer + +} // end namespace ML From 5770850306f305b862db71ec8ae79b1ca73b7286 Mon Sep 17 00:00:00 2001 From: Thejaswi Rao Date: Mon, 27 Apr 2020 23:28:36 -0700 Subject: [PATCH 06/56] FEA added cusolver/cublas/cusparse wrappers and updated raft/handle.hpp --- cpp/include/raft/handle.hpp | 3 + cpp/include/raft/linalg/cublas_wrappers.h | 544 +++++++++++++++ cpp/include/raft/linalg/cusolver_wrappers.h | 711 ++++++++++++++++++++ cpp/include/raft/sparse/cusparse_wrappers.h | 167 +++++ 4 files changed, 1425 insertions(+) create mode 100644 cpp/include/raft/linalg/cublas_wrappers.h create mode 100644 cpp/include/raft/linalg/cusolver_wrappers.h create mode 100644 cpp/include/raft/sparse/cusparse_wrappers.h diff --git a/cpp/include/raft/handle.hpp b/cpp/include/raft/handle.hpp index ee7c3a8a95..21b7a8327d 100644 --- a/cpp/include/raft/handle.hpp +++ b/cpp/include/raft/handle.hpp @@ -32,6 +32,9 @@ #include "allocator.hpp" #include "cudart_utils.h" +#include +#include +#include namespace raft { diff --git a/cpp/include/raft/linalg/cublas_wrappers.h b/cpp/include/raft/linalg/cublas_wrappers.h new file mode 100644 index 0000000000..e732d7d424 --- /dev/null +++ b/cpp/include/raft/linalg/cublas_wrappers.h @@ -0,0 +1,544 @@ +/* + * Copyright (c) 2018-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 +///@todo: enable this once we have logger enabled +//#include +#include + +namespace raft { +namespace linal { + +#define _CUBLAS_ERR_TO_STR(err) \ + case err: \ + return #err +inline const char *cublasErr2Str(cublasStatus_t err) { + switch (err) { + _CUBLAS_ERR_TO_STR(CUBLAS_STATUS_SUCCESS); + _CUBLAS_ERR_TO_STR(CUBLAS_STATUS_NOT_INITIALIZED); + _CUBLAS_ERR_TO_STR(CUBLAS_STATUS_ALLOC_FAILED); + _CUBLAS_ERR_TO_STR(CUBLAS_STATUS_INVALID_VALUE); + _CUBLAS_ERR_TO_STR(CUBLAS_STATUS_ARCH_MISMATCH); + _CUBLAS_ERR_TO_STR(CUBLAS_STATUS_MAPPING_ERROR); + _CUBLAS_ERR_TO_STR(CUBLAS_STATUS_EXECUTION_FAILED); + _CUBLAS_ERR_TO_STR(CUBLAS_STATUS_INTERNAL_ERROR); + _CUBLAS_ERR_TO_STR(CUBLAS_STATUS_NOT_SUPPORTED); + _CUBLAS_ERR_TO_STR(CUBLAS_STATUS_LICENSE_ERROR); + default: + return "CUBLAS_STATUS_UNKNOWN"; + }; +} +#undef _CUBLAS_ERR_TO_STR + +/** check for cublas runtime API errors and assert accordingly */ +#define CUBLAS_CHECK(call) \ + do { \ + cublasStatus_t err = call; \ + ASSERT(err == CUBLAS_STATUS_SUCCESS, \ + "CUBLAS call='%s' got errorcode=%d err=%s", #call, err, \ + raft::linalg::cublasErr2Str(err)); \ + } while (0) + +///@todo: enable this once we have logging enabled +// /** check for cublas runtime API errors but do not assert */ +// #define CUBLAS_CHECK_NO_THROW(call) \ +// do { \ +// cublasStatus_t err = call; \ +// if (err != CUBLAS_STATUS_SUCCESS) { \ +// CUML_LOG_ERROR("CUBLAS call='%s' got errorcode=%d err=%s", #call, err, \ +// raft::linalg::cublasErr2Str(err)); \ +// } \ +// } while (0) + +/** + * @defgroup Axpy cublas ax+y operations + * @{ + */ +template +cublasStatus_t cublasaxpy(cublasHandle_t handle, int n, const T *alpha, + const T *x, int incx, T *y, int incy, + cudaStream_t stream); + +template <> +inline cublasStatus_t cublasaxpy(cublasHandle_t handle, int n, + const float *alpha, const float *x, int incx, + float *y, int incy, cudaStream_t stream) { + CUBLAS_CHECK(cublasSetStream(handle, stream)); + return cublasSaxpy(handle, n, alpha, x, incx, y, incy); +} + +template <> +inline cublasStatus_t cublasaxpy(cublasHandle_t handle, int n, + const double *alpha, const double *x, int incx, + double *y, int incy, cudaStream_t stream) { + CUBLAS_CHECK(cublasSetStream(handle, stream)); + return cublasDaxpy(handle, n, alpha, x, incx, y, incy); +} +/** @} */ + +/** + * @defgroup gemv cublas gemv calls + * @{ + */ +template +cublasStatus_t cublasgemv(cublasHandle_t handle, cublasOperation_t transA, + int m, int n, const T *alfa, const T *A, int lda, + const T *x, int incx, const T *beta, T *y, int incy, + cudaStream_t stream); + +template <> +inline cublasStatus_t cublasgemv(cublasHandle_t handle, + cublasOperation_t transA, int m, int n, + const float *alfa, const float *A, int lda, + const float *x, int incx, const float *beta, + float *y, int incy, cudaStream_t stream) { + CUBLAS_CHECK(cublasSetStream(handle, stream)); + return cublasSgemv(handle, transA, m, n, alfa, A, lda, x, incx, beta, y, + incy); +} + +template <> +inline cublasStatus_t cublasgemv(cublasHandle_t handle, + cublasOperation_t transA, int m, int n, + const double *alfa, const double *A, int lda, + const double *x, int incx, const double *beta, + double *y, int incy, cudaStream_t stream) { + CUBLAS_CHECK(cublasSetStream(handle, stream)); + return cublasDgemv(handle, transA, m, n, alfa, A, lda, x, incx, beta, y, + incy); +} +/** @} */ + +/** + * @defgroup ger cublas a(x*y.T) + A calls + * @{ + */ +template +cublasStatus_t cublasger(cublasHandle_t handle, int m, int n, const T *alpha, + const T *x, int incx, const T *y, int incy, T *A, + int lda, cudaStream_t stream); +template <> +inline cublasStatus_t cublasger(cublasHandle_t handle, int m, int n, + const float *alpha, const float *x, int incx, + const float *y, int incy, float *A, int lda, + cudaStream_t stream) { + CUBLAS_CHECK(cublasSetStream(handle, stream)); + return cublasSger(handle, m, n, alpha, x, incx, y, incy, A, lda); +} + +template <> +inline cublasStatus_t cublasger(cublasHandle_t handle, int m, int n, + const double *alpha, const double *x, int incx, + const double *y, int incy, double *A, int lda, + cudaStream_t stream) { + CUBLAS_CHECK(cublasSetStream(handle, stream)); + return cublasDger(handle, m, n, alpha, x, incx, y, incy, A, lda); +} +/** @} */ + +/** + * @defgroup gemm cublas gemm calls + * @{ + */ +template +cublasStatus_t cublasgemm(cublasHandle_t handle, cublasOperation_t transA, + cublasOperation_t transB, int m, int n, int k, + const T *alfa, const T *A, int lda, const T *B, + int ldb, const T *beta, T *C, int ldc, + cudaStream_t stream); + +template <> +inline cublasStatus_t cublasgemm(cublasHandle_t handle, + cublasOperation_t transA, + cublasOperation_t transB, int m, int n, int k, + const float *alfa, const float *A, int lda, + const float *B, int ldb, const float *beta, + float *C, int ldc, cudaStream_t stream) { + CUBLAS_CHECK(cublasSetStream(handle, stream)); + return cublasSgemm(handle, transA, transB, m, n, k, alfa, A, lda, B, ldb, + beta, C, ldc); +} + +template <> +inline cublasStatus_t cublasgemm(cublasHandle_t handle, + cublasOperation_t transA, + cublasOperation_t transB, int m, int n, int k, + const double *alfa, const double *A, int lda, + const double *B, int ldb, const double *beta, + double *C, int ldc, cudaStream_t stream) { + CUBLAS_CHECK(cublasSetStream(handle, stream)); + return cublasDgemm(handle, transA, transB, m, n, k, alfa, A, lda, B, ldb, + beta, C, ldc); +} +/** @} */ + +/** + * @defgroup gemmbatched cublas gemmbatched calls + * @{ + */ +template +cublasStatus_t cublasgemmBatched(cublasHandle_t handle, + cublasOperation_t transa, + cublasOperation_t transb, int m, int n, int k, + const T *alpha, const T *const Aarray[], + int lda, const T *const Barray[], int ldb, + const T *beta, T *Carray[], int ldc, + int batchCount, cudaStream_t stream); + +template <> +inline cublasStatus_t cublasgemmBatched( + cublasHandle_t handle, cublasOperation_t transa, cublasOperation_t transb, + int m, int n, int k, const float *alpha, const float *const Aarray[], int lda, + const float *const Barray[], int ldb, const float *beta, float *Carray[], + int ldc, int batchCount, cudaStream_t stream) { + CUBLAS_CHECK(cublasSetStream(handle, stream)); + return cublasSgemmBatched(handle, transa, transb, m, n, k, alpha, Aarray, lda, + Barray, ldb, beta, Carray, ldc, batchCount); +} + +template <> +inline cublasStatus_t cublasgemmBatched( + cublasHandle_t handle, cublasOperation_t transa, cublasOperation_t transb, + int m, int n, int k, const double *alpha, const double *const Aarray[], + int lda, const double *const Barray[], int ldb, const double *beta, + double *Carray[], int ldc, int batchCount, cudaStream_t stream) { + CUBLAS_CHECK(cublasSetStream(handle, stream)); + return cublasDgemmBatched(handle, transa, transb, m, n, k, alpha, Aarray, lda, + Barray, ldb, beta, Carray, ldc, batchCount); +} +/** @} */ + +/** + * @defgroup gemmbatched cublas gemmbatched calls + * @{ + */ +template +cublasStatus_t cublasgemmStridedBatched( + cublasHandle_t handle, cublasOperation_t transa, cublasOperation_t transb, + int m, int n, int k, const T *alpha, const T *const Aarray, int lda, + long long int strideA, const T *const Barray, int ldb, long long int strideB, + const T *beta, T *Carray, int ldc, long long int strideC, int batchCount, + cudaStream_t stream); + +template <> +inline cublasStatus_t cublasgemmStridedBatched( + cublasHandle_t handle, cublasOperation_t transa, cublasOperation_t transb, + int m, int n, int k, const float *alpha, const float *const Aarray, int lda, + long long int strideA, const float *const Barray, int ldb, + long long int strideB, const float *beta, float *Carray, int ldc, + long long int strideC, int batchCount, cudaStream_t stream) { + CUBLAS_CHECK(cublasSetStream(handle, stream)); + return cublasSgemmStridedBatched(handle, transa, transb, m, n, k, alpha, + Aarray, lda, strideA, Barray, ldb, strideB, + beta, Carray, ldc, strideC, batchCount); +} + +template <> +inline cublasStatus_t cublasgemmStridedBatched( + cublasHandle_t handle, cublasOperation_t transa, cublasOperation_t transb, + int m, int n, int k, const double *alpha, const double *const Aarray, int lda, + long long int strideA, const double *const Barray, int ldb, + long long int strideB, const double *beta, double *Carray, int ldc, + long long int strideC, int batchCount, cudaStream_t stream) { + CUBLAS_CHECK(cublasSetStream(handle, stream)); + return cublasDgemmStridedBatched(handle, transa, transb, m, n, k, alpha, + Aarray, lda, strideA, Barray, ldb, strideB, + beta, Carray, ldc, strideC, batchCount); +} +/** @} */ + +/** + * @defgroup solverbatched cublas getrf/gettribatched calls + * @{ + */ + +template +cublasStatus_t cublasgetrfBatched(cublasHandle_t handle, int n, + T *const A[], /*Device pointer*/ + int lda, int *P, /*Device Pointer*/ + int *info, /*Device Pointer*/ + int batchSize, cudaStream_t stream); + +template <> +inline cublasStatus_t cublasgetrfBatched(cublasHandle_t handle, int n, + float *const A[], /*Device pointer*/ + int lda, int *P, /*Device Pointer*/ + int *info, /*Device Pointer*/ + int batchSize, cudaStream_t stream) { + CUBLAS_CHECK(cublasSetStream(handle, stream)); + return cublasSgetrfBatched(handle, n, A, lda, P, info, batchSize); +} + +template <> +inline cublasStatus_t cublasgetrfBatched(cublasHandle_t handle, int n, + double *const A[], /*Device pointer*/ + int lda, int *P, /*Device Pointer*/ + int *info, /*Device Pointer*/ + int batchSize, cudaStream_t stream) { + CUBLAS_CHECK(cublasSetStream(handle, stream)); + return cublasDgetrfBatched(handle, n, A, lda, P, info, batchSize); +} + +template +cublasStatus_t cublasgetriBatched(cublasHandle_t handle, int n, + const T *const A[], /*Device pointer*/ + int lda, const int *P, /*Device pointer*/ + T *const C[], /*Device pointer*/ + int ldc, int *info, int batchSize, + cudaStream_t stream); + +template <> +inline cublasStatus_t cublasgetriBatched( + cublasHandle_t handle, int n, const float *const A[], /*Device pointer*/ + int lda, const int *P, /*Device pointer*/ + float *const C[], /*Device pointer*/ + int ldc, int *info, int batchSize, cudaStream_t stream) { + CUBLAS_CHECK(cublasSetStream(handle, stream)); + return cublasSgetriBatched(handle, n, A, lda, P, C, ldc, info, batchSize); +} + +template <> +inline cublasStatus_t cublasgetriBatched( + cublasHandle_t handle, int n, const double *const A[], /*Device pointer*/ + int lda, const int *P, /*Device pointer*/ + double *const C[], /*Device pointer*/ + int ldc, int *info, int batchSize, cudaStream_t stream) { + CUBLAS_CHECK(cublasSetStream(handle, stream)); + return cublasDgetriBatched(handle, n, A, lda, P, C, ldc, info, batchSize); +} + +/** @} */ + +/** + * @defgroup gelsbatched cublas gelsbatched calls + * @{ + */ + +template +inline cublasStatus_t cublasgelsBatched(cublasHandle_t handle, + cublasOperation_t trans, int m, int n, + int nrhs, T *Aarray[], int lda, + T *Carray[], int ldc, int *info, + int *devInfoArray, int batchSize, + cudaStream_t stream = 0); + +template <> +inline cublasStatus_t cublasgelsBatched(cublasHandle_t handle, + cublasOperation_t trans, int m, int n, + int nrhs, float *Aarray[], int lda, + float *Carray[], int ldc, int *info, + int *devInfoArray, int batchSize, + cudaStream_t stream) { + CUBLAS_CHECK(cublasSetStream(handle, stream)); + return cublasSgelsBatched(handle, trans, m, n, nrhs, Aarray, lda, Carray, ldc, + info, devInfoArray, batchSize); +} + +template <> +inline cublasStatus_t cublasgelsBatched(cublasHandle_t handle, + cublasOperation_t trans, int m, int n, + int nrhs, double *Aarray[], int lda, + double *Carray[], int ldc, int *info, + int *devInfoArray, int batchSize, + cudaStream_t stream) { + CUBLAS_CHECK(cublasSetStream(handle, stream)); + return cublasDgelsBatched(handle, trans, m, n, nrhs, Aarray, lda, Carray, ldc, + info, devInfoArray, batchSize); +} + +/** @} */ + +/** + * @defgroup geam cublas geam calls + * @{ + */ +template +cublasStatus_t cublasgeam(cublasHandle_t handle, cublasOperation_t transA, + cublasOperation_t transB, int m, int n, const T *alfa, + const T *A, int lda, const T *beta, const T *B, + int ldb, T *C, int ldc, cudaStream_t stream); + +template <> +inline cublasStatus_t cublasgeam(cublasHandle_t handle, + cublasOperation_t transA, + cublasOperation_t transB, int m, int n, + const float *alfa, const float *A, int lda, + const float *beta, const float *B, int ldb, + float *C, int ldc, cudaStream_t stream) { + CUBLAS_CHECK(cublasSetStream(handle, stream)); + return cublasSgeam(handle, transA, transB, m, n, alfa, A, lda, beta, B, ldb, + C, ldc); +} + +template <> +inline cublasStatus_t cublasgeam(cublasHandle_t handle, + cublasOperation_t transA, + cublasOperation_t transB, int m, int n, + const double *alfa, const double *A, int lda, + const double *beta, const double *B, int ldb, + double *C, int ldc, cudaStream_t stream) { + CUBLAS_CHECK(cublasSetStream(handle, stream)); + return cublasDgeam(handle, transA, transB, m, n, alfa, A, lda, beta, B, ldb, + C, ldc); +} +/** @} */ + +/** + * @defgroup symm cublas symm calls + * @{ + */ +template +cublasStatus_t cublassymm(cublasHandle_t handle, cublasSideMode_t side, + cublasFillMode_t uplo, int m, int n, const T *alpha, + const T *A, int lda, const T *B, int ldb, + const T *beta, T *C, int ldc, cudaStream_t stream); + +template <> +inline cublasStatus_t cublassymm(cublasHandle_t handle, cublasSideMode_t side, + cublasFillMode_t uplo, int m, int n, + const float *alpha, const float *A, int lda, + const float *B, int ldb, const float *beta, + float *C, int ldc, cudaStream_t stream) { + CUBLAS_CHECK(cublasSetStream(handle, stream)); + return cublasSsymm(handle, side, uplo, m, n, alpha, A, lda, B, ldb, beta, C, + ldc); +} + +template <> +inline cublasStatus_t cublassymm(cublasHandle_t handle, cublasSideMode_t side, + cublasFillMode_t uplo, int m, int n, + const double *alpha, const double *A, int lda, + const double *B, int ldb, const double *beta, + double *C, int ldc, cudaStream_t stream) { + CUBLAS_CHECK(cublasSetStream(handle, stream)); + return cublasDsymm(handle, side, uplo, m, n, alpha, A, lda, B, ldb, beta, C, + ldc); +} +/** @} */ + +/** + * @defgroup syrk cublas syrk calls + * @{ + */ +template +cublasStatus_t cublassyrk(cublasHandle_t handle, cublasFillMode_t uplo, + cublasOperation_t trans, int n, int k, const T *alpha, + const T *A, int lda, const T *beta, T *C, int ldc, + cudaStream_t stream); + +template <> +inline cublasStatus_t cublassyrk(cublasHandle_t handle, cublasFillMode_t uplo, + cublasOperation_t trans, int n, int k, + const float *alpha, const float *A, int lda, + const float *beta, float *C, int ldc, + cudaStream_t stream) { + CUBLAS_CHECK(cublasSetStream(handle, stream)); + return cublasSsyrk(handle, uplo, trans, n, k, alpha, A, lda, beta, C, ldc); +} + +template <> +inline cublasStatus_t cublassyrk(cublasHandle_t handle, cublasFillMode_t uplo, + cublasOperation_t trans, int n, int k, + const double *alpha, const double *A, int lda, + const double *beta, double *C, int ldc, + cudaStream_t stream) { + CUBLAS_CHECK(cublasSetStream(handle, stream)); + return cublasDsyrk(handle, uplo, trans, n, k, alpha, A, lda, beta, C, ldc); +} +/** @} */ + +/** + * @defgroup nrm2 cublas nrm2 calls + * @{ + */ +template +cublasStatus_t cublasnrm2(cublasHandle_t handle, int n, const T *x, int incx, + T *result, cudaStream_t stream); + +template <> +inline cublasStatus_t cublasnrm2(cublasHandle_t handle, int n, const float *x, + int incx, float *result, cudaStream_t stream) { + CUBLAS_CHECK(cublasSetStream(handle, stream)); + return cublasSnrm2(handle, n, x, incx, result); +} + +template <> +inline cublasStatus_t cublasnrm2(cublasHandle_t handle, int n, const double *x, + int incx, double *result, + cudaStream_t stream) { + CUBLAS_CHECK(cublasSetStream(handle, stream)); + return cublasDnrm2(handle, n, x, incx, result); +} +/** @} */ + +template +cublasStatus_t cublastrsm(cublasHandle_t handle, cublasSideMode_t side, + cublasFillMode_t uplo, cublasOperation_t trans, + cublasDiagType_t diag, int m, int n, const T *alpha, + const T *A, int lda, T *B, int ldb, + cudaStream_t stream); + +template <> +inline cublasStatus_t cublastrsm(cublasHandle_t handle, cublasSideMode_t side, + cublasFillMode_t uplo, cublasOperation_t trans, + cublasDiagType_t diag, int m, int n, + const float *alpha, const float *A, int lda, + float *B, int ldb, cudaStream_t stream) { + CUBLAS_CHECK(cublasSetStream(handle, stream)); + return cublasStrsm(handle, side, uplo, trans, diag, m, n, alpha, A, lda, B, + ldb); +} + +template <> +inline cublasStatus_t cublastrsm(cublasHandle_t handle, cublasSideMode_t side, + cublasFillMode_t uplo, cublasOperation_t trans, + cublasDiagType_t diag, int m, int n, + const double *alpha, const double *A, int lda, + double *B, int ldb, cudaStream_t stream) { + CUBLAS_CHECK(cublasSetStream(handle, stream)); + return cublasDtrsm(handle, side, uplo, trans, diag, m, n, alpha, A, lda, B, + ldb); +} + +/** + * @defgroup dot cublas dot calls + * @{ + */ +template +cublasStatus_t cublasdot(cublasHandle_t handle, int n, const T *x, int incx, + const T *y, int incy, T *result, cudaStream_t stream); + +template <> +inline cublasStatus_t cublasdot(cublasHandle_t handle, int n, const float *x, + int incx, const float *y, int incy, + float *result, cudaStream_t stream) { + CUBLAS_CHECK(cublasSetStream(handle, stream)); + return cublasSdot(handle, n, x, incx, y, incy, result); +} + +template <> +inline cublasStatus_t cublasdot(cublasHandle_t handle, int n, const double *x, + int incx, const double *y, int incy, + double *result, cudaStream_t stream) { + CUBLAS_CHECK(cublasSetStream(handle, stream)); + return cublasDdot(handle, n, x, incx, y, incy, result); +} +/** @} */ + +}; // namespace linalg +}; // namespace raft diff --git a/cpp/include/raft/linalg/cusolver_wrappers.h b/cpp/include/raft/linalg/cusolver_wrappers.h new file mode 100644 index 0000000000..6d35c632f2 --- /dev/null +++ b/cpp/include/raft/linalg/cusolver_wrappers.h @@ -0,0 +1,711 @@ +/* + * Copyright (c) 2018-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 +///@todo: enable this once logging is enabled +//#include +#include + +namespace raft { +namespace linalg { + +#define _CUSOLVER_ERR_TO_STR(err) \ + case err: \ + return #err; +inline const char *cusolverErr2Str(cusolverStatus_t err) { + switch (err) { + _CUSOLVER_ERR_TO_STR(CUSOLVER_STATUS_SUCCESS); + _CUSOLVER_ERR_TO_STR(CUSOLVER_STATUS_NOT_INITIALIZED); + _CUSOLVER_ERR_TO_STR(CUSOLVER_STATUS_ALLOC_FAILED); + _CUSOLVER_ERR_TO_STR(CUSOLVER_STATUS_INVALID_VALUE); + _CUSOLVER_ERR_TO_STR(CUSOLVER_STATUS_ARCH_MISMATCH); + _CUSOLVER_ERR_TO_STR(CUSOLVER_STATUS_EXECUTION_FAILED); + _CUSOLVER_ERR_TO_STR(CUSOLVER_STATUS_INTERNAL_ERROR); + _CUSOLVER_ERR_TO_STR(CUSOLVER_STATUS_MATRIX_TYPE_NOT_SUPPORTED); + _CUSOLVER_ERR_TO_STR(CUSOLVER_STATUS_ZERO_PIVOT); + _CUSOLVER_ERR_TO_STR(CUSOLVER_STATUS_NOT_SUPPORTED); + default: + return "CUSOLVER_STATUS_UNKNOWN"; + }; +} +#undef _CUSOLVER_ERR_TO_STR + +/** check for cusolver runtime API errors and assert accordingly */ +#define CUSOLVER_CHECK(call) \ + do { \ + cusolverStatus_t err = call; \ + ASSERT(err == CUSOLVER_STATUS_SUCCESS, \ + "CUSOLVER call='%s' got errorcode=%d err=%s", #call, err, \ + MLCommon::LinAlg::cusolverErr2Str(err)); \ + } while (0) + +///@todo: enable this once logging is enabled +// /** check for cusolver runtime API errors but do not assert */ +// #define CUSOLVER_CHECK_NO_THROW(call) \ +// do { \ +// cusolverStatus_t err = call; \ +// if (err != CUSOLVER_STATUS_SUCCESS) { \ +// CUML_LOG_ERROR("CUSOLVER call='%s' got errorcode=%d err=%s", #call, err, \ +// MLCommon::LinAlg::cusolverErr2Str(err)); \ +// } \ +// } while (0) + +/** + * @defgroup Getrf cusolver getrf operations + * @{ + */ +template +cusolverStatus_t cusolverDngetrf(cusolverDnHandle_t handle, int m, int n, T *A, + int lda, T *Workspace, int *devIpiv, + int *devInfo, cudaStream_t stream); + +template <> +inline cusolverStatus_t cusolverDngetrf(cusolverDnHandle_t handle, int m, int n, + float *A, int lda, float *Workspace, + int *devIpiv, int *devInfo, + cudaStream_t stream) { + CUSOLVER_CHECK(cusolverDnSetStream(handle, stream)); + return cusolverDnSgetrf(handle, m, n, A, lda, Workspace, devIpiv, devInfo); +} + +template <> +inline cusolverStatus_t cusolverDngetrf(cusolverDnHandle_t handle, int m, int n, + double *A, int lda, double *Workspace, + int *devIpiv, int *devInfo, + cudaStream_t stream) { + CUSOLVER_CHECK(cusolverDnSetStream(handle, stream)); + return cusolverDnDgetrf(handle, m, n, A, lda, Workspace, devIpiv, devInfo); +} + +template +cusolverStatus_t cusolverDngetrf_bufferSize(cusolverDnHandle_t handle, int m, + int n, T *A, int lda, int *Lwork); + +template <> +inline cusolverStatus_t cusolverDngetrf_bufferSize(cusolverDnHandle_t handle, + int m, int n, float *A, + int lda, int *Lwork) { + return cusolverDnSgetrf_bufferSize(handle, m, n, A, lda, Lwork); +} + +template <> +inline cusolverStatus_t cusolverDngetrf_bufferSize(cusolverDnHandle_t handle, + int m, int n, double *A, + int lda, int *Lwork) { + return cusolverDnDgetrf_bufferSize(handle, m, n, A, lda, Lwork); +} + +/** + * @defgroup Getrs cusolver getrs operations + * @{ + */ +template +cusolverStatus_t cusolverDngetrs(cusolverDnHandle_t handle, + cublasOperation_t trans, int n, int nrhs, + const T *A, int lda, const int *devIpiv, T *B, + int ldb, int *devInfo, cudaStream_t stream); + +template <> +inline cusolverStatus_t cusolverDngetrs(cusolverDnHandle_t handle, + cublasOperation_t trans, int n, + int nrhs, const float *A, int lda, + const int *devIpiv, float *B, int ldb, + int *devInfo, cudaStream_t stream) { + CUSOLVER_CHECK(cusolverDnSetStream(handle, stream)); + return cusolverDnSgetrs(handle, trans, n, nrhs, A, lda, devIpiv, B, ldb, + devInfo); +} + +template <> +inline cusolverStatus_t cusolverDngetrs(cusolverDnHandle_t handle, + cublasOperation_t trans, int n, + int nrhs, const double *A, int lda, + const int *devIpiv, double *B, int ldb, + int *devInfo, cudaStream_t stream) { + CUSOLVER_CHECK(cusolverDnSetStream(handle, stream)); + return cusolverDnDgetrs(handle, trans, n, nrhs, A, lda, devIpiv, B, ldb, + devInfo); +} +/** @} */ + +/** + * @defgroup syevd cusolver syevd operations + * @{ + */ +template +cusolverStatus_t cusolverDnsyevd_bufferSize(cusolverDnHandle_t handle, + cusolverEigMode_t jobz, + cublasFillMode_t uplo, int n, + const T *A, int lda, const T *W, + int *lwork); + +template <> +inline cusolverStatus_t cusolverDnsyevd_bufferSize(cusolverDnHandle_t handle, + cusolverEigMode_t jobz, + cublasFillMode_t uplo, int n, + const float *A, int lda, + const float *W, int *lwork) { + return cusolverDnSsyevd_bufferSize(handle, jobz, uplo, n, A, lda, W, lwork); +} + +template <> +inline cusolverStatus_t cusolverDnsyevd_bufferSize( + cusolverDnHandle_t handle, cusolverEigMode_t jobz, cublasFillMode_t uplo, + int n, const double *A, int lda, const double *W, int *lwork) { + return cusolverDnDsyevd_bufferSize(handle, jobz, uplo, n, A, lda, W, lwork); +} +/** @} */ + +/** + * @defgroup syevj cusolver syevj operations + * @{ + */ +template +cusolverStatus_t cusolverDnsyevj(cusolverDnHandle_t handle, + cusolverEigMode_t jobz, cublasFillMode_t uplo, + int n, T *A, int lda, T *W, T *work, int lwork, + int *info, syevjInfo_t params, + cudaStream_t stream); + +template <> +inline cusolverStatus_t cusolverDnsyevj( + cusolverDnHandle_t handle, cusolverEigMode_t jobz, cublasFillMode_t uplo, + int n, float *A, int lda, float *W, float *work, int lwork, int *info, + syevjInfo_t params, cudaStream_t stream) { + CUSOLVER_CHECK(cusolverDnSetStream(handle, stream)); + return cusolverDnSsyevj(handle, jobz, uplo, n, A, lda, W, work, lwork, info, + params); +} + +template <> +inline cusolverStatus_t cusolverDnsyevj( + cusolverDnHandle_t handle, cusolverEigMode_t jobz, cublasFillMode_t uplo, + int n, double *A, int lda, double *W, double *work, int lwork, int *info, + syevjInfo_t params, cudaStream_t stream) { + CUSOLVER_CHECK(cusolverDnSetStream(handle, stream)); + return cusolverDnDsyevj(handle, jobz, uplo, n, A, lda, W, work, lwork, info, + params); +} + +template +cusolverStatus_t cusolverDnsyevj_bufferSize(cusolverDnHandle_t handle, + cusolverEigMode_t jobz, + cublasFillMode_t uplo, int n, + const T *A, int lda, const T *W, + int *lwork, syevjInfo_t params); + +template <> +inline cusolverStatus_t cusolverDnsyevj_bufferSize(cusolverDnHandle_t handle, + cusolverEigMode_t jobz, + cublasFillMode_t uplo, int n, + const float *A, int lda, + const float *W, int *lwork, + syevjInfo_t params) { + return cusolverDnSsyevj_bufferSize(handle, jobz, uplo, n, A, lda, W, lwork, + params); +} + +template <> +inline cusolverStatus_t cusolverDnsyevj_bufferSize(cusolverDnHandle_t handle, + cusolverEigMode_t jobz, + cublasFillMode_t uplo, int n, + const double *A, int lda, + const double *W, int *lwork, + syevjInfo_t params) { + return cusolverDnDsyevj_bufferSize(handle, jobz, uplo, n, A, lda, W, lwork, + params); +} +/** @} */ + +/** + * @defgroup syevd cusolver syevd operations + * @{ + */ +template +cusolverStatus_t cusolverDnsyevd(cusolverDnHandle_t handle, + cusolverEigMode_t jobz, cublasFillMode_t uplo, + int n, T *A, int lda, T *W, T *work, int lwork, + int *devInfo, cudaStream_t stream); + +template <> +inline cusolverStatus_t cusolverDnsyevd(cusolverDnHandle_t handle, + cusolverEigMode_t jobz, + cublasFillMode_t uplo, int n, float *A, + int lda, float *W, float *work, + int lwork, int *devInfo, + cudaStream_t stream) { + CUSOLVER_CHECK(cusolverDnSetStream(handle, stream)); + return cusolverDnSsyevd(handle, jobz, uplo, n, A, lda, W, work, lwork, + devInfo); +} + +template <> +inline cusolverStatus_t cusolverDnsyevd(cusolverDnHandle_t handle, + cusolverEigMode_t jobz, + cublasFillMode_t uplo, int n, double *A, + int lda, double *W, double *work, + int lwork, int *devInfo, + cudaStream_t stream) { + CUSOLVER_CHECK(cusolverDnSetStream(handle, stream)); + return cusolverDnDsyevd(handle, jobz, uplo, n, A, lda, W, work, lwork, + devInfo); +} +/** @} */ + +#if CUDART_VERSION >= 10010 +/** + * @defgroup syevdx cusolver syevdx operations + * @{ +*/ +template +cusolverStatus_t cusolverDnsyevdx_bufferSize( + cusolverDnHandle_t handle, cusolverEigMode_t jobz, cusolverEigRange_t range, + cublasFillMode_t uplo, int n, const T *A, int lda, T vl, T vu, int il, int iu, + int *h_meig, const T *W, int *lwork); + +template <> +inline cusolverStatus_t cusolverDnsyevdx_bufferSize( + cusolverDnHandle_t handle, cusolverEigMode_t jobz, cusolverEigRange_t range, + cublasFillMode_t uplo, int n, const float *A, int lda, float vl, float vu, + int il, int iu, int *h_meig, const float *W, int *lwork) { + return cusolverDnSsyevdx_bufferSize(handle, jobz, range, uplo, n, A, lda, vl, + vu, il, iu, h_meig, W, lwork); +} + +template <> +inline cusolverStatus_t cusolverDnsyevdx_bufferSize( + cusolverDnHandle_t handle, cusolverEigMode_t jobz, cusolverEigRange_t range, + cublasFillMode_t uplo, int n, const double *A, int lda, double vl, double vu, + int il, int iu, int *h_meig, const double *W, int *lwork) { + return cusolverDnDsyevdx_bufferSize(handle, jobz, range, uplo, n, A, lda, vl, + vu, il, iu, h_meig, W, lwork); +} + +template +cusolverStatus_t cusolverDnsyevdx( + cusolverDnHandle_t handle, cusolverEigMode_t jobz, cusolverEigRange_t range, + cublasFillMode_t uplo, int n, T *A, int lda, T vl, T vu, int il, int iu, + int *h_meig, T *W, T *work, int lwork, int *devInfo, cudaStream_t stream); + +template <> +inline cusolverStatus_t cusolverDnsyevdx( + cusolverDnHandle_t handle, cusolverEigMode_t jobz, cusolverEigRange_t range, + cublasFillMode_t uplo, int n, float *A, int lda, float vl, float vu, int il, + int iu, int *h_meig, float *W, float *work, int lwork, int *devInfo, + cudaStream_t stream) { + CUSOLVER_CHECK(cusolverDnSetStream(handle, stream)); + return cusolverDnSsyevdx(handle, jobz, range, uplo, n, A, lda, vl, vu, il, iu, + h_meig, W, work, lwork, devInfo); +} + +template <> +inline cusolverStatus_t cusolverDnsyevdx( + cusolverDnHandle_t handle, cusolverEigMode_t jobz, cusolverEigRange_t range, + cublasFillMode_t uplo, int n, double *A, int lda, double vl, double vu, + int il, int iu, int *h_meig, double *W, double *work, int lwork, int *devInfo, + cudaStream_t stream) { + CUSOLVER_CHECK(cusolverDnSetStream(handle, stream)); + return cusolverDnDsyevdx(handle, jobz, range, uplo, n, A, lda, vl, vu, il, iu, + h_meig, W, work, lwork, devInfo); +} +/** @} */ +#endif + +/** + * @defgroup svd cusolver svd operations + * @{ + */ +template +cusolverStatus_t cusolverDngesvd_bufferSize(cusolverDnHandle_t handle, int m, + int n, int *lwork) { + if (typeid(T) == typeid(float)) { + return cusolverDnSgesvd_bufferSize(handle, m, n, lwork); + } else { + return cusolverDnDgesvd_bufferSize(handle, m, n, lwork); + } +} +template +cusolverStatus_t cusolverDngesvd(cusolverDnHandle_t handle, signed char jobu, + signed char jobvt, int m, int n, T *A, int lda, + T *S, T *U, int ldu, T *VT, int ldvt, T *work, + int lwork, T *rwork, int *devInfo, + cudaStream_t stream); +template <> +inline cusolverStatus_t cusolverDngesvd( + cusolverDnHandle_t handle, signed char jobu, signed char jobvt, int m, int n, + float *A, int lda, float *S, float *U, int ldu, float *VT, int ldvt, + float *work, int lwork, float *rwork, int *devInfo, cudaStream_t stream) { + CUSOLVER_CHECK(cusolverDnSetStream(handle, stream)); + return cusolverDnSgesvd(handle, jobu, jobvt, m, n, A, lda, S, U, ldu, VT, + ldvt, work, lwork, rwork, devInfo); +} +template <> +inline cusolverStatus_t cusolverDngesvd( + cusolverDnHandle_t handle, signed char jobu, signed char jobvt, int m, int n, + double *A, int lda, double *S, double *U, int ldu, double *VT, int ldvt, + double *work, int lwork, double *rwork, int *devInfo, cudaStream_t stream) { + CUSOLVER_CHECK(cusolverDnSetStream(handle, stream)); + return cusolverDnDgesvd(handle, jobu, jobvt, m, n, A, lda, S, U, ldu, VT, + ldvt, work, lwork, rwork, devInfo); +} + +template +inline cusolverStatus_t CUSOLVERAPI cusolverDngesvdj_bufferSize( + cusolverDnHandle_t handle, cusolverEigMode_t jobz, int econ, int m, int n, + const T *A, int lda, const T *S, const T *U, int ldu, const T *V, int ldv, + int *lwork, gesvdjInfo_t params); +template <> +inline cusolverStatus_t CUSOLVERAPI cusolverDngesvdj_bufferSize( + cusolverDnHandle_t handle, cusolverEigMode_t jobz, int econ, int m, int n, + const float *A, int lda, const float *S, const float *U, int ldu, + const float *V, int ldv, int *lwork, gesvdjInfo_t params) { + return cusolverDnSgesvdj_bufferSize(handle, jobz, econ, m, n, A, lda, S, U, + ldu, V, ldv, lwork, params); +} +template <> +inline cusolverStatus_t CUSOLVERAPI cusolverDngesvdj_bufferSize( + cusolverDnHandle_t handle, cusolverEigMode_t jobz, int econ, int m, int n, + const double *A, int lda, const double *S, const double *U, int ldu, + const double *V, int ldv, int *lwork, gesvdjInfo_t params) { + return cusolverDnDgesvdj_bufferSize(handle, jobz, econ, m, n, A, lda, S, U, + ldu, V, ldv, lwork, params); +} +template +inline cusolverStatus_t CUSOLVERAPI cusolverDngesvdj( + cusolverDnHandle_t handle, cusolverEigMode_t jobz, int econ, int m, int n, + T *A, int lda, T *S, T *U, int ldu, T *V, int ldv, T *work, int lwork, + int *info, gesvdjInfo_t params, cudaStream_t stream); +template <> +inline cusolverStatus_t CUSOLVERAPI cusolverDngesvdj( + cusolverDnHandle_t handle, cusolverEigMode_t jobz, int econ, int m, int n, + float *A, int lda, float *S, float *U, int ldu, float *V, int ldv, + float *work, int lwork, int *info, gesvdjInfo_t params, cudaStream_t stream) { + CUSOLVER_CHECK(cusolverDnSetStream(handle, stream)); + return cusolverDnSgesvdj(handle, jobz, econ, m, n, A, lda, S, U, ldu, V, ldv, + work, lwork, info, params); +} +template <> +inline cusolverStatus_t CUSOLVERAPI +cusolverDngesvdj(cusolverDnHandle_t handle, cusolverEigMode_t jobz, int econ, + int m, int n, double *A, int lda, double *S, double *U, + int ldu, double *V, int ldv, double *work, int lwork, + int *info, gesvdjInfo_t params, cudaStream_t stream) { + CUSOLVER_CHECK(cusolverDnSetStream(handle, stream)); + return cusolverDnDgesvdj(handle, jobz, econ, m, n, A, lda, S, U, ldu, V, ldv, + work, lwork, info, params); +} +/** @} */ + +/** + * @defgroup potrf cusolver potrf operations + * @{ + */ +template +cusolverStatus_t cusolverDnpotrf_bufferSize(cusolverDnHandle_t handle, + cublasFillMode_t uplo, int n, T *A, + int lda, int *Lwork); + +template <> +inline cusolverStatus_t cusolverDnpotrf_bufferSize(cusolverDnHandle_t handle, + cublasFillMode_t uplo, int n, + float *A, int lda, + int *Lwork) { + return cusolverDnSpotrf_bufferSize(handle, uplo, n, A, lda, Lwork); +} + +template <> +inline cusolverStatus_t cusolverDnpotrf_bufferSize(cusolverDnHandle_t handle, + cublasFillMode_t uplo, int n, + double *A, int lda, + int *Lwork) { + return cusolverDnDpotrf_bufferSize(handle, uplo, n, A, lda, Lwork); +} + +template +inline cusolverStatus_t cusolverDnpotrf(cusolverDnHandle_t handle, + cublasFillMode_t uplo, int n, T *A, + int lda, T *Workspace, int Lwork, + int *devInfo, cudaStream_t stream); + +template <> +inline cusolverStatus_t cusolverDnpotrf(cusolverDnHandle_t handle, + cublasFillMode_t uplo, int n, float *A, + int lda, float *Workspace, int Lwork, + int *devInfo, cudaStream_t stream) { + CUSOLVER_CHECK(cusolverDnSetStream(handle, stream)); + return cusolverDnSpotrf(handle, uplo, n, A, lda, Workspace, Lwork, devInfo); +} + +template <> +inline cusolverStatus_t cusolverDnpotrf(cusolverDnHandle_t handle, + cublasFillMode_t uplo, int n, double *A, + int lda, double *Workspace, int Lwork, + int *devInfo, cudaStream_t stream) { + CUSOLVER_CHECK(cusolverDnSetStream(handle, stream)); + return cusolverDnDpotrf(handle, uplo, n, A, lda, Workspace, Lwork, devInfo); +} +/** @} */ + +/** + * @defgroup potrs cusolver potrs operations + * @{ + */ +template +cusolverStatus_t cusolverDnpotrs(cusolverDnHandle_t handle, + cublasFillMode_t uplo, int n, int nrhs, + const T *A, int lda, T *B, int ldb, + int *devInfo, cudaStream_t stream); + +template <> +inline cusolverStatus_t cusolverDnpotrs(cusolverDnHandle_t handle, + cublasFillMode_t uplo, int n, int nrhs, + const float *A, int lda, float *B, + int ldb, int *devInfo, + cudaStream_t stream) { + CUSOLVER_CHECK(cusolverDnSetStream(handle, stream)); + return cusolverDnSpotrs(handle, uplo, n, nrhs, A, lda, B, ldb, devInfo); +} + +template <> +inline cusolverStatus_t cusolverDnpotrs(cusolverDnHandle_t handle, + cublasFillMode_t uplo, int n, int nrhs, + const double *A, int lda, double *B, + int ldb, int *devInfo, + cudaStream_t stream) { + CUSOLVER_CHECK(cusolverDnSetStream(handle, stream)); + return cusolverDnDpotrs(handle, uplo, n, nrhs, A, lda, B, ldb, devInfo); +} +/** @} */ + +/** + * @defgroup geqrf cusolver geqrf operations + * @{ + */ +template +cusolverStatus_t cusolverDngeqrf(cusolverDnHandle_t handle, int m, int n, T *A, + int lda, T *TAU, T *Workspace, int Lwork, + int *devInfo, cudaStream_t stream); +template <> +inline cusolverStatus_t cusolverDngeqrf(cusolverDnHandle_t handle, int m, int n, + float *A, int lda, float *TAU, + float *Workspace, int Lwork, + int *devInfo, cudaStream_t stream) { + CUSOLVER_CHECK(cusolverDnSetStream(handle, stream)); + return cusolverDnSgeqrf(handle, m, n, A, lda, TAU, Workspace, Lwork, devInfo); +} +template <> +inline cusolverStatus_t cusolverDngeqrf(cusolverDnHandle_t handle, int m, int n, + double *A, int lda, double *TAU, + double *Workspace, int Lwork, + int *devInfo, cudaStream_t stream) { + CUSOLVER_CHECK(cusolverDnSetStream(handle, stream)); + return cusolverDnDgeqrf(handle, m, n, A, lda, TAU, Workspace, Lwork, devInfo); +} + +template +cusolverStatus_t cusolverDngeqrf_bufferSize(cusolverDnHandle_t handle, int m, + int n, T *A, int lda, int *Lwork); +template <> +inline cusolverStatus_t cusolverDngeqrf_bufferSize(cusolverDnHandle_t handle, + int m, int n, float *A, + int lda, int *Lwork) { + return cusolverDnSgeqrf_bufferSize(handle, m, n, A, lda, Lwork); +} +template <> +inline cusolverStatus_t cusolverDngeqrf_bufferSize(cusolverDnHandle_t handle, + int m, int n, double *A, + int lda, int *Lwork) { + return cusolverDnDgeqrf_bufferSize(handle, m, n, A, lda, Lwork); +} +/** @} */ + +/** + * @defgroup orgqr cusolver orgqr operations + * @{ + */ +template +cusolverStatus_t cusolverDnorgqr(cusolverDnHandle_t handle, int m, int n, int k, + T *A, int lda, const T *tau, T *work, + int lwork, int *devInfo, cudaStream_t stream); +template <> +inline cusolverStatus_t cusolverDnorgqr(cusolverDnHandle_t handle, int m, int n, + int k, float *A, int lda, + const float *tau, float *work, + int lwork, int *devInfo, + cudaStream_t stream) { + CUSOLVER_CHECK(cusolverDnSetStream(handle, stream)); + return cusolverDnSorgqr(handle, m, n, k, A, lda, tau, work, lwork, devInfo); +} +template <> +inline cusolverStatus_t cusolverDnorgqr(cusolverDnHandle_t handle, int m, int n, + int k, double *A, int lda, + const double *tau, double *work, + int lwork, int *devInfo, + cudaStream_t stream) { + CUSOLVER_CHECK(cusolverDnSetStream(handle, stream)); + return cusolverDnDorgqr(handle, m, n, k, A, lda, tau, work, lwork, devInfo); +} + +template +cusolverStatus_t cusolverDnorgqr_bufferSize(cusolverDnHandle_t handle, int m, + int n, int k, const T *A, int lda, + const T *TAU, int *lwork); +template <> +inline cusolverStatus_t cusolverDnorgqr_bufferSize(cusolverDnHandle_t handle, + int m, int n, int k, + const float *A, int lda, + const float *TAU, + int *lwork) { + return cusolverDnSorgqr_bufferSize(handle, m, n, k, A, lda, TAU, lwork); +} +template <> +inline cusolverStatus_t cusolverDnorgqr_bufferSize(cusolverDnHandle_t handle, + int m, int n, int k, + const double *A, int lda, + const double *TAU, + int *lwork) { + return cusolverDnDorgqr_bufferSize(handle, m, n, k, A, lda, TAU, lwork); +} +/** @} */ + +/** + * @defgroup ormqr cusolver ormqr operations + * @{ + */ +template +cusolverStatus_t cusolverDnormqr(cusolverDnHandle_t handle, + cublasSideMode_t side, cublasOperation_t trans, + int m, int n, int k, const T *A, int lda, + const T *tau, T *C, int ldc, T *work, + int lwork, int *devInfo, cudaStream_t stream); + +template <> +inline cusolverStatus_t cusolverDnormqr( + cusolverDnHandle_t handle, cublasSideMode_t side, cublasOperation_t trans, + int m, int n, int k, const float *A, int lda, const float *tau, float *C, + int ldc, float *work, int lwork, int *devInfo, cudaStream_t stream) { + CUSOLVER_CHECK(cusolverDnSetStream(handle, stream)); + return cusolverDnSormqr(handle, side, trans, m, n, k, A, lda, tau, C, ldc, + work, lwork, devInfo); +} + +template <> +inline cusolverStatus_t cusolverDnormqr( + cusolverDnHandle_t handle, cublasSideMode_t side, cublasOperation_t trans, + int m, int n, int k, const double *A, int lda, const double *tau, double *C, + int ldc, double *work, int lwork, int *devInfo, cudaStream_t stream) { + CUSOLVER_CHECK(cusolverDnSetStream(handle, stream)); + return cusolverDnDormqr(handle, side, trans, m, n, k, A, lda, tau, C, ldc, + work, lwork, devInfo); +} + +template +cusolverStatus_t cusolverDnormqr_bufferSize(cusolverDnHandle_t handle, + cublasSideMode_t side, + cublasOperation_t trans, int m, + int n, int k, const T *A, int lda, + const T *tau, const T *C, int ldc, + int *lwork); + +template <> +inline cusolverStatus_t cusolverDnormqr_bufferSize( + cusolverDnHandle_t handle, cublasSideMode_t side, cublasOperation_t trans, + int m, int n, int k, const float *A, int lda, const float *tau, + const float *C, int ldc, int *lwork) { + return cusolverDnSormqr_bufferSize(handle, side, trans, m, n, k, A, lda, tau, + C, ldc, lwork); +} + +template <> +inline cusolverStatus_t cusolverDnormqr_bufferSize( + cusolverDnHandle_t handle, cublasSideMode_t side, cublasOperation_t trans, + int m, int n, int k, const double *A, int lda, const double *tau, + const double *C, int ldc, int *lwork) { + return cusolverDnDormqr_bufferSize(handle, side, trans, m, n, k, A, lda, tau, + C, ldc, lwork); +} +/** @} */ + +/** + * @defgroup csrqrBatched cusolver batched + * @{ + */ +template +cusolverStatus_t cusolverSpcsrqrBufferInfoBatched( + cusolverSpHandle_t handle, int m, int n, int nnzA, + const cusparseMatDescr_t descrA, const T *csrValA, const int *csrRowPtrA, + const int *csrColIndA, int batchSize, csrqrInfo_t info, + size_t *internalDataInBytes, size_t *workspaceInBytes); + +template <> +inline cusolverStatus_t cusolverSpcsrqrBufferInfoBatched( + cusolverSpHandle_t handle, int m, int n, int nnzA, + const cusparseMatDescr_t descrA, const float *csrValA, const int *csrRowPtrA, + const int *csrColIndA, int batchSize, csrqrInfo_t info, + size_t *internalDataInBytes, size_t *workspaceInBytes) { + return cusolverSpScsrqrBufferInfoBatched( + handle, m, n, nnzA, descrA, csrValA, csrRowPtrA, csrColIndA, batchSize, + info, internalDataInBytes, workspaceInBytes); +} + +template <> +inline cusolverStatus_t cusolverSpcsrqrBufferInfoBatched( + cusolverSpHandle_t handle, int m, int n, int nnzA, + const cusparseMatDescr_t descrA, const double *csrValA, const int *csrRowPtrA, + const int *csrColIndA, int batchSize, csrqrInfo_t info, + size_t *internalDataInBytes, size_t *workspaceInBytes) { + return cusolverSpDcsrqrBufferInfoBatched( + handle, m, n, nnzA, descrA, csrValA, csrRowPtrA, csrColIndA, batchSize, + info, internalDataInBytes, workspaceInBytes); +} + +template +cusolverStatus_t cusolverSpcsrqrsvBatched( + cusolverSpHandle_t handle, int m, int n, int nnzA, + const cusparseMatDescr_t descrA, const T *csrValA, const int *csrRowPtrA, + const int *csrColIndA, const T *b, T *x, int batchSize, csrqrInfo_t info, + void *pBuffer, cudaStream_t stream); + +template <> +inline cusolverStatus_t cusolverSpcsrqrsvBatched( + cusolverSpHandle_t handle, int m, int n, int nnzA, + const cusparseMatDescr_t descrA, const float *csrValA, const int *csrRowPtrA, + const int *csrColIndA, const float *b, float *x, int batchSize, + csrqrInfo_t info, void *pBuffer, cudaStream_t stream) { + CUSOLVER_CHECK(cusolverSpSetStream(handle, stream)); + return cusolverSpScsrqrsvBatched(handle, m, n, nnzA, descrA, csrValA, + csrRowPtrA, csrColIndA, b, x, batchSize, + info, pBuffer); +} + +template <> +inline cusolverStatus_t cusolverSpcsrqrsvBatched( + cusolverSpHandle_t handle, int m, int n, int nnzA, + const cusparseMatDescr_t descrA, const double *csrValA, const int *csrRowPtrA, + const int *csrColIndA, const double *b, double *x, int batchSize, + csrqrInfo_t info, void *pBuffer, cudaStream_t stream) { + CUSOLVER_CHECK(cusolverSpSetStream(handle, stream)); + return cusolverSpDcsrqrsvBatched(handle, m, n, nnzA, descrA, csrValA, + csrRowPtrA, csrColIndA, b, x, batchSize, + info, pBuffer); +} +/** @} */ + +}; // namespace linalg +}; // namespace raft diff --git a/cpp/include/raft/sparse/cusparse_wrappers.h b/cpp/include/raft/sparse/cusparse_wrappers.h new file mode 100644 index 0000000000..369624786d --- /dev/null +++ b/cpp/include/raft/sparse/cusparse_wrappers.h @@ -0,0 +1,167 @@ +/* + * 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 +///@todo: enable this once logging is enabled +//#include +#include + +namespace raft { +namespace sparse { + +#define _CUSPARSE_ERR_TO_STR(err) \ + case err: \ + return #err; +inline const char* cusparseErr2Str(cusparseStatus_t err) { +#if defined(CUDART_VERSION) && CUDART_VERSION >= 10100 + return cusparseGetErrorString(status); +#else // CUDART_VERSION + switch (err) { + _CUSPARSE_ERR_TO_STR(CUSPARSE_STATUS_SUCCESS); + _CUSPARSE_ERR_TO_STR(CUSPARSE_STATUS_NOT_INITIALIZED); + _CUSPARSE_ERR_TO_STR(CUSPARSE_STATUS_ALLOC_FAILED); + _CUSPARSE_ERR_TO_STR(CUSPARSE_STATUS_INVALID_VALUE); + _CUSPARSE_ERR_TO_STR(CUSPARSE_STATUS_ARCH_MISMATCH); + _CUSPARSE_ERR_TO_STR(CUSPARSE_STATUS_EXECUTION_FAILED); + _CUSPARSE_ERR_TO_STR(CUSPARSE_STATUS_INTERNAL_ERROR); + _CUSPARSE_ERR_TO_STR(CUSPARSE_STATUS_MATRIX_TYPE_NOT_SUPPORTED); + default: + return "CUSPARSE_STATUS_UNKNOWN"; + }; +#endif // CUDART_VERSION +} +#undef _CUSPARSE_ERR_TO_STR + +/** check for cusparse runtime API errors and assert accordingly */ +#define CUSPARSE_CHECK(call) \ + do { \ + cusparseStatus_t err = call; \ + ASSERT(err == CUSPARSE_STATUS_SUCCESS, \ + "CUSPARSE call='%s' got errorcode=%d err=%s", #call, err, \ + raft::sparse::cusparseErr2Str(err)); \ + } while (0) + +///@todo: enable this once logging is enabled +// /** check for cusparse runtime API errors but do not assert */ +// #define CUSPARSE_CHECK_NO_THROW(call) \ +// do { \ +// cusparseStatus_t err = call; \ +// if (err != CUSPARSE_STATUS_SUCCESS) { \ +// CUML_LOG_ERROR("CUSPARSE call='%s' got errorcode=%d err=%s", #call, err, \ +// raft::sparse::cusparseErr2Str(err)); \ +// } \ +// } while (0) + +/** + * @defgroup gthr cusparse gather methods + * @{ + */ +template +cusparseStatus_t cusparsegthr(cusparseHandle_t handle, int nnz, const T* vals, + T* vals_sorted, int* d_P, cudaStream_t stream); +template <> +inline cusparseStatus_t cusparsegthr(cusparseHandle_t handle, int nnz, + const double* vals, double* vals_sorted, + int* d_P, cudaStream_t stream) { + CUSPARSE_CHECK(cusparseSetStream(handle, stream)); + return cusparseDgthr(handle, nnz, vals, vals_sorted, d_P, + CUSPARSE_INDEX_BASE_ZERO); +} +template <> +inline cusparseStatus_t cusparsegthr(cusparseHandle_t handle, int nnz, + const float* vals, float* vals_sorted, + int* d_P, cudaStream_t stream) { + CUSPARSE_CHECK(cusparseSetStream(handle, stream)); + return cusparseSgthr(handle, nnz, vals, vals_sorted, d_P, + CUSPARSE_INDEX_BASE_ZERO); +} +/** @} */ + +/** + * @defgroup coo2csr cusparse COO to CSR converter methods + * @{ + */ +template +void cusparsecoo2csr(cusparseHandle_t handle, const T* cooRowInd, int nnz, + int m, T* csrRowPtr, cudaStream_t stream); +template <> +inline void cusparsecoo2csr(cusparseHandle_t handle, const int* cooRowInd, + int nnz, int m, int* csrRowPtr, + cudaStream_t stream) { + CUSPARSE_CHECK(cusparseSetStream(handle, stream)); + CUSPARSE_CHECK(cusparseXcoo2csr(handle, cooRowInd, nnz, m, csrRowPtr, + CUSPARSE_INDEX_BASE_ZERO)); +} +/** @} */ + +/** + * @defgroup coosort cusparse coo sort methods + * @{ + */ +template +size_t cusparsecoosort_bufferSizeExt(cusparseHandle_t handle, int m, int n, + int nnz, const T* cooRows, + const T* cooCols, cudaStream_t stream); +template <> +inline size_t cusparsecoosort_bufferSizeExt(cusparseHandle_t handle, int m, + int n, int nnz, const int* cooRows, + const int* cooCols, + cudaStream_t stream) { + size_t val; + CUSPARSE_CHECK(cusparseSetStream(handle, stream)); + CUSPARSE_CHECK( + cusparseXcoosort_bufferSizeExt(handle, m, n, nnz, cooRows, cooCols, &val)); + return val; +} + +template +void cusparsecoosortByRow(cusparseHandle_t handle, int m, int n, int nnz, + T* cooRows, T* cooCols, T* P, void* pBuffer, + cudaStream_t stream); +template <> +inline void cusparsecoosortByRow(cusparseHandle_t handle, int m, int n, int nnz, + int* cooRows, int* cooCols, int* P, + void* pBuffer, cudaStream_t stream) { + CUSPARSE_CHECK(cusparseSetStream(handle, stream)); + CUSPARSE_CHECK( + cusparseXcoosortByRow(handle, m, n, nnz, cooRows, cooCols, P, pBuffer)); +} +/** @} */ + +/** + * @defgroup Gemmi cusparse gemmi operations + * @{ + */ +inline cusparseStatus_t cusparsegemmi( + cusparseHandle_t handle, int m, int n, int k, int nnz, const float* alpha, + const float* A, int lda, const float* cscValB, const int* cscColPtrB, + const int* cscRowIndB, const float* beta, float* C, int ldc) { + return cusparseSgemmi(handle, m, n, k, nnz, alpha, A, lda, cscValB, + cscColPtrB, cscRowIndB, beta, C, ldc); +} +inline cusparseStatus_t cusparsegemmi( + cusparseHandle_t handle, int m, int n, int k, int nnz, const double* alpha, + const double* A, int lda, const double* cscValB, const int* cscColPtrB, + const int* cscRowIndB, const double* beta, double* C, int ldc) { + return cusparseDgemmi(handle, m, n, k, nnz, alpha, A, lda, cscValB, + cscColPtrB, cscRowIndB, beta, C, ldc); +} +/** @} */ + +}; // namespace sparse +}; // namespace raft From 92daf4d9f36b490c11d4ccef930868b7440ff457 Mon Sep 17 00:00:00 2001 From: Thejaswi Rao Date: Mon, 27 Apr 2020 23:45:35 -0700 Subject: [PATCH 07/56] FIX updated project name in cmake file --- cpp/CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cpp/CMakeLists.txt b/cpp/CMakeLists.txt index 7163b0dd77..5603e3203d 100644 --- a/cpp/CMakeLists.txt +++ b/cpp/CMakeLists.txt @@ -17,7 +17,7 @@ cmake_minimum_required(VERSION 3.14 FATAL_ERROR) -project(CUML VERSION 0.14.0 LANGUAGES CXX CUDA) +project(RAFT VERSION 0.14.0 LANGUAGES CXX CUDA) ############################################################################## # - build type --------------------------------------------------------------- From c93afda15816b6b3f727f3ff96a5c76bbafc3771 Mon Sep 17 00:00:00 2001 From: Thejaswi Rao Date: Tue, 28 Apr 2020 01:46:00 -0700 Subject: [PATCH 08/56] ENH added googletest build for unit-testing --- cpp/CMakeLists.txt | 25 +++++++++++++++------ cpp/cmake/Dependencies.cmake | 42 ++++++++++++++++++++++++++++++++++++ cpp/test/test.cpp | 10 ++++++--- 3 files changed, 68 insertions(+), 9 deletions(-) create mode 100644 cpp/cmake/Dependencies.cmake diff --git a/cpp/CMakeLists.txt b/cpp/CMakeLists.txt index 5603e3203d..edc236e93f 100644 --- a/cpp/CMakeLists.txt +++ b/cpp/CMakeLists.txt @@ -45,6 +45,8 @@ option(LINE_INFO "Enable lineinfo in nvcc" OFF) option(NVTX "Enable nvtx markers" OFF) +option(BUILD_RAFT_TESTS "Build raft unit-tests" ON) + set(PARALLEL_LEVEL "" CACHE STRING "Sub-projects parallel level for compilation. Currently only affects FAISS" ) @@ -53,12 +55,13 @@ set(GPU_ARCHS "" CACHE STRING Pass 'ALL' if you want to compile for all supported GPU architectures. Empty string means to auto-detect the GPUs on the current system") - ############################################################################## # - Requirements ------------------------------------------------------------- find_package(CUDA 10.0 REQUIRED) +set(CMAKE_MODULE_PATH ${CMAKE_CURRENT_SOURCE_DIR}/cmake) + ############################################################################## # - Compiler Options -------------------------------------------------------- @@ -149,8 +152,10 @@ endif(CMAKE_COMPILER_IS_GNUCXX) set(CMAKE_CUDA_FLAGS "${CMAKE_CUDA_FLAGS} -Xcudafe --diag_suppress=unrecognized_gcc_pragma") +############################################################################## +# - dependencies ------------------------------------------------------------- - +include(cmake/Dependencies.cmake) ############################################################################## # - include paths ------------------------------------------------------------ @@ -165,11 +170,19 @@ set(RAFT_INCLUDE_DIRECTORIES ############################################################################## # - build test executables --------------------------------------------------- -add_executable(test_raft - test/test.cpp) +if(BUILD_RAFT_TESTS) + add_executable(test_raft + test/test.cpp) + + target_include_directories(test_raft PRIVATE + ${RAFT_INCLUDE_DIRECTORIES} + ${GTEST_DIR}/include) -target_include_directories(test_raft PRIVATE - ${RAFT_INCLUDE_DIRECTORIES}) + target_link_libraries(test_raft + gtestlib + gtest_mainlib + pthread) +endif(BUILD_RAFT_TESTS) ############################################################################## # - doxygen targets ---------------------------------------------------------- diff --git a/cpp/cmake/Dependencies.cmake b/cpp/cmake/Dependencies.cmake new file mode 100644 index 0000000000..d9d15a0ea8 --- /dev/null +++ b/cpp/cmake/Dependencies.cmake @@ -0,0 +1,42 @@ +#============================================================================= +# 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. +#============================================================================= + +include(ExternalProject) + +############################################################################## +# - googletest --------------------------------------------------------------- + +set(GTEST_DIR ${CMAKE_CURRENT_BINARY_DIR}/googletest CACHE STRING + "Path to googletest repo") +include(ExternalProject) +ExternalProject_Add(googletest + GIT_REPOSITORY https://github.com/google/googletest.git + GIT_TAG 6ce9b98f541b8bcd84c5c5b3483f29a933c4aefb + PREFIX ${GTEST_DIR} + CMAKE_ARGS -DCMAKE_INSTALL_PREFIX= + -DBUILD_SHARED_LIBS=OFF + -DCMAKE_INSTALL_LIBDIR=lib + BUILD_BYPRODUCTS ${GTEST_DIR}/lib/libgtest.a + ${GTEST_DIR}/lib/libgtest_main.a + UPDATE_COMMAND "") +add_library(gtestlib STATIC IMPORTED) +add_library(gtest_mainlib STATIC IMPORTED) +set_property(TARGET gtestlib PROPERTY + IMPORTED_LOCATION ${GTEST_DIR}/lib/libgtest.a) +set_property(TARGET gtest_mainlib PROPERTY + IMPORTED_LOCATION ${GTEST_DIR}/lib/libgtest_main.a) +add_dependencies(gtestlib googletest) +add_dependencies(gtest_mainlib googletest) diff --git a/cpp/test/test.cpp b/cpp/test/test.cpp index 1cb0434587..32c527492b 100644 --- a/cpp/test/test.cpp +++ b/cpp/test/test.cpp @@ -17,8 +17,12 @@ #include #include +#include -int main(){ - std::string result = raft::test_raft(); - std::cout << result; +namespace raft { + +TEST(Raft, print) { + std::cout << test_raft() << std::endl; } + +} // namespace raft From 0d369d23af6e0c7dcb969c5d67acfcaa1eaf8e27 Mon Sep 17 00:00:00 2001 From: Thejaswi Rao Date: Tue, 28 Apr 2020 11:05:45 -0700 Subject: [PATCH 09/56] ENH added proper include-dirs and link libraries for the unit-test to run successfully --- cpp/CMakeLists.txt | 20 ++++++++++++++++---- cpp/test/test.cpp | 1 - 2 files changed, 16 insertions(+), 5 deletions(-) diff --git a/cpp/CMakeLists.txt b/cpp/CMakeLists.txt index edc236e93f..9e583f8417 100644 --- a/cpp/CMakeLists.txt +++ b/cpp/CMakeLists.txt @@ -164,14 +164,25 @@ set(RAFT_INCLUDE_DIR ${CMAKE_CURRENT_SOURCE_DIR}/include CACHE STRING "Path to RAFT include directories") set(RAFT_INCLUDE_DIRECTORIES - ${RAFT_INCLUDE_DIR} -) + ${RAFT_INCLUDE_DIR} + ${CMAKE_CUDA_TOOLKIT_INCLUDE_DIRECTORIES}) ############################################################################## -# - build test executables --------------------------------------------------- +# - libraries ---------------------------------------------------------------- + +set(RAFT_LINK_LIBRARIES + ${CUDA_cublas_LIBRARY} + ${CUDA_cusolver_LIBRARY} + ${CUDA_CUDART_LIBRARY} + ${CUDA_cusparse_LIBRARY}) + +############################################################################## +# - build test executable ---------------------------------------------------- if(BUILD_RAFT_TESTS) + # keep the files in alphabetical order! add_executable(test_raft + test/cudart_utils.cpp test/test.cpp) target_include_directories(test_raft PRIVATE @@ -181,7 +192,8 @@ if(BUILD_RAFT_TESTS) target_link_libraries(test_raft gtestlib gtest_mainlib - pthread) + pthread + ${RAFT_LINK_LIBRARIES}) endif(BUILD_RAFT_TESTS) ############################################################################## diff --git a/cpp/test/test.cpp b/cpp/test/test.cpp index 32c527492b..2305bc9dc8 100644 --- a/cpp/test/test.cpp +++ b/cpp/test/test.cpp @@ -14,7 +14,6 @@ * limitations under the License. */ - #include #include #include From 81d5a70d0f9c618e3252830fd7ae98f1aa37c071 Mon Sep 17 00:00:00 2001 From: Thejaswi Rao Date: Tue, 28 Apr 2020 11:17:07 -0700 Subject: [PATCH 10/56] ENH initial commit for cudart_utils unit-tests --- cpp/test/cudart_utils.cpp | 30 ++++++++++++++++++++++++++++++ 1 file changed, 30 insertions(+) create mode 100644 cpp/test/cudart_utils.cpp diff --git a/cpp/test/cudart_utils.cpp b/cpp/test/cudart_utils.cpp new file mode 100644 index 0000000000..23be71cd70 --- /dev/null +++ b/cpp/test/cudart_utils.cpp @@ -0,0 +1,30 @@ +/* + * 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. + */ + +#include +#include +#include + +namespace raft { + +TEST(Raft, Utils) { + ASSERT_NO_THROW(ASSERT(1 == 1, "Should not assert!")); + ASSERT_THROW(ASSERT(1 != 1, "Should assert!"), Exception); + ASSERT_THROW(THROW("Should throw!"), Exception); + //ASSERT_NO_THROW(CUDA_CHECK(cudaFree(nullptr))); +} + +} // namespace raft From 7039a55e339f1aec2b7f0440c2e8b86226b6dddd Mon Sep 17 00:00:00 2001 From: Thejaswi Rao Date: Mon, 4 May 2020 11:18:21 -0700 Subject: [PATCH 11/56] FIX enabled ABI to make gtests running again --- cpp/CMakeLists.txt | 11 ++++++++--- cpp/test/cudart_utils.cpp | 2 +- 2 files changed, 9 insertions(+), 4 deletions(-) diff --git a/cpp/CMakeLists.txt b/cpp/CMakeLists.txt index 9e583f8417..93e9760767 100644 --- a/cpp/CMakeLists.txt +++ b/cpp/CMakeLists.txt @@ -37,6 +37,8 @@ endif() ############################################################################## # - User Options ------------------------------------------------------------ +option(CMAKE_CXX11_ABI "Enable the GLIBCXX11 ABI" ON) + option(EMPTY_MARKER_KERNEL "Enable empty marker kernel after nvtxRangePop" ON) option(KERNEL_INFO "Enable kernel resource usage info" OFF) @@ -88,7 +90,7 @@ endif(OPENMP_FOUND) set(CMAKE_CUDA_FLAGS "${CMAKE_CUDA_FLAGS} --expt-extended-lambda") if(${CMAKE_VERSION} VERSION_LESS "3.17.0") - set(CMAKE_CUDA_FLAGS "${CMAKE_CUDA_FLAGS} --std=c++11") + set(CMAKE_CUDA_FLAGS "${CMAKE_CUDA_FLAGS} --std=c++14") endif(${CMAKE_VERSION} VERSION_LESS "3.17.0") if(LINE_INFO) @@ -180,6 +182,8 @@ set(RAFT_LINK_LIBRARIES # - build test executable ---------------------------------------------------- if(BUILD_RAFT_TESTS) + find_package(OpenMP REQUIRED) + # keep the files in alphabetical order! add_executable(test_raft test/cudart_utils.cpp @@ -190,10 +194,11 @@ if(BUILD_RAFT_TESTS) ${GTEST_DIR}/include) target_link_libraries(test_raft + ${RAFT_LINK_LIBRARIES} gtestlib gtest_mainlib - pthread - ${RAFT_LINK_LIBRARIES}) + OpenMP::OpenMP_CXX + Threads::Threads) endif(BUILD_RAFT_TESTS) ############################################################################## diff --git a/cpp/test/cudart_utils.cpp b/cpp/test/cudart_utils.cpp index 23be71cd70..5ed44288a3 100644 --- a/cpp/test/cudart_utils.cpp +++ b/cpp/test/cudart_utils.cpp @@ -24,7 +24,7 @@ TEST(Raft, Utils) { ASSERT_NO_THROW(ASSERT(1 == 1, "Should not assert!")); ASSERT_THROW(ASSERT(1 != 1, "Should assert!"), Exception); ASSERT_THROW(THROW("Should throw!"), Exception); - //ASSERT_NO_THROW(CUDA_CHECK(cudaFree(nullptr))); + ASSERT_NO_THROW(CUDA_CHECK(cudaFree(nullptr))); } } // namespace raft From b398243e19c8eb7dd82ea10d2fe21ea3547aa801 Mon Sep 17 00:00:00 2001 From: Thejaswi Rao Date: Mon, 4 May 2020 23:23:03 -0700 Subject: [PATCH 12/56] FIX updated allocator header include in device buffer --- cpp/include/raft/device_buffer.hpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/cpp/include/raft/device_buffer.hpp b/cpp/include/raft/device_buffer.hpp index 359c69ef4a..dbaa7a0321 100644 --- a/cpp/include/raft/device_buffer.hpp +++ b/cpp/include/raft/device_buffer.hpp @@ -17,8 +17,7 @@ #pragma once #include "buffer_base.hpp" -//TODO: update allocator header -#include +#include "allocator.hpp" namespace raft { From 64377cb0c3822979bc76d346e4bfaa5302a0946d Mon Sep 17 00:00:00 2001 From: Thejaswi Rao Date: Mon, 4 May 2020 23:23:18 -0700 Subject: [PATCH 13/56] TEST added unit-tests for device_buffer --- cpp/CMakeLists.txt | 1 + cpp/test/device_buffer.cpp | 44 ++++++++++++++++++++++++++++++++++++++ 2 files changed, 45 insertions(+) create mode 100644 cpp/test/device_buffer.cpp diff --git a/cpp/CMakeLists.txt b/cpp/CMakeLists.txt index 93e9760767..9a0f578b72 100644 --- a/cpp/CMakeLists.txt +++ b/cpp/CMakeLists.txt @@ -187,6 +187,7 @@ if(BUILD_RAFT_TESTS) # keep the files in alphabetical order! add_executable(test_raft test/cudart_utils.cpp + test/device_buffer.cpp test/test.cpp) target_include_directories(test_raft PRIVATE diff --git a/cpp/test/device_buffer.cpp b/cpp/test/device_buffer.cpp new file mode 100644 index 0000000000..0f717a5adb --- /dev/null +++ b/cpp/test/device_buffer.cpp @@ -0,0 +1,44 @@ +/* + * 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. + */ + +#include +#include +#include +#include + +namespace raft { + +TEST(Raft, DeviceBuffer) { + auto allocator = std::make_shared(); + cudaStream_t stream; + CUDA_CHECK(cudaStreamCreate(&stream)); + // no allocation at construction + device_buffer buff(allocator, stream); + ASSERT_EQ(0, buff.size()); + // explicit allocation after construction + buff.resize(20, stream); + ASSERT_EQ(20, buff.size()); + // resizing to a smaller buffer size + buff.resize(10, stream); + ASSERT_EQ(10, buff.size()); + // explicit deallocation + buff.release(stream); + ASSERT_EQ(0, buff.size()); + CUDA_CHECK(cudaStreamSynchronize(stream)); + CUDA_CHECK(cudaStreamDestroy(stream)); +} + +} // namespace raft From 2946799159dca7bce63c8bb831a7b07852a2657a Mon Sep 17 00:00:00 2001 From: Thejaswi Rao Date: Mon, 4 May 2020 23:26:10 -0700 Subject: [PATCH 14/56] FIX updated allocator header include in host buffer --- cpp/include/raft/host_buffer.hpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/cpp/include/raft/host_buffer.hpp b/cpp/include/raft/host_buffer.hpp index 2df3dbd71a..747de8fd12 100644 --- a/cpp/include/raft/host_buffer.hpp +++ b/cpp/include/raft/host_buffer.hpp @@ -17,8 +17,7 @@ #pragma once #include "buffer_base.hpp" -//TODO: update allocator header -#include +#include "allocator.hpp" namespace raft { From b058aca1e1782bca80dca541c6a080d2650b5bac Mon Sep 17 00:00:00 2001 From: Thejaswi Rao Date: Mon, 4 May 2020 23:27:17 -0700 Subject: [PATCH 15/56] TEST added unit-tests for host_buffer --- cpp/CMakeLists.txt | 1 + cpp/test/host_buffer.cpp | 44 ++++++++++++++++++++++++++++++++++++++++ 2 files changed, 45 insertions(+) create mode 100644 cpp/test/host_buffer.cpp diff --git a/cpp/CMakeLists.txt b/cpp/CMakeLists.txt index 9a0f578b72..bba9c371db 100644 --- a/cpp/CMakeLists.txt +++ b/cpp/CMakeLists.txt @@ -188,6 +188,7 @@ if(BUILD_RAFT_TESTS) add_executable(test_raft test/cudart_utils.cpp test/device_buffer.cpp + test/host_buffer.cpp test/test.cpp) target_include_directories(test_raft PRIVATE diff --git a/cpp/test/host_buffer.cpp b/cpp/test/host_buffer.cpp new file mode 100644 index 0000000000..bbfc6a4a03 --- /dev/null +++ b/cpp/test/host_buffer.cpp @@ -0,0 +1,44 @@ +/* + * 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. + */ + +#include +#include +#include +#include + +namespace raft { + +TEST(Raft, HostBuffer) { + auto allocator = std::make_shared(); + cudaStream_t stream; + CUDA_CHECK(cudaStreamCreate(&stream)); + // no allocation at construction + host_buffer buff(allocator, stream); + ASSERT_EQ(0, buff.size()); + // explicit allocation after construction + buff.resize(20, stream); + ASSERT_EQ(20, buff.size()); + // resizing to a smaller buffer size + buff.resize(10, stream); + ASSERT_EQ(10, buff.size()); + // explicit deallocation + buff.release(stream); + ASSERT_EQ(0, buff.size()); + CUDA_CHECK(cudaStreamSynchronize(stream)); + CUDA_CHECK(cudaStreamDestroy(stream)); +} + +} // namespace raft From 48390f6ae9de7dff991abee051612ff01e85ebc7 Mon Sep 17 00:00:00 2001 From: Thejaswi Rao Date: Mon, 4 May 2020 23:36:18 -0700 Subject: [PATCH 16/56] FIX handle/cublas/cusolver header compilation issues --- cpp/include/raft/handle.hpp | 3 ++- cpp/include/raft/linalg/cublas_wrappers.h | 2 +- cpp/include/raft/linalg/cusolver_wrappers.h | 4 ++-- 3 files changed, 5 insertions(+), 4 deletions(-) diff --git a/cpp/include/raft/handle.hpp b/cpp/include/raft/handle.hpp index 21b7a8327d..bc6b739447 100644 --- a/cpp/include/raft/handle.hpp +++ b/cpp/include/raft/handle.hpp @@ -20,6 +20,7 @@ #include #include #include +#include #include #include @@ -52,7 +53,7 @@ class handle_t { * * @param[in] n_streams number worker streams to be created */ - handle_t(int n_streams = NumDefaultWorkerStreams) : + handle_t(int n_streams = NumDefaultWorkerStreams) : _dev_id([]() -> int { int cur_dev = -1; CUDA_CHECK(cudaGetDevice(&cur_dev)); diff --git a/cpp/include/raft/linalg/cublas_wrappers.h b/cpp/include/raft/linalg/cublas_wrappers.h index e732d7d424..642448f89a 100644 --- a/cpp/include/raft/linalg/cublas_wrappers.h +++ b/cpp/include/raft/linalg/cublas_wrappers.h @@ -22,7 +22,7 @@ #include namespace raft { -namespace linal { +namespace linalg { #define _CUBLAS_ERR_TO_STR(err) \ case err: \ diff --git a/cpp/include/raft/linalg/cusolver_wrappers.h b/cpp/include/raft/linalg/cusolver_wrappers.h index 6d35c632f2..a9b27814d6 100644 --- a/cpp/include/raft/linalg/cusolver_wrappers.h +++ b/cpp/include/raft/linalg/cusolver_wrappers.h @@ -52,7 +52,7 @@ inline const char *cusolverErr2Str(cusolverStatus_t err) { cusolverStatus_t err = call; \ ASSERT(err == CUSOLVER_STATUS_SUCCESS, \ "CUSOLVER call='%s' got errorcode=%d err=%s", #call, err, \ - MLCommon::LinAlg::cusolverErr2Str(err)); \ + raft::linalg::cusolverErr2Str(err)); \ } while (0) ///@todo: enable this once logging is enabled @@ -62,7 +62,7 @@ inline const char *cusolverErr2Str(cusolverStatus_t err) { // cusolverStatus_t err = call; \ // if (err != CUSOLVER_STATUS_SUCCESS) { \ // CUML_LOG_ERROR("CUSOLVER call='%s' got errorcode=%d err=%s", #call, err, \ -// MLCommon::LinAlg::cusolverErr2Str(err)); \ +// raft::linalg::cusolverErr2Str(err)); \ // } \ // } while (0) From ed543ce0db2cdb078880ed1c11d9db00ecf49f4a Mon Sep 17 00:00:00 2001 From: Thejaswi Rao Date: Mon, 4 May 2020 23:38:56 -0700 Subject: [PATCH 17/56] TEST added unit-tests for handle_t --- cpp/CMakeLists.txt | 1 + cpp/test/handle.cpp | 46 +++++++++++++++++++++++++++++++++++++++++++++ 2 files changed, 47 insertions(+) create mode 100644 cpp/test/handle.cpp diff --git a/cpp/CMakeLists.txt b/cpp/CMakeLists.txt index bba9c371db..51b46f8aac 100644 --- a/cpp/CMakeLists.txt +++ b/cpp/CMakeLists.txt @@ -188,6 +188,7 @@ if(BUILD_RAFT_TESTS) add_executable(test_raft test/cudart_utils.cpp test/device_buffer.cpp + test/handle.cpp test/host_buffer.cpp test/test.cpp) diff --git a/cpp/test/handle.cpp b/cpp/test/handle.cpp new file mode 100644 index 0000000000..c9deb4d10d --- /dev/null +++ b/cpp/test/handle.cpp @@ -0,0 +1,46 @@ +/* + * 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. + */ + +#include +#include +#include +#include + +namespace raft { + +TEST(Raft, HandleDefault) { + handle_t h; + ASSERT_EQ(0, h.getNumInternalStreams()); + ASSERT_EQ(0, h.getDevice()); + ASSERT_EQ(nullptr, h.getStream()); + ASSERT_NE(nullptr, h.getCublasHandle()); + ASSERT_NE(nullptr, h.getcusolverDnHandle()); + ASSERT_NE(nullptr, h.getcusolverSpHandle()); + ASSERT_NE(nullptr, h.getcusparseHandle()); +} + +TEST(Raft, Handle) { + handle_t h(4); + ASSERT_EQ(4, h.getNumInternalStreams()); + cudaStream_t stream; + CUDA_CHECK(cudaStreamCreate(&stream)); + h.setStream(stream); + ASSERT_EQ(stream, h.getStream()); + CUDA_CHECK(cudaStreamSynchronize(stream)); + CUDA_CHECK(cudaStreamDestroy(stream)); +} + +} // namespace raft From db38255a2e1928bd2e062c84cee39a42ab23beb8 Mon Sep 17 00:00:00 2001 From: Thejaswi Rao Date: Tue, 5 May 2020 02:51:03 -0700 Subject: [PATCH 18/56] CI updated style.sh with the latest checks --- ci/checks/style.sh | 11 ++--------- 1 file changed, 2 insertions(+), 9 deletions(-) diff --git a/ci/checks/style.sh b/ci/checks/style.sh index 160c3a2c69..cc5bb08907 100644 --- a/ci/checks/style.sh +++ b/ci/checks/style.sh @@ -1,7 +1,7 @@ #!/bin/bash # Copyright (c) 2020, NVIDIA CORPORATION. ##################### -# cuML Style Tester # +# RAFT Style Tester # ##################### # Ignore errors and set path @@ -41,15 +41,9 @@ else fi # Check for a consistent #include syntax -# TODO: keep adding more dirs as and when we update the syntax HASH_INCLUDE=`python cpp/scripts/include_checker.py \ - cpp/bench \ - cpp/comms/mpi/include \ - cpp/comms/mpi/src \ - cpp/comms/std/include \ - cpp/comms/std/src \ cpp/include \ - cpp/examples \ + cpp/test \ 2>&1` HASH_RETVAL=$? if [ "$RETVAL" = "0" ]; then @@ -66,7 +60,6 @@ else fi # Check for a consistent code format -# TODO: keep adding more dirs when we add more source folders in cuml FORMAT=`python cpp/scripts/run-clang-format.py 2>&1` FORMAT_RETVAL=$? if [ "$RETVAL" = "0" ]; then From a07b3af02f2bd90f8103b9e74407f81e0b9acd67 Mon Sep 17 00:00:00 2001 From: Thejaswi Rao Date: Tue, 5 May 2020 02:55:36 -0700 Subject: [PATCH 19/56] FIX ignore commented lines in include_checker --- cpp/scripts/include_checker.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/cpp/scripts/include_checker.py b/cpp/scripts/include_checker.py index e8e752380e..1ced05e743 100644 --- a/cpp/scripts/include_checker.py +++ b/cpp/scripts/include_checker.py @@ -22,6 +22,7 @@ IncludeRegex = re.compile(r"\s*#include\s*(\S+)") +RemoveComments = re.compile(r"//.*") def parse_args(): @@ -52,6 +53,7 @@ def check_includes_in(src): errs = [] dir = os.path.dirname(src) for line_number, line in enumerate(open(src)): + line = RemoveComments.sub("", line) match = IncludeRegex.search(line) if match is None: continue From 6aecf504b08f8e1d0b1068d9d2c4fbf1720f200f Mon Sep 17 00:00:00 2001 From: Thejaswi Rao Date: Tue, 5 May 2020 02:56:47 -0700 Subject: [PATCH 20/56] ENH added .clang-format file --- cpp/.clang-format | 157 ++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 157 insertions(+) create mode 100644 cpp/.clang-format diff --git a/cpp/.clang-format b/cpp/.clang-format new file mode 100644 index 0000000000..2af7510304 --- /dev/null +++ b/cpp/.clang-format @@ -0,0 +1,157 @@ +--- +# Refer to the following link for the explanation of each params: +# http://releases.llvm.org/8.0.0/tools/clang/docs/ClangFormatStyleOptions.html +Language: Cpp +# BasedOnStyle: Google +AccessModifierOffset: -1 +AlignAfterOpenBracket: Align +AlignConsecutiveAssignments: false +AlignConsecutiveDeclarations: false +AlignEscapedNewlines: Left +AlignOperands: true +AlignTrailingComments: true +AllowAllParametersOfDeclarationOnNextLine: true +AllowShortBlocksOnASingleLine: false +AllowShortCaseLabelsOnASingleLine: false +AllowShortFunctionsOnASingleLine: All +AllowShortIfStatementsOnASingleLine: true +AllowShortLoopsOnASingleLine: true +# This is deprecated +AlwaysBreakAfterDefinitionReturnType: None +AlwaysBreakAfterReturnType: None +AlwaysBreakBeforeMultilineStrings: true +AlwaysBreakTemplateDeclarations: Yes +BinPackArguments: true +BinPackParameters: true +BraceWrapping: + AfterClass: false + AfterControlStatement: false + AfterEnum: false + AfterFunction: false + AfterNamespace: false + AfterObjCDeclaration: false + AfterStruct: false + AfterUnion: false + AfterExternBlock: false + BeforeCatch: false + BeforeElse: false + IndentBraces: false + # disabling the below splits, else, they'll just add to the vertical length of source files! + SplitEmptyFunction: false + SplitEmptyRecord: false + SplitEmptyNamespace: false +BreakBeforeBinaryOperators: None +BreakBeforeBraces: Attach +BreakBeforeInheritanceComma: false +BreakInheritanceList: BeforeColon +BreakBeforeTernaryOperators: true +BreakConstructorInitializersBeforeComma: false +BreakConstructorInitializers: BeforeColon +BreakAfterJavaFieldAnnotations: false +BreakStringLiterals: true +ColumnLimit: 80 +CommentPragmas: '^ IWYU pragma:' +CompactNamespaces: false +ConstructorInitializerAllOnOneLineOrOnePerLine: true +# Kept the below 2 to be the same as `IndentWidth` to keep everything uniform +ConstructorInitializerIndentWidth: 2 +ContinuationIndentWidth: 2 +Cpp11BracedListStyle: true +DerivePointerAlignment: true +DisableFormat: false +ExperimentalAutoDetectBinPacking: false +FixNamespaceComments: true +ForEachMacros: + - foreach + - Q_FOREACH + - BOOST_FOREACH +IncludeBlocks: Preserve +IncludeCategories: + - Regex: '^' + Priority: 2 + - Regex: '^<.*\.h>' + Priority: 1 + - Regex: '^<.*' + Priority: 2 + - Regex: '.*' + Priority: 3 +IncludeIsMainRegex: '([-_](test|unittest))?$' +IndentCaseLabels: true +IndentPPDirectives: None +IndentWidth: 2 +IndentWrappedFunctionNames: false +JavaScriptQuotes: Leave +JavaScriptWrapImports: true +KeepEmptyLinesAtTheStartOfBlocks: false +MacroBlockBegin: '' +MacroBlockEnd: '' +MaxEmptyLinesToKeep: 1 +NamespaceIndentation: None +ObjCBinPackProtocolList: Never +ObjCBlockIndentWidth: 2 +ObjCSpaceAfterProperty: false +ObjCSpaceBeforeProtocolList: true +PenaltyBreakAssignment: 2 +PenaltyBreakBeforeFirstCallParameter: 1 +PenaltyBreakComment: 300 +PenaltyBreakFirstLessLess: 120 +PenaltyBreakString: 1000 +PenaltyBreakTemplateDeclaration: 10 +PenaltyExcessCharacter: 1000000 +PenaltyReturnTypeOnItsOwnLine: 200 +PointerAlignment: Left +RawStringFormats: + - Language: Cpp + Delimiters: + - cc + - CC + - cpp + - Cpp + - CPP + - 'c++' + - 'C++' + CanonicalDelimiter: '' + - Language: TextProto + Delimiters: + - pb + - PB + - proto + - PROTO + EnclosingFunctions: + - EqualsProto + - EquivToProto + - PARSE_PARTIAL_TEXT_PROTO + - PARSE_TEST_PROTO + - PARSE_TEXT_PROTO + - ParseTextOrDie + - ParseTextProtoOrDie + CanonicalDelimiter: '' + BasedOnStyle: google +# Enabling comment reflow causes doxygen comments to be messed up in their formats! +ReflowComments: false +SortIncludes: true +SortUsingDeclarations: true +SpaceAfterCStyleCast: false +SpaceAfterTemplateKeyword: true +SpaceBeforeAssignmentOperators: true +SpaceBeforeCpp11BracedList: false +SpaceBeforeCtorInitializerColon: true +SpaceBeforeInheritanceColon: true +SpaceBeforeParens: ControlStatements +SpaceBeforeRangeBasedForLoopColon: true +SpaceInEmptyParentheses: false +SpacesBeforeTrailingComments: 2 +SpacesInAngles: false +SpacesInContainerLiterals: true +SpacesInCStyleCastParentheses: false +SpacesInParentheses: false +SpacesInSquareBrackets: false +# We are C++14, but clang-format puts this under `Cpp11` itself +Standard: Cpp11 +StatementMacros: + - Q_UNUSED + - QT_REQUIRE_VERSION +# Be consistent with indent-width, even for people who use tab for indentation! +TabWidth: 2 +UseTab: Never +... From 46949e8fe140ee9ab9d7938bac6bbeffb12a4423 Mon Sep 17 00:00:00 2001 From: Thejaswi Rao Date: Tue, 5 May 2020 09:57:11 +0000 Subject: [PATCH 21/56] FIX clang-format fixes --- cpp/include/raft.hpp | 10 ++++----- cpp/include/raft/allocator.hpp | 6 ++--- cpp/include/raft/cudart_utils.h | 6 ++--- cpp/include/raft/device_buffer.hpp | 2 +- cpp/include/raft/handle.hpp | 36 +++++++++++++++--------------- cpp/include/raft/host_buffer.hpp | 2 +- cpp/test/device_buffer.cpp | 2 +- cpp/test/handle.cpp | 2 +- cpp/test/host_buffer.cpp | 2 +- cpp/test/test.cpp | 8 +++---- 10 files changed, 35 insertions(+), 41 deletions(-) mode change 100755 => 100644 cpp/include/raft.hpp diff --git a/cpp/include/raft.hpp b/cpp/include/raft.hpp old mode 100755 new mode 100644 index 7c12cbf3c8..f380d276b2 --- a/cpp/include/raft.hpp +++ b/cpp/include/raft.hpp @@ -14,7 +14,6 @@ * limitations under the License. */ - #include namespace raft { @@ -22,10 +21,9 @@ namespace raft { /* Function for testing RAFT include * * @return message indicating RAFT has been included succesfully*/ -inline std::string test_raft() -{ - std::string status = "RAFT Setup succesfully"; - return status; +inline std::string test_raft() { + std::string status = "RAFT Setup succesfully"; + return status; } -} // namespace raft +} // namespace raft diff --git a/cpp/include/raft/allocator.hpp b/cpp/include/raft/allocator.hpp index 6751393013..89d9cf2915 100644 --- a/cpp/include/raft/allocator.hpp +++ b/cpp/include/raft/allocator.hpp @@ -69,8 +69,7 @@ class Allocator { * further to the ones listed in `Allocator`: * - Allocations may be always on the device that was specified on construction. */ -class deviceAllocator : public Allocator { -}; // class deviceAllocator +class deviceAllocator : public Allocator {}; // class deviceAllocator /** * @brief An explicit interface for an asynchronous host allocations. @@ -80,8 +79,7 @@ class deviceAllocator : public Allocator { * further to the ones listed in `Allocator`: * - Allocations don't need to be zero copy accessible form a device. */ -class hostAllocator : public Allocator { -}; // class hostAllocator +class hostAllocator : public Allocator {}; // class hostAllocator /** Default cudaMalloc/cudaFree based device allocator */ class defaultDeviceAllocator : public deviceAllocator { diff --git a/cpp/include/raft/cudart_utils.h b/cpp/include/raft/cudart_utils.h index fd23295b68..e41c3e57e6 100644 --- a/cpp/include/raft/cudart_utils.h +++ b/cpp/include/raft/cudart_utils.h @@ -18,12 +18,12 @@ #include #include +#include #include +#include #include #include #include -#include -#include ///@todo: enable once logging has been enabled in raft //#include "logger.hpp" @@ -90,7 +90,7 @@ class Exception : public std::exception { msg += errMsg; \ std::snprintf(errMsg, sizeof(errMsg), fmt, ##__VA_ARGS__); \ msg += errMsg; \ - throw raft::Exception(msg); \ + throw raft::Exception(msg); \ } while (0) /** macro to check for a conditional and assert on failure */ diff --git a/cpp/include/raft/device_buffer.hpp b/cpp/include/raft/device_buffer.hpp index dbaa7a0321..fac3740464 100644 --- a/cpp/include/raft/device_buffer.hpp +++ b/cpp/include/raft/device_buffer.hpp @@ -16,8 +16,8 @@ #pragma once -#include "buffer_base.hpp" #include "allocator.hpp" +#include "buffer_base.hpp" namespace raft { diff --git a/cpp/include/raft/handle.hpp b/cpp/include/raft/handle.hpp index bc6b739447..7c8898fd96 100644 --- a/cpp/include/raft/handle.hpp +++ b/cpp/include/raft/handle.hpp @@ -16,14 +16,14 @@ #pragma once +#include #include #include #include #include -#include -#include #include +#include #include #include #include @@ -31,11 +31,11 @@ ///@todo: enable once we have migrated cuml-comms layer too //#include -#include "allocator.hpp" -#include "cudart_utils.h" #include #include #include +#include "allocator.hpp" +#include "cudart_utils.h" namespace raft { @@ -55,19 +55,19 @@ class handle_t { */ handle_t(int n_streams = NumDefaultWorkerStreams) : _dev_id([]() -> int { - int cur_dev = -1; - CUDA_CHECK(cudaGetDevice(&cur_dev)); - return cur_dev; - }()), - _num_streams(n_streams), - _cublasInitialized(false), - _cusolverDnInitialized(false), - _cusolverSpInitialized(false), - _cusparseInitialized(false), - _deviceAllocator(std::make_shared()), - _hostAllocator(std::make_shared()), - _userStream(NULL), - _devicePropInitialized(false) { + int cur_dev = -1; + CUDA_CHECK(cudaGetDevice(&cur_dev)); + return cur_dev; + }()), + _num_streams(n_streams), + _cublasInitialized(false), + _cusolverDnInitialized(false), + _cusolverSpInitialized(false), + _cusparseInitialized(false), + _deviceAllocator(std::make_shared()), + _hostAllocator(std::make_shared()), + _userStream(NULL), + _devicePropInitialized(false) { createResources(); } @@ -241,4 +241,4 @@ class streamSyncer { const handle_t& _handle; }; // class streamSyncer -} // end namespace ML +} // namespace raft diff --git a/cpp/include/raft/host_buffer.hpp b/cpp/include/raft/host_buffer.hpp index 747de8fd12..75991072b1 100644 --- a/cpp/include/raft/host_buffer.hpp +++ b/cpp/include/raft/host_buffer.hpp @@ -16,8 +16,8 @@ #pragma once -#include "buffer_base.hpp" #include "allocator.hpp" +#include "buffer_base.hpp" namespace raft { diff --git a/cpp/test/device_buffer.cpp b/cpp/test/device_buffer.cpp index 0f717a5adb..00b39d6040 100644 --- a/cpp/test/device_buffer.cpp +++ b/cpp/test/device_buffer.cpp @@ -15,9 +15,9 @@ */ #include -#include #include #include +#include namespace raft { diff --git a/cpp/test/handle.cpp b/cpp/test/handle.cpp index c9deb4d10d..209caa9cc4 100644 --- a/cpp/test/handle.cpp +++ b/cpp/test/handle.cpp @@ -15,9 +15,9 @@ */ #include -#include #include #include +#include namespace raft { diff --git a/cpp/test/host_buffer.cpp b/cpp/test/host_buffer.cpp index bbfc6a4a03..dc3857dc39 100644 --- a/cpp/test/host_buffer.cpp +++ b/cpp/test/host_buffer.cpp @@ -15,9 +15,9 @@ */ #include -#include #include #include +#include namespace raft { diff --git a/cpp/test/test.cpp b/cpp/test/test.cpp index 2305bc9dc8..7477d7d0b5 100644 --- a/cpp/test/test.cpp +++ b/cpp/test/test.cpp @@ -14,14 +14,12 @@ * limitations under the License. */ -#include -#include #include +#include +#include namespace raft { -TEST(Raft, print) { - std::cout << test_raft() << std::endl; -} +TEST(Raft, print) { std::cout << test_raft() << std::endl; } } // namespace raft From f8b9b4caa8865feac82297292626b56f4e4ba4fe Mon Sep 17 00:00:00 2001 From: Thejaswi Rao Date: Tue, 5 May 2020 03:14:24 -0700 Subject: [PATCH 22/56] DOC update changelog --- CHANGELOG.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 640b4c90bb..7a9dfb4310 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -3,6 +3,7 @@ ## New Features - Initial RAFT version +- PR #3: defining raft::handle_t, device_buffer, host_buffer, allocator classes ## Improvements From bfa5218ddac4cbae43a53069997d3760ab93dd02 Mon Sep 17 00:00:00 2001 From: Thejaswi Rao Date: Wed, 6 May 2020 11:32:02 -0700 Subject: [PATCH 23/56] ENH added .clang-tidy. Courtesy: xgboost --- cpp/.clang-tidy | 168 ++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 168 insertions(+) create mode 100644 cpp/.clang-tidy diff --git a/cpp/.clang-tidy b/cpp/.clang-tidy new file mode 100644 index 0000000000..0831a55d35 --- /dev/null +++ b/cpp/.clang-tidy @@ -0,0 +1,168 @@ +--- +Checks: 'clang-diagnostic-*,clang-analyzer-*,modernize-*,-modernize-make-*,-modernize-raw-string-literal,google-*,-google-default-arguments,-clang-diagnostic-#pragma-messages,readability-identifier-naming,-*,modernize-*,-modernize-make-*,-modernize-raw-string-literal,google-*,-google-default-arguments,-clang-diagnostic-#pragma-messages,readability-identifier-naming' +WarningsAsErrors: '' +HeaderFilterRegex: '' +AnalyzeTemporaryDtors: false +FormatStyle: none +User: snanditale +CheckOptions: + - key: google-build-namespaces.HeaderFileExtensions + value: ',h,hh,hpp,hxx' + - key: google-global-names-in-headers.HeaderFileExtensions + value: ',h,hh,hpp,hxx' + - key: google-readability-braces-around-statements.ShortStatementLines + value: '1' + - key: google-readability-function-size.BranchThreshold + value: '4294967295' + - key: google-readability-function-size.LineThreshold + value: '4294967295' + - key: google-readability-function-size.NestingThreshold + value: '4294967295' + - key: google-readability-function-size.ParameterThreshold + value: '4294967295' + - key: google-readability-function-size.StatementThreshold + value: '800' + - key: google-readability-function-size.VariableThreshold + value: '4294967295' + - key: google-readability-namespace-comments.ShortNamespaceLines + value: '10' + - key: google-readability-namespace-comments.SpacesBeforeComments + value: '2' + - key: google-runtime-int.SignedTypePrefix + value: int + - key: google-runtime-int.TypeSuffix + value: '' + - key: google-runtime-int.UnsignedTypePrefix + value: uint + - key: google-runtime-references.WhiteListTypes + value: '' + - key: modernize-loop-convert.MaxCopySize + value: '16' + - key: modernize-loop-convert.MinConfidence + value: reasonable + - key: modernize-loop-convert.NamingStyle + value: CamelCase + - key: modernize-pass-by-value.IncludeStyle + value: llvm + - key: modernize-pass-by-value.ValuesOnly + value: '0' + - key: modernize-replace-auto-ptr.IncludeStyle + value: llvm + - key: modernize-replace-random-shuffle.IncludeStyle + value: llvm + - key: modernize-use-auto.MinTypeNameLength + value: '5' + - key: modernize-use-auto.RemoveStars + value: '0' + - key: modernize-use-default-member-init.IgnoreMacros + value: '1' + - key: modernize-use-default-member-init.UseAssignment + value: '0' + - key: modernize-use-emplace.ContainersWithPushBack + value: '::std::vector;::std::list;::std::deque' + - key: modernize-use-emplace.SmartPointers + value: '::std::shared_ptr;::std::unique_ptr;::std::auto_ptr;::std::weak_ptr' + - key: modernize-use-emplace.TupleMakeFunctions + value: '::std::make_pair;::std::make_tuple' + - key: modernize-use-emplace.TupleTypes + value: '::std::pair;::std::tuple' + - key: modernize-use-equals-default.IgnoreMacros + value: '1' + - key: modernize-use-noexcept.ReplacementString + value: '' + - key: modernize-use-noexcept.UseNoexceptFalse + value: '1' + - key: modernize-use-nullptr.NullMacros + value: 'NULL' + - key: modernize-use-transparent-functors.SafeMode + value: '0' + - key: modernize-use-using.IgnoreMacros + value: '1' + - key: readability-identifier-naming.ClassCase + value: CamelCase + - key: readability-identifier-naming.ClassPrefix + value: '' + - key: readability-identifier-naming.ClassSuffix + value: '' + - key: readability-identifier-naming.ConstexprVariableCase + value: CamelCase + - key: readability-identifier-naming.ConstexprVariablePrefix + value: k + - key: readability-identifier-naming.ConstexprVariableSuffix + value: '' + - key: readability-identifier-naming.EnumCase + value: CamelCase + - key: readability-identifier-naming.EnumConstantPrefix + value: k + - key: readability-identifier-naming.EnumConstantSuffix + value: '' + - key: readability-identifier-naming.EnumPrefix + value: '' + - key: readability-identifier-naming.EnumSuffix + value: '' + - key: readability-identifier-naming.FunctionCase + value: CamelCase + - key: readability-identifier-naming.FunctionPrefix + value: '' + - key: readability-identifier-naming.FunctionSuffix + value: '' + - key: readability-identifier-naming.GlobalConstantCase + value: CamelCase + - key: readability-identifier-naming.GlobalConstantPrefix + value: k + - key: readability-identifier-naming.GlobalConstantSuffix + value: '' + - key: readability-identifier-naming.IgnoreFailedSplit + value: '0' + - key: readability-identifier-naming.MemberCase + value: lower_case + - key: readability-identifier-naming.MemberPrefix + value: '' + - key: readability-identifier-naming.MemberSuffix + value: '' + - key: readability-identifier-naming.NamespaceCase + value: lower_case + - key: readability-identifier-naming.NamespacePrefix + value: '' + - key: readability-identifier-naming.NamespaceSuffix + value: '' + - key: readability-identifier-naming.PrivateMemberPrefix + value: '' + - key: readability-identifier-naming.PrivateMemberSuffix + value: _ + - key: readability-identifier-naming.ProtectedMemberPrefix + value: '' + - key: readability-identifier-naming.ProtectedMemberSuffix + value: _ + - key: readability-identifier-naming.StaticConstantCase + value: CamelCase + - key: readability-identifier-naming.StaticConstantPrefix + value: k + - key: readability-identifier-naming.StaticConstantSuffix + value: '' + - key: readability-identifier-naming.StructCase + value: CamelCase + - key: readability-identifier-naming.StructPrefix + value: '' + - key: readability-identifier-naming.StructSuffix + value: '' + - key: readability-identifier-naming.TypeAliasCase + value: CamelCase + - key: readability-identifier-naming.TypeAliasPrefix + value: '' + - key: readability-identifier-naming.TypeAliasSuffix + value: '' + - key: readability-identifier-naming.TypeTemplateParameterCase + value: CamelCase + - key: readability-identifier-naming.TypeTemplateParameterPrefix + value: '' + - key: readability-identifier-naming.TypeTemplateParameterSuffix + value: '' + - key: readability-identifier-naming.TypedefCase + value: CamelCase + - key: readability-identifier-naming.TypedefPrefix + value: '' + - key: readability-identifier-naming.TypedefSuffix + value: '' +... + From 30d47d7933fbb8f5b38fbc4b75a2d5343ba3d1c3 Mon Sep 17 00:00:00 2001 From: Thejaswi Rao Date: Wed, 6 May 2020 11:34:06 -0700 Subject: [PATCH 24/56] FEA added clang tidy checker script from cuml --- cpp/scripts/run-clang-tidy.py | 259 ++++++++++++++++++++++++++++++++++ 1 file changed, 259 insertions(+) create mode 100644 cpp/scripts/run-clang-tidy.py diff --git a/cpp/scripts/run-clang-tidy.py b/cpp/scripts/run-clang-tidy.py new file mode 100644 index 0000000000..836c825a5d --- /dev/null +++ b/cpp/scripts/run-clang-tidy.py @@ -0,0 +1,259 @@ +# 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 __future__ import print_function +import sys +import re +import os +import subprocess +import argparse +import json +import multiprocessing as mp + + +EXPECTED_VERSION = "8.0.1" +VERSION_REGEX = re.compile(r" LLVM version ([0-9.]+)") +GPU_ARCH_REGEX = re.compile(r"sm_(\d+)") +SPACES = re.compile(r"\s+") +SEPARATOR = "-" * 16 + + +def parse_args(): + argparser = argparse.ArgumentParser("Runs clang-tidy on a project") + argparser.add_argument("-cdb", type=str, default="compile_commands.json", + help="Path to cmake-generated compilation database") + argparser.add_argument("-exe", type=str, default="clang-tidy", + help="Path to clang-tidy exe") + argparser.add_argument("-ignore", type=str, default="[.]cu$|examples/kmeans/", + help="Regex used to ignore files from checking") + argparser.add_argument("-select", type=str, default=None, + help="Regex used to select files for checking") + argparser.add_argument("-j", type=int, default=-1, + help="Number of parallel jobs to launch.") + args = argparser.parse_args() + if args.j <= 0: + args.j = mp.cpu_count() + args.ignore_compiled = re.compile(args.ignore) if args.ignore else None + args.select_compiled = re.compile(args.select) if args.select else None + ret = subprocess.check_output("%s --version" % args.exe, shell=True) + ret = ret.decode("utf-8") + version = VERSION_REGEX.search(ret) + if version is None: + raise Exception("Failed to figure out clang-tidy version!") + version = version.group(1) + if version != EXPECTED_VERSION: + raise Exception("clang-tidy exe must be v%s found '%s'" % \ + (EXPECTED_VERSION, version)) + if not os.path.exists(args.cdb): + raise Exception("Compilation database '%s' missing" % args.cdb) + return args + + +def list_all_cmds(cdb): + with open(cdb, "r") as fp: + return json.load(fp) + + +def get_gpu_archs(command): + archs = [] + for loc in range(len(command)): + if command[loc] != "-gencode": + continue + arch_flag = command[loc + 1] + match = GPU_ARCH_REGEX.search(arch_flag) + if match is not None: + archs.append("--cuda-gpu-arch=sm_%s" % match.group(1)) + return archs + + +def get_index(arr, item): + try: + return arr.index(item) + except: + return -1 + + +def remove_item(arr, item): + loc = get_index(arr, item) + if loc >= 0: + del arr[loc] + return loc + + +def remove_item_plus_one(arr, item): + loc = get_index(arr, item) + if loc >= 0: + del arr[loc + 1] + del arr[loc] + return loc + + +def get_clang_includes(exe): + dir = os.getenv("CONDA_PREFIX") + if dir is None: + ret = subprocess.check_output("which %s 2>&1" % exe, shell=True) + ret = ret.decode("utf-8") + dir = os.path.dirname(os.path.dirname(ret)) + header = os.path.join(dir, "include", "ClangHeaders") + return ["-I", header] + + +def get_tidy_args(cmd, exe): + command, file = cmd["command"], cmd["file"] + is_cuda = file.endswith(".cu") + command = re.split(SPACES, command) + # compiler is always clang++! + command[0] = "clang++" + # remove compilation and output targets from the original command + remove_item_plus_one(command, "-c") + remove_item_plus_one(command, "-o") + if is_cuda: + # replace nvcc's "-gencode ..." with clang's "--cuda-gpu-arch ..." + archs = get_gpu_archs(command) + command.extend(archs) + while True: + loc = remove_item_plus_one(command, "-gencode") + if loc < 0: + break + # "-x cuda" is the right usage in clang + loc = get_index(command, "-x") + if loc >= 0: + command[loc + 1] = "cuda" + remove_item_plus_one(command, "-ccbin") + remove_item(command, "--expt-extended-lambda") + remove_item(command, "--diag_suppress=unrecognized_gcc_pragma") + command.extend(get_clang_includes(exe)) + return command, is_cuda + + +def run_clang_tidy_command(tidy_cmd): + cmd = " ".join(tidy_cmd) + result = subprocess.run(cmd, check=False, shell=True, + stdout=subprocess.PIPE, stderr=subprocess.STDOUT) + status = result.returncode == 0 + if status: + out = "" + else: + out = "CMD: " + cmd + out += result.stdout.decode("utf-8").rstrip() + return status, out + + +def run_clang_tidy(cmd, args): + command, is_cuda = get_tidy_args(cmd, args.exe) + tidy_cmd = [args.exe, "-header-filter=.*raft/cpp/.*", cmd["file"], "--", ] + tidy_cmd.extend(command) + status = True + out = "" + if is_cuda: + tidy_cmd.append("--cuda-device-only") + tidy_cmd.append(cmd["file"]) + ret, out1 = run_clang_tidy_command(tidy_cmd) + out += out1 + out += "%s" % SEPARATOR + if not ret: + status = ret + tidy_cmd[-2] = "--cuda-host-only" + ret, out1 = run_clang_tidy_command(tidy_cmd) + if not ret: + status = ret + out += out1 + else: + tidy_cmd.append(cmd["file"]) + ret, out1 = run_clang_tidy_command(tidy_cmd) + if not ret: + status = ret + out += out1 + return status, out, cmd["file"] + + +# yikes! global var :( +results = [] +def collect_result(result): + global results + results.append(result) + + +def print_result(passed, stdout, file): + status_str = "PASSED" if passed else "FAILED" + print("%s File:%s %s %s" % (SEPARATOR, file, status_str, SEPARATOR)) + if stdout: + print(stdout) + print("%s File:%s ENDS %s" % (SEPARATOR, file, SEPARATOR)) + + +def print_results(): + global results + status = True + for passed, stdout, file in results: + print_result(passed, stdout, file) + if not passed: + status = False + return status + + +# mostly used for debugging purposes +def run_sequential(args, all_files): + status = True + # actual tidy checker + for cmd in all_files: + # skip files that we don't want to look at + if args.ignore_compiled is not None and \ + re.search(args.ignore_compiled, cmd["file"]) is not None: + continue + if args.select_compiled is not None and \ + re.search(args.select_compiled, cmd["file"]) is None: + continue + passed, stdout, file = run_clang_tidy(cmd, args) + print_result(passed, stdout, file) + if not passed: + status = False + return status + + +def run_parallel(args, all_files): + pool = mp.Pool(args.j) + # actual tidy checker + for cmd in all_files: + # skip files that we don't want to look at + if args.ignore_compiled is not None and \ + re.search(args.ignore_compiled, cmd["file"]) is not None: + continue + if args.select_compiled is not None and \ + re.search(args.select_compiled, cmd["file"]) is None: + continue + pool.apply_async(run_clang_tidy, args=(cmd, args), + callback=collect_result) + pool.close() + pool.join() + return print_results() + + +def main(): + args = parse_args() + # Attempt to making sure that we run this script from root of repo always + if not os.path.exists(".git"): + raise Exception("This needs to always be run from the root of repo") + all_files = list_all_cmds(args.cdb) + if args.j == 1: + status = run_sequential(args, all_files) + else: + status = run_parallel(args, all_files) + if not status: + raise Exception("clang-tidy failed! Refer to the errors above.") + + +if __name__ == "__main__": + main() From 8e49e383442d03e293bd6e9d3d4c2911d4d212e2 Mon Sep 17 00:00:00 2001 From: Thejaswi Rao Date: Wed, 6 May 2020 11:35:17 -0700 Subject: [PATCH 25/56] ENH updated cmakelists to always dump compilation database for clang-tidy purposes --- cpp/CMakeLists.txt | 3 +++ 1 file changed, 3 insertions(+) diff --git a/cpp/CMakeLists.txt b/cpp/CMakeLists.txt index 51b46f8aac..f5696cf121 100644 --- a/cpp/CMakeLists.txt +++ b/cpp/CMakeLists.txt @@ -34,6 +34,9 @@ if(NOT CMAKE_BUILD_TYPE AND NOT CMAKE_CONFIGURATION_TYPES) "Debug" "Release") endif() +# this is needed for clang-tidy runs +set(CMAKE_EXPORT_COMPILE_COMMANDS ON) + ############################################################################## # - User Options ------------------------------------------------------------ From b094809306bf66abe1b144e63dbff3c16e5de960 Mon Sep 17 00:00:00 2001 From: Thejaswi Rao Date: Wed, 6 May 2020 11:46:20 -0700 Subject: [PATCH 26/56] BUG updated ignore arg to clang tidy script --- cpp/scripts/run-clang-tidy.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cpp/scripts/run-clang-tidy.py b/cpp/scripts/run-clang-tidy.py index 836c825a5d..23260d2f4d 100644 --- a/cpp/scripts/run-clang-tidy.py +++ b/cpp/scripts/run-clang-tidy.py @@ -36,7 +36,7 @@ def parse_args(): help="Path to cmake-generated compilation database") argparser.add_argument("-exe", type=str, default="clang-tidy", help="Path to clang-tidy exe") - argparser.add_argument("-ignore", type=str, default="[.]cu$|examples/kmeans/", + argparser.add_argument("-ignore", type=str, default="[.]cu$", help="Regex used to ignore files from checking") argparser.add_argument("-select", type=str, default=None, help="Regex used to select files for checking") From 32030357b01cc8ac4886ac30bd0bc1b96e559bc6 Mon Sep 17 00:00:00 2001 From: Thejaswi Rao Date: Wed, 6 May 2020 11:56:11 -0700 Subject: [PATCH 27/56] DOC added a note about RAII in the Allocator class doxygen comments --- cpp/include/raft/allocator.hpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/cpp/include/raft/allocator.hpp b/cpp/include/raft/allocator.hpp index 89d9cf2915..a1c981e149 100644 --- a/cpp/include/raft/allocator.hpp +++ b/cpp/include/raft/allocator.hpp @@ -26,6 +26,9 @@ namespace raft { * * An implementation of this interface can make the following assumptions: * - It does not need to be but it can allow async allocate and deallocate. + * + * @note This interface does NOT support RAII. Thus, if you need RAII-enabled + * interface, better to use `device_buffer` or `host_buffer`. */ class Allocator { public: From 124249773b6def29f94de646d2326ce62b08d0cf Mon Sep 17 00:00:00 2001 From: Thejaswi Rao Date: Wed, 6 May 2020 23:54:13 -0700 Subject: [PATCH 28/56] FIX keep the C-style string for THROW macro, as using std::array inside macros does not compile properly --- cpp/.clang-tidy | 4 ++-- cpp/include/raft/cudart_utils.h | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/cpp/.clang-tidy b/cpp/.clang-tidy index 0831a55d35..d299e9fe60 100644 --- a/cpp/.clang-tidy +++ b/cpp/.clang-tidy @@ -79,7 +79,7 @@ CheckOptions: - key: modernize-use-using.IgnoreMacros value: '1' - key: readability-identifier-naming.ClassCase - value: CamelCase + value: lower_case - key: readability-identifier-naming.ClassPrefix value: '' - key: readability-identifier-naming.ClassSuffix @@ -101,7 +101,7 @@ CheckOptions: - key: readability-identifier-naming.EnumSuffix value: '' - key: readability-identifier-naming.FunctionCase - value: CamelCase + value: lower_case - key: readability-identifier-naming.FunctionPrefix value: '' - key: readability-identifier-naming.FunctionSuffix diff --git a/cpp/include/raft/cudart_utils.h b/cpp/include/raft/cudart_utils.h index e41c3e57e6..8904d7c136 100644 --- a/cpp/include/raft/cudart_utils.h +++ b/cpp/include/raft/cudart_utils.h @@ -84,7 +84,7 @@ class Exception : public std::exception { #define THROW(fmt, ...) \ do { \ std::string msg; \ - char errMsg[2048]; \ + char errMsg[2048]; /* NOLINT */ \ std::snprintf(errMsg, sizeof(errMsg), \ "Exception occured! file=%s line=%d: ", __FILE__, __LINE__); \ msg += errMsg; \ From a2858851a1ad41e540a039462d9889bbdd304f33 Mon Sep 17 00:00:00 2001 From: Thejaswi Rao Date: Thu, 7 May 2020 00:05:56 -0700 Subject: [PATCH 29/56] FIX clang-tidy fixes for cudart_utils.h --- cpp/include/raft/cudart_utils.h | 75 +++++++++++++-------------------- 1 file changed, 29 insertions(+), 46 deletions(-) diff --git a/cpp/include/raft/cudart_utils.h b/cpp/include/raft/cudart_utils.h index 8904d7c136..8f7be839a6 100644 --- a/cpp/include/raft/cudart_utils.h +++ b/cpp/include/raft/cudart_utils.h @@ -30,44 +30,41 @@ namespace raft { /** base exception class for the whole of raft */ -class Exception : public std::exception { +class exception : public std::exception { public: /** default ctor */ - Exception() throw() : std::exception(), msg() {} + explicit exception() noexcept : std::exception(), msg_() {} /** copy ctor */ - Exception(const Exception& src) throw() : std::exception(), msg(src.what()) { - collectCallStack(); + exception(const exception& src) noexcept : std::exception(), msg_(src.what()) { + collect_call_stack(); } /** ctor from an input message */ - Exception(const std::string& _msg) throw() : std::exception(), msg(_msg) { - collectCallStack(); + explicit exception(const std::string& _msg) noexcept : std::exception(), msg_(_msg) { + collect_call_stack(); } - /** dtor */ - virtual ~Exception() throw() {} - /** get the message associated with this exception */ - virtual const char* what() const throw() { return msg.c_str(); } + const char* what() const noexcept override { return msg_.c_str(); } private: /** message associated with this exception */ - std::string msg; + std::string msg_; /** append call stack info to this exception's message for ease of debug */ // Courtesy: https://www.gnu.org/software/libc/manual/html_node/Backtraces.html - void collectCallStack() throw() { + void collect_call_stack() noexcept { #ifdef __GNUC__ - const int MaxStackDepth = 64; - void* stack[MaxStackDepth]; - auto depth = backtrace(stack, MaxStackDepth); + const int kMaxStackDepth = 64; + void* stack[kMaxStackDepth]; // NOLINT + auto depth = backtrace(stack, kMaxStackDepth); std::ostringstream oss; oss << std::endl << "Obtained " << depth << " stack frames" << std::endl; char** strings = backtrace_symbols(stack, depth); if (strings == nullptr) { oss << "But no stack trace could be found!" << std::endl; - msg += oss.str(); + msg_ += oss.str(); return; } ///@todo: support for demangling of C++ symbol names @@ -75,7 +72,7 @@ class Exception : public std::exception { oss << "#" << i << " in " << strings[i] << std::endl; } free(strings); - msg += oss.str(); + msg_ += oss.str(); #endif // __GNUC__ } }; @@ -86,11 +83,11 @@ class Exception : public std::exception { std::string msg; \ char errMsg[2048]; /* NOLINT */ \ std::snprintf(errMsg, sizeof(errMsg), \ - "Exception occured! file=%s line=%d: ", __FILE__, __LINE__); \ + "exception occured! file=%s line=%d: ", __FILE__, __LINE__); \ msg += errMsg; \ std::snprintf(errMsg, sizeof(errMsg), fmt, ##__VA_ARGS__); \ msg += errMsg; \ - throw raft::Exception(msg); \ + throw raft::exception(msg); \ } while (0) /** macro to check for a conditional and assert on failure */ @@ -122,7 +119,7 @@ class Exception : public std::exception { // } while (0) /** helper method to get max usable shared mem per block parameter */ -inline int getSharedMemPerBlock() { +inline int get_shared_memory_per_block() { int devId; CUDA_CHECK(cudaGetDevice(&devId)); int smemPerBlk; @@ -131,7 +128,7 @@ inline int getSharedMemPerBlock() { return smemPerBlk; } /** helper method to get multi-processor count parameter */ -inline int getMultiProcessorCount() { +inline int get_multi_processor_count() { int devId; CUDA_CHECK(cudaGetDevice(&devId)); int mpCount; @@ -162,32 +159,32 @@ void copy(Type* dst, const Type* src, size_t len, cudaStream_t stream) { */ /** performs a host to device copy */ template -void updateDevice(Type* dPtr, const Type* hPtr, size_t len, - cudaStream_t stream) { +void update_device(Type* dPtr, const Type* hPtr, size_t len, + cudaStream_t stream) { copy(dPtr, hPtr, len, stream); } /** performs a device to host copy */ template -void updateHost(Type* hPtr, const Type* dPtr, size_t len, cudaStream_t stream) { +void update_host(Type* hPtr, const Type* dPtr, size_t len, cudaStream_t stream) { copy(hPtr, dPtr, len, stream); } template -void copyAsync(Type* dPtr1, const Type* dPtr2, size_t len, - cudaStream_t stream) { +void copy_async(Type* dPtr1, const Type* dPtr2, size_t len, + cudaStream_t stream) { CUDA_CHECK(cudaMemcpyAsync(dPtr1, dPtr2, len * sizeof(Type), cudaMemcpyDeviceToDevice, stream)); } /** @} */ /** - * @defgroup Debug Utils for debugging device buffers + * @defgroup Debug Utils for debugging host/device buffers * @{ */ template -void printHostVector(const char* variableName, const T* hostMem, - size_t componentsCount, OutStream& out) { +void print_host_vector(const char* variableName, const T* hostMem, + size_t componentsCount, OutStream& out) { out << variableName << "=["; for (size_t i = 0; i < componentsCount; ++i) { if (i != 0) out << ","; @@ -196,29 +193,15 @@ void printHostVector(const char* variableName, const T* hostMem, out << "];\n"; } -template -void printHostVector(const char* variableName, const T* hostMem, - size_t componentsCount) { - printHostVector(variableName, hostMem, componentsCount, std::cout); - std::cout.flush(); -} - template -void printDevVector(const char* variableName, const T* devMem, - size_t componentsCount, OutStream& out) { +void print_device_vector(const char* variableName, const T* devMem, + size_t componentsCount, OutStream& out) { T* hostMem = new T[componentsCount]; CUDA_CHECK(cudaMemcpy(hostMem, devMem, componentsCount * sizeof(T), cudaMemcpyDeviceToHost)); - printHostVector(variableName, hostMem, componentsCount, out); + print_host_vector(variableName, hostMem, componentsCount, out); delete[] hostMem; } - -template -void printDevVector(const char* variableName, const T* devMem, - size_t componentsCount) { - printDevVector(variableName, devMem, componentsCount, std::cout); - std::cout.flush(); -} /** @} */ }; // namespace raft From 03fc6ede1023cf4d3f97105a9543a786d4823734 Mon Sep 17 00:00:00 2001 From: Thejaswi Rao Date: Thu, 7 May 2020 00:11:25 -0700 Subject: [PATCH 30/56] FIX use pass-by-value + std::move in ctor --- cpp/include/raft/cudart_utils.h | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/cpp/include/raft/cudart_utils.h b/cpp/include/raft/cudart_utils.h index 8f7be839a6..93224b9c54 100644 --- a/cpp/include/raft/cudart_utils.h +++ b/cpp/include/raft/cudart_utils.h @@ -24,6 +24,7 @@ #include #include #include +#include ///@todo: enable once logging has been enabled in raft //#include "logger.hpp" @@ -41,7 +42,7 @@ class exception : public std::exception { } /** ctor from an input message */ - explicit exception(const std::string& _msg) noexcept : std::exception(), msg_(_msg) { + explicit exception(const std::string _msg) noexcept : std::exception(), msg_(std::move(_msg)) { collect_call_stack(); } From 2884c2d5bdf5e169673867e42ea1372992a5afb9 Mon Sep 17 00:00:00 2001 From: Thejaswi Rao Date: Thu, 7 May 2020 00:11:38 -0700 Subject: [PATCH 31/56] FIX clang tidy fixes for allocator.hpp --- cpp/include/raft/allocator.hpp | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/cpp/include/raft/allocator.hpp b/cpp/include/raft/allocator.hpp index a1c981e149..565b9e60fb 100644 --- a/cpp/include/raft/allocator.hpp +++ b/cpp/include/raft/allocator.hpp @@ -30,7 +30,7 @@ namespace raft { * @note This interface does NOT support RAII. Thus, if you need RAII-enabled * interface, better to use `device_buffer` or `host_buffer`. */ -class Allocator { +class allocator { public: /** * @brief Asynchronously allocates a memory region. @@ -61,7 +61,7 @@ class Allocator { */ virtual void deallocate(void* p, std::size_t n, cudaStream_t stream) = 0; - virtual ~Allocator() {} + virtual ~allocator() = default; }; // class Allocator /** @@ -72,7 +72,7 @@ class Allocator { * further to the ones listed in `Allocator`: * - Allocations may be always on the device that was specified on construction. */ -class deviceAllocator : public Allocator {}; // class deviceAllocator +class device_allocator : public allocator {}; /** * @brief An explicit interface for an asynchronous host allocations. @@ -82,10 +82,10 @@ class deviceAllocator : public Allocator {}; // class deviceAllocator * further to the ones listed in `Allocator`: * - Allocations don't need to be zero copy accessible form a device. */ -class hostAllocator : public Allocator {}; // class hostAllocator +class host_allocator : public allocator {}; /** Default cudaMalloc/cudaFree based device allocator */ -class defaultDeviceAllocator : public deviceAllocator { +class default_device_allocator : public device_allocator { public: void* allocate(std::size_t n, cudaStream_t stream) override { void* ptr = 0; @@ -98,10 +98,10 @@ class defaultDeviceAllocator : public deviceAllocator { //CUDA_CHECK_NO_THROW(cudaFree(p)); CUDA_CHECK(cudaFree(p)); } -}; // class defaultDeviceAllocator +}; // class default_device_allocator /** Default cudaMallocHost/cudaFreeHost based host allocator */ -class defaultHostAllocator : public hostAllocator { +class default_host_allocator : public host_allocator { public: void* allocate(std::size_t n, cudaStream_t stream) override { void* ptr = 0; @@ -114,6 +114,6 @@ class defaultHostAllocator : public hostAllocator { //CUDA_CHECK_NO_THROW(cudaFreeHost(p)); CUDA_CHECK(cudaFreeHost(p)); } -}; // class defaultHostAllocator +}; // class default_host_allocator }; // end namespace raft From eab2c5e46be8e50fc286c140bda0037ed1a1a6f6 Mon Sep 17 00:00:00 2001 From: Thejaswi Rao Date: Thu, 7 May 2020 00:12:53 -0700 Subject: [PATCH 32/56] FIX compilation issue with cudart_utils.cpp --- cpp/test/cudart_utils.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/cpp/test/cudart_utils.cpp b/cpp/test/cudart_utils.cpp index 5ed44288a3..c14d880efd 100644 --- a/cpp/test/cudart_utils.cpp +++ b/cpp/test/cudart_utils.cpp @@ -22,8 +22,8 @@ namespace raft { TEST(Raft, Utils) { ASSERT_NO_THROW(ASSERT(1 == 1, "Should not assert!")); - ASSERT_THROW(ASSERT(1 != 1, "Should assert!"), Exception); - ASSERT_THROW(THROW("Should throw!"), Exception); + ASSERT_THROW(ASSERT(1 != 1, "Should assert!"), exception); + ASSERT_THROW(THROW("Should throw!"), exception); ASSERT_NO_THROW(CUDA_CHECK(cudaFree(nullptr))); } From 05c6b3e96191cf66b192d83f628b1fe54f3854ec Mon Sep 17 00:00:00 2001 From: Thejaswi Rao Date: Thu, 7 May 2020 00:27:00 -0700 Subject: [PATCH 33/56] FIX clang tidy fixes for allocator and buffer classes --- cpp/.clang-tidy | 6 +- cpp/include/raft/allocator.hpp | 4 +- cpp/include/raft/buffer_base.hpp | 89 +++++++++++++++--------------- cpp/include/raft/device_buffer.hpp | 2 +- cpp/include/raft/host_buffer.hpp | 26 ++++----- cpp/test/device_buffer.cpp | 2 +- cpp/test/host_buffer.cpp | 2 +- 7 files changed, 65 insertions(+), 66 deletions(-) diff --git a/cpp/.clang-tidy b/cpp/.clang-tidy index d299e9fe60..ee4cc6c24d 100644 --- a/cpp/.clang-tidy +++ b/cpp/.clang-tidy @@ -141,13 +141,13 @@ CheckOptions: - key: readability-identifier-naming.StaticConstantSuffix value: '' - key: readability-identifier-naming.StructCase - value: CamelCase + value: lower_case - key: readability-identifier-naming.StructPrefix value: '' - key: readability-identifier-naming.StructSuffix value: '' - key: readability-identifier-naming.TypeAliasCase - value: CamelCase + value: lower_case - key: readability-identifier-naming.TypeAliasPrefix value: '' - key: readability-identifier-naming.TypeAliasSuffix @@ -159,7 +159,7 @@ CheckOptions: - key: readability-identifier-naming.TypeTemplateParameterSuffix value: '' - key: readability-identifier-naming.TypedefCase - value: CamelCase + value: lower_case - key: readability-identifier-naming.TypedefPrefix value: '' - key: readability-identifier-naming.TypedefSuffix diff --git a/cpp/include/raft/allocator.hpp b/cpp/include/raft/allocator.hpp index 565b9e60fb..56d15bd6cb 100644 --- a/cpp/include/raft/allocator.hpp +++ b/cpp/include/raft/allocator.hpp @@ -88,7 +88,7 @@ class host_allocator : public allocator {}; class default_device_allocator : public device_allocator { public: void* allocate(std::size_t n, cudaStream_t stream) override { - void* ptr = 0; + void* ptr = nullptr; CUDA_CHECK(cudaMalloc(&ptr, n)); return ptr; } @@ -104,7 +104,7 @@ class default_device_allocator : public device_allocator { class default_host_allocator : public host_allocator { public: void* allocate(std::size_t n, cudaStream_t stream) override { - void* ptr = 0; + void* ptr = nullptr; CUDA_CHECK(cudaMallocHost(&ptr, n)); return ptr; } diff --git a/cpp/include/raft/buffer_base.hpp b/cpp/include/raft/buffer_base.hpp index 4af4f86ccd..c287b4b4d0 100644 --- a/cpp/include/raft/buffer_base.hpp +++ b/cpp/include/raft/buffer_base.hpp @@ -18,6 +18,7 @@ #include #include +#include #include "cudart_utils.h" namespace raft { @@ -52,39 +53,39 @@ class buffer_base { */ buffer_base(std::shared_ptr allocator, cudaStream_t stream, size_type n = 0) - : _size(n), - _capacity(n), - _data(nullptr), - _stream(stream), - _allocator(allocator) { - if (_capacity > 0) { - _data = static_cast( - _allocator->allocate(_capacity * sizeof(value_type), _stream)); - CUDA_CHECK(cudaStreamSynchronize(_stream)); + : size_(n), + capacity_(n), + data_(nullptr), + stream_(stream), + allocator_(std::move(allocator)) { + if (capacity_ > 0) { + data_ = static_cast( + allocator_->allocate(capacity_ * sizeof(value_type), stream_)); + CUDA_CHECK(cudaStreamSynchronize(stream_)); } } ~buffer_base() { - if (nullptr != _data) { - _allocator->deallocate(_data, _capacity * sizeof(value_type), _stream); + if (nullptr != data_) { + allocator_->deallocate(data_, capacity_ * sizeof(value_type), stream_); } } - value_type* data() { return _data; } + value_type* data() { return data_; } - const value_type* data() const { return _data; } + const value_type* data() const { return data_; } - size_type size() const { return _size; } + size_type size() const { return size_; } - void clear() { _size = 0; } + void clear() { size_ = 0; } - iterator begin() { return _data; } + iterator begin() { return data_; } - const_iterator begin() const { return _data; } + const_iterator begin() const { return data_; } - iterator end() { return _data + _size; } + iterator end() { return data_ + size_; } - const_iterator end() const { return _data + _size; } + const_iterator end() const { return data_ + size_; } /** * @brief Reserve new memory size for this buffer. @@ -98,18 +99,18 @@ class buffer_base { */ void reserve(const size_type new_capacity, cudaStream_t stream) { set_stream(stream); - if (new_capacity > _capacity) { - value_type* new_data = static_cast( - _allocator->allocate(new_capacity * sizeof(value_type), _stream)); - if (_size > 0) { - CUDA_CHECK(cudaMemcpyAsync(new_data, _data, _size * sizeof(value_type), - cudaMemcpyDefault, _stream)); + if (new_capacity > capacity_) { + auto* new_data = static_cast( + allocator_->allocate(new_capacity * sizeof(value_type), stream_)); + if (size_ > 0) { + CUDA_CHECK(cudaMemcpyAsync(new_data, data_, size_ * sizeof(value_type), + cudaMemcpyDefault, stream_)); } - if (nullptr != _data) { - _allocator->deallocate(_data, _capacity * sizeof(value_type), _stream); + if (nullptr != data_) { + allocator_->deallocate(data_, capacity_ * sizeof(value_type), stream_); } - _data = new_data; - _capacity = new_capacity; + data_ = new_data; + capacity_ = new_capacity; } } @@ -121,7 +122,7 @@ class buffer_base { */ void resize(const size_type new_size, cudaStream_t stream) { reserve(new_size, stream); - _size = new_size; + size_ = new_size; } /** @@ -133,12 +134,12 @@ class buffer_base { */ void release(cudaStream_t stream) { set_stream(stream); - if (nullptr != _data) { - _allocator->deallocate(_data, _capacity * sizeof(value_type), _stream); + if (nullptr != data_) { + allocator_->deallocate(data_, capacity_ * sizeof(value_type), stream_); } - _data = nullptr; - _capacity = 0; - _size = 0; + data_ = nullptr; + capacity_ = 0; + size_ = 0; } /** @@ -146,16 +147,16 @@ class buffer_base { * * @return the allocator pointer */ - std::shared_ptr get_allocator() const { return _allocator; } + std::shared_ptr get_allocator() const { return allocator_; } protected: - value_type* _data; + value_type* data_; private: - size_type _size; - size_type _capacity; - cudaStream_t _stream; - std::shared_ptr _allocator; + size_type size_; + size_type capacity_; + cudaStream_t stream_; + std::shared_ptr allocator_; /** * @brief Sets a new cuda stream where the future operations will be queued @@ -169,12 +170,12 @@ class buffer_base { * current one, then this method will be a no-op. */ void set_stream(cudaStream_t stream) { - if (_stream != stream) { + if (stream_ != stream) { cudaEvent_t event; CUDA_CHECK(cudaEventCreateWithFlags(&event, cudaEventDisableTiming)); - CUDA_CHECK(cudaEventRecord(event, _stream)); + CUDA_CHECK(cudaEventRecord(event, stream_)); CUDA_CHECK(cudaStreamWaitEvent(stream, event, 0)); - _stream = stream; + stream_ = stream; CUDA_CHECK(cudaEventDestroy(event)); } } diff --git a/cpp/include/raft/device_buffer.hpp b/cpp/include/raft/device_buffer.hpp index fac3740464..799518b25c 100644 --- a/cpp/include/raft/device_buffer.hpp +++ b/cpp/include/raft/device_buffer.hpp @@ -40,6 +40,6 @@ namespace raft { * @endcode */ template -using device_buffer = buffer_base; +using device_buffer = buffer_base; } // namespace raft diff --git a/cpp/include/raft/host_buffer.hpp b/cpp/include/raft/host_buffer.hpp index 75991072b1..48fdbc576b 100644 --- a/cpp/include/raft/host_buffer.hpp +++ b/cpp/include/raft/host_buffer.hpp @@ -41,15 +41,15 @@ namespace raft { * @endcode */ template -class host_buffer : public buffer_base { +class host_buffer : public buffer_base { public: - using size_type = typename buffer_base::size_type; - using value_type = typename buffer_base::value_type; - using iterator = typename buffer_base::iterator; - using const_iterator = typename buffer_base::const_iterator; - using reference = typename buffer_base::reference; + using size_type = typename buffer_base::size_type; + using value_type = typename buffer_base::value_type; + using iterator = typename buffer_base::iterator; + using const_iterator = typename buffer_base::const_iterator; + using reference = typename buffer_base::reference; using const_reference = - typename buffer_base::const_reference; + typename buffer_base::const_reference; host_buffer() = delete; @@ -57,18 +57,16 @@ class host_buffer : public buffer_base { host_buffer& operator=(const host_buffer& other) = delete; - host_buffer(std::shared_ptr allocator, cudaStream_t stream, + host_buffer(std::shared_ptr allocator, cudaStream_t stream, size_type n = 0) - : buffer_base(allocator, stream, n) {} + : buffer_base(allocator, stream, n) {} - ~host_buffer() {} + reference operator[](size_type pos) { return data_[pos]; } - reference operator[](size_type pos) { return _data[pos]; } - - const_reference operator[](size_type pos) const { return _data[pos]; } + const_reference operator[](size_type pos) const { return data_[pos]; } private: - using buffer_base::_data; + using buffer_base::data_; }; } // namespace raft diff --git a/cpp/test/device_buffer.cpp b/cpp/test/device_buffer.cpp index 00b39d6040..0600e26385 100644 --- a/cpp/test/device_buffer.cpp +++ b/cpp/test/device_buffer.cpp @@ -22,7 +22,7 @@ namespace raft { TEST(Raft, DeviceBuffer) { - auto allocator = std::make_shared(); + auto allocator = std::make_shared(); cudaStream_t stream; CUDA_CHECK(cudaStreamCreate(&stream)); // no allocation at construction diff --git a/cpp/test/host_buffer.cpp b/cpp/test/host_buffer.cpp index dc3857dc39..8c398e5145 100644 --- a/cpp/test/host_buffer.cpp +++ b/cpp/test/host_buffer.cpp @@ -22,7 +22,7 @@ namespace raft { TEST(Raft, HostBuffer) { - auto allocator = std::make_shared(); + auto allocator = std::make_shared(); cudaStream_t stream; CUDA_CHECK(cudaStreamCreate(&stream)); // no allocation at construction From fa1af0d1f985efcfcade0be938e1284f8f2c0828 Mon Sep 17 00:00:00 2001 From: Thejaswi Rao Date: Thu, 7 May 2020 00:36:33 -0700 Subject: [PATCH 34/56] FIX clang tidy fixes for cublas wrappers --- cpp/include/raft/linalg/cublas_wrappers.h | 131 +++++++++++----------- 1 file changed, 66 insertions(+), 65 deletions(-) diff --git a/cpp/include/raft/linalg/cublas_wrappers.h b/cpp/include/raft/linalg/cublas_wrappers.h index 642448f89a..83021edb80 100644 --- a/cpp/include/raft/linalg/cublas_wrappers.h +++ b/cpp/include/raft/linalg/cublas_wrappers.h @@ -27,7 +27,7 @@ namespace linalg { #define _CUBLAS_ERR_TO_STR(err) \ case err: \ return #err -inline const char *cublasErr2Str(cublasStatus_t err) { +inline const char *cublas_error_to_string(cublasStatus_t err) { switch (err) { _CUBLAS_ERR_TO_STR(CUBLAS_STATUS_SUCCESS); _CUBLAS_ERR_TO_STR(CUBLAS_STATUS_NOT_INITIALIZED); @@ -51,7 +51,7 @@ inline const char *cublasErr2Str(cublasStatus_t err) { cublasStatus_t err = call; \ ASSERT(err == CUBLAS_STATUS_SUCCESS, \ "CUBLAS call='%s' got errorcode=%d err=%s", #call, err, \ - raft::linalg::cublasErr2Str(err)); \ + raft::linalg::cublas_error_to_string(err)); \ } while (0) ///@todo: enable this once we have logging enabled @@ -61,7 +61,7 @@ inline const char *cublasErr2Str(cublasStatus_t err) { // cublasStatus_t err = call; \ // if (err != CUBLAS_STATUS_SUCCESS) { \ // CUML_LOG_ERROR("CUBLAS call='%s' got errorcode=%d err=%s", #call, err, \ -// raft::linalg::cublasErr2Str(err)); \ +// raft::linalg::cublas_error_to_string(err)); \ // } \ // } while (0) @@ -192,19 +192,23 @@ inline cublasStatus_t cublasgemm(cublasHandle_t handle, * @{ */ template -cublasStatus_t cublasgemmBatched(cublasHandle_t handle, +cublasStatus_t cublasgemmBatched(cublasHandle_t handle, // NOLINT cublasOperation_t transa, cublasOperation_t transb, int m, int n, int k, - const T *alpha, const T *const Aarray[], - int lda, const T *const Barray[], int ldb, - const T *beta, T *Carray[], int ldc, - int batchCount, cudaStream_t stream); + const T *alpha, + const T *const Aarray[], // NOLINT + int lda, const T *const Barray[], // NOLINT + int ldb, const T *beta, + T *Carray[], // NOLINT + int ldc, int batchCount, cudaStream_t stream); template <> -inline cublasStatus_t cublasgemmBatched( +inline cublasStatus_t cublasgemmBatched( // NOLINT cublasHandle_t handle, cublasOperation_t transa, cublasOperation_t transb, - int m, int n, int k, const float *alpha, const float *const Aarray[], int lda, - const float *const Barray[], int ldb, const float *beta, float *Carray[], + int m, int n, int k, const float *alpha, + const float *const Aarray[], // NOLINT + int lda, const float *const Barray[], // NOLINT + int ldb, const float *beta, float *Carray[], // NOLINT int ldc, int batchCount, cudaStream_t stream) { CUBLAS_CHECK(cublasSetStream(handle, stream)); return cublasSgemmBatched(handle, transa, transb, m, n, k, alpha, Aarray, lda, @@ -212,11 +216,13 @@ inline cublasStatus_t cublasgemmBatched( } template <> -inline cublasStatus_t cublasgemmBatched( +inline cublasStatus_t cublasgemmBatched( // NOLINT cublasHandle_t handle, cublasOperation_t transa, cublasOperation_t transb, - int m, int n, int k, const double *alpha, const double *const Aarray[], - int lda, const double *const Barray[], int ldb, const double *beta, - double *Carray[], int ldc, int batchCount, cudaStream_t stream) { + int m, int n, int k, const double *alpha, + const double *const Aarray[], // NOLINT + int lda, const double *const Barray[], // NOLINT + int ldb, const double *beta, double *Carray[], // NOLINT + int ldc, int batchCount, cudaStream_t stream) { CUBLAS_CHECK(cublasSetStream(handle, stream)); return cublasDgemmBatched(handle, transa, transb, m, n, k, alpha, Aarray, lda, Barray, ldb, beta, Carray, ldc, batchCount); @@ -228,20 +234,20 @@ inline cublasStatus_t cublasgemmBatched( * @{ */ template -cublasStatus_t cublasgemmStridedBatched( +cublasStatus_t cublasgemmStridedBatched( // NOLINT cublasHandle_t handle, cublasOperation_t transa, cublasOperation_t transb, int m, int n, int k, const T *alpha, const T *const Aarray, int lda, - long long int strideA, const T *const Barray, int ldb, long long int strideB, - const T *beta, T *Carray, int ldc, long long int strideC, int batchCount, + int64_t strideA, const T *const Barray, int ldb, int64_t strideB, + const T *beta, T *Carray, int ldc, int64_t strideC, int batchCount, cudaStream_t stream); template <> -inline cublasStatus_t cublasgemmStridedBatched( +inline cublasStatus_t cublasgemmStridedBatched( // NOLINT cublasHandle_t handle, cublasOperation_t transa, cublasOperation_t transb, int m, int n, int k, const float *alpha, const float *const Aarray, int lda, - long long int strideA, const float *const Barray, int ldb, - long long int strideB, const float *beta, float *Carray, int ldc, - long long int strideC, int batchCount, cudaStream_t stream) { + int64_t strideA, const float *const Barray, int ldb, + int64_t strideB, const float *beta, float *Carray, int ldc, + int64_t strideC, int batchCount, cudaStream_t stream) { CUBLAS_CHECK(cublasSetStream(handle, stream)); return cublasSgemmStridedBatched(handle, transa, transb, m, n, k, alpha, Aarray, lda, strideA, Barray, ldb, strideB, @@ -249,12 +255,12 @@ inline cublasStatus_t cublasgemmStridedBatched( } template <> -inline cublasStatus_t cublasgemmStridedBatched( +inline cublasStatus_t cublasgemmStridedBatched( // NOLINT cublasHandle_t handle, cublasOperation_t transa, cublasOperation_t transb, int m, int n, int k, const double *alpha, const double *const Aarray, int lda, - long long int strideA, const double *const Barray, int ldb, - long long int strideB, const double *beta, double *Carray, int ldc, - long long int strideC, int batchCount, cudaStream_t stream) { + int64_t strideA, const double *const Barray, int ldb, + int64_t strideB, const double *beta, double *Carray, int ldc, + int64_t strideC, int batchCount, cudaStream_t stream) { CUBLAS_CHECK(cublasSetStream(handle, stream)); return cublasDgemmStridedBatched(handle, transa, transb, m, n, k, alpha, Aarray, lda, strideA, Barray, ldb, strideB, @@ -268,55 +274,50 @@ inline cublasStatus_t cublasgemmStridedBatched( */ template -cublasStatus_t cublasgetrfBatched(cublasHandle_t handle, int n, - T *const A[], /*Device pointer*/ - int lda, int *P, /*Device Pointer*/ - int *info, /*Device Pointer*/ +cublasStatus_t cublasgetrfBatched(cublasHandle_t handle, int n, // NOLINT + T *const A[], // NOLINT + int lda, int *P, int *info, int batchSize, cudaStream_t stream); template <> -inline cublasStatus_t cublasgetrfBatched(cublasHandle_t handle, int n, - float *const A[], /*Device pointer*/ - int lda, int *P, /*Device Pointer*/ - int *info, /*Device Pointer*/ +inline cublasStatus_t cublasgetrfBatched(cublasHandle_t handle, // NOLINT + int n, float *const A[], // NOLINT + int lda, int *P, int *info, int batchSize, cudaStream_t stream) { CUBLAS_CHECK(cublasSetStream(handle, stream)); return cublasSgetrfBatched(handle, n, A, lda, P, info, batchSize); } template <> -inline cublasStatus_t cublasgetrfBatched(cublasHandle_t handle, int n, - double *const A[], /*Device pointer*/ - int lda, int *P, /*Device Pointer*/ - int *info, /*Device Pointer*/ +inline cublasStatus_t cublasgetrfBatched(cublasHandle_t handle, // NOLINT + int n, double *const A[], // NOLINT + int lda, int *P, int *info, int batchSize, cudaStream_t stream) { CUBLAS_CHECK(cublasSetStream(handle, stream)); return cublasDgetrfBatched(handle, n, A, lda, P, info, batchSize); } template -cublasStatus_t cublasgetriBatched(cublasHandle_t handle, int n, - const T *const A[], /*Device pointer*/ - int lda, const int *P, /*Device pointer*/ - T *const C[], /*Device pointer*/ +cublasStatus_t cublasgetriBatched(cublasHandle_t handle, int n, // NOLINT + const T *const A[], // NOLINT + int lda, const int *P, + T *const C[], // NOLINT int ldc, int *info, int batchSize, cudaStream_t stream); template <> -inline cublasStatus_t cublasgetriBatched( - cublasHandle_t handle, int n, const float *const A[], /*Device pointer*/ - int lda, const int *P, /*Device pointer*/ - float *const C[], /*Device pointer*/ +inline cublasStatus_t cublasgetriBatched( // NOLINT + cublasHandle_t handle, int n, const float *const A[], // NOLINT + int lda, const int *P, float *const C[], // NOLINT int ldc, int *info, int batchSize, cudaStream_t stream) { CUBLAS_CHECK(cublasSetStream(handle, stream)); return cublasSgetriBatched(handle, n, A, lda, P, C, ldc, info, batchSize); } template <> -inline cublasStatus_t cublasgetriBatched( - cublasHandle_t handle, int n, const double *const A[], /*Device pointer*/ - int lda, const int *P, /*Device pointer*/ - double *const C[], /*Device pointer*/ +inline cublasStatus_t cublasgetriBatched( // NOLINT + cublasHandle_t handle, int n, const double *const A[], // NOLINT + int lda, const int *P, double *const C[], // NOLINT int ldc, int *info, int batchSize, cudaStream_t stream) { CUBLAS_CHECK(cublasSetStream(handle, stream)); return cublasDgetriBatched(handle, n, A, lda, P, C, ldc, info, batchSize); @@ -330,32 +331,32 @@ inline cublasStatus_t cublasgetriBatched( */ template -inline cublasStatus_t cublasgelsBatched(cublasHandle_t handle, +inline cublasStatus_t cublasgelsBatched(cublasHandle_t handle, // NOLINT cublasOperation_t trans, int m, int n, - int nrhs, T *Aarray[], int lda, - T *Carray[], int ldc, int *info, - int *devInfoArray, int batchSize, - cudaStream_t stream = 0); + int nrhs, T *Aarray[], // NOLINT + int lda, T *Carray[], // NOLINT + int ldc, int *info, int *devInfoArray, + int batchSize, cudaStream_t stream); template <> -inline cublasStatus_t cublasgelsBatched(cublasHandle_t handle, +inline cublasStatus_t cublasgelsBatched(cublasHandle_t handle, // NOLINT cublasOperation_t trans, int m, int n, - int nrhs, float *Aarray[], int lda, - float *Carray[], int ldc, int *info, - int *devInfoArray, int batchSize, - cudaStream_t stream) { + int nrhs, float *Aarray[], // NOLINT + int lda, float *Carray[], // NOLINT + int ldc, int *info, int *devInfoArray, + int batchSize, cudaStream_t stream) { CUBLAS_CHECK(cublasSetStream(handle, stream)); return cublasSgelsBatched(handle, trans, m, n, nrhs, Aarray, lda, Carray, ldc, info, devInfoArray, batchSize); } template <> -inline cublasStatus_t cublasgelsBatched(cublasHandle_t handle, +inline cublasStatus_t cublasgelsBatched(cublasHandle_t handle, // NOLINT cublasOperation_t trans, int m, int n, - int nrhs, double *Aarray[], int lda, - double *Carray[], int ldc, int *info, - int *devInfoArray, int batchSize, - cudaStream_t stream) { + int nrhs, double *Aarray[], // NOLINT + int lda, double *Carray[], // NOLINT + int ldc, int *info, int *devInfoArray, + int batchSize, cudaStream_t stream) { CUBLAS_CHECK(cublasSetStream(handle, stream)); return cublasDgelsBatched(handle, trans, m, n, nrhs, Aarray, lda, Carray, ldc, info, devInfoArray, batchSize); From 1acc966850e5768232ebe99877f31eebd07b4ce0 Mon Sep 17 00:00:00 2001 From: Thejaswi Rao Date: Thu, 7 May 2020 00:46:52 -0700 Subject: [PATCH 35/56] FIX clang tidy fixes for cusolver wrappers --- cpp/include/raft/linalg/cublas_wrappers.h | 1 + cpp/include/raft/linalg/cusolver_wrappers.h | 256 ++++++++++---------- 2 files changed, 126 insertions(+), 131 deletions(-) diff --git a/cpp/include/raft/linalg/cublas_wrappers.h b/cpp/include/raft/linalg/cublas_wrappers.h index 83021edb80..3594069921 100644 --- a/cpp/include/raft/linalg/cublas_wrappers.h +++ b/cpp/include/raft/linalg/cublas_wrappers.h @@ -20,6 +20,7 @@ ///@todo: enable this once we have logger enabled //#include #include +#include namespace raft { namespace linalg { diff --git a/cpp/include/raft/linalg/cusolver_wrappers.h b/cpp/include/raft/linalg/cusolver_wrappers.h index a9b27814d6..850ea9654d 100644 --- a/cpp/include/raft/linalg/cusolver_wrappers.h +++ b/cpp/include/raft/linalg/cusolver_wrappers.h @@ -28,7 +28,7 @@ namespace linalg { #define _CUSOLVER_ERR_TO_STR(err) \ case err: \ return #err; -inline const char *cusolverErr2Str(cusolverStatus_t err) { +inline const char *cusolver_error_to_string(cusolverStatus_t err) { switch (err) { _CUSOLVER_ERR_TO_STR(CUSOLVER_STATUS_SUCCESS); _CUSOLVER_ERR_TO_STR(CUSOLVER_STATUS_NOT_INITIALIZED); @@ -52,7 +52,7 @@ inline const char *cusolverErr2Str(cusolverStatus_t err) { cusolverStatus_t err = call; \ ASSERT(err == CUSOLVER_STATUS_SUCCESS, \ "CUSOLVER call='%s' got errorcode=%d err=%s", #call, err, \ - raft::linalg::cusolverErr2Str(err)); \ + raft::linalg::cusolver_error_to_string(err)); \ } while (0) ///@todo: enable this once logging is enabled @@ -62,7 +62,7 @@ inline const char *cusolverErr2Str(cusolverStatus_t err) { // cusolverStatus_t err = call; \ // if (err != CUSOLVER_STATUS_SUCCESS) { \ // CUML_LOG_ERROR("CUSOLVER call='%s' got errorcode=%d err=%s", #call, err, \ -// raft::linalg::cusolverErr2Str(err)); \ +// raft::linalg::cusolver_error_to_string(err)); \ // } \ // } while (0) @@ -71,12 +71,14 @@ inline const char *cusolverErr2Str(cusolverStatus_t err) { * @{ */ template -cusolverStatus_t cusolverDngetrf(cusolverDnHandle_t handle, int m, int n, T *A, - int lda, T *Workspace, int *devIpiv, - int *devInfo, cudaStream_t stream); +cusolverStatus_t cusolverDngetrf(cusolverDnHandle_t handle, int m, // NOLINT + int n, T *A, int lda, T *Workspace, + int *devIpiv, int *devInfo, + cudaStream_t stream); template <> -inline cusolverStatus_t cusolverDngetrf(cusolverDnHandle_t handle, int m, int n, +inline cusolverStatus_t cusolverDngetrf(cusolverDnHandle_t handle, // NOLINT + int m, int n, float *A, int lda, float *Workspace, int *devIpiv, int *devInfo, cudaStream_t stream) { @@ -85,7 +87,8 @@ inline cusolverStatus_t cusolverDngetrf(cusolverDnHandle_t handle, int m, int n, } template <> -inline cusolverStatus_t cusolverDngetrf(cusolverDnHandle_t handle, int m, int n, +inline cusolverStatus_t cusolverDngetrf(cusolverDnHandle_t handle, // NOLINT + int m, int n, double *A, int lda, double *Workspace, int *devIpiv, int *devInfo, cudaStream_t stream) { @@ -94,20 +97,18 @@ inline cusolverStatus_t cusolverDngetrf(cusolverDnHandle_t handle, int m, int n, } template -cusolverStatus_t cusolverDngetrf_bufferSize(cusolverDnHandle_t handle, int m, - int n, T *A, int lda, int *Lwork); +cusolverStatus_t cusolverDngetrf_bufferSize( // NOLINT + cusolverDnHandle_t handle, int m, int n, T *A, int lda, int *Lwork); template <> -inline cusolverStatus_t cusolverDngetrf_bufferSize(cusolverDnHandle_t handle, - int m, int n, float *A, - int lda, int *Lwork) { +inline cusolverStatus_t cusolverDngetrf_bufferSize( // NOLINT + cusolverDnHandle_t handle, int m, int n, float *A, int lda, int *Lwork) { return cusolverDnSgetrf_bufferSize(handle, m, n, A, lda, Lwork); } template <> -inline cusolverStatus_t cusolverDngetrf_bufferSize(cusolverDnHandle_t handle, - int m, int n, double *A, - int lda, int *Lwork) { +inline cusolverStatus_t cusolverDngetrf_bufferSize( // NOLINT + cusolverDnHandle_t handle, int m, int n, double *A, int lda, int *Lwork) { return cusolverDnDgetrf_bufferSize(handle, m, n, A, lda, Lwork); } @@ -116,13 +117,13 @@ inline cusolverStatus_t cusolverDngetrf_bufferSize(cusolverDnHandle_t handle, * @{ */ template -cusolverStatus_t cusolverDngetrs(cusolverDnHandle_t handle, +cusolverStatus_t cusolverDngetrs(cusolverDnHandle_t handle, // NOLINT cublasOperation_t trans, int n, int nrhs, const T *A, int lda, const int *devIpiv, T *B, int ldb, int *devInfo, cudaStream_t stream); template <> -inline cusolverStatus_t cusolverDngetrs(cusolverDnHandle_t handle, +inline cusolverStatus_t cusolverDngetrs(cusolverDnHandle_t handle, // NOLINT cublasOperation_t trans, int n, int nrhs, const float *A, int lda, const int *devIpiv, float *B, int ldb, @@ -133,7 +134,7 @@ inline cusolverStatus_t cusolverDngetrs(cusolverDnHandle_t handle, } template <> -inline cusolverStatus_t cusolverDngetrs(cusolverDnHandle_t handle, +inline cusolverStatus_t cusolverDngetrs(cusolverDnHandle_t handle, // NOLINT cublasOperation_t trans, int n, int nrhs, const double *A, int lda, const int *devIpiv, double *B, int ldb, @@ -149,23 +150,19 @@ inline cusolverStatus_t cusolverDngetrs(cusolverDnHandle_t handle, * @{ */ template -cusolverStatus_t cusolverDnsyevd_bufferSize(cusolverDnHandle_t handle, - cusolverEigMode_t jobz, - cublasFillMode_t uplo, int n, - const T *A, int lda, const T *W, - int *lwork); +cusolverStatus_t cusolverDnsyevd_bufferSize( // NOLINT + cusolverDnHandle_t handle, cusolverEigMode_t jobz, cublasFillMode_t uplo, + int n, const T *A, int lda, const T *W, int *lwork); template <> -inline cusolverStatus_t cusolverDnsyevd_bufferSize(cusolverDnHandle_t handle, - cusolverEigMode_t jobz, - cublasFillMode_t uplo, int n, - const float *A, int lda, - const float *W, int *lwork) { +inline cusolverStatus_t cusolverDnsyevd_bufferSize( // NOLINT + cusolverDnHandle_t handle, cusolverEigMode_t jobz, cublasFillMode_t uplo, + int n, const float *A, int lda, const float *W, int *lwork) { return cusolverDnSsyevd_bufferSize(handle, jobz, uplo, n, A, lda, W, lwork); } template <> -inline cusolverStatus_t cusolverDnsyevd_bufferSize( +inline cusolverStatus_t cusolverDnsyevd_bufferSize( // NOLINT cusolverDnHandle_t handle, cusolverEigMode_t jobz, cublasFillMode_t uplo, int n, const double *A, int lda, const double *W, int *lwork) { return cusolverDnDsyevd_bufferSize(handle, jobz, uplo, n, A, lda, W, lwork); @@ -177,14 +174,14 @@ inline cusolverStatus_t cusolverDnsyevd_bufferSize( * @{ */ template -cusolverStatus_t cusolverDnsyevj(cusolverDnHandle_t handle, +cusolverStatus_t cusolverDnsyevj(cusolverDnHandle_t handle, // NOLINT cusolverEigMode_t jobz, cublasFillMode_t uplo, int n, T *A, int lda, T *W, T *work, int lwork, int *info, syevjInfo_t params, cudaStream_t stream); template <> -inline cusolverStatus_t cusolverDnsyevj( +inline cusolverStatus_t cusolverDnsyevj( // NOLINT cusolverDnHandle_t handle, cusolverEigMode_t jobz, cublasFillMode_t uplo, int n, float *A, int lda, float *W, float *work, int lwork, int *info, syevjInfo_t params, cudaStream_t stream) { @@ -194,7 +191,7 @@ inline cusolverStatus_t cusolverDnsyevj( } template <> -inline cusolverStatus_t cusolverDnsyevj( +inline cusolverStatus_t cusolverDnsyevj( // NOLINT cusolverDnHandle_t handle, cusolverEigMode_t jobz, cublasFillMode_t uplo, int n, double *A, int lda, double *W, double *work, int lwork, int *info, syevjInfo_t params, cudaStream_t stream) { @@ -204,30 +201,23 @@ inline cusolverStatus_t cusolverDnsyevj( } template -cusolverStatus_t cusolverDnsyevj_bufferSize(cusolverDnHandle_t handle, - cusolverEigMode_t jobz, - cublasFillMode_t uplo, int n, - const T *A, int lda, const T *W, - int *lwork, syevjInfo_t params); +cusolverStatus_t cusolverDnsyevj_bufferSize( // NOLINT + cusolverDnHandle_t handle, cusolverEigMode_t jobz, cublasFillMode_t uplo, + int n, const T *A, int lda, const T *W, int *lwork, syevjInfo_t params); template <> -inline cusolverStatus_t cusolverDnsyevj_bufferSize(cusolverDnHandle_t handle, - cusolverEigMode_t jobz, - cublasFillMode_t uplo, int n, - const float *A, int lda, - const float *W, int *lwork, - syevjInfo_t params) { +inline cusolverStatus_t cusolverDnsyevj_bufferSize( // NOLINT + cusolverDnHandle_t handle, cusolverEigMode_t jobz, cublasFillMode_t uplo, + int n, const float *A, int lda, const float *W, int *lwork, syevjInfo_t params) { return cusolverDnSsyevj_bufferSize(handle, jobz, uplo, n, A, lda, W, lwork, params); } template <> -inline cusolverStatus_t cusolverDnsyevj_bufferSize(cusolverDnHandle_t handle, - cusolverEigMode_t jobz, - cublasFillMode_t uplo, int n, - const double *A, int lda, - const double *W, int *lwork, - syevjInfo_t params) { +inline cusolverStatus_t cusolverDnsyevj_bufferSize( // NOLINT + cusolverDnHandle_t handle, cusolverEigMode_t jobz, cublasFillMode_t uplo, + int n, const double *A, int lda, const double *W, int *lwork, + syevjInfo_t params) { return cusolverDnDsyevj_bufferSize(handle, jobz, uplo, n, A, lda, W, lwork, params); } @@ -238,13 +228,13 @@ inline cusolverStatus_t cusolverDnsyevj_bufferSize(cusolverDnHandle_t handle, * @{ */ template -cusolverStatus_t cusolverDnsyevd(cusolverDnHandle_t handle, +cusolverStatus_t cusolverDnsyevd(cusolverDnHandle_t handle, // NOLINT cusolverEigMode_t jobz, cublasFillMode_t uplo, int n, T *A, int lda, T *W, T *work, int lwork, int *devInfo, cudaStream_t stream); template <> -inline cusolverStatus_t cusolverDnsyevd(cusolverDnHandle_t handle, +inline cusolverStatus_t cusolverDnsyevd(cusolverDnHandle_t handle, // NOLINT cusolverEigMode_t jobz, cublasFillMode_t uplo, int n, float *A, int lda, float *W, float *work, @@ -256,7 +246,7 @@ inline cusolverStatus_t cusolverDnsyevd(cusolverDnHandle_t handle, } template <> -inline cusolverStatus_t cusolverDnsyevd(cusolverDnHandle_t handle, +inline cusolverStatus_t cusolverDnsyevd(cusolverDnHandle_t handle, // NOLINT cusolverEigMode_t jobz, cublasFillMode_t uplo, int n, double *A, int lda, double *W, double *work, @@ -274,13 +264,13 @@ inline cusolverStatus_t cusolverDnsyevd(cusolverDnHandle_t handle, * @{ */ template -cusolverStatus_t cusolverDnsyevdx_bufferSize( +cusolverStatus_t cusolverDnsyevdx_bufferSize( // NOLINT cusolverDnHandle_t handle, cusolverEigMode_t jobz, cusolverEigRange_t range, cublasFillMode_t uplo, int n, const T *A, int lda, T vl, T vu, int il, int iu, int *h_meig, const T *W, int *lwork); template <> -inline cusolverStatus_t cusolverDnsyevdx_bufferSize( +inline cusolverStatus_t cusolverDnsyevdx_bufferSize( // NOLINT cusolverDnHandle_t handle, cusolverEigMode_t jobz, cusolverEigRange_t range, cublasFillMode_t uplo, int n, const float *A, int lda, float vl, float vu, int il, int iu, int *h_meig, const float *W, int *lwork) { @@ -289,7 +279,7 @@ inline cusolverStatus_t cusolverDnsyevdx_bufferSize( } template <> -inline cusolverStatus_t cusolverDnsyevdx_bufferSize( +inline cusolverStatus_t cusolverDnsyevdx_bufferSize( // NOLINT cusolverDnHandle_t handle, cusolverEigMode_t jobz, cusolverEigRange_t range, cublasFillMode_t uplo, int n, const double *A, int lda, double vl, double vu, int il, int iu, int *h_meig, const double *W, int *lwork) { @@ -298,13 +288,13 @@ inline cusolverStatus_t cusolverDnsyevdx_bufferSize( } template -cusolverStatus_t cusolverDnsyevdx( +cusolverStatus_t cusolverDnsyevdx( // NOLINT cusolverDnHandle_t handle, cusolverEigMode_t jobz, cusolverEigRange_t range, cublasFillMode_t uplo, int n, T *A, int lda, T vl, T vu, int il, int iu, int *h_meig, T *W, T *work, int lwork, int *devInfo, cudaStream_t stream); template <> -inline cusolverStatus_t cusolverDnsyevdx( +inline cusolverStatus_t cusolverDnsyevdx( // NOLINT cusolverDnHandle_t handle, cusolverEigMode_t jobz, cusolverEigRange_t range, cublasFillMode_t uplo, int n, float *A, int lda, float vl, float vu, int il, int iu, int *h_meig, float *W, float *work, int lwork, int *devInfo, @@ -315,7 +305,7 @@ inline cusolverStatus_t cusolverDnsyevdx( } template <> -inline cusolverStatus_t cusolverDnsyevdx( +inline cusolverStatus_t cusolverDnsyevdx( // NOLINT cusolverDnHandle_t handle, cusolverEigMode_t jobz, cusolverEigRange_t range, cublasFillMode_t uplo, int n, double *A, int lda, double vl, double vu, int il, int iu, int *h_meig, double *W, double *work, int lwork, int *devInfo, @@ -332,8 +322,8 @@ inline cusolverStatus_t cusolverDnsyevdx( * @{ */ template -cusolverStatus_t cusolverDngesvd_bufferSize(cusolverDnHandle_t handle, int m, - int n, int *lwork) { +cusolverStatus_t cusolverDngesvd_bufferSize( // NOLINT + cusolverDnHandle_t handle, int m, int n, int *lwork) { if (typeid(T) == typeid(float)) { return cusolverDnSgesvd_bufferSize(handle, m, n, lwork); } else { @@ -341,13 +331,12 @@ cusolverStatus_t cusolverDngesvd_bufferSize(cusolverDnHandle_t handle, int m, } } template -cusolverStatus_t cusolverDngesvd(cusolverDnHandle_t handle, signed char jobu, - signed char jobvt, int m, int n, T *A, int lda, - T *S, T *U, int ldu, T *VT, int ldvt, T *work, - int lwork, T *rwork, int *devInfo, - cudaStream_t stream); +cusolverStatus_t cusolverDngesvd( // NOLINT + cusolverDnHandle_t handle, signed char jobu, signed char jobvt, int m, int n, + T *A, int lda, T *S, T *U, int ldu, T *VT, int ldvt, T *work, int lwork, + T *rwork, int *devInfo, cudaStream_t stream); template <> -inline cusolverStatus_t cusolverDngesvd( +inline cusolverStatus_t cusolverDngesvd( // NOLINT cusolverDnHandle_t handle, signed char jobu, signed char jobvt, int m, int n, float *A, int lda, float *S, float *U, int ldu, float *VT, int ldvt, float *work, int lwork, float *rwork, int *devInfo, cudaStream_t stream) { @@ -356,7 +345,7 @@ inline cusolverStatus_t cusolverDngesvd( ldvt, work, lwork, rwork, devInfo); } template <> -inline cusolverStatus_t cusolverDngesvd( +inline cusolverStatus_t cusolverDngesvd( // NOLINT cusolverDnHandle_t handle, signed char jobu, signed char jobvt, int m, int n, double *A, int lda, double *S, double *U, int ldu, double *VT, int ldvt, double *work, int lwork, double *rwork, int *devInfo, cudaStream_t stream) { @@ -366,12 +355,12 @@ inline cusolverStatus_t cusolverDngesvd( } template -inline cusolverStatus_t CUSOLVERAPI cusolverDngesvdj_bufferSize( +inline cusolverStatus_t CUSOLVERAPI cusolverDngesvdj_bufferSize( // NOLINT cusolverDnHandle_t handle, cusolverEigMode_t jobz, int econ, int m, int n, const T *A, int lda, const T *S, const T *U, int ldu, const T *V, int ldv, int *lwork, gesvdjInfo_t params); template <> -inline cusolverStatus_t CUSOLVERAPI cusolverDngesvdj_bufferSize( +inline cusolverStatus_t CUSOLVERAPI cusolverDngesvdj_bufferSize( // NOLINT cusolverDnHandle_t handle, cusolverEigMode_t jobz, int econ, int m, int n, const float *A, int lda, const float *S, const float *U, int ldu, const float *V, int ldv, int *lwork, gesvdjInfo_t params) { @@ -379,7 +368,7 @@ inline cusolverStatus_t CUSOLVERAPI cusolverDngesvdj_bufferSize( ldu, V, ldv, lwork, params); } template <> -inline cusolverStatus_t CUSOLVERAPI cusolverDngesvdj_bufferSize( +inline cusolverStatus_t CUSOLVERAPI cusolverDngesvdj_bufferSize( // NOLINT cusolverDnHandle_t handle, cusolverEigMode_t jobz, int econ, int m, int n, const double *A, int lda, const double *S, const double *U, int ldu, const double *V, int ldv, int *lwork, gesvdjInfo_t params) { @@ -387,12 +376,12 @@ inline cusolverStatus_t CUSOLVERAPI cusolverDngesvdj_bufferSize( ldu, V, ldv, lwork, params); } template -inline cusolverStatus_t CUSOLVERAPI cusolverDngesvdj( +inline cusolverStatus_t CUSOLVERAPI cusolverDngesvdj( // NOLINT cusolverDnHandle_t handle, cusolverEigMode_t jobz, int econ, int m, int n, T *A, int lda, T *S, T *U, int ldu, T *V, int ldv, T *work, int lwork, int *info, gesvdjInfo_t params, cudaStream_t stream); template <> -inline cusolverStatus_t CUSOLVERAPI cusolverDngesvdj( +inline cusolverStatus_t CUSOLVERAPI cusolverDngesvdj( // NOLINT cusolverDnHandle_t handle, cusolverEigMode_t jobz, int econ, int m, int n, float *A, int lda, float *S, float *U, int ldu, float *V, int ldv, float *work, int lwork, int *info, gesvdjInfo_t params, cudaStream_t stream) { @@ -402,10 +391,11 @@ inline cusolverStatus_t CUSOLVERAPI cusolverDngesvdj( } template <> inline cusolverStatus_t CUSOLVERAPI -cusolverDngesvdj(cusolverDnHandle_t handle, cusolverEigMode_t jobz, int econ, - int m, int n, double *A, int lda, double *S, double *U, - int ldu, double *V, int ldv, double *work, int lwork, - int *info, gesvdjInfo_t params, cudaStream_t stream) { +cusolverDngesvdj( // NOLINT + cusolverDnHandle_t handle, cusolverEigMode_t jobz, int econ, int m, int n, + double *A, int lda, double *S, double *U, int ldu, double *V, int ldv, + double *work, int lwork, int *info, gesvdjInfo_t params, + cudaStream_t stream) { CUSOLVER_CHECK(cusolverDnSetStream(handle, stream)); return cusolverDnDgesvdj(handle, jobz, econ, m, n, A, lda, S, U, ldu, V, ldv, work, lwork, info, params); @@ -417,34 +407,32 @@ cusolverDngesvdj(cusolverDnHandle_t handle, cusolverEigMode_t jobz, int econ, * @{ */ template -cusolverStatus_t cusolverDnpotrf_bufferSize(cusolverDnHandle_t handle, - cublasFillMode_t uplo, int n, T *A, - int lda, int *Lwork); +cusolverStatus_t cusolverDnpotrf_bufferSize( // NOLINT + cusolverDnHandle_t handle, cublasFillMode_t uplo, int n, T *A, int lda, + int *Lwork); template <> -inline cusolverStatus_t cusolverDnpotrf_bufferSize(cusolverDnHandle_t handle, - cublasFillMode_t uplo, int n, - float *A, int lda, - int *Lwork) { +inline cusolverStatus_t cusolverDnpotrf_bufferSize( // NOLINT + cusolverDnHandle_t handle, cublasFillMode_t uplo, int n, float *A, int lda, + int *Lwork) { return cusolverDnSpotrf_bufferSize(handle, uplo, n, A, lda, Lwork); } template <> -inline cusolverStatus_t cusolverDnpotrf_bufferSize(cusolverDnHandle_t handle, - cublasFillMode_t uplo, int n, - double *A, int lda, - int *Lwork) { +inline cusolverStatus_t cusolverDnpotrf_bufferSize( // NOLINT + cusolverDnHandle_t handle, cublasFillMode_t uplo, int n, double *A, int lda, + int *Lwork) { return cusolverDnDpotrf_bufferSize(handle, uplo, n, A, lda, Lwork); } template -inline cusolverStatus_t cusolverDnpotrf(cusolverDnHandle_t handle, +inline cusolverStatus_t cusolverDnpotrf(cusolverDnHandle_t handle, // NOLINT cublasFillMode_t uplo, int n, T *A, int lda, T *Workspace, int Lwork, int *devInfo, cudaStream_t stream); template <> -inline cusolverStatus_t cusolverDnpotrf(cusolverDnHandle_t handle, +inline cusolverStatus_t cusolverDnpotrf(cusolverDnHandle_t handle, // NOLINT cublasFillMode_t uplo, int n, float *A, int lda, float *Workspace, int Lwork, int *devInfo, cudaStream_t stream) { @@ -453,7 +441,7 @@ inline cusolverStatus_t cusolverDnpotrf(cusolverDnHandle_t handle, } template <> -inline cusolverStatus_t cusolverDnpotrf(cusolverDnHandle_t handle, +inline cusolverStatus_t cusolverDnpotrf(cusolverDnHandle_t handle, // NOLINT cublasFillMode_t uplo, int n, double *A, int lda, double *Workspace, int Lwork, int *devInfo, cudaStream_t stream) { @@ -467,13 +455,13 @@ inline cusolverStatus_t cusolverDnpotrf(cusolverDnHandle_t handle, * @{ */ template -cusolverStatus_t cusolverDnpotrs(cusolverDnHandle_t handle, +cusolverStatus_t cusolverDnpotrs(cusolverDnHandle_t handle, // NOLINT cublasFillMode_t uplo, int n, int nrhs, const T *A, int lda, T *B, int ldb, int *devInfo, cudaStream_t stream); template <> -inline cusolverStatus_t cusolverDnpotrs(cusolverDnHandle_t handle, +inline cusolverStatus_t cusolverDnpotrs(cusolverDnHandle_t handle, // NOLINT cublasFillMode_t uplo, int n, int nrhs, const float *A, int lda, float *B, int ldb, int *devInfo, @@ -483,7 +471,7 @@ inline cusolverStatus_t cusolverDnpotrs(cusolverDnHandle_t handle, } template <> -inline cusolverStatus_t cusolverDnpotrs(cusolverDnHandle_t handle, +inline cusolverStatus_t cusolverDnpotrs(cusolverDnHandle_t handle, // NOLINT cublasFillMode_t uplo, int n, int nrhs, const double *A, int lda, double *B, int ldb, int *devInfo, @@ -498,39 +486,38 @@ inline cusolverStatus_t cusolverDnpotrs(cusolverDnHandle_t handle, * @{ */ template -cusolverStatus_t cusolverDngeqrf(cusolverDnHandle_t handle, int m, int n, T *A, - int lda, T *TAU, T *Workspace, int Lwork, - int *devInfo, cudaStream_t stream); +cusolverStatus_t cusolverDngeqrf(cusolverDnHandle_t handle, int m, // NOLINT + int n, T *A, int lda, T *TAU, T *Workspace, + int Lwork, int *devInfo, cudaStream_t stream); template <> -inline cusolverStatus_t cusolverDngeqrf(cusolverDnHandle_t handle, int m, int n, - float *A, int lda, float *TAU, - float *Workspace, int Lwork, +inline cusolverStatus_t cusolverDngeqrf(cusolverDnHandle_t handle, // NOLINT + int m, int n, float *A, int lda, + float *TAU, float *Workspace, int Lwork, int *devInfo, cudaStream_t stream) { CUSOLVER_CHECK(cusolverDnSetStream(handle, stream)); return cusolverDnSgeqrf(handle, m, n, A, lda, TAU, Workspace, Lwork, devInfo); } template <> -inline cusolverStatus_t cusolverDngeqrf(cusolverDnHandle_t handle, int m, int n, - double *A, int lda, double *TAU, - double *Workspace, int Lwork, - int *devInfo, cudaStream_t stream) { +inline cusolverStatus_t cusolverDngeqrf(cusolverDnHandle_t handle, // NOLINT + int m, int n, double *A, int lda, + double *TAU, double *Workspace, + int Lwork, int *devInfo, + cudaStream_t stream) { CUSOLVER_CHECK(cusolverDnSetStream(handle, stream)); return cusolverDnDgeqrf(handle, m, n, A, lda, TAU, Workspace, Lwork, devInfo); } template -cusolverStatus_t cusolverDngeqrf_bufferSize(cusolverDnHandle_t handle, int m, - int n, T *A, int lda, int *Lwork); +cusolverStatus_t cusolverDngeqrf_bufferSize( // NOLINT + cusolverDnHandle_t handle, int m, int n, T *A, int lda, int *Lwork); template <> -inline cusolverStatus_t cusolverDngeqrf_bufferSize(cusolverDnHandle_t handle, - int m, int n, float *A, - int lda, int *Lwork) { +inline cusolverStatus_t cusolverDngeqrf_bufferSize( // NOLINT + cusolverDnHandle_t handle, int m, int n, float *A, int lda, int *Lwork) { return cusolverDnSgeqrf_bufferSize(handle, m, n, A, lda, Lwork); } template <> -inline cusolverStatus_t cusolverDngeqrf_bufferSize(cusolverDnHandle_t handle, - int m, int n, double *A, - int lda, int *Lwork) { +inline cusolverStatus_t cusolverDngeqrf_bufferSize( // NOLINT + cusolverDnHandle_t handle, int m, int n, double *A, int lda, int *Lwork) { return cusolverDnDgeqrf_bufferSize(handle, m, n, A, lda, Lwork); } /** @} */ @@ -540,11 +527,13 @@ inline cusolverStatus_t cusolverDngeqrf_bufferSize(cusolverDnHandle_t handle, * @{ */ template -cusolverStatus_t cusolverDnorgqr(cusolverDnHandle_t handle, int m, int n, int k, +cusolverStatus_t cusolverDnorgqr( // NOLINT + cusolverDnHandle_t handle, int m, int n, int k, T *A, int lda, const T *tau, T *work, int lwork, int *devInfo, cudaStream_t stream); template <> -inline cusolverStatus_t cusolverDnorgqr(cusolverDnHandle_t handle, int m, int n, +inline cusolverStatus_t cusolverDnorgqr( // NOLINT + cusolverDnHandle_t handle, int m, int n, int k, float *A, int lda, const float *tau, float *work, int lwork, int *devInfo, @@ -553,7 +542,8 @@ inline cusolverStatus_t cusolverDnorgqr(cusolverDnHandle_t handle, int m, int n, return cusolverDnSorgqr(handle, m, n, k, A, lda, tau, work, lwork, devInfo); } template <> -inline cusolverStatus_t cusolverDnorgqr(cusolverDnHandle_t handle, int m, int n, +inline cusolverStatus_t cusolverDnorgqr( // NOLINT + cusolverDnHandle_t handle, int m, int n, int k, double *A, int lda, const double *tau, double *work, int lwork, int *devInfo, @@ -563,11 +553,13 @@ inline cusolverStatus_t cusolverDnorgqr(cusolverDnHandle_t handle, int m, int n, } template -cusolverStatus_t cusolverDnorgqr_bufferSize(cusolverDnHandle_t handle, int m, +cusolverStatus_t cusolverDnorgqr_bufferSize( // NOLINT + cusolverDnHandle_t handle, int m, int n, int k, const T *A, int lda, const T *TAU, int *lwork); template <> -inline cusolverStatus_t cusolverDnorgqr_bufferSize(cusolverDnHandle_t handle, +inline cusolverStatus_t cusolverDnorgqr_bufferSize( // NOLINT + cusolverDnHandle_t handle, int m, int n, int k, const float *A, int lda, const float *TAU, @@ -575,7 +567,8 @@ inline cusolverStatus_t cusolverDnorgqr_bufferSize(cusolverDnHandle_t handle, return cusolverDnSorgqr_bufferSize(handle, m, n, k, A, lda, TAU, lwork); } template <> -inline cusolverStatus_t cusolverDnorgqr_bufferSize(cusolverDnHandle_t handle, +inline cusolverStatus_t cusolverDnorgqr_bufferSize( // NOLINT + cusolverDnHandle_t handle, int m, int n, int k, const double *A, int lda, const double *TAU, @@ -589,14 +582,14 @@ inline cusolverStatus_t cusolverDnorgqr_bufferSize(cusolverDnHandle_t handle, * @{ */ template -cusolverStatus_t cusolverDnormqr(cusolverDnHandle_t handle, +cusolverStatus_t cusolverDnormqr(cusolverDnHandle_t handle, // NOLINT cublasSideMode_t side, cublasOperation_t trans, int m, int n, int k, const T *A, int lda, const T *tau, T *C, int ldc, T *work, int lwork, int *devInfo, cudaStream_t stream); template <> -inline cusolverStatus_t cusolverDnormqr( +inline cusolverStatus_t cusolverDnormqr( // NOLINT cusolverDnHandle_t handle, cublasSideMode_t side, cublasOperation_t trans, int m, int n, int k, const float *A, int lda, const float *tau, float *C, int ldc, float *work, int lwork, int *devInfo, cudaStream_t stream) { @@ -606,7 +599,7 @@ inline cusolverStatus_t cusolverDnormqr( } template <> -inline cusolverStatus_t cusolverDnormqr( +inline cusolverStatus_t cusolverDnormqr( // NOLINT cusolverDnHandle_t handle, cublasSideMode_t side, cublasOperation_t trans, int m, int n, int k, const double *A, int lda, const double *tau, double *C, int ldc, double *work, int lwork, int *devInfo, cudaStream_t stream) { @@ -616,15 +609,16 @@ inline cusolverStatus_t cusolverDnormqr( } template -cusolverStatus_t cusolverDnormqr_bufferSize(cusolverDnHandle_t handle, - cublasSideMode_t side, +cusolverStatus_t cusolverDnormqr_bufferSize( // NOLINT + cusolverDnHandle_t handle, + cublasSideMode_t side, cublasOperation_t trans, int m, int n, int k, const T *A, int lda, const T *tau, const T *C, int ldc, int *lwork); template <> -inline cusolverStatus_t cusolverDnormqr_bufferSize( +inline cusolverStatus_t cusolverDnormqr_bufferSize( // NOLINT cusolverDnHandle_t handle, cublasSideMode_t side, cublasOperation_t trans, int m, int n, int k, const float *A, int lda, const float *tau, const float *C, int ldc, int *lwork) { @@ -633,7 +627,7 @@ inline cusolverStatus_t cusolverDnormqr_bufferSize( } template <> -inline cusolverStatus_t cusolverDnormqr_bufferSize( +inline cusolverStatus_t cusolverDnormqr_bufferSize( // NOLINT cusolverDnHandle_t handle, cublasSideMode_t side, cublasOperation_t trans, int m, int n, int k, const double *A, int lda, const double *tau, const double *C, int ldc, int *lwork) { @@ -647,14 +641,14 @@ inline cusolverStatus_t cusolverDnormqr_bufferSize( * @{ */ template -cusolverStatus_t cusolverSpcsrqrBufferInfoBatched( +cusolverStatus_t cusolverSpcsrqrBufferInfoBatched( // NOLINT cusolverSpHandle_t handle, int m, int n, int nnzA, const cusparseMatDescr_t descrA, const T *csrValA, const int *csrRowPtrA, const int *csrColIndA, int batchSize, csrqrInfo_t info, size_t *internalDataInBytes, size_t *workspaceInBytes); template <> -inline cusolverStatus_t cusolverSpcsrqrBufferInfoBatched( +inline cusolverStatus_t cusolverSpcsrqrBufferInfoBatched( // NOLINT cusolverSpHandle_t handle, int m, int n, int nnzA, const cusparseMatDescr_t descrA, const float *csrValA, const int *csrRowPtrA, const int *csrColIndA, int batchSize, csrqrInfo_t info, @@ -665,7 +659,7 @@ inline cusolverStatus_t cusolverSpcsrqrBufferInfoBatched( } template <> -inline cusolverStatus_t cusolverSpcsrqrBufferInfoBatched( +inline cusolverStatus_t cusolverSpcsrqrBufferInfoBatched( // NOLINT cusolverSpHandle_t handle, int m, int n, int nnzA, const cusparseMatDescr_t descrA, const double *csrValA, const int *csrRowPtrA, const int *csrColIndA, int batchSize, csrqrInfo_t info, @@ -676,14 +670,14 @@ inline cusolverStatus_t cusolverSpcsrqrBufferInfoBatched( } template -cusolverStatus_t cusolverSpcsrqrsvBatched( +cusolverStatus_t cusolverSpcsrqrsvBatched( // NOLINT cusolverSpHandle_t handle, int m, int n, int nnzA, const cusparseMatDescr_t descrA, const T *csrValA, const int *csrRowPtrA, const int *csrColIndA, const T *b, T *x, int batchSize, csrqrInfo_t info, void *pBuffer, cudaStream_t stream); template <> -inline cusolverStatus_t cusolverSpcsrqrsvBatched( +inline cusolverStatus_t cusolverSpcsrqrsvBatched( // NOLINT cusolverSpHandle_t handle, int m, int n, int nnzA, const cusparseMatDescr_t descrA, const float *csrValA, const int *csrRowPtrA, const int *csrColIndA, const float *b, float *x, int batchSize, @@ -695,7 +689,7 @@ inline cusolverStatus_t cusolverSpcsrqrsvBatched( } template <> -inline cusolverStatus_t cusolverSpcsrqrsvBatched( +inline cusolverStatus_t cusolverSpcsrqrsvBatched( // NOLINT cusolverSpHandle_t handle, int m, int n, int nnzA, const cusparseMatDescr_t descrA, const double *csrValA, const int *csrRowPtrA, const int *csrColIndA, const double *b, double *x, int batchSize, From 235eac6228a0c70dfc36a2da26a52e1c672842d7 Mon Sep 17 00:00:00 2001 From: Thejaswi Rao Date: Thu, 7 May 2020 00:47:24 -0700 Subject: [PATCH 36/56] FIX clang-format fixes --- cpp/include/raft/cudart_utils.h | 9 ++- cpp/include/raft/host_buffer.hpp | 3 +- cpp/include/raft/linalg/cublas_wrappers.h | 50 +++++++-------- cpp/include/raft/linalg/cusolver_wrappers.h | 70 ++++++++------------- 4 files changed, 59 insertions(+), 73 deletions(-) diff --git a/cpp/include/raft/cudart_utils.h b/cpp/include/raft/cudart_utils.h index 93224b9c54..ac31ca917a 100644 --- a/cpp/include/raft/cudart_utils.h +++ b/cpp/include/raft/cudart_utils.h @@ -37,12 +37,14 @@ class exception : public std::exception { explicit exception() noexcept : std::exception(), msg_() {} /** copy ctor */ - exception(const exception& src) noexcept : std::exception(), msg_(src.what()) { + exception(const exception& src) noexcept + : std::exception(), msg_(src.what()) { collect_call_stack(); } /** ctor from an input message */ - explicit exception(const std::string _msg) noexcept : std::exception(), msg_(std::move(_msg)) { + explicit exception(const std::string _msg) noexcept + : std::exception(), msg_(std::move(_msg)) { collect_call_stack(); } @@ -167,7 +169,8 @@ void update_device(Type* dPtr, const Type* hPtr, size_t len, /** performs a device to host copy */ template -void update_host(Type* hPtr, const Type* dPtr, size_t len, cudaStream_t stream) { +void update_host(Type* hPtr, const Type* dPtr, size_t len, + cudaStream_t stream) { copy(hPtr, dPtr, len, stream); } diff --git a/cpp/include/raft/host_buffer.hpp b/cpp/include/raft/host_buffer.hpp index 48fdbc576b..b0d54d75b0 100644 --- a/cpp/include/raft/host_buffer.hpp +++ b/cpp/include/raft/host_buffer.hpp @@ -46,7 +46,8 @@ class host_buffer : public buffer_base { using size_type = typename buffer_base::size_type; using value_type = typename buffer_base::value_type; using iterator = typename buffer_base::iterator; - using const_iterator = typename buffer_base::const_iterator; + using const_iterator = + typename buffer_base::const_iterator; using reference = typename buffer_base::reference; using const_reference = typename buffer_base::const_reference; diff --git a/cpp/include/raft/linalg/cublas_wrappers.h b/cpp/include/raft/linalg/cublas_wrappers.h index 3594069921..cd8a508a84 100644 --- a/cpp/include/raft/linalg/cublas_wrappers.h +++ b/cpp/include/raft/linalg/cublas_wrappers.h @@ -197,7 +197,7 @@ cublasStatus_t cublasgemmBatched(cublasHandle_t handle, // NOLINT cublasOperation_t transa, cublasOperation_t transb, int m, int n, int k, const T *alpha, - const T *const Aarray[], // NOLINT + const T *const Aarray[], // NOLINT int lda, const T *const Barray[], // NOLINT int ldb, const T *beta, T *Carray[], // NOLINT @@ -207,8 +207,8 @@ template <> inline cublasStatus_t cublasgemmBatched( // NOLINT cublasHandle_t handle, cublasOperation_t transa, cublasOperation_t transb, int m, int n, int k, const float *alpha, - const float *const Aarray[], // NOLINT - int lda, const float *const Barray[], // NOLINT + const float *const Aarray[], // NOLINT + int lda, const float *const Barray[], // NOLINT int ldb, const float *beta, float *Carray[], // NOLINT int ldc, int batchCount, cudaStream_t stream) { CUBLAS_CHECK(cublasSetStream(handle, stream)); @@ -220,8 +220,8 @@ template <> inline cublasStatus_t cublasgemmBatched( // NOLINT cublasHandle_t handle, cublasOperation_t transa, cublasOperation_t transb, int m, int n, int k, const double *alpha, - const double *const Aarray[], // NOLINT - int lda, const double *const Barray[], // NOLINT + const double *const Aarray[], // NOLINT + int lda, const double *const Barray[], // NOLINT int ldb, const double *beta, double *Carray[], // NOLINT int ldc, int batchCount, cudaStream_t stream) { CUBLAS_CHECK(cublasSetStream(handle, stream)); @@ -246,9 +246,9 @@ template <> inline cublasStatus_t cublasgemmStridedBatched( // NOLINT cublasHandle_t handle, cublasOperation_t transa, cublasOperation_t transb, int m, int n, int k, const float *alpha, const float *const Aarray, int lda, - int64_t strideA, const float *const Barray, int ldb, - int64_t strideB, const float *beta, float *Carray, int ldc, - int64_t strideC, int batchCount, cudaStream_t stream) { + int64_t strideA, const float *const Barray, int ldb, int64_t strideB, + const float *beta, float *Carray, int ldc, int64_t strideC, int batchCount, + cudaStream_t stream) { CUBLAS_CHECK(cublasSetStream(handle, stream)); return cublasSgemmStridedBatched(handle, transa, transb, m, n, k, alpha, Aarray, lda, strideA, Barray, ldb, strideB, @@ -259,9 +259,9 @@ template <> inline cublasStatus_t cublasgemmStridedBatched( // NOLINT cublasHandle_t handle, cublasOperation_t transa, cublasOperation_t transb, int m, int n, int k, const double *alpha, const double *const Aarray, int lda, - int64_t strideA, const double *const Barray, int ldb, - int64_t strideB, const double *beta, double *Carray, int ldc, - int64_t strideC, int batchCount, cudaStream_t stream) { + int64_t strideA, const double *const Barray, int ldb, int64_t strideB, + const double *beta, double *Carray, int ldc, int64_t strideC, int batchCount, + cudaStream_t stream) { CUBLAS_CHECK(cublasSetStream(handle, stream)); return cublasDgemmStridedBatched(handle, transa, transb, m, n, k, alpha, Aarray, lda, strideA, Barray, ldb, strideB, @@ -275,13 +275,13 @@ inline cublasStatus_t cublasgemmStridedBatched( // NOLINT */ template -cublasStatus_t cublasgetrfBatched(cublasHandle_t handle, int n, // NOLINT - T *const A[], // NOLINT - int lda, int *P, int *info, - int batchSize, cudaStream_t stream); +cublasStatus_t cublasgetrfBatched(cublasHandle_t handle, int n, // NOLINT + T *const A[], // NOLINT + int lda, int *P, int *info, int batchSize, + cudaStream_t stream); template <> -inline cublasStatus_t cublasgetrfBatched(cublasHandle_t handle, // NOLINT +inline cublasStatus_t cublasgetrfBatched(cublasHandle_t handle, // NOLINT int n, float *const A[], // NOLINT int lda, int *P, int *info, int batchSize, cudaStream_t stream) { @@ -290,7 +290,7 @@ inline cublasStatus_t cublasgetrfBatched(cublasHandle_t handle, // NOLINT } template <> -inline cublasStatus_t cublasgetrfBatched(cublasHandle_t handle, // NOLINT +inline cublasStatus_t cublasgetrfBatched(cublasHandle_t handle, // NOLINT int n, double *const A[], // NOLINT int lda, int *P, int *info, int batchSize, cudaStream_t stream) { @@ -300,25 +300,25 @@ inline cublasStatus_t cublasgetrfBatched(cublasHandle_t handle, // NOLINT template cublasStatus_t cublasgetriBatched(cublasHandle_t handle, int n, // NOLINT - const T *const A[], // NOLINT + const T *const A[], // NOLINT int lda, const int *P, T *const C[], // NOLINT int ldc, int *info, int batchSize, cudaStream_t stream); template <> -inline cublasStatus_t cublasgetriBatched( // NOLINT +inline cublasStatus_t cublasgetriBatched( // NOLINT cublasHandle_t handle, int n, const float *const A[], // NOLINT - int lda, const int *P, float *const C[], // NOLINT + int lda, const int *P, float *const C[], // NOLINT int ldc, int *info, int batchSize, cudaStream_t stream) { CUBLAS_CHECK(cublasSetStream(handle, stream)); return cublasSgetriBatched(handle, n, A, lda, P, C, ldc, info, batchSize); } template <> -inline cublasStatus_t cublasgetriBatched( // NOLINT +inline cublasStatus_t cublasgetriBatched( // NOLINT cublasHandle_t handle, int n, const double *const A[], // NOLINT - int lda, const int *P, double *const C[], // NOLINT + int lda, const int *P, double *const C[], // NOLINT int ldc, int *info, int batchSize, cudaStream_t stream) { CUBLAS_CHECK(cublasSetStream(handle, stream)); return cublasDgetriBatched(handle, n, A, lda, P, C, ldc, info, batchSize); @@ -335,7 +335,7 @@ template inline cublasStatus_t cublasgelsBatched(cublasHandle_t handle, // NOLINT cublasOperation_t trans, int m, int n, int nrhs, T *Aarray[], // NOLINT - int lda, T *Carray[], // NOLINT + int lda, T *Carray[], // NOLINT int ldc, int *info, int *devInfoArray, int batchSize, cudaStream_t stream); @@ -343,7 +343,7 @@ template <> inline cublasStatus_t cublasgelsBatched(cublasHandle_t handle, // NOLINT cublasOperation_t trans, int m, int n, int nrhs, float *Aarray[], // NOLINT - int lda, float *Carray[], // NOLINT + int lda, float *Carray[], // NOLINT int ldc, int *info, int *devInfoArray, int batchSize, cudaStream_t stream) { CUBLAS_CHECK(cublasSetStream(handle, stream)); @@ -355,7 +355,7 @@ template <> inline cublasStatus_t cublasgelsBatched(cublasHandle_t handle, // NOLINT cublasOperation_t trans, int m, int n, int nrhs, double *Aarray[], // NOLINT - int lda, double *Carray[], // NOLINT + int lda, double *Carray[], // NOLINT int ldc, int *info, int *devInfoArray, int batchSize, cudaStream_t stream) { CUBLAS_CHECK(cublasSetStream(handle, stream)); diff --git a/cpp/include/raft/linalg/cusolver_wrappers.h b/cpp/include/raft/linalg/cusolver_wrappers.h index 850ea9654d..92ba1a2194 100644 --- a/cpp/include/raft/linalg/cusolver_wrappers.h +++ b/cpp/include/raft/linalg/cusolver_wrappers.h @@ -78,20 +78,18 @@ cusolverStatus_t cusolverDngetrf(cusolverDnHandle_t handle, int m, // NOLINT template <> inline cusolverStatus_t cusolverDngetrf(cusolverDnHandle_t handle, // NOLINT - int m, int n, - float *A, int lda, float *Workspace, - int *devIpiv, int *devInfo, - cudaStream_t stream) { + int m, int n, float *A, int lda, + float *Workspace, int *devIpiv, + int *devInfo, cudaStream_t stream) { CUSOLVER_CHECK(cusolverDnSetStream(handle, stream)); return cusolverDnSgetrf(handle, m, n, A, lda, Workspace, devIpiv, devInfo); } template <> inline cusolverStatus_t cusolverDngetrf(cusolverDnHandle_t handle, // NOLINT - int m, int n, - double *A, int lda, double *Workspace, - int *devIpiv, int *devInfo, - cudaStream_t stream) { + int m, int n, double *A, int lda, + double *Workspace, int *devIpiv, + int *devInfo, cudaStream_t stream) { CUSOLVER_CHECK(cusolverDnSetStream(handle, stream)); return cusolverDnDgetrf(handle, m, n, A, lda, Workspace, devIpiv, devInfo); } @@ -208,7 +206,8 @@ cusolverStatus_t cusolverDnsyevj_bufferSize( // NOLINT template <> inline cusolverStatus_t cusolverDnsyevj_bufferSize( // NOLINT cusolverDnHandle_t handle, cusolverEigMode_t jobz, cublasFillMode_t uplo, - int n, const float *A, int lda, const float *W, int *lwork, syevjInfo_t params) { + int n, const float *A, int lda, const float *W, int *lwork, + syevjInfo_t params) { return cusolverDnSsyevj_bufferSize(handle, jobz, uplo, n, A, lda, W, lwork, params); } @@ -390,8 +389,7 @@ inline cusolverStatus_t CUSOLVERAPI cusolverDngesvdj( // NOLINT work, lwork, info, params); } template <> -inline cusolverStatus_t CUSOLVERAPI -cusolverDngesvdj( // NOLINT +inline cusolverStatus_t CUSOLVERAPI cusolverDngesvdj( // NOLINT cusolverDnHandle_t handle, cusolverEigMode_t jobz, int econ, int m, int n, double *A, int lda, double *S, double *U, int ldu, double *V, int ldv, double *work, int lwork, int *info, gesvdjInfo_t params, @@ -490,7 +488,7 @@ cusolverStatus_t cusolverDngeqrf(cusolverDnHandle_t handle, int m, // NOLINT int n, T *A, int lda, T *TAU, T *Workspace, int Lwork, int *devInfo, cudaStream_t stream); template <> -inline cusolverStatus_t cusolverDngeqrf(cusolverDnHandle_t handle, // NOLINT +inline cusolverStatus_t cusolverDngeqrf(cusolverDnHandle_t handle, // NOLINT int m, int n, float *A, int lda, float *TAU, float *Workspace, int Lwork, int *devInfo, cudaStream_t stream) { @@ -528,51 +526,38 @@ inline cusolverStatus_t cusolverDngeqrf_bufferSize( // NOLINT */ template cusolverStatus_t cusolverDnorgqr( // NOLINT - cusolverDnHandle_t handle, int m, int n, int k, - T *A, int lda, const T *tau, T *work, - int lwork, int *devInfo, cudaStream_t stream); + cusolverDnHandle_t handle, int m, int n, int k, T *A, int lda, const T *tau, + T *work, int lwork, int *devInfo, cudaStream_t stream); template <> inline cusolverStatus_t cusolverDnorgqr( // NOLINT - cusolverDnHandle_t handle, int m, int n, - int k, float *A, int lda, - const float *tau, float *work, - int lwork, int *devInfo, - cudaStream_t stream) { + cusolverDnHandle_t handle, int m, int n, int k, float *A, int lda, + const float *tau, float *work, int lwork, int *devInfo, cudaStream_t stream) { CUSOLVER_CHECK(cusolverDnSetStream(handle, stream)); return cusolverDnSorgqr(handle, m, n, k, A, lda, tau, work, lwork, devInfo); } template <> inline cusolverStatus_t cusolverDnorgqr( // NOLINT - cusolverDnHandle_t handle, int m, int n, - int k, double *A, int lda, - const double *tau, double *work, - int lwork, int *devInfo, - cudaStream_t stream) { + cusolverDnHandle_t handle, int m, int n, int k, double *A, int lda, + const double *tau, double *work, int lwork, int *devInfo, + cudaStream_t stream) { CUSOLVER_CHECK(cusolverDnSetStream(handle, stream)); return cusolverDnDorgqr(handle, m, n, k, A, lda, tau, work, lwork, devInfo); } template cusolverStatus_t cusolverDnorgqr_bufferSize( // NOLINT - cusolverDnHandle_t handle, int m, - int n, int k, const T *A, int lda, - const T *TAU, int *lwork); + cusolverDnHandle_t handle, int m, int n, int k, const T *A, int lda, + const T *TAU, int *lwork); template <> inline cusolverStatus_t cusolverDnorgqr_bufferSize( // NOLINT - cusolverDnHandle_t handle, - int m, int n, int k, - const float *A, int lda, - const float *TAU, - int *lwork) { + cusolverDnHandle_t handle, int m, int n, int k, const float *A, int lda, + const float *TAU, int *lwork) { return cusolverDnSorgqr_bufferSize(handle, m, n, k, A, lda, TAU, lwork); } template <> inline cusolverStatus_t cusolverDnorgqr_bufferSize( // NOLINT - cusolverDnHandle_t handle, - int m, int n, int k, - const double *A, int lda, - const double *TAU, - int *lwork) { + cusolverDnHandle_t handle, int m, int n, int k, const double *A, int lda, + const double *TAU, int *lwork) { return cusolverDnDorgqr_bufferSize(handle, m, n, k, A, lda, TAU, lwork); } /** @} */ @@ -610,12 +595,9 @@ inline cusolverStatus_t cusolverDnormqr( // NOLINT template cusolverStatus_t cusolverDnormqr_bufferSize( // NOLINT - cusolverDnHandle_t handle, - cublasSideMode_t side, - cublasOperation_t trans, int m, - int n, int k, const T *A, int lda, - const T *tau, const T *C, int ldc, - int *lwork); + cusolverDnHandle_t handle, cublasSideMode_t side, cublasOperation_t trans, + int m, int n, int k, const T *A, int lda, const T *tau, const T *C, int ldc, + int *lwork); template <> inline cusolverStatus_t cusolverDnormqr_bufferSize( // NOLINT From e670c3a9b039dacb8c31a10513b60d00aaa1c668 Mon Sep 17 00:00:00 2001 From: Thejaswi Rao Date: Thu, 7 May 2020 00:49:17 -0700 Subject: [PATCH 37/56] FIX clang tidy fixes for cusparse wrappers --- cpp/include/raft/sparse/cusparse_wrappers.h | 18 +++++++++++------- 1 file changed, 11 insertions(+), 7 deletions(-) diff --git a/cpp/include/raft/sparse/cusparse_wrappers.h b/cpp/include/raft/sparse/cusparse_wrappers.h index 369624786d..304b2bf0c3 100644 --- a/cpp/include/raft/sparse/cusparse_wrappers.h +++ b/cpp/include/raft/sparse/cusparse_wrappers.h @@ -27,7 +27,7 @@ namespace sparse { #define _CUSPARSE_ERR_TO_STR(err) \ case err: \ return #err; -inline const char* cusparseErr2Str(cusparseStatus_t err) { +inline const char* cusparse_error_to_string(cusparseStatus_t err) { #if defined(CUDART_VERSION) && CUDART_VERSION >= 10100 return cusparseGetErrorString(status); #else // CUDART_VERSION @@ -53,7 +53,7 @@ inline const char* cusparseErr2Str(cusparseStatus_t err) { cusparseStatus_t err = call; \ ASSERT(err == CUSPARSE_STATUS_SUCCESS, \ "CUSPARSE call='%s' got errorcode=%d err=%s", #call, err, \ - raft::sparse::cusparseErr2Str(err)); \ + raft::sparse::cusparse_error_to_string(err)); \ } while (0) ///@todo: enable this once logging is enabled @@ -63,7 +63,7 @@ inline const char* cusparseErr2Str(cusparseStatus_t err) { // cusparseStatus_t err = call; \ // if (err != CUSPARSE_STATUS_SUCCESS) { \ // CUML_LOG_ERROR("CUSPARSE call='%s' got errorcode=%d err=%s", #call, err, \ -// raft::sparse::cusparseErr2Str(err)); \ +// raft::sparse::cusparse_error_to_string(err)); \ // } \ // } while (0) @@ -114,11 +114,13 @@ inline void cusparsecoo2csr(cusparseHandle_t handle, const int* cooRowInd, * @{ */ template -size_t cusparsecoosort_bufferSizeExt(cusparseHandle_t handle, int m, int n, +size_t cusparsecoosort_bufferSizeExt( // NOLINT + cusparseHandle_t handle, int m, int n, int nnz, const T* cooRows, const T* cooCols, cudaStream_t stream); template <> -inline size_t cusparsecoosort_bufferSizeExt(cusparseHandle_t handle, int m, +inline size_t cusparsecoosort_bufferSizeExt( // NOLINT + cusparseHandle_t handle, int m, int n, int nnz, const int* cooRows, const int* cooCols, cudaStream_t stream) { @@ -130,11 +132,13 @@ inline size_t cusparsecoosort_bufferSizeExt(cusparseHandle_t handle, int m, } template -void cusparsecoosortByRow(cusparseHandle_t handle, int m, int n, int nnz, +void cusparsecoosortByRow( // NOLINT + cusparseHandle_t handle, int m, int n, int nnz, T* cooRows, T* cooCols, T* P, void* pBuffer, cudaStream_t stream); template <> -inline void cusparsecoosortByRow(cusparseHandle_t handle, int m, int n, int nnz, +inline void cusparsecoosortByRow( // NOLINT + cusparseHandle_t handle, int m, int n, int nnz, int* cooRows, int* cooCols, int* P, void* pBuffer, cudaStream_t stream) { CUSPARSE_CHECK(cusparseSetStream(handle, stream)); From 619b5c5a48cf73f1001942559fc0962bb9fc3383 Mon Sep 17 00:00:00 2001 From: Thejaswi Rao Date: Thu, 7 May 2020 01:08:53 -0700 Subject: [PATCH 38/56] FIX clang-tidy errors for handle_t --- cpp/include/raft/handle.hpp | 217 ++++++++++++++++++------------------ cpp/test/handle.cpp | 20 ++-- 2 files changed, 116 insertions(+), 121 deletions(-) diff --git a/cpp/include/raft/handle.hpp b/cpp/include/raft/handle.hpp index 7c8898fd96..381696abaf 100644 --- a/cpp/include/raft/handle.hpp +++ b/cpp/include/raft/handle.hpp @@ -45,7 +45,7 @@ namespace raft { */ class handle_t { private: - static constexpr int NumDefaultWorkerStreams = 0; + static constexpr int kNumDefaultWorkerStreams = 0; public: /** @@ -53,99 +53,93 @@ class handle_t { * * @param[in] n_streams number worker streams to be created */ - handle_t(int n_streams = NumDefaultWorkerStreams) - : _dev_id([]() -> int { + explicit handle_t(int n_streams = kNumDefaultWorkerStreams) + : dev_id_([]() -> int { int cur_dev = -1; CUDA_CHECK(cudaGetDevice(&cur_dev)); return cur_dev; }()), - _num_streams(n_streams), - _cublasInitialized(false), - _cusolverDnInitialized(false), - _cusolverSpInitialized(false), - _cusparseInitialized(false), - _deviceAllocator(std::make_shared()), - _hostAllocator(std::make_shared()), - _userStream(NULL), - _devicePropInitialized(false) { - createResources(); + num_streams_(n_streams), + device_allocator_(std::make_shared()), + host_allocator_(std::make_shared()) { + create_resources(); } /** Destroys all held-up resources */ - ~handle_t() { destroyResources(); } + ~handle_t() { destroy_resources(); } - int getDevice() const { return _dev_id; } + int get_device() const { return dev_id_; } - void setStream(cudaStream_t stream) { _userStream = stream; } - cudaStream_t getStream() const { return _userStream; } + void set_stream(cudaStream_t stream) { user_stream_ = stream; } + cudaStream_t get_stream() const { return user_stream_; } - void setDeviceAllocator(std::shared_ptr allocator) { - _deviceAllocator = allocator; + void set_device_allocator(std::shared_ptr allocator) { + device_allocator_ = allocator; } - std::shared_ptr getDeviceAllocator() const { - return _deviceAllocator; + std::shared_ptr get_device_allocator() const { + return device_allocator_; } - void setHostAllocator(std::shared_ptr allocator) { - _hostAllocator = allocator; + void set_host_allocator(std::shared_ptr allocator) { + host_allocator_ = allocator; } - std::shared_ptr getHostAllocator() const { - return _hostAllocator; + std::shared_ptr get_host_allocator() const { + return host_allocator_; } - cublasHandle_t getCublasHandle() const { - if (!_cublasInitialized) { - CUBLAS_CHECK(cublasCreate(&_cublas_handle)); - _cublasInitialized = true; + cublasHandle_t get_cublas_handle() const { + if (!cublas_initialized_) { + CUBLAS_CHECK(cublasCreate(&cublas_handle_)); + cublas_initialized_ = true; } - return _cublas_handle; + return cublas_handle_; } - cusolverDnHandle_t getcusolverDnHandle() const { - if (!_cusolverDnInitialized) { - CUSOLVER_CHECK(cusolverDnCreate(&_cusolverDn_handle)); - _cusolverDnInitialized = true; + cusolverDnHandle_t get_cusolver_dn_handle() const { + if (!cusolver_dn_initialized_) { + CUSOLVER_CHECK(cusolverDnCreate(&cusolver_dn_handle_)); + cusolver_dn_initialized_ = true; } - return _cusolverDn_handle; + return cusolver_dn_handle_; } - cusolverSpHandle_t getcusolverSpHandle() const { - if (!_cusolverSpInitialized) { - CUSOLVER_CHECK(cusolverSpCreate(&_cusolverSp_handle)); - _cusolverSpInitialized = true; + cusolverSpHandle_t get_cusolver_sp_handle() const { + if (!cusolver_sp_initialized_) { + CUSOLVER_CHECK(cusolverSpCreate(&cusolver_sp_handle_)); + cusolver_sp_initialized_ = true; } - return _cusolverSp_handle; + return cusolver_sp_handle_; } - cusparseHandle_t getcusparseHandle() const { - if (!_cusparseInitialized) { - CUSPARSE_CHECK(cusparseCreate(&_cusparse_handle)); - _cusparseInitialized = true; + cusparseHandle_t get_cusparse_handle() const { + if (!cusparse_initialized_) { + CUSPARSE_CHECK(cusparseCreate(&cusparse_handle_)); + cusparse_initialized_ = true; } - return _cusparse_handle; + return cusparse_handle_; } - cudaStream_t getInternalStream(int sid) const { return _streams[sid]; } - int getNumInternalStreams() const { return _num_streams; } - std::vector getInternalStreams() const { - std::vector int_streams_vec(_num_streams); - for (auto s : _streams) { + cudaStream_t get_internal_stream(int sid) const { return streams_[sid]; } + int get_num_internal_streams() const { return num_streams_; } + std::vector get_internal_streams() const { + std::vector int_streams_vec(num_streams_); + for (auto s : streams_) { int_streams_vec.push_back(s); } return int_streams_vec; } - void waitOnUserStream() const { - CUDA_CHECK(cudaEventRecord(_event, _userStream)); - for (auto s : _streams) { - CUDA_CHECK(cudaStreamWaitEvent(s, _event, 0)); + void wait_on_user_stream() const { + CUDA_CHECK(cudaEventRecord(event_, user_stream_)); + for (auto s : streams_) { + CUDA_CHECK(cudaStreamWaitEvent(s, event_, 0)); } } - void waitOnInternalStreams() const { - for (auto s : _streams) { - CUDA_CHECK(cudaEventRecord(_event, s)); - CUDA_CHECK(cudaStreamWaitEvent(_userStream, _event, 0)); + void wait_on_internal_streams() const { + for (auto s : streams_) { + CUDA_CHECK(cudaEventRecord(event_, s)); + CUDA_CHECK(cudaStreamWaitEvent(user_stream_, event_, 0)); } } @@ -155,90 +149,91 @@ class handle_t { // const MLCommon::cumlCommunicator& getCommunicator() const; // bool commsInitialized() const; - const cudaDeviceProp& getDeviceProperties() const { - if (!_devicePropInitialized) { - CUDA_CHECK(cudaGetDeviceProperties(&_prop, _dev_id)); - _devicePropInitialized = true; + const cudaDeviceProp& get_device_properties() const { + if (!device_prop_initialized_) { + CUDA_CHECK(cudaGetDeviceProperties(&prop_, dev_id_)); + device_prop_initialized_ = true; } - return _prop; + return prop_; } private: - const int _dev_id; - const int _num_streams; - std::vector _streams; - mutable cublasHandle_t _cublas_handle; - mutable bool _cublasInitialized; - mutable cusolverDnHandle_t _cusolverDn_handle; - mutable bool _cusolverDnInitialized; - mutable cusolverSpHandle_t _cusolverSp_handle; - mutable bool _cusolverSpInitialized; - mutable cusparseHandle_t _cusparse_handle; - mutable bool _cusparseInitialized; - std::shared_ptr _deviceAllocator; - std::shared_ptr _hostAllocator; - cudaStream_t _userStream; - cudaEvent_t _event; - mutable cudaDeviceProp _prop; - mutable bool _devicePropInitialized; + const int dev_id_; + const int num_streams_; + std::vector streams_; + mutable cublasHandle_t cublas_handle_; + mutable bool cublas_initialized_{false}; + mutable cusolverDnHandle_t cusolver_dn_handle_; + mutable bool cusolver_dn_initialized_{false}; + mutable cusolverSpHandle_t cusolver_sp_handle_; + mutable bool cusolver_sp_initialized_{false}; + mutable cusparseHandle_t cusparse_handle_; + mutable bool cusparse_initialized_{false}; + std::shared_ptr device_allocator_; + std::shared_ptr host_allocator_; + cudaStream_t user_stream_{nullptr}; + cudaEvent_t event_; + mutable cudaDeviceProp prop_; + mutable bool device_prop_initialized_{false}; ///@todo: enable this once we have migrated cuml-comms //std::shared_ptr _communicator; - void createResources() { + void create_resources() { cudaStream_t stream; CUDA_CHECK(cudaStreamCreateWithFlags(&stream, cudaStreamNonBlocking)); - _streams.push_back(stream); - for (int i = 1; i < _num_streams; ++i) { + streams_.push_back(stream); + for (int i = 1; i < num_streams_; ++i) { cudaStream_t stream; CUDA_CHECK(cudaStreamCreateWithFlags(&stream, cudaStreamNonBlocking)); - _streams.push_back(stream); + streams_.push_back(stream); } - CUDA_CHECK(cudaEventCreateWithFlags(&_event, cudaEventDisableTiming)); + CUDA_CHECK(cudaEventCreateWithFlags(&event_, cudaEventDisableTiming)); } - void destroyResources() { + void destroy_resources() { ///@todo: enable *_NO_THROW variants once we have enabled logging - if (_cusparseInitialized) { - //CUSPARSE_CHECK_NO_THROW(cusparseDestroy(_cusparse_handle)); - CUSPARSE_CHECK(cusparseDestroy(_cusparse_handle)); + if (cusparse_initialized_) { + //CUSPARSE_CHECK_NO_THROW(cusparseDestroy(cusparse_handle_)); + CUSPARSE_CHECK(cusparseDestroy(cusparse_handle_)); } - if (_cusolverDnInitialized) { - //CUSOLVER_CHECK_NO_THROW(cusolverDnDestroy(_cusolverDn_handle)); + if (cusolver_dn_initialized_) { + //CUSOLVER_CHECK_NO_THROW(cusolverDnDestroy(cusolver_dn_handle_)); + CUSOLVER_CHECK(cusolverDnDestroy(cusolver_dn_handle_)); } - if (_cusolverSpInitialized) { - //CUSOLVER_CHECK_NO_THROW(cusolverSpDestroy(_cusolverSp_handle)); - CUSOLVER_CHECK(cusolverSpDestroy(_cusolverSp_handle)); + if (cusolver_sp_initialized_) { + //CUSOLVER_CHECK_NO_THROW(cusolverSpDestroy(cusolver_sp_handle_)); + CUSOLVER_CHECK(cusolverSpDestroy(cusolver_sp_handle_)); } - if (_cublasInitialized) { - //CUBLAS_CHECK_NO_THROW(cublasDestroy(_cublas_handle)); - CUBLAS_CHECK(cublasDestroy(_cublas_handle)); + if (cublas_initialized_) { + //CUBLAS_CHECK_NO_THROW(cublasDestroy(cublas_handle_)); + CUBLAS_CHECK(cublasDestroy(cublas_handle_)); } - while (!_streams.empty()) { - //CUDA_CHECK_NO_THROW(cudaStreamDestroy(_streams.back())); - CUDA_CHECK(cudaStreamDestroy(_streams.back())); - _streams.pop_back(); + while (!streams_.empty()) { + //CUDA_CHECK_NO_THROW(cudaStreamDestroy(streams_.back())); + CUDA_CHECK(cudaStreamDestroy(streams_.back())); + streams_.pop_back(); } - //CUDA_CHECK_NO_THROW(cudaEventDestroy(_event)); - CUDA_CHECK(cudaEventDestroy(_event)); + //CUDA_CHECK_NO_THROW(cudaEventDestroy(event_)); + CUDA_CHECK(cudaEventDestroy(event_)); } }; // class handle_t /** * @brief RAII approach to synchronizing across all streams in the handle */ -class streamSyncer { +class stream_syncer { public: - streamSyncer(const handle_t& handle) : _handle(handle) { - _handle.waitOnUserStream(); + explicit stream_syncer(const handle_t& handle) : handle_(handle) { + handle_.wait_on_user_stream(); } - ~streamSyncer() { _handle.waitOnInternalStreams(); } + ~stream_syncer() { handle_.wait_on_internal_streams(); } - streamSyncer(const streamSyncer& other) = delete; - streamSyncer& operator=(const streamSyncer& other) = delete; + stream_syncer(const stream_syncer& other) = delete; + stream_syncer& operator=(const stream_syncer& other) = delete; private: - const handle_t& _handle; -}; // class streamSyncer + const handle_t& handle_; +}; // class stream_syncer } // namespace raft diff --git a/cpp/test/handle.cpp b/cpp/test/handle.cpp index 209caa9cc4..2c5280199d 100644 --- a/cpp/test/handle.cpp +++ b/cpp/test/handle.cpp @@ -23,22 +23,22 @@ namespace raft { TEST(Raft, HandleDefault) { handle_t h; - ASSERT_EQ(0, h.getNumInternalStreams()); - ASSERT_EQ(0, h.getDevice()); - ASSERT_EQ(nullptr, h.getStream()); - ASSERT_NE(nullptr, h.getCublasHandle()); - ASSERT_NE(nullptr, h.getcusolverDnHandle()); - ASSERT_NE(nullptr, h.getcusolverSpHandle()); - ASSERT_NE(nullptr, h.getcusparseHandle()); + ASSERT_EQ(0, h.get_num_internal_streams()); + ASSERT_EQ(0, h.get_device()); + ASSERT_EQ(nullptr, h.get_stream()); + ASSERT_NE(nullptr, h.get_cublas_handle()); + ASSERT_NE(nullptr, h.get_cusolver_dn_handle()); + ASSERT_NE(nullptr, h.get_cusolver_sp_handle()); + ASSERT_NE(nullptr, h.get_cusparse_handle()); } TEST(Raft, Handle) { handle_t h(4); - ASSERT_EQ(4, h.getNumInternalStreams()); + ASSERT_EQ(4, h.get_num_internal_streams()); cudaStream_t stream; CUDA_CHECK(cudaStreamCreate(&stream)); - h.setStream(stream); - ASSERT_EQ(stream, h.getStream()); + h.set_stream(stream); + ASSERT_EQ(stream, h.get_stream()); CUDA_CHECK(cudaStreamSynchronize(stream)); CUDA_CHECK(cudaStreamDestroy(stream)); } From 5eba7be8f10aa76c6b4052dc15ac7d7c64106330 Mon Sep 17 00:00:00 2001 From: Thejaswi Rao Date: Thu, 7 May 2020 08:09:22 +0000 Subject: [PATCH 39/56] FIX clang-format fixes --- cpp/include/raft/sparse/cusparse_wrappers.h | 21 ++++++++------------- 1 file changed, 8 insertions(+), 13 deletions(-) diff --git a/cpp/include/raft/sparse/cusparse_wrappers.h b/cpp/include/raft/sparse/cusparse_wrappers.h index 304b2bf0c3..1c63d2348b 100644 --- a/cpp/include/raft/sparse/cusparse_wrappers.h +++ b/cpp/include/raft/sparse/cusparse_wrappers.h @@ -115,15 +115,12 @@ inline void cusparsecoo2csr(cusparseHandle_t handle, const int* cooRowInd, */ template size_t cusparsecoosort_bufferSizeExt( // NOLINT - cusparseHandle_t handle, int m, int n, - int nnz, const T* cooRows, - const T* cooCols, cudaStream_t stream); + cusparseHandle_t handle, int m, int n, int nnz, const T* cooRows, + const T* cooCols, cudaStream_t stream); template <> inline size_t cusparsecoosort_bufferSizeExt( // NOLINT - cusparseHandle_t handle, int m, - int n, int nnz, const int* cooRows, - const int* cooCols, - cudaStream_t stream) { + cusparseHandle_t handle, int m, int n, int nnz, const int* cooRows, + const int* cooCols, cudaStream_t stream) { size_t val; CUSPARSE_CHECK(cusparseSetStream(handle, stream)); CUSPARSE_CHECK( @@ -133,14 +130,12 @@ inline size_t cusparsecoosort_bufferSizeExt( // NOLINT template void cusparsecoosortByRow( // NOLINT - cusparseHandle_t handle, int m, int n, int nnz, - T* cooRows, T* cooCols, T* P, void* pBuffer, - cudaStream_t stream); + cusparseHandle_t handle, int m, int n, int nnz, T* cooRows, T* cooCols, T* P, + void* pBuffer, cudaStream_t stream); template <> inline void cusparsecoosortByRow( // NOLINT - cusparseHandle_t handle, int m, int n, int nnz, - int* cooRows, int* cooCols, int* P, - void* pBuffer, cudaStream_t stream) { + cusparseHandle_t handle, int m, int n, int nnz, int* cooRows, int* cooCols, + int* P, void* pBuffer, cudaStream_t stream) { CUSPARSE_CHECK(cusparseSetStream(handle, stream)); CUSPARSE_CHECK( cusparseXcoosortByRow(handle, m, n, nnz, cooRows, cooCols, P, pBuffer)); From ad5db0f494c1127a38836d13dcb6f2b90f590e84 Mon Sep 17 00:00:00 2001 From: Thejaswi Rao Date: Thu, 7 May 2020 01:12:59 -0700 Subject: [PATCH 40/56] DOC dropped documentation url for clang-format/clang-tidy options --- cpp/.clang-format | 2 +- cpp/.clang-tidy | 2 ++ 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/cpp/.clang-format b/cpp/.clang-format index 2af7510304..779ca0033a 100644 --- a/cpp/.clang-format +++ b/cpp/.clang-format @@ -1,6 +1,6 @@ --- # Refer to the following link for the explanation of each params: -# http://releases.llvm.org/8.0.0/tools/clang/docs/ClangFormatStyleOptions.html +# http://releases.llvm.org/8.0.1/tools/clang/docs/ClangFormatStyleOptions.html Language: Cpp # BasedOnStyle: Google AccessModifierOffset: -1 diff --git a/cpp/.clang-tidy b/cpp/.clang-tidy index ee4cc6c24d..b845dcf27c 100644 --- a/cpp/.clang-tidy +++ b/cpp/.clang-tidy @@ -1,4 +1,6 @@ --- +# Refer to the following link for the explanation of each params: +# https://releases.llvm.org/8.0.1/tools/clang/tools/extra/docs/clang-tidy/checks/list.html Checks: 'clang-diagnostic-*,clang-analyzer-*,modernize-*,-modernize-make-*,-modernize-raw-string-literal,google-*,-google-default-arguments,-clang-diagnostic-#pragma-messages,readability-identifier-naming,-*,modernize-*,-modernize-make-*,-modernize-raw-string-literal,google-*,-google-default-arguments,-clang-diagnostic-#pragma-messages,readability-identifier-naming' WarningsAsErrors: '' HeaderFilterRegex: '' From deb02e6edefbd84eeb3abe775b8918c48de18f5a Mon Sep 17 00:00:00 2001 From: Thejaswi Rao Date: Sun, 10 May 2020 23:09:17 -0700 Subject: [PATCH 41/56] ENH added check for local variable naming as well --- cpp/.clang-tidy | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/cpp/.clang-tidy b/cpp/.clang-tidy index b845dcf27c..ee452f3422 100644 --- a/cpp/.clang-tidy +++ b/cpp/.clang-tidy @@ -116,6 +116,12 @@ CheckOptions: value: '' - key: readability-identifier-naming.IgnoreFailedSplit value: '0' + - key: readability-identifier-naming.LocalVariableCase + value: 'lower_case' + - key: readability-identifier-naming.LocalVariablePrefix + value: '' + - key: readability-identifier-naming.LocalVariableSuffix + value: '' - key: readability-identifier-naming.MemberCase value: lower_case - key: readability-identifier-naming.MemberPrefix From 1acb2b6d82d253e5bc47a303ca89a763a23f1813 Mon Sep 17 00:00:00 2001 From: Thejaswi Rao Date: Sun, 10 May 2020 23:13:15 -0700 Subject: [PATCH 42/56] ENH added check for constexpr variable as well --- cpp/.clang-tidy | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/cpp/.clang-tidy b/cpp/.clang-tidy index ee452f3422..2cbda104a4 100644 --- a/cpp/.clang-tidy +++ b/cpp/.clang-tidy @@ -122,6 +122,12 @@ CheckOptions: value: '' - key: readability-identifier-naming.LocalVariableSuffix value: '' + - key: readability-identifier-naming.ConstExprVariableCase + value: 'CamelCase' + - key: readability-identifier-naming.ConstExprVariablePrefix + value: 'k' + - key: readability-identifier-naming.ConstExprVariableSuffix + value: '' - key: readability-identifier-naming.MemberCase value: lower_case - key: readability-identifier-naming.MemberPrefix From 6126ab6e68b70538cb35d51388485187257c648b Mon Sep 17 00:00:00 2001 From: Thejaswi Rao Date: Sun, 10 May 2020 23:16:51 -0700 Subject: [PATCH 43/56] FIX all clang-tidy fixes --- cpp/include/raft/cudart_utils.h | 52 ++++++++++++++++----------------- 1 file changed, 26 insertions(+), 26 deletions(-) diff --git a/cpp/include/raft/cudart_utils.h b/cpp/include/raft/cudart_utils.h index ac31ca917a..ebfa358ec5 100644 --- a/cpp/include/raft/cudart_utils.h +++ b/cpp/include/raft/cudart_utils.h @@ -59,7 +59,7 @@ class exception : public std::exception { // Courtesy: https://www.gnu.org/software/libc/manual/html_node/Backtraces.html void collect_call_stack() noexcept { #ifdef __GNUC__ - const int kMaxStackDepth = 64; + constexpr int kMaxStackDepth = 64; void* stack[kMaxStackDepth]; // NOLINT auto depth = backtrace(stack, kMaxStackDepth); std::ostringstream oss; @@ -123,21 +123,21 @@ class exception : public std::exception { /** helper method to get max usable shared mem per block parameter */ inline int get_shared_memory_per_block() { - int devId; - CUDA_CHECK(cudaGetDevice(&devId)); - int smemPerBlk; - CUDA_CHECK(cudaDeviceGetAttribute(&smemPerBlk, - cudaDevAttrMaxSharedMemoryPerBlock, devId)); - return smemPerBlk; + int dev_id; + CUDA_CHECK(cudaGetDevice(&dev_id)); + int smem_per_blk; + CUDA_CHECK(cudaDeviceGetAttribute(&smem_per_blk, + cudaDevAttrMaxSharedMemoryPerBlock, dev_id)); + return smem_per_blk; } /** helper method to get multi-processor count parameter */ inline int get_multi_processor_count() { - int devId; - CUDA_CHECK(cudaGetDevice(&devId)); - int mpCount; + int dev_id; + CUDA_CHECK(cudaGetDevice(&dev_id)); + int mp_count; CUDA_CHECK( - cudaDeviceGetAttribute(&mpCount, cudaDevAttrMultiProcessorCount, devId)); - return mpCount; + cudaDeviceGetAttribute(&mp_count, cudaDevAttrMultiProcessorCount, dev_id)); + return mp_count; } /** @@ -162,22 +162,22 @@ void copy(Type* dst, const Type* src, size_t len, cudaStream_t stream) { */ /** performs a host to device copy */ template -void update_device(Type* dPtr, const Type* hPtr, size_t len, +void update_device(Type* d_ptr, const Type* h_ptr, size_t len, cudaStream_t stream) { - copy(dPtr, hPtr, len, stream); + copy(d_ptr, h_ptr, len, stream); } /** performs a device to host copy */ template -void update_host(Type* hPtr, const Type* dPtr, size_t len, +void update_host(Type* h_ptr, const Type* d_ptr, size_t len, cudaStream_t stream) { - copy(hPtr, dPtr, len, stream); + copy(h_ptr, d_ptr, len, stream); } template -void copy_async(Type* dPtr1, const Type* dPtr2, size_t len, +void copy_async(Type* d_ptr1, const Type* d_ptr2, size_t len, cudaStream_t stream) { - CUDA_CHECK(cudaMemcpyAsync(dPtr1, dPtr2, len * sizeof(Type), + CUDA_CHECK(cudaMemcpyAsync(d_ptr1, d_ptr2, len * sizeof(Type), cudaMemcpyDeviceToDevice, stream)); } /** @} */ @@ -187,24 +187,24 @@ void copy_async(Type* dPtr1, const Type* dPtr2, size_t len, * @{ */ template -void print_host_vector(const char* variableName, const T* hostMem, +void print_host_vector(const char* variable_name, const T* host_mem, size_t componentsCount, OutStream& out) { - out << variableName << "=["; + out << variable_name << "=["; for (size_t i = 0; i < componentsCount; ++i) { if (i != 0) out << ","; - out << hostMem[i]; + out << host_mem[i]; } out << "];\n"; } template -void print_device_vector(const char* variableName, const T* devMem, +void print_device_vector(const char* variable_name, const T* devMem, size_t componentsCount, OutStream& out) { - T* hostMem = new T[componentsCount]; - CUDA_CHECK(cudaMemcpy(hostMem, devMem, componentsCount * sizeof(T), + T* host_mem = new T[componentsCount]; + CUDA_CHECK(cudaMemcpy(host_mem, devMem, componentsCount * sizeof(T), cudaMemcpyDeviceToHost)); - print_host_vector(variableName, hostMem, componentsCount, out); - delete[] hostMem; + print_host_vector(variable_name, host_mem, componentsCount, out); + delete[] host_mem; } /** @} */ From ef9d0c2b661e59470fac540b18a5d4e26ff0d782 Mon Sep 17 00:00:00 2001 From: Thejaswi Rao Date: Sun, 10 May 2020 23:44:22 -0700 Subject: [PATCH 44/56] ENH added more explicit identifier-naming checks --- cpp/.clang-tidy | 66 ++++++++++++++++++++++++++++++++++++++++++------- 1 file changed, 57 insertions(+), 9 deletions(-) diff --git a/cpp/.clang-tidy b/cpp/.clang-tidy index 2cbda104a4..30d96069b0 100644 --- a/cpp/.clang-tidy +++ b/cpp/.clang-tidy @@ -80,28 +80,66 @@ CheckOptions: value: '0' - key: modernize-use-using.IgnoreMacros value: '1' + - key: readability-identifier-naming.AbstractClassCase + value: lower_case + - key: readability-identifier-naming.AbstractClassPrefix + value: '' + - key: readability-identifier-naming.AbstractClassSuffix + value: '' - key: readability-identifier-naming.ClassCase value: lower_case - key: readability-identifier-naming.ClassPrefix value: '' - key: readability-identifier-naming.ClassSuffix value: '' + - key: readability-identifier-naming.ClassConstantCase + value: CamelCase + - key: readability-identifier-naming.ClassConstantPrefix + value: 'k' + - key: readability-identifier-naming.ClassConstantSuffix + value: '' + - key: readability-identifier-naming.ClassMemberCase + value: lower_case + - key: readability-identifier-naming.ClassMemberPrefix + value: '' + - key: readability-identifier-naming.ClassMemberSuffix + value: '_' + - key: readability-identifier-naming.ClassMethodCase + value: lower_case + - key: readability-identifier-naming.ClassMethodPrefix + value: '' + - key: readability-identifier-naming.ClassMethodSuffix + value: '' + - key: readability-identifier-naming.ConstexprFunctionCase + value: lower_case + - key: readability-identifier-naming.ConstexprFunctionPrefix + value: '' + - key: readability-identifier-naming.ConstexprFunctionSuffix + value: '' + - key: readability-identifier-naming.ConstexprMethodCase + value: lower_case + - key: readability-identifier-naming.ConstexprMethodPrefix + value: '' + - key: readability-identifier-naming.ConstexprMethodSuffix + value: '' - key: readability-identifier-naming.ConstexprVariableCase value: CamelCase - key: readability-identifier-naming.ConstexprVariablePrefix - value: k + value: 'k' - key: readability-identifier-naming.ConstexprVariableSuffix value: '' - key: readability-identifier-naming.EnumCase value: CamelCase - - key: readability-identifier-naming.EnumConstantPrefix - value: k - - key: readability-identifier-naming.EnumConstantSuffix - value: '' - key: readability-identifier-naming.EnumPrefix value: '' - key: readability-identifier-naming.EnumSuffix value: '' + - key: readability-identifier-naming.EnumConstantCase + value: CamelCase + - key: readability-identifier-naming.EnumConstantPrefix + value: 'k' + - key: readability-identifier-naming.EnumConstantSuffix + value: '' - key: readability-identifier-naming.FunctionCase value: lower_case - key: readability-identifier-naming.FunctionPrefix @@ -111,7 +149,7 @@ CheckOptions: - key: readability-identifier-naming.GlobalConstantCase value: CamelCase - key: readability-identifier-naming.GlobalConstantPrefix - value: k + value: 'k' - key: readability-identifier-naming.GlobalConstantSuffix value: '' - key: readability-identifier-naming.IgnoreFailedSplit @@ -140,18 +178,22 @@ CheckOptions: value: '' - key: readability-identifier-naming.NamespaceSuffix value: '' + - key: readability-identifier-naming.PrivateMemberCase + value: lower_case - key: readability-identifier-naming.PrivateMemberPrefix value: '' - key: readability-identifier-naming.PrivateMemberSuffix - value: _ + value: '_' + - key: readability-identifier-naming.ProtectedMemberCase + value: lower_case - key: readability-identifier-naming.ProtectedMemberPrefix value: '' - key: readability-identifier-naming.ProtectedMemberSuffix - value: _ + value: '_' - key: readability-identifier-naming.StaticConstantCase value: CamelCase - key: readability-identifier-naming.StaticConstantPrefix - value: k + value: 'k' - key: readability-identifier-naming.StaticConstantSuffix value: '' - key: readability-identifier-naming.StructCase @@ -178,5 +220,11 @@ CheckOptions: value: '' - key: readability-identifier-naming.TypedefSuffix value: '' + - key: readability-identifier-naming.VariableCase + value: lower_case + - key: readability-identifier-naming.VariablePrefix + value: '' + - key: readability-identifier-naming.VariableSuffix + value: '' ... From 873a51fde6472865c0c2415c1d767d7c04a87fbd Mon Sep 17 00:00:00 2001 From: Thejaswi Rao Date: Sun, 10 May 2020 23:52:05 -0700 Subject: [PATCH 45/56] ENH added warp_size constexpr method --- cpp/include/raft/cudart_utils.h | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/cpp/include/raft/cudart_utils.h b/cpp/include/raft/cudart_utils.h index ebfa358ec5..0340086c41 100644 --- a/cpp/include/raft/cudart_utils.h +++ b/cpp/include/raft/cudart_utils.h @@ -140,6 +140,11 @@ inline int get_multi_processor_count() { return mp_count; } +/** Helper method to get to know warp size in device code */ +constexpr inline int warp_size() { + return 32; +} + /** * @brief Generic copy method for all kinds of transfers * @tparam Type data type From 73b75f519f033381deee0cf329d1bf4636e17985 Mon Sep 17 00:00:00 2001 From: Thejaswi Rao Date: Mon, 11 May 2020 00:00:12 -0700 Subject: [PATCH 46/56] FIX reduced code duplication in create_resources --- cpp/include/raft/handle.hpp | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/cpp/include/raft/handle.hpp b/cpp/include/raft/handle.hpp index 381696abaf..2ebed6eea2 100644 --- a/cpp/include/raft/handle.hpp +++ b/cpp/include/raft/handle.hpp @@ -180,10 +180,7 @@ class handle_t { //std::shared_ptr _communicator; void create_resources() { - cudaStream_t stream; - CUDA_CHECK(cudaStreamCreateWithFlags(&stream, cudaStreamNonBlocking)); - streams_.push_back(stream); - for (int i = 1; i < num_streams_; ++i) { + for (int i = 0; i < num_streams_; ++i) { cudaStream_t stream; CUDA_CHECK(cudaStreamCreateWithFlags(&stream, cudaStreamNonBlocking)); streams_.push_back(stream); From 93c1a9879006f00a7aa0dc561586cb3f0bcfc842 Mon Sep 17 00:00:00 2001 From: Thejaswi Rao Date: Mon, 11 May 2020 00:05:52 -0700 Subject: [PATCH 47/56] FIX apply critical section on various "get handle" methods --- cpp/include/raft/handle.hpp | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/cpp/include/raft/handle.hpp b/cpp/include/raft/handle.hpp index 2ebed6eea2..7fd00dc882 100644 --- a/cpp/include/raft/handle.hpp +++ b/cpp/include/raft/handle.hpp @@ -88,34 +88,42 @@ class handle_t { } cublasHandle_t get_cublas_handle() const { + mutex_.lock(); if (!cublas_initialized_) { CUBLAS_CHECK(cublasCreate(&cublas_handle_)); cublas_initialized_ = true; } + mutex_.unlock(); return cublas_handle_; } cusolverDnHandle_t get_cusolver_dn_handle() const { + mutex_.lock(); if (!cusolver_dn_initialized_) { CUSOLVER_CHECK(cusolverDnCreate(&cusolver_dn_handle_)); cusolver_dn_initialized_ = true; } + mutex_.unlock(); return cusolver_dn_handle_; } cusolverSpHandle_t get_cusolver_sp_handle() const { + mutex_.lock(); if (!cusolver_sp_initialized_) { CUSOLVER_CHECK(cusolverSpCreate(&cusolver_sp_handle_)); cusolver_sp_initialized_ = true; } + mutex_.unlock(); return cusolver_sp_handle_; } cusparseHandle_t get_cusparse_handle() const { + mutex_.lock(); if (!cusparse_initialized_) { CUSPARSE_CHECK(cusparseCreate(&cusparse_handle_)); cusparse_initialized_ = true; } + mutex_.unlock(); return cusparse_handle_; } @@ -150,10 +158,12 @@ class handle_t { // bool commsInitialized() const; const cudaDeviceProp& get_device_properties() const { + mutex_.lock(); if (!device_prop_initialized_) { CUDA_CHECK(cudaGetDeviceProperties(&prop_, dev_id_)); device_prop_initialized_ = true; } + mutex_.unlock(); return prop_; } @@ -175,6 +185,7 @@ class handle_t { cudaEvent_t event_; mutable cudaDeviceProp prop_; mutable bool device_prop_initialized_{false}; + mutable std::mutex mutex_; ///@todo: enable this once we have migrated cuml-comms //std::shared_ptr _communicator; From 4a55d57434aefdd4be3bf593704f3987f1a8ba6c Mon Sep 17 00:00:00 2001 From: Thejaswi Rao Date: Mon, 11 May 2020 00:32:02 -0700 Subject: [PATCH 48/56] ENH updated build to find rmm installation from conda-env --- cpp/CMakeLists.txt | 36 +++++++++++++++++++++++++++--------- 1 file changed, 27 insertions(+), 9 deletions(-) diff --git a/cpp/CMakeLists.txt b/cpp/CMakeLists.txt index f5696cf121..28b6aacbc2 100644 --- a/cpp/CMakeLists.txt +++ b/cpp/CMakeLists.txt @@ -172,6 +172,11 @@ set(RAFT_INCLUDE_DIRECTORIES ${RAFT_INCLUDE_DIR} ${CMAKE_CUDA_TOOLKIT_INCLUDE_DIRECTORIES}) +if(DEFINED ENV{CONDA_PREFIX}) + message(STATUS "Using RMM installation froM $ENV{CONDA_PREFIX}") + list(APPEND RAFT_INCLUDE_DIRECTORIES $ENV{CONDA_PREFIX}/include) +endif(DEFINED ENV{CONDA_PREFIX}) + ############################################################################## # - libraries ---------------------------------------------------------------- @@ -179,7 +184,14 @@ set(RAFT_LINK_LIBRARIES ${CUDA_cublas_LIBRARY} ${CUDA_cusolver_LIBRARY} ${CUDA_CUDART_LIBRARY} - ${CUDA_cusparse_LIBRARY}) + ${CUDA_cusparse_LIBRARY} + rmm) + +set(RAFT_LINK_DIRECTORIES "") + +if(DEFINED ENV{CONDA_PREFIX}) + list(APPEND RAFT_LINK_DIRECTORIES $ENV{CONDA_PREFIX}/lib) +endif(DEFINED ENV{CONDA_PREFIX}) ############################################################################## # - build test executable ---------------------------------------------------- @@ -195,16 +207,22 @@ if(BUILD_RAFT_TESTS) test/host_buffer.cpp test/test.cpp) - target_include_directories(test_raft PRIVATE - ${RAFT_INCLUDE_DIRECTORIES} - ${GTEST_DIR}/include) + target_include_directories(test_raft + PRIVATE + ${RAFT_INCLUDE_DIRECTORIES} + ${GTEST_DIR}/include) + + target_link_directories(test_raft + PRIVATE + ${RAFT_LINK_DIRECTORIES}) target_link_libraries(test_raft - ${RAFT_LINK_LIBRARIES} - gtestlib - gtest_mainlib - OpenMP::OpenMP_CXX - Threads::Threads) + PRIVATE + ${RAFT_LINK_LIBRARIES} + gtestlib + gtest_mainlib + OpenMP::OpenMP_CXX + Threads::Threads) endif(BUILD_RAFT_TESTS) ############################################################################## From 0ac33c70bff637b35972953308f3394a1e94f779 Mon Sep 17 00:00:00 2001 From: Thejaswi Rao Date: Mon, 11 May 2020 00:32:13 -0700 Subject: [PATCH 49/56] ENH made default allocator to use RMM --- cpp/include/raft/allocator.hpp | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/cpp/include/raft/allocator.hpp b/cpp/include/raft/allocator.hpp index 56d15bd6cb..ba39755791 100644 --- a/cpp/include/raft/allocator.hpp +++ b/cpp/include/raft/allocator.hpp @@ -18,6 +18,7 @@ #include #include "cudart_utils.h" +#include namespace raft { @@ -84,19 +85,16 @@ class device_allocator : public allocator {}; */ class host_allocator : public allocator {}; -/** Default cudaMalloc/cudaFree based device allocator */ +/** Default device allocator based on the one provided by RMM */ class default_device_allocator : public device_allocator { public: void* allocate(std::size_t n, cudaStream_t stream) override { - void* ptr = nullptr; - CUDA_CHECK(cudaMalloc(&ptr, n)); + void* ptr = rmm::mr::get_default_resource()->allocate(n, stream); return ptr; } void deallocate(void* p, std::size_t n, cudaStream_t stream) override { - ///@todo: enable this once logging is enabled in raft - //CUDA_CHECK_NO_THROW(cudaFree(p)); - CUDA_CHECK(cudaFree(p)); + rmm::mr::get_default_resource()->deallocate(p, n, stream); } }; // class default_device_allocator From 90c139accde7598069aee2a0d4f27c1eab74922f Mon Sep 17 00:00:00 2001 From: Thejaswi Rao Date: Mon, 11 May 2020 02:06:40 -0700 Subject: [PATCH 50/56] ENH moved all allocator and buffer logic under appropriate folders/namespaces for clarity --- cpp/include/raft/device_buffer.hpp | 45 ----------- cpp/include/raft/host_buffer.hpp | 73 ----------------- cpp/include/raft/{ => mr}/allocator.hpp | 63 ++------------- cpp/include/raft/{ => mr}/buffer_base.hpp | 27 ++++--- cpp/include/raft/mr/device/allocator.hpp | 94 ++++++++++++++++++++++ cpp/include/raft/mr/device/buffer.hpp | 70 ++++++++++++++++ cpp/include/raft/mr/host/allocator.hpp | 98 +++++++++++++++++++++++ cpp/include/raft/mr/host/buffer.hpp | 78 ++++++++++++++++++ 8 files changed, 360 insertions(+), 188 deletions(-) delete mode 100644 cpp/include/raft/device_buffer.hpp delete mode 100644 cpp/include/raft/host_buffer.hpp rename cpp/include/raft/{ => mr}/allocator.hpp (53%) rename cpp/include/raft/{ => mr}/buffer_base.hpp (89%) create mode 100644 cpp/include/raft/mr/device/allocator.hpp create mode 100644 cpp/include/raft/mr/device/buffer.hpp create mode 100644 cpp/include/raft/mr/host/allocator.hpp create mode 100644 cpp/include/raft/mr/host/buffer.hpp diff --git a/cpp/include/raft/device_buffer.hpp b/cpp/include/raft/device_buffer.hpp deleted file mode 100644 index 799518b25c..0000000000 --- a/cpp/include/raft/device_buffer.hpp +++ /dev/null @@ -1,45 +0,0 @@ -/* - * 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 "allocator.hpp" -#include "buffer_base.hpp" - -namespace raft { - -/** - * @brief RAII object owning a contiguous typed device buffer. The passed in - * allocator supports asynchronous allocation and deallocation so this - * can also be used for temporary memory - * - * @code{.cpp} - * template - * void foo(const raft_handle& h, ..., cudaStream_t stream) { - * ... - * device_buffer temp( h.getDeviceAllocator(), stream, 0 ); - * - * temp.resize(n, stream); - * kernelA<<>>(...,temp.data(),...); - * kernelB<<>>(...,temp.data(),...); - * temp.release(stream); - * } - * @endcode - */ -template -using device_buffer = buffer_base; - -} // namespace raft diff --git a/cpp/include/raft/host_buffer.hpp b/cpp/include/raft/host_buffer.hpp deleted file mode 100644 index b0d54d75b0..0000000000 --- a/cpp/include/raft/host_buffer.hpp +++ /dev/null @@ -1,73 +0,0 @@ -/* - * 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 "allocator.hpp" -#include "buffer_base.hpp" - -namespace raft { - -/** - * @brief RAII object owning a contigous typed host buffer (aka pinned memory). - * The passed in allocator supports asynchronus allocation and - * deallocation so this can also be used for temporary memory - * - * @code{.cpp} - * template - * void foo(const raft_handle& h, const T* in_d , T* out_d, ..., cudaStream_t stream ) { - * ... - * host_buffer temp( handle->getHostAllocator(), stream, 0 ) - * - * temp.resize(n, stream); - * cudaMemcpyAsync( temp.data(), in_d, temp.size()*sizeof(T), cudaMemcpyDeviceToHost ); - * ... - * cudaMemcpyAsync( out_d, temp.data(), temp.size()*sizeof(T), cudaMemcpyHostToDevice ); - * temp.release(stream); - * } - * @endcode - */ -template -class host_buffer : public buffer_base { - public: - using size_type = typename buffer_base::size_type; - using value_type = typename buffer_base::value_type; - using iterator = typename buffer_base::iterator; - using const_iterator = - typename buffer_base::const_iterator; - using reference = typename buffer_base::reference; - using const_reference = - typename buffer_base::const_reference; - - host_buffer() = delete; - - host_buffer(const host_buffer& other) = delete; - - host_buffer& operator=(const host_buffer& other) = delete; - - host_buffer(std::shared_ptr allocator, cudaStream_t stream, - size_type n = 0) - : buffer_base(allocator, stream, n) {} - - reference operator[](size_type pos) { return data_[pos]; } - - const_reference operator[](size_type pos) const { return data_[pos]; } - - private: - using buffer_base::data_; -}; - -} // namespace raft diff --git a/cpp/include/raft/allocator.hpp b/cpp/include/raft/mr/allocator.hpp similarity index 53% rename from cpp/include/raft/allocator.hpp rename to cpp/include/raft/mr/allocator.hpp index ba39755791..a636ae8eed 100644 --- a/cpp/include/raft/allocator.hpp +++ b/cpp/include/raft/mr/allocator.hpp @@ -16,11 +16,8 @@ #pragma once -#include -#include "cudart_utils.h" -#include - namespace raft { +namespace mr { /** * @brief Interface for an asynchronous device/host allocator. @@ -31,7 +28,7 @@ namespace raft { * @note This interface does NOT support RAII. Thus, if you need RAII-enabled * interface, better to use `device_buffer` or `host_buffer`. */ -class allocator { +class base_allocator { public: /** * @brief Asynchronously allocates a memory region. @@ -62,56 +59,8 @@ class allocator { */ virtual void deallocate(void* p, std::size_t n, cudaStream_t stream) = 0; - virtual ~allocator() = default; -}; // class Allocator - -/** - * @brief An explicit interface for an asynchronous device allocator. - * - * This is mostly done in order to reduce work needed in cuML codebase. - * An implementation of this interface can make the following assumptions, - * further to the ones listed in `Allocator`: - * - Allocations may be always on the device that was specified on construction. - */ -class device_allocator : public allocator {}; - -/** - * @brief An explicit interface for an asynchronous host allocations. - * - * This is mostly done in order to reduce work needed in cuML codebase. - * An implementation of this interface can make the following assumptions, - * further to the ones listed in `Allocator`: - * - Allocations don't need to be zero copy accessible form a device. - */ -class host_allocator : public allocator {}; - -/** Default device allocator based on the one provided by RMM */ -class default_device_allocator : public device_allocator { - public: - void* allocate(std::size_t n, cudaStream_t stream) override { - void* ptr = rmm::mr::get_default_resource()->allocate(n, stream); - return ptr; - } - - void deallocate(void* p, std::size_t n, cudaStream_t stream) override { - rmm::mr::get_default_resource()->deallocate(p, n, stream); - } -}; // class default_device_allocator - -/** Default cudaMallocHost/cudaFreeHost based host allocator */ -class default_host_allocator : public host_allocator { - public: - void* allocate(std::size_t n, cudaStream_t stream) override { - void* ptr = nullptr; - CUDA_CHECK(cudaMallocHost(&ptr, n)); - return ptr; - } - - void deallocate(void* p, std::size_t n, cudaStream_t stream) override { - ///@todo: enable this once logging is enabled in raft - //CUDA_CHECK_NO_THROW(cudaFreeHost(p)); - CUDA_CHECK(cudaFreeHost(p)); - } -}; // class default_host_allocator + virtual ~base_allocator() = default; +}; // class base_allocator -}; // end namespace raft +}; // namespace mr +}; // namespace raft diff --git a/cpp/include/raft/buffer_base.hpp b/cpp/include/raft/mr/buffer_base.hpp similarity index 89% rename from cpp/include/raft/buffer_base.hpp rename to cpp/include/raft/mr/buffer_base.hpp index c287b4b4d0..fe683cef36 100644 --- a/cpp/include/raft/buffer_base.hpp +++ b/cpp/include/raft/mr/buffer_base.hpp @@ -19,16 +19,20 @@ #include #include #include -#include "cudart_utils.h" +#include namespace raft { +namespace mr { /** * @brief Base for all RAII-based owning of temporary memory allocations. This * class should ideally not be used by users directly, but instead via * the child classes `device_buffer` and `host_buffer`. + * + * @tparam T data type + * @tparam AllocatorT The underly allocator object */ -template +template class buffer_base { public: using size_type = std::size_t; @@ -51,13 +55,12 @@ class buffer_base { * @param[in] stream cuda stream where this allocation operations are async * @param[in] n size of the buffer (in number of elements) */ - buffer_base(std::shared_ptr allocator, cudaStream_t stream, - size_type n = 0) + buffer_base(AllocatorT* allocator, cudaStream_t stream, size_type n = 0) { : size_(n), capacity_(n), data_(nullptr), stream_(stream), - allocator_(std::move(allocator)) { + allocator_(allocator) { if (capacity_ > 0) { data_ = static_cast( allocator_->allocate(capacity_ * sizeof(value_type), stream_)); @@ -97,16 +100,13 @@ class buffer_base { * @param[in] new_capacity new capacity (in number of elements) * @param[in] stream cuda stream where allocation operations are queued */ - void reserve(const size_type new_capacity, cudaStream_t stream) { + void reserve(size_type new_capacity, cudaStream_t stream) { set_stream(stream); if (new_capacity > capacity_) { auto* new_data = static_cast( allocator_->allocate(new_capacity * sizeof(value_type), stream_)); if (size_ > 0) { - CUDA_CHECK(cudaMemcpyAsync(new_data, data_, size_ * sizeof(value_type), - cudaMemcpyDefault, stream_)); - } - if (nullptr != data_) { + raft::copy(new_data, data_t, size_, stream_); allocator_->deallocate(data_, capacity_ * sizeof(value_type), stream_); } data_ = new_data; @@ -147,7 +147,7 @@ class buffer_base { * * @return the allocator pointer */ - std::shared_ptr get_allocator() const { return allocator_; } + AllocatorT* get_allocator() const { return allocator_; } protected: value_type* data_; @@ -156,7 +156,7 @@ class buffer_base { size_type size_; size_type capacity_; cudaStream_t stream_; - std::shared_ptr allocator_; + AllocatorT* allocator_; /** * @brief Sets a new cuda stream where the future operations will be queued @@ -181,4 +181,5 @@ class buffer_base { } }; // class buffer_base -} // namespace raft +}; // namespace mr +}; // namespace raft diff --git a/cpp/include/raft/mr/device/allocator.hpp b/cpp/include/raft/mr/device/allocator.hpp new file mode 100644 index 0000000000..9acd3986c1 --- /dev/null +++ b/cpp/include/raft/mr/device/allocator.hpp @@ -0,0 +1,94 @@ +/* + * 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 + +namespace raft { +namespace mr { +namespace device { + +/** + * @brief An explicit interface for an asynchronous device allocator. + * + * This is mostly done in order to reduce work needed in cuML codebase. + * An implementation of this interface can make the following assumptions, + * further to the ones listed in `Allocator`: + * - Allocations may be always on the device that was specified on construction. + */ +class allocator : public base_allocator {}; + +/** Default device allocator based on the one provided by RMM */ +class default_allocator : public allocator { + public: + void* allocate(std::size_t n, cudaStream_t stream) override { + void* ptr = rmm::mr::get_default_resource()->allocate(n, stream); + return ptr; + } + + void deallocate(void* p, std::size_t n, cudaStream_t stream) override { + rmm::mr::get_default_resource()->deallocate(p, n, stream); + } +}; // class default_allocator + +namespace { + +allocator* get_default_impl() { + static default_allocator obj; + return &obj; +} + +std::atomic& get_default() { + static std::atomic alloc{get_default_impl()}; + return alloc; +} + +} // namespace + +/** + * @brief Gets the default device allocator + * + * This is thread-safe + * + * @return the allocator object + */ +allocator* get_default_allocator() { + return get_default().load(); +} + +/** + * @brief Sets the new default device allocator + * + * This is thread-safe + * + * @param[in] new_allocator the new device allocator that will be the default + * If a nullptr is passed, the default allocator will + * be reset to the one based on `default_allocator` + * @return the old allocator + */ +allocator* set_default_allocator(allocator* new_allocator) { + if (new_allocator == nullptr) { + new_allocator = get_default(); + } + return get_default().exchange(new_allocator); +} + +}; // namespace device +}; // namespace mr +}; // namespace raft diff --git a/cpp/include/raft/mr/device/buffer.hpp b/cpp/include/raft/mr/device/buffer.hpp new file mode 100644 index 0000000000..2ec0dfb508 --- /dev/null +++ b/cpp/include/raft/mr/device/buffer.hpp @@ -0,0 +1,70 @@ +/* + * 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 "allocator.hpp" +#include + +namespace raft { +namespace mr { +namespace device { + +/** + * @brief RAII object owning a contiguous typed device buffer. The passed in + * allocator supports asynchronous allocation and deallocation so this + * can also be used for temporary memory + * + * @code{.cpp} + * template + * void foo(..., cudaStream_t stream) { + * ... + * raft::mr::device::buffer temp(stream, 0); + * ... + * temp.resize(n); + * kernelA<<>>(...,temp.data(),...); + * kernelB<<>>(...,temp.data(),...); + * temp.release(); + * ... + * } + * @endcode + */ +template +class buffer : public buffer_base { + public: + using size_type = typename buffer_base::size_type; + using value_type = typename buffer_base::value_type; + using iterator = typename buffer_base::iterator; + using const_iterator = + typename buffer_base::const_iterator; + using reference = typename buffer_base::reference; + using const_reference = + typename buffer_base::const_reference; + + buffer() = delete; + + buffer(const buffer& other) = delete; + + buffer& operator=(const buffer& other) = delete; + + buffer(cudaStream_t stream, size_type n = 0, + allocator* alloc = get_default_allocator()) + : buffer_base(alloc, stream, n) {} +}; // class buffer + +}; // namespace device +}; // namespace mr +}; // namespace raft diff --git a/cpp/include/raft/mr/host/allocator.hpp b/cpp/include/raft/mr/host/allocator.hpp new file mode 100644 index 0000000000..ddd313b152 --- /dev/null +++ b/cpp/include/raft/mr/host/allocator.hpp @@ -0,0 +1,98 @@ +/* + * 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 + +namespace raft { +namespace mr { +namespace host { + +/** + * @brief An explicit interface for an asynchronous host allocations. + * + * This is mostly done in order to reduce work needed in cuML codebase. + * An implementation of this interface can make the following assumptions, + * further to the ones listed in `Allocator`: + * - Allocations don't need to be zero copy accessible form a device. + */ +class allocator : public base_allocator {}; + +/** Default cudaMallocHost/cudaFreeHost based host allocator */ +class default_allocator : public allocator { + public: + void* allocate(std::size_t n, cudaStream_t stream) override { + void* ptr = nullptr; + CUDA_CHECK(cudaMallocHost(&ptr, n)); + return ptr; + } + + void deallocate(void* p, std::size_t n, cudaStream_t stream) override { + ///@todo: enable this once logging is enabled in raft + //CUDA_CHECK_NO_THROW(cudaFreeHost(p)); + CUDA_CHECK(cudaFreeHost(p)); + } +}; // class default_allocator + +namespace { + +allocator* get_default_impl() { + static default_allocator obj; + return &obj; +} + +std::atomic& get_default() { + static std::atomic alloc{get_default_impl()}; + return alloc; +} + +} // namespace + +/** + * @brief Gets the default host allocator + * + * This is thread-safe + * + * @return the allocator object + */ +allocator* get_default_allocator() { + return get_default().load(); +} + +/** + * @brief Sets the new default host allocator + * + * This is thread-safe + * + * @param[in] new_allocator the new host allocator that will be the default + * If a nullptr is passed, the default allocator will + * be reset to the one based on `default_allocator` + * @return the old allocator + */ +allocator* set_default_allocator(allocator* new_allocator) { + if (new_allocator == nullptr) { + new_allocator = get_default(); + } + return get_default().exchange(new_allocator); +} + +}; // namespace host +}; // namespace mr +}; // namespace raft diff --git a/cpp/include/raft/mr/host/buffer.hpp b/cpp/include/raft/mr/host/buffer.hpp new file mode 100644 index 0000000000..e3f5dc07f8 --- /dev/null +++ b/cpp/include/raft/mr/host/buffer.hpp @@ -0,0 +1,78 @@ +/* + * 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 "allocator.hpp" +#include + +namespace raft { +namespace mr { +namespace host { + +/** + * @brief RAII object owning a contigous typed host buffer (aka pinned memory). + * The passed in allocator supports asynchronus allocation and + * deallocation so this can also be used for temporary memory + * + * @code{.cpp} + * template + * void foo(const T* in_d , T* out_d, ..., cudaStream_t stream) { + * ... + * raft::mr::host::buffer temp(stream, 0); + * ... + * temp.resize(n); + * raft::copy(temp.data(), in_d, temp.size()); + * ... + * raft::copy(out_d, temp.data(), temp.size()); + * temp.release(stream); + * ... + * } + * @endcode + */ +template +class buffer : public buffer_base { + public: + using size_type = typename buffer_base::size_type; + using value_type = typename buffer_base::value_type; + using iterator = typename buffer_base::iterator; + using const_iterator = + typename buffer_base::const_iterator; + using reference = typename buffer_base::reference; + using const_reference = + typename buffer_base::const_reference; + + buffer() = delete; + + buffer(const buffer& other) = delete; + + buffer& operator=(const buffer& other) = delete; + + buffer(cudaStream_t stream, size_type n = 0, + allocator* alloc = get_default_allocator()) + : buffer_base(stream, n, alloc) {} + + reference operator[](size_type pos) { return data_[pos]; } + + const_reference operator[](size_type pos) const { return data_[pos]; } + + private: + using buffer_base::data_; +}; + +}; // namespace host +}; // namespace mr +}; // namespace raft From e09393ffd3da2144339d9259bff9034bffe4869a Mon Sep 17 00:00:00 2001 From: Thejaswi Rao Date: Mon, 11 May 2020 03:03:44 -0700 Subject: [PATCH 51/56] ENH updated tests and other interfaces to the new mr design --- cpp/CMakeLists.txt | 4 +- cpp/include/raft/handle.hpp | 19 ++++---- cpp/include/raft/mr/allocator.hpp | 2 + cpp/include/raft/mr/buffer_base.hpp | 11 ++--- cpp/include/raft/mr/device/allocator.hpp | 43 ------------------- cpp/include/raft/mr/device/buffer.hpp | 4 +- cpp/include/raft/mr/host/allocator.hpp | 43 ------------------- cpp/include/raft/mr/host/buffer.hpp | 7 +-- .../device/buffer.cpp} | 10 +++-- .../{host_buffer.cpp => mr/host/buffer.cpp} | 10 +++-- 10 files changed, 40 insertions(+), 113 deletions(-) rename cpp/test/{device_buffer.cpp => mr/device/buffer.cpp} (85%) rename cpp/test/{host_buffer.cpp => mr/host/buffer.cpp} (86%) diff --git a/cpp/CMakeLists.txt b/cpp/CMakeLists.txt index 28b6aacbc2..947d0318cb 100644 --- a/cpp/CMakeLists.txt +++ b/cpp/CMakeLists.txt @@ -202,9 +202,9 @@ if(BUILD_RAFT_TESTS) # keep the files in alphabetical order! add_executable(test_raft test/cudart_utils.cpp - test/device_buffer.cpp test/handle.cpp - test/host_buffer.cpp + test/mr/device/buffer.cpp + test/mr/host/buffer.cpp test/test.cpp) target_include_directories(test_raft diff --git a/cpp/include/raft/handle.hpp b/cpp/include/raft/handle.hpp index 7fd00dc882..017de5f221 100644 --- a/cpp/include/raft/handle.hpp +++ b/cpp/include/raft/handle.hpp @@ -33,8 +33,9 @@ #include #include +#include +#include #include -#include "allocator.hpp" #include "cudart_utils.h" namespace raft { @@ -60,8 +61,8 @@ class handle_t { return cur_dev; }()), num_streams_(n_streams), - device_allocator_(std::make_shared()), - host_allocator_(std::make_shared()) { + device_allocator_(std::make_shared()), + host_allocator_(std::make_shared()) { create_resources(); } @@ -73,17 +74,17 @@ class handle_t { void set_stream(cudaStream_t stream) { user_stream_ = stream; } cudaStream_t get_stream() const { return user_stream_; } - void set_device_allocator(std::shared_ptr allocator) { + void set_device_allocator(std::shared_ptr allocator) { device_allocator_ = allocator; } - std::shared_ptr get_device_allocator() const { + std::shared_ptr get_device_allocator() const { return device_allocator_; } - void set_host_allocator(std::shared_ptr allocator) { + void set_host_allocator(std::shared_ptr allocator) { host_allocator_ = allocator; } - std::shared_ptr get_host_allocator() const { + std::shared_ptr get_host_allocator() const { return host_allocator_; } @@ -179,8 +180,8 @@ class handle_t { mutable bool cusolver_sp_initialized_{false}; mutable cusparseHandle_t cusparse_handle_; mutable bool cusparse_initialized_{false}; - std::shared_ptr device_allocator_; - std::shared_ptr host_allocator_; + std::shared_ptr device_allocator_; + std::shared_ptr host_allocator_; cudaStream_t user_stream_{nullptr}; cudaEvent_t event_; mutable cudaDeviceProp prop_; diff --git a/cpp/include/raft/mr/allocator.hpp b/cpp/include/raft/mr/allocator.hpp index a636ae8eed..707b71d468 100644 --- a/cpp/include/raft/mr/allocator.hpp +++ b/cpp/include/raft/mr/allocator.hpp @@ -16,6 +16,8 @@ #pragma once +#include + namespace raft { namespace mr { diff --git a/cpp/include/raft/mr/buffer_base.hpp b/cpp/include/raft/mr/buffer_base.hpp index fe683cef36..e04bd6d04b 100644 --- a/cpp/include/raft/mr/buffer_base.hpp +++ b/cpp/include/raft/mr/buffer_base.hpp @@ -55,12 +55,13 @@ class buffer_base { * @param[in] stream cuda stream where this allocation operations are async * @param[in] n size of the buffer (in number of elements) */ - buffer_base(AllocatorT* allocator, cudaStream_t stream, size_type n = 0) { + buffer_base(std::shared_ptr allocator, cudaStream_t stream, + size_type n = 0) : size_(n), capacity_(n), data_(nullptr), stream_(stream), - allocator_(allocator) { + allocator_(std::move(allocator)) { if (capacity_ > 0) { data_ = static_cast( allocator_->allocate(capacity_ * sizeof(value_type), stream_)); @@ -106,7 +107,7 @@ class buffer_base { auto* new_data = static_cast( allocator_->allocate(new_capacity * sizeof(value_type), stream_)); if (size_ > 0) { - raft::copy(new_data, data_t, size_, stream_); + raft::copy(new_data, data_, size_, stream_); allocator_->deallocate(data_, capacity_ * sizeof(value_type), stream_); } data_ = new_data; @@ -147,7 +148,7 @@ class buffer_base { * * @return the allocator pointer */ - AllocatorT* get_allocator() const { return allocator_; } + std::shared_ptr get_allocator() const { return allocator_; } protected: value_type* data_; @@ -156,7 +157,7 @@ class buffer_base { size_type size_; size_type capacity_; cudaStream_t stream_; - AllocatorT* allocator_; + std::shared_ptr allocator_; /** * @brief Sets a new cuda stream where the future operations will be queued diff --git a/cpp/include/raft/mr/device/allocator.hpp b/cpp/include/raft/mr/device/allocator.hpp index 9acd3986c1..be6ea6fc67 100644 --- a/cpp/include/raft/mr/device/allocator.hpp +++ b/cpp/include/raft/mr/device/allocator.hpp @@ -18,7 +18,6 @@ #include #include -#include namespace raft { namespace mr { @@ -47,48 +46,6 @@ class default_allocator : public allocator { } }; // class default_allocator -namespace { - -allocator* get_default_impl() { - static default_allocator obj; - return &obj; -} - -std::atomic& get_default() { - static std::atomic alloc{get_default_impl()}; - return alloc; -} - -} // namespace - -/** - * @brief Gets the default device allocator - * - * This is thread-safe - * - * @return the allocator object - */ -allocator* get_default_allocator() { - return get_default().load(); -} - -/** - * @brief Sets the new default device allocator - * - * This is thread-safe - * - * @param[in] new_allocator the new device allocator that will be the default - * If a nullptr is passed, the default allocator will - * be reset to the one based on `default_allocator` - * @return the old allocator - */ -allocator* set_default_allocator(allocator* new_allocator) { - if (new_allocator == nullptr) { - new_allocator = get_default(); - } - return get_default().exchange(new_allocator); -} - }; // namespace device }; // namespace mr }; // namespace raft diff --git a/cpp/include/raft/mr/device/buffer.hpp b/cpp/include/raft/mr/device/buffer.hpp index 2ec0dfb508..902531b366 100644 --- a/cpp/include/raft/mr/device/buffer.hpp +++ b/cpp/include/raft/mr/device/buffer.hpp @@ -16,6 +16,7 @@ #pragma once +#include #include "allocator.hpp" #include @@ -60,8 +61,7 @@ class buffer : public buffer_base { buffer& operator=(const buffer& other) = delete; - buffer(cudaStream_t stream, size_type n = 0, - allocator* alloc = get_default_allocator()) + buffer(std::shared_ptr alloc, cudaStream_t stream, size_type n = 0) : buffer_base(alloc, stream, n) {} }; // class buffer diff --git a/cpp/include/raft/mr/host/allocator.hpp b/cpp/include/raft/mr/host/allocator.hpp index ddd313b152..92a57150e0 100644 --- a/cpp/include/raft/mr/host/allocator.hpp +++ b/cpp/include/raft/mr/host/allocator.hpp @@ -19,7 +19,6 @@ #include #include #include -#include namespace raft { namespace mr { @@ -51,48 +50,6 @@ class default_allocator : public allocator { } }; // class default_allocator -namespace { - -allocator* get_default_impl() { - static default_allocator obj; - return &obj; -} - -std::atomic& get_default() { - static std::atomic alloc{get_default_impl()}; - return alloc; -} - -} // namespace - -/** - * @brief Gets the default host allocator - * - * This is thread-safe - * - * @return the allocator object - */ -allocator* get_default_allocator() { - return get_default().load(); -} - -/** - * @brief Sets the new default host allocator - * - * This is thread-safe - * - * @param[in] new_allocator the new host allocator that will be the default - * If a nullptr is passed, the default allocator will - * be reset to the one based on `default_allocator` - * @return the old allocator - */ -allocator* set_default_allocator(allocator* new_allocator) { - if (new_allocator == nullptr) { - new_allocator = get_default(); - } - return get_default().exchange(new_allocator); -} - }; // namespace host }; // namespace mr }; // namespace raft diff --git a/cpp/include/raft/mr/host/buffer.hpp b/cpp/include/raft/mr/host/buffer.hpp index e3f5dc07f8..c00be165dc 100644 --- a/cpp/include/raft/mr/host/buffer.hpp +++ b/cpp/include/raft/mr/host/buffer.hpp @@ -16,6 +16,7 @@ #pragma once +#include #include "allocator.hpp" #include @@ -61,9 +62,9 @@ class buffer : public buffer_base { buffer& operator=(const buffer& other) = delete; - buffer(cudaStream_t stream, size_type n = 0, - allocator* alloc = get_default_allocator()) - : buffer_base(stream, n, alloc) {} + buffer(std::shared_ptr alloc, cudaStream_t stream, + size_type n = 0) + : buffer_base(alloc, stream, n) {} reference operator[](size_type pos) { return data_[pos]; } diff --git a/cpp/test/device_buffer.cpp b/cpp/test/mr/device/buffer.cpp similarity index 85% rename from cpp/test/device_buffer.cpp rename to cpp/test/mr/device/buffer.cpp index 0600e26385..da3c7dc1e2 100644 --- a/cpp/test/device_buffer.cpp +++ b/cpp/test/mr/device/buffer.cpp @@ -17,16 +17,18 @@ #include #include #include -#include +#include namespace raft { +namespace mr { +namespace device { TEST(Raft, DeviceBuffer) { - auto allocator = std::make_shared(); + auto alloc = std::make_shared(); cudaStream_t stream; CUDA_CHECK(cudaStreamCreate(&stream)); // no allocation at construction - device_buffer buff(allocator, stream); + buffer buff(alloc, stream); ASSERT_EQ(0, buff.size()); // explicit allocation after construction buff.resize(20, stream); @@ -41,4 +43,6 @@ TEST(Raft, DeviceBuffer) { CUDA_CHECK(cudaStreamDestroy(stream)); } +} // namespace device +} // namespace mr } // namespace raft diff --git a/cpp/test/host_buffer.cpp b/cpp/test/mr/host/buffer.cpp similarity index 86% rename from cpp/test/host_buffer.cpp rename to cpp/test/mr/host/buffer.cpp index 8c398e5145..dd5356d260 100644 --- a/cpp/test/host_buffer.cpp +++ b/cpp/test/mr/host/buffer.cpp @@ -17,16 +17,18 @@ #include #include #include -#include +#include namespace raft { +namespace mr { +namespace host { TEST(Raft, HostBuffer) { - auto allocator = std::make_shared(); + auto alloc = std::make_shared(); cudaStream_t stream; CUDA_CHECK(cudaStreamCreate(&stream)); // no allocation at construction - host_buffer buff(allocator, stream); + buffer buff(alloc, stream); ASSERT_EQ(0, buff.size()); // explicit allocation after construction buff.resize(20, stream); @@ -41,4 +43,6 @@ TEST(Raft, HostBuffer) { CUDA_CHECK(cudaStreamDestroy(stream)); } +} // namespace host +} // namespace mr } // namespace raft From 7a5b15f2d0b11869ae949b377912457543fe24cd Mon Sep 17 00:00:00 2001 From: Thejaswi Rao Date: Mon, 11 May 2020 03:11:31 -0700 Subject: [PATCH 52/56] allow users to call buffer operations without an explicit stream parameter --- cpp/include/raft/mr/buffer_base.hpp | 31 +++++++++++++++++++++++------ cpp/test/mr/device/buffer.cpp | 7 +++++++ cpp/test/mr/host/buffer.cpp | 7 +++++++ 3 files changed, 39 insertions(+), 6 deletions(-) diff --git a/cpp/include/raft/mr/buffer_base.hpp b/cpp/include/raft/mr/buffer_base.hpp index e04bd6d04b..64eef69aa0 100644 --- a/cpp/include/raft/mr/buffer_base.hpp +++ b/cpp/include/raft/mr/buffer_base.hpp @@ -100,9 +100,9 @@ class buffer_base { * * @param[in] new_capacity new capacity (in number of elements) * @param[in] stream cuda stream where allocation operations are queued + * @{ */ - void reserve(size_type new_capacity, cudaStream_t stream) { - set_stream(stream); + void reserve(size_type new_capacity) { if (new_capacity > capacity_) { auto* new_data = static_cast( allocator_->allocate(new_capacity * sizeof(value_type), stream_)); @@ -115,26 +115,39 @@ class buffer_base { } } + void reserve(size_type new_capacity, cudaStream_t stream) { + set_stream(stream); + reserve(new_capacity); + } + /** @} */ + /** * @brief Resize the underlying buffer (uses `reserve` method internally) * * @param[in] new_size new buffer size * @param[in] stream cuda stream where the work will be queued + * @{ */ - void resize(const size_type new_size, cudaStream_t stream) { - reserve(new_size, stream); + void resize(const size_type new_size) { + reserve(new_size); size_ = new_size; } + void resize(const size_type new_size, cudaStream_t stream) { + set_stream(stream); + resize(new_size); + } + /** @} */ + /** * @brief Deletes the underlying buffer * * If this method is not explicitly called, it will be during the destructor * * @param[in] stream cuda stream where the work will be queued + * @{ */ - void release(cudaStream_t stream) { - set_stream(stream); + void release() { if (nullptr != data_) { allocator_->deallocate(data_, capacity_ * sizeof(value_type), stream_); } @@ -143,6 +156,12 @@ class buffer_base { size_ = 0; } + void release(cudaStream_t stream) { + set_stream(stream); + release(); + } + /** @} */ + /** * @brief returns the underlying allocator used * diff --git a/cpp/test/mr/device/buffer.cpp b/cpp/test/mr/device/buffer.cpp index da3c7dc1e2..86aee43ce3 100644 --- a/cpp/test/mr/device/buffer.cpp +++ b/cpp/test/mr/device/buffer.cpp @@ -39,6 +39,13 @@ TEST(Raft, DeviceBuffer) { // explicit deallocation buff.release(stream); ASSERT_EQ(0, buff.size()); + // use these methods without the explicit stream parameter + buff.resize(20); + ASSERT_EQ(20, buff.size()); + buff.resize(10); + ASSERT_EQ(10, buff.size()); + buff.release(); + ASSERT_EQ(0, buff.size()); CUDA_CHECK(cudaStreamSynchronize(stream)); CUDA_CHECK(cudaStreamDestroy(stream)); } diff --git a/cpp/test/mr/host/buffer.cpp b/cpp/test/mr/host/buffer.cpp index dd5356d260..80b759fd74 100644 --- a/cpp/test/mr/host/buffer.cpp +++ b/cpp/test/mr/host/buffer.cpp @@ -39,6 +39,13 @@ TEST(Raft, HostBuffer) { // explicit deallocation buff.release(stream); ASSERT_EQ(0, buff.size()); + // use these methods without the explicit stream parameter + buff.resize(20); + ASSERT_EQ(20, buff.size()); + buff.resize(10); + ASSERT_EQ(10, buff.size()); + buff.release(); + ASSERT_EQ(0, buff.size()); CUDA_CHECK(cudaStreamSynchronize(stream)); CUDA_CHECK(cudaStreamDestroy(stream)); } From 58c490c4bfaf4224d429f6b853217e43852eed68 Mon Sep 17 00:00:00 2001 From: Thejaswi Rao Date: Mon, 11 May 2020 03:21:42 -0700 Subject: [PATCH 53/56] ENH added support for constructing host buffer from device buffer --- cpp/include/raft/mr/buffer_base.hpp | 11 ++++++++--- cpp/include/raft/mr/host/buffer.hpp | 9 +++++++++ cpp/test/mr/host/buffer.cpp | 15 +++++++++++++++ 3 files changed, 32 insertions(+), 3 deletions(-) diff --git a/cpp/include/raft/mr/buffer_base.hpp b/cpp/include/raft/mr/buffer_base.hpp index 64eef69aa0..74729134b9 100644 --- a/cpp/include/raft/mr/buffer_base.hpp +++ b/cpp/include/raft/mr/buffer_base.hpp @@ -70,9 +70,7 @@ class buffer_base { } ~buffer_base() { - if (nullptr != data_) { - allocator_->deallocate(data_, capacity_ * sizeof(value_type), stream_); - } + release(); } value_type* data() { return data_; } @@ -169,6 +167,13 @@ class buffer_base { */ std::shared_ptr get_allocator() const { return allocator_; } + /** + * @brief returns the underlying stream used + * + * @return the cuda stream + */ + cudaStream_t get_stream() const { return stream_; } + protected: value_type* data_; diff --git a/cpp/include/raft/mr/host/buffer.hpp b/cpp/include/raft/mr/host/buffer.hpp index c00be165dc..6a91e55d51 100644 --- a/cpp/include/raft/mr/host/buffer.hpp +++ b/cpp/include/raft/mr/host/buffer.hpp @@ -19,6 +19,8 @@ #include #include "allocator.hpp" #include +#include +#include namespace raft { namespace mr { @@ -62,6 +64,13 @@ class buffer : public buffer_base { buffer& operator=(const buffer& other) = delete; + buffer(std::shared_ptr alloc, const device::buffer& other) + : buffer_base(alloc, other.get_stream(), other.size()) { + if (other.size() > 0) { + raft::copy(data_, other.data(), other.size(), other.get_stream()); + } + } + buffer(std::shared_ptr alloc, cudaStream_t stream, size_type n = 0) : buffer_base(alloc, stream, n) {} diff --git a/cpp/test/mr/host/buffer.cpp b/cpp/test/mr/host/buffer.cpp index 80b759fd74..fcbdcccf20 100644 --- a/cpp/test/mr/host/buffer.cpp +++ b/cpp/test/mr/host/buffer.cpp @@ -18,6 +18,7 @@ #include #include #include +#include namespace raft { namespace mr { @@ -50,6 +51,20 @@ TEST(Raft, HostBuffer) { CUDA_CHECK(cudaStreamDestroy(stream)); } +TEST(Raft, DeviceToHostBuffer) { + auto d_alloc = std::make_shared(); + auto h_alloc = std::make_shared(); + cudaStream_t stream; + CUDA_CHECK(cudaStreamCreate(&stream)); + device::buffer d_buff(d_alloc, stream, 32); + CUDA_CHECK(cudaMemsetAsync(d_buff.data(), 0, sizeof(char) * d_buff.size(), + stream)); + buffer h_buff(h_alloc, d_buff); + ASSERT_EQ(d_buff.size(), h_buff.size()); + CUDA_CHECK(cudaStreamSynchronize(stream)); + CUDA_CHECK(cudaStreamDestroy(stream)); +} + } // namespace host } // namespace mr } // namespace raft From 2fb8f746db62c15e6df3e4566b1eadd3df685eec Mon Sep 17 00:00:00 2001 From: Thejaswi Rao Date: Mon, 11 May 2020 10:32:29 +0000 Subject: [PATCH 54/56] FIX clang-format fixes --- cpp/include/raft/cudart_utils.h | 8 +++----- cpp/include/raft/handle.hpp | 2 +- cpp/include/raft/mr/buffer_base.hpp | 6 ++---- cpp/include/raft/mr/device/buffer.hpp | 8 +++----- cpp/include/raft/mr/host/allocator.hpp | 2 +- cpp/include/raft/mr/host/buffer.hpp | 13 +++++-------- cpp/test/mr/host/buffer.cpp | 6 +++--- 7 files changed, 18 insertions(+), 27 deletions(-) diff --git a/cpp/include/raft/cudart_utils.h b/cpp/include/raft/cudart_utils.h index 0340086c41..8bd4caf121 100644 --- a/cpp/include/raft/cudart_utils.h +++ b/cpp/include/raft/cudart_utils.h @@ -126,8 +126,8 @@ inline int get_shared_memory_per_block() { int dev_id; CUDA_CHECK(cudaGetDevice(&dev_id)); int smem_per_blk; - CUDA_CHECK(cudaDeviceGetAttribute(&smem_per_blk, - cudaDevAttrMaxSharedMemoryPerBlock, dev_id)); + CUDA_CHECK(cudaDeviceGetAttribute( + &smem_per_blk, cudaDevAttrMaxSharedMemoryPerBlock, dev_id)); return smem_per_blk; } /** helper method to get multi-processor count parameter */ @@ -141,9 +141,7 @@ inline int get_multi_processor_count() { } /** Helper method to get to know warp size in device code */ -constexpr inline int warp_size() { - return 32; -} +constexpr inline int warp_size() { return 32; } /** * @brief Generic copy method for all kinds of transfers diff --git a/cpp/include/raft/handle.hpp b/cpp/include/raft/handle.hpp index 017de5f221..91f1047fa6 100644 --- a/cpp/include/raft/handle.hpp +++ b/cpp/include/raft/handle.hpp @@ -33,9 +33,9 @@ #include #include +#include #include #include -#include #include "cudart_utils.h" namespace raft { diff --git a/cpp/include/raft/mr/buffer_base.hpp b/cpp/include/raft/mr/buffer_base.hpp index 74729134b9..f1d74d4b24 100644 --- a/cpp/include/raft/mr/buffer_base.hpp +++ b/cpp/include/raft/mr/buffer_base.hpp @@ -17,9 +17,9 @@ #pragma once #include +#include #include #include -#include namespace raft { namespace mr { @@ -69,9 +69,7 @@ class buffer_base { } } - ~buffer_base() { - release(); - } + ~buffer_base() { release(); } value_type* data() { return data_; } diff --git a/cpp/include/raft/mr/device/buffer.hpp b/cpp/include/raft/mr/device/buffer.hpp index 902531b366..39b5674ce4 100644 --- a/cpp/include/raft/mr/device/buffer.hpp +++ b/cpp/include/raft/mr/device/buffer.hpp @@ -17,8 +17,8 @@ #pragma once #include -#include "allocator.hpp" #include +#include "allocator.hpp" namespace raft { namespace mr { @@ -49,11 +49,9 @@ class buffer : public buffer_base { using size_type = typename buffer_base::size_type; using value_type = typename buffer_base::value_type; using iterator = typename buffer_base::iterator; - using const_iterator = - typename buffer_base::const_iterator; + using const_iterator = typename buffer_base::const_iterator; using reference = typename buffer_base::reference; - using const_reference = - typename buffer_base::const_reference; + using const_reference = typename buffer_base::const_reference; buffer() = delete; diff --git a/cpp/include/raft/mr/host/allocator.hpp b/cpp/include/raft/mr/host/allocator.hpp index 92a57150e0..9ad6ea7532 100644 --- a/cpp/include/raft/mr/host/allocator.hpp +++ b/cpp/include/raft/mr/host/allocator.hpp @@ -16,9 +16,9 @@ #pragma once -#include #include #include +#include namespace raft { namespace mr { diff --git a/cpp/include/raft/mr/host/buffer.hpp b/cpp/include/raft/mr/host/buffer.hpp index 6a91e55d51..c26617e072 100644 --- a/cpp/include/raft/mr/host/buffer.hpp +++ b/cpp/include/raft/mr/host/buffer.hpp @@ -16,11 +16,11 @@ #pragma once +#include #include -#include "allocator.hpp" #include #include -#include +#include "allocator.hpp" namespace raft { namespace mr { @@ -52,11 +52,9 @@ class buffer : public buffer_base { using size_type = typename buffer_base::size_type; using value_type = typename buffer_base::value_type; using iterator = typename buffer_base::iterator; - using const_iterator = - typename buffer_base::const_iterator; + using const_iterator = typename buffer_base::const_iterator; using reference = typename buffer_base::reference; - using const_reference = - typename buffer_base::const_reference; + using const_reference = typename buffer_base::const_reference; buffer() = delete; @@ -71,8 +69,7 @@ class buffer : public buffer_base { } } - buffer(std::shared_ptr alloc, cudaStream_t stream, - size_type n = 0) + buffer(std::shared_ptr alloc, cudaStream_t stream, size_type n = 0) : buffer_base(alloc, stream, n) {} reference operator[](size_type pos) { return data_[pos]; } diff --git a/cpp/test/mr/host/buffer.cpp b/cpp/test/mr/host/buffer.cpp index fcbdcccf20..953f65ddfb 100644 --- a/cpp/test/mr/host/buffer.cpp +++ b/cpp/test/mr/host/buffer.cpp @@ -17,8 +17,8 @@ #include #include #include -#include #include +#include namespace raft { namespace mr { @@ -57,8 +57,8 @@ TEST(Raft, DeviceToHostBuffer) { cudaStream_t stream; CUDA_CHECK(cudaStreamCreate(&stream)); device::buffer d_buff(d_alloc, stream, 32); - CUDA_CHECK(cudaMemsetAsync(d_buff.data(), 0, sizeof(char) * d_buff.size(), - stream)); + CUDA_CHECK( + cudaMemsetAsync(d_buff.data(), 0, sizeof(char) * d_buff.size(), stream)); buffer h_buff(h_alloc, d_buff); ASSERT_EQ(d_buff.size(), h_buff.size()); CUDA_CHECK(cudaStreamSynchronize(stream)); From eb4db028b90fd15c6e0a674849f32d4be5fdb98e Mon Sep 17 00:00:00 2001 From: Thejaswi Rao Date: Mon, 11 May 2020 20:02:54 -0700 Subject: [PATCH 55/56] CI enabled googletest --- ci/gpu/build.sh | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/ci/gpu/build.sh b/ci/gpu/build.sh index 62a511eaf0..6c3fc0dc7b 100644 --- a/ci/gpu/build.sh +++ b/ci/gpu/build.sh @@ -108,9 +108,7 @@ nvidia-smi logger "GoogleTest for raft..." cd $WORKSPACE/cpp/build -# Googletests haven't been moved over/integrated yet -# GTEST_OUTPUT="xml:${WORKSPACE}/test-results/raft_cpp/" ./test_raft -# running simple tests meanwhile +GTEST_OUTPUT="xml:${WORKSPACE}/test-results/raft_cpp/" ./test_raft logger "Python pytest for cuml..." cd $WORKSPACE/python From 4b0619ed1a739aa1ffe497bc77ccebc2feeda05d Mon Sep 17 00:00:00 2001 From: Thejaswi Rao Date: Mon, 11 May 2020 22:52:18 -0700 Subject: [PATCH 56/56] FIX use RAII-style lock_guard --- cpp/include/raft/handle.hpp | 15 +++++---------- 1 file changed, 5 insertions(+), 10 deletions(-) diff --git a/cpp/include/raft/handle.hpp b/cpp/include/raft/handle.hpp index 91f1047fa6..81e63342ce 100644 --- a/cpp/include/raft/handle.hpp +++ b/cpp/include/raft/handle.hpp @@ -89,42 +89,38 @@ class handle_t { } cublasHandle_t get_cublas_handle() const { - mutex_.lock(); + std::lock_guard _(mutex_); if (!cublas_initialized_) { CUBLAS_CHECK(cublasCreate(&cublas_handle_)); cublas_initialized_ = true; } - mutex_.unlock(); return cublas_handle_; } cusolverDnHandle_t get_cusolver_dn_handle() const { - mutex_.lock(); + std::lock_guard _(mutex_); if (!cusolver_dn_initialized_) { CUSOLVER_CHECK(cusolverDnCreate(&cusolver_dn_handle_)); cusolver_dn_initialized_ = true; } - mutex_.unlock(); return cusolver_dn_handle_; } cusolverSpHandle_t get_cusolver_sp_handle() const { - mutex_.lock(); + std::lock_guard _(mutex_); if (!cusolver_sp_initialized_) { CUSOLVER_CHECK(cusolverSpCreate(&cusolver_sp_handle_)); cusolver_sp_initialized_ = true; } - mutex_.unlock(); return cusolver_sp_handle_; } cusparseHandle_t get_cusparse_handle() const { - mutex_.lock(); + std::lock_guard _(mutex_); if (!cusparse_initialized_) { CUSPARSE_CHECK(cusparseCreate(&cusparse_handle_)); cusparse_initialized_ = true; } - mutex_.unlock(); return cusparse_handle_; } @@ -159,12 +155,11 @@ class handle_t { // bool commsInitialized() const; const cudaDeviceProp& get_device_properties() const { - mutex_.lock(); + std::lock_guard _(mutex_); if (!device_prop_initialized_) { CUDA_CHECK(cudaGetDeviceProperties(&prop_, dev_id_)); device_prop_initialized_ = true; } - mutex_.unlock(); return prop_; }