Przeglądaj źródła

sched_ctx: move worker functions back to starpu_sched_ctx.h and rename them to better identify their purpose

Nathalie Furmento 12 lat temu
rodzic
commit
11dd8ecb24

+ 2 - 2
doc/chapters/advanced-api.texi

@@ -696,7 +696,7 @@ Description of the policy.
 @end table
 @end deftp
 
-@deftypefun void starpu_worker_set_sched_condition (unsigned @var{sched_ctx_id}, int @var{workerid}, pthread_mutex_t *@var{sched_mutex}, {pthread_cond_t *}@var{sched_cond})
+@deftypefun void starpu_sched_ctx_set_worker_mutex_and_cond (unsigned @var{sched_ctx_id}, int @var{workerid}, pthread_mutex_t *@var{sched_mutex}, {pthread_cond_t *}@var{sched_cond})
 This function specifies the condition variable associated to a worker per context
 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
@@ -708,7 +708,7 @@ The initialization method of a scheduling strategy (@code{init_sched}) must
 call this function once per worker.
 @end deftypefun
 
-@deftypefun void starpu_worker_get_sched_condition (unsigned @var{sched_ctx_id}, int @var{workerid}, {pthread_mutex_t **}@var{sched_mutex}, {pthread_cond_t **}@var{sched_cond})
+@deftypefun void starpu_sched_ctx_get_worker_mutex_and_cond (unsigned @var{sched_ctx_id}, int @var{workerid}, {pthread_mutex_t **}@var{sched_mutex}, {pthread_cond_t **}@var{sched_cond})
 This function returns the condition variables associated to a worker in a context
 It is used in the policy to access to the local queue of the worker
 @end deftypefun

+ 2 - 2
examples/scheduler/dummy_sched.c

@@ -36,7 +36,7 @@ static void dummy_sched_add_workers(unsigned sched_ctx_id, int *workerids, unsig
 	for(i = 0; i < nworkers; i++)
 	{
 		workerid = workerids[i];
-		starpu_worker_set_sched_condition(sched_ctx_id, workerid, &data->sched_mutex,  &data->sched_cond);
+		starpu_sched_ctx_set_worker_mutex_and_cond(sched_ctx_id, workerid, &data->sched_mutex,  &data->sched_cond);
 	}
 }
 
@@ -47,7 +47,7 @@ static void dummy_sched_remove_workers(unsigned sched_ctx_id, int *workerids, un
 	for(i = 0; i < nworkers; i++)
 	{
 		workerid = workerids[i];
-		starpu_worker_set_sched_condition(sched_ctx_id, workerid, NULL,  NULL);
+		starpu_sched_ctx_set_worker_mutex_and_cond(sched_ctx_id, workerid, NULL,  NULL);
 	}
 }
 

+ 12 - 0
include/starpu_sched_ctx.h

@@ -92,6 +92,18 @@ void starpu_sched_ctx_set_policy_data(unsigned sched_ctx_id, void *policy_data);
 
 void* starpu_sched_ctx_get_policy_data(unsigned sched_ctx);
 
+#ifdef STARPU_DEVEL
+#warning do we really need both starpu_sched_ctx_set_worker_mutex_and_cond and starpu_sched_ctx_init_worker_mutex_and_cond functions
+#endif
+
+void starpu_sched_ctx_set_worker_mutex_and_cond(unsigned sched_ctx_id, int workerid, pthread_mutex_t *sched_mutex, pthread_cond_t *sched_cond);
+
+void starpu_sched_ctx_get_worker_mutex_and_cond(unsigned sched_ctx_id, int workerid, pthread_mutex_t **sched_mutex, pthread_cond_t **sched_cond);
+
+void starpu_sched_ctx_init_worker_mutex_and_cond(unsigned sched_ctx_id, int workerid);
+
+void starpu_sched_ctx_deinit_worker_mutex_and_cond(unsigned sched_ctx_id, int workerid);
+
 struct starpu_sched_ctx_worker_collection* starpu_create_worker_collection_for_sched_ctx(unsigned sched_ctx_id, int type);
 
 void starpu_delete_worker_collection_for_sched_ctx(unsigned sched_ctx_id);

+ 0 - 12
include/starpu_worker.h

@@ -99,17 +99,5 @@ int starpu_worker_get_devid(int id);
 }
 #endif
 
-#ifdef STARPU_DEVEL
-#warning do we really need both starpu_worker_set_sched_condition and starpu_worker_init_sched_condition functions
-#endif
-
-void starpu_worker_set_sched_condition(unsigned sched_ctx_id, int workerid, pthread_mutex_t *sched_mutex, pthread_cond_t *sched_cond);
-
-void starpu_worker_get_sched_condition(unsigned sched_ctx_id, int workerid, pthread_mutex_t **sched_mutex, pthread_cond_t **sched_cond);
-
-void starpu_worker_init_sched_condition(unsigned sched_ctx_id, int workerid);
-
-void starpu_worker_deinit_sched_condition(unsigned sched_ctx_id, int workerid);
-
 #endif /* __STARPU_WORKER_H__ */
 

+ 5 - 5
src/core/sched_ctx.c

@@ -741,7 +741,7 @@ _starpu_pthread_mutex_t *_starpu_get_sched_mutex(struct _starpu_sched_ctx *sched
 		return NULL;
 }
 
-void starpu_worker_set_sched_condition(unsigned sched_ctx_id, int workerid, _starpu_pthread_mutex_t *sched_mutex, _starpu_pthread_cond_t *sched_cond)
+void starpu_sched_ctx_set_worker_mutex_and_cond(unsigned sched_ctx_id, int workerid, _starpu_pthread_mutex_t *sched_mutex, _starpu_pthread_cond_t *sched_cond)
 {
 	struct _starpu_sched_ctx *sched_ctx = _starpu_get_sched_ctx_struct(sched_ctx_id);
 	if(sched_ctx->sched_mutex && sched_ctx->sched_cond)
@@ -751,7 +751,7 @@ void starpu_worker_set_sched_condition(unsigned sched_ctx_id, int workerid, _sta
 	}
 }
 
-void starpu_worker_get_sched_condition(unsigned sched_ctx_id, int workerid, _starpu_pthread_mutex_t **sched_mutex, _starpu_pthread_cond_t **sched_cond)
+void starpu_sched_ctx_get_worker_mutex_and_cond(unsigned sched_ctx_id, int workerid, _starpu_pthread_mutex_t **sched_mutex, _starpu_pthread_cond_t **sched_cond)
 {
 	struct _starpu_sched_ctx *sched_ctx = _starpu_get_sched_ctx_struct(sched_ctx_id);
 	*sched_mutex = sched_ctx->sched_mutex[workerid];
@@ -763,12 +763,12 @@ void starpu_worker_get_sched_condition(unsigned sched_ctx_id, int workerid, _sta
 		struct _starpu_worker *workerarg = _starpu_get_worker_struct(workerid);
 		*sched_mutex = &workerarg->sched_mutex;
 		*sched_cond = &workerarg->sched_cond;
-		starpu_worker_set_sched_condition(sched_ctx_id, workerid, *sched_mutex, *sched_cond);
+		starpu_sched_ctx_set_worker_mutex_and_cond(sched_ctx_id, workerid, *sched_mutex, *sched_cond);
 	}
 
 }
 
-void starpu_worker_init_sched_condition(unsigned sched_ctx_id, int workerid)
+void starpu_sched_ctx_init_worker_mutex_and_cond(unsigned sched_ctx_id, int workerid)
 {
 	struct _starpu_sched_ctx *sched_ctx = _starpu_get_sched_ctx_struct(sched_ctx_id);
 	sched_ctx->sched_mutex[workerid] = (_starpu_pthread_mutex_t*)malloc(sizeof(_starpu_pthread_mutex_t));
@@ -777,7 +777,7 @@ void starpu_worker_init_sched_condition(unsigned sched_ctx_id, int workerid)
 	_STARPU_PTHREAD_COND_INIT(sched_ctx->sched_cond[workerid], NULL);
 }
 
-void starpu_worker_deinit_sched_condition(unsigned sched_ctx_id, int workerid)
+void starpu_sched_ctx_deinit_worker_mutex_and_cond(unsigned sched_ctx_id, int workerid)
 {
 	struct _starpu_sched_ctx *sched_ctx = _starpu_get_sched_ctx_struct(sched_ctx_id);
 	_STARPU_PTHREAD_MUTEX_DESTROY(sched_ctx->sched_mutex[workerid]);

+ 8 - 8
src/sched_policies/deque_modeling_policy_data_aware.c

@@ -230,7 +230,7 @@ static struct starpu_task *dmda_pop_every_task(unsigned sched_ctx_id)
 
 	pthread_mutex_t *sched_mutex;
 	pthread_cond_t *sched_cond;
-	starpu_worker_get_sched_condition(sched_ctx_id, workerid, &sched_mutex, &sched_cond);
+	starpu_sched_ctx_get_worker_mutex_and_cond(sched_ctx_id, workerid, &sched_mutex, &sched_cond);
 	new_list = _starpu_fifo_pop_every_task(fifo, sched_mutex, workerid);
 
 	while (new_list)
@@ -259,7 +259,7 @@ static int push_task_on_best_worker(struct starpu_task *task, int best_workerid,
 
 	pthread_mutex_t *sched_mutex;
 	pthread_cond_t *sched_cond;
-	starpu_worker_get_sched_condition(sched_ctx_id, best_workerid, &sched_mutex, &sched_cond);
+	starpu_sched_ctx_get_worker_mutex_and_cond(sched_ctx_id, best_workerid, &sched_mutex, &sched_cond);
 
 #ifdef STARPU_USE_SCHED_CTX_HYPERVISOR
 	starpu_call_pushed_task_cb(best_workerid, sched_ctx_id);
@@ -367,7 +367,7 @@ static int _dm_push_task(struct starpu_task *task, unsigned prio, unsigned sched
 			double exp_end;
 			pthread_mutex_t *sched_mutex;
 			pthread_cond_t *sched_cond;
-			starpu_worker_get_sched_condition(sched_ctx_id, worker, &sched_mutex, &sched_cond);
+			starpu_sched_ctx_get_worker_mutex_and_cond(sched_ctx_id, worker, &sched_mutex, &sched_cond);
 			
 			/* Sometimes workers didn't take the tasks as early as we expected */
 			_STARPU_PTHREAD_MUTEX_LOCK(sched_mutex);
@@ -485,7 +485,7 @@ static void compute_all_performance_predictions(struct starpu_task *task,
 			/* Sometimes workers didn't take the tasks as early as we expected */
 			pthread_mutex_t *sched_mutex;
 			pthread_cond_t *sched_cond;
-			starpu_worker_get_sched_condition(sched_ctx_id, worker, &sched_mutex, &sched_cond);
+			starpu_sched_ctx_get_worker_mutex_and_cond(sched_ctx_id, worker, &sched_mutex, &sched_cond);
 			_STARPU_PTHREAD_MUTEX_LOCK(sched_mutex);
 			fifo->exp_start = STARPU_MAX(fifo->exp_start, starpu_timing_now());
 			_STARPU_PTHREAD_MUTEX_UNLOCK(sched_mutex);
@@ -754,7 +754,7 @@ static void dmda_add_workers(unsigned sched_ctx_id, int *workerids, unsigned nwo
 	{
 		workerid = workerids[i];
 		dt->queue_array[workerid] = _starpu_create_fifo();
-		starpu_worker_init_sched_condition(sched_ctx_id, workerid);
+		starpu_sched_ctx_init_worker_mutex_and_cond(sched_ctx_id, workerid);
 	}
 }
 
@@ -768,7 +768,7 @@ static void dmda_remove_workers(unsigned sched_ctx_id, int *workerids, unsigned
 	{
 		workerid = workerids[i];
 		_starpu_destroy_fifo(dt->queue_array[workerid]);
-		starpu_worker_deinit_sched_condition(sched_ctx_id, workerid);
+		starpu_sched_ctx_deinit_worker_mutex_and_cond(sched_ctx_id, workerid);
 	}
 }
 
@@ -847,7 +847,7 @@ static void dmda_pre_exec_hook(struct starpu_task *task)
 
 	pthread_mutex_t *sched_mutex;
 	pthread_cond_t *sched_cond;
-	starpu_worker_get_sched_condition(sched_ctx_id, workerid, &sched_mutex, &sched_cond);
+	starpu_sched_ctx_get_worker_mutex_and_cond(sched_ctx_id, workerid, &sched_mutex, &sched_cond);
 	/* Once the task is executing, we can update the predicted amount
 	 * of work. */
 	_STARPU_PTHREAD_MUTEX_LOCK(sched_mutex);
@@ -871,7 +871,7 @@ static void dmda_push_task_notify(struct starpu_task *task, int workerid, unsign
 	double predicted_transfer = starpu_task_expected_data_transfer_time(memory_node, task);
 	pthread_mutex_t *sched_mutex;
 	pthread_cond_t *sched_cond;
-	starpu_worker_get_sched_condition(sched_ctx_id, workerid, &sched_mutex, &sched_cond);
+	starpu_sched_ctx_get_worker_mutex_and_cond(sched_ctx_id, workerid, &sched_mutex, &sched_cond);
 
 
 	/* Update the predictions */

+ 2 - 2
src/sched_policies/eager_central_policy.c

@@ -38,7 +38,7 @@ static void eager_add_workers(unsigned sched_ctx_id, int *workerids, unsigned nw
 	for (i = 0; i < nworkers; i++)
 	{
 		workerid = workerids[i];
-		starpu_worker_set_sched_condition(sched_ctx_id, workerid, &data->sched_mutex, &data->sched_cond);
+		starpu_sched_ctx_set_worker_mutex_and_cond(sched_ctx_id, workerid, &data->sched_mutex, &data->sched_cond);
 	}
 }
 
@@ -49,7 +49,7 @@ static void eager_remove_workers(unsigned sched_ctx_id, int *workerids, unsigned
 	for (i = 0; i < nworkers; i++)
 	{
 		workerid = workerids[i];
-		starpu_worker_set_sched_condition(sched_ctx_id, workerid, NULL, NULL);
+		starpu_sched_ctx_set_worker_mutex_and_cond(sched_ctx_id, workerid, NULL, NULL);
 	}
 }
 

+ 2 - 2
src/sched_policies/eager_central_priority_policy.c

@@ -83,7 +83,7 @@ static void eager_priority_add_workers(unsigned sched_ctx_id, int *workerids, un
 	for (i = 0; i < nworkers; i++)
 	{
 		workerid = workerids[i];
-		starpu_worker_set_sched_condition(sched_ctx_id, workerid, &data->sched_mutex, &data->sched_cond);
+		starpu_sched_ctx_set_worker_mutex_and_cond(sched_ctx_id, workerid, &data->sched_mutex, &data->sched_cond);
 	}
 }
 
@@ -94,7 +94,7 @@ static void eager_priority_remove_workers(unsigned sched_ctx_id, int *workerids,
 	for (i = 0; i < nworkers; i++)
 	{
 		workerid = workerids[i];
-		starpu_worker_set_sched_condition(sched_ctx_id, workerid, NULL, NULL);
+		starpu_sched_ctx_set_worker_mutex_and_cond(sched_ctx_id, workerid, NULL, NULL);
 	}	
 }
 

+ 6 - 6
src/sched_policies/heft.c

@@ -78,7 +78,7 @@ static void heft_add_workers(unsigned sched_ctx_id, int *workerids, unsigned nwo
 	{
 		workerid = workerids[i];
 		hd->queue_array[workerid] = _starpu_create_fifo();
-		starpu_worker_init_sched_condition(sched_ctx_id, workerid);
+		starpu_sched_ctx_init_worker_mutex_and_cond(sched_ctx_id, workerid);
 
 		current_time[workerid][sched_ctx_id] = 0.0;
 	}
@@ -94,7 +94,7 @@ static void heft_remove_workers(unsigned sched_ctx_id, int *workerids, unsigned
 	{
 		workerid = workerids[i];
 		_starpu_destroy_fifo(hd->queue_array[workerid]);
-		starpu_worker_deinit_sched_condition(sched_ctx_id, workerid);
+		starpu_sched_ctx_deinit_worker_mutex_and_cond(sched_ctx_id, workerid);
 		current_time[workerid][sched_ctx_id] = 0.0;
 	}
 }
@@ -150,7 +150,7 @@ static void heft_pre_exec_hook(struct starpu_task *task)
 
 	pthread_mutex_t *sched_mutex;
 	pthread_cond_t *sched_cond;
-	starpu_worker_get_sched_condition(sched_ctx_id, workerid, &sched_mutex, &sched_cond);
+	starpu_sched_ctx_get_worker_mutex_and_cond(sched_ctx_id, workerid, &sched_mutex, &sched_cond);
 	/* Once the task is executing, we can update the predicted amount
 	 * of work. */
 	_STARPU_PTHREAD_MUTEX_LOCK(sched_mutex);
@@ -175,7 +175,7 @@ static void heft_push_task_notify(struct starpu_task *task, int workerid)
 	double predicted_transfer = starpu_task_expected_data_transfer_time(memory_node, task);
 	pthread_mutex_t *sched_mutex;
 	pthread_cond_t *sched_cond;
-	starpu_worker_get_sched_condition(sched_ctx_id, workerid, &sched_mutex, &sched_cond);
+	starpu_sched_ctx_get_worker_mutex_and_cond(sched_ctx_id, workerid, &sched_mutex, &sched_cond);
 
 
 	/* Update the predictions */
@@ -227,7 +227,7 @@ static int push_task_on_best_worker(struct starpu_task *task, int best_workerid,
 
 	pthread_mutex_t *sched_mutex;
 	pthread_cond_t *sched_cond;
-	starpu_worker_get_sched_condition(sched_ctx_id, best_workerid, &sched_mutex, &sched_cond);
+	starpu_sched_ctx_get_worker_mutex_and_cond(sched_ctx_id, best_workerid, &sched_mutex, &sched_cond);
 
 #ifdef STARPU_USE_SCHED_CTX_HYPERVISOR
 	starpu_call_pushed_task_cb(best_workerid, sched_ctx_id);
@@ -341,7 +341,7 @@ static void compute_all_performance_predictions(struct starpu_task *task,
 				struct _starpu_fifo_taskq *fifo = hd->queue_array[worker];
 				pthread_mutex_t *sched_mutex;
 				pthread_cond_t *sched_cond;
-				starpu_worker_get_sched_condition(sched_ctx_id, worker, &sched_mutex, &sched_cond);
+				starpu_sched_ctx_get_worker_mutex_and_cond(sched_ctx_id, worker, &sched_mutex, &sched_cond);
 				_STARPU_PTHREAD_MUTEX_LOCK(sched_mutex);
 				fifo->exp_start = STARPU_MAX(fifo->exp_start, starpu_timing_now());
 				exp_end[worker_ctx][nimpl] = fifo->exp_start + fifo->exp_len;

+ 3 - 3
src/sched_policies/parallel_greedy.c

@@ -113,9 +113,9 @@ static void pgreedy_add_workers(unsigned sched_ctx_id, int *workerids, unsigned
 
 		/* All masters use the same condition/mutex */
 		if (master == workerid)
-			starpu_worker_set_sched_condition(sched_ctx_id, workerid, &data->sched_mutex, &data->sched_cond);
+			starpu_sched_ctx_set_worker_mutex_and_cond(sched_ctx_id, workerid, &data->sched_mutex, &data->sched_cond);
 		else
-			starpu_worker_set_sched_condition(sched_ctx_id, workerid, &data->master_sched_mutex[master], &data->master_sched_cond[master]);
+			starpu_sched_ctx_set_worker_mutex_and_cond(sched_ctx_id, workerid, &data->master_sched_mutex[master], &data->master_sched_cond[master]);
 	}
 
 #if 0
@@ -137,7 +137,7 @@ static void pgreedy_remove_workers(unsigned sched_ctx_id, int *workerids, unsign
         {
 		workerid = workerids[i];
 		_starpu_destroy_fifo(data->local_fifo[workerid]);
-		starpu_worker_set_sched_condition(sched_ctx_id, workerid, NULL, NULL);
+		starpu_sched_ctx_set_worker_mutex_and_cond(sched_ctx_id, workerid, NULL, NULL);
 		_STARPU_PTHREAD_MUTEX_DESTROY(&data->master_sched_mutex[workerid]);
 		_STARPU_PTHREAD_COND_DESTROY(&data->master_sched_cond[workerid]);
 	}

+ 6 - 6
src/sched_policies/parallel_heft.c

@@ -73,7 +73,7 @@ static void parallel_heft_pre_exec_hook(struct starpu_task *task)
 
 	pthread_mutex_t *sched_mutex;
 	pthread_cond_t *sched_cond;
-	starpu_worker_get_sched_condition(sched_ctx_id, workerid, &sched_mutex, &sched_cond);
+	starpu_sched_ctx_get_worker_mutex_and_cond(sched_ctx_id, workerid, &sched_mutex, &sched_cond);
 
 	/* Once we have executed the task, we can update the predicted amount
 	 * of work. */
@@ -108,7 +108,7 @@ static int push_task_on_best_worker(struct starpu_task *task, int best_workerid,
 		task->predicted_transfer = 0;
 		pthread_mutex_t *sched_mutex;
 		pthread_cond_t *sched_cond;
-		starpu_worker_get_sched_condition(sched_ctx_id, best_workerid, &sched_mutex, &sched_cond);
+		starpu_sched_ctx_get_worker_mutex_and_cond(sched_ctx_id, best_workerid, &sched_mutex, &sched_cond);
 
 		_STARPU_PTHREAD_MUTEX_LOCK(sched_mutex);
 		worker_exp_len[best_workerid] += task->predicted;
@@ -162,7 +162,7 @@ static int push_task_on_best_worker(struct starpu_task *task, int best_workerid,
 			alias->predicted_transfer = 0;
 			pthread_mutex_t *sched_mutex;
 			pthread_cond_t *sched_cond;
-			starpu_worker_get_sched_condition(sched_ctx_id, local_worker, &sched_mutex, &sched_cond);
+			starpu_sched_ctx_get_worker_mutex_and_cond(sched_ctx_id, local_worker, &sched_mutex, &sched_cond);
 			_STARPU_PTHREAD_MUTEX_LOCK(sched_mutex);
 			worker_exp_len[local_worker] += alias->predicted;
 			worker_exp_end[local_worker] = exp_end_predicted;
@@ -284,7 +284,7 @@ static int _parallel_heft_push_task(struct starpu_task *task, unsigned prio, uns
 		{
 			pthread_mutex_t *sched_mutex;
 			pthread_cond_t *sched_cond;
-			starpu_worker_get_sched_condition(sched_ctx_id, worker, &sched_mutex, &sched_cond);
+			starpu_sched_ctx_get_worker_mutex_and_cond(sched_ctx_id, worker, &sched_mutex, &sched_cond);
 			/* Sometimes workers didn't take the tasks as early as we expected */
 			_STARPU_PTHREAD_MUTEX_LOCK(sched_mutex);
 			worker_exp_start[worker] = STARPU_MAX(worker_exp_start[worker], starpu_timing_now());
@@ -503,7 +503,7 @@ static void parallel_heft_add_workers(unsigned sched_ctx_id, int *workerids, uns
 			workerarg->has_prev_init = 1;
 		}
 
-		starpu_worker_init_sched_condition(sched_ctx_id, workerid);
+		starpu_sched_ctx_init_worker_mutex_and_cond(sched_ctx_id, workerid);
 	}
 	_starpu_sched_find_worker_combinations(workerids, nworkers);
 
@@ -543,7 +543,7 @@ static void parallel_heft_remove_workers(unsigned sched_ctx_id, int *workerids,
 	for(i = 0; i < nworkers; i++)
 	{
 		worker = workerids[i];
-		starpu_worker_deinit_sched_condition(sched_ctx_id, worker);
+		starpu_sched_ctx_deinit_worker_mutex_and_cond(sched_ctx_id, worker);
 	}
 }
 static void initialize_parallel_heft_policy(unsigned sched_ctx_id) 

+ 2 - 2
src/sched_policies/random_policy.c

@@ -107,7 +107,7 @@ static void random_add_workers(unsigned sched_ctx_id, int *workerids, unsigned n
 	{
 		workerid = workerids[i];
 		struct _starpu_worker *workerarg = _starpu_get_worker_struct(workerid);
-		starpu_worker_set_sched_condition(sched_ctx_id, workerid, &workerarg->sched_mutex, &workerarg->sched_cond);
+		starpu_sched_ctx_set_worker_mutex_and_cond(sched_ctx_id, workerid, &workerarg->sched_mutex, &workerarg->sched_cond);
 	}
 }
 
@@ -118,7 +118,7 @@ static void random_remove_workers(unsigned sched_ctx_id, int *workerids, unsigne
 	for (i = 0; i < nworkers; i++)
 	{
 		workerid = workerids[i];
-		starpu_worker_set_sched_condition(sched_ctx_id, workerid, NULL, NULL);
+		starpu_sched_ctx_set_worker_mutex_and_cond(sched_ctx_id, workerid, NULL, NULL);
 	}
 
 }

+ 2 - 2
src/sched_policies/work_stealing_policy.c

@@ -358,7 +358,7 @@ static void ws_add_workers(unsigned sched_ctx_id, int *workerids,unsigned nworke
 		ws->queue_array[workerid]->nprocessed = -1;
 		ws->queue_array[workerid]->njobs = 0;
 
-		starpu_worker_set_sched_condition(sched_ctx_id, workerid, &ws->sched_mutex, &ws->sched_cond);
+		starpu_sched_ctx_set_worker_mutex_and_cond(sched_ctx_id, workerid, &ws->sched_mutex, &ws->sched_cond);
 	}
 }
 
@@ -373,7 +373,7 @@ static void ws_remove_workers(unsigned sched_ctx_id, int *workerids, unsigned nw
 	{
 		workerid = workerids[i];
 		_starpu_destroy_deque(ws->queue_array[workerid]);
-		starpu_worker_set_sched_condition(sched_ctx_id, workerid, NULL, NULL);
+		starpu_sched_ctx_set_worker_mutex_and_cond(sched_ctx_id, workerid, NULL, NULL);
 	}
 }