driver_cpu.c 18 KB

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