starpu_mpi.c 65 KB

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