| 123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300 | 
							- /* StarPU --- Runtime system for heterogeneous multicore architectures.
 
-  *
 
-  * Copyright (C) 2011-2013                                Inria
 
-  * Copyright (C) 2010-2018                                CNRS
 
-  * Copyright (C) 2009-2011,2014-2018                      Université de Bordeaux
 
-  *
 
-  * StarPU is free software; you can redistribute it and/or modify
 
-  * it under the terms of the GNU Lesser General Public License as published by
 
-  * the Free Software Foundation; either version 2.1 of the License, or (at
 
-  * your option) any later version.
 
-  *
 
-  * StarPU is distributed in the hope that it will be useful, but
 
-  * WITHOUT ANY WARRANTY; without even the implied warranty of
 
-  * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE.
 
-  *
 
-  * See the GNU Lesser General Public License in COPYING.LGPL for more details.
 
-  */
 
- /*! \defgroup API_Scheduling_Policy Scheduling Policy
 
- \brief TODO. While StarPU comes with a variety of scheduling policies
 
- (see \ref TaskSchedulingPolicy), it may sometimes be desirable to
 
- implement custom policies to address specific problems. The API
 
- described below allows users to write their own scheduling policy.
 
- \def STARPU_MAXIMPLEMENTATIONS
 
- \ingroup API_Scheduling_Policy
 
- Define the maximum number of implementations per architecture. The default value can be modified at
 
- configure by using the option \ref enable-maximplementations "--enable-maximplementations".
 
- \struct starpu_sched_policy
 
- \ingroup API_Scheduling_Policy
 
- Contain all the methods that implement a
 
- scheduling policy. An application may specify which scheduling
 
- strategy in the field starpu_conf::sched_policy passed to the function
 
- starpu_init().
 
- For each task going through the scheduler, the following methods get called in the given order:
 
- <ul>
 
- <li>starpu_sched_policy::submit_hook when the task is submitted</li>
 
- <li>starpu_sched_policy::push_task when the task becomes ready. The scheduler is here <b>given</b> the task</li>
 
- <li>starpu_sched_policy::pop_task when the worker is idle. The scheduler here <b>gives</b> back the task to the core</li>
 
- <li>starpu_sched_policy::pre_exec_hook right before the worker actually starts the task computation (after transferring any missing data).</li>
 
- <li>starpu_sched_policy::post_exec_hook right after the worker actually completes the task computation.</li>
 
- </ul>
 
- For each task not going through the scheduler (because starpu_task::execute_on_a_specific_worker was set), these get called:
 
- <ul>
 
- <li>starpu_sched_policy::submit_hook when the task is submitted</li>
 
- <li>starpu_sched_policy::push_task_notify when the task becomes ready. This is just a notification, the scheduler does not have to do anything about the task.</li>
 
- <li>starpu_sched_policy::pre_exec_hook right before the worker actually starts the task computation (after transferring any missing data).</li>
 
- <li>starpu_sched_policy::post_exec_hook right after the worker actually completes the task computation.</li>
 
- </ul>
 
- \var void (*starpu_sched_policy::init_sched)(unsigned sched_ctx_id)
 
-         Initialize the scheduling policy, called before any other method.
 
- \var void (*starpu_sched_policy::deinit_sched)(unsigned sched_ctx_id)
 
-         Cleanup the scheduling policy, called before any other method.
 
- \var int (*starpu_sched_policy::push_task)(struct starpu_task *)
 
-         Insert a task into the scheduler, called when the task becomes ready for
 
-         execution.
 
- \var void (*starpu_sched_policy::push_task_notify)(struct starpu_task *, int workerid, int perf_workerid, unsigned sched_ctx_id)
 
-         Notify the scheduler that a task was pushed on a given worker.
 
- 	This method is called when a task that was explicitly
 
- 	assigned to a worker becomes ready and is about to be executed
 
- 	by the worker. This method therefore permits to keep the state
 
- 	of the scheduler coherent even when StarPU bypasses the
 
- 	scheduling strategy.
 
- \var struct starpu_task *(*starpu_sched_policy::pop_task)(unsigned sched_ctx_id)
 
-         Get a task from the scheduler.
 
-         If this method returns NULL, the worker will start sleeping. If later on
 
-         some task are pushed for this worker, starpu_wake_worker() must be
 
-         called to wake the worker so it can call the pop_task() method again.
 
- 	
 
-         The mutex associated to the worker is already taken when this method
 
-         is called. This method may release it (e.g. for scalability reasons
 
-         when doing work stealing), but it must acquire it again before taking
 
-         the decision whether to return a task or NULL, so the atomicity of
 
-         deciding to return NULL and making the worker actually sleep is
 
-         preserved. Otherwise in simgrid or blocking driver mode the worker might start
 
-         sleeping while a task has just been pushed for it.
 
-         If this method is defined as <c>NULL</c>, the worker will only execute
 
-         tasks from its local queue. In this case, the push_task method should
 
-         use the starpu_push_local_task method to assign tasks to the different
 
-         workers.
 
- \var struct starpu_task *(*starpu_sched_policy::pop_every_task)(unsigned sched_ctx_id)
 
-         Remove all available tasks from the scheduler (tasks are
 
- 	chained by the means of the field starpu_task::prev and
 
- 	starpu_task::next). The mutex associated to the worker is
 
- 	already taken when this method is called. This is currently
 
- 	not used and can be discarded.
 
- \var void (*starpu_sched_policy::submit_hook)(struct starpu_task *)
 
-         Optional field. This method is called when a task is submitted.
 
- \var void (*starpu_sched_policy::pre_exec_hook)(struct starpu_task *)
 
-         Optional field. This method is called every time a task is starting.
 
- \var void (*starpu_sched_policy::post_exec_hook)(struct starpu_task *)
 
-         Optional field. This method is called every time a task has been executed.
 
- \var void (*starpu_sched_policy::do_schedule)(unsigned sched_ctx_id)
 
-         Optional field. This method is called when it is a good time to start
 
-         scheduling tasks. This is notably called when the application calls
 
-         starpu_task_wait_for_all or starpu_do_schedule explicitly.
 
- \var void (*starpu_sched_policy::add_workers)(unsigned sched_ctx_id, int *workerids, unsigned nworkers)
 
-         Initialize scheduling structures corresponding to each worker used by the policy.
 
- \var void (*starpu_sched_policy::remove_workers)(unsigned sched_ctx_id, int *workerids, unsigned nworkers)
 
-         Deinitialize scheduling structures corresponding to each worker used by the policy.
 
- \var const char *starpu_sched_policy::policy_name
 
-         Optional field. Name of the policy.
 
- \var const char *starpu_sched_policy::policy_description
 
-         Optional field. Human readable description of the policy.
 
- \fn struct starpu_sched_policy **starpu_sched_get_predefined_policies()
 
- \ingroup API_Scheduling_Policy
 
- Return an <c>NULL</c>-terminated array of all the predefined scheduling
 
- policies.
 
- \fn void starpu_worker_get_sched_condition(int workerid, starpu_pthread_mutex_t **sched_mutex, starpu_pthread_cond_t **sched_cond)
 
- \ingroup API_Scheduling_Policy
 
- When there is no available task for a worker, StarPU blocks this
 
- worker on a condition variable. This function specifies which
 
- condition variable (and the associated mutex) should be used to block
 
- (and to wake up) a worker. Note that multiple workers may use the same
 
- condition variable. For instance, in the case of a scheduling strategy
 
- with a single task queue, the same condition variable would be used to
 
- block and wake up all workers.
 
- \fn int starpu_wake_worker_no_relax(int workerid)
 
- \ingroup API_Scheduling_Policy
 
- Must be called to wake up a worker that is sleeping on the cond.
 
- Return 0 whenever the worker is not in a sleeping state or has the
 
- state_keep_awake flag on.
 
- \fn int starpu_wake_worker_locked(int workerid)
 
- \ingroup API_Scheduling_Policy
 
- Version of starpu_wake_worker_no_relax() which assumes that the sched
 
- mutex is locked
 
- \fn int starpu_sched_set_min_priority(int min_prio)
 
- \ingroup API_Scheduling_Policy
 
- TODO: check if this is correct
 
- Define the minimum task priority level supported by the scheduling
 
- policy. The default minimum priority level is the same as the default
 
- priority level which is 0 by convention. The application may access
 
- that value by calling the function starpu_sched_get_min_priority().
 
- This function should only be called from the initialization method of
 
- the scheduling policy, and should not be used directly from the
 
- application.
 
- \fn int starpu_sched_set_max_priority(int max_prio)
 
- \ingroup API_Scheduling_Policy
 
- TODO: check if this is correct
 
- Define the maximum priority level supported by the scheduling policy.
 
- The default maximum priority level is 1. The application may access
 
- that value by calling the function starpu_sched_get_max_priority().
 
- This function should only be called from the initialization method of
 
- the scheduling policy, and should not be used directly from the
 
- application.
 
- \fn int starpu_sched_get_min_priority(void)
 
- \ingroup API_Scheduling_Policy
 
- TODO: check if this is correct
 
- Return the current minimum priority level supported by the scheduling
 
- policy
 
- \fn int starpu_sched_get_max_priority(void)
 
- \ingroup API_Scheduling_Policy
 
- TODO: check if this is correct
 
- Return the current maximum priority level supported by the scheduling
 
- policy
 
- \fn int starpu_push_local_task(int workerid, struct starpu_task *task, int back)
 
- \ingroup API_Scheduling_Policy
 
- The scheduling policy may put tasks directly into a worker’s local
 
- queue so that it is not always necessary to create its own queue when
 
- the local queue is sufficient. If \p back is not 0, \p task is put
 
- at the back of the queue where the worker will pop tasks first.
 
- Setting \p back to 0 therefore ensures a FIFO ordering.
 
- \fn int starpu_push_task_end(struct starpu_task *task)
 
- \ingroup API_Scheduling_Policy
 
- Must be called by a scheduler to notify that the given
 
- task has just been pushed.
 
- \fn int starpu_worker_can_execute_task(unsigned workerid, struct starpu_task *task, unsigned nimpl)
 
- \ingroup API_Scheduling_Policy
 
- Check if the worker specified by workerid can execute the codelet.
 
- Schedulers need to call it before assigning a task to a worker,
 
- otherwise the task may fail to execute.
 
- \fn int starpu_worker_can_execute_task_impl(unsigned workerid, struct starpu_task *task, unsigned *impl_mask)
 
- \ingroup API_Scheduling_Policy
 
- Check if the worker specified by workerid can execute the codelet and returns
 
- which implementation numbers can be used.
 
- Schedulers need to call it before assigning a task to a worker,
 
- otherwise the task may fail to execute.
 
- This should be preferred rather than calling starpu_worker_can_execute_task for
 
- each and every implementation. It can also be used with <c>impl_mask == NULL</c> to
 
- check for at least one implementation without determining which.
 
- \fn int starpu_worker_can_execute_task_first_impl(unsigned workerid, struct starpu_task *task, unsigned *nimpl)
 
- \ingroup API_Scheduling_Policy
 
- Check if the worker specified by workerid can execute the codelet and returns
 
- the first implementation which can be used.
 
- Schedulers need to call it before assigning a task to a worker,
 
- otherwise the task may fail to execute.
 
- This should be preferred rather than calling starpu_worker_can_execute_task for
 
- each and every implementation. It can also be used with <c>impl_mask == NULL</c> to
 
- check for at least one implementation without determining which.
 
- \fn uint32_t starpu_task_footprint(struct starpu_perfmodel *model, struct starpu_task *task, struct starpu_perfmodel_arch *arch, unsigned nimpl)
 
- \ingroup API_Scheduling_Policy
 
- Return the footprint for a given task, taking into account user-provided
 
- perfmodel footprint or size_base functions.
 
- \fn uint32_t starpu_task_data_footprint(struct starpu_task *task)
 
- \ingroup API_Scheduling_Policy
 
- Return the raw footprint for the data of a given task (without taking into account user-provided functions).
 
- \fn double starpu_task_expected_length(struct starpu_task *task, struct starpu_perfmodel_arch *arch, unsigned nimpl)
 
- \ingroup API_Scheduling_Policy
 
- Return expected task duration in micro-seconds.
 
- \fn double starpu_worker_get_relative_speedup(struct starpu_perfmodel_arch *perf_arch)
 
- \ingroup API_Scheduling_Policy
 
- Return an estimated speedup factor relative to CPU speed
 
- \fn double starpu_task_expected_data_transfer_time(unsigned memory_node, struct starpu_task *task)
 
- \ingroup API_Scheduling_Policy
 
- Return expected data transfer time in micro-seconds for the given \p
 
- memory_node. Prefer using starpu_task_expected_data_transfer_time_for() which is
 
- more precise.
 
- \fn double starpu_task_expected_data_transfer_time_for(struct starpu_task *task, unsigned worker)
 
- \ingroup API_Scheduling_Policy
 
- Return expected data transfer time in micro-seconds for the given \p worker.
 
- \fn double starpu_data_expected_transfer_time(starpu_data_handle_t handle, unsigned memory_node, enum starpu_data_access_mode mode)
 
- \ingroup API_Scheduling_Policy
 
- Predict the transfer time (in micro-seconds) to move \p handle to a memory node
 
- \fn double starpu_task_expected_energy(struct starpu_task *task, struct starpu_perfmodel_arch *arch, unsigned nimpl)
 
- \ingroup API_Scheduling_Policy
 
- Return expected energy consumption in J
 
- \fn double starpu_task_expected_conversion_time(struct starpu_task *task, struct starpu_perfmodel_arch *arch, unsigned nimpl)
 
- \ingroup API_Scheduling_Policy
 
- Return expected conversion time in ms (multiformat interface only)
 
- \fn int starpu_get_prefetch_flag(void)
 
- \ingroup API_Scheduling_Policy
 
- Whether \ref STARPU_PREFETCH was set
 
- \fn int starpu_prefetch_task_input_on_node(struct starpu_task *task, unsigned node)
 
- \ingroup API_Scheduling_Policy
 
- Prefetch data for a given p task on a given p node
 
- \fn int starpu_prefetch_task_input_on_node_prio(struct starpu_task *task, unsigned node, int prio)
 
- \ingroup API_Scheduling_Policy
 
- Prefetch data for a given p task on a given p node with a given priority
 
- \fn int starpu_idle_prefetch_task_input_on_node(struct starpu_task *task, unsigned node)
 
- \ingroup API_Scheduling_Policy
 
- Prefetch data for a given p task on a given p node when the bus is idle
 
- \fn int starpu_idle_prefetch_task_input_on_node_prio(struct starpu_task *task, unsigned node, int prio)
 
- \ingroup API_Scheduling_Policy
 
- Prefetch data for a given p task on a given p node when the bus is idle with a given priority
 
- \fn int starpu_prefetch_task_input_for(struct starpu_task *task, unsigned worker)
 
- \ingroup API_Scheduling_Policy
 
- Prefetch data for a given p task on a given p worker
 
- \fn int starpu_prefetch_task_input_for_prio(struct starpu_task *task, unsigned worker, int prio)
 
- \ingroup API_Scheduling_Policy
 
- Prefetch data for a given p task on a given p worker with a given priority
 
- \fn int starpu_idle_prefetch_task_input_for(struct starpu_task *task, unsigned worker)
 
- \ingroup API_Scheduling_Policy
 
- Prefetch data for a given p task on a given p worker when the bus is idle
 
- \fn int starpu_idle_prefetch_task_input_for_prio(struct starpu_task *task, unsigned worker, int prio)
 
- \ingroup API_Scheduling_Policy
 
- Prefetch data for a given p task on a given p worker when the bus is idle with a given priority
 
- \fn void starpu_task_notify_ready_soon_register(starpu_notify_ready_soon_func f, void *data);
 
- \ingroup API_Scheduling_Policy
 
- Register a callback to be called when it is determined when a task will be ready
 
- an estimated amount of time from now, because its last dependency has just
 
- started and we know how long it will take.
 
- \fn void starpu_sched_ctx_worker_shares_tasks_lists(int workerid, int sched_ctx_id)
 
- \ingroup API_Scheduling_Policy
 
- The scheduling policies indicates if the worker may pop tasks from the list of other workers
 
- or if there is a central list with task for all the workers
 
- */
 
 
  |