driver_cuda.c 24 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593594595596597598599600601602603604605606607608609610611612613614615616617618619620621622623624625626627628629630631632633634635636637638639640641642643644645646647648649650651652653654655656657658659660661662663664665666667668669670671672673674675676677678679680681682683684685686687688689690691692693694695696697698699700701702703704705706707708709710711712713714715716717718719720721722723724725726727728729730731732733734735736737738739740741742743744745746747748749750751752753754755756757758759760761762763764765766767768769770771772773774775776777778779780781782783784785786787788789790791792793794795796797798799800801802803804805806807808809810811812813814815816817818819820821822823824825826827828829830831832833834835836837838839840841842843844845846847848849850851852853854855856857858859860861862863864865866867868869870871872873874
  1. /* StarPU --- Runtime system for heterogeneous multicore architectures.
  2. *
  3. * Copyright (C) 2009-2014 Université de Bordeaux 1
  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. static cudaEvent_t task_events[STARPU_NMAXWORKERS][STARPU_MAX_PIPELINE];
  49. #endif /* STARPU_USE_CUDA */
  50. void
  51. _starpu_cuda_discover_devices (struct _starpu_machine_config *config)
  52. {
  53. /* Discover the number of CUDA devices. Fill the result in CONFIG. */
  54. #ifdef STARPU_SIMGRID
  55. config->topology.nhwcudagpus = _starpu_simgrid_get_nbhosts("CUDA");
  56. #else
  57. int cnt;
  58. cudaError_t cures;
  59. cures = cudaGetDeviceCount (&cnt);
  60. if (STARPU_UNLIKELY(cures != cudaSuccess))
  61. cnt = 0;
  62. config->topology.nhwcudagpus = cnt;
  63. #endif
  64. }
  65. /* In case we want to cap the amount of memory available on the GPUs by the
  66. * mean of the STARPU_LIMIT_CUDA_MEM, we decrease the value of
  67. * global_mem[devid] which is the value returned by
  68. * _starpu_cuda_get_global_mem_size() to indicate how much memory can
  69. * be allocated on the device
  70. */
  71. static void _starpu_cuda_limit_gpu_mem_if_needed(unsigned devid)
  72. {
  73. starpu_ssize_t limit;
  74. size_t STARPU_ATTRIBUTE_UNUSED totalGlobalMem = 0;
  75. size_t STARPU_ATTRIBUTE_UNUSED to_waste = 0;
  76. char name[30];
  77. #ifdef STARPU_SIMGRID
  78. totalGlobalMem = _starpu_simgrid_get_memsize("CUDA", devid);
  79. #elif defined(STARPU_USE_CUDA)
  80. /* Find the size of the memory on the device */
  81. totalGlobalMem = props[devid].totalGlobalMem;
  82. #endif
  83. limit = starpu_get_env_number("STARPU_LIMIT_CUDA_MEM");
  84. if (limit == -1)
  85. {
  86. sprintf(name, "STARPU_LIMIT_CUDA_%u_MEM", devid);
  87. limit = starpu_get_env_number(name);
  88. }
  89. #if defined(STARPU_USE_CUDA) || defined(STARPU_SIMGRID)
  90. if (limit == -1)
  91. {
  92. /* Use 90% of the available memory by default. */
  93. limit = totalGlobalMem / (1024*1024) * 0.9;
  94. }
  95. #endif
  96. global_mem[devid] = limit * 1024*1024;
  97. #ifdef STARPU_USE_CUDA
  98. /* How much memory to waste ? */
  99. to_waste = totalGlobalMem - global_mem[devid];
  100. props[devid].totalGlobalMem -= to_waste;
  101. #endif /* STARPU_USE_CUDA */
  102. _STARPU_DEBUG("CUDA device %u: Wasting %ld MB / Limit %ld MB / Total %ld MB / Remains %ld MB\n",
  103. devid, (long) to_waste/(1024*1024), (long) limit, (long) totalGlobalMem/(1024*1024),
  104. (long) (totalGlobalMem - to_waste)/(1024*1024));
  105. }
  106. #ifdef STARPU_USE_CUDA
  107. cudaStream_t starpu_cuda_get_local_in_transfer_stream()
  108. {
  109. int worker = starpu_worker_get_id();
  110. int devid = starpu_worker_get_devid(worker);
  111. cudaStream_t stream;
  112. stream = in_transfer_streams[devid];
  113. STARPU_ASSERT(stream);
  114. return stream;
  115. }
  116. cudaStream_t starpu_cuda_get_local_out_transfer_stream()
  117. {
  118. int worker = starpu_worker_get_id();
  119. int devid = starpu_worker_get_devid(worker);
  120. cudaStream_t stream;
  121. stream = out_transfer_streams[devid];
  122. STARPU_ASSERT(stream);
  123. return stream;
  124. }
  125. cudaStream_t starpu_cuda_get_peer_transfer_stream(unsigned src_node, unsigned dst_node)
  126. {
  127. int worker = starpu_worker_get_id();
  128. int devid = starpu_worker_get_devid(worker);
  129. int src_devid = _starpu_memory_node_get_devid(src_node);
  130. int dst_devid = _starpu_memory_node_get_devid(dst_node);
  131. cudaStream_t stream;
  132. STARPU_ASSERT(devid == src_devid || devid == dst_devid);
  133. if (devid == dst_devid)
  134. stream = in_peer_transfer_streams[src_devid][dst_devid];
  135. else
  136. stream = out_peer_transfer_streams[src_devid][dst_devid];
  137. STARPU_ASSERT(stream);
  138. return stream;
  139. }
  140. cudaStream_t starpu_cuda_get_local_stream(void)
  141. {
  142. int worker = starpu_worker_get_id();
  143. return streams[worker];
  144. }
  145. const struct cudaDeviceProp *starpu_cuda_get_device_properties(unsigned workerid)
  146. {
  147. struct _starpu_machine_config *config = _starpu_get_machine_config();
  148. unsigned devid = config->workers[workerid].devid;
  149. return &props[devid];
  150. }
  151. #endif /* STARPU_USE_CUDA */
  152. void starpu_cuda_set_device(unsigned devid STARPU_ATTRIBUTE_UNUSED)
  153. {
  154. #ifdef STARPU_SIMGRID
  155. STARPU_ABORT();
  156. #else
  157. cudaError_t cures;
  158. struct starpu_conf *conf = _starpu_get_machine_config()->conf;
  159. #if !defined(HAVE_CUDA_MEMCPY_PEER) && defined(HAVE_CUDA_GL_INTEROP_H)
  160. unsigned i;
  161. #endif
  162. #ifdef HAVE_CUDA_MEMCPY_PEER
  163. if (conf->n_cuda_opengl_interoperability)
  164. {
  165. 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");
  166. STARPU_ABORT();
  167. }
  168. #elif !defined(HAVE_CUDA_GL_INTEROP_H)
  169. if (conf->n_cuda_opengl_interoperability)
  170. {
  171. 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.");
  172. STARPU_ABORT();
  173. }
  174. #else
  175. for (i = 0; i < conf->n_cuda_opengl_interoperability; i++)
  176. if (conf->cuda_opengl_interoperability[i] == devid)
  177. {
  178. cures = cudaGLSetGLDevice(devid);
  179. goto done;
  180. }
  181. #endif
  182. cures = cudaSetDevice(devid);
  183. #if !defined(HAVE_CUDA_MEMCPY_PEER) && defined(HAVE_CUDA_GL_INTEROP_H)
  184. done:
  185. #endif
  186. if (STARPU_UNLIKELY(cures))
  187. STARPU_CUDA_REPORT_ERROR(cures);
  188. #endif
  189. }
  190. #ifndef STARPU_SIMGRID
  191. static void init_context(struct _starpu_worker_set *worker_set, unsigned devid)
  192. {
  193. cudaError_t cures;
  194. int workerid;
  195. unsigned i, j;
  196. /* TODO: cudaSetDeviceFlag(cudaDeviceMapHost) */
  197. starpu_cuda_set_device(devid);
  198. #ifdef HAVE_CUDA_MEMCPY_PEER
  199. if (starpu_get_env_number("STARPU_ENABLE_CUDA_GPU_GPU_DIRECT") != 0)
  200. {
  201. int nworkers = starpu_worker_get_count();
  202. for (workerid = 0; workerid < nworkers; workerid++)
  203. {
  204. struct _starpu_worker *worker = _starpu_get_worker_struct(workerid);
  205. if (worker->arch == STARPU_CUDA_WORKER && worker->devid != devid)
  206. {
  207. int can;
  208. cures = cudaDeviceCanAccessPeer(&can, devid, worker->devid);
  209. if (!cures && can)
  210. {
  211. cures = cudaDeviceEnablePeerAccess(worker->devid, 0);
  212. if (!cures)
  213. _STARPU_DEBUG("Enabled GPU-Direct %d -> %d\n", worker->devid, devid);
  214. }
  215. }
  216. }
  217. }
  218. #endif
  219. /* force CUDA to initialize the context for real */
  220. cures = cudaFree(0);
  221. if (STARPU_UNLIKELY(cures))
  222. {
  223. if (cures == cudaErrorDevicesUnavailable)
  224. {
  225. fprintf(stderr,"All CUDA-capable devices are busy or unavailable\n");
  226. exit(77);
  227. }
  228. STARPU_CUDA_REPORT_ERROR(cures);
  229. }
  230. cures = cudaGetDeviceProperties(&props[devid], devid);
  231. if (STARPU_UNLIKELY(cures))
  232. STARPU_CUDA_REPORT_ERROR(cures);
  233. #ifdef HAVE_CUDA_MEMCPY_PEER
  234. if (props[devid].computeMode == cudaComputeModeExclusive)
  235. {
  236. 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");
  237. STARPU_ABORT();
  238. }
  239. #endif
  240. for (i = 0; i < worker_set->nworkers; i++)
  241. {
  242. workerid = worker_set->workers[i].workerid;
  243. for (j = 0; j < STARPU_MAX_PIPELINE; j++)
  244. cures = cudaEventCreateWithFlags(&task_events[workerid][j], cudaEventDisableTiming);
  245. if (STARPU_UNLIKELY(cures))
  246. STARPU_CUDA_REPORT_ERROR(cures);
  247. cures = cudaStreamCreate(&streams[workerid]);
  248. if (STARPU_UNLIKELY(cures))
  249. STARPU_CUDA_REPORT_ERROR(cures);
  250. }
  251. cures = cudaStreamCreate(&in_transfer_streams[devid]);
  252. if (STARPU_UNLIKELY(cures))
  253. STARPU_CUDA_REPORT_ERROR(cures);
  254. cures = cudaStreamCreate(&out_transfer_streams[devid]);
  255. if (STARPU_UNLIKELY(cures))
  256. STARPU_CUDA_REPORT_ERROR(cures);
  257. for (i = 0; i < ncudagpus; i++)
  258. {
  259. cures = cudaStreamCreate(&in_peer_transfer_streams[i][devid]);
  260. if (STARPU_UNLIKELY(cures))
  261. STARPU_CUDA_REPORT_ERROR(cures);
  262. cures = cudaStreamCreate(&out_peer_transfer_streams[devid][i]);
  263. if (STARPU_UNLIKELY(cures))
  264. STARPU_CUDA_REPORT_ERROR(cures);
  265. }
  266. }
  267. static void deinit_context(struct _starpu_worker_set *worker_set)
  268. {
  269. cudaError_t cures;
  270. unsigned i, j;
  271. int workerid = worker_set->workers[0].workerid;
  272. int devid = starpu_worker_get_devid(workerid);
  273. for (i = 0; i < worker_set->nworkers; i++)
  274. {
  275. workerid = worker_set->workers[i].workerid;
  276. devid = starpu_worker_get_devid(workerid);
  277. for (j = 0; j < STARPU_MAX_PIPELINE; j++)
  278. cudaEventDestroy(task_events[workerid][j]);
  279. cudaStreamDestroy(streams[workerid]);
  280. }
  281. cudaStreamDestroy(in_transfer_streams[devid]);
  282. cudaStreamDestroy(out_transfer_streams[devid]);
  283. for (i = 0; i < ncudagpus; i++)
  284. {
  285. cudaStreamDestroy(in_peer_transfer_streams[i][devid]);
  286. cudaStreamDestroy(out_peer_transfer_streams[devid][i]);
  287. }
  288. /* cleanup the runtime API internal stuffs (which CUBLAS is using) */
  289. cures = cudaThreadExit();
  290. if (cures)
  291. STARPU_CUDA_REPORT_ERROR(cures);
  292. }
  293. #endif /* !SIMGRID */
  294. static size_t _starpu_cuda_get_global_mem_size(unsigned devid)
  295. {
  296. return global_mem[devid];
  297. }
  298. /* Return the number of devices usable in the system.
  299. * The value returned cannot be greater than MAXCUDADEVS */
  300. unsigned _starpu_get_cuda_device_count(void)
  301. {
  302. #ifdef STARPU_SIMGRID
  303. return _starpu_simgrid_get_nbhosts("CUDA");
  304. #else
  305. int cnt;
  306. cudaError_t cures;
  307. cures = cudaGetDeviceCount(&cnt);
  308. if (STARPU_UNLIKELY(cures))
  309. return 0;
  310. if (cnt > STARPU_MAXCUDADEVS)
  311. {
  312. 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);
  313. cnt = STARPU_MAXCUDADEVS;
  314. }
  315. return (unsigned)cnt;
  316. #endif
  317. }
  318. void _starpu_init_cuda(void)
  319. {
  320. ncudagpus = _starpu_get_cuda_device_count();
  321. STARPU_ASSERT(ncudagpus <= STARPU_MAXCUDADEVS);
  322. }
  323. static int start_job_on_cuda(struct _starpu_job *j, struct _starpu_worker *worker)
  324. {
  325. int ret;
  326. STARPU_ASSERT(j);
  327. struct starpu_task *task = j->task;
  328. int profiling = starpu_profiling_status_get();
  329. STARPU_ASSERT(task);
  330. struct starpu_codelet *cl = task->cl;
  331. STARPU_ASSERT(cl);
  332. _starpu_set_current_task(task);
  333. ret = _starpu_fetch_task_input(j);
  334. if (ret != 0)
  335. {
  336. /* there was not enough memory, so the input of
  337. * the codelet cannot be fetched ... put the
  338. * codelet back, and try it later */
  339. return -EAGAIN;
  340. }
  341. if (worker->ntasks == 1)
  342. {
  343. /* We are alone in the pipeline, the kernel will start now, record it */
  344. _starpu_driver_start_job(worker, j, &worker->perf_arch, &j->cl_start, 0, profiling);
  345. }
  346. #if defined(HAVE_CUDA_MEMCPY_PEER) && !defined(STARPU_SIMGRID)
  347. /* We make sure we do manipulate the proper device */
  348. starpu_cuda_set_device(worker->devid);
  349. #endif
  350. starpu_cuda_func_t func = _starpu_task_get_cuda_nth_implementation(cl, j->nimpl);
  351. STARPU_ASSERT_MSG(func, "when STARPU_CUDA is defined in 'where', cuda_func or cuda_funcs has to be defined");
  352. if (starpu_get_env_number("STARPU_DISABLE_KERNELS") <= 0)
  353. {
  354. _STARPU_TRACE_START_EXECUTING();
  355. #ifdef STARPU_SIMGRID
  356. _starpu_simgrid_execute_job(j, &worker->perf_arch, NAN);
  357. #else
  358. func(_STARPU_TASK_GET_INTERFACES(task), task->cl_arg);
  359. #endif
  360. _STARPU_TRACE_END_EXECUTING();
  361. }
  362. return 0;
  363. }
  364. static void finish_job_on_cuda(struct _starpu_job *j, struct _starpu_worker *worker)
  365. {
  366. struct timespec codelet_end;
  367. int profiling = starpu_profiling_status_get();
  368. _starpu_set_current_task(NULL);
  369. worker->current_tasks[worker->first_task] = NULL;
  370. worker->first_task = (worker->first_task + 1) % STARPU_MAX_PIPELINE;
  371. worker->ntasks--;
  372. _starpu_driver_end_job(worker, j, &worker->perf_arch, &codelet_end, 0, profiling);
  373. _starpu_driver_update_job_feedback(j, worker, &worker->perf_arch, &j->cl_start, &codelet_end, profiling);
  374. _starpu_push_task_output(j);
  375. _starpu_handle_job_termination(j);
  376. }
  377. /* Execute a job, up to completion for synchronous jobs */
  378. static void execute_job_on_cuda(struct starpu_task *task, struct _starpu_worker *worker)
  379. {
  380. int workerid = worker->workerid;
  381. int res;
  382. struct _starpu_job *j = _starpu_get_job_associated_to_task(task);
  383. res = start_job_on_cuda(j, worker);
  384. if (res)
  385. {
  386. switch (res)
  387. {
  388. case -EAGAIN:
  389. _STARPU_DISP("ouch, CUDA could not actually run task %p, putting it back...\n", task);
  390. _starpu_push_task_to_workers(task);
  391. STARPU_ABORT();
  392. default:
  393. STARPU_ABORT();
  394. }
  395. }
  396. #ifndef STARPU_SIMGRID
  397. if (task->cl->cuda_flags[j->nimpl] & STARPU_CUDA_ASYNC)
  398. {
  399. /* Record event to synchronize with task termination later */
  400. cudaEventRecord(task_events[workerid][(worker->first_task + worker->ntasks - 1)%STARPU_MAX_PIPELINE], starpu_cuda_get_local_stream());
  401. #ifdef STARPU_USE_FXT
  402. int k;
  403. for (k = 0; k < (int) worker->set->nworkers; k++)
  404. if (worker->set->workers[k].ntasks == worker->set->workers[k].pipeline_length)
  405. break;
  406. if (k == (int) worker->set->nworkers)
  407. /* Everybody busy */
  408. _STARPU_TRACE_START_EXECUTING()
  409. #endif
  410. }
  411. else
  412. #else
  413. #ifdef STARPU_DEVEL
  414. #warning No CUDA asynchronous execution with simgrid yet.
  415. #endif
  416. #endif
  417. /* Synchronous execution */
  418. {
  419. #if defined(STARPU_DEBUG) && !defined(STARPU_SIMGRID)
  420. STARPU_ASSERT_MSG(cudaStreamQuery(starpu_cuda_get_local_stream()) == cudaSuccess, "CUDA codelets have to wait for termination of their kernels on the starpu_cuda_get_local_stream() stream");
  421. #endif
  422. finish_job_on_cuda(j, worker);
  423. }
  424. }
  425. /* XXX Should this be merged with _starpu_init_cuda ? */
  426. int _starpu_cuda_driver_init(struct _starpu_worker_set *worker_set)
  427. {
  428. struct _starpu_worker *worker0 = &worker_set->workers[0];
  429. unsigned devid = worker0->devid;
  430. unsigned i;
  431. _starpu_worker_start(worker0, _STARPU_FUT_CUDA_KEY);
  432. #ifdef STARPU_USE_FXT
  433. unsigned memnode = worker0->memory_node;
  434. for (i = 1; i < worker_set->nworkers; i++)
  435. {
  436. struct _starpu_worker *worker = &worker_set->workers[i];
  437. _STARPU_TRACE_WORKER_INIT_START(_STARPU_FUT_CUDA_KEY, worker->workerid, devid, memnode);
  438. }
  439. #endif
  440. #ifndef STARPU_SIMGRID
  441. init_context(worker_set, devid);
  442. if (worker_set->nworkers > 1 && props[devid].concurrentKernels == 0)
  443. _STARPU_DISP("Warning: STARPU_NWORKER_PER_CUDA is %u, but the device does not support concurrent kernel execution!\n", worker_set->nworkers);
  444. #endif
  445. _starpu_cuda_limit_gpu_mem_if_needed(devid);
  446. _starpu_memory_manager_set_global_memory_size(worker0->memory_node, _starpu_cuda_get_global_mem_size(devid));
  447. _starpu_malloc_init(worker0->memory_node);
  448. /* one more time to avoid hacks from third party lib :) */
  449. _starpu_bind_thread_on_cpu(worker0->config, worker0->bindid);
  450. worker0->status = STATUS_UNKNOWN;
  451. float size = (float) global_mem[devid] / (1<<30);
  452. #ifdef STARPU_SIMGRID
  453. const char *devname = "Simgrid";
  454. #else
  455. /* get the device's name */
  456. char devname[128];
  457. strncpy(devname, props[devid].name, 128);
  458. #endif
  459. for (i = 0; i < worker_set->nworkers; i++)
  460. {
  461. struct _starpu_worker *worker = &worker_set->workers[i];
  462. #if defined(STARPU_HAVE_BUSID) && !defined(STARPU_SIMGRID)
  463. #if defined(STARPU_HAVE_DOMAINID) && !defined(STARPU_SIMGRID)
  464. if (props[devid].pciDomainID)
  465. 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);
  466. else
  467. #endif
  468. 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);
  469. #else
  470. snprintf(worker->name, sizeof(worker->name), "CUDA %u.%u (%s %.1f GiB)", devid, i, devname, size);
  471. #endif
  472. snprintf(worker->short_name, sizeof(worker->short_name), "CUDA %u.%u", devid, i);
  473. _STARPU_DEBUG("cuda (%s) dev id %u worker %u thread is ready to run on CPU %d !\n", devname, devid, i, worker->bindid);
  474. worker->pipeline_length = starpu_get_env_number_default("STARPU_CUDA_PIPELINE", 2);
  475. _STARPU_TRACE_WORKER_INIT_END(worker_set->workers[i].workerid);
  476. }
  477. /* tell the main thread that this one is ready */
  478. STARPU_PTHREAD_MUTEX_LOCK(&worker0->mutex);
  479. worker0->worker_is_initialized = 1;
  480. STARPU_PTHREAD_COND_SIGNAL(&worker0->ready_cond);
  481. STARPU_PTHREAD_MUTEX_UNLOCK(&worker0->mutex);
  482. /* tell the main thread that this one is ready */
  483. STARPU_PTHREAD_MUTEX_LOCK(&worker_set->mutex);
  484. worker_set->set_is_initialized = 1;
  485. STARPU_PTHREAD_COND_SIGNAL(&worker_set->ready_cond);
  486. STARPU_PTHREAD_MUTEX_UNLOCK(&worker_set->mutex);
  487. return 0;
  488. }
  489. int _starpu_cuda_driver_run_once(struct _starpu_worker_set *worker_set)
  490. {
  491. struct _starpu_worker *worker0 = &worker_set->workers[0];
  492. unsigned memnode = worker0->memory_node;
  493. struct starpu_task *tasks[worker_set->nworkers], *task;
  494. struct _starpu_job *j;
  495. int i, res;
  496. #ifndef STARPU_SIMGRID
  497. int idle;
  498. /* First poll for completed jobs */
  499. idle = 0;
  500. for (i = 0; i < (int) worker_set->nworkers; i++)
  501. {
  502. struct _starpu_worker *worker = &worker_set->workers[i];
  503. int workerid = worker->workerid;
  504. if (!worker->ntasks)
  505. {
  506. idle++;
  507. /* Even nothing to test */
  508. continue;
  509. }
  510. task = worker->current_tasks[worker->first_task];
  511. /* On-going asynchronous task, check for its termination first */
  512. cudaError_t cures = cudaEventQuery(task_events[workerid][worker->first_task]);
  513. if (cures != cudaSuccess)
  514. {
  515. STARPU_ASSERT(cures == cudaErrorNotReady);
  516. }
  517. else
  518. {
  519. /* Asynchronous task completed! */
  520. _starpu_set_local_worker_key(worker);
  521. finish_job_on_cuda(_starpu_get_job_associated_to_task(task), worker);
  522. /* See next task if any */
  523. if (worker->ntasks)
  524. {
  525. task = worker->current_tasks[worker->first_task];
  526. j = _starpu_get_job_associated_to_task(task);
  527. if (task->cl->cuda_flags[j->nimpl] & STARPU_CUDA_ASYNC)
  528. {
  529. /* An asynchronous task, it was already
  530. * queued, it's now running, record its start time. */
  531. _starpu_driver_start_job(worker, j, &worker->perf_arch, &j->cl_start, 0, starpu_profiling_status_get());
  532. }
  533. else
  534. {
  535. /* A synchronous task, we have finished
  536. * flushing the pipeline, we can now at
  537. * last execute it. */
  538. _STARPU_TRACE_END_PROGRESS(memnode);
  539. _STARPU_TRACE_EVENT("sync_task");
  540. execute_job_on_cuda(task, worker);
  541. _STARPU_TRACE_EVENT("end_sync_task");
  542. _STARPU_TRACE_START_PROGRESS(memnode);
  543. worker->pipeline_stuck = 0;
  544. }
  545. }
  546. #ifdef STARPU_USE_FXT
  547. int k;
  548. for (k = 0; k < (int) worker_set->nworkers; k++)
  549. if (worker_set->workers[k].ntasks)
  550. break;
  551. if (k == (int) worker_set->nworkers)
  552. /* Everybody busy */
  553. _STARPU_TRACE_END_EXECUTING()
  554. #endif
  555. }
  556. if (worker->ntasks < worker->pipeline_length)
  557. idle++;
  558. }
  559. if (!idle)
  560. {
  561. /* Nothing ready yet, no better thing to do than waiting */
  562. __starpu_datawizard_progress(memnode, 1, 0);
  563. return 0;
  564. }
  565. #endif /* STARPU_SIMGRID */
  566. /* Something done, make some progress */
  567. __starpu_datawizard_progress(memnode, 1, 1);
  568. /* And pull tasks */
  569. res = _starpu_get_multi_worker_task(worker_set->workers, tasks, worker_set->nworkers);
  570. if (!res)
  571. return 0;
  572. for (i = 0; i < (int) worker_set->nworkers; i++)
  573. {
  574. struct _starpu_worker *worker = &worker_set->workers[i];
  575. task = tasks[i];
  576. if (!task)
  577. continue;
  578. j = _starpu_get_job_associated_to_task(task);
  579. /* can CUDA do that task ? */
  580. if (!_STARPU_CUDA_MAY_PERFORM(j))
  581. {
  582. /* this is neither a cuda or a cublas task */
  583. worker->ntasks--;
  584. _starpu_push_task_to_workers(task);
  585. continue;
  586. }
  587. if (worker->ntasks > 1 && !(task->cl->cuda_flags[j->nimpl] & STARPU_CUDA_ASYNC))
  588. {
  589. /* We have to execute a non-asynchronous task but we
  590. * still have tasks in the pipeline... Record it to
  591. * prevent more tasks from coming, and do it later */
  592. worker->pipeline_stuck = 1;
  593. continue;
  594. }
  595. _starpu_set_local_worker_key(worker);
  596. _STARPU_TRACE_END_PROGRESS(memnode);
  597. execute_job_on_cuda(task, worker);
  598. _STARPU_TRACE_START_PROGRESS(memnode);
  599. }
  600. return 0;
  601. }
  602. int _starpu_cuda_driver_deinit(struct _starpu_worker_set *arg)
  603. {
  604. struct _starpu_worker *worker = &arg->workers[0];
  605. unsigned memnode = worker->memory_node;
  606. _STARPU_TRACE_WORKER_DEINIT_START;
  607. _starpu_handle_all_pending_node_data_requests(memnode);
  608. /* In case there remains some memory that was automatically
  609. * allocated by StarPU, we release it now. Note that data
  610. * coherency is not maintained anymore at that point ! */
  611. _starpu_free_all_automatically_allocated_buffers(memnode);
  612. _starpu_malloc_shutdown(memnode);
  613. #ifndef STARPU_SIMGRID
  614. deinit_context(arg);
  615. #endif
  616. _STARPU_TRACE_WORKER_DEINIT_END(_STARPU_FUT_CUDA_KEY);
  617. return 0;
  618. }
  619. void *_starpu_cuda_worker(void *_arg)
  620. {
  621. struct _starpu_worker_set* worker = _arg;
  622. _starpu_cuda_driver_init(worker);
  623. _STARPU_TRACE_START_PROGRESS(memnode);
  624. while (_starpu_machine_is_running())
  625. _starpu_cuda_driver_run_once(worker);
  626. _STARPU_TRACE_END_PROGRESS(memnode);
  627. _starpu_cuda_driver_deinit(worker);
  628. return NULL;
  629. }
  630. #ifdef STARPU_USE_CUDA
  631. void starpu_cublas_report_error(const char *func, const char *file, int line, cublasStatus status)
  632. {
  633. char *errormsg;
  634. switch (status)
  635. {
  636. case CUBLAS_STATUS_SUCCESS:
  637. errormsg = "success";
  638. break;
  639. case CUBLAS_STATUS_NOT_INITIALIZED:
  640. errormsg = "not initialized";
  641. break;
  642. case CUBLAS_STATUS_ALLOC_FAILED:
  643. errormsg = "alloc failed";
  644. break;
  645. case CUBLAS_STATUS_INVALID_VALUE:
  646. errormsg = "invalid value";
  647. break;
  648. case CUBLAS_STATUS_ARCH_MISMATCH:
  649. errormsg = "arch mismatch";
  650. break;
  651. case CUBLAS_STATUS_EXECUTION_FAILED:
  652. errormsg = "execution failed";
  653. break;
  654. case CUBLAS_STATUS_INTERNAL_ERROR:
  655. errormsg = "internal error";
  656. break;
  657. default:
  658. errormsg = "unknown error";
  659. break;
  660. }
  661. fprintf(stderr, "oops in %s (%s:%d)... %d: %s \n", func, file, line, status, errormsg);
  662. STARPU_ABORT();
  663. }
  664. void starpu_cuda_report_error(const char *func, const char *file, int line, cudaError_t status)
  665. {
  666. const char *errormsg = cudaGetErrorString(status);
  667. printf("oops in %s (%s:%d)... %d: %s \n", func, file, line, status, errormsg);
  668. STARPU_ABORT();
  669. }
  670. #endif /* STARPU_USE_CUDA */
  671. #ifdef STARPU_USE_CUDA
  672. int
  673. starpu_cuda_copy_async_sync(void *src_ptr, unsigned src_node,
  674. void *dst_ptr, unsigned dst_node,
  675. size_t ssize, cudaStream_t stream,
  676. enum cudaMemcpyKind kind)
  677. {
  678. #ifdef HAVE_CUDA_MEMCPY_PEER
  679. int peer_copy = 0;
  680. int src_dev = -1, dst_dev = -1;
  681. #endif
  682. cudaError_t cures = 0;
  683. if (kind == cudaMemcpyDeviceToDevice && src_node != dst_node)
  684. {
  685. #ifdef HAVE_CUDA_MEMCPY_PEER
  686. peer_copy = 1;
  687. src_dev = _starpu_memory_node_get_devid(src_node);
  688. dst_dev = _starpu_memory_node_get_devid(dst_node);
  689. #else
  690. STARPU_ABORT();
  691. #endif
  692. }
  693. if (stream)
  694. {
  695. _STARPU_TRACE_START_DRIVER_COPY_ASYNC(src_node, dst_node);
  696. #ifdef HAVE_CUDA_MEMCPY_PEER
  697. if (peer_copy)
  698. {
  699. cures = cudaMemcpyPeerAsync((char *) dst_ptr, dst_dev,
  700. (char *) src_ptr, src_dev,
  701. ssize, stream);
  702. }
  703. else
  704. #endif
  705. {
  706. cures = cudaMemcpyAsync((char *)dst_ptr, (char *)src_ptr, ssize, kind, stream);
  707. }
  708. _STARPU_TRACE_END_DRIVER_COPY_ASYNC(src_node, dst_node);
  709. }
  710. /* Test if the asynchronous copy has failed or if the caller only asked for a synchronous copy */
  711. if (stream == NULL || cures)
  712. {
  713. /* do it in a synchronous fashion */
  714. #ifdef HAVE_CUDA_MEMCPY_PEER
  715. if (peer_copy)
  716. {
  717. cures = cudaMemcpyPeer((char *) dst_ptr, dst_dev,
  718. (char *) src_ptr, src_dev,
  719. ssize);
  720. }
  721. else
  722. #endif
  723. {
  724. cures = cudaMemcpy((char *)dst_ptr, (char *)src_ptr, ssize, kind);
  725. }
  726. if (STARPU_UNLIKELY(cures))
  727. STARPU_CUDA_REPORT_ERROR(cures);
  728. return 0;
  729. }
  730. return -EAGAIN;
  731. }
  732. #endif /* STARPU_USE_CUDA */
  733. int _starpu_run_cuda(struct _starpu_worker_set *workerarg)
  734. {
  735. /* Let's go ! */
  736. _starpu_cuda_worker(workerarg);
  737. return 0;
  738. }