starpu_scheduler.h 16 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465
  1. /* StarPU --- Runtime system for heterogeneous multicore architectures.
  2. *
  3. * Copyright (C) 2011-2013,2015-2017 Inria
  4. * Copyright (C) 2010-2018 Université de Bordeaux
  5. * Copyright (C) 2011-2013,2015,2017,2019 CNRS
  6. * Copyright (C) 2013 Thibaut Lambert
  7. * Copyright (C) 2011 Télécom-SudParis
  8. * Copyright (C) 2016 Uppsala University
  9. *
  10. * StarPU is free software; you can redistribute it and/or modify
  11. * it under the terms of the GNU Lesser General Public License as published by
  12. * the Free Software Foundation; either version 2.1 of the License, or (at
  13. * your option) any later version.
  14. *
  15. * StarPU is distributed in the hope that it will be useful, but
  16. * WITHOUT ANY WARRANTY; without even the implied warranty of
  17. * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE.
  18. *
  19. * See the GNU Lesser General Public License in COPYING.LGPL for more details.
  20. */
  21. #ifndef __STARPU_SCHEDULER_H__
  22. #define __STARPU_SCHEDULER_H__
  23. #include <starpu.h>
  24. #ifdef __cplusplus
  25. extern "C"
  26. {
  27. #endif
  28. /**
  29. @defgroup API_Scheduling_Policy Scheduling Policy
  30. @brief TODO. While StarPU comes with a variety of scheduling
  31. policies (see \ref TaskSchedulingPolicy), it may sometimes be
  32. desirable to implement custom policies to address specific
  33. problems. The API described below allows users to write their own
  34. scheduling policy.
  35. @{
  36. */
  37. struct starpu_task;
  38. /**
  39. Contain all the methods that implement a scheduling policy. An
  40. application may specify which scheduling strategy in the field
  41. starpu_conf::sched_policy passed to the function starpu_init().
  42. For each task going through the scheduler, the following methods
  43. get called in the given order:
  44. <ul>
  45. <li>starpu_sched_policy::submit_hook when the task is
  46. submitted</li>
  47. <li>starpu_sched_policy::push_task when the task becomes ready. The
  48. scheduler is here <b>given</b> the task</li>
  49. <li>starpu_sched_policy::pop_task when the worker is idle. The
  50. scheduler here <b>gives</b> back the task to the core. It must not
  51. access this task any more</li>
  52. <li>starpu_sched_policy::pre_exec_hook right before the worker
  53. actually starts the task computation (after transferring any
  54. missing data).</li>
  55. <li>starpu_sched_policy::post_exec_hook right after the worker
  56. actually completes the task computation.</li>
  57. </ul>
  58. For each task not going through the scheduler (because
  59. starpu_task::execute_on_a_specific_worker was set), these get
  60. called:
  61. <ul>
  62. <li>starpu_sched_policy::submit_hook when the task is
  63. submitted</li>
  64. <li>starpu_sched_policy::push_task_notify when the task becomes
  65. ready. This is just a notification, the scheduler does not have to
  66. do anything about the task.</li>
  67. <li>starpu_sched_policy::pre_exec_hook right before the worker
  68. actually starts the task computation (after transferring any
  69. missing data).</li>
  70. <li>starpu_sched_policy::post_exec_hook right after the worker
  71. actually completes the task computation.</li>
  72. </ul>
  73. */
  74. struct starpu_sched_policy
  75. {
  76. /**
  77. Initialize the scheduling policy, called before any other
  78. method.
  79. */
  80. void (*init_sched)(unsigned sched_ctx_id);
  81. /**
  82. Cleanup the scheduling policy
  83. */
  84. void (*deinit_sched)(unsigned sched_ctx_id);
  85. /**
  86. Insert a task into the scheduler, called when the task
  87. becomes ready for execution. This must call
  88. starpu_push_task_end() once it has effectively pushed the
  89. task to a queue (to note the time when this was done in the
  90. task), but before releasing mutexes (so that the task
  91. hasn't been already taken by a worker).
  92. */
  93. int (*push_task)(struct starpu_task *);
  94. double (*simulate_push_task)(struct starpu_task *);
  95. /**
  96. Notify the scheduler that a task was pushed on a given
  97. worker. This method is called when a task that was
  98. explicitly assigned to a worker becomes ready and is about
  99. to be executed by the worker. This method therefore permits
  100. to keep the state of the scheduler coherent even when
  101. StarPU bypasses the scheduling strategy.
  102. */
  103. void (*push_task_notify)(struct starpu_task *, int workerid, int perf_workerid, unsigned sched_ctx_id);
  104. /**
  105. Get a task from the scheduler.
  106. If this method returns NULL, the worker will start
  107. sleeping. If later on some task are pushed for this worker,
  108. starpu_wake_worker() must be called to wake the worker so
  109. it can call the pop_task() method again.
  110. The mutex associated to the worker is already taken when
  111. this method is called. This method may release it (e.g. for
  112. scalability reasons when doing work stealing), but it must
  113. acquire it again before taking the decision whether to
  114. return a task or NULL, so the atomicity of deciding to
  115. return NULL and making the worker actually sleep is
  116. preserved. Otherwise in simgrid or blocking driver mode the
  117. worker might start sleeping while a task has just been
  118. pushed for it.
  119. If this method is defined as <c>NULL</c>, the worker will
  120. only execute tasks from its local queue. In this case, the
  121. push_task method should use the starpu_push_local_task
  122. method to assign tasks to the different workers.
  123. */
  124. struct starpu_task *(*pop_task)(unsigned sched_ctx_id);
  125. /**
  126. Remove all available tasks from the scheduler (tasks are
  127. chained by the means of the field starpu_task::prev and
  128. starpu_task::next). The mutex associated to the worker is
  129. already taken when this method is called. This is currently
  130. not used and can be discarded.
  131. */
  132. struct starpu_task *(*pop_every_task)(unsigned sched_ctx_id);
  133. /**
  134. Optional field. This method is called when a task is
  135. submitted.
  136. */
  137. void (*submit_hook)(struct starpu_task *task);
  138. /**
  139. Optional field. This method is called every time a task is
  140. starting.
  141. */
  142. void (*pre_exec_hook)(struct starpu_task *, unsigned sched_ctx_id);
  143. /**
  144. Optional field. This method is called every time a task has
  145. been executed.
  146. */
  147. void (*post_exec_hook)(struct starpu_task *, unsigned sched_ctx_id);
  148. /**
  149. Optional field. This method is called when it is a good
  150. time to start scheduling tasks. This is notably called when
  151. the application calls starpu_task_wait_for_all() or
  152. starpu_do_schedule() explicitly.
  153. */
  154. void (*do_schedule)(unsigned sched_ctx_id);
  155. /**
  156. Initialize scheduling structures corresponding to each
  157. worker used by the policy.
  158. */
  159. void (*add_workers)(unsigned sched_ctx_id, int *workerids, unsigned nworkers);
  160. /**
  161. Deinitialize scheduling structures corresponding to each
  162. worker used by the policy.
  163. */
  164. void (*remove_workers)(unsigned sched_ctx_id, int *workerids, unsigned nworkers);
  165. /**
  166. Optional field. Name of the policy.
  167. */
  168. const char *policy_name;
  169. /**
  170. Optional field. Human readable description of the policy.
  171. */
  172. const char *policy_description;
  173. enum starpu_worker_collection_type worker_type;
  174. };
  175. /**
  176. Return an <c>NULL</c>-terminated array of all the predefined
  177. scheduling policies.
  178. */
  179. struct starpu_sched_policy **starpu_sched_get_predefined_policies();
  180. /**
  181. When there is no available task for a worker, StarPU blocks this
  182. worker on a condition variable. This function specifies which
  183. condition variable (and the associated mutex) should be used to
  184. block (and to wake up) a worker. Note that multiple workers may use
  185. the same condition variable. For instance, in the case of a
  186. scheduling strategy with a single task queue, the same condition
  187. variable would be used to block and wake up all workers.
  188. */
  189. void starpu_worker_get_sched_condition(int workerid, starpu_pthread_mutex_t **sched_mutex, starpu_pthread_cond_t **sched_cond);
  190. unsigned long starpu_task_get_job_id(struct starpu_task *task);
  191. /**
  192. TODO: check if this is correct
  193. Return the current minimum priority level supported by the scheduling
  194. policy
  195. */
  196. int starpu_sched_get_min_priority(void);
  197. /**
  198. TODO: check if this is correct
  199. Return the current maximum priority level supported by the
  200. scheduling policy
  201. */
  202. int starpu_sched_get_max_priority(void);
  203. /**
  204. TODO: check if this is correct
  205. Define the minimum task priority level supported by the scheduling
  206. policy. The default minimum priority level is the same as the
  207. default priority level which is 0 by convention. The application
  208. may access that value by calling the function
  209. starpu_sched_get_min_priority(). This function should only be
  210. called from the initialization method of the scheduling policy, and
  211. should not be used directly from the application.
  212. */
  213. int starpu_sched_set_min_priority(int min_prio);
  214. /**
  215. TODO: check if this is correct
  216. Define the maximum priority level supported by the scheduling
  217. policy. The default maximum priority level is 1. The application
  218. may access that value by calling the function
  219. starpu_sched_get_max_priority(). This function should only be
  220. called from the initialization method of the scheduling policy, and
  221. should not be used directly from the application.
  222. */
  223. int starpu_sched_set_max_priority(int max_prio);
  224. /**
  225. Check if the worker specified by workerid can execute the codelet.
  226. Schedulers need to call it before assigning a task to a worker,
  227. otherwise the task may fail to execute.
  228. */
  229. int starpu_worker_can_execute_task(unsigned workerid, struct starpu_task *task, unsigned nimpl);
  230. /**
  231. Check if the worker specified by workerid can execute the codelet
  232. and return which implementation numbers can be used.
  233. Schedulers need to call it before assigning a task to a worker,
  234. otherwise the task may fail to execute.
  235. This should be preferred rather than calling
  236. starpu_worker_can_execute_task() for each and every implementation.
  237. It can also be used with <c>impl_mask == NULL</c> to check for at
  238. least one implementation without determining which.
  239. */
  240. int starpu_worker_can_execute_task_impl(unsigned workerid, struct starpu_task *task, unsigned *impl_mask);
  241. /**
  242. Check if the worker specified by workerid can execute the codelet
  243. and return the first implementation which can be used.
  244. Schedulers need to call it before assigning a task to a worker,
  245. otherwise the task may fail to execute. This should be preferred
  246. rather than calling starpu_worker_can_execute_task() for
  247. each and every implementation. It can also be used with
  248. <c>impl_mask == NULL</c> to check for at least one implementation
  249. without determining which.
  250. */
  251. int starpu_worker_can_execute_task_first_impl(unsigned workerid, struct starpu_task *task, unsigned *nimpl);
  252. /**
  253. The scheduling policy may put tasks directly into a worker’s local
  254. queue so that it is not always necessary to create its own queue
  255. when the local queue is sufficient. If \p back is not 0, \p task is
  256. put at the back of the queue where the worker will pop tasks first.
  257. Setting \p back to 0 therefore ensures a FIFO ordering.
  258. */
  259. int starpu_push_local_task(int workerid, struct starpu_task *task, int back);
  260. /**
  261. Must be called by a scheduler to notify that the given
  262. task has just been pushed.
  263. */
  264. int starpu_push_task_end(struct starpu_task *task);
  265. /**
  266. Whether \ref STARPU_PREFETCH was set
  267. */
  268. int starpu_get_prefetch_flag(void);
  269. /**
  270. Prefetch data for a given p task on a given p node with a given
  271. priority
  272. */
  273. int starpu_prefetch_task_input_on_node_prio(struct starpu_task *task, unsigned node, int prio);
  274. /**
  275. Prefetch data for a given p task on a given p node
  276. */
  277. int starpu_prefetch_task_input_on_node(struct starpu_task *task, unsigned node);
  278. /**
  279. Prefetch data for a given p task on a given p node when the bus is
  280. idle with a given priority
  281. */
  282. int starpu_idle_prefetch_task_input_on_node_prio(struct starpu_task *task, unsigned node, int prio);
  283. /**
  284. Prefetch data for a given p task on a given p node when the bus is
  285. idle
  286. */
  287. int starpu_idle_prefetch_task_input_on_node(struct starpu_task *task, unsigned node);
  288. /**
  289. Prefetch data for a given p task on a given p worker with a given
  290. priority
  291. */
  292. int starpu_prefetch_task_input_for_prio(struct starpu_task *task, unsigned worker, int prio);
  293. /**
  294. Prefetch data for a given p task on a given p worker
  295. */
  296. int starpu_prefetch_task_input_for(struct starpu_task *task, unsigned worker);
  297. /**
  298. Prefetch data for a given p task on a given p worker when the bus
  299. is idle with a given priority
  300. */
  301. int starpu_idle_prefetch_task_input_for_prio(struct starpu_task *task, unsigned worker, int prio);
  302. /**
  303. Prefetch data for a given p task on a given p worker when the bus
  304. is idle
  305. */
  306. int starpu_idle_prefetch_task_input_for(struct starpu_task *task, unsigned worker);
  307. /**
  308. Return the footprint for a given task, taking into account
  309. user-provided perfmodel footprint or size_base functions.
  310. */
  311. uint32_t starpu_task_footprint(struct starpu_perfmodel *model, struct starpu_task *task, struct starpu_perfmodel_arch *arch, unsigned nimpl);
  312. /**
  313. Return the raw footprint for the data of a given task (without
  314. taking into account user-provided functions).
  315. */
  316. uint32_t starpu_task_data_footprint(struct starpu_task *task);
  317. /**
  318. Return expected task duration in micro-seconds.
  319. */
  320. double starpu_task_expected_length(struct starpu_task *task, struct starpu_perfmodel_arch *arch, unsigned nimpl);
  321. /**
  322. Return an estimated speedup factor relative to CPU speed
  323. */
  324. double starpu_worker_get_relative_speedup(struct starpu_perfmodel_arch *perf_arch);
  325. /**
  326. Return expected data transfer time in micro-seconds for the given \p
  327. memory_node. Prefer using starpu_task_expected_data_transfer_time_for() which is
  328. more precise.
  329. */
  330. double starpu_task_expected_data_transfer_time(unsigned memory_node, struct starpu_task *task);
  331. /**
  332. Return expected data transfer time in micro-seconds for the given
  333. \p worker.
  334. */
  335. double starpu_task_expected_data_transfer_time_for(struct starpu_task *task, unsigned worker);
  336. /**
  337. Predict the transfer time (in micro-seconds) to move \p handle to a
  338. memory node
  339. */
  340. double starpu_data_expected_transfer_time(starpu_data_handle_t handle, unsigned memory_node, enum starpu_data_access_mode mode);
  341. /**
  342. Return expected energy consumption in J
  343. */
  344. double starpu_task_expected_energy(struct starpu_task *task, struct starpu_perfmodel_arch *arch, unsigned nimpl);
  345. /**
  346. Return expected conversion time in ms (multiformat interface only)
  347. */
  348. double starpu_task_expected_conversion_time(struct starpu_task *task, struct starpu_perfmodel_arch *arch, unsigned nimpl);
  349. typedef void (*starpu_notify_ready_soon_func)(void *data, struct starpu_task *task, double delay);
  350. /**
  351. Register a callback to be called when it is determined when a task
  352. will be ready an estimated amount of time from now, because its
  353. last dependency has just started and we know how long it will take.
  354. */
  355. void starpu_task_notify_ready_soon_register(starpu_notify_ready_soon_func f, void *data);
  356. /**
  357. The scheduling policies indicates if the worker may pop tasks from
  358. the list of other workers or if there is a central list with task
  359. for all the workers
  360. */
  361. void starpu_sched_ctx_worker_shares_tasks_lists(int workerid, int sched_ctx_id);
  362. void starpu_sched_task_break(struct starpu_task *task);
  363. /**
  364. @name Worker operations
  365. @{
  366. */
  367. /**
  368. Wake up \p workerid while temporarily entering the current worker
  369. relax state if needed during the waiting process. Return 1 if \p
  370. workerid has been woken up or its state_keep_awake flag has been
  371. set to \c 1, and \c 0 otherwise (if \p workerid was not in the
  372. STATE_SLEEPING or in the STATE_SCHEDULING).
  373. */
  374. int starpu_wake_worker_relax(int workerid);
  375. /**
  376. Must be called to wake up a worker that is sleeping on the cond.
  377. Return 0 whenever the worker is not in a sleeping state or has the
  378. state_keep_awake flag on.
  379. */
  380. int starpu_wake_worker_no_relax(int workerid);
  381. /**
  382. Version of starpu_wake_worker_no_relax() which assumes that the
  383. sched mutex is locked
  384. */
  385. int starpu_wake_worker_locked(int workerid);
  386. /**
  387. Light version of starpu_wake_worker_relax() which, when possible,
  388. speculatively set keep_awake on the target worker without waiting
  389. for the worker to enter the relax state.
  390. */
  391. int starpu_wake_worker_relax_light(int workerid);
  392. /** @} */
  393. /** @} */
  394. #ifdef __cplusplus
  395. }
  396. #endif
  397. #endif /* __STARPU_SCHEDULER_H__ */