|
@@ -21,6 +21,8 @@
|
|
#include <drivers/mp_common/mp_common.h>
|
|
#include <drivers/mp_common/mp_common.h>
|
|
#include <datawizard/interfaces/data_interface.h>
|
|
#include <datawizard/interfaces/data_interface.h>
|
|
#include <common/barrier.h>
|
|
#include <common/barrier.h>
|
|
|
|
+#include <core/workers.h>
|
|
|
|
+#include <common/barrier_counter.h>
|
|
#ifdef STARPU_USE_MIC
|
|
#ifdef STARPU_USE_MIC
|
|
#include <common/COISysInfo_common.h>
|
|
#include <common/COISysInfo_common.h>
|
|
#endif
|
|
#endif
|
|
@@ -150,6 +152,81 @@ static void _starpu_sink_common_copy_to_sink(const struct _starpu_mp_node *mp_no
|
|
mp_node->dt_send_to_device(mp_node, cmd->devid, cmd->addr, cmd->size);
|
|
mp_node->dt_send_to_device(mp_node, cmd->devid, cmd->addr, cmd->size);
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+
|
|
|
|
+/* Receive workers and combined workers and store them into the struct config
|
|
|
|
+ */
|
|
|
|
+static void _starpu_sink_common_recv_workers(struct _starpu_mp_node * node, void *arg, int arg_size)
|
|
|
|
+{
|
|
|
|
+ /* Retrieve information from the message */
|
|
|
|
+ STARPU_ASSERT(arg_size == (sizeof(int)*5));
|
|
|
|
+ void * arg_ptr = arg;
|
|
|
|
+ int i;
|
|
|
|
+
|
|
|
|
+ int nworkers = *(int *)arg_ptr;
|
|
|
|
+ arg_ptr += sizeof(nworkers);
|
|
|
|
+
|
|
|
|
+ int worker_size = *(int *)arg_ptr;
|
|
|
|
+ arg_ptr += sizeof(worker_size);
|
|
|
|
+
|
|
|
|
+ int combined_worker_size = *(int *)arg_ptr;
|
|
|
|
+ arg_ptr += sizeof(combined_worker_size);
|
|
|
|
+
|
|
|
|
+ int baseworkerid = *(int *)arg_ptr;
|
|
|
|
+ arg_ptr += sizeof(baseworkerid);
|
|
|
|
+
|
|
|
|
+ struct _starpu_machine_config *config = _starpu_get_machine_config();
|
|
|
|
+ config->topology.nworkers = *(int *)arg_ptr;
|
|
|
|
+
|
|
|
|
+
|
|
|
|
+ /* Retrieve workers */
|
|
|
|
+ struct _starpu_worker * workers = &config->workers[baseworkerid];
|
|
|
|
+ node->dt_recv(node,workers,worker_size);
|
|
|
|
+
|
|
|
|
+ /* Update workers to have coherent field */
|
|
|
|
+ for(i=0; i<nworkers; i++)
|
|
|
|
+ {
|
|
|
|
+ workers[i].config = config;
|
|
|
|
+ starpu_pthread_mutex_init(&workers[i].mutex,NULL);
|
|
|
|
+ starpu_pthread_mutex_destroy(&workers[i].mutex);
|
|
|
|
+
|
|
|
|
+ starpu_pthread_cond_init(&workers[i].started_cond,NULL);
|
|
|
|
+ starpu_pthread_cond_destroy(&workers[i].started_cond);
|
|
|
|
+
|
|
|
|
+ starpu_pthread_cond_init(&workers[i].ready_cond,NULL);
|
|
|
|
+ starpu_pthread_cond_destroy(&workers[i].ready_cond);
|
|
|
|
+
|
|
|
|
+ starpu_pthread_mutex_init(&workers[i].sched_mutex,NULL);
|
|
|
|
+ starpu_pthread_mutex_destroy(&workers[i].sched_mutex);
|
|
|
|
+
|
|
|
|
+ starpu_pthread_cond_init(&workers[i].sched_cond,NULL);
|
|
|
|
+ starpu_pthread_cond_destroy(&workers[i].sched_cond);
|
|
|
|
+
|
|
|
|
+ workers[i].current_task = NULL;
|
|
|
|
+ workers[i].set = NULL;
|
|
|
|
+ workers[i].terminated_jobs = NULL;
|
|
|
|
+ workers[i].sched_ctx = NULL;
|
|
|
|
+
|
|
|
|
+ //_starpu_barrier_counter_init(&workers[i].tasks_barrier, 1);
|
|
|
|
+ //_starpu_barrier_counter_destroy(&workers[i].tasks_barrier);
|
|
|
|
+
|
|
|
|
+ starpu_pthread_mutex_init(&workers[i].parallel_sect_mutex,NULL);
|
|
|
|
+ starpu_pthread_mutex_destroy(&workers[i].parallel_sect_mutex);
|
|
|
|
+
|
|
|
|
+ starpu_pthread_cond_init(&workers[i].parallel_sect_cond,NULL);
|
|
|
|
+ starpu_pthread_cond_destroy(&workers[i].parallel_sect_cond);
|
|
|
|
+
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /* Retrieve combined workers */
|
|
|
|
+ struct _starpu_combined_worker * combined_workers = config->combined_workers;
|
|
|
|
+ node->dt_recv(node, combined_workers, combined_worker_size);
|
|
|
|
+
|
|
|
|
+ node->baseworkerid = baseworkerid;
|
|
|
|
+ STARPU_PTHREAD_BARRIER_WAIT(&node->init_completed_barrier);
|
|
|
|
+}
|
|
|
|
+
|
|
|
|
+
|
|
|
|
+
|
|
/* Function looping on the sink, waiting for tasks to execute.
|
|
/* Function looping on the sink, waiting for tasks to execute.
|
|
* If the caller is the host, don't do anything.
|
|
* If the caller is the host, don't do anything.
|
|
*/
|
|
*/
|
|
@@ -170,6 +247,10 @@ void _starpu_sink_common_worker(void)
|
|
/* Create and initialize the node */
|
|
/* Create and initialize the node */
|
|
node = _starpu_mp_common_node_create(node_kind, -1);
|
|
node = _starpu_mp_common_node_create(node_kind, -1);
|
|
|
|
|
|
|
|
+ starpu_pthread_key_t worker_key;
|
|
|
|
+ STARPU_PTHREAD_KEY_CREATE(&worker_key, NULL);
|
|
|
|
+
|
|
|
|
+
|
|
while (!exit_starpu)
|
|
while (!exit_starpu)
|
|
{
|
|
{
|
|
/* If we have received a message */
|
|
/* If we have received a message */
|
|
@@ -216,6 +297,9 @@ void _starpu_sink_common_worker(void)
|
|
_starpu_sink_common_copy_to_sink(node, arg, arg_size);
|
|
_starpu_sink_common_copy_to_sink(node, arg, arg_size);
|
|
break;
|
|
break;
|
|
|
|
|
|
|
|
+ case STARPU_SYNC_WORKERS:
|
|
|
|
+ _starpu_sink_common_recv_workers(node, arg, arg_size);
|
|
|
|
+ break;
|
|
default:
|
|
default:
|
|
printf("Oops, command %x unrecognized\n", command);
|
|
printf("Oops, command %x unrecognized\n", command);
|
|
}
|
|
}
|
|
@@ -245,97 +329,187 @@ void _starpu_sink_common_worker(void)
|
|
exit(0);
|
|
exit(0);
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+
|
|
|
|
+/* Search for the mp_barrier correspondind to the specified combined worker
|
|
|
|
+ * and create it if it doesn't exist
|
|
|
|
+ */
|
|
|
|
+static struct mp_barrier * _starpu_sink_common_get_barrier(struct _starpu_mp_node * node, int cb_workerid, int cb_workersize)
|
|
|
|
+{
|
|
|
|
+ struct mp_barrier * b = NULL;
|
|
|
|
+ STARPU_PTHREAD_MUTEX_LOCK(&node->barrier_mutex);
|
|
|
|
+ /* Search if the barrier already exist */
|
|
|
|
+ for(b = mp_barrier_list_begin(node->barrier_list);
|
|
|
|
+ b != mp_barrier_list_end(node->barrier_list) && b->id != cb_workerid;
|
|
|
|
+ b = mp_barrier_list_next(b));
|
|
|
|
+
|
|
|
|
+ /* If we found the barrier */
|
|
|
|
+ if(b != NULL)
|
|
|
|
+ {
|
|
|
|
+ STARPU_PTHREAD_MUTEX_UNLOCK(&node->barrier_mutex);
|
|
|
|
+ return b;
|
|
|
|
+ }
|
|
|
|
+ else
|
|
|
|
+ {
|
|
|
|
+
|
|
|
|
+ /* Else we create, initialize and add it to the list*/
|
|
|
|
+ b = mp_barrier_new();
|
|
|
|
+ b->id = cb_workerid;
|
|
|
|
+ STARPU_PTHREAD_BARRIER_INIT(&b->before_work_barrier,NULL,cb_workersize);
|
|
|
|
+ STARPU_PTHREAD_BARRIER_INIT(&b->after_work_barrier,NULL,cb_workersize);
|
|
|
|
+ mp_barrier_list_push_back(node->barrier_list,b);
|
|
|
|
+ STARPU_PTHREAD_MUTEX_UNLOCK(&node->barrier_mutex);
|
|
|
|
+ return b;
|
|
|
|
+ }
|
|
|
|
+}
|
|
|
|
+
|
|
|
|
+
|
|
|
|
+/* Erase for the mp_barrier correspondind to the specified combined worker
|
|
|
|
+*/
|
|
|
|
+static void _starpu_sink_common_erase_barrier(struct _starpu_mp_node * node, struct mp_barrier *barrier)
|
|
|
|
+{
|
|
|
|
+ STARPU_PTHREAD_MUTEX_LOCK(&node->barrier_mutex);
|
|
|
|
+ mp_barrier_list_erase(node->barrier_list,barrier);
|
|
|
|
+ STARPU_PTHREAD_MUTEX_UNLOCK(&node->barrier_mutex);
|
|
|
|
+}
|
|
|
|
+
|
|
|
|
+/* Append the message given in parameter to the message list
|
|
|
|
+ */
|
|
|
|
+static void _starpu_sink_common_append_message(struct _starpu_mp_node *node, struct mp_message * message)
|
|
|
|
+{
|
|
|
|
+ STARPU_PTHREAD_MUTEX_LOCK(&node->message_queue_mutex);
|
|
|
|
+ mp_message_list_push_front(node->message_queue,message);
|
|
|
|
+ STARPU_PTHREAD_MUTEX_UNLOCK(&node->message_queue_mutex);
|
|
|
|
+
|
|
|
|
+}
|
|
|
|
+/* Append to the message list a "STARPU_PRE_EXECUTION" message
|
|
|
|
+ */
|
|
|
|
+static void _starpu_sink_common_pre_execution_message(struct _starpu_mp_node *node, struct mp_task *task)
|
|
|
|
+{
|
|
|
|
+ /* Init message to tell the sink that the execution has begun */
|
|
|
|
+ struct mp_message * message = mp_message_new();
|
|
|
|
+ message->type = STARPU_PRE_EXECUTION;
|
|
|
|
+ *(int *) message->buffer = task->combined_workerid;
|
|
|
|
+ message->size = sizeof(task->combined_workerid);
|
|
|
|
+
|
|
|
|
+
|
|
|
|
+ /* Append the message to the queue */
|
|
|
|
+ _starpu_sink_common_append_message(node, message);
|
|
|
|
+
|
|
|
|
+}
|
|
|
|
+
|
|
|
|
+/* Append to the message list a "STARPU_EXECUTION_COMPLETED" message
|
|
|
|
+ */
|
|
|
|
+static void _starpu_sink_common_execution_completed_message(struct _starpu_mp_node *node, struct mp_task *task)
|
|
|
|
+{
|
|
|
|
+ /* Init message to tell the sink that the execution is completed */
|
|
|
|
+ struct mp_message * message = mp_message_new();
|
|
|
|
+ message->type = STARPU_EXECUTION_COMPLETED;
|
|
|
|
+ message->size = sizeof(task->coreid);
|
|
|
|
+ *(int*) message->buffer = task->coreid;
|
|
|
|
+
|
|
|
|
+ /* Append the message to the queue */
|
|
|
|
+ _starpu_sink_common_append_message(node, message);
|
|
|
|
+}
|
|
|
|
+
|
|
|
|
+
|
|
|
|
+/* Bind the thread which is running on the specified core to the combined worker */
|
|
|
|
+static void _starpu_sink_common_bind_to_combined_worker(struct _starpu_mp_node *node, int coreid, struct _starpu_combined_worker * combined_worker)
|
|
|
|
+{
|
|
|
|
+ int i;
|
|
|
|
+ int * bind_set = malloc(sizeof(int)*combined_worker->worker_size);
|
|
|
|
+ for(i=0;i<combined_worker->worker_size;i++)
|
|
|
|
+ bind_set[i] = combined_worker->combined_workerid[i] - node->baseworkerid;
|
|
|
|
+ node->bind_thread(node, coreid, bind_set, combined_worker->worker_size);
|
|
|
|
+}
|
|
|
|
+
|
|
|
|
+
|
|
/* The main function executed by the thread
|
|
/* The main function executed by the thread
|
|
* thread_arg is a structure containing the information needed by the thread
|
|
* thread_arg is a structure containing the information needed by the thread
|
|
*/
|
|
*/
|
|
void* _starpu_sink_thread(void * thread_arg)
|
|
void* _starpu_sink_thread(void * thread_arg)
|
|
{
|
|
{
|
|
/* Retrieve the information from the structure */
|
|
/* Retrieve the information from the structure */
|
|
- struct mp_task **task = ((struct arg_sink_thread *)thread_arg)->task;
|
|
|
|
struct _starpu_mp_node *node = ((struct arg_sink_thread *)thread_arg)->node;
|
|
struct _starpu_mp_node *node = ((struct arg_sink_thread *)thread_arg)->node;
|
|
sem_t * sem = ((struct arg_sink_thread *)thread_arg)->sem;
|
|
sem_t * sem = ((struct arg_sink_thread *)thread_arg)->sem;
|
|
int coreid =((struct arg_sink_thread *)thread_arg)->coreid;
|
|
int coreid =((struct arg_sink_thread *)thread_arg)->coreid;
|
|
/* free the structure */
|
|
/* free the structure */
|
|
free(thread_arg);
|
|
free(thread_arg);
|
|
|
|
|
|
|
|
+ STARPU_PTHREAD_BARRIER_WAIT(&node->init_completed_barrier);
|
|
|
|
+
|
|
|
|
+ struct _starpu_machine_config *config = _starpu_get_machine_config();
|
|
|
|
+ struct _starpu_worker * worker = &config->workers[node->baseworkerid + coreid];
|
|
|
|
+
|
|
|
|
+ _starpu_set_local_worker_key(worker);
|
|
|
|
+ struct _starpu_combined_worker * combined_worker;
|
|
while(node->is_running)
|
|
while(node->is_running)
|
|
{
|
|
{
|
|
/*Wait there is a task available */
|
|
/*Wait there is a task available */
|
|
sem_wait(sem);
|
|
sem_wait(sem);
|
|
-
|
|
|
|
- if((*task) != NULL)
|
|
|
|
|
|
+ struct mp_task *task = node->run_table[coreid];
|
|
|
|
+ combined_worker = NULL;
|
|
|
|
+ if(task != NULL)
|
|
{
|
|
{
|
|
|
|
|
|
/* If it's a parallel task */
|
|
/* If it's a parallel task */
|
|
- if((*task)->is_parallel_task)
|
|
|
|
|
|
+ if(task->is_parallel_task)
|
|
{
|
|
{
|
|
|
|
+ combined_worker = _starpu_get_combined_worker_struct(task->combined_workerid);
|
|
/* Synchronize with others threads of the combined worker*/
|
|
/* Synchronize with others threads of the combined worker*/
|
|
- STARPU_PTHREAD_BARRIER_WAIT(&(*task)->mp_barrier->before_work_barrier);
|
|
|
|
|
|
+ STARPU_PTHREAD_BARRIER_WAIT(&task->mp_barrier->before_work_barrier);
|
|
|
|
|
|
- /* The first thread of the combined worker
|
|
|
|
- * tell the sink that the execution has begun
|
|
|
|
- */
|
|
|
|
- if((*task)->coreid == (*task)->combined_worker[0])
|
|
|
|
- {
|
|
|
|
- /* Init message to tell the sink that the execution has begun */
|
|
|
|
- struct mp_message * message = mp_message_new();
|
|
|
|
- message->type = STARPU_PRE_EXECUTION;
|
|
|
|
- *(int *) message->buffer = (*task)->combined_workerid;
|
|
|
|
- message->size = sizeof((*task)->combined_workerid);
|
|
|
|
|
|
|
|
- /* Append the message to the queue */
|
|
|
|
- pthread_mutex_lock(&node->message_queue_mutex);
|
|
|
|
- mp_message_list_push_front(node->message_queue,message);
|
|
|
|
- pthread_mutex_unlock(&node->message_queue_mutex);
|
|
|
|
|
|
+ /* The first thread of the combined worker */
|
|
|
|
+ if(worker->workerid == combined_worker->combined_workerid[0])
|
|
|
|
+ {
|
|
|
|
+ /* tell the sink that the execution has begun */
|
|
|
|
+ _starpu_sink_common_pre_execution_message(node,task);
|
|
|
|
|
|
/* If the mode is FORKJOIN,
|
|
/* If the mode is FORKJOIN,
|
|
- * the first thread binds himself on all core of the combined worker
|
|
|
|
- */
|
|
|
|
- if((*task)->type == STARPU_FORKJOIN)
|
|
|
|
- node->bind_thread(node, coreid, (*task)->combined_worker, (*task)->combined_worker_size);
|
|
|
|
|
|
+ * the first thread binds himself
|
|
|
|
+ * on all core of the combined worker*/
|
|
|
|
+ if(task->type == STARPU_FORKJOIN)
|
|
|
|
+ {
|
|
|
|
+ _starpu_sink_common_bind_to_combined_worker(node, coreid, combined_worker);
|
|
|
|
+ }
|
|
}
|
|
}
|
|
}
|
|
}
|
|
- if((*task)->type != STARPU_FORKJOIN || (*task)->coreid == (*task)->combined_worker[0])
|
|
|
|
|
|
+ if(task->type != STARPU_FORKJOIN || worker->workerid == combined_worker->combined_workerid[0])
|
|
{
|
|
{
|
|
|
|
+ if(combined_worker != NULL)
|
|
|
|
+ worker->worker_size = combined_worker->worker_size;
|
|
|
|
+ else
|
|
|
|
+ worker->worker_size = 1;
|
|
/* execute the task */
|
|
/* execute the task */
|
|
- (*task)->kernel((*task)->interfaces,(*task)->cl_arg);
|
|
|
|
|
|
+ task->kernel(task->interfaces,task->cl_arg);
|
|
}
|
|
}
|
|
|
|
|
|
/* If it's a parallel task */
|
|
/* If it's a parallel task */
|
|
- if((*task)->is_parallel_task)
|
|
|
|
|
|
+ if(task->is_parallel_task)
|
|
{
|
|
{
|
|
/* Synchronize with others threads of the combined worker*/
|
|
/* Synchronize with others threads of the combined worker*/
|
|
- STARPU_PTHREAD_BARRIER_WAIT(&(*task)->mp_barrier->after_work_barrier);
|
|
|
|
|
|
+ STARPU_PTHREAD_BARRIER_WAIT(&task->mp_barrier->after_work_barrier);
|
|
|
|
|
|
/* The fisrt thread of the combined */
|
|
/* The fisrt thread of the combined */
|
|
- if((*task)->coreid == (*task)->combined_worker[0])
|
|
|
|
|
|
+ if(worker->workerid == combined_worker->combined_workerid[0])
|
|
{
|
|
{
|
|
/* Erase the barrier from the list */
|
|
/* Erase the barrier from the list */
|
|
- pthread_mutex_lock(&node->barrier_mutex);
|
|
|
|
- mp_barrier_list_erase(node->barrier_list,(*task)->mp_barrier);
|
|
|
|
- pthread_mutex_unlock(&node->barrier_mutex);
|
|
|
|
|
|
+ _starpu_sink_common_erase_barrier(node,task->mp_barrier);
|
|
|
|
|
|
/* If the mode is FORKJOIN,
|
|
/* If the mode is FORKJOIN,
|
|
- * the first thread rebinds himself on his own core
|
|
|
|
- */
|
|
|
|
- if((*task)->type == STARPU_FORKJOIN)
|
|
|
|
|
|
+ * the first thread rebinds himself on his own core */
|
|
|
|
+ if(task->type == STARPU_FORKJOIN)
|
|
node->bind_thread(node, coreid, &coreid, 1);
|
|
node->bind_thread(node, coreid, &coreid, 1);
|
|
|
|
|
|
}
|
|
}
|
|
}
|
|
}
|
|
- /* Init message to tell the sink that the execution is completed */
|
|
|
|
- struct mp_message * message = mp_message_new();
|
|
|
|
- message->type = STARPU_EXECUTION_COMPLETED;
|
|
|
|
- message->size = sizeof((*task)->coreid);
|
|
|
|
- *(int*) message->buffer = (*task)->coreid;
|
|
|
|
-
|
|
|
|
- free(*task);
|
|
|
|
- (*task) = NULL;
|
|
|
|
-
|
|
|
|
- /* Append the message to the queue */
|
|
|
|
- pthread_mutex_lock(&node->message_queue_mutex);
|
|
|
|
- mp_message_list_push_front(node->message_queue,message);
|
|
|
|
- pthread_mutex_unlock(&node->message_queue_mutex);
|
|
|
|
|
|
+
|
|
|
|
+ node->run_table[coreid] = NULL;
|
|
|
|
+
|
|
|
|
+ /* tell the sink that the execution is completed */
|
|
|
|
+ _starpu_sink_common_execution_completed_message(node,task);
|
|
|
|
+
|
|
|
|
+ free(task);
|
|
}
|
|
}
|
|
|
|
|
|
}
|
|
}
|
|
@@ -353,39 +527,6 @@ static void _starpu_sink_common_execute_thread(struct _starpu_mp_node *node, str
|
|
sem_post(&node->sem_run_table[task->coreid]);
|
|
sem_post(&node->sem_run_table[task->coreid]);
|
|
}
|
|
}
|
|
|
|
|
|
-/* Search for the mp_barrier correspondind to the specified combined worker
|
|
|
|
- * and create it if it doesn't exist
|
|
|
|
- */
|
|
|
|
-static struct mp_barrier * _starpu_sink_common_get_barrier(struct _starpu_mp_node * node, int cb_workerid, int cb_workersize)
|
|
|
|
-{
|
|
|
|
- struct mp_barrier * b = NULL;
|
|
|
|
- pthread_mutex_lock(&node->barrier_mutex);
|
|
|
|
-
|
|
|
|
- /* Search if the barrier already exist */
|
|
|
|
- for(b = mp_barrier_list_begin(node->barrier_list);
|
|
|
|
- b != mp_barrier_list_end(node->barrier_list) && b->id != cb_workerid;
|
|
|
|
- b = mp_barrier_list_next(b));
|
|
|
|
-
|
|
|
|
- /* If we found the barrier */
|
|
|
|
- if(b != NULL && b->id == cb_workerid)
|
|
|
|
- {
|
|
|
|
- pthread_mutex_unlock(&node->barrier_mutex);
|
|
|
|
- return b;
|
|
|
|
- }
|
|
|
|
- else
|
|
|
|
- {
|
|
|
|
-
|
|
|
|
- /* Else we create, initialize and add it to the list*/
|
|
|
|
- b = mp_barrier_new();
|
|
|
|
- b->id = cb_workerid;
|
|
|
|
- STARPU_PTHREAD_BARRIER_INIT(&b->before_work_barrier,NULL,cb_workersize);
|
|
|
|
- STARPU_PTHREAD_BARRIER_INIT(&b->after_work_barrier,NULL,cb_workersize);
|
|
|
|
- mp_barrier_list_push_back(node->barrier_list,b);
|
|
|
|
- pthread_mutex_unlock(&node->barrier_mutex);
|
|
|
|
- return b;
|
|
|
|
- }
|
|
|
|
-}
|
|
|
|
-
|
|
|
|
|
|
|
|
|
|
|
|
/* Receive paquet from _starpu_src_common_execute_kernel in the form below :
|
|
/* Receive paquet from _starpu_src_common_execute_kernel in the form below :
|
|
@@ -398,7 +539,6 @@ static struct mp_barrier * _starpu_sink_common_get_barrier(struct _starpu_mp_nod
|
|
void _starpu_sink_common_execute(struct _starpu_mp_node *node,
|
|
void _starpu_sink_common_execute(struct _starpu_mp_node *node,
|
|
void *arg, int arg_size)
|
|
void *arg, int arg_size)
|
|
{
|
|
{
|
|
- int id = 0;
|
|
|
|
unsigned nb_interfaces, i;
|
|
unsigned nb_interfaces, i;
|
|
|
|
|
|
void *arg_ptr = arg;
|
|
void *arg_ptr = arg;
|
|
@@ -418,16 +558,7 @@ void _starpu_sink_common_execute(struct _starpu_mp_node *node,
|
|
task->combined_workerid= *(int *) arg_ptr;
|
|
task->combined_workerid= *(int *) arg_ptr;
|
|
arg_ptr += sizeof(task->combined_workerid);
|
|
arg_ptr += sizeof(task->combined_workerid);
|
|
|
|
|
|
- task->combined_worker_size = *(int *) arg_ptr;
|
|
|
|
- arg_ptr += sizeof(task->combined_worker_size);
|
|
|
|
-
|
|
|
|
- for (id = 0; id < task->combined_worker_size; id++)
|
|
|
|
- {
|
|
|
|
-
|
|
|
|
- task->combined_worker[id] = *(int*) arg_ptr;
|
|
|
|
- arg_ptr += sizeof(task->combined_worker[id]);
|
|
|
|
- }
|
|
|
|
- task->mp_barrier = _starpu_sink_common_get_barrier(node,task->combined_workerid,task->combined_worker_size);
|
|
|
|
|
|
+ task->mp_barrier = _starpu_sink_common_get_barrier(node,task->combined_workerid,_starpu_get_combined_worker_struct(task->combined_workerid)->worker_size);
|
|
}
|
|
}
|
|
|
|
|
|
task->coreid = *(unsigned *) arg_ptr;
|
|
task->coreid = *(unsigned *) arg_ptr;
|