未验证 提交 da3743fd 编写于 作者: L Leo Chen 提交者: GitHub

move CUDAStream to phi (#44529)

* init

* move CUDAStream to phi

* fix compilation

* merge develop

* add stream_owned_ member

* split cuda_stream.h

* fix cpu compile

* fix constructor

* fix bug

* fix windows compile

* fix inference test_levit

* fix windows tests
上级 23ad0cc4
...@@ -18,24 +18,12 @@ if(WITH_NCCL OR WITH_RCCL) ...@@ -18,24 +18,12 @@ if(WITH_NCCL OR WITH_RCCL)
cc_library( cc_library(
processgroup_nccl processgroup_nccl
SRCS ProcessGroupNCCL.cc NCCLTools.cc Common.cc SRCS ProcessGroupNCCL.cc NCCLTools.cc Common.cc
DEPS place DEPS place enforce collective_helper device_context phi_api eager_api)
cuda_stream
enforce
collective_helper
device_context
phi_api
eager_api)
if(WITH_DISTRIBUTE AND WITH_PSCORE) if(WITH_DISTRIBUTE AND WITH_PSCORE)
cc_library( cc_library(
processgroup_heter processgroup_heter
SRCS ProcessGroupHeter.cc NCCLTools.cc Common.cc SRCS ProcessGroupHeter.cc NCCLTools.cc Common.cc
DEPS place DEPS place enforce collective_helper device_context phi_api eager_api)
cuda_stream
enforce
collective_helper
device_context
phi_api
eager_api)
endif() endif()
endif() endif()
......
...@@ -33,7 +33,6 @@ ...@@ -33,7 +33,6 @@
#include "paddle/fluid/platform/enforce.h" #include "paddle/fluid/platform/enforce.h"
#include "paddle/fluid/platform/gen_comm_id_helper.h" #include "paddle/fluid/platform/gen_comm_id_helper.h"
#include "paddle/fluid/platform/place.h" #include "paddle/fluid/platform/place.h"
#include "paddle/fluid/platform/stream/cuda_stream.h"
#if defined(PADDLE_WITH_NCCL) || defined(PADDLE_WITH_RCCL) #if defined(PADDLE_WITH_NCCL) || defined(PADDLE_WITH_RCCL)
#include "paddle/fluid/distributed/collective/NCCLTools.h" #include "paddle/fluid/distributed/collective/NCCLTools.h"
......
...@@ -28,7 +28,6 @@ ...@@ -28,7 +28,6 @@
#include "paddle/fluid/platform/enforce.h" #include "paddle/fluid/platform/enforce.h"
#include "paddle/fluid/platform/gen_comm_id_helper.h" #include "paddle/fluid/platform/gen_comm_id_helper.h"
#include "paddle/fluid/platform/place.h" #include "paddle/fluid/platform/place.h"
#include "paddle/fluid/platform/stream/cuda_stream.h"
#if defined(PADDLE_WITH_NCCL) || defined(PADDLE_WITH_RCCL) #if defined(PADDLE_WITH_NCCL) || defined(PADDLE_WITH_RCCL)
#include "paddle/fluid/distributed/collective/NCCLTools.h" #include "paddle/fluid/distributed/collective/NCCLTools.h"
...@@ -46,7 +45,6 @@ namespace paddle { ...@@ -46,7 +45,6 @@ namespace paddle {
namespace distributed { namespace distributed {
using Place = paddle::platform::Place; using Place = paddle::platform::Place;
using CUDAStream = platform::stream::CUDAStream;
using CUDADeviceContext = paddle::platform::CUDADeviceContext; using CUDADeviceContext = paddle::platform::CUDADeviceContext;
class ProcessGroupNCCL : public ProcessGroup { class ProcessGroupNCCL : public ProcessGroup {
......
...@@ -25,6 +25,7 @@ ...@@ -25,6 +25,7 @@
#ifdef PADDLE_WITH_MLU #ifdef PADDLE_WITH_MLU
#include "paddle/fluid/platform/device/mlu/device_context.h" #include "paddle/fluid/platform/device/mlu/device_context.h"
#endif #endif
#include "paddle/fluid/platform/stream_callback_manager.h"
namespace paddle { namespace paddle {
namespace framework { namespace framework {
......
...@@ -30,7 +30,6 @@ limitations under the License. */ ...@@ -30,7 +30,6 @@ limitations under the License. */
#include "paddle/fluid/platform/place.h" #include "paddle/fluid/platform/place.h"
#include "paddle/phi/core/ddim.h" #include "paddle/phi/core/ddim.h"
#include "paddle/phi/core/dense_tensor.h" #include "paddle/phi/core/dense_tensor.h"
#include "paddle/phi/core/stream.h"
namespace paddle { namespace paddle {
......
...@@ -195,7 +195,7 @@ phi::Allocation* StreamSafeCUDAAllocator::AllocateImpl(size_t size) { ...@@ -195,7 +195,7 @@ phi::Allocation* StreamSafeCUDAAllocator::AllocateImpl(size_t size) {
default_stream_, default_stream_,
this); this);
VLOG(8) << "Allocate " << allocation->size() << " bytes at address " VLOG(8) << "Allocate " << allocation->size() << " bytes at address "
<< allocation->ptr(); << allocation->ptr() << " , stream: " << default_stream_;
return allocation; return allocation;
} }
......
...@@ -19,7 +19,6 @@ ...@@ -19,7 +19,6 @@
#include "paddle/fluid/framework/op_registry.h" #include "paddle/fluid/framework/op_registry.h"
#include "paddle/fluid/platform/dynload/nvjpeg.h" #include "paddle/fluid/platform/dynload/nvjpeg.h"
#include "paddle/fluid/platform/enforce.h" #include "paddle/fluid/platform/enforce.h"
#include "paddle/fluid/platform/stream/cuda_stream.h"
namespace paddle { namespace paddle {
namespace operators { namespace operators {
...@@ -69,7 +68,7 @@ class GPUDecodeJpegKernel : public framework::OpKernel<T> { ...@@ -69,7 +68,7 @@ class GPUDecodeJpegKernel : public framework::OpKernel<T> {
nvjpegStatus_t info_status = nvjpegStatus_t info_status =
platform::dynload::nvjpegGetImageInfo(nvjpeg_handle, platform::dynload::nvjpegGetImageInfo(nvjpeg_handle,
x_data, x_data,
(size_t)x->numel(), static_cast<size_t>(x->numel()),
&components, &components,
&subsampling, &subsampling,
widths, widths,
......
...@@ -113,7 +113,6 @@ endif() ...@@ -113,7 +113,6 @@ endif()
add_subdirectory(device) add_subdirectory(device)
add_subdirectory(dynload) add_subdirectory(dynload)
add_subdirectory(stream)
cc_library( cc_library(
cpu_helper cpu_helper
...@@ -130,7 +129,7 @@ if(WITH_DGC) ...@@ -130,7 +129,7 @@ if(WITH_DGC)
endif() endif()
if(WITH_GPU OR WITH_ROCM) if(WITH_GPU OR WITH_ROCM)
set(GPU_CTX_DEPS dynload_cuda dynamic_loader cuda_stream) set(GPU_CTX_DEPS dynload_cuda dynamic_loader)
endif() endif()
if(WITH_IPU) if(WITH_IPU)
......
...@@ -26,7 +26,6 @@ limitations under the License. */ ...@@ -26,7 +26,6 @@ limitations under the License. */
#include "paddle/fluid/platform/place.h" #include "paddle/fluid/platform/place.h"
#include "paddle/fluid/platform/profiler.h" #include "paddle/fluid/platform/profiler.h"
#include "paddle/fluid/platform/profiler/event_tracing.h" #include "paddle/fluid/platform/profiler/event_tracing.h"
#include "paddle/fluid/platform/stream/cuda_stream.h"
#include "paddle/phi/backends/gpu/gpu_context.h" #include "paddle/phi/backends/gpu/gpu_context.h"
#include "paddle/phi/core/allocator.h" #include "paddle/phi/core/allocator.h"
...@@ -534,24 +533,11 @@ void CudnnWorkspaceHandle::ReallocWorkspace(size_t required_workspace_bytes) { ...@@ -534,24 +533,11 @@ void CudnnWorkspaceHandle::ReallocWorkspace(size_t required_workspace_bytes) {
allocation_ = memory::Alloc(device_context_, required_workspace_bytes); allocation_ = memory::Alloc(device_context_, required_workspace_bytes);
} }
CUDADeviceContext::CUDADeviceContext(CUDAPlace place) : phi::GPUContext(place) { CUDADeviceContext::CUDADeviceContext(CUDAPlace place)
phi::GPUContext::PartialInitWithoutAllocator(); : phi::GPUContext(place) {}
cuda_stream_.reset(new stream::CUDAStream(phi::GPUContext::stream(), place));
}
CUDADeviceContext::~CUDADeviceContext() = default; CUDADeviceContext::~CUDADeviceContext() = default;
stream::CUDAStream* CUDADeviceContext::GetCudaStream() const {
return cuda_stream_.get();
}
stream::CUDAStream* CUDADeviceContext::SetCudaStream(
stream::CUDAStream* new_stream_ptr) {
auto* old_stream_ptr = cuda_stream_.release();
cuda_stream_.reset(new_stream_ptr);
return old_stream_ptr;
}
CUDAPinnedDeviceContext::CUDAPinnedDeviceContext() { CUDAPinnedDeviceContext::CUDAPinnedDeviceContext() {
eigen_device_.reset(new Eigen::DefaultDevice()); eigen_device_.reset(new Eigen::DefaultDevice());
} }
......
...@@ -67,9 +67,6 @@ limitations under the License. */ ...@@ -67,9 +67,6 @@ limitations under the License. */
#include "glog/logging.h" #include "glog/logging.h"
#include "paddle/fluid/platform/enforce.h" #include "paddle/fluid/platform/enforce.h"
#include "paddle/fluid/platform/place.h" #include "paddle/fluid/platform/place.h"
#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP)
#include "paddle/fluid/platform/stream/cuda_stream.h"
#endif
#ifdef PADDLE_WITH_ASCEND_CL #ifdef PADDLE_WITH_ASCEND_CL
#include "paddle/fluid/platform/device/npu/enforce_npu.h" #include "paddle/fluid/platform/device/npu/enforce_npu.h"
#include "paddle/fluid/platform/device/npu/npu_stream.h" #include "paddle/fluid/platform/device/npu/npu_stream.h"
...@@ -279,16 +276,8 @@ class CUDADeviceContext : public phi::GPUContext { ...@@ -279,16 +276,8 @@ class CUDADeviceContext : public phi::GPUContext {
explicit CUDADeviceContext(CUDAPlace place); explicit CUDADeviceContext(CUDAPlace place);
virtual ~CUDADeviceContext(); virtual ~CUDADeviceContext();
// NOTE: Just for compatibility with the past, please delete if there is an
// elegant way.
stream::CUDAStream* GetCudaStream() const;
stream::CUDAStream* SetCudaStream(stream::CUDAStream*);
private: private:
// NOTE: Just for compatibility with the past, please delete if there is an int place_holder_; // TO BE REMOVED
// elegant way.
std::unique_ptr<stream::CUDAStream> cuda_stream_;
DISABLE_COPY_AND_ASSIGN(CUDADeviceContext); DISABLE_COPY_AND_ASSIGN(CUDADeviceContext);
}; };
......
...@@ -25,7 +25,6 @@ limitations under the License. */ ...@@ -25,7 +25,6 @@ limitations under the License. */
#include <hip/hip_runtime.h> #include <hip/hip_runtime.h>
#endif #endif
#include "paddle/fluid/platform/place.h" #include "paddle/fluid/platform/place.h"
#include "paddle/fluid/platform/stream/cuda_stream.h"
namespace paddle { namespace paddle {
namespace platform { namespace platform {
......
if(WITH_GPU OR WITH_ROCM)
cc_library(
cuda_stream
SRCS cuda_stream.cc
DEPS enforce eigen3 ${MKLDNN_CTX_DEPS})
endif()
/* Copyright (c) 2020 PaddlePaddle Authors. All Rights Reserved.
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 "paddle/fluid/platform/stream/cuda_stream.h"
#include "paddle/fluid/platform/cuda_device_guard.h"
#include "paddle/fluid/platform/device/gpu/gpu_types.h"
#include "paddle/fluid/platform/device_context.h"
#include "paddle/fluid/platform/enforce.h"
namespace paddle {
namespace platform {
namespace stream {
bool CUDAStream::Init(const Place& place,
const Priority& priority,
const StreamFlag& flag) {
PADDLE_ENFORCE_EQ(is_gpu_place(place),
true,
platform::errors::InvalidArgument(
"Cuda stream must be created using cuda place."));
place_ = place;
CUDADeviceGuard guard(place_.device);
if (priority == Priority::kHigh) {
#ifdef PADDLE_WITH_HIP
PADDLE_ENFORCE_GPU_SUCCESS(hipStreamCreateWithPriority(
&stream_, static_cast<unsigned int>(flag), -1));
#else
PADDLE_ENFORCE_GPU_SUCCESS(cudaStreamCreateWithPriority(
&stream_, static_cast<unsigned int>(flag), -1));
#endif
} else if (priority == Priority::kNormal) {
#ifdef PADDLE_WITH_HIP
PADDLE_ENFORCE_GPU_SUCCESS(hipStreamCreateWithPriority(
&stream_, static_cast<unsigned int>(flag), 0));
#else
PADDLE_ENFORCE_GPU_SUCCESS(cudaStreamCreateWithPriority(
&stream_, static_cast<unsigned int>(flag), 0));
#endif
}
callback_manager_.reset(new StreamCallbackManager<gpuStream_t>(stream_));
VLOG(3) << "GPUStream Init stream: " << stream_
<< ", priority: " << static_cast<int>(priority)
<< ", flag:" << static_cast<int>(flag);
return true;
}
void CUDAStream::Destroy() {
CUDADeviceGuard guard(place_.device);
Wait();
WaitCallback();
if (stream_ && owned_stream_) {
#ifdef PADDLE_WITH_HIP
PADDLE_ENFORCE_GPU_SUCCESS(hipStreamDestroy(stream_));
#else
PADDLE_ENFORCE_GPU_SUCCESS(cudaStreamDestroy(stream_));
#endif
}
stream_ = nullptr;
}
void CUDAStream::Wait() const {
#ifdef PADDLE_WITH_HIP
hipError_t e_sync = hipSuccess;
#if !defined(_WIN32)
e_sync = hipStreamSynchronize(stream_);
#else
while (e_sync = hipStreamQuery(stream_)) {
if (e_sync == hipErrorNotReady) continue;
break;
}
#endif
#else
cudaError_t e_sync = cudaSuccess;
#if !defined(_WIN32)
e_sync = cudaStreamSynchronize(stream_);
#else
while (e_sync = cudaStreamQuery(stream_)) {
if (e_sync == cudaErrorNotReady) continue;
break;
}
#endif
#endif // PADDLE_WITH_HIP
PADDLE_ENFORCE_GPU_SUCCESS(e_sync);
}
// Note: Can only be used under thread_local semantics.
void CUDAStream::SetStream(gpuStream_t stream) {
if (owned_stream_ && stream_) {
#ifdef PADDLE_WITH_HIP
PADDLE_ENFORCE_GPU_SUCCESS(hipStreamDestroy(stream_));
#else
PADDLE_ENFORCE_GPU_SUCCESS(cudaStreamDestroy(stream_));
#endif
}
owned_stream_ = false;
stream_ = stream;
callback_manager_.reset(new StreamCallbackManager<gpuStream_t>(stream_));
}
CUDAStream* get_current_stream(int deviceId) {
#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP)
if (deviceId == -1) {
deviceId = platform::GetCurrentDeviceId();
}
auto& pool = platform::DeviceContextPool::Instance();
platform::Place device = CUDAPlace(deviceId);
return static_cast<platform::CUDADeviceContext*>(pool.Get(device))
->GetCudaStream();
#else
PADDLE_THROW(platform::errors::Unavailable(
"Paddle is not compiled with CUDA. Cannot visit cuda current stream."));
return nullptr;
#endif
}
CUDAStream* set_current_stream(CUDAStream* stream) {
#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP)
auto& device = stream->GetPlace();
auto& pool = platform::DeviceContextPool::Instance();
return static_cast<platform::CUDADeviceContext*>(pool.Get(device))
->SetCudaStream(stream);
#else
PADDLE_THROW(platform::errors::Unavailable(
"Paddle is not compiled with CUDA. Cannot visit cuda current"
"stream."));
return CUDAStream(nullptr);
#endif
}
} // namespace stream
} // namespace platform
} // namespace paddle
/* Copyright (c) 2020 PaddlePaddle Authors. All Rights Reserved.
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 <cstdint>
#include <functional>
#include <memory>
#include "paddle/fluid/platform/device/gpu/gpu_info.h"
#include "paddle/fluid/platform/device/gpu/gpu_types.h"
#include "paddle/fluid/platform/macros.h"
#include "paddle/fluid/platform/place.h"
#include "paddle/fluid/platform/stream_callback_manager.h"
namespace paddle {
namespace platform {
namespace stream {
#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP)
enum class Priority : uint8_t {
kNull = 0x0,
kHigh = 0x1,
kNormal = 0x2,
};
enum class StreamFlag : uint8_t {
kDefaultFlag = 0x0,
kStreamNonBlocking = 0x1,
};
#endif
class CUDAStream final {
#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP)
public:
CUDAStream() = default;
explicit CUDAStream(const Place& place,
const Priority& priority = Priority::kNormal,
const StreamFlag& flag = StreamFlag::kDefaultFlag) {
Init(place, priority, flag);
}
explicit CUDAStream(gpuStream_t stream, const Place& place)
: place_(place), stream_(stream) {
owned_stream_ = false;
callback_manager_.reset(new StreamCallbackManager<gpuStream_t>(stream_));
}
virtual ~CUDAStream() { Destroy(); }
bool Init(const Place& place,
const Priority& priority = Priority::kNormal,
const StreamFlag& flag = StreamFlag::kDefaultFlag);
void AddCallback(std::function<void()> callback) const {
callback_manager_->AddCallback(callback);
}
#ifdef PADDLE_WITH_HIP
void RecordEvent(hipEvent_t ev, const std::function<void()>& callback) const {
callback();
PADDLE_ENFORCE_GPU_SUCCESS(hipEventRecord(ev, stream_));
}
#else
void RecordEvent(cudaEvent_t ev,
const std::function<void()>& callback) const {
callback();
PADDLE_ENFORCE_GPU_SUCCESS(cudaEventRecord(ev, stream_));
}
#endif
#ifdef PADDLE_WITH_HIP
void RecordEvent(hipEvent_t ev) const {
PADDLE_ENFORCE_GPU_SUCCESS(hipEventRecord(ev, stream_));
}
#else
void RecordEvent(cudaEvent_t ev) const {
PADDLE_ENFORCE_GPU_SUCCESS(cudaEventRecord(ev, stream_));
}
#endif
#ifdef PADDLE_WITH_HIP
void WaitEvent(hipEvent_t ev) const {
PADDLE_ENFORCE_GPU_SUCCESS(hipStreamWaitEvent(stream_, ev, 0));
}
#else
void WaitEvent(cudaEvent_t ev) const {
PADDLE_ENFORCE_GPU_SUCCESS(cudaStreamWaitEvent(stream_, ev, 0));
}
#endif
void Wait() const;
void WaitCallback() const { callback_manager_->Wait(); }
#ifdef PADDLE_WITH_HIP
const hipStream_t& raw_stream() const { return stream_; }
#else
const cudaStream_t& raw_stream() const { return stream_; }
#endif
void Destroy();
bool Query() const {
#ifdef PADDLE_WITH_HIP
hipError_t err = hipStreamQuery(stream_);
if (err == hipSuccess) {
return true;
}
if (err == hipErrorNotReady) {
return false;
}
#else
cudaError_t err = cudaStreamQuery(stream_);
if (err == cudaSuccess) {
return true;
}
if (err == cudaErrorNotReady) {
return false;
}
#endif
PADDLE_ENFORCE_GPU_SUCCESS(err);
return false;
}
void Synchronize() const { platform::GpuStreamSync(stream_); }
const Place& GetPlace() const { return place_; }
// Note: Can only be used under thread_local semantics.
void SetStream(gpuStream_t stream);
private:
Place place_;
bool owned_stream_{true};
#ifdef PADDLE_WITH_HIP
hipStream_t stream_{nullptr};
#else
cudaStream_t stream_{nullptr};
#endif
Priority priority_{Priority::kNormal};
std::unique_ptr<StreamCallbackManager<gpuStream_t>> callback_manager_;
#endif
DISABLE_COPY_AND_ASSIGN(CUDAStream);
};
CUDAStream* get_current_stream(int deviceId);
// NOTE: There is a problem with the interface and needs to be fixed
CUDAStream* set_current_stream(CUDAStream* stream);
} // namespace stream
} // namespace platform
} // namespace paddle
...@@ -31,6 +31,7 @@ ...@@ -31,6 +31,7 @@
#include <mutex> // NOLINT #include <mutex> // NOLINT
#include "paddle/fluid/platform/enforce.h" #include "paddle/fluid/platform/enforce.h"
#include "paddle/phi/backends/gpu/gpu_decls.h"
namespace paddle { namespace paddle {
namespace platform { namespace platform {
......
...@@ -19,11 +19,30 @@ ...@@ -19,11 +19,30 @@
#include "paddle/fluid/platform/device_event_base.h" #include "paddle/fluid/platform/device_event_base.h"
#include "paddle/fluid/platform/event.h" #include "paddle/fluid/platform/event.h"
#include "paddle/fluid/platform/stream/cuda_stream.h"
namespace py = pybind11; namespace py = pybind11;
namespace paddle { namespace paddle {
namespace platform {
#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP)
phi::CUDAStream *get_current_stream(int device_id) {
if (device_id == -1) {
device_id = phi::backends::gpu::GetCurrentDeviceId();
}
auto *gpu_context = static_cast<const phi::GPUContext *>(
DeviceContextPool::Instance().Get(GPUPlace(device_id)));
return gpu_context->cuda_stream();
}
phi::CUDAStream *set_current_stream(phi::CUDAStream *stream) {
auto *original_stream = get_current_stream(stream->place().GetDeviceId());
auto *gpu_context = static_cast<phi::GPUContext *>(
DeviceContextPool::Instance().Get(stream->place()));
gpu_context->SetCUDAStream(stream, /*clear=*/false);
return original_stream;
}
#endif
} // namespace platform
namespace pybind { namespace pybind {
void BindCudaStream(py::module *m_ptr) { void BindCudaStream(py::module *m_ptr) {
auto &m = *m_ptr; auto &m = *m_ptr;
...@@ -33,24 +52,24 @@ void BindCudaStream(py::module *m_ptr) { ...@@ -33,24 +52,24 @@ void BindCudaStream(py::module *m_ptr) {
"_get_current_stream", "_get_current_stream",
[](int deviceId) { [](int deviceId) {
#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) #if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP)
return paddle::platform::stream::get_current_stream(deviceId); return platform::get_current_stream(deviceId);
#else #else
PADDLE_THROW(platform::errors::Unavailable( PADDLE_THROW(
"Paddle is not compiled with CUDA. Cannot visit cuda current" platform::errors::Unavailable("Paddle is not compiled with CUDA. "
"stream.")); "Cannot visit device synchronize."));
#endif #endif
}, },
py::return_value_policy::reference); py::return_value_policy::reference);
m.def( m.def(
"_set_current_stream", "_set_current_stream",
[](paddle::platform::stream::CUDAStream &stream) { [](phi::CUDAStream *stream) {
#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) #if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP)
return paddle::platform::stream::set_current_stream(&stream); return platform::set_current_stream(stream);
#else #else
PADDLE_THROW(platform::errors::Unavailable( PADDLE_THROW(
"Paddle is not compiled with CUDA. Cannot set cuda current " platform::errors::Unavailable("Paddle is not compiled with CUDA. "
"stream.")); "Cannot visit device synchronize."));
#endif #endif
}, },
py::return_value_policy::reference); py::return_value_policy::reference);
...@@ -75,7 +94,7 @@ void BindCudaStream(py::module *m_ptr) { ...@@ -75,7 +94,7 @@ void BindCudaStream(py::module *m_ptr) {
#endif #endif
}); });
py::class_<paddle::platform::stream::CUDAStream>(m, "CUDAStream", R"DOC( py::class_<phi::CUDAStream>(m, "CUDAStream", R"DOC(
The handle of the CUDA stream. The handle of the CUDA stream.
Parameters: Parameters:
...@@ -99,8 +118,7 @@ void BindCudaStream(py::module *m_ptr) { ...@@ -99,8 +118,7 @@ void BindCudaStream(py::module *m_ptr) {
#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) #if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP)
.def( .def(
"wait_event", "wait_event",
[](paddle::platform::stream::CUDAStream &self, [](phi::CUDAStream &self, paddle::platform::CudaEvent &event) {
paddle::platform::CudaEvent &event) {
self.WaitEvent(event.GetRawCudaEvent()); self.WaitEvent(event.GetRawCudaEvent());
}, },
R"DOC( R"DOC(
...@@ -121,11 +139,9 @@ void BindCudaStream(py::module *m_ptr) { ...@@ -121,11 +139,9 @@ void BindCudaStream(py::module *m_ptr) {
)DOC") )DOC")
.def( .def(
"wait_stream", "wait_stream",
[](paddle::platform::stream::CUDAStream &self, [](phi::CUDAStream &self, phi::CUDAStream &stream) {
paddle::platform::stream::CUDAStream &stream) {
paddle::platform::CudaEvent event; paddle::platform::CudaEvent event;
event.Record(stream.raw_stream()); event.Record(stream.raw_stream());
self.WaitEvent(event.GetRawCudaEvent()); self.WaitEvent(event.GetRawCudaEvent());
}, },
R"DOC( R"DOC(
...@@ -146,9 +162,7 @@ void BindCudaStream(py::module *m_ptr) { ...@@ -146,9 +162,7 @@ void BindCudaStream(py::module *m_ptr) {
)DOC") )DOC")
.def( .def(
"query", "query",
[](paddle::platform::stream::CUDAStream &self) { [](phi::CUDAStream &self) { return self.Query(); },
return self.Query();
},
R"DOC( R"DOC(
Return the status whether if all operations in stream have completed. Return the status whether if all operations in stream have completed.
...@@ -165,9 +179,7 @@ void BindCudaStream(py::module *m_ptr) { ...@@ -165,9 +179,7 @@ void BindCudaStream(py::module *m_ptr) {
)DOC") )DOC")
.def( .def(
"synchronize", "synchronize",
[](paddle::platform::stream::CUDAStream &self) { [](phi::CUDAStream &self) { self.Synchronize(); },
self.Synchronize();
},
R"DOC( R"DOC(
Waits for stream tasks to complete. Waits for stream tasks to complete.
...@@ -182,8 +194,7 @@ void BindCudaStream(py::module *m_ptr) { ...@@ -182,8 +194,7 @@ void BindCudaStream(py::module *m_ptr) {
)DOC") )DOC")
.def( .def(
"record_event", "record_event",
[](paddle::platform::stream::CUDAStream &self, [](phi::CUDAStream &self, paddle::platform::CudaEvent *event) {
paddle::platform::CudaEvent *event) {
if (event == nullptr) { if (event == nullptr) {
event = new paddle::platform::CudaEvent(); event = new paddle::platform::CudaEvent();
} }
...@@ -212,7 +223,7 @@ void BindCudaStream(py::module *m_ptr) { ...@@ -212,7 +223,7 @@ void BindCudaStream(py::module *m_ptr) {
py::arg("event") = nullptr) py::arg("event") = nullptr)
.def_property_readonly( .def_property_readonly(
"cuda_stream", "cuda_stream",
[](paddle::platform::stream::CUDAStream &self) { [](phi::CUDAStream &self) {
VLOG(10) << self.raw_stream(); VLOG(10) << self.raw_stream();
return reinterpret_cast<std::uintptr_t>(self.raw_stream()); return reinterpret_cast<std::uintptr_t>(self.raw_stream());
}, },
...@@ -235,26 +246,22 @@ void BindCudaStream(py::module *m_ptr) { ...@@ -235,26 +246,22 @@ void BindCudaStream(py::module *m_ptr) {
#endif #endif
.def( .def(
"__init__", "__init__",
[](paddle::platform::stream::CUDAStream &self, [](phi::CUDAStream &self, platform::CUDAPlace *place, int priority) {
platform::CUDAPlace *device,
int priority) {
#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) #if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP)
if (priority != 1 && priority != 2) { if (priority != 1 && priority != 2) {
PADDLE_THROW(platform::errors::InvalidArgument( PADDLE_THROW(platform::errors::InvalidArgument(
"Priority should be 1(high) or 2(normal) ")); "Priority should be 1(high) or 2(normal) "));
} }
auto prio = paddle::platform::stream::Priority(priority); auto prio = phi::CUDAStream::Priority(priority);
auto stream_flag = auto stream_flag = phi::CUDAStream::StreamFlag::kStreamNonBlocking;
paddle::platform::stream::StreamFlag::kStreamNonBlocking;
if (device == nullptr) { if (place == nullptr) {
int curr_device_id = platform::GetCurrentDeviceId(); int curr_device_id = platform::GetCurrentDeviceId();
auto device_tmp = platform::CUDAPlace(curr_device_id); auto place_tmp = platform::CUDAPlace(curr_device_id);
device = &device_tmp; place = &place_tmp;
} }
new (&self) paddle::platform::stream::CUDAStream( new (&self) phi::CUDAStream(*place, prio, stream_flag);
*device, prio, stream_flag);
#else #else
PADDLE_THROW(platform::errors::Unavailable( PADDLE_THROW(platform::errors::Unavailable(
"Class CUDAStream can only be initialized on the GPU platform.")); "Class CUDAStream can only be initialized on the GPU platform."));
...@@ -264,17 +271,14 @@ void BindCudaStream(py::module *m_ptr) { ...@@ -264,17 +271,14 @@ void BindCudaStream(py::module *m_ptr) {
py::arg("priority") = 2) py::arg("priority") = 2)
.def( .def(
"__init__", "__init__",
[](paddle::platform::stream::CUDAStream &self, [](phi::CUDAStream &self, int device, int priority) {
int device,
int priority) {
#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) #if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP)
if (priority != 1 && priority != 2) { if (priority != 1 && priority != 2) {
PADDLE_THROW(platform::errors::InvalidArgument( PADDLE_THROW(platform::errors::InvalidArgument(
"Priority should be 1(high) or 2(normal) ")); "Priority should be 1(high) or 2(normal) "));
} }
auto prio = paddle::platform::stream::Priority(priority); auto prio = phi::CUDAStream::Priority(priority);
auto stream_flag = auto stream_flag = phi::CUDAStream::StreamFlag::kStreamNonBlocking;
paddle::platform::stream::StreamFlag::kStreamNonBlocking;
int device_count = platform::GetGPUDeviceCount(); int device_count = platform::GetGPUDeviceCount();
if (device < 0) { if (device < 0) {
...@@ -287,8 +291,8 @@ void BindCudaStream(py::module *m_ptr) { ...@@ -287,8 +291,8 @@ void BindCudaStream(py::module *m_ptr) {
device)); device));
} }
new (&self) paddle::platform::stream::CUDAStream( new (&self)
platform::CUDAPlace(device), prio, stream_flag); phi::CUDAStream(platform::CUDAPlace(device), prio, stream_flag);
#else #else
PADDLE_THROW(platform::errors::Unavailable( PADDLE_THROW(platform::errors::Unavailable(
"Class CUDAStream can only be initialized on the GPU platform.")); "Class CUDAStream can only be initialized on the GPU platform."));
...@@ -296,16 +300,15 @@ void BindCudaStream(py::module *m_ptr) { ...@@ -296,16 +300,15 @@ void BindCudaStream(py::module *m_ptr) {
}, },
py::arg("device") = -1, py::arg("device") = -1,
py::arg("priority") = 2) py::arg("priority") = 2)
.def("__init__", [](paddle::platform::stream::CUDAStream &self) { .def("__init__", [](phi::CUDAStream &self) {
#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) #if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP)
auto prio = paddle::platform::stream::Priority::kNormal; auto prio = phi::CUDAStream::Priority::kNormal;
auto stream_flag = auto stream_flag = phi::CUDAStream::StreamFlag::kStreamNonBlocking;
paddle::platform::stream::StreamFlag::kStreamNonBlocking;
int device_id = platform::GetCurrentDeviceId(); int device_id = platform::GetCurrentDeviceId();
new (&self) paddle::platform::stream::CUDAStream( new (&self)
platform::CUDAPlace(device_id), prio, stream_flag); phi::CUDAStream(platform::CUDAPlace(device_id), prio, stream_flag);
#else #else
PADDLE_THROW(platform::errors::Unavailable( PADDLE_THROW(platform::errors::Unavailable(
"Class CUDAStream can only be initialized on the GPU platform.")); "Class CUDAStream can only be initialized on the GPU platform."));
...@@ -331,10 +334,9 @@ void BindCudaStream(py::module *m_ptr) { ...@@ -331,10 +334,9 @@ void BindCudaStream(py::module *m_ptr) {
#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP) #if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP)
.def( .def(
"record", "record",
[](paddle::platform::CudaEvent &self, [](paddle::platform::CudaEvent &self, phi::CUDAStream *stream) {
paddle::platform::stream::CUDAStream *stream) {
if (stream == nullptr) { if (stream == nullptr) {
stream = paddle::platform::stream::get_current_stream(-1); stream = paddle::platform::get_current_stream(-1);
} }
self.Record(stream->raw_stream()); self.Record(stream->raw_stream());
}, },
...@@ -410,5 +412,4 @@ void BindCudaStream(py::module *m_ptr) { ...@@ -410,5 +412,4 @@ void BindCudaStream(py::module *m_ptr) {
} }
} // namespace pybind } // namespace pybind
} // namespace paddle } // namespace paddle
...@@ -17,9 +17,23 @@ ...@@ -17,9 +17,23 @@
#include "pybind11/pybind11.h" #include "pybind11/pybind11.h"
#include "pybind11/stl.h" #include "pybind11/stl.h"
#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP)
#include "paddle/phi/core/cuda_stream.h"
#else
namespace phi {
class CUDAStream {};
} // namespace phi
#endif
namespace py = pybind11; namespace py = pybind11;
namespace paddle { namespace paddle {
namespace platform {
#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP)
phi::CUDAStream* get_current_stream(int device_id = -1);
phi::CUDAStream* set_current_stream(phi::CUDAStream* stream);
#endif
} // namespace platform
namespace pybind { namespace pybind {
void BindCudaStream(py::module* m); void BindCudaStream(py::module* m);
......
...@@ -35,7 +35,6 @@ typedef SSIZE_T ssize_t; ...@@ -35,7 +35,6 @@ typedef SSIZE_T ssize_t;
#include "paddle/fluid/platform/device/gpu/gpu_info.h" #include "paddle/fluid/platform/device/gpu/gpu_info.h"
#include "paddle/fluid/platform/dynload/dynamic_loader.h" #include "paddle/fluid/platform/dynload/dynamic_loader.h"
#include "paddle/fluid/platform/enforce.h" #include "paddle/fluid/platform/enforce.h"
#include "paddle/fluid/platform/stream/cuda_stream.h"
#include "paddle/fluid/pybind/eager.h" #include "paddle/fluid/pybind/eager.h"
#include "paddle/fluid/pybind/eager_utils.h" #include "paddle/fluid/pybind/eager_utils.h"
#include "paddle/fluid/pybind/exception.h" #include "paddle/fluid/pybind/exception.h"
...@@ -51,6 +50,10 @@ typedef SSIZE_T ssize_t; ...@@ -51,6 +50,10 @@ typedef SSIZE_T ssize_t;
#include "pybind11/numpy.h" #include "pybind11/numpy.h"
#include "pybind11/pybind11.h" #include "pybind11/pybind11.h"
#if defined(PADDLE_WITH_CUDA) || defined(PADDLE_WITH_HIP)
#include "paddle/fluid/pybind/cuda_streams_py.h"
#endif
namespace paddle { namespace paddle {
namespace pybind { namespace pybind {
...@@ -673,8 +676,7 @@ static PyObject* eager_api_async_read(PyObject* self, ...@@ -673,8 +676,7 @@ static PyObject* eager_api_async_read(PyObject* self,
platform::errors::InvalidArgument( platform::errors::InvalidArgument(
"`index` tensor should be one-dimensional.")); "`index` tensor should be one-dimensional."));
auto stream = auto stream = paddle::platform::get_current_stream(deviceId)->raw_stream();
paddle::platform::stream::get_current_stream(deviceId)->raw_stream();
int64_t numel = 0; // total copy length int64_t numel = 0; // total copy length
int64_t copy_flag = offset_tensor.dims()[0]; int64_t copy_flag = offset_tensor.dims()[0];
...@@ -828,8 +830,7 @@ static PyObject* eager_api_async_write(PyObject* self, ...@@ -828,8 +830,7 @@ static PyObject* eager_api_async_write(PyObject* self,
"except for the first dimension.")); "except for the first dimension."));
} }
auto stream = auto stream = paddle::platform::get_current_stream(deviceId)->raw_stream();
paddle::platform::stream::get_current_stream(deviceId)->raw_stream();
int64_t size = src_tensor.numel() / src_tensor.dims()[0]; int64_t size = src_tensor.numel() / src_tensor.dims()[0];
auto* src_data = src_tensor.data<float>(); auto* src_data = src_tensor.data<float>();
......
...@@ -52,6 +52,7 @@ limitations under the License. */ ...@@ -52,6 +52,7 @@ limitations under the License. */
#include "paddle/fluid/imperative/type_defs.h" #include "paddle/fluid/imperative/type_defs.h"
#include "paddle/fluid/memory/allocation/mmap_allocator.h" #include "paddle/fluid/memory/allocation/mmap_allocator.h"
#include "paddle/fluid/operators/utils.h" #include "paddle/fluid/operators/utils.h"
#include "paddle/fluid/pybind/cuda_streams_py.h"
#include "paddle/fluid/pybind/eager_utils.h" #include "paddle/fluid/pybind/eager_utils.h"
#include "paddle/fluid/pybind/op_function.h" #include "paddle/fluid/pybind/op_function.h"
#include "paddle/fluid/pybind/pybind_variant_caster.h" #include "paddle/fluid/pybind/pybind_variant_caster.h"
...@@ -2796,8 +2797,8 @@ void BindImperative(py::module *m_ptr) { ...@@ -2796,8 +2797,8 @@ void BindImperative(py::module *m_ptr) {
"except for the first dimension.")); "except for the first dimension."));
} }
auto stream = paddle::platform::stream::get_current_stream(deviceId) auto stream =
->raw_stream(); paddle::platform::get_current_stream(deviceId)->raw_stream();
int64_t size = src_tensor.numel() / src_tensor.dims()[0]; int64_t size = src_tensor.numel() / src_tensor.dims()[0];
auto *src_data = src_tensor.data<float>(); auto *src_data = src_tensor.data<float>();
...@@ -2960,8 +2961,8 @@ void BindImperative(py::module *m_ptr) { ...@@ -2960,8 +2961,8 @@ void BindImperative(py::module *m_ptr) {
platform::errors::InvalidArgument( platform::errors::InvalidArgument(
"`index` tensor should be one-dimensional.")); "`index` tensor should be one-dimensional."));
auto stream = paddle::platform::stream::get_current_stream(deviceId) auto stream =
->raw_stream(); paddle::platform::get_current_stream(deviceId)->raw_stream();
int64_t numel = 0; // total copy length int64_t numel = 0; // total copy length
int64_t copy_flag = offset_tensor.dims()[0]; int64_t copy_flag = offset_tensor.dims()[0];
......
...@@ -806,7 +806,7 @@ void BindTensor(pybind11::module &m) { // NOLINT ...@@ -806,7 +806,7 @@ void BindTensor(pybind11::module &m) { // NOLINT
// TODO(ZHUI): use cuda event, to avoid sync. // TODO(ZHUI): use cuda event, to avoid sync.
const auto &device_id = paddle::platform::GetCurrentDeviceId(); const auto &device_id = paddle::platform::GetCurrentDeviceId();
auto stream = auto stream =
paddle::platform::stream::get_current_stream(device_id); paddle::platform::get_current_stream(device_id);
stream->Synchronize(); stream->Synchronize();
int type_idx = static_cast<int>(self.type()); int type_idx = static_cast<int>(self.type());
......
...@@ -30,6 +30,7 @@ limitations under the License. */ ...@@ -30,6 +30,7 @@ limitations under the License. */
#include "paddle/phi/common/float16.h" #include "paddle/phi/common/float16.h"
#include "paddle/phi/common/place.h" #include "paddle/phi/common/place.h"
#include "paddle/phi/core/allocator.h" #include "paddle/phi/core/allocator.h"
#include "paddle/phi/core/cuda_stream.h"
#ifdef PADDLE_WITH_CUDA #ifdef PADDLE_WITH_CUDA
#include "paddle/phi/backends/dynload/cublas.h" #include "paddle/phi/backends/dynload/cublas.h"
...@@ -117,10 +118,10 @@ class EigenGpuStreamDevice : public Eigen::StreamInterface { ...@@ -117,10 +118,10 @@ class EigenGpuStreamDevice : public Eigen::StreamInterface {
semaphore_ = reinterpret_cast<unsigned int*>(scratch); semaphore_ = reinterpret_cast<unsigned int*>(scratch);
#ifdef PADDLE_WITH_HIP #ifdef PADDLE_WITH_HIP
PADDLE_ENFORCE_GPU_SUCCESS( PADDLE_ENFORCE_GPU_SUCCESS(
hipMemsetAsync(semaphore_, 0, sizeof(unsigned int), stream_)); hipMemsetAsync(semaphore_, 0, sizeof(unsigned int), stream()));
#else #else
PADDLE_ENFORCE_GPU_SUCCESS( PADDLE_ENFORCE_GPU_SUCCESS(
cudaMemsetAsync(semaphore_, 0, sizeof(unsigned int), stream_)); cudaMemsetAsync(semaphore_, 0, sizeof(unsigned int), stream()));
#endif #endif
} }
return semaphore_; return semaphore_;
...@@ -212,12 +213,14 @@ struct GPUContext::Impl { ...@@ -212,12 +213,14 @@ struct GPUContext::Impl {
&max_threads_per_mp_, &max_threads_per_mp_,
&max_threads_per_block_, &max_threads_per_block_,
&max_grid_dim_size_); &max_grid_dim_size_);
phi::InitStream(&stream_); stream_ = new CUDAStream(place_);
InitEigenDevice();
InitDnnWorkspace(); InitDnnWorkspace();
} }
void PartialInitWithoutAllocator() { void PartialInitWithoutAllocator() {
owned_ = true; owned_ = true;
stream_owned_ = true;
backends::gpu::GPUDeviceGuard guard(place_.device); backends::gpu::GPUDeviceGuard guard(place_.device);
phi::InitGpuProperties(place_, phi::InitGpuProperties(place_,
&compute_capability_, &compute_capability_,
...@@ -227,11 +230,12 @@ struct GPUContext::Impl { ...@@ -227,11 +230,12 @@ struct GPUContext::Impl {
&max_threads_per_mp_, &max_threads_per_mp_,
&max_threads_per_block_, &max_threads_per_block_,
&max_grid_dim_size_); &max_grid_dim_size_);
phi::InitStream(&stream_); stream_ = new CUDAStream(place_);
} }
void PartialInitWithAllocator() { void PartialInitWithAllocator() {
owned_ = true; owned_ = true;
stream_owned_ = true;
backends::gpu::GPUDeviceGuard guard(place_.device); backends::gpu::GPUDeviceGuard guard(place_.device);
InitDnnWorkspace(); InitDnnWorkspace();
} }
...@@ -257,7 +261,9 @@ struct GPUContext::Impl { ...@@ -257,7 +261,9 @@ struct GPUContext::Impl {
phi::DestroyBlasHandle(blas_tensor_core_handle_); phi::DestroyBlasHandle(blas_tensor_core_handle_);
phi::DestroyBlasHandle(blas_tf32_tensor_core_handle_); phi::DestroyBlasHandle(blas_tf32_tensor_core_handle_);
phi::DestroyBlasLtHandle(blaslt_handle_); phi::DestroyBlasLtHandle(blaslt_handle_);
phi::DestoryStream(stream_); }
if (stream_owned_ && stream_) {
delete stream_;
} }
} }
...@@ -270,7 +276,7 @@ struct GPUContext::Impl { ...@@ -270,7 +276,7 @@ struct GPUContext::Impl {
void InitDnnWorkspace() { void InitDnnWorkspace() {
PD_CHECK(allocator_ != nullptr, PD_CHECK(allocator_ != nullptr,
"the device allocator for gpu context is nullptr."); "the device allocator for gpu context is nullptr.");
workspace_ = new DnnWorkspaceHandle(allocator_, stream_); workspace_ = new DnnWorkspaceHandle(allocator_, stream());
} }
void DestoryInternalWorkspace() { void DestoryInternalWorkspace() {
...@@ -288,12 +294,34 @@ struct GPUContext::Impl { ...@@ -288,12 +294,34 @@ struct GPUContext::Impl {
DnnWorkspaceHandle GetDnnWorkspace() { DnnWorkspaceHandle GetDnnWorkspace() {
PD_CHECK(allocator_ != nullptr, PD_CHECK(allocator_ != nullptr,
"the device allocator for gpu context is nullptr."); "the device allocator for gpu context is nullptr.");
return DnnWorkspaceHandle(allocator_, stream_); return DnnWorkspaceHandle(allocator_, stream());
}
void SetStream(gpuStream_t stream) {
if (stream_ == nullptr) {
auto s = Stream(reinterpret_cast<StreamId>(stream));
stream_ = new CUDAStream(place_, s);
stream_owned_ = true;
}
stream_->set_raw_stream(stream);
} }
void SetStream(gpuStream_t stream) { stream_ = stream; } void SetCUDAStream(CUDAStream* stream, bool clear = true) {
if (clear && stream_owned_ && stream_) {
delete stream_;
}
stream_owned_ = false;
stream_ = stream;
// TODO(phi): reset related handles?
}
gpuStream_t GetStream() const { gpuStream_t stream() const {
auto s = stream_->raw_stream();
PD_CHECK(s != nullptr, "the gpu stream is nullptr.");
return s;
}
CUDAStream* cuda_stream() const {
PD_CHECK(stream_ != nullptr, "the gpu stream is nullptr."); PD_CHECK(stream_ != nullptr, "the gpu stream is nullptr.");
return stream_; return stream_;
} }
...@@ -302,7 +330,7 @@ struct GPUContext::Impl { ...@@ -302,7 +330,7 @@ struct GPUContext::Impl {
PD_CHECK(allocator_ != nullptr, PD_CHECK(allocator_ != nullptr,
"the allocator for eigen device is nullptr."); "the allocator for eigen device is nullptr.");
eigen_stream_.reset(new internal::EigenGpuStreamDevice()); eigen_stream_.reset(new internal::EigenGpuStreamDevice());
eigen_stream_->Reinitialize(stream_, allocator_, place_); eigen_stream_->Reinitialize(stream(), allocator_, place_);
eigen_device_ = new Eigen::GpuDevice(eigen_stream_.get()); eigen_device_ = new Eigen::GpuDevice(eigen_stream_.get());
} }
...@@ -335,29 +363,32 @@ struct GPUContext::Impl { ...@@ -335,29 +363,32 @@ struct GPUContext::Impl {
blasHandle_t GetBlasHandle() { blasHandle_t GetBlasHandle() {
std::call_once(flag_blas_, [&]() { std::call_once(flag_blas_, [&]() {
if (!blas_handle_) { if (!blas_handle_) {
if (!blas_handle_creator_) if (!blas_handle_creator_) {
phi::InitBlasHandle(&blas_handle_, stream_); phi::InitBlasHandle(&blas_handle_, stream());
else } else {
blas_handle_ = blas_handle_creator_(); blas_handle_ = blas_handle_creator_();
} }
}
#ifdef PADDLE_WITH_CUDA #ifdef PADDLE_WITH_CUDA
#if CUDA_VERSION >= 9000 #if CUDA_VERSION >= 9000
if (!blas_tensor_core_handle_) { if (!blas_tensor_core_handle_) {
if (!blas_tensor_core_handle_creator_) if (!blas_tensor_core_handle_creator_) {
phi::InitBlasHandle(&blas_tensor_core_handle_, stream_); phi::InitBlasHandle(&blas_tensor_core_handle_, stream());
else } else {
blas_tensor_core_handle_ = blas_tensor_core_handle_creator_(); blas_tensor_core_handle_ = blas_tensor_core_handle_creator_();
}
PADDLE_RETRY_CUDA_SUCCESS(phi::dynload::cublasSetMathMode( PADDLE_RETRY_CUDA_SUCCESS(phi::dynload::cublasSetMathMode(
blas_tensor_core_handle_, CUBLAS_TENSOR_OP_MATH)); blas_tensor_core_handle_, CUBLAS_TENSOR_OP_MATH));
} }
#endif #endif
#if CUDA_VERSION >= 11000 #if CUDA_VERSION >= 11000
if (!blas_tf32_tensor_core_handle_) { if (!blas_tf32_tensor_core_handle_) {
if (!blas_tf32_tensor_core_handle_creator_) if (!blas_tf32_tensor_core_handle_creator_) {
phi::InitBlasHandle(&blas_tf32_tensor_core_handle_, stream_); phi::InitBlasHandle(&blas_tf32_tensor_core_handle_, stream());
else } else {
blas_tf32_tensor_core_handle_ = blas_tf32_tensor_core_handle_ =
blas_tf32_tensor_core_handle_creator_(); blas_tf32_tensor_core_handle_creator_();
}
PADDLE_RETRY_CUDA_SUCCESS(phi::dynload::cublasSetMathMode( PADDLE_RETRY_CUDA_SUCCESS(phi::dynload::cublasSetMathMode(
blas_tf32_tensor_core_handle_, CUBLAS_TF32_TENSOR_OP_MATH)); blas_tf32_tensor_core_handle_, CUBLAS_TF32_TENSOR_OP_MATH));
} }
...@@ -412,11 +443,12 @@ struct GPUContext::Impl { ...@@ -412,11 +443,12 @@ struct GPUContext::Impl {
dnnHandle_t GetDnnHandle() { dnnHandle_t GetDnnHandle() {
std::call_once(flag_dnn_, [&]() { std::call_once(flag_dnn_, [&]() {
if (!dnn_handle_) { if (!dnn_handle_) {
if (!dnn_handle_creator_) if (!dnn_handle_creator_) {
phi::InitDnnHandle(&dnn_handle_, stream_, place_); phi::InitDnnHandle(&dnn_handle_, stream(), place_);
else } else {
dnn_handle_ = dnn_handle_creator_(); dnn_handle_ = dnn_handle_creator_();
} }
}
}); });
PD_CHECK(dnn_handle_ != nullptr, "the gpu dnn handle is nullptr."); PD_CHECK(dnn_handle_ != nullptr, "the gpu dnn handle is nullptr.");
return dnn_handle_; return dnn_handle_;
...@@ -445,11 +477,12 @@ struct GPUContext::Impl { ...@@ -445,11 +477,12 @@ struct GPUContext::Impl {
solverHandle_t GetSolverHandle() { solverHandle_t GetSolverHandle() {
std::call_once(flag_slover_, [&]() { std::call_once(flag_slover_, [&]() {
if (!solver_handle_) { if (!solver_handle_) {
if (!solver_handle_creator_) if (!solver_handle_creator_) {
phi::InitSolverHandle(&solver_handle_, stream_); phi::InitSolverHandle(&solver_handle_, stream());
else } else {
solver_handle_ = solver_handle_creator_(); solver_handle_ = solver_handle_creator_();
} }
}
}); });
PD_CHECK(solver_handle_ != nullptr, "the gpu solver handle is nullptr."); PD_CHECK(solver_handle_ != nullptr, "the gpu solver handle is nullptr.");
return solver_handle_; return solver_handle_;
...@@ -464,11 +497,12 @@ struct GPUContext::Impl { ...@@ -464,11 +497,12 @@ struct GPUContext::Impl {
sparseHandle_t GetSparseHandle() { sparseHandle_t GetSparseHandle() {
std::call_once(flag_sparse_, [&]() { std::call_once(flag_sparse_, [&]() {
if (!sparse_handle_) { if (!sparse_handle_) {
if (!sparse_handle_creator_) if (!sparse_handle_creator_) {
phi::InitSparseHandle(&sparse_handle_, stream_); phi::InitSparseHandle(&sparse_handle_, stream());
else } else {
sparse_handle_ = sparse_handle_creator_(); sparse_handle_ = sparse_handle_creator_();
} }
}
}); });
PD_CHECK(sparse_handle_ != nullptr, "the gpu sparse handle is nullptr."); PD_CHECK(sparse_handle_ != nullptr, "the gpu sparse handle is nullptr.");
return sparse_handle_; return sparse_handle_;
...@@ -484,9 +518,9 @@ struct GPUContext::Impl { ...@@ -484,9 +518,9 @@ struct GPUContext::Impl {
#ifdef PADDLE_WITH_HIP #ifdef PADDLE_WITH_HIP
hipError_t e_sync = hipSuccess; hipError_t e_sync = hipSuccess;
#if !defined(_WIN32) #if !defined(_WIN32)
e_sync = hipStreamSynchronize(stream_); e_sync = hipStreamSynchronize(stream());
#else #else
while (e_sync = hipStreamQuery(stream_)) { while (e_sync = hipStreamQuery(stream())) {
if (e_sync == hipErrorNotReady) continue; if (e_sync == hipErrorNotReady) continue;
break; break;
} }
...@@ -494,9 +528,9 @@ struct GPUContext::Impl { ...@@ -494,9 +528,9 @@ struct GPUContext::Impl {
#else // PADDLE_WITH_HIP #else // PADDLE_WITH_HIP
cudaError_t e_sync = cudaSuccess; cudaError_t e_sync = cudaSuccess;
#if !defined(_WIN32) #if !defined(_WIN32)
e_sync = cudaStreamSynchronize(stream_); e_sync = cudaStreamSynchronize(stream());
#else #else
while (e_sync = cudaStreamQuery(stream_)) { while (e_sync = cudaStreamQuery(stream())) {
if (e_sync == cudaErrorNotReady) continue; if (e_sync == cudaErrorNotReady) continue;
break; break;
} }
...@@ -508,9 +542,9 @@ struct GPUContext::Impl { ...@@ -508,9 +542,9 @@ struct GPUContext::Impl {
void WaitEvent(gpuEvent_t ev) const { void WaitEvent(gpuEvent_t ev) const {
#ifdef PADDLE_WITH_HIP #ifdef PADDLE_WITH_HIP
PADDLE_ENFORCE_GPU_SUCCESS(hipStreamWaitEvent(stream_, ev, 0)); PADDLE_ENFORCE_GPU_SUCCESS(hipStreamWaitEvent(stream(), ev, 0));
#else #else
PADDLE_ENFORCE_GPU_SUCCESS(cudaStreamWaitEvent(stream_, ev, 0)); PADDLE_ENFORCE_GPU_SUCCESS(cudaStreamWaitEvent(stream(), ev, 0));
#endif #endif
} }
...@@ -531,29 +565,32 @@ struct GPUContext::Impl { ...@@ -531,29 +565,32 @@ struct GPUContext::Impl {
inline void CublasCall(const std::function<void(blasHandle_t)>& callback) { inline void CublasCall(const std::function<void(blasHandle_t)>& callback) {
std::call_once(flag_cublas_, [&]() { std::call_once(flag_cublas_, [&]() {
if (!blas_handle_) { if (!blas_handle_) {
if (!blas_handle_creator_) if (!blas_handle_creator_) {
phi::InitBlasHandle(&blas_handle_, stream_); phi::InitBlasHandle(&blas_handle_, stream());
else } else {
blas_handle_ = blas_handle_creator_(); blas_handle_ = blas_handle_creator_();
} }
}
#ifdef PADDLE_WITH_CUDA #ifdef PADDLE_WITH_CUDA
#if CUDA_VERSION >= 9000 #if CUDA_VERSION >= 9000
if (!blas_tensor_core_handle_) { if (!blas_tensor_core_handle_) {
if (!blas_tensor_core_handle_creator_) if (!blas_tensor_core_handle_creator_) {
phi::InitBlasHandle(&blas_tensor_core_handle_, stream_); phi::InitBlasHandle(&blas_tensor_core_handle_, stream());
else } else {
blas_tensor_core_handle_ = blas_tensor_core_handle_creator_(); phi::InitBlasHandle(&blas_tensor_core_handle_, stream());
}
PADDLE_RETRY_CUDA_SUCCESS(phi::dynload::cublasSetMathMode( PADDLE_RETRY_CUDA_SUCCESS(phi::dynload::cublasSetMathMode(
blas_tensor_core_handle_, CUBLAS_TENSOR_OP_MATH)); blas_tensor_core_handle_, CUBLAS_TENSOR_OP_MATH));
} }
#endif #endif
#if CUDA_VERSION >= 11000 #if CUDA_VERSION >= 11000
if (!blas_tf32_tensor_core_handle_) { if (!blas_tf32_tensor_core_handle_) {
if (!blas_tf32_tensor_core_handle_creator_) if (!blas_tf32_tensor_core_handle_creator_) {
phi::InitBlasHandle(&blas_tf32_tensor_core_handle_, stream_); phi::InitBlasHandle(&blas_tf32_tensor_core_handle_, stream());
else } else {
blas_tf32_tensor_core_handle_ = blas_tf32_tensor_core_handle_ =
blas_tf32_tensor_core_handle_creator_(); blas_tf32_tensor_core_handle_creator_();
}
PADDLE_RETRY_CUDA_SUCCESS(phi::dynload::cublasSetMathMode( PADDLE_RETRY_CUDA_SUCCESS(phi::dynload::cublasSetMathMode(
blas_tf32_tensor_core_handle_, CUBLAS_TF32_TENSOR_OP_MATH)); blas_tf32_tensor_core_handle_, CUBLAS_TF32_TENSOR_OP_MATH));
} }
...@@ -573,29 +610,32 @@ struct GPUContext::Impl { ...@@ -573,29 +610,32 @@ struct GPUContext::Impl {
const std::function<void(blasHandle_t)>& callback) { const std::function<void(blasHandle_t)>& callback) {
std::call_once(flag_tensorcore_cublas_, [&]() { std::call_once(flag_tensorcore_cublas_, [&]() {
if (!blas_handle_) { if (!blas_handle_) {
if (!blas_handle_creator_) if (!blas_handle_creator_) {
phi::InitBlasHandle(&blas_handle_, stream_); phi::InitBlasHandle(&blas_handle_, stream());
else } else {
blas_handle_ = blas_handle_creator_(); blas_handle_ = blas_handle_creator_();
} }
}
#ifdef PADDLE_WITH_CUDA #ifdef PADDLE_WITH_CUDA
#if CUDA_VERSION >= 9000 #if CUDA_VERSION >= 9000
if (!blas_tensor_core_handle_) { if (!blas_tensor_core_handle_) {
if (!blas_tensor_core_handle_creator_) if (!blas_tensor_core_handle_creator_) {
phi::InitBlasHandle(&blas_tensor_core_handle_, stream_); phi::InitBlasHandle(&blas_tensor_core_handle_, stream());
else } else {
blas_tensor_core_handle_ = blas_tensor_core_handle_creator_(); blas_tensor_core_handle_ = blas_tensor_core_handle_creator_();
}
PADDLE_RETRY_CUDA_SUCCESS(phi::dynload::cublasSetMathMode( PADDLE_RETRY_CUDA_SUCCESS(phi::dynload::cublasSetMathMode(
blas_tensor_core_handle_, CUBLAS_TENSOR_OP_MATH)); blas_tensor_core_handle_, CUBLAS_TENSOR_OP_MATH));
} }
#endif #endif
#if CUDA_VERSION >= 11000 #if CUDA_VERSION >= 11000
if (!blas_tf32_tensor_core_handle_) { if (!blas_tf32_tensor_core_handle_) {
if (!blas_tf32_tensor_core_handle_creator_) if (!blas_tf32_tensor_core_handle_creator_) {
phi::InitBlasHandle(&blas_tf32_tensor_core_handle_, stream_); phi::InitBlasHandle(&blas_tf32_tensor_core_handle_, stream());
else } else {
blas_tf32_tensor_core_handle_ = blas_tf32_tensor_core_handle_ =
blas_tf32_tensor_core_handle_creator_(); blas_tf32_tensor_core_handle_creator_();
}
PADDLE_RETRY_CUDA_SUCCESS(phi::dynload::cublasSetMathMode( PADDLE_RETRY_CUDA_SUCCESS(phi::dynload::cublasSetMathMode(
blas_tf32_tensor_core_handle_, CUBLAS_TF32_TENSOR_OP_MATH)); blas_tf32_tensor_core_handle_, CUBLAS_TF32_TENSOR_OP_MATH));
} }
...@@ -615,11 +655,12 @@ struct GPUContext::Impl { ...@@ -615,11 +655,12 @@ struct GPUContext::Impl {
const std::function<void(sparseHandle_t)>& callback) { const std::function<void(sparseHandle_t)>& callback) {
std::call_once(flag_sparse_, [&]() { std::call_once(flag_sparse_, [&]() {
if (!sparse_handle_) { if (!sparse_handle_) {
if (!sparse_handle_creator_) if (!sparse_handle_creator_) {
phi::InitSparseHandle(&sparse_handle_, stream_); phi::InitSparseHandle(&sparse_handle_, stream());
else } else {
sparse_handle_ = sparse_handle_creator_(); sparse_handle_ = sparse_handle_creator_();
} }
}
}); });
std::lock_guard<std::mutex> guard(sparse_mtx_); std::lock_guard<std::mutex> guard(sparse_mtx_);
callback(sparse_handle_); callback(sparse_handle_);
...@@ -632,9 +673,9 @@ struct GPUContext::Impl { ...@@ -632,9 +673,9 @@ struct GPUContext::Impl {
void RecordEvent(gpuEvent_t ev) const { void RecordEvent(gpuEvent_t ev) const {
#ifdef PADDLE_WITH_HIP #ifdef PADDLE_WITH_HIP
PADDLE_ENFORCE_GPU_SUCCESS(hipEventRecord(ev, stream_)); PADDLE_ENFORCE_GPU_SUCCESS(hipEventRecord(ev, stream()));
#else #else
PADDLE_ENFORCE_GPU_SUCCESS(cudaEventRecord(ev, stream_)); PADDLE_ENFORCE_GPU_SUCCESS(cudaEventRecord(ev, stream()));
#endif #endif
} }
...@@ -653,22 +694,22 @@ struct GPUContext::Impl { ...@@ -653,22 +694,22 @@ struct GPUContext::Impl {
#ifdef PADDLE_WITH_HIP #ifdef PADDLE_WITH_HIP
PADDLE_ENFORCE_GPU_SUCCESS( PADDLE_ENFORCE_GPU_SUCCESS(
hipStreamAddCallback(stream_, internal::StreamCallbackFunc, func, 0)); hipStreamAddCallback(stream(), internal::StreamCallbackFunc, func, 0));
#endif #endif
#ifdef PADDLE_WITH_CUDA #ifdef PADDLE_WITH_CUDA
#if CUDA_VERSION >= 10000 #if CUDA_VERSION >= 10000
PADDLE_ENFORCE_GPU_SUCCESS( PADDLE_ENFORCE_GPU_SUCCESS(
cudaLaunchHostFunc(stream_, internal::StreamCallbackFunc, func)); cudaLaunchHostFunc(stream(), internal::StreamCallbackFunc, func));
#else #else
PADDLE_ENFORCE_GPU_SUCCESS( PADDLE_ENFORCE_GPU_SUCCESS(
cudaStreamAddCallback(stream_, internal::StreamCallbackFunc, func, 0)); cudaStreamAddCallback(stream(), internal::StreamCallbackFunc, func, 0));
#endif #endif
#endif #endif
} }
void WaitStreamCallback() const { void WaitStreamCallback() const {
#if defined(PADDLE_WITH_HIP) || defined(PADDLE_WITH_CUDA) #if defined(PADDLE_WITH_HIP) || defined(PADDLE_WITH_CUDA)
phi::backends::gpu::GpuStreamSync(stream_); phi::backends::gpu::GpuStreamSync(stream());
#endif #endif
{ {
std::lock_guard<std::mutex> lock(stream_call_back_mtx_); std::lock_guard<std::mutex> lock(stream_call_back_mtx_);
...@@ -678,7 +719,10 @@ struct GPUContext::Impl { ...@@ -678,7 +719,10 @@ struct GPUContext::Impl {
} }
} }
// use one flag for all handles?
// they should be accessed consistently
bool owned_{false}; bool owned_{false};
bool stream_owned_{false};
Place place_; Place place_;
int compute_capability_; int compute_capability_;
int runtime_version_; int runtime_version_;
...@@ -688,7 +732,7 @@ struct GPUContext::Impl { ...@@ -688,7 +732,7 @@ struct GPUContext::Impl {
int max_threads_per_block_; int max_threads_per_block_;
std::array<int, 3> max_grid_dim_size_; std::array<int, 3> max_grid_dim_size_;
gpuStream_t stream_{nullptr}; CUDAStream* stream_{nullptr};
Eigen::GpuDevice* eigen_device_{nullptr}; Eigen::GpuDevice* eigen_device_{nullptr};
std::function<Eigen::GpuDevice*()> eigen_device_creator_{nullptr}; std::function<Eigen::GpuDevice*()> eigen_device_creator_{nullptr};
blasHandle_t blas_handle_{nullptr}; blasHandle_t blas_handle_{nullptr};
...@@ -746,14 +790,20 @@ GPUContext::GPUContext(GPUContext&&) = default; ...@@ -746,14 +790,20 @@ GPUContext::GPUContext(GPUContext&&) = default;
GPUContext& GPUContext::operator=(GPUContext&&) = default; GPUContext& GPUContext::operator=(GPUContext&&) = default;
GPUContext::GPUContext(const GPUPlace& place) GPUContext::GPUContext(const GPUPlace& place, bool init)
: DeviceContext(), impl_(std::make_unique<Impl>(place)) {} : DeviceContext(), impl_(std::make_unique<Impl>(place)) {
if (init) {
impl_->PartialInitWithoutAllocator();
}
}
GPUContext::~GPUContext() = default; GPUContext::~GPUContext() = default;
const Place& GPUContext::GetPlace() const { return impl_->GetPlace(); } const Place& GPUContext::GetPlace() const { return impl_->GetPlace(); }
gpuStream_t GPUContext::stream() const { return impl_->GetStream(); } gpuStream_t GPUContext::stream() const { return impl_->stream(); }
CUDAStream* GPUContext::cuda_stream() const { return impl_->cuda_stream(); }
dnnHandle_t GPUContext::cudnn_handle() const { return impl_->GetDnnHandle(); } dnnHandle_t GPUContext::cudnn_handle() const { return impl_->GetDnnHandle(); }
...@@ -850,6 +900,11 @@ void GPUContext::SetStream(gpuStream_t stream) { ...@@ -850,6 +900,11 @@ void GPUContext::SetStream(gpuStream_t stream) {
impl_->SetStream(stream); impl_->SetStream(stream);
} }
void GPUContext::SetCUDAStream(CUDAStream* stream, bool clear) {
impl_->allocator_ = const_cast<Allocator*>(&this->GetAllocator());
impl_->SetCUDAStream(stream, clear);
}
void GPUContext::SetEigenDevice(Eigen::GpuDevice* device) { void GPUContext::SetEigenDevice(Eigen::GpuDevice* device) {
impl_->SetEigenDevice(device); impl_->SetEigenDevice(device);
} }
......
...@@ -28,6 +28,8 @@ limitations under the License. */ ...@@ -28,6 +28,8 @@ limitations under the License. */
namespace phi { namespace phi {
class CUDAStream;
class DnnWorkspaceHandle { class DnnWorkspaceHandle {
public: public:
inline DnnWorkspaceHandle(Allocator* allocator, gpuStream_t stream) inline DnnWorkspaceHandle(Allocator* allocator, gpuStream_t stream)
...@@ -78,11 +80,11 @@ class DnnWorkspaceHandle { ...@@ -78,11 +80,11 @@ class DnnWorkspaceHandle {
class PADDLE_API GPUContext : public DeviceContext { class PADDLE_API GPUContext : public DeviceContext {
public: public:
GPUContext(); GPUContext();
explicit GPUContext(const GPUPlace& place, bool init = true);
GPUContext(GPUContext&&); GPUContext(GPUContext&&);
GPUContext& operator=(GPUContext&&); GPUContext& operator=(GPUContext&&);
explicit GPUContext(const GPUPlace& place);
virtual ~GPUContext(); virtual ~GPUContext();
/*! \brief Return place in the device context. */ /*! \brief Return place in the device context. */
...@@ -91,6 +93,9 @@ class PADDLE_API GPUContext : public DeviceContext { ...@@ -91,6 +93,9 @@ class PADDLE_API GPUContext : public DeviceContext {
/*! \brief Return gpu stream in the device context. */ /*! \brief Return gpu stream in the device context. */
gpuStream_t stream() const; gpuStream_t stream() const;
/*! \brief Return CUDAStream in the device context. */
CUDAStream* cuda_stream() const;
/*! \brief Return cudnn handle in the device context. */ /*! \brief Return cudnn handle in the device context. */
dnnHandle_t cudnn_handle() const; dnnHandle_t cudnn_handle() const;
...@@ -190,6 +195,11 @@ class PADDLE_API GPUContext : public DeviceContext { ...@@ -190,6 +195,11 @@ class PADDLE_API GPUContext : public DeviceContext {
// called. // called.
void PartialInitWithAllocator(); void PartialInitWithAllocator();
// Note that this function is a trick implementation since all 'set' methods
// are protected by default.
// clear: whether clear the original CUDAStream or not
void SetCUDAStream(CUDAStream*, bool clear = true);
protected: protected:
// NOTE: External users manage resources. Used in inference scenarios. // NOTE: External users manage resources. Used in inference scenarios.
// The Set interface is for inference only, DeviceContext will mark the // The Set interface is for inference only, DeviceContext will mark the
......
/* Copyright (c) 2022 PaddlePaddle Authors. All Rights Reserved.
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 "paddle/phi/backends/gpu/gpu_info.h"
#include "paddle/phi/common/place.h"
#include "paddle/phi/core/stream.h"
#ifdef PADDLE_WITH_CUDA
#include <cuda_runtime.h>
using gpuStream_t = cudaStream_t;
#endif
#ifdef PADDLE_WITH_HIP
#include <hip/hip_runtime.h>
using gpuStream_t = hipStream_t;
#endif
// TODO(phi): remove fluid headers.
#include "paddle/fluid/platform/enforce.h"
namespace phi {
// Currently, CudaStream is used in python-side API only
class CUDAStream {
public:
enum class Priority : uint8_t {
kNull = 0x0,
kHigh = 0x1,
kNormal = 0x2,
};
enum class StreamFlag : uint8_t {
kDefaultFlag = 0x0,
kStreamNonBlocking = 0x1,
};
public:
CUDAStream(const Place& place, const Stream& stream)
: place_(place), stream_(stream) {}
CUDAStream(const Place& place,
const Priority& priority = Priority::kNormal,
const StreamFlag& flag = StreamFlag::kDefaultFlag) {
place_ = place;
gpuStream_t stream = nullptr;
backends::gpu::GPUDeviceGuard guard(place_.device);
if (priority == Priority::kHigh) {
#ifdef PADDLE_WITH_HIP
PADDLE_ENFORCE_GPU_SUCCESS(hipStreamCreateWithPriority(
&stream, static_cast<unsigned int>(flag), -1));
#else
PADDLE_ENFORCE_GPU_SUCCESS(cudaStreamCreateWithPriority(
&stream, static_cast<unsigned int>(flag), -1));
#endif
} else if (priority == Priority::kNormal) {
#ifdef PADDLE_WITH_HIP
PADDLE_ENFORCE_GPU_SUCCESS(hipStreamCreateWithPriority(
&stream, static_cast<unsigned int>(flag), 0));
#else
PADDLE_ENFORCE_GPU_SUCCESS(cudaStreamCreateWithPriority(
&stream, static_cast<unsigned int>(flag), 0));
#endif
}
VLOG(10) << "CUDAStream " << stream;
stream_ = Stream(reinterpret_cast<StreamId>(stream));
owned_ = true;
}
gpuStream_t raw_stream() const { return reinterpret_cast<gpuStream_t>(id()); }
void set_raw_stream(gpuStream_t stream) {
if (owned_ && stream_.id() != 0) {
backends::gpu::GPUDeviceGuard guard(place_.device);
#ifdef PADDLE_WITH_HIP
PADDLE_ENFORCE_GPU_SUCCESS(hipStreamDestroy(raw_stream()));
#else
PADDLE_ENFORCE_GPU_SUCCESS(cudaStreamDestroy(raw_stream()));
#endif
}
stream_ = Stream(reinterpret_cast<StreamId>(stream));
}
StreamId id() const { return stream_.id(); }
Place place() const { return place_; }
bool Query() const {
#ifdef PADDLE_WITH_HIP
hipError_t err = hipStreamQuery(raw_stream());
if (err == hipSuccess) {
return true;
}
if (err == hipErrorNotReady) {
return false;
}
#else
cudaError_t err = cudaStreamQuery(raw_stream());
if (err == cudaSuccess) {
return true;
}
if (err == cudaErrorNotReady) {
return false;
}
#endif
PADDLE_ENFORCE_GPU_SUCCESS(err);
return false;
}
void Synchronize() const {
VLOG(10) << "Synchronize " << raw_stream();
backends::gpu::GpuStreamSync(raw_stream());
}
void WaitEvent(gpuEvent_t ev) const {
#ifdef PADDLE_WITH_HIP
PADDLE_ENFORCE_GPU_SUCCESS(hipStreamWaitEvent(raw_stream(), ev, 0));
#else
PADDLE_ENFORCE_GPU_SUCCESS(cudaStreamWaitEvent(raw_stream(), ev, 0));
#endif
}
~CUDAStream() {
VLOG(10) << "~CUDAStream " << raw_stream();
Synchronize();
if (owned_ && stream_.id() != 0) {
backends::gpu::GPUDeviceGuard guard(place_.device);
#ifdef PADDLE_WITH_HIP
hipStreamDestroy(raw_stream());
#else
cudaStreamDestroy(raw_stream());
#endif
}
}
private:
Place place_;
Stream stream_;
bool owned_{false}; // whether the stream is created and onwed by self
};
} // namespace phi
...@@ -14,19 +14,19 @@ limitations under the License. */ ...@@ -14,19 +14,19 @@ limitations under the License. */
#pragma once #pragma once
#include <cstdint>
#include <memory>
namespace phi { namespace phi {
using StreamId = uint64_t; using StreamId = uint64_t;
// device-agnostic abstraction of any-stream
class Stream final { class Stream final {
public: public:
Stream() {}
explicit Stream(StreamId id) : id_(id) {} explicit Stream(StreamId id) : id_(id) {}
StreamId id() const { return id_; } StreamId id() const { return id_; }
private: private:
StreamId id_; StreamId id_{0}; // not onwed the stream
}; };
} // namespace phi } // namespace phi
...@@ -124,6 +124,9 @@ class TestStreamGuard(unittest.TestCase): ...@@ -124,6 +124,9 @@ class TestStreamGuard(unittest.TestCase):
c = a + b c = a + b
with paddle.device.cuda.stream_guard(s): with paddle.device.cuda.stream_guard(s):
d = a + b d = a + b
# NOTE(zhiqiu): it is strange that cudaMemcpy d2h not waits all
# kernels to be completed on windows.
s.synchronize()
self.assertTrue(np.array_equal(np.array(c), np.array(d))) self.assertTrue(np.array_equal(np.array(c), np.array(d)))
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册