sched_ctx.c 41 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593594595596597598599600601602603604605606607608609610611612613614615616617618619620621622623624625626627628629630631632633634635636637638639640641642643644645646647648649650651652653654655656657658659660661662663664665666667668669670671672673674675676677678679680681682683684685686687688689690691692693694695696697698699700701702703704705706707708709710711712713714715716717718719720721722723724725726727728729730731732733734735736737738739740741742743744745746747748749750751752753754755756757758759760761762763764765766767768769770771772773774775776777778779780781782783784785786787788789790791792793794795796797798799800801802803804805806807808809810811812813814815816817818819820821822823824825826827828829830831832833834835836837838839840841842843844845846847848849850851852853854855856857858859860861862863864865866867868869870871872873874875876877878879880881882883884885886887888889890891892893894895896897898899900901902903904905906907908909910911912913914915916917918919920921922923924925926927928929930931932933934935936937938939940941942943944945946947948949950951952953954955956957958959960961962963964965966967968969970971972973974975976977978979980981982983984985986987988989990991992993994995996997998999100010011002100310041005100610071008100910101011101210131014101510161017101810191020102110221023102410251026102710281029103010311032103310341035103610371038103910401041104210431044104510461047104810491050105110521053105410551056105710581059106010611062106310641065106610671068106910701071107210731074107510761077107810791080108110821083108410851086108710881089109010911092109310941095109610971098109911001101110211031104110511061107110811091110111111121113111411151116111711181119112011211122112311241125112611271128112911301131113211331134113511361137113811391140114111421143114411451146114711481149115011511152115311541155115611571158115911601161116211631164116511661167116811691170117111721173117411751176117711781179118011811182118311841185118611871188118911901191119211931194119511961197119811991200120112021203120412051206120712081209121012111212121312141215121612171218121912201221122212231224122512261227122812291230123112321233123412351236123712381239124012411242124312441245124612471248124912501251125212531254125512561257125812591260126112621263126412651266126712681269127012711272127312741275127612771278127912801281128212831284128512861287128812891290129112921293129412951296129712981299130013011302130313041305
  1. /* StarPU --- Runtime system for heterogeneous multicore architectures.
  2. *
  3. * Copyright (C) 2011, 2013 INRIA
  4. *
  5. * StarPU is free software; you can redistribute it and/or modify
  6. * it under the terms of the GNU Lesser General Public License as published by
  7. * the Free Software Foundation; either version 2.1 of the License, or (at
  8. * your option) any later version.
  9. *
  10. * StarPU is distributed in the hope that it will be useful, but
  11. * WITHOUT ANY WARRANTY; without even the implied warranty of
  12. * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE.
  13. *
  14. * See the GNU Lesser General Public License in COPYING.LGPL for more details.
  15. */
  16. #include <core/sched_policy.h>
  17. #include <core/sched_ctx.h>
  18. #include <common/utils.h>
  19. starpu_pthread_mutex_t changing_ctx_mutex[STARPU_NMAX_SCHED_CTXS];
  20. extern struct starpu_worker_collection worker_list;
  21. static starpu_pthread_mutex_t sched_ctx_manag = STARPU_PTHREAD_MUTEX_INITIALIZER;
  22. static starpu_pthread_mutex_t finished_submit_mutex = STARPU_PTHREAD_MUTEX_INITIALIZER;
  23. struct starpu_task stop_submission_task = STARPU_TASK_INITIALIZER;
  24. starpu_pthread_key_t sched_ctx_key;
  25. unsigned with_hypervisor = 0;
  26. double max_time_worker_on_ctx = -1.0;
  27. static unsigned _starpu_get_first_free_sched_ctx(struct _starpu_machine_config *config);
  28. static void _starpu_worker_gets_into_ctx(unsigned sched_ctx_id, struct _starpu_worker *worker)
  29. {
  30. unsigned ret_sched_ctx = _starpu_sched_ctx_list_get_sched_ctx(worker->sched_ctx_list, sched_ctx_id);
  31. /* the worker was planning to go away in another ctx but finally he changed his mind &
  32. he's staying */
  33. if (ret_sched_ctx == STARPU_NMAX_SCHED_CTXS)
  34. {
  35. /* add context to worker */
  36. _starpu_sched_ctx_list_add(&worker->sched_ctx_list, sched_ctx_id);
  37. worker->nsched_ctxs++;
  38. worker->active_ctx = sched_ctx_id;
  39. }
  40. worker->removed_from_ctx[sched_ctx_id] = 0;
  41. return;
  42. }
  43. void _starpu_worker_gets_out_of_ctx(unsigned sched_ctx_id, struct _starpu_worker *worker)
  44. {
  45. unsigned ret_sched_ctx = _starpu_sched_ctx_list_get_sched_ctx(worker->sched_ctx_list, sched_ctx_id);
  46. /* remove context from worker */
  47. if(ret_sched_ctx != STARPU_NMAX_SCHED_CTXS)
  48. {
  49. struct _starpu_sched_ctx *sched_ctx = _starpu_get_sched_ctx_struct(sched_ctx_id);
  50. if(sched_ctx && sched_ctx->sched_policy && sched_ctx->sched_policy->remove_workers)
  51. sched_ctx->sched_policy->remove_workers(sched_ctx_id, &worker->workerid, 1);
  52. _starpu_sched_ctx_list_remove(&worker->sched_ctx_list, sched_ctx_id);
  53. worker->nsched_ctxs--;
  54. }
  55. return;
  56. }
  57. static void _starpu_update_workers_with_ctx(int *workerids, int nworkers, int sched_ctx_id)
  58. {
  59. int i;
  60. struct _starpu_worker *worker = NULL;
  61. struct _starpu_worker *curr_worker = _starpu_get_local_worker_key();
  62. for(i = 0; i < nworkers; i++)
  63. {
  64. worker = _starpu_get_worker_struct(workerids[i]);
  65. /* if the current thread requires resize it's no need
  66. to lock it in order to change its sched_ctx info */
  67. if(curr_worker && curr_worker == worker)
  68. _starpu_worker_gets_into_ctx(sched_ctx_id, worker);
  69. else
  70. {
  71. STARPU_PTHREAD_MUTEX_LOCK(&worker->sched_mutex);
  72. _starpu_worker_gets_into_ctx(sched_ctx_id, worker);
  73. STARPU_PTHREAD_MUTEX_UNLOCK(&worker->sched_mutex);
  74. }
  75. }
  76. return;
  77. }
  78. static void _starpu_update_workers_without_ctx(int *workerids, int nworkers, int sched_ctx_id, unsigned now)
  79. {
  80. int i;
  81. struct _starpu_worker *worker = NULL;
  82. struct _starpu_worker *curr_worker = _starpu_get_local_worker_key();
  83. for(i = 0; i < nworkers; i++)
  84. {
  85. worker = _starpu_get_worker_struct(workerids[i]);
  86. if(now)
  87. {
  88. if(curr_worker && curr_worker == worker)
  89. _starpu_worker_gets_out_of_ctx(sched_ctx_id, worker);
  90. else
  91. {
  92. STARPU_PTHREAD_MUTEX_LOCK(&worker->sched_mutex);
  93. _starpu_worker_gets_out_of_ctx(sched_ctx_id, worker);
  94. STARPU_PTHREAD_MUTEX_UNLOCK(&worker->sched_mutex);
  95. }
  96. }
  97. else
  98. {
  99. if(curr_worker && curr_worker == worker)
  100. worker->removed_from_ctx[sched_ctx_id] = 1;
  101. else
  102. {
  103. STARPU_PTHREAD_MUTEX_LOCK(&worker->sched_mutex);
  104. worker->removed_from_ctx[sched_ctx_id] = 1;
  105. STARPU_PTHREAD_MUTEX_UNLOCK(&worker->sched_mutex);
  106. }
  107. }
  108. }
  109. return;
  110. }
  111. void starpu_sched_ctx_stop_task_submission()
  112. {
  113. _starpu_exclude_task_from_dag(&stop_submission_task);
  114. _starpu_task_submit_internally(&stop_submission_task);
  115. }
  116. void starpu_sched_ctx_worker_shares_tasks_lists(int workerid, int sched_ctx_id)
  117. {
  118. struct _starpu_worker *worker = _starpu_get_worker_struct(workerid);
  119. struct _starpu_sched_ctx *sched_ctx = _starpu_get_sched_ctx_struct(sched_ctx_id);
  120. /* if is the initial sched_ctx no point in taking the mutex, the workers are
  121. not launched yet */
  122. if(!sched_ctx->is_initial_sched)
  123. STARPU_PTHREAD_MUTEX_LOCK(&worker->sched_mutex);
  124. worker->shares_tasks_lists[sched_ctx_id] = 1;
  125. if(!sched_ctx->is_initial_sched)
  126. STARPU_PTHREAD_MUTEX_UNLOCK(&worker->sched_mutex);
  127. }
  128. static void _starpu_add_workers_to_sched_ctx(struct _starpu_sched_ctx *sched_ctx, int *workerids, int nworkers,
  129. int *added_workers, int *n_added_workers)
  130. {
  131. struct starpu_worker_collection *workers = sched_ctx->workers;
  132. struct _starpu_machine_config *config = (struct _starpu_machine_config *)_starpu_get_machine_config();
  133. int nworkers_to_add = nworkers == -1 ? (int)config->topology.nworkers : nworkers;
  134. int workers_to_add[nworkers_to_add];
  135. int i = 0;
  136. for(i = 0; i < nworkers_to_add; i++)
  137. {
  138. /* added_workers is NULL for the call of this func at the creation of the context*/
  139. /* if the function is called at the creation of the context it's no need to do this verif */
  140. if(added_workers)
  141. {
  142. int worker = workers->add(workers, (workerids == NULL ? i : workerids[i]));
  143. if(worker >= 0)
  144. added_workers[(*n_added_workers)++] = worker;
  145. else
  146. {
  147. struct _starpu_worker *worker_str = _starpu_get_worker_struct(workerids[i]);
  148. STARPU_PTHREAD_MUTEX_LOCK(&worker_str->sched_mutex);
  149. worker_str->removed_from_ctx[sched_ctx->id] = 0;
  150. STARPU_PTHREAD_MUTEX_UNLOCK(&worker_str->sched_mutex);
  151. }
  152. }
  153. else
  154. {
  155. int worker = (workerids == NULL ? i : workerids[i]);
  156. workers->add(workers, worker);
  157. workers_to_add[i] = worker;
  158. }
  159. }
  160. if(sched_ctx->sched_policy->add_workers)
  161. {
  162. if(added_workers)
  163. {
  164. if(*n_added_workers > 0)
  165. sched_ctx->sched_policy->add_workers(sched_ctx->id, added_workers, *n_added_workers);
  166. }
  167. else
  168. sched_ctx->sched_policy->add_workers(sched_ctx->id, workers_to_add, nworkers_to_add);
  169. }
  170. return;
  171. }
  172. static void _starpu_remove_workers_from_sched_ctx(struct _starpu_sched_ctx *sched_ctx, int *workerids,
  173. int nworkers, int *removed_workers, int *n_removed_workers)
  174. {
  175. struct starpu_worker_collection *workers = sched_ctx->workers;
  176. int i = 0;
  177. for(i = 0; i < nworkers; i++)
  178. {
  179. if(workers->nworkers > 0)
  180. {
  181. if(_starpu_worker_belongs_to_a_sched_ctx(workerids[i], sched_ctx->id))
  182. {
  183. int worker = workers->remove(workers, workerids[i]);
  184. if(worker >= 0)
  185. removed_workers[(*n_removed_workers)++] = worker;
  186. }
  187. }
  188. }
  189. return;
  190. }
  191. static void _starpu_sched_ctx_free_scheduling_data(struct _starpu_sched_ctx *sched_ctx)
  192. {
  193. int *workerids = NULL;
  194. unsigned nworkers_ctx = starpu_sched_ctx_get_workers_list(sched_ctx->id, &workerids);
  195. if(nworkers_ctx > 0 && sched_ctx->sched_policy->remove_workers)
  196. sched_ctx->sched_policy->remove_workers(sched_ctx->id, workerids, nworkers_ctx);
  197. free(workerids);
  198. return;
  199. }
  200. #ifdef STARPU_HAVE_HWLOC
  201. static void _starpu_sched_ctx_create_hwloc_tree(struct _starpu_sched_ctx *sched_ctx)
  202. {
  203. struct _starpu_machine_config *config = (struct _starpu_machine_config *)_starpu_get_machine_config();
  204. sched_ctx->hwloc_workers_set = hwloc_bitmap_alloc();
  205. struct starpu_worker_collection *workers = sched_ctx->workers;
  206. int worker;
  207. struct starpu_sched_ctx_iterator it;
  208. if(workers->init_iterator)
  209. workers->init_iterator(workers, &it);
  210. while(workers->has_next(workers, &it))
  211. {
  212. worker = workers->get_next(workers, &it);
  213. if(!starpu_worker_is_combined_worker(worker))
  214. {
  215. hwloc_bitmap_or(sched_ctx->hwloc_workers_set,
  216. sched_ctx->hwloc_workers_set,
  217. config->workers[worker].hwloc_cpu_set);
  218. }
  219. }
  220. return;
  221. }
  222. #endif
  223. struct _starpu_sched_ctx* _starpu_create_sched_ctx(struct starpu_sched_policy *policy, int *workerids,
  224. int nworkers_ctx, unsigned is_initial_sched,
  225. const char *sched_name)
  226. {
  227. struct _starpu_machine_config *config = (struct _starpu_machine_config *)_starpu_get_machine_config();
  228. STARPU_PTHREAD_MUTEX_LOCK(&sched_ctx_manag);
  229. STARPU_ASSERT(config->topology.nsched_ctxs < STARPU_NMAX_SCHED_CTXS);
  230. unsigned id = _starpu_get_first_free_sched_ctx(config);
  231. struct _starpu_sched_ctx *sched_ctx = &config->sched_ctxs[id];
  232. sched_ctx->id = id;
  233. config->topology.nsched_ctxs++;
  234. STARPU_PTHREAD_MUTEX_UNLOCK(&sched_ctx_manag);
  235. int nworkers = config->topology.nworkers;
  236. STARPU_ASSERT(nworkers_ctx <= nworkers);
  237. STARPU_PTHREAD_MUTEX_INIT(&sched_ctx->empty_ctx_mutex, NULL);
  238. starpu_task_list_init(&sched_ctx->empty_ctx_tasks);
  239. sched_ctx->sched_policy = (struct starpu_sched_policy*)malloc(sizeof(struct starpu_sched_policy));
  240. sched_ctx->is_initial_sched = is_initial_sched;
  241. sched_ctx->name = sched_name;
  242. sched_ctx->inheritor = STARPU_NMAX_SCHED_CTXS;
  243. sched_ctx->finished_submit = 0;
  244. sched_ctx->min_priority = 0;
  245. sched_ctx->max_priority = 1;
  246. sem_init(&sched_ctx->parallel_code_sem, 0, 0);
  247. _starpu_barrier_counter_init(&sched_ctx->tasks_barrier, 0);
  248. /*init the strategy structs and the worker_collection of the ressources of the context */
  249. _starpu_init_sched_policy(config, sched_ctx, policy);
  250. /* construct the collection of workers(list/tree/etc.) */
  251. sched_ctx->workers->init(sched_ctx->workers);
  252. /* after having an worker_collection on the ressources add them */
  253. _starpu_add_workers_to_sched_ctx(sched_ctx, workerids, nworkers_ctx, NULL, NULL);
  254. #ifdef STARPU_HAVE_HWLOC
  255. /* build hwloc tree of the context */
  256. _starpu_sched_ctx_create_hwloc_tree(sched_ctx);
  257. #endif //STARPU_HAVE_HWLOC
  258. /* if we create the initial big sched ctx we can update workers' status here
  259. because they haven't been launched yet */
  260. if(is_initial_sched)
  261. {
  262. int i;
  263. /*initialize the mutexes for all contexts */
  264. for(i = 0; i < STARPU_NMAX_SCHED_CTXS; i++)
  265. STARPU_PTHREAD_MUTEX_INIT(&changing_ctx_mutex[i], NULL);
  266. for(i = 0; i < nworkers; i++)
  267. {
  268. struct _starpu_worker *worker = _starpu_get_worker_struct(i);
  269. worker->sched_ctx_list = (struct _starpu_sched_ctx_list*)malloc(sizeof(struct _starpu_sched_ctx_list));
  270. _starpu_sched_ctx_list_init(worker->sched_ctx_list);
  271. _starpu_sched_ctx_list_add(&worker->sched_ctx_list, sched_ctx->id);
  272. worker->nsched_ctxs++;
  273. }
  274. }
  275. int w;
  276. for(w = 0; w < STARPU_NMAXWORKERS; w++)
  277. {
  278. sched_ctx->pop_counter[w] = 0;
  279. }
  280. return sched_ctx;
  281. }
  282. static void _get_workers(int min, int max, int *workers, int *nw, enum starpu_worker_archtype arch, unsigned allow_overlap)
  283. {
  284. int pus[max];
  285. int npus = 0;
  286. int i;
  287. int n = 0;
  288. struct _starpu_machine_config *config = (struct _starpu_machine_config *)_starpu_get_machine_config();
  289. if(config->topology.nsched_ctxs == 1)
  290. {
  291. /*we have all available resources */
  292. npus = starpu_worker_get_nids_by_type(arch, pus, max);
  293. /*TODO: hierarchical ctxs: get max good workers: close one to another */
  294. for(i = 0; i < npus; i++)
  295. workers[(*nw)++] = pus[i];
  296. }
  297. else
  298. {
  299. unsigned enough_ressources = 0;
  300. npus = starpu_worker_get_nids_ctx_free_by_type(arch, pus, max);
  301. for(i = 0; i < npus; i++)
  302. workers[(*nw)++] = pus[i];
  303. if(npus == max)
  304. /*we have enough available resources */
  305. enough_ressources = 1;
  306. if(!enough_ressources && npus >= min)
  307. /*we have enough available resources */
  308. enough_ressources = 1;
  309. if(!enough_ressources)
  310. {
  311. /* try to get ressources from ctx who have more than the min of workers they need */
  312. int s;
  313. for(s = 1; s < STARPU_NMAX_SCHED_CTXS; s++)
  314. {
  315. if(config->sched_ctxs[s].id != STARPU_NMAX_SCHED_CTXS)
  316. {
  317. int _npus = 0;
  318. int _pus[STARPU_NMAXWORKERS];
  319. _npus = starpu_get_workers_of_sched_ctx(config->sched_ctxs[s].id, _pus, arch);
  320. int ctx_min = arch == STARPU_CPU_WORKER ? config->sched_ctxs[s].min_ncpus : config->sched_ctxs[s].min_ngpus;
  321. if(_npus > ctx_min)
  322. {
  323. if(npus < min)
  324. {
  325. n = (_npus - ctx_min) > (min - npus) ? min - npus : (_npus - ctx_min);
  326. npus += n;
  327. }
  328. /*TODO: hierarchical ctxs: get n good workers: close to the other ones I already assigned to the ctx */
  329. for(i = 0; i < n; i++)
  330. workers[(*nw)++] = _pus[i];
  331. starpu_sched_ctx_remove_workers(_pus, n, config->sched_ctxs[s].id);
  332. }
  333. }
  334. }
  335. if(npus >= min)
  336. enough_ressources = 1;
  337. }
  338. if(!enough_ressources)
  339. {
  340. /* if there is no available workers to satisfy the minimum required
  341. give them workers proportional to their requirements*/
  342. int global_npus = starpu_worker_get_count_by_type(arch);
  343. int req_npus = 0;
  344. int s;
  345. for(s = 1; s < STARPU_NMAX_SCHED_CTXS; s++)
  346. if(config->sched_ctxs[s].id != STARPU_NMAX_SCHED_CTXS)
  347. req_npus += arch == STARPU_CPU_WORKER ? config->sched_ctxs[s].min_ncpus : config->sched_ctxs[s].min_ngpus;
  348. req_npus += min;
  349. for(s = 1; s < STARPU_NMAX_SCHED_CTXS; s++)
  350. {
  351. if(config->sched_ctxs[s].id != STARPU_NMAX_SCHED_CTXS)
  352. {
  353. int ctx_min = arch == STARPU_CPU_WORKER ? config->sched_ctxs[s].min_ncpus : config->sched_ctxs[s].min_ngpus;
  354. double needed_npus = ((double)ctx_min * (double)global_npus) / (double)req_npus;
  355. int _npus = 0;
  356. int _pus[STARPU_NMAXWORKERS];
  357. _npus = starpu_get_workers_of_sched_ctx(config->sched_ctxs[s].id, _pus, arch);
  358. if(needed_npus < (double)_npus)
  359. {
  360. double npus_to_rem = (double)_npus - needed_npus;
  361. int x = floor(npus_to_rem);
  362. double x_double = (double)x;
  363. double diff = npus_to_rem - x_double;
  364. int npus_to_remove = diff >= 0.5 ? x+1 : x;
  365. int pus_to_remove[npus_to_remove];
  366. int c = 0;
  367. /*TODO: hierarchical ctxs: get npus_to_remove good workers: close to the other ones I already assigned to the ctx */
  368. for(i = _npus-1; i >= (_npus - npus_to_remove); i--)
  369. {
  370. workers[(*nw)++] = _pus[i];
  371. pus_to_remove[c++] = _pus[i];
  372. }
  373. if(!allow_overlap)
  374. starpu_sched_ctx_remove_workers(pus_to_remove, npus_to_remove, config->sched_ctxs[s].id);
  375. }
  376. }
  377. }
  378. }
  379. }
  380. }
  381. unsigned starpu_sched_ctx_create_inside_interval(const char *policy_name, const char *sched_name,
  382. int min_ncpus, int max_ncpus, int min_ngpus, int max_ngpus,
  383. unsigned allow_overlap)
  384. {
  385. struct _starpu_machine_config *config = (struct _starpu_machine_config *)_starpu_get_machine_config();
  386. struct starpu_sched_policy *selected_policy = _starpu_select_sched_policy(config, policy_name);
  387. struct _starpu_sched_ctx *sched_ctx = NULL;
  388. int workers[max_ncpus + max_ngpus];
  389. int nw = 0;
  390. STARPU_PTHREAD_MUTEX_LOCK(&sched_ctx_manag);
  391. _get_workers(min_ncpus, max_ncpus, workers, &nw, STARPU_CPU_WORKER, allow_overlap);
  392. _get_workers(min_ngpus, max_ngpus, workers, &nw, STARPU_CUDA_WORKER, allow_overlap);
  393. STARPU_PTHREAD_MUTEX_UNLOCK(&sched_ctx_manag);
  394. int i;
  395. printf("%d: ", nw);
  396. for(i = 0; i < nw; i++)
  397. printf("%d ", workers[i]);
  398. printf("\n");
  399. sched_ctx = _starpu_create_sched_ctx(selected_policy, workers, nw, 0, sched_name);
  400. sched_ctx->min_ncpus = min_ncpus;
  401. sched_ctx->max_ncpus = max_ncpus;
  402. sched_ctx->min_ngpus = min_ngpus;
  403. sched_ctx->max_ngpus = max_ngpus;
  404. _starpu_update_workers_without_ctx(sched_ctx->workers->workerids, sched_ctx->workers->nworkers, sched_ctx->id, 0);
  405. #ifdef STARPU_USE_SC_HYPERVISOR
  406. sched_ctx->perf_counters = NULL;
  407. #endif
  408. return sched_ctx->id;
  409. }
  410. unsigned starpu_sched_ctx_create(const char *policy_name, int *workerids,
  411. int nworkers, const char *sched_name)
  412. {
  413. struct _starpu_machine_config *config = (struct _starpu_machine_config *)_starpu_get_machine_config();
  414. struct starpu_sched_policy *selected_policy = _starpu_select_sched_policy(config, policy_name);
  415. struct _starpu_sched_ctx *sched_ctx = NULL;
  416. sched_ctx = _starpu_create_sched_ctx(selected_policy, workerids, nworkers, 0, sched_name);
  417. _starpu_update_workers_with_ctx(sched_ctx->workers->workerids, sched_ctx->workers->nworkers, sched_ctx->id);
  418. #ifdef STARPU_USE_SC_HYPERVISOR
  419. sched_ctx->perf_counters = NULL;
  420. #endif
  421. return sched_ctx->id;
  422. }
  423. unsigned starpu_sched_ctx_create_with_custom_policy(struct starpu_sched_policy *policy, int *workerids, int nworkers, const char *sched_name)
  424. {
  425. struct _starpu_sched_ctx *sched_ctx = NULL;
  426. sched_ctx = _starpu_create_sched_ctx(policy, workerids, nworkers, 0, sched_name);
  427. _starpu_update_workers_with_ctx(sched_ctx->workers->workerids, sched_ctx->workers->nworkers, sched_ctx->id);
  428. #ifdef STARPU_USE_SC_HYPERVISOR
  429. sched_ctx->perf_counters = NULL;
  430. #endif
  431. return sched_ctx->id;
  432. }
  433. #ifdef STARPU_USE_SC_HYPERVISOR
  434. void starpu_sched_ctx_set_perf_counters(unsigned sched_ctx_id, struct starpu_sched_ctx_performance_counters *perf_counters)
  435. {
  436. struct _starpu_sched_ctx *sched_ctx = _starpu_get_sched_ctx_struct(sched_ctx_id);
  437. sched_ctx->perf_counters = perf_counters;
  438. return;
  439. }
  440. #endif
  441. /* free all structures for the context */
  442. static void _starpu_delete_sched_ctx(struct _starpu_sched_ctx *sched_ctx)
  443. {
  444. STARPU_ASSERT(sched_ctx->id != STARPU_NMAX_SCHED_CTXS);
  445. _starpu_deinit_sched_policy(sched_ctx);
  446. free(sched_ctx->sched_policy);
  447. sched_ctx->sched_policy = NULL;
  448. STARPU_PTHREAD_MUTEX_DESTROY(&sched_ctx->empty_ctx_mutex);
  449. sem_destroy(&sched_ctx->parallel_code_sem);
  450. sched_ctx->id = STARPU_NMAX_SCHED_CTXS;
  451. #ifdef STARPU_HAVE_HWLOC
  452. hwloc_bitmap_free(sched_ctx->hwloc_workers_set);
  453. #endif //STARPU_HAVE_HWLOC
  454. struct _starpu_machine_config *config = _starpu_get_machine_config();
  455. STARPU_PTHREAD_MUTEX_LOCK(&sched_ctx_manag);
  456. config->topology.nsched_ctxs--;
  457. STARPU_PTHREAD_MUTEX_UNLOCK(&sched_ctx_manag);
  458. }
  459. void starpu_sched_ctx_delete(unsigned sched_ctx_id)
  460. {
  461. struct _starpu_sched_ctx *sched_ctx = _starpu_get_sched_ctx_struct(sched_ctx_id);
  462. #ifdef STARPU_USE_SC_HYPERVISOR
  463. if(sched_ctx != NULL && sched_ctx_id != 0 && sched_ctx_id != STARPU_NMAX_SCHED_CTXS
  464. && sched_ctx->perf_counters != NULL)
  465. sched_ctx->perf_counters->notify_delete_context(sched_ctx_id);
  466. #endif //STARPU_USE_SC_HYPERVISOR
  467. unsigned inheritor_sched_ctx_id = sched_ctx->inheritor;
  468. struct _starpu_sched_ctx *inheritor_sched_ctx = _starpu_get_sched_ctx_struct(sched_ctx->inheritor);
  469. STARPU_PTHREAD_MUTEX_LOCK(&changing_ctx_mutex[sched_ctx_id]);
  470. STARPU_ASSERT(sched_ctx->id != STARPU_NMAX_SCHED_CTXS);
  471. int *workerids;
  472. unsigned nworkers_ctx = starpu_sched_ctx_get_workers_list(sched_ctx->id, &workerids);
  473. /*if both of them have all the ressources is pointless*/
  474. /*trying to transfer ressources from one ctx to the other*/
  475. struct _starpu_machine_config *config = (struct _starpu_machine_config *)_starpu_get_machine_config();
  476. unsigned nworkers = config->topology.nworkers;
  477. if(nworkers_ctx > 0 && inheritor_sched_ctx && inheritor_sched_ctx->id != STARPU_NMAX_SCHED_CTXS &&
  478. !(nworkers_ctx == nworkers && nworkers_ctx == inheritor_sched_ctx->workers->nworkers))
  479. {
  480. starpu_sched_ctx_add_workers(workerids, nworkers_ctx, inheritor_sched_ctx_id);
  481. }
  482. if(!_starpu_wait_for_all_tasks_of_sched_ctx(sched_ctx_id))
  483. {
  484. /*if btw the mutex release & the mutex lock the context has changed take care to free all
  485. scheduling data before deleting the context */
  486. _starpu_update_workers_without_ctx(workerids, nworkers_ctx, sched_ctx_id, 1);
  487. // _starpu_sched_ctx_free_scheduling_data(sched_ctx);
  488. _starpu_delete_sched_ctx(sched_ctx);
  489. }
  490. /* workerids is malloc-ed in starpu_sched_ctx_get_workers_list, don't forget to free it when
  491. you don't use it anymore */
  492. free(workerids);
  493. STARPU_PTHREAD_MUTEX_UNLOCK(&changing_ctx_mutex[sched_ctx_id]);
  494. return;
  495. }
  496. /* called after the workers are terminated so we don't have anything else to do but free the memory*/
  497. void _starpu_delete_all_sched_ctxs()
  498. {
  499. unsigned i;
  500. for(i = 0; i < STARPU_NMAX_SCHED_CTXS; i++)
  501. {
  502. struct _starpu_sched_ctx *sched_ctx = _starpu_get_sched_ctx_struct(i);
  503. STARPU_PTHREAD_MUTEX_LOCK(&changing_ctx_mutex[i]);
  504. if(sched_ctx->id != STARPU_NMAX_SCHED_CTXS)
  505. {
  506. _starpu_sched_ctx_free_scheduling_data(sched_ctx);
  507. _starpu_barrier_counter_destroy(&sched_ctx->tasks_barrier);
  508. _starpu_delete_sched_ctx(sched_ctx);
  509. }
  510. STARPU_PTHREAD_MUTEX_UNLOCK(&changing_ctx_mutex[i]);
  511. STARPU_PTHREAD_MUTEX_DESTROY(&changing_ctx_mutex[i]);
  512. }
  513. return;
  514. }
  515. static void _starpu_check_workers(int *workerids, int nworkers)
  516. {
  517. struct _starpu_machine_config *config = (struct _starpu_machine_config *)_starpu_get_machine_config();
  518. int nworkers_conf = config->topology.nworkers;
  519. int i;
  520. for(i = 0; i < nworkers; i++)
  521. {
  522. /* take care the user does not ask for a resource that does not exist */
  523. STARPU_ASSERT_MSG(workerids[i] >= 0 && workerids[i] <= nworkers_conf, "requested to add workerid = %d, but that is beyond the range 0 to %d", workerids[i], nworkers_conf);
  524. }
  525. }
  526. void _starpu_fetch_tasks_from_empty_ctx_list(struct _starpu_sched_ctx *sched_ctx)
  527. {
  528. unsigned unlocked = 0;
  529. STARPU_PTHREAD_MUTEX_LOCK(&sched_ctx->empty_ctx_mutex);
  530. if(starpu_task_list_empty(&sched_ctx->empty_ctx_tasks))
  531. {
  532. STARPU_PTHREAD_MUTEX_UNLOCK(&sched_ctx->empty_ctx_mutex);
  533. return;
  534. }
  535. else
  536. /* you're not suppose to get here if you deleted the context
  537. so no point in having the mutex locked */
  538. STARPU_PTHREAD_MUTEX_UNLOCK(&changing_ctx_mutex[sched_ctx->id]);
  539. while(!starpu_task_list_empty(&sched_ctx->empty_ctx_tasks))
  540. {
  541. if(unlocked)
  542. STARPU_PTHREAD_MUTEX_LOCK(&sched_ctx->empty_ctx_mutex);
  543. struct starpu_task *old_task = starpu_task_list_pop_back(&sched_ctx->empty_ctx_tasks);
  544. unlocked = 1;
  545. STARPU_PTHREAD_MUTEX_UNLOCK(&sched_ctx->empty_ctx_mutex);
  546. if(old_task == &stop_submission_task)
  547. break;
  548. int ret = _starpu_push_task_to_workers(old_task);
  549. /* if we should stop poping from empty ctx tasks */
  550. if(ret == -EAGAIN) break;
  551. }
  552. if(!unlocked)
  553. STARPU_PTHREAD_MUTEX_UNLOCK(&sched_ctx->empty_ctx_mutex);
  554. /* leave the mutex as it was to avoid pbs in the caller function */
  555. STARPU_PTHREAD_MUTEX_LOCK(&changing_ctx_mutex[sched_ctx->id]);
  556. return;
  557. }
  558. void starpu_sched_ctx_add_workers(int *workers_to_add, int nworkers_to_add, unsigned sched_ctx_id)
  559. {
  560. struct _starpu_sched_ctx *sched_ctx = _starpu_get_sched_ctx_struct(sched_ctx_id);
  561. int added_workers[nworkers_to_add];
  562. int n_added_workers = 0;
  563. STARPU_PTHREAD_MUTEX_LOCK(&changing_ctx_mutex[sched_ctx_id]);
  564. STARPU_ASSERT(workers_to_add != NULL && nworkers_to_add > 0);
  565. _starpu_check_workers(workers_to_add, nworkers_to_add);
  566. /* if the context has not already been deleted */
  567. if(sched_ctx->id != STARPU_NMAX_SCHED_CTXS)
  568. {
  569. _starpu_add_workers_to_sched_ctx(sched_ctx, workers_to_add, nworkers_to_add, added_workers, &n_added_workers);
  570. if(n_added_workers > 0)
  571. {
  572. _starpu_update_workers_with_ctx(added_workers, n_added_workers, sched_ctx->id);
  573. }
  574. _starpu_fetch_tasks_from_empty_ctx_list(sched_ctx);
  575. }
  576. STARPU_PTHREAD_MUTEX_UNLOCK(&changing_ctx_mutex[sched_ctx_id]);
  577. return;
  578. }
  579. void starpu_sched_ctx_remove_workers(int *workers_to_remove, int nworkers_to_remove, unsigned sched_ctx_id)
  580. {
  581. struct _starpu_sched_ctx *sched_ctx = _starpu_get_sched_ctx_struct(sched_ctx_id);
  582. int removed_workers[sched_ctx->workers->nworkers];
  583. int n_removed_workers = 0;
  584. _starpu_check_workers(workers_to_remove, nworkers_to_remove);
  585. STARPU_PTHREAD_MUTEX_LOCK(&changing_ctx_mutex[sched_ctx_id]);
  586. /* if the context has not already been deleted */
  587. if(sched_ctx->id != STARPU_NMAX_SCHED_CTXS)
  588. {
  589. _starpu_remove_workers_from_sched_ctx(sched_ctx, workers_to_remove, nworkers_to_remove, removed_workers, &n_removed_workers);
  590. if(n_removed_workers > 0)
  591. _starpu_update_workers_without_ctx(removed_workers, n_removed_workers, sched_ctx->id, 0);
  592. }
  593. STARPU_PTHREAD_MUTEX_UNLOCK(&changing_ctx_mutex[sched_ctx_id]);
  594. return;
  595. }
  596. /* unused sched_ctx have the id STARPU_NMAX_SCHED_CTXS */
  597. void _starpu_init_all_sched_ctxs(struct _starpu_machine_config *config)
  598. {
  599. starpu_pthread_key_create(&sched_ctx_key, NULL);
  600. unsigned i;
  601. for(i = 0; i < STARPU_NMAX_SCHED_CTXS; i++)
  602. config->sched_ctxs[i].id = STARPU_NMAX_SCHED_CTXS;
  603. char* max_time_on_ctx = getenv("STARPU_MAX_TIME_ON_CTX");
  604. if (max_time_on_ctx != NULL)
  605. max_time_worker_on_ctx = atof(max_time_on_ctx);
  606. return;
  607. }
  608. /* sched_ctx aren't necessarly one next to another */
  609. /* for eg when we remove one its place is free */
  610. /* when we add new one we reuse its place */
  611. static unsigned _starpu_get_first_free_sched_ctx(struct _starpu_machine_config *config)
  612. {
  613. unsigned i;
  614. for(i = 0; i < STARPU_NMAX_SCHED_CTXS; i++)
  615. if(config->sched_ctxs[i].id == STARPU_NMAX_SCHED_CTXS)
  616. return i;
  617. STARPU_ASSERT(0);
  618. return STARPU_NMAX_SCHED_CTXS;
  619. }
  620. int _starpu_wait_for_all_tasks_of_sched_ctx(unsigned sched_ctx_id)
  621. {
  622. struct _starpu_sched_ctx *sched_ctx = _starpu_get_sched_ctx_struct(sched_ctx_id);
  623. if (STARPU_UNLIKELY(!_starpu_worker_may_perform_blocking_calls()))
  624. return -EDEADLK;
  625. return _starpu_barrier_counter_wait_for_empty_counter(&sched_ctx->tasks_barrier);
  626. }
  627. void _starpu_decrement_nsubmitted_tasks_of_sched_ctx(unsigned sched_ctx_id)
  628. {
  629. struct _starpu_sched_ctx *sched_ctx = _starpu_get_sched_ctx_struct(sched_ctx_id);
  630. int finished = _starpu_barrier_counter_decrement_until_empty_counter(&sched_ctx->tasks_barrier);
  631. /*when finished decrementing the tasks if the user signaled he will not submit tasks anymore
  632. we can move all its workers to the inheritor context */
  633. if(finished && sched_ctx->inheritor != STARPU_NMAX_SCHED_CTXS)
  634. {
  635. STARPU_PTHREAD_MUTEX_LOCK(&finished_submit_mutex);
  636. if(sched_ctx->finished_submit)
  637. {
  638. STARPU_PTHREAD_MUTEX_UNLOCK(&finished_submit_mutex);
  639. /* take care the context is not deleted or changed at the same time */
  640. STARPU_PTHREAD_MUTEX_LOCK(&changing_ctx_mutex[sched_ctx_id]);
  641. if(sched_ctx->id != STARPU_NMAX_SCHED_CTXS)
  642. {
  643. int *workerids = NULL;
  644. unsigned nworkers = starpu_sched_ctx_get_workers_list(sched_ctx->id, &workerids);
  645. if(nworkers > 0)
  646. {
  647. starpu_sched_ctx_add_workers(workerids, nworkers, sched_ctx->inheritor);
  648. free(workerids);
  649. }
  650. }
  651. STARPU_PTHREAD_MUTEX_UNLOCK(&changing_ctx_mutex[sched_ctx_id]);
  652. return;
  653. }
  654. STARPU_PTHREAD_MUTEX_UNLOCK(&finished_submit_mutex);
  655. }
  656. return;
  657. }
  658. void _starpu_increment_nsubmitted_tasks_of_sched_ctx(unsigned sched_ctx_id)
  659. {
  660. struct _starpu_sched_ctx *sched_ctx = _starpu_get_sched_ctx_struct(sched_ctx_id);
  661. _starpu_barrier_counter_increment(&sched_ctx->tasks_barrier);
  662. }
  663. void starpu_sched_ctx_set_context(unsigned *sched_ctx)
  664. {
  665. starpu_pthread_setspecific(sched_ctx_key, (void*)sched_ctx);
  666. }
  667. unsigned starpu_sched_ctx_get_context()
  668. {
  669. unsigned *sched_ctx = (unsigned*)starpu_pthread_getspecific(sched_ctx_key);
  670. if(sched_ctx == NULL)
  671. return STARPU_NMAX_SCHED_CTXS;
  672. STARPU_ASSERT(*sched_ctx < STARPU_NMAX_SCHED_CTXS);
  673. return *sched_ctx;
  674. }
  675. void starpu_sched_ctx_notify_hypervisor_exists()
  676. {
  677. with_hypervisor = 1;
  678. }
  679. unsigned starpu_sched_ctx_check_if_hypervisor_exists()
  680. {
  681. return with_hypervisor;
  682. }
  683. unsigned _starpu_get_nsched_ctxs()
  684. {
  685. struct _starpu_machine_config *config = (struct _starpu_machine_config *)_starpu_get_machine_config();
  686. return config->topology.nsched_ctxs;
  687. }
  688. void starpu_sched_ctx_set_policy_data(unsigned sched_ctx_id, void* policy_data)
  689. {
  690. struct _starpu_sched_ctx *sched_ctx = _starpu_get_sched_ctx_struct(sched_ctx_id);
  691. sched_ctx->policy_data = policy_data;
  692. }
  693. void* starpu_sched_ctx_get_policy_data(unsigned sched_ctx_id)
  694. {
  695. struct _starpu_sched_ctx *sched_ctx = _starpu_get_sched_ctx_struct(sched_ctx_id);
  696. return sched_ctx->policy_data;
  697. }
  698. struct starpu_worker_collection* starpu_sched_ctx_create_worker_collection(unsigned sched_ctx_id, enum starpu_worker_collection_type worker_collection_type)
  699. {
  700. struct _starpu_sched_ctx *sched_ctx = _starpu_get_sched_ctx_struct(sched_ctx_id);
  701. sched_ctx->workers = (struct starpu_worker_collection*)malloc(sizeof(struct starpu_worker_collection));
  702. switch(worker_collection_type)
  703. {
  704. case STARPU_WORKER_LIST:
  705. sched_ctx->workers->has_next = worker_list.has_next;
  706. sched_ctx->workers->get_next = worker_list.get_next;
  707. sched_ctx->workers->add = worker_list.add;
  708. sched_ctx->workers->remove = worker_list.remove;
  709. sched_ctx->workers->init = worker_list.init;
  710. sched_ctx->workers->deinit = worker_list.deinit;
  711. sched_ctx->workers->init_iterator = worker_list.init_iterator;
  712. sched_ctx->workers->type = STARPU_WORKER_LIST;
  713. break;
  714. }
  715. return sched_ctx->workers;
  716. }
  717. unsigned starpu_sched_ctx_get_workers_list(unsigned sched_ctx_id, int **workerids)
  718. {
  719. struct _starpu_sched_ctx *sched_ctx = _starpu_get_sched_ctx_struct(sched_ctx_id);
  720. struct starpu_worker_collection *workers = sched_ctx->workers;
  721. *workerids = (int*)malloc(workers->nworkers*sizeof(int));
  722. int worker;
  723. unsigned nworkers = 0;
  724. struct starpu_sched_ctx_iterator it;
  725. if(workers->init_iterator)
  726. workers->init_iterator(workers, &it);
  727. while(workers->has_next(workers, &it))
  728. {
  729. worker = workers->get_next(workers, &it);
  730. (*workerids)[nworkers++] = worker;
  731. }
  732. return nworkers;
  733. }
  734. void starpu_sched_ctx_delete_worker_collection(unsigned sched_ctx_id)
  735. {
  736. struct _starpu_sched_ctx *sched_ctx = _starpu_get_sched_ctx_struct(sched_ctx_id);
  737. sched_ctx->workers->deinit(sched_ctx->workers);
  738. free(sched_ctx->workers);
  739. }
  740. struct starpu_worker_collection* starpu_sched_ctx_get_worker_collection(unsigned sched_ctx_id)
  741. {
  742. struct _starpu_sched_ctx *sched_ctx = _starpu_get_sched_ctx_struct(sched_ctx_id);
  743. return sched_ctx->workers;
  744. }
  745. int starpu_get_workers_of_sched_ctx(unsigned sched_ctx_id, int *pus, enum starpu_worker_archtype arch)
  746. {
  747. struct _starpu_sched_ctx *sched_ctx = _starpu_get_sched_ctx_struct(sched_ctx_id);
  748. struct starpu_worker_collection *workers = sched_ctx->workers;
  749. int worker;
  750. int npus = 0;
  751. struct starpu_sched_ctx_iterator it;
  752. if(workers->init_iterator)
  753. workers->init_iterator(workers, &it);
  754. while(workers->has_next(workers, &it))
  755. {
  756. worker = workers->get_next(workers, &it);
  757. enum starpu_worker_archtype curr_arch = starpu_worker_get_type(worker);
  758. if(curr_arch == arch)
  759. pus[npus++] = worker;
  760. }
  761. return npus;
  762. }
  763. starpu_pthread_mutex_t* _starpu_sched_ctx_get_changing_ctx_mutex(unsigned sched_ctx_id)
  764. {
  765. return &changing_ctx_mutex[sched_ctx_id];
  766. }
  767. unsigned starpu_sched_ctx_get_nworkers(unsigned sched_ctx_id)
  768. {
  769. struct _starpu_sched_ctx *sched_ctx = _starpu_get_sched_ctx_struct(sched_ctx_id);
  770. if(sched_ctx != NULL)
  771. return sched_ctx->workers->nworkers;
  772. else
  773. return 0;
  774. }
  775. unsigned starpu_sched_ctx_get_nshared_workers(unsigned sched_ctx_id, unsigned sched_ctx_id2)
  776. {
  777. struct _starpu_sched_ctx *sched_ctx = _starpu_get_sched_ctx_struct(sched_ctx_id);
  778. struct _starpu_sched_ctx *sched_ctx2 = _starpu_get_sched_ctx_struct(sched_ctx_id2);
  779. struct starpu_worker_collection *workers = sched_ctx->workers;
  780. struct starpu_worker_collection *workers2 = sched_ctx2->workers;
  781. int worker, worker2;
  782. int shared_workers = 0;
  783. struct starpu_sched_ctx_iterator it1, it2;
  784. if(workers->init_iterator)
  785. workers->init_iterator(workers, &it1);
  786. if(workers2->init_iterator)
  787. workers2->init_iterator(workers2, &it2);
  788. while(workers->has_next(workers, &it1))
  789. {
  790. worker = workers->get_next(workers, &it1);
  791. while(workers2->has_next(workers2, &it2))
  792. {
  793. worker2 = workers2->get_next(workers2, &it2);
  794. if(worker == worker2)
  795. shared_workers++;
  796. }
  797. }
  798. return shared_workers;
  799. }
  800. unsigned starpu_sched_ctx_contains_worker(int workerid, unsigned sched_ctx_id)
  801. {
  802. struct _starpu_sched_ctx *sched_ctx = _starpu_get_sched_ctx_struct(sched_ctx_id);
  803. struct starpu_worker_collection *workers = sched_ctx->workers;
  804. int worker;
  805. struct starpu_sched_ctx_iterator it;
  806. if(workers->init_iterator)
  807. workers->init_iterator(workers, &it);
  808. while(workers->has_next(workers, &it))
  809. {
  810. worker = workers->get_next(workers, &it);
  811. if(worker == workerid)
  812. return 1;
  813. }
  814. return 0;
  815. }
  816. unsigned starpu_sched_ctx_contains_type_of_worker(enum starpu_worker_archtype arch, unsigned sched_ctx_id)
  817. {
  818. struct starpu_worker_collection *workers = starpu_sched_ctx_get_worker_collection(sched_ctx_id);
  819. int worker;
  820. struct starpu_sched_ctx_iterator it;
  821. if(workers->init_iterator)
  822. workers->init_iterator(workers, &it);
  823. while(workers->has_next(workers, &it))
  824. {
  825. worker = workers->get_next(workers, &it);
  826. enum starpu_worker_archtype curr_arch = starpu_worker_get_type(worker);
  827. if(curr_arch == arch)
  828. return 1;
  829. }
  830. return 0;
  831. }
  832. unsigned _starpu_worker_belongs_to_a_sched_ctx(int workerid, unsigned sched_ctx_id)
  833. {
  834. struct _starpu_machine_config *config = (struct _starpu_machine_config *)_starpu_get_machine_config();
  835. int i;
  836. struct _starpu_sched_ctx *sched_ctx = NULL;
  837. for(i = 0; i < STARPU_NMAX_SCHED_CTXS; i++)
  838. {
  839. sched_ctx = &config->sched_ctxs[i];
  840. if(sched_ctx && sched_ctx->id != STARPU_NMAX_SCHED_CTXS && sched_ctx->id != sched_ctx_id)
  841. if(starpu_sched_ctx_contains_worker(workerid, sched_ctx->id))
  842. return 1;
  843. }
  844. return 0;
  845. }
  846. unsigned starpu_sched_ctx_overlapping_ctxs_on_worker(int workerid)
  847. {
  848. struct _starpu_worker *worker = _starpu_get_worker_struct(workerid);
  849. return worker->nsched_ctxs > 1;
  850. }
  851. unsigned starpu_sched_ctx_is_ctxs_turn(int workerid, unsigned sched_ctx_id)
  852. {
  853. if(max_time_worker_on_ctx == -1.0) return 1;
  854. struct _starpu_worker *worker = _starpu_get_worker_struct(workerid);
  855. return worker->active_ctx == sched_ctx_id;
  856. }
  857. void starpu_sched_ctx_set_turn_to_other_ctx(int workerid, unsigned sched_ctx_id)
  858. {
  859. struct _starpu_worker *worker = _starpu_get_worker_struct(workerid);
  860. struct _starpu_sched_ctx *other_sched_ctx = NULL;
  861. struct _starpu_sched_ctx *active_sched_ctx = NULL;
  862. struct _starpu_sched_ctx_list *l = NULL;
  863. for (l = worker->sched_ctx_list; l; l = l->next)
  864. {
  865. other_sched_ctx = _starpu_get_sched_ctx_struct(l->sched_ctx);
  866. if(other_sched_ctx != NULL && other_sched_ctx->id != STARPU_NMAX_SCHED_CTXS &&
  867. other_sched_ctx->id != 0 && other_sched_ctx->id != sched_ctx_id)
  868. {
  869. worker->active_ctx = other_sched_ctx->id;
  870. active_sched_ctx = other_sched_ctx;
  871. break;
  872. }
  873. }
  874. if(active_sched_ctx != NULL && worker->active_ctx != sched_ctx_id)
  875. {
  876. _starpu_fetch_tasks_from_empty_ctx_list(active_sched_ctx);
  877. }
  878. }
  879. double starpu_sched_ctx_get_max_time_worker_on_ctx(void)
  880. {
  881. return max_time_worker_on_ctx;
  882. }
  883. void starpu_sched_ctx_set_inheritor(unsigned sched_ctx_id, unsigned inheritor)
  884. {
  885. STARPU_ASSERT(inheritor < STARPU_NMAX_SCHED_CTXS);
  886. struct _starpu_sched_ctx *sched_ctx = _starpu_get_sched_ctx_struct(sched_ctx_id);
  887. sched_ctx->inheritor = inheritor;
  888. return;
  889. }
  890. void starpu_sched_ctx_finished_submit(unsigned sched_ctx_id)
  891. {
  892. struct _starpu_sched_ctx *sched_ctx = _starpu_get_sched_ctx_struct(sched_ctx_id);
  893. STARPU_PTHREAD_MUTEX_LOCK(&finished_submit_mutex);
  894. sched_ctx->finished_submit = 1;
  895. STARPU_PTHREAD_MUTEX_UNLOCK(&finished_submit_mutex);
  896. return;
  897. }
  898. #ifdef STARPU_USE_SC_HYPERVISOR
  899. void _starpu_sched_ctx_call_poped_task_cb(int workerid, struct starpu_task *task, size_t data_size, uint32_t footprint)
  900. {
  901. struct _starpu_sched_ctx *sched_ctx = _starpu_get_sched_ctx_struct(task->sched_ctx);
  902. if(sched_ctx != NULL && task->sched_ctx != _starpu_get_initial_sched_ctx()->id && task->sched_ctx != STARPU_NMAX_SCHED_CTXS
  903. && sched_ctx->perf_counters != NULL)
  904. sched_ctx->perf_counters->notify_poped_task(task->sched_ctx, workerid, task, data_size, footprint);
  905. }
  906. void starpu_sched_ctx_call_pushed_task_cb(int workerid, unsigned sched_ctx_id)
  907. {
  908. struct _starpu_sched_ctx *sched_ctx = _starpu_get_sched_ctx_struct(sched_ctx_id);
  909. if(sched_ctx != NULL && sched_ctx_id != _starpu_get_initial_sched_ctx()->id && sched_ctx_id != STARPU_NMAX_SCHED_CTXS
  910. && sched_ctx->perf_counters != NULL)
  911. sched_ctx->perf_counters->notify_pushed_task(sched_ctx_id, workerid);
  912. }
  913. #endif //STARPU_USE_SC_HYPERVISOR
  914. int starpu_sched_get_min_priority(void)
  915. {
  916. return starpu_sched_ctx_get_min_priority(_starpu_get_initial_sched_ctx()->id);
  917. }
  918. int starpu_sched_get_max_priority(void)
  919. {
  920. return starpu_sched_ctx_get_max_priority(_starpu_get_initial_sched_ctx()->id);
  921. }
  922. int starpu_sched_set_min_priority(int min_prio)
  923. {
  924. return starpu_sched_ctx_set_min_priority(_starpu_get_initial_sched_ctx()->id, min_prio);
  925. }
  926. int starpu_sched_set_max_priority(int max_prio)
  927. {
  928. return starpu_sched_ctx_set_max_priority(_starpu_get_initial_sched_ctx()->id, max_prio);
  929. }
  930. int starpu_sched_ctx_get_min_priority(unsigned sched_ctx_id)
  931. {
  932. struct _starpu_sched_ctx *sched_ctx = _starpu_get_sched_ctx_struct(sched_ctx_id);
  933. return sched_ctx->min_priority;
  934. }
  935. int starpu_sched_ctx_get_max_priority(unsigned sched_ctx_id)
  936. {
  937. struct _starpu_sched_ctx *sched_ctx = _starpu_get_sched_ctx_struct(sched_ctx_id);
  938. return sched_ctx->max_priority;
  939. }
  940. int starpu_sched_ctx_set_min_priority(unsigned sched_ctx_id, int min_prio)
  941. {
  942. struct _starpu_sched_ctx *sched_ctx = _starpu_get_sched_ctx_struct(sched_ctx_id);
  943. sched_ctx->min_priority = min_prio;
  944. return 0;
  945. }
  946. int starpu_sched_ctx_set_max_priority(unsigned sched_ctx_id, int max_prio)
  947. {
  948. struct _starpu_sched_ctx *sched_ctx = _starpu_get_sched_ctx_struct(sched_ctx_id);
  949. sched_ctx->max_priority = max_prio;
  950. return 0;
  951. }
  952. static void _starpu_sched_ctx_bind_thread_to_ctx_cpus(unsigned sched_ctx_id)
  953. {
  954. struct _starpu_sched_ctx *sched_ctx = _starpu_get_sched_ctx_struct(sched_ctx_id);
  955. struct _starpu_machine_config *config = _starpu_get_machine_config();
  956. #ifdef STARPU_HAVE_HWLOC
  957. const struct hwloc_topology_support *support = hwloc_topology_get_support(config->topology.hwtopology);
  958. if (support->cpubind->set_thisthread_cpubind)
  959. {
  960. hwloc_bitmap_t set = sched_ctx->hwloc_workers_set;
  961. int ret;
  962. ret = hwloc_set_cpubind (config->topology.hwtopology, set,
  963. HWLOC_CPUBIND_THREAD);
  964. if (ret)
  965. {
  966. perror("binding thread");
  967. STARPU_ABORT();
  968. }
  969. }
  970. #else
  971. #warning no sched ctx CPU binding support
  972. #endif
  973. return;
  974. }
  975. void _starpu_sched_ctx_rebind_thread_to_its_cpu(unsigned cpuid)
  976. {
  977. struct _starpu_machine_config *config = _starpu_get_machine_config();
  978. #ifdef STARPU_SIMGRID
  979. return;
  980. #endif
  981. if (starpu_get_env_number("STARPU_WORKERS_NOBIND") > 0)
  982. return;
  983. #ifdef STARPU_HAVE_HWLOC
  984. const struct hwloc_topology_support *support = hwloc_topology_get_support (config->topology.hwtopology);
  985. if (support->cpubind->set_thisthread_cpubind)
  986. {
  987. hwloc_obj_t obj = hwloc_get_obj_by_depth (config->topology.hwtopology,
  988. config->cpu_depth, cpuid);
  989. hwloc_bitmap_t set = obj->cpuset;
  990. int ret;
  991. hwloc_bitmap_singlify(set);
  992. ret = hwloc_set_cpubind (config->topology.hwtopology, set,
  993. HWLOC_CPUBIND_THREAD);
  994. if (ret)
  995. {
  996. perror("hwloc_set_cpubind");
  997. STARPU_ABORT();
  998. }
  999. }
  1000. #elif defined(HAVE_PTHREAD_SETAFFINITY_NP) && defined(__linux__)
  1001. int ret;
  1002. /* fix the thread on the correct cpu */
  1003. cpu_set_t aff_mask;
  1004. CPU_ZERO(&aff_mask);
  1005. CPU_SET(cpuid, &aff_mask);
  1006. starpu_pthread_t self = pthread_self();
  1007. ret = pthread_setaffinity_np(self, sizeof(aff_mask), &aff_mask);
  1008. if (ret)
  1009. {
  1010. perror("binding thread");
  1011. STARPU_ABORT();
  1012. }
  1013. #elif defined(__MINGW32__) || defined(__CYGWIN__)
  1014. DWORD mask = 1 << cpuid;
  1015. if (!SetThreadAffinityMask(GetCurrentThread(), mask))
  1016. {
  1017. _STARPU_ERROR("SetThreadMaskAffinity(%lx) failed\n", mask);
  1018. }
  1019. #else
  1020. #warning no CPU binding support
  1021. #endif
  1022. }
  1023. static void _starpu_sched_ctx_get_workers_to_sleep(unsigned sched_ctx_id)
  1024. {
  1025. struct _starpu_sched_ctx *sched_ctx = _starpu_get_sched_ctx_struct(sched_ctx_id);
  1026. struct starpu_worker_collection *workers = sched_ctx->workers;
  1027. struct starpu_sched_ctx_iterator it;
  1028. struct _starpu_worker *worker = NULL;
  1029. if(workers->init_iterator)
  1030. workers->init_iterator(workers, &it);
  1031. while(workers->has_next(workers, &it))
  1032. {
  1033. worker = _starpu_get_worker_struct(workers->get_next(workers, &it));
  1034. STARPU_PTHREAD_MUTEX_LOCK(&worker->sched_mutex);
  1035. worker->parallel_sect = 1;
  1036. STARPU_PTHREAD_MUTEX_UNLOCK(&worker->sched_mutex);
  1037. }
  1038. while(workers->has_next(workers, &it))
  1039. {
  1040. workers->get_next(workers, &it);
  1041. sem_wait(&sched_ctx->parallel_code_sem);
  1042. }
  1043. return;
  1044. }
  1045. void _starpu_sched_ctx_signal_worker_blocked(int workerid)
  1046. {
  1047. struct _starpu_worker *worker = _starpu_get_worker_struct(workerid);
  1048. struct _starpu_sched_ctx *sched_ctx = NULL;
  1049. struct _starpu_sched_ctx_list *l = NULL;
  1050. for (l = worker->sched_ctx_list; l; l = l->next)
  1051. {
  1052. sched_ctx = _starpu_get_sched_ctx_struct(l->sched_ctx);
  1053. if(sched_ctx->id != 0)
  1054. sem_post(&sched_ctx->parallel_code_sem);
  1055. }
  1056. return;
  1057. }
  1058. static void _starpu_sched_ctx_wake_up_workers(unsigned sched_ctx_id)
  1059. {
  1060. struct _starpu_sched_ctx *sched_ctx = _starpu_get_sched_ctx_struct(sched_ctx_id);
  1061. struct starpu_worker_collection *workers = sched_ctx->workers;
  1062. struct starpu_sched_ctx_iterator it;
  1063. struct _starpu_worker *worker = NULL;
  1064. if(workers->init_iterator)
  1065. workers->init_iterator(workers, &it);
  1066. while(workers->has_next(workers, &it))
  1067. {
  1068. worker = _starpu_get_worker_struct(workers->get_next(workers, &it));
  1069. STARPU_PTHREAD_MUTEX_LOCK(&worker->parallel_sect_mutex);
  1070. STARPU_PTHREAD_COND_SIGNAL(&worker->parallel_sect_cond);
  1071. STARPU_PTHREAD_MUTEX_UNLOCK(&worker->parallel_sect_mutex);
  1072. }
  1073. return;
  1074. }
  1075. void* starpu_sched_ctx_exec_parallel_code(void* (*func)(void*), void* param, unsigned sched_ctx_id)
  1076. {
  1077. /* get starpu workers to sleep */
  1078. _starpu_sched_ctx_get_workers_to_sleep(sched_ctx_id);
  1079. /* bind current thread on all workers of the context */
  1080. _starpu_sched_ctx_bind_thread_to_ctx_cpus(sched_ctx_id);
  1081. /* execute parallel code */
  1082. void* ret = func(param);
  1083. /* wake up starpu workers */
  1084. _starpu_sched_ctx_wake_up_workers(sched_ctx_id);
  1085. return ret;
  1086. }