driver_cpu.c 18 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593594595596597598599600601602603604605606607608609610611612613614615616
  1. /* StarPU --- Runtime system for heterogeneous multicore architectures.
  2. *
  3. * Copyright (C) 2008-2021 Université de Bordeaux, CNRS (LaBRI UMR 5800), Inria
  4. * Copyright (C) 2010 Mehdi Juhoor
  5. * Copyright (C) 2011 Télécom-SudParis
  6. * Copyright (C) 2013 Thibaut Lambert
  7. * Copyright (C) 2020 Federal University of Rio Grande do Sul (UFRGS)
  8. *
  9. * StarPU is free software; you can redistribute it and/or modify
  10. * it under the terms of the GNU Lesser General Public License as published by
  11. * the Free Software Foundation; either version 2.1 of the License, or (at
  12. * your option) any later version.
  13. *
  14. * StarPU is distributed in the hope that it will be useful, but
  15. * WITHOUT ANY WARRANTY; without even the implied warranty of
  16. * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE.
  17. *
  18. * See the GNU Lesser General Public License in COPYING.LGPL for more details.
  19. */
  20. #include <common/config.h>
  21. #include <math.h>
  22. #include <starpu.h>
  23. #include <starpu_profiling.h>
  24. #include <drivers/driver_common/driver_common.h>
  25. #include <common/utils.h>
  26. #include <core/debug.h>
  27. #include <core/workers.h>
  28. #include <core/drivers.h>
  29. #include <core/idle_hook.h>
  30. #include <drivers/cpu/driver_cpu.h>
  31. #include <drivers/disk/driver_disk.h>
  32. #include <drivers/opencl/driver_opencl.h>
  33. #include <drivers/cuda/driver_cuda.h>
  34. #include <drivers/mpi/driver_mpi_source.h>
  35. #include <drivers/disk/driver_disk.h>
  36. #include <core/sched_policy.h>
  37. #include <datawizard/memory_manager.h>
  38. #include <datawizard/memory_nodes.h>
  39. #include <datawizard/malloc.h>
  40. #include <datawizard/datawizard.h>
  41. #include <core/simgrid.h>
  42. #include <core/task.h>
  43. #include <core/disk.h>
  44. #include <common/knobs.h>
  45. #ifdef STARPU_HAVE_HWLOC
  46. #include <hwloc.h>
  47. #ifndef HWLOC_API_VERSION
  48. #define HWLOC_OBJ_PU HWLOC_OBJ_PROC
  49. #endif
  50. #if HWLOC_API_VERSION < 0x00010b00
  51. #define HWLOC_OBJ_NUMANODE HWLOC_OBJ_NODE
  52. #endif
  53. #endif
  54. #ifdef STARPU_HAVE_WINDOWS
  55. #include <windows.h>
  56. #endif
  57. static struct _starpu_driver_info driver_info =
  58. {
  59. .name_upper = "CPU",
  60. .name_var = "CPU",
  61. .name_lower = "cpu",
  62. .memory_kind = STARPU_CPU_RAM,
  63. .alpha = 0.5f,
  64. };
  65. static struct _starpu_memory_driver_info memory_driver_info =
  66. {
  67. .name_upper = "NUMA",
  68. .worker_archtype = STARPU_CPU_WORKER,
  69. };
  70. void _starpu_cpu_preinit(void)
  71. {
  72. _starpu_driver_info_register(STARPU_CPU_WORKER, &driver_info);
  73. _starpu_memory_driver_info_register(STARPU_CPU_RAM, &memory_driver_info);
  74. }
  75. #ifdef STARPU_USE_CPU
  76. /* Actually launch the job on a cpu worker.
  77. * Handle binding CPUs on cores.
  78. * In the case of a combined worker WORKER_TASK != J->TASK */
  79. static int execute_job_on_cpu(struct _starpu_job *j, struct starpu_task *worker_task, struct _starpu_worker *cpu_args, int rank, struct starpu_perfmodel_arch* perf_arch)
  80. {
  81. int is_parallel_task = (j->task_size > 1);
  82. int profiling = starpu_profiling_status_get();
  83. struct starpu_task *task = j->task;
  84. struct starpu_codelet *cl = task->cl;
  85. STARPU_ASSERT(cl);
  86. if (is_parallel_task)
  87. {
  88. STARPU_PTHREAD_BARRIER_WAIT(&j->before_work_barrier);
  89. /* In the case of a combined worker, the scheduler needs to know
  90. * when each actual worker begins the execution */
  91. _starpu_sched_pre_exec_hook(worker_task);
  92. }
  93. /* Give profiling variable */
  94. _starpu_driver_start_job(cpu_args, j, perf_arch, rank, profiling);
  95. /* In case this is a Fork-join parallel task, the worker does not
  96. * execute the kernel at all. */
  97. if ((rank == 0) || (cl->type != STARPU_FORKJOIN))
  98. {
  99. _starpu_cl_func_t func = _starpu_task_get_cpu_nth_implementation(cl, j->nimpl);
  100. if (is_parallel_task && cl->type == STARPU_FORKJOIN)
  101. /* bind to parallel worker */
  102. _starpu_bind_thread_on_cpus(_starpu_get_combined_worker_struct(j->combined_workerid));
  103. STARPU_ASSERT_MSG(func, "when STARPU_CPU is defined in 'where', cpu_func or cpu_funcs has to be defined");
  104. if (_starpu_get_disable_kernels() <= 0)
  105. {
  106. _STARPU_TRACE_START_EXECUTING();
  107. #ifdef STARPU_SIMGRID
  108. if (cl->flags & STARPU_CODELET_SIMGRID_EXECUTE)
  109. func(_STARPU_TASK_GET_INTERFACES(task), task->cl_arg);
  110. else if (cl->flags & STARPU_CODELET_SIMGRID_EXECUTE_AND_INJECT)
  111. {
  112. _SIMGRID_TIMER_BEGIN(1);
  113. func(_STARPU_TASK_GET_INTERFACES(task), task->cl_arg);
  114. _SIMGRID_TIMER_END;
  115. }
  116. else
  117. {
  118. struct _starpu_sched_ctx *sched_ctx = _starpu_sched_ctx_get_sched_ctx_for_worker_and_job(cpu_args, j);
  119. _starpu_simgrid_submit_job(cpu_args->workerid, sched_ctx->id, j, perf_arch, NAN, NAN, NULL);
  120. }
  121. #else
  122. # ifdef STARPU_PAPI
  123. if (rank == 0)
  124. _starpu_profiling_papi_task_start_counters(task);
  125. # endif
  126. func(_STARPU_TASK_GET_INTERFACES(task), task->cl_arg);
  127. # ifdef STARPU_PAPI
  128. if (rank == 0)
  129. _starpu_profiling_papi_task_stop_counters(task);
  130. # endif
  131. #endif
  132. _STARPU_TRACE_END_EXECUTING();
  133. }
  134. if (is_parallel_task && cl->type == STARPU_FORKJOIN)
  135. /* rebind to single CPU */
  136. _starpu_bind_thread_on_cpu(cpu_args->bindid, cpu_args->workerid, NULL);
  137. }
  138. else
  139. {
  140. _STARPU_TRACE_START_EXECUTING();
  141. }
  142. if (is_parallel_task)
  143. {
  144. STARPU_PTHREAD_BARRIER_WAIT(&j->after_work_barrier);
  145. if (rank != 0)
  146. _STARPU_TRACE_END_EXECUTING();
  147. }
  148. _starpu_driver_end_job(cpu_args, j, perf_arch, rank, profiling);
  149. if (is_parallel_task)
  150. {
  151. #ifdef STARPU_SIMGRID
  152. if (rank == 0)
  153. {
  154. /* Wait for other threads to exit barrier_wait so we
  155. * can safely drop the job structure */
  156. starpu_sleep(0.0000001);
  157. j->after_work_busy_barrier = 0;
  158. }
  159. #else
  160. ANNOTATE_HAPPENS_BEFORE(&j->after_work_busy_barrier);
  161. (void) STARPU_ATOMIC_ADD(&j->after_work_busy_barrier, -1);
  162. if (rank == 0)
  163. {
  164. /* Wait with a busy barrier for other workers to have
  165. * finished with the blocking barrier before we can
  166. * safely drop the job structure */
  167. while (j->after_work_busy_barrier > 0)
  168. {
  169. STARPU_UYIELD();
  170. STARPU_SYNCHRONIZE();
  171. }
  172. ANNOTATE_HAPPENS_AFTER(&j->after_work_busy_barrier);
  173. }
  174. #endif
  175. }
  176. if (rank == 0)
  177. {
  178. _starpu_driver_update_job_feedback(j, cpu_args, perf_arch, profiling);
  179. #ifdef STARPU_OPENMP
  180. if (!j->continuation)
  181. #endif
  182. {
  183. _starpu_push_task_output(j);
  184. }
  185. }
  186. return 0;
  187. }
  188. int _starpu_cpu_driver_init(struct _starpu_worker *cpu_worker)
  189. {
  190. int devid = cpu_worker->devid;
  191. _starpu_driver_start(cpu_worker, STARPU_CPU_WORKER, 1);
  192. snprintf(cpu_worker->name, sizeof(cpu_worker->name), "CPU %d", devid);
  193. snprintf(cpu_worker->short_name, sizeof(cpu_worker->short_name), "CPU %d", devid);
  194. starpu_pthread_setname(cpu_worker->short_name);
  195. _STARPU_TRACE_WORKER_INIT_END(cpu_worker->workerid);
  196. STARPU_PTHREAD_MUTEX_LOCK_SCHED(&cpu_worker->sched_mutex);
  197. cpu_worker->status = STATUS_UNKNOWN;
  198. STARPU_PTHREAD_MUTEX_UNLOCK_SCHED(&cpu_worker->sched_mutex);
  199. /* tell the main thread that we are ready */
  200. STARPU_PTHREAD_MUTEX_LOCK(&cpu_worker->mutex);
  201. cpu_worker->worker_is_initialized = 1;
  202. STARPU_PTHREAD_COND_SIGNAL(&cpu_worker->ready_cond);
  203. STARPU_PTHREAD_MUTEX_UNLOCK(&cpu_worker->mutex);
  204. return 0;
  205. }
  206. static int _starpu_cpu_driver_execute_task(struct _starpu_worker *cpu_worker, struct starpu_task *task, struct _starpu_job *j)
  207. {
  208. int res;
  209. int rank;
  210. int is_parallel_task = (j->task_size > 1);
  211. struct starpu_perfmodel_arch* perf_arch;
  212. rank = cpu_worker->current_rank;
  213. /* Get the rank in case it is a parallel task */
  214. if (is_parallel_task)
  215. {
  216. if(j->combined_workerid != -1)
  217. {
  218. struct _starpu_combined_worker *combined_worker;
  219. combined_worker = _starpu_get_combined_worker_struct(j->combined_workerid);
  220. cpu_worker->combined_workerid = j->combined_workerid;
  221. cpu_worker->worker_size = combined_worker->worker_size;
  222. perf_arch = &combined_worker->perf_arch;
  223. }
  224. else
  225. {
  226. struct _starpu_sched_ctx *sched_ctx = _starpu_sched_ctx_get_sched_ctx_for_worker_and_job(cpu_worker, j);
  227. STARPU_ASSERT_MSG(sched_ctx != NULL, "there should be a worker %d in the ctx of this job \n", cpu_worker->workerid);
  228. perf_arch = &sched_ctx->perf_arch;
  229. }
  230. }
  231. else
  232. {
  233. cpu_worker->combined_workerid = cpu_worker->workerid;
  234. cpu_worker->worker_size = 1;
  235. struct _starpu_sched_ctx *sched_ctx = _starpu_sched_ctx_get_sched_ctx_for_worker_and_job(cpu_worker, j);
  236. if (sched_ctx && !sched_ctx->sched_policy && !sched_ctx->awake_workers && sched_ctx->main_master == cpu_worker->workerid)
  237. perf_arch = &sched_ctx->perf_arch;
  238. else
  239. perf_arch = &cpu_worker->perf_arch;
  240. }
  241. _starpu_set_current_task(j->task);
  242. cpu_worker->current_task = j->task;
  243. res = execute_job_on_cpu(j, task, cpu_worker, rank, perf_arch);
  244. _starpu_set_current_task(NULL);
  245. cpu_worker->current_task = NULL;
  246. if (res)
  247. {
  248. switch (res)
  249. {
  250. case -EAGAIN:
  251. _starpu_push_task_to_workers(task);
  252. return 0;
  253. default:
  254. STARPU_ABORT();
  255. }
  256. }
  257. /* In the case of combined workers, we need to inform the
  258. * scheduler each worker's execution is over.
  259. * Then we free the workers' task alias */
  260. if (is_parallel_task)
  261. {
  262. _starpu_sched_post_exec_hook(task);
  263. free(task);
  264. }
  265. if (rank == 0)
  266. _starpu_handle_job_termination(j);
  267. return 0;
  268. }
  269. int _starpu_cpu_driver_run_once(struct _starpu_worker *cpu_worker)
  270. {
  271. unsigned memnode = cpu_worker->memory_node;
  272. int workerid = cpu_worker->workerid;
  273. int res;
  274. struct _starpu_job *j;
  275. struct starpu_task *task = NULL, *pending_task;
  276. int rank = 0;
  277. #ifdef STARPU_SIMGRID
  278. starpu_pthread_wait_reset(&cpu_worker->wait);
  279. #endif
  280. /* Test if async transfers are completed */
  281. pending_task = cpu_worker->task_transferring;
  282. if (pending_task != NULL && cpu_worker->nb_buffers_transferred == cpu_worker->nb_buffers_totransfer)
  283. {
  284. int ret;
  285. STARPU_RMB();
  286. _STARPU_TRACE_END_PROGRESS(memnode);
  287. j = _starpu_get_job_associated_to_task(pending_task);
  288. _starpu_fetch_task_input_tail(pending_task, j, cpu_worker);
  289. _starpu_set_worker_status(cpu_worker, STATUS_UNKNOWN);
  290. /* Reset it */
  291. cpu_worker->task_transferring = NULL;
  292. ret = _starpu_cpu_driver_execute_task(cpu_worker, pending_task, j);
  293. _STARPU_TRACE_START_PROGRESS(memnode);
  294. return ret;
  295. }
  296. res = __starpu_datawizard_progress(_STARPU_DATAWIZARD_DO_ALLOC, 1);
  297. if (!pending_task)
  298. task = _starpu_get_worker_task(cpu_worker, workerid, memnode);
  299. #ifdef STARPU_SIMGRID
  300. #ifndef STARPU_OPENMP
  301. if (!res && !task)
  302. /* No progress, wait */
  303. starpu_pthread_wait_wait(&cpu_worker->wait);
  304. #else
  305. #if SIMGRID_VERSION >= 31800
  306. if (!res && !task)
  307. {
  308. /* No progress, wait (but at most 1s for OpenMP support) */
  309. /* TODO: ideally, make OpenMP wake worker when run_once should return */
  310. struct timespec abstime;
  311. _starpu_clock_gettime(&abstime);
  312. abstime.tv_sec++;
  313. starpu_pthread_wait_timedwait(&cpu_worker->wait, &abstime);
  314. }
  315. #else
  316. /* Previous simgrid versions don't really permit to use wait_timedwait in C */
  317. starpu_sleep(0.001);
  318. #endif
  319. #endif
  320. #endif
  321. if (!task)
  322. {
  323. /* No task or task still pending transfers */
  324. _starpu_execute_registered_idle_hooks();
  325. return 0;
  326. }
  327. j = _starpu_get_job_associated_to_task(task);
  328. /* NOTE: j->task is != task for parallel tasks, which share the same
  329. * job. */
  330. /* can a cpu perform that task ? */
  331. if (!_STARPU_MAY_PERFORM(j, CPU))
  332. {
  333. /* put it and the end of the queue ... XXX */
  334. _starpu_push_task_to_workers(task);
  335. return 0;
  336. }
  337. _STARPU_TRACE_END_PROGRESS(memnode);
  338. /* Get the rank in case it is a parallel task */
  339. if (j->task_size > 1)
  340. {
  341. STARPU_PTHREAD_MUTEX_LOCK(&j->sync_mutex);
  342. rank = j->active_task_alias_count++;
  343. STARPU_PTHREAD_MUTEX_UNLOCK(&j->sync_mutex);
  344. }
  345. else
  346. {
  347. rank = 0;
  348. }
  349. cpu_worker->current_rank = rank;
  350. #ifdef STARPU_OPENMP
  351. /* At this point, j->continuation as been cleared as the task is being
  352. * woken up, thus we use j->discontinuous instead for the check */
  353. const unsigned continuation_wake_up = j->discontinuous;
  354. #else
  355. const unsigned continuation_wake_up = 0;
  356. #endif
  357. if (rank == 0 && !continuation_wake_up)
  358. {
  359. res = _starpu_fetch_task_input(task, j, 1);
  360. STARPU_ASSERT(res == 0);
  361. }
  362. else
  363. {
  364. int ret = _starpu_cpu_driver_execute_task(cpu_worker, task, j);
  365. _STARPU_TRACE_END_PROGRESS(memnode);
  366. return ret;
  367. }
  368. _STARPU_TRACE_END_PROGRESS(memnode);
  369. return 0;
  370. }
  371. int _starpu_cpu_driver_deinit(struct _starpu_worker *cpu_worker)
  372. {
  373. _STARPU_TRACE_WORKER_DEINIT_START;
  374. unsigned memnode = cpu_worker->memory_node;
  375. _starpu_datawizard_handle_all_pending_node_data_requests(memnode);
  376. /* In case there remains some memory that was automatically
  377. * allocated by StarPU, we release it now. Note that data
  378. * coherency is not maintained anymore at that point ! */
  379. _starpu_free_all_automatically_allocated_buffers(memnode);
  380. cpu_worker->worker_is_initialized = 0;
  381. _STARPU_TRACE_WORKER_DEINIT_END(STARPU_CPU_WORKER);
  382. return 0;
  383. }
  384. void *_starpu_cpu_worker(void *arg)
  385. {
  386. struct _starpu_worker *worker = arg;
  387. _starpu_cpu_driver_init(worker);
  388. _STARPU_TRACE_START_PROGRESS(worker->memory_node);
  389. while (_starpu_machine_is_running())
  390. {
  391. _starpu_may_pause();
  392. _starpu_cpu_driver_run_once(worker);
  393. }
  394. _STARPU_TRACE_END_PROGRESS(worker->memory_node);
  395. _starpu_cpu_driver_deinit(worker);
  396. return NULL;
  397. }
  398. int _starpu_cpu_driver_run(struct _starpu_worker *worker)
  399. {
  400. worker->set = NULL;
  401. worker->worker_is_initialized = 0;
  402. _starpu_cpu_worker(worker);
  403. return 0;
  404. }
  405. struct _starpu_driver_ops _starpu_driver_cpu_ops =
  406. {
  407. .init = _starpu_cpu_driver_init,
  408. .run = _starpu_cpu_driver_run,
  409. .run_once = _starpu_cpu_driver_run_once,
  410. .deinit = _starpu_cpu_driver_deinit
  411. };
  412. #endif /* STARPU_USE_CPU */
  413. int _starpu_cpu_copy_interface(starpu_data_handle_t handle, void *src_interface, unsigned src_node, void *dst_interface, unsigned dst_node, struct _starpu_data_request *req)
  414. {
  415. int src_kind = starpu_node_get_kind(src_node);
  416. int dst_kind = starpu_node_get_kind(dst_node);
  417. STARPU_ASSERT(src_kind == STARPU_CPU_RAM && dst_kind == STARPU_CPU_RAM);
  418. int ret = 0;
  419. const struct starpu_data_copy_methods *copy_methods = handle->ops->copy_methods;
  420. if (copy_methods->ram_to_ram)
  421. copy_methods->ram_to_ram(src_interface, src_node, dst_interface, dst_node);
  422. else
  423. copy_methods->any_to_any(src_interface, src_node, dst_interface, dst_node, req ? &req->async_channel : NULL);
  424. return ret;
  425. }
  426. int _starpu_cpu_copy_data(uintptr_t src, size_t src_offset, unsigned src_node, uintptr_t dst, size_t dst_offset, unsigned dst_node, size_t size, struct _starpu_async_channel *async_channel)
  427. {
  428. int src_kind = starpu_node_get_kind(src_node);
  429. int dst_kind = starpu_node_get_kind(dst_node);
  430. STARPU_ASSERT(src_kind == STARPU_CPU_RAM && dst_kind == STARPU_CPU_RAM);
  431. (void) async_channel;
  432. memcpy((void *) (dst + dst_offset), (void *) (src + src_offset), size);
  433. return 0;
  434. }
  435. int _starpu_cpu_is_direct_access_supported(unsigned node, unsigned handling_node)
  436. {
  437. (void) node;
  438. (void) handling_node;
  439. return 1;
  440. }
  441. uintptr_t _starpu_cpu_malloc_on_node(unsigned dst_node, size_t size, int flags)
  442. {
  443. uintptr_t addr = 0;
  444. _starpu_malloc_flags_on_node(dst_node, (void**) &addr, size,
  445. #if defined(STARPU_USE_CUDA) && !defined(STARPU_HAVE_CUDA_MEMCPY_PEER) && !defined(STARPU_SIMGRID)
  446. /* without memcpy_peer, we can not
  447. * allocated pinned memory, since it
  448. * requires waiting for a task, and we
  449. * may be called with a spinlock held
  450. */
  451. flags & ~STARPU_MALLOC_PINNED
  452. #else
  453. flags
  454. #endif
  455. );
  456. return addr;
  457. }
  458. void _starpu_cpu_free_on_node(unsigned dst_node, uintptr_t addr, size_t size, int flags)
  459. {
  460. _starpu_free_flags_on_node(dst_node, (void*)addr, size,
  461. #if defined(STARPU_USE_CUDA) && !defined(STARPU_HAVE_CUDA_MEMCPY_PEER) && !defined(STARPU_SIMGRID)
  462. flags & ~STARPU_MALLOC_PINNED
  463. #else
  464. flags
  465. #endif
  466. );
  467. }
  468. struct _starpu_node_ops _starpu_driver_cpu_node_ops =
  469. {
  470. .copy_interface_to[STARPU_UNUSED] = NULL,
  471. .copy_interface_to[STARPU_CPU_RAM] = _starpu_cpu_copy_interface,
  472. #ifdef STARPU_USE_CUDA
  473. .copy_interface_to[STARPU_CUDA_RAM] = _starpu_cuda_copy_interface_from_cpu_to_cuda,
  474. #else
  475. .copy_interface_to[STARPU_CUDA_RAM] = NULL,
  476. #endif
  477. #ifdef STARPU_USE_OPENCL
  478. .copy_interface_to[STARPU_OPENCL_RAM] = _starpu_opencl_copy_interface_from_cpu_to_opencl,
  479. #else
  480. .copy_interface_to[STARPU_OPENCL_RAM] = NULL,
  481. #endif
  482. .copy_interface_to[STARPU_DISK_RAM] = _starpu_disk_copy_interface_from_cpu_to_disk,
  483. #ifdef STARPU_USE_MPI_MASTER_SLAVE
  484. .copy_interface_to[STARPU_MPI_MS_RAM] = _starpu_mpi_copy_interface_from_cpu_to_mpi,
  485. #else
  486. .copy_interface_to[STARPU_MPI_MS_RAM] = NULL,
  487. #endif
  488. .copy_data_to[STARPU_UNUSED] = NULL,
  489. .copy_data_to[STARPU_CPU_RAM] = _starpu_cpu_copy_data,
  490. #ifdef STARPU_USE_CUDA
  491. .copy_data_to[STARPU_CUDA_RAM] = _starpu_cuda_copy_data_from_cpu_to_cuda,
  492. #else
  493. .copy_data_to[STARPU_CUDA_RAM] = NULL,
  494. #endif
  495. #ifdef STARPU_USE_OPENCL
  496. .copy_data_to[STARPU_OPENCL_RAM] = _starpu_opencl_copy_data_from_cpu_to_opencl,
  497. #else
  498. .copy_data_to[STARPU_OPENCL_RAM] = NULL,
  499. #endif
  500. .copy_data_to[STARPU_DISK_RAM] = _starpu_disk_copy_data_from_cpu_to_disk,
  501. #ifdef STARPU_USE_MPI_MASTER_SLAVE
  502. .copy_data_to[STARPU_MPI_MS_RAM] = _starpu_mpi_copy_data_from_cpu_to_mpi,
  503. #else
  504. .copy_data_to[STARPU_MPI_MS_RAM] = NULL,
  505. #endif
  506. .copy2d_data_to[STARPU_UNUSED] = NULL,
  507. .copy2d_data_to[STARPU_CPU_RAM] = NULL,
  508. #ifdef STARPU_USE_CUDA
  509. .copy2d_data_to[STARPU_CUDA_RAM] = _starpu_cuda_copy2d_data_from_cpu_to_cuda,
  510. #else
  511. .copy2d_data_to[STARPU_CUDA_RAM] = NULL,
  512. #endif
  513. .copy2d_data_to[STARPU_OPENCL_RAM] = NULL,
  514. .copy2d_data_to[STARPU_DISK_RAM] = NULL,
  515. .copy2d_data_to[STARPU_MPI_MS_RAM] = NULL,
  516. .copy3d_data_to[STARPU_UNUSED] = NULL,
  517. .copy3d_data_to[STARPU_CPU_RAM] = NULL,
  518. #if 0
  519. #ifdef STARPU_USE_CUDA
  520. .copy3d_data_to[STARPU_CUDA_RAM] = _starpu_cuda_copy3d_data_from_cpu_to_cuda,
  521. #else
  522. .copy3d_data_to[STARPU_CUDA_RAM] = NULL,
  523. #endif
  524. #else
  525. .copy3d_data_to[STARPU_CUDA_RAM] = NULL,
  526. #endif
  527. .copy3d_data_to[STARPU_OPENCL_RAM] = NULL,
  528. .copy3d_data_to[STARPU_DISK_RAM] = NULL,
  529. .copy3d_data_to[STARPU_MPI_MS_RAM] = NULL,
  530. .wait_request_completion = NULL,
  531. .test_request_completion = NULL,
  532. .is_direct_access_supported = _starpu_cpu_is_direct_access_supported,
  533. .malloc_on_node = _starpu_cpu_malloc_on_node,
  534. .free_on_node = _starpu_cpu_free_on_node,
  535. .name = "cpu driver"
  536. };