Explorar el Código

Add initial simgrid support

Samuel Thibault hace 12 años
padre
commit
f60eb1288d

+ 2 - 0
ChangeLog

@@ -56,6 +56,8 @@ New features:
         - When exchanging user-defined data interfaces, the size of
         - When exchanging user-defined data interfaces, the size of
 	  the data is the size returned by the pack operation, i.e
 	  the data is the size returned by the pack operation, i.e
 	  data with dynamic size can now be exchanged with StarPU-MPI.
 	  data with dynamic size can now be exchanged with StarPU-MPI.
+  * Add experimental simgrid support, to simulation execution with various
+    number of CPUs, GPUs, amount of memory, etc.
 
 
 Changes:
 Changes:
   * Fix the block filter functions.
   * Fix the block filter functions.

+ 21 - 5
configure.ac

@@ -777,6 +777,25 @@ AC_DEFINE_UNQUOTED(STARPU_MAXGORDONDEVS, [1], [maximum number of GORDON devices]
 #                                                                             #
 #                                                                             #
 ###############################################################################
 ###############################################################################
 
 
+AC_ARG_ENABLE(simgrid, [AS_HELP_STRING([--enable-simgrid],
+			[Enable simulating execution in simgrid])],
+			enable_simgrid=$enableval, enable_simgrid=no)
+if test x$enable_simgrid = xyes ; then
+	AC_HAVE_LIBRARY([simgrid], [],
+		[
+			AC_MSG_ERROR(Simgrid support needs simgrid installed)
+		]
+	)
+	SIMGRID_LIBS=-lsimgrid
+	AC_DEFINE(STARPU_SIMGRID, 1, [Define this to enable simgrid execution])
+	# Avoid the starpu top thread compilation
+	enable_starpu_top=no
+	# We won't bind or detect anything
+	with_hwloc=no
+	# In simgrid, it's much better to let workers block than spinlock
+	enable_blocking=yes
+fi
+
 AC_MSG_CHECKING(whether blocking drivers should be disabled)
 AC_MSG_CHECKING(whether blocking drivers should be disabled)
 AC_ARG_ENABLE(blocking-drivers, [AS_HELP_STRING([--enable-blocking-drivers], [enable blocking drivers])],
 AC_ARG_ENABLE(blocking-drivers, [AS_HELP_STRING([--enable-blocking-drivers], [enable blocking drivers])],
 				enable_blocking=$enableval, enable_blocking=no)
 				enable_blocking=$enableval, enable_blocking=no)
@@ -1183,10 +1202,6 @@ else
   build_starpu_top=no
   build_starpu_top=no
 fi
 fi
 
 
-if test "x$build_starpu_top" != "xno" ; then
-	AC_DEFINE(STARPU_TOP, [1], [Define this to 1 to enable building StarPU Top])
-fi
-
 AM_CONDITIONAL(BUILD_STARPU_TOP, test x$build_starpu_top = xyes)
 AM_CONDITIONAL(BUILD_STARPU_TOP, test x$build_starpu_top = xyes)
 
 
 ###############################################################################
 ###############################################################################
@@ -1712,7 +1727,7 @@ AC_SUBST([LIBSTARPU_LDFLAGS])
 
 
 LIBSTARPU_LINK=libstarpu-$STARPU_EFFECTIVE_VERSION.la
 LIBSTARPU_LINK=libstarpu-$STARPU_EFFECTIVE_VERSION.la
 if test x$enable_perf_debug = xyes; then
 if test x$enable_perf_debug = xyes; then
-	LIBSTARPU_LINK=".libs/libstarpu-$STARPU_EFFECTIVE_VERSION.a $LIBSTARPU_LDFLAGS $HWLOC_LIBS $STARPU_CUDA_LDFLAGS $STARPU_OPENCL_LDFLAGS"
+	LIBSTARPU_LINK=".libs/libstarpu-$STARPU_EFFECTIVE_VERSION.a $LIBSTARPU_LDFLAGS $HWLOC_LIBS $SIMGRID_LIBS $STARPU_CUDA_LDFLAGS $STARPU_OPENCL_LDFLAGS"
 fi
 fi
 AC_SUBST([LIBSTARPU_LINK])
 AC_SUBST([LIBSTARPU_LINK])
 
 
@@ -1808,6 +1823,7 @@ AC_MSG_NOTICE([
 	       GCC plug-in test suite (requires GNU Guile): $run_gcc_plugin_test_suite
 	       GCC plug-in test suite (requires GNU Guile): $run_gcc_plugin_test_suite
 	       SOCL enabled:                                $build_socl
 	       SOCL enabled:                                $build_socl
                SOCL test suite:                             $run_socl_check
                SOCL test suite:                             $run_socl_check
+               simgrid enabled:                             $enable_simgrid
 ])
 ])
 
 
 if test "$build_socl" = "yes" -a "$run_socl_check" = "no" ; then
 if test "$build_socl" = "yes" -a "$run_socl_check" = "no" ; then

+ 5 - 0
doc/chapters/configuration.texi

@@ -283,6 +283,11 @@ Disable the build of examples.
 Enable memory statistics (@pxref{Memory feedback}).
 Enable memory statistics (@pxref{Memory feedback}).
 @end defvr
 @end defvr
 
 
+@defvr {Configure option} --enable-simgrid
+Enable simulation of execution in simgrid, to allow easy experimentation with
+various numbers of cores and GPUs, or amount of memory, etc. Experimental.
+@end defvr
+
 @node Execution configuration through environment variables
 @node Execution configuration through environment variables
 @section Execution configuration through environment variables
 @section Execution configuration through environment variables
 
 

+ 19 - 0
doc/chapters/perf-optimization.texi

@@ -21,6 +21,7 @@ TODO: improve!
 * Profiling::
 * Profiling::
 * CUDA-specific optimizations::
 * CUDA-specific optimizations::
 * Performance debugging::
 * Performance debugging::
+* Simulated performance::
 @end menu
 @end menu
 
 
 Simply encapsulating application kernels into tasks already permits to
 Simply encapsulating application kernels into tasks already permits to
@@ -422,3 +423,21 @@ detailed in the next chapter. The various informations should be checked for.
   greedy algorithm which thus performs badly.
   greedy algorithm which thus performs badly.
 @end itemize
 @end itemize
 @end itemize
 @end itemize
+
+@node Simulated performance
+@section Simulated performance
+
+StarPU can use Simgrid in order to simulate execution on an arbitrary
+platform. The idea is to first compile StarPU normally, and run the application,
+so as to automatically benchmark the bus and the codelets. Then, recompile
+StarPU, passing @code{--enable-simgrid} to @code{./configure}, and re-run the
+application, specifying the requested number of devices:
+
+@cartouche
+@smallexample
+STARPU_NCPU=12 STARPU_NCUDA=2 STARPU_NOPENCL=0 ./example
+@end smallexample
+@end cartouche
+
+For now, only the number of cpus can be arbitrarily chosen. The number of CUDA
+and OpenCL devices have to be lower than the real number on the current machine.

+ 4 - 0
include/starpu.h

@@ -66,6 +66,10 @@ extern "C"
 {
 {
 #endif
 #endif
 
 
+#ifdef STARPU_SIMGRID
+#define main starpu_main
+#endif
+
 enum starpu_archtype
 enum starpu_archtype
 {
 {
 	STARPU_CPU_WORKER,    /* CPU core */
 	STARPU_CPU_WORKER,    /* CPU core */

+ 2 - 0
include/starpu_config.h.in

@@ -26,6 +26,8 @@
 #undef STARPU_USE_OPENCL
 #undef STARPU_USE_OPENCL
 #undef STARPU_USE_GORDON
 #undef STARPU_USE_GORDON
 
 
+#undef STARPU_SIMGRID
+
 #undef STARPU_HAVE_ICC
 #undef STARPU_HAVE_ICC
 
 
 #undef STARPU_USE_MPI
 #undef STARPU_USE_MPI

+ 1 - 1
include/starpu_perfmodel.h

@@ -188,7 +188,7 @@ struct starpu_perfmodel
 	unsigned is_loaded;
 	unsigned is_loaded;
 	unsigned benchmarking;
 	unsigned benchmarking;
 
 
-#if defined(_MSC_VER)
+#if defined(_MSC_VER) || defined(STARPU_SIMGRID)
 	void *model_rwlock;
 	void *model_rwlock;
 #else
 #else
 	pthread_rwlock_t model_rwlock;
 	pthread_rwlock_t model_rwlock;

+ 1 - 1
include/starpu_scheduler.h

@@ -118,7 +118,7 @@ multiple workers may use the same condition variable. For instance, in the case
 of a scheduling strategy with a single task queue, the same condition variable
 of a scheduling strategy with a single task queue, the same condition variable
 would be used to block and wake up all workers.  The initialization method of a
 would be used to block and wake up all workers.  The initialization method of a
 scheduling strategy (init_sched) must call this function once per worker. */
 scheduling strategy (init_sched) must call this function once per worker. */
-#if !defined(_MSC_VER)
+#if !defined(_MSC_VER) && !defined(STARPU_SIMGRID)
 void starpu_worker_set_sched_condition(int workerid, pthread_cond_t *sched_cond, pthread_mutex_t *sched_mutex);
 void starpu_worker_set_sched_condition(int workerid, pthread_cond_t *sched_cond, pthread_mutex_t *sched_mutex);
 #endif
 #endif
 
 

+ 1 - 1
mpi/src/starpu_mpi.c

@@ -855,7 +855,7 @@ int _starpu_mpi_initialize(int *argc, char ***argv)
 	struct _starpu_mpi_argc_argv *argc_argv = malloc(sizeof(struct _starpu_mpi_argc_argv));
 	struct _starpu_mpi_argc_argv *argc_argv = malloc(sizeof(struct _starpu_mpi_argc_argv));
 	argc_argv->argc = argc;
 	argc_argv->argc = argc;
 	argc_argv->argv = argv;
 	argc_argv->argv = argv;
-	_STARPU_PTHREAD_CREATE(&progress_thread, NULL, _starpu_mpi_progress_thread_func, argc_argv);
+	_STARPU_PTHREAD_CREATE("MPI progress", &progress_thread, NULL, _starpu_mpi_progress_thread_func, argc_argv);
 
 
 	_STARPU_PTHREAD_MUTEX_LOCK(&mutex);
 	_STARPU_PTHREAD_MUTEX_LOCK(&mutex);
 	while (!running)
 	while (!running)

+ 2 - 0
src/Makefile.am

@@ -72,6 +72,7 @@ noinst_HEADERS = 						\
 	core/errorcheck.h					\
 	core/errorcheck.h					\
 	core/combined_workers.h					\
 	core/combined_workers.h					\
 	core/parallel_task.h					\
 	core/parallel_task.h					\
+	core/simgrid.h						\
 	core/task_bundle.h					\
 	core/task_bundle.h					\
 	sched_policies/detect_combined_workers.h		\
 	sched_policies/detect_combined_workers.h		\
 	sched_policies/fifo_queues.h				\
 	sched_policies/fifo_queues.h				\
@@ -147,6 +148,7 @@ libstarpu_@STARPU_EFFECTIVE_VERSION@_la_SOURCES = 						\
 	core/perfmodel/perfmodel_print.c			\
 	core/perfmodel/perfmodel_print.c			\
 	core/perfmodel/regression.c				\
 	core/perfmodel/regression.c				\
 	core/sched_policy.c					\
 	core/sched_policy.c					\
+	core/simgrid.c						\
 	core/priorities.c					\
 	core/priorities.c					\
 	core/parallel_task.c					\
 	core/parallel_task.c					\
 	sched_policies/eager_central_policy.c			\
 	sched_policies/eager_central_policy.c			\

+ 5 - 0
src/common/barrier.h

@@ -17,6 +17,11 @@
 #ifndef __COMMON_BARRIER_H__
 #ifndef __COMMON_BARRIER_H__
 #define __COMMON_BARRIER_H__
 #define __COMMON_BARRIER_H__
 
 
+#ifdef STARPU_SIMGRID
+/* Force using our implementation of barriers, so it can be simgridish */
+#undef PTHREAD_BARRIER_SERIAL_THREAD
+#endif
+
 #include <pthread.h>
 #include <pthread.h>
 #include <common/utils.h>
 #include <common/utils.h>
 
 

+ 17 - 0
src/common/starpu_spinlock.c

@@ -20,6 +20,10 @@
 #include <common/utils.h>
 #include <common/utils.h>
 #include <starpu_util.h>
 #include <starpu_util.h>
 
 
+#ifdef STARPU_SIMGRID
+#include <msg/msg.h>
+#endif
+
 int _starpu_spin_init(struct _starpu_spinlock *lock)
 int _starpu_spin_init(struct _starpu_spinlock *lock)
 {
 {
 #ifdef STARPU_SPINLOCK_CHECK
 #ifdef STARPU_SPINLOCK_CHECK
@@ -70,9 +74,22 @@ int _starpu_spin_lock(struct _starpu_spinlock *lock)
 	return ret;
 	return ret;
 #else
 #else
 #ifdef HAVE_PTHREAD_SPIN_LOCK
 #ifdef HAVE_PTHREAD_SPIN_LOCK
+#ifdef STARPU_SIMGRID
+	while (1) {
+		int ret = pthread_spin_trylock(&lock->lock);
+		if (ret <= 0)
+			return ret;
+#ifdef STARPU_DEVEL
+#warning FIXME: better way to spinlock?
+#endif
+		/* Sleep for 10µs */
+		MSG_process_sleep(0.000010);
+	}
+#else
 	int ret = pthread_spin_lock(&lock->lock);
 	int ret = pthread_spin_lock(&lock->lock);
 	STARPU_ASSERT(!ret);
 	STARPU_ASSERT(!ret);
 	return ret;
 	return ret;
+#endif
 #else
 #else
 	uint32_t prev;
 	uint32_t prev;
 	do
 	do

+ 21 - 1
src/common/timing.c

@@ -21,7 +21,23 @@
 #include <profiling/profiling.h>
 #include <profiling/profiling.h>
 #include <common/timing.h>
 #include <common/timing.h>
 
 
-#if defined(HAVE_CLOCK_GETTIME) && defined(CLOCK_MONOTONIC)
+#ifdef STARPU_SIMGRID
+#include <msg/msg.h>
+#endif
+
+#ifdef STARPU_SIMGRID
+void _starpu_timing_init(void)
+{
+}
+
+void _starpu_clock_gettime(struct timespec *ts)
+{
+	double now = MSG_get_clock();
+	ts->tv_sec = floor(now);
+	ts->tv_nsec = floor((now - ts->tv_sec) * 1000000000);
+}
+
+#elif defined(HAVE_CLOCK_GETTIME) && defined(CLOCK_MONOTONIC)
 #include <time.h>
 #include <time.h>
 #ifndef _POSIX_C_SOURCE
 #ifndef _POSIX_C_SOURCE
 /* for clock_gettime */
 /* for clock_gettime */
@@ -193,8 +209,12 @@ double starpu_timing_timespec_to_us(struct timespec *ts)
 
 
 double starpu_timing_now(void)
 double starpu_timing_now(void)
 {
 {
+#ifdef STARPU_SIMGRID
+	return MSG_get_clock()*1000000;
+#else
 	struct timespec now;
 	struct timespec now;
 	_starpu_clock_gettime(&now);
 	_starpu_clock_gettime(&now);
 
 
 	return starpu_timing_timespec_to_us(&now);
 	return starpu_timing_timespec_to_us(&now);
+#endif
 }
 }

+ 140 - 1
src/common/utils.h

@@ -26,6 +26,11 @@
 #include <stdlib.h>
 #include <stdlib.h>
 #include <math.h>
 #include <math.h>
 
 
+#ifdef STARPU_SIMGRID
+#include <xbt/synchro_core.h>
+#include <msg/msg.h>
+#endif
+
 #ifdef STARPU_VERBOSE
 #ifdef STARPU_VERBOSE
 #  define _STARPU_DEBUG(fmt, args ...) do { if (!getenv("STARPU_SILENT")) {fprintf(stderr, "[starpu][%s] " fmt ,__func__ ,##args); fflush(stderr); }} while(0)
 #  define _STARPU_DEBUG(fmt, args ...) do { if (!getenv("STARPU_SILENT")) {fprintf(stderr, "[starpu][%s] " fmt ,__func__ ,##args); fflush(stderr); }} while(0)
 #else
 #else
@@ -52,7 +57,11 @@
 
 
 #define _STARPU_IS_ZERO(a) (fpclassify(a) == FP_ZERO)
 #define _STARPU_IS_ZERO(a) (fpclassify(a) == FP_ZERO)
 
 
+#ifdef STARPU_SIMGRID
+typedef xbt_mutex_t _starpu_pthread_mutex_t;
+#else
 typedef pthread_mutex_t _starpu_pthread_mutex_t;
 typedef pthread_mutex_t _starpu_pthread_mutex_t;
+#endif
 int _starpu_mkpath(const char *s, mode_t mode);
 int _starpu_mkpath(const char *s, mode_t mode);
 void _starpu_mkpath_and_check(const char *s, mode_t mode);
 void _starpu_mkpath_and_check(const char *s, mode_t mode);
 int _starpu_check_mutex_deadlock(_starpu_pthread_mutex_t *mutex);
 int _starpu_check_mutex_deadlock(_starpu_pthread_mutex_t *mutex);
@@ -69,7 +78,25 @@ struct starpu_codelet;
 /* Returns the symbol associated to that job if any. */
 /* Returns the symbol associated to that job if any. */
 const char *_starpu_codelet_get_model_name(struct starpu_codelet *cl);
 const char *_starpu_codelet_get_model_name(struct starpu_codelet *cl);
 
 
-#define _STARPU_PTHREAD_CREATE(thread, attr, routine, arg) do {                \
+struct _starpu_pthread_args {
+	void *(*f)(void*);
+	void *arg;
+};
+
+int _starpu_simgrid_thread_start(int argc, char *argv[]);
+#ifdef STARPU_SIMGRID
+#define _STARPU_PTHREAD_CREATE_ON(name, thread, attr, routine, threadarg, where) do {\
+	struct _starpu_pthread_args *_args = malloc(sizeof(*_args));           \
+	xbt_dynar_t _hosts;                                                    \
+	_args->f = routine;                                                    \
+	_args->arg = threadarg;                                                \
+	_hosts = MSG_hosts_as_dynar();                                         \
+	MSG_process_create((name), _starpu_simgrid_thread_start, _args,        \
+			xbt_dynar_get_as(_hosts, (where), msg_host_t));        \
+	xbt_dynar_free(&_hosts);                                               \
+} while (0)
+#else
+#define _STARPU_PTHREAD_CREATE_ON(name, thread, attr, routine, arg, where) do {\
 	int p_ret = pthread_create((thread), (attr), (routine), (arg));	       \
 	int p_ret = pthread_create((thread), (attr), (routine), (arg));	       \
 	if (STARPU_UNLIKELY(p_ret != 0)) {                                     \
 	if (STARPU_UNLIKELY(p_ret != 0)) {                                     \
 		fprintf(stderr,                                                \
 		fprintf(stderr,                                                \
@@ -77,10 +104,24 @@ const char *_starpu_codelet_get_model_name(struct starpu_codelet *cl);
 			__FILE__, __LINE__, strerror(p_ret));                  \
 			__FILE__, __LINE__, strerror(p_ret));                  \
 	}                                                                      \
 	}                                                                      \
 } while (0)
 } while (0)
+#endif
+#define _STARPU_PTHREAD_CREATE(name, thread, attr, routine, arg)               \
+	_STARPU_PTHREAD_CREATE_ON(name, thread, attr, routine, arg, 0)
 
 
 /*
 /*
  * Encapsulation of the pthread_key_* functions.
  * Encapsulation of the pthread_key_* functions.
  */
  */
+#ifdef STARPU_SIMGRID
+typedef int _starpu_pthread_key_t;
+int _starpu_pthread_key_create(_starpu_pthread_key_t *key);
+#define _STARPU_PTHREAD_KEY_CREATE(key, destr) _starpu_pthread_key_create(key)
+int _starpu_pthread_key_delete(_starpu_pthread_key_t key);
+#define _STARPU_PTHREAD_KEY_DELETE(key) _starpu_pthread_key_delete(key)
+int _starpu_pthread_setspecific(_starpu_pthread_key_t key, void *ptr);
+#define _STARPU_PTHREAD_SETSPECIFIC(key, ptr) _starpu_pthread_setspecific(key, ptr)
+void *_starpu_pthread_getspecific(_starpu_pthread_key_t key);
+#define _STARPU_PTHREAD_GETSPECIFIC(key) _starpu_pthread_getspecific(key)
+#else
 typedef pthread_key_t _starpu_pthread_key_t;
 typedef pthread_key_t _starpu_pthread_key_t;
 #define _STARPU_PTHREAD_KEY_CREATE(key, destr) do {                            \
 #define _STARPU_PTHREAD_KEY_CREATE(key, destr) do {                            \
 	int p_ret = pthread_key_create((key), (destr));	                       \
 	int p_ret = pthread_key_create((key), (destr));	                       \
@@ -110,10 +151,17 @@ typedef pthread_key_t _starpu_pthread_key_t;
 } while (0)
 } while (0)
 
 
 #define _STARPU_PTHREAD_GETSPECIFIC(key) pthread_getspecific((key))
 #define _STARPU_PTHREAD_GETSPECIFIC(key) pthread_getspecific((key))
+#endif
 
 
 /*
 /*
  * Encapsulation of the pthread_mutex_* functions.
  * Encapsulation of the pthread_mutex_* functions.
  */
  */
+#ifdef STARPU_SIMGRID
+#define _STARPU_PTHREAD_MUTEX_INITIALIZER NULL
+#define _STARPU_PTHREAD_MUTEX_INIT(mutex, attr) do {                           \
+	(*mutex) = xbt_mutex_init();                                           \
+} while (0)
+#else
 #define _STARPU_PTHREAD_MUTEX_INITIALIZER PTHREAD_MUTEX_INITIALIZER
 #define _STARPU_PTHREAD_MUTEX_INITIALIZER PTHREAD_MUTEX_INITIALIZER
 #define _STARPU_PTHREAD_MUTEX_INIT(mutex, attr) do {                           \
 #define _STARPU_PTHREAD_MUTEX_INIT(mutex, attr) do {                           \
 	int p_ret = pthread_mutex_init((mutex), (attr));                       \
 	int p_ret = pthread_mutex_init((mutex), (attr));                       \
@@ -124,7 +172,14 @@ typedef pthread_key_t _starpu_pthread_key_t;
 		STARPU_ABORT();                                                \
 		STARPU_ABORT();                                                \
 	}                                                                      \
 	}                                                                      \
 } while (0)
 } while (0)
+#endif
 
 
+#ifdef STARPU_SIMGRID
+#define _STARPU_PTHREAD_MUTEX_DESTROY(mutex) do {                              \
+	if (*mutex)                                                            \
+		xbt_mutex_destroy((*mutex));                                   \
+} while (0)
+#else
 #define _STARPU_PTHREAD_MUTEX_DESTROY(mutex) do {                              \
 #define _STARPU_PTHREAD_MUTEX_DESTROY(mutex) do {                              \
 	int p_ret = pthread_mutex_destroy(mutex);                              \
 	int p_ret = pthread_mutex_destroy(mutex);                              \
 	if (STARPU_UNLIKELY(p_ret)) {                                          \
 	if (STARPU_UNLIKELY(p_ret)) {                                          \
@@ -134,7 +189,14 @@ typedef pthread_key_t _starpu_pthread_key_t;
 		STARPU_ABORT();                                                \
 		STARPU_ABORT();                                                \
 	}                                                                      \
 	}                                                                      \
 } while(0)
 } while(0)
+#endif
 
 
+#ifdef STARPU_SIMGRID
+#define _STARPU_PTHREAD_MUTEX_LOCK(mutex) do {                                 \
+	if (!(*mutex)) _STARPU_PTHREAD_MUTEX_INIT((mutex), NULL);              \
+	xbt_mutex_acquire((*mutex));                                           \
+} while (0)
+#else
 #define _STARPU_PTHREAD_MUTEX_LOCK(mutex) do {                                 \
 #define _STARPU_PTHREAD_MUTEX_LOCK(mutex) do {                                 \
 	int p_ret = pthread_mutex_lock(mutex);                                 \
 	int p_ret = pthread_mutex_lock(mutex);                                 \
 	if (STARPU_UNLIKELY(p_ret)) {                                          \
 	if (STARPU_UNLIKELY(p_ret)) {                                          \
@@ -144,9 +206,19 @@ typedef pthread_key_t _starpu_pthread_key_t;
 		STARPU_ABORT();                                                \
 		STARPU_ABORT();                                                \
 	}                                                                      \
 	}                                                                      \
 } while (0)
 } while (0)
+#endif
 
 
+#ifdef STARPU_SIMGRID
+#define _STARPU_PTHREAD_MUTEX_TRYLOCK(mutex) (xbt_mutex_acquire(*mutex), 0)
+#else
 #define _STARPU_PTHREAD_MUTEX_TRYLOCK(mutex) pthread_mutex_trylock(mutex)
 #define _STARPU_PTHREAD_MUTEX_TRYLOCK(mutex) pthread_mutex_trylock(mutex)
+#endif
 
 
+#ifdef STARPU_SIMGRID
+#define _STARPU_PTHREAD_MUTEX_UNLOCK(mutex) do {                               \
+	xbt_mutex_release((*mutex));                                           \
+} while (0)
+#else
 #define _STARPU_PTHREAD_MUTEX_UNLOCK(mutex) do {                               \
 #define _STARPU_PTHREAD_MUTEX_UNLOCK(mutex) do {                               \
 	int p_ret = pthread_mutex_unlock(mutex);                               \
 	int p_ret = pthread_mutex_unlock(mutex);                               \
 	if (STARPU_UNLIKELY(p_ret)) {                                          \
 	if (STARPU_UNLIKELY(p_ret)) {                                          \
@@ -156,11 +228,19 @@ typedef pthread_key_t _starpu_pthread_key_t;
 		STARPU_ABORT();                                                \
 		STARPU_ABORT();                                                \
 	}                                                                      \
 	}                                                                      \
 } while (0)
 } while (0)
+#endif
 
 
+#ifdef STARPU_SIMGRID
+typedef xbt_mutex_t _starpu_pthread_rwlock_t;
+#else
 typedef pthread_rwlock_t _starpu_pthread_rwlock_t;
 typedef pthread_rwlock_t _starpu_pthread_rwlock_t;
+#endif
 /*
 /*
  * Encapsulation of the pthread_rwlock_* functions.
  * Encapsulation of the pthread_rwlock_* functions.
  */
  */
+#ifdef STARPU_SIMGRID
+#define _STARPU_PTHREAD_RWLOCK_INIT(rwlock, attr) _STARPU_PTHREAD_MUTEX_INIT(rwlock, attr)
+#else
 #define _STARPU_PTHREAD_RWLOCK_INIT(rwlock, attr) do {                         \
 #define _STARPU_PTHREAD_RWLOCK_INIT(rwlock, attr) do {                         \
 	int p_ret = pthread_rwlock_init((rwlock), (attr));                     \
 	int p_ret = pthread_rwlock_init((rwlock), (attr));                     \
 	if (STARPU_UNLIKELY(p_ret)) {                                          \
 	if (STARPU_UNLIKELY(p_ret)) {                                          \
@@ -170,7 +250,11 @@ typedef pthread_rwlock_t _starpu_pthread_rwlock_t;
 		STARPU_ABORT();                                                \
 		STARPU_ABORT();                                                \
 	}                                                                      \
 	}                                                                      \
 } while (0)
 } while (0)
+#endif
 
 
+#ifdef STARPU_SIMGRID
+#define _STARPU_PTHREAD_RWLOCK_RDLOCK(rwlock) _STARPU_PTHREAD_MUTEX_LOCK(rwlock)
+#else
 #define _STARPU_PTHREAD_RWLOCK_RDLOCK(rwlock) do {                             \
 #define _STARPU_PTHREAD_RWLOCK_RDLOCK(rwlock) do {                             \
 	int p_ret = pthread_rwlock_rdlock(rwlock);                             \
 	int p_ret = pthread_rwlock_rdlock(rwlock);                             \
 	if (STARPU_UNLIKELY(p_ret)) {                                          \
 	if (STARPU_UNLIKELY(p_ret)) {                                          \
@@ -180,7 +264,11 @@ typedef pthread_rwlock_t _starpu_pthread_rwlock_t;
 		STARPU_ABORT();                                                \
 		STARPU_ABORT();                                                \
 	}                                                                      \
 	}                                                                      \
 } while (0)
 } while (0)
+#endif
 
 
+#ifdef STARPU_SIMGRID
+#define _STARPU_PTHREAD_RWLOCK_WRLOCK(rwlock) _STARPU_PTHREAD_MUTEX_LOCK(rwlock)
+#else
 #define _STARPU_PTHREAD_RWLOCK_WRLOCK(rwlock) do {                             \
 #define _STARPU_PTHREAD_RWLOCK_WRLOCK(rwlock) do {                             \
 	int p_ret = pthread_rwlock_wrlock(rwlock);                             \
 	int p_ret = pthread_rwlock_wrlock(rwlock);                             \
 	if (STARPU_UNLIKELY(p_ret)) {                                          \
 	if (STARPU_UNLIKELY(p_ret)) {                                          \
@@ -190,7 +278,11 @@ typedef pthread_rwlock_t _starpu_pthread_rwlock_t;
 		STARPU_ABORT();                                                \
 		STARPU_ABORT();                                                \
 	}                                                                      \
 	}                                                                      \
 } while (0)
 } while (0)
+#endif
 
 
+#ifdef STARPU_SIMGRID
+#define _STARPU_PTHREAD_RWLOCK_UNLOCK(rwlock) _STARPU_PTHREAD_MUTEX_UNLOCK(rwlock)
+#else
 #define _STARPU_PTHREAD_RWLOCK_UNLOCK(rwlock) do {                             \
 #define _STARPU_PTHREAD_RWLOCK_UNLOCK(rwlock) do {                             \
 	int p_ret = pthread_rwlock_unlock(rwlock);                             \
 	int p_ret = pthread_rwlock_unlock(rwlock);                             \
 	if (STARPU_UNLIKELY(p_ret)) {                                          \
 	if (STARPU_UNLIKELY(p_ret)) {                                          \
@@ -200,7 +292,11 @@ typedef pthread_rwlock_t _starpu_pthread_rwlock_t;
 		STARPU_ABORT();                                                \
 		STARPU_ABORT();                                                \
 	}                                                                      \
 	}                                                                      \
 } while (0)
 } while (0)
+#endif
 
 
+#ifdef STARPU_SIMGRID
+#define _STARPU_PTHREAD_RWLOCK_DESTROY(rwlock) _STARPU_PTHREAD_MUTEX_DESTROY(rwlock)
+#else
 #define _STARPU_PTHREAD_RWLOCK_DESTROY(rwlock) do {                            \
 #define _STARPU_PTHREAD_RWLOCK_DESTROY(rwlock) do {                            \
 	int p_ret = pthread_rwlock_destroy(rwlock);                            \
 	int p_ret = pthread_rwlock_destroy(rwlock);                            \
 	if (STARPU_UNLIKELY(p_ret)) {                                          \
 	if (STARPU_UNLIKELY(p_ret)) {                                          \
@@ -210,11 +306,22 @@ typedef pthread_rwlock_t _starpu_pthread_rwlock_t;
 		STARPU_ABORT();                                                \
 		STARPU_ABORT();                                                \
 	}                                                                      \
 	}                                                                      \
 } while (0)
 } while (0)
+#endif
 
 
+#ifdef STARPU_SIMGRID
+typedef xbt_cond_t _starpu_pthread_cond_t;
+#else
 typedef pthread_cond_t _starpu_pthread_cond_t;
 typedef pthread_cond_t _starpu_pthread_cond_t;
+#endif
 /*
 /*
  * Encapsulation of the pthread_cond_* functions.
  * Encapsulation of the pthread_cond_* functions.
  */
  */
+#ifdef STARPU_SIMGRID
+#define _STARPU_PTHREAD_COND_INITIALIZER NULL
+#define _STARPU_PTHREAD_COND_INIT(cond, attr) do {                             \
+	(*cond) = xbt_cond_init();                                             \
+} while (0)
+#else
 #define _STARPU_PTHREAD_COND_INITIALIZER PTHREAD_COND_INITIALIZER
 #define _STARPU_PTHREAD_COND_INITIALIZER PTHREAD_COND_INITIALIZER
 #define _STARPU_PTHREAD_COND_INIT(cond, attr) do {                             \
 #define _STARPU_PTHREAD_COND_INIT(cond, attr) do {                             \
 	int p_ret = pthread_cond_init((cond), (attr));                         \
 	int p_ret = pthread_cond_init((cond), (attr));                         \
@@ -225,7 +332,14 @@ typedef pthread_cond_t _starpu_pthread_cond_t;
 		STARPU_ABORT();                                                \
 		STARPU_ABORT();                                                \
 	}                                                                      \
 	}                                                                      \
 } while (0)
 } while (0)
+#endif
 
 
+#ifdef STARPU_SIMGRID
+#define _STARPU_PTHREAD_COND_DESTROY(cond) do {                                \
+	if (*cond)                                                             \
+		xbt_cond_destroy((*cond));                                     \
+} while (0)
+#else
 #define _STARPU_PTHREAD_COND_DESTROY(cond) do {                                \
 #define _STARPU_PTHREAD_COND_DESTROY(cond) do {                                \
 	int p_ret = pthread_cond_destroy(cond);                                \
 	int p_ret = pthread_cond_destroy(cond);                                \
 	if (STARPU_UNLIKELY(p_ret)) {                                          \
 	if (STARPU_UNLIKELY(p_ret)) {                                          \
@@ -235,7 +349,15 @@ typedef pthread_cond_t _starpu_pthread_cond_t;
 			STARPU_ABORT();                                        \
 			STARPU_ABORT();                                        \
 	}                                                                      \
 	}                                                                      \
 } while (0)
 } while (0)
+#endif
 
 
+#ifdef STARPU_SIMGRID
+#define _STARPU_PTHREAD_COND_SIGNAL(cond) do {                                 \
+	if (!*cond)                                                            \
+		_STARPU_PTHREAD_COND_INIT(cond, NULL);                         \
+	xbt_cond_signal((*cond));                                              \
+} while (0)
+#else
 #define _STARPU_PTHREAD_COND_SIGNAL(cond) do {                                 \
 #define _STARPU_PTHREAD_COND_SIGNAL(cond) do {                                 \
 	int p_ret = pthread_cond_signal(cond);                                 \
 	int p_ret = pthread_cond_signal(cond);                                 \
 	if (STARPU_UNLIKELY(p_ret)) {                                          \
 	if (STARPU_UNLIKELY(p_ret)) {                                          \
@@ -245,7 +367,15 @@ typedef pthread_cond_t _starpu_pthread_cond_t;
 		STARPU_ABORT();                                                \
 		STARPU_ABORT();                                                \
 	}                                                                      \
 	}                                                                      \
 } while (0)
 } while (0)
+#endif
 
 
+#ifdef STARPU_SIMGRID
+#define _STARPU_PTHREAD_COND_BROADCAST(cond) do {                              \
+	if (!*cond)                                                            \
+		_STARPU_PTHREAD_COND_INIT(cond, NULL);                         \
+	xbt_cond_broadcast((*cond));                                           \
+} while (0)
+#else
 #define _STARPU_PTHREAD_COND_BROADCAST(cond) do {                              \
 #define _STARPU_PTHREAD_COND_BROADCAST(cond) do {                              \
 	int p_ret = pthread_cond_broadcast(cond);                              \
 	int p_ret = pthread_cond_broadcast(cond);                              \
 	if (STARPU_UNLIKELY(p_ret)) {                                          \
 	if (STARPU_UNLIKELY(p_ret)) {                                          \
@@ -255,7 +385,15 @@ typedef pthread_cond_t _starpu_pthread_cond_t;
 		STARPU_ABORT();                                                \
 		STARPU_ABORT();                                                \
 	}                                                                      \
 	}                                                                      \
 } while (0)
 } while (0)
+#endif
 
 
+#ifdef STARPU_SIMGRID
+#define _STARPU_PTHREAD_COND_WAIT(cond, mutex) do {                            \
+	if (!*cond)                                                            \
+		_STARPU_PTHREAD_COND_INIT(cond, NULL);                         \
+	xbt_cond_wait((*cond), (*mutex));                                      \
+} while (0)
+#else
 #define _STARPU_PTHREAD_COND_WAIT(cond, mutex) do {                            \
 #define _STARPU_PTHREAD_COND_WAIT(cond, mutex) do {                            \
 	int p_ret = pthread_cond_wait((cond), (mutex));                        \
 	int p_ret = pthread_cond_wait((cond), (mutex));                        \
 	if (STARPU_UNLIKELY(p_ret)) {                                          \
 	if (STARPU_UNLIKELY(p_ret)) {                                          \
@@ -265,6 +403,7 @@ typedef pthread_cond_t _starpu_pthread_cond_t;
 		STARPU_ABORT();                                                \
 		STARPU_ABORT();                                                \
 	}                                                                      \
 	}                                                                      \
 } while (0)
 } while (0)
+#endif
 
 
 #include <common/barrier.h>
 #include <common/barrier.h>
 
 

+ 3 - 1
src/core/perfmodel/perfmodel.h

@@ -1,6 +1,6 @@
 /* StarPU --- Runtime system for heterogeneous multicore architectures.
 /* StarPU --- Runtime system for heterogeneous multicore architectures.
  *
  *
- * Copyright (C) 2009, 2010, 2011  Université de Bordeaux 1
+ * Copyright (C) 2009-2012  Université de Bordeaux 1
  * Copyright (C) 2010, 2011, 2012  Centre National de la Recherche Scientifique
  * Copyright (C) 2010, 2011, 2012  Centre National de la Recherche Scientifique
  * Copyright (C) 2011  Télécom-SudParis
  * Copyright (C) 2011  Télécom-SudParis
  *
  *
@@ -60,6 +60,8 @@ void _starpu_update_perfmodel_history(struct _starpu_job *j, struct starpu_perfm
 void _starpu_create_sampling_directory_if_needed(void);
 void _starpu_create_sampling_directory_if_needed(void);
 
 
 void _starpu_load_bus_performance_files(void);
 void _starpu_load_bus_performance_files(void);
+double _starpu_transfer_bandwidth(unsigned src_node, unsigned dst_node);
+double _starpu_transfer_latency(unsigned src_node, unsigned dst_node);
 double _starpu_predict_transfer_time(unsigned src_node, unsigned dst_node, size_t size);
 double _starpu_predict_transfer_time(unsigned src_node, unsigned dst_node, size_t size);
 
 
 
 

+ 26 - 1
src/core/perfmodel/perfmodel_bus.c

@@ -44,7 +44,9 @@
 #define SIZE	(32*1024*1024*sizeof(char))
 #define SIZE	(32*1024*1024*sizeof(char))
 #define NITER	128
 #define NITER	128
 
 
+#ifndef STARPU_SIMGRID
 static void starpu_force_bus_sampling(void);
 static void starpu_force_bus_sampling(void);
+#endif
 
 
 /* timing is in µs per byte (i.e. slowness, inverse of bandwidth) */
 /* timing is in µs per byte (i.e. slowness, inverse of bandwidth) */
 struct dev_timing
 struct dev_timing
@@ -84,7 +86,7 @@ static struct dev_timing opencldev_timing_per_cpu[STARPU_MAXNODES*STARPU_MAXCPUS
 static hwloc_topology_t hwtopology;
 static hwloc_topology_t hwtopology;
 #endif
 #endif
 
 
-#if defined(STARPU_USE_CUDA) || defined(STARPU_USE_OPENCL)
+#if (defined(STARPU_USE_CUDA) || defined(STARPU_USE_OPENCL)) && !defined(STARPU_SIMGRID)
 
 
 #ifdef STARPU_USE_CUDA
 #ifdef STARPU_USE_CUDA
 
 
@@ -529,6 +531,10 @@ static void measure_bandwidth_between_host_and_dev(int dev, double *dev_timing_h
 
 
 static void benchmark_all_gpu_devices(void)
 static void benchmark_all_gpu_devices(void)
 {
 {
+#ifdef STARPU_SIMGRID
+	_STARPU_DISP("can not measure bus in simgrid mode\n");
+	STARPU_ABORT();
+#else /* !SIMGRID */
 #if defined(STARPU_USE_CUDA) || defined(STARPU_USE_OPENCL)
 #if defined(STARPU_USE_CUDA) || defined(STARPU_USE_OPENCL)
 	int i;
 	int i;
 #endif
 #endif
@@ -611,6 +617,7 @@ static void benchmark_all_gpu_devices(void)
 	_STARPU_DEBUG("Benchmarking the speed of the bus is done.\n");
 	_STARPU_DEBUG("Benchmarking the speed of the bus is done.\n");
 
 
 	was_benchmarked = 1;
 	was_benchmarked = 1;
+#endif /* !SIMGRID */
 }
 }
 
 
 static void get_bus_path(const char *type, char *path, size_t maxlen)
 static void get_bus_path(const char *type, char *path, size_t maxlen)
@@ -632,6 +639,7 @@ static void get_bus_path(const char *type, char *path, size_t maxlen)
  *	Affinity
  *	Affinity
  */
  */
 
 
+#ifndef STARPU_SIMGRID
 static void get_affinity_path(char *path, size_t maxlen)
 static void get_affinity_path(char *path, size_t maxlen)
 {
 {
 	get_bus_path("affinity", path, maxlen);
 	get_bus_path("affinity", path, maxlen);
@@ -788,6 +796,7 @@ static void load_bus_affinity_file(void)
 
 
 	load_bus_affinity_file_content();
 	load_bus_affinity_file_content();
 }
 }
+#endif /* !SIMGRID */
 
 
 #ifdef STARPU_USE_CUDA
 #ifdef STARPU_USE_CUDA
 int *_starpu_get_cuda_affinity_vector(unsigned gpuid)
 int *_starpu_get_cuda_affinity_vector(unsigned gpuid)
@@ -1213,6 +1222,7 @@ static void load_bus_bandwidth_file(void)
 	}
 	}
 }
 }
 
 
+#ifndef STARPU_SIMGRID
 /*
 /*
  *	Config
  *	Config
  */
  */
@@ -1329,17 +1339,32 @@ static void starpu_force_bus_sampling(void)
 	generate_bus_bandwidth_file();
 	generate_bus_bandwidth_file();
         generate_bus_config_file();
         generate_bus_config_file();
 }
 }
+#endif /* !SIMGRID */
 
 
 void _starpu_load_bus_performance_files(void)
 void _starpu_load_bus_performance_files(void)
 {
 {
 	_starpu_create_sampling_directory_if_needed();
 	_starpu_create_sampling_directory_if_needed();
 
 
+#ifndef STARPU_SIMGRID
         check_bus_config_file();
         check_bus_config_file();
 	load_bus_affinity_file();
 	load_bus_affinity_file();
+#endif
 	load_bus_latency_file();
 	load_bus_latency_file();
 	load_bus_bandwidth_file();
 	load_bus_bandwidth_file();
 }
 }
 
 
+/* (in MB/s) */
+double _starpu_transfer_bandwidth(unsigned src_node, unsigned dst_node)
+{
+	return bandwidth_matrix[src_node][dst_node];
+}
+
+/* (in µs) */
+double _starpu_transfer_latency(unsigned src_node, unsigned dst_node)
+{
+	return latency_matrix[src_node][dst_node];
+}
+
 /* (in µs) */
 /* (in µs) */
 double _starpu_predict_transfer_time(unsigned src_node, unsigned dst_node, size_t size)
 double _starpu_predict_transfer_time(unsigned src_node, unsigned dst_node, size_t size)
 {
 {

+ 275 - 0
src/core/simgrid.c

@@ -0,0 +1,275 @@
+/* StarPU --- Runtime system for heterogeneous multicore architectures.
+ *
+ * Copyright (C) 2012  Université de Bordeaux 1
+ *
+ * StarPU is free software; you can redistribute it and/or modify
+ * it under the terms of the GNU Lesser General Public License as published by
+ * the Free Software Foundation; either version 2.1 of the License, or (at
+ * your option) any later version.
+ *
+ * StarPU is distributed in the hope that it will be useful, but
+ * WITHOUT ANY WARRANTY; without even the implied warranty of
+ * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE.
+ *
+ * See the GNU Lesser General Public License in COPYING.LGPL for more details.
+ */
+
+#include <starpu.h>
+#include <datawizard/memory_nodes.h>
+#include <unistd.h>
+#include <core/perfmodel/perfmodel.h>
+#include <core/workers.h>
+
+#ifdef STARPU_SIMGRID
+#include <msg/msg.h>
+
+#define MAX_TSD 16
+
+#pragma weak starpu_main
+extern int starpu_main(int argc, char *argv[]);
+
+static void bus_name(struct starpu_conf *conf, char *s, size_t size, int num)
+{
+	if (!num)
+		snprintf(s, size, "RAM");
+	else if (num < conf->ncuda + 1)
+		snprintf(s, size, "CUDA%d", num - 1);
+	else
+		snprintf(s, size, "OpenCL%d", num - conf->ncuda - 1);
+}
+
+#ifdef STARPU_DEVEL
+#warning TODO: use another way to start main, when simgrid provides it, and then include the application-provided configuration for platform numbers
+#endif
+#undef main
+int main(int argc, char **argv)
+{
+	xbt_dynar_t hosts;
+	int i, j;
+	char name[] = "/tmp/starpu-simgrid-platform.xml.XXXXXX";
+	int fd;
+	FILE *file;
+	struct starpu_machine_topology *topology = &_starpu_get_machine_config()->topology;
+
+	if (!starpu_main)
+	{
+		fprintf(stderr,"The main file of this application needs to be compiled with starpu.h included, to properly define starpu_main\n");
+		exit(EXIT_FAILURE);
+	}
+
+	MSG_init(&argc, argv);
+	MSG_config("workstation/model", "ptask_L07");
+
+	/* Create platform file */
+	struct starpu_conf conf;
+	starpu_conf_init(&conf);
+	if (!getenv("STARPU_NCPUS")
+#ifdef STARPU_USE_CUDA
+	 || !getenv("STARPU_NCUDA")
+#endif
+#ifdef STARPU_USE_OPENCL
+	 || !getenv("STARPU_NOPENCL")
+#endif
+			)
+	{
+		fprintf(stderr,"Please specify the number of cpus and gpus\n");
+		exit(EXIT_FAILURE);
+	}
+	_starpu_conf_check_environment(&conf);
+
+	_starpu_load_bus_performance_files();
+
+	topology->ncpus = conf.ncpus;
+	topology->ncudagpus = conf.ncuda;
+	topology->nopenclgpus = conf.nopencl;
+
+	fd = mkstemp(name);
+	file = fdopen(fd, "w");
+	fprintf(file,
+"<?xml version='1.0'?>\n"
+" <!DOCTYPE platform SYSTEM 'http://simgrid.gforge.inria.fr/simgrid.dtd'>\n"
+" <platform version='3'>\n"
+" <AS  id='AS0'  routing='Full'>\n"
+"   <host id='MAIN' power='1'/>\n"
+		);
+
+	for (i = 0; i < conf.ncpus; i++)
+		fprintf(file, "   <host id='CPU%d' power='2000000000'/>\n", i);
+
+	for (i = 0; i < conf.ncuda; i++)
+		fprintf(file, "   <host id='CUDA%d' power='2000000000'/>\n", i);
+
+	for (i = 0; i < conf.nopencl; i++)
+		fprintf(file, "   <host id='OpenCL%d' power='2000000000'/>\n", i);
+
+	fprintf(file, "\n   <host id='RAM' power='1'/>\n");
+
+	/* Compute maximum bandwidth, taken as machine bandwidth */
+	double max_bandwidth = 0;
+	for (i = 1; i < conf.ncuda + conf.nopencl + 1; i++)
+	{
+		if (max_bandwidth < _starpu_transfer_bandwidth(0, i))
+			max_bandwidth = _starpu_transfer_bandwidth(0, i);
+		if (max_bandwidth < _starpu_transfer_bandwidth(i, 0))
+			max_bandwidth = _starpu_transfer_bandwidth(i, 0);
+	}
+	fprintf(file, "\n   <link id='Share' bandwidth='%f' latency='0.000000'/>\n\n", max_bandwidth*1000000);
+
+	for (i = 0; i < conf.ncuda + conf.nopencl + 1; i++)
+	{
+		char i_name[16];
+		bus_name(&conf, i_name, sizeof(i_name), i);
+
+		for (j = 0; j < conf.ncuda + conf.nopencl + 1; j++)
+		{
+			char j_name[16];
+			if (j == i)
+				continue;
+			bus_name(&conf, j_name, sizeof(j_name), j);
+			fprintf(file, "   <link id='%s-%s' bandwidth='%f' latency='%f'/>\n",
+				i_name, j_name,
+				_starpu_transfer_bandwidth(i, j) * 1000000,
+				_starpu_transfer_latency(i, j) / 1000000);
+		}
+	}
+
+	for (i = 0; i < conf.ncuda + conf.nopencl + 1; i++)
+	{
+		char i_name[16];
+		bus_name(&conf, i_name, sizeof(i_name), i);
+
+		for (j = 0; j < conf.ncuda + conf.nopencl + 1; j++)
+		{
+			char j_name[16];
+			if (j == i)
+				continue;
+			bus_name(&conf, j_name, sizeof(j_name), j);
+			fprintf(file,
+"   <route src='%s' dst='%s' symmetrical='NO'><link_ctn id='%s-%s'/><link_ctn id='Share'/></route>\n",
+				i_name, j_name, i_name, j_name);
+		}
+	}
+
+	fprintf(file, 
+" </AS>\n"
+" </platform>\n"
+		);
+	fclose(file);
+	close(fd);
+
+	/* and load it */
+	MSG_create_environment(name);
+	unlink(name);
+
+	hosts = MSG_hosts_as_dynar();
+	int nb = xbt_dynar_length(hosts);
+	for (i = 0; i < nb; i++)
+		MSG_host_set_data(xbt_dynar_get_as(hosts, i, msg_host_t), calloc(MAX_TSD, sizeof(void*)));
+	MSG_process_create("main", &starpu_main, NULL, xbt_dynar_get_as(hosts, 0, msg_host_t));
+	xbt_dynar_free(&hosts);
+
+	MSG_main();
+	return 0;
+}
+
+void _starpu_simgrid_execute_job(struct _starpu_job *j, enum starpu_perf_archtype perf_arch)
+{
+	struct starpu_task *task = j->task;
+	msg_task_t simgrid_task;
+	double length = starpu_task_expected_length(task, perf_arch, j->nimpl);
+
+	STARPU_ASSERT_MSG(!_STARPU_IS_ZERO(length) && !isnan(length), "Codelets need to have a calibrated perfmodel");
+
+	simgrid_task = MSG_task_create(_starpu_job_get_model_name(j),
+			length/1000000.0*MSG_get_host_speed(MSG_host_self()),
+			0, NULL);
+	MSG_task_execute(simgrid_task);
+}
+
+msg_task_t _starpu_simgrid_transfer_task_create(unsigned src_node, unsigned dst_node, size_t size)
+{
+	msg_host_t *hosts = calloc(2, sizeof(*hosts));
+	double *computation = calloc(2, sizeof(*hosts));
+	double *communication = calloc(4, sizeof(*hosts));
+
+	hosts[0] = _starpu_simgrid_memory_node_get_host(src_node);
+	hosts[1] = _starpu_simgrid_memory_node_get_host(dst_node);
+	communication[1] = size;
+
+	return MSG_parallel_task_create("copy", 2, hosts, computation, communication, NULL);
+}
+
+struct completion {
+	msg_task_t task;
+	unsigned *finished;
+	_starpu_pthread_mutex_t *mutex;
+	_starpu_pthread_cond_t *cond;
+};
+
+int transfer_execute(int argc STARPU_ATTRIBUTE_UNUSED, char *argv[] STARPU_ATTRIBUTE_UNUSED)
+{
+	struct completion *completion = MSG_process_get_data(MSG_process_self());
+	MSG_task_execute(completion->task);
+	MSG_task_destroy(completion->task);
+	_STARPU_PTHREAD_MUTEX_LOCK(completion->mutex);
+	*completion->finished = 1;
+	fprintf(stderr, "transfer finished\n");
+	_STARPU_PTHREAD_COND_BROADCAST(completion->cond);
+	_STARPU_PTHREAD_MUTEX_UNLOCK(completion->mutex);
+	free(completion);
+	return 0;
+}
+
+void _starpu_simgrid_post_task(msg_task_t task, unsigned *finished, _starpu_pthread_mutex_t *mutex, _starpu_pthread_cond_t *cond)
+{
+	struct completion *completion = malloc(sizeof (*completion));
+	completion->task = task;
+	completion->finished = finished;
+	completion->mutex = mutex;
+	completion->cond = cond;
+	xbt_dynar_t hosts = MSG_hosts_as_dynar();
+	MSG_process_create("transfer task", transfer_execute, completion, xbt_dynar_get_as(hosts, 0, msg_host_t));
+	fprintf(stderr,"process started\n");
+	xbt_dynar_free(&hosts);
+}
+
+static int last_key;
+
+int _starpu_pthread_key_create(_starpu_pthread_key_t *key)
+{
+	/* Note: no synchronization here, we are actually monothreaded anyway. */
+	STARPU_ASSERT(last_key < MAX_TSD);
+	*key = last_key++;
+	return 0;
+}
+
+int _starpu_pthread_key_delete(_starpu_pthread_key_t key)
+{
+#ifdef STARPU_DEVEL
+#warning TODO: implement pthread_key_delete so simgridified starpu can be restarted at will
+#endif
+	return 0;
+}
+
+int _starpu_pthread_setspecific(_starpu_pthread_key_t key, void *ptr)
+{
+	void **array = MSG_host_get_data(MSG_host_self());
+	array[key] = ptr;
+	return 0;
+}
+
+void* _starpu_pthread_getspecific(_starpu_pthread_key_t key)
+{
+	void **array = MSG_host_get_data(MSG_host_self());
+	return array[key];
+}
+
+int
+_starpu_simgrid_thread_start(int argc STARPU_ATTRIBUTE_UNUSED, char *argv[] STARPU_ATTRIBUTE_UNUSED)
+{
+	struct _starpu_pthread_args *args = MSG_process_get_data(MSG_process_self());
+	args->f(args->arg);
+	free(args);
+	return 0;
+}
+#endif

+ 28 - 0
src/core/simgrid.h

@@ -0,0 +1,28 @@
+/* StarPU --- Runtime system for heterogeneous multicore architectures.
+ *
+ * Copyright (C) 2012  Université de Bordeaux 1
+ *
+ * StarPU is free software; you can redistribute it and/or modify
+ * it under the terms of the GNU Lesser General Public License as published by
+ * the Free Software Foundation; either version 2.1 of the License, or (at
+ * your option) any later version.
+ *
+ * StarPU is distributed in the hope that it will be useful, but
+ * WITHOUT ANY WARRANTY; without even the implied warranty of
+ * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE.
+ *
+ * See the GNU Lesser General Public License in COPYING.LGPL for more details.
+ */
+
+#ifndef __SIMGRID_H__
+#define __SIMGRID_H__
+
+#ifdef STARPU_SIMGRID
+#include <msg/msg.h>
+
+void _starpu_simgrid_execute_job(struct _starpu_job *job, enum starpu_perf_archtype perf_arch);
+msg_task_t _starpu_simgrid_transfer_task_create(unsigned src_node, unsigned dst_node, size_t size);
+void _starpu_simgrid_post_task(msg_task_t task, unsigned *finished, _starpu_pthread_mutex_t *mutex, _starpu_pthread_cond_t *cond);
+#endif
+
+#endif // __SIMGRID_H__

+ 44 - 1
src/core/topology.c

@@ -37,6 +37,11 @@
 #include <windows.h>
 #include <windows.h>
 #endif
 #endif
 
 
+#ifdef STARPU_SIMGRID
+#include <msg/msg.h>
+#include <core/simgrid.h>
+#endif
+
 static unsigned topology_is_initialized = 0;
 static unsigned topology_is_initialized = 0;
 
 
 #if defined(STARPU_USE_CUDA) || defined(STARPU_USE_OPENCL)
 #if defined(STARPU_USE_CUDA) || defined(STARPU_USE_OPENCL)
@@ -266,6 +271,12 @@ _starpu_init_topology (struct _starpu_machine_config *config)
 	if (topology_is_initialized)
 	if (topology_is_initialized)
 		return;
 		return;
 
 
+#ifdef STARPU_SIMGRID
+	struct starpu_conf *conf = config->conf;
+	topology->nhwcpus = conf->ncpus;
+	topology->nhwcudagpus = conf->ncuda;
+	topology->nhwopenclgpus = conf->nopencl;
+#else
 	topology->nhwcpus = 0;
 	topology->nhwcpus = 0;
 
 
 #ifdef STARPU_HAVE_HWLOC
 #ifdef STARPU_HAVE_HWLOC
@@ -276,6 +287,7 @@ _starpu_init_topology (struct _starpu_machine_config *config)
 	_starpu_cpu_discover_devices(config);
 	_starpu_cpu_discover_devices(config);
 	_starpu_cuda_discover_devices(config);
 	_starpu_cuda_discover_devices(config);
 	_starpu_opencl_discover_devices(config);
 	_starpu_opencl_discover_devices(config);
+#endif
 
 
 	topology_is_initialized = 1;
 	topology_is_initialized = 1;
 }
 }
@@ -450,6 +462,7 @@ _starpu_init_machine_config (struct _starpu_machine_config *config)
 #ifdef STARPU_USE_CUDA
 #ifdef STARPU_USE_CUDA
 	int ncuda = config->conf->ncuda;
 	int ncuda = config->conf->ncuda;
 
 
+#ifndef STARPU_SIMGRID
 	if (ncuda != 0)
 	if (ncuda != 0)
 	{
 	{
 		/* The user did not disable CUDA. We need to initialize CUDA
 		/* The user did not disable CUDA. We need to initialize CUDA
@@ -473,6 +486,7 @@ _starpu_init_machine_config (struct _starpu_machine_config *config)
 			}
 			}
 		}
 		}
 	}
 	}
+#endif
 
 
 	/* Now we know how many CUDA devices will be used */
 	/* Now we know how many CUDA devices will be used */
 	topology->ncudagpus = ncuda;
 	topology->ncudagpus = ncuda;
@@ -506,6 +520,7 @@ _starpu_init_machine_config (struct _starpu_machine_config *config)
 #ifdef STARPU_USE_OPENCL
 #ifdef STARPU_USE_OPENCL
 	int nopencl = config->conf->nopencl;
 	int nopencl = config->conf->nopencl;
 
 
+#ifndef STARPU_SIMGRID
 	if (nopencl != 0)
 	if (nopencl != 0)
 	{
 	{
 		/* The user did not disable OPENCL. We need to initialize
 		/* The user did not disable OPENCL. We need to initialize
@@ -542,6 +557,7 @@ _starpu_init_machine_config (struct _starpu_machine_config *config)
 			}
 			}
 		}
 		}
 	}
 	}
+#endif
 
 
 	topology->nopenclgpus = nopencl;
 	topology->nopenclgpus = nopencl;
 	STARPU_ASSERT(topology->nopenclgpus + topology->nworkers <= STARPU_NMAXWORKERS);
 	STARPU_ASSERT(topology->nopenclgpus + topology->nworkers <= STARPU_NMAXWORKERS);
@@ -669,6 +685,9 @@ _starpu_bind_thread_on_cpu (
 	struct _starpu_machine_config *config STARPU_ATTRIBUTE_UNUSED,
 	struct _starpu_machine_config *config STARPU_ATTRIBUTE_UNUSED,
 	unsigned cpuid)
 	unsigned cpuid)
 {
 {
+#ifdef STARPU_SIMGRID
+	return;
+#endif
 	if (starpu_get_env_number("STARPU_WORKERS_NOBIND") > 0)
 	if (starpu_get_env_number("STARPU_WORKERS_NOBIND") > 0)
 		return;
 		return;
 #ifdef STARPU_HAVE_HWLOC
 #ifdef STARPU_HAVE_HWLOC
@@ -732,8 +751,11 @@ _starpu_bind_thread_on_cpu (
 void
 void
 _starpu_bind_thread_on_cpus (
 _starpu_bind_thread_on_cpus (
 	struct _starpu_machine_config *config STARPU_ATTRIBUTE_UNUSED,
 	struct _starpu_machine_config *config STARPU_ATTRIBUTE_UNUSED,
-	struct _starpu_combined_worker *combined_worker)
+	struct _starpu_combined_worker *combined_worker STARPU_ATTRIBUTE_UNUSED)
 {
 {
+#ifdef STARPU_SIMGRID
+	return;
+#endif
 #ifdef STARPU_HAVE_HWLOC
 #ifdef STARPU_HAVE_HWLOC
 	const struct hwloc_topology_support *support;
 	const struct hwloc_topology_support *support;
 
 
@@ -779,6 +801,18 @@ _starpu_init_workers_binding (struct _starpu_machine_config *config)
 	/* TODO : support NUMA  ;) */
 	/* TODO : support NUMA  ;) */
 	ram_memory_node = _starpu_register_memory_node(STARPU_CPU_RAM, -1);
 	ram_memory_node = _starpu_register_memory_node(STARPU_CPU_RAM, -1);
 
 
+#ifdef STARPU_SIMGRID
+	xbt_dynar_t hosts = MSG_hosts_as_dynar();
+	int nb = xbt_dynar_length(hosts), i;
+	for (i = 0; i < nb; i++) {
+		msg_host_t host = xbt_dynar_get_as(hosts, i, msg_host_t);
+		if (!memcmp(MSG_host_get_name(host), "RAM", 4)) {
+			_starpu_simgrid_memory_node_set_host(0, host);
+			break;
+		}
+	}
+#endif
+
 	/* We will store all the busid of the different (src, dst)
 	/* We will store all the busid of the different (src, dst)
 	 * combinations in a matrix which we initialize here. */
 	 * combinations in a matrix which we initialize here. */
 	_starpu_initialize_busid_matrix();
 	_starpu_initialize_busid_matrix();
@@ -820,6 +854,9 @@ _starpu_init_workers_binding (struct _starpu_machine_config *config)
 				}
 				}
 				is_a_set_of_accelerators = 0;
 				is_a_set_of_accelerators = 0;
 				memory_node = _starpu_register_memory_node(STARPU_CUDA_RAM, workerarg->devid);
 				memory_node = _starpu_register_memory_node(STARPU_CUDA_RAM, workerarg->devid);
+#ifdef STARPU_SIMGRID
+				_starpu_simgrid_memory_node_set_host(memory_node, xbt_dynar_get_as(hosts, worker+1, msg_host_t));
+#endif
 				_starpu_memory_node_worker_add(memory_node);
 				_starpu_memory_node_worker_add(memory_node);
 
 
 				_starpu_register_bus(0, memory_node);
 				_starpu_register_bus(0, memory_node);
@@ -850,6 +887,9 @@ _starpu_init_workers_binding (struct _starpu_machine_config *config)
 				}
 				}
 				is_a_set_of_accelerators = 0;
 				is_a_set_of_accelerators = 0;
 				memory_node = _starpu_register_memory_node(STARPU_OPENCL_RAM, workerarg->devid);
 				memory_node = _starpu_register_memory_node(STARPU_OPENCL_RAM, workerarg->devid);
+#ifdef STARPU_SIMGRID
+				_starpu_simgrid_memory_node_set_host(memory_node, xbt_dynar_get_as(hosts, worker+1, msg_host_t));
+#endif
 				_starpu_memory_node_worker_add(memory_node);
 				_starpu_memory_node_worker_add(memory_node);
 				_starpu_register_bus(0, memory_node);
 				_starpu_register_bus(0, memory_node);
 				_starpu_register_bus(memory_node, 0);
 				_starpu_register_bus(memory_node, 0);
@@ -899,6 +939,9 @@ _starpu_init_workers_binding (struct _starpu_machine_config *config)
 			hwloc_bitmap_dup (worker_obj->cpuset);
 			hwloc_bitmap_dup (worker_obj->cpuset);
 #endif
 #endif
 	}
 	}
+#ifdef STARPU_SIMGRID
+	xbt_dynar_free(&hosts);
+#endif
 }
 }
 
 
 
 

+ 32 - 9
src/core/workers.c

@@ -32,6 +32,10 @@
 #include <drivers/cuda/driver_cuda.h>
 #include <drivers/cuda/driver_cuda.h>
 #include <drivers/opencl/driver_opencl.h>
 #include <drivers/opencl/driver_opencl.h>
 
 
+#ifdef STARPU_SIMGRID
+#include <msg/msg.h>
+#endif
+
 #ifdef __MINGW32__
 #ifdef __MINGW32__
 #include <windows.h>
 #include <windows.h>
 #endif
 #endif
@@ -383,11 +387,13 @@ static void _starpu_launch_drivers(struct _starpu_machine_config *config)
 				driver.id.cpu_id = cpu;
 				driver.id.cpu_id = cpu;
 				if (_starpu_may_launch_driver(config->conf, &driver))
 				if (_starpu_may_launch_driver(config->conf, &driver))
 				{
 				{
-					_STARPU_PTHREAD_CREATE(
+					_STARPU_PTHREAD_CREATE_ON(
+						workerarg->name,
 						&workerarg->worker_thread,
 						&workerarg->worker_thread,
 						NULL,
 						NULL,
 						_starpu_cpu_worker,
 						_starpu_cpu_worker,
-						workerarg);
+						workerarg,
+						worker+1);
 				}
 				}
 				else
 				else
 				{
 				{
@@ -403,11 +409,13 @@ static void _starpu_launch_drivers(struct _starpu_machine_config *config)
 				driver.id.cuda_id = cuda;
 				driver.id.cuda_id = cuda;
 				if (_starpu_may_launch_driver(config->conf, &driver))
 				if (_starpu_may_launch_driver(config->conf, &driver))
 				{
 				{
-					_STARPU_PTHREAD_CREATE(
+					_STARPU_PTHREAD_CREATE_ON(
+						workerarg->name,
 						&workerarg->worker_thread,
 						&workerarg->worker_thread,
 						NULL,
 						NULL,
 						_starpu_cuda_worker,
 						_starpu_cuda_worker,
-						workerarg);
+						workerarg,
+						worker+1);
 				}
 				}
 				else
 				else
 				{
 				{
@@ -426,11 +434,13 @@ static void _starpu_launch_drivers(struct _starpu_machine_config *config)
 				}
 				}
 				workerarg->set = NULL;
 				workerarg->set = NULL;
 				workerarg->worker_is_initialized = 0;
 				workerarg->worker_is_initialized = 0;
-				_STARPU_PTHREAD_CREATE(
+				_STARPU_PTHREAD_CREATE_ON(
+					workerarg->name,
 					&workerarg->worker_thread,
 					&workerarg->worker_thread,
 					NULL,
 					NULL,
 					_starpu_opencl_worker,
 					_starpu_opencl_worker,
-					workerarg);
+					workerarg,
+					worker+1);
 				break;
 				break;
 #endif
 #endif
 #ifdef STARPU_USE_GORDON
 #ifdef STARPU_USE_GORDON
@@ -444,11 +454,13 @@ static void _starpu_launch_drivers(struct _starpu_machine_config *config)
 
 
 					gordon_worker_set.set_is_initialized = 0;
 					gordon_worker_set.set_is_initialized = 0;
 
 
-					_STARPU_PTHREAD_CREATE(
+					_STARPU_PTHREAD_CREATE_ON(
+						workerarg->name
 						&gordon_worker_set.worker_thread,
 						&gordon_worker_set.worker_thread,
 						NULL,
 						NULL,
 						_starpu_gordon_worker,
 						_starpu_gordon_worker,
-						&gordon_worker_set);
+						&gordon_worker_set,
+						worker+1);
 
 
 					_STARPU_PTHREAD_MUTEX_LOCK(&gordon_worker_set.mutex);
 					_STARPU_PTHREAD_MUTEX_LOCK(&gordon_worker_set.mutex);
 					while (!gordon_worker_set.set_is_initialized)
 					while (!gordon_worker_set.set_is_initialized)
@@ -615,7 +627,7 @@ static void _starpu_conf_set_value_against_environment(char *name, int *value)
 	}
 	}
 }
 }
 
 
-static void _starpu_conf_check_environment(struct starpu_conf *conf)
+void _starpu_conf_check_environment(struct starpu_conf *conf)
 {
 {
 	char *sched = getenv("STARPU_SCHED");
 	char *sched = getenv("STARPU_SCHED");
 	if (sched)
 	if (sched)
@@ -787,6 +799,12 @@ static void _starpu_terminate_workers(struct _starpu_machine_config *config)
 		{
 		{
 			if (!set->joined)
 			if (!set->joined)
 			{
 			{
+#ifdef STARPU_SIMGRID
+#ifdef STARPU_DEVEL
+#warning TODO: use a simgrid_join when it becomes available
+#endif
+				MSG_process_sleep(1);
+#else
 				if (!pthread_equal(pthread_self(), set->worker_thread))
 				if (!pthread_equal(pthread_self(), set->worker_thread))
 				{
 				{
 					status = pthread_join(set->worker_thread, NULL);
 					status = pthread_join(set->worker_thread, NULL);
@@ -797,6 +815,7 @@ static void _starpu_terminate_workers(struct _starpu_machine_config *config)
                                         }
                                         }
 #endif
 #endif
 				}
 				}
+#endif
 
 
 				set->joined = 1;
 				set->joined = 1;
 			}
 			}
@@ -806,6 +825,9 @@ static void _starpu_terminate_workers(struct _starpu_machine_config *config)
 			if (!worker->run_by_starpu)
 			if (!worker->run_by_starpu)
 				goto out;
 				goto out;
 
 
+#ifdef STARPU_SIMGRID
+			MSG_process_sleep(1);
+#else
 			if (!pthread_equal(pthread_self(), worker->worker_thread))
 			if (!pthread_equal(pthread_self(), worker->worker_thread))
 			{
 			{
 				status = pthread_join(worker->worker_thread, NULL);
 				status = pthread_join(worker->worker_thread, NULL);
@@ -816,6 +838,7 @@ static void _starpu_terminate_workers(struct _starpu_machine_config *config)
                                 }
                                 }
 #endif
 #endif
 			}
 			}
+#endif
 		}
 		}
 
 
 out:
 out:

+ 3 - 0
src/core/workers.h

@@ -166,6 +166,9 @@ struct _starpu_machine_config
 	unsigned submitting;
 	unsigned submitting;
 };
 };
 
 
+/* Fill conf with environment variables */
+void _starpu_conf_check_environment(struct starpu_conf *conf);
+
 /* Has starpu_shutdown already been called ? */
 /* Has starpu_shutdown already been called ? */
 unsigned _starpu_machine_is_running(void);
 unsigned _starpu_machine_is_running(void);
 
 

+ 46 - 9
src/datawizard/copy_driver.c

@@ -27,6 +27,11 @@
 #include <starpu_cuda.h>
 #include <starpu_cuda.h>
 #include <profiling/profiling.h>
 #include <profiling/profiling.h>
 
 
+#ifdef STARPU_SIMGRID
+#include <core/simgrid.h>
+#include <msg/msg.h>
+#endif
+
 void _starpu_wake_all_blocked_workers_on_node(unsigned nodeid)
 void _starpu_wake_all_blocked_workers_on_node(unsigned nodeid)
 {
 {
 	/* wake up all workers on that memory node */
 	/* wake up all workers on that memory node */
@@ -87,29 +92,47 @@ static int copy_data_1_to_1_generic(starpu_data_handle_t handle,
 				    struct _starpu_data_replicate *dst_replicate,
 				    struct _starpu_data_replicate *dst_replicate,
 				    struct _starpu_data_request *req STARPU_ATTRIBUTE_UNUSED)
 				    struct _starpu_data_request *req STARPU_ATTRIBUTE_UNUSED)
 {
 {
-	int ret = 0;
-
-	const struct starpu_data_copy_methods *copy_methods = handle->ops->copy_methods;
-
 	unsigned src_node = src_replicate->memory_node;
 	unsigned src_node = src_replicate->memory_node;
 	unsigned dst_node = dst_replicate->memory_node;
 	unsigned dst_node = dst_replicate->memory_node;
 
 
-	enum starpu_node_kind src_kind = starpu_node_get_kind(src_node);
-	enum starpu_node_kind dst_kind = starpu_node_get_kind(dst_node);
-
 	STARPU_ASSERT(src_replicate->refcnt);
 	STARPU_ASSERT(src_replicate->refcnt);
 	STARPU_ASSERT(dst_replicate->refcnt);
 	STARPU_ASSERT(dst_replicate->refcnt);
 
 
 	STARPU_ASSERT(src_replicate->allocated);
 	STARPU_ASSERT(src_replicate->allocated);
 	STARPU_ASSERT(dst_replicate->allocated);
 	STARPU_ASSERT(dst_replicate->allocated);
 
 
+	_starpu_comm_amounts_inc(src_node, dst_node, handle->ops->get_size(handle));
+
+#ifdef STARPU_SIMGRID
+	msg_task_t task = _starpu_simgrid_transfer_task_create(src_node, dst_node, handle->ops->get_size(handle));
+	if (!req) {
+		/* this is not associated to a request so it's synchronous */
+		MSG_task_execute(task);
+		MSG_task_destroy(task);
+		return 0;
+	}
+	_STARPU_TRACE_START_DRIVER_COPY_ASYNC(src_node, dst_node);
+	req->async_channel.event.finished = 0;
+	_STARPU_PTHREAD_MUTEX_INIT(&req->async_channel.event.mutex, NULL);
+	_STARPU_PTHREAD_COND_INIT(&req->async_channel.event.cond, NULL);
+	_starpu_simgrid_post_task(task, &req->async_channel.event.finished, &req->async_channel.event.mutex, &req->async_channel.event.cond);
+	_STARPU_TRACE_END_DRIVER_COPY_ASYNC(src_node, dst_node);
+	_STARPU_TRACE_DATA_COPY(src_node, dst_node, handle->ops->get_size(handle));
+	return -EAGAIN;
+#else /* !SIMGRID */
+
+	int ret = 0;
+
+	const struct starpu_data_copy_methods *copy_methods = handle->ops->copy_methods;
+
+	enum starpu_node_kind src_kind = starpu_node_get_kind(src_node);
+	enum starpu_node_kind dst_kind = starpu_node_get_kind(dst_node);
+
 #ifdef STARPU_USE_CUDA
 #ifdef STARPU_USE_CUDA
 	cudaError_t cures;
 	cudaError_t cures;
 	cudaStream_t stream;
 	cudaStream_t stream;
 #endif
 #endif
 
 
-	_starpu_comm_amounts_inc(src_node, dst_node, handle->ops->get_size(handle));
-
 	void *src_interface = src_replicate->data_interface;
 	void *src_interface = src_replicate->data_interface;
 	void *dst_interface = dst_replicate->data_interface;
 	void *dst_interface = dst_replicate->data_interface;
 
 
@@ -248,6 +271,7 @@ static int copy_data_1_to_1_generic(starpu_data_handle_t handle,
 	}
 	}
 
 
 	return ret;
 	return ret;
+#endif /* !SIMGRID */
 }
 }
 
 
 int __attribute__((warn_unused_result)) _starpu_driver_copy_data_1_to_1(starpu_data_handle_t handle,
 int __attribute__((warn_unused_result)) _starpu_driver_copy_data_1_to_1(starpu_data_handle_t handle,
@@ -315,6 +339,14 @@ int __attribute__((warn_unused_result)) _starpu_driver_copy_data_1_to_1(starpu_d
 
 
 void _starpu_driver_wait_request_completion(struct _starpu_async_channel *async_channel)
 void _starpu_driver_wait_request_completion(struct _starpu_async_channel *async_channel)
 {
 {
+#ifdef STARPU_SIMGRID
+	if (async_channel->event.finished)
+		return;
+	_STARPU_PTHREAD_MUTEX_LOCK(&async_channel->event.mutex);
+	while (!async_channel->event.finished)
+		_STARPU_PTHREAD_COND_WAIT(&async_channel->event.cond, &async_channel->event.mutex);
+	_STARPU_PTHREAD_MUTEX_UNLOCK(&async_channel->event.mutex);
+#else /* !SIMGRID */
 	enum starpu_node_kind kind = async_channel->type;
 	enum starpu_node_kind kind = async_channel->type;
 #ifdef STARPU_USE_CUDA
 #ifdef STARPU_USE_CUDA
 	cudaEvent_t event;
 	cudaEvent_t event;
@@ -351,10 +383,14 @@ void _starpu_driver_wait_request_completion(struct _starpu_async_channel *async_
 	default:
 	default:
 		STARPU_ABORT();
 		STARPU_ABORT();
 	}
 	}
+#endif /* !SIMGRID */
 }
 }
 
 
 unsigned _starpu_driver_test_request_completion(struct _starpu_async_channel *async_channel)
 unsigned _starpu_driver_test_request_completion(struct _starpu_async_channel *async_channel)
 {
 {
+#ifdef STARPU_SIMGRID
+	return async_channel->event.finished;
+#else /* !SIMGRID */
 	enum starpu_node_kind kind = async_channel->type;
 	enum starpu_node_kind kind = async_channel->type;
 	unsigned success = 0;
 	unsigned success = 0;
 #ifdef STARPU_USE_CUDA
 #ifdef STARPU_USE_CUDA
@@ -396,4 +432,5 @@ unsigned _starpu_driver_test_request_completion(struct _starpu_async_channel *as
 	}
 	}
 
 
 	return success;
 	return success;
+#endif /* !SIMGRID */
 }
 }

+ 8 - 1
src/datawizard/copy_driver.h

@@ -1,6 +1,6 @@
 /* StarPU --- Runtime system for heterogeneous multicore architectures.
 /* StarPU --- Runtime system for heterogeneous multicore architectures.
  *
  *
- * Copyright (C) 2010  Université de Bordeaux 1
+ * Copyright (C) 2010, 2012  Université de Bordeaux 1
  * Copyright (C) 2010, 2011  Centre National de la Recherche Scientifique
  * Copyright (C) 2010, 2011  Centre National de la Recherche Scientifique
  *
  *
  * StarPU is free software; you can redistribute it and/or modify
  * StarPU is free software; you can redistribute it and/or modify
@@ -41,6 +41,13 @@ struct _starpu_data_replicate;
 union _starpu_async_channel_event
 union _starpu_async_channel_event
 {
 {
 	int dummy;
 	int dummy;
+#ifdef STARPU_SIMGRID
+	struct {
+		unsigned finished;
+		_starpu_pthread_mutex_t mutex;
+		_starpu_pthread_cond_t cond;
+	};
+#endif
 #ifdef STARPU_USE_CUDA
 #ifdef STARPU_USE_CUDA
 	cudaEvent_t cuda_event;
 	cudaEvent_t cuda_event;
 #endif
 #endif

+ 45 - 3
src/datawizard/memalloc.c

@@ -242,7 +242,7 @@ static size_t free_memory_on_node(struct _starpu_mem_chunk *mc, uint32_t node)
 		if (handle && !data_was_deleted)
 		if (handle && !data_was_deleted)
 			STARPU_ASSERT(replicate->allocated);
 			STARPU_ASSERT(replicate->allocated);
 
 
-#if defined(STARPU_USE_CUDA) && defined(HAVE_CUDA_MEMCPY_PEER)
+#if defined(STARPU_USE_CUDA) && defined(HAVE_CUDA_MEMCPY_PEER) && !defined(STARPU_SIMGRID)
 		if (starpu_node_get_kind(node) == STARPU_CUDA_RAM)
 		if (starpu_node_get_kind(node) == STARPU_CUDA_RAM)
 		{
 		{
 			/* To facilitate the design of interface, we set the
 			/* To facilitate the design of interface, we set the
@@ -750,6 +750,11 @@ static size_t _starpu_get_global_mem_size(int dst_node)
 	return global_mem_size;
 	return global_mem_size;
 }
 }
 
 
+#ifdef STARPU_SIMGRID
+static _starpu_pthread_mutex_t cuda_alloc_mutex = _STARPU_PTHREAD_MUTEX_INITIALIZER;
+static _starpu_pthread_mutex_t opencl_alloc_mutex = _STARPU_PTHREAD_MUTEX_INITIALIZER;
+#endif
+
 uintptr_t
 uintptr_t
 starpu_allocate_buffer_on_node(uint32_t dst_node, size_t size)
 starpu_allocate_buffer_on_node(uint32_t dst_node, size_t size)
 {
 {
@@ -766,7 +771,19 @@ starpu_allocate_buffer_on_node(uint32_t dst_node, size_t size)
 			break;
 			break;
 #ifdef STARPU_USE_CUDA
 #ifdef STARPU_USE_CUDA
 		case STARPU_CUDA_RAM:
 		case STARPU_CUDA_RAM:
+#ifdef STARPU_SIMGRID
+#ifdef STARPU_DEVEL
+#warning TODO: record used memory, using a simgrid property to know the available memory
+#endif
+			/* Sleep 10µs for the allocation */
+			_STARPU_PTHREAD_MUTEX_LOCK(&cuda_alloc_mutex);
+			MSG_process_sleep(0.000010);
+			addr = 1;
+			status = cudaSuccess;
+			_STARPU_PTHREAD_MUTEX_UNLOCK(&cuda_alloc_mutex);
+#else
 			status = cudaMalloc((void **)&addr, size);
 			status = cudaMalloc((void **)&addr, size);
+#endif
 			if (!addr || (status != cudaSuccess))
 			if (!addr || (status != cudaSuccess))
 			{
 			{
 				if (STARPU_UNLIKELY(status != cudaErrorMemoryAllocation))
 				if (STARPU_UNLIKELY(status != cudaErrorMemoryAllocation))
@@ -781,7 +798,16 @@ starpu_allocate_buffer_on_node(uint32_t dst_node, size_t size)
 			{
 			{
                                 int ret;
                                 int ret;
 				cl_mem ptr;
 				cl_mem ptr;
-                                ret = starpu_opencl_allocate_memory(&ptr, size, CL_MEM_READ_WRITE);
+#ifdef STARPU_SIMGRID
+				/* Sleep 10µs for the allocation */
+				_STARPU_PTHREAD_MUTEX_LOCK(&opencl_alloc_mutex);
+				MSG_process_sleep(0.000010);
+				ptr = (cl_mem) 1;
+				ret = CL_SUCCESS;
+				_STARPU_PTHREAD_MUTEX_UNLOCK(&opencl_alloc_mutex);
+#else
+				ret = starpu_opencl_allocate_memory(&ptr, size, CL_MEM_READ_WRITE);
+#endif
 				if (ret)
 				if (ret)
 					addr = 0;
 					addr = 0;
 				else
 				else
@@ -809,7 +835,15 @@ starpu_free_buffer_on_node(uint32_t dst_node, uintptr_t addr)
 		case STARPU_CUDA_RAM:
 		case STARPU_CUDA_RAM:
 		{
 		{
 			cudaError_t err;
 			cudaError_t err;
+#ifdef STARPU_SIMGRID
+			_STARPU_PTHREAD_MUTEX_LOCK(&cuda_alloc_mutex);
+			/* Sleep 10µs for the free */
+			MSG_process_sleep(0.000010);
+			err = cudaSuccess;
+			_STARPU_PTHREAD_MUTEX_UNLOCK(&cuda_alloc_mutex);
+#else
 			err = cudaFree((void*)addr);
 			err = cudaFree((void*)addr);
+#endif
 			if (STARPU_UNLIKELY(err != cudaSuccess))
 			if (STARPU_UNLIKELY(err != cudaSuccess))
 				STARPU_CUDA_REPORT_ERROR(err);
 				STARPU_CUDA_REPORT_ERROR(err);
 			break;
 			break;
@@ -819,7 +853,15 @@ starpu_free_buffer_on_node(uint32_t dst_node, uintptr_t addr)
                 case STARPU_OPENCL_RAM:
                 case STARPU_OPENCL_RAM:
 		{
 		{
 			cl_int err;
 			cl_int err;
+#ifdef STARPU_SIMGRID
+			_STARPU_PTHREAD_MUTEX_LOCK(&opencl_alloc_mutex);
+			/* Sleep 10µs for the free */
+			MSG_process_sleep(0.000010);
+			err = CL_SUCCESS;
+			_STARPU_PTHREAD_MUTEX_UNLOCK(&opencl_alloc_mutex);
+#else
                         err = clReleaseMemObject((void*)addr);
                         err = clReleaseMemObject((void*)addr);
+#endif
 			if (STARPU_UNLIKELY(err != CL_SUCCESS))
 			if (STARPU_UNLIKELY(err != CL_SUCCESS))
 				STARPU_OPENCL_REPORT_ERROR(err);
 				STARPU_OPENCL_REPORT_ERROR(err);
                         break;
                         break;
@@ -878,7 +920,7 @@ static ssize_t _starpu_allocate_interface(starpu_data_handle_t handle, struct _s
 		_STARPU_TRACE_START_ALLOC(dst_node);
 		_STARPU_TRACE_START_ALLOC(dst_node);
 		STARPU_ASSERT(replicate->data_interface);
 		STARPU_ASSERT(replicate->data_interface);
 
 
-#if defined(STARPU_USE_CUDA) && defined(HAVE_CUDA_MEMCPY_PEER)
+#if defined(STARPU_USE_CUDA) && defined(HAVE_CUDA_MEMCPY_PEER) && !defined(STARPU_SIMGRID)
 		if (starpu_node_get_kind(dst_node) == STARPU_CUDA_RAM)
 		if (starpu_node_get_kind(dst_node) == STARPU_CUDA_RAM)
 		{
 		{
 			/* To facilitate the design of interface, we set the
 			/* To facilitate the design of interface, we set the

+ 13 - 1
src/datawizard/memory_nodes.c

@@ -1,6 +1,6 @@
 /* StarPU --- Runtime system for heterogeneous multicore architectures.
 /* StarPU --- Runtime system for heterogeneous multicore architectures.
  *
  *
- * Copyright (C) 2009-2011  Université de Bordeaux 1
+ * Copyright (C) 2009-2012  Université de Bordeaux 1
  * Copyright (C) 2010, 2011  Centre National de la Recherche Scientifique
  * Copyright (C) 2010, 2011  Centre National de la Recherche Scientifique
  *
  *
  * StarPU is free software; you can redistribute it and/or modify
  * StarPU is free software; you can redistribute it and/or modify
@@ -121,6 +121,18 @@ unsigned _starpu_register_memory_node(enum starpu_node_kind kind, int devid)
 	return (nnodes-1);
 	return (nnodes-1);
 }
 }
 
 
+#ifdef STARPU_SIMGRID
+void _starpu_simgrid_memory_node_set_host(unsigned node, msg_host_t host)
+{
+	descr.host[node] = host;
+}
+
+msg_host_t _starpu_simgrid_memory_node_get_host(unsigned node)
+{
+	return descr.host[node];
+}
+#endif
+
 /* TODO move in a more appropriate file  !! */
 /* TODO move in a more appropriate file  !! */
 /* Register a condition variable associated to worker which is associated to a
 /* Register a condition variable associated to worker which is associated to a
  * memory node itself. */
  * memory node itself. */

+ 11 - 0
src/datawizard/memory_nodes.h

@@ -24,6 +24,9 @@
 #include <datawizard/memalloc.h>
 #include <datawizard/memalloc.h>
 #include <common/utils.h>
 #include <common/utils.h>
 
 
+#ifdef STARPU_SIMGRID
+#include <msg/msg.h>
+#endif
 
 
 #define _STARPU_MEMORY_NODE_TUPLE(node1,node2) (node1 | (node2 << 4))
 #define _STARPU_MEMORY_NODE_TUPLE(node1,node2) (node1 | (node2 << 4))
 #define _STARPU_MEMORY_NODE_TUPLE_FIRST(tuple) (tuple & 0x0F)
 #define _STARPU_MEMORY_NODE_TUPLE_FIRST(tuple) (tuple & 0x0F)
@@ -45,6 +48,10 @@ struct _starpu_mem_node_descr
 
 
 	unsigned nworkers[STARPU_MAXNODES];
 	unsigned nworkers[STARPU_MAXNODES];
 
 
+#ifdef STARPU_SIMGRID
+	msg_host_t host[STARPU_MAXNODES];
+#endif
+
 	// TODO move this 2 lists outside struct _starpu_mem_node_descr
 	// TODO move this 2 lists outside struct _starpu_mem_node_descr
 	/* Every worker is associated to a condition variable on which the
 	/* Every worker is associated to a condition variable on which the
 	 * worker waits when there is task available. It is possible that
 	 * worker waits when there is task available. It is possible that
@@ -66,6 +73,10 @@ void _starpu_set_local_memory_node_key(unsigned *node);
 unsigned _starpu_get_local_memory_node(void);
 unsigned _starpu_get_local_memory_node(void);
 void _starpu_memory_node_worker_add(unsigned node);
 void _starpu_memory_node_worker_add(unsigned node);
 unsigned _starpu_memory_node_workers(unsigned node);
 unsigned _starpu_memory_node_workers(unsigned node);
+#ifdef STARPU_SIMGRID
+void _starpu_simgrid_memory_node_set_host(unsigned node, msg_host_t host);
+msg_host_t _starpu_simgrid_memory_node_get_host(unsigned node);
+#endif
 unsigned _starpu_register_memory_node(enum starpu_node_kind kind, int devid);
 unsigned _starpu_register_memory_node(enum starpu_node_kind kind, int devid);
 //void _starpu_memory_node_attach_queue(struct starpu_jobq_s *q, unsigned nodeid);
 //void _starpu_memory_node_attach_queue(struct starpu_jobq_s *q, unsigned nodeid);
 void _starpu_memory_node_register_condition(_starpu_pthread_cond_t *cond, _starpu_pthread_mutex_t *mutex, unsigned memory_node);
 void _starpu_memory_node_register_condition(_starpu_pthread_cond_t *cond, _starpu_pthread_mutex_t *mutex, unsigned memory_node);

+ 10 - 1
src/datawizard/progress.c

@@ -1,6 +1,6 @@
 /* StarPU --- Runtime system for heterogeneous multicore architectures.
 /* StarPU --- Runtime system for heterogeneous multicore architectures.
  *
  *
- * Copyright (C) 2009, 2010  Université de Bordeaux 1
+ * Copyright (C) 2009, 2010, 2012  Université de Bordeaux 1
  * Copyright (C) 2010, 2011  Centre National de la Recherche Scientifique
  * Copyright (C) 2010, 2011  Centre National de la Recherche Scientifique
  *
  *
  * StarPU is free software; you can redistribute it and/or modify
  * StarPU is free software; you can redistribute it and/or modify
@@ -19,9 +19,18 @@
 #include <common/config.h>
 #include <common/config.h>
 #include <datawizard/datawizard.h>
 #include <datawizard/datawizard.h>
 #include <core/workers.h>
 #include <core/workers.h>
+#ifdef STARPU_SIMGRID
+#include <msg/msg.h>
+#endif
 
 
 void _starpu_datawizard_progress(uint32_t memory_node, unsigned may_alloc)
 void _starpu_datawizard_progress(uint32_t memory_node, unsigned may_alloc)
 {
 {
+#ifdef STARPU_SIMGRID
+#if STARPU_DEVEL
+#warning FIXME
+#endif
+	MSG_process_sleep(0.000010);
+#endif
 	/* in case some other driver requested data */
 	/* in case some other driver requested data */
 	_starpu_handle_pending_node_data_requests(memory_node);
 	_starpu_handle_pending_node_data_requests(memory_node);
 	_starpu_handle_node_data_requests(memory_node, may_alloc);
 	_starpu_handle_node_data_requests(memory_node, may_alloc);

+ 10 - 0
src/drivers/cpu/driver_cpu.c

@@ -40,6 +40,11 @@
 #include <windows.h>
 #include <windows.h>
 #endif
 #endif
 
 
+#ifdef STARPU_SIMGRID
+#include <core/simgrid.h>
+#endif
+
+#ifndef STARPU_SIMGRID
 #ifdef STARPU_HAVE_HWLOC
 #ifdef STARPU_HAVE_HWLOC
 void
 void
 _starpu_cpu_discover_devices(struct _starpu_machine_config *config)
 _starpu_cpu_discover_devices(struct _starpu_machine_config *config)
@@ -92,6 +97,7 @@ _starpu_cpu_discover_devices(struct _starpu_machine_config *config)
 	config->topology.nhwcpus = 1;
 	config->topology.nhwcpus = 1;
 }
 }
 #endif
 #endif
+#endif
 
 
 
 
 /* Actually launch the job on a cpu worker.
 /* Actually launch the job on a cpu worker.
@@ -142,7 +148,11 @@ static int execute_job_on_cpu(struct _starpu_job *j, struct starpu_task *worker_
 			/* bind to parallel worker */
 			/* bind to parallel worker */
 			_starpu_bind_thread_on_cpus(cpu_args->config, _starpu_get_combined_worker_struct(j->combined_workerid));
 			_starpu_bind_thread_on_cpus(cpu_args->config, _starpu_get_combined_worker_struct(j->combined_workerid));
 		STARPU_ASSERT(func);
 		STARPU_ASSERT(func);
+#ifdef STARPU_SIMGRID
+		_starpu_simgrid_execute_job(j, perf_arch);
+#else
 		func(task->interfaces, task->cl_arg);
 		func(task->interfaces, task->cl_arg);
+#endif
 		if (is_parallel_task && cl->type == STARPU_FORKJOIN)
 		if (is_parallel_task && cl->type == STARPU_FORKJOIN)
 			/* rebind to single CPU */
 			/* rebind to single CPU */
 			_starpu_bind_thread_on_cpu(cpu_args->config, cpu_args->bindid);
 			_starpu_bind_thread_on_cpu(cpu_args->config, cpu_args->bindid);

+ 35 - 10
src/drivers/cuda/driver_cuda.c

@@ -30,6 +30,10 @@
 #include <cuda_gl_interop.h>
 #include <cuda_gl_interop.h>
 #endif
 #endif
 
 
+#ifdef STARPU_SIMGRID
+#include <core/simgrid.h>
+#endif
+
 /* the number of CUDA devices */
 /* the number of CUDA devices */
 static int ncudagpus;
 static int ncudagpus;
 
 
@@ -37,11 +41,7 @@ static cudaStream_t streams[STARPU_NMAXWORKERS];
 static cudaStream_t transfer_streams[STARPU_NMAXWORKERS];
 static cudaStream_t transfer_streams[STARPU_NMAXWORKERS];
 static struct cudaDeviceProp props[STARPU_MAXCUDADEVS];
 static struct cudaDeviceProp props[STARPU_MAXCUDADEVS];
 
 
-/* In case we want to cap the amount of memory available on the GPUs by the
- * mean of the STARPU_LIMIT_GPU_MEM, we allocate a big buffer when the driver
- * is launched. */
-static char *wasted_memory[STARPU_NMAXWORKERS];
-
+#ifndef STARPU_SIMGRID
 void
 void
 _starpu_cuda_discover_devices (struct _starpu_machine_config *config)
 _starpu_cuda_discover_devices (struct _starpu_machine_config *config)
 {
 {
@@ -56,6 +56,11 @@ _starpu_cuda_discover_devices (struct _starpu_machine_config *config)
 	config->topology.nhwcudagpus = cnt;
 	config->topology.nhwcudagpus = cnt;
 }
 }
 
 
+/* In case we want to cap the amount of memory available on the GPUs by the
+ * mean of the STARPU_LIMIT_GPU_MEM, we allocate a big buffer when the driver
+ * is launched. */
+static char *wasted_memory[STARPU_NMAXWORKERS];
+
 static void limit_gpu_mem_if_needed(unsigned devid)
 static void limit_gpu_mem_if_needed(unsigned devid)
 {
 {
 	cudaError_t cures;
 	cudaError_t cures;
@@ -98,6 +103,7 @@ static void unlimit_gpu_mem_if_needed(unsigned devid)
 		wasted_memory[devid] = NULL;
 		wasted_memory[devid] = NULL;
 	}
 	}
 }
 }
+#endif
 
 
 size_t starpu_cuda_get_global_mem_size(unsigned devid)
 size_t starpu_cuda_get_global_mem_size(unsigned devid)
 {
 {
@@ -125,8 +131,11 @@ const struct cudaDeviceProp *starpu_cuda_get_device_properties(unsigned workerid
 	return &props[devid];
 	return &props[devid];
 }
 }
 
 
-void starpu_cuda_set_device(unsigned devid)
+void starpu_cuda_set_device(unsigned devid STARPU_ATTRIBUTE_UNUSED)
 {
 {
+#ifdef STARPU_SIMGRID
+	STARPU_ABORT();
+#else
 	cudaError_t cures;
 	cudaError_t cures;
 	struct starpu_conf *conf = _starpu_get_machine_config()->conf;
 	struct starpu_conf *conf = _starpu_get_machine_config()->conf;
 #if !defined(HAVE_CUDA_MEMCPY_PEER) && defined(HAVE_CUDA_GL_INTEROP_H)
 #if !defined(HAVE_CUDA_MEMCPY_PEER) && defined(HAVE_CUDA_GL_INTEROP_H)
@@ -158,8 +167,10 @@ done:
 #endif
 #endif
 	if (STARPU_UNLIKELY(cures))
 	if (STARPU_UNLIKELY(cures))
 		STARPU_CUDA_REPORT_ERROR(cures);
 		STARPU_CUDA_REPORT_ERROR(cures);
+#endif
 }
 }
 
 
+#ifndef STARPU_SIMGRID
 static void init_context(unsigned devid)
 static void init_context(unsigned devid)
 {
 {
 	cudaError_t cures;
 	cudaError_t cures;
@@ -234,7 +245,7 @@ static void deinit_context(int workerid, unsigned devid)
 	if (cures)
 	if (cures)
 		STARPU_CUDA_REPORT_ERROR(cures);
 		STARPU_CUDA_REPORT_ERROR(cures);
 }
 }
-
+#endif /* !SIMGRID */
 
 
 /* Return the number of devices usable in the system.
 /* Return the number of devices usable in the system.
  * The value returned cannot be greater than MAXCUDADEVS */
  * The value returned cannot be greater than MAXCUDADEVS */
@@ -289,14 +300,19 @@ static int execute_job_on_cuda(struct _starpu_job *j, struct _starpu_worker *arg
 
 
 	_starpu_driver_start_job(args, j, &codelet_start, 0, profiling);
 	_starpu_driver_start_job(args, j, &codelet_start, 0, profiling);
 
 
-#ifdef HAVE_CUDA_MEMCPY_PEER
+#if defined(HAVE_CUDA_MEMCPY_PEER) && !defined(STARPU_SIMGRID)
 	/* We make sure we do manipulate the proper device */
 	/* We make sure we do manipulate the proper device */
 	starpu_cuda_set_device(args->devid);
 	starpu_cuda_set_device(args->devid);
 #endif
 #endif
 
 
 	starpu_cuda_func_t func = _starpu_task_get_cuda_nth_implementation(cl, j->nimpl);
 	starpu_cuda_func_t func = _starpu_task_get_cuda_nth_implementation(cl, j->nimpl);
 	STARPU_ASSERT(func);
 	STARPU_ASSERT(func);
+
+#ifdef STARPU_SIMGRID
+	_starpu_simgrid_execute_job(j, args->perf_arch);
+#else
 	func(task->interfaces, task->cl_arg);
 	func(task->interfaces, task->cl_arg);
+#endif
 
 
 	_starpu_driver_end_job(args, j, args->perf_arch, &codelet_end, 0, profiling);
 	_starpu_driver_end_job(args, j, args->perf_arch, &codelet_end, 0, profiling);
 
 
@@ -332,17 +348,23 @@ int _starpu_cuda_driver_init(struct starpu_driver *d)
 	struct _starpu_worker* args = _starpu_get_worker_from_driver(d);
 	struct _starpu_worker* args = _starpu_get_worker_from_driver(d);
 	STARPU_ASSERT(args);
 	STARPU_ASSERT(args);
 
 
-	unsigned devid = args->devid;
-
 	_starpu_worker_init(args, _STARPU_FUT_CUDA_KEY);
 	_starpu_worker_init(args, _STARPU_FUT_CUDA_KEY);
 
 
+#ifndef STARPU_SIMGRID
+	unsigned devid = args->devid;
+
 	init_context(devid);
 	init_context(devid);
+#endif
 
 
 	/* one more time to avoid hacks from third party lib :) */
 	/* one more time to avoid hacks from third party lib :) */
 	_starpu_bind_thread_on_cpu(args->config, args->bindid);
 	_starpu_bind_thread_on_cpu(args->config, args->bindid);
 
 
 	args->status = STATUS_UNKNOWN;
 	args->status = STATUS_UNKNOWN;
 
 
+#ifdef STARPU_SIMGRID
+	const char *devname = "Simgrid";
+	snprintf(args->name, sizeof(args->name), "CUDA %u (%s TODO GiB)", args->devid, devname);
+#else
 	/* get the device's name */
 	/* get the device's name */
 	char devname[128];
 	char devname[128];
 	strncpy(devname, props[devid].name, 128);
 	strncpy(devname, props[devid].name, 128);
@@ -358,6 +380,7 @@ int _starpu_cuda_driver_init(struct starpu_driver *d)
 #else
 #else
 	snprintf(args->name, sizeof(args->name), "CUDA %u (%s %.1f GiB)", args->devid, devname, size);
 	snprintf(args->name, sizeof(args->name), "CUDA %u (%s %.1f GiB)", args->devid, devname, size);
 #endif
 #endif
+#endif
 	snprintf(args->short_name, sizeof(args->short_name), "CUDA %u", args->devid);
 	snprintf(args->short_name, sizeof(args->short_name), "CUDA %u", args->devid);
 	_STARPU_DEBUG("cuda (%s) dev id %u thread is ready to run on CPU %d !\n", devname, devid, args->bindid);
 	_STARPU_DEBUG("cuda (%s) dev id %u thread is ready to run on CPU %d !\n", devname, devid, args->bindid);
 
 
@@ -443,7 +466,9 @@ int _starpu_cuda_driver_deinit(struct starpu_driver *d)
 	 * coherency is not maintained anymore at that point ! */
 	 * coherency is not maintained anymore at that point ! */
 	_starpu_free_all_automatically_allocated_buffers(memnode);
 	_starpu_free_all_automatically_allocated_buffers(memnode);
 
 
+#ifndef STARPU_SIMGRID
 	deinit_context(args->workerid, args->devid);
 	deinit_context(args->workerid, args->devid);
+#endif
 
 
 	_STARPU_TRACE_WORKER_DEINIT_END(_STARPU_FUT_CUDA_KEY);
 	_STARPU_TRACE_WORKER_DEINIT_END(_STARPU_FUT_CUDA_KEY);
 
 

+ 1 - 1
src/drivers/gordon/driver_gordon.c

@@ -464,7 +464,7 @@ void *_starpu_gordon_worker(void *arg)
 	_STARPU_PTHREAD_MUTEX_INIT(&progress_mutex, NULL);
 	_STARPU_PTHREAD_MUTEX_INIT(&progress_mutex, NULL);
 	_STARPU_PTHREAD_COND_INIT(&progress_cond, NULL);
 	_STARPU_PTHREAD_COND_INIT(&progress_cond, NULL);
 
 
-	_STARPU_PTHREAD_CREATE(&progress_thread, NULL,
+	_STARPU_PTHREAD_CREATE("Gordon progress", &progress_thread, NULL,
 			       gordon_worker_progress, gordon_set_arg);
 			       gordon_worker_progress, gordon_set_arg);
 
 
 	/* wait for the progression thread to be ready */
 	/* wait for the progression thread to be ready */

+ 33 - 4
src/drivers/opencl/driver_opencl.c

@@ -29,6 +29,10 @@
 #include "driver_opencl_utils.h"
 #include "driver_opencl_utils.h"
 #include <common/utils.h>
 #include <common/utils.h>
 
 
+#ifdef STARPU_SIMGRID
+#include <core/simgrid.h>
+#endif
+
 static _starpu_pthread_mutex_t big_lock = _STARPU_PTHREAD_MUTEX_INITIALIZER;
 static _starpu_pthread_mutex_t big_lock = _STARPU_PTHREAD_MUTEX_INITIALIZER;
 
 
 static cl_context contexts[STARPU_MAXOPENCLDEVS];
 static cl_context contexts[STARPU_MAXOPENCLDEVS];
@@ -38,6 +42,7 @@ static cl_command_queue transfer_queues[STARPU_MAXOPENCLDEVS];
 static cl_uint nb_devices = -1;
 static cl_uint nb_devices = -1;
 static int init_done = 0;
 static int init_done = 0;
 
 
+#ifndef STARPU_SIMGRID
 void
 void
 _starpu_opencl_discover_devices(struct _starpu_machine_config *config)
 _starpu_opencl_discover_devices(struct _starpu_machine_config *config)
 {
 {
@@ -92,6 +97,7 @@ static void unlimit_gpu_mem_if_needed(int devid)
 		wasted_memory[devid] = NULL;
 		wasted_memory[devid] = NULL;
 	}
 	}
 }
 }
+#endif
 
 
 size_t starpu_opencl_get_global_mem_size(int devid)
 size_t starpu_opencl_get_global_mem_size(int devid)
 {
 {
@@ -135,6 +141,7 @@ void starpu_opencl_get_current_context(cl_context *context)
         *context = contexts[worker->devid];
         *context = contexts[worker->devid];
 }
 }
 
 
+#ifndef STARPU_SIMGRID
 cl_int _starpu_opencl_init_context(int devid)
 cl_int _starpu_opencl_init_context(int devid)
 {
 {
 	cl_int err;
 	cl_int err;
@@ -193,9 +200,13 @@ cl_int _starpu_opencl_deinit_context(int devid)
 
 
         return CL_SUCCESS;
         return CL_SUCCESS;
 }
 }
+#endif
 
 
-cl_int starpu_opencl_allocate_memory(cl_mem *mem, size_t size, cl_mem_flags flags)
+cl_int starpu_opencl_allocate_memory(cl_mem *mem STARPU_ATTRIBUTE_UNUSED, size_t size STARPU_ATTRIBUTE_UNUSED, cl_mem_flags flags STARPU_ATTRIBUTE_UNUSED)
 {
 {
+#ifdef STARPU_SIMGRID
+	STARPU_ABORT();
+#else
 	cl_int err;
 	cl_int err;
         cl_mem memory;
         cl_mem memory;
         struct _starpu_worker *worker = _starpu_get_local_worker_key();
         struct _starpu_worker *worker = _starpu_get_local_worker_key();
@@ -223,6 +234,7 @@ cl_int starpu_opencl_allocate_memory(cl_mem *mem, size_t size, cl_mem_flags flag
 
 
         *mem = memory;
         *mem = memory;
         return CL_SUCCESS;
         return CL_SUCCESS;
+#endif
 }
 }
 
 
 cl_int starpu_opencl_copy_ram_to_opencl(void *ptr, unsigned src_node STARPU_ATTRIBUTE_UNUSED, cl_mem buffer, unsigned dst_node STARPU_ATTRIBUTE_UNUSED, size_t size, size_t offset, cl_event *event, int *ret)
 cl_int starpu_opencl_copy_ram_to_opencl(void *ptr, unsigned src_node STARPU_ATTRIBUTE_UNUSED, cl_mem buffer, unsigned dst_node STARPU_ATTRIBUTE_UNUSED, size_t size, size_t offset, cl_event *event, int *ret)
@@ -410,7 +422,9 @@ void _starpu_opencl_init(void)
 	_STARPU_PTHREAD_MUTEX_UNLOCK(&big_lock);
 	_STARPU_PTHREAD_MUTEX_UNLOCK(&big_lock);
 }
 }
 
 
+#ifndef STARPU_SIMGRID
 static unsigned _starpu_opencl_get_device_name(int dev, char *name, int lname);
 static unsigned _starpu_opencl_get_device_name(int dev, char *name, int lname);
+#endif
 static int _starpu_opencl_execute_job(struct _starpu_job *j, struct _starpu_worker *args);
 static int _starpu_opencl_execute_job(struct _starpu_job *j, struct _starpu_worker *args);
 
 
 static struct _starpu_worker*
 static struct _starpu_worker*
@@ -444,20 +458,26 @@ int _starpu_opencl_driver_init(struct starpu_driver *d)
 	args = _starpu_opencl_get_worker_from_driver(d);
 	args = _starpu_opencl_get_worker_from_driver(d);
 	STARPU_ASSERT(args);
 	STARPU_ASSERT(args);
 
 
-	int devid = args->devid;
-
 	_starpu_worker_init(args, _STARPU_FUT_OPENCL_KEY);
 	_starpu_worker_init(args, _STARPU_FUT_OPENCL_KEY);
 
 
+#ifndef STARPU_SIMGRID
+	int devid = args->devid;
+
 	_starpu_opencl_init_context(devid);
 	_starpu_opencl_init_context(devid);
+#endif
 
 
 	/* one more time to avoid hacks from third party lib :) */
 	/* one more time to avoid hacks from third party lib :) */
 	_starpu_bind_thread_on_cpu(args->config, args->bindid);
 	_starpu_bind_thread_on_cpu(args->config, args->bindid);
 
 
 	args->status = STATUS_UNKNOWN;
 	args->status = STATUS_UNKNOWN;
 
 
+#ifdef STARPU_SIMGRID
+	const char *devname = "Simgrid";
+#else
 	/* get the device's name */
 	/* get the device's name */
 	char devname[128];
 	char devname[128];
 	_starpu_opencl_get_device_name(devid, devname, 128);
 	_starpu_opencl_get_device_name(devid, devname, 128);
+#endif
 	snprintf(args->name, sizeof(args->name), "OpenCL %u (%s)", args->devid, devname);
 	snprintf(args->name, sizeof(args->name), "OpenCL %u (%s)", args->devid, devname);
 	snprintf(args->short_name, sizeof(args->short_name), "OpenCL %u", args->devid);
 	snprintf(args->short_name, sizeof(args->short_name), "OpenCL %u", args->devid);
 
 
@@ -540,11 +560,13 @@ int _starpu_opencl_driver_deinit(struct starpu_driver *d)
 	args = _starpu_opencl_get_worker_from_driver(d);
 	args = _starpu_opencl_get_worker_from_driver(d);
 	STARPU_ASSERT(args);
 	STARPU_ASSERT(args);
 
 
-	unsigned devid   = args->devid;
 	unsigned memnode = args->memory_node;
 	unsigned memnode = args->memory_node;
 
 
 	_starpu_handle_all_pending_node_data_requests(memnode);
 	_starpu_handle_all_pending_node_data_requests(memnode);
+#ifndef STARPU_SIMGRID
+	unsigned devid   = args->devid;
         _starpu_opencl_deinit_context(devid);
         _starpu_opencl_deinit_context(devid);
+#endif
 
 
 	return 0;
 	return 0;
 }
 }
@@ -568,6 +590,7 @@ void *_starpu_opencl_worker(void *arg)
 	return NULL;
 	return NULL;
 }
 }
 
 
+#ifndef STARPU_SIMGRID
 static unsigned _starpu_opencl_get_device_name(int dev, char *name, int lname)
 static unsigned _starpu_opencl_get_device_name(int dev, char *name, int lname)
 {
 {
 	int err;
 	int err;
@@ -584,6 +607,7 @@ static unsigned _starpu_opencl_get_device_name(int dev, char *name, int lname)
 	_STARPU_DEBUG("Device %d : [%s]\n", dev, name);
 	_STARPU_DEBUG("Device %d : [%s]\n", dev, name);
 	return EXIT_SUCCESS;
 	return EXIT_SUCCESS;
 }
 }
+#endif
 
 
 unsigned _starpu_opencl_get_device_count(void)
 unsigned _starpu_opencl_get_device_count(void)
 {
 {
@@ -637,7 +661,12 @@ static int _starpu_opencl_execute_job(struct _starpu_job *j, struct _starpu_work
 
 
 	starpu_opencl_func_t func = _starpu_task_get_opencl_nth_implementation(cl, j->nimpl);
 	starpu_opencl_func_t func = _starpu_task_get_opencl_nth_implementation(cl, j->nimpl);
 	STARPU_ASSERT(func);
 	STARPU_ASSERT(func);
+
+#ifdef STARPU_SIMGRID
+	_starpu_simgrid_execute_job(j, args->perf_arch);
+#else
 	func(task->interfaces, task->cl_arg);
 	func(task->interfaces, task->cl_arg);
+#endif
 
 
 	_starpu_driver_end_job(args, j, args->perf_arch, &codelet_end, 0, profiling);
 	_starpu_driver_end_job(args, j, args->perf_arch, &codelet_end, 0, profiling);
 
 

+ 2 - 4
src/top/starpu_top_connection.c

@@ -100,7 +100,6 @@ void * message_to_ui(void * p)
 
 
 void _starpu_top_communications_threads_launcher(void)
 void _starpu_top_communications_threads_launcher(void)
 {
 {
-#ifdef STARPU_TOP
 	pthread_t from_ui;
 	pthread_t from_ui;
 	pthread_t to_ui;
 	pthread_t to_ui;
 	pthread_attr_t threads_attr;
 	pthread_attr_t threads_attr;
@@ -163,8 +162,7 @@ void _starpu_top_communications_threads_launcher(void)
 	pthread_attr_init(&threads_attr);
 	pthread_attr_init(&threads_attr);
 	pthread_attr_setdetachstate(&threads_attr, PTHREAD_CREATE_DETACHED);
 	pthread_attr_setdetachstate(&threads_attr, PTHREAD_CREATE_DETACHED);
 
 
-	_STARPU_PTHREAD_CREATE(&from_ui, &threads_attr, message_from_ui, NULL);
-	_STARPU_PTHREAD_CREATE(&to_ui, &threads_attr, message_to_ui, NULL);
-#endif
+	_STARPU_PTHREAD_CREATE("StarPU-Top from", &from_ui, &threads_attr, message_from_ui, NULL);
+	_STARPU_PTHREAD_CREATE("StarPU-Top to", &to_ui, &threads_attr, message_to_ui, NULL);
 }
 }
 
 

+ 7 - 3
src/util/malloc.c

@@ -41,7 +41,7 @@ struct malloc_pinned_codelet_struct
 //}
 //}
 //#endif
 //#endif
 
 
-#if defined(STARPU_USE_CUDA) && !defined(HAVE_CUDA_MEMCPY_PEER)
+#if defined(STARPU_USE_CUDA) && !defined(HAVE_CUDA_MEMCPY_PEER) && !defined(STARPU_SIMGRID)
 static void malloc_pinned_cuda_codelet(void *buffers[] STARPU_ATTRIBUTE_UNUSED, void *arg)
 static void malloc_pinned_cuda_codelet(void *buffers[] STARPU_ATTRIBUTE_UNUSED, void *arg)
 {
 {
 	struct malloc_pinned_codelet_struct *s = arg;
 	struct malloc_pinned_codelet_struct *s = arg;
@@ -53,7 +53,7 @@ static void malloc_pinned_cuda_codelet(void *buffers[] STARPU_ATTRIBUTE_UNUSED,
 }
 }
 #endif
 #endif
 
 
-#if (defined(STARPU_USE_CUDA) && !defined(HAVE_CUDA_MEMCPY_PEER))// || defined(STARPU_USE_OPENCL)
+#if (defined(STARPU_USE_CUDA) && !defined(HAVE_CUDA_MEMCPY_PEER)) && !defined(STARPU_SIMGRID)// || defined(STARPU_USE_OPENCL)
 static struct starpu_perfmodel malloc_pinned_model =
 static struct starpu_perfmodel malloc_pinned_model =
 {
 {
 	.type = STARPU_HISTORY_BASED,
 	.type = STARPU_HISTORY_BASED,
@@ -78,6 +78,7 @@ int starpu_malloc(void **A, size_t dim)
 
 
 	STARPU_ASSERT(A);
 	STARPU_ASSERT(A);
 
 
+#ifndef STARPU_SIMGRID
 	if (_starpu_can_submit_cuda_task())
 	if (_starpu_can_submit_cuda_task())
 	{
 	{
 #ifdef STARPU_USE_CUDA
 #ifdef STARPU_USE_CUDA
@@ -136,6 +137,7 @@ int starpu_malloc(void **A, size_t dim)
 //#endif
 //#endif
 //        }
 //        }
         else
         else
+#endif
 	{
 	{
 		*A = malloc(dim);
 		*A = malloc(dim);
 	}
 	}
@@ -145,7 +147,7 @@ int starpu_malloc(void **A, size_t dim)
 	return 0;
 	return 0;
 }
 }
 
 
-#if defined(STARPU_USE_CUDA) && !defined(HAVE_CUDA_MEMCPY_PEER)
+#if defined(STARPU_USE_CUDA) && !defined(HAVE_CUDA_MEMCPY_PEER) && !defined(STARPU_SIMGRID)
 static void free_pinned_cuda_codelet(void *buffers[] STARPU_ATTRIBUTE_UNUSED, void *arg)
 static void free_pinned_cuda_codelet(void *buffers[] STARPU_ATTRIBUTE_UNUSED, void *arg)
 {
 {
 	cudaError_t cures;
 	cudaError_t cures;
@@ -187,6 +189,7 @@ int starpu_free(void *A)
 	if (STARPU_UNLIKELY(!_starpu_worker_may_perform_blocking_calls()))
 	if (STARPU_UNLIKELY(!_starpu_worker_may_perform_blocking_calls()))
 		return -EDEADLK;
 		return -EDEADLK;
 
 
+#ifndef STARPU_SIMGRID
 #ifdef STARPU_USE_CUDA
 #ifdef STARPU_USE_CUDA
 	if (_starpu_can_submit_cuda_task())
 	if (_starpu_can_submit_cuda_task())
 	{
 	{
@@ -241,6 +244,7 @@ int starpu_free(void *A)
 //	}
 //	}
 	} else
 	} else
 #endif
 #endif
+#endif
 	{
 	{
 		free(A);
 		free(A);
 	}
 	}