sched_policy.c 22 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593594595596597598599600601602603604605606607608609610611612613614615616617618619620621622623624625626627628629630631632633634635636637638639640641642643644645646647648649650651652653654655656657658659660661662663664665666667668669670671672673674675676677678679680681682683684685686687688689690691692693694695696697698699700701702703704705706707708709710711712713714715716717718719720721722723724725726727728729730731732733734735736737738739740741742743744745746747748749750751752753754755756
  1. /* StarPU --- Runtime system for heterogeneous multicore architectures.
  2. *
  3. * Copyright (C) 2010-2013 Université de Bordeaux 1
  4. * Copyright (C) 2010-2013 Centre National de la Recherche Scientifique
  5. * Copyright (C) 2011 INRIA
  6. *
  7. * StarPU is free software; you can redistribute it and/or modify
  8. * it under the terms of the GNU Lesser General Public License as published by
  9. * the Free Software Foundation; either version 2.1 of the License, or (at
  10. * your option) any later version.
  11. *
  12. * StarPU is distributed in the hope that it will be useful, but
  13. * WITHOUT ANY WARRANTY; without even the implied warranty of
  14. * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE.
  15. *
  16. * See the GNU Lesser General Public License in COPYING.LGPL for more details.
  17. */
  18. #include <pthread.h>
  19. #include <starpu.h>
  20. #include <common/config.h>
  21. #include <common/utils.h>
  22. #include <core/sched_policy.h>
  23. #include <profiling/profiling.h>
  24. #include <common/barrier.h>
  25. #include <core/debug.h>
  26. #include <core/parallel_task.h>
  27. static int use_prefetch = 0;
  28. int starpu_get_prefetch_flag(void)
  29. {
  30. return use_prefetch;
  31. }
  32. static struct starpu_sched_policy *predefined_policies[] =
  33. {
  34. &_starpu_sched_eager_policy,
  35. &_starpu_sched_prio_policy,
  36. &_starpu_sched_random_policy,
  37. &_starpu_sched_ws_policy,
  38. &_starpu_sched_dm_policy,
  39. &_starpu_sched_dmda_policy,
  40. &_starpu_sched_dmda_ready_policy,
  41. &_starpu_sched_dmda_sorted_policy,
  42. &_starpu_sched_parallel_heft_policy,
  43. &_starpu_sched_peager_policy,
  44. NULL
  45. };
  46. struct starpu_sched_policy **starpu_sched_get_predefined_policies()
  47. {
  48. return predefined_policies;
  49. }
  50. struct starpu_sched_policy *_starpu_get_sched_policy(struct _starpu_sched_ctx *sched_ctx)
  51. {
  52. return sched_ctx->sched_policy;
  53. }
  54. /*
  55. * Methods to initialize the scheduling policy
  56. */
  57. static void load_sched_policy(struct starpu_sched_policy *sched_policy, struct _starpu_sched_ctx *sched_ctx)
  58. {
  59. STARPU_ASSERT(sched_policy);
  60. #ifdef STARPU_VERBOSE
  61. if (sched_policy->policy_name)
  62. {
  63. if (sched_policy->policy_description)
  64. _STARPU_DEBUG("Use %s scheduler (%s)\n", sched_policy->policy_name, sched_policy->policy_description);
  65. else
  66. _STARPU_DEBUG("Use %s scheduler \n", sched_policy->policy_name);
  67. }
  68. #endif
  69. struct starpu_sched_policy *policy = sched_ctx->sched_policy;
  70. memcpy(policy, sched_policy, sizeof(*policy));
  71. }
  72. static struct starpu_sched_policy *find_sched_policy_from_name(const char *policy_name)
  73. {
  74. if (!policy_name)
  75. return NULL;
  76. if (strncmp(policy_name, "heft", 5) == 0)
  77. {
  78. _STARPU_DISP("Warning: heft is now called \"dmda\".\n");
  79. return &_starpu_sched_dmda_policy;
  80. }
  81. struct starpu_sched_policy **policy;
  82. for(policy=predefined_policies ; *policy!=NULL ; policy++)
  83. {
  84. struct starpu_sched_policy *p = *policy;
  85. if (p->policy_name)
  86. {
  87. if (strcmp(policy_name, p->policy_name) == 0)
  88. {
  89. /* we found a policy with the requested name */
  90. return p;
  91. }
  92. }
  93. }
  94. fprintf(stderr, "Warning: scheduling policy \"%s\" was not found, try \"help\" to get a list\n", policy_name);
  95. /* nothing was found */
  96. return NULL;
  97. }
  98. static void display_sched_help_message(void)
  99. {
  100. const char *sched_env = getenv("STARPU_SCHED");
  101. if (sched_env && (strcmp(sched_env, "help") == 0))
  102. {
  103. /* display the description of all predefined policies */
  104. struct starpu_sched_policy **policy;
  105. fprintf(stderr, "STARPU_SCHED can be either of\n");
  106. for(policy=predefined_policies ; *policy!=NULL ; policy++)
  107. {
  108. struct starpu_sched_policy *p = *policy;
  109. fprintf(stderr, "%s\t-> %s\n", p->policy_name, p->policy_description);
  110. }
  111. }
  112. }
  113. static struct starpu_sched_policy *select_sched_policy(struct _starpu_machine_config *config, const char *required_policy)
  114. {
  115. struct starpu_sched_policy *selected_policy = NULL;
  116. struct starpu_conf *user_conf = config->conf;
  117. if(required_policy)
  118. selected_policy = find_sched_policy_from_name(required_policy);
  119. /* First, we check whether the application explicitely gave a scheduling policy or not */
  120. if (!selected_policy && user_conf && (user_conf->sched_policy))
  121. return user_conf->sched_policy;
  122. /* Otherwise, we look if the application specified the name of a policy to load */
  123. const char *sched_pol_name;
  124. sched_pol_name = getenv("STARPU_SCHED");
  125. if (sched_pol_name == NULL && user_conf && user_conf->sched_policy_name)
  126. sched_pol_name = user_conf->sched_policy_name;
  127. if (!selected_policy && sched_pol_name)
  128. selected_policy = find_sched_policy_from_name(sched_pol_name);
  129. /* Perhaps there was no policy that matched the name */
  130. if (selected_policy)
  131. return selected_policy;
  132. /* If no policy was specified, we use the greedy policy as a default */
  133. return &_starpu_sched_eager_policy;
  134. }
  135. void _starpu_init_sched_policy(struct _starpu_machine_config *config, struct _starpu_sched_ctx *sched_ctx, const char *required_policy)
  136. {
  137. /* Perhaps we have to display some help */
  138. display_sched_help_message();
  139. /* Prefetch is activated by default */
  140. use_prefetch = starpu_get_env_number("STARPU_PREFETCH");
  141. if (use_prefetch == -1)
  142. use_prefetch = 1;
  143. /* Set calibrate flag */
  144. _starpu_set_calibrate_flag(config->conf->calibrate);
  145. struct starpu_sched_policy *selected_policy;
  146. selected_policy = select_sched_policy(config, required_policy);
  147. load_sched_policy(selected_policy, sched_ctx);
  148. sched_ctx->sched_policy->init_sched(sched_ctx->id);
  149. }
  150. void _starpu_deinit_sched_policy(struct _starpu_sched_ctx *sched_ctx)
  151. {
  152. struct starpu_sched_policy *policy = sched_ctx->sched_policy;
  153. if (policy->deinit_sched)
  154. policy->deinit_sched(sched_ctx->id);
  155. }
  156. /* Enqueue a task into the list of tasks explicitely attached to a worker. In
  157. * case workerid identifies a combined worker, a task will be enqueued into
  158. * each worker of the combination. */
  159. static int _starpu_push_task_on_specific_worker(struct starpu_task *task, int workerid)
  160. {
  161. int nbasic_workers = (int)starpu_worker_get_count();
  162. /* Is this a basic worker or a combined worker ? */
  163. int is_basic_worker = (workerid < nbasic_workers);
  164. unsigned memory_node;
  165. struct _starpu_worker *worker = NULL;
  166. struct _starpu_combined_worker *combined_worker = NULL;
  167. if (is_basic_worker)
  168. {
  169. worker = _starpu_get_worker_struct(workerid);
  170. memory_node = worker->memory_node;
  171. }
  172. else
  173. {
  174. combined_worker = _starpu_get_combined_worker_struct(workerid);
  175. memory_node = combined_worker->memory_node;
  176. }
  177. if (use_prefetch)
  178. starpu_prefetch_task_input_on_node(task, memory_node);
  179. /* if we push a task on a specific worker, notify all the sched_ctxs the worker belongs to */
  180. unsigned i;
  181. struct _starpu_sched_ctx *sched_ctx;
  182. for(i = 0; i < STARPU_NMAX_SCHED_CTXS; i++)
  183. {
  184. sched_ctx = worker->sched_ctx[i];
  185. if (sched_ctx != NULL && sched_ctx->sched_policy != NULL && sched_ctx->sched_policy->push_task_notify)
  186. sched_ctx->sched_policy->push_task_notify(task, workerid, sched_ctx->id);
  187. }
  188. #ifdef STARPU_USE_SCHED_CTX_HYPERVISOR
  189. starpu_sched_ctx_call_pushed_task_cb(workerid, task->sched_ctx);
  190. #endif //STARPU_USE_SCHED_CTX_HYPERVISOR
  191. if (is_basic_worker)
  192. {
  193. unsigned node = starpu_worker_get_memory_node(workerid);
  194. if (_starpu_task_uses_multiformat_handles(task))
  195. {
  196. for (i = 0; i < task->cl->nbuffers; i++)
  197. {
  198. struct starpu_task *conversion_task;
  199. starpu_data_handle_t handle;
  200. handle = task->handles[i];
  201. if (!_starpu_handle_needs_conversion_task(handle, node))
  202. continue;
  203. conversion_task = _starpu_create_conversion_task(handle, node);
  204. conversion_task->mf_skip = 1;
  205. conversion_task->execute_on_a_specific_worker = 1;
  206. conversion_task->workerid = workerid;
  207. _starpu_task_submit_conversion_task(conversion_task, workerid);
  208. //_STARPU_DEBUG("Pushing a conversion task\n");
  209. }
  210. for (i = 0; i < task->cl->nbuffers; i++)
  211. task->handles[i]->mf_node = node;
  212. }
  213. // if(task->sched_ctx != _starpu_get_initial_sched_ctx()->id)
  214. if(task->priority > 0)
  215. return _starpu_push_local_task(worker, task, 1);
  216. else
  217. return _starpu_push_local_task(worker, task, 0);
  218. }
  219. else
  220. {
  221. /* This is a combined worker so we create task aliases */
  222. int worker_size = combined_worker->worker_size;
  223. int *combined_workerid = combined_worker->combined_workerid;
  224. int ret = 0;
  225. struct _starpu_job *job = _starpu_get_job_associated_to_task(task);
  226. job->task_size = worker_size;
  227. job->combined_workerid = workerid;
  228. job->active_task_alias_count = 0;
  229. _STARPU_PTHREAD_BARRIER_INIT(&job->before_work_barrier, NULL, worker_size);
  230. _STARPU_PTHREAD_BARRIER_INIT(&job->after_work_barrier, NULL, worker_size);
  231. /* Note: we have to call that early, or else the task may have
  232. * disappeared already */
  233. _starpu_push_task_end(task);
  234. int j;
  235. for (j = 0; j < worker_size; j++)
  236. {
  237. struct starpu_task *alias = _starpu_create_task_alias(task);
  238. worker = _starpu_get_worker_struct(combined_workerid[j]);
  239. ret |= _starpu_push_local_task(worker, alias, 0);
  240. }
  241. return ret;
  242. }
  243. }
  244. static int _starpu_nworkers_able_to_execute_task(struct starpu_task *task, struct _starpu_sched_ctx *sched_ctx)
  245. {
  246. int worker = -1, nworkers = 0;
  247. struct starpu_sched_ctx_worker_collection *workers = sched_ctx->workers;
  248. struct starpu_sched_ctx_iterator it;
  249. if(workers->init_iterator)
  250. workers->init_iterator(workers, &it);
  251. while(workers->has_next(workers, &it))
  252. {
  253. worker = workers->get_next(workers, &it);
  254. if (starpu_worker_can_execute_task(worker, task, 0) && starpu_sched_ctx_is_ctxs_turn(worker, sched_ctx->id))
  255. nworkers++;
  256. }
  257. return nworkers;
  258. }
  259. /* the generic interface that call the proper underlying implementation */
  260. int _starpu_push_task(struct _starpu_job *j)
  261. {
  262. struct starpu_task *task = j->task;
  263. struct _starpu_sched_ctx *sched_ctx = _starpu_get_sched_ctx_struct(task->sched_ctx);
  264. unsigned nworkers = 0;
  265. int ret;
  266. _STARPU_LOG_IN();
  267. _STARPU_TRACE_JOB_PUSH(task, task->priority > 0);
  268. _starpu_increment_nready_tasks();
  269. task->status = STARPU_TASK_READY;
  270. #ifdef HAVE_AYUDAME_H
  271. if (AYU_event)
  272. {
  273. int id = -1;
  274. AYU_event(AYU_ADDTASKTOQUEUE, j->job_id, &id);
  275. }
  276. #endif
  277. /* if the context does not have any workers save the tasks in a temp list */
  278. if(!sched_ctx->is_initial_sched)
  279. {
  280. /*if there are workers in the ctx that are not able to execute tasks
  281. we consider the ctx empty */
  282. nworkers = _starpu_nworkers_able_to_execute_task(task, sched_ctx);
  283. if(nworkers == 0)
  284. {
  285. _STARPU_PTHREAD_MUTEX_LOCK(&sched_ctx->empty_ctx_mutex);
  286. starpu_task_list_push_front(&sched_ctx->empty_ctx_tasks, task);
  287. _STARPU_PTHREAD_MUTEX_UNLOCK(&sched_ctx->empty_ctx_mutex);
  288. return 0;
  289. }
  290. }
  291. /* in case there is no codelet associated to the task (that's a control
  292. * task), we directly execute its callback and enforce the
  293. * corresponding dependencies */
  294. if (task->cl == NULL)
  295. {
  296. _starpu_handle_job_termination(j);
  297. _STARPU_LOG_OUT_TAG("handle_job_termination");
  298. return 0;
  299. }
  300. ret = _starpu_push_task_to_workers(task);
  301. if (ret == -EAGAIN)
  302. /* pushed to empty context, that's fine */
  303. ret = 0;
  304. return ret;
  305. }
  306. int _starpu_push_task_to_workers(struct starpu_task *task)
  307. {
  308. struct _starpu_sched_ctx *sched_ctx = _starpu_get_sched_ctx_struct(task->sched_ctx);
  309. unsigned nworkers = 0;
  310. /* if the contexts still does not have workers put the task back to its place in
  311. the empty ctx list */
  312. if(!sched_ctx->is_initial_sched)
  313. {
  314. /*if there are workers in the ctx that are not able to execute tasks
  315. we consider the ctx empty */
  316. nworkers = _starpu_nworkers_able_to_execute_task(task, sched_ctx);
  317. if (nworkers == 0)
  318. {
  319. _STARPU_PTHREAD_MUTEX_LOCK(&sched_ctx->empty_ctx_mutex);
  320. starpu_task_list_push_back(&sched_ctx->empty_ctx_tasks, task);
  321. _STARPU_PTHREAD_MUTEX_UNLOCK(&sched_ctx->empty_ctx_mutex);
  322. return -EAGAIN;
  323. }
  324. }
  325. _starpu_profiling_set_task_push_start_time(task);
  326. int ret;
  327. if (STARPU_UNLIKELY(task->execute_on_a_specific_worker))
  328. {
  329. ret = _starpu_push_task_on_specific_worker(task, task->workerid);
  330. }
  331. else
  332. {
  333. STARPU_ASSERT(sched_ctx->sched_policy->push_task);
  334. ret = sched_ctx->sched_policy->push_task(task);
  335. if(ret == -1)
  336. {
  337. fprintf(stderr, "repush task \n");
  338. _STARPU_TRACE_JOB_POP(task, task->priority > 0);
  339. ret = _starpu_push_task_to_workers(task);
  340. }
  341. }
  342. /* Note: from here, the task might have been destroyed already! */
  343. _STARPU_LOG_OUT();
  344. return ret;
  345. }
  346. /* This is called right after the scheduler has pushed a task to a queue
  347. * but just before releasing mutexes: we need the task to still be alive!
  348. */
  349. int _starpu_push_task_end(struct starpu_task *task)
  350. {
  351. _starpu_profiling_set_task_push_end_time(task);
  352. task->scheduled = 1;
  353. return 0;
  354. }
  355. /*
  356. * Given a handle that needs to be converted in order to be used on the given
  357. * node, returns a task that takes care of the conversion.
  358. */
  359. struct starpu_task *_starpu_create_conversion_task(starpu_data_handle_t handle,
  360. unsigned int node)
  361. {
  362. return _starpu_create_conversion_task_for_arch(handle, starpu_node_get_kind(node));
  363. }
  364. struct starpu_task *_starpu_create_conversion_task_for_arch(starpu_data_handle_t handle,
  365. enum starpu_node_kind node_kind)
  366. {
  367. struct starpu_task *conversion_task;
  368. struct starpu_multiformat_interface *format_interface;
  369. conversion_task = starpu_task_create();
  370. conversion_task->synchronous = 0;
  371. conversion_task->handles[0] = handle;
  372. /* The node does not really matter here */
  373. format_interface = (struct starpu_multiformat_interface *) starpu_data_get_interface_on_node(handle, 0);
  374. _starpu_spin_lock(&handle->header_lock);
  375. handle->refcnt++;
  376. handle->busy_count++;
  377. _starpu_spin_unlock(&handle->header_lock);
  378. switch(node_kind)
  379. {
  380. case STARPU_CPU_RAM:
  381. switch (starpu_node_get_kind(handle->mf_node))
  382. {
  383. case STARPU_CPU_RAM:
  384. STARPU_ABORT();
  385. #if defined(STARPU_USE_CUDA) || defined(STARPU_SIMGRID)
  386. case STARPU_CUDA_RAM:
  387. {
  388. struct starpu_multiformat_data_interface_ops *mf_ops;
  389. mf_ops = (struct starpu_multiformat_data_interface_ops *) handle->ops->get_mf_ops(format_interface);
  390. conversion_task->cl = mf_ops->cuda_to_cpu_cl;
  391. break;
  392. }
  393. #endif
  394. #if defined(STARPU_USE_OPENCL) || defined(STARPU_SIMGRID)
  395. case STARPU_OPENCL_RAM:
  396. {
  397. struct starpu_multiformat_data_interface_ops *mf_ops;
  398. mf_ops = (struct starpu_multiformat_data_interface_ops *) handle->ops->get_mf_ops(format_interface);
  399. conversion_task->cl = mf_ops->opencl_to_cpu_cl;
  400. break;
  401. }
  402. #endif
  403. default:
  404. _STARPU_ERROR("Oops : %u\n", handle->mf_node);
  405. }
  406. break;
  407. #if defined(STARPU_USE_CUDA) || defined(STARPU_SIMGRID)
  408. case STARPU_CUDA_RAM:
  409. {
  410. struct starpu_multiformat_data_interface_ops *mf_ops;
  411. mf_ops = (struct starpu_multiformat_data_interface_ops *) handle->ops->get_mf_ops(format_interface);
  412. conversion_task->cl = mf_ops->cpu_to_cuda_cl;
  413. break;
  414. }
  415. #endif
  416. #if defined(STARPU_USE_OPENCL) || defined(STARPU_SIMGRID)
  417. case STARPU_OPENCL_RAM:
  418. {
  419. struct starpu_multiformat_data_interface_ops *mf_ops;
  420. mf_ops = (struct starpu_multiformat_data_interface_ops *) handle->ops->get_mf_ops(format_interface);
  421. conversion_task->cl = mf_ops->cpu_to_opencl_cl;
  422. break;
  423. }
  424. #endif
  425. default:
  426. STARPU_ABORT();
  427. }
  428. conversion_task->cl->modes[0] = STARPU_RW;
  429. return conversion_task;
  430. }
  431. struct _starpu_sched_ctx* _get_next_sched_ctx_to_pop_into(struct _starpu_worker *worker)
  432. {
  433. struct _starpu_sched_ctx *sched_ctx, *good_sched_ctx = NULL;
  434. unsigned smallest_counter = worker->nsched_ctxs;
  435. unsigned i;
  436. for(i = 0; i < STARPU_NMAX_SCHED_CTXS; i++)
  437. {
  438. sched_ctx = worker->sched_ctx[i];
  439. if(sched_ctx != NULL && sched_ctx->id != STARPU_NMAX_SCHED_CTXS && worker->removed_from_ctx[sched_ctx->id])
  440. return sched_ctx;
  441. if(sched_ctx != NULL && sched_ctx->id != STARPU_NMAX_SCHED_CTXS &&
  442. sched_ctx->pop_counter[worker->workerid] < worker->nsched_ctxs &&
  443. smallest_counter > sched_ctx->pop_counter[worker->workerid])
  444. {
  445. good_sched_ctx = sched_ctx;
  446. smallest_counter = sched_ctx->pop_counter[worker->workerid];
  447. }
  448. }
  449. if(good_sched_ctx == NULL)
  450. {
  451. for(i = 0; i < STARPU_NMAX_SCHED_CTXS; i++)
  452. {
  453. sched_ctx = worker->sched_ctx[i];
  454. if(sched_ctx != NULL && sched_ctx->id != STARPU_NMAX_SCHED_CTXS)
  455. sched_ctx->pop_counter[worker->workerid] = 0;
  456. }
  457. return _get_next_sched_ctx_to_pop_into(worker);
  458. }
  459. return good_sched_ctx;
  460. }
  461. struct starpu_task *_starpu_pop_task(struct _starpu_worker *worker)
  462. {
  463. struct starpu_task *task;
  464. int worker_id;
  465. unsigned node;
  466. /* We can't tell in advance which task will be picked up, so we measure
  467. * a timestamp, and will attribute it afterwards to the task. */
  468. int profiling = starpu_profiling_status_get();
  469. struct timespec pop_start_time;
  470. if (profiling)
  471. _starpu_clock_gettime(&pop_start_time);
  472. pick:
  473. /* perhaps there is some local task to be executed first */
  474. task = _starpu_pop_local_task(worker);
  475. /* get tasks from the stacks of the strategy */
  476. if(!task)
  477. {
  478. struct _starpu_sched_ctx *sched_ctx;
  479. //unsigned lucky_ctx = STARPU_NMAX_SCHED_CTXS;
  480. int been_here[STARPU_NMAX_SCHED_CTXS];
  481. int i;
  482. for(i = 0; i < STARPU_NMAX_SCHED_CTXS; i++)
  483. been_here[i] = 0;
  484. while(!task)
  485. {
  486. if(worker->nsched_ctxs == 1)
  487. sched_ctx = _starpu_get_initial_sched_ctx();
  488. else
  489. sched_ctx = _get_next_sched_ctx_to_pop_into(worker);
  490. if(sched_ctx != NULL && sched_ctx->id != STARPU_NMAX_SCHED_CTXS)
  491. {
  492. if (sched_ctx->sched_policy && sched_ctx->sched_policy->pop_task)
  493. {
  494. task = sched_ctx->sched_policy->pop_task(sched_ctx->id);
  495. //lucky_ctx = sched_ctx->id;
  496. }
  497. }
  498. if(!task && worker->removed_from_ctx[sched_ctx->id])
  499. {
  500. _starpu_worker_gets_out_of_ctx(sched_ctx->id, worker);
  501. worker->removed_from_ctx[sched_ctx->id] = 0;
  502. }
  503. if((!task && sched_ctx->pop_counter[worker->workerid] == 0 && been_here[sched_ctx->id]) || worker->nsched_ctxs == 1)
  504. break;
  505. been_here[sched_ctx->id] = 1;
  506. sched_ctx->pop_counter[worker->workerid]++;
  507. }
  508. }
  509. #ifdef STARPU_USE_SCHED_CTX_HYPERVISOR
  510. struct _starpu_sched_ctx *sched_ctx = NULL;
  511. struct starpu_sched_ctx_performance_counters *perf_counters = NULL;
  512. int j;
  513. for(j = 0; j < STARPU_NMAX_SCHED_CTXS; j++)
  514. {
  515. sched_ctx = worker->sched_ctx[j];
  516. if(sched_ctx != NULL && sched_ctx->id != 0 && sched_ctx->id != STARPU_NMAX_SCHED_CTXS)
  517. {
  518. perf_counters = sched_ctx->perf_counters;
  519. if(perf_counters != NULL && perf_counters->notify_idle_cycle && perf_counters->notify_idle_end)
  520. {
  521. if(!task)
  522. perf_counters->notify_idle_cycle(sched_ctx->id, worker->workerid, 1.0);
  523. else
  524. perf_counters->notify_idle_end(sched_ctx->id, worker->workerid);
  525. }
  526. }
  527. }
  528. #endif //STARPU_USE_SCHED_CTX_HYPERVISOR
  529. if (!task)
  530. return NULL;
  531. /* Make sure we do not bother with all the multiformat-specific code if
  532. * it is not necessary. */
  533. if (!_starpu_task_uses_multiformat_handles(task))
  534. goto profiling;
  535. /* This is either a conversion task, or a regular task for which the
  536. * conversion tasks have already been created and submitted */
  537. if (task->mf_skip)
  538. goto profiling;
  539. worker_id = starpu_worker_get_id();
  540. if (!starpu_worker_can_execute_task(worker_id, task, 0))
  541. return task;
  542. node = starpu_worker_get_memory_node(worker_id);
  543. /*
  544. * We do have a task that uses multiformat handles. Let's create the
  545. * required conversion tasks.
  546. */
  547. unsigned i;
  548. for (i = 0; i < task->cl->nbuffers; i++)
  549. {
  550. struct starpu_task *conversion_task;
  551. starpu_data_handle_t handle;
  552. handle = task->handles[i];
  553. if (!_starpu_handle_needs_conversion_task(handle, node))
  554. continue;
  555. conversion_task = _starpu_create_conversion_task(handle, node);
  556. conversion_task->mf_skip = 1;
  557. conversion_task->execute_on_a_specific_worker = 1;
  558. conversion_task->workerid = worker_id;
  559. /*
  560. * Next tasks will need to know where these handles have gone.
  561. */
  562. handle->mf_node = node;
  563. _starpu_task_submit_conversion_task(conversion_task, worker_id);
  564. }
  565. task->mf_skip = 1;
  566. starpu_task_list_push_front(&worker->local_tasks, task);
  567. goto pick;
  568. profiling:
  569. if (profiling)
  570. {
  571. struct starpu_task_profiling_info *profiling_info;
  572. profiling_info = task->profiling_info;
  573. /* The task may have been created before profiling was enabled,
  574. * so we check if the profiling_info structure is available
  575. * even though we already tested if profiling is enabled. */
  576. if (profiling_info)
  577. {
  578. memcpy(&profiling_info->pop_start_time,
  579. &pop_start_time, sizeof(struct timespec));
  580. _starpu_clock_gettime(&profiling_info->pop_end_time);
  581. }
  582. }
  583. return task;
  584. }
  585. struct starpu_task *_starpu_pop_every_task(struct _starpu_sched_ctx *sched_ctx)
  586. {
  587. STARPU_ASSERT(sched_ctx->sched_policy->pop_every_task);
  588. /* TODO set profiling info */
  589. if(sched_ctx->sched_policy->pop_every_task)
  590. return sched_ctx->sched_policy->pop_every_task(sched_ctx->id);
  591. return NULL;
  592. }
  593. void _starpu_sched_pre_exec_hook(struct starpu_task *task)
  594. {
  595. struct _starpu_sched_ctx *sched_ctx = _starpu_get_sched_ctx_struct(task->sched_ctx);
  596. if (sched_ctx->sched_policy->pre_exec_hook)
  597. sched_ctx->sched_policy->pre_exec_hook(task);
  598. }
  599. void _starpu_sched_post_exec_hook(struct starpu_task *task)
  600. {
  601. struct _starpu_sched_ctx *sched_ctx = _starpu_get_sched_ctx_struct(task->sched_ctx);
  602. #ifdef STARPU_USE_SCHED_CTX_HYPERVISOR
  603. if(task->hypervisor_tag > 0 && sched_ctx != NULL &&
  604. sched_ctx->id != 0 && sched_ctx->perf_counters != NULL)
  605. sched_ctx->perf_counters->notify_post_exec_hook(sched_ctx->id, task->hypervisor_tag);
  606. #endif //STARPU_USE_SCHED_CTX_HYPERVISOR
  607. if (sched_ctx->sched_policy->post_exec_hook)
  608. sched_ctx->sched_policy->post_exec_hook(task);
  609. }
  610. void _starpu_wait_on_sched_event(void)
  611. {
  612. struct _starpu_worker *worker = _starpu_get_local_worker_key();
  613. _STARPU_PTHREAD_MUTEX_LOCK(&worker->sched_mutex);
  614. _starpu_handle_all_pending_node_data_requests(worker->memory_node);
  615. if (_starpu_machine_is_running())
  616. {
  617. #ifndef STARPU_NON_BLOCKING_DRIVERS
  618. _STARPU_PTHREAD_COND_WAIT(&worker->sched_cond,
  619. &worker->sched_mutex);
  620. #endif
  621. }
  622. _STARPU_PTHREAD_MUTEX_UNLOCK(&worker->sched_mutex);
  623. }
  624. /* The scheduling policy may put tasks directly into a worker's local queue so
  625. * that it is not always necessary to create its own queue when the local queue
  626. * is sufficient. If "back" not null, the task is put at the back of the queue
  627. * where the worker will pop tasks first. Setting "back" to 0 therefore ensures
  628. * a FIFO ordering. */
  629. int starpu_push_local_task(int workerid, struct starpu_task *task, int back)
  630. {
  631. struct _starpu_worker *worker = _starpu_get_worker_struct(workerid);
  632. return _starpu_push_local_task(worker, task, back);
  633. }