topology.c 94 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593594595596597598599600601602603604605606607608609610611612613614615616617618619620621622623624625626627628629630631632633634635636637638639640641642643644645646647648649650651652653654655656657658659660661662663664665666667668669670671672673674675676677678679680681682683684685686687688689690691692693694695696697698699700701702703704705706707708709710711712713714715716717718719720721722723724725726727728729730731732733734735736737738739740741742743744745746747748749750751752753754755756757758759760761762763764765766767768769770771772773774775776777778779780781782783784785786787788789790791792793794795796797798799800801802803804805806807808809810811812813814815816817818819820821822823824825826827828829830831832833834835836837838839840841842843844845846847848849850851852853854855856857858859860861862863864865866867868869870871872873874875876877878879880881882883884885886887888889890891892893894895896897898899900901902903904905906907908909910911912913914915916917918919920921922923924925926927928929930931932933934935936937938939940941942943944945946947948949950951952953954955956957958959960961962963964965966967968969970971972973974975976977978979980981982983984985986987988989990991992993994995996997998999100010011002100310041005100610071008100910101011101210131014101510161017101810191020102110221023102410251026102710281029103010311032103310341035103610371038103910401041104210431044104510461047104810491050105110521053105410551056105710581059106010611062106310641065106610671068106910701071107210731074107510761077107810791080108110821083108410851086108710881089109010911092109310941095109610971098109911001101110211031104110511061107110811091110111111121113111411151116111711181119112011211122112311241125112611271128112911301131113211331134113511361137113811391140114111421143114411451146114711481149115011511152115311541155115611571158115911601161116211631164116511661167116811691170117111721173117411751176117711781179118011811182118311841185118611871188118911901191119211931194119511961197119811991200120112021203120412051206120712081209121012111212121312141215121612171218121912201221122212231224122512261227122812291230123112321233123412351236123712381239124012411242124312441245124612471248124912501251125212531254125512561257125812591260126112621263126412651266126712681269127012711272127312741275127612771278127912801281128212831284128512861287128812891290129112921293129412951296129712981299130013011302130313041305130613071308130913101311131213131314131513161317131813191320132113221323132413251326132713281329133013311332133313341335133613371338133913401341134213431344134513461347134813491350135113521353135413551356135713581359136013611362136313641365136613671368136913701371137213731374137513761377137813791380138113821383138413851386138713881389139013911392139313941395139613971398139914001401140214031404140514061407140814091410141114121413141414151416141714181419142014211422142314241425142614271428142914301431143214331434143514361437143814391440144114421443144414451446144714481449145014511452145314541455145614571458145914601461146214631464146514661467146814691470147114721473147414751476147714781479148014811482148314841485148614871488148914901491149214931494149514961497149814991500150115021503150415051506150715081509151015111512151315141515151615171518151915201521152215231524152515261527152815291530153115321533153415351536153715381539154015411542154315441545154615471548154915501551155215531554155515561557155815591560156115621563156415651566156715681569157015711572157315741575157615771578157915801581158215831584158515861587158815891590159115921593159415951596159715981599160016011602160316041605160616071608160916101611161216131614161516161617161816191620162116221623162416251626162716281629163016311632163316341635163616371638163916401641164216431644164516461647164816491650165116521653165416551656165716581659166016611662166316641665166616671668166916701671167216731674167516761677167816791680168116821683168416851686168716881689169016911692169316941695169616971698169917001701170217031704170517061707170817091710171117121713171417151716171717181719172017211722172317241725172617271728172917301731173217331734173517361737173817391740174117421743174417451746174717481749175017511752175317541755175617571758175917601761176217631764176517661767176817691770177117721773177417751776177717781779178017811782178317841785178617871788178917901791179217931794179517961797179817991800180118021803180418051806180718081809181018111812181318141815181618171818181918201821182218231824182518261827182818291830183118321833183418351836183718381839184018411842184318441845184618471848184918501851185218531854185518561857185818591860186118621863186418651866186718681869187018711872187318741875187618771878187918801881188218831884188518861887188818891890189118921893189418951896189718981899190019011902190319041905190619071908190919101911191219131914191519161917191819191920192119221923192419251926192719281929193019311932193319341935193619371938193919401941194219431944194519461947194819491950195119521953195419551956195719581959196019611962196319641965196619671968196919701971197219731974197519761977197819791980198119821983198419851986198719881989199019911992199319941995199619971998199920002001200220032004200520062007200820092010201120122013201420152016201720182019202020212022202320242025202620272028202920302031203220332034203520362037203820392040204120422043204420452046204720482049205020512052205320542055205620572058205920602061206220632064206520662067206820692070207120722073207420752076207720782079208020812082208320842085208620872088208920902091209220932094209520962097209820992100210121022103210421052106210721082109211021112112211321142115211621172118211921202121212221232124212521262127212821292130213121322133213421352136213721382139214021412142214321442145214621472148214921502151215221532154215521562157215821592160216121622163216421652166216721682169217021712172217321742175217621772178217921802181218221832184218521862187218821892190219121922193219421952196219721982199220022012202220322042205220622072208220922102211221222132214221522162217221822192220222122222223222422252226222722282229223022312232223322342235223622372238223922402241224222432244224522462247224822492250225122522253225422552256225722582259226022612262226322642265226622672268226922702271227222732274227522762277227822792280228122822283228422852286228722882289229022912292229322942295229622972298229923002301230223032304230523062307230823092310231123122313231423152316231723182319232023212322232323242325232623272328232923302331233223332334233523362337233823392340234123422343234423452346234723482349235023512352235323542355235623572358235923602361236223632364236523662367236823692370237123722373237423752376237723782379238023812382238323842385238623872388238923902391239223932394239523962397239823992400240124022403240424052406240724082409241024112412241324142415241624172418241924202421242224232424242524262427242824292430243124322433243424352436243724382439244024412442244324442445244624472448244924502451245224532454245524562457245824592460246124622463246424652466246724682469247024712472247324742475247624772478247924802481248224832484248524862487248824892490249124922493249424952496249724982499250025012502250325042505250625072508250925102511251225132514251525162517251825192520252125222523252425252526252725282529253025312532253325342535253625372538253925402541254225432544254525462547254825492550255125522553255425552556255725582559256025612562256325642565256625672568256925702571257225732574257525762577257825792580258125822583258425852586258725882589259025912592259325942595259625972598259926002601260226032604260526062607260826092610261126122613261426152616261726182619262026212622262326242625262626272628262926302631263226332634263526362637263826392640264126422643264426452646264726482649265026512652265326542655265626572658265926602661266226632664266526662667266826692670267126722673267426752676267726782679268026812682268326842685268626872688268926902691269226932694269526962697269826992700270127022703270427052706270727082709271027112712271327142715271627172718271927202721272227232724272527262727272827292730273127322733273427352736273727382739274027412742274327442745274627472748274927502751275227532754275527562757275827592760276127622763276427652766276727682769277027712772277327742775277627772778277927802781278227832784278527862787278827892790279127922793279427952796279727982799280028012802280328042805280628072808280928102811281228132814281528162817281828192820282128222823282428252826282728282829283028312832283328342835283628372838283928402841284228432844284528462847284828492850285128522853285428552856285728582859286028612862286328642865286628672868286928702871287228732874287528762877287828792880288128822883288428852886288728882889289028912892289328942895289628972898289929002901290229032904290529062907290829092910291129122913291429152916291729182919292029212922292329242925292629272928292929302931293229332934293529362937293829392940294129422943294429452946294729482949295029512952295329542955295629572958295929602961296229632964296529662967296829692970297129722973297429752976297729782979298029812982298329842985298629872988298929902991299229932994299529962997299829993000300130023003300430053006
  1. /* StarPU --- Runtime system for heterogeneous multicore architectures.
  2. *
  3. * Copyright (C) 2011-2017 Inria
  4. * Copyright (C) 2009-2019 Université de Bordeaux
  5. * Copyright (C) 2010-2017, 2019 CNRS
  6. * Copyright (C) 2013 Thibaut Lambert
  7. * Copyright (C) 2016 Uppsala University
  8. *
  9. * StarPU is free software; you can redistribute it and/or modify
  10. * it under the terms of the GNU Lesser General Public License as published by
  11. * the Free Software Foundation; either version 2.1 of the License, or (at
  12. * your option) any later version.
  13. *
  14. * StarPU is distributed in the hope that it will be useful, but
  15. * WITHOUT ANY WARRANTY; without even the implied warranty of
  16. * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE.
  17. *
  18. * See the GNU Lesser General Public License in COPYING.LGPL for more details.
  19. */
  20. #include <stdlib.h>
  21. #include <stdio.h>
  22. #include <common/config.h>
  23. #include <core/workers.h>
  24. #include <core/debug.h>
  25. #include <core/topology.h>
  26. #include <drivers/cuda/driver_cuda.h>
  27. #include <drivers/cpu/driver_cpu.h>
  28. #include <drivers/mic/driver_mic_source.h>
  29. #include <drivers/mpi/driver_mpi_source.h>
  30. #include <drivers/mpi/driver_mpi_common.h>
  31. #include <drivers/mp_common/source_common.h>
  32. #include <drivers/opencl/driver_opencl.h>
  33. #include <drivers/opencl/driver_opencl_utils.h>
  34. #include <profiling/profiling.h>
  35. #include <datawizard/datastats.h>
  36. #include <datawizard/memory_nodes.h>
  37. #include <datawizard/memory_manager.h>
  38. #include <common/uthash.h>
  39. #ifdef STARPU_HAVE_HWLOC
  40. #include <hwloc.h>
  41. #ifndef HWLOC_API_VERSION
  42. #define HWLOC_OBJ_PU HWLOC_OBJ_PROC
  43. #endif
  44. #if HWLOC_API_VERSION < 0x00010b00
  45. #define HWLOC_OBJ_NUMANODE HWLOC_OBJ_NODE
  46. #endif
  47. #endif
  48. #ifdef STARPU_HAVE_WINDOWS
  49. #include <windows.h>
  50. #endif
  51. #ifdef STARPU_SIMGRID
  52. #include <core/simgrid.h>
  53. #endif
  54. #if defined(HAVE_DECL_HWLOC_CUDA_GET_DEVICE_OSDEV_BY_INDEX) && HAVE_DECL_HWLOC_CUDA_GET_DEVICE_OSDEV_BY_INDEX
  55. #include <hwloc/cuda.h>
  56. #endif
  57. #if defined(STARPU_HAVE_HWLOC) && defined(STARPU_USE_OPENCL)
  58. #include <hwloc/opencl.h>
  59. #endif
  60. static unsigned topology_is_initialized = 0;
  61. static int nobind;
  62. static int numa_enabled = -1;
  63. /* For checking whether two workers share the same PU, indexed by PU number */
  64. static int cpu_worker[STARPU_MAXCPUS];
  65. static char * cpu_name[STARPU_MAXCPUS];
  66. static unsigned nb_numa_nodes = 0;
  67. static int numa_memory_nodes_to_hwloclogid[STARPU_MAXNUMANODES]; /* indexed by StarPU numa node to convert in hwloc logid */
  68. static int numa_memory_nodes_to_physicalid[STARPU_MAXNUMANODES]; /* indexed by StarPU numa node to convert in physical id */
  69. static unsigned numa_bus_id[STARPU_MAXNUMANODES*STARPU_MAXNUMANODES];
  70. static int _starpu_get_logical_numa_node_worker(unsigned workerid);
  71. #define STARPU_NUMA_UNINITIALIZED (-2)
  72. #define STARPU_NUMA_MAIN_RAM (-1)
  73. #if defined(STARPU_USE_CUDA) || defined(STARPU_USE_OPENCL) || defined(STARPU_SIMGRID) || defined(STARPU_USE_MPI_MASTER_SLAVE)
  74. struct handle_entry
  75. {
  76. UT_hash_handle hh;
  77. unsigned gpuid;
  78. };
  79. # if defined(STARPU_USE_CUDA) || defined(STARPU_SIMGRID)
  80. /* Entry in the `devices_using_cuda' hash table. */
  81. static struct handle_entry *devices_using_cuda;
  82. # endif
  83. static unsigned may_bind_automatically[STARPU_NARCH] = { 0 };
  84. #endif // defined(STARPU_USE_CUDA) || defined(STARPU_USE_OPENCL)
  85. #if defined(STARPU_USE_CUDA) || defined(STARPU_SIMGRID)
  86. static struct _starpu_worker_set cuda_worker_set[STARPU_MAXCUDADEVS];
  87. #endif
  88. #ifdef STARPU_USE_MIC
  89. static struct _starpu_worker_set mic_worker_set[STARPU_MAXMICDEVS];
  90. #endif
  91. #ifdef STARPU_USE_MPI_MASTER_SLAVE
  92. struct _starpu_worker_set mpi_worker_set[STARPU_MAXMPIDEVS];
  93. #endif
  94. int starpu_memory_nodes_get_numa_count(void)
  95. {
  96. return nb_numa_nodes;
  97. }
  98. #if defined(STARPU_HAVE_HWLOC)
  99. static hwloc_obj_t numa_get_obj(hwloc_obj_t obj)
  100. {
  101. #if HWLOC_API_VERSION >= 0x00020000
  102. while (obj->memory_first_child == NULL)
  103. {
  104. obj = obj->parent;
  105. if (!obj)
  106. return NULL;
  107. }
  108. return obj->memory_first_child;
  109. #else
  110. while (obj->type != HWLOC_OBJ_NUMANODE)
  111. {
  112. obj = obj->parent;
  113. /* If we don't find a "node" obj before the root, this means
  114. * hwloc does not know whether there are numa nodes or not, so
  115. * we should not use a per-node sampling in that case. */
  116. if (!obj)
  117. return NULL;
  118. }
  119. return obj;
  120. #endif
  121. }
  122. static int numa_get_logical_id(hwloc_obj_t obj)
  123. {
  124. STARPU_ASSERT(obj);
  125. obj = numa_get_obj(obj);
  126. if (!obj)
  127. return 0;
  128. return obj->logical_index;
  129. }
  130. static int numa_get_physical_id(hwloc_obj_t obj)
  131. {
  132. STARPU_ASSERT(obj);
  133. obj = numa_get_obj(obj);
  134. if (!obj)
  135. return 0;
  136. return obj->os_index;
  137. }
  138. #endif
  139. /* This returns the exact NUMA node next to a worker */
  140. static int _starpu_get_logical_numa_node_worker(unsigned workerid)
  141. {
  142. #if defined(STARPU_HAVE_HWLOC)
  143. STARPU_ASSERT(numa_enabled != -1);
  144. if (numa_enabled)
  145. {
  146. struct _starpu_worker *worker = _starpu_get_worker_struct(workerid);
  147. struct _starpu_machine_config *config = (struct _starpu_machine_config *)_starpu_get_machine_config() ;
  148. struct _starpu_machine_topology *topology = &config->topology ;
  149. hwloc_obj_t obj;
  150. switch(worker->arch)
  151. {
  152. case STARPU_CPU_WORKER:
  153. obj = hwloc_get_obj_by_type(topology->hwtopology, HWLOC_OBJ_PU, worker->bindid) ;
  154. break;
  155. default:
  156. STARPU_ABORT();
  157. }
  158. return numa_get_logical_id(obj);
  159. }
  160. else
  161. #endif
  162. {
  163. (void) workerid; /* unused */
  164. return STARPU_NUMA_MAIN_RAM;
  165. }
  166. }
  167. /* This returns the exact NUMA node next to a worker */
  168. static int _starpu_get_physical_numa_node_worker(unsigned workerid)
  169. {
  170. #if defined(STARPU_HAVE_HWLOC)
  171. STARPU_ASSERT(numa_enabled != -1);
  172. if (numa_enabled)
  173. {
  174. struct _starpu_worker *worker = _starpu_get_worker_struct(workerid);
  175. struct _starpu_machine_config *config = (struct _starpu_machine_config *)_starpu_get_machine_config() ;
  176. struct _starpu_machine_topology *topology = &config->topology ;
  177. hwloc_obj_t obj;
  178. switch(worker->arch)
  179. {
  180. case STARPU_CPU_WORKER:
  181. obj = hwloc_get_obj_by_type(topology->hwtopology, HWLOC_OBJ_PU, worker->bindid) ;
  182. break;
  183. default:
  184. STARPU_ABORT();
  185. }
  186. return numa_get_physical_id(obj);
  187. }
  188. else
  189. #endif
  190. {
  191. (void) workerid; /* unused */
  192. return STARPU_NUMA_MAIN_RAM;
  193. }
  194. }
  195. /* This returns the CPU NUMA memory close to a worker */
  196. static int _starpu_get_logical_close_numa_node_worker(unsigned workerid)
  197. {
  198. #if defined(STARPU_HAVE_HWLOC)
  199. STARPU_ASSERT(numa_enabled != -1);
  200. if (numa_enabled)
  201. {
  202. struct _starpu_worker *worker = _starpu_get_worker_struct(workerid);
  203. struct _starpu_machine_config *config = (struct _starpu_machine_config *)_starpu_get_machine_config() ;
  204. struct _starpu_machine_topology *topology = &config->topology ;
  205. hwloc_obj_t obj;
  206. switch(worker->arch)
  207. {
  208. default:
  209. obj = hwloc_get_obj_by_type(topology->hwtopology, HWLOC_OBJ_PU, worker->bindid) ;
  210. break;
  211. #ifndef STARPU_SIMGRID
  212. #if defined(HAVE_DECL_HWLOC_CUDA_GET_DEVICE_OSDEV_BY_INDEX) && HAVE_DECL_HWLOC_CUDA_GET_DEVICE_OSDEV_BY_INDEX
  213. case STARPU_CUDA_WORKER:
  214. obj = hwloc_cuda_get_device_osdev_by_index(topology->hwtopology, worker->devid);
  215. if (!obj)
  216. obj = hwloc_get_obj_by_type(topology->hwtopology, HWLOC_OBJ_PU, worker->bindid) ;
  217. break;
  218. #endif
  219. #endif
  220. }
  221. return numa_get_logical_id(obj);
  222. }
  223. else
  224. #endif
  225. {
  226. (void) workerid; /* unused */
  227. return STARPU_NUMA_MAIN_RAM;
  228. }
  229. }
  230. //TODO change this in an array
  231. int starpu_memory_nodes_numa_hwloclogid_to_id(int logid)
  232. {
  233. unsigned n;
  234. for (n = 0; n < nb_numa_nodes; n++)
  235. if (numa_memory_nodes_to_hwloclogid[n] == logid)
  236. return n;
  237. return -1;
  238. }
  239. int starpu_memory_nodes_numa_id_to_hwloclogid(unsigned id)
  240. {
  241. STARPU_ASSERT(id < STARPU_MAXNUMANODES);
  242. return numa_memory_nodes_to_hwloclogid[id];
  243. }
  244. int starpu_memory_nodes_numa_devid_to_id(unsigned id)
  245. {
  246. STARPU_ASSERT(id < STARPU_MAXNUMANODES);
  247. return numa_memory_nodes_to_physicalid[id];
  248. }
  249. //TODO change this in an array
  250. int starpu_memory_nodes_numa_id_to_devid(int osid)
  251. {
  252. unsigned n;
  253. for (n = 0; n < nb_numa_nodes; n++)
  254. if (numa_memory_nodes_to_physicalid[n] == osid)
  255. return n;
  256. return -1;
  257. }
  258. // TODO: cache the values instead of looking in hwloc each time
  259. /* Avoid using this one, prefer _starpu_task_data_get_node_on_worker */
  260. int _starpu_task_data_get_node_on_node(struct starpu_task *task, unsigned index, unsigned local_node)
  261. {
  262. int node = STARPU_SPECIFIC_NODE_LOCAL;
  263. if (task->cl->specific_nodes)
  264. node = STARPU_CODELET_GET_NODE(task->cl, index);
  265. switch (node) {
  266. case STARPU_SPECIFIC_NODE_LOCAL:
  267. // TODO: rather find MCDRAM
  268. node = local_node;
  269. break;
  270. case STARPU_SPECIFIC_NODE_CPU:
  271. switch (starpu_node_get_kind(local_node)) {
  272. case STARPU_CPU_RAM:
  273. node = local_node;
  274. break;
  275. default:
  276. // TODO: rather take close NUMA node
  277. node = STARPU_MAIN_RAM;
  278. break;
  279. }
  280. break;
  281. case STARPU_SPECIFIC_NODE_SLOW:
  282. // TODO: rather leave in DDR
  283. node = local_node;
  284. break;
  285. case STARPU_SPECIFIC_NODE_LOCAL_OR_CPU:
  286. if (task->handles[index]->per_node[local_node].state != STARPU_INVALID) {
  287. /* It is here already, rather access it from here */
  288. node = local_node;
  289. } else {
  290. /* It is not here already, do not bother moving it */
  291. node = STARPU_MAIN_RAM;
  292. }
  293. break;
  294. }
  295. return node;
  296. }
  297. int _starpu_task_data_get_node_on_worker(struct starpu_task *task, unsigned index, unsigned worker)
  298. {
  299. unsigned local_node = starpu_worker_get_memory_node(worker);
  300. int node = STARPU_SPECIFIC_NODE_LOCAL;
  301. if (task->cl->specific_nodes)
  302. node = STARPU_CODELET_GET_NODE(task->cl, index);
  303. switch (node) {
  304. case STARPU_SPECIFIC_NODE_LOCAL:
  305. // TODO: rather find MCDRAM
  306. node = local_node;
  307. break;
  308. case STARPU_SPECIFIC_NODE_CPU:
  309. node = starpu_memory_nodes_numa_hwloclogid_to_id(_starpu_get_logical_close_numa_node_worker(worker));
  310. if (node == -1)
  311. node = STARPU_MAIN_RAM;
  312. break;
  313. case STARPU_SPECIFIC_NODE_SLOW:
  314. // TODO: rather leave in DDR
  315. node = local_node;
  316. break;
  317. case STARPU_SPECIFIC_NODE_LOCAL_OR_CPU:
  318. if (task->handles[index]->per_node[local_node].state != STARPU_INVALID) {
  319. /* It is here already, rather access it from here */
  320. node = local_node;
  321. } else {
  322. /* It is not here already, do not bother moving it */
  323. node = STARPU_MAIN_RAM;
  324. }
  325. break;
  326. }
  327. return node;
  328. }
  329. struct _starpu_worker *_starpu_get_worker_from_driver(struct starpu_driver *d)
  330. {
  331. unsigned nworkers = starpu_worker_get_count();
  332. unsigned workerid;
  333. for (workerid = 0; workerid < nworkers; workerid++)
  334. {
  335. if (starpu_worker_get_type(workerid) == d->type)
  336. {
  337. struct _starpu_worker *worker;
  338. worker = _starpu_get_worker_struct(workerid);
  339. switch (d->type)
  340. {
  341. #ifdef STARPU_USE_CPU
  342. case STARPU_CPU_WORKER:
  343. if (worker->devid == d->id.cpu_id)
  344. return worker;
  345. break;
  346. #endif
  347. #ifdef STARPU_USE_OPENCL
  348. case STARPU_OPENCL_WORKER:
  349. {
  350. cl_device_id device;
  351. starpu_opencl_get_device(worker->devid, &device);
  352. if (device == d->id.opencl_id)
  353. return worker;
  354. break;
  355. }
  356. #endif
  357. #ifdef STARPU_USE_CUDA
  358. case STARPU_CUDA_WORKER:
  359. {
  360. if (worker->devid == d->id.cuda_id)
  361. return worker;
  362. break;
  363. }
  364. #endif
  365. default:
  366. (void) worker;
  367. _STARPU_DEBUG("Invalid device type\n");
  368. return NULL;
  369. }
  370. }
  371. }
  372. return NULL;
  373. }
  374. /*
  375. * Discover the topology of the machine
  376. */
  377. #if defined(STARPU_USE_CUDA) || defined(STARPU_USE_OPENCL) || defined(STARPU_SIMGRID) || defined(STARPU_USE_MPI_MASTER_SLAVE)
  378. static void _starpu_initialize_workers_deviceid(int *explicit_workers_gpuid,
  379. int *current, int *workers_gpuid,
  380. const char *varname, unsigned nhwgpus,
  381. enum starpu_worker_archtype type)
  382. {
  383. char *strval;
  384. unsigned i;
  385. *current = 0;
  386. /* conf->workers_gpuid indicates the successive GPU identifier that
  387. * should be used to bind the workers. It should be either filled
  388. * according to the user's explicit parameters (from starpu_conf) or
  389. * according to the STARPU_WORKERS_CUDAID env. variable. Otherwise, a
  390. * round-robin policy is used to distributed the workers over the
  391. * cores. */
  392. /* what do we use, explicit value, env. variable, or round-robin ? */
  393. strval = starpu_getenv(varname);
  394. if (strval)
  395. {
  396. /* STARPU_WORKERS_CUDAID certainly contains less entries than
  397. * STARPU_NMAXWORKERS, so we reuse its entries in a round
  398. * robin fashion: "1 2" is equivalent to "1 2 1 2 1 2 .... 1
  399. * 2". */
  400. unsigned wrap = 0;
  401. unsigned number_of_entries = 0;
  402. char *endptr;
  403. /* we use the content of the STARPU_WORKERS_CUDAID
  404. * env. variable */
  405. for (i = 0; i < STARPU_NMAXWORKERS; i++)
  406. {
  407. if (!wrap)
  408. {
  409. long int val;
  410. val = strtol(strval, &endptr, 10);
  411. if (endptr != strval)
  412. {
  413. workers_gpuid[i] = (unsigned)val;
  414. strval = endptr;
  415. }
  416. else
  417. {
  418. /* there must be at least one entry */
  419. STARPU_ASSERT(i != 0);
  420. number_of_entries = i;
  421. /* there is no more values in the
  422. * string */
  423. wrap = 1;
  424. workers_gpuid[i] = workers_gpuid[0];
  425. }
  426. }
  427. else
  428. {
  429. workers_gpuid[i] =
  430. workers_gpuid[i % number_of_entries];
  431. }
  432. }
  433. }
  434. else if (explicit_workers_gpuid)
  435. {
  436. /* we use the explicit value from the user */
  437. memcpy(workers_gpuid,
  438. explicit_workers_gpuid,
  439. STARPU_NMAXWORKERS*sizeof(unsigned));
  440. }
  441. else
  442. {
  443. /* by default, we take a round robin policy */
  444. if (nhwgpus > 0)
  445. for (i = 0; i < STARPU_NMAXWORKERS; i++)
  446. workers_gpuid[i] = (unsigned)(i % nhwgpus);
  447. /* StarPU can use sampling techniques to bind threads
  448. * correctly */
  449. may_bind_automatically[type] = 1;
  450. }
  451. }
  452. #endif
  453. #if defined(STARPU_USE_CUDA) || defined(STARPU_SIMGRID)
  454. static void _starpu_initialize_workers_cuda_gpuid(struct _starpu_machine_config *config)
  455. {
  456. struct _starpu_machine_topology *topology = &config->topology;
  457. struct starpu_conf *uconf = &config->conf;
  458. _starpu_initialize_workers_deviceid(uconf->use_explicit_workers_cuda_gpuid == 0
  459. ? NULL
  460. : (int *)uconf->workers_cuda_gpuid,
  461. &(config->current_cuda_gpuid),
  462. (int *)topology->workers_cuda_gpuid,
  463. "STARPU_WORKERS_CUDAID",
  464. topology->nhwcudagpus,
  465. STARPU_CUDA_WORKER);
  466. }
  467. static inline int _starpu_get_next_cuda_gpuid(struct _starpu_machine_config *config)
  468. {
  469. unsigned i = ((config->current_cuda_gpuid++) % config->topology.ncudagpus);
  470. return (int)config->topology.workers_cuda_gpuid[i];
  471. }
  472. #endif
  473. #if defined(STARPU_USE_OPENCL) || defined(STARPU_SIMGRID)
  474. static void _starpu_initialize_workers_opencl_gpuid(struct _starpu_machine_config*config)
  475. {
  476. struct _starpu_machine_topology *topology = &config->topology;
  477. struct starpu_conf *uconf = &config->conf;
  478. _starpu_initialize_workers_deviceid(uconf->use_explicit_workers_opencl_gpuid == 0
  479. ? NULL
  480. : (int *)uconf->workers_opencl_gpuid,
  481. &(config->current_opencl_gpuid),
  482. (int *)topology->workers_opencl_gpuid,
  483. "STARPU_WORKERS_OPENCLID",
  484. topology->nhwopenclgpus,
  485. STARPU_OPENCL_WORKER);
  486. #if defined(STARPU_USE_CUDA) || defined(STARPU_SIMGRID)
  487. // Detect devices which are already used with CUDA
  488. {
  489. unsigned tmp[STARPU_NMAXWORKERS];
  490. unsigned nb=0;
  491. int i;
  492. for(i=0 ; i<STARPU_NMAXWORKERS ; i++)
  493. {
  494. struct handle_entry *entry;
  495. int devid = config->topology.workers_opencl_gpuid[i];
  496. HASH_FIND_INT(devices_using_cuda, &devid, entry);
  497. if (entry == NULL)
  498. {
  499. tmp[nb] = topology->workers_opencl_gpuid[i];
  500. nb++;
  501. }
  502. }
  503. for (i=nb ; i<STARPU_NMAXWORKERS ; i++)
  504. tmp[i] = -1;
  505. memcpy(topology->workers_opencl_gpuid, tmp, sizeof(unsigned)*STARPU_NMAXWORKERS);
  506. }
  507. #endif /* STARPU_USE_CUDA */
  508. {
  509. // Detect identical devices
  510. struct handle_entry *devices_already_used = NULL;
  511. unsigned tmp[STARPU_NMAXWORKERS];
  512. unsigned nb=0;
  513. int i;
  514. for(i=0 ; i<STARPU_NMAXWORKERS ; i++)
  515. {
  516. int devid = topology->workers_opencl_gpuid[i];
  517. struct handle_entry *entry;
  518. HASH_FIND_INT(devices_already_used, &devid, entry);
  519. if (entry == NULL)
  520. {
  521. struct handle_entry *entry2;
  522. _STARPU_MALLOC(entry2, sizeof(*entry2));
  523. entry2->gpuid = devid;
  524. HASH_ADD_INT(devices_already_used, gpuid,
  525. entry2);
  526. tmp[nb] = devid;
  527. nb ++;
  528. }
  529. }
  530. struct handle_entry *entry=NULL, *tempo=NULL;
  531. HASH_ITER(hh, devices_already_used, entry, tempo)
  532. {
  533. HASH_DEL(devices_already_used, entry);
  534. free(entry);
  535. }
  536. for (i=nb ; i<STARPU_NMAXWORKERS ; i++)
  537. tmp[i] = -1;
  538. memcpy(topology->workers_opencl_gpuid, tmp, sizeof(unsigned)*STARPU_NMAXWORKERS);
  539. }
  540. }
  541. static inline int _starpu_get_next_opencl_gpuid(struct _starpu_machine_config *config)
  542. {
  543. unsigned i = ((config->current_opencl_gpuid++) % config->topology.nopenclgpus);
  544. return (int)config->topology.workers_opencl_gpuid[i];
  545. }
  546. #endif
  547. #if 0
  548. #if defined(STARPU_USE_MIC) || defined(STARPU_SIMGRID)
  549. static void _starpu_initialize_workers_mic_deviceid(struct _starpu_machine_config *config)
  550. {
  551. struct _starpu_machine_topology *topology = &config->topology;
  552. struct starpu_conf *uconf = &config->conf;
  553. _starpu_initialize_workers_deviceid(uconf->use_explicit_workers_mic_deviceid == 0
  554. ? NULL
  555. : (int *)config->user_conf->workers_mic_deviceid,
  556. &(config->current_mic_deviceid),
  557. (int *)topology->workers_mic_deviceid,
  558. "STARPU_WORKERS_MICID",
  559. topology->nhwmiccores,
  560. STARPU_MIC_WORKER);
  561. }
  562. #endif
  563. #endif
  564. #if 0
  565. #ifdef STARPU_USE_MIC
  566. static inline int _starpu_get_next_mic_deviceid(struct _starpu_machine_config *config)
  567. {
  568. unsigned i = ((config->current_mic_deviceid++) % config->topology.nmicdevices);
  569. return (int)config->topology.workers_mic_deviceid[i];
  570. }
  571. #endif
  572. #endif
  573. #ifdef STARPU_USE_MPI_MASTER_SLAVE
  574. static inline int _starpu_get_next_mpi_deviceid(struct _starpu_machine_config *config)
  575. {
  576. unsigned i = ((config->current_mpi_deviceid++) % config->topology.nmpidevices);
  577. return (int)config->topology.workers_mpi_ms_deviceid[i];
  578. }
  579. static void _starpu_init_mpi_topology(struct _starpu_machine_config *config, long mpi_idx)
  580. {
  581. /* Discover the topology of the mpi node identifier by MPI_IDX. That
  582. * means, make this StarPU instance aware of the number of cores available
  583. * on this MPI device. Update the `nhwmpicores' topology field
  584. * accordingly. */
  585. struct _starpu_machine_topology *topology = &config->topology;
  586. int nbcores;
  587. _starpu_src_common_sink_nbcores(_starpu_mpi_ms_nodes[mpi_idx], &nbcores);
  588. topology->nhwmpicores[mpi_idx] = nbcores;
  589. }
  590. #endif /* STARPU_USE_MPI_MASTER_SLAVE */
  591. #ifdef STARPU_USE_MIC
  592. static void _starpu_init_mic_topology(struct _starpu_machine_config *config, long mic_idx)
  593. {
  594. /* Discover the topology of the mic node identifier by MIC_IDX. That
  595. * means, make this StarPU instance aware of the number of cores available
  596. * on this MIC device. Update the `nhwmiccores' topology field
  597. * accordingly. */
  598. struct _starpu_machine_topology *topology = &config->topology;
  599. int nbcores;
  600. _starpu_src_common_sink_nbcores(_starpu_mic_nodes[mic_idx], &nbcores);
  601. topology->nhwmiccores[mic_idx] = nbcores;
  602. }
  603. static int _starpu_init_mic_node(struct _starpu_machine_config *config, int mic_idx,
  604. COIENGINE *coi_handle, COIPROCESS *coi_process)
  605. {
  606. /* Initialize the MIC node of index MIC_IDX. */
  607. struct starpu_conf *user_conf = &config->conf;
  608. char ***argv = _starpu_get_argv();
  609. const char *suffixes[] = {"-mic", "_mic", NULL};
  610. /* Environment variables to send to the Sink, it informs it what kind
  611. * of node it is (architecture and type) as there is no way to discover
  612. * it itself */
  613. char mic_idx_env[32];
  614. snprintf(mic_idx_env, sizeof(mic_idx_env), "_STARPU_MIC_DEVID=%d", mic_idx);
  615. /* XXX: this is currently necessary so that the remote process does not
  616. * segfault. */
  617. char nb_mic_env[32];
  618. snprintf(nb_mic_env, sizeof(nb_mic_env), "_STARPU_MIC_NB=%d", 2);
  619. const char *mic_sink_env[] = {"STARPU_SINK=STARPU_MIC", mic_idx_env, nb_mic_env, NULL};
  620. char mic_sink_program_path[1024];
  621. /* Let's get the helper program to run on the MIC device */
  622. int mic_file_found = _starpu_src_common_locate_file(mic_sink_program_path,
  623. sizeof(mic_sink_program_path),
  624. starpu_getenv("STARPU_MIC_SINK_PROGRAM_NAME"),
  625. starpu_getenv("STARPU_MIC_SINK_PROGRAM_PATH"),
  626. user_conf->mic_sink_program_path,
  627. (argv ? (*argv)[0] : NULL),
  628. suffixes);
  629. if (0 != mic_file_found)
  630. {
  631. _STARPU_MSG("No MIC program specified, use the environment\n"
  632. "variable STARPU_MIC_SINK_PROGRAM_NAME or the environment\n"
  633. "or the field 'starpu_conf.mic_sink_program_path'\n"
  634. "to define it.\n");
  635. return -1;
  636. }
  637. COIRESULT res;
  638. /* Let's get the handle which let us manage the remote MIC device */
  639. res = COIEngineGetHandle(COI_ISA_MIC, mic_idx, coi_handle);
  640. if (STARPU_UNLIKELY(res != COI_SUCCESS))
  641. STARPU_MIC_SRC_REPORT_COI_ERROR(res);
  642. /* We launch the helper on the MIC device, which will wait for us
  643. * to give it work to do.
  644. * As we will communicate further with the device throught scif we
  645. * don't need to keep the process pointer */
  646. res = COIProcessCreateFromFile(*coi_handle, mic_sink_program_path, 0, NULL, 0,
  647. mic_sink_env, 1, NULL, 0, NULL,
  648. coi_process);
  649. if (STARPU_UNLIKELY(res != COI_SUCCESS))
  650. STARPU_MIC_SRC_REPORT_COI_ERROR(res);
  651. /* Let's create the node structure, we'll communicate with the peer
  652. * through scif thanks to it */
  653. _starpu_mic_nodes[mic_idx] =
  654. _starpu_mp_common_node_create(STARPU_NODE_MIC_SOURCE, mic_idx);
  655. return 0;
  656. }
  657. #endif
  658. #ifndef STARPU_SIMGRID
  659. #ifdef STARPU_HAVE_HWLOC
  660. static void _starpu_allocate_topology_userdata(hwloc_obj_t obj)
  661. {
  662. unsigned i;
  663. _STARPU_CALLOC(obj->userdata, 1, sizeof(struct _starpu_hwloc_userdata));
  664. for (i = 0; i < obj->arity; i++)
  665. _starpu_allocate_topology_userdata(obj->children[i]);
  666. #if HWLOC_API_VERSION >= 0x00020000
  667. hwloc_obj_t child;
  668. for (child = obj->io_first_child; child; child = child->next_sibling)
  669. _starpu_allocate_topology_userdata(child);
  670. #endif
  671. }
  672. static void _starpu_deallocate_topology_userdata(hwloc_obj_t obj)
  673. {
  674. unsigned i;
  675. struct _starpu_hwloc_userdata *data = obj->userdata;
  676. STARPU_ASSERT(!data->worker_list || data->worker_list == (void*)-1);
  677. free(data);
  678. for (i = 0; i < obj->arity; i++)
  679. _starpu_deallocate_topology_userdata(obj->children[i]);
  680. #if HWLOC_API_VERSION >= 0x00020000
  681. hwloc_obj_t child;
  682. for (child = obj->io_first_child; child; child = child->next_sibling)
  683. _starpu_deallocate_topology_userdata(child);
  684. #endif
  685. }
  686. #endif
  687. #endif
  688. static void _starpu_init_topology(struct _starpu_machine_config *config)
  689. {
  690. /* Discover the topology, meaning finding all the available PUs for
  691. the compiled drivers. These drivers MUST have been initialized
  692. before calling this function. The discovered topology is filled in
  693. CONFIG. */
  694. struct _starpu_machine_topology *topology = &config->topology;
  695. if (topology_is_initialized)
  696. return;
  697. nobind = starpu_get_env_number("STARPU_WORKERS_NOBIND");
  698. topology->nhwcpus = 0;
  699. topology->nhwpus = 0;
  700. #ifndef STARPU_SIMGRID
  701. #ifdef STARPU_HAVE_HWLOC
  702. hwloc_topology_init(&topology->hwtopology);
  703. char *hwloc_input = starpu_getenv("STARPU_HWLOC_INPUT");
  704. if (hwloc_input && hwloc_input[0])
  705. {
  706. int err = hwloc_topology_set_xml(topology->hwtopology, hwloc_input);
  707. if (err < 0) _STARPU_DISP("Could not load hwloc input %s\n", hwloc_input);
  708. }
  709. _starpu_topology_filter(topology->hwtopology);
  710. hwloc_topology_load(topology->hwtopology);
  711. _starpu_allocate_topology_userdata(hwloc_get_root_obj(topology->hwtopology));
  712. #endif
  713. #endif
  714. #ifdef STARPU_SIMGRID
  715. config->topology.nhwcpus = config->topology.nhwpus = _starpu_simgrid_get_nbhosts("CPU");
  716. #elif defined(STARPU_HAVE_HWLOC)
  717. /* Discover the CPUs relying on the hwloc interface and fills CONFIG
  718. * accordingly. */
  719. config->cpu_depth = hwloc_get_type_depth(topology->hwtopology, HWLOC_OBJ_CORE);
  720. config->pu_depth = hwloc_get_type_depth(topology->hwtopology, HWLOC_OBJ_PU);
  721. /* Would be very odd */
  722. STARPU_ASSERT(config->cpu_depth != HWLOC_TYPE_DEPTH_MULTIPLE);
  723. if (config->cpu_depth == HWLOC_TYPE_DEPTH_UNKNOWN)
  724. {
  725. /* unknown, using logical procesors as fallback */
  726. _STARPU_DISP("Warning: The OS did not report CPU cores. Assuming there is only one hardware thread per core.\n");
  727. config->cpu_depth = hwloc_get_type_depth(topology->hwtopology,
  728. HWLOC_OBJ_PU);
  729. }
  730. topology->nhwcpus = hwloc_get_nbobjs_by_depth(topology->hwtopology, config->cpu_depth);
  731. topology->nhwpus = hwloc_get_nbobjs_by_depth(topology->hwtopology, config->pu_depth);
  732. #elif defined(HAVE_SYSCONF)
  733. /* Discover the CPUs relying on the sysconf(3) function and fills
  734. * CONFIG accordingly. */
  735. config->topology.nhwcpus = config->topology.nhwpus = sysconf(_SC_NPROCESSORS_ONLN);
  736. #elif defined(_WIN32)
  737. /* Discover the CPUs on Cygwin and MinGW systems. */
  738. SYSTEM_INFO sysinfo;
  739. GetSystemInfo(&sysinfo);
  740. config->topology.nhwcpus = config->topology.nhwpus = sysinfo.dwNumberOfProcessors;
  741. #else
  742. #warning no way to know number of cores, assuming 1
  743. config->topology.nhwcpus = config->topology.nhwpus = 1;
  744. #endif
  745. if (config->conf.ncuda != 0)
  746. _starpu_cuda_discover_devices(config);
  747. if (config->conf.nopencl != 0)
  748. _starpu_opencl_discover_devices(config);
  749. #ifdef STARPU_USE_MPI_MASTER_SLAVE
  750. config->topology.nhwmpi = _starpu_mpi_src_get_device_count();
  751. #endif
  752. topology_is_initialized = 1;
  753. }
  754. /*
  755. * Bind workers on the different processors
  756. */
  757. static void _starpu_initialize_workers_bindid(struct _starpu_machine_config *config)
  758. {
  759. char *strval;
  760. unsigned i;
  761. struct _starpu_machine_topology *topology = &config->topology;
  762. config->current_bindid = 0;
  763. /* conf->workers_bindid indicates the successive logical PU identifier that
  764. * should be used to bind the workers. It should be either filled
  765. * according to the user's explicit parameters (from starpu_conf) or
  766. * according to the STARPU_WORKERS_CPUID env. variable. Otherwise, a
  767. * round-robin policy is used to distributed the workers over the
  768. * cores. */
  769. /* what do we use, explicit value, env. variable, or round-robin ? */
  770. strval = starpu_getenv("STARPU_WORKERS_CPUID");
  771. if (strval)
  772. {
  773. /* STARPU_WORKERS_CPUID certainly contains less entries than
  774. * STARPU_NMAXWORKERS, so we reuse its entries in a round
  775. * robin fashion: "1 2" is equivalent to "1 2 1 2 1 2 .... 1
  776. * 2". */
  777. unsigned wrap = 0;
  778. unsigned number_of_entries = 0;
  779. char *endptr;
  780. /* we use the content of the STARPU_WORKERS_CPUID
  781. * env. variable */
  782. for (i = 0; i < STARPU_NMAXWORKERS; i++)
  783. {
  784. if (!wrap)
  785. {
  786. long int val;
  787. val = strtol(strval, &endptr, 10);
  788. if (endptr != strval)
  789. {
  790. topology->workers_bindid[i] = (unsigned)(val % topology->nhwpus);
  791. strval = endptr;
  792. if (*strval == '-')
  793. {
  794. /* range of values */
  795. long int endval;
  796. strval++;
  797. if (*strval && *strval != ' ' && *strval != ',')
  798. {
  799. endval = strtol(strval, &endptr, 10);
  800. strval = endptr;
  801. }
  802. else
  803. {
  804. endval = topology->nhwpus-1;
  805. if (*strval)
  806. strval++;
  807. }
  808. for (val++; val <= endval && i < STARPU_NMAXWORKERS-1; val++)
  809. {
  810. i++;
  811. topology->workers_bindid[i] = (unsigned)(val % topology->nhwpus);
  812. }
  813. }
  814. if (*strval == ',')
  815. strval++;
  816. }
  817. else
  818. {
  819. /* there must be at least one entry */
  820. STARPU_ASSERT(i != 0);
  821. number_of_entries = i;
  822. /* there is no more values in the
  823. * string */
  824. wrap = 1;
  825. topology->workers_bindid[i] =
  826. topology->workers_bindid[0];
  827. }
  828. }
  829. else
  830. {
  831. topology->workers_bindid[i] =
  832. topology->workers_bindid[i % number_of_entries];
  833. }
  834. }
  835. }
  836. else if (config->conf.use_explicit_workers_bindid)
  837. {
  838. /* we use the explicit value from the user */
  839. memcpy(topology->workers_bindid,
  840. config->conf.workers_bindid,
  841. STARPU_NMAXWORKERS*sizeof(unsigned));
  842. }
  843. else
  844. {
  845. int nth_per_core = starpu_get_env_number_default("STARPU_NTHREADS_PER_CORE", 1);
  846. int k;
  847. int nbindids=0;
  848. int nhyperthreads = topology->nhwpus / topology->nhwcpus;
  849. STARPU_ASSERT_MSG(nth_per_core > 0 && nth_per_core <= nhyperthreads , "Incorrect number of hyperthreads");
  850. i = 0; /* PU number currently assigned */
  851. k = 0; /* Number of threads already put on the current core */
  852. while(nbindids < STARPU_NMAXWORKERS)
  853. {
  854. if (k >= nth_per_core)
  855. {
  856. /* We have already put enough workers on this
  857. * core, skip remaining PUs from this core, and
  858. * proceed with next core */
  859. i += nhyperthreads-nth_per_core;
  860. k = 0;
  861. continue;
  862. }
  863. /* Add a worker to this core, by using this logical PU */
  864. topology->workers_bindid[nbindids++] = (unsigned)(i % topology->nhwpus);
  865. k++;
  866. i++;
  867. }
  868. }
  869. for (i = 0; i < STARPU_MAXCPUS;i++)
  870. cpu_worker[i] = STARPU_NOWORKERID;
  871. /* no binding yet */
  872. memset(&config->currently_bound, 0, sizeof(config->currently_bound));
  873. memset(&config->currently_shared, 0, sizeof(config->currently_shared));
  874. }
  875. static void _starpu_deinitialize_workers_bindid(struct _starpu_machine_config *config STARPU_ATTRIBUTE_UNUSED)
  876. {
  877. unsigned i;
  878. for (i = 0; i < STARPU_MAXCPUS;i++)
  879. {
  880. if (cpu_name[i])
  881. {
  882. free(cpu_name[i]);
  883. cpu_name[i] = NULL;
  884. }
  885. }
  886. }
  887. /* This function gets the identifier of the next core on which to bind a
  888. * worker. In case a list of preferred cores was specified (logical indexes),
  889. * we look for a an available core among the list if possible, otherwise a
  890. * round-robin policy is used. */
  891. static inline unsigned _starpu_get_next_bindid(struct _starpu_machine_config *config, unsigned flags,
  892. unsigned *preferred_binding, unsigned npreferred)
  893. {
  894. struct _starpu_machine_topology *topology = &config->topology;
  895. unsigned current_preferred;
  896. unsigned nhyperthreads = topology->nhwpus / topology->nhwcpus;
  897. unsigned ncores = topology->nhwpus / nhyperthreads;
  898. unsigned i;
  899. if (npreferred)
  900. {
  901. STARPU_ASSERT_MSG(preferred_binding, "Passing NULL pointer for parameter preferred_binding with a non-0 value of parameter npreferred");
  902. }
  903. /* loop over the preference list */
  904. for (current_preferred = 0;
  905. current_preferred < npreferred;
  906. current_preferred++)
  907. {
  908. /* can we bind the worker on the preferred core ? */
  909. unsigned requested_core = preferred_binding[current_preferred];
  910. unsigned requested_bindid = requested_core * nhyperthreads;
  911. /* Look at the remaining cores to be bound to */
  912. for (i = 0; i < ncores; i++)
  913. {
  914. if (topology->workers_bindid[i] == requested_bindid &&
  915. (!config->currently_bound[i] ||
  916. (config->currently_shared[i] && !(flags & STARPU_THREAD_ACTIVE)))
  917. )
  918. {
  919. /* the cpu is available, or shareable with us, we use it ! */
  920. config->currently_bound[i] = 1;
  921. if (!(flags & STARPU_THREAD_ACTIVE))
  922. config->currently_shared[i] = 1;
  923. return requested_bindid;
  924. }
  925. }
  926. }
  927. if (!(flags & STARPU_THREAD_ACTIVE))
  928. {
  929. /* Try to find a shareable PU */
  930. for (i = 0; i < ncores; i++)
  931. if (config->currently_shared[i])
  932. return topology->workers_bindid[i];
  933. }
  934. /* Try to find an available PU from last used PU */
  935. for (i = config->current_bindid; i < ncores; i++)
  936. if (!config->currently_bound[i])
  937. /* Found a cpu ready for use, use it! */
  938. break;
  939. if (i == ncores)
  940. {
  941. /* Finished binding on all cpus, restart from start in
  942. * case the user really wants overloading */
  943. memset(&config->currently_bound, 0, sizeof(config->currently_bound));
  944. i = 0;
  945. }
  946. STARPU_ASSERT(i < ncores);
  947. unsigned bindid = topology->workers_bindid[i];
  948. config->currently_bound[i] = 1;
  949. if (!(flags & STARPU_THREAD_ACTIVE))
  950. config->currently_shared[i] = 1;
  951. config->current_bindid = i;
  952. return bindid;
  953. }
  954. unsigned starpu_get_next_bindid(unsigned flags, unsigned *preferred, unsigned npreferred)
  955. {
  956. return _starpu_get_next_bindid(_starpu_get_machine_config(), flags, preferred, npreferred);
  957. }
  958. unsigned _starpu_topology_get_nhwcpu(struct _starpu_machine_config *config)
  959. {
  960. #if defined(STARPU_USE_OPENCL) || defined(STARPU_SIMGRID)
  961. if (config->conf.nopencl != 0)
  962. _starpu_opencl_init();
  963. #endif
  964. #if defined(STARPU_USE_CUDA) || defined(STARPU_SIMGRID)
  965. if (config->conf.ncuda != 0)
  966. _starpu_init_cuda();
  967. #endif
  968. _starpu_init_topology(config);
  969. return config->topology.nhwcpus;
  970. }
  971. unsigned _starpu_topology_get_nhwpu(struct _starpu_machine_config *config)
  972. {
  973. #if defined(STARPU_USE_OPENCL) || defined(STARPU_SIMGRID)
  974. if (config->conf.nopencl != 0)
  975. _starpu_opencl_init();
  976. #endif
  977. #if defined(STARPU_USE_CUDA) || defined(STARPU_SIMGRID)
  978. if (config->conf.ncuda != 0)
  979. _starpu_init_cuda();
  980. #endif
  981. _starpu_init_topology(config);
  982. return config->topology.nhwpus;
  983. }
  984. unsigned _starpu_topology_get_nnumanodes(struct _starpu_machine_config *config STARPU_ATTRIBUTE_UNUSED)
  985. {
  986. #if defined(STARPU_USE_OPENCL) || defined(STARPU_SIMGRID)
  987. if (config->conf.nopencl != 0)
  988. _starpu_opencl_init();
  989. #endif
  990. #if defined(STARPU_USE_CUDA) || defined(STARPU_SIMGRID)
  991. if (config->conf.ncuda != 0)
  992. _starpu_init_cuda();
  993. #endif
  994. _starpu_init_topology(config);
  995. int res;
  996. #if defined(STARPU_HAVE_HWLOC)
  997. if (numa_enabled == -1)
  998. numa_enabled = starpu_get_env_number_default("STARPU_USE_NUMA", 0);
  999. if (numa_enabled)
  1000. {
  1001. struct _starpu_machine_topology *topology = &config->topology ;
  1002. int nnumanodes = hwloc_get_nbobjs_by_type(topology->hwtopology, HWLOC_OBJ_NUMANODE) ;
  1003. res = nnumanodes > 0 ? nnumanodes : 1 ;
  1004. }
  1005. else
  1006. #endif
  1007. {
  1008. res = 1;
  1009. }
  1010. STARPU_ASSERT_MSG(res <= STARPU_MAXNUMANODES, "Number of NUMA nodes discovered is higher than maximum accepted ! Use configure option --enable-maxnumanodes=xxx to increase the maximum value of supported NUMA nodes.\n");
  1011. return res;
  1012. }
  1013. #ifdef STARPU_HAVE_HWLOC
  1014. void _starpu_topology_filter(hwloc_topology_t topology)
  1015. {
  1016. #if HWLOC_API_VERSION >= 0x20000
  1017. hwloc_topology_set_io_types_filter(topology, HWLOC_TYPE_FILTER_KEEP_IMPORTANT);
  1018. hwloc_topology_set_flags(topology, HWLOC_TOPOLOGY_FLAG_IS_THISSYSTEM);
  1019. #else
  1020. hwloc_topology_set_flags(topology, HWLOC_TOPOLOGY_FLAG_IS_THISSYSTEM | HWLOC_TOPOLOGY_FLAG_IO_DEVICES | HWLOC_TOPOLOGY_FLAG_IO_BRIDGES);
  1021. #endif
  1022. #ifdef HAVE_HWLOC_TOPOLOGY_SET_COMPONENTS
  1023. # ifndef STARPU_USE_CUDA
  1024. hwloc_topology_set_components(topology, HWLOC_TOPOLOGY_COMPONENTS_FLAG_BLACKLIST, "cuda");
  1025. hwloc_topology_set_components(topology, HWLOC_TOPOLOGY_COMPONENTS_FLAG_BLACKLIST, "nvml");
  1026. # endif
  1027. # ifndef STARPU_USE_OPENCL
  1028. hwloc_topology_set_components(topology, HWLOC_TOPOLOGY_COMPONENTS_FLAG_BLACKLIST, "opencl");
  1029. # endif
  1030. #endif
  1031. }
  1032. #endif
  1033. #ifdef STARPU_USE_MIC
  1034. static void _starpu_init_mic_config(struct _starpu_machine_config *config,
  1035. struct starpu_conf *user_conf,
  1036. unsigned mic_idx)
  1037. {
  1038. // Configure the MIC device of index MIC_IDX.
  1039. struct _starpu_machine_topology *topology = &config->topology;
  1040. topology->nhwmiccores[mic_idx] = 0;
  1041. _starpu_init_mic_topology(config, mic_idx);
  1042. int nmiccores;
  1043. nmiccores = starpu_get_env_number("STARPU_NMICTHREADS");
  1044. STARPU_ASSERT_MSG(nmiccores >= -1, "nmiccores can not be negative and different from -1 (is is %d)", nmiccores);
  1045. if (nmiccores == -1)
  1046. {
  1047. /* Nothing was specified, so let's use the number of
  1048. * detected mic cores. ! */
  1049. nmiccores = topology->nhwmiccores[mic_idx];
  1050. }
  1051. else
  1052. {
  1053. if ((unsigned) nmiccores > topology->nhwmiccores[mic_idx])
  1054. {
  1055. /* The user requires more MIC cores than there is available */
  1056. _STARPU_MSG("# Warning: %d MIC cores requested. Only %u available.\n", nmiccores, topology->nhwmiccores[mic_idx]);
  1057. nmiccores = topology->nhwmiccores[mic_idx];
  1058. }
  1059. }
  1060. topology->nmiccores[mic_idx] = nmiccores;
  1061. STARPU_ASSERT_MSG(topology->nmiccores[mic_idx] + topology->nworkers <= STARPU_NMAXWORKERS,
  1062. "topology->nmiccores[mic_idx(%u)] (%u) + topology->nworkers (%u) <= STARPU_NMAXWORKERS (%d)",
  1063. mic_idx, topology->nmiccores[mic_idx], topology->nworkers, STARPU_NMAXWORKERS);
  1064. /* _starpu_initialize_workers_mic_deviceid (config); */
  1065. mic_worker_set[mic_idx].workers = &config->workers[topology->nworkers];
  1066. mic_worker_set[mic_idx].nworkers = topology->nmiccores[mic_idx];
  1067. unsigned miccore_id;
  1068. for (miccore_id = 0; miccore_id < topology->nmiccores[mic_idx]; miccore_id++)
  1069. {
  1070. int worker_idx = topology->nworkers + miccore_id;
  1071. config->workers[worker_idx].set = &mic_worker_set[mic_idx];
  1072. config->workers[worker_idx].arch = STARPU_MIC_WORKER;
  1073. _STARPU_MALLOC(config->workers[worker_idx].perf_arch.devices, sizeof(struct starpu_perfmodel_device));
  1074. config->workers[worker_idx].perf_arch.ndevices = 1;
  1075. config->workers[worker_idx].perf_arch.devices[0].type = STARPU_MIC_WORKER;
  1076. config->workers[worker_idx].perf_arch.devices[0].devid = mic_idx;
  1077. config->workers[worker_idx].perf_arch.devices[0].ncores = 1;
  1078. config->workers[worker_idx].devid = mic_idx;
  1079. config->workers[worker_idx].subworkerid = miccore_id;
  1080. config->workers[worker_idx].worker_mask = STARPU_MIC;
  1081. config->worker_mask |= STARPU_MIC;
  1082. }
  1083. _starpu_mic_nodes[mic_idx]->baseworkerid = topology->nworkers;
  1084. topology->nworkers += topology->nmiccores[mic_idx];
  1085. }
  1086. static COIENGINE mic_handles[STARPU_MAXMICDEVS];
  1087. COIPROCESS _starpu_mic_process[STARPU_MAXMICDEVS];
  1088. #endif
  1089. #ifdef STARPU_USE_MPI_MASTER_SLAVE
  1090. static void _starpu_init_mpi_config(struct _starpu_machine_config *config,
  1091. struct starpu_conf *user_conf,
  1092. unsigned mpi_idx)
  1093. {
  1094. struct _starpu_machine_topology *topology = &config->topology;
  1095. topology->nhwmpicores[mpi_idx] = 0;
  1096. _starpu_init_mpi_topology(config, mpi_idx);
  1097. int nmpicores;
  1098. nmpicores = starpu_get_env_number("STARPU_NMPIMSTHREADS");
  1099. if (nmpicores == -1)
  1100. {
  1101. /* Nothing was specified, so let's use the number of
  1102. * detected mpi cores. ! */
  1103. nmpicores = topology->nhwmpicores[mpi_idx];
  1104. }
  1105. else
  1106. {
  1107. if ((unsigned) nmpicores > topology->nhwmpicores[mpi_idx])
  1108. {
  1109. /* The user requires more MPI cores than there is available */
  1110. _STARPU_MSG("# Warning: %d MPI cores requested. Only %u available.\n",
  1111. nmpicores, topology->nhwmpicores[mpi_idx]);
  1112. nmpicores = topology->nhwmpicores[mpi_idx];
  1113. }
  1114. }
  1115. topology->nmpicores[mpi_idx] = nmpicores;
  1116. STARPU_ASSERT_MSG(topology->nmpicores[mpi_idx] + topology->nworkers <= STARPU_NMAXWORKERS,
  1117. "topology->nmpicores[mpi_idx(%u)] (%u) + topology->nworkers (%u) <= STARPU_NMAXWORKERS (%d)",
  1118. mpi_idx, topology->nmpicores[mpi_idx], topology->nworkers, STARPU_NMAXWORKERS);
  1119. mpi_worker_set[mpi_idx].workers = &config->workers[topology->nworkers];
  1120. mpi_worker_set[mpi_idx].nworkers = topology->nmpicores[mpi_idx];
  1121. unsigned mpicore_id;
  1122. for (mpicore_id = 0; mpicore_id < topology->nmpicores[mpi_idx]; mpicore_id++)
  1123. {
  1124. int worker_idx = topology->nworkers + mpicore_id;
  1125. config->workers[worker_idx].set = &mpi_worker_set[mpi_idx];
  1126. config->workers[worker_idx].arch = STARPU_MPI_MS_WORKER;
  1127. _STARPU_MALLOC(config->workers[worker_idx].perf_arch.devices, sizeof(struct starpu_perfmodel_device));
  1128. config->workers[worker_idx].perf_arch.ndevices = 1;
  1129. config->workers[worker_idx].perf_arch.devices[0].type = STARPU_MPI_MS_WORKER;
  1130. config->workers[worker_idx].perf_arch.devices[0].devid = mpi_idx;
  1131. config->workers[worker_idx].perf_arch.devices[0].ncores = 1;
  1132. config->workers[worker_idx].devid = mpi_idx;
  1133. config->workers[worker_idx].subworkerid = mpicore_id;
  1134. config->workers[worker_idx].worker_mask = STARPU_MPI_MS;
  1135. config->worker_mask |= STARPU_MPI_MS;
  1136. }
  1137. _starpu_mpi_ms_nodes[mpi_idx]->baseworkerid = topology->nworkers;
  1138. topology->nworkers += topology->nmpicores[mpi_idx];
  1139. }
  1140. #endif
  1141. #if defined(STARPU_USE_MIC) || defined(STARPU_USE_MPI_MASTER_SLAVE)
  1142. static void _starpu_init_mp_config(struct _starpu_machine_config *config,
  1143. struct starpu_conf *user_conf, int no_mp_config)
  1144. {
  1145. /* Discover and configure the mp topology. That means:
  1146. * - discover the number of mp nodes;
  1147. * - initialize each discovered node;
  1148. * - discover the local topology (number of PUs/devices) of each node;
  1149. * - configure the workers accordingly.
  1150. */
  1151. #ifdef STARPU_USE_MIC
  1152. if (!no_mp_config)
  1153. {
  1154. struct _starpu_machine_topology *topology = &config->topology;
  1155. /* Discover and initialize the number of MIC nodes through the mp
  1156. * infrastructure. */
  1157. unsigned nhwmicdevices = _starpu_mic_src_get_device_count();
  1158. int reqmicdevices = starpu_get_env_number("STARPU_NMIC");
  1159. if (reqmicdevices == -1 && user_conf)
  1160. reqmicdevices = user_conf->nmic;
  1161. if (reqmicdevices == -1)
  1162. /* Nothing was specified, so let's use the number of
  1163. * detected mic devices. ! */
  1164. reqmicdevices = nhwmicdevices;
  1165. STARPU_ASSERT_MSG(reqmicdevices >= -1, "nmic can not be negative and different from -1 (is is %d)", reqmicdevices);
  1166. if (reqmicdevices != -1)
  1167. {
  1168. if ((unsigned) reqmicdevices > nhwmicdevices)
  1169. {
  1170. /* The user requires more MIC devices than there is available */
  1171. _STARPU_MSG("# Warning: %d MIC devices requested. Only %u available.\n", reqmicdevices, nhwmicdevices);
  1172. reqmicdevices = nhwmicdevices;
  1173. }
  1174. }
  1175. topology->nmicdevices = 0;
  1176. unsigned i;
  1177. for (i = 0; i < (unsigned) reqmicdevices; i++)
  1178. if (0 == _starpu_init_mic_node(config, i, &mic_handles[i], &_starpu_mic_process[i]))
  1179. topology->nmicdevices++;
  1180. for (i = 0; i < topology->nmicdevices; i++)
  1181. _starpu_init_mic_config(config, user_conf, i);
  1182. }
  1183. #endif
  1184. #ifdef STARPU_USE_MPI_MASTER_SLAVE
  1185. {
  1186. struct _starpu_machine_topology *topology = &config->topology;
  1187. /* Discover and initialize the number of MPI nodes through the mp
  1188. * infrastructure. */
  1189. unsigned nhwmpidevices = _starpu_mpi_src_get_device_count();
  1190. int reqmpidevices = starpu_get_env_number("STARPU_NMPI_MS");
  1191. if (reqmpidevices == -1 && user_conf)
  1192. reqmpidevices = user_conf->nmpi_ms;
  1193. if (reqmpidevices == -1)
  1194. /* Nothing was specified, so let's use the number of
  1195. * detected mpi devices. ! */
  1196. reqmpidevices = nhwmpidevices;
  1197. if (reqmpidevices != -1)
  1198. {
  1199. if ((unsigned) reqmpidevices > nhwmpidevices)
  1200. {
  1201. /* The user requires more MPI devices than there is available */
  1202. _STARPU_MSG("# Warning: %d MPI Master-Slave devices requested. Only %u available.\n",
  1203. reqmpidevices, nhwmpidevices);
  1204. reqmpidevices = nhwmpidevices;
  1205. }
  1206. }
  1207. topology->nmpidevices = reqmpidevices;
  1208. /* if user don't want to use MPI slaves, we close the slave processes */
  1209. if (no_mp_config && topology->nmpidevices == 0)
  1210. {
  1211. _starpu_mpi_common_mp_deinit();
  1212. exit(0);
  1213. }
  1214. if (!no_mp_config)
  1215. {
  1216. unsigned i;
  1217. for (i = 0; i < topology->nmpidevices; i++)
  1218. _starpu_mpi_ms_nodes[i] = _starpu_mp_common_node_create(STARPU_NODE_MPI_SOURCE, i);
  1219. for (i = 0; i < topology->nmpidevices; i++)
  1220. _starpu_init_mpi_config(config, user_conf, i);
  1221. }
  1222. }
  1223. #endif
  1224. }
  1225. #endif
  1226. #ifdef STARPU_USE_MIC
  1227. static void _starpu_deinit_mic_node(unsigned mic_idx)
  1228. {
  1229. _starpu_mp_common_send_command(_starpu_mic_nodes[mic_idx], STARPU_MP_COMMAND_EXIT, NULL, 0);
  1230. COIProcessDestroy(_starpu_mic_process[mic_idx], -1, 0, NULL, NULL);
  1231. _starpu_mp_common_node_destroy(_starpu_mic_nodes[mic_idx]);
  1232. }
  1233. #endif
  1234. #ifdef STARPU_USE_MPI_MASTER_SLAVE
  1235. static void _starpu_deinit_mpi_node(int devid)
  1236. {
  1237. _starpu_mp_common_send_command(_starpu_mpi_ms_nodes[devid], STARPU_MP_COMMAND_EXIT, NULL, 0);
  1238. _starpu_mp_common_node_destroy(_starpu_mpi_ms_nodes[devid]);
  1239. }
  1240. #endif
  1241. #if defined(STARPU_USE_MIC) || defined(STARPU_USE_MPI_MASTER_SLAVE)
  1242. static void _starpu_deinit_mp_config(struct _starpu_machine_config *config)
  1243. {
  1244. struct _starpu_machine_topology *topology = &config->topology;
  1245. unsigned i;
  1246. #ifdef STARPU_USE_MIC
  1247. for (i = 0; i < topology->nmicdevices; i++)
  1248. _starpu_deinit_mic_node(i);
  1249. _starpu_mic_clear_kernels();
  1250. #endif
  1251. #ifdef STARPU_USE_MPI_MASTER_SLAVE
  1252. for (i = 0; i < topology->nmpidevices; i++)
  1253. _starpu_deinit_mpi_node(i);
  1254. #endif
  1255. }
  1256. #endif
  1257. #ifdef STARPU_HAVE_HWLOC
  1258. static unsigned _starpu_topology_count_ngpus(hwloc_obj_t obj)
  1259. {
  1260. struct _starpu_hwloc_userdata *data = obj->userdata;
  1261. unsigned n = data->ngpus;
  1262. unsigned i;
  1263. for (i = 0; i < obj->arity; i++)
  1264. n += _starpu_topology_count_ngpus(obj->children[i]);
  1265. data->ngpus = n;
  1266. //#ifdef STARPU_VERBOSE
  1267. // {
  1268. // char name[64];
  1269. // hwloc_obj_type_snprintf(name, sizeof(name), obj, 0);
  1270. // _STARPU_DEBUG("hwloc obj %s has %u GPUs below\n", name, n);
  1271. // }
  1272. //#endif
  1273. return n;
  1274. }
  1275. #endif
  1276. static int _starpu_init_machine_config(struct _starpu_machine_config *config, int no_mp_config STARPU_ATTRIBUTE_UNUSED)
  1277. {
  1278. int i;
  1279. for (i = 0; i < STARPU_NMAXWORKERS; i++)
  1280. {
  1281. config->workers[i].workerid = i;
  1282. config->workers[i].set = NULL;
  1283. }
  1284. struct _starpu_machine_topology *topology = &config->topology;
  1285. topology->nworkers = 0;
  1286. topology->ncombinedworkers = 0;
  1287. topology->nsched_ctxs = 0;
  1288. #if defined(STARPU_USE_OPENCL) || defined(STARPU_SIMGRID)
  1289. if (config->conf.nopencl != 0)
  1290. _starpu_opencl_init();
  1291. #endif
  1292. #if defined(STARPU_USE_CUDA) || defined(STARPU_SIMGRID)
  1293. if (config->conf.ncuda != 0)
  1294. _starpu_init_cuda();
  1295. #endif
  1296. _starpu_init_topology(config);
  1297. _starpu_initialize_workers_bindid(config);
  1298. #if defined(STARPU_USE_CUDA) || defined(STARPU_SIMGRID)
  1299. for (i = 0; i < (int) (sizeof(cuda_worker_set)/sizeof(cuda_worker_set[0])); i++)
  1300. cuda_worker_set[i].workers = NULL;
  1301. #endif
  1302. #ifdef STARPU_USE_MIC
  1303. for (i = 0; i < (int) (sizeof(mic_worker_set)/sizeof(mic_worker_set[0])); i++)
  1304. mic_worker_set[i].workers = NULL;
  1305. #endif
  1306. #ifdef STARPU_USE_MPI_MASTER_SLAVE
  1307. for (i = 0; i < (int) (sizeof(mpi_worker_set)/sizeof(mpi_worker_set[0])); i++)
  1308. mpi_worker_set[i].workers = NULL;
  1309. #endif
  1310. #if defined(STARPU_USE_CUDA) || defined(STARPU_SIMGRID)
  1311. int ncuda = config->conf.ncuda;
  1312. int nworker_per_cuda = starpu_get_env_number_default("STARPU_NWORKER_PER_CUDA", 1);
  1313. STARPU_ASSERT_MSG(nworker_per_cuda > 0, "STARPU_NWORKER_PER_CUDA has to be > 0");
  1314. STARPU_ASSERT_MSG(nworker_per_cuda < STARPU_NMAXWORKERS, "STARPU_NWORKER_PER_CUDA (%d) cannot be higher than STARPU_NMAXWORKERS (%d)\n", nworker_per_cuda, STARPU_NMAXWORKERS);
  1315. #ifndef STARPU_NON_BLOCKING_DRIVERS
  1316. if (nworker_per_cuda > 1)
  1317. {
  1318. _STARPU_DISP("Warning: reducing STARPU_NWORKER_PER_CUDA to 1 because blocking drivers are enabled\n");
  1319. nworker_per_cuda = 1;
  1320. }
  1321. #endif
  1322. if (ncuda != 0)
  1323. {
  1324. /* The user did not disable CUDA. We need to initialize CUDA
  1325. * early to count the number of devices */
  1326. _starpu_init_cuda();
  1327. int nb_devices = _starpu_get_cuda_device_count();
  1328. STARPU_ASSERT_MSG(ncuda >= -1, "ncuda can not be negative and different from -1 (is is %d)", ncuda);
  1329. if (ncuda == -1)
  1330. {
  1331. /* Nothing was specified, so let's choose ! */
  1332. ncuda = nb_devices;
  1333. }
  1334. else
  1335. {
  1336. if (ncuda > nb_devices)
  1337. {
  1338. /* The user requires more CUDA devices than
  1339. * there is available */
  1340. _STARPU_DISP("Warning: %d CUDA devices requested. Only %d available.\n", ncuda, nb_devices);
  1341. ncuda = nb_devices;
  1342. }
  1343. }
  1344. }
  1345. /* Now we know how many CUDA devices will be used */
  1346. topology->ncudagpus = ncuda;
  1347. topology->nworkerpercuda = nworker_per_cuda;
  1348. STARPU_ASSERT(topology->ncudagpus <= STARPU_MAXCUDADEVS);
  1349. _starpu_initialize_workers_cuda_gpuid(config);
  1350. /* allow having one worker per stream */
  1351. topology->cuda_th_per_stream = starpu_get_env_number_default("STARPU_CUDA_THREAD_PER_WORKER", -1);
  1352. topology->cuda_th_per_dev = starpu_get_env_number_default("STARPU_CUDA_THREAD_PER_DEV", -1);
  1353. STARPU_ASSERT_MSG(!(topology->cuda_th_per_stream == 1 && topology->cuda_th_per_dev != -1), "It does not make sense to set both STARPU_CUDA_THREAD_PER_WORKER to 1 and to set STARPU_CUDA_THREAD_PER_DEV, please choose either per worker or per device or none");
  1354. /* per device by default */
  1355. if (topology->cuda_th_per_dev == -1)
  1356. {
  1357. if (topology->cuda_th_per_stream == 1)
  1358. topology->cuda_th_per_dev = 0;
  1359. else
  1360. topology->cuda_th_per_dev = 1;
  1361. }
  1362. /* Not per stream by default */
  1363. if (topology->cuda_th_per_stream == -1)
  1364. {
  1365. topology->cuda_th_per_stream = 0;
  1366. }
  1367. if (!topology->cuda_th_per_dev)
  1368. {
  1369. cuda_worker_set[0].workers = &config->workers[topology->nworkers];
  1370. cuda_worker_set[0].nworkers = topology->ncudagpus * nworker_per_cuda;
  1371. }
  1372. unsigned cudagpu;
  1373. for (cudagpu = 0; cudagpu < topology->ncudagpus; cudagpu++)
  1374. {
  1375. int devid = _starpu_get_next_cuda_gpuid(config);
  1376. int worker_idx0 = topology->nworkers + cudagpu * nworker_per_cuda;
  1377. struct _starpu_worker_set *worker_set;
  1378. if (topology->cuda_th_per_dev)
  1379. {
  1380. worker_set = &cuda_worker_set[devid];
  1381. worker_set->workers = &config->workers[worker_idx0];
  1382. worker_set->nworkers = nworker_per_cuda;
  1383. }
  1384. else
  1385. {
  1386. /* Same worker set for all devices */
  1387. worker_set = &cuda_worker_set[0];
  1388. }
  1389. for (i = 0; i < nworker_per_cuda; i++)
  1390. {
  1391. int worker_idx = worker_idx0 + i;
  1392. if(topology->cuda_th_per_stream)
  1393. {
  1394. /* Just one worker in the set */
  1395. _STARPU_CALLOC(config->workers[worker_idx].set, 1, sizeof(struct _starpu_worker_set));
  1396. config->workers[worker_idx].set->workers = &config->workers[worker_idx];
  1397. config->workers[worker_idx].set->nworkers = 1;
  1398. }
  1399. else
  1400. config->workers[worker_idx].set = worker_set;
  1401. config->workers[worker_idx].arch = STARPU_CUDA_WORKER;
  1402. _STARPU_MALLOC(config->workers[worker_idx].perf_arch.devices, sizeof(struct starpu_perfmodel_device));
  1403. config->workers[worker_idx].perf_arch.ndevices = 1;
  1404. config->workers[worker_idx].perf_arch.devices[0].type = STARPU_CUDA_WORKER;
  1405. config->workers[worker_idx].perf_arch.devices[0].devid = devid;
  1406. // TODO: fix perfmodels etc.
  1407. //config->workers[worker_idx].perf_arch.ncore = nworker_per_cuda - 1;
  1408. config->workers[worker_idx].perf_arch.devices[0].ncores = 1;
  1409. config->workers[worker_idx].devid = devid;
  1410. config->workers[worker_idx].subworkerid = i;
  1411. config->workers[worker_idx].worker_mask = STARPU_CUDA;
  1412. config->worker_mask |= STARPU_CUDA;
  1413. struct handle_entry *entry;
  1414. HASH_FIND_INT(devices_using_cuda, &devid, entry);
  1415. if (!entry)
  1416. {
  1417. _STARPU_MALLOC(entry, sizeof(*entry));
  1418. entry->gpuid = devid;
  1419. HASH_ADD_INT(devices_using_cuda, gpuid, entry);
  1420. }
  1421. }
  1422. #ifndef STARPU_SIMGRID
  1423. #if defined(HAVE_DECL_HWLOC_CUDA_GET_DEVICE_OSDEV_BY_INDEX) && HAVE_DECL_HWLOC_CUDA_GET_DEVICE_OSDEV_BY_INDEX
  1424. {
  1425. hwloc_obj_t obj = hwloc_cuda_get_device_osdev_by_index(topology->hwtopology, devid);
  1426. if (obj)
  1427. {
  1428. struct _starpu_hwloc_userdata *data = obj->userdata;
  1429. data->ngpus++;
  1430. }
  1431. else
  1432. {
  1433. _STARPU_DEBUG("Warning: could not find location of CUDA%u, do you have the hwloc CUDA plugin installed?\n", devid);
  1434. }
  1435. }
  1436. #endif
  1437. #endif
  1438. }
  1439. topology->nworkers += topology->ncudagpus * nworker_per_cuda;
  1440. #endif
  1441. #if defined(STARPU_USE_OPENCL) || defined(STARPU_SIMGRID)
  1442. int nopencl = config->conf.nopencl;
  1443. if (nopencl != 0)
  1444. {
  1445. /* The user did not disable OPENCL. We need to initialize
  1446. * OpenCL early to count the number of devices */
  1447. _starpu_opencl_init();
  1448. int nb_devices;
  1449. nb_devices = _starpu_opencl_get_device_count();
  1450. STARPU_ASSERT_MSG(nopencl >= -1, "nopencl can not be negative and different from -1 (is is %d)", nopencl);
  1451. if (nopencl == -1)
  1452. {
  1453. /* Nothing was specified, so let's choose ! */
  1454. nopencl = nb_devices;
  1455. if (nopencl > STARPU_MAXOPENCLDEVS)
  1456. {
  1457. _STARPU_DISP("Warning: %d OpenCL devices available. Only %d enabled. Use configure option --enable-maxopencldadev=xxx to update the maximum value of supported OpenCL devices.\n", nb_devices, STARPU_MAXOPENCLDEVS);
  1458. nopencl = STARPU_MAXOPENCLDEVS;
  1459. }
  1460. }
  1461. else
  1462. {
  1463. /* Let's make sure this value is OK. */
  1464. if (nopencl > nb_devices)
  1465. {
  1466. /* The user requires more OpenCL devices than
  1467. * there is available */
  1468. _STARPU_DISP("Warning: %d OpenCL devices requested. Only %d available.\n", nopencl, nb_devices);
  1469. nopencl = nb_devices;
  1470. }
  1471. /* Let's make sure this value is OK. */
  1472. if (nopencl > STARPU_MAXOPENCLDEVS)
  1473. {
  1474. _STARPU_DISP("Warning: %d OpenCL devices requested. Only %d enabled. Use configure option --enable-maxopencldev=xxx to update the maximum value of supported OpenCL devices.\n", nopencl, STARPU_MAXOPENCLDEVS);
  1475. nopencl = STARPU_MAXOPENCLDEVS;
  1476. }
  1477. }
  1478. }
  1479. topology->nopenclgpus = nopencl;
  1480. STARPU_ASSERT(topology->nopenclgpus + topology->nworkers <= STARPU_NMAXWORKERS);
  1481. _starpu_initialize_workers_opencl_gpuid(config);
  1482. unsigned openclgpu;
  1483. for (openclgpu = 0; openclgpu < topology->nopenclgpus; openclgpu++)
  1484. {
  1485. int worker_idx = topology->nworkers + openclgpu;
  1486. int devid = _starpu_get_next_opencl_gpuid(config);
  1487. if (devid == -1)
  1488. {
  1489. // There is no more devices left
  1490. topology->nopenclgpus = openclgpu;
  1491. break;
  1492. }
  1493. config->workers[worker_idx].arch = STARPU_OPENCL_WORKER;
  1494. _STARPU_MALLOC(config->workers[worker_idx].perf_arch.devices, sizeof(struct starpu_perfmodel_device));
  1495. config->workers[worker_idx].perf_arch.ndevices = 1;
  1496. config->workers[worker_idx].perf_arch.devices[0].type = STARPU_OPENCL_WORKER;
  1497. config->workers[worker_idx].perf_arch.devices[0].devid = devid;
  1498. config->workers[worker_idx].perf_arch.devices[0].ncores = 1;
  1499. config->workers[worker_idx].subworkerid = 0;
  1500. config->workers[worker_idx].devid = devid;
  1501. config->workers[worker_idx].worker_mask = STARPU_OPENCL;
  1502. config->worker_mask |= STARPU_OPENCL;
  1503. }
  1504. topology->nworkers += topology->nopenclgpus;
  1505. #endif
  1506. #if defined(STARPU_USE_MIC) || defined(STARPU_USE_MPI_MASTER_SLAVE)
  1507. _starpu_init_mp_config(config, &config->conf, no_mp_config);
  1508. #endif
  1509. /* we put the CPU section after the accelerator : in case there was an
  1510. * accelerator found, we devote one cpu */
  1511. #if defined(STARPU_USE_CPU) || defined(STARPU_SIMGRID)
  1512. int ncpu = config->conf.ncpus;
  1513. if (ncpu != 0)
  1514. {
  1515. STARPU_ASSERT_MSG(ncpu >= -1, "ncpus can not be negative and different from -1 (is is %d)", ncpu);
  1516. if (ncpu == -1)
  1517. {
  1518. unsigned mic_busy_cpus = 0;
  1519. int j = 0;
  1520. for (j = 0; j < STARPU_MAXMICDEVS; j++)
  1521. mic_busy_cpus += (topology->nmiccores[j] ? 1 : 0);
  1522. unsigned mpi_ms_busy_cpus = 0;
  1523. #ifdef STARPU_USE_MPI_MASTER_SLAVE
  1524. #ifdef STARPU_MPI_MASTER_SLAVE_MULTIPLE_THREAD
  1525. for (j = 0; j < STARPU_MAXMPIDEVS; j++)
  1526. mpi_ms_busy_cpus += (topology->nmpicores[j] ? 1 : 0);
  1527. #else
  1528. mpi_ms_busy_cpus = 1; /* we launch one thread to control all slaves */
  1529. #endif
  1530. #endif /* STARPU_USE_MPI_MASTER_SLAVE */
  1531. unsigned cuda_busy_cpus = 0;
  1532. #if defined(STARPU_USE_CUDA) || defined(STARPU_SIMGRID)
  1533. cuda_busy_cpus =
  1534. topology->cuda_th_per_dev == 0 && topology->cuda_th_per_stream == 0 ? (topology->ncudagpus ? 1 : 0) :
  1535. topology->cuda_th_per_stream ? (nworker_per_cuda * topology->ncudagpus) : topology->ncudagpus;
  1536. #endif
  1537. unsigned already_busy_cpus = mpi_ms_busy_cpus + mic_busy_cpus
  1538. + cuda_busy_cpus
  1539. + topology->nopenclgpus;
  1540. long avail_cpus = (long) topology->nhwcpus - (long) already_busy_cpus;
  1541. if (avail_cpus < 0)
  1542. avail_cpus = 0;
  1543. int nth_per_core = starpu_get_env_number_default("STARPU_NTHREADS_PER_CORE", 1);
  1544. avail_cpus *= nth_per_core;
  1545. ncpu = avail_cpus;
  1546. }
  1547. if (ncpu > STARPU_MAXCPUS)
  1548. {
  1549. _STARPU_DISP("Warning: %d CPU cores requested. Only %d enabled. Use configure option --enable-maxcpus=xxx to update the maximum value of supported CPU devices.\n", ncpu, STARPU_MAXCPUS);
  1550. ncpu = STARPU_MAXCPUS;
  1551. }
  1552. if (config->conf.reserve_ncpus > 0)
  1553. {
  1554. if (ncpu < config->conf.reserve_ncpus)
  1555. {
  1556. _STARPU_DISP("Warning: %d CPU cores were requested to be reserved, but only %d were available,\n", config->conf.reserve_ncpus, ncpu);
  1557. ncpu = 0;
  1558. }
  1559. else
  1560. {
  1561. ncpu -= config->conf.reserve_ncpus;
  1562. }
  1563. }
  1564. }
  1565. topology->ncpus = ncpu;
  1566. STARPU_ASSERT(topology->ncpus + topology->nworkers <= STARPU_NMAXWORKERS);
  1567. unsigned cpu;
  1568. unsigned homogeneous = starpu_get_env_number_default("STARPU_PERF_MODEL_HOMOGENEOUS_CPU", 1);
  1569. for (cpu = 0; cpu < topology->ncpus; cpu++)
  1570. {
  1571. int worker_idx = topology->nworkers + cpu;
  1572. config->workers[worker_idx].arch = STARPU_CPU_WORKER;
  1573. _STARPU_MALLOC(config->workers[worker_idx].perf_arch.devices, sizeof(struct starpu_perfmodel_device));
  1574. config->workers[worker_idx].perf_arch.ndevices = 1;
  1575. config->workers[worker_idx].perf_arch.devices[0].type = STARPU_CPU_WORKER;
  1576. config->workers[worker_idx].perf_arch.devices[0].devid = homogeneous ? 0 : cpu;
  1577. config->workers[worker_idx].perf_arch.devices[0].ncores = 1;
  1578. config->workers[worker_idx].subworkerid = 0;
  1579. config->workers[worker_idx].devid = cpu;
  1580. config->workers[worker_idx].worker_mask = STARPU_CPU;
  1581. config->worker_mask |= STARPU_CPU;
  1582. }
  1583. topology->nworkers += topology->ncpus;
  1584. #endif
  1585. if (topology->nworkers == 0)
  1586. {
  1587. _STARPU_DEBUG("No worker found, aborting ...\n");
  1588. return -ENODEV;
  1589. }
  1590. return 0;
  1591. }
  1592. void _starpu_destroy_machine_config(struct _starpu_machine_config *config)
  1593. {
  1594. _starpu_close_debug_logfile();
  1595. unsigned worker;
  1596. for (worker = 0; worker < config->topology.nworkers; worker++)
  1597. {
  1598. struct _starpu_worker *workerarg = &config->workers[worker];
  1599. int bindid = workerarg->bindid;
  1600. free(workerarg->perf_arch.devices);
  1601. #ifdef STARPU_HAVE_HWLOC
  1602. hwloc_bitmap_free(workerarg->hwloc_cpu_set);
  1603. if (bindid != -1)
  1604. {
  1605. hwloc_obj_t worker_obj = hwloc_get_obj_by_depth(config->topology.hwtopology,
  1606. config->pu_depth,
  1607. bindid);
  1608. struct _starpu_hwloc_userdata *data = worker_obj->userdata;
  1609. if (data->worker_list)
  1610. {
  1611. _starpu_worker_list_delete(data->worker_list);
  1612. data->worker_list = NULL;
  1613. }
  1614. }
  1615. #endif
  1616. if (bindid != -1)
  1617. {
  1618. free(config->bindid_workers[bindid].workerids);
  1619. config->bindid_workers[bindid].workerids = NULL;
  1620. }
  1621. }
  1622. free(config->bindid_workers);
  1623. config->bindid_workers = NULL;
  1624. config->nbindid = 0;
  1625. unsigned combined_worker_id;
  1626. for(combined_worker_id=0 ; combined_worker_id < config->topology.ncombinedworkers ; combined_worker_id++)
  1627. {
  1628. struct _starpu_combined_worker *combined_worker = &config->combined_workers[combined_worker_id];
  1629. #ifdef STARPU_HAVE_HWLOC
  1630. hwloc_bitmap_free(combined_worker->hwloc_cpu_set);
  1631. #endif
  1632. free(combined_worker->perf_arch.devices);
  1633. }
  1634. #ifdef STARPU_HAVE_HWLOC
  1635. _starpu_deallocate_topology_userdata(hwloc_get_root_obj(config->topology.hwtopology));
  1636. hwloc_topology_destroy(config->topology.hwtopology);
  1637. #endif
  1638. topology_is_initialized = 0;
  1639. #ifdef STARPU_USE_CUDA
  1640. struct handle_entry *entry=NULL, *tmp=NULL;
  1641. HASH_ITER(hh, devices_using_cuda, entry, tmp)
  1642. {
  1643. HASH_DEL(devices_using_cuda, entry);
  1644. free(entry);
  1645. }
  1646. devices_using_cuda = NULL;
  1647. #endif
  1648. #if defined(STARPU_USE_CUDA) || defined(STARPU_USE_OPENCL)
  1649. int i;
  1650. for (i=0; i<STARPU_NARCH; i++)
  1651. may_bind_automatically[i] = 0;
  1652. #endif
  1653. }
  1654. int _starpu_bind_thread_on_cpu(int cpuid STARPU_ATTRIBUTE_UNUSED, int workerid STARPU_ATTRIBUTE_UNUSED, const char *name STARPU_ATTRIBUTE_UNUSED)
  1655. {
  1656. int ret = 0;
  1657. #ifdef STARPU_SIMGRID
  1658. return ret;
  1659. #else
  1660. if (nobind > 0)
  1661. return ret;
  1662. if (cpuid < 0)
  1663. return ret;
  1664. #ifdef STARPU_HAVE_HWLOC
  1665. const struct hwloc_topology_support *support;
  1666. struct _starpu_machine_config *config = _starpu_get_machine_config();
  1667. #ifdef STARPU_USE_OPENCL
  1668. if (config->conf.nopencl != 0)
  1669. _starpu_opencl_init();
  1670. #endif
  1671. #ifdef STARPU_USE_CUDA
  1672. if (config->conf.ncuda != 0)
  1673. _starpu_init_cuda();
  1674. #endif
  1675. _starpu_init_topology(config);
  1676. if (workerid != STARPU_NOWORKERID && cpuid < STARPU_MAXCPUS)
  1677. {
  1678. /* TODO: mutex... */
  1679. int previous = cpu_worker[cpuid];
  1680. /* We would like the PU to be available, or we are perhaps fine to share it */
  1681. if ( !( previous == STARPU_NOWORKERID ||
  1682. (previous == STARPU_NONACTIVETHREAD && workerid == STARPU_NONACTIVETHREAD) ||
  1683. (previous >= 0 && previous == workerid) ||
  1684. (name && cpu_name[cpuid] && !strcmp(name, cpu_name[cpuid])) ) )
  1685. {
  1686. if (previous == STARPU_ACTIVETHREAD)
  1687. _STARPU_DISP("Warning: active thread %s was already bound to PU %d\n", cpu_name[cpuid], cpuid);
  1688. else if (previous == STARPU_NONACTIVETHREAD)
  1689. _STARPU_DISP("Warning: non-active thread %s was already bound to PU %d\n", cpu_name[cpuid], cpuid);
  1690. else
  1691. _STARPU_DISP("Warning: worker %d was already bound to PU %d\n", previous, cpuid);
  1692. if (workerid == STARPU_ACTIVETHREAD)
  1693. _STARPU_DISP("and we were told to also bind active thread %s to it.\n", name);
  1694. else if (previous == STARPU_NONACTIVETHREAD)
  1695. _STARPU_DISP("and we were told to also bind non-active thread %s to it.\n", name);
  1696. else
  1697. _STARPU_DISP("and we were told to also bind worker %d to it.\n", workerid);
  1698. _STARPU_DISP("This will strongly degrade performance.\n");
  1699. if (workerid >= 0)
  1700. /* This shouldn't happen for workers */
  1701. _STARPU_DISP("Maybe check starpu_machine_display's output to determine what wrong binding happened. Hwloc reported %d cores and %d threads, perhaps there is misdetection between hwloc, the kernel and the BIOS, or an administrative allocation issue from e.g. the job scheduler?\n", config->topology.nhwcpus, config->topology.nhwpus);
  1702. ret = -1;
  1703. }
  1704. else
  1705. {
  1706. cpu_worker[cpuid] = workerid;
  1707. if (name)
  1708. cpu_name[cpuid] = strdup(name);
  1709. }
  1710. }
  1711. support = hwloc_topology_get_support(config->topology.hwtopology);
  1712. if (support->cpubind->set_thisthread_cpubind)
  1713. {
  1714. hwloc_obj_t obj = hwloc_get_obj_by_depth(config->topology.hwtopology, config->pu_depth, cpuid);
  1715. hwloc_bitmap_t set = obj->cpuset;
  1716. int res;
  1717. hwloc_bitmap_singlify(set);
  1718. res = hwloc_set_cpubind(config->topology.hwtopology, set, HWLOC_CPUBIND_THREAD);
  1719. if (res)
  1720. {
  1721. perror("hwloc_set_cpubind");
  1722. STARPU_ABORT();
  1723. }
  1724. }
  1725. #elif defined(HAVE_PTHREAD_SETAFFINITY_NP) && defined(__linux__)
  1726. int res;
  1727. /* fix the thread on the correct cpu */
  1728. cpu_set_t aff_mask;
  1729. CPU_ZERO(&aff_mask);
  1730. CPU_SET(cpuid, &aff_mask);
  1731. starpu_pthread_t self = starpu_pthread_self();
  1732. res = pthread_setaffinity_np(self, sizeof(aff_mask), &aff_mask);
  1733. if (res)
  1734. {
  1735. const char *msg = strerror(res);
  1736. _STARPU_MSG("pthread_setaffinity_np: %s\n", msg);
  1737. STARPU_ABORT();
  1738. }
  1739. #elif defined(_WIN32)
  1740. DWORD mask = 1 << cpuid;
  1741. if (!SetThreadAffinityMask(GetCurrentThread(), mask))
  1742. {
  1743. _STARPU_ERROR("SetThreadMaskAffinity(%lx) failed\n", mask);
  1744. }
  1745. #else
  1746. #warning no CPU binding support
  1747. #endif
  1748. #endif
  1749. return ret;
  1750. }
  1751. int
  1752. starpu_bind_thread_on(int cpuid, unsigned flags, const char *name)
  1753. {
  1754. int workerid;
  1755. STARPU_ASSERT_MSG(name, "starpu_bind_thread_on must be provided with a name");
  1756. starpu_pthread_setname(name);
  1757. if (flags & STARPU_THREAD_ACTIVE)
  1758. workerid = STARPU_ACTIVETHREAD;
  1759. else
  1760. workerid = STARPU_NONACTIVETHREAD;
  1761. return _starpu_bind_thread_on_cpu(cpuid, workerid, name);
  1762. }
  1763. void _starpu_bind_thread_on_cpus(struct _starpu_combined_worker *combined_worker STARPU_ATTRIBUTE_UNUSED)
  1764. {
  1765. #ifdef STARPU_SIMGRID
  1766. return;
  1767. #endif
  1768. #ifdef STARPU_HAVE_HWLOC
  1769. const struct hwloc_topology_support *support;
  1770. struct _starpu_machine_config *config = _starpu_get_machine_config();
  1771. #ifdef STARPU_USE_OPENC
  1772. if (config->conf.nopencl != 0)
  1773. _starpu_opencl_init();
  1774. #endif
  1775. #ifdef STARPU_USE_CUDA
  1776. if (config->conf.ncuda != 0)
  1777. _starpu_init_cuda();
  1778. #endif
  1779. _starpu_init_topology(config);
  1780. support = hwloc_topology_get_support(config->topology.hwtopology);
  1781. if (support->cpubind->set_thisthread_cpubind)
  1782. {
  1783. hwloc_bitmap_t set = combined_worker->hwloc_cpu_set;
  1784. int ret;
  1785. ret = hwloc_set_cpubind(config->topology.hwtopology, set, HWLOC_CPUBIND_THREAD);
  1786. if (ret)
  1787. {
  1788. perror("binding thread");
  1789. STARPU_ABORT();
  1790. }
  1791. }
  1792. #else
  1793. #ifdef __GLIBC__
  1794. sched_setaffinity(0,sizeof(combined_worker->cpu_set),&combined_worker->cpu_set);
  1795. #else
  1796. # warning no parallel worker CPU binding support
  1797. #endif
  1798. #endif
  1799. }
  1800. static void _starpu_init_binding_cpu(struct _starpu_machine_config *config)
  1801. {
  1802. unsigned worker;
  1803. for (worker = 0; worker < config->topology.nworkers; worker++)
  1804. {
  1805. struct _starpu_worker *workerarg = &config->workers[worker];
  1806. switch (workerarg->arch)
  1807. {
  1808. case STARPU_CPU_WORKER:
  1809. {
  1810. /* Dedicate a cpu core to that worker */
  1811. workerarg->bindid = _starpu_get_next_bindid(config, STARPU_THREAD_ACTIVE, NULL, 0);
  1812. break;
  1813. }
  1814. default:
  1815. /* Do nothing */
  1816. break;
  1817. }
  1818. }
  1819. }
  1820. static size_t _starpu_cpu_get_global_mem_size(int nodeid STARPU_ATTRIBUTE_UNUSED, struct _starpu_machine_config *config STARPU_ATTRIBUTE_UNUSED)
  1821. {
  1822. size_t global_mem;
  1823. starpu_ssize_t limit = -1;
  1824. #if defined(STARPU_HAVE_HWLOC)
  1825. struct _starpu_machine_topology *topology = &config->topology;
  1826. STARPU_ASSERT(numa_enabled != -1);
  1827. if (numa_enabled)
  1828. {
  1829. int depth_node = hwloc_get_type_depth(topology->hwtopology, HWLOC_OBJ_NUMANODE);
  1830. if (depth_node == HWLOC_TYPE_DEPTH_UNKNOWN)
  1831. {
  1832. #if HWLOC_API_VERSION >= 0x00020000
  1833. global_mem = hwloc_get_root_obj(topology->hwtopology)->total_memory;
  1834. #else
  1835. global_mem = hwloc_get_root_obj(topology->hwtopology)->memory.total_memory;
  1836. #endif
  1837. }
  1838. else
  1839. {
  1840. char name[32];
  1841. hwloc_obj_t obj = hwloc_get_obj_by_depth(topology->hwtopology, depth_node, nodeid);
  1842. #if HWLOC_API_VERSION >= 0x00020000
  1843. global_mem = obj->attr->numanode.local_memory;
  1844. #else
  1845. global_mem = obj->memory.local_memory;
  1846. #endif
  1847. snprintf(name, sizeof(name), "STARPU_LIMIT_CPU_NUMA_%d_MEM", obj->os_index);
  1848. limit = starpu_get_env_number(name);
  1849. }
  1850. }
  1851. else
  1852. {
  1853. /* Do not limit ourself to a single NUMA node */
  1854. #if HWLOC_API_VERSION >= 0x00020000
  1855. global_mem = hwloc_get_root_obj(topology->hwtopology)->total_memory;
  1856. #else
  1857. global_mem = hwloc_get_root_obj(topology->hwtopology)->memory.total_memory;
  1858. #endif
  1859. }
  1860. #else /* STARPU_HAVE_HWLOC */
  1861. #ifdef STARPU_DEVEL
  1862. # warning TODO: use sysinfo when available to get global size
  1863. #endif
  1864. global_mem = 0;
  1865. #endif
  1866. if (limit == -1)
  1867. limit = starpu_get_env_number("STARPU_LIMIT_CPU_MEM");
  1868. if (limit < 0)
  1869. // No limit is defined, we return the global memory size
  1870. return global_mem;
  1871. else if (global_mem && (size_t)limit * 1024*1024 > global_mem)
  1872. // The requested limit is higher than what is available, we return the global memory size
  1873. return global_mem;
  1874. else
  1875. // We limit the memory
  1876. return limit*1024*1024;
  1877. }
  1878. //TODO : Check SIMGRID
  1879. static void _starpu_init_numa_node(struct _starpu_machine_config *config)
  1880. {
  1881. nb_numa_nodes = 0;
  1882. unsigned i;
  1883. for (i = 0; i < STARPU_MAXNUMANODES; i++)
  1884. {
  1885. numa_memory_nodes_to_hwloclogid[i] = STARPU_NUMA_UNINITIALIZED;
  1886. numa_memory_nodes_to_physicalid[i] = STARPU_NUMA_UNINITIALIZED;
  1887. }
  1888. #ifdef STARPU_SIMGRID
  1889. char name[16];
  1890. starpu_sg_host_t host;
  1891. #endif
  1892. numa_enabled = starpu_get_env_number_default("STARPU_USE_NUMA", 0);
  1893. /* NUMA mode activated */
  1894. if (numa_enabled)
  1895. {
  1896. /* Take all NUMA nodes used by CPU workers */
  1897. unsigned worker;
  1898. for (worker = 0; worker < config->topology.nworkers; worker++)
  1899. {
  1900. struct _starpu_worker *workerarg = &config->workers[worker];
  1901. if (workerarg->arch == STARPU_CPU_WORKER)
  1902. {
  1903. int numa_logical_id = _starpu_get_logical_numa_node_worker(worker);
  1904. /* Convert logical id to StarPU id to check if this NUMA node is already saved or not */
  1905. int numa_starpu_id = starpu_memory_nodes_numa_hwloclogid_to_id(numa_logical_id);
  1906. /* This shouldn't happen */
  1907. if (numa_starpu_id == -1 && nb_numa_nodes == STARPU_MAXNUMANODES)
  1908. {
  1909. _STARPU_MSG("Warning: %u NUMA nodes available. Only %u enabled. Use configure option --enable-maxnumanodes=xxx to update the maximum value of supported NUMA nodes.\n", _starpu_topology_get_nnumanodes(config), STARPU_MAXNUMANODES);
  1910. STARPU_ABORT();
  1911. }
  1912. if (numa_starpu_id == -1)
  1913. {
  1914. int devid = numa_logical_id == STARPU_NUMA_MAIN_RAM ? 0 : numa_logical_id;
  1915. int memnode = _starpu_memory_node_register(STARPU_CPU_RAM, devid, &_starpu_driver_cpu_node_ops);
  1916. _starpu_memory_manager_set_global_memory_size(memnode, _starpu_cpu_get_global_mem_size(devid, config));
  1917. STARPU_ASSERT_MSG(memnode < STARPU_MAXNUMANODES, "Wrong Memory Node : %d (only %d available)", memnode, STARPU_MAXNUMANODES);
  1918. numa_memory_nodes_to_hwloclogid[memnode] = numa_logical_id;
  1919. int numa_physical_id = _starpu_get_physical_numa_node_worker(worker);
  1920. numa_memory_nodes_to_physicalid[memnode] = numa_physical_id;
  1921. nb_numa_nodes++;
  1922. #ifdef STARPU_SIMGRID
  1923. snprintf(name, sizeof(name), "RAM%d", memnode);
  1924. host = _starpu_simgrid_get_host_by_name(name);
  1925. STARPU_ASSERT(host);
  1926. _starpu_simgrid_memory_node_set_host(memnode, host);
  1927. #endif
  1928. }
  1929. }
  1930. }
  1931. /* If we found NUMA nodes from CPU workers, it's good */
  1932. if (nb_numa_nodes != 0)
  1933. return;
  1934. _STARPU_DISP("No NUMA nodes found when checking CPU workers...\n");
  1935. #if (defined(STARPU_USE_CUDA) || defined(STARPU_USE_OPENCL)) && defined(STARPU_HAVE_HWLOC)
  1936. _STARPU_DISP("Take NUMA nodes attached to CUDA and OpenCL devices...\n");
  1937. #endif
  1938. #if defined(STARPU_USE_CUDA) && defined(STARPU_HAVE_HWLOC)
  1939. for (i = 0; i < config->topology.ncudagpus; i++)
  1940. {
  1941. hwloc_obj_t obj = hwloc_cuda_get_device_osdev_by_index(config->topology.hwtopology, i);
  1942. if (obj)
  1943. obj = numa_get_obj(obj);
  1944. /* Hwloc cannot recognize some devices */
  1945. if (!obj)
  1946. continue;
  1947. int numa_starpu_id = starpu_memory_nodes_numa_hwloclogid_to_id(obj->logical_index);
  1948. /* This shouldn't happen */
  1949. if (numa_starpu_id == -1 && nb_numa_nodes == STARPU_MAXNUMANODES)
  1950. {
  1951. _STARPU_MSG("Warning: %u NUMA nodes available. Only %u enabled. Use configure option --enable-maxnumanodes=xxx to update the maximum value of supported NUMA nodes.\n", _starpu_topology_get_nnumanodes(config), STARPU_MAXNUMANODES);
  1952. STARPU_ABORT();
  1953. }
  1954. if (numa_starpu_id == -1)
  1955. {
  1956. int memnode = _starpu_memory_node_register(STARPU_CPU_RAM, obj->logical_index, &_starpu_driver_cpu_node_ops);
  1957. _starpu_memory_manager_set_global_memory_size(memnode, _starpu_cpu_get_global_mem_size(obj->logical_index, config));
  1958. STARPU_ASSERT_MSG(memnode < STARPU_MAXNUMANODES, "Wrong Memory Node : %d (only %d available)", memnode, STARPU_MAXNUMANODES);
  1959. numa_memory_nodes_to_hwloclogid[memnode] = obj->logical_index;
  1960. numa_memory_nodes_to_physicalid[memnode] = obj->os_index;
  1961. nb_numa_nodes++;
  1962. #ifdef STARPU_SIMGRID
  1963. snprintf(name, sizeof(name), "RAM%d", memnode);
  1964. host = _starpu_simgrid_get_host_by_name(name);
  1965. STARPU_ASSERT(host);
  1966. _starpu_simgrid_memory_node_set_host(memnode, host);
  1967. #endif
  1968. }
  1969. }
  1970. #endif
  1971. #if defined(STARPU_USE_OPENCL) && defined(STARPU_HAVE_HWLOC)
  1972. if (config->topology.nopenclgpus > 0)
  1973. {
  1974. cl_int err;
  1975. cl_platform_id platform_id[_STARPU_OPENCL_PLATFORM_MAX];
  1976. cl_uint nb_platforms;
  1977. unsigned platform;
  1978. unsigned nb_opencl_devices = 0, num = 0;
  1979. err = clGetPlatformIDs(_STARPU_OPENCL_PLATFORM_MAX, platform_id, &nb_platforms);
  1980. if (STARPU_UNLIKELY(err != CL_SUCCESS))
  1981. nb_platforms=0;
  1982. cl_device_type device_type = CL_DEVICE_TYPE_GPU|CL_DEVICE_TYPE_ACCELERATOR;
  1983. if (starpu_get_env_number("STARPU_OPENCL_ON_CPUS") > 0)
  1984. device_type |= CL_DEVICE_TYPE_CPU;
  1985. if (starpu_get_env_number("STARPU_OPENCL_ONLY_ON_CPUS") > 0)
  1986. device_type = CL_DEVICE_TYPE_CPU;
  1987. for (platform = 0; platform < nb_platforms ; platform++)
  1988. {
  1989. err = clGetDeviceIDs(platform_id[platform], device_type, 0, NULL, &num);
  1990. if (err != CL_SUCCESS)
  1991. num = 0;
  1992. nb_opencl_devices += num;
  1993. for (i = 0; i < num; i++)
  1994. {
  1995. hwloc_obj_t obj = hwloc_opencl_get_device_osdev_by_index(config->topology.hwtopology, platform, i);
  1996. if (obj)
  1997. obj = numa_get_obj(obj);
  1998. /* Hwloc cannot recognize some devices */
  1999. if (!obj)
  2000. continue;
  2001. int numa_starpu_id = starpu_memory_nodes_numa_hwloclogid_to_id(obj->logical_index);
  2002. /* This shouldn't happen */
  2003. if (numa_starpu_id == -1 && nb_numa_nodes == STARPU_MAXNUMANODES)
  2004. {
  2005. _STARPU_MSG("Warning: %u NUMA nodes available. Only %u enabled. Use configure option --enable-maxnumanodes=xxx to update the maximum value of supported NUMA nodes.\n", _starpu_topology_get_nnumanodes(config), STARPU_MAXNUMANODES);
  2006. STARPU_ABORT();
  2007. }
  2008. if (numa_starpu_id == -1)
  2009. {
  2010. int memnode = _starpu_memory_node_register(STARPU_CPU_RAM, obj->logical_index, &_starpu_driver_cpu_node_ops);
  2011. _starpu_memory_manager_set_global_memory_size(memnode, _starpu_cpu_get_global_mem_size(obj->logical_index, config));
  2012. STARPU_ASSERT_MSG(memnode < STARPU_MAXNUMANODES, "Wrong Memory Node : %d (only %d available)", memnode, STARPU_MAXNUMANODES);
  2013. numa_memory_nodes_to_hwloclogid[memnode] = obj->logical_index;
  2014. numa_memory_nodes_to_physicalid[memnode] = obj->os_index;
  2015. nb_numa_nodes++;
  2016. #ifdef STARPU_SIMGRID
  2017. snprintf(name, sizeof(name), "RAM%d", memnode);
  2018. host = _starpu_simgrid_get_host_by_name(name);
  2019. STARPU_ASSERT(host);
  2020. _starpu_simgrid_memory_node_set_host(memnode, host);
  2021. #endif
  2022. }
  2023. }
  2024. }
  2025. }
  2026. #endif
  2027. }
  2028. #if (defined(STARPU_USE_CUDA) || defined(STARPU_USE_OPENCL)) && defined(STARPU_HAVE_HWLOC)
  2029. //Found NUMA nodes from CUDA nodes
  2030. if (nb_numa_nodes != 0)
  2031. return;
  2032. /* In case, we do not find any NUMA nodes when checking NUMA nodes attached to GPUs, we take all of them */
  2033. if (numa_enabled)
  2034. _STARPU_DISP("No NUMA nodes found when checking GPUs devices...\n");
  2035. #endif
  2036. if (numa_enabled)
  2037. _STARPU_DISP("Finally, take all NUMA nodes available... \n");
  2038. unsigned nnuma = _starpu_topology_get_nnumanodes(config);
  2039. if (nnuma > STARPU_MAXNUMANODES)
  2040. {
  2041. _STARPU_MSG("Warning: %u NUMA nodes available. Only %u enabled. Use configure option --enable-maxnumanodes=xxx to update the maximum value of supported NUMA nodes.\n", _starpu_topology_get_nnumanodes(config), STARPU_MAXNUMANODES);
  2042. nnuma = STARPU_MAXNUMANODES;
  2043. }
  2044. unsigned numa;
  2045. for (numa = 0; numa < nnuma; numa++)
  2046. {
  2047. unsigned numa_logical_id;
  2048. unsigned numa_physical_id;
  2049. #if defined(STARPU_HAVE_HWLOC)
  2050. hwloc_obj_t obj = hwloc_get_obj_by_type(config->topology.hwtopology, HWLOC_OBJ_NUMANODE, numa);
  2051. if (obj)
  2052. {
  2053. numa_logical_id = obj->logical_index;
  2054. numa_physical_id = obj->os_index;
  2055. }
  2056. else
  2057. #endif
  2058. {
  2059. numa_logical_id = 0;
  2060. numa_physical_id = 0;
  2061. }
  2062. int memnode = _starpu_memory_node_register(STARPU_CPU_RAM, numa_logical_id, &_starpu_driver_cpu_node_ops);
  2063. _starpu_memory_manager_set_global_memory_size(memnode, _starpu_cpu_get_global_mem_size(numa_logical_id, config));
  2064. numa_memory_nodes_to_hwloclogid[memnode] = numa_logical_id;
  2065. numa_memory_nodes_to_physicalid[memnode] = numa_physical_id;
  2066. nb_numa_nodes++;
  2067. if (numa == 0)
  2068. STARPU_ASSERT_MSG(memnode == STARPU_MAIN_RAM, "Wrong Memory Node : %d (expected %d) \n", memnode, STARPU_MAIN_RAM);
  2069. STARPU_ASSERT_MSG(memnode < STARPU_MAXNUMANODES, "Wrong Memory Node : %d (only %d available) \n", memnode, STARPU_MAXNUMANODES);
  2070. #ifdef STARPU_SIMGRID
  2071. if (nnuma > 1)
  2072. {
  2073. snprintf(name, sizeof(name), "RAM%d", memnode);
  2074. host = _starpu_simgrid_get_host_by_name(name);
  2075. }
  2076. else
  2077. {
  2078. /* In this case, nnuma has only one node */
  2079. host = _starpu_simgrid_get_host_by_name("RAM");
  2080. }
  2081. STARPU_ASSERT(host);
  2082. _starpu_simgrid_memory_node_set_host(memnode, host);
  2083. #endif
  2084. }
  2085. STARPU_ASSERT_MSG(nb_numa_nodes > 0, "No NUMA node found... We need at least one memory node !\n");
  2086. }
  2087. static void _starpu_init_numa_bus()
  2088. {
  2089. unsigned i, j;
  2090. for (i = 0; i < nb_numa_nodes; i++)
  2091. for (j = 0; j < nb_numa_nodes; j++)
  2092. if (i != j)
  2093. numa_bus_id[i*nb_numa_nodes+j] = _starpu_register_bus(i, j);
  2094. }
  2095. #if defined(STARPU_HAVE_HWLOC) && !defined(STARPU_SIMGRID)
  2096. static int _starpu_find_pu_driving_numa_from(hwloc_obj_t root, unsigned node)
  2097. {
  2098. unsigned i;
  2099. int found = 0;
  2100. if (!root->arity)
  2101. {
  2102. if (root->type == HWLOC_OBJ_PU)
  2103. {
  2104. struct _starpu_hwloc_userdata *userdata = root->userdata;
  2105. if (userdata->pu_worker)
  2106. {
  2107. /* Cool, found a worker! */
  2108. _STARPU_DEBUG("found PU %d to drive memory node %d\n", userdata->pu_worker->bindid, node);
  2109. _starpu_worker_drives_memory_node(userdata->pu_worker, node);
  2110. found = 1;
  2111. }
  2112. }
  2113. }
  2114. for (i = 0; i < root->arity; i++)
  2115. {
  2116. if (_starpu_find_pu_driving_numa_from(root->children[i], node))
  2117. found = 1;
  2118. }
  2119. return found;
  2120. }
  2121. /* Look upward to find a level containing the given NUMA node and workers to drive it */
  2122. static int _starpu_find_pu_driving_numa_up(hwloc_obj_t root, unsigned node)
  2123. {
  2124. if (_starpu_find_pu_driving_numa_from(root, node))
  2125. /* Ok, we already managed to find drivers */
  2126. return 1;
  2127. if (!root->parent)
  2128. /* And no parent!? nobody can drive this... */
  2129. return 0;
  2130. /* Try from parent */
  2131. return _starpu_find_pu_driving_numa_up(root->parent, node);
  2132. }
  2133. #endif
  2134. static void _starpu_init_workers_binding_and_memory(struct _starpu_machine_config *config, int no_mp_config STARPU_ATTRIBUTE_UNUSED)
  2135. {
  2136. /* We will store all the busid of the different (src, dst)
  2137. * combinations in a matrix which we initialize here. */
  2138. _starpu_initialize_busid_matrix();
  2139. #if defined(STARPU_USE_CUDA) || defined(STARPU_SIMGRID)
  2140. unsigned cuda_init[STARPU_MAXCUDADEVS] = { };
  2141. unsigned cuda_memory_nodes[STARPU_MAXCUDADEVS];
  2142. unsigned cuda_bindid[STARPU_MAXCUDADEVS];
  2143. int cuda_globalbindid = -1;
  2144. #endif
  2145. #if defined(STARPU_USE_OPENCL) || defined(STARPU_SIMGRID)
  2146. unsigned opencl_init[STARPU_MAXOPENCLDEVS] = { };
  2147. unsigned opencl_memory_nodes[STARPU_MAXOPENCLDEVS];
  2148. unsigned opencl_bindid[STARPU_MAXOPENCLDEVS];
  2149. #endif
  2150. #ifdef STARPU_USE_MIC
  2151. unsigned mic_init[STARPU_MAXMICDEVS] = { };
  2152. unsigned mic_memory_nodes[STARPU_MAXMICDEVS];
  2153. unsigned mic_bindid[STARPU_MAXMICDEVS];
  2154. #endif
  2155. #ifdef STARPU_USE_MPI_MASTER_SLAVE
  2156. unsigned mpi_init[STARPU_MAXMPIDEVS] = { };
  2157. unsigned mpi_memory_nodes[STARPU_MAXMPIDEVS];
  2158. unsigned mpi_bindid[STARPU_MAXMPIDEVS];
  2159. #endif
  2160. unsigned bindid;
  2161. for (bindid = 0; bindid < config->nbindid; bindid++)
  2162. {
  2163. free(config->bindid_workers[bindid].workerids);
  2164. config->bindid_workers[bindid].workerids = NULL;
  2165. config->bindid_workers[bindid].nworkers = 0;
  2166. }
  2167. /* Init CPU binding before NUMA nodes, because we use it to discover NUMA nodes */
  2168. _starpu_init_binding_cpu(config);
  2169. /* Initialize NUMA nodes */
  2170. _starpu_init_numa_node(config);
  2171. _starpu_init_numa_bus();
  2172. unsigned worker;
  2173. for (worker = 0; worker < config->topology.nworkers; worker++)
  2174. {
  2175. unsigned memory_node = -1;
  2176. struct _starpu_worker *workerarg = &config->workers[worker];
  2177. unsigned devid STARPU_ATTRIBUTE_UNUSED = workerarg->devid;
  2178. #if defined(STARPU_USE_CUDA) || defined(STARPU_USE_OPENCL) || defined(STARPU_USE_MIC) || defined(STARPU_SIMGRID) || defined(STARPU_USE_MPI_MASTER_SLAVE)
  2179. /* Perhaps the worker has some "favourite" bindings */
  2180. unsigned *preferred_binding = NULL;
  2181. unsigned npreferred = 0;
  2182. #endif
  2183. /* select the memory node that contains worker's memory */
  2184. switch (workerarg->arch)
  2185. {
  2186. case STARPU_CPU_WORKER:
  2187. {
  2188. int numa_logical_id = _starpu_get_logical_numa_node_worker(worker);
  2189. int numa_starpu_id = starpu_memory_nodes_numa_hwloclogid_to_id(numa_logical_id);
  2190. if (numa_starpu_id < 0 || numa_starpu_id >= STARPU_MAXNUMANODES)
  2191. numa_starpu_id = STARPU_MAIN_RAM;
  2192. #if defined(STARPU_HAVE_HWLOC) && !defined(STARPU_SIMGRID)
  2193. hwloc_obj_t pu_obj = hwloc_get_obj_by_type(config->topology.hwtopology, HWLOC_OBJ_PU, workerarg->bindid);
  2194. struct _starpu_hwloc_userdata *userdata = pu_obj->userdata;
  2195. userdata->pu_worker = workerarg;
  2196. #endif
  2197. workerarg->numa_memory_node = memory_node = numa_starpu_id;
  2198. _starpu_memory_node_add_nworkers(memory_node);
  2199. _starpu_worker_drives_memory_node(workerarg, numa_starpu_id);
  2200. break;
  2201. }
  2202. #if defined(STARPU_USE_CUDA) || defined(STARPU_SIMGRID)
  2203. case STARPU_CUDA_WORKER:
  2204. {
  2205. unsigned numa;
  2206. #ifndef STARPU_SIMGRID
  2207. if (may_bind_automatically[STARPU_CUDA_WORKER])
  2208. {
  2209. /* StarPU is allowed to bind threads automatically */
  2210. preferred_binding = _starpu_get_cuda_affinity_vector(devid);
  2211. npreferred = config->topology.nhwpus;
  2212. }
  2213. #endif /* SIMGRID */
  2214. if (cuda_init[devid])
  2215. {
  2216. memory_node = cuda_memory_nodes[devid];
  2217. if (config->topology.cuda_th_per_stream == 0)
  2218. workerarg->bindid = cuda_bindid[devid];
  2219. else
  2220. workerarg->bindid = _starpu_get_next_bindid(config, STARPU_THREAD_ACTIVE, preferred_binding, npreferred);
  2221. }
  2222. else
  2223. {
  2224. cuda_init[devid] = 1;
  2225. if (config->topology.cuda_th_per_dev == 0 && config->topology.cuda_th_per_stream == 0)
  2226. {
  2227. if (cuda_globalbindid == -1)
  2228. cuda_globalbindid = _starpu_get_next_bindid(config, STARPU_THREAD_ACTIVE, preferred_binding, npreferred);
  2229. workerarg->bindid = cuda_bindid[devid] = cuda_globalbindid;
  2230. }
  2231. else
  2232. workerarg->bindid = cuda_bindid[devid] = _starpu_get_next_bindid(config, STARPU_THREAD_ACTIVE, preferred_binding, npreferred);
  2233. memory_node = cuda_memory_nodes[devid] = _starpu_memory_node_register(STARPU_CUDA_RAM, devid, &_starpu_driver_cuda_node_ops);
  2234. for (numa = 0; numa < nb_numa_nodes; numa++)
  2235. {
  2236. _starpu_cuda_bus_ids[numa][devid+STARPU_MAXNUMANODES] = _starpu_register_bus(numa, memory_node);
  2237. _starpu_cuda_bus_ids[devid+STARPU_MAXNUMANODES][numa] = _starpu_register_bus(memory_node, numa);
  2238. }
  2239. #ifdef STARPU_SIMGRID
  2240. const char* cuda_memcpy_peer;
  2241. char name[16];
  2242. snprintf(name, sizeof(name), "CUDA%u", devid);
  2243. starpu_sg_host_t host = _starpu_simgrid_get_host_by_name(name);
  2244. STARPU_ASSERT(host);
  2245. _starpu_simgrid_memory_node_set_host(memory_node, host);
  2246. # ifdef STARPU_HAVE_SIMGRID_ACTOR_H
  2247. cuda_memcpy_peer = sg_host_get_property_value(host, "memcpy_peer");
  2248. # else
  2249. cuda_memcpy_peer = MSG_host_get_property_value(host, "memcpy_peer");
  2250. # endif
  2251. #endif /* SIMGRID */
  2252. if (
  2253. #ifdef STARPU_SIMGRID
  2254. cuda_memcpy_peer && atoll(cuda_memcpy_peer)
  2255. #elif defined(STARPU_HAVE_CUDA_MEMCPY_PEER)
  2256. 1
  2257. #else /* MEMCPY_PEER */
  2258. 0
  2259. #endif /* MEMCPY_PEER */
  2260. )
  2261. {
  2262. unsigned worker2;
  2263. for (worker2 = 0; worker2 < worker; worker2++)
  2264. {
  2265. struct _starpu_worker *workerarg2 = &config->workers[worker2];
  2266. int devid2 = workerarg2->devid;
  2267. if (workerarg2->arch == STARPU_CUDA_WORKER)
  2268. {
  2269. unsigned memory_node2 = starpu_worker_get_memory_node(worker2);
  2270. _starpu_cuda_bus_ids[devid2+STARPU_MAXNUMANODES][devid+STARPU_MAXNUMANODES] = _starpu_register_bus(memory_node2, memory_node);
  2271. _starpu_cuda_bus_ids[devid+STARPU_MAXNUMANODES][devid2+STARPU_MAXNUMANODES] = _starpu_register_bus(memory_node, memory_node2);
  2272. #ifndef STARPU_SIMGRID
  2273. #if defined(HAVE_DECL_HWLOC_CUDA_GET_DEVICE_OSDEV_BY_INDEX) && HAVE_DECL_HWLOC_CUDA_GET_DEVICE_OSDEV_BY_INDEX
  2274. {
  2275. hwloc_obj_t obj, obj2, ancestor;
  2276. obj = hwloc_cuda_get_device_osdev_by_index(config->topology.hwtopology, devid);
  2277. obj2 = hwloc_cuda_get_device_osdev_by_index(config->topology.hwtopology, devid2);
  2278. ancestor = hwloc_get_common_ancestor_obj(config->topology.hwtopology, obj, obj2);
  2279. if (ancestor)
  2280. {
  2281. struct _starpu_hwloc_userdata *data = ancestor->userdata;
  2282. #ifdef STARPU_VERBOSE
  2283. {
  2284. char name[64];
  2285. hwloc_obj_type_snprintf(name, sizeof(name), ancestor, 0);
  2286. _STARPU_DEBUG("CUDA%u and CUDA%u are linked through %s, along %u GPUs\n", devid, devid2, name, data->ngpus);
  2287. }
  2288. #endif
  2289. starpu_bus_set_ngpus(_starpu_cuda_bus_ids[devid2+STARPU_MAXNUMANODES][devid+STARPU_MAXNUMANODES], data->ngpus);
  2290. starpu_bus_set_ngpus(_starpu_cuda_bus_ids[devid+STARPU_MAXNUMANODES][devid2+STARPU_MAXNUMANODES], data->ngpus);
  2291. }
  2292. }
  2293. #endif
  2294. #endif
  2295. }
  2296. }
  2297. }
  2298. }
  2299. _starpu_memory_node_add_nworkers(memory_node);
  2300. //This worker can manage transfers on NUMA nodes
  2301. for (numa = 0; numa < nb_numa_nodes; numa++)
  2302. _starpu_worker_drives_memory_node(&workerarg->set->workers[0], numa);
  2303. _starpu_worker_drives_memory_node(&workerarg->set->workers[0], memory_node);
  2304. break;
  2305. }
  2306. #endif
  2307. #if defined(STARPU_USE_OPENCL) || defined(STARPU_SIMGRID)
  2308. case STARPU_OPENCL_WORKER:
  2309. {
  2310. unsigned numa;
  2311. #ifndef STARPU_SIMGRID
  2312. if (may_bind_automatically[STARPU_OPENCL_WORKER])
  2313. {
  2314. /* StarPU is allowed to bind threads automatically */
  2315. preferred_binding = _starpu_get_opencl_affinity_vector(devid);
  2316. npreferred = config->topology.nhwpus;
  2317. }
  2318. #endif /* SIMGRID */
  2319. if (opencl_init[devid])
  2320. {
  2321. memory_node = opencl_memory_nodes[devid];
  2322. #ifndef STARPU_SIMGRID
  2323. workerarg->bindid = opencl_bindid[devid];
  2324. #endif /* SIMGRID */
  2325. }
  2326. else
  2327. {
  2328. opencl_init[devid] = 1;
  2329. workerarg->bindid = opencl_bindid[devid] = _starpu_get_next_bindid(config, STARPU_THREAD_ACTIVE, preferred_binding, npreferred);
  2330. memory_node = opencl_memory_nodes[devid] = _starpu_memory_node_register(STARPU_OPENCL_RAM, devid, &_starpu_driver_opencl_node_ops);
  2331. for (numa = 0; numa < nb_numa_nodes; numa++)
  2332. {
  2333. _starpu_register_bus(numa, memory_node);
  2334. _starpu_register_bus(memory_node, numa);
  2335. }
  2336. #ifdef STARPU_SIMGRID
  2337. char name[16];
  2338. snprintf(name, sizeof(name), "OpenCL%u", devid);
  2339. starpu_sg_host_t host = _starpu_simgrid_get_host_by_name(name);
  2340. STARPU_ASSERT(host);
  2341. _starpu_simgrid_memory_node_set_host(memory_node, host);
  2342. #endif /* SIMGRID */
  2343. }
  2344. _starpu_memory_node_add_nworkers(memory_node);
  2345. //This worker can manage transfers on NUMA nodes
  2346. for (numa = 0; numa < nb_numa_nodes; numa++)
  2347. _starpu_worker_drives_memory_node(workerarg, numa);
  2348. _starpu_worker_drives_memory_node(workerarg, memory_node);
  2349. break;
  2350. }
  2351. #endif
  2352. #ifdef STARPU_USE_MIC
  2353. case STARPU_MIC_WORKER:
  2354. {
  2355. unsigned numa;
  2356. if (mic_init[devid])
  2357. {
  2358. memory_node = mic_memory_nodes[devid];
  2359. }
  2360. else
  2361. {
  2362. mic_init[devid] = 1;
  2363. /* TODO */
  2364. //if (may_bind_automatically)
  2365. //{
  2366. // /* StarPU is allowed to bind threads automatically */
  2367. // preferred_binding = _starpu_get_mic_affinity_vector(devid);
  2368. // npreferred = config->topology.nhwpus;
  2369. //}
  2370. mic_bindid[devid] = _starpu_get_next_bindid(config, STARPU_THREAD_ACTIVE, preferred_binding, npreferred);
  2371. memory_node = mic_memory_nodes[devid] = _starpu_memory_node_register(STARPU_MIC_RAM, devid, &_starpu_driver_mic_node_ops);
  2372. for (numa = 0; numa < nb_numa_nodes; numa++)
  2373. {
  2374. _starpu_register_bus(numa, memory_node);
  2375. _starpu_register_bus(memory_node, numa);
  2376. }
  2377. }
  2378. workerarg->bindid = mic_bindid[devid];
  2379. _starpu_memory_node_add_nworkers(memory_node);
  2380. //This worker can manage transfers on NUMA nodes
  2381. for (numa = 0; numa < nb_numa_nodes; numa++)
  2382. _starpu_worker_drives_memory_node(&workerarg->set->workers[0], numa);
  2383. _starpu_worker_drives_memory_node(&workerarg->set->workers[0], memory_node);
  2384. break;
  2385. }
  2386. #endif /* STARPU_USE_MIC */
  2387. #ifdef STARPU_USE_MPI_MASTER_SLAVE
  2388. case STARPU_MPI_MS_WORKER:
  2389. {
  2390. unsigned numa;
  2391. if (mpi_init[devid])
  2392. {
  2393. memory_node = mpi_memory_nodes[devid];
  2394. }
  2395. else
  2396. {
  2397. mpi_init[devid] = 1;
  2398. mpi_bindid[devid] = _starpu_get_next_bindid(config, STARPU_THREAD_ACTIVE, preferred_binding, npreferred);
  2399. memory_node = mpi_memory_nodes[devid] = _starpu_memory_node_register(STARPU_MPI_MS_RAM, devid, &_starpu_driver_mpi_node_ops);
  2400. for (numa = 0; numa < nb_numa_nodes; numa++)
  2401. {
  2402. _starpu_register_bus(numa, memory_node);
  2403. _starpu_register_bus(memory_node, numa);
  2404. }
  2405. }
  2406. //This worker can manage transfers on NUMA nodes
  2407. for (numa = 0; numa < nb_numa_nodes; numa++)
  2408. _starpu_worker_drives_memory_node(&workerarg->set->workers[0], numa);
  2409. _starpu_worker_drives_memory_node(&workerarg->set->workers[0], memory_node);
  2410. #ifndef STARPU_MPI_MASTER_SLAVE_MULTIPLE_THREAD
  2411. /* MPI driver thread can manage all slave memories if we disable the MPI multiple thread */
  2412. unsigned findworker;
  2413. for (findworker = 0; findworker < worker; findworker++)
  2414. {
  2415. struct _starpu_worker *findworkerarg = &config->workers[findworker];
  2416. if (findworkerarg->arch == STARPU_MPI_MS_WORKER)
  2417. {
  2418. _starpu_worker_drives_memory_node(workerarg, findworkerarg->memory_node);
  2419. _starpu_worker_drives_memory_node(findworkerarg, memory_node);
  2420. }
  2421. }
  2422. #endif
  2423. workerarg->bindid = mpi_bindid[devid];
  2424. _starpu_memory_node_add_nworkers(memory_node);
  2425. break;
  2426. }
  2427. #endif /* STARPU_USE_MPI_MASTER_SLAVE */
  2428. default:
  2429. STARPU_ABORT();
  2430. }
  2431. workerarg->memory_node = memory_node;
  2432. _STARPU_DEBUG("worker %u type %d devid %u bound to cpu %d, STARPU memory node %u\n", worker, workerarg->arch, devid, workerarg->bindid, memory_node);
  2433. #ifdef __GLIBC__
  2434. if (workerarg->bindid != -1)
  2435. {
  2436. /* Save the initial cpuset */
  2437. CPU_ZERO(&workerarg->cpu_set);
  2438. CPU_SET(workerarg->bindid, &workerarg->cpu_set);
  2439. }
  2440. #endif /* __GLIBC__ */
  2441. #ifdef STARPU_HAVE_HWLOC
  2442. if (workerarg->bindid == -1)
  2443. {
  2444. workerarg->hwloc_cpu_set = hwloc_bitmap_alloc();
  2445. workerarg->hwloc_obj = NULL;
  2446. }
  2447. else
  2448. {
  2449. /* Put the worker descriptor in the userdata field of the
  2450. * hwloc object describing the CPU */
  2451. hwloc_obj_t worker_obj = hwloc_get_obj_by_depth(config->topology.hwtopology,
  2452. config->pu_depth,
  2453. workerarg->bindid);
  2454. struct _starpu_hwloc_userdata *data = worker_obj->userdata;
  2455. if (data->worker_list == NULL)
  2456. data->worker_list = _starpu_worker_list_new();
  2457. _starpu_worker_list_push_front(data->worker_list, workerarg);
  2458. /* Clear the cpu set and set the cpu */
  2459. workerarg->hwloc_cpu_set = hwloc_bitmap_dup(worker_obj->cpuset);
  2460. workerarg->hwloc_obj = worker_obj;
  2461. }
  2462. #endif
  2463. if (workerarg->bindid != -1)
  2464. {
  2465. bindid = workerarg->bindid;
  2466. unsigned old_nbindid = config->nbindid;
  2467. if (bindid >= old_nbindid)
  2468. {
  2469. /* More room needed */
  2470. if (!old_nbindid)
  2471. config->nbindid = STARPU_NMAXWORKERS;
  2472. else
  2473. config->nbindid = 2 * old_nbindid;
  2474. if (bindid > config->nbindid)
  2475. {
  2476. config->nbindid = bindid+1;
  2477. }
  2478. _STARPU_REALLOC(config->bindid_workers, config->nbindid * sizeof(config->bindid_workers[0]));
  2479. memset(&config->bindid_workers[old_nbindid], 0, (config->nbindid - old_nbindid) * sizeof(config->bindid_workers[0]));
  2480. }
  2481. /* Add slot for this worker */
  2482. /* Don't care about amortizing the cost, there are usually very few workers sharing the same bindid */
  2483. config->bindid_workers[bindid].nworkers++;
  2484. _STARPU_REALLOC(config->bindid_workers[bindid].workerids, config->bindid_workers[bindid].nworkers * sizeof(config->bindid_workers[bindid].workerids[0]));
  2485. config->bindid_workers[bindid].workerids[config->bindid_workers[bindid].nworkers-1] = worker;
  2486. }
  2487. }
  2488. #if defined(STARPU_HAVE_HWLOC) && !defined(STARPU_SIMGRID)
  2489. /* If some NUMA nodes don't have drivers, attribute some */
  2490. unsigned node, nnodes = starpu_memory_nodes_get_count();;
  2491. for (node = 0; node < nnodes; node++)
  2492. {
  2493. if (starpu_node_get_kind(node) != STARPU_CPU_RAM)
  2494. /* Only RAM nodes can be processed by any CPU */
  2495. continue;
  2496. for (worker = 0; worker < config->topology.nworkers; worker++)
  2497. {
  2498. if (_starpu_worker_drives_memory[worker][node])
  2499. break;
  2500. }
  2501. if (worker < config->topology.nworkers)
  2502. /* Already somebody driving it */
  2503. continue;
  2504. /* Nobody driving this node! Attribute some */
  2505. _STARPU_DEBUG("nobody drives memory node %d\n", node);
  2506. hwloc_obj_t numa_node_obj = hwloc_get_obj_by_type(config->topology.hwtopology, HWLOC_OBJ_NUMANODE, starpu_memory_nodes_numa_id_to_hwloclogid(node));
  2507. int ret = _starpu_find_pu_driving_numa_up(numa_node_obj, node);
  2508. STARPU_ASSERT_MSG(ret, "oops, didn't find any worker to drive memory node %d!?", node);
  2509. }
  2510. #endif
  2511. #ifdef STARPU_SIMGRID
  2512. _starpu_simgrid_count_ngpus();
  2513. #else
  2514. #ifdef STARPU_HAVE_HWLOC
  2515. _starpu_topology_count_ngpus(hwloc_get_root_obj(config->topology.hwtopology));
  2516. #endif
  2517. #endif
  2518. }
  2519. int _starpu_build_topology(struct _starpu_machine_config *config, int no_mp_config)
  2520. {
  2521. int ret;
  2522. unsigned i;
  2523. ret = _starpu_init_machine_config(config, no_mp_config);
  2524. if (ret)
  2525. return ret;
  2526. /* for the data management library */
  2527. _starpu_memory_nodes_init();
  2528. _starpu_datastats_init();
  2529. _starpu_init_workers_binding_and_memory(config, no_mp_config);
  2530. _starpu_mem_chunk_init_last();
  2531. config->cpus_nodeid = -1;
  2532. config->cuda_nodeid = -1;
  2533. config->opencl_nodeid = -1;
  2534. config->mic_nodeid = -1;
  2535. config->mpi_nodeid = -1;
  2536. for (i = 0; i < starpu_worker_get_count(); i++)
  2537. {
  2538. switch (starpu_worker_get_type(i))
  2539. {
  2540. case STARPU_CPU_WORKER:
  2541. if (config->cpus_nodeid == -1)
  2542. config->cpus_nodeid = starpu_worker_get_memory_node(i);
  2543. else if (config->cpus_nodeid != (int) starpu_worker_get_memory_node(i))
  2544. config->cpus_nodeid = -2;
  2545. break;
  2546. case STARPU_CUDA_WORKER:
  2547. if (config->cuda_nodeid == -1)
  2548. config->cuda_nodeid = starpu_worker_get_memory_node(i);
  2549. else if (config->cuda_nodeid != (int) starpu_worker_get_memory_node(i))
  2550. config->cuda_nodeid = -2;
  2551. break;
  2552. case STARPU_OPENCL_WORKER:
  2553. if (config->opencl_nodeid == -1)
  2554. config->opencl_nodeid = starpu_worker_get_memory_node(i);
  2555. else if (config->opencl_nodeid != (int) starpu_worker_get_memory_node(i))
  2556. config->opencl_nodeid = -2;
  2557. break;
  2558. case STARPU_MIC_WORKER:
  2559. if (config->mic_nodeid == -1)
  2560. config->mic_nodeid = starpu_worker_get_memory_node(i);
  2561. else if (config->mic_nodeid != (int) starpu_worker_get_memory_node(i))
  2562. config->mic_nodeid = -2;
  2563. break;
  2564. case STARPU_MPI_MS_WORKER:
  2565. if (config->mpi_nodeid == -1)
  2566. config->mpi_nodeid = starpu_worker_get_memory_node(i);
  2567. else if (config->mpi_nodeid != (int) starpu_worker_get_memory_node(i))
  2568. config->mpi_nodeid = -2;
  2569. break;
  2570. case STARPU_ANY_WORKER:
  2571. STARPU_ASSERT(0);
  2572. }
  2573. }
  2574. return 0;
  2575. }
  2576. void _starpu_destroy_topology(struct _starpu_machine_config *config STARPU_ATTRIBUTE_UNUSED)
  2577. {
  2578. #if defined(STARPU_USE_MIC) || defined(STARPU_USE_MPI_MASTER_SLAVE)
  2579. _starpu_deinit_mp_config(config);
  2580. #endif
  2581. /* cleanup StarPU internal data structures */
  2582. _starpu_memory_nodes_deinit();
  2583. _starpu_destroy_machine_config(config);
  2584. _starpu_deinitialize_workers_bindid(config);
  2585. }
  2586. void starpu_topology_print(FILE *output)
  2587. {
  2588. struct _starpu_machine_config *config = _starpu_get_machine_config();
  2589. struct _starpu_machine_topology *topology = &config->topology;
  2590. unsigned pu;
  2591. unsigned worker;
  2592. unsigned nworkers = starpu_worker_get_count();
  2593. unsigned ncombinedworkers = topology->ncombinedworkers;
  2594. unsigned nthreads_per_core = topology->nhwpus / topology->nhwcpus;
  2595. #ifdef STARPU_HAVE_HWLOC
  2596. hwloc_topology_t topo = topology->hwtopology;
  2597. hwloc_obj_t pu_obj;
  2598. hwloc_obj_t last_numa_obj = NULL, numa_obj;
  2599. hwloc_obj_t last_package_obj = NULL, package_obj;
  2600. #endif
  2601. for (pu = 0; pu < topology->nhwpus; pu++)
  2602. {
  2603. #ifdef STARPU_HAVE_HWLOC
  2604. pu_obj = hwloc_get_obj_by_type(topo, HWLOC_OBJ_PU, pu);
  2605. numa_obj = numa_get_obj(pu_obj);
  2606. if (numa_obj != last_numa_obj)
  2607. {
  2608. fprintf(output, "numa %u", numa_obj->logical_index);
  2609. last_numa_obj = numa_obj;
  2610. }
  2611. fprintf(output, "\t");
  2612. package_obj = hwloc_get_ancestor_obj_by_type(topo, HWLOC_OBJ_SOCKET, pu_obj);
  2613. if (package_obj != last_package_obj)
  2614. {
  2615. fprintf(output, "pack %u", package_obj->logical_index);
  2616. last_package_obj = package_obj;
  2617. }
  2618. fprintf(output, "\t");
  2619. #endif
  2620. if ((pu % nthreads_per_core) == 0)
  2621. fprintf(output, "core %u", pu / nthreads_per_core);
  2622. fprintf(output, "\tPU %u\t", pu);
  2623. for (worker = 0;
  2624. worker < nworkers + ncombinedworkers;
  2625. worker++)
  2626. {
  2627. if (worker < nworkers)
  2628. {
  2629. struct _starpu_worker *workerarg = &config->workers[worker];
  2630. if (workerarg->bindid == (int) pu)
  2631. {
  2632. char name[256];
  2633. starpu_worker_get_name(worker, name, sizeof(name));
  2634. fprintf(output, "%s\t", name);
  2635. }
  2636. }
  2637. else
  2638. {
  2639. int worker_size, i;
  2640. int *combined_workerid;
  2641. starpu_combined_worker_get_description(worker, &worker_size, &combined_workerid);
  2642. for (i = 0; i < worker_size; i++)
  2643. {
  2644. if (topology->workers_bindid[combined_workerid[i]] == pu)
  2645. fprintf(output, "comb %u\t", worker-nworkers);
  2646. }
  2647. }
  2648. }
  2649. fprintf(output, "\n");
  2650. }
  2651. }