Andra Hugo 12 gadi atpakaļ
vecāks
revīzija
e020608b29

+ 1 - 0
include/starpu.h

@@ -101,6 +101,7 @@ void starpu_shutdown(void);
  * StarPU tasks). The returned value should be at most STARPU_NMAXWORKERS. */
 unsigned starpu_worker_get_count(void);
 unsigned starpu_combined_worker_get_count(void);
+unsigned starpu_worker_is_combined_worker(int id);
 
 unsigned starpu_cpu_worker_get_count(void);
 unsigned starpu_cuda_worker_get_count(void);

+ 2 - 0
src/core/combined_workers.c

@@ -54,8 +54,10 @@ static void sort_workerid_array(int nworkers, int workerid_array[])
  * workerid_array array which has nworkers entries. This function returns
  * the identifier of the combined worker in case of success, a negative value
  * is returned otherwise. */
+
 int starpu_combined_worker_assign_workerid(int nworkers, int workerid_array[])
 {
+	unsigned sched_ctx_id = starpu_get_sched_ctx();
 	int new_workerid;
 
 	/* Return the number of actual workers. */

+ 1 - 0
src/core/sched_ctx.c

@@ -176,6 +176,7 @@ static void _starpu_remove_workers_from_sched_ctx(struct _starpu_sched_ctx *sche
 	return;
 }
 
+
 struct _starpu_sched_ctx*  _starpu_create_sched_ctx(const char *policy_name, int *workerids, 
 				  int nworkers_ctx, unsigned is_initial_sched,
 				  const char *sched_name)

+ 36 - 20
src/core/sched_policy.c

@@ -471,34 +471,50 @@ struct starpu_task *_starpu_create_conversion_task(starpu_data_handle_t handle,
 
 struct _starpu_sched_ctx* _get_next_sched_ctx_to_pop_into(struct _starpu_worker *worker)
 {
-	struct _starpu_sched_ctx *sched_ctx, *good_sched_ctx = NULL;
-	int smallest_counter =  worker->nsched_ctxs;
-	unsigned i;
-	for(i = 0; i < STARPU_NMAX_SCHED_CTXS; i++)
-	{
-		sched_ctx = worker->sched_ctx[i];
+	double max_time_on_ctx = starpu_get_max_time_worker_on_ctx();
+/* 	if(max_time_on_ctx != -1.0 && starpu_are_overlapping_ctxs_on_worker(worker->workerid)) */
+/* 	{ */
+/* 		unsigned current_active_ctx = worker->active_ctx; */
+/* //		current_time[worker->workerid][current_active_ctx] += predicted; */
 		
-		if(sched_ctx != NULL && sched_ctx->id != STARPU_NMAX_SCHED_CTXS && 
-		   sched_ctx->pop_counter[worker->workerid] < worker->nsched_ctxs && 
-		   smallest_counter > sched_ctx->pop_counter[worker->workerid])
-		{
-			good_sched_ctx = sched_ctx;
-			smallest_counter = sched_ctx->pop_counter[worker->workerid];
-		}
-	}
-
-	if(good_sched_ctx == NULL)
+/* 		if(current_time[worker->workerid][current_active_ctx] >= max_time_on_ctx) */
+/* 		{ */
+/* 			current_time[worker->workerid][current_active_ctx] = 0.0; */
+/* 			starpu_set_turn_to_other_ctx(worker->workerid, current_active_ctx); */
+/* 		} */
+/* 		return worker->active_ctx; */
+/* 	} */
+/* 	else */
 	{
+		struct _starpu_sched_ctx *sched_ctx, *good_sched_ctx = NULL;
+		int smallest_counter =  worker->nsched_ctxs;
+		unsigned i;
 		for(i = 0; i < STARPU_NMAX_SCHED_CTXS; i++)
 		{
 			sched_ctx = worker->sched_ctx[i];
-			if(sched_ctx != NULL && sched_ctx->id != STARPU_NMAX_SCHED_CTXS)
-				sched_ctx->pop_counter[worker->workerid] = 0;
+			
+			if(sched_ctx != NULL && sched_ctx->id != STARPU_NMAX_SCHED_CTXS && 
+			   sched_ctx->pop_counter[worker->workerid] < worker->nsched_ctxs && 
+			   smallest_counter > sched_ctx->pop_counter[worker->workerid])
+			{
+				good_sched_ctx = sched_ctx;
+				smallest_counter = sched_ctx->pop_counter[worker->workerid];
+			}
 		}
 		
-		return _get_next_sched_ctx_to_pop_into(worker);
+		if(good_sched_ctx == NULL)
+		{
+			for(i = 0; i < STARPU_NMAX_SCHED_CTXS; i++)
+			{
+				sched_ctx = worker->sched_ctx[i];
+				if(sched_ctx != NULL && sched_ctx->id != STARPU_NMAX_SCHED_CTXS)
+					sched_ctx->pop_counter[worker->workerid] = 0;
+			}
+			
+			return _get_next_sched_ctx_to_pop_into(worker);
+		}
+		return good_sched_ctx;
 	}
-	return good_sched_ctx;
 }
 
 struct starpu_task *_starpu_pop_task(struct _starpu_worker *worker)

+ 5 - 0
src/core/workers.c

@@ -858,6 +858,11 @@ struct _starpu_worker *_starpu_get_worker_struct(unsigned id)
 	return &config.workers[id];
 }
 
+unsigned starpu_worker_is_combined_worker(int id)
+{
+	return id >= (int)config.topology.nworkers;
+}
+
 struct _starpu_sched_ctx *_starpu_get_sched_ctx_struct(unsigned id)
 {
         STARPU_ASSERT(id <= STARPU_NMAX_SCHED_CTXS);

+ 70 - 37
src/sched_policies/detect_combined_workers.c

@@ -277,8 +277,15 @@ static unsigned find_and_assign_combinations_with_hwloc_recursive(struct _starpu
     {
 	gather_trees(tree, subtrees, obj->arity);
 
-	int ret = starpu_combined_worker_assign_workerid(nb_workers, workers);
-	STARPU_ASSERT(ret >= 0);
+	unsigned sched_ctx_id = starpu_get_sched_ctx();
+	int i;
+	for(i = 0; i < nb_workers; i++)
+		if(!starpu_worker_belongs_to_sched_ctx(workers[i], sched_ctx_id))
+			return 0;
+	struct worker_collection* workers_coll = starpu_get_worker_collection_of_sched_ctx(sched_ctx_id);
+	int newworkerid = starpu_combined_worker_assign_workerid(nb_workers, workers);
+	STARPU_ASSERT(newworkerid >= 0);
+	workers_coll->add(workers_coll, newworkerid);
 	nb_workers = 0;
     }
 
@@ -361,7 +368,7 @@ static void get_min_max_sizes(unsigned int *min_size, unsigned int *max_size, st
 
 /* find_and_assign_combinations_with_hwloc
  * =======================================
- * Purpose
+ * * Purpose
  * =======
  * Launches find_and_assign_combinations_with_hwloc_recursive function on the root
  * of the hwloc tree to gather and assign combined cpu workers in an efficient manner.
@@ -376,8 +383,17 @@ static void get_min_max_sizes(unsigned int *min_size, unsigned int *max_size, st
  *			to get the hwloc tree.
  */
 
-static void find_and_assign_combinations_with_hwloc(struct starpu_machine_topology *topology)
+static void find_and_assign_combinations_with_hwloc(int *workerids, int nworkers)
 {
+    struct _starpu_machine_config *config = _starpu_get_machine_config();
+    struct starpu_machine_topology *topology = &config->topology;
+
+    unsigned sched_ctx_id  = starpu_get_sched_ctx();
+    if(sched_ctx_id == STARPU_NMAX_SCHED_CTXS)
+	    sched_ctx_id = 0; 
+
+    struct worker_collection* workers = starpu_get_worker_collection_of_sched_ctx(sched_ctx_id);
+
     unsigned nb_workers;
     unsigned int min_size, max_size;
 
@@ -404,8 +420,9 @@ static void find_and_assign_combinations_with_hwloc(struct starpu_machine_topolo
 	 * while there are enough workers to assign regarding the min_size value */
 	STARPU_ASSERT(nb_workers <= max_size);
 
-	int ret = starpu_combined_worker_assign_workerid(nb_workers, tree.workers);
-	STARPU_ASSERT(ret >= 0);
+	int newworkerid = starpu_combined_worker_assign_workerid(nb_workers, tree.workers);
+	STARPU_ASSERT(newworkerid >= 0);
+	workers->add(workers, newworkerid);
     }
 
     free(tree.workers);
@@ -413,19 +430,27 @@ static void find_and_assign_combinations_with_hwloc(struct starpu_machine_topolo
 
 #else /* STARPU_HAVE_HWLOC */
 
-static void find_and_assign_combinations_without_hwloc(struct starpu_machine_topology *topology)
+static void find_and_assign_combinations_without_hwloc(int *workerids, int nworkers)
 {
-    struct _starpu_machine_config *config = _starpu_get_machine_config();
+    unsigned sched_ctx_id  = starpu_get_sched_ctx();
+    if(sched_ctx_id == STARPU_NMAX_SCHED_CTXS)
+	    sched_ctx_id = 0; 
+
+    struct worker_collection* workers = starpu_get_worker_collection_of_sched_ctx(sched_ctx_id);
+
 
     /* We put the id of all CPU workers in this array */
     int cpu_workers[STARPU_NMAXWORKERS];
     unsigned ncpus = 0;
 
+    struct _starpu_worker *worker;
     unsigned i;
-    for (i = 0; i < topology->nworkers; i++)
+    for (i = 0; i < nworkers; i++)
     {
-	if (config->workers[i].perf_arch == STARPU_CPU_DEFAULT)
-	    cpu_workers[ncpus++] = i;
+	    worker = _starpu_get_worker_struct(workerids[i]);
+	   
+	    if (worker.perf_arch == STARPU_CPU_DEFAULT)
+		    cpu_workers[ncpus++] = i;
     }
 
     unsigned size;
@@ -442,9 +467,10 @@ static void find_and_assign_combinations_without_hwloc(struct starpu_machine_top
 		    workerids[i] = cpu_workers[first_cpu + i];
 
 		/* We register this combination */
-		int ret;
-		ret = starpu_combined_worker_assign_workerid(size, workerids);
-		STARPU_ASSERT(ret >= 0);
+		int newworkerid;
+		newworkerid = starpu_combined_worker_assign_workerid(size, workerids);
+		STARPU_ASSERT(newworkerid >= 0);
+		workers->add(workers, bewworkerid);
 	    }
 	}
     }
@@ -452,40 +478,47 @@ static void find_and_assign_combinations_without_hwloc(struct starpu_machine_top
 
 #endif /* STARPU_HAVE_HWLOC */
 
-static void combine_all_cpu_workers(struct starpu_machine_topology *topology)
-{
-    struct _starpu_machine_config *config = _starpu_get_machine_config();
 
-    int cpu_workers[STARPU_NMAXWORKERS];
-    unsigned ncpus = 0;
-
-    unsigned i;
-    for (i = 0; i < topology->nworkers; i++)
-    {
-	if (config->workers[i].perf_arch == STARPU_CPU_DEFAULT)
-	    cpu_workers[ncpus++] = i;
-    }
-
-    for (i = 1; i <= ncpus; i++)
-    {
-	int ret;
-	ret = starpu_combined_worker_assign_workerid(i, cpu_workers);
-	STARPU_ASSERT(ret >= 0);
-    }
+static void combine_all_cpu_workers(int *workerids, int nworkers)
+{
+	unsigned sched_ctx_id  = starpu_get_sched_ctx();
+	if(sched_ctx_id == STARPU_NMAX_SCHED_CTXS)
+		sched_ctx_id = 0;
+	struct worker_collection* workers = starpu_get_worker_collection_of_sched_ctx(sched_ctx_id);
+	int cpu_workers[STARPU_NMAXWORKERS];
+	unsigned ncpus = 0;
+	struct _starpu_worker *worker;
+	unsigned i;
+	for (i = 0; i < nworkers; i++)
+	{
+		worker = _starpu_get_worker_struct(workerids[i]);
+		
+		if (worker->perf_arch == STARPU_CPU_DEFAULT)
+			cpu_workers[ncpus++] = workerids[i];
+	}
+	
+	for (i = 1; i <= ncpus; i++)
+	{
+		int newworkerid;
+		newworkerid = starpu_combined_worker_assign_workerid(i, cpu_workers);
+		STARPU_ASSERT(newworkerid >= 0);
+		workers->add(workers, newworkerid);
+	}
 }
 
-void _starpu_sched_find_worker_combinations(struct starpu_machine_topology *topology)
+void _starpu_sched_find_worker_combinations(int *workerids, int nworkers)
 {
     struct _starpu_machine_config *config = _starpu_get_machine_config();
 
+    
     if (config->conf->single_combined_worker > 0)
-	combine_all_cpu_workers(topology);
+	    combine_all_cpu_workers(workerids, nworkers);
     else
     {
 #ifdef STARPU_HAVE_HWLOC
-	find_and_assign_combinations_with_hwloc(topology);
+	    find_and_assign_combinations_with_hwloc(workerids, nworkers);
 #else
-	find_and_assign_combinations_without_hwloc(topology);
+	    find_and_assign_combinations_without_hwloc(workerids, nworkers);
 #endif
     }
 }

+ 1 - 1
src/sched_policies/detect_combined_workers.h

@@ -17,5 +17,5 @@
 #include <starpu.h>
 
 /* Initialize combined workers */
-void _starpu_sched_find_worker_combinations(struct starpu_machine_topology *topology);
+void _starpu_sched_find_worker_combinations(int *workerids, int nworkers);
 

+ 1 - 1
src/sched_policies/heft.c

@@ -479,7 +479,7 @@ static int _heft_push_task(struct starpu_task *task, unsigned prio, unsigned sch
 	 *	and detect if there is some calibration that needs to be done.
 	 */
 
-	starpu_task_bundle_t bundle = NULL; //task->bundle;
+	starpu_task_bundle_t bundle = task->bundle;
 
 	if(workers->init_cursor)
 		workers->init_cursor(workers);

+ 1 - 1
src/sched_policies/parallel_greedy.c

@@ -50,7 +50,7 @@ static void pgreedy_add_workers(unsigned sched_ctx_id, int *workerids, unsigned
 	struct _starpu_machine_config *config = _starpu_get_machine_config();
 	struct starpu_machine_topology *topology = &config->topology;
 
-	_starpu_sched_find_worker_combinations(topology);
+	_starpu_sched_find_worker_combinations(workerids, nworkers);
 
 	unsigned workerid, i;
 	unsigned ncombinedworkers;

+ 46 - 48
src/sched_policies/parallel_heft.c

@@ -34,7 +34,7 @@
 #define DBL_MAX __DBL_MAX__
 #endif
 
-static unsigned ncombinedworkers;
+//static unsigned ncombinedworkers;
 //static enum starpu_perf_archtype applicable_perf_archtypes[STARPU_NARCH_VARIATIONS];
 //static unsigned napplicable_perf_archtypes = 0;
 
@@ -93,9 +93,6 @@ static int push_task_on_best_worker(struct starpu_task *task, int best_workerid,
 	pheft_data *hd = (pheft_data*)starpu_get_sched_ctx_policy_data(sched_ctx_id);
 
 	/* Is this a basic worker or a combined worker ? */
-	int nbasic_workers = starpu_get_nworkers_of_sched_ctx(sched_ctx_id);
-	int is_basic_worker = (best_workerid < nbasic_workers);
-
 	unsigned memory_node;
 	memory_node = starpu_worker_get_memory_node(best_workerid);
 
@@ -104,7 +101,7 @@ static int push_task_on_best_worker(struct starpu_task *task, int best_workerid,
 
 	int ret = 0;
 
-	if (is_basic_worker)
+	if (!starpu_worker_is_combined_worker(best_workerid))
 	{
 		task->predicted = exp_end_predicted - worker_exp_end[best_workerid];
 		/* TODO */
@@ -188,8 +185,7 @@ static int push_task_on_best_worker(struct starpu_task *task, int best_workerid,
 
 static double compute_expected_end(int workerid, double length, unsigned sched_ctx_id)
 {
-	unsigned nworkers = starpu_get_nworkers_of_sched_ctx(sched_ctx_id);
-	if (workerid < (int)nworkers)
+	if (!starpu_worker_is_combined_worker(workerid))
 	{
 		/* This is a basic worker */
 		return worker_exp_start[workerid] + worker_exp_len[workerid] + length;
@@ -218,9 +214,8 @@ static double compute_expected_end(int workerid, double length, unsigned sched_c
 
 static double compute_ntasks_end(int workerid, unsigned sched_ctx_id)
 {
-	unsigned nworkers = starpu_get_nworkers_of_sched_ctx(sched_ctx_id);
 	enum starpu_perf_archtype perf_arch = starpu_worker_get_perf_archtype(workerid);
-	if (workerid < (int)nworkers)
+	if (!starpu_worker_is_combined_worker(workerid))
 	{
 		/* This is a basic worker */
 		return ntasks[workerid] / starpu_worker_get_relative_speedup(perf_arch);
@@ -259,15 +254,15 @@ static int _parallel_heft_push_task(struct starpu_task *task, unsigned prio, uns
 	   there is no performance prediction available yet */
 	int forced_best = -1, forced_best_ctx = -1, forced_nimpl = -1;
 
-	double local_task_length[nworkers_ctx + ncombinedworkers][STARPU_MAXIMPLEMENTATIONS];
-	double local_data_penalty[nworkers_ctx + ncombinedworkers][STARPU_MAXIMPLEMENTATIONS];
-	double local_power[nworkers_ctx + ncombinedworkers][STARPU_MAXIMPLEMENTATIONS];
-	double local_exp_end[nworkers_ctx + ncombinedworkers][STARPU_MAXIMPLEMENTATIONS];
-	double fitness[nworkers_ctx + ncombinedworkers][STARPU_MAXIMPLEMENTATIONS];
+	double local_task_length[nworkers_ctx][STARPU_MAXIMPLEMENTATIONS];
+	double local_data_penalty[nworkers_ctx][STARPU_MAXIMPLEMENTATIONS];
+	double local_power[nworkers_ctx][STARPU_MAXIMPLEMENTATIONS];
+	double local_exp_end[nworkers_ctx][STARPU_MAXIMPLEMENTATIONS];
+	double fitness[nworkers_ctx][STARPU_MAXIMPLEMENTATIONS];
 
 	double max_exp_end = 0.0;
 
-	int skip_worker[nworkers_ctx + ncombinedworkers][STARPU_MAXIMPLEMENTATIONS];
+	int skip_worker[nworkers_ctx][STARPU_MAXIMPLEMENTATIONS];
 
 	double best_exp_end = DBL_MAX;
 	//double penality_best = 0.0;
@@ -285,25 +280,26 @@ static int _parallel_heft_push_task(struct starpu_task *task, unsigned prio, uns
         {
                 worker = workers->get_next(workers);
 
-		pthread_mutex_t *sched_mutex;
-		pthread_cond_t *sched_cond;
-		starpu_worker_get_sched_condition(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());
-		worker_exp_end[worker] = worker_exp_start[worker] + worker_exp_len[worker];
-		if (worker_exp_end[worker] > max_exp_end)
-			max_exp_end = worker_exp_end[worker];
-		_STARPU_PTHREAD_MUTEX_UNLOCK(sched_mutex);
+		if(!starpu_worker_is_combined_worker(worker))
+		{
+			pthread_mutex_t *sched_mutex;
+			pthread_cond_t *sched_cond;
+			starpu_worker_get_sched_condition(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());
+			worker_exp_end[worker] = worker_exp_start[worker] + worker_exp_len[worker];
+			if (worker_exp_end[worker] > max_exp_end)
+				max_exp_end = worker_exp_end[worker];
+			_STARPU_PTHREAD_MUTEX_UNLOCK(sched_mutex);
+		}
 	}
 
 	unsigned nimpl;
 	worker_ctx = 0;
-	while((workers->has_next(workers) && worker_ctx < nworkers_ctx) || 
-	      (worker_ctx >= nworkers_ctx && worker_ctx < (nworkers_ctx + ncombinedworkers)))
+	while(workers->has_next(workers))
 	{
-                worker = (workers->has_next(workers) && worker_ctx < nworkers_ctx) ? 
-			workers->get_next(workers) : worker_ctx;
+                worker = workers->get_next(workers);
 
 		for (nimpl = 0; nimpl < STARPU_MAXIMPLEMENTATIONS; nimpl++)
 		{
@@ -388,13 +384,10 @@ static int _parallel_heft_push_task(struct starpu_task *task, unsigned prio, uns
 	if (forced_best == -1)
 	{
 		worker_ctx = 0;
-		while((workers->has_next(workers) && worker_ctx < nworkers_ctx) || 
-		      (worker_ctx >= nworkers_ctx && worker_ctx < (nworkers_ctx + ncombinedworkers)))
+		while(workers->has_next(workers))
 		{
-			worker = (workers->has_next(workers) && worker_ctx < nworkers_ctx) ? 
-				workers->get_next(workers) : worker_ctx;
+			worker = workers->get_next(workers);
 			
-
 			for (nimpl = 0; nimpl < STARPU_MAXIMPLEMENTATIONS; nimpl++)
 			{
 				if (skip_worker[worker_ctx][nimpl])
@@ -428,6 +421,8 @@ static int _parallel_heft_push_task(struct starpu_task *task, unsigned prio, uns
 		}
 	}
 
+        if(workers->init_cursor)                                                                                                                                                                                                    workers->deinit_cursor(workers);
+
 	STARPU_ASSERT(forced_best != -1 || best != -1);
 
 	if (forced_best != -1)
@@ -461,14 +456,15 @@ static int parallel_heft_push_task(struct starpu_task *task)
 	int ret_val = -1;
 
 	if (task->priority == STARPU_MAX_PRIO)
-	{  _STARPU_PTHREAD_MUTEX_LOCK(changing_ctx_mutex);
+	{  
+		_STARPU_PTHREAD_MUTEX_LOCK(changing_ctx_mutex);
                 nworkers = starpu_get_nworkers_of_sched_ctx(sched_ctx_id);
                 if(nworkers == 0)
                 {
                         _STARPU_PTHREAD_MUTEX_UNLOCK(changing_ctx_mutex);
                         return ret_val;
                 }
-
+		
 		ret_val = _parallel_heft_push_task(task, 1, sched_ctx_id);
 		_STARPU_PTHREAD_MUTEX_UNLOCK(changing_ctx_mutex);
                 return ret_val;
@@ -508,23 +504,25 @@ static void parallel_heft_add_workers(unsigned sched_ctx_id, int *workerids, uns
 
 		starpu_worker_init_sched_condition(sched_ctx_id, workerid);
 	}
+	_starpu_sched_find_worker_combinations(workerids, nworkers);
 
-	struct _starpu_machine_config *config = (struct _starpu_machine_config *)_starpu_get_machine_config();
-	_starpu_sched_find_worker_combinations(&config->topology);
+// start_unclear_part: not very clear where this is used
+/* 	struct _starpu_machine_config *config = (struct _starpu_machine_config *)_starpu_get_machine_config(); */
+/* 	ncombinedworkers = config->topology.ncombinedworkers; */
 
-	ncombinedworkers = config->topology.ncombinedworkers;
+/* 	/\* We pre-compute an array of all the perfmodel archs that are applicable *\/ */
+/* 	unsigned total_worker_count = nworkers + ncombinedworkers; */
 
-	/* We pre-compute an array of all the perfmodel archs that are applicable */
-	unsigned total_worker_count = nworkers + ncombinedworkers;
+/* 	unsigned used_perf_archtypes[STARPU_NARCH_VARIATIONS]; */
+/* 	memset(used_perf_archtypes, 0, sizeof(used_perf_archtypes)); */
 
-	unsigned used_perf_archtypes[STARPU_NARCH_VARIATIONS];
-	memset(used_perf_archtypes, 0, sizeof(used_perf_archtypes));
+/* 	for (workerid = 0; workerid < total_worker_count; workerid++) */
+/* 	{ */
+/* 		enum starpu_perf_archtype perf_archtype = starpu_worker_get_perf_archtype(workerid); */
+/* 		used_perf_archtypes[perf_archtype] = 1; */
+/* 	} */
 
-	for (workerid = 0; workerid < total_worker_count; workerid++)
-	{
-		enum starpu_perf_archtype perf_archtype = starpu_worker_get_perf_archtype(workerid);
-		used_perf_archtypes[perf_archtype] = 1;
-	}
+// end_unclear_part
 
 //	napplicable_perf_archtypes = 0;