X-Git-Url: http://info.iut-bm.univ-fcomte.fr/pub/gitweb/simgrid.git/blobdiff_plain/d47debaeffa6807956ee9cc7da90759efe49beda..0c5fdc16c1cd593d4434110621ca3b6ed81bdec5:/src/smpi/src/smpi_base.c diff --git a/src/smpi/src/smpi_base.c b/src/smpi/src/smpi_base.c index 02bb9a84ae..7d66cf0a32 100644 --- a/src/smpi/src/smpi_base.c +++ b/src/smpi/src/smpi_base.c @@ -1,557 +1,820 @@ #include +#include #include -#include "msg/msg.h" -#include "xbt/sysdep.h" -#include "xbt/xbt_portability.h" -#include "smpi.h" -smpi_mpi_request_t **smpi_pending_send_requests = NULL; -smpi_mpi_request_t **smpi_last_pending_send_requests = NULL; +#include "private.h" -smpi_mpi_request_t **smpi_pending_recv_requests = NULL; -smpi_mpi_request_t **smpi_last_pending_recv_requests = NULL; +SMPI_Global_t smpi_global = NULL; -smpi_received_t **smpi_received = NULL; -smpi_received_t **smpi_last_received = NULL; +SMPI_MPI_Global_t smpi_mpi_global = NULL; -m_process_t *smpi_sender_processes = NULL; -m_process_t *smpi_receiver_processes = NULL; +XBT_LOG_NEW_DEFAULT_CATEGORY(smpi, "SMPI"); -int smpi_running_hosts = 0; - -smpi_mpi_communicator_t smpi_mpi_comm_world; +int inline smpi_mpi_comm_size(smpi_mpi_communicator_t *comm) +{ + return comm->size; +} -smpi_mpi_status_t smpi_mpi_status_ignore; +// FIXME: smarter algorithm? +int smpi_mpi_comm_rank(smpi_mpi_communicator_t *comm, smx_host_t host) +{ + int i; -smpi_mpi_datatype_t smpi_mpi_byte; -smpi_mpi_datatype_t smpi_mpi_int; -smpi_mpi_datatype_t smpi_mpi_double; + for(i = comm->size - 1; i > 0 && host != comm->hosts[i]; i--); -smpi_mpi_op_t smpi_mpi_land; -smpi_mpi_op_t smpi_mpi_sum; + return i; +} -static xbt_os_timer_t smpi_timer; -static int smpi_benchmarking; -static double smpi_reference; +int inline smpi_mpi_comm_rank_self(smpi_mpi_communicator_t *comm) +{ + return smpi_mpi_comm_rank(comm, SIMIX_host_self()); +} -void smpi_mpi_land_func(void *x, void *y, void *z) { - *(int *)z = *(int *)x && *(int *)y; +int inline smpi_mpi_comm_world_rank_self() +{ + return smpi_mpi_comm_rank(smpi_mpi_global->mpi_comm_world, SIMIX_host_self()); } -void smpi_mpi_sum_func(void *x, void *y, void *z) { - *(int *)z = *(int *)x + *(int *)y; +int smpi_sender(int argc, char **argv) +{ + smx_process_t self; + smx_host_t shost; + int rank; + + xbt_fifo_t request_queue; + smx_mutex_t request_queue_mutex; + int size; + + int running_hosts_count; + + smpi_mpi_request_t *request; + + smx_host_t dhost; + + smx_action_t communicate_action; + + smpi_received_message_t *message; + + int drank; + + smx_process_t receiver_process; + + self = SIMIX_process_self(); + shost = SIMIX_host_self(); + rank = smpi_mpi_comm_rank(smpi_mpi_global->mpi_comm_world, shost); + + // make sure root is done before own initialization + SIMIX_mutex_lock(smpi_global->start_stop_mutex); + if (!smpi_global->root_ready) { + SIMIX_cond_wait(smpi_global->start_stop_cond, smpi_global->start_stop_mutex); + } + SIMIX_mutex_unlock(smpi_global->start_stop_mutex); + + request_queue = smpi_global->pending_send_request_queues[rank]; + request_queue_mutex = smpi_global->pending_send_request_queues_mutexes[rank]; + size = smpi_mpi_comm_size(smpi_mpi_global->mpi_comm_world); + + smpi_global->sender_processes[rank] = self; + + // wait for all nodes to signal initializatin complete + SIMIX_mutex_lock(smpi_global->start_stop_mutex); + smpi_global->ready_process_count++; + if (smpi_global->ready_process_count < 3 * size) { + SIMIX_cond_wait(smpi_global->start_stop_cond, smpi_global->start_stop_mutex); + } else { + SIMIX_cond_broadcast(smpi_global->start_stop_cond); + } + SIMIX_mutex_unlock(smpi_global->start_stop_mutex); + + do { + + SIMIX_mutex_lock(request_queue_mutex); + request = xbt_fifo_shift(request_queue); + SIMIX_mutex_unlock(request_queue_mutex); + + if (NULL == request) { + SIMIX_process_suspend(self); + } else { + + SIMIX_mutex_lock(request->mutex); + + // copy request to appropriate received queue + message = xbt_mallocator_get(smpi_global->message_mallocator); + message->comm = request->comm; + message->src = request->src; + message->dst = request->dst; + message->tag = request->tag; + message->buf = xbt_malloc(request->datatype->size * request->count); + memcpy(message->buf, request->buf, request->datatype->size * request->count); + + dhost = request->comm->hosts[request->dst]; + drank = smpi_mpi_comm_rank(smpi_mpi_global->mpi_comm_world, dhost); + + SIMIX_mutex_lock(smpi_global->received_message_queues_mutexes[drank]); + xbt_fifo_push(smpi_global->received_message_queues[drank], message); + SIMIX_mutex_unlock(smpi_global->received_message_queues_mutexes[drank]); + + request->completed = 1; + + communicate_action = SIMIX_action_communicate(shost, dhost, + "communication", request->datatype->size * request->count * 1.0, -1.0); + + SIMIX_register_condition_to_action(communicate_action, request->cond); + SIMIX_register_action_to_condition(communicate_action, request->cond); + + SIMIX_cond_wait(request->cond, request->mutex); + + SIMIX_mutex_unlock(request->mutex); + + // wake up receiver if necessary + receiver_process = smpi_global->receiver_processes[drank]; + + if (SIMIX_process_is_suspended(receiver_process)) { + SIMIX_process_resume(receiver_process); + } + + } + + SIMIX_mutex_lock(smpi_global->running_hosts_count_mutex); + running_hosts_count = smpi_global->running_hosts_count; + SIMIX_mutex_unlock(smpi_global->running_hosts_count_mutex); + + } while (0 < running_hosts_count); + + SIMIX_mutex_lock(smpi_global->start_stop_mutex); + smpi_global->ready_process_count--; + if (smpi_global->ready_process_count == 0) { + SIMIX_cond_broadcast(smpi_global->start_stop_cond); + } else if (smpi_global->ready_process_count < 0) { + // FIXME: can't happen! abort! + } + SIMIX_mutex_unlock(smpi_global->start_stop_mutex); + + return 0; } -void smpi_mpi_init() { - int i; - int size, rank; - m_host_t *hosts; - m_host_t host; - double duration; - m_task_t mtask; - - // will eventually need mutex - smpi_running_hosts++; - - // initialize some local variables - size = MSG_get_host_number(); - host = MSG_host_self(); - hosts = MSG_get_host_table(); - for(i = 0; i < size && host != hosts[i]; i++); - rank = i; - - // node 0 sets the globals - if (0 == rank) { - - // global communicator - smpi_mpi_comm_world.id = 0; - smpi_mpi_comm_world.size = size; - smpi_mpi_comm_world.barrier = 0; - smpi_mpi_comm_world.hosts = hosts; - smpi_mpi_comm_world.processes = xbt_malloc(sizeof(m_process_t) * size); - smpi_mpi_comm_world.processes[0] = MSG_process_self(); - - // mpi datatypes - smpi_mpi_byte.size = (size_t)1; - smpi_mpi_int.size = sizeof(int); - smpi_mpi_double.size = sizeof(double); - - // mpi operations - smpi_mpi_land.func = &smpi_mpi_land_func; - smpi_mpi_sum.func = &smpi_mpi_sum_func; - - // smpi globals - smpi_pending_send_requests = xbt_malloc(sizeof(smpi_mpi_request_t*) * size); - smpi_last_pending_send_requests = xbt_malloc(sizeof(smpi_mpi_request_t*) * size); - smpi_pending_recv_requests = xbt_malloc(sizeof(smpi_mpi_request_t*) * size); - smpi_last_pending_recv_requests = xbt_malloc(sizeof(smpi_mpi_request_t*) * size); - smpi_received = xbt_malloc(sizeof(smpi_received_t*) * size); - smpi_last_received = xbt_malloc(sizeof(smpi_received_t*) * size); - smpi_sender_processes = xbt_malloc(sizeof(m_process_t) * size); - smpi_receiver_processes = xbt_malloc(sizeof(m_process_t) * size); - for(i = 0; i < size; i++) { - smpi_pending_send_requests[i] = NULL; - smpi_last_pending_send_requests[i] = NULL; - smpi_pending_recv_requests[i] = NULL; - smpi_last_pending_recv_requests[i] = NULL; - smpi_received[i] = NULL; - smpi_last_received[i] = NULL; - } - smpi_timer = xbt_os_timer_new(); - smpi_reference = DEFAULT_POWER; - smpi_benchmarking = 0; - - // tell send/recv nodes to begin - for(i = 0; i < size; i++) { - mtask = MSG_task_create("READY", 0, 0, NULL); - MSG_task_put(mtask, hosts[i], SEND_SYNC_PORT); - mtask = (m_task_t)0; - MSG_task_get_from_host(&mtask, SEND_SYNC_PORT, hosts[i]); - MSG_task_destroy(mtask); - mtask = MSG_task_create("READY", 0, 0, NULL); - MSG_task_put(mtask, hosts[i], RECV_SYNC_PORT); - mtask = (m_task_t)0; - MSG_task_get_from_host(&mtask, RECV_SYNC_PORT, hosts[i]); - MSG_task_destroy(mtask); - } - - // now everyone else - for(i = 1; i < size; i++) { - mtask = MSG_task_create("READY", 0, 0, NULL); - MSG_task_put(mtask, hosts[i], MPI_PORT); - } - - } else { - // everyone needs to wait for node 0 to finish - mtask = (m_task_t)0; - MSG_task_get(&mtask, MPI_PORT); - MSG_task_destroy(mtask); - smpi_mpi_comm_world.processes[rank] = MSG_process_self(); - } - - // now that mpi_comm_world_processes is set, it's safe to set a barrier - smpi_barrier(&smpi_mpi_comm_world); +int smpi_receiver(int argc, char **argv) +{ + smx_process_t self; + int rank; + + xbt_fifo_t request_queue; + smx_mutex_t request_queue_mutex; + xbt_fifo_t message_queue; + smx_mutex_t message_queue_mutex; + int size; + + int running_hosts_count; + + smpi_mpi_request_t *request; + smpi_received_message_t *message; + + xbt_fifo_item_t request_item; + xbt_fifo_item_t message_item; + + smx_process_t waitproc; + + self = SIMIX_process_self(); + rank = smpi_mpi_comm_world_rank_self(); + + // make sure root is done before own initialization + SIMIX_mutex_lock(smpi_global->start_stop_mutex); + if (!smpi_global->root_ready) { + SIMIX_cond_wait(smpi_global->start_stop_cond, smpi_global->start_stop_mutex); + } + SIMIX_mutex_unlock(smpi_global->start_stop_mutex); + + request_queue = smpi_global->pending_recv_request_queues[rank]; + request_queue_mutex = smpi_global->pending_recv_request_queues_mutexes[rank]; + message_queue = smpi_global->received_message_queues[rank]; + message_queue_mutex = smpi_global->received_message_queues_mutexes[rank]; + size = smpi_mpi_comm_size(smpi_mpi_global->mpi_comm_world); + + smpi_global->receiver_processes[rank] = self; + + // wait for all nodes to signal initializatin complete + SIMIX_mutex_lock(smpi_global->start_stop_mutex); + smpi_global->ready_process_count++; + if (smpi_global->ready_process_count < 3 * size) { + SIMIX_cond_wait(smpi_global->start_stop_cond, smpi_global->start_stop_mutex); + } else { + SIMIX_cond_broadcast(smpi_global->start_stop_cond); + } + SIMIX_mutex_unlock(smpi_global->start_stop_mutex); + + do { + request = NULL; + message = NULL; + + // FIXME: better algorithm, maybe some kind of balanced tree? or a heap? + + // FIXME: not the best way to request multiple locks... + SIMIX_mutex_lock(request_queue_mutex); + SIMIX_mutex_lock(message_queue_mutex); + for (request_item = xbt_fifo_get_first_item(request_queue); + NULL != request_item; + request_item = xbt_fifo_get_next_item(request_item)) { + request = xbt_fifo_get_item_content(request_item); + for (message_item = xbt_fifo_get_first_item(message_queue); + NULL != message_item; + message_item = xbt_fifo_get_next_item(message_item)) { + message = xbt_fifo_get_item_content(message_item); + if (request->comm == message->comm && + (MPI_ANY_SOURCE == request->src || request->src == message->src) && + request->tag == message->tag) { + xbt_fifo_remove_item(request_queue, request_item); + xbt_fifo_remove_item(message_queue, message_item); + goto stopsearch; + } + } + } +stopsearch: + SIMIX_mutex_unlock(message_queue_mutex); + SIMIX_mutex_unlock(request_queue_mutex); + + if (NULL == request || NULL == message) { + SIMIX_process_suspend(self); + } else { + SIMIX_mutex_lock(request->mutex); + + memcpy(request->buf, message->buf, request->datatype->size * request->count); + request->src = message->src; + request->completed = 1; + SIMIX_cond_broadcast(request->cond); + + SIMIX_mutex_unlock(request->mutex); + + xbt_free(message->buf); + xbt_mallocator_release(smpi_global->message_mallocator, message); + } + + SIMIX_mutex_lock(smpi_global->running_hosts_count_mutex); + running_hosts_count = smpi_global->running_hosts_count; + SIMIX_mutex_unlock(smpi_global->running_hosts_count_mutex); + + } while (0 < running_hosts_count); + + SIMIX_mutex_lock(smpi_global->start_stop_mutex); + smpi_global->ready_process_count--; + if (smpi_global->ready_process_count == 0) { + SIMIX_cond_broadcast(smpi_global->start_stop_cond); + } else if (smpi_global->ready_process_count < 0) { + // FIXME: can't happen, abort! + } + SIMIX_mutex_unlock(smpi_global->start_stop_mutex); + + return 0; } -void smpi_mpi_finalize() { - int i; - smpi_running_hosts--; - if (0 <= smpi_running_hosts) { - for(i = 0; i < smpi_mpi_comm_world.size; i++) { - if(MSG_process_is_suspended(smpi_sender_processes[i])) { - MSG_process_resume(smpi_sender_processes[i]); - } - if(MSG_process_is_suspended(smpi_receiver_processes[i])) { - MSG_process_resume(smpi_receiver_processes[i]); - } - } - } else { - xbt_free(smpi_mpi_comm_world.processes); - xbt_free(smpi_pending_send_requests); - xbt_free(smpi_last_pending_send_requests); - xbt_free(smpi_pending_recv_requests); - xbt_free(smpi_last_pending_recv_requests); - xbt_free(smpi_received); - xbt_free(smpi_last_received); - xbt_free(smpi_sender_processes); - xbt_free(smpi_receiver_processes); - xbt_os_timer_free(smpi_timer); - } +void *smpi_request_new() +{ + smpi_mpi_request_t *request = xbt_new(smpi_mpi_request_t, 1); + + request->completed = 0; + request->mutex = SIMIX_mutex_init(); + request->cond = SIMIX_cond_init(); + + return request; } -void smpi_complete(smpi_mpi_request_t *request) { - smpi_waitlist_node_t *current, *next; - request->completed = 1; - request->next = NULL; - current = request->waitlist; - while(NULL != current) { - if(MSG_process_is_suspended(current->process)) { - MSG_process_resume(current->process); - } - next = current->next; - xbt_free(current); - current = next; - } - request->waitlist = NULL; +void smpi_request_free(void *pointer) { + + smpi_mpi_request_t *request = pointer; + + if (NULL != request) { + SIMIX_cond_destroy(request->cond); + SIMIX_mutex_destroy(request->mutex); + xbt_free(request); + } + + return; } -int smpi_host_rank_self() { - return smpi_comm_rank(&smpi_mpi_comm_world, MSG_host_self()); +void smpi_request_reset(void *pointer) { + return; } -void smpi_isend(smpi_mpi_request_t *sendreq) { - int rank = smpi_host_rank_self(); - if (NULL == smpi_last_pending_send_requests[rank]) { - smpi_pending_send_requests[rank] = sendreq; - } else { - smpi_last_pending_send_requests[rank]->next = sendreq; - } - smpi_last_pending_send_requests[rank] = sendreq; - if (MSG_process_is_suspended(smpi_sender_processes[rank])) { - MSG_process_resume(smpi_sender_processes[rank]); - } + +void *smpi_message_new() +{ + return xbt_new(smpi_received_message_t, 1); } -void smpi_match_requests(int rank) { - smpi_mpi_request_t *frequest, *prequest, *crequest; - smpi_received_t *freceived, *preceived, *creceived; - size_t dsize; - short int match; - frequest = smpi_pending_recv_requests[rank]; - prequest = NULL; - crequest = frequest; - while(NULL != crequest) { - freceived = smpi_received[rank]; - preceived = NULL; - creceived = freceived; - match = 0; - while(NULL != creceived && !match) { - if(crequest->comm->id == creceived->commid && - (MPI_ANY_SOURCE == crequest->src || crequest->src == creceived->src) && - crequest->tag == creceived->tag) { - - // we have a match! - match = 1; - - // pull the request from the queue - if(NULL == prequest) { - frequest = crequest->next; - smpi_pending_recv_requests[rank] = frequest; - } else { - prequest->next = crequest->next; - } - if(crequest == smpi_last_pending_recv_requests[rank]) { - smpi_last_pending_recv_requests[rank] = prequest; - } - - // pull the received data from the queue - if(NULL == preceived) { - freceived = creceived->next; - smpi_received[rank] = freceived; - } else { - preceived->next = creceived->next; - } - if(creceived == smpi_last_received[rank]) { - smpi_last_received[rank] = preceived; - } - - // for when request->src is any source - crequest->src = creceived->src; - - // calculate data size - dsize = crequest->count * crequest->datatype->size; - - // copy data to buffer - memcpy(crequest->buf, creceived->data, dsize); - - // fwd through - crequest->fwdthrough = creceived->fwdthrough; - - // get rid of received data node, no longer needed - xbt_free(creceived->data); - xbt_free(creceived); - - if (crequest->fwdthrough == rank) { - smpi_complete(crequest); - } else { - crequest->src = rank; - crequest->dst = (rank + 1) % crequest->comm->size; - smpi_isend(crequest); - } - - } else { - preceived = creceived; - creceived = creceived->next; - } - } - prequest = crequest; - crequest = crequest->next; - } +void smpi_message_free(void *pointer) +{ + if (NULL != pointer) { + xbt_free(pointer); + } + + return; } -void smpi_bench_begin() { - xbt_assert0(!smpi_benchmarking, "Already benchmarking"); - smpi_benchmarking = 1; - xbt_os_timer_start(smpi_timer); - return; +void smpi_message_reset(void *pointer) +{ + return; } -void smpi_bench_end() { - m_task_t ctask = NULL; - double duration; - xbt_assert0(smpi_benchmarking, "Not benchmarking yet"); - smpi_benchmarking = 0; - xbt_os_timer_stop(smpi_timer); - duration = xbt_os_timer_elapsed(smpi_timer); - ctask = MSG_task_create("computation", duration * smpi_reference, 0 , NULL); - MSG_task_execute(ctask); - MSG_task_destroy(ctask); - return; +void smpi_global_init() +{ + int i; + + int size = SIMIX_host_get_number(); + + smpi_global = xbt_new(s_SMPI_Global_t, 1); + + // config variable + smpi_global->reference_speed = SMPI_DEFAULT_SPEED; + + smpi_global->root_ready = 0; + smpi_global->ready_process_count = 0; + + // start/stop + smpi_global->start_stop_mutex = SIMIX_mutex_init(); + smpi_global->start_stop_cond = SIMIX_cond_init(); + + // processes + smpi_global->sender_processes = xbt_new(smx_process_t, size); + smpi_global->receiver_processes = xbt_new(smx_process_t, size); + + // running hosts + smpi_global->running_hosts_count_mutex = SIMIX_mutex_init(); + smpi_global->running_hosts_count = 0; + + // mallocators + smpi_global->request_mallocator = xbt_mallocator_new(SMPI_REQUEST_MALLOCATOR_SIZE, + smpi_request_new, smpi_request_free, smpi_request_reset); + smpi_global->message_mallocator = xbt_mallocator_new(SMPI_MESSAGE_MALLOCATOR_SIZE, + smpi_message_new, smpi_message_free, smpi_message_reset); + + // + smpi_global->pending_send_request_queues = xbt_new(xbt_fifo_t, size); + smpi_global->pending_send_request_queues_mutexes = xbt_new(smx_mutex_t, size); + smpi_global->pending_recv_request_queues = xbt_new(xbt_fifo_t, size); + smpi_global->pending_recv_request_queues_mutexes = xbt_new(smx_mutex_t, size); + smpi_global->received_message_queues = xbt_new(xbt_fifo_t, size); + smpi_global->received_message_queues_mutexes = xbt_new(smx_mutex_t, size); + smpi_global->timers = xbt_new(xbt_os_timer_t, size); + smpi_global->timers_mutexes = xbt_new(smx_mutex_t, size); + + for(i = 0; i < size; i++) { + smpi_global->pending_send_request_queues[i] = xbt_fifo_new(); + smpi_global->pending_send_request_queues_mutexes[i] = SIMIX_mutex_init(); + smpi_global->pending_recv_request_queues[i] = xbt_fifo_new(); + smpi_global->pending_recv_request_queues_mutexes[i] = SIMIX_mutex_init(); + smpi_global->received_message_queues[i] = xbt_fifo_new(); + smpi_global->received_message_queues_mutexes[i] = SIMIX_mutex_init(); + smpi_global->timers[i] = xbt_os_timer_new(); + smpi_global->timers_mutexes[i] = SIMIX_mutex_init(); + } + } -int smpi_create_request(void *buf, int count, smpi_mpi_datatype_t *datatype, - int src, int dst, int tag, smpi_mpi_communicator_t *comm, smpi_mpi_request_t **request) { - int retval = MPI_SUCCESS; - *request = NULL; - if (NULL == buf && 0 < count) { - retval = MPI_ERR_INTERN; - } else if (0 > count) { - retval = MPI_ERR_COUNT; - } else if (NULL == datatype) { - retval = MPI_ERR_TYPE; - } else if (NULL == comm) { - retval = MPI_ERR_COMM; - } else if (MPI_ANY_SOURCE != src && (0 > src || comm->size <= src)) { - retval = MPI_ERR_RANK; - } else if (0 > dst || comm->size <= dst) { - retval = MPI_ERR_RANK; - } else if (0 > tag) { - retval = MPI_ERR_TAG; - } else { - *request = xbt_malloc(sizeof(smpi_mpi_request_t)); - (*request)->buf = buf; - (*request)->count = count; - (*request)->datatype = datatype; - (*request)->src = src; - (*request)->dst = dst; - (*request)->tag = tag; - (*request)->comm = comm; - (*request)->completed = 0; - (*request)->fwdthrough = dst; - (*request)->waitlist = NULL; - (*request)->next = NULL; - } - return retval; +void smpi_global_destroy() +{ + int i; + + int size = SIMIX_host_get_number(); + + // start/stop + SIMIX_mutex_destroy(smpi_global->start_stop_mutex); + SIMIX_cond_destroy(smpi_global->start_stop_cond); + + // processes + xbt_free(smpi_global->sender_processes); + xbt_free(smpi_global->receiver_processes); + + // running hosts + SIMIX_mutex_destroy(smpi_global->running_hosts_count_mutex); + + // mallocators + xbt_mallocator_free(smpi_global->request_mallocator); + xbt_mallocator_free(smpi_global->message_mallocator); + + for(i = 0; i < size; i++) { + xbt_fifo_free(smpi_global->pending_send_request_queues[i]); + SIMIX_mutex_destroy(smpi_global->pending_send_request_queues_mutexes[i]); + xbt_fifo_free(smpi_global->pending_recv_request_queues[i]); + SIMIX_mutex_destroy(smpi_global->pending_recv_request_queues_mutexes[i]); + xbt_fifo_free(smpi_global->received_message_queues[i]); + SIMIX_mutex_destroy(smpi_global->received_message_queues_mutexes[i]); + xbt_os_timer_free(smpi_global->timers[i]); + SIMIX_mutex_destroy(smpi_global->timers_mutexes[i]); + } + + xbt_free(smpi_global->pending_send_request_queues); + xbt_free(smpi_global->pending_send_request_queues_mutexes); + xbt_free(smpi_global->pending_recv_request_queues); + xbt_free(smpi_global->pending_recv_request_queues_mutexes); + xbt_free(smpi_global->received_message_queues); + xbt_free(smpi_global->received_message_queues_mutexes); + xbt_free(smpi_global->timers); + xbt_free(smpi_global->timers_mutexes); + + xbt_free(smpi_global); } -void smpi_barrier(smpi_mpi_communicator_t *comm) { - int i; - comm->barrier++; - if(comm->barrier < comm->size) { - MSG_process_suspend(MSG_process_self()); - } else { - comm->barrier = 0; - for(i = 0; i < comm->size; i++) { - if (MSG_process_is_suspended(comm->processes[i])) { - MSG_process_resume(comm->processes[i]); - } - } - } +int smpi_run_simulation(int argc, char **argv) +{ + smx_cond_t cond = NULL; + smx_action_t action = NULL; + + xbt_fifo_t actions_failed = xbt_fifo_new(); + xbt_fifo_t actions_done = xbt_fifo_new(); + + srand(SMPI_RAND_SEED); + + SIMIX_global_init(&argc, argv); + + SIMIX_function_register("smpi_simulated_main", smpi_simulated_main); + SIMIX_function_register("smpi_sender", smpi_sender); + SIMIX_function_register("smpi_receiver", smpi_receiver); + + // FIXME: ought to verify these files... + SIMIX_create_environment(argv[1]); + + // must initialize globals between creating environment and launching app.... + smpi_global_init(); + + SIMIX_launch_application(argv[2]); + + /* Prepare to display some more info when dying on Ctrl-C pressing */ + // FIXME: doesn't work + //signal(SIGINT, inthandler); + + /* Clean IO before the run */ + fflush(stdout); + fflush(stderr); + + while (SIMIX_solve(actions_done, actions_failed) != -1.0) { + while (action = xbt_fifo_pop(actions_failed)) { + DEBUG1("** %s failed **", action->name); + while (cond = xbt_fifo_pop(action->cond_list)) { + SIMIX_cond_broadcast(cond); + } + SIMIX_action_destroy(action); + } + while (action = xbt_fifo_pop(actions_done)) { + DEBUG1("** %s done **",action->name); + while (cond = xbt_fifo_pop(action->cond_list)) { + SIMIX_cond_broadcast(cond); + } + SIMIX_action_destroy(action); + } + } + + xbt_fifo_free(actions_failed); + xbt_fifo_free(actions_done); + + INFO1("simulation time %g", SIMIX_get_clock()); + + smpi_global_destroy(); + + SIMIX_clean(); + + return 0; } -int smpi_comm_rank(smpi_mpi_communicator_t *comm, m_host_t host) { - int i; - for(i = 0; i < comm->size && host != comm->hosts[i]; i++); - if (i >= comm->size) i = -1; - return i; +void smpi_mpi_land_func(void *x, void *y, void *z) +{ + *(int *)z = *(int *)x && *(int *)y; } -void smpi_irecv(smpi_mpi_request_t *recvreq) { - int rank = smpi_host_rank_self(); - if (NULL == smpi_pending_recv_requests[rank]) { - smpi_pending_recv_requests[rank] = recvreq; - } else if (NULL != smpi_last_pending_recv_requests[rank]) { - smpi_last_pending_recv_requests[rank]->next = recvreq; - } else { // can't happen! - fprintf(stderr, "smpi_pending_recv_requests not null while smpi_last_pending_recv_requests null!\n"); - } - smpi_last_pending_recv_requests[rank] = recvreq; - smpi_match_requests(rank); - if (MSG_process_is_suspended(smpi_receiver_processes[rank])) { - MSG_process_resume(smpi_receiver_processes[rank]); - } +void smpi_mpi_sum_func(void *x, void *y, void *z) +{ + *(int *)z = *(int *)x + *(int *)y; } -void smpi_wait(smpi_mpi_request_t *request, smpi_mpi_status_t *status) { - smpi_waitlist_node_t *waitnode, *current; - if (NULL != request) { - if (!request->completed) { - waitnode = xbt_malloc(sizeof(smpi_waitlist_node_t)); - waitnode->process = MSG_process_self(); - waitnode->next = NULL; - if (NULL == request->waitlist) { - request->waitlist = waitnode; - } else { - for(current = request->waitlist; NULL != current->next; current = current->next); - current->next = waitnode; - } - MSG_process_suspend(waitnode->process); - } - if (NULL != status && MPI_STATUS_IGNORE != status) { - status->MPI_SOURCE = request->src; - } - } + +void smpi_mpi_init() +{ + int i; + smx_process_t process; + smx_host_t host; + smx_host_t *hosts; + int size; + double duration; + + SIMIX_mutex_lock(smpi_global->running_hosts_count_mutex); + smpi_global->running_hosts_count++; + SIMIX_mutex_unlock(smpi_global->running_hosts_count_mutex); + + // initialize some local variables + process = SIMIX_process_self(); + host = SIMIX_host_self(); + hosts = SIMIX_host_get_table(); + size = SIMIX_host_get_number(); + + // node 0 sets the globals + if (host == hosts[0]) { + + smpi_mpi_global = xbt_new(s_SMPI_MPI_Global_t, 1); + + // global communicator + smpi_mpi_global->mpi_comm_world = xbt_new(smpi_mpi_communicator_t, 1); + smpi_mpi_global->mpi_comm_world->size = size; + smpi_mpi_global->mpi_comm_world->barrier_count = 0; + smpi_mpi_global->mpi_comm_world->barrier_mutex = SIMIX_mutex_init(); + smpi_mpi_global->mpi_comm_world->barrier_cond = SIMIX_cond_init(); + smpi_mpi_global->mpi_comm_world->hosts = hosts; + smpi_mpi_global->mpi_comm_world->processes = xbt_new(smx_process_t, size); + smpi_mpi_global->mpi_comm_world->processes[0] = process; + + // mpi datatypes + smpi_mpi_global->mpi_byte = xbt_new(smpi_mpi_datatype_t, 1); + smpi_mpi_global->mpi_byte->size = (size_t)1; + smpi_mpi_global->mpi_int = xbt_new(smpi_mpi_datatype_t, 1); + smpi_mpi_global->mpi_int->size = sizeof(int); + smpi_mpi_global->mpi_double = xbt_new(smpi_mpi_datatype_t, 1); + smpi_mpi_global->mpi_double->size = sizeof(double); + + // mpi operations + smpi_mpi_global->mpi_land = xbt_new(smpi_mpi_op_t, 1); + smpi_mpi_global->mpi_land->func = smpi_mpi_land_func; + smpi_mpi_global->mpi_sum = xbt_new(smpi_mpi_op_t, 1); + smpi_mpi_global->mpi_sum->func = smpi_mpi_sum_func; + + // signal all nodes to perform initialization + SIMIX_mutex_lock(smpi_global->start_stop_mutex); + smpi_global->root_ready = 1; + SIMIX_cond_broadcast(smpi_global->start_stop_cond); + SIMIX_mutex_unlock(smpi_global->start_stop_mutex); + + } else { + + // make sure root is done before own initialization + SIMIX_mutex_lock(smpi_global->start_stop_mutex); + if (!smpi_global->root_ready) { + SIMIX_cond_wait(smpi_global->start_stop_cond, smpi_global->start_stop_mutex); + } + SIMIX_mutex_unlock(smpi_global->start_stop_mutex); + + smpi_mpi_global->mpi_comm_world->processes[smpi_mpi_comm_world_rank_self()] = process; + } + + // wait for all nodes to signal initializatin complete + SIMIX_mutex_lock(smpi_global->start_stop_mutex); + smpi_global->ready_process_count++; + if (smpi_global->ready_process_count < 3 * size) { + SIMIX_cond_wait(smpi_global->start_stop_cond, smpi_global->start_stop_mutex); + } else { + SIMIX_cond_broadcast(smpi_global->start_stop_cond); + } + SIMIX_mutex_unlock(smpi_global->start_stop_mutex); + + return; } -void smpi_wait_all(int count, smpi_mpi_request_t **requests, smpi_mpi_status_t *statuses) { - int i; - for (i = 0; i < count; i++) { - smpi_wait(requests[i], &statuses[i]); - } +void smpi_mpi_finalize() +{ + int i; + + SIMIX_mutex_lock(smpi_global->running_hosts_count_mutex); + i = --smpi_global->running_hosts_count; + SIMIX_mutex_unlock(smpi_global->running_hosts_count_mutex); + + SIMIX_mutex_lock(smpi_global->start_stop_mutex); + smpi_global->ready_process_count--; + SIMIX_mutex_unlock(smpi_global->start_stop_mutex); + + if (0 >= i) { + + // wake up senders/receivers + for (i = 0; i < smpi_mpi_global->mpi_comm_world->size; i++) { + if (SIMIX_process_is_suspended(smpi_global->sender_processes[i])) { + SIMIX_process_resume(smpi_global->sender_processes[i]); + } + if (SIMIX_process_is_suspended(smpi_global->receiver_processes[i])) { + SIMIX_process_resume(smpi_global->receiver_processes[i]); + } + } + + // wait for senders/receivers to exit... + SIMIX_mutex_lock(smpi_global->start_stop_mutex); + if (smpi_global->ready_process_count > 0) { + SIMIX_cond_wait(smpi_global->start_stop_cond, smpi_global->start_stop_mutex); + } + SIMIX_mutex_unlock(smpi_global->start_stop_mutex); + + SIMIX_mutex_destroy(smpi_mpi_global->mpi_comm_world->barrier_mutex); + SIMIX_cond_destroy(smpi_mpi_global->mpi_comm_world->barrier_cond); + xbt_free(smpi_mpi_global->mpi_comm_world->processes); + xbt_free(smpi_mpi_global->mpi_comm_world); + + xbt_free(smpi_mpi_global->mpi_byte); + xbt_free(smpi_mpi_global->mpi_int); + xbt_free(smpi_mpi_global->mpi_double); + + xbt_free(smpi_mpi_global->mpi_land); + xbt_free(smpi_mpi_global->mpi_sum); + + xbt_free(smpi_mpi_global); + } + } -void smpi_wait_all_nostatus(int count, smpi_mpi_request_t **requests) { - int i; - for (i = 0; i < count; i++) { - smpi_wait(requests[i], MPI_STATUS_IGNORE); - } +// FIXME: could cause trouble with multithreaded procs on same host... +void smpi_bench_begin() +{ + int rank = smpi_mpi_comm_world_rank_self(); + SIMIX_mutex_lock(smpi_global->timers_mutexes[rank]); + xbt_os_timer_start(smpi_global->timers[rank]); + return; } -int smpi_sender(int argc, char *argv[]) { - m_process_t process; - char taskname[50]; - size_t dsize; - void *data; - m_host_t dhost; - m_task_t mtask; - int rank, fc, ft; - smpi_mpi_request_t *sendreq; - - process = MSG_process_self(); - - // wait for init - mtask = (m_task_t)0; - MSG_task_get(&mtask, SEND_SYNC_PORT); - - rank = smpi_host_rank_self(); - - smpi_sender_processes[rank] = process; - - // ready! - MSG_task_put(mtask, MSG_task_get_source(mtask), SEND_SYNC_PORT); - - while (0 < smpi_running_hosts) { - sendreq = smpi_pending_send_requests[rank]; - if (NULL != sendreq) { - - // pull from queue if not a fwd or no more to fwd - if (sendreq->dst == sendreq->fwdthrough) { - smpi_pending_send_requests[rank] = sendreq->next; - if(sendreq == smpi_last_pending_send_requests[rank]) { - smpi_last_pending_send_requests[rank] = NULL; - } - ft = sendreq->dst; - } else { - fc = ((sendreq->fwdthrough - sendreq->dst + sendreq->comm->size) % sendreq->comm->size) / 2; - ft = (sendreq->dst + fc) % sendreq->comm->size; - //printf("node %d sending broadcast to node %d through node %d\n", rank, sendreq->dst, ft); - } - - // create task to send - sprintf(taskname, "comm:%d,src:%d,dst:%d,tag:%d,ft:%d", sendreq->comm->id, sendreq->src, sendreq->dst, sendreq->tag, ft); - dsize = sendreq->count * sendreq->datatype->size; - data = xbt_malloc(dsize); - memcpy(data, sendreq->buf, dsize); - mtask = MSG_task_create(taskname, 0, dsize, data); - - // figure out which host to send it to - dhost = sendreq->comm->hosts[sendreq->dst]; - - // send task - #ifdef DEBUG - printf("host %s attempting to send to host %s\n", MSG_host_get_name(MSG_host_self()), MSG_host_get_name(dhost)); - #endif - MSG_task_put(mtask, dhost, MPI_PORT); - - if (sendreq->dst == sendreq->fwdthrough) { - smpi_complete(sendreq); - } else { - sendreq->dst = (sendreq->dst + fc + 1) % sendreq->comm->size; - } - - } else { - MSG_process_suspend(process); - } - } - return 0; +void smpi_bench_end() +{ + int rank = smpi_mpi_comm_world_rank_self(); + double duration; + smx_host_t host; + smx_action_t compute_action; + smx_mutex_t mutex; + smx_cond_t cond; + + xbt_os_timer_stop(smpi_global->timers[rank]); + + duration = xbt_os_timer_elapsed(smpi_global->timers[rank]); + SIMIX_mutex_unlock(smpi_global->timers_mutexes[rank]); + + host = smpi_mpi_global->mpi_comm_world->hosts[rank]; + compute_action = SIMIX_action_execute(host, "computation", duration * SMPI_DEFAULT_SPEED); + mutex = SIMIX_mutex_init(); + cond = SIMIX_cond_init(); + + SIMIX_register_condition_to_action(compute_action, cond); + SIMIX_register_action_to_condition(compute_action, cond); + SIMIX_mutex_lock(mutex); + SIMIX_cond_wait(cond, mutex); + SIMIX_mutex_unlock(mutex); + + SIMIX_mutex_destroy(mutex); + SIMIX_cond_destroy(cond); + + // FIXME: check for success/failure? + + return; } -int smpi_receiver(int argc, char **argv) { - m_process_t process; - m_task_t mtask; - smpi_received_t *received; - int rank; - smpi_mpi_request_t *recvreq; +void smpi_barrier(smpi_mpi_communicator_t *comm) { - process = MSG_process_self(); + SIMIX_mutex_lock(comm->barrier_mutex); + if(++comm->barrier_count < comm->size) { + SIMIX_cond_wait(comm->barrier_cond, comm->barrier_mutex); + } else { + comm->barrier_count = 0; + SIMIX_cond_broadcast(comm->barrier_cond); + } + SIMIX_mutex_unlock(comm->barrier_mutex); - // wait for init - mtask = (m_task_t)0; - MSG_task_get(&mtask, RECV_SYNC_PORT); + return; +} - rank = smpi_host_rank_self(); +// FIXME: smarter algorithm... +int smpi_comm_rank(smpi_mpi_communicator_t *comm, smx_host_t host) +{ + int i; + for(i = 0; i < comm->size && host != comm->hosts[i]; i++); + if (i >= comm->size) i = -1; + return i; +} - // potential race condition... - smpi_receiver_processes[rank] = process; +int smpi_create_request(void *buf, int count, smpi_mpi_datatype_t *datatype, + int src, int dst, int tag, smpi_mpi_communicator_t *comm, smpi_mpi_request_t **request) +{ + int retval = MPI_SUCCESS; + + *request = NULL; + + if (0 > count) { + retval = MPI_ERR_COUNT; + } else if (NULL == buf) { + retval = MPI_ERR_INTERN; + } else if (NULL == datatype) { + retval = MPI_ERR_TYPE; + } else if (NULL == comm) { + retval = MPI_ERR_COMM; + } else if (MPI_ANY_SOURCE != src && (0 > src || comm->size <= src)) { + retval = MPI_ERR_RANK; + } else if (0 > dst || comm->size <= dst) { + retval = MPI_ERR_RANK; + } else if (0 > tag) { + retval = MPI_ERR_TAG; + } else { + *request = xbt_mallocator_get(smpi_global->request_mallocator); + (*request)->comm = comm; + (*request)->src = src; + (*request)->dst = dst; + (*request)->tag = tag; + (*request)->buf = buf; + (*request)->count = count; + (*request)->datatype = datatype; + } + return retval; +} - // ready! - MSG_task_put(mtask, MSG_task_get_source(mtask), RECV_SYNC_PORT); +int smpi_isend(smpi_mpi_request_t *request) +{ + int retval = MPI_SUCCESS; + int rank = smpi_mpi_comm_world_rank_self(); - while (0 < smpi_running_hosts) { - recvreq = smpi_pending_recv_requests[rank]; - if (NULL != recvreq) { - mtask = (m_task_t)0; + if (NULL != request) { + SIMIX_mutex_lock(smpi_global->pending_send_request_queues_mutexes[rank]); + xbt_fifo_push(smpi_global->pending_send_request_queues[rank], request); + SIMIX_mutex_unlock(smpi_global->pending_send_request_queues_mutexes[rank]); + } - #ifdef DEBUG - printf("host %s waiting to receive from anyone, but first in queue is (%d,%d,%d).\n", - MSG_host_get_name(MSG_host_self()), recvreq->src, recvreq->dst, recvreq->tag); - #endif - MSG_task_get(&mtask, MPI_PORT); + if (SIMIX_process_is_suspended(smpi_global->sender_processes[rank])) { + SIMIX_process_resume(smpi_global->sender_processes[rank]); + } - received = xbt_malloc(sizeof(smpi_received_t)); + return retval; +} - sscanf(MSG_task_get_name(mtask), "comm:%d,src:%d,dst:%d,tag:%d,ft:%d", - &received->commid, &received->src, &received->dst, &received->tag, &received->fwdthrough); - received->data = MSG_task_get_data(mtask); - received->next = NULL; +int smpi_irecv(smpi_mpi_request_t *request) +{ + int retval = MPI_SUCCESS; + int rank = smpi_mpi_comm_world_rank_self(); - if (NULL == smpi_last_received[rank]) { - smpi_received[rank] = received; - } else { - smpi_last_received[rank]->next = received; - } - smpi_last_received[rank] = received; + if (NULL != request) { + SIMIX_mutex_lock(smpi_global->pending_recv_request_queues_mutexes[rank]); + xbt_fifo_push(smpi_global->pending_recv_request_queues[rank], request); + SIMIX_mutex_unlock(smpi_global->pending_recv_request_queues_mutexes[rank]); + } - MSG_task_destroy(mtask); + if (SIMIX_process_is_suspended(smpi_global->receiver_processes[rank])) { + SIMIX_process_resume(smpi_global->receiver_processes[rank]); + } - smpi_match_requests(rank); + return retval; +} - } else { - MSG_process_suspend(process); - } - } - return 0; +void smpi_wait(smpi_mpi_request_t *request, smpi_mpi_status_t *status) +{ + smx_process_t self = SIMIX_process_self(); + int suspend = 0; + + if (NULL != request) { + SIMIX_mutex_lock(request->mutex); + if (!request->completed) { + SIMIX_cond_wait(request->cond, request->mutex); + } + if (NULL != status) { + status->MPI_SOURCE = request->src; + } + SIMIX_mutex_unlock(request->mutex); + } } // FIXME: move into own file -int smpi_gettimeofday(struct timeval *tv, struct timezone *tz) { - double now; - int retval = 0; - smpi_bench_end(); - if (NULL == tv) { - retval = -1; - } else { - now = MSG_get_clock(); - tv->tv_sec = now; - tv->tv_usec = ((now - (double)tv->tv_sec) * 1000000.0); - } - smpi_bench_begin(); - return retval; +int smpi_gettimeofday(struct timeval *tv, struct timezone *tz) +{ + double now; + int retval = 0; + smpi_bench_end(); + if (NULL == tv) { + retval = -1; + } else { + now = SIMIX_get_clock(); + tv->tv_sec = now; + tv->tv_usec = ((now - (double)tv->tv_sec) * 1000000.0); + } + smpi_bench_begin(); + return retval; } -unsigned int smpi_sleep(unsigned int seconds) { - m_task_t task = NULL; - smpi_bench_end(); - task = MSG_task_create("sleep", seconds * DEFAULT_POWER, 0, NULL); - MSG_task_execute(task); - MSG_task_destroy(task); - smpi_bench_begin(); - return 0; +unsigned int smpi_sleep(unsigned int seconds) +{ + smx_mutex_t mutex; + smx_cond_t cond; + smx_host_t host; + smx_action_t sleep_action; + + smpi_bench_end(); + host = SIMIX_host_self(); + sleep_action = SIMIX_action_sleep(host, seconds); + mutex = SIMIX_mutex_init(); + cond = SIMIX_cond_init(); + + SIMIX_register_condition_to_action(sleep_action, cond); + SIMIX_register_action_to_condition(sleep_action, cond); + SIMIX_mutex_lock(mutex); + SIMIX_cond_wait(cond, mutex); + SIMIX_mutex_unlock(mutex); + + SIMIX_mutex_destroy(mutex); + SIMIX_cond_destroy(cond); + + // FIXME: check for success/failure? + + smpi_bench_begin(); + return 0; } -void smpi_exit(int status) { - smpi_bench_end(); - smpi_running_hosts--; - MSG_process_kill(MSG_process_self()); - return; +void smpi_exit(int status) +{ + smpi_bench_end(); + SIMIX_mutex_lock(smpi_global->running_hosts_count_mutex); + smpi_global->running_hosts_count--; + SIMIX_mutex_unlock(smpi_global->running_hosts_count_mutex); + SIMIX_process_kill(SIMIX_process_self()); + return; }