coherency.h 13 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343
  1. /* StarPU --- Runtime system for heterogeneous multicore architectures.
  2. *
  3. * Copyright (C) 2011,2014-2017 Inria
  4. * Copyright (C) 2008-2018 Université de Bordeaux
  5. * Copyright (C) 2010-2015,2017 CNRS
  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. #ifndef __COHERENCY__H__
  19. #define __COHERENCY__H__
  20. #include <starpu.h>
  21. #include <common/config.h>
  22. #include <common/starpu_spinlock.h>
  23. #include <common/rwlock.h>
  24. #include <common/timing.h>
  25. #include <common/fxt.h>
  26. #include <common/list.h>
  27. #include <datawizard/interfaces/data_interface.h>
  28. #include <datawizard/datastats.h>
  29. #include <datawizard/memstats.h>
  30. #include <datawizard/data_request.h>
  31. enum _starpu_cache_state
  32. {
  33. STARPU_OWNER,
  34. STARPU_SHARED,
  35. STARPU_INVALID
  36. };
  37. /* this should contain the information relative to a given data replicate */
  38. struct _starpu_data_replicate
  39. {
  40. starpu_data_handle_t handle;
  41. /* describe the actual data layout, as manipulated by data interfaces in *_interface.c */
  42. void *data_interface;
  43. /* How many requests or tasks are currently working with this replicate */
  44. int refcnt;
  45. char memory_node;
  46. /* describes the state of the local data in term of coherency */
  47. enum _starpu_cache_state state: 2;
  48. /* A buffer that is used for SCRATCH or reduction cannnot be used with
  49. * filters. */
  50. unsigned relaxed_coherency:2;
  51. /* We may need to initialize the replicate with some value before using it. */
  52. unsigned initialized:1;
  53. /* is the data locally allocated ? */
  54. unsigned allocated:1;
  55. /* was it automatically allocated ? (else it's the application-provided
  56. * buffer, don't ever try to free it!) */
  57. /* perhaps the allocation was perform higher in the hiearchy
  58. * for now this is just translated into !automatically_allocated
  59. * */
  60. unsigned automatically_allocated:1;
  61. /* To help the scheduling policies to make some decision, we
  62. may keep a track of the tasks that are likely to request
  63. this data on the current node.
  64. It is the responsability of the scheduling _policy_ to set that
  65. flag when it assigns a task to a queue, policies which do not
  66. use this hint can simply ignore it.
  67. */
  68. uint32_t requested;
  69. struct _starpu_data_request *request[STARPU_MAXNODES];
  70. /* Pointer to memchunk for LRU strategy */
  71. struct _starpu_mem_chunk * mc;
  72. };
  73. struct _starpu_data_requester_list;
  74. struct _starpu_jobid_list
  75. {
  76. unsigned long id;
  77. struct _starpu_jobid_list *next;
  78. };
  79. /* This structure describes a simply-linked list of task */
  80. struct _starpu_task_wrapper_list
  81. {
  82. struct starpu_task *task;
  83. struct _starpu_task_wrapper_list *next;
  84. };
  85. /* This structure describes a doubly-linked list of task */
  86. struct _starpu_task_wrapper_dlist
  87. {
  88. struct starpu_task *task;
  89. struct _starpu_task_wrapper_dlist *next;
  90. struct _starpu_task_wrapper_dlist *prev;
  91. };
  92. extern int _starpu_has_not_important_data;
  93. typedef void (*_starpu_data_handle_unregister_hook)(starpu_data_handle_t);
  94. /* This is initialized in both _starpu_register_new_data and _starpu_data_partition */
  95. struct _starpu_data_state
  96. {
  97. int magic;
  98. struct _starpu_data_requester_list req_list;
  99. /* the number of requests currently in the scheduling engine (not in
  100. * the req_list anymore), i.e. the number of holders of the
  101. * current_mode rwlock */
  102. unsigned refcnt;
  103. /* whether we are already unlocking data requests */
  104. unsigned unlocking_reqs;
  105. /* Current access mode. Is always either STARPU_R, STARPU_W,
  106. * STARPU_SCRATCH or STARPU_REDUX, but never a combination such as
  107. * STARPU_RW. */
  108. enum starpu_data_access_mode current_mode;
  109. /* protect meta data */
  110. struct _starpu_spinlock header_lock;
  111. /* Condition to make application wait for all transfers before freeing handle */
  112. /* busy_count is the number of handle->refcnt, handle->per_node[*]->refcnt, number of starpu_data_requesters, and number of tasks that have released it but are still registered on the implicit data dependency lists. */
  113. /* Core code which releases busy_count has to call
  114. * _starpu_data_check_not_busy to let starpu_data_unregister proceed */
  115. unsigned busy_count;
  116. /* Is starpu_data_unregister waiting for busy_count? */
  117. unsigned busy_waiting;
  118. starpu_pthread_mutex_t busy_mutex;
  119. starpu_pthread_cond_t busy_cond;
  120. /* In case we user filters, the handle may describe a sub-data */
  121. struct _starpu_data_state *root_handle; /* root of the tree */
  122. struct _starpu_data_state *father_handle; /* father of the node, NULL if the current node is the root */
  123. starpu_data_handle_t *active_children; /* The currently active set of read-write children */
  124. starpu_data_handle_t **active_readonly_children; /* The currently active set of read-only children */
  125. unsigned nactive_readonly_children; /* Size of active_readonly_children array */
  126. /* Our siblings in the father partitioning */
  127. unsigned nsiblings; /* How many siblings */
  128. starpu_data_handle_t *siblings;
  129. unsigned sibling_index; /* indicate which child this node is from the father's perpsective (if any) */
  130. unsigned depth; /* what's the depth of the tree ? */
  131. /* Synchronous partitioning */
  132. starpu_data_handle_t children;
  133. unsigned nchildren;
  134. /* How many partition plans this handle has */
  135. unsigned nplans;
  136. /* Switch codelet for asynchronous partitioning */
  137. struct starpu_codelet *switch_cl;
  138. /* size of dyn_nodes recorded in switch_cl */
  139. unsigned switch_cl_nparts;
  140. /* Whether a partition plan is currently submitted and the
  141. * corresponding unpartition has not been yet
  142. *
  143. * Or the number of partition plans currently submitted in readonly
  144. * mode.
  145. */
  146. unsigned partitioned;
  147. /* Whether a partition plan is currently submitted in readonly mode */
  148. unsigned readonly:1;
  149. /* Whether our father is currently partitioned into ourself */
  150. unsigned active:1;
  151. unsigned active_ro:1;
  152. /* describe the state of the data in term of coherency */
  153. struct _starpu_data_replicate per_node[STARPU_MAXNODES];
  154. struct _starpu_data_replicate *per_worker;
  155. struct starpu_data_interface_ops *ops;
  156. /* Footprint which identifies data layout */
  157. uint32_t footprint;
  158. /* where is the data home, i.e. which node it was registered from ? -1 if none yet */
  159. int home_node;
  160. /* what is the default write-through mask for that data ? */
  161. uint32_t wt_mask;
  162. /* in some case, the application may explicitly tell StarPU that a
  163. * piece of data is not likely to be used soon again */
  164. unsigned is_not_important;
  165. /* Does StarPU have to enforce some implicit data-dependencies ? */
  166. unsigned sequential_consistency;
  167. /* Is the data initialized, or a task is already submitted to initialize it */
  168. unsigned initialized;
  169. /* This lock should protect any operation to enforce
  170. * sequential_consistency */
  171. starpu_pthread_mutex_t sequential_consistency_mutex;
  172. /* The last submitted task (or application data request) that declared
  173. * it would modify the piece of data ? Any task accessing the data in a
  174. * read-only mode should depend on that task implicitely if the
  175. * sequential_consistency flag is enabled. */
  176. enum starpu_data_access_mode last_submitted_mode;
  177. struct starpu_task *last_sync_task;
  178. struct _starpu_task_wrapper_dlist last_submitted_accessors;
  179. /* If FxT is enabled, we keep track of "ghost dependencies": that is to
  180. * say the dependencies that are not needed anymore, but that should
  181. * appear in the post-mortem DAG. For instance if we have the sequence
  182. * f(Aw) g(Aw), and that g is submitted after the termination of f, we
  183. * want to have f->g appear in the DAG even if StarPU does not need to
  184. * enforce this dependency anymore.*/
  185. unsigned last_submitted_ghost_sync_id_is_valid;
  186. unsigned long last_submitted_ghost_sync_id;
  187. struct _starpu_jobid_list *last_submitted_ghost_accessors_id;
  188. /* protected by sequential_consistency_mutex */
  189. struct _starpu_task_wrapper_list *post_sync_tasks;
  190. unsigned post_sync_tasks_cnt;
  191. /*
  192. * Reductions
  193. */
  194. /* During reduction we need some specific methods: redux_func performs
  195. * the reduction of an interface into another one (eg. "+="), and init_func
  196. * initializes the data interface to a default value that is stable by
  197. * reduction (eg. 0 for +=). */
  198. struct starpu_codelet *redux_cl;
  199. struct starpu_codelet *init_cl;
  200. /* Are we currently performing a reduction on that handle ? If so the
  201. * reduction_refcnt should be non null until there are pending tasks
  202. * that are performing the reduction. */
  203. unsigned reduction_refcnt;
  204. /* List of requesters that are specific to the pending reduction. This
  205. * list is used when the requests in the req_list list are frozen until
  206. * the end of the reduction. */
  207. struct _starpu_data_requester_list reduction_req_list;
  208. starpu_data_handle_t *reduction_tmp_handles;
  209. /* Final request for write invalidation */
  210. struct _starpu_data_request *write_invalidation_req;
  211. unsigned lazy_unregister;
  212. #ifdef STARPU_OPENMP
  213. unsigned removed_from_context_hash;
  214. #endif
  215. /* Used for MPI */
  216. void *mpi_data;
  217. _starpu_memory_stats_t memory_stats;
  218. unsigned int mf_node; //XXX
  219. /* hook to be called when unregistering the data */
  220. _starpu_data_handle_unregister_hook unregister_hook;
  221. struct starpu_arbiter *arbiter;
  222. /* This is protected by the arbiter mutex */
  223. struct _starpu_data_requester_list arbitered_req_list;
  224. /* Data maintained by schedulers themselves */
  225. /* Last worker that took this data in locality mode, or -1 if nobody
  226. * took it yet */
  227. int last_locality;
  228. /* A generic pointer to data in the user land (could be anything and this
  229. * is not manage by StarPU) */
  230. void *user_data;
  231. };
  232. /* This does not take a reference on the handle, the caller has to do it,
  233. * e.g. through _starpu_attempt_to_submit_data_request_from_apps()
  234. * detached means that the core is allowed to drop the request. The caller
  235. * should thus *not* take a reference since it can not know whether the request will complete
  236. * async means that _starpu_fetch_data_on_node will wait for completion of the request
  237. */
  238. int _starpu_fetch_data_on_node(starpu_data_handle_t handle, int node, struct _starpu_data_replicate *replicate,
  239. enum starpu_data_access_mode mode, unsigned detached, unsigned is_prefetch, unsigned async,
  240. void (*callback_func)(void *), void *callback_arg, int prio, const char *origin);
  241. /* This releases a reference on the handle */
  242. void _starpu_release_data_on_node(struct _starpu_data_state *state, uint32_t default_wt_mask,
  243. struct _starpu_data_replicate *replicate);
  244. void _starpu_update_data_state(starpu_data_handle_t handle,
  245. struct _starpu_data_replicate *requesting_replicate,
  246. enum starpu_data_access_mode mode);
  247. uint32_t _starpu_get_data_refcnt(struct _starpu_data_state *state, unsigned node);
  248. size_t _starpu_data_get_size(starpu_data_handle_t handle);
  249. uint32_t _starpu_data_get_footprint(starpu_data_handle_t handle);
  250. void __starpu_push_task_output(struct _starpu_job *j);
  251. /* Version with driver trace */
  252. void _starpu_push_task_output(struct _starpu_job *j);
  253. void _starpu_release_nowhere_task_output(struct _starpu_job *j);
  254. struct _starpu_worker;
  255. STARPU_ATTRIBUTE_WARN_UNUSED_RESULT
  256. int _starpu_fetch_task_input(struct starpu_task *task, struct _starpu_job *j, int async);
  257. void _starpu_fetch_task_input_tail(struct starpu_task *task, struct _starpu_job *j, struct _starpu_worker *worker);
  258. void _starpu_fetch_nowhere_task_input(struct _starpu_job *j);
  259. unsigned _starpu_is_data_present_or_requested(struct _starpu_data_state *state, unsigned node);
  260. int _starpu_select_src_node(struct _starpu_data_state *state, unsigned destination);
  261. /* is_prefetch is whether the DSM may drop the request (when there is not enough memory for instance
  262. * async is whether the caller wants a reference on the last request, to be
  263. * able to wait for it (which will release that reference).
  264. */
  265. struct _starpu_data_request *_starpu_create_request_to_fetch_data(starpu_data_handle_t handle,
  266. struct _starpu_data_replicate *dst_replicate,
  267. enum starpu_data_access_mode mode, unsigned is_prefetch,
  268. unsigned async,
  269. void (*callback_func)(void *), void *callback_arg, int prio, const char *origin);
  270. void _starpu_redux_init_data_replicate(starpu_data_handle_t handle, struct _starpu_data_replicate *replicate, int workerid);
  271. void _starpu_data_start_reduction_mode(starpu_data_handle_t handle);
  272. void _starpu_data_end_reduction_mode(starpu_data_handle_t handle);
  273. void _starpu_data_end_reduction_mode_terminate(starpu_data_handle_t handle);
  274. void _starpu_data_set_unregister_hook(starpu_data_handle_t handle, _starpu_data_handle_unregister_hook func);
  275. struct _starpu_data_replicate *get_replicate(starpu_data_handle_t handle, enum starpu_data_access_mode mode, int workerid, unsigned node);
  276. #endif // __COHERENCY__H__