starpu_mpi.c 67 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593594595596597598599600601602603604605606607608609610611612613614615616617618619620621622623624625626627628629630631632633634635636637638639640641642643644645646647648649650651652653654655656657658659660661662663664665666667668669670671672673674675676677678679680681682683684685686687688689690691692693694695696697698699700701702703704705706707708709710711712713714715716717718719720721722723724725726727728729730731732733734735736737738739740741742743744745746747748749750751752753754755756757758759760761762763764765766767768769770771772773774775776777778779780781782783784785786787788789790791792793794795796797798799800801802803804805806807808809810811812813814815816817818819820821822823824825826827828829830831832833834835836837838839840841842843844845846847848849850851852853854855856857858859860861862863864865866867868869870871872873874875876877878879880881882883884885886887888889890891892893894895896897898899900901902903904905906907908909910911912913914915916917918919920921922923924925926927928929930931932933934935936937938939940941942943944945946947948949950951952953954955956957958959960961962963964965966967968969970971972973974975976977978979980981982983984985986987988989990991992993994995996997998999100010011002100310041005100610071008100910101011101210131014101510161017101810191020102110221023102410251026102710281029103010311032103310341035103610371038103910401041104210431044104510461047104810491050105110521053105410551056105710581059106010611062106310641065106610671068106910701071107210731074107510761077107810791080108110821083108410851086108710881089109010911092109310941095109610971098109911001101110211031104110511061107110811091110111111121113111411151116111711181119112011211122112311241125112611271128112911301131113211331134113511361137113811391140114111421143114411451146114711481149115011511152115311541155115611571158115911601161116211631164116511661167116811691170117111721173117411751176117711781179118011811182118311841185118611871188118911901191119211931194119511961197119811991200120112021203120412051206120712081209121012111212121312141215121612171218121912201221122212231224122512261227122812291230123112321233123412351236123712381239124012411242124312441245124612471248124912501251125212531254125512561257125812591260126112621263126412651266126712681269127012711272127312741275127612771278127912801281128212831284128512861287128812891290129112921293129412951296129712981299130013011302130313041305130613071308130913101311131213131314131513161317131813191320132113221323132413251326132713281329133013311332133313341335133613371338133913401341134213431344134513461347134813491350135113521353135413551356135713581359136013611362136313641365136613671368136913701371137213731374137513761377137813791380138113821383138413851386138713881389139013911392139313941395139613971398139914001401140214031404140514061407140814091410141114121413141414151416141714181419142014211422142314241425142614271428142914301431143214331434143514361437143814391440144114421443144414451446144714481449145014511452145314541455145614571458145914601461146214631464146514661467146814691470147114721473147414751476147714781479148014811482148314841485148614871488148914901491149214931494149514961497149814991500150115021503150415051506150715081509151015111512151315141515151615171518151915201521152215231524152515261527152815291530153115321533153415351536153715381539154015411542154315441545154615471548154915501551155215531554155515561557155815591560156115621563156415651566156715681569157015711572157315741575157615771578157915801581158215831584158515861587158815891590159115921593159415951596159715981599160016011602160316041605160616071608160916101611161216131614161516161617161816191620162116221623162416251626162716281629163016311632163316341635163616371638163916401641164216431644164516461647164816491650165116521653165416551656165716581659166016611662166316641665166616671668166916701671167216731674167516761677167816791680168116821683168416851686168716881689169016911692169316941695169616971698169917001701170217031704170517061707170817091710171117121713171417151716171717181719172017211722172317241725172617271728172917301731173217331734173517361737173817391740174117421743174417451746174717481749175017511752175317541755175617571758175917601761176217631764176517661767176817691770177117721773177417751776177717781779178017811782178317841785178617871788178917901791179217931794179517961797179817991800180118021803180418051806180718081809181018111812181318141815181618171818181918201821182218231824182518261827182818291830183118321833183418351836183718381839184018411842184318441845184618471848184918501851185218531854185518561857185818591860186118621863186418651866186718681869187018711872187318741875187618771878187918801881
  1. /* StarPU --- Runtime system for heterogeneous multicore architectures.
  2. *
  3. * Copyright (C) 2009, 2010-2017 Université de Bordeaux
  4. * Copyright (C) 2010, 2011, 2012, 2013, 2014, 2015, 2016, 2017 CNRS
  5. * Copyright (C) 2016 Inria
  6. *
  7. * StarPU is free software; you can redistribute it and/or modify
  8. * it under the terms of the GNU Lesser General Public License as published by
  9. * the Free Software Foundation; either version 2.1 of the License, or (at
  10. * your option) any later version.
  11. *
  12. * StarPU is distributed in the hope that it will be useful, but
  13. * WITHOUT ANY WARRANTY; without even the implied warranty of
  14. * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE.
  15. *
  16. * See the GNU Lesser General Public License in COPYING.LGPL for more details.
  17. */
  18. #include <stdlib.h>
  19. #include <starpu_mpi.h>
  20. #include <starpu_mpi_datatype.h>
  21. #include <starpu_mpi_private.h>
  22. #include <starpu_mpi_cache.h>
  23. #include <starpu_profiling.h>
  24. #include <starpu_mpi_stats.h>
  25. #include <starpu_mpi_cache.h>
  26. #include <starpu_mpi_sync_data.h>
  27. #include <starpu_mpi_early_data.h>
  28. #include <starpu_mpi_early_request.h>
  29. #include <starpu_mpi_select_node.h>
  30. #include <starpu_mpi_tag.h>
  31. #include <starpu_mpi_comm.h>
  32. #include <starpu_mpi_init.h>
  33. #include <common/config.h>
  34. #include <common/thread.h>
  35. #include <datawizard/interfaces/data_interface.h>
  36. #include <datawizard/coherency.h>
  37. #include <core/simgrid.h>
  38. #include <core/task.h>
  39. /* Number of ready requests to process before polling for completed requests */
  40. #define NREADY_PROCESS 10
  41. static void _starpu_mpi_add_sync_point_in_fxt(void);
  42. static void _starpu_mpi_submit_ready_request(void *arg);
  43. static void _starpu_mpi_handle_ready_request(struct _starpu_mpi_req *req);
  44. static void _starpu_mpi_handle_request_termination(struct _starpu_mpi_req *req);
  45. #ifdef STARPU_VERBOSE
  46. static char *_starpu_mpi_request_type(enum _starpu_mpi_request_type request_type);
  47. #endif
  48. static struct _starpu_mpi_req *_starpu_mpi_isend_common(starpu_data_handle_t data_handle,
  49. int dest, int data_tag, MPI_Comm comm,
  50. unsigned detached, unsigned sync, void (*callback)(void *), void *arg,
  51. int sequential_consistency);
  52. static struct _starpu_mpi_req *_starpu_mpi_irecv_common(starpu_data_handle_t data_handle,
  53. int source, int data_tag, MPI_Comm comm,
  54. unsigned detached, unsigned sync, void (*callback)(void *), void *arg,
  55. int sequential_consistency, int is_internal_req,
  56. starpu_ssize_t count);
  57. static void _starpu_mpi_handle_detached_request(struct _starpu_mpi_req *req);
  58. static void _starpu_mpi_early_data_cb(void* arg);
  59. /* The list of ready requests */
  60. static struct _starpu_mpi_req_list *ready_requests;
  61. /* The list of detached requests that have already been submitted to MPI */
  62. static struct _starpu_mpi_req_list *detached_requests;
  63. static starpu_pthread_mutex_t detached_requests_mutex;
  64. /* Condition to wake up progression thread */
  65. static starpu_pthread_cond_t progress_cond;
  66. /* Condition to wake up waiting for all current MPI requests to finish */
  67. static starpu_pthread_cond_t barrier_cond;
  68. static starpu_pthread_mutex_t progress_mutex;
  69. #ifndef STARPU_SIMGRID
  70. static starpu_pthread_t progress_thread;
  71. #endif
  72. static int running = 0;
  73. #ifdef STARPU_SIMGRID
  74. static int wait_counter;
  75. static starpu_pthread_cond_t wait_counter_cond;
  76. static starpu_pthread_mutex_t wait_counter_mutex;
  77. #endif
  78. int _starpu_mpi_fake_world_size = -1;
  79. int _starpu_mpi_fake_world_rank = -1;
  80. /* Count requests posted by the application and not yet submitted to MPI */
  81. static starpu_pthread_mutex_t mutex_posted_requests;
  82. static int posted_requests = 0, newer_requests, barrier_running = 0;
  83. #define _STARPU_MPI_INC_POSTED_REQUESTS(value) { STARPU_PTHREAD_MUTEX_LOCK(&mutex_posted_requests); posted_requests += value; STARPU_PTHREAD_MUTEX_UNLOCK(&mutex_posted_requests); }
  84. #pragma weak smpi_simulated_main_
  85. extern int smpi_simulated_main_(int argc, char *argv[]);
  86. #ifdef HAVE_SMPI_PROCESS_SET_USER_DATA
  87. #if !HAVE_DECL_SMPI_PROCESS_SET_USER_DATA
  88. extern void smpi_process_set_user_data(void *);
  89. #endif
  90. #endif
  91. static void _starpu_mpi_request_init(struct _starpu_mpi_req **req)
  92. {
  93. _STARPU_MPI_CALLOC(*req, 1, sizeof(struct _starpu_mpi_req));
  94. /* Initialize the request structure */
  95. (*req)->data_handle = NULL;
  96. (*req)->datatype = 0;
  97. (*req)->datatype_name = NULL;
  98. (*req)->ptr = NULL;
  99. (*req)->count = -1;
  100. (*req)->registered_datatype = -1;
  101. (*req)->node_tag.rank = -1;
  102. (*req)->node_tag.data_tag = -1;
  103. (*req)->node_tag.comm = 0;
  104. (*req)->func = NULL;
  105. (*req)->status = NULL;
  106. (*req)->data_request = 0;
  107. (*req)->flag = NULL;
  108. (*req)->ret = -1;
  109. STARPU_PTHREAD_MUTEX_INIT(&((*req)->req_mutex), NULL);
  110. STARPU_PTHREAD_COND_INIT(&((*req)->req_cond), NULL);
  111. STARPU_PTHREAD_MUTEX_INIT(&((*req)->posted_mutex), NULL);
  112. STARPU_PTHREAD_COND_INIT(&((*req)->posted_cond), NULL);
  113. (*req)->request_type = UNKNOWN_REQ;
  114. (*req)->submitted = 0;
  115. (*req)->completed = 0;
  116. (*req)->posted = 0;
  117. (*req)->other_request = NULL;
  118. (*req)->sync = 0;
  119. (*req)->detached = -1;
  120. (*req)->callback = NULL;
  121. (*req)->callback_arg = NULL;
  122. (*req)->size_req = 0;
  123. (*req)->internal_req = NULL;
  124. (*req)->is_internal_req = 0;
  125. (*req)->early_data_handle = NULL;
  126. (*req)->envelope = NULL;
  127. (*req)->sequential_consistency = 1;
  128. (*req)->pre_sync_jobid = -1;
  129. (*req)->post_sync_jobid = -1;
  130. #ifdef STARPU_SIMGRID
  131. starpu_pthread_queue_init(&((*req)->queue));
  132. starpu_pthread_queue_register(&wait, &((*req)->queue));
  133. (*req)->done = 0;
  134. #endif
  135. }
  136. static void _starpu_mpi_request_destroy(struct _starpu_mpi_req *req)
  137. {
  138. STARPU_PTHREAD_MUTEX_DESTROY(&req->req_mutex);
  139. STARPU_PTHREAD_COND_DESTROY(&req->req_cond);
  140. STARPU_PTHREAD_MUTEX_DESTROY(&req->posted_mutex);
  141. STARPU_PTHREAD_COND_DESTROY(&req->posted_cond);
  142. free(req->datatype_name);
  143. req->datatype_name = NULL;
  144. #ifdef STARPU_SIMGRID
  145. starpu_pthread_queue_unregister(&wait, &req->queue);
  146. starpu_pthread_queue_destroy(&req->queue);
  147. #endif
  148. free(req);
  149. }
  150. /********************************************************/
  151. /* */
  152. /* Send/Receive functionalities */
  153. /* */
  154. /********************************************************/
  155. struct _starpu_mpi_early_data_cb_args
  156. {
  157. starpu_data_handle_t data_handle;
  158. starpu_data_handle_t early_handle;
  159. struct _starpu_mpi_req *req;
  160. void *buffer;
  161. };
  162. static void _starpu_mpi_submit_ready_request(void *arg)
  163. {
  164. _STARPU_MPI_LOG_IN();
  165. struct _starpu_mpi_req *req = arg;
  166. _STARPU_MPI_INC_POSTED_REQUESTS(-1);
  167. _STARPU_MPI_DEBUG(3, "new req %p srcdst %d tag %d and type %s %d\n", req, req->node_tag.rank, req->node_tag.data_tag, _starpu_mpi_request_type(req->request_type), req->is_internal_req);
  168. STARPU_PTHREAD_MUTEX_LOCK(&progress_mutex);
  169. if (req->request_type == RECV_REQ)
  170. {
  171. /* Case : the request is the internal receive request submitted
  172. * by StarPU-MPI to receive incoming data without a matching
  173. * early_request from the application. We immediately allocate the
  174. * pointer associated to the data_handle, and push it into the
  175. * ready_requests list, so as the real MPI request can be submitted
  176. * before the next submission of the envelope-catching request. */
  177. if (req->is_internal_req)
  178. {
  179. _starpu_mpi_datatype_allocate(req->data_handle, req);
  180. if (req->registered_datatype == 1)
  181. {
  182. req->count = 1;
  183. req->ptr = starpu_data_get_local_ptr(req->data_handle);
  184. }
  185. else
  186. {
  187. STARPU_ASSERT(req->count);
  188. _STARPU_MPI_MALLOC(req->ptr, req->count);
  189. }
  190. _STARPU_MPI_DEBUG(3, "Pushing internal starpu_mpi_irecv request %p type %s tag %d src %d data %p ptr %p datatype '%s' count %d registered_datatype %d \n",
  191. req, _starpu_mpi_request_type(req->request_type), req->node_tag.data_tag, req->node_tag.rank, req->data_handle, req->ptr,
  192. req->datatype_name, (int)req->count, req->registered_datatype);
  193. _starpu_mpi_req_list_push_front(ready_requests, req);
  194. /* inform the starpu mpi thread that the request has been pushed in the ready_requests list */
  195. STARPU_PTHREAD_MUTEX_UNLOCK(&progress_mutex);
  196. STARPU_PTHREAD_MUTEX_LOCK(&req->posted_mutex);
  197. req->posted = 1;
  198. STARPU_PTHREAD_COND_BROADCAST(&req->posted_cond);
  199. STARPU_PTHREAD_MUTEX_UNLOCK(&req->posted_mutex);
  200. STARPU_PTHREAD_MUTEX_LOCK(&progress_mutex);
  201. }
  202. else
  203. {
  204. /* test whether some data with the given tag and source have already been received by StarPU-MPI*/
  205. struct _starpu_mpi_early_data_handle *early_data_handle = _starpu_mpi_early_data_find(&req->node_tag);
  206. /* Case: a receive request for a data with the given tag and source has already been
  207. * posted by StarPU. Asynchronously requests a Read permission over the temporary handle ,
  208. * so as when the internal receive is completed, the _starpu_mpi_early_data_cb function
  209. * will be called to bring the data back to the original data handle associated to the request.*/
  210. if (early_data_handle)
  211. {
  212. STARPU_PTHREAD_MUTEX_UNLOCK(&progress_mutex);
  213. STARPU_PTHREAD_MUTEX_LOCK(&(early_data_handle->req_mutex));
  214. while (!(early_data_handle->req_ready))
  215. STARPU_PTHREAD_COND_WAIT(&(early_data_handle->req_cond), &(early_data_handle->req_mutex));
  216. STARPU_PTHREAD_MUTEX_UNLOCK(&(early_data_handle->req_mutex));
  217. STARPU_PTHREAD_MUTEX_LOCK(&progress_mutex);
  218. _STARPU_MPI_DEBUG(3, "The RECV request %p with tag %d has already been received, copying previously received data into handle's pointer..\n", req, req->node_tag.data_tag);
  219. STARPU_ASSERT(req->data_handle != early_data_handle->handle);
  220. req->internal_req = early_data_handle->req;
  221. req->early_data_handle = early_data_handle;
  222. struct _starpu_mpi_early_data_cb_args *cb_args;
  223. _STARPU_MPI_MALLOC(cb_args, sizeof(struct _starpu_mpi_early_data_cb_args));
  224. cb_args->data_handle = req->data_handle;
  225. cb_args->early_handle = early_data_handle->handle;
  226. cb_args->buffer = early_data_handle->buffer;
  227. cb_args->req = req;
  228. _STARPU_MPI_DEBUG(3, "Calling data_acquire_cb on starpu_mpi_copy_cb..\n");
  229. STARPU_PTHREAD_MUTEX_UNLOCK(&progress_mutex);
  230. starpu_data_acquire_cb(early_data_handle->handle,STARPU_R,_starpu_mpi_early_data_cb,(void*) cb_args);
  231. STARPU_PTHREAD_MUTEX_LOCK(&progress_mutex);
  232. }
  233. /* Case: no matching data has been received. Store the receive request as an early_request. */
  234. else
  235. {
  236. struct _starpu_mpi_req *sync_req = _starpu_mpi_sync_data_find(req->node_tag.data_tag, req->node_tag.rank, req->node_tag.comm);
  237. _STARPU_MPI_DEBUG(3, "----------> Looking for sync data for tag %d and src %d = %p\n", req->node_tag.data_tag, req->node_tag.rank, sync_req);
  238. if (sync_req)
  239. {
  240. req->sync = 1;
  241. _starpu_mpi_datatype_allocate(req->data_handle, req);
  242. if (req->registered_datatype == 1)
  243. {
  244. req->count = 1;
  245. req->ptr = starpu_data_get_local_ptr(req->data_handle);
  246. }
  247. else
  248. {
  249. req->count = sync_req->count;
  250. STARPU_ASSERT(req->count);
  251. _STARPU_MPI_MALLOC(req->ptr, req->count);
  252. }
  253. _starpu_mpi_req_list_push_front(ready_requests, req);
  254. _starpu_mpi_request_destroy(sync_req);
  255. }
  256. else
  257. {
  258. _STARPU_MPI_DEBUG(3, "Adding the pending receive request %p (srcdst %d tag %d) into the request hashmap\n", req, req->node_tag.rank, req->node_tag.data_tag);
  259. _starpu_mpi_early_request_enqueue(req);
  260. }
  261. }
  262. }
  263. }
  264. else
  265. {
  266. _starpu_mpi_req_list_push_front(ready_requests, req);
  267. _STARPU_MPI_DEBUG(3, "Pushing new request %p type %s tag %d src %d data %p ptr %p datatype '%s' count %d registered_datatype %d \n",
  268. req, _starpu_mpi_request_type(req->request_type), req->node_tag.data_tag, req->node_tag.rank, req->data_handle, req->ptr,
  269. req->datatype_name, (int)req->count, req->registered_datatype);
  270. }
  271. newer_requests = 1;
  272. STARPU_PTHREAD_COND_BROADCAST(&progress_cond);
  273. #ifdef STARPU_SIMGRID
  274. starpu_pthread_queue_signal(&dontsleep);
  275. #endif
  276. STARPU_PTHREAD_MUTEX_UNLOCK(&progress_mutex);
  277. _STARPU_MPI_LOG_OUT();
  278. }
  279. static void nop_acquire_cb(void *arg)
  280. {
  281. starpu_data_release(arg);
  282. }
  283. static struct _starpu_mpi_req *_starpu_mpi_isend_irecv_common(starpu_data_handle_t data_handle,
  284. int srcdst, int data_tag, MPI_Comm comm,
  285. unsigned detached, unsigned sync, void (*callback)(void *), void *arg,
  286. enum _starpu_mpi_request_type request_type, void (*func)(struct _starpu_mpi_req *),
  287. enum starpu_data_access_mode mode,
  288. int sequential_consistency,
  289. int is_internal_req,
  290. starpu_ssize_t count)
  291. {
  292. struct _starpu_mpi_req *req;
  293. if (_starpu_mpi_fake_world_size != -1)
  294. {
  295. /* Don't actually do the communication */
  296. starpu_data_acquire_on_node_cb_sequential_consistency(data_handle, STARPU_MAIN_RAM, mode, nop_acquire_cb, data_handle, sequential_consistency);
  297. return NULL;
  298. }
  299. _STARPU_MPI_LOG_IN();
  300. _STARPU_MPI_INC_POSTED_REQUESTS(1);
  301. _starpu_mpi_comm_register(comm);
  302. /* Initialize the request structure */
  303. _starpu_mpi_request_init(&req);
  304. req->request_type = request_type;
  305. req->data_handle = data_handle;
  306. req->node_tag.rank = srcdst;
  307. req->node_tag.data_tag = data_tag;
  308. req->node_tag.comm = comm;
  309. req->detached = detached;
  310. req->sync = sync;
  311. req->callback = callback;
  312. req->callback_arg = arg;
  313. req->func = func;
  314. req->sequential_consistency = sequential_consistency;
  315. req->is_internal_req = is_internal_req;
  316. req->count = count;
  317. /* Asynchronously request StarPU to fetch the data in main memory: when
  318. * it is available in main memory, _starpu_mpi_submit_ready_request(req) is called and
  319. * the request is actually submitted */
  320. starpu_data_acquire_on_node_cb_sequential_consistency_sync_jobids(data_handle, STARPU_MAIN_RAM, mode, _starpu_mpi_submit_ready_request, (void *)req, sequential_consistency, &req->pre_sync_jobid, &req->post_sync_jobid);
  321. _STARPU_MPI_LOG_OUT();
  322. return req;
  323. }
  324. #ifdef STARPU_SIMGRID
  325. int _starpu_mpi_simgrid_mpi_test(unsigned *done, int *flag)
  326. {
  327. *flag = 0;
  328. if (*done)
  329. {
  330. starpu_pthread_queue_signal(&dontsleep);
  331. *flag = 1;
  332. }
  333. return MPI_SUCCESS;
  334. }
  335. static void _starpu_mpi_simgrid_wait_req_func(void* arg)
  336. {
  337. struct _starpu_simgrid_mpi_req *sim_req = arg;
  338. int ret;
  339. STARPU_PTHREAD_MUTEX_LOCK(&wait_counter_mutex);
  340. wait_counter++;
  341. STARPU_PTHREAD_MUTEX_UNLOCK(&wait_counter_mutex);
  342. ret = MPI_Wait(sim_req->request, sim_req->status);
  343. STARPU_MPI_ASSERT_MSG(ret == MPI_SUCCESS, "MPI_Wait returning %s", _starpu_mpi_get_mpi_error_code(ret));
  344. *(sim_req->done) = 1;
  345. starpu_pthread_queue_signal(sim_req->queue);
  346. free(sim_req);
  347. STARPU_PTHREAD_MUTEX_LOCK(&wait_counter_mutex);
  348. if (--wait_counter == 0)
  349. STARPU_PTHREAD_COND_SIGNAL(&wait_counter_cond);
  350. STARPU_PTHREAD_MUTEX_UNLOCK(&wait_counter_mutex);
  351. }
  352. void _starpu_mpi_simgrid_wait_req(MPI_Request *request, MPI_Status *status, starpu_pthread_queue_t *queue, unsigned *done)
  353. {
  354. struct _starpu_simgrid_mpi_req *sim_req;
  355. _STARPU_MPI_CALLOC(sim_req, 1, sizeof(struct _starpu_simgrid_mpi_req));
  356. sim_req->request = request;
  357. sim_req->status = status;
  358. sim_req->queue = queue;
  359. sim_req->done = done;
  360. *done = 0;
  361. _starpu_simgrid_xbt_thread_create("wait for mpi transfer", _starpu_mpi_simgrid_wait_req_func, sim_req);
  362. }
  363. #endif
  364. /********************************************************/
  365. /* */
  366. /* Send functionalities */
  367. /* */
  368. /********************************************************/
  369. static void _starpu_mpi_isend_data_func(struct _starpu_mpi_req *req)
  370. {
  371. _STARPU_MPI_LOG_IN();
  372. _STARPU_MPI_DEBUG(30, "post MPI isend request %p type %s tag %d src %d data %p datasize %ld ptr %p datatype '%s' count %d registered_datatype %d sync %d\n", req, _starpu_mpi_request_type(req->request_type), req->node_tag.data_tag, req->node_tag.rank, req->data_handle, starpu_data_get_size(req->data_handle), req->ptr, req->datatype_name, (int)req->count, req->registered_datatype, req->sync);
  373. _starpu_mpi_comm_amounts_inc(req->node_tag.comm, req->node_tag.rank, req->datatype, req->count);
  374. _STARPU_MPI_TRACE_ISEND_SUBMIT_BEGIN(req->node_tag.rank, req->node_tag.data_tag, 0);
  375. if (req->sync == 0)
  376. {
  377. _STARPU_MPI_COMM_TO_DEBUG(req, req->count, req->datatype, req->node_tag.rank, _STARPU_MPI_TAG_DATA, req->node_tag.data_tag, req->node_tag.comm);
  378. req->ret = MPI_Isend(req->ptr, req->count, req->datatype, req->node_tag.rank, _STARPU_MPI_TAG_DATA, req->node_tag.comm, &req->data_request);
  379. STARPU_MPI_ASSERT_MSG(req->ret == MPI_SUCCESS, "MPI_Isend returning %s", _starpu_mpi_get_mpi_error_code(req->ret));
  380. }
  381. else
  382. {
  383. _STARPU_MPI_COMM_TO_DEBUG(req, req->count, req->datatype, req->node_tag.rank, _STARPU_MPI_TAG_SYNC_DATA, req->node_tag.data_tag, req->node_tag.comm);
  384. req->ret = MPI_Issend(req->ptr, req->count, req->datatype, req->node_tag.rank, _STARPU_MPI_TAG_SYNC_DATA, req->node_tag.comm, &req->data_request);
  385. STARPU_MPI_ASSERT_MSG(req->ret == MPI_SUCCESS, "MPI_Issend returning %s", _starpu_mpi_get_mpi_error_code(req->ret));
  386. }
  387. #ifdef STARPU_SIMGRID
  388. _starpu_mpi_simgrid_wait_req(&req->data_request, &req->status_store, &req->queue, &req->done);
  389. #endif
  390. _STARPU_MPI_TRACE_ISEND_SUBMIT_END(req->node_tag.rank, req->node_tag.data_tag, starpu_data_get_size(req->data_handle), req->pre_sync_jobid);
  391. /* somebody is perhaps waiting for the MPI request to be posted */
  392. STARPU_PTHREAD_MUTEX_LOCK(&req->req_mutex);
  393. req->submitted = 1;
  394. STARPU_PTHREAD_COND_BROADCAST(&req->req_cond);
  395. STARPU_PTHREAD_MUTEX_UNLOCK(&req->req_mutex);
  396. _starpu_mpi_handle_detached_request(req);
  397. _STARPU_MPI_LOG_OUT();
  398. }
  399. static void _starpu_mpi_isend_size_func(struct _starpu_mpi_req *req)
  400. {
  401. _starpu_mpi_datatype_allocate(req->data_handle, req);
  402. _STARPU_MPI_CALLOC(req->envelope, 1,sizeof(struct _starpu_mpi_envelope));
  403. req->envelope->mode = _STARPU_MPI_ENVELOPE_DATA;
  404. req->envelope->data_tag = req->node_tag.data_tag;
  405. req->envelope->sync = req->sync;
  406. if (req->registered_datatype == 1)
  407. {
  408. int size;
  409. req->count = 1;
  410. req->ptr = starpu_data_get_local_ptr(req->data_handle);
  411. MPI_Type_size(req->datatype, &size);
  412. req->envelope->size = (starpu_ssize_t)req->count * size;
  413. _STARPU_MPI_DEBUG(20, "Post MPI isend count (%ld) datatype_size %ld request to %d\n",req->count,starpu_data_get_size(req->data_handle), req->node_tag.rank);
  414. _STARPU_MPI_COMM_TO_DEBUG(req->envelope, sizeof(struct _starpu_mpi_envelope), MPI_BYTE, req->node_tag.rank, _STARPU_MPI_TAG_ENVELOPE, req->envelope->data_tag, req->node_tag.comm);
  415. MPI_Isend(req->envelope, sizeof(struct _starpu_mpi_envelope), MPI_BYTE, req->node_tag.rank, _STARPU_MPI_TAG_ENVELOPE, req->node_tag.comm, &req->size_req);
  416. }
  417. else
  418. {
  419. int ret;
  420. // Do not pack the data, just try to find out the size
  421. starpu_data_pack(req->data_handle, NULL, &(req->envelope->size));
  422. if (req->envelope->size != -1)
  423. {
  424. // We already know the size of the data, let's send it to overlap with the packing of the data
  425. _STARPU_MPI_DEBUG(20, "Sending size %ld (%ld %s) to node %d (first call to pack)\n", req->envelope->size, sizeof(req->count), "MPI_BYTE", req->node_tag.rank);
  426. req->count = req->envelope->size;
  427. _STARPU_MPI_COMM_TO_DEBUG(req->envelope, sizeof(struct _starpu_mpi_envelope), MPI_BYTE, req->node_tag.rank, _STARPU_MPI_TAG_ENVELOPE, req->envelope->data_tag, req->node_tag.comm);
  428. ret = MPI_Isend(req->envelope, sizeof(struct _starpu_mpi_envelope), MPI_BYTE, req->node_tag.rank, _STARPU_MPI_TAG_ENVELOPE, req->node_tag.comm, &req->size_req);
  429. STARPU_MPI_ASSERT_MSG(ret == MPI_SUCCESS, "when sending size, MPI_Isend returning %s", _starpu_mpi_get_mpi_error_code(ret));
  430. }
  431. // Pack the data
  432. starpu_data_pack(req->data_handle, &req->ptr, &req->count);
  433. if (req->envelope->size == -1)
  434. {
  435. // We know the size now, let's send it
  436. _STARPU_MPI_DEBUG(20, "Sending size %ld (%ld %s) to node %d (second call to pack)\n", req->envelope->size, sizeof(req->count), "MPI_BYTE", req->node_tag.rank);
  437. _STARPU_MPI_COMM_TO_DEBUG(req->envelope, sizeof(struct _starpu_mpi_envelope), MPI_BYTE, req->node_tag.rank, _STARPU_MPI_TAG_ENVELOPE, req->envelope->data_tag, req->node_tag.comm);
  438. ret = MPI_Isend(req->envelope, sizeof(struct _starpu_mpi_envelope), MPI_BYTE, req->node_tag.rank, _STARPU_MPI_TAG_ENVELOPE, req->node_tag.comm, &req->size_req);
  439. STARPU_MPI_ASSERT_MSG(ret == MPI_SUCCESS, "when sending size, MPI_Isend returning %s", _starpu_mpi_get_mpi_error_code(ret));
  440. }
  441. else
  442. {
  443. // We check the size returned with the 2 calls to pack is the same
  444. STARPU_MPI_ASSERT_MSG(req->count == req->envelope->size, "Calls to pack_data returned different sizes %ld != %ld", req->count, req->envelope->size);
  445. }
  446. // We can send the data now
  447. }
  448. if (req->sync)
  449. {
  450. // If the data is to be sent in synchronous mode, we need to wait for the receiver ready message
  451. _starpu_mpi_sync_data_add(req);
  452. }
  453. else
  454. {
  455. // Otherwise we can send the data
  456. _starpu_mpi_isend_data_func(req);
  457. }
  458. }
  459. static struct _starpu_mpi_req *_starpu_mpi_isend_common(starpu_data_handle_t data_handle,
  460. int dest, int data_tag, MPI_Comm comm,
  461. unsigned detached, unsigned sync, void (*callback)(void *), void *arg,
  462. int sequential_consistency)
  463. {
  464. return _starpu_mpi_isend_irecv_common(data_handle, dest, data_tag, comm, detached, sync, callback, arg, SEND_REQ, _starpu_mpi_isend_size_func,
  465. #ifdef STARPU_MPI_PEDANTIC_ISEND
  466. STARPU_RW,
  467. #else
  468. STARPU_R,
  469. #endif
  470. sequential_consistency, 0, 0);
  471. }
  472. int starpu_mpi_isend(starpu_data_handle_t data_handle, starpu_mpi_req *public_req, int dest, int data_tag, MPI_Comm comm)
  473. {
  474. _STARPU_MPI_LOG_IN();
  475. STARPU_MPI_ASSERT_MSG(public_req, "starpu_mpi_isend needs a valid starpu_mpi_req");
  476. struct _starpu_mpi_req *req;
  477. _STARPU_MPI_TRACE_ISEND_COMPLETE_BEGIN(dest, data_tag, 0);
  478. req = _starpu_mpi_isend_common(data_handle, dest, data_tag, comm, 0, 0, NULL, NULL, 1);
  479. _STARPU_MPI_TRACE_ISEND_COMPLETE_END(dest, data_tag, 0);
  480. STARPU_MPI_ASSERT_MSG(req, "Invalid return for _starpu_mpi_isend_common");
  481. *public_req = req;
  482. _STARPU_MPI_LOG_OUT();
  483. return 0;
  484. }
  485. int starpu_mpi_isend_detached(starpu_data_handle_t data_handle,
  486. int dest, int data_tag, MPI_Comm comm, void (*callback)(void *), void *arg)
  487. {
  488. _STARPU_MPI_LOG_IN();
  489. _starpu_mpi_isend_common(data_handle, dest, data_tag, comm, 1, 0, callback, arg, 1);
  490. _STARPU_MPI_LOG_OUT();
  491. return 0;
  492. }
  493. int starpu_mpi_send(starpu_data_handle_t data_handle, int dest, int data_tag, MPI_Comm comm)
  494. {
  495. starpu_mpi_req req;
  496. MPI_Status status;
  497. _STARPU_MPI_LOG_IN();
  498. memset(&status, 0, sizeof(MPI_Status));
  499. starpu_mpi_isend(data_handle, &req, dest, data_tag, comm);
  500. starpu_mpi_wait(&req, &status);
  501. _STARPU_MPI_LOG_OUT();
  502. return 0;
  503. }
  504. int starpu_mpi_issend(starpu_data_handle_t data_handle, starpu_mpi_req *public_req, int dest, int data_tag, MPI_Comm comm)
  505. {
  506. _STARPU_MPI_LOG_IN();
  507. STARPU_MPI_ASSERT_MSG(public_req, "starpu_mpi_issend needs a valid starpu_mpi_req");
  508. struct _starpu_mpi_req *req;
  509. req = _starpu_mpi_isend_common(data_handle, dest, data_tag, comm, 0, 1, NULL, NULL, 1);
  510. STARPU_MPI_ASSERT_MSG(req, "Invalid return for _starpu_mpi_isend_common");
  511. *public_req = req;
  512. _STARPU_MPI_LOG_OUT();
  513. return 0;
  514. }
  515. int starpu_mpi_issend_detached(starpu_data_handle_t data_handle, int dest, int data_tag, MPI_Comm comm, void (*callback)(void *), void *arg)
  516. {
  517. _STARPU_MPI_LOG_IN();
  518. _starpu_mpi_isend_common(data_handle, dest, data_tag, comm, 1, 1, callback, arg, 1);
  519. _STARPU_MPI_LOG_OUT();
  520. return 0;
  521. }
  522. /********************************************************/
  523. /* */
  524. /* receive functionalities */
  525. /* */
  526. /********************************************************/
  527. static void _starpu_mpi_irecv_data_func(struct _starpu_mpi_req *req)
  528. {
  529. _STARPU_MPI_LOG_IN();
  530. _STARPU_MPI_DEBUG(20, "post MPI irecv request %p type %s tag %d src %d data %p ptr %p datatype '%s' count %d registered_datatype %d \n", req, _starpu_mpi_request_type(req->request_type), req->node_tag.data_tag, req->node_tag.rank, req->data_handle, req->ptr, req->datatype_name, (int)req->count, req->registered_datatype);
  531. _STARPU_MPI_TRACE_IRECV_SUBMIT_BEGIN(req->node_tag.rank, req->node_tag.data_tag);
  532. if (req->sync)
  533. {
  534. struct _starpu_mpi_envelope *_envelope;
  535. _STARPU_MPI_CALLOC(_envelope, 1, sizeof(struct _starpu_mpi_envelope));
  536. _envelope->mode = _STARPU_MPI_ENVELOPE_SYNC_READY;
  537. _envelope->data_tag = req->node_tag.data_tag;
  538. _STARPU_MPI_DEBUG(20, "Telling node %d it can send the data and waiting for the data back ...\n", req->node_tag.rank);
  539. _STARPU_MPI_COMM_TO_DEBUG(_envelope, sizeof(struct _starpu_mpi_envelope), MPI_BYTE, req->node_tag.rank, _STARPU_MPI_TAG_ENVELOPE, _envelope->data_tag, req->node_tag.comm);
  540. req->ret = MPI_Send(_envelope, sizeof(struct _starpu_mpi_envelope), MPI_BYTE, req->node_tag.rank, _STARPU_MPI_TAG_ENVELOPE, req->node_tag.comm);
  541. STARPU_MPI_ASSERT_MSG(req->ret == MPI_SUCCESS, "MPI_Send returning %s", _starpu_mpi_get_mpi_error_code(req->ret));
  542. free(_envelope);
  543. _envelope = NULL;
  544. }
  545. if (req->sync)
  546. {
  547. _STARPU_MPI_COMM_FROM_DEBUG(req, req->count, req->datatype, req->node_tag.rank, _STARPU_MPI_TAG_SYNC_DATA, req->node_tag.data_tag, req->node_tag.comm);
  548. req->ret = MPI_Irecv(req->ptr, req->count, req->datatype, req->node_tag.rank, _STARPU_MPI_TAG_SYNC_DATA, req->node_tag.comm, &req->data_request);
  549. }
  550. else
  551. {
  552. _STARPU_MPI_COMM_FROM_DEBUG(req, req->count, req->datatype, req->node_tag.rank, _STARPU_MPI_TAG_DATA, req->node_tag.data_tag, req->node_tag.comm);
  553. req->ret = MPI_Irecv(req->ptr, req->count, req->datatype, req->node_tag.rank, _STARPU_MPI_TAG_DATA, req->node_tag.comm, &req->data_request);
  554. #ifdef STARPU_SIMGRID
  555. _starpu_mpi_simgrid_wait_req(&req->data_request, &req->status_store, &req->queue, &req->done);
  556. #endif
  557. }
  558. STARPU_MPI_ASSERT_MSG(req->ret == MPI_SUCCESS, "MPI_IRecv returning %s", _starpu_mpi_get_mpi_error_code(req->ret));
  559. _STARPU_MPI_TRACE_IRECV_SUBMIT_END(req->node_tag.rank, req->node_tag.data_tag);
  560. /* somebody is perhaps waiting for the MPI request to be posted */
  561. STARPU_PTHREAD_MUTEX_LOCK(&req->req_mutex);
  562. req->submitted = 1;
  563. STARPU_PTHREAD_COND_BROADCAST(&req->req_cond);
  564. STARPU_PTHREAD_MUTEX_UNLOCK(&req->req_mutex);
  565. _starpu_mpi_handle_detached_request(req);
  566. _STARPU_MPI_LOG_OUT();
  567. }
  568. static struct _starpu_mpi_req *_starpu_mpi_irecv_common(starpu_data_handle_t data_handle, int source, int data_tag, MPI_Comm comm, unsigned detached, unsigned sync, void (*callback)(void *), void *arg, int sequential_consistency, int is_internal_req, starpu_ssize_t count)
  569. {
  570. return _starpu_mpi_isend_irecv_common(data_handle, source, data_tag, comm, detached, sync, callback, arg, RECV_REQ, _starpu_mpi_irecv_data_func, STARPU_W, sequential_consistency, is_internal_req, count);
  571. }
  572. int starpu_mpi_irecv(starpu_data_handle_t data_handle, starpu_mpi_req *public_req, int source, int data_tag, MPI_Comm comm)
  573. {
  574. _STARPU_MPI_LOG_IN();
  575. STARPU_MPI_ASSERT_MSG(public_req, "starpu_mpi_irecv needs a valid starpu_mpi_req");
  576. // // We check if a tag is defined for the data handle, if not,
  577. // // we define the one given for the communication.
  578. // // A tag is necessary for the internal mpi engine.
  579. // int tag = starpu_data_get_tag(data_handle);
  580. // if (tag == -1)
  581. // starpu_data_set_tag(data_handle, data_tag);
  582. struct _starpu_mpi_req *req;
  583. _STARPU_MPI_TRACE_IRECV_COMPLETE_BEGIN(source, data_tag);
  584. req = _starpu_mpi_irecv_common(data_handle, source, data_tag, comm, 0, 0, NULL, NULL, 1, 0, 0);
  585. _STARPU_MPI_TRACE_IRECV_COMPLETE_END(source, data_tag);
  586. STARPU_MPI_ASSERT_MSG(req, "Invalid return for _starpu_mpi_irecv_common");
  587. *public_req = req;
  588. _STARPU_MPI_LOG_OUT();
  589. return 0;
  590. }
  591. int starpu_mpi_irecv_detached(starpu_data_handle_t data_handle, int source, int data_tag, MPI_Comm comm, void (*callback)(void *), void *arg)
  592. {
  593. _STARPU_MPI_LOG_IN();
  594. // // We check if a tag is defined for the data handle, if not,
  595. // // we define the one given for the communication.
  596. // // A tag is necessary for the internal mpi engine.
  597. // int tag = starpu_data_get_tag(data_handle);
  598. // if (tag == -1)
  599. // starpu_data_set_tag(data_handle, data_tag);
  600. _starpu_mpi_irecv_common(data_handle, source, data_tag, comm, 1, 0, callback, arg, 1, 0, 0);
  601. _STARPU_MPI_LOG_OUT();
  602. return 0;
  603. }
  604. int starpu_mpi_irecv_detached_sequential_consistency(starpu_data_handle_t data_handle, int source, int data_tag, MPI_Comm comm, void (*callback)(void *), void *arg, int sequential_consistency)
  605. {
  606. _STARPU_MPI_LOG_IN();
  607. // // We check if a tag is defined for the data handle, if not,
  608. // // we define the one given for the communication.
  609. // // A tag is necessary for the internal mpi engine.
  610. // int tag = starpu_data_get_tag(data_handle);
  611. // if (tag == -1)
  612. // starpu_data_set_tag(data_handle, data_tag);
  613. _starpu_mpi_irecv_common(data_handle, source, data_tag, comm, 1, 0, callback, arg, sequential_consistency, 0, 0);
  614. _STARPU_MPI_LOG_OUT();
  615. return 0;
  616. }
  617. int starpu_mpi_recv(starpu_data_handle_t data_handle, int source, int data_tag, MPI_Comm comm, MPI_Status *status)
  618. {
  619. starpu_mpi_req req;
  620. _STARPU_MPI_LOG_IN();
  621. // // We check if a tag is defined for the data handle, if not,
  622. // // we define the one given for the communication.
  623. // // A tag is necessary for the internal mpi engine.
  624. // int tag = starpu_data_get_tag(data_handle);
  625. // if (tag == -1)
  626. // starpu_data_set_tag(data_handle, data_tag);
  627. starpu_mpi_irecv(data_handle, &req, source, data_tag, comm);
  628. starpu_mpi_wait(&req, status);
  629. _STARPU_MPI_LOG_OUT();
  630. return 0;
  631. }
  632. /********************************************************/
  633. /* */
  634. /* Wait functionalities */
  635. /* */
  636. /********************************************************/
  637. static void _starpu_mpi_wait_func(struct _starpu_mpi_req *waiting_req)
  638. {
  639. _STARPU_MPI_LOG_IN();
  640. /* Which is the mpi request we are waiting for ? */
  641. struct _starpu_mpi_req *req = waiting_req->other_request;
  642. _STARPU_MPI_TRACE_UWAIT_BEGIN(req->node_tag.rank, req->node_tag.data_tag);
  643. if (req->data_request != MPI_REQUEST_NULL)
  644. {
  645. // TODO: Fix for STARPU_SIMGRID
  646. #ifdef STARPU_SIMGRID
  647. STARPU_MPI_ASSERT_MSG(0, "Implement this in STARPU_SIMGRID");
  648. #endif
  649. req->ret = MPI_Wait(&req->data_request, waiting_req->status);
  650. STARPU_MPI_ASSERT_MSG(req->ret == MPI_SUCCESS, "MPI_Wait returning %s", _starpu_mpi_get_mpi_error_code(req->ret));
  651. }
  652. _STARPU_MPI_TRACE_UWAIT_END(req->node_tag.rank, req->node_tag.data_tag);
  653. _starpu_mpi_handle_request_termination(req);
  654. _STARPU_MPI_LOG_OUT();
  655. }
  656. int starpu_mpi_wait(starpu_mpi_req *public_req, MPI_Status *status)
  657. {
  658. int ret;
  659. struct _starpu_mpi_req *req = *public_req;
  660. struct _starpu_mpi_req *waiting_req;
  661. _STARPU_MPI_LOG_IN();
  662. _STARPU_MPI_INC_POSTED_REQUESTS(1);
  663. /* We cannot try to complete a MPI request that was not actually posted
  664. * to MPI yet. */
  665. STARPU_PTHREAD_MUTEX_LOCK(&(req->req_mutex));
  666. while (!(req->submitted))
  667. STARPU_PTHREAD_COND_WAIT(&(req->req_cond), &(req->req_mutex));
  668. STARPU_PTHREAD_MUTEX_UNLOCK(&(req->req_mutex));
  669. /* Initialize the request structure */
  670. _starpu_mpi_request_init(&waiting_req);
  671. waiting_req->status = status;
  672. waiting_req->other_request = req;
  673. waiting_req->func = _starpu_mpi_wait_func;
  674. waiting_req->request_type = WAIT_REQ;
  675. _starpu_mpi_submit_ready_request(waiting_req);
  676. /* We wait for the MPI request to finish */
  677. STARPU_PTHREAD_MUTEX_LOCK(&req->req_mutex);
  678. while (!req->completed)
  679. STARPU_PTHREAD_COND_WAIT(&req->req_cond, &req->req_mutex);
  680. STARPU_PTHREAD_MUTEX_UNLOCK(&req->req_mutex);
  681. ret = req->ret;
  682. /* The internal request structure was automatically allocated */
  683. *public_req = NULL;
  684. if (req->internal_req)
  685. {
  686. _starpu_mpi_request_destroy(req->internal_req);
  687. }
  688. _starpu_mpi_request_destroy(req);
  689. _starpu_mpi_request_destroy(waiting_req);
  690. _STARPU_MPI_LOG_OUT();
  691. return ret;
  692. }
  693. /********************************************************/
  694. /* */
  695. /* Test functionalities */
  696. /* */
  697. /********************************************************/
  698. static void _starpu_mpi_test_func(struct _starpu_mpi_req *testing_req)
  699. {
  700. _STARPU_MPI_LOG_IN();
  701. /* Which is the mpi request we are testing for ? */
  702. struct _starpu_mpi_req *req = testing_req->other_request;
  703. _STARPU_MPI_DEBUG(2, "Test request %p type %s tag %d src %d data %p ptr %p datatype '%s' count %d registered_datatype %d \n",
  704. req, _starpu_mpi_request_type(req->request_type), req->node_tag.data_tag, req->node_tag.rank, req->data_handle, req->ptr,
  705. req->datatype_name, (int)req->count, req->registered_datatype);
  706. _STARPU_MPI_TRACE_UTESTING_BEGIN(req->node_tag.rank, req->node_tag.data_tag);
  707. #ifdef STARPU_SIMGRID
  708. req->ret = _starpu_mpi_simgrid_mpi_test(&req->done, testing_req->flag);
  709. memcpy(testing_req->status, &req->status_store, sizeof(*testing_req->status));
  710. #else
  711. req->ret = MPI_Test(&req->data_request, testing_req->flag, testing_req->status);
  712. #endif
  713. STARPU_MPI_ASSERT_MSG(req->ret == MPI_SUCCESS, "MPI_Test returning %s", _starpu_mpi_get_mpi_error_code(req->ret));
  714. _STARPU_MPI_TRACE_UTESTING_END(req->node_tag.rank, req->node_tag.data_tag);
  715. if (*testing_req->flag)
  716. {
  717. testing_req->ret = req->ret;
  718. _starpu_mpi_handle_request_termination(req);
  719. }
  720. STARPU_PTHREAD_MUTEX_LOCK(&testing_req->req_mutex);
  721. testing_req->completed = 1;
  722. STARPU_PTHREAD_COND_SIGNAL(&testing_req->req_cond);
  723. STARPU_PTHREAD_MUTEX_UNLOCK(&testing_req->req_mutex);
  724. _STARPU_MPI_LOG_OUT();
  725. }
  726. int starpu_mpi_test(starpu_mpi_req *public_req, int *flag, MPI_Status *status)
  727. {
  728. _STARPU_MPI_LOG_IN();
  729. int ret = 0;
  730. STARPU_MPI_ASSERT_MSG(public_req, "starpu_mpi_test needs a valid starpu_mpi_req");
  731. struct _starpu_mpi_req *req = *public_req;
  732. STARPU_MPI_ASSERT_MSG(!req->detached, "MPI_Test cannot be called on a detached request");
  733. STARPU_PTHREAD_MUTEX_LOCK(&req->req_mutex);
  734. unsigned submitted = req->submitted;
  735. STARPU_PTHREAD_MUTEX_UNLOCK(&req->req_mutex);
  736. if (submitted)
  737. {
  738. struct _starpu_mpi_req *testing_req;
  739. /* Initialize the request structure */
  740. _starpu_mpi_request_init(&testing_req);
  741. testing_req->flag = flag;
  742. testing_req->status = status;
  743. testing_req->other_request = req;
  744. testing_req->func = _starpu_mpi_test_func;
  745. testing_req->completed = 0;
  746. testing_req->request_type = TEST_REQ;
  747. _STARPU_MPI_INC_POSTED_REQUESTS(1);
  748. _starpu_mpi_submit_ready_request(testing_req);
  749. /* We wait for the test request to finish */
  750. STARPU_PTHREAD_MUTEX_LOCK(&(testing_req->req_mutex));
  751. while (!(testing_req->completed))
  752. STARPU_PTHREAD_COND_WAIT(&(testing_req->req_cond), &(testing_req->req_mutex));
  753. STARPU_PTHREAD_MUTEX_UNLOCK(&(testing_req->req_mutex));
  754. ret = testing_req->ret;
  755. if (*(testing_req->flag))
  756. {
  757. /* The request was completed so we free the internal
  758. * request structure which was automatically allocated
  759. * */
  760. *public_req = NULL;
  761. if (req->internal_req)
  762. {
  763. _starpu_mpi_request_destroy(req->internal_req);
  764. }
  765. _starpu_mpi_request_destroy(req);
  766. }
  767. _starpu_mpi_request_destroy(testing_req);
  768. }
  769. else
  770. {
  771. *flag = 0;
  772. }
  773. _STARPU_MPI_LOG_OUT();
  774. return ret;
  775. }
  776. /********************************************************/
  777. /* */
  778. /* Barrier functionalities */
  779. /* */
  780. /********************************************************/
  781. static void _starpu_mpi_barrier_func(struct _starpu_mpi_req *barrier_req)
  782. {
  783. _STARPU_MPI_LOG_IN();
  784. barrier_req->ret = MPI_Barrier(barrier_req->node_tag.comm);
  785. STARPU_MPI_ASSERT_MSG(barrier_req->ret == MPI_SUCCESS, "MPI_Barrier returning %s", _starpu_mpi_get_mpi_error_code(barrier_req->ret));
  786. _starpu_mpi_handle_request_termination(barrier_req);
  787. _STARPU_MPI_LOG_OUT();
  788. }
  789. int _starpu_mpi_barrier(MPI_Comm comm)
  790. {
  791. _STARPU_MPI_LOG_IN();
  792. int ret = posted_requests;
  793. struct _starpu_mpi_req *barrier_req;
  794. /* First wait for *both* all tasks and MPI requests to finish, in case
  795. * some tasks generate MPI requests, MPI requests generate tasks, etc.
  796. */
  797. STARPU_PTHREAD_MUTEX_LOCK(&progress_mutex);
  798. STARPU_MPI_ASSERT_MSG(!barrier_running, "Concurrent starpu_mpi_barrier is not implemented, even on different communicators");
  799. barrier_running = 1;
  800. do
  801. {
  802. while (posted_requests)
  803. /* Wait for all current MPI requests to finish */
  804. STARPU_PTHREAD_COND_WAIT(&barrier_cond, &progress_mutex);
  805. /* No current request, clear flag */
  806. newer_requests = 0;
  807. STARPU_PTHREAD_MUTEX_UNLOCK(&progress_mutex);
  808. /* Now wait for all tasks */
  809. starpu_task_wait_for_all();
  810. STARPU_PTHREAD_MUTEX_LOCK(&progress_mutex);
  811. /* Check newer_requests again, in case some MPI requests
  812. * triggered by tasks completed and triggered tasks between
  813. * wait_for_all finished and we take the lock */
  814. } while (posted_requests || newer_requests);
  815. barrier_running = 0;
  816. STARPU_PTHREAD_MUTEX_UNLOCK(&progress_mutex);
  817. /* Initialize the request structure */
  818. _starpu_mpi_request_init(&barrier_req);
  819. barrier_req->func = _starpu_mpi_barrier_func;
  820. barrier_req->request_type = BARRIER_REQ;
  821. barrier_req->node_tag.comm = comm;
  822. _STARPU_MPI_INC_POSTED_REQUESTS(1);
  823. _starpu_mpi_submit_ready_request(barrier_req);
  824. /* We wait for the MPI request to finish */
  825. STARPU_PTHREAD_MUTEX_LOCK(&barrier_req->req_mutex);
  826. while (!barrier_req->completed)
  827. STARPU_PTHREAD_COND_WAIT(&barrier_req->req_cond, &barrier_req->req_mutex);
  828. STARPU_PTHREAD_MUTEX_UNLOCK(&barrier_req->req_mutex);
  829. _starpu_mpi_request_destroy(barrier_req);
  830. _STARPU_MPI_LOG_OUT();
  831. return ret;
  832. }
  833. int starpu_mpi_barrier(MPI_Comm comm)
  834. {
  835. _starpu_mpi_barrier(comm);
  836. return 0;
  837. }
  838. /********************************************************/
  839. /* */
  840. /* Progression */
  841. /* */
  842. /********************************************************/
  843. #ifdef STARPU_VERBOSE
  844. static char *_starpu_mpi_request_type(enum _starpu_mpi_request_type request_type)
  845. {
  846. switch (request_type)
  847. {
  848. case SEND_REQ: return "SEND_REQ";
  849. case RECV_REQ: return "RECV_REQ";
  850. case WAIT_REQ: return "WAIT_REQ";
  851. case TEST_REQ: return "TEST_REQ";
  852. case BARRIER_REQ: return "BARRIER_REQ";
  853. case UNKNOWN_REQ: return "UNSET_REQ";
  854. default: return "unknown request type";
  855. }
  856. }
  857. #endif
  858. static void _starpu_mpi_handle_request_termination(struct _starpu_mpi_req *req)
  859. {
  860. _STARPU_MPI_LOG_IN();
  861. _STARPU_MPI_DEBUG(2, "complete MPI request %p type %s tag %d src %d data %p ptr %p datatype '%s' count %d registered_datatype %d internal_req %p\n",
  862. req, _starpu_mpi_request_type(req->request_type), req->node_tag.data_tag, req->node_tag.rank, req->data_handle, req->ptr,
  863. req->datatype_name, (int)req->count, req->registered_datatype, req->internal_req);
  864. if (req->internal_req)
  865. {
  866. free(req->early_data_handle);
  867. req->early_data_handle = NULL;
  868. }
  869. else
  870. {
  871. if (req->request_type == RECV_REQ || req->request_type == SEND_REQ)
  872. {
  873. if (req->registered_datatype == 0)
  874. {
  875. if (req->request_type == SEND_REQ)
  876. {
  877. // We need to make sure the communication for sending the size
  878. // has completed, as MPI can re-order messages, let's call
  879. // MPI_Wait to make sure data have been sent
  880. int ret;
  881. ret = MPI_Wait(&req->size_req, MPI_STATUS_IGNORE);
  882. STARPU_MPI_ASSERT_MSG(ret == MPI_SUCCESS, "MPI_Wait returning %s", _starpu_mpi_get_mpi_error_code(ret));
  883. free(req->ptr);
  884. req->ptr = NULL;
  885. }
  886. else if (req->request_type == RECV_REQ)
  887. {
  888. // req->ptr is freed by starpu_data_unpack
  889. starpu_data_unpack(req->data_handle, req->ptr, req->count);
  890. starpu_memory_deallocate(STARPU_MAIN_RAM, req->count);
  891. }
  892. }
  893. else
  894. {
  895. _starpu_mpi_datatype_free(req->data_handle, &req->datatype);
  896. }
  897. }
  898. _STARPU_MPI_TRACE_TERMINATED(req, req->node_tag.rank, req->node_tag.data_tag);
  899. }
  900. if (req->data_handle)
  901. starpu_data_release(req->data_handle);
  902. if (req->envelope)
  903. {
  904. free(req->envelope);
  905. req->envelope = NULL;
  906. }
  907. /* Execute the specified callback, if any */
  908. if (req->callback)
  909. req->callback(req->callback_arg);
  910. /* tell anyone potentially waiting on the request that it is
  911. * terminated now */
  912. STARPU_PTHREAD_MUTEX_LOCK(&req->req_mutex);
  913. req->completed = 1;
  914. STARPU_PTHREAD_COND_BROADCAST(&req->req_cond);
  915. STARPU_PTHREAD_MUTEX_UNLOCK(&req->req_mutex);
  916. _STARPU_MPI_LOG_OUT();
  917. }
  918. static void _starpu_mpi_early_data_cb(void* arg)
  919. {
  920. struct _starpu_mpi_early_data_cb_args *args = arg;
  921. if (args->buffer)
  922. {
  923. /* Data has been received as a raw memory, it has to be unpacked */
  924. struct starpu_data_interface_ops *itf_src = starpu_data_get_interface_ops(args->early_handle);
  925. struct starpu_data_interface_ops *itf_dst = starpu_data_get_interface_ops(args->data_handle);
  926. STARPU_MPI_ASSERT_MSG(itf_dst->unpack_data, "The data interface does not define an unpack function\n");
  927. itf_dst->unpack_data(args->data_handle, STARPU_MAIN_RAM, args->buffer, itf_src->get_size(args->early_handle));
  928. free(args->buffer);
  929. args->buffer = NULL;
  930. }
  931. else
  932. {
  933. struct starpu_data_interface_ops *itf = starpu_data_get_interface_ops(args->early_handle);
  934. void* itf_src = starpu_data_get_interface_on_node(args->early_handle, STARPU_MAIN_RAM);
  935. void* itf_dst = starpu_data_get_interface_on_node(args->data_handle, STARPU_MAIN_RAM);
  936. if (!itf->copy_methods->ram_to_ram)
  937. {
  938. _STARPU_MPI_DEBUG(3, "Initiating any_to_any copy..\n");
  939. itf->copy_methods->any_to_any(itf_src, STARPU_MAIN_RAM, itf_dst, STARPU_MAIN_RAM, NULL);
  940. }
  941. else
  942. {
  943. _STARPU_MPI_DEBUG(3, "Initiating ram_to_ram copy..\n");
  944. itf->copy_methods->ram_to_ram(itf_src, STARPU_MAIN_RAM, itf_dst, STARPU_MAIN_RAM);
  945. }
  946. }
  947. _STARPU_MPI_DEBUG(3, "Done, handling release of early_handle..\n");
  948. starpu_data_release(args->early_handle);
  949. _STARPU_MPI_DEBUG(3, "Done, handling unregister of early_handle..\n");
  950. starpu_data_unregister_submit(args->early_handle);
  951. _STARPU_MPI_DEBUG(3, "Done, handling request %p termination of the already received request\n",args->req);
  952. // If the request is detached, we need to call _starpu_mpi_handle_request_termination
  953. // as it will not be called automatically as the request is not in the list detached_requests
  954. if (args->req)
  955. {
  956. if (args->req->detached)
  957. {
  958. _starpu_mpi_handle_request_termination(args->req);
  959. _starpu_mpi_request_destroy(args->req);
  960. }
  961. else
  962. {
  963. // else: If the request is not detached its termination will
  964. // be handled when calling starpu_mpi_wait
  965. // We store in the application request the internal MPI
  966. // request so that it can be used by starpu_mpi_wait
  967. args->req->data_request = args->req->internal_req->data_request;
  968. STARPU_PTHREAD_MUTEX_LOCK(&args->req->req_mutex);
  969. args->req->submitted = 1;
  970. STARPU_PTHREAD_COND_BROADCAST(&args->req->req_cond);
  971. STARPU_PTHREAD_MUTEX_UNLOCK(&args->req->req_mutex);
  972. }
  973. }
  974. free(args);
  975. args = NULL;
  976. }
  977. static void _starpu_mpi_test_detached_requests(void)
  978. {
  979. //_STARPU_MPI_LOG_IN();
  980. int flag;
  981. struct _starpu_mpi_req *req;
  982. STARPU_PTHREAD_MUTEX_LOCK(&detached_requests_mutex);
  983. req = _starpu_mpi_req_list_begin(detached_requests);
  984. while (req != _starpu_mpi_req_list_end(detached_requests))
  985. {
  986. STARPU_PTHREAD_MUTEX_UNLOCK(&detached_requests_mutex);
  987. STARPU_MPI_ASSERT_MSG(req->data_request != MPI_REQUEST_NULL, "Cannot test completion of the request MPI_REQUEST_NULL");
  988. //_STARPU_MPI_DEBUG(3, "Test detached request %p - mpitag %d - TYPE %s %d\n", &req->data_request, req->node_tag.data_tag, _starpu_mpi_request_type(req->request_type), req->node_tag.rank);
  989. #ifdef STARPU_SIMGRID
  990. req->ret = _starpu_mpi_simgrid_mpi_test(&req->done, &flag);
  991. #else
  992. req->ret = MPI_Test(&req->data_request, &flag, MPI_STATUS_IGNORE);
  993. #endif
  994. STARPU_MPI_ASSERT_MSG(req->ret == MPI_SUCCESS, "MPI_Test returning %s", _starpu_mpi_get_mpi_error_code(req->ret));
  995. if (!flag)
  996. {
  997. req = _starpu_mpi_req_list_next(req);
  998. }
  999. else
  1000. {
  1001. struct _starpu_mpi_req *next_req;
  1002. next_req = _starpu_mpi_req_list_next(req);
  1003. _STARPU_MPI_TRACE_COMPLETE_BEGIN(req->request_type, req->node_tag.rank, req->node_tag.data_tag);
  1004. STARPU_PTHREAD_MUTEX_LOCK(&detached_requests_mutex);
  1005. _starpu_mpi_req_list_erase(detached_requests, req);
  1006. STARPU_PTHREAD_MUTEX_UNLOCK(&detached_requests_mutex);
  1007. _starpu_mpi_handle_request_termination(req);
  1008. _STARPU_MPI_TRACE_COMPLETE_END(req->request_type, req->node_tag.rank, req->node_tag.data_tag);
  1009. if (req->is_internal_req == 0)
  1010. {
  1011. _starpu_mpi_request_destroy(req);
  1012. }
  1013. req = next_req;
  1014. }
  1015. STARPU_PTHREAD_MUTEX_LOCK(&detached_requests_mutex);
  1016. }
  1017. STARPU_PTHREAD_MUTEX_UNLOCK(&detached_requests_mutex);
  1018. //_STARPU_MPI_LOG_OUT();
  1019. }
  1020. static void _starpu_mpi_handle_detached_request(struct _starpu_mpi_req *req)
  1021. {
  1022. if (req->detached)
  1023. {
  1024. /* put the submitted request into the list of pending requests
  1025. * so that it can be handled by the progression mechanisms */
  1026. STARPU_PTHREAD_MUTEX_LOCK(&detached_requests_mutex);
  1027. _starpu_mpi_req_list_push_front(detached_requests, req);
  1028. STARPU_PTHREAD_MUTEX_UNLOCK(&detached_requests_mutex);
  1029. starpu_wake_all_blocked_workers();
  1030. STARPU_PTHREAD_MUTEX_LOCK(&progress_mutex);
  1031. STARPU_PTHREAD_COND_SIGNAL(&progress_cond);
  1032. STARPU_PTHREAD_MUTEX_UNLOCK(&progress_mutex);
  1033. }
  1034. }
  1035. static void _starpu_mpi_handle_ready_request(struct _starpu_mpi_req *req)
  1036. {
  1037. _STARPU_MPI_LOG_IN();
  1038. STARPU_MPI_ASSERT_MSG(req, "Invalid request");
  1039. /* submit the request to MPI */
  1040. _STARPU_MPI_DEBUG(2, "Handling new request %p type %s tag %d src %d data %p ptr %p datatype '%s' count %d registered_datatype %d \n",
  1041. req, _starpu_mpi_request_type(req->request_type), req->node_tag.data_tag, req->node_tag.rank, req->data_handle,
  1042. req->ptr, req->datatype_name, (int)req->count, req->registered_datatype);
  1043. req->func(req);
  1044. _STARPU_MPI_LOG_OUT();
  1045. }
  1046. static void _starpu_mpi_receive_early_data(struct _starpu_mpi_envelope *envelope, MPI_Status status, MPI_Comm comm)
  1047. {
  1048. _STARPU_MPI_DEBUG(20, "Request with tag %d and source %d not found, creating a early_data_handle to receive incoming data..\n", envelope->data_tag, status.MPI_SOURCE);
  1049. _STARPU_MPI_DEBUG(20, "Request sync %d\n", envelope->sync);
  1050. struct _starpu_mpi_early_data_handle* early_data_handle = _starpu_mpi_early_data_create(envelope, status.MPI_SOURCE, comm);
  1051. _starpu_mpi_early_data_add(early_data_handle);
  1052. starpu_data_handle_t data_handle = NULL;
  1053. STARPU_PTHREAD_MUTEX_UNLOCK(&progress_mutex);
  1054. data_handle = _starpu_mpi_tag_get_data_handle_from_tag(envelope->data_tag);
  1055. STARPU_PTHREAD_MUTEX_LOCK(&progress_mutex);
  1056. if (data_handle && starpu_data_get_interface_id(data_handle) < STARPU_MAX_INTERFACE_ID)
  1057. {
  1058. /* We know which data will receive it and we won't have to unpack, use just the same kind of data. */
  1059. early_data_handle->buffer = NULL;
  1060. starpu_data_register_same(&early_data_handle->handle, data_handle);
  1061. //_starpu_mpi_early_data_add(early_data_handle);
  1062. }
  1063. else
  1064. {
  1065. /* The application has not registered yet a data with the tag,
  1066. * we are going to receive the data as a raw memory, and give it
  1067. * to the application when it post a receive for this tag
  1068. */
  1069. _STARPU_MPI_DEBUG(3, "Posting a receive for a data of size %d which has not yet been registered\n", (int)early_data_handle->env->size);
  1070. _STARPU_MPI_MALLOC(early_data_handle->buffer, early_data_handle->env->size);
  1071. starpu_variable_data_register(&early_data_handle->handle, STARPU_MAIN_RAM, (uintptr_t) early_data_handle->buffer, early_data_handle->env->size);
  1072. //_starpu_mpi_early_data_add(early_data_handle);
  1073. }
  1074. _STARPU_MPI_DEBUG(20, "Posting internal detached irecv on early_data_handle with tag %d from comm %ld src %d ..\n",
  1075. early_data_handle->node_tag.data_tag, (long int)comm, status.MPI_SOURCE);
  1076. STARPU_PTHREAD_MUTEX_UNLOCK(&progress_mutex);
  1077. early_data_handle->req = _starpu_mpi_irecv_common(early_data_handle->handle, status.MPI_SOURCE,
  1078. early_data_handle->node_tag.data_tag, comm, 1, 0,
  1079. NULL, NULL, 1, 1, envelope->size);
  1080. STARPU_PTHREAD_MUTEX_LOCK(&progress_mutex);
  1081. // We wait until the request is pushed in the
  1082. // ready_request list
  1083. STARPU_PTHREAD_MUTEX_UNLOCK(&progress_mutex);
  1084. STARPU_PTHREAD_MUTEX_LOCK(&(early_data_handle->req->posted_mutex));
  1085. while (!(early_data_handle->req->posted))
  1086. STARPU_PTHREAD_COND_WAIT(&(early_data_handle->req->posted_cond), &(early_data_handle->req->posted_mutex));
  1087. STARPU_PTHREAD_MUTEX_UNLOCK(&(early_data_handle->req->posted_mutex));
  1088. #ifdef STARPU_DEVEL
  1089. #warning check if req_ready is still necessary
  1090. #endif
  1091. STARPU_PTHREAD_MUTEX_LOCK(&early_data_handle->req_mutex);
  1092. early_data_handle->req_ready = 1;
  1093. STARPU_PTHREAD_COND_BROADCAST(&early_data_handle->req_cond);
  1094. STARPU_PTHREAD_MUTEX_UNLOCK(&early_data_handle->req_mutex);
  1095. STARPU_PTHREAD_MUTEX_LOCK(&progress_mutex);
  1096. // Handle the request immediatly to make sure the mpi_irecv is
  1097. // posted before receiving an other envelope
  1098. _starpu_mpi_req_list_erase(ready_requests, early_data_handle->req);
  1099. STARPU_PTHREAD_MUTEX_UNLOCK(&progress_mutex);
  1100. _starpu_mpi_handle_ready_request(early_data_handle->req);
  1101. STARPU_PTHREAD_MUTEX_LOCK(&progress_mutex);
  1102. }
  1103. static void *_starpu_mpi_progress_thread_func(void *arg)
  1104. {
  1105. struct _starpu_mpi_argc_argv *argc_argv = (struct _starpu_mpi_argc_argv *) arg;
  1106. starpu_pthread_setname("MPI");
  1107. #ifndef STARPU_SIMGRID
  1108. _starpu_mpi_do_initialize(argc_argv);
  1109. #endif
  1110. _starpu_mpi_fake_world_size = starpu_get_env_number("STARPU_MPI_FAKE_SIZE");
  1111. _starpu_mpi_fake_world_rank = starpu_get_env_number("STARPU_MPI_FAKE_RANK");
  1112. #ifdef STARPU_SIMGRID
  1113. /* Now that MPI is set up, let the rest of simgrid get initialized */
  1114. char **argv_cpy;
  1115. _STARPU_MPI_MALLOC(argv_cpy, *(argc_argv->argc) * sizeof(char*));
  1116. int i;
  1117. for (i = 0; i < *(argc_argv->argc); i++)
  1118. argv_cpy[i] = strdup((*(argc_argv->argv))[i]);
  1119. MSG_process_create_with_arguments("main", smpi_simulated_main_, NULL, _starpu_simgrid_get_host_by_name("MAIN"), *(argc_argv->argc), argv_cpy);
  1120. /* And set TSD for us */
  1121. #ifdef HAVE_SMPI_PROCESS_SET_USER_DATA
  1122. void **tsd;
  1123. _STARPU_CALLOC(tsd, MAX_TSD + 1, sizeof(void*));
  1124. smpi_process_set_user_data(tsd);
  1125. #endif
  1126. #endif
  1127. #ifdef STARPU_USE_FXT
  1128. _starpu_fxt_wait_initialisation();
  1129. #endif //STARPU_USE_FXT
  1130. {
  1131. _STARPU_MPI_TRACE_START(argc_argv->rank, argc_argv->world_size);
  1132. #ifdef STARPU_USE_FXT
  1133. starpu_profiling_set_id(argc_argv->rank);
  1134. #endif //STARPU_USE_FXT
  1135. }
  1136. _starpu_mpi_add_sync_point_in_fxt();
  1137. _starpu_mpi_comm_amounts_init(argc_argv->comm);
  1138. _starpu_mpi_cache_init(argc_argv->comm);
  1139. _starpu_mpi_select_node_init();
  1140. _starpu_mpi_tag_init();
  1141. _starpu_mpi_comm_init(argc_argv->comm);
  1142. _starpu_mpi_early_request_init();
  1143. _starpu_mpi_early_data_init();
  1144. _starpu_mpi_sync_data_init();
  1145. _starpu_mpi_datatype_init();
  1146. /* notify the main thread that the progression thread is ready */
  1147. STARPU_PTHREAD_MUTEX_LOCK(&progress_mutex);
  1148. running = 1;
  1149. STARPU_PTHREAD_COND_SIGNAL(&progress_cond);
  1150. STARPU_PTHREAD_MUTEX_UNLOCK(&progress_mutex);
  1151. #ifdef STARPU_SIMGRID
  1152. starpu_pthread_wait_init(&wait);
  1153. starpu_pthread_queue_init(&dontsleep);
  1154. starpu_pthread_queue_register(&wait, &dontsleep);
  1155. #endif
  1156. STARPU_PTHREAD_MUTEX_LOCK(&progress_mutex);
  1157. int envelope_request_submitted = 0;
  1158. while (running || posted_requests || !(_starpu_mpi_req_list_empty(ready_requests)) || !(_starpu_mpi_req_list_empty(detached_requests)))// || !(_starpu_mpi_early_request_count()) || !(_starpu_mpi_sync_data_count()))
  1159. {
  1160. #ifdef STARPU_SIMGRID
  1161. starpu_pthread_wait_reset(&wait);
  1162. #endif
  1163. /* shall we block ? */
  1164. unsigned block = _starpu_mpi_req_list_empty(ready_requests) && _starpu_mpi_early_request_count() == 0 && _starpu_mpi_sync_data_count() == 0 && _starpu_mpi_req_list_empty(detached_requests);
  1165. if (block)
  1166. {
  1167. _STARPU_MPI_DEBUG(3, "NO MORE REQUESTS TO HANDLE\n");
  1168. _STARPU_MPI_TRACE_SLEEP_BEGIN();
  1169. if (barrier_running)
  1170. /* Tell mpi_barrier */
  1171. STARPU_PTHREAD_COND_SIGNAL(&barrier_cond);
  1172. STARPU_PTHREAD_COND_WAIT(&progress_cond, &progress_mutex);
  1173. _STARPU_MPI_TRACE_SLEEP_END();
  1174. }
  1175. /* get one request */
  1176. int n = 0;
  1177. while (!_starpu_mpi_req_list_empty(ready_requests))
  1178. {
  1179. struct _starpu_mpi_req *req;
  1180. if (n++ == NREADY_PROCESS)
  1181. /* Already spent some time on submitting ready requests, poll before processing more ready requests */
  1182. break;
  1183. req = _starpu_mpi_req_list_pop_back(ready_requests);
  1184. /* handling a request is likely to block for a while
  1185. * (on a sync_data_with_mem call), we want to let the
  1186. * application submit requests in the meantime, so we
  1187. * release the lock. */
  1188. STARPU_PTHREAD_MUTEX_UNLOCK(&progress_mutex);
  1189. _starpu_mpi_handle_ready_request(req);
  1190. STARPU_PTHREAD_MUTEX_LOCK(&progress_mutex);
  1191. }
  1192. /* If there is no currently submitted envelope_request submitted to
  1193. * catch envelopes from senders, and there is some pending
  1194. * receive requests on our side, we resubmit a header request. */
  1195. if (((_starpu_mpi_early_request_count() > 0) || (_starpu_mpi_sync_data_count() > 0)) && (envelope_request_submitted == 0))// && (HASH_COUNT(_starpu_mpi_early_data_handle_hashmap) == 0))
  1196. {
  1197. _starpu_mpi_comm_post_recv();
  1198. envelope_request_submitted = 1;
  1199. }
  1200. /* test whether there are some terminated "detached request" */
  1201. STARPU_PTHREAD_MUTEX_UNLOCK(&progress_mutex);
  1202. _starpu_mpi_test_detached_requests();
  1203. STARPU_PTHREAD_MUTEX_LOCK(&progress_mutex);
  1204. if (envelope_request_submitted == 1)
  1205. {
  1206. int flag;
  1207. struct _starpu_mpi_envelope *envelope;
  1208. MPI_Status envelope_status;
  1209. MPI_Comm envelope_comm;
  1210. /* test whether an envelope has arrived. */
  1211. flag = _starpu_mpi_comm_test_recv(&envelope_status, &envelope, &envelope_comm);
  1212. if (flag)
  1213. {
  1214. _STARPU_MPI_COMM_FROM_DEBUG(envelope, sizeof(struct _starpu_mpi_envelope), MPI_BYTE, envelope_status.MPI_SOURCE, _STARPU_MPI_TAG_ENVELOPE, envelope->data_tag, envelope_comm);
  1215. _STARPU_MPI_DEBUG(4, "Envelope received with mode %d\n", envelope->mode);
  1216. if (envelope->mode == _STARPU_MPI_ENVELOPE_SYNC_READY)
  1217. {
  1218. struct _starpu_mpi_req *_sync_req = _starpu_mpi_sync_data_find(envelope->data_tag, envelope_status.MPI_SOURCE, envelope_comm);
  1219. _STARPU_MPI_DEBUG(20, "Sending data with tag %d to node %d\n", _sync_req->node_tag.data_tag, envelope_status.MPI_SOURCE);
  1220. STARPU_MPI_ASSERT_MSG(envelope->data_tag == _sync_req->node_tag.data_tag, "Tag mismatch (envelope %d != req %d)\n", envelope->data_tag, _sync_req->node_tag.data_tag);
  1221. STARPU_PTHREAD_MUTEX_UNLOCK(&progress_mutex);
  1222. _starpu_mpi_isend_data_func(_sync_req);
  1223. STARPU_PTHREAD_MUTEX_LOCK(&progress_mutex);
  1224. }
  1225. else
  1226. {
  1227. _STARPU_MPI_DEBUG(3, "Searching for application request with tag %d and source %d (size %ld)\n", envelope->data_tag, envelope_status.MPI_SOURCE, envelope->size);
  1228. struct _starpu_mpi_req *early_request = _starpu_mpi_early_request_dequeue(envelope->data_tag, envelope_status.MPI_SOURCE, envelope_comm);
  1229. /* Case: a data will arrive before a matching receive is
  1230. * posted by the application. Create a temporary handle to
  1231. * store the incoming data, submit a starpu_mpi_irecv_detached
  1232. * on this handle, and store it as an early_data
  1233. */
  1234. if (early_request == NULL)
  1235. {
  1236. if (envelope->sync)
  1237. {
  1238. _STARPU_MPI_DEBUG(2000, "-------------------------> adding request for tag %d\n", envelope->data_tag);
  1239. struct _starpu_mpi_req *new_req;
  1240. #ifdef STARPU_DEVEL
  1241. #warning creating a request is not really useful.
  1242. #endif
  1243. /* Initialize the request structure */
  1244. _starpu_mpi_request_init(&new_req);
  1245. new_req->request_type = RECV_REQ;
  1246. new_req->data_handle = NULL;
  1247. new_req->node_tag.rank = envelope_status.MPI_SOURCE;
  1248. new_req->node_tag.data_tag = envelope->data_tag;
  1249. new_req->node_tag.comm = envelope_comm;
  1250. new_req->detached = 1;
  1251. new_req->sync = 1;
  1252. new_req->callback = NULL;
  1253. new_req->callback_arg = NULL;
  1254. new_req->func = _starpu_mpi_irecv_data_func;
  1255. new_req->sequential_consistency = 1;
  1256. new_req->is_internal_req = 0; // ????
  1257. new_req->count = envelope->size;
  1258. _starpu_mpi_sync_data_add(new_req);
  1259. }
  1260. else
  1261. {
  1262. _starpu_mpi_receive_early_data(envelope, envelope_status, envelope_comm);
  1263. }
  1264. }
  1265. /* Case: a matching application request has been found for
  1266. * the incoming data, we handle the correct allocation
  1267. * of the pointer associated to the data handle, then
  1268. * submit the corresponding receive with
  1269. * _starpu_mpi_handle_ready_request. */
  1270. else
  1271. {
  1272. _STARPU_MPI_DEBUG(2000, "A matching application request has been found for the incoming data with tag %d\n", envelope->data_tag);
  1273. _STARPU_MPI_DEBUG(2000, "Request sync %d\n", envelope->sync);
  1274. early_request->sync = envelope->sync;
  1275. _starpu_mpi_datatype_allocate(early_request->data_handle, early_request);
  1276. if (early_request->registered_datatype == 1)
  1277. {
  1278. early_request->count = 1;
  1279. early_request->ptr = starpu_data_get_local_ptr(early_request->data_handle);
  1280. }
  1281. else
  1282. {
  1283. early_request->count = envelope->size;
  1284. _STARPU_MPI_MALLOC(early_request->ptr, early_request->count);
  1285. starpu_memory_allocate(STARPU_MAIN_RAM, early_request->count, STARPU_MEMORY_OVERFLOW);
  1286. STARPU_MPI_ASSERT_MSG(early_request->ptr, "cannot allocate message of size %ld\n", early_request->count);
  1287. }
  1288. _STARPU_MPI_DEBUG(3, "Handling new request... \n");
  1289. /* handling a request is likely to block for a while
  1290. * (on a sync_data_with_mem call), we want to let the
  1291. * application submit requests in the meantime, so we
  1292. * release the lock. */
  1293. STARPU_PTHREAD_MUTEX_UNLOCK(&progress_mutex);
  1294. _starpu_mpi_handle_ready_request(early_request);
  1295. STARPU_PTHREAD_MUTEX_LOCK(&progress_mutex);
  1296. }
  1297. }
  1298. envelope_request_submitted = 0;
  1299. }
  1300. else
  1301. {
  1302. //_STARPU_MPI_DEBUG(4, "Nothing received, continue ..\n");
  1303. }
  1304. }
  1305. #ifdef STARPU_SIMGRID
  1306. STARPU_PTHREAD_MUTEX_UNLOCK(&progress_mutex);
  1307. starpu_pthread_wait_wait(&wait);
  1308. STARPU_PTHREAD_MUTEX_LOCK(&progress_mutex);
  1309. #endif
  1310. }
  1311. if (envelope_request_submitted)
  1312. {
  1313. _starpu_mpi_comm_cancel_recv();
  1314. envelope_request_submitted = 0;
  1315. }
  1316. #ifdef STARPU_SIMGRID
  1317. STARPU_PTHREAD_MUTEX_LOCK(&wait_counter_mutex);
  1318. while (wait_counter != 0)
  1319. STARPU_PTHREAD_COND_WAIT(&wait_counter_cond, &wait_counter_mutex);
  1320. STARPU_PTHREAD_MUTEX_UNLOCK(&wait_counter_mutex);
  1321. STARPU_PTHREAD_MUTEX_DESTROY(&wait_counter_mutex);
  1322. STARPU_PTHREAD_COND_DESTROY(&wait_counter_cond);
  1323. starpu_pthread_queue_unregister(&wait, &dontsleep);
  1324. starpu_pthread_queue_destroy(&dontsleep);
  1325. starpu_pthread_wait_destroy(&wait);
  1326. #endif
  1327. STARPU_MPI_ASSERT_MSG(_starpu_mpi_req_list_empty(detached_requests), "List of detached requests not empty");
  1328. STARPU_MPI_ASSERT_MSG(_starpu_mpi_req_list_empty(ready_requests), "List of ready requests not empty");
  1329. STARPU_MPI_ASSERT_MSG(posted_requests == 0, "Number of posted request is not zero");
  1330. _starpu_mpi_early_request_check_termination();
  1331. _starpu_mpi_early_data_check_termination();
  1332. _starpu_mpi_sync_data_check_termination();
  1333. if (argc_argv->initialize_mpi)
  1334. {
  1335. _STARPU_MPI_DEBUG(3, "Calling MPI_Finalize()\n");
  1336. MPI_Finalize();
  1337. }
  1338. STARPU_PTHREAD_MUTEX_UNLOCK(&progress_mutex);
  1339. _starpu_mpi_sync_data_shutdown();
  1340. _starpu_mpi_early_data_shutdown();
  1341. _starpu_mpi_early_request_shutdown();
  1342. _starpu_mpi_datatype_shutdown();
  1343. free(argc_argv);
  1344. return NULL;
  1345. }
  1346. static void _starpu_mpi_add_sync_point_in_fxt(void)
  1347. {
  1348. #ifdef STARPU_USE_FXT
  1349. int rank;
  1350. int worldsize;
  1351. int ret;
  1352. starpu_mpi_comm_rank(MPI_COMM_WORLD, &rank);
  1353. starpu_mpi_comm_size(MPI_COMM_WORLD, &worldsize);
  1354. ret = MPI_Barrier(MPI_COMM_WORLD);
  1355. STARPU_MPI_ASSERT_MSG(ret == MPI_SUCCESS, "MPI_Barrier returning %s", _starpu_mpi_get_mpi_error_code(ret));
  1356. /* We generate a "unique" key so that we can make sure that different
  1357. * FxT traces come from the same MPI run. */
  1358. int random_number;
  1359. /* XXX perhaps we don't want to generate a new seed if the application
  1360. * specified some reproductible behaviour ? */
  1361. if (rank == 0)
  1362. {
  1363. srand(time(NULL));
  1364. random_number = rand();
  1365. }
  1366. ret = MPI_Bcast(&random_number, 1, MPI_INT, 0, MPI_COMM_WORLD);
  1367. STARPU_MPI_ASSERT_MSG(ret == MPI_SUCCESS, "MPI_Bcast returning %s", _starpu_mpi_get_mpi_error_code(ret));
  1368. _STARPU_MPI_TRACE_BARRIER(rank, worldsize, random_number);
  1369. _STARPU_MPI_DEBUG(3, "unique key %x\n", random_number);
  1370. #endif
  1371. }
  1372. int _starpu_mpi_progress_init(struct _starpu_mpi_argc_argv *argc_argv)
  1373. {
  1374. STARPU_PTHREAD_MUTEX_INIT(&progress_mutex, NULL);
  1375. STARPU_PTHREAD_COND_INIT(&progress_cond, NULL);
  1376. STARPU_PTHREAD_COND_INIT(&barrier_cond, NULL);
  1377. ready_requests = _starpu_mpi_req_list_new();
  1378. STARPU_PTHREAD_MUTEX_INIT(&detached_requests_mutex, NULL);
  1379. detached_requests = _starpu_mpi_req_list_new();
  1380. STARPU_PTHREAD_MUTEX_INIT(&mutex_posted_requests, NULL);
  1381. _starpu_mpi_comm_debug = starpu_getenv("STARPU_MPI_COMM") != NULL;
  1382. #ifdef STARPU_SIMGRID
  1383. STARPU_PTHREAD_MUTEX_INIT(&wait_counter_mutex, NULL);
  1384. STARPU_PTHREAD_COND_INIT(&wait_counter_cond, NULL);
  1385. #endif
  1386. #ifdef STARPU_SIMGRID
  1387. _starpu_mpi_progress_thread_func(argc_argv);
  1388. return 0;
  1389. #else
  1390. STARPU_PTHREAD_CREATE(&progress_thread, NULL, _starpu_mpi_progress_thread_func, argc_argv);
  1391. STARPU_PTHREAD_MUTEX_LOCK(&progress_mutex);
  1392. while (!running)
  1393. STARPU_PTHREAD_COND_WAIT(&progress_cond, &progress_mutex);
  1394. STARPU_PTHREAD_MUTEX_UNLOCK(&progress_mutex);
  1395. return 0;
  1396. #endif
  1397. }
  1398. #ifdef STARPU_SIMGRID
  1399. void _starpu_mpi_wait_for_initialization()
  1400. {
  1401. /* Wait for MPI initialization to finish */
  1402. STARPU_PTHREAD_MUTEX_LOCK(&progress_mutex);
  1403. while (!running)
  1404. STARPU_PTHREAD_COND_WAIT(&progress_cond, &progress_mutex);
  1405. STARPU_PTHREAD_MUTEX_UNLOCK(&progress_mutex);
  1406. }
  1407. #endif
  1408. void _starpu_mpi_progress_shutdown(int *value)
  1409. {
  1410. STARPU_PTHREAD_MUTEX_LOCK(&progress_mutex);
  1411. running = 0;
  1412. STARPU_PTHREAD_COND_BROADCAST(&progress_cond);
  1413. #ifdef STARPU_SIMGRID
  1414. starpu_pthread_queue_signal(&dontsleep);
  1415. #endif
  1416. STARPU_PTHREAD_MUTEX_UNLOCK(&progress_mutex);
  1417. #ifdef STARPU_SIMGRID
  1418. /* FIXME: should rather properly wait for _starpu_mpi_progress_thread_func to finish */
  1419. (void) value;
  1420. MSG_process_sleep(1);
  1421. #else
  1422. starpu_pthread_join(progress_thread, (void *)value);
  1423. #endif
  1424. /* free the request queues */
  1425. _starpu_mpi_req_list_delete(detached_requests);
  1426. _starpu_mpi_req_list_delete(ready_requests);
  1427. STARPU_PTHREAD_MUTEX_DESTROY(&mutex_posted_requests);
  1428. STARPU_PTHREAD_MUTEX_DESTROY(&progress_mutex);
  1429. STARPU_PTHREAD_COND_DESTROY(&barrier_cond);
  1430. }
  1431. void _starpu_mpi_data_clear(starpu_data_handle_t data_handle)
  1432. {
  1433. _starpu_mpi_tag_data_release(data_handle);
  1434. _starpu_mpi_cache_data_clear(data_handle);
  1435. free(data_handle->mpi_data);
  1436. }
  1437. void starpu_mpi_data_register_comm(starpu_data_handle_t data_handle, int tag, int rank, MPI_Comm comm)
  1438. {
  1439. struct _starpu_mpi_data *mpi_data;
  1440. if (data_handle->mpi_data)
  1441. {
  1442. mpi_data = data_handle->mpi_data;
  1443. }
  1444. else
  1445. {
  1446. _STARPU_CALLOC(mpi_data, 1, sizeof(struct _starpu_mpi_data));
  1447. mpi_data->magic = 42;
  1448. mpi_data->node_tag.data_tag = -1;
  1449. mpi_data->node_tag.rank = -1;
  1450. mpi_data->node_tag.comm = MPI_COMM_WORLD;
  1451. data_handle->mpi_data = mpi_data;
  1452. _starpu_mpi_cache_data_init(data_handle);
  1453. _starpu_mpi_tag_data_register(data_handle, tag);
  1454. _starpu_data_set_unregister_hook(data_handle, _starpu_mpi_data_clear);
  1455. }
  1456. if (tag != -1)
  1457. {
  1458. mpi_data->node_tag.data_tag = tag;
  1459. }
  1460. if (rank != -1)
  1461. {
  1462. _STARPU_MPI_TRACE_DATA_SET_RANK(data_handle, rank);
  1463. mpi_data->node_tag.rank = rank;
  1464. mpi_data->node_tag.comm = comm;
  1465. _starpu_mpi_comm_register(comm);
  1466. }
  1467. }
  1468. void starpu_mpi_data_set_rank_comm(starpu_data_handle_t handle, int rank, MPI_Comm comm)
  1469. {
  1470. starpu_mpi_data_register_comm(handle, -1, rank, comm);
  1471. }
  1472. void starpu_mpi_data_set_tag(starpu_data_handle_t handle, int tag)
  1473. {
  1474. starpu_mpi_data_register_comm(handle, tag, -1, MPI_COMM_WORLD);
  1475. }
  1476. int starpu_mpi_data_get_rank(starpu_data_handle_t data)
  1477. {
  1478. STARPU_ASSERT_MSG(data->mpi_data, "starpu_mpi_data_register MUST be called for data %p\n", data);
  1479. return ((struct _starpu_mpi_data *)(data->mpi_data))->node_tag.rank;
  1480. }
  1481. int starpu_mpi_data_get_tag(starpu_data_handle_t data)
  1482. {
  1483. STARPU_ASSERT_MSG(data->mpi_data, "starpu_mpi_data_register MUST be called for data %p\n", data);
  1484. return ((struct _starpu_mpi_data *)(data->mpi_data))->node_tag.data_tag;
  1485. }
  1486. void starpu_mpi_get_data_on_node_detached(MPI_Comm comm, starpu_data_handle_t data_handle, int node, void (*callback)(void*), void *arg)
  1487. {
  1488. int me, rank, tag;
  1489. rank = starpu_mpi_data_get_rank(data_handle);
  1490. if (rank == -1)
  1491. {
  1492. _STARPU_ERROR("StarPU needs to be told the MPI rank of this data, using starpu_mpi_data_register() or starpu_mpi_data_register()\n");
  1493. }
  1494. starpu_mpi_comm_rank(comm, &me);
  1495. if (node == rank) return;
  1496. tag = starpu_mpi_data_get_tag(data_handle);
  1497. if (tag == -1)
  1498. {
  1499. _STARPU_ERROR("StarPU needs to be told the MPI tag of this data, using starpu_mpi_data_register() or starpu_mpi_data_register()\n");
  1500. }
  1501. if (me == node)
  1502. {
  1503. _STARPU_MPI_DEBUG(1, "Migrating data %p from %d to %d\n", data_handle, rank, node);
  1504. int already_received = _starpu_mpi_cache_received_data_set(data_handle);
  1505. if (already_received == 0)
  1506. {
  1507. _STARPU_MPI_DEBUG(1, "Receiving data %p from %d\n", data_handle, rank);
  1508. starpu_mpi_irecv_detached(data_handle, rank, tag, comm, callback, arg);
  1509. }
  1510. }
  1511. else if (me == rank)
  1512. {
  1513. _STARPU_MPI_DEBUG(1, "Migrating data %p from %d to %d\n", data_handle, rank, node);
  1514. int already_sent = _starpu_mpi_cache_sent_data_set(data_handle, node);
  1515. if (already_sent == 0)
  1516. {
  1517. _STARPU_MPI_DEBUG(1, "Sending data %p to %d\n", data_handle, node);
  1518. starpu_mpi_isend_detached(data_handle, node, tag, comm, NULL, NULL);
  1519. }
  1520. }
  1521. }
  1522. void starpu_mpi_get_data_on_node(MPI_Comm comm, starpu_data_handle_t data_handle, int node)
  1523. {
  1524. int me, rank, tag;
  1525. rank = starpu_mpi_data_get_rank(data_handle);
  1526. if (rank == -1)
  1527. {
  1528. _STARPU_ERROR("StarPU needs to be told the MPI rank of this data, using starpu_mpi_data_register\n");
  1529. }
  1530. starpu_mpi_comm_rank(comm, &me);
  1531. if (node == rank) return;
  1532. tag = starpu_mpi_data_get_tag(data_handle);
  1533. if (tag == -1)
  1534. {
  1535. _STARPU_ERROR("StarPU needs to be told the MPI tag of this data, using starpu_mpi_data_register\n");
  1536. }
  1537. if (me == node)
  1538. {
  1539. MPI_Status status;
  1540. _STARPU_MPI_DEBUG(1, "Migrating data %p from %d to %d\n", data_handle, rank, node);
  1541. int already_received = _starpu_mpi_cache_received_data_set(data_handle);
  1542. if (already_received == 0)
  1543. {
  1544. _STARPU_MPI_DEBUG(1, "Receiving data %p from %d\n", data_handle, rank);
  1545. starpu_mpi_recv(data_handle, rank, tag, comm, &status);
  1546. }
  1547. }
  1548. else if (me == rank)
  1549. {
  1550. _STARPU_MPI_DEBUG(1, "Migrating data %p from %d to %d\n", data_handle, rank, node);
  1551. int already_sent = _starpu_mpi_cache_sent_data_set(data_handle, node);
  1552. if (already_sent == 0)
  1553. {
  1554. _STARPU_MPI_DEBUG(1, "Sending data %p to %d\n", data_handle, node);
  1555. starpu_mpi_send(data_handle, node, tag, comm);
  1556. }
  1557. }
  1558. }
  1559. void starpu_mpi_data_migrate(MPI_Comm comm, starpu_data_handle_t data, int new_rank)
  1560. {
  1561. int old_rank = starpu_mpi_data_get_rank(data);
  1562. if (new_rank == old_rank)
  1563. /* Already there */
  1564. return;
  1565. /* First submit data migration if it's not already on destination */
  1566. starpu_mpi_get_data_on_node_detached(comm, data, new_rank, NULL, NULL);
  1567. /* And note new owner */
  1568. starpu_mpi_data_set_rank_comm(data, new_rank, comm);
  1569. /* Flush cache in all other nodes */
  1570. /* TODO: Ideally we'd transmit the knowledge of who owns it */
  1571. starpu_mpi_cache_flush(comm, data);
  1572. return;
  1573. }
  1574. int starpu_mpi_wait_for_all(MPI_Comm comm)
  1575. {
  1576. int mpi = 1;
  1577. int task = 1;
  1578. while (task || mpi)
  1579. {
  1580. task = _starpu_task_wait_for_all_and_return_nb_waited_tasks();
  1581. mpi = _starpu_mpi_barrier(comm);
  1582. }
  1583. return 0;
  1584. }