device_context.h 11.3 KB
Newer Older
1
/* Copyright (c) 2016 PaddlePaddle Authors. All Rights Reserved.
Q
QI JUN 已提交
2 3 4 5 6 7 8 9 10 11 12
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

13
#include <future>  // NOLINT
D
dzhwinter 已提交
14
#include <memory>
Y
yuyang18 已提交
15
#include <mutex>  // NOLINT
16
#include <string>
D
dzhwinter 已提交
17
#include <unordered_map>
18
#include <utility>
19
#include <vector>
Y
Yu Yang 已提交
20
#include "paddle/fluid/memory/malloc.h"
21
#include "paddle/fluid/platform/temporary_allocator.h"
22
#ifdef PADDLE_WITH_CUDA
Y
Yi Wang 已提交
23 24 25
#include "paddle/fluid/platform/dynload/cublas.h"
#include "paddle/fluid/platform/dynload/cudnn.h"
#include "paddle/fluid/platform/gpu_info.h"
Q
QI JUN 已提交
26
#endif
D
dzhwinter 已提交
27

T
tensor-tang 已提交
28
#ifdef PADDLE_WITH_MKLDNN
L
luotao1 已提交
29
#include "mkldnn.hpp"
T
tensor-tang 已提交
30 31
#endif

32 33
#include <map>
#include "glog/logging.h"
Y
Yi Wang 已提交
34 35
#include "paddle/fluid/platform/enforce.h"
#include "paddle/fluid/platform/place.h"
S
sneaxiy 已提交
36 37 38
#ifdef PADDLE_WITH_CUDA
#include "paddle/fluid/platform/stream_callback_manager.h"
#endif
Q
qijun 已提交
39
#include "unsupported/Eigen/CXX11/Tensor"
Q
QI JUN 已提交
40 41 42 43

namespace paddle {
namespace platform {

44 45 46 47 48 49 50 51 52 53 54 55 56 57 58 59 60 61 62 63 64 65 66 67 68 69 70 71 72 73 74 75 76 77 78 79 80 81 82 83 84 85 86 87
/*! \brief device temporary allocator singleton */
class DeviceTemporaryAllocator {
 public:
  static DeviceTemporaryAllocator& Instance() {
    PADDLE_ENFORCE_NOT_NULL(allocators,
                            "Need to Create DeviceTemporaryAllocator first!");
    return *allocators;
  }

  static DeviceTemporaryAllocator& Init() {
    if (allocators == nullptr) {
      allocators = new DeviceTemporaryAllocator();
    }
    return *allocators;
  }

/*! \brief  Return handle of single temporary allocator. */
#ifdef PADDLE_WITH_CUDA
  platform::TemporaryAllocator& Get(const platform::Place& place,
                                    const cudaStream_t& stream);
#endif
  template <typename DeviceContext>
  platform::TemporaryAllocator& Get(const DeviceContext& dev_ctx);

  platform::TemporaryAllocator& Get(const platform::Place& place);

 private:
  DeviceTemporaryAllocator() : cpu_allocator_(platform::CPUPlace()) {}

  static DeviceTemporaryAllocator* allocators;

  platform::TemporaryAllocator cpu_allocator_;

#ifdef PADDLE_WITH_CUDA
  std::map<std::pair<platform::Place, cudaStream_t>,
           std::unique_ptr<platform::TemporaryAllocator>>
      device_allocator_;
#endif

  std::mutex mtx_;

  DISABLE_COPY_AND_ASSIGN(DeviceTemporaryAllocator);
};

Q
QI JUN 已提交
88 89 90
class DeviceContext {
 public:
  virtual ~DeviceContext() {}
L
liaogang 已提交
91
  virtual Place GetPlace() const = 0;
Q
QI JUN 已提交
92

93
  virtual void Wait() const {}
Q
QI JUN 已提交
94 95
};

Q
qijun 已提交
96 97
class CPUDeviceContext : public DeviceContext {
 public:
98
  CPUDeviceContext();
Q
qijun 已提交
99
  explicit CPUDeviceContext(CPUPlace place);
Q
qijun 已提交
100

101
  Eigen::DefaultDevice* eigen_device() const;
Q
qijun 已提交
102

L
liaogang 已提交
103
  Place GetPlace() const override;
Y
Yu Yang 已提交
104

Q
qijun 已提交
105
 private:
D
dzhwinter 已提交
106
  CPUPlace place_;
Q
qijun 已提交
107
  std::unique_ptr<Eigen::DefaultDevice> eigen_device_;
Q
QI JUN 已提交
108 109
};

Y
Yang Yu 已提交
110 111 112 113 114 115 116 117
template <typename Place>
struct DefaultDeviceContextType;

template <>
struct DefaultDeviceContextType<platform::CPUPlace> {
  using TYPE = CPUDeviceContext;
};

118
#ifdef PADDLE_WITH_CUDA
119

Q
qijun 已提交
120
class EigenCudaStreamDevice;
S
sneaxiy 已提交
121 122 123 124 125 126 127 128 129 130 131 132
class CudnnHolder {
 public:
  CudnnHolder(const cudaStream_t* stream, const CUDAPlace& place);
  ~CudnnHolder();
  cudnnHandle_t cudnn_handle() const { return cudnn_handle_; }

 private:
  friend class CudnnWorkspaceHandle;
  void ReallocateWorkspace(size_t required_workspace_len);

  template <typename Callback>
  void RunFuncImpl(Callback&& cudnn_func, size_t required_workspace_len) {
Y
Yu Yang 已提交
133
    if (required_workspace_len > WorkspaceSize()) {
S
sneaxiy 已提交
134 135
      ReallocateWorkspace(required_workspace_len);
    }
Y
Yu Yang 已提交
136 137 138 139 140 141 142 143 144 145 146 147 148 149 150 151 152
    cudnn_func(WorkspacePtr());
  }

  inline void* WorkspacePtr() {
    if (workspace_) {
      return workspace_->ptr();
    } else {
      return nullptr;
    }
  }

  inline size_t WorkspaceSize() {
    if (workspace_) {
      return workspace_->size();
    } else {
      return 0;
    }
S
sneaxiy 已提交
153 154 155 156 157
  }

  std::mutex& Mutex() { return mtx_; }

  cudnnHandle_t cudnn_handle_;
Y
Yu Yang 已提交
158
  memory::AllocationPtr workspace_;
S
sneaxiy 已提交
159 160 161 162 163 164

  const cudaStream_t* stream_;  // not owned;
  const CUDAPlace place_;

  std::mutex mtx_;
};
D
dongzhihong 已提交
165

S
sneaxiy 已提交
166 167 168 169
class CudnnWorkspaceHandle {
 public:
  /*! \brief The lock would not be acquired when constructor calls.
   *  The lock would be acquired when RunFunc() is called first time. */
S
sneaxiy 已提交
170
  inline explicit CudnnWorkspaceHandle(CudnnHolder* holder) : holder_(holder) {}
S
sneaxiy 已提交
171 172 173

  /*! \brief Thread which call RunFunc() would acquire the lock first
   *  before invoking cudnn functions. */
S
sneaxiy 已提交
174 175 176 177 178 179 180 181
  template <typename Callback>
  inline void RunFunc(Callback&& cudnn_func, size_t required_workspace_len) {
    if (!guard_) {
      guard_.reset(new std::lock_guard<std::mutex>(holder_->Mutex()));
    }
    holder_->RunFuncImpl(std::forward<Callback>(cudnn_func),
                         required_workspace_len);
  }
S
sneaxiy 已提交
182

S
sneaxiy 已提交
183 184
  CudnnWorkspaceHandle(CudnnWorkspaceHandle&&) = default;
  CudnnWorkspaceHandle& operator=(CudnnWorkspaceHandle&&) = delete;
S
sneaxiy 已提交
185 186 187 188 189 190

 private:
  CudnnHolder* holder_;  // not own
  std::unique_ptr<std::lock_guard<std::mutex>> guard_;
};

191 192 193 194 195 196 197 198 199 200 201 202 203 204 205 206 207 208 209 210 211 212 213 214 215 216 217 218 219 220 221 222 223
#if CUDA_VERSION >= 9000
class ScopedCublasMathMode {
 public:
  ScopedCublasMathMode(cublasHandle_t handle, cublasMath_t new_math_mode)
      : handle_(handle) {
    need_reset = false;
    PADDLE_ENFORCE(
        platform::dynload::cublasGetMathMode(handle_, &old_math_mode_),
        "Failed to get old cublas math mode");
    if (old_math_mode_ != new_math_mode) {
      PADDLE_ENFORCE(
          platform::dynload::cublasSetMathMode(handle_, new_math_mode),
          "Failed to set old cublas math mode");
      need_reset = true;
    }
  }

  ~ScopedCublasMathMode() {
    if (need_reset) {
      PADDLE_ENFORCE(
          platform::dynload::cublasSetMathMode(handle_, old_math_mode_),
          "Failed to set old cublas math mode");
    }
  }

 private:
  cublasHandle_t handle_;
  cublasMath_t old_math_mode_;
  bool need_reset;
};

#endif

224
class CUDADeviceContext : public DeviceContext {
Q
QI JUN 已提交
225
 public:
D
dzhwinter 已提交
226
  explicit CUDADeviceContext(CUDAPlace place);
227
  virtual ~CUDADeviceContext();
Q
QI JUN 已提交
228

229
  /*! \brief  Wait for all operations completion in the stream. */
230
  void Wait() const override;
Q
QI JUN 已提交
231

232
  /*! \brief  Return place in the device context. */
L
liaogang 已提交
233
  Place GetPlace() const override;
234

K
Kexin Zhao 已提交
235
  /*! \brief  Return compute capability in the device context. */
K
Kexin Zhao 已提交
236 237
  int GetComputeCapability() const;

238 239 240
  /*! \brief  Return the max physical thread count in the device context */
  int GetMaxPhysicalThreadCount() const;

241 242 243 244
  /*! \brief  Return eigen device in the device context. */
  Eigen::GpuDevice* eigen_device() const;

  /*! \brief  Return cublas handle in the device context. */
245
  cublasHandle_t cublas_handle() const;
246 247

  /*! \brief  Return cudnn  handle in the device context. */
248
  cudnnHandle_t cudnn_handle() const;
249

S
sneaxiy 已提交
250 251 252 253 254 255 256 257 258
  /*! \brief  Return a cudnn workspace handle to call multiple cudnn
   *  functions without interrupting by other threads.
   *  Once the first cudnn function is called by the handle, a lock
   *  would be acquired to prevent other threads from accessing the
   *  workspace. Once the handle is destructed, the lock would be released.
   *  CudnnWorkspaceHandle is an RAII object to implement thread-safe
   *  sequential cudnn function calls. */
  CudnnWorkspaceHandle cudnn_workspace_handle() const;

Q
init  
qijun 已提交
259
  /*! \brief  Return cuda stream in the device context. */
260
  cudaStream_t stream() const;
Q
QI JUN 已提交
261

Y
Yu Yang 已提交
262 263
  template <typename Callback>
  void RecordEvent(cudaEvent_t ev, Callback callback) {
Y
yuyang18 已提交
264
    std::lock_guard<std::mutex> guard(mtx_);
Y
Yu Yang 已提交
265 266 267 268
    callback();
    PADDLE_ENFORCE(cudaEventRecord(ev, stream_));
  }

S
sneaxiy 已提交
269 270 271 272 273
  template <typename Callback>
  void AddStreamCallback(Callback&& callback) const {
    callback_manager_->AddCallback(callback);
  }

S
fix bug  
sneaxiy 已提交
274
  void WaitStreamCallback() const { callback_manager_->Wait(); }
S
sneaxiy 已提交
275

276 277 278 279 280 281 282 283 284 285 286 287
#if CUDA_VERSION >= 9000
  /*! \brief CublasCall may need to change cublas's config,
   *  but the cublas may be hold by multi-thread, so we should
   *  add lock here. */
  template <typename Callback>
  void CublasCall(Callback callback, cublasMath_t new_math) {
    std::lock_guard<std::mutex> guard(cublas_mtx_);
    ScopedCublasMathMode scoped_cublas_math(cublas_handle_, new_math);
    callback();
  }
#endif

Q
QI JUN 已提交
288
 private:
D
dzhwinter 已提交
289
  CUDAPlace place_;
Q
QI JUN 已提交
290

Q
qijun 已提交
291
  std::unique_ptr<Eigen::GpuDevice> eigen_device_;
Q
init  
qijun 已提交
292
  std::unique_ptr<EigenCudaStreamDevice> eigen_stream_;
293
  std::unique_ptr<CudnnHolder> cudnn_holder_;
294 295
  cudaStream_t stream_;
  cublasHandle_t cublas_handle_;
296

C
chengduo 已提交
297 298 299 300 301
  int compute_capability_;
  int runtime_version_;
  int driver_version_;
  int multi_process_;
  int max_threads_per_mp_;
Y
yuyang18 已提交
302

S
sneaxiy 已提交
303 304
  mutable std::mutex mtx_;

S
fix bug  
sneaxiy 已提交
305
  // StreamCallbackManager is thread-safe
S
sneaxiy 已提交
306
  std::unique_ptr<StreamCallbackManager> callback_manager_;
307 308

  mutable std::mutex cublas_mtx_;
Q
QI JUN 已提交
309
};
Q
qijun 已提交
310

Y
Yang Yu 已提交
311 312
template <>
struct DefaultDeviceContextType<platform::CUDAPlace> {
Y
Yang Yu 已提交
313
  using TYPE = CUDADeviceContext;
Y
Yang Yu 已提交
314 315
};

C
chengduoZH 已提交
316
// Currently, CUDAPinnedDeviceContext is only used to data copying.
C
chengduoZH 已提交
317 318 319 320 321 322
class CUDAPinnedDeviceContext : public DeviceContext {
 public:
  CUDAPinnedDeviceContext();
  explicit CUDAPinnedDeviceContext(CUDAPinnedPlace place);

  Place GetPlace() const override;
C
chengduoZH 已提交
323

C
chengduoZH 已提交
324 325 326 327 328 329 330 331 332 333 334
  Eigen::DefaultDevice* eigen_device() const;

 private:
  CUDAPinnedPlace place_;
  std::unique_ptr<Eigen::DefaultDevice> eigen_device_;
};

template <>
struct DefaultDeviceContextType<platform::CUDAPinnedPlace> {
  using TYPE = CUDAPinnedDeviceContext;
};
Q
QI JUN 已提交
335
#endif
Q
qijun 已提交
336

T
tensor-tang 已提交
337
#ifdef PADDLE_WITH_MKLDNN
S
Sylwester Fraczek 已提交
338 339 340 341 342 343
using KeyBlob = std::unordered_map<std::string, std::shared_ptr<void>>;
using BlobMap = std::unordered_map<int, std::shared_ptr<KeyBlob>>;

void set_cur_thread_id(int);
int get_cur_thread_id(void);

T
tensor-tang 已提交
344 345 346 347 348
class MKLDNNDeviceContext : public CPUDeviceContext {
 public:
  explicit MKLDNNDeviceContext(CPUPlace place);

  /* \brief  Get the active engine */
349
  const mkldnn::engine& GetEngine() const { return engine_; }
T
tensor-tang 已提交
350

351 352
  // Set data to blob (i.e. name/data pair). Create blob if not existing
  void SetBlob(const std::string& name, std::shared_ptr<void> data) const;
T
tensor-tang 已提交
353

354 355
  // Find a saved blob. Return nullptr if not found
  std::shared_ptr<void> GetBlob(const std::string& name) const;
T
tensor-tang 已提交
356 357

 private:
358
  mkldnn::engine engine_;
359 360
  std::shared_ptr<BlobMap> p_blobmap_;
  std::shared_ptr<std::mutex> p_mutex_;
T
tensor-tang 已提交
361 362 363
};
#endif

D
dzhwinter 已提交
364 365 366 367 368
/*! \brief device context pool singleton */
class DeviceContextPool {
 public:
  explicit DeviceContextPool(const std::vector<platform::Place>& places);

Y
Yang Yu 已提交
369
  static DeviceContextPool& Instance() {
D
dzhwinter 已提交
370 371 372 373 374
    PADDLE_ENFORCE_NOT_NULL(pool, "Need to Create DeviceContextPool first!");
    return *pool;
  }

  /*! \brief  Create should only called by Init function */
Y
Yang Yu 已提交
375
  static DeviceContextPool& Init(const std::vector<platform::Place>& places) {
D
dzhwinter 已提交
376 377 378 379 380 381 382
    if (pool == nullptr) {
      pool = new DeviceContextPool(places);
    }
    return *pool;
  }

  /*! \brief  Return handle of single device context. */
Y
Yu Yang 已提交
383
  platform::DeviceContext* Get(const platform::Place& place);
D
dzhwinter 已提交
384

Y
Yang Yu 已提交
385 386 387 388 389 390 391
  template <typename Place>
  const typename DefaultDeviceContextType<Place>::TYPE* GetByPlace(
      const Place& place) {
    return reinterpret_cast<
        const typename DefaultDeviceContextType<Place>::TYPE*>(Get(place));
  }

392 393
  size_t size() const { return device_contexts_.size(); }

D
dzhwinter 已提交
394 395
 private:
  static DeviceContextPool* pool;
396 397
  std::map<Place, std::shared_future<std::unique_ptr<DeviceContext>>>
      device_contexts_;
D
dzhwinter 已提交
398 399 400
  DISABLE_COPY_AND_ASSIGN(DeviceContextPool);
};

Q
QI JUN 已提交
401 402
}  // namespace platform
}  // namespace paddle