starpu_sched_component.h 28 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593594595596597598599600601602603604605606607608609610611612613614615616617618619620621622623624625626627628629630631632633634635636637638639640641642643644645646647648649650651652653654655656657658659660661662663664665666667668669670671672673674675676677678679680681682683684685686687688689690691692693694695696697698699700701702703704705706707708709710711712713714715716717718719720721722723724725726727728729730731732733734735736737738739740741742743744745746747748749750751752753754755
  1. /* StarPU --- Runtime system for heterogeneous multicore architectures.
  2. *
  3. * Copyright (C) 2017 Arthur Chevalier
  4. * Copyright (C) 2013,2014,2017 Inria
  5. * Copyright (C) 2014,2015,2017,2019 CNRS
  6. * Copyright (C) 2014-2019 Université de Bordeaux
  7. * Copyright (C) 2013 Simon Archipoff
  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. #ifndef __STARPU_SCHED_COMPONENT_H__
  21. #define __STARPU_SCHED_COMPONENT_H__
  22. #include <starpu.h>
  23. #ifdef STARPU_HAVE_HWLOC
  24. #include <hwloc.h>
  25. #endif
  26. #ifdef __cplusplus
  27. extern "C"
  28. {
  29. #endif
  30. /**
  31. @defgroup API_Modularized_Scheduler Modularized Scheduler Interface
  32. @{
  33. */
  34. /**
  35. flags for starpu_sched_component::properties
  36. */
  37. enum starpu_sched_component_properties
  38. {
  39. /** indicate that all workers have the same starpu_worker_archtype */
  40. STARPU_SCHED_COMPONENT_HOMOGENEOUS = (1<<0),
  41. /** indicate that all workers have the same memory component */
  42. STARPU_SCHED_COMPONENT_SINGLE_MEMORY_NODE = (1<<1)
  43. };
  44. /**
  45. indicate if component is homogeneous
  46. */
  47. #define STARPU_SCHED_COMPONENT_IS_HOMOGENEOUS(component) ((component)->properties & STARPU_SCHED_COMPONENT_HOMOGENEOUS)
  48. /**
  49. indicate if all workers have the same memory component
  50. */
  51. #define STARPU_SCHED_COMPONENT_IS_SINGLE_MEMORY_NODE(component) ((component)->properties & STARPU_SCHED_COMPONENT_SINGLE_MEMORY_NODE)
  52. /**
  53. Structure for a scheduler module. A scheduler is a
  54. tree-like structure of them, some parts of scheduler can be shared by
  55. several contexes to perform some local optimisations, so, for all
  56. components, a list of parent is defined by \c sched_ctx_id. They
  57. embed there specialised method in a pseudo object-style, so calls are
  58. like <c>component->push_task(component,task)</c>
  59. */
  60. struct starpu_sched_component
  61. {
  62. /** The tree containing the component*/
  63. struct starpu_sched_tree *tree;
  64. /** set of underlying workers */
  65. struct starpu_bitmap *workers;
  66. /**
  67. subset of starpu_sched_component::workers that is currently available in the context
  68. The push method should take this value into account, it is set with:
  69. component->workers UNION tree->workers UNION
  70. component->child[i]->workers_in_ctx iff exist x such as component->children[i]->parents[x] == component
  71. */
  72. struct starpu_bitmap *workers_in_ctx;
  73. /** private data */
  74. void *data;
  75. char *name;
  76. /** number of compoments's children */
  77. unsigned nchildren;
  78. /** vector of component's children */
  79. struct starpu_sched_component **children;
  80. /** number of component's parents */
  81. unsigned nparents;
  82. /** vector of component's parents */
  83. struct starpu_sched_component **parents;
  84. /** add a child to component */
  85. void (*add_child)(struct starpu_sched_component *component, struct starpu_sched_component *child);
  86. /** remove a child from component */
  87. void (*remove_child)(struct starpu_sched_component *component, struct starpu_sched_component *child);
  88. void (*add_parent)(struct starpu_sched_component *component, struct starpu_sched_component *parent);
  89. void (*remove_parent)(struct starpu_sched_component *component, struct starpu_sched_component *parent);
  90. /**
  91. push a task in the scheduler module. this function is called to
  92. push a task on component subtree, this can either perform a
  93. recursive call on a child or store the task in the component,
  94. then it will be returned by a further pull_task call.
  95. the caller must ensure that component is able to execute task.
  96. This method must either return 0 if it the task was properly stored or
  97. passed over to a child component, or return a value different from 0 if the
  98. task could not be consumed (e.g. the queue is full).
  99. */
  100. int (*push_task)(struct starpu_sched_component *, struct starpu_task *);
  101. /**
  102. pop a task from the scheduler module. this function is called by workers to get a task from their
  103. parents. this function should first return a locally stored task
  104. or perform a recursive call on the parents.
  105. the task returned by this function should be executable by the caller
  106. */
  107. struct starpu_task *(*pull_task)(struct starpu_sched_component *from, struct starpu_sched_component *to);
  108. /**
  109. This function is called by a component which implements a queue,
  110. allowing it to signify to its parents that an empty slot is
  111. available in its queue. This should return 1 if some tasks could be pushed
  112. The basic implementation of this function
  113. is a recursive call to its parents, the user has to specify a
  114. personally-made function to catch those calls.
  115. */
  116. int (*can_push)(struct starpu_sched_component *from, struct starpu_sched_component *to);
  117. /**
  118. This function allow a component to wake up a worker. It is
  119. currently called by component which implements a queue, to
  120. signify to its children that a task have been pushed in its local
  121. queue, and is available to be popped by a worker, for example.
  122. This should return 1 if some some container or worker could (or will) pull
  123. some tasks.
  124. The basic implementation of this function is a recursive call to
  125. its children, until at least one worker have been woken up.
  126. */
  127. int (*can_pull)(struct starpu_sched_component *component);
  128. int (*notify)(struct starpu_sched_component* component, int message_ID, void* arg);
  129. /**
  130. heuristic to compute load of scheduler module. Basically the number of tasks divided by the sum
  131. of relatives speedup of workers available in context.
  132. estimated_load(component) = sum(estimated_load(component_children)) + nb_local_tasks / average(relative_speedup(underlying_worker))
  133. */
  134. double (*estimated_load)(struct starpu_sched_component *component);
  135. /**
  136. return the time when a worker will enter in starvation. This function is relevant only if the task->predicted
  137. member has been set.
  138. */
  139. double (*estimated_end)(struct starpu_sched_component *component);
  140. /**
  141. called by starpu_sched_component_destroy. Should free data allocated during creation
  142. */
  143. void (*deinit_data)(struct starpu_sched_component *component);
  144. /**
  145. this function is called for each component when workers are added or removed from a context
  146. */
  147. void (*notify_change_workers)(struct starpu_sched_component *component);
  148. int properties;
  149. #ifdef STARPU_HAVE_HWLOC
  150. /**
  151. the hwloc object associated to scheduler module. points to the
  152. part of topology that is binded to this component, eg: a numa
  153. node for a ws component that would balance load between
  154. underlying sockets
  155. */
  156. hwloc_obj_t obj;
  157. #else
  158. void *obj;
  159. #endif
  160. };
  161. /**
  162. The actual scheduler
  163. */
  164. struct starpu_sched_tree
  165. {
  166. /**
  167. entry module of the scheduler
  168. */
  169. struct starpu_sched_component *root;
  170. /**
  171. set of workers available in this context, this value is used to mask workers in modules
  172. */
  173. struct starpu_bitmap *workers;
  174. /**
  175. context id of the scheduler
  176. */
  177. unsigned sched_ctx_id;
  178. /**
  179. lock used to protect the scheduler, it is taken in read mode pushing a task and in write mode for adding or
  180. removing workers
  181. */
  182. starpu_pthread_mutex_t lock;
  183. };
  184. void starpu_initialize_prio_center_policy(unsigned sched_ctx_id);
  185. /**
  186. @name Scheduling Tree API
  187. @{
  188. */
  189. /**
  190. create a empty initialized starpu_sched_tree
  191. */
  192. struct starpu_sched_tree *starpu_sched_tree_create(unsigned sched_ctx_id) STARPU_ATTRIBUTE_MALLOC;
  193. /**
  194. destroy tree and free all non shared component in it.
  195. */
  196. void starpu_sched_tree_destroy(struct starpu_sched_tree *tree);
  197. struct starpu_sched_tree *starpu_sched_tree_get(unsigned sched_ctx_id);
  198. /**
  199. recursively set all starpu_sched_component::workers, do not take into account shared parts (except workers).
  200. */
  201. void starpu_sched_tree_update_workers(struct starpu_sched_tree *t);
  202. /**
  203. recursively set all starpu_sched_component::workers_in_ctx, do not take into account shared parts (except workers)
  204. */
  205. void starpu_sched_tree_update_workers_in_ctx(struct starpu_sched_tree *t);
  206. /**
  207. compatibility with starpu_sched_policy interface
  208. */
  209. int starpu_sched_tree_push_task(struct starpu_task *task);
  210. /**
  211. compatibility with starpu_sched_policy interface
  212. */
  213. struct starpu_task *starpu_sched_tree_pop_task(unsigned sched_ctx);
  214. /**
  215. Push a task to a component. This is a helper for <c>component->push_task(component, task)</c> plus tracing.
  216. */
  217. int starpu_sched_component_push_task(struct starpu_sched_component *from, struct starpu_sched_component *to, struct starpu_task *task);
  218. /**
  219. Pull a task from a component. This is a helper for <c>component->pull_task(component)</c> plus tracing.
  220. */
  221. struct starpu_task *starpu_sched_component_pull_task(struct starpu_sched_component *from, struct starpu_sched_component *to);
  222. struct starpu_task* starpu_sched_component_pump_to(struct starpu_sched_component *component, struct starpu_sched_component *to, int* success);
  223. struct starpu_task* starpu_sched_component_pump_downstream(struct starpu_sched_component *component, int* success);
  224. int starpu_sched_component_send_can_push_to_parents(struct starpu_sched_component * component);
  225. /**
  226. compatibility with starpu_sched_policy interface
  227. */
  228. void starpu_sched_tree_add_workers(unsigned sched_ctx_id, int *workerids, unsigned nworkers);
  229. /**
  230. compatibility with starpu_sched_policy interface
  231. */
  232. void starpu_sched_tree_remove_workers(unsigned sched_ctx_id, int *workerids, unsigned nworkers);
  233. /**
  234. Attach component \p child to parent \p parent. Some component may accept only one child, others accept several (e.g. MCT)
  235. */
  236. void starpu_sched_component_connect(struct starpu_sched_component *parent, struct starpu_sched_component *child);
  237. /** @} */
  238. /**
  239. @name Generic Scheduling Component API
  240. @{
  241. */
  242. typedef struct starpu_sched_component * (*starpu_sched_component_create_t)(struct starpu_sched_tree *tree, void *data);
  243. /**
  244. allocate and initialize component field with defaults values :
  245. .pop_task make recursive call on father
  246. .estimated_load compute relative speedup and tasks in sub tree
  247. .estimated_end return the minimum of recursive call on children
  248. .add_child is starpu_sched_component_add_child
  249. .remove_child is starpu_sched_component_remove_child
  250. .notify_change_workers does nothing
  251. .deinit_data does nothing
  252. */
  253. struct starpu_sched_component *starpu_sched_component_create(struct starpu_sched_tree *tree, const char *name) STARPU_ATTRIBUTE_MALLOC;
  254. /**
  255. free data allocated by starpu_sched_component_create and call component->deinit_data(component)
  256. set to <c>NULL</c> the member starpu_sched_component::fathers[sched_ctx_id] of all child if its equal to \p component
  257. */
  258. void starpu_sched_component_destroy(struct starpu_sched_component *component);
  259. /**
  260. recursively destroy non shared parts of a \p component 's tree
  261. */
  262. void starpu_sched_component_destroy_rec(struct starpu_sched_component *component);
  263. void starpu_sched_component_add_child(struct starpu_sched_component* component, struct starpu_sched_component * child);
  264. /**
  265. return true iff \p component can execute \p task, this function take into account the workers available in the scheduling context
  266. */
  267. int starpu_sched_component_can_execute_task(struct starpu_sched_component *component, struct starpu_task *task);
  268. /**
  269. return a non <c>NULL</c> value if \p component can execute \p task.
  270. write the execution prediction length for the best implementation of the best worker available and write this at \p length address.
  271. this result is more relevant if starpu_sched_component::is_homogeneous is non <c>NULL</c>.
  272. if a worker need to be calibrated for an implementation, nan is set to \p length.
  273. */
  274. int STARPU_WARN_UNUSED_RESULT starpu_sched_component_execute_preds(struct starpu_sched_component *component, struct starpu_task *task, double *length);
  275. /**
  276. return the average time to transfer \p task data to underlying \p component workers.
  277. */
  278. double starpu_sched_component_transfer_length(struct starpu_sched_component *component, struct starpu_task *task);
  279. void starpu_sched_component_prefetch_on_node(struct starpu_sched_component *component, struct starpu_task *task);
  280. /** @} */
  281. /**
  282. @name Worker Component API
  283. @{
  284. */
  285. /**
  286. return the struct starpu_sched_component corresponding to \p workerid. Undefined if \p workerid is not a valid workerid
  287. */
  288. struct starpu_sched_component *starpu_sched_component_worker_get(unsigned sched_ctx, int workerid);
  289. struct starpu_sched_component *starpu_sched_component_worker_new(unsigned sched_ctx, int workerid);
  290. /**
  291. Create a combined worker that pushes tasks in parallel to workers \p workers (size \p nworkers).
  292. */
  293. struct starpu_sched_component *starpu_sched_component_parallel_worker_create(struct starpu_sched_tree *tree, unsigned nworkers, unsigned *workers);
  294. /**
  295. return the workerid of \p worker_component, undefined if starpu_sched_component_is_worker(worker_component) == 0
  296. */
  297. int starpu_sched_component_worker_get_workerid(struct starpu_sched_component *worker_component);
  298. /**
  299. return true iff \p component is a worker component
  300. */
  301. int starpu_sched_component_is_worker(struct starpu_sched_component *component);
  302. /**
  303. return true iff \p component is a simple worker component
  304. */
  305. int starpu_sched_component_is_simple_worker(struct starpu_sched_component *component);
  306. /**
  307. return true iff \p component is a combined worker component
  308. */
  309. int starpu_sched_component_is_combined_worker(struct starpu_sched_component *component);
  310. /**
  311. compatibility with starpu_sched_policy interface
  312. update predictions for workers
  313. */
  314. void starpu_sched_component_worker_pre_exec_hook(struct starpu_task *task, unsigned sched_ctx_id);
  315. /**
  316. compatibility with starpu_sched_policy interface
  317. */
  318. void starpu_sched_component_worker_post_exec_hook(struct starpu_task *task, unsigned sched_ctx_id);
  319. /** @} */
  320. /**
  321. @name Flow-control Fifo Component API
  322. @{
  323. */
  324. /**
  325. default function for the can_push component method, just call can_push of parents until one of them returns non-zero
  326. */
  327. int starpu_sched_component_can_push(struct starpu_sched_component * component, struct starpu_sched_component * to);
  328. /**
  329. default function for the can_pull component method, just call can_pull of children until one of them returns non-zero
  330. */
  331. int starpu_sched_component_can_pull(struct starpu_sched_component * component);
  332. /**
  333. function for the can_pull component method, call can_pull of all children
  334. */
  335. int starpu_sched_component_can_pull_all(struct starpu_sched_component * component);
  336. /**
  337. default function for the estimated_load component method, just sum up the loads
  338. of the children of the component.
  339. */
  340. double starpu_sched_component_estimated_load(struct starpu_sched_component * component);
  341. /**
  342. function that can be used for the estimated_end component method, compute the minimum completion time of the children.
  343. */
  344. double starpu_sched_component_estimated_end_min(struct starpu_sched_component * component);
  345. /**
  346. function that can be used for the estimated_end component method, compute
  347. the minimum completion time of the children, and add to it an estimation of how
  348. existing queued work, plus the exp_len work, can be completed. This is typically
  349. used instead of starpu_sched_component_estimated_end_min when the component
  350. contains a queue of tasks, which thus needs to be added to the estimations.
  351. */
  352. double starpu_sched_component_estimated_end_min_add(struct starpu_sched_component * component, double exp_len);
  353. /**
  354. default function for the estimated_end component method, compute the average completion time of the children.
  355. */
  356. double starpu_sched_component_estimated_end_average(struct starpu_sched_component * component);
  357. struct starpu_sched_component_fifo_data
  358. {
  359. unsigned ntasks_threshold;
  360. double exp_len_threshold;
  361. };
  362. /**
  363. Return a struct starpu_sched_component with a fifo. A stable sort is performed according to tasks priorities.
  364. A push_task call on this component does not perform recursive calls, underlying components will have to call pop_task to get it.
  365. starpu_sched_component::estimated_end function compute the estimated length by dividing the sequential length by the number of underlying workers.
  366. */
  367. struct starpu_sched_component *starpu_sched_component_fifo_create(struct starpu_sched_tree *tree, struct starpu_sched_component_fifo_data *fifo_data) STARPU_ATTRIBUTE_MALLOC;
  368. /**
  369. return true iff \p component is a fifo component
  370. */
  371. int starpu_sched_component_is_fifo(struct starpu_sched_component *component);
  372. /** @} */
  373. /**
  374. @name Flow-control Prio Component API
  375. @{
  376. */
  377. struct starpu_sched_component_prio_data
  378. {
  379. unsigned ntasks_threshold;
  380. double exp_len_threshold;
  381. };
  382. struct starpu_sched_component *starpu_sched_component_prio_create(struct starpu_sched_tree *tree, struct starpu_sched_component_prio_data *prio_data) STARPU_ATTRIBUTE_MALLOC;
  383. int starpu_sched_component_is_prio(struct starpu_sched_component *component);
  384. /** @} */
  385. /**
  386. @name Resource-mapping Work-Stealing Component API
  387. @{
  388. */
  389. /**
  390. return a component that perform a work stealing scheduling. Tasks are pushed in a round robin way. estimated_end return the average of expected length of fifos, starting at the average of the expected_end of his children. When a worker have to steal a task, it steal a task in a round robin way, and get the last pushed task of the higher priority.
  391. */
  392. struct starpu_sched_component *starpu_sched_component_work_stealing_create(struct starpu_sched_tree *tree, void *arg) STARPU_ATTRIBUTE_MALLOC;
  393. /**
  394. return true iff \p component is a work stealing component
  395. */
  396. int starpu_sched_component_is_work_stealing(struct starpu_sched_component *component);
  397. /**
  398. undefined if there is no work stealing component in the scheduler. If any, \p task is pushed in a default way if the caller is the application, and in the caller's fifo if its a worker.
  399. */
  400. int starpu_sched_tree_work_stealing_push_task(struct starpu_task *task);
  401. /** @} */
  402. /**
  403. @name Resource-mapping Random Component API
  404. @{
  405. */
  406. /**
  407. create a component that perform a random scheduling
  408. */
  409. struct starpu_sched_component *starpu_sched_component_random_create(struct starpu_sched_tree *tree, void *arg) STARPU_ATTRIBUTE_MALLOC;
  410. /**
  411. return true iff \p component is a random component
  412. */
  413. int starpu_sched_component_is_random(struct starpu_sched_component *);
  414. /** @} */
  415. /**
  416. @name Resource-mapping Eager Component API
  417. @{
  418. */
  419. struct starpu_sched_component *starpu_sched_component_eager_create(struct starpu_sched_tree *tree, void *arg) STARPU_ATTRIBUTE_MALLOC;
  420. int starpu_sched_component_is_eager(struct starpu_sched_component *);
  421. /**
  422. @name Resource-mapping Eager-Calibration Component API
  423. @{
  424. */
  425. struct starpu_sched_component *starpu_sched_component_eager_calibration_create(struct starpu_sched_tree *tree, void *arg) STARPU_ATTRIBUTE_MALLOC;
  426. int starpu_sched_component_is_eager_calibration(struct starpu_sched_component *);
  427. /** @} */
  428. /**
  429. @name Resource-mapping MCT Component API
  430. @{
  431. */
  432. struct starpu_sched_component_mct_data
  433. {
  434. double alpha;
  435. double beta;
  436. double _gamma;
  437. double idle_power;
  438. };
  439. /**
  440. create a component with mct_data paremeters. the mct component doesnt
  441. do anything but pushing tasks on no_perf_model_component and
  442. calibrating_component
  443. */
  444. struct starpu_sched_component *starpu_sched_component_mct_create(struct starpu_sched_tree *tree, struct starpu_sched_component_mct_data *mct_data) STARPU_ATTRIBUTE_MALLOC;
  445. int starpu_sched_component_is_mct(struct starpu_sched_component *component);
  446. /** @} */
  447. /**
  448. @name Resource-mapping Heft Component API
  449. @{
  450. */
  451. struct starpu_sched_component *starpu_sched_component_heft_create(struct starpu_sched_tree *tree, struct starpu_sched_component_mct_data *mct_data) STARPU_ATTRIBUTE_MALLOC;
  452. int starpu_sched_component_is_heft(struct starpu_sched_component *component);
  453. /** @} */
  454. /**
  455. @name Special-purpose Best_Implementation Component API
  456. @{
  457. */
  458. /**
  459. Select the implementation that offer the shortest computation length for the first worker that can execute the task.
  460. Or an implementation that need to be calibrated.
  461. Also set starpu_task::predicted and starpu_task::predicted_transfer for memory component of the first suitable workerid.
  462. If starpu_sched_component::push method is called and starpu_sched_component::nchild > 1 the result is undefined.
  463. */
  464. struct starpu_sched_component *starpu_sched_component_best_implementation_create(struct starpu_sched_tree *tree, void *arg) STARPU_ATTRIBUTE_MALLOC;
  465. /** @} */
  466. /**
  467. @name Special-purpose Perfmodel_Select Component API
  468. @{
  469. */
  470. struct starpu_sched_component_perfmodel_select_data
  471. {
  472. struct starpu_sched_component *calibrator_component;
  473. struct starpu_sched_component *no_perfmodel_component;
  474. struct starpu_sched_component *perfmodel_component;
  475. };
  476. struct starpu_sched_component *starpu_sched_component_perfmodel_select_create(struct starpu_sched_tree *tree, struct starpu_sched_component_perfmodel_select_data *perfmodel_select_data) STARPU_ATTRIBUTE_MALLOC;
  477. int starpu_sched_component_is_perfmodel_select(struct starpu_sched_component *component);
  478. /** @} */
  479. /**
  480. @name Recipe Component API
  481. @{
  482. */
  483. /**
  484. parameters for starpu_sched_component_composed_component_create
  485. */
  486. struct starpu_sched_component_composed_recipe;
  487. /**
  488. return an empty recipe for a composed component, it should not be used without modification
  489. */
  490. struct starpu_sched_component_composed_recipe *starpu_sched_component_composed_recipe_create(void) STARPU_ATTRIBUTE_MALLOC;
  491. /**
  492. return a recipe to build a composed component with a \p create_component
  493. */
  494. struct starpu_sched_component_composed_recipe *starpu_sched_component_composed_recipe_create_singleton(struct starpu_sched_component *(*create_component)(struct starpu_sched_tree *tree, void *arg), void *arg) STARPU_ATTRIBUTE_MALLOC;
  495. /**
  496. add \p create_component under all previous components in recipe
  497. */
  498. void starpu_sched_component_composed_recipe_add(struct starpu_sched_component_composed_recipe *recipe, struct starpu_sched_component *(*create_component)(struct starpu_sched_tree *tree, void *arg), void *arg);
  499. /**
  500. destroy composed_sched_component, this should be done after starpu_sched_component_composed_component_create was called
  501. */
  502. void starpu_sched_component_composed_recipe_destroy(struct starpu_sched_component_composed_recipe *);
  503. /**
  504. create a component that behave as all component of recipe where linked. Except that you cant use starpu_sched_component_is_foo function
  505. if recipe contain a single create_foo arg_foo pair, create_foo(arg_foo) is returned instead of a composed component
  506. */
  507. struct starpu_sched_component *starpu_sched_component_composed_component_create(struct starpu_sched_tree *tree, struct starpu_sched_component_composed_recipe *recipe) STARPU_ATTRIBUTE_MALLOC;
  508. #ifdef STARPU_HAVE_HWLOC
  509. /**
  510. Define how build a scheduler according to topology. Each level (except for hwloc_machine_composed_sched_component) can be <c>NULL</c>, then
  511. the level is just skipped. Bugs everywhere, do not rely on.
  512. */
  513. struct starpu_sched_component_specs
  514. {
  515. /**
  516. the composed component to put on the top of the scheduler
  517. this member must not be <c>NULL</c> as it is the root of the topology
  518. */
  519. struct starpu_sched_component_composed_recipe *hwloc_machine_composed_sched_component;
  520. /**
  521. the composed component to put for each memory component
  522. */
  523. struct starpu_sched_component_composed_recipe *hwloc_component_composed_sched_component;
  524. /**
  525. the composed component to put for each socket
  526. */
  527. struct starpu_sched_component_composed_recipe *hwloc_socket_composed_sched_component;
  528. /**
  529. the composed component to put for each cache
  530. */
  531. struct starpu_sched_component_composed_recipe *hwloc_cache_composed_sched_component;
  532. /**
  533. a function that return a starpu_sched_component_composed_recipe to put on top of a worker of type \p archtype.
  534. <c>NULL</c> is a valid return value, then no component will be added on top
  535. */
  536. struct starpu_sched_component_composed_recipe *(*worker_composed_sched_component)(enum starpu_worker_archtype archtype);
  537. /**
  538. this flag is a dirty hack because of the poor expressivity of this interface. As example, if you want to build
  539. a heft component with a fifo component per numa component, and you also have GPUs, if this flag is set, GPUs will share those fifos.
  540. If this flag is not set, a new fifo will be built for each of them (if they have the same starpu_perf_arch and the same
  541. numa component it will be shared. it indicates if heterogenous workers should be brothers or cousins, as example, if a gpu and a cpu should share or not there numa node
  542. */
  543. int mix_heterogeneous_workers;
  544. };
  545. /**
  546. build a scheduler for \p sched_ctx_id according to \p s and the hwloc topology of the machine.
  547. */
  548. struct starpu_sched_tree *starpu_sched_component_make_scheduler(unsigned sched_ctx_id, struct starpu_sched_component_specs s);
  549. #endif /* STARPU_HAVE_HWLOC */
  550. /**
  551. @name Basic API
  552. @{
  553. */
  554. #define STARPU_SCHED_SIMPLE_DECIDE_MASK (3<<0)
  555. /**
  556. Request to create downstream queues per worker, i.e. the scheduling decision-making component will choose exactly which workers tasks should got to.
  557. */
  558. #define STARPU_SCHED_SIMPLE_DECIDE_WORKERS (1<<0)
  559. /**
  560. Request to create downstream queues per memory nodes, i.e. the scheduling decision-making component will choose which memory node tasks will go to.
  561. */
  562. #define STARPU_SCHED_SIMPLE_DECIDE_MEMNODES (2<<0)
  563. /**
  564. Request to create downstream queues per computation arch, i.e. the scheduling decision-making component will choose whether tasks go to CPUs, or CUDA, or OpenCL, etc.
  565. */
  566. #define STARPU_SCHED_SIMPLE_DECIDE_ARCHS (3<<0)
  567. /**
  568. Request to add a perfmodel selector above the scheduling decision-making component. That way, only tasks with a calibrated performance model will be given to the component, other tasks will go to an eager branch that will distributed tasks so that their performance models will get calibrated.
  569. In other words, this is needed when using a component which needs performance models for tasks.
  570. */
  571. #define STARPU_SCHED_SIMPLE_PERFMODEL (1<<4)
  572. /**
  573. Request that a component be added just above workers, that chooses the best task implementation.
  574. */
  575. #define STARPU_SCHED_SIMPLE_IMPL (1<<5)
  576. /**
  577. Request to create a fifo above the scheduling decision-making component, otherwise tasks will be pushed directly to the component.
  578. This is useful to store tasks if there is a fifo below which limits the number of tasks to be scheduld in advance. The scheduling decision-making component can also store tasks itself, in which case this flag is not useful.
  579. */
  580. #define STARPU_SCHED_SIMPLE_FIFO_ABOVE (1<<6)
  581. /**
  582. Request that the fifo above be sorted by priorities
  583. */
  584. #define STARPU_SCHED_SIMPLE_FIFO_ABOVE_PRIO (1<<7)
  585. /**
  586. Request to create fifos below the scheduling decision-making component, otherwise tasks will be pulled directly from workers.
  587. This is useful to be able to schedule a (tunable) small number of tasks in advance only.
  588. */
  589. #define STARPU_SCHED_SIMPLE_FIFOS_BELOW (1<<8)
  590. /**
  591. Request that the fifos below be sorted by priorities
  592. */
  593. #define STARPU_SCHED_SIMPLE_FIFOS_BELOW_PRIO (1<<9)
  594. /**
  595. Request that work between workers using the same fifo below be distributed using a work stealing component.
  596. */
  597. #define STARPU_SCHED_SIMPLE_WS_BELOW (1<<10)
  598. /**
  599. Request to not only choose between simple workers, but also choose between combined workers.
  600. */
  601. #define STARPU_SCHED_SIMPLE_COMBINED_WORKERS (1<<11)
  602. /**
  603. Create a simple modular scheduler tree around a scheduling decision-making
  604. component \p component. The details of what should be built around \p component
  605. is described by \p flags. The different STARPU_SCHED_SIMPL_DECIDE_* flags are
  606. mutually exclusive. \p data is passed to the \p create_decision_component
  607. function when creating the decision component.
  608. */
  609. void starpu_sched_component_initialize_simple_scheduler(starpu_sched_component_create_t create_decision_component, void *data, unsigned flags, unsigned sched_ctx_id);
  610. /** @} */
  611. #define STARPU_COMPONENT_MUTEX_LOCK(m) \
  612. do \
  613. { \
  614. const int _relaxed_state = starpu_worker_get_relax_state(); \
  615. if (!_relaxed_state) \
  616. starpu_worker_relax_on(); \
  617. STARPU_PTHREAD_MUTEX_LOCK((m)); \
  618. if (!_relaxed_state) \
  619. starpu_worker_relax_off(); \
  620. } \
  621. while(0)
  622. #define STARPU_COMPONENT_MUTEX_TRYLOCK(m) STARPU_PTHREAD_MUTEX_TRYLOCK((m))
  623. #define STARPU_COMPONENT_MUTEX_UNLOCK(m) STARPU_PTHREAD_MUTEX_UNLOCK((m))
  624. /** @} */
  625. #ifdef __cplusplus
  626. }
  627. #endif
  628. #endif /* __STARPU_SCHED_COMPONENT_H__ */