Selaa lähdekoodia

src: introduce new function _starpu_worker_get_id_check() which fails when called from outside a worker, otherwise returns starpu_worker_get_id()

Nathalie Furmento 9 vuotta sitten
vanhempi
commit
b7d6e2d4ce

+ 7 - 0
src/core/workers.c

@@ -1686,6 +1686,13 @@ int starpu_worker_get_id(void)
 }
 #define starpu_worker_get_id _starpu_worker_get_id
 
+unsigned _starpu_worker_get_id_check(void)
+{
+	int id = starpu_worker_get_id();
+	STARPU_ASSERT_MSG(id>=0, "Cannot be called from outside a worker\n");
+	return id;
+}
+
 int starpu_combined_worker_get_id(void)
 {
 	struct _starpu_worker *worker;

+ 3 - 1
src/core/workers.h

@@ -1,7 +1,7 @@
 /* StarPU --- Runtime system for heterogeneous multicore architectures.
  *
  * Copyright (C) 2009-2016  Université de Bordeaux
- * Copyright (C) 2010, 2011, 2012, 2013, 2014  CNRS
+ * Copyright (C) 2010, 2011, 2012, 2013, 2014, 2016  CNRS
  * Copyright (C) 2011  INRIA
  *
  * StarPU is free software; you can redistribute it and/or modify
@@ -558,5 +558,7 @@ static inline int _starpu_worker_get_id(void)
 }
 #define starpu_worker_get_id _starpu_worker_get_id
 
+/* Similar behaviour to starpu_worker_get_id() but fails when called from outside a worker */
+unsigned _starpu_worker_get_id_check(void);
 
 #endif // __WORKERS_H__

+ 5 - 3
src/sched_policies/component_best_implementation.c

@@ -1,6 +1,7 @@
 /* StarPU --- Runtime system for heterogeneous multicore architectures.
  *
  * Copyright (C) 2013  INRIA
+ * Copyright (C) 2016  CNRS
  * Copyright (C) 2013  Simon Archipoff
  *
  * StarPU is free software; you can redistribute it and/or modify
@@ -15,10 +16,11 @@
  * See the GNU Lesser General Public License in COPYING.LGPL for more details.
  */
 
+#include <float.h>
+
 #include <starpu_sched_component.h>
 #include <starpu_scheduler.h>
-
-#include <float.h>
+#include <core/workers.h>
 
 /* return true if workerid can execute task, and fill task->predicted and task->predicted_transfer
  *  according to best implementation predictions
@@ -100,7 +102,7 @@ static struct starpu_task * best_implementation_pull_task(struct starpu_sched_co
 	}
 	if(task)
 		/* this worker can execute this task as it was returned by a pop*/
-		(void)find_best_impl(component->tree->sched_ctx_id, task, starpu_worker_get_id());
+		(void)find_best_impl(component->tree->sched_ctx_id, task, _starpu_worker_get_id_check());
 	return task;
 }
 

+ 10 - 9
src/sched_policies/component_fifo.c

@@ -17,6 +17,7 @@
 
 #include <starpu_sched_component.h>
 #include <starpu_scheduler.h>
+#include <core/workers.h>
 
 #include "fifo_queues.h"
 #include "sched_component.h"
@@ -110,7 +111,7 @@ static int fifo_push_local_task(struct starpu_sched_component * component, struc
 	else
 		exp_len = fifo->exp_len;
 
-	if((data->ntasks_threshold != 0) && (data->exp_len_threshold != 0.0) && 
+	if((data->ntasks_threshold != 0) && (data->exp_len_threshold != 0.0) &&
 			((fifo->ntasks >= data->ntasks_threshold) || (exp_len >= data->exp_len_threshold)))
 	{
 		static int warned;
@@ -141,12 +142,12 @@ static int fifo_push_local_task(struct starpu_sched_component * component, struc
 		STARPU_ASSERT(!isnan(fifo->exp_end));
 		STARPU_ASSERT(!isnan(fifo->exp_len));
 		STARPU_ASSERT(!isnan(fifo->exp_start));
-		
+
 		if(!is_pushback)
 			component->can_pull(component);
 		STARPU_PTHREAD_MUTEX_UNLOCK(mutex);
 	}
-	
+
 	return ret;
 }
 
@@ -162,7 +163,7 @@ static struct starpu_task * fifo_pull_task(struct starpu_sched_component * compo
 	struct _starpu_fifo_taskq * fifo = data->fifo;
 	starpu_pthread_mutex_t * mutex = &data->mutex;
 	STARPU_PTHREAD_MUTEX_LOCK(mutex);
-	struct starpu_task * task = _starpu_fifo_pop_task(fifo, starpu_worker_get_id());
+	struct starpu_task * task = _starpu_fifo_pop_task(fifo, _starpu_worker_get_id_check());
 	if(task)
 	{
 		if(!isnan(task->predicted))
@@ -193,7 +194,7 @@ static struct starpu_task * fifo_pull_task(struct starpu_sched_component * compo
 				break;
 		}
 	}
-	
+
 	if(task)
 		return task;
 
@@ -216,18 +217,18 @@ static int fifo_can_push(struct starpu_sched_component * component)
 
 	struct starpu_task * task = starpu_sched_component_pull_task(component,NULL);
 	if(task)
-		ret = starpu_sched_component_push_task(NULL,child,task);	
-	while(task && !ret) 
+		ret = starpu_sched_component_push_task(NULL,child,task);
+	while(task && !ret)
 	{
 		if(!res)
 			res = 1;
 
 		task = starpu_sched_component_pull_task(component,NULL);
 		if(task)
-			ret = starpu_sched_component_push_task(NULL,child,task);	
+			ret = starpu_sched_component_push_task(NULL,child,task);
 	}
 	if(task && ret)
-		fifo_push_local_task(component,task,1); 
+		fifo_push_local_task(component,task,1);
 
 	return res;
 }

+ 2 - 1
src/sched_policies/component_sched.c

@@ -2,6 +2,7 @@
  *
  * Copyright (C) 2013  INRIA
  * Copyright (C) 2013  Simon Archipoff
+ * Copyright (C) 2016  CNRS
  *
  * 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
@@ -349,7 +350,7 @@ int starpu_sched_component_push_task(struct starpu_sched_component *from STARPU_
 
 struct starpu_task * starpu_sched_tree_pop_task(unsigned sched_ctx)
 {
-	int workerid = starpu_worker_get_id();
+	unsigned workerid = _starpu_worker_get_id_check();
 	struct starpu_sched_component * component = starpu_sched_component_worker_get(sched_ctx, workerid);
 
 	/* _starpu_sched_component_lock_worker(workerid) is called by component->pull_task()

+ 12 - 10
src/sched_policies/component_work_stealing.c

@@ -1,6 +1,7 @@
 /* StarPU --- Runtime system for heterogeneous multicore architectures.
  *
  * Copyright (C) 2013  INRIA
+ * Copyright (C) 2013  CNRS
  * Copyright (C) 2013  Simon Archipoff
  *
  * StarPU is free software; you can redistribute it and/or modify
@@ -15,11 +16,12 @@
  * See the GNU Lesser General Public License in COPYING.LGPL for more details.
  */
 
+#include <float.h>
+
 #include <starpu_sched_component.h>
 #include <starpu_scheduler.h>
 #include <starpu.h>
-
-#include <float.h>
+#include <core/workers.h>
 
 #include "prio_deque.h"
 
@@ -30,7 +32,7 @@ struct _starpu_work_stealing_data
  */
 	unsigned performed_total, last_pop_child, last_push_child;
 
-	struct _starpu_prio_deque ** fifos;	
+	struct _starpu_prio_deque ** fifos;
 	starpu_pthread_mutex_t ** mutexes;
 	int size;
 };
@@ -70,7 +72,7 @@ static struct starpu_task *  steal_task_round_robin(struct starpu_sched_componen
 			return NULL;
 		}
 		i = (i + 1) % component->nchildren;
-	
+
 	}
 	return task;
 }
@@ -118,7 +120,7 @@ static int is_worker_of_component(struct starpu_sched_component * component, int
 
 static struct starpu_task * pull_task(struct starpu_sched_component * component)
 {
-	int workerid = starpu_worker_get_id();
+	unsigned workerid = _starpu_worker_get_id_check();
 	int i;
 	for(i = 0; i < component->nchildren; i++)
 	{
@@ -145,7 +147,7 @@ static struct starpu_task * pull_task(struct starpu_sched_component * component)
 	{
 		return task;
 	}
-	
+
 	task  = steal_task(component, workerid);
 	if(task)
 	{
@@ -213,7 +215,7 @@ double _ws_estimated_load(struct starpu_sched_component * component)
 	{
 		speedup += starpu_worker_get_relative_speedup(starpu_worker_get_perf_archtype(workerid, component->tree->sched_ctx_id));
 	}
-	
+
 	return ntasks / speedup;
 }
 
@@ -256,14 +258,14 @@ int starpu_sched_tree_work_stealing_push_task(struct starpu_task *task)
 				if(is_worker_of_component(component->children[i], workerid))
 					break;
 			STARPU_ASSERT(i < component->nchildren);
-			
+
 			struct _starpu_work_stealing_data * wsd = component->data;
 			STARPU_PTHREAD_MUTEX_LOCK(wsd->mutexes[i]);
 			int ret = _starpu_prio_deque_push_task(wsd->fifos[i] , task);
 			if(ret == 0 && !isnan(task->predicted))
 				wsd->fifos[i]->exp_len += task->predicted;
 			STARPU_PTHREAD_MUTEX_UNLOCK(wsd->mutexes[i]);
-			
+
 			//we need to wake all workers
 			component->can_pull(component);
 			return ret;
@@ -313,7 +315,7 @@ void _ws_remove_child(struct starpu_sched_component * component, struct starpu_s
 	struct _starpu_prio_deque * tmp_fifo = wsd->fifos[i_component];
 	wsd->fifos[i_component] = wsd->fifos[component->nchildren - 1];
 
-	
+
 	component->children[i_component] = component->children[component->nchildren - 1];
 	component->nchildren--;
 	struct starpu_task * task;

+ 7 - 7
src/sched_policies/component_worker.c

@@ -150,8 +150,8 @@ static struct _starpu_task_grid * _starpu_task_grid_create(void)
 
 static struct _starpu_worker_task_list * _worker_get_list(unsigned sched_ctx_id)
 {
-	int workerid = starpu_worker_get_id();
-	STARPU_ASSERT(0 <= workerid && workerid < (int) starpu_worker_get_count());
+	unsigned workerid = _starpu_worker_get_id_check();
+	STARPU_ASSERT(workerid < (int) starpu_worker_get_count());
 	struct _starpu_worker_component_data * d = starpu_sched_component_worker_get(sched_ctx_id, workerid)->data;
 	return d->list;
 }
@@ -342,7 +342,7 @@ void _starpu_sched_component_unlock_worker(unsigned sched_ctx_id, int workerid)
  */
 static void _starpu_sched_component_worker_lock_scheduling(unsigned sched_ctx_id)
 {
-	int workerid = starpu_worker_get_id();
+	unsigned workerid = _starpu_worker_get_id_check();
 	starpu_pthread_mutex_t *sched_mutex;
 	starpu_pthread_cond_t *sched_cond;
 	starpu_worker_get_sched_condition(workerid, &sched_mutex, &sched_cond);
@@ -356,7 +356,7 @@ static void _starpu_sched_component_worker_lock_scheduling(unsigned sched_ctx_id
 
 static void _starpu_sched_component_worker_unlock_scheduling(unsigned sched_ctx_id)
 {
-	int workerid = starpu_worker_get_id();
+	unsigned workerid = _starpu_worker_get_id_check();
 	starpu_pthread_mutex_t *sched_mutex;
 	starpu_pthread_cond_t *sched_cond;
 	starpu_worker_get_sched_condition(workerid, &sched_mutex, &sched_cond);
@@ -439,7 +439,7 @@ static void simple_worker_can_pull(struct starpu_sched_component * worker_compon
 	if(_starpu_sched_component_worker_is_reset_status(worker_component))
 		_starpu_sched_component_worker_set_changed_status(worker_component);
 
-	if(w->workerid == starpu_worker_get_id())
+	if(w->workerid == _starpu_worker_get_id_check())
 	{
 		_starpu_sched_component_unlock_worker(worker_component->tree->sched_ctx_id, w->workerid);
 		return;
@@ -482,7 +482,7 @@ static int simple_worker_push_task(struct starpu_sched_component * component, st
 
 static struct starpu_task * simple_worker_pull_task(struct starpu_sched_component *component)
 {
-	int workerid = starpu_worker_get_id();
+	unsigned workerid = _starpu_worker_get_id_check();
 	struct _starpu_worker_component_data * data = component->data;
 	struct _starpu_worker_task_list * list = data->list;
 	STARPU_PTHREAD_MUTEX_LOCK(&list->mutex);
@@ -640,7 +640,7 @@ static void combined_worker_can_pull(struct starpu_sched_component * component)
 	(void) component;
 	STARPU_ASSERT(starpu_sched_component_is_combined_worker(component));
 	struct _starpu_worker_component_data * data = component->data;
-	int workerid = starpu_worker_get_id();
+	unsigned workerid = _starpu_worker_get_id_check();
 	int i;
 	for(i = 0; i < data->combined_worker->worker_size; i++)
 	{

+ 22 - 21
src/sched_policies/deque_modeling_policy_data_aware.c

@@ -1,7 +1,7 @@
 /* StarPU --- Runtime system for heterogeneous multicore architectures.
  *
  * Copyright (C) 2010-2016  Université de Bordeaux
- * Copyright (C) 2010, 2011, 2012, 2013, 2015  CNRS
+ * Copyright (C) 2010, 2011, 2012, 2013, 2015, 2016  CNRS
  * Copyright (C) 2011  Télécom-SudParis
  * Copyright (C) 2011-2012  INRIA
  *
@@ -24,6 +24,7 @@
 
 #include <common/fxt.h>
 #include <core/task.h>
+#include <core/workers.h>
 
 #include <sched_policies/fifo_queues.h>
 #include <limits.h>
@@ -184,7 +185,7 @@ static struct starpu_task *dmda_pop_ready_task(unsigned sched_ctx_id)
 
 	struct starpu_task *task;
 
-	int workerid = starpu_worker_get_id();
+	unsigned workerid = _starpu_worker_get_id_check();
 	struct _starpu_fifo_taskq *fifo = dt->queue_array[workerid];
 
 	unsigned node = starpu_worker_get_memory_node(workerid);
@@ -195,7 +196,7 @@ static struct starpu_task *dmda_pop_ready_task(unsigned sched_ctx_id)
 		/* We now start the transfer, get rid of it in the completion
 		 * prediction */
 		double transfer_model = task->predicted_transfer;
-		if(!isnan(transfer_model)) 
+		if(!isnan(transfer_model))
 		{
 			fifo->exp_len -= transfer_model;
 			fifo->exp_start = starpu_timing_now() + transfer_model;
@@ -207,7 +208,7 @@ static struct starpu_task *dmda_pop_ready_task(unsigned sched_ctx_id)
 				for(i = 0; i <= task_prio; i++)
 					fifo->exp_len_per_priority[i] -= transfer_model;
 			}
-			
+
 		}
 
 		starpu_sched_ctx_list_task_counters_decrement(sched_ctx_id, workerid);
@@ -233,7 +234,7 @@ static struct starpu_task *dmda_pop_task(unsigned sched_ctx_id)
 
 	struct starpu_task *task;
 
-	int workerid = starpu_worker_get_id();
+	unsigned workerid = _starpu_worker_get_id_check();
 	struct _starpu_fifo_taskq *fifo = dt->queue_array[workerid];
 
 	STARPU_ASSERT_MSG(fifo, "worker %d does not belong to ctx %d anymore.\n", workerid, sched_ctx_id);
@@ -245,7 +246,7 @@ static struct starpu_task *dmda_pop_task(unsigned sched_ctx_id)
 		/* We now start the transfer, get rid of it in the completion
 		 * prediction */
 
-		if(!isnan(transfer_model)) 
+		if(!isnan(transfer_model))
 		{
 			double model = task->predicted;
 			fifo->exp_len -= transfer_model;
@@ -262,7 +263,7 @@ static struct starpu_task *dmda_pop_task(unsigned sched_ctx_id)
 		}
 
 		starpu_sched_ctx_list_task_counters_decrement(sched_ctx_id, workerid);
-		  
+
 #ifdef STARPU_VERBOSE
 		if (task->cl)
 		{
@@ -270,7 +271,7 @@ static struct starpu_task *dmda_pop_task(unsigned sched_ctx_id)
 			if (non_ready == 0)
 				dt->ready_task_cnt++;
 		}
-		
+
 		dt->total_task_cnt++;
 #endif
 	}
@@ -284,7 +285,7 @@ static struct starpu_task *dmda_pop_every_task(unsigned sched_ctx_id)
 
 	struct starpu_task *new_list;
 
-	int workerid = starpu_worker_get_id();
+	unsigned workerid = _starpu_worker_get_id_check();
 	struct _starpu_fifo_taskq *fifo = dt->queue_array[workerid];
 
 	starpu_pthread_mutex_t *sched_mutex;
@@ -301,7 +302,7 @@ static struct starpu_task *dmda_pop_every_task(unsigned sched_ctx_id)
 		double transfer_model = new_list->predicted_transfer;
 		/* We now start the transfer, get rid of it in the completion
 		 * prediction */
-		if(!isnan(transfer_model)) 
+		if(!isnan(transfer_model))
 		{
 			fifo->exp_len -= transfer_model;
 			fifo->exp_start = starpu_timing_now() + transfer_model;
@@ -312,7 +313,7 @@ static struct starpu_task *dmda_pop_every_task(unsigned sched_ctx_id)
 				for(i = 0; i < new_list->priority; i++)
 					fifo->exp_len_per_priority[i] -= transfer_model;
 			}
-		
+
 		}
 
 		new_list = new_list->next;
@@ -366,7 +367,7 @@ static int push_task_on_best_worker(struct starpu_task *task, int best_workerid,
 		predicted_transfer = (starpu_timing_now() + predicted_transfer) - fifo->exp_end;
 	}
 
-	if(!isnan(predicted_transfer)) 
+	if(!isnan(predicted_transfer))
 	{
 		fifo->exp_end += predicted_transfer;
 		fifo->exp_len += predicted_transfer;
@@ -525,7 +526,7 @@ static int _dm_push_task(struct starpu_task *task, unsigned prio, unsigned sched
 			 * no performance model, or is not calibrated yet.
 			 */
 			if (ntasks_best == -1
-			
+
 			    /* Always compute the greedy decision, at least for
 			     * the tasks with no performance model. */
 			    || (!calibrating && ntasks_end < ntasks_best_end)
@@ -619,7 +620,7 @@ static void compute_all_performance_predictions(struct starpu_task *task,
 
 	starpu_task_bundle_t bundle = task->bundle;
 	struct _starpu_dmda_data *dt = (struct _starpu_dmda_data*)starpu_sched_ctx_get_policy_data(sched_ctx_id);
-	
+
 	if(sorted_decision && dt->num_priorities != -1)
 		task_prio = _normalize_prio(task->priority, dt->num_priorities, sched_ctx_id);
 
@@ -650,7 +651,7 @@ static void compute_all_performance_predictions(struct starpu_task *task,
 
 			int fifo_ntasks = fifo->ntasks;
 			double prev_exp_len = fifo->exp_len;
-			/* consider the priority of the task when deciding on which worker to schedule, 
+			/* consider the priority of the task when deciding on which worker to schedule,
 			   compute the expected_end of the task if it is inserted before other tasks already scheduled */
 			if(sorted_decision)
 			{
@@ -669,7 +670,7 @@ static void compute_all_performance_predictions(struct starpu_task *task,
 					STARPU_PTHREAD_MUTEX_UNLOCK_SCHED(sched_mutex);
 				}
 			}
-				
+
 			exp_end[worker_ctx][nimpl] = exp_start + prev_exp_len;
 			if (exp_end[worker_ctx][nimpl] > max_exp_end)
 				max_exp_end = exp_end[worker_ctx][nimpl];
@@ -811,8 +812,8 @@ static double _dmda_push_task(struct starpu_task *task, unsigned prio, unsigned
 					    local_power,
 					    &forced_best,
 					    &forced_impl, sched_ctx_id, sorted_decision);
-	
-	
+
+
 	double best_fitness = -1;
 
 	unsigned nimpl;
@@ -1014,7 +1015,7 @@ static void initialize_dmda_policy(unsigned sched_ctx_id)
 
 	if(starpu_sched_ctx_min_priority_is_set(sched_ctx_id) != 0 && starpu_sched_ctx_max_priority_is_set(sched_ctx_id) != 0)
 		dt->num_priorities = starpu_sched_ctx_get_max_priority(sched_ctx_id) - starpu_sched_ctx_get_min_priority(sched_ctx_id) + 1;
-	else 
+	else
 		dt->num_priorities = -1;
 
 
@@ -1058,7 +1059,7 @@ static void deinitialize_dmda_policy(unsigned sched_ctx_id)
 static void dmda_pre_exec_hook(struct starpu_task *task)
 {
 	unsigned sched_ctx_id = starpu_sched_ctx_get_ctx_for_task(task);
-	int workerid = starpu_worker_get_id();
+	unsigned workerid = _starpu_worker_get_id_check();
 	struct _starpu_dmda_data *dt = (struct _starpu_dmda_data*)starpu_sched_ctx_get_policy_data(sched_ctx_id);
 	struct _starpu_fifo_taskq *fifo = dt->queue_array[workerid];
 	double model = task->predicted;
@@ -1171,7 +1172,7 @@ static void dmda_post_exec_hook(struct starpu_task * task)
 {
 	unsigned sched_ctx_id = starpu_sched_ctx_get_ctx_for_task(task);
 	struct _starpu_dmda_data *dt = (struct _starpu_dmda_data*)starpu_sched_ctx_get_policy_data(sched_ctx_id);
-	int workerid = starpu_worker_get_id();
+	unsigned workerid = _starpu_worker_get_id_check();
 	struct _starpu_fifo_taskq *fifo = dt->queue_array[workerid];
 	starpu_pthread_mutex_t *sched_mutex;
 	starpu_pthread_cond_t *sched_cond;

+ 5 - 4
src/sched_policies/eager_central_policy.c

@@ -1,7 +1,7 @@
 /* StarPU --- Runtime system for heterogeneous multicore architectures.
  *
  * Copyright (C) 2010-2016  Université de Bordeaux
- * Copyright (C) 2010-2013  CNRS
+ * Copyright (C) 2010-2013, 2016  CNRS
  * Copyright (C) 2011  INRIA
  *
  * StarPU is free software; you can redistribute it and/or modify
@@ -25,6 +25,7 @@
 #include <sched_policies/fifo_queues.h>
 #include <common/thread.h>
 #include <starpu_bitmap.h>
+#include <core/workers.h>
 
 struct _starpu_eager_center_policy_data
 {
@@ -136,7 +137,7 @@ static int push_task_eager_policy(struct starpu_task *task)
 static struct starpu_task *pop_every_task_eager_policy(unsigned sched_ctx_id)
 {
 	struct _starpu_eager_center_policy_data *data = (struct _starpu_eager_center_policy_data*)starpu_sched_ctx_get_policy_data(sched_ctx_id);
-	int workerid = starpu_worker_get_id();;
+	unsigned workerid = _starpu_worker_get_id_check();
 	STARPU_PTHREAD_MUTEX_LOCK(&data->policy_mutex);
 	struct starpu_task* task = _starpu_fifo_pop_every_task(data->fifo, workerid);
 	STARPU_PTHREAD_MUTEX_UNLOCK(&data->policy_mutex);
@@ -149,7 +150,7 @@ static struct starpu_task *pop_every_task_eager_policy(unsigned sched_ctx_id)
 static struct starpu_task *pop_task_eager_policy(unsigned sched_ctx_id)
 {
 	struct starpu_task *chosen_task = NULL;
-	unsigned workerid = starpu_worker_get_id();
+	unsigned workerid = _starpu_worker_get_id_check();
 	struct _starpu_eager_center_policy_data *data = (struct _starpu_eager_center_policy_data*)starpu_sched_ctx_get_policy_data(sched_ctx_id);
 
 	/* block until some event happens */
@@ -199,7 +200,7 @@ static void eager_add_workers(unsigned sched_ctx_id, int *workerids, unsigned nw
 	for (i = 0; i < nworkers; i++)
 	{
 		workerid = workerids[i];
-		int curr_workerid = starpu_worker_get_id();
+		unsigned curr_workerid = _starpu_worker_get_id_check();
 		if(workerid != curr_workerid)
 			starpu_wake_worker(workerid);
 

+ 10 - 9
src/sched_policies/eager_central_priority_policy.c

@@ -1,7 +1,7 @@
 /* StarPU --- Runtime system for heterogeneous multicore architectures.
  *
  * Copyright (C) 2010-2016  Université de Bordeaux
- * Copyright (C) 2010, 2011, 2012, 2013, 2015  CNRS
+ * Copyright (C) 2010, 2011, 2012, 2013, 2015, 2016  CNRS
  * Copyright (C) 2011  INRIA
  *
  * StarPU is free software; you can redistribute it and/or modify
@@ -26,6 +26,7 @@
 #include <starpu_bitmap.h>
 
 #include <common/fxt.h>
+#include <core/workers.h>
 
 #define DEFAULT_MIN_LEVEL	(-5)
 #define DEFAULT_MAX_LEVEL	(+5)
@@ -125,7 +126,7 @@ static int _starpu_priority_push_task(struct starpu_task *task)
 
 	STARPU_PTHREAD_MUTEX_LOCK(&data->policy_mutex);
 	unsigned priolevel = task->priority - STARPU_MIN_PRIO;
-	
+
 	starpu_task_list_push_back(&taskq->taskq[priolevel], task);
 	taskq->ntasks[priolevel]++;
 	taskq->total_ntasks++;
@@ -135,7 +136,7 @@ static int _starpu_priority_push_task(struct starpu_task *task)
 	/* wake people waiting for a task */
 	unsigned worker = 0;
 	struct starpu_worker_collection *workers = starpu_sched_ctx_get_worker_collection(sched_ctx_id);
-	
+
 	struct starpu_sched_ctx_iterator it;
 #ifndef STARPU_NON_BLOCKING_DRIVERS
 	char dowake[STARPU_NMAXWORKERS] = { 0 };
@@ -188,7 +189,7 @@ static int _starpu_priority_push_task(struct starpu_task *task)
 static struct starpu_task *_starpu_priority_pop_task(unsigned sched_ctx_id)
 {
 	struct starpu_task *chosen_task = NULL, *task, *nexttask;
-	unsigned workerid = starpu_worker_get_id();
+	unsigned workerid = _starpu_worker_get_id_check();
 	int skipped = 0;
 
 	struct _starpu_eager_central_prio_data *data = (struct _starpu_eager_central_prio_data*)starpu_sched_ctx_get_policy_data(sched_ctx_id);
@@ -213,8 +214,8 @@ static struct starpu_task *_starpu_priority_pop_task(unsigned sched_ctx_id)
 	starpu_pthread_cond_t *curr_sched_cond;
 	starpu_worker_get_sched_condition(workerid, &curr_sched_mutex, &curr_sched_cond);
 	STARPU_PTHREAD_MUTEX_UNLOCK_SCHED(curr_sched_mutex);
-	
-	/* all workers will block on this mutex anyway so 
+
+	/* all workers will block on this mutex anyway so
 	   there's no need for their own mutex to be locked */
 	STARPU_PTHREAD_MUTEX_LOCK(&data->policy_mutex);
 
@@ -225,7 +226,7 @@ static struct starpu_task *_starpu_priority_pop_task(unsigned sched_ctx_id)
 		{
 			for (task  = starpu_task_list_begin(&taskq->taskq[priolevel]);
 			     task != starpu_task_list_end(&taskq->taskq[priolevel]) && !chosen_task;
-			     task  = nexttask) 
+			     task  = nexttask)
 			{
 				unsigned nimpl;
 				nexttask = starpu_task_list_next(task);
@@ -268,7 +269,7 @@ static struct starpu_task *_starpu_priority_pop_task(unsigned sched_ctx_id)
 #endif
 			}
 		}
-	
+
 	}
 
 	if (!chosen_task)
@@ -305,7 +306,7 @@ static void eager_center_priority_add_workers(unsigned sched_ctx_id, int *worker
         for (i = 0; i < nworkers; i++)
         {
 		workerid = workerids[i];
-		int curr_workerid = starpu_worker_get_id();
+		unsigned curr_workerid = _starpu_worker_get_id_check();
 		if(workerid != curr_workerid)
 			starpu_wake_worker(workerid);
 

+ 3 - 2
src/sched_policies/graph_test_policy.c

@@ -1,7 +1,7 @@
 /* StarPU --- Runtime system for heterogeneous multicore architectures.
  *
  * Copyright (C) 2010-2016  Université de Bordeaux
- * Copyright (C) 2010-2013  CNRS
+ * Copyright (C) 2010-2013, 2016  CNRS
  * Copyright (C) 2011  INRIA
  *
  * StarPU is free software; you can redistribute it and/or modify
@@ -34,6 +34,7 @@
 #include <common/thread.h>
 #include <starpu_bitmap.h>
 #include <core/task.h>
+#include <core/workers.h>
 
 struct _starpu_graph_test_policy_data
 {
@@ -282,7 +283,7 @@ static int push_task_graph_test_policy(struct starpu_task *task)
 static struct starpu_task *pop_task_graph_test_policy(unsigned sched_ctx_id)
 {
 	struct starpu_task *chosen_task = NULL;
-	unsigned workerid = starpu_worker_get_id();
+	unsigned workerid = _starpu_worker_get_id_check();
 	struct _starpu_graph_test_policy_data *data = (struct _starpu_graph_test_policy_data*)starpu_sched_ctx_get_policy_data(sched_ctx_id);
 	struct _starpu_prio_deque *prio;
 

+ 3 - 1
src/sched_policies/heteroprio.c

@@ -1,6 +1,7 @@
 /* StarPU --- Runtime system for heterogeneous multicore architectures.
  *
  * Copyright (C) 2015  INRIA
+ * Copyright (C) 2016  CNRS
  *
  * 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
@@ -22,6 +23,7 @@
 
 #include <common/fxt.h>
 #include <core/task.h>
+#include <core/workers.h>
 
 #include <sched_policies/fifo_queues.h>
 #include <limits.h>
@@ -446,7 +448,7 @@ static int push_task_heteroprio_policy(struct starpu_task *task)
 
 static struct starpu_task *pop_task_heteroprio_policy(unsigned sched_ctx_id)
 {
-	const unsigned workerid = starpu_worker_get_id();
+	const unsigned workerid = _starpu_worker_get_id_check();
 	struct _starpu_heteroprio_data *hp = (struct _starpu_heteroprio_data*)starpu_sched_ctx_get_policy_data(sched_ctx_id);
 	struct _heteroprio_worker_wrapper* worker = &hp->workers_heteroprio[workerid];
 

+ 2 - 1
src/sched_policies/parallel_eager.c

@@ -3,6 +3,7 @@
  * Copyright (C) 2011-2016  Université de Bordeaux
  * Copyright (C) 2011  Télécom-SudParis
  * Copyright (C) 2011-2013  INRIA
+ * Copyright (C) 2016       CNRS
  *
  * 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
@@ -181,7 +182,7 @@ static struct starpu_task *pop_task_peager_policy(unsigned sched_ctx_id)
 {
 	struct _starpu_peager_data *data = (struct _starpu_peager_data*)starpu_sched_ctx_get_policy_data(sched_ctx_id);
 
-	int workerid = starpu_worker_get_id();
+	unsigned workerid = _starpu_worker_get_id_check();
 
 	/* If this is not a CPU or a MIC, then the worker simply grabs tasks from the fifo */
 	if (starpu_worker_get_type(workerid) != STARPU_CPU_WORKER && starpu_worker_get_type(workerid) != STARPU_MIC_WORKER)

+ 2 - 1
src/sched_policies/parallel_heft.c

@@ -3,6 +3,7 @@
  * Copyright (C) 2012 INRIA
  * Copyright (C) 2010-2016  Université de Bordeaux
  * Copyright (C) 2011  Télécom-SudParis
+ * Copyright (C) 2016  CNRS
  *
  * 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
@@ -75,7 +76,7 @@ static void parallel_heft_pre_exec_hook(struct starpu_task *task)
 	if (!task->cl || task->execute_on_a_specific_worker)
 		return;
 
-	int workerid = starpu_worker_get_id();
+	unsigned workerid = _starpu_worker_get_id_check();
 	double model = task->predicted;
 	double transfer_model = task->predicted_transfer;
 

+ 2 - 4
src/sched_policies/work_stealing_policy.c

@@ -1,7 +1,7 @@
 /* StarPU --- Runtime system for heterogeneous multicore architectures.
  *
  * Copyright (C) 2010-2016  Université de Bordeaux
- * Copyright (C) 2010, 2011, 2012, 2013  CNRS
+ * Copyright (C) 2010, 2011, 2012, 2013, 2016  CNRS
  * Copyright (C) 2011, 2012  INRIA
  *
  * StarPU is free software; you can redistribute it and/or modify
@@ -525,9 +525,7 @@ static struct starpu_task *ws_pop_task(unsigned sched_ctx_id)
 	struct _starpu_work_stealing_data *ws = (struct _starpu_work_stealing_data*)starpu_sched_ctx_get_policy_data(sched_ctx_id);
 
 	struct starpu_task *task;
-	int workerid = starpu_worker_get_id();
-
-	STARPU_ASSERT(workerid != -1);
+	int workerid = _starpu_worker_get_id_check();
 
 	task = ws_pick_task(workerid, workerid, sched_ctx_id);
 	if (task)