driver_cuda.c 28 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593594595596597598599600601602603604605606607608609610611612613614615616617618619620621622623624625626627628629630631632633634635636637638639640641642643644645646647648649650651652653654655656657658659660661662663664665666667668669670671672673674675676677678679680681682683684685686687688689690691692693694695696697698699700701702703704705706707708709710711712713714715716717718719720721722723724725726727728729730731732733734735736737738739740741742743744745746747748749750751752753754755756757758759760761762763764765766767768769770771772773774775776777778779780781782783784785786787788789790791792793794795796797798799800801802803804805806807808809810811812813814815816817818819820821822823824825826827828829830831832833834835836837838839840841842843844845846847848849850851852853854855856857858859860861862863864865866867868869870871872873874875876877878879880881882883884885886887888889890891892893894895896897898899900901902903904905906907908909910911912913914915916917918919920921922923924925926927928929930931932933934935936937938939940941942943944945946947948949950951952953954955956957958959960961962963964965966967
  1. /* StarPU --- Runtime system for heterogeneous multicore architectures.
  2. *
  3. * Copyright (C) 2009-2014 Université de Bordeaux
  4. * Copyright (C) 2010 Mehdi Juhoor <mjuhoor@gmail.com>
  5. * Copyright (C) 2010, 2011, 2012, 2013, 2014 Centre National de la Recherche Scientifique
  6. * Copyright (C) 2011 Télécom-SudParis
  7. *
  8. * StarPU is free software; you can redistribute it and/or modify
  9. * it under the terms of the GNU Lesser General Public License as published by
  10. * the Free Software Foundation; either version 2.1 of the License, or (at
  11. * your option) any later version.
  12. *
  13. * StarPU is distributed in the hope that it will be useful, but
  14. * WITHOUT ANY WARRANTY; without even the implied warranty of
  15. * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE.
  16. *
  17. * See the GNU Lesser General Public License in COPYING.LGPL for more details.
  18. */
  19. #include <starpu.h>
  20. #include <starpu_cuda.h>
  21. #include <starpu_profiling.h>
  22. #include <common/utils.h>
  23. #include <common/config.h>
  24. #include <core/debug.h>
  25. #include <drivers/driver_common/driver_common.h>
  26. #include "driver_cuda.h"
  27. #include <core/sched_policy.h>
  28. #ifdef HAVE_CUDA_GL_INTEROP_H
  29. #include <cuda_gl_interop.h>
  30. #endif
  31. #include <datawizard/memory_manager.h>
  32. #include <datawizard/malloc.h>
  33. #ifdef STARPU_SIMGRID
  34. #include <core/simgrid.h>
  35. #endif
  36. /* the number of CUDA devices */
  37. static unsigned ncudagpus;
  38. static size_t global_mem[STARPU_MAXCUDADEVS];
  39. #ifdef STARPU_USE_CUDA
  40. static cudaStream_t streams[STARPU_NMAXWORKERS];
  41. static cudaStream_t out_transfer_streams[STARPU_MAXCUDADEVS];
  42. static cudaStream_t in_transfer_streams[STARPU_MAXCUDADEVS];
  43. /* Note: streams are not thread-safe, so we define them for each CUDA worker
  44. * emitting a GPU-GPU transfer */
  45. static cudaStream_t in_peer_transfer_streams[STARPU_MAXCUDADEVS][STARPU_MAXCUDADEVS];
  46. static cudaStream_t out_peer_transfer_streams[STARPU_MAXCUDADEVS][STARPU_MAXCUDADEVS];
  47. static struct cudaDeviceProp props[STARPU_MAXCUDADEVS];
  48. #ifndef STARPU_SIMGRID
  49. static cudaEvent_t task_events[STARPU_NMAXWORKERS][STARPU_MAX_PIPELINE];
  50. #endif
  51. #endif /* STARPU_USE_CUDA */
  52. #ifdef STARPU_SIMGRID
  53. static unsigned task_finished[STARPU_NMAXWORKERS][STARPU_MAX_PIPELINE];
  54. static starpu_pthread_mutex_t task_mutex[STARPU_NMAXWORKERS][STARPU_MAX_PIPELINE];
  55. static starpu_pthread_cond_t task_cond[STARPU_NMAXWORKERS][STARPU_MAX_PIPELINE];
  56. #endif /* STARPU_SIMGRID */
  57. void
  58. _starpu_cuda_discover_devices (struct _starpu_machine_config *config)
  59. {
  60. /* Discover the number of CUDA devices. Fill the result in CONFIG. */
  61. #ifdef STARPU_SIMGRID
  62. config->topology.nhwcudagpus = _starpu_simgrid_get_nbhosts("CUDA");
  63. #else
  64. int cnt;
  65. cudaError_t cures;
  66. cures = cudaGetDeviceCount (&cnt);
  67. if (STARPU_UNLIKELY(cures != cudaSuccess))
  68. cnt = 0;
  69. config->topology.nhwcudagpus = cnt;
  70. #endif
  71. }
  72. /* In case we want to cap the amount of memory available on the GPUs by the
  73. * mean of the STARPU_LIMIT_CUDA_MEM, we decrease the value of
  74. * global_mem[devid] which is the value returned by
  75. * _starpu_cuda_get_global_mem_size() to indicate how much memory can
  76. * be allocated on the device
  77. */
  78. static void _starpu_cuda_limit_gpu_mem_if_needed(unsigned devid)
  79. {
  80. starpu_ssize_t limit;
  81. size_t STARPU_ATTRIBUTE_UNUSED totalGlobalMem = 0;
  82. size_t STARPU_ATTRIBUTE_UNUSED to_waste = 0;
  83. char name[30];
  84. #ifdef STARPU_SIMGRID
  85. totalGlobalMem = _starpu_simgrid_get_memsize("CUDA", devid);
  86. #elif defined(STARPU_USE_CUDA)
  87. /* Find the size of the memory on the device */
  88. totalGlobalMem = props[devid].totalGlobalMem;
  89. #endif
  90. limit = starpu_get_env_number("STARPU_LIMIT_CUDA_MEM");
  91. if (limit == -1)
  92. {
  93. sprintf(name, "STARPU_LIMIT_CUDA_%u_MEM", devid);
  94. limit = starpu_get_env_number(name);
  95. }
  96. #if defined(STARPU_USE_CUDA) || defined(STARPU_SIMGRID)
  97. if (limit == -1)
  98. {
  99. /* Use 90% of the available memory by default. */
  100. limit = totalGlobalMem / (1024*1024) * 0.9;
  101. }
  102. #endif
  103. global_mem[devid] = limit * 1024*1024;
  104. #ifdef STARPU_USE_CUDA
  105. /* How much memory to waste ? */
  106. to_waste = totalGlobalMem - global_mem[devid];
  107. props[devid].totalGlobalMem -= to_waste;
  108. #endif /* STARPU_USE_CUDA */
  109. _STARPU_DEBUG("CUDA device %u: Wasting %ld MB / Limit %ld MB / Total %ld MB / Remains %ld MB\n",
  110. devid, (long) to_waste/(1024*1024), (long) limit, (long) totalGlobalMem/(1024*1024),
  111. (long) (totalGlobalMem - to_waste)/(1024*1024));
  112. }
  113. #ifdef STARPU_USE_CUDA
  114. cudaStream_t starpu_cuda_get_local_in_transfer_stream()
  115. {
  116. int worker = starpu_worker_get_id();
  117. int devid = starpu_worker_get_devid(worker);
  118. cudaStream_t stream;
  119. stream = in_transfer_streams[devid];
  120. STARPU_ASSERT(stream);
  121. return stream;
  122. }
  123. cudaStream_t starpu_cuda_get_local_out_transfer_stream()
  124. {
  125. int worker = starpu_worker_get_id();
  126. int devid = starpu_worker_get_devid(worker);
  127. cudaStream_t stream;
  128. stream = out_transfer_streams[devid];
  129. STARPU_ASSERT(stream);
  130. return stream;
  131. }
  132. cudaStream_t starpu_cuda_get_peer_transfer_stream(unsigned src_node, unsigned dst_node)
  133. {
  134. int worker = starpu_worker_get_id();
  135. int devid = starpu_worker_get_devid(worker);
  136. int src_devid = _starpu_memory_node_get_devid(src_node);
  137. int dst_devid = _starpu_memory_node_get_devid(dst_node);
  138. cudaStream_t stream;
  139. STARPU_ASSERT(devid == src_devid || devid == dst_devid);
  140. if (devid == dst_devid)
  141. stream = in_peer_transfer_streams[src_devid][dst_devid];
  142. else
  143. stream = out_peer_transfer_streams[src_devid][dst_devid];
  144. STARPU_ASSERT(stream);
  145. return stream;
  146. }
  147. cudaStream_t starpu_cuda_get_local_stream(void)
  148. {
  149. int worker = starpu_worker_get_id();
  150. return streams[worker];
  151. }
  152. const struct cudaDeviceProp *starpu_cuda_get_device_properties(unsigned workerid)
  153. {
  154. struct _starpu_machine_config *config = _starpu_get_machine_config();
  155. unsigned devid = config->workers[workerid].devid;
  156. return &props[devid];
  157. }
  158. #endif /* STARPU_USE_CUDA */
  159. void starpu_cuda_set_device(unsigned devid STARPU_ATTRIBUTE_UNUSED)
  160. {
  161. #ifdef STARPU_SIMGRID
  162. STARPU_ABORT();
  163. #else
  164. cudaError_t cures;
  165. struct starpu_conf *conf = _starpu_get_machine_config()->conf;
  166. #if !defined(HAVE_CUDA_MEMCPY_PEER) && defined(HAVE_CUDA_GL_INTEROP_H)
  167. unsigned i;
  168. #endif
  169. #ifdef HAVE_CUDA_MEMCPY_PEER
  170. if (conf->n_cuda_opengl_interoperability)
  171. {
  172. fprintf(stderr, "OpenGL interoperability was requested, but StarPU was built with multithread GPU control support, please reconfigure with --disable-cuda-memcpy-peer but that will disable the memcpy-peer optimizations\n");
  173. STARPU_ABORT();
  174. }
  175. #elif !defined(HAVE_CUDA_GL_INTEROP_H)
  176. if (conf->n_cuda_opengl_interoperability)
  177. {
  178. fprintf(stderr,"OpenGL interoperability was requested, but cuda_gl_interop.h could not be compiled, please make sure that OpenGL headers were available before ./configure run.");
  179. STARPU_ABORT();
  180. }
  181. #else
  182. for (i = 0; i < conf->n_cuda_opengl_interoperability; i++)
  183. if (conf->cuda_opengl_interoperability[i] == devid)
  184. {
  185. cures = cudaGLSetGLDevice(devid);
  186. goto done;
  187. }
  188. #endif
  189. cures = cudaSetDevice(devid);
  190. #if !defined(HAVE_CUDA_MEMCPY_PEER) && defined(HAVE_CUDA_GL_INTEROP_H)
  191. done:
  192. #endif
  193. if (STARPU_UNLIKELY(cures
  194. #ifdef STARPU_OPENMP
  195. /* When StarPU is used as Open Runtime support,
  196. * starpu_omp_shutdown() will usually be called from a
  197. * destructor, in which case cudaThreadExit() reports a
  198. * cudaErrorCudartUnloading here. There should not
  199. * be any remaining tasks running at this point so
  200. * we can probably ignore it without much consequences. */
  201. && cures != cudaErrorCudartUnloading
  202. #endif /* STARPU_OPENMP */
  203. ))
  204. STARPU_CUDA_REPORT_ERROR(cures);
  205. #endif
  206. }
  207. static void init_context(struct _starpu_worker_set *worker_set, unsigned devid STARPU_ATTRIBUTE_UNUSED)
  208. {
  209. int workerid;
  210. unsigned i, j;
  211. #ifdef STARPU_SIMGRID
  212. for (i = 0; i < worker_set->nworkers; i++)
  213. {
  214. workerid = worker_set->workers[i].workerid;
  215. for (j = 0; j < STARPU_MAX_PIPELINE; j++)
  216. {
  217. task_finished[workerid][j] = 0;
  218. STARPU_PTHREAD_MUTEX_INIT(&task_mutex[workerid][j], NULL);
  219. STARPU_PTHREAD_COND_INIT(&task_cond[workerid][j], NULL);
  220. }
  221. }
  222. #else /* !SIMGRID */
  223. cudaError_t cures;
  224. /* TODO: cudaSetDeviceFlag(cudaDeviceMapHost) */
  225. starpu_cuda_set_device(devid);
  226. #ifdef HAVE_CUDA_MEMCPY_PEER
  227. if (starpu_get_env_number("STARPU_ENABLE_CUDA_GPU_GPU_DIRECT") != 0)
  228. {
  229. int nworkers = starpu_worker_get_count();
  230. for (workerid = 0; workerid < nworkers; workerid++)
  231. {
  232. struct _starpu_worker *worker = _starpu_get_worker_struct(workerid);
  233. if (worker->arch == STARPU_CUDA_WORKER && worker->devid != devid)
  234. {
  235. int can;
  236. cures = cudaDeviceCanAccessPeer(&can, devid, worker->devid);
  237. if (!cures && can)
  238. {
  239. cures = cudaDeviceEnablePeerAccess(worker->devid, 0);
  240. if (!cures)
  241. _STARPU_DEBUG("Enabled GPU-Direct %d -> %d\n", worker->devid, devid);
  242. }
  243. }
  244. }
  245. }
  246. #endif
  247. /* force CUDA to initialize the context for real */
  248. cures = cudaFree(0);
  249. if (STARPU_UNLIKELY(cures))
  250. {
  251. if (cures == cudaErrorDevicesUnavailable)
  252. {
  253. fprintf(stderr,"All CUDA-capable devices are busy or unavailable\n");
  254. exit(77);
  255. }
  256. STARPU_CUDA_REPORT_ERROR(cures);
  257. }
  258. cures = cudaGetDeviceProperties(&props[devid], devid);
  259. if (STARPU_UNLIKELY(cures))
  260. STARPU_CUDA_REPORT_ERROR(cures);
  261. #ifdef HAVE_CUDA_MEMCPY_PEER
  262. if (props[devid].computeMode == cudaComputeModeExclusive)
  263. {
  264. fprintf(stderr, "CUDA is in EXCLUSIVE-THREAD mode, but StarPU was built with multithread GPU control support, please either ask your administrator to use EXCLUSIVE-PROCESS mode (which should really be fine), or reconfigure with --disable-cuda-memcpy-peer but that will disable the memcpy-peer optimizations\n");
  265. STARPU_ABORT();
  266. }
  267. #endif
  268. for (i = 0; i < worker_set->nworkers; i++)
  269. {
  270. workerid = worker_set->workers[i].workerid;
  271. for (j = 0; j < STARPU_MAX_PIPELINE; j++)
  272. cures = cudaEventCreateWithFlags(&task_events[workerid][j], cudaEventDisableTiming);
  273. if (STARPU_UNLIKELY(cures))
  274. STARPU_CUDA_REPORT_ERROR(cures);
  275. cures = cudaStreamCreate(&streams[workerid]);
  276. if (STARPU_UNLIKELY(cures))
  277. STARPU_CUDA_REPORT_ERROR(cures);
  278. }
  279. cures = cudaStreamCreate(&in_transfer_streams[devid]);
  280. if (STARPU_UNLIKELY(cures))
  281. STARPU_CUDA_REPORT_ERROR(cures);
  282. cures = cudaStreamCreate(&out_transfer_streams[devid]);
  283. if (STARPU_UNLIKELY(cures))
  284. STARPU_CUDA_REPORT_ERROR(cures);
  285. for (i = 0; i < ncudagpus; i++)
  286. {
  287. cures = cudaStreamCreate(&in_peer_transfer_streams[i][devid]);
  288. if (STARPU_UNLIKELY(cures))
  289. STARPU_CUDA_REPORT_ERROR(cures);
  290. cures = cudaStreamCreate(&out_peer_transfer_streams[devid][i]);
  291. if (STARPU_UNLIKELY(cures))
  292. STARPU_CUDA_REPORT_ERROR(cures);
  293. }
  294. #endif /* !SIMGRID */
  295. }
  296. static void deinit_context(struct _starpu_worker_set *worker_set)
  297. {
  298. unsigned i, j;
  299. int workerid;
  300. #ifdef STARPU_SIMGRID
  301. for (i = 0; i < worker_set->nworkers; i++)
  302. {
  303. workerid = worker_set->workers[i].workerid;
  304. for (j = 0; j < STARPU_MAX_PIPELINE; j++)
  305. {
  306. STARPU_PTHREAD_MUTEX_DESTROY(&task_mutex[workerid][j]);
  307. STARPU_PTHREAD_COND_DESTROY(&task_cond[workerid][j]);
  308. }
  309. }
  310. #else /* !STARPU_SIMGRID */
  311. cudaError_t cures;
  312. workerid = worker_set->workers[0].workerid;
  313. int devid = starpu_worker_get_devid(workerid);
  314. for (i = 0; i < worker_set->nworkers; i++)
  315. {
  316. workerid = worker_set->workers[i].workerid;
  317. devid = starpu_worker_get_devid(workerid);
  318. for (j = 0; j < STARPU_MAX_PIPELINE; j++)
  319. cudaEventDestroy(task_events[workerid][j]);
  320. cudaStreamDestroy(streams[workerid]);
  321. }
  322. cudaStreamDestroy(in_transfer_streams[devid]);
  323. cudaStreamDestroy(out_transfer_streams[devid]);
  324. for (i = 0; i < ncudagpus; i++)
  325. {
  326. cudaStreamDestroy(in_peer_transfer_streams[i][devid]);
  327. cudaStreamDestroy(out_peer_transfer_streams[devid][i]);
  328. }
  329. }
  330. static size_t _starpu_cuda_get_global_mem_size(unsigned devid)
  331. {
  332. return global_mem[devid];
  333. }
  334. /* Return the number of devices usable in the system.
  335. * The value returned cannot be greater than MAXCUDADEVS */
  336. unsigned _starpu_get_cuda_device_count(void)
  337. {
  338. #ifdef STARPU_SIMGRID
  339. return _starpu_simgrid_get_nbhosts("CUDA");
  340. #else
  341. int cnt;
  342. cudaError_t cures;
  343. cures = cudaGetDeviceCount(&cnt);
  344. if (STARPU_UNLIKELY(cures))
  345. return 0;
  346. if (cnt > STARPU_MAXCUDADEVS)
  347. {
  348. fprintf(stderr, "# Warning: %d CUDA devices available. Only %d enabled. Use configure option --enable-maxcudadev=xxx to update the maximum value of supported CUDA devices.\n", cnt, STARPU_MAXCUDADEVS);
  349. cnt = STARPU_MAXCUDADEVS;
  350. }
  351. return (unsigned)cnt;
  352. #endif
  353. }
  354. void _starpu_init_cuda(void)
  355. {
  356. ncudagpus = _starpu_get_cuda_device_count();
  357. STARPU_ASSERT(ncudagpus <= STARPU_MAXCUDADEVS);
  358. }
  359. static int start_job_on_cuda(struct _starpu_job *j, struct _starpu_worker *worker, unsigned char pipeline_idx STARPU_ATTRIBUTE_UNUSED)
  360. {
  361. int ret;
  362. STARPU_ASSERT(j);
  363. struct starpu_task *task = j->task;
  364. int profiling = starpu_profiling_status_get();
  365. STARPU_ASSERT(task);
  366. struct starpu_codelet *cl = task->cl;
  367. STARPU_ASSERT(cl);
  368. _starpu_set_current_task(task);
  369. ret = _starpu_fetch_task_input(j);
  370. if (ret != 0)
  371. {
  372. /* there was not enough memory, so the input of
  373. * the codelet cannot be fetched ... put the
  374. * codelet back, and try it later */
  375. return -EAGAIN;
  376. }
  377. if (worker->ntasks == 1)
  378. {
  379. /* We are alone in the pipeline, the kernel will start now, record it */
  380. _starpu_driver_start_job(worker, j, &worker->perf_arch, &j->cl_start, 0, profiling);
  381. }
  382. #if defined(HAVE_CUDA_MEMCPY_PEER) && !defined(STARPU_SIMGRID)
  383. /* We make sure we do manipulate the proper device */
  384. starpu_cuda_set_device(worker->devid);
  385. #endif
  386. starpu_cuda_func_t func = _starpu_task_get_cuda_nth_implementation(cl, j->nimpl);
  387. STARPU_ASSERT_MSG(func, "when STARPU_CUDA is defined in 'where', cuda_func or cuda_funcs has to be defined");
  388. if (starpu_get_env_number("STARPU_DISABLE_KERNELS") <= 0)
  389. {
  390. _STARPU_TRACE_START_EXECUTING();
  391. #ifdef STARPU_SIMGRID
  392. int async = task->cl->cuda_flags[j->nimpl] & STARPU_CUDA_ASYNC;
  393. unsigned workerid = worker->workerid;
  394. _starpu_simgrid_submit_job(workerid, j, &worker->perf_arch, NAN,
  395. async ? &task_finished[workerid][pipeline_idx] : NULL,
  396. async ? &task_mutex[workerid][pipeline_idx] : NULL,
  397. async ? &task_cond[workerid][pipeline_idx] : NULL);
  398. #else
  399. func(_STARPU_TASK_GET_INTERFACES(task), task->cl_arg);
  400. #endif
  401. _STARPU_TRACE_END_EXECUTING();
  402. }
  403. return 0;
  404. }
  405. static void finish_job_on_cuda(struct _starpu_job *j, struct _starpu_worker *worker)
  406. {
  407. struct timespec codelet_end;
  408. int profiling = starpu_profiling_status_get();
  409. _starpu_set_current_task(NULL);
  410. if (worker->pipeline_length)
  411. worker->current_tasks[worker->first_task] = NULL;
  412. else
  413. worker->current_task = NULL;
  414. worker->first_task = (worker->first_task + 1) % STARPU_MAX_PIPELINE;
  415. worker->ntasks--;
  416. _starpu_driver_end_job(worker, j, &worker->perf_arch, &codelet_end, 0, profiling);
  417. struct _starpu_sched_ctx *sched_ctx = _starpu_sched_ctx_get_sched_ctx_for_worker_and_job(worker, j);
  418. if(!sched_ctx)
  419. sched_ctx = _starpu_get_sched_ctx_struct(j->task->sched_ctx);
  420. if(!sched_ctx->sched_policy)
  421. _starpu_driver_update_job_feedback(j, worker, &sched_ctx->perf_arch, &j->cl_start, &codelet_end, profiling);
  422. else
  423. _starpu_driver_update_job_feedback(j, worker, &worker->perf_arch, &j->cl_start, &codelet_end, profiling);
  424. _starpu_push_task_output(j);
  425. _starpu_handle_job_termination(j);
  426. }
  427. /* Execute a job, up to completion for synchronous jobs */
  428. static void execute_job_on_cuda(struct starpu_task *task, struct _starpu_worker *worker)
  429. {
  430. int workerid = worker->workerid;
  431. int res;
  432. struct _starpu_job *j = _starpu_get_job_associated_to_task(task);
  433. unsigned char pipeline_idx = (worker->first_task + worker->ntasks - 1)%STARPU_MAX_PIPELINE;
  434. res = start_job_on_cuda(j, worker, pipeline_idx);
  435. if (res)
  436. {
  437. switch (res)
  438. {
  439. case -EAGAIN:
  440. _STARPU_DISP("ouch, CUDA could not actually run task %p, putting it back...\n", task);
  441. _starpu_push_task_to_workers(task);
  442. STARPU_ABORT();
  443. default:
  444. STARPU_ABORT();
  445. }
  446. }
  447. if (task->cl->cuda_flags[j->nimpl] & STARPU_CUDA_ASYNC)
  448. {
  449. if (worker->pipeline_length == 0)
  450. {
  451. #ifdef STARPU_SIMGRID
  452. _starpu_simgrid_wait_tasks(workerid);
  453. #else
  454. /* Forced synchronous execution */
  455. cudaStreamSynchronize(starpu_cuda_get_local_stream());
  456. #endif
  457. finish_job_on_cuda(j, worker);
  458. }
  459. else
  460. {
  461. #ifndef STARPU_SIMGRID
  462. /* Record event to synchronize with task termination later */
  463. cudaEventRecord(task_events[workerid][pipeline_idx], starpu_cuda_get_local_stream());
  464. #endif
  465. #ifdef STARPU_USE_FXT
  466. int k;
  467. for (k = 0; k < (int) worker->set->nworkers; k++)
  468. if (worker->set->workers[k].ntasks == worker->set->workers[k].pipeline_length)
  469. break;
  470. if (k == (int) worker->set->nworkers)
  471. /* Everybody busy */
  472. _STARPU_TRACE_START_EXECUTING();
  473. #endif
  474. }
  475. }
  476. else
  477. /* Synchronous execution */
  478. {
  479. #if !defined(STARPU_SIMGRID)
  480. STARPU_ASSERT_MSG(cudaStreamQuery(starpu_cuda_get_local_stream()) == cudaSuccess, "Unless when using the STARPU_CUDA_ASYNC flag, CUDA codelets have to wait for termination of their kernels on the starpu_cuda_get_local_stream() stream");
  481. #endif
  482. finish_job_on_cuda(j, worker);
  483. }
  484. }
  485. /* XXX Should this be merged with _starpu_init_cuda ? */
  486. int _starpu_cuda_driver_init(struct _starpu_worker_set *worker_set)
  487. {
  488. struct _starpu_worker *worker0 = &worker_set->workers[0];
  489. unsigned devid = worker0->devid;
  490. unsigned i;
  491. _starpu_worker_start(worker0, _STARPU_FUT_CUDA_KEY);
  492. #ifdef STARPU_USE_FXT
  493. unsigned memnode = worker0->memory_node;
  494. for (i = 1; i < worker_set->nworkers; i++)
  495. {
  496. struct _starpu_worker *worker = &worker_set->workers[i];
  497. _STARPU_TRACE_WORKER_INIT_START(_STARPU_FUT_CUDA_KEY, worker->workerid, devid, memnode);
  498. }
  499. #endif
  500. init_context(worker_set, devid);
  501. #ifdef STARPU_SIMGRID
  502. STARPU_ASSERT_MSG (worker_set->nworkers = 1, "Simgrid mode does not support concurrent kernel execution yet\n");
  503. #else /* !STARPU_SIMGRID */
  504. if (worker_set->nworkers > 1 && props[devid].concurrentKernels == 0)
  505. _STARPU_DISP("Warning: STARPU_NWORKER_PER_CUDA is %u, but the device does not support concurrent kernel execution!\n", worker_set->nworkers);
  506. #endif /* !STARPU_SIMGRID */
  507. _starpu_cuda_limit_gpu_mem_if_needed(devid);
  508. _starpu_memory_manager_set_global_memory_size(worker0->memory_node, _starpu_cuda_get_global_mem_size(devid));
  509. _starpu_malloc_init(worker0->memory_node);
  510. /* one more time to avoid hacks from third party lib :) */
  511. _starpu_bind_thread_on_cpu(worker0->config, worker0->bindid);
  512. float size = (float) global_mem[devid] / (1<<30);
  513. #ifdef STARPU_SIMGRID
  514. const char *devname = "Simgrid";
  515. #else
  516. /* get the device's name */
  517. char devname[128];
  518. strncpy(devname, props[devid].name, 128);
  519. #endif
  520. for (i = 0; i < worker_set->nworkers; i++)
  521. {
  522. struct _starpu_worker *worker = &worker_set->workers[i];
  523. #if defined(STARPU_HAVE_BUSID) && !defined(STARPU_SIMGRID)
  524. #if defined(STARPU_HAVE_DOMAINID) && !defined(STARPU_SIMGRID)
  525. if (props[devid].pciDomainID)
  526. snprintf(worker->name, sizeof(worker->name), "CUDA %u.%u (%s %.1f GiB %04x:%02x:%02x.0)", devid, i, devname, size, props[devid].pciDomainID, props[devid].pciBusID, props[devid].pciDeviceID);
  527. else
  528. #endif
  529. snprintf(worker->name, sizeof(worker->name), "CUDA %u.%u (%s %.1f GiB %02x:%02x.0)", devid, i, devname, size, props[devid].pciBusID, props[devid].pciDeviceID);
  530. #else
  531. snprintf(worker->name, sizeof(worker->name), "CUDA %u.%u (%s %.1f GiB)", devid, i, devname, size);
  532. #endif
  533. snprintf(worker->short_name, sizeof(worker->short_name), "CUDA %u.%u", devid, i);
  534. _STARPU_DEBUG("cuda (%s) dev id %u worker %u thread is ready to run on CPU %d !\n", devname, devid, i, worker->bindid);
  535. worker->pipeline_length = starpu_get_env_number_default("STARPU_CUDA_PIPELINE", 2);
  536. if (worker->pipeline_length > STARPU_MAX_PIPELINE)
  537. {
  538. _STARPU_DISP("Warning: STARPU_CUDA_PIPELINE is %u, but STARPU_MAX_PIPELINE is only %u", worker->pipeline_length, STARPU_MAX_PIPELINE);
  539. worker->pipeline_length = STARPU_MAX_PIPELINE;
  540. }
  541. #if defined(STARPU_SIMGRID) && defined(STARPU_NON_BLOCKING_DRIVERS)
  542. if (worker->pipeline_length >= 1)
  543. {
  544. /* We need blocking drivers, otherwise idle drivers
  545. * would keep consuming real CPU time while just
  546. * polling for task termination */
  547. _STARPU_DISP("Warning: reducing STARPU_CUDA_PIPELINE to 0 because simgrid is enabled and blocking drivers are not enabled\n");
  548. worker->pipeline_length = 0;
  549. }
  550. #endif
  551. #if !defined(STARPU_SIMGRID) && !defined(STARPU_NON_BLOCKING_DRIVERS)
  552. if (worker->pipeline_length >= 1)
  553. {
  554. /* We need non-blocking drivers, to poll for CUDA task
  555. * termination */
  556. _STARPU_DISP("Warning: reducing STARPU_CUDA_PIPELINE to 0 because blocking drivers are enabled (and simgrid is not enabled)\n");
  557. worker->pipeline_length = 0;
  558. }
  559. #endif
  560. _STARPU_TRACE_WORKER_INIT_END(worker_set->workers[i].workerid);
  561. }
  562. /* tell the main thread that this one is ready */
  563. STARPU_PTHREAD_MUTEX_LOCK(&worker0->mutex);
  564. worker0->status = STATUS_UNKNOWN;
  565. worker0->worker_is_initialized = 1;
  566. STARPU_PTHREAD_COND_SIGNAL(&worker0->ready_cond);
  567. STARPU_PTHREAD_MUTEX_UNLOCK(&worker0->mutex);
  568. /* tell the main thread that this one is ready */
  569. STARPU_PTHREAD_MUTEX_LOCK(&worker_set->mutex);
  570. worker_set->set_is_initialized = 1;
  571. STARPU_PTHREAD_COND_SIGNAL(&worker_set->ready_cond);
  572. STARPU_PTHREAD_MUTEX_UNLOCK(&worker_set->mutex);
  573. return 0;
  574. }
  575. int _starpu_cuda_driver_run_once(struct _starpu_worker_set *worker_set)
  576. {
  577. struct _starpu_worker *worker0 = &worker_set->workers[0];
  578. unsigned memnode = worker0->memory_node;
  579. struct starpu_task *tasks[worker_set->nworkers], *task;
  580. struct _starpu_job *j;
  581. int i, res;
  582. int idle;
  583. /* First poll for completed jobs */
  584. idle = 0;
  585. for (i = 0; i < (int) worker_set->nworkers; i++)
  586. {
  587. struct _starpu_worker *worker = &worker_set->workers[i];
  588. int workerid = worker->workerid;
  589. if (!worker->ntasks)
  590. {
  591. idle++;
  592. /* Even nothing to test */
  593. continue;
  594. }
  595. task = worker->current_tasks[worker->first_task];
  596. /* On-going asynchronous task, check for its termination first */
  597. #ifdef STARPU_SIMGRID
  598. if (task_finished[workerid][worker->first_task])
  599. #else /* !STARPU_SIMGRID */
  600. cudaError_t cures = cudaEventQuery(task_events[workerid][worker->first_task]);
  601. if (cures != cudaSuccess)
  602. {
  603. STARPU_ASSERT_MSG(cures == cudaErrorNotReady, "CUDA error on task %p, codelet %p (%s): %s (%d)", task, task->cl, _starpu_codelet_get_model_name(task->cl), cudaGetErrorString(cures), cures);
  604. }
  605. else
  606. #endif /* !STARPU_SIMGRID */
  607. {
  608. /* Asynchronous task completed! */
  609. _starpu_set_local_worker_key(worker);
  610. finish_job_on_cuda(_starpu_get_job_associated_to_task(task), worker);
  611. /* See next task if any */
  612. if (worker->ntasks)
  613. {
  614. task = worker->current_tasks[worker->first_task];
  615. j = _starpu_get_job_associated_to_task(task);
  616. if (task->cl->cuda_flags[j->nimpl] & STARPU_CUDA_ASYNC)
  617. {
  618. /* An asynchronous task, it was already
  619. * queued, it's now running, record its start time. */
  620. _starpu_driver_start_job(worker, j, &worker->perf_arch, &j->cl_start, 0, starpu_profiling_status_get());
  621. }
  622. else
  623. {
  624. /* A synchronous task, we have finished
  625. * flushing the pipeline, we can now at
  626. * last execute it. */
  627. _STARPU_TRACE_END_PROGRESS(memnode);
  628. _STARPU_TRACE_EVENT("sync_task");
  629. execute_job_on_cuda(task, worker);
  630. _STARPU_TRACE_EVENT("end_sync_task");
  631. _STARPU_TRACE_START_PROGRESS(memnode);
  632. worker->pipeline_stuck = 0;
  633. }
  634. }
  635. #ifdef STARPU_USE_FXT
  636. int k;
  637. for (k = 0; k < (int) worker_set->nworkers; k++)
  638. if (worker_set->workers[k].ntasks)
  639. break;
  640. if (k == (int) worker_set->nworkers)
  641. /* Everybody busy */
  642. _STARPU_TRACE_END_EXECUTING()
  643. #endif
  644. }
  645. if (worker->ntasks < worker->pipeline_length)
  646. idle++;
  647. }
  648. #ifdef STARPU_NON_BLOCKING_DRIVERS
  649. if (!idle)
  650. {
  651. /* Nothing ready yet, no better thing to do than waiting */
  652. __starpu_datawizard_progress(memnode, 1, 0);
  653. return 0;
  654. }
  655. #endif
  656. /* Something done, make some progress */
  657. __starpu_datawizard_progress(memnode, 1, 1);
  658. /* And pull tasks */
  659. res = _starpu_get_multi_worker_task(worker_set->workers, tasks, worker_set->nworkers, memnode);
  660. if (!res)
  661. return 0;
  662. for (i = 0; i < (int) worker_set->nworkers; i++)
  663. {
  664. struct _starpu_worker *worker = &worker_set->workers[i];
  665. task = tasks[i];
  666. if (!task)
  667. continue;
  668. j = _starpu_get_job_associated_to_task(task);
  669. /* can CUDA do that task ? */
  670. if (!_STARPU_CUDA_MAY_PERFORM(j))
  671. {
  672. /* this is neither a cuda or a cublas task */
  673. worker->ntasks--;
  674. _starpu_push_task_to_workers(task);
  675. continue;
  676. }
  677. if (worker->ntasks > 1 && !(task->cl->cuda_flags[j->nimpl] & STARPU_CUDA_ASYNC))
  678. {
  679. /* We have to execute a non-asynchronous task but we
  680. * still have tasks in the pipeline... Record it to
  681. * prevent more tasks from coming, and do it later */
  682. worker->pipeline_stuck = 1;
  683. continue;
  684. }
  685. _starpu_set_local_worker_key(worker);
  686. _STARPU_TRACE_END_PROGRESS(memnode);
  687. execute_job_on_cuda(task, worker);
  688. _STARPU_TRACE_START_PROGRESS(memnode);
  689. }
  690. return 0;
  691. }
  692. int _starpu_cuda_driver_deinit(struct _starpu_worker_set *arg)
  693. {
  694. struct _starpu_worker *worker = &arg->workers[0];
  695. unsigned memnode = worker->memory_node;
  696. _STARPU_TRACE_WORKER_DEINIT_START;
  697. _starpu_handle_all_pending_node_data_requests(memnode);
  698. /* In case there remains some memory that was automatically
  699. * allocated by StarPU, we release it now. Note that data
  700. * coherency is not maintained anymore at that point ! */
  701. _starpu_free_all_automatically_allocated_buffers(memnode);
  702. _starpu_malloc_shutdown(memnode);
  703. deinit_context(arg);
  704. _STARPU_TRACE_WORKER_DEINIT_END(_STARPU_FUT_CUDA_KEY);
  705. return 0;
  706. }
  707. void *_starpu_cuda_worker(void *_arg)
  708. {
  709. struct _starpu_worker_set* worker = _arg;
  710. _starpu_cuda_driver_init(worker);
  711. _STARPU_TRACE_START_PROGRESS(memnode);
  712. while (_starpu_machine_is_running())
  713. {
  714. _starpu_may_pause();
  715. _starpu_cuda_driver_run_once(worker);
  716. }
  717. _STARPU_TRACE_END_PROGRESS(memnode);
  718. _starpu_cuda_driver_deinit(worker);
  719. return NULL;
  720. }
  721. #ifdef STARPU_USE_CUDA
  722. void starpu_cublas_report_error(const char *func, const char *file, int line, int status)
  723. {
  724. char *errormsg;
  725. switch (status)
  726. {
  727. case CUBLAS_STATUS_SUCCESS:
  728. errormsg = "success";
  729. break;
  730. case CUBLAS_STATUS_NOT_INITIALIZED:
  731. errormsg = "not initialized";
  732. break;
  733. case CUBLAS_STATUS_ALLOC_FAILED:
  734. errormsg = "alloc failed";
  735. break;
  736. case CUBLAS_STATUS_INVALID_VALUE:
  737. errormsg = "invalid value";
  738. break;
  739. case CUBLAS_STATUS_ARCH_MISMATCH:
  740. errormsg = "arch mismatch";
  741. break;
  742. case CUBLAS_STATUS_EXECUTION_FAILED:
  743. errormsg = "execution failed";
  744. break;
  745. case CUBLAS_STATUS_INTERNAL_ERROR:
  746. errormsg = "internal error";
  747. break;
  748. default:
  749. errormsg = "unknown error";
  750. break;
  751. }
  752. fprintf(stderr, "oops in %s (%s:%d)... %d: %s \n", func, file, line, status, errormsg);
  753. STARPU_ABORT();
  754. }
  755. void starpu_cuda_report_error(const char *func, const char *file, int line, cudaError_t status)
  756. {
  757. const char *errormsg = cudaGetErrorString(status);
  758. printf("oops in %s (%s:%d)... %d: %s \n", func, file, line, status, errormsg);
  759. STARPU_ABORT();
  760. }
  761. #endif /* STARPU_USE_CUDA */
  762. #ifdef STARPU_USE_CUDA
  763. int
  764. starpu_cuda_copy_async_sync(void *src_ptr, unsigned src_node,
  765. void *dst_ptr, unsigned dst_node,
  766. size_t ssize, cudaStream_t stream,
  767. enum cudaMemcpyKind kind)
  768. {
  769. #ifdef HAVE_CUDA_MEMCPY_PEER
  770. int peer_copy = 0;
  771. int src_dev = -1, dst_dev = -1;
  772. #endif
  773. cudaError_t cures = 0;
  774. if (kind == cudaMemcpyDeviceToDevice && src_node != dst_node)
  775. {
  776. #ifdef HAVE_CUDA_MEMCPY_PEER
  777. peer_copy = 1;
  778. src_dev = _starpu_memory_node_get_devid(src_node);
  779. dst_dev = _starpu_memory_node_get_devid(dst_node);
  780. #else
  781. STARPU_ABORT();
  782. #endif
  783. }
  784. if (stream)
  785. {
  786. _STARPU_TRACE_START_DRIVER_COPY_ASYNC(src_node, dst_node);
  787. #ifdef HAVE_CUDA_MEMCPY_PEER
  788. if (peer_copy)
  789. {
  790. cures = cudaMemcpyPeerAsync((char *) dst_ptr, dst_dev,
  791. (char *) src_ptr, src_dev,
  792. ssize, stream);
  793. }
  794. else
  795. #endif
  796. {
  797. cures = cudaMemcpyAsync((char *)dst_ptr, (char *)src_ptr, ssize, kind, stream);
  798. }
  799. _STARPU_TRACE_END_DRIVER_COPY_ASYNC(src_node, dst_node);
  800. }
  801. /* Test if the asynchronous copy has failed or if the caller only asked for a synchronous copy */
  802. if (stream == NULL || cures)
  803. {
  804. /* do it in a synchronous fashion */
  805. #ifdef HAVE_CUDA_MEMCPY_PEER
  806. if (peer_copy)
  807. {
  808. cures = cudaMemcpyPeer((char *) dst_ptr, dst_dev,
  809. (char *) src_ptr, src_dev,
  810. ssize);
  811. }
  812. else
  813. #endif
  814. {
  815. cures = cudaMemcpy((char *)dst_ptr, (char *)src_ptr, ssize, kind);
  816. }
  817. if (STARPU_UNLIKELY(cures))
  818. STARPU_CUDA_REPORT_ERROR(cures);
  819. return 0;
  820. }
  821. return -EAGAIN;
  822. }
  823. #endif /* STARPU_USE_CUDA */
  824. int _starpu_run_cuda(struct _starpu_worker_set *workerarg)
  825. {
  826. /* Let's go ! */
  827. _starpu_cuda_worker(workerarg);
  828. return 0;
  829. }