bound.c 32 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593594595596597598599600601602603604605606607608609610611612613614615616617618619620621622623624625626627628629630631632633634635636637638639640641642643644645646647648649650651652653654655656657658659660661662663664665666667668669670671672673674675676677678679680681682683684685686687688689690691692693694695696697698699700701702703704705706707708709710711712713714715716717718719720721722723724725726727728729730731732733734735736737738739740741742743744745746747748749750751752753754755756757758759760761762763764765766767768769770771772773774775776777778779780781782783784785786787788789790791792793794795796797798799800801802803804805806807808809810811812813814815816817818819820821822823824825826827828829830831832833834835836837838839840841842843844845846847848849850851852853854855856857858859860861862863864865866867868869870871872873874875876877878879880881882883884885886887888889890891892893894895896897898899900901902903904905906907908909910911912913914915916917918919920921922923924925926927928929930931932933934935936937938939940941942943944945946947948949950951952953954955956957958959960961962963964965966967968969970971972973974975976977978979980981982983984985986987988989990991992993994995996997998999100010011002100310041005100610071008100910101011101210131014101510161017101810191020102110221023102410251026102710281029103010311032103310341035103610371038103910401041104210431044104510461047104810491050105110521053105410551056105710581059106010611062106310641065106610671068106910701071107210731074107510761077107810791080108110821083108410851086108710881089109010911092109310941095109610971098109911001101110211031104110511061107110811091110111111121113111411151116111711181119112011211122112311241125112611271128112911301131113211331134113511361137113811391140114111421143114411451146114711481149115011511152115311541155115611571158115911601161116211631164116511661167116811691170117111721173117411751176117711781179
  1. /* StarPU --- Runtime system for heterogeneous multicore architectures.
  2. *
  3. * Copyright (C) 2011-2012 Inria
  4. * Copyright (C) 2010-2017 Université de Bordeaux
  5. * Copyright (C) 2010-2017 CNRS
  6. * Copyright (C) 2011 Télécom-SudParis
  7. *
  8. * StarPU is free software; you can redistribute it and/or modify
  9. * it under the terms of the GNU Lesser General Public License as published by
  10. * the Free Software Foundation; either version 2.1 of the License, or (at
  11. * your option) any later version.
  12. *
  13. * StarPU is distributed in the hope that it will be useful, but
  14. * WITHOUT ANY WARRANTY; without even the implied warranty of
  15. * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE.
  16. *
  17. * See the GNU Lesser General Public License in COPYING.LGPL for more details.
  18. */
  19. /*
  20. * Record which kinds of tasks have been executed, to later on compute an upper
  21. * bound of the performance that could have theoretically been achieved
  22. */
  23. #include <starpu.h>
  24. #include <starpu_config.h>
  25. #include <profiling/bound.h>
  26. #include <core/jobs.h>
  27. #include <core/workers.h>
  28. #ifdef STARPU_HAVE_GLPK_H
  29. #include <glpk.h>
  30. #endif /* STARPU_HAVE_GLPK_H */
  31. /* TODO: output duration between starpu_bound_start and starpu_bound_stop */
  32. /* TODO: compute critical path and introduce it in the LP */
  33. /*
  34. * Record without dependencies: just count each kind of task
  35. *
  36. * The linear programming problem will just have as variables:
  37. * - the number of tasks of kind `t' executed by worker `w'
  38. * - the total duration
  39. *
  40. * and the constraints will be:
  41. * - the time taken by each worker to complete its assigned tasks is lower than
  42. * the total duration.
  43. * - the total numer of tasks of a given kind is equal to the number run by the
  44. * application.
  45. */
  46. struct bound_task_pool
  47. {
  48. /* Which codelet has been executed */
  49. struct starpu_codelet *cl;
  50. /* Task footprint key (for history-based perfmodel) */
  51. uint32_t footprint;
  52. /* Number of tasks of this kind */
  53. unsigned long n;
  54. /* Other task kinds */
  55. struct bound_task_pool *next;
  56. };
  57. /*
  58. * Record with dependencies: each task is recorded separately
  59. *
  60. * The linear programming problem will have as variables:
  61. * - The start time of each task
  62. * - The completion time of each tag
  63. * - The total duration
  64. * - For each task and for each worker, whether the task is executing on that worker.
  65. * - For each pair of task, which task is scheduled first.
  66. *
  67. * and the constraints will be:
  68. * - All task start time plus duration are less than total duration
  69. * - Each task is executed on exactly one worker.
  70. * - Each task starts after all its task dependencies finish.
  71. * - Each task starts after all its tag dependencies finish.
  72. * - For each task pair and each worker, if both tasks are executed by that worker,
  73. * one is started after the other's completion.
  74. */
  75. struct task_dep
  76. {
  77. /* Task this depends on */
  78. struct bound_task *dep;
  79. /* Data transferred between tasks (i.e. implicit data dep size) */
  80. size_t size;
  81. };
  82. struct bound_task
  83. {
  84. /* Unique ID */
  85. unsigned long id;
  86. /* Tag ID, if any */
  87. starpu_tag_t tag_id;
  88. int use_tag;
  89. /* Which codelet has been executed */
  90. struct starpu_codelet *cl;
  91. /* Task footprint key */
  92. uint32_t footprint;
  93. /* Task priority */
  94. int priority;
  95. /* Tasks this one depends on */
  96. struct task_dep *deps;
  97. int depsn;
  98. /* Estimated duration */
  99. double** duration[STARPU_NARCH];
  100. /* Other tasks */
  101. struct bound_task *next;
  102. };
  103. struct bound_tag_dep
  104. {
  105. starpu_tag_t tag;
  106. starpu_tag_t dep_tag;
  107. struct bound_tag_dep *next;
  108. };
  109. static struct bound_task_pool *task_pools, *last;
  110. static struct bound_task *tasks;
  111. static struct bound_tag_dep *tag_deps;
  112. int _starpu_bound_recording;
  113. static int recorddeps;
  114. static int recordprio;
  115. static starpu_pthread_mutex_t mutex = STARPU_PTHREAD_MUTEX_INITIALIZER;
  116. /* Initialization */
  117. void starpu_bound_start(int deps, int prio)
  118. {
  119. struct bound_task_pool *tp;
  120. struct bound_task *t;
  121. struct bound_tag_dep *td;
  122. STARPU_PTHREAD_MUTEX_LOCK(&mutex);
  123. tp = task_pools;
  124. task_pools = NULL;
  125. last = NULL;
  126. t = tasks;
  127. tasks = NULL;
  128. td = tag_deps;
  129. tag_deps = NULL;
  130. _starpu_bound_recording = 1;
  131. recorddeps = deps;
  132. recordprio = prio;
  133. STARPU_PTHREAD_MUTEX_UNLOCK(&mutex);
  134. while (tp != NULL)
  135. {
  136. struct bound_task_pool *next = tp->next;
  137. free(tp);
  138. tp = next;
  139. }
  140. while (t != NULL)
  141. {
  142. struct bound_task *next = t->next;
  143. free(t);
  144. t = next;
  145. }
  146. while (td != NULL)
  147. {
  148. struct bound_tag_dep *next = td->next;
  149. free(td);
  150. td = next;
  151. }
  152. }
  153. /* Whether we will include it in the computation */
  154. static int good_job(struct _starpu_job *j)
  155. {
  156. /* No codelet, nothing to measure */
  157. if (j->exclude_from_dag)
  158. return 0;
  159. if (!j->task->cl)
  160. return 0;
  161. /* No performance model, no time duration estimation */
  162. if (!j->task->cl->model)
  163. return 0;
  164. /* Only support history based */
  165. if (j->task->cl->model->type != STARPU_HISTORY_BASED
  166. && j->task->cl->model->type != STARPU_NL_REGRESSION_BASED)
  167. return 0;
  168. return 1;
  169. }
  170. static double** initialize_arch_duration(int maxdevid, unsigned* maxncore_table)
  171. {
  172. int devid, maxncore;
  173. double ** arch_model;
  174. _STARPU_MALLOC(arch_model, sizeof(*arch_model)*(maxdevid+1));
  175. arch_model[maxdevid] = NULL;
  176. for(devid=0; devid<maxdevid; devid++)
  177. {
  178. if(maxncore_table != NULL)
  179. maxncore = maxncore_table[devid];
  180. else
  181. maxncore = 1;
  182. _STARPU_CALLOC(arch_model[devid], maxncore+1,sizeof(*arch_model[devid]));
  183. }
  184. return arch_model;
  185. }
  186. static void initialize_duration(struct bound_task *task)
  187. {
  188. struct _starpu_machine_config *conf = _starpu_get_machine_config();
  189. task->duration[STARPU_CPU_WORKER] = initialize_arch_duration(1,&conf->topology.nhwcpus);
  190. task->duration[STARPU_CUDA_WORKER] = initialize_arch_duration(conf->topology.nhwcudagpus,NULL);
  191. task->duration[STARPU_OPENCL_WORKER] = initialize_arch_duration(conf->topology.nhwopenclgpus,NULL);
  192. task->duration[STARPU_MIC_WORKER] = initialize_arch_duration(conf->topology.nhwmicdevices,conf->topology.nmiccores);
  193. task->duration[STARPU_SCC_WORKER] = initialize_arch_duration(conf->topology.nhwscc,NULL);
  194. }
  195. static struct starpu_perfmodel_device device =
  196. {
  197. .type = STARPU_CPU_WORKER,
  198. .devid = 0,
  199. .ncores = 1,
  200. };
  201. static struct starpu_perfmodel_arch dumb_arch =
  202. {
  203. .ndevices = 1,
  204. .devices = &device,
  205. };
  206. /* Create a new task (either because it has just been submitted, or a
  207. * dependency was added before submission) */
  208. static void new_task(struct _starpu_job *j)
  209. {
  210. struct bound_task *t;
  211. if (j->bound_task)
  212. return;
  213. _STARPU_MALLOC(t, sizeof(*t));
  214. memset(t, 0, sizeof(*t));
  215. t->id = j->job_id;
  216. t->tag_id = j->task->tag_id;
  217. t->use_tag = j->task->use_tag;
  218. t->cl = j->task->cl;
  219. t->footprint = _starpu_compute_buffers_footprint(j->task->cl?j->task->cl->model:NULL, &dumb_arch, 0, j);
  220. t->priority = j->task->priority;
  221. t->deps = NULL;
  222. t->depsn = 0;
  223. initialize_duration(t);
  224. t->next = tasks;
  225. j->bound_task = t;
  226. tasks = t;
  227. }
  228. /* A new task was submitted, record it */
  229. void _starpu_bound_record(struct _starpu_job *j)
  230. {
  231. if (!_starpu_bound_recording)
  232. return;
  233. if (!good_job(j))
  234. return;
  235. STARPU_PTHREAD_MUTEX_LOCK(&mutex);
  236. /* Re-check, this time with mutex held */
  237. if (!_starpu_bound_recording)
  238. {
  239. STARPU_PTHREAD_MUTEX_UNLOCK(&mutex);
  240. return;
  241. }
  242. if (recorddeps)
  243. {
  244. new_task(j);
  245. }
  246. else
  247. {
  248. struct bound_task_pool *tp;
  249. _starpu_compute_buffers_footprint(j->task->cl?j->task->cl->model:NULL, NULL, 0, j);
  250. if (last && last->cl == j->task->cl && last->footprint == j->footprint)
  251. tp = last;
  252. else
  253. for (tp = task_pools; tp; tp = tp->next)
  254. if (tp->cl == j->task->cl && tp->footprint == j->footprint)
  255. break;
  256. if (!tp)
  257. {
  258. _STARPU_MALLOC(tp, sizeof(*tp));
  259. tp->cl = j->task->cl;
  260. tp->footprint = j->footprint;
  261. tp->n = 0;
  262. tp->next = task_pools;
  263. task_pools = tp;
  264. }
  265. /* One more task of this kind */
  266. tp->n++;
  267. }
  268. STARPU_PTHREAD_MUTEX_UNLOCK(&mutex);
  269. }
  270. /* A tag dependency was emitted, record it */
  271. void _starpu_bound_tag_dep(starpu_tag_t id, starpu_tag_t dep_id)
  272. {
  273. struct bound_tag_dep *td;
  274. if (!_starpu_bound_recording || !recorddeps)
  275. return;
  276. STARPU_PTHREAD_MUTEX_LOCK(&mutex);
  277. /* Re-check, this time with mutex held */
  278. if (!_starpu_bound_recording || !recorddeps)
  279. {
  280. STARPU_PTHREAD_MUTEX_UNLOCK(&mutex);
  281. return;
  282. }
  283. _STARPU_MALLOC(td, sizeof(*td));
  284. td->tag = id;
  285. td->dep_tag = dep_id;
  286. td->next = tag_deps;
  287. tag_deps = td;
  288. STARPU_PTHREAD_MUTEX_UNLOCK(&mutex);
  289. }
  290. /* A task dependency was emitted, record it */
  291. void _starpu_bound_task_dep(struct _starpu_job *j, struct _starpu_job *dep_j)
  292. {
  293. struct bound_task *t;
  294. int i;
  295. if (!_starpu_bound_recording || !recorddeps)
  296. return;
  297. if (!good_job(j) || !good_job(dep_j))
  298. return;
  299. STARPU_PTHREAD_MUTEX_LOCK(&mutex);
  300. /* Re-check, this time with mutex held */
  301. if (!_starpu_bound_recording || !recorddeps)
  302. {
  303. STARPU_PTHREAD_MUTEX_UNLOCK(&mutex);
  304. return;
  305. }
  306. new_task(j);
  307. new_task(dep_j);
  308. t = j->bound_task;
  309. for (i = 0; i < t->depsn; i++)
  310. if (t->deps[i].dep == dep_j->bound_task)
  311. break;
  312. if (i == t->depsn)
  313. {
  314. /* Not already there, add */
  315. _STARPU_REALLOC(t->deps, ++t->depsn * sizeof(t->deps[0]));
  316. t->deps[t->depsn-1].dep = dep_j->bound_task;
  317. t->deps[t->depsn-1].size = 0; /* We don't have data information in that case */
  318. }
  319. STARPU_PTHREAD_MUTEX_UNLOCK(&mutex);
  320. }
  321. /* Look for job with id ID among our tasks */
  322. static struct bound_task *find_job(unsigned long id)
  323. {
  324. struct bound_task *t;
  325. for (t = tasks; t; t = t->next)
  326. if (t->id == id)
  327. return t;
  328. return NULL;
  329. }
  330. /* Job J depends on previous job of id ID (which is already finished) */
  331. void _starpu_bound_job_id_dep(starpu_data_handle_t handle, struct _starpu_job *j, unsigned long id)
  332. {
  333. struct bound_task *t, *dep_t;
  334. int i;
  335. if (!_starpu_bound_recording || !recorddeps)
  336. return;
  337. if (!good_job(j))
  338. return;
  339. STARPU_PTHREAD_MUTEX_LOCK(&mutex);
  340. /* Re-check, this time with mutex held */
  341. if (!_starpu_bound_recording || !recorddeps)
  342. {
  343. STARPU_PTHREAD_MUTEX_UNLOCK(&mutex);
  344. return;
  345. }
  346. new_task(j);
  347. dep_t = find_job(id);
  348. if (!dep_t)
  349. {
  350. _STARPU_MSG("dependency %lu not found !\n", id);
  351. STARPU_PTHREAD_MUTEX_UNLOCK(&mutex);
  352. return;
  353. }
  354. t = j->bound_task;
  355. for (i = 0; i < t->depsn; i++)
  356. if (t->deps[i].dep == dep_t)
  357. {
  358. /* Found, just add size */
  359. t->deps[i].size += _starpu_data_get_size(handle);
  360. break;
  361. }
  362. if (i == t->depsn)
  363. {
  364. /* Not already there, add */
  365. _STARPU_REALLOC(t->deps, ++t->depsn * sizeof(t->deps[0]));
  366. t->deps[t->depsn-1].dep = dep_t;
  367. t->deps[t->depsn-1].size = _starpu_data_get_size(handle);
  368. }
  369. STARPU_PTHREAD_MUTEX_UNLOCK(&mutex);
  370. }
  371. void starpu_bound_stop(void)
  372. {
  373. STARPU_PTHREAD_MUTEX_LOCK(&mutex);
  374. _starpu_bound_recording = 0;
  375. STARPU_PTHREAD_MUTEX_UNLOCK(&mutex);
  376. }
  377. /* Compute all tasks times on all workers */
  378. static void _starpu_get_tasks_times(int nw, int nt, double *times)
  379. {
  380. struct bound_task_pool *tp;
  381. int w, t;
  382. for (w = 0; w < nw; w++)
  383. {
  384. for (t = 0, tp = task_pools; tp; t++, tp = tp->next)
  385. {
  386. struct _starpu_job j =
  387. {
  388. .footprint = tp->footprint,
  389. .footprint_is_computed = 1,
  390. };
  391. struct starpu_perfmodel_arch* arch = starpu_worker_get_perf_archtype(w, STARPU_NMAX_SCHED_CTXS);
  392. double length = _starpu_history_based_job_expected_perf(tp->cl->model, arch, &j, j.nimpl);
  393. if (isnan(length))
  394. times[w*nt+t] = NAN;
  395. else
  396. times[w*nt+t] = length / 1000.;
  397. }
  398. }
  399. }
  400. /* Return whether PARENT is an ancestor of CHILD */
  401. static int ancestor(struct bound_task *child, struct bound_task *parent)
  402. {
  403. int i;
  404. for (i = 0; i < child->depsn; i++)
  405. {
  406. if (parent == child->deps[i].dep)
  407. return 1;
  408. if (ancestor(child->deps[i].dep, parent))
  409. return -1;
  410. }
  411. return 0;
  412. }
  413. /* Print bound recording in .dot format */
  414. void starpu_bound_print_dot(FILE *output)
  415. {
  416. struct bound_task *t;
  417. struct bound_tag_dep *td;
  418. int i;
  419. if (!recorddeps)
  420. {
  421. fprintf(output, "Not supported\n");
  422. return;
  423. }
  424. fprintf(output, "strict digraph bounddeps {\n");
  425. for (t = tasks; t; t = t->next)
  426. {
  427. fprintf(output, "\"t%lu\" [label=\"%lu: %s\"]\n", t->id, t->id, _starpu_codelet_get_model_name(t->cl));
  428. for (i = 0; i < t->depsn; i++)
  429. fprintf(output, "\"t%lu\" -> \"t%lu\"\n", t->deps[i].dep->id, t->id);
  430. }
  431. for (td = tag_deps; td; td = td->next)
  432. fprintf(output, "\"tag%lu\" -> \"tag%lu\";\n", (unsigned long) td->dep_tag, (unsigned long) td->tag);
  433. fprintf(output, "}\n");
  434. }
  435. /*
  436. * Print bound system in lp_solve format
  437. *
  438. * When dependencies are enabled, you can check the set of tasks and deps that
  439. * were recorded by using tools/lp2paje and vite.
  440. */
  441. void starpu_bound_print_lp(FILE *output)
  442. {
  443. int nt; /* Number of different kinds of tasks */
  444. int nw; /* Number of different workers */
  445. int t;
  446. int w, w2; /* worker */
  447. unsigned n, n2;
  448. STARPU_PTHREAD_MUTEX_LOCK(&mutex);
  449. nw = starpu_worker_get_count();
  450. if (!nw)
  451. /* Make llvm happy about the VLA below */
  452. return;
  453. if (recorddeps)
  454. {
  455. struct bound_task *t1, *t2;
  456. struct bound_tag_dep *td;
  457. int i;
  458. nt = 0;
  459. for (t1 = tasks; t1; t1 = t1->next)
  460. {
  461. if (t1->cl->model->type != STARPU_HISTORY_BASED &&
  462. t1->cl->model->type != STARPU_NL_REGRESSION_BASED)
  463. /* TODO: */
  464. _STARPU_MSG("Warning: task %s uses a perf model which is neither history nor non-linear regression-based, support for such model is not implemented yet, system will not be solvable.\n", _starpu_codelet_get_model_name(t1->cl));
  465. struct _starpu_job j =
  466. {
  467. .footprint = t1->footprint,
  468. .footprint_is_computed = 1,
  469. };
  470. for (w = 0; w < nw; w++)
  471. {
  472. struct starpu_perfmodel_arch* arch = starpu_worker_get_perf_archtype(w, STARPU_NMAX_SCHED_CTXS);
  473. if (_STARPU_IS_ZERO(t1->duration[arch->devices[0].type][arch->devices[0].devid][arch->devices[0].ncores]))
  474. {
  475. double length = _starpu_history_based_job_expected_perf(t1->cl->model, arch, &j,j.nimpl);
  476. if (isnan(length))
  477. /* Avoid problems with binary coding of doubles */
  478. t1->duration[arch->devices[0].type][arch->devices[0].devid][arch->devices[0].ncores] = NAN;
  479. else
  480. t1->duration[arch->devices[0].type][arch->devices[0].devid][arch->devices[0].ncores] = length / 1000.;
  481. }
  482. }
  483. nt++;
  484. }
  485. if (!nt)
  486. return;
  487. fprintf(output, "/* StarPU upper bound linear programming problem, to be run in lp_solve. */\n\n");
  488. fprintf(output, "/* !! This is a big system, it will be long to solve !! */\n\n");
  489. fprintf(output, "/* We want to minimize total execution time (ms) */\n");
  490. fprintf(output, "min: tmax;\n\n");
  491. fprintf(output, "/* Number of tasks */\n");
  492. fprintf(output, "nt = %d;\n", nt);
  493. fprintf(output, "/* Number of workers */\n");
  494. fprintf(output, "nw = %d;\n", nw);
  495. fprintf(output, "/* The total execution time is the maximum of all task completion times (ms) */\n");
  496. for (t1 = tasks; t1; t1 = t1->next)
  497. fprintf(output, "c%lu <= tmax;\n", t1->id);
  498. fprintf(output, "\n/* We have tasks executing on workers, exactly one worker executes each task */\n");
  499. for (t1 = tasks; t1; t1 = t1->next)
  500. {
  501. for (w = 0; w < nw; w++)
  502. {
  503. struct starpu_perfmodel_arch* arch = starpu_worker_get_perf_archtype(w, STARPU_NMAX_SCHED_CTXS);
  504. if (!isnan(t1->duration[arch->devices[0].type][arch->devices[0].devid][arch->devices[0].ncores]))
  505. fprintf(output, " +t%luw%d", t1->id, w);
  506. }
  507. fprintf(output, " = 1;\n");
  508. }
  509. fprintf(output, "\n/* Completion time is start time plus computation time */\n");
  510. fprintf(output, "/* According to where the task is indeed executed */\n");
  511. for (t1 = tasks; t1; t1 = t1->next)
  512. {
  513. fprintf(output, "/* %s %x */\tc%lu = s%lu", _starpu_codelet_get_model_name(t1->cl), (unsigned) t1->footprint, t1->id, t1->id);
  514. for (w = 0; w < nw; w++)
  515. {
  516. struct starpu_perfmodel_arch* arch = starpu_worker_get_perf_archtype(w, STARPU_NMAX_SCHED_CTXS);
  517. if (!isnan(t1->duration[arch->devices[0].type][arch->devices[0].devid][arch->devices[0].ncores]))
  518. fprintf(output, " + %f t%luw%d", t1->duration[arch->devices[0].type][arch->devices[0].devid][arch->devices[0].ncores], t1->id, w);
  519. }
  520. fprintf(output, ";\n");
  521. }
  522. fprintf(output, "\n/* Each task starts after all its task dependencies finish and data is transferred. */\n");
  523. fprintf(output, "/* Note that the dependency finish time depends on the worker where it's working */\n");
  524. for (t1 = tasks; t1; t1 = t1->next)
  525. for (i = 0; i < t1->depsn; i++)
  526. {
  527. fprintf(output, "/* %lu bytes transferred */\n", (unsigned long) t1->deps[i].size);
  528. fprintf(output, "s%lu >= c%lu", t1->id, t1->deps[i].dep->id);
  529. /* Transfer time: pick up one source node and a worker on it */
  530. for (n = 0; n < starpu_memory_nodes_get_count(); n++)
  531. for (w = 0; w < nw; w++)
  532. if (starpu_worker_get_memory_node(w) == n)
  533. {
  534. /* pick up another destination node and a worker on it */
  535. for (n2 = 0; n2 < starpu_memory_nodes_get_count(); n2++)
  536. if (n2 != n)
  537. {
  538. for (w2 = 0; w2 < nw; w2++)
  539. if (starpu_worker_get_memory_node(w2) == n2)
  540. {
  541. /* If predecessor is on worker w and successor
  542. * on worker w2 on different nodes, we need to
  543. * transfer the data. */
  544. fprintf(output, " + d_t%luw%dt%luw%d", t1->deps[i].dep->id, w, t1->id, w2);
  545. }
  546. }
  547. }
  548. fprintf(output, ";\n");
  549. /* Transfer time: pick up one source node and a worker on it */
  550. for (n = 0; n < starpu_memory_nodes_get_count(); n++)
  551. for (w = 0; w < nw; w++)
  552. if (starpu_worker_get_memory_node(w) == n)
  553. {
  554. /* pick up another destination node and a worker on it */
  555. for (n2 = 0; n2 < starpu_memory_nodes_get_count(); n2++)
  556. if (n2 != n)
  557. {
  558. for (w2 = 0; w2 < nw; w2++)
  559. if (starpu_worker_get_memory_node(w2) == n2)
  560. {
  561. /* The data transfer is at least 0ms */
  562. fprintf(output, "d_t%luw%dt%luw%d >= 0;\n", t1->deps[i].dep->id, w, t1->id, w2);
  563. /* The data transfer from w to w2 only happens if tasks run there */
  564. fprintf(output, "d_t%luw%dt%luw%d >= %f - 2e5 + 1e5 t%luw%d + 1e5 t%luw%d;\n",
  565. t1->deps[i].dep->id, w, t1->id, w2,
  566. starpu_transfer_predict(n, n2, t1->deps[i].size)/1000.,
  567. t1->deps[i].dep->id, w, t1->id, w2);
  568. }
  569. }
  570. }
  571. }
  572. fprintf(output, "\n/* Each tag finishes when its corresponding task finishes */\n");
  573. for (t1 = tasks; t1; t1 = t1->next)
  574. if (t1->use_tag)
  575. {
  576. for (w = 0; w < nw; w++)
  577. fprintf(output, "c%lu = tag%lu;\n", t1->id, (unsigned long) t1->tag_id);
  578. }
  579. fprintf(output, "\n/* tags start after all their tag dependencies finish. */\n");
  580. for (td = tag_deps; td; td = td->next)
  581. fprintf(output, "tag%lu >= tag%lu;\n", (unsigned long) td->tag, (unsigned long) td->dep_tag);
  582. /* TODO: factorize ancestor calls */
  583. fprintf(output, "\n/* For each task pair and each worker, if both tasks are executed by the same worker,\n");
  584. fprintf(output, " one is started after the other's completion */\n");
  585. for (t1 = tasks; t1; t1 = t1->next)
  586. {
  587. for (t2 = t1->next; t2; t2 = t2->next)
  588. {
  589. if (!ancestor(t1, t2) && !ancestor(t2, t1))
  590. {
  591. for (w = 0; w < nw; w++)
  592. {
  593. struct starpu_perfmodel_arch* arch = starpu_worker_get_perf_archtype(w, STARPU_NMAX_SCHED_CTXS);
  594. if (!isnan(t1->duration[arch->devices[0].type][arch->devices[0].devid][arch->devices[0].ncores]))
  595. {
  596. fprintf(output, "s%lu - c%lu >= -3e5 + 1e5 t%luw%d + 1e5 t%luw%d + 1e5 t%luafter%lu;\n",
  597. t1->id, t2->id, t1->id, w, t2->id, w, t1->id, t2->id);
  598. fprintf(output, "s%lu - c%lu >= -2e5 + 1e5 t%luw%d + 1e5 t%luw%d - 1e5 t%luafter%lu;\n",
  599. t2->id, t1->id, t1->id, w, t2->id, w, t1->id, t2->id);
  600. }
  601. }
  602. }
  603. }
  604. }
  605. #if 0
  606. /* Doesn't help at all to actually express what "after" means */
  607. for (t1 = tasks; t1; t1 = t1->next)
  608. for (t2 = t1->next; t2; t2 = t2->next)
  609. if (!ancestor(t1, t2) && !ancestor(t2, t1))
  610. {
  611. fprintf(output, "s%lu - s%lu >= -1e5 + 1e5 t%luafter%lu;\n", t1->id, t2->id, t1->id, t2->id);
  612. fprintf(output, "s%lu - s%lu >= -1e5 t%luafter%lu;\n", t2->id, t1->id, t1->id, t2->id);
  613. }
  614. #endif
  615. if (recordprio)
  616. {
  617. fprintf(output, "\n/* For StarPU, a priority means given schedulable tasks it will consider the\n");
  618. fprintf(output, " * more prioritized first */\n");
  619. for (t1 = tasks; t1; t1 = t1->next)
  620. {
  621. for (t2 = t1->next; t2; t2 = t2->next)
  622. {
  623. if (!ancestor(t1, t2) && !ancestor(t2, t1)
  624. && t1->priority != t2->priority)
  625. {
  626. if (t1->priority > t2->priority)
  627. {
  628. /* Either t2 is scheduled before t1, but then it
  629. needs to be scheduled before some t dep finishes */
  630. /* One of the t1 deps to give the maximum start time for t2 */
  631. if (t1->depsn > 1)
  632. {
  633. for (i = 0; i < t1->depsn; i++)
  634. fprintf(output, " + t%lut%lud%d", t2->id, t1->id, i);
  635. fprintf(output, " = 1;\n");
  636. }
  637. for (i = 0; i < t1->depsn; i++)
  638. {
  639. fprintf(output, "c%lu - s%lu >= ", t1->deps[i].dep->id, t2->id);
  640. if (t1->depsn > 1)
  641. /* Only checks this when it's this dependency that is chosen */
  642. fprintf(output, "-2e5 + 1e5 t%lut%lud%d", t2->id, t1->id, i);
  643. else
  644. fprintf(output, "-1e5");
  645. /* Only check this if t1 is after t2 */
  646. fprintf(output, " + 1e5 t%luafter%lu", t1->id, t2->id);
  647. fprintf(output, ";\n");
  648. }
  649. /* Or t2 is scheduled after t1 is. */
  650. fprintf(output, "s%lu - s%lu >= -1e5 t%luafter%lu;\n", t2->id, t1->id, t1->id, t2->id);
  651. }
  652. else
  653. {
  654. /* Either t1 is scheduled before t2, but then it
  655. needs to be scheduled before some t2 dep finishes */
  656. /* One of the t2 deps to give the maximum start time for t1 */
  657. if (t2->depsn > 1)
  658. {
  659. for (i = 0; i < t2->depsn; i++)
  660. fprintf(output, " + t%lut%lud%d", t1->id, t2->id, i);
  661. fprintf(output, " = 1;\n");
  662. }
  663. for (i = 0; i < t2->depsn; i++)
  664. {
  665. fprintf(output, "c%lu - s%lu >= ", t2->deps[i].dep->id, t1->id);
  666. if (t2->depsn > 1)
  667. /* Only checks this when it's this dependency that is chosen */
  668. fprintf(output, "-1e5 + 1e5 t%lut%lud%d", t1->id, t2->id, i);
  669. /* Only check this if t2 is after t1 */
  670. fprintf(output, " - 1e5 t%luafter%lu;\n", t1->id, t2->id);
  671. }
  672. /* Or t1 is scheduled after t2 is. */
  673. fprintf(output, "s%lu - s%lu >= -1e5 + 1e5 t%luafter%lu;\n", t1->id, t2->id, t1->id, t2->id);
  674. }
  675. }
  676. }
  677. }
  678. }
  679. for (t1 = tasks; t1; t1 = t1->next)
  680. for (t2 = t1->next; t2; t2 = t2->next)
  681. if (!ancestor(t1, t2) && !ancestor(t2, t1))
  682. {
  683. fprintf(output, "bin t%luafter%lu;\n", t1->id, t2->id);
  684. if (recordprio && t1->priority != t2->priority)
  685. {
  686. if (t1->priority > t2->priority)
  687. {
  688. if (t1->depsn > 1)
  689. for (i = 0; i < t1->depsn; i++)
  690. fprintf(output, "bin t%lut%lud%d;\n", t2->id, t1->id, i);
  691. }
  692. else
  693. {
  694. if (t2->depsn > 1)
  695. for (i = 0; i < t2->depsn; i++)
  696. fprintf(output, "bin t%lut%lud%d;\n", t1->id, t2->id, i);
  697. }
  698. }
  699. }
  700. for (t1 = tasks; t1; t1 = t1->next)
  701. for (w = 0; w < nw; w++)
  702. fprintf(output, "bin t%luw%d;\n", t1->id, w);
  703. }
  704. else
  705. {
  706. struct bound_task_pool *tp;
  707. nt = 0;
  708. for (tp = task_pools; tp; tp = tp->next)
  709. nt++;
  710. if (!nt)
  711. return;
  712. {
  713. double times[nw*nt];
  714. _starpu_get_tasks_times(nw, nt, times);
  715. fprintf(output, "/* StarPU upper bound linear programming problem, to be run in lp_solve. */\n\n");
  716. fprintf(output, "/* We want to minimize total execution time (ms) */\n");
  717. fprintf(output, "min: tmax;\n\n");
  718. fprintf(output, "/* Which is the maximum of all worker execution times (ms) */\n");
  719. for (w = 0; w < nw; w++)
  720. {
  721. char name[32];
  722. starpu_worker_get_name(w, name, sizeof(name));
  723. fprintf(output, "/* worker %s */\n0", name);
  724. for (t = 0, tp = task_pools; tp; t++, tp = tp->next)
  725. {
  726. if (!isnan(times[w*nt+t]))
  727. fprintf(output, "\t%+f * w%dt%dn", (float) times[w*nt+t], w, t);
  728. }
  729. fprintf(output, " <= tmax;\n");
  730. }
  731. fprintf(output, "\n");
  732. fprintf(output, "/* And we have to have computed exactly all tasks */\n");
  733. for (t = 0, tp = task_pools; tp; t++, tp = tp->next)
  734. {
  735. int got_one = 0;
  736. fprintf(output, "/* task %s key %x */\n0", _starpu_codelet_get_model_name(tp->cl), (unsigned) tp->footprint);
  737. for (w = 0; w < nw; w++)
  738. {
  739. if (isnan(times[w*nt+t]))
  740. _STARPU_MSG("Warning: task %s has no performance measurement for worker %d.\n", _starpu_codelet_get_model_name(tp->cl), w);
  741. else
  742. {
  743. got_one = 1;
  744. fprintf(output, "\t+w%dt%dn", w, t);
  745. }
  746. }
  747. fprintf(output, " = %lu;\n", tp->n);
  748. if (!got_one)
  749. _STARPU_MSG("Warning: task %s has no performance measurement for any worker, system will not be solvable!\n", _starpu_codelet_get_model_name(tp->cl));
  750. /* Show actual values */
  751. fprintf(output, "/*");
  752. for (w = 0; w < nw; w++)
  753. fprintf(output, "\t+%lu", tp->cl->per_worker_stats[w]);
  754. fprintf(output, "\t*/\n\n");
  755. }
  756. fprintf(output, "/* Optionally tell that tasks can not be divided */\n");
  757. fprintf(output, "/* int ");
  758. int first = 1;
  759. for (w = 0; w < nw; w++)
  760. for (t = 0, tp = task_pools; tp; t++, tp = tp->next)
  761. {
  762. if (!first)
  763. fprintf(output, ",");
  764. else
  765. first = 0;
  766. fprintf(output, "w%dt%dn", w, t);
  767. }
  768. fprintf(output, "; */\n");
  769. }
  770. }
  771. STARPU_PTHREAD_MUTEX_UNLOCK(&mutex);
  772. }
  773. /*
  774. * Print bound system in MPS output format
  775. */
  776. void starpu_bound_print_mps(FILE *output)
  777. {
  778. struct bound_task_pool * tp;
  779. int nt; /* Number of different kinds of tasks */
  780. int nw; /* Number of different workers */
  781. int t, w;
  782. if (recorddeps)
  783. {
  784. fprintf(output, "Not supported\n");
  785. return;
  786. }
  787. nw = starpu_worker_get_count();
  788. if (!nw)
  789. /* Make llvm happy about the VLA below */
  790. return;
  791. STARPU_PTHREAD_MUTEX_LOCK(&mutex);
  792. nt = 0;
  793. for (tp = task_pools; tp; tp = tp->next)
  794. nt++;
  795. if (!nt)
  796. {
  797. STARPU_PTHREAD_MUTEX_UNLOCK(&mutex);
  798. return;
  799. }
  800. {
  801. double times[nw*nt];
  802. _starpu_get_tasks_times(nw, nt, times);
  803. fprintf(output, "NAME StarPU theoretical bound\n");
  804. fprintf(output, "*\nROWS\n");
  805. fprintf(output, "* We want to minimize total execution time (ms)\n");
  806. fprintf(output, " N TMAX\n");
  807. fprintf(output, "* Which is the maximum of all worker execution times (ms)\n");
  808. for (w = 0; w < nw; w++)
  809. {
  810. char name[32];
  811. starpu_worker_get_name(w, name, sizeof(name));
  812. fprintf(output, "* worker %s\n", name);
  813. fprintf(output, " L W%d\n", w);
  814. }
  815. fprintf(output, "*\n* And we have to have computed exactly all tasks\n*\n");
  816. for (t = 0, tp = task_pools; tp; t++, tp = tp->next)
  817. {
  818. fprintf(output, "* task %s key %x\n", _starpu_codelet_get_model_name(tp->cl), (unsigned) tp->footprint);
  819. fprintf(output, " E T%d\n", t);
  820. }
  821. fprintf(output, "*\nCOLUMNS\n*\n");
  822. fprintf(output, "*\n* Execution times and completion of all tasks\n*\n");
  823. for (w = 0; w < nw; w++)
  824. for (t = 0, tp = task_pools; tp; t++, tp = tp->next)
  825. if (!isnan(times[w*nt+t]))
  826. {
  827. char name[23];
  828. snprintf(name, sizeof(name), "W%dT%d", w, t);
  829. fprintf(output," %-8s W%-7d %12f\n", name, w, times[w*nt+t]);
  830. fprintf(output," %-8s T%-7d %12d\n", name, t, 1);
  831. }
  832. fprintf(output, "*\n* Total execution time\n*\n");
  833. for (w = 0; w < nw; w++)
  834. fprintf(output," TMAX W%-2d %12d\n", w, -1);
  835. fprintf(output," TMAX TMAX %12d\n", 1);
  836. fprintf(output, "*\nRHS\n*\n");
  837. fprintf(output, "*\n* Total number of tasks\n*\n");
  838. for (t = 0, tp = task_pools; tp; t++, tp = tp->next)
  839. fprintf(output," NT%-2d T%-7d %12lu\n", t, t, tp->n);
  840. fprintf(output, "ENDATA\n");
  841. }
  842. STARPU_PTHREAD_MUTEX_UNLOCK(&mutex);
  843. }
  844. /*
  845. * Solve bound system thanks to GNU Linear Programming Kit backend
  846. */
  847. #ifdef STARPU_HAVE_GLPK_H
  848. static glp_prob *_starpu_bound_glp_resolve(int integer)
  849. {
  850. struct bound_task_pool * tp;
  851. int nt; /* Number of different kinds of tasks */
  852. int nw; /* Number of different workers */
  853. int t, w;
  854. glp_prob *lp;
  855. int ret;
  856. nw = starpu_worker_get_count();
  857. if (!nw)
  858. /* Make llvm happy about the VLA below */
  859. return NULL;
  860. nt = 0;
  861. for (tp = task_pools; tp; tp = tp->next)
  862. nt++;
  863. if (!nt)
  864. return NULL;
  865. lp = glp_create_prob();
  866. glp_set_prob_name(lp, "StarPU theoretical bound");
  867. glp_set_obj_dir(lp, GLP_MIN);
  868. glp_set_obj_name(lp, "total execution time");
  869. {
  870. double times[nw*nt];
  871. int ne =
  872. nw * (nt+1) /* worker execution time */
  873. + nt * nw
  874. + 1; /* glp dumbness */
  875. int n = 1;
  876. int ia[ne], ja[ne];
  877. double ar[ne];
  878. _starpu_get_tasks_times(nw, nt, times);
  879. /* Variables: number of tasks i assigned to worker j, and tmax */
  880. glp_add_cols(lp, nw*nt+1);
  881. #define colnum(w, t) ((t)*nw+(w)+1)
  882. glp_set_obj_coef(lp, nw*nt+1, 1.);
  883. for (w = 0; w < nw; w++)
  884. for (t = 0, tp = task_pools; tp; t++, tp = tp->next)
  885. {
  886. char name[32];
  887. snprintf(name, sizeof(name), "w%dt%dn", w, t);
  888. glp_set_col_name(lp, colnum(w, t), name);
  889. if (integer)
  890. glp_set_col_kind(lp, colnum(w, t), GLP_IV);
  891. glp_set_col_bnds(lp, colnum(w, t), GLP_LO, 0., 0.);
  892. }
  893. glp_set_col_bnds(lp, nw*nt+1, GLP_LO, 0., 0.);
  894. /* Total worker execution time */
  895. glp_add_rows(lp, nw);
  896. for (t = 0, tp = task_pools; tp; t++, tp = tp->next)
  897. {
  898. int someone = 0;
  899. for (w = 0; w < nw; w++)
  900. if (!isnan(times[w*nt+t]))
  901. someone = 1;
  902. if (!someone)
  903. {
  904. /* This task does not have any performance model at all, abort */
  905. glp_delete_prob(lp);
  906. return NULL;
  907. }
  908. }
  909. for (w = 0; w < nw; w++)
  910. {
  911. char name[32], title[64];
  912. starpu_worker_get_name(w, name, sizeof(name));
  913. snprintf(title, sizeof(title), "worker %s", name);
  914. glp_set_row_name(lp, w+1, title);
  915. for (t = 0, tp = task_pools; tp; t++, tp = tp->next)
  916. {
  917. ia[n] = w+1;
  918. ja[n] = colnum(w, t);
  919. if (isnan(times[w*nt+t]))
  920. ar[n] = 1000000000.;
  921. else
  922. ar[n] = times[w*nt+t];
  923. n++;
  924. }
  925. /* tmax */
  926. ia[n] = w+1;
  927. ja[n] = nw*nt+1;
  928. ar[n] = -1;
  929. n++;
  930. glp_set_row_bnds(lp, w+1, GLP_UP, 0, 0);
  931. }
  932. /* Total task completion */
  933. glp_add_rows(lp, nt);
  934. for (t = 0, tp = task_pools; tp; t++, tp = tp->next)
  935. {
  936. char name[32], title[64];
  937. starpu_worker_get_name(w, name, sizeof(name));
  938. snprintf(title, sizeof(title), "task %s key %x", _starpu_codelet_get_model_name(tp->cl), (unsigned) tp->footprint);
  939. glp_set_row_name(lp, nw+t+1, title);
  940. for (w = 0; w < nw; w++)
  941. {
  942. ia[n] = nw+t+1;
  943. ja[n] = colnum(w, t);
  944. ar[n] = 1;
  945. n++;
  946. }
  947. glp_set_row_bnds(lp, nw+t+1, GLP_FX, tp->n, tp->n);
  948. }
  949. STARPU_ASSERT(n == ne);
  950. glp_load_matrix(lp, ne-1, ia, ja, ar);
  951. }
  952. glp_smcp parm;
  953. glp_init_smcp(&parm);
  954. parm.msg_lev = GLP_MSG_OFF;
  955. ret = glp_simplex(lp, &parm);
  956. if (ret)
  957. {
  958. glp_delete_prob(lp);
  959. lp = NULL;
  960. return NULL;
  961. }
  962. if (integer)
  963. {
  964. glp_iocp iocp;
  965. glp_init_iocp(&iocp);
  966. iocp.msg_lev = GLP_MSG_OFF;
  967. glp_intopt(lp, &iocp);
  968. }
  969. return lp;
  970. }
  971. #endif /* STARPU_HAVE_GLPK_H */
  972. /* Print the computed bound as well as the optimized distribution of tasks */
  973. void starpu_bound_print(FILE *output, int integer)
  974. {
  975. #ifdef STARPU_HAVE_GLPK_H
  976. if (recorddeps)
  977. {
  978. fprintf(output, "Not supported\n");
  979. return;
  980. }
  981. STARPU_PTHREAD_MUTEX_LOCK(&mutex);
  982. glp_prob *lp = _starpu_bound_glp_resolve(integer);
  983. if (lp)
  984. {
  985. struct bound_task_pool * tp;
  986. int t, w;
  987. int nw; /* Number of different workers */
  988. double tmax;
  989. nw = starpu_worker_get_count();
  990. if (integer)
  991. tmax = glp_mip_obj_val(lp);
  992. else
  993. tmax = glp_get_obj_val(lp);
  994. fprintf(output, "Theoretical minimum execution time: %f ms\n", tmax);
  995. for (t = 0, tp = task_pools; tp; t++, tp = tp->next)
  996. {
  997. fprintf(output, "%s key %x\n", _starpu_codelet_get_model_name(tp->cl), (unsigned) tp->footprint);
  998. for (w = 0; w < nw; w++)
  999. if (integer)
  1000. fprintf(output, "\tw%dt%dn %f", w, t, glp_mip_col_val(lp, colnum(w, t)));
  1001. else
  1002. fprintf(output, "\tw%dt%dn %f", w, t, glp_get_col_prim(lp, colnum(w, t)));
  1003. fprintf(output, "\n");
  1004. }
  1005. glp_delete_prob(lp);
  1006. }
  1007. else
  1008. {
  1009. _STARPU_MSG("Simplex failed\n");
  1010. }
  1011. STARPU_PTHREAD_MUTEX_UNLOCK(&mutex);
  1012. #else /* STARPU_HAVE_GLPK_H */
  1013. (void) integer;
  1014. fprintf(output, "Please rebuild StarPU with glpk installed.\n");
  1015. #endif /* STARPU_HAVE_GLPK_H */
  1016. }
  1017. /* Compute and return the bound */
  1018. void starpu_bound_compute(double *res, double *integer_res, int integer)
  1019. {
  1020. #ifdef STARPU_HAVE_GLPK_H
  1021. double ret;
  1022. if (recorddeps)
  1023. {
  1024. *res = 0.;
  1025. return;
  1026. }
  1027. STARPU_PTHREAD_MUTEX_LOCK(&mutex);
  1028. glp_prob *lp = _starpu_bound_glp_resolve(integer);
  1029. if (lp)
  1030. {
  1031. ret = glp_get_obj_val(lp);
  1032. if (integer)
  1033. *integer_res = glp_mip_obj_val(lp);
  1034. glp_delete_prob(lp);
  1035. }
  1036. else
  1037. ret = 0.;
  1038. STARPU_PTHREAD_MUTEX_UNLOCK(&mutex);
  1039. *res = ret;
  1040. #else /* STARPU_HAVE_GLPK_H */
  1041. (void) integer_res;
  1042. (void) integer;
  1043. *res = 0.;
  1044. #endif /* STARPU_HAVE_GLPK_H */
  1045. }