Explorar o código

wrong version+but keep it just in case

Andra Hugo %!s(int64=13) %!d(string=hai) anos
pai
achega
0b4907cb09

+ 0 - 1
examples/cholesky/cholesky_implicit.c

@@ -198,7 +198,6 @@ static void execute_cholesky(unsigned size, unsigned nblocks)
 		FPRINTF(stdout, "\n");
 	}
 #endif
-
 	cholesky(mat, size, size, nblocks);
 
 #ifdef PRINT_OUTPUT

+ 3 - 2
examples/sched_ctx_utils/sched_ctx_utils.c

@@ -64,7 +64,7 @@ void update_sched_ctx_timing_results(double flops, double avg_timing)
 {
 	unsigned *id = pthread_getspecific(key);
 	rv[*id].flops += flops;
-	rv[*id].avg_timing += avg_timing;	
+	rv[*id].avg_timing += avg_timing;
 }
 
 void* start_bench(void *val){
@@ -92,6 +92,7 @@ void* start_bench(void *val){
 
 	rv[p->id].flops /= NSAMPLES;
 	rv[p->id].avg_timing /= NSAMPLES;
+	
 }
 
 void start_2benchs(void (*bench)(unsigned, unsigned))
@@ -217,7 +218,7 @@ void construct_contexts(void (*bench)(unsigned, unsigned))
 	for(i = n_all_gpus  + cpu1; i < n_all_gpus + cpu1 + cpu2; i++)
 		procs2[k++] = i;
 
-	p2.ctx = starpu_create_sched_ctx("prio", procs2, nprocs2, "sched_ctx2");
+	p2.ctx = starpu_create_sched_ctx("heft", procs2, nprocs2, "sched_ctx2");
 	p1.the_other_ctx = (int)p2.ctx;
 	p2.procs = procs2;
 	p2.nprocs = nprocs2;

+ 74 - 43
src/core/sched_ctx.c

@@ -40,19 +40,22 @@ static void update_workers_func(void *buffers[] __attribute__ ((unused)), void *
 	unsigned sched_ctx_id = sched_ctx_info_args->sched_ctx_id;
 	
 	if(current_sched_ctx != NULL)
-	  {
+	{
 		/* add context to worker */
 		worker->sched_ctx[sched_ctx_id] = current_sched_ctx;
 		worker->nsched_ctxs++;
 		current_sched_ctx->workerids_to_add[worker->workerid] = NO_RESIZE;
-	  }
+	}
 	else
-	  {
+	{
 		/* remove context from worker */
-		  worker->sched_ctx[sched_ctx_id]->workerids_to_remove[worker->workerid] = NO_RESIZE;
-		  worker->sched_ctx[sched_ctx_id] = NULL;
-		  worker->nsched_ctxs--;
-	  }
+
+		worker->sched_ctx[sched_ctx_id]->workerids_to_remove[worker->workerid] = NO_RESIZE;
+		worker->sched_ctx[sched_ctx_id]->sched_mutex[worker->workerid] = NULL;
+		worker->sched_ctx[sched_ctx_id]->sched_cond[worker->workerid] = NULL;
+		worker->sched_ctx[sched_ctx_id] = NULL;
+		worker->nsched_ctxs--;
+	}
 }
 
 struct starpu_codelet_t sched_ctx_info_cl = {
@@ -72,7 +75,7 @@ static void _starpu_update_workers(int *workerids, int nworkers,
 	struct starpu_worker_s *worker[nworkers];
 	struct sched_ctx_info sched_info_args[nworkers];
 	for(i = 0; i < nworkers; i++)
-	  {
+	{
 		worker[i] = _starpu_get_worker_struct(workerids[i]);
 		
 		sched_info_args[i].sched_ctx_id = sched_ctx_id == -1  ? 
@@ -101,7 +104,7 @@ static void _starpu_update_workers(int *workerids, int nworkers,
 			starpu_task_destroy(tasks[i]);
 			tasks[i] = NULL;
 		}
-	  }
+	}
 	
 	for (i = 0; i < nworkers; i++)
 	{
@@ -110,10 +113,11 @@ static void _starpu_update_workers(int *workerids, int nworkers,
 			ret = starpu_task_wait(tasks[i]);
 			STARPU_ASSERT(!ret);
 			starpu_task_destroy(tasks[i]);
-	      }
+		}
 	}
 	
 }
+
 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)
@@ -126,6 +130,7 @@ struct starpu_sched_ctx*  _starpu_create_sched_ctx(const char *policy_name, int
 	struct starpu_sched_ctx *sched_ctx = &config->sched_ctxs[id];
 	_starpu_init_sched_ctx(sched_ctx);
 	sched_ctx->id = id;
+	sched_ctx->modified = 0;
 	int nworkers = config->topology.nworkers;
 	
 	STARPU_ASSERT(nworkers_ctx <= nworkers);
@@ -142,25 +147,25 @@ struct starpu_sched_ctx*  _starpu_create_sched_ctx(const char *policy_name, int
 	int j;
 	/* if null add all the workers are to the contex */
 	if(workerids == NULL)
-	  {
+	{
 		for(j = 0; j < nworkers; j++)
-		  {
+		{
 			sched_ctx->workerids[j] = j;
-		  }
+		}
 		sched_ctx->nworkers = nworkers;
-	  } 
+	} 
 	else 
-	  {
+	{
 		int i;
 		for(i = 0; i < nworkers_ctx; i++)
-		  {
+		{
 			/* the user should not ask for a resource that does not exist */
 			STARPU_ASSERT( workerids[i] >= 0 &&  workerids[i] <= nworkers);		    
 			sched_ctx->workerids[i] = workerids[i];
-
-		  }
-	  }
-
+			
+		}
+	}
+	
 	/* initialise all sync structures bc the number of workers can modify */
 	sched_ctx->sched_mutex = (pthread_mutex_t**)malloc(STARPU_NMAXWORKERS* sizeof(pthread_mutex_t*));
 	sched_ctx->sched_cond = (pthread_cond_t**)malloc(STARPU_NMAXWORKERS *sizeof(pthread_cond_t*));
@@ -242,7 +247,9 @@ static void _starpu_add_workers_to_sched_ctx(int *new_workers, int nnew_workers,
 {
         struct starpu_machine_config_s *config = (struct starpu_machine_config_s *)_starpu_get_machine_config();
         int nworkers = config->topology.nworkers;
-	
+	int nworkers_ctx = sched_ctx->nworkers;
+	int n_added_workers = 0;
+        int added_workers[nworkers];
         /*if null add the rest of the workers which don't already belong to this ctx*/
         if(new_workers == NULL)
 	{        
@@ -250,7 +257,12 @@ static void _starpu_add_workers_to_sched_ctx(int *new_workers, int nnew_workers,
                 for(j = 0; j < nworkers; j++)
                         if(!_starpu_worker_belongs_to_ctx(j, sched_ctx) &&
 			   sched_ctx->workerids_to_add[j] == NO_RESIZE)
-                                sched_ctx->workerids_to_add[j] = REQ_RESIZE;
+				sched_ctx->workerids_to_add[j] = REQ_RESIZE;
+			/* { */
+			/* 	sched_ctx->workerids[++nworkers_ctx]= j; */
+			/* 	added_workers[n_added_workers++] = j; */
+
+			/* } */
 	}
         else
 	{
@@ -263,25 +275,35 @@ static void _starpu_add_workers_to_sched_ctx(int *new_workers, int nnew_workers,
 			if(!_starpu_worker_belongs_to_ctx(new_workers[i], sched_ctx) &&
 			   sched_ctx->workerids_to_add[new_workers[i]] == NO_RESIZE)
 				sched_ctx->workerids_to_add[new_workers[i]] = REQ_RESIZE;
+			/* { */
+			/* 	sched_ctx->workerids[nworkers_ctx + n_added_workers] = new_workers[i]; */
+			/* 	added_workers[n_added_workers++] = new_workers[i]; */
+			/* } */
 		}
 	}
+	/* sched_ctx->sched_policy->init_sched_for_workers(sched_ctx->id, added_workers, n_added_workers); */
+
+        /* _starpu_update_workers(added_workers, n_added_workers, -1, sched_ctx); */
 
         return;
 }
 
 void _starpu_actually_add_workers_to_sched_ctx(struct starpu_sched_ctx *sched_ctx)
 {
-        struct starpu_machine_config_s *config = (struct starpu_machine_config_s *)_starpu_get_machine_config();
-        int nworkers = config->topology.nworkers;
 	int nworkers_ctx = sched_ctx->nworkers;
 
+	struct starpu_machine_config_s *config = (struct starpu_machine_config_s *)_starpu_get_machine_config();
+	int nworkers = config->topology.nworkers;
+ 
 	int n_added_workers = 0;
 	int added_workers[nworkers];
 
 	unsigned modified = 0;
 	int workerid;
 	for(workerid = 0; workerid < nworkers; workerid++)
+	{
 		if(sched_ctx->workerids_to_add[workerid] == REQ_RESIZE)
+		{
 			if(!_starpu_worker_belongs_to_ctx(workerid, sched_ctx))
 			{
 				added_workers[n_added_workers++] = workerid;
@@ -289,15 +311,17 @@ void _starpu_actually_add_workers_to_sched_ctx(struct starpu_sched_ctx *sched_ct
 				sched_ctx->workerids_to_add[workerid] = DO_RESIZE;
 				modified = 1;
 			}
-	
+		}
+	}
+
 	if(modified)
 	{
-		
 		sched_ctx->sched_policy->init_sched_for_workers(sched_ctx->id, added_workers, n_added_workers);
 		sched_ctx->nworkers += n_added_workers;
 		
 		_starpu_update_workers(added_workers, n_added_workers, -1, sched_ctx);
 	}
+
         return;
 }
 
@@ -305,7 +329,6 @@ void starpu_delete_sched_ctx(unsigned sched_ctx_id, unsigned inheritor_sched_ctx
 {
 	struct starpu_sched_ctx *sched_ctx = _starpu_get_sched_ctx_struct(sched_ctx_id);
 	struct starpu_sched_ctx *inheritor_sched_ctx = _starpu_get_sched_ctx_struct(inheritor_sched_ctx_id);
-	
 	_starpu_manage_delete_sched_ctx(sched_ctx);
 
 	/*if both of them have all the ressources is pointless*/
@@ -315,12 +338,12 @@ void starpu_delete_sched_ctx(unsigned sched_ctx_id, unsigned inheritor_sched_ctx
 	
 	if(!(sched_ctx->nworkers == nworkers && sched_ctx->nworkers == inheritor_sched_ctx->nworkers))
 		_starpu_add_workers_to_sched_ctx(sched_ctx->workerids, sched_ctx->nworkers, inheritor_sched_ctx);
-	
+	inheritor_sched_ctx->modified = 1;
 	if(!starpu_wait_for_all_tasks_of_sched_ctx(sched_ctx_id))
 	{
 		free_sched_ctx_mem(sched_ctx);
 		
-	}	
+	}
 	return;	
 }
 
@@ -341,7 +364,6 @@ void _starpu_delete_all_sched_ctxs()
 void starpu_add_workers_to_sched_ctx(int *workers_to_add, int nworkers_to_add,
 				     unsigned sched_ctx_id)
 {
-	printf("add workers\n");
 	struct starpu_sched_ctx *sched_ctx = _starpu_get_sched_ctx_struct(sched_ctx_id);
 	_starpu_add_workers_to_sched_ctx(workers_to_add, nworkers_to_add, sched_ctx);
 	return;
@@ -357,6 +379,8 @@ static void _starpu_remove_workers_from_sched_ctx(int *workerids, int nworkers_t
 	STARPU_ASSERT(nworkers_to_remove  <= nworkers_ctx);
 
 	int i, workerid;
+	int nremoved_workers = 0;
+        int removed_workers[nworkers_ctx];
 
 	/*if null remove all the workers that belong to this ctx*/
 	if(workerids == NULL)
@@ -364,17 +388,30 @@ static void _starpu_remove_workers_from_sched_ctx(int *workerids, int nworkers_t
 		for(i = 0; i < nworkers_ctx; i++)
 			if(sched_ctx->workerids_to_remove[i] == NO_RESIZE)
 				sched_ctx->workerids_to_remove[i] = REQ_RESIZE;
+		/* 	{ */
+		/* 		removed_workers[i] = sched_ctx->workerids[i]; */
+		/* 		sched_ctx->workerids[i] = -1; */
+		/* 		nremoved_workers++; */
+		/* 	} */
+		/* sched_ctx->nworkers = 0; */
 	} 
 	else 
 	{
-		for(i = 0; i < nworkers_ctx; i++)
+		for(i = 0; i < nworkers_to_remove; i++)
 		{
 			workerid = workerids[i]; 
 			/* take care the user does not ask for a resource that does not exist */
 			STARPU_ASSERT( workerid >= 0 &&  workerid <= nworkers);
-			if(sched_ctx->workerids_to_add[workerid] == NO_RESIZE)
+ 			if(sched_ctx->workerids_to_remove[workerid] == NO_RESIZE)
 				sched_ctx->workerids_to_remove[workerid] = REQ_RESIZE;
+			/* removed_workers[nremoved_workers++] = workerid; */
+			/* int workerid_ctx = _starpu_get_index_in_ctx_of_workerid(sched_ctx->id, workerid); */
+			/* sched_ctx->workerids[workerid_ctx] = -1; */
+
 		}
+		/* sched_ctx->nworkers -= nremoved_workers; */
+		/* _starpu_rearange_sched_ctx_workerids(sched_ctx, nworkers_ctx); */
+		/* _starpu_update_workers(removed_workers, nremoved_workers, sched_ctx->id, NULL); */
 	}
 
 	return;
@@ -382,6 +419,7 @@ static void _starpu_remove_workers_from_sched_ctx(int *workerids, int nworkers_t
 
 void _starpu_actually_remove_workers_from_sched_ctx(struct starpu_sched_ctx *sched_ctx)
 {
+
 	int nworkers_ctx =  sched_ctx->nworkers;
 
 	int i, workerid, worker_ctx;
@@ -529,10 +567,6 @@ static void _starpu_rearange_sched_ctx_workerids(struct starpu_sched_ctx *sched_
 			if(first_free_id != -1)
 			  {
 				sched_ctx->workerids[first_free_id] = sched_ctx->workerids[i];
-				sched_ctx->sched_mutex[first_free_id] = sched_ctx->sched_mutex[i];
-				sched_ctx->sched_cond[first_free_id] = sched_ctx->sched_cond[i];
-				sched_ctx->sched_mutex[i] = NULL;
-				sched_ctx->sched_cond[i] = NULL;
 				sched_ctx->workerids[i] = -1;
 			  }
 		  }
@@ -624,16 +658,14 @@ int _starpu_get_index_in_ctx_of_workerid(unsigned sched_ctx_id, unsigned workeri
 	return -1;
 }
 
-pthread_mutex_t *_starpu_get_sched_mutex(struct starpu_sched_ctx *sched_ctx, int worker)
+pthread_mutex_t *_starpu_get_sched_mutex(struct starpu_sched_ctx *sched_ctx, int workerid)
 {
-	int workerid_ctx = _starpu_get_index_in_ctx_of_workerid(sched_ctx->id, worker);
-	return (workerid_ctx == -1 ? NULL : sched_ctx->sched_mutex[workerid_ctx]);
+	return sched_ctx->sched_mutex[workerid];
 }
 
-pthread_cond_t *_starpu_get_sched_cond(struct starpu_sched_ctx *sched_ctx, int worker)
+pthread_cond_t *_starpu_get_sched_cond(struct starpu_sched_ctx *sched_ctx, int workerid)
 {
-	int workerid_ctx = _starpu_get_index_in_ctx_of_workerid(sched_ctx->id, worker);
-	return (workerid_ctx == -1 ? NULL : sched_ctx->sched_cond[workerid_ctx]);
+	return sched_ctx->sched_cond[workerid];
 }
 
 int* starpu_get_workers_of_ctx(unsigned sched_ctx_id)
@@ -642,7 +674,6 @@ int* starpu_get_workers_of_ctx(unsigned sched_ctx_id)
 	return sched_ctx->workerids;
 }
 
-
 void starpu_set_sched_ctx(unsigned *sched_ctx)
 {
 	pthread_setspecific(sched_ctx_key, (void*)sched_ctx);

+ 1 - 0
src/core/sched_ctx.h

@@ -70,6 +70,7 @@ struct starpu_sched_ctx {
 	/* a structure containing a series of criteria determining the resize procedure */
 	struct starpu_sched_ctx_hypervisor_criteria *criteria;
 #endif //STARPU_USE_SCHED_CTX_HYPERVISOR
+	unsigned modified;
 };
 
 struct starpu_machine_config_s;

+ 33 - 28
src/core/sched_policy.c

@@ -310,9 +310,14 @@ int _starpu_push_task(starpu_job_t j, unsigned job_is_already_locked)
 	else {
 		struct starpu_sched_ctx *sched_ctx = _starpu_get_sched_ctx_struct(task->sched_ctx);
 		STARPU_ASSERT(sched_ctx->sched_policy->push_task);
-		_starpu_actually_add_workers_to_sched_ctx(sched_ctx);
-		_starpu_actually_remove_workers_from_sched_ctx(sched_ctx);
 
+		if(sched_ctx->modified)
+		{
+			_starpu_actually_remove_workers_from_sched_ctx(sched_ctx);
+			_starpu_actually_add_workers_to_sched_ctx(sched_ctx);
+			sched_ctx->modified = 0;
+		}
+			
 		ret = sched_ctx->sched_policy->push_task(task, sched_ctx->id);
 	}
 
@@ -332,43 +337,43 @@ struct starpu_task *_starpu_pop_task(struct starpu_worker_s *worker)
 	struct timespec pop_start_time;
 	if (profiling)
 		starpu_clock_gettime(&pop_start_time);
-
+	
 	PTHREAD_MUTEX_LOCK(worker->sched_mutex);
 	/* perhaps there is some local task to be executed first */
 	task = _starpu_pop_local_task(worker);
 	PTHREAD_MUTEX_UNLOCK(worker->sched_mutex);
-
-
+	
+	
 	/* get tasks from the stacks of the strategy */
 	if(!task)
-	  {
+	{
 		struct starpu_sched_ctx *sched_ctx;
 		pthread_mutex_t *sched_ctx_mutex;
-
+		
 		unsigned i;
 		for(i = 0; i < STARPU_NMAX_SCHED_CTXS; i++)
-		  {
-		    sched_ctx = worker->sched_ctx[i];
-		    
-		    if(sched_ctx != NULL && 
-		       sched_ctx->workerids_to_add[worker->workerid] == NO_RESIZE)
-		      {
-			sched_ctx_mutex = _starpu_get_sched_mutex(sched_ctx, worker->workerid);
-			if(sched_ctx_mutex != NULL)
-			  {
-			    PTHREAD_MUTEX_LOCK(sched_ctx_mutex);
-			    if (sched_ctx->sched_policy->pop_task)
-			      {
-				PTHREAD_MUTEX_UNLOCK(sched_ctx_mutex);
-				task = sched_ctx->sched_policy->pop_task(sched_ctx->id);
-				break;
-			      }
-			    PTHREAD_MUTEX_UNLOCK(sched_ctx_mutex);
-			  }
-		      }
-		  }
+		{
+			sched_ctx = worker->sched_ctx[i];
+			
+			if(sched_ctx != NULL &&
+			   sched_ctx->workerids_to_add[worker->workerid] == NO_RESIZE)
+			{
+				sched_ctx_mutex = _starpu_get_sched_mutex(sched_ctx, worker->workerid);
+				if(sched_ctx_mutex != NULL)
+				{
+					PTHREAD_MUTEX_LOCK(sched_ctx_mutex);
+					if (sched_ctx->sched_policy->pop_task)
+					{
+						PTHREAD_MUTEX_UNLOCK(sched_ctx_mutex);
+						task = sched_ctx->sched_policy->pop_task(sched_ctx->id);
+						break;
+					}
+					PTHREAD_MUTEX_UNLOCK(sched_ctx_mutex);
+				}
+			}
+		}
 	  }
-
+	
 	/* Note that we may get a NULL task in case the scheduler was unlocked
 	 * for some reason. */
 	if (profiling && task)

+ 2 - 3
src/core/task.c

@@ -218,7 +218,7 @@ int _starpu_submit_job(starpu_job_t j, unsigned do_not_increment_nsubmitted)
 	PTHREAD_MUTEX_LOCK(&j->sync_mutex);
 	
 	j->submitted = 1;
-
+       
 	int ret = _starpu_enforce_deps_and_schedule(j, 1);
 
 	PTHREAD_MUTEX_UNLOCK(&j->sync_mutex);
@@ -232,9 +232,8 @@ int starpu_task_submit(struct starpu_task *task)
 {
 	unsigned nsched_ctxs = _starpu_get_nsched_ctxs();
 
-	task->sched_ctx = nsched_ctxs == 1 || task->control_task ? 
+	task->sched_ctx = (nsched_ctxs == 1 || task->control_task) ? 
 		0 : starpu_get_sched_ctx();
-
 	int ret;
 	unsigned is_sync = task->synchronous;
         _STARPU_LOG_IN();

+ 66 - 65
src/sched_policies/heft.c

@@ -83,9 +83,8 @@ static void heft_init_for_workers(unsigned sched_ctx_id, int *workerids, unsigne
 		/* we push the tasks on the local lists of the workers
 		   therefore the synchronisations mechanisms of the strategy
 		   are the global ones */
-		sched_ctx->sched_mutex[nworkers_ctx] = workerarg->sched_mutex;
-		sched_ctx->sched_cond[nworkers_ctx] = workerarg->sched_cond;
-		nworkers_ctx++;
+		sched_ctx->sched_mutex[workerid] = workerarg->sched_mutex;
+		sched_ctx->sched_cond[workerid] = workerarg->sched_cond;
 	}
 }
 static void heft_init(unsigned sched_ctx_id)
@@ -98,7 +97,7 @@ static void heft_init(unsigned sched_ctx_id)
 	
 	struct starpu_sched_ctx *sched_ctx = _starpu_get_sched_ctx_struct(sched_ctx_id);
 
-	unsigned nworkers = sched_ctx->nworkers;
+	unsigned nworkers_ctx = sched_ctx->nworkers;
 	sched_ctx->policy_data = (void*)hd;
 
 	const char *strval_alpha = getenv("STARPU_SCHED_ALPHA");
@@ -124,7 +123,7 @@ static void heft_init(unsigned sched_ctx_id)
 
 	unsigned workerid_ctx;
 
-	for (workerid_ctx = 0; workerid_ctx < nworkers; workerid_ctx++)
+	for (workerid_ctx = 0; workerid_ctx < nworkers_ctx; workerid_ctx++)
 	{
 		int workerid = sched_ctx->workerids[workerid_ctx];
 		struct starpu_worker_s *workerarg = _starpu_get_worker_struct(workerid);
@@ -140,8 +139,8 @@ static void heft_init(unsigned sched_ctx_id)
 		/* we push the tasks on the local lists of the workers
 		   therefore the synchronisations mechanisms of the strategy
 		   are the global ones */
-		sched_ctx->sched_mutex[workerid_ctx] = workerarg->sched_mutex;
-		sched_ctx->sched_cond[workerid_ctx] = workerarg->sched_cond;
+		sched_ctx->sched_mutex[workerid] = workerarg->sched_mutex;
+		sched_ctx->sched_cond[workerid] = workerarg->sched_cond;
 		
 	}
 }
@@ -228,22 +227,22 @@ static void compute_all_performance_predictions(struct starpu_task *task,
 					struct starpu_task_bundle *bundle,
 					struct starpu_sched_ctx *sched_ctx )
 {
-  int calibrating = 0;
-  double max_exp_end = DBL_MIN;
-  double best_exp_end = DBL_MAX;
-  int ntasks_best = -1;
-  double ntasks_best_end = 0.0;
-
-  /* A priori, we know all estimations */
-  int unknown = 0;
-
-  unsigned nworkers = sched_ctx->nworkers;
-
-  unsigned nimpl;
-  unsigned best_impl = 0;
-  unsigned worker, worker_ctx;
-  for (worker_ctx = 0; worker_ctx < nworkers; worker_ctx++)
-    {
+	int calibrating = 0;
+	double max_exp_end = DBL_MIN;
+	double best_exp_end = DBL_MAX;
+	int ntasks_best = -1;
+	double ntasks_best_end = 0.0;
+	
+	/* A priori, we know all estimations */
+	int unknown = 0;
+	
+	unsigned nworkers_ctx = sched_ctx->nworkers;
+	
+	unsigned nimpl;
+	unsigned best_impl = 0;
+	unsigned worker, worker_ctx;
+	for (worker_ctx = 0; worker_ctx < nworkers_ctx; worker_ctx++)
+	{
 		worker = sched_ctx->workerids[worker_ctx];
 		for (nimpl = 0; nimpl <STARPU_MAXIMPLEMENTATIONS; nimpl++) 
 		{
@@ -261,66 +260,67 @@ static void compute_all_performance_predictions(struct starpu_task *task,
 			
 			enum starpu_perf_archtype perf_arch = starpu_worker_get_perf_archtype(worker);
 			unsigned memory_node = starpu_worker_get_memory_node(worker);
-
-      		if (bundle)
-      		{
-      			local_task_length[worker_ctx] = starpu_task_bundle_expected_length(bundle, perf_arch, nimpl);
-      	  		local_data_penalty[worker_ctx] = starpu_task_bundle_expected_data_transfer_time(bundle, memory_node);
-      	  		local_power[worker_ctx] = starpu_task_bundle_expected_power(bundle, perf_arch, nimpl);
+			
+			if (bundle)
+			{
+				local_task_length[worker_ctx] = starpu_task_bundle_expected_length(bundle, perf_arch, nimpl);
+				local_data_penalty[worker_ctx] = starpu_task_bundle_expected_data_transfer_time(bundle, memory_node);
+				local_power[worker_ctx] = starpu_task_bundle_expected_power(bundle, perf_arch, nimpl);
 				//_STARPU_DEBUG("Scheduler heft bundle: task length (%lf) local power (%lf) worker (%u) kernel (%u) \n", local_task_length[worker_ctx],local_power[worker_ctx],worker,nimpl);
-      		}
-      		else 
+			}
+			else 
 			{
 				local_task_length[worker_ctx] = starpu_task_expected_length(task, perf_arch, nimpl);
 				local_data_penalty[worker_ctx] = starpu_task_expected_data_transfer_time(memory_node, task);
 				local_power[worker_ctx] = starpu_task_expected_power(task, perf_arch, nimpl);
 				//_STARPU_DEBUG("Scheduler heft: task length (%lf) local power (%lf) worker (%u) kernel (%u) \n", local_task_length[worker_ctx],local_power[worker_ctx],worker,nimpl);
-      		}
-
-      		double ntasks_end = ntasks[worker] / starpu_worker_get_relative_speedup(perf_arch);
-
-      		if (ntasks_best == -1
-	  			|| (!calibrating && ntasks_end < ntasks_best_end) /* Not calibrating, take better task */
-	  			|| (!calibrating && local_task_length[worker_ctx] == -1.0) /* Not calibrating but this worker is being calibrated */
-	  			|| (calibrating && local_task_length[worker_ctx] == -1.0 && ntasks_end < ntasks_best_end) /* Calibrating, compete this worker with other non-calibrated */
-	  		) {
+			}
+			
+			double ntasks_end = ntasks[worker] / starpu_worker_get_relative_speedup(perf_arch);
+			
+			if (ntasks_best == -1
+			    || (!calibrating && ntasks_end < ntasks_best_end) /* Not calibrating, take better task */
+			    || (!calibrating && local_task_length[worker_ctx] == -1.0) /* Not calibrating but this worker is being calibrated */
+			    || (calibrating && local_task_length[worker_ctx] == -1.0 && ntasks_end < ntasks_best_end) /* Calibrating, compete this worker with other non-calibrated */
+				) 
+			{
 				ntasks_best_end = ntasks_end;
 				ntasks_best = worker;
-      		}
-
-     		if (local_task_length[worker_ctx] == -1.0)
+			}
+			
+			if (local_task_length[worker_ctx] == -1.0)
 				/* we are calibrating, we want to speed-up calibration time
-	 			* so we privilege non-calibrated tasks (but still
-	 			* greedily distribute them to avoid dumb schedules) */
+				 * so we privilege non-calibrated tasks (but still
+				 * greedily distribute them to avoid dumb schedules) */
 				calibrating = 1;
-
-      		if (local_task_length[worker_ctx] <= 0.0)
+			
+			if (local_task_length[worker_ctx] <= 0.0)
 				/* there is no prediction available for that task
-	 			* with that arch yet, so switch to a greedy strategy */
+				 * with that arch yet, so switch to a greedy strategy */
 				unknown = 1;
-
+			
 			if (unknown)
 				continue;
 
-      		exp_end[worker_ctx] = exp_start[worker] + exp_len[worker] + local_task_length[worker_ctx];
-
-      		if (exp_end[worker_ctx] < best_exp_end)
+			exp_end[worker_ctx] = exp_start[worker] + exp_len[worker] + local_task_length[worker_ctx];
+			
+			if (exp_end[worker_ctx] < best_exp_end)
 			{
-	  			/* a better solution was found */
-	  			best_exp_end = exp_end[worker_ctx];
+				/* a better solution was found */
+				best_exp_end = exp_end[worker_ctx];
 				best_impl = nimpl;
 			}
-
-      		if (local_power[worker_ctx] == -1.0)
+			
+			if (local_power[worker_ctx] == -1.0)
 				local_power[worker_ctx] = 0.;
-    	}
+		}
 	}
 
 	*forced_best = unknown?ntasks_best:-1;
 
 	*best_exp_endp = best_exp_end;
 	*max_exp_endp = max_exp_end;
-
+	
 	/* save the best implementation */
 	//_STARPU_DEBUG("Scheduler heft: kernel (%u)\n", best_impl);
 	_starpu_get_job_associated_to_task(task)->nimpl = best_impl;
@@ -354,17 +354,17 @@ static int _heft_push_task(struct starpu_task *task, unsigned prio, unsigned sch
 	struct starpu_task_bundle *bundle = task->bundle;
 
 	compute_all_performance_predictions(task, local_task_length, exp_end,
-					&max_exp_end, &best_exp_end,
-					local_data_penalty,
-					local_power, &forced_best, bundle, sched_ctx);
-
+					    &max_exp_end, &best_exp_end,
+					    local_data_penalty,
+					    local_power, &forced_best, bundle, sched_ctx);
+	
 	/* If there is no prediction available for that task with that arch we
 	 * want to speed-up calibration time so we force this measurement */
 	if (forced_best != -1){
 		_starpu_increment_nsubmitted_tasks_of_worker(forced_best);
 		return push_task_on_best_worker(task, forced_best, 0.0, prio);
 	}
-
+	
 	/*
 	 *	Determine which worker optimizes the fitness metric which is a
 	 *	trade-off between load-balacing, data locality, and energy
@@ -422,14 +422,15 @@ static int _heft_push_task(struct starpu_task *task, unsigned prio, unsigned sch
 		 * anymore. */
 		PTHREAD_MUTEX_LOCK(&bundle->mutex);
 		int ret = starpu_task_bundle_remove(bundle, task);
-
+		
 		/* Perhaps the bundle was destroyed when removing the last
 		 * entry */
 		if (ret != 1)
 			PTHREAD_MUTEX_UNLOCK(&bundle->mutex);
 
 	}
-	else {
+	else 
+	{
 		model_best = local_task_length[best_id_ctx];
 	}