Browse Source

src: introduce macros STARPU_MALLOC, STARPU_CALLOC and STARPU_REALLOC to make sure we always check the return value of stdlib malloc functions

Nathalie Furmento 9 years ago
parent
commit
d4761fddf6
87 changed files with 576 additions and 538 deletions
  1. 5 5
      src/common/bitmap.c
  2. 6 3
      src/common/graph.c
  3. 3 3
      src/common/list.h
  4. 1 1
      src/common/prio_list.h
  5. 4 4
      src/common/thread.c
  6. 3 2
      src/common/utils.c
  7. 6 1
      src/common/utils.h
  8. 4 5
      src/core/combined_workers.c
  9. 2 2
      src/core/debug.c
  10. 2 3
      src/core/dependencies/cg.c
  11. 9 5
      src/core/dependencies/data_arbiter_concurrency.c
  12. 5 4
      src/core/dependencies/implicit_data_deps.c
  13. 7 9
      src/core/dependencies/tags.c
  14. 3 3
      src/core/dependencies/task_deps.c
  15. 7 7
      src/core/detect_combined_workers.c
  16. 6 16
      src/core/disk.c
  17. 13 13
      src/core/disk_ops/disk_leveldb.cpp
  18. 6 6
      src/core/disk_ops/disk_stdio.c
  19. 4 4
      src/core/disk_ops/disk_unistd.c
  20. 4 4
      src/core/disk_ops/disk_unistd_o_direct.c
  21. 8 6
      src/core/disk_ops/unistd/disk_unistd_global.c
  22. 5 5
      src/core/jobs.c
  23. 2 2
      src/core/parallel_task.c
  24. 26 23
      src/core/perfmodel/multiple_regression.c
  25. 4 4
      src/core/perfmodel/perfmodel.c
  26. 8 17
      src/core/perfmodel/perfmodel_bus.c
  27. 33 37
      src/core/perfmodel/perfmodel_history.c
  28. 5 5
      src/core/perfmodel/perfmodel_print.c
  29. 5 4
      src/core/perfmodel/regression.c
  30. 17 7
      src/core/sched_ctx.c
  31. 5 3
      src/core/sched_ctx_list.c
  32. 14 6
      src/core/simgrid.c
  33. 3 5
      src/core/task.c
  34. 6 8
      src/core/task_bundle.c
  35. 9 11
      src/core/topology.c
  36. 1 1
      src/core/tree.c
  37. 7 4
      src/core/workers.c
  38. 1 1
      src/datawizard/coherency.c
  39. 3 3
      src/datawizard/data_request.c
  40. 7 5
      src/datawizard/filters.c
  41. 6 9
      src/datawizard/interfaces/data_interface.c
  42. 4 4
      src/datawizard/interfaces/multiformat_interface.c
  43. 2 2
      src/datawizard/malloc.c
  44. 2 2
      src/datawizard/memstats.c
  45. 2 2
      src/datawizard/reduction.c
  46. 5 4
      src/datawizard/user_interactions.c
  47. 3 2
      src/debug/traces/anim.c
  48. 12 8
      src/debug/traces/starpu_fxt.c
  49. 3 3
      src/debug/traces/starpu_fxt_mpi.c
  50. 10 11
      src/drivers/mic/driver_mic_sink.c
  51. 4 4
      src/drivers/mp_common/mp_common.c
  52. 12 9
      src/drivers/mp_common/sink_common.c
  53. 5 3
      src/drivers/mp_common/source_common.c
  54. 13 11
      src/drivers/opencl/driver_opencl_utils.c
  55. 9 8
      src/profiling/bound.c
  56. 2 3
      src/profiling/profiling.c
  57. 4 2
      src/sched_policies/component_composed.c
  58. 3 2
      src/sched_policies/component_fifo.c
  59. 2 1
      src/sched_policies/component_heft.c
  60. 5 3
      src/sched_policies/component_perfmodel_select.c
  61. 2 1
      src/sched_policies/component_prio.c
  62. 6 6
      src/sched_policies/component_sched.c
  63. 10 8
      src/sched_policies/component_work_stealing.c
  64. 8 4
      src/sched_policies/component_worker.c
  65. 5 4
      src/sched_policies/deque_modeling_policy_data_aware.c
  66. 2 1
      src/sched_policies/eager_central_policy.c
  67. 5 4
      src/sched_policies/eager_central_priority_policy.c
  68. 2 2
      src/sched_policies/fifo_queues.c
  69. 2 1
      src/sched_policies/graph_test_policy.c
  70. 2 1
      src/sched_policies/helper_mct.c
  71. 2 1
      src/sched_policies/heteroprio.c
  72. 9 8
      src/sched_policies/parallel_eager.c
  73. 2 1
      src/sched_policies/parallel_heft.c
  74. 1 1
      src/sched_policies/prio_deque.c
  75. 1 1
      src/sched_policies/scheduler_maker.c
  76. 5 4
      src/sched_policies/work_stealing_policy.c
  77. 51 34
      src/top/starpu_top.c
  78. 2 6
      src/top/starpu_top_message_queue.c
  79. 7 4
      src/top/starpu_top_task.c
  80. 12 5
      src/util/fstarpu.c
  81. 23 42
      src/util/openmp_runtime_support.c
  82. 8 24
      src/util/openmp_runtime_support_environment.c
  83. 15 15
      src/util/starpu_clusters_create.c
  84. 2 1
      src/util/starpu_data_cpy.c
  85. 7 5
      src/util/starpu_task_insert_utils.c
  86. 7 3
      src/worker_collection/worker_list.c
  87. 1 1
      src/worker_collection/worker_tree.c

+ 5 - 5
src/common/bitmap.c

@@ -74,8 +74,8 @@ static int _count_bit(unsigned long e)
 
 struct starpu_bitmap * starpu_bitmap_create(void)
 {
-	struct starpu_bitmap * b = malloc(sizeof(*b));
-	memset(b,0,sizeof(*b));
+	struct starpu_bitmap *b;
+	STARPU_CALLOC(b, 1, sizeof(*b));
 	return b;
 }
 void starpu_bitmap_destroy(struct starpu_bitmap * b)
@@ -96,7 +96,7 @@ void starpu_bitmap_set(struct starpu_bitmap * b, int e)
 		return;
 	if((e/LONG_BIT) + 1 > b->size)
 	{
-		b->bits = realloc(b->bits, sizeof(unsigned long) * ((e/LONG_BIT) + 1));
+		STARPU_REALLOC(b->bits, sizeof(unsigned long) * ((e/LONG_BIT) + 1));
 		memset(b->bits + b->size, 0, sizeof(unsigned long) * ((e/LONG_BIT + 1) - b->size));
 		b->size = (e/LONG_BIT) + 1;
 	}
@@ -125,7 +125,7 @@ void starpu_bitmap_unset_all(struct starpu_bitmap * b)
 void starpu_bitmap_unset_and(struct starpu_bitmap * a, struct starpu_bitmap * b, struct starpu_bitmap * c)
 {
 	int n = STARPU_MIN(b->size, c->size);
-	a->bits = realloc(a->bits, sizeof(unsigned long) * n);
+	STARPU_REALLOC(a->bits, sizeof(unsigned long) * n);
 	a->size = n;
 	a->cardinal = 0;
 	int i;
@@ -149,7 +149,7 @@ void starpu_bitmap_or(struct starpu_bitmap * a, struct starpu_bitmap * b)
 {
 	if(a->size < b->size)
 	{
-		a->bits = realloc(a->bits, b->size * sizeof(unsigned long));
+		STARPU_REALLOC(a->bits, b->size * sizeof(unsigned long));
 		memset(a->bits + a->size, 0, (b->size - a->size) * sizeof(unsigned long));
 		a->size = b->size;
 

+ 6 - 3
src/common/graph.c

@@ -112,7 +112,8 @@ static void __starpu_graph_foreach(void (*func)(void *data, struct _starpu_graph
 /* Add a node to the graph */
 void _starpu_graph_add_job(struct _starpu_job *job)
 {
-	struct _starpu_graph_node *node = calloc(1, sizeof(*node));
+	struct _starpu_graph_node *node;
+	STARPU_CALLOC(node, 1, sizeof(*node));
 	node->job = job;
 	job->graph_node = node;
 	STARPU_PTHREAD_MUTEX_INIT(&node->mutex, NULL);
@@ -137,9 +138,11 @@ static unsigned add_node(struct _starpu_graph_node *node, struct _starpu_graph_n
 			*alloc_nodes *= 2;
 		else
 			*alloc_nodes = 4;
-		*nodes = realloc(*nodes, *alloc_nodes * sizeof(**nodes));
+		STARPU_REALLOC(*nodes, *alloc_nodes * sizeof(**nodes));
 		if (slot)
-			*slot = realloc(*slot, *alloc_nodes * sizeof(**slot));
+		{
+			STARPU_REALLOC(*slot, *alloc_nodes * sizeof(**slot));
+		}
 	}
 	ret = (*n_nodes)++;
 	(*nodes)[ret] = node;

+ 3 - 3
src/common/list.h

@@ -1,7 +1,7 @@
 /* StarPU --- Runtime system for heterogeneous multicore architectures.
  *
  * Copyright (C) 2009-2012, 2015-2016  Université de Bordeaux
- * Copyright (C) 2010, 2011  CNRS
+ * Copyright (C) 2010, 2011, 2016  CNRS
  *
  * 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
@@ -135,7 +135,7 @@
     struct ENAME *_tail; /**< @internal tail of the list */ \
   }; \
   /** @internal */static inline struct ENAME *ENAME##_new(void) \
-    { struct ENAME *e = (struct ENAME *)malloc(sizeof(struct ENAME)); \
+    { struct ENAME *e; STARPU_MALLOC(e, sizeof(struct ENAME)); \
       e->_next = NULL; e->_prev = NULL; return e; } \
   /** @internal */static inline void ENAME##_delete(struct ENAME *e) \
     { free(e); } \
@@ -164,7 +164,7 @@
   /** @internal */static inline void ENAME##_list_init(struct ENAME##_list *l) \
     { l->_head=NULL; l->_tail=l->_head; } \
   /** @internal */static inline struct ENAME##_list *ENAME##_list_new(void) \
-    { struct ENAME##_list *l; l=(struct ENAME##_list *)malloc(sizeof(struct ENAME##_list)); \
+    { struct ENAME##_list *l; STARPU_MALLOC(l, sizeof(struct ENAME##_list)); \
       ENAME##_list_init(l); return l; } \
   /** @internal */static inline int ENAME##_list_empty(const struct ENAME##_list *l) \
     { return (l->_head == NULL); } \

+ 1 - 1
src/common/prio_list.h

@@ -112,7 +112,7 @@
 		if (node) \
 			stage = ENAME##_node_to_list_stage(node); \
 		else { \
-			stage = malloc(sizeof(*stage)); \
+			STARPU_MALLOC(stage, sizeof(*stage));	\
 			starpu_rbtree_node_init(&stage->node); \
 			stage->prio = prio; \
 			_starpu_data_request_list_init(&stage->list); \

+ 4 - 4
src/common/thread.c

@@ -1,7 +1,7 @@
 /* StarPU --- Runtime system for heterogeneous multicore architectures.
  *
  * Copyright (C) 2010, 2012-2016  Université de Bordeaux
- * Copyright (C) 2010, 2011, 2012, 2013, 2014, 2015  CNRS
+ * Copyright (C) 2010, 2011, 2012, 2013, 2014, 2015, 2016  CNRS
  *
  * 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
@@ -46,7 +46,8 @@ extern int _starpu_simgrid_thread_start(int argc, char *argv[]);
 
 int starpu_pthread_create_on(char *name, starpu_pthread_t *thread, const starpu_pthread_attr_t *attr STARPU_ATTRIBUTE_UNUSED, void *(*start_routine) (void *), void *arg, msg_host_t host)
 {
-	char **_args = malloc(3*sizeof(char*));
+	char **_args;
+	STARPU_MALLOC(_args, 3*sizeof(char*));
 	asprintf(&_args[0], "%p", start_routine);
 	asprintf(&_args[1], "%p", arg);
 	_args[2] = NULL;
@@ -406,8 +407,7 @@ int starpu_pthread_queue_register(starpu_pthread_wait_t *w, starpu_pthread_queue
 		newalloc = q->allocqueue * 2;
 		if (!newalloc)
 			newalloc = 1;
-		newqueue = realloc(q->queue, newalloc * sizeof(*(q->queue)));
-		STARPU_ASSERT(newqueue);
+		STARPU_REALLOC(newqueue, newalloc * sizeof(*(q->queue)));
 		q->queue = newqueue;
 		q->allocqueue = newalloc;
 	}

+ 3 - 2
src/common/utils.c

@@ -1,7 +1,7 @@
 /* StarPU --- Runtime system for heterogeneous multicore architectures.
  *
  * Copyright (C) 2010, 2012-2016  Université de Bordeaux
- * Copyright (C) 2010, 2011, 2012, 2013, 2014, 2015  CNRS
+ * Copyright (C) 2010, 2011, 2012, 2013, 2014, 2015, 2016  CNRS
  *
  * 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
@@ -144,7 +144,8 @@ char *_starpu_mktemp(const char *directory, int flags, int *fd)
 {
 	/* create template for mkstemp */
 	const char *tmp = "STARPU_XXXXXX";
-	char *baseCpy = malloc(strlen(directory)+1+strlen(tmp)+1);
+	char *baseCpy;
+	STARPU_MALLOC(baseCpy, strlen(directory)+1+strlen(tmp)+1);
 	STARPU_ASSERT(baseCpy != NULL);
 
 	strcpy(baseCpy, directory);

+ 6 - 1
src/common/utils.h

@@ -1,7 +1,7 @@
 /* StarPU --- Runtime system for heterogeneous multicore architectures.
  *
  * Copyright (C) 2010, 2012-2016  Université de Bordeaux
- * Copyright (C) 2010, 2011, 2012, 2013, 2015  CNRS
+ * Copyright (C) 2010, 2011, 2012, 2013, 2015, 2016  CNRS
  *
  * 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
@@ -116,6 +116,11 @@
 	} while (0)
 
 
+#define STARPU_MALLOC(ptr, size) do { ptr = malloc(size); STARPU_ASSERT_MSG(ptr != NULL, "Cannot allocate %ld bytes\n", size); } while (0)
+#define STARPU_CALLOC(ptr, nmemb, size) do { ptr = calloc(nmemb, size); STARPU_ASSERT_MSG(ptr != NULL, "Cannot allocate %ld bytes\n", nmemb*size); } while (0)
+#define STARPU_REALLOC(ptr, size) do { ptr = realloc(ptr, size); STARPU_ASSERT_MSG(ptr != NULL, "Cannot reallocate %ld bytes\n", size); } while (0)
+#define STARPU_FREE(ptr) do { free(ptr); ptr = NULL; } while(0)
+
 #ifdef _MSC_VER
 #define _STARPU_IS_ZERO(a) (a == 0.0)
 #else

+ 4 - 5
src/core/combined_workers.c

@@ -1,7 +1,7 @@
 /* StarPU --- Runtime system for heterogeneous multicore architectures.
  *
  * Copyright (C) 2010-2015  Université de Bordeaux
- * Copyright (C) 2010, 2011, 2014  CNRS
+ * Copyright (C) 2010, 2011, 2014, 2016  CNRS
  *
  * 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
@@ -102,13 +102,13 @@ int starpu_combined_worker_assign_workerid(int nworkers, int workerid_array[])
 		&config->combined_workers[combined_worker_id];
 
 	combined_worker->worker_size = nworkers;
-	combined_worker->perf_arch.devices = (struct starpu_perfmodel_device*)malloc(sizeof(struct starpu_perfmodel_device));
+	STARPU_MALLOC(combined_worker->perf_arch.devices, sizeof(struct starpu_perfmodel_device));
 	combined_worker->perf_arch.ndevices = 1;
 	combined_worker->perf_arch.devices[0].type = config->workers[workerid_array[0]].perf_arch.devices[0].type;
-	combined_worker->perf_arch.devices[0].devid = config->workers[workerid_array[0]].perf_arch.devices[0].devid; 
+	combined_worker->perf_arch.devices[0].devid = config->workers[workerid_array[0]].perf_arch.devices[0].devid;
 	combined_worker->perf_arch.devices[0].ncores = nworkers;
 	combined_worker->worker_mask = config->workers[workerid_array[0]].worker_mask;
-	
+
 #ifdef STARPU_USE_MP
 	combined_worker->count = nworkers -1;
 	STARPU_PTHREAD_MUTEX_INIT(&combined_worker->count_mutex,NULL);
@@ -180,4 +180,3 @@ int starpu_combined_worker_get_description(int workerid, int *worker_size, int *
 
 	return 0;
 }
-

+ 2 - 2
src/core/debug.c

@@ -1,7 +1,7 @@
 /* StarPU --- Runtime system for heterogeneous multicore architectures.
  *
  * Copyright (C) 2009-2013  Université de Bordeaux
- * Copyright (C) 2010, 2011, 2013, 2015  CNRS
+ * Copyright (C) 2010, 2011, 2013, 2015, 2016  CNRS
  * Copyright (C) 2016  Inria
  *
  * StarPU is free software; you can redistribute it and/or modify
@@ -106,7 +106,7 @@ int64_t _starpu_ayudame_get_func_id(struct starpu_codelet *cl)
 			ncodelets_alloc = 16;
 		else
 			ncodelets_alloc *= 2;
-		codelets = realloc(codelets, ncodelets_alloc * sizeof(*codelets));
+		STARPU_REALLOC(codelets, ncodelets_alloc * sizeof(*codelets));
 	}
 	codelets[ncodelets].cl = cl;
 	if (name)

+ 2 - 3
src/core/dependencies/cg.c

@@ -1,7 +1,7 @@
 /* StarPU --- Runtime system for heterogeneous multicore architectures.
  *
  * Copyright (C) 2010-2012, 2014-2016  Université de Bordeaux
- * Copyright (C) 2010, 2011, 2012, 2013, 2015  CNRS
+ * Copyright (C) 2010, 2011, 2012, 2013, 2015, 2016  CNRS
  * Copyright (C) 2012 INRIA
  *
  * StarPU is free software; you can redistribute it and/or modify
@@ -82,8 +82,7 @@ int _starpu_add_successor_to_cg_list(struct _starpu_cg_list *successors, struct
 		else
 			successors->succ_list_size = 4;
 
-		successors->succ = (struct _starpu_cg **) realloc(successors->succ,
-			successors->succ_list_size*sizeof(struct _starpu_cg *));
+		STARPU_REALLOC(successors->succ, successors->succ_list_size*sizeof(struct _starpu_cg *));
 	}
 #else
 	STARPU_ASSERT(index < STARPU_NMAXDEPS);

+ 9 - 5
src/core/dependencies/data_arbiter_concurrency.c

@@ -127,9 +127,10 @@ struct LockOrDelegateListNode
  */
 static int _starpu_LockOrDelegatePostOrPerform(starpu_arbiter_t arbiter, void (*func)(void*), void* data)
 {
-	struct LockOrDelegateListNode* newNode = malloc(sizeof(*newNode)), *iter, *next;
+	struct LockOrDelegateListNode *newNode, *iter, *next;
 	int did = 0;
-	STARPU_ASSERT(newNode);
+
+	STARPU_MALLOC(newNode, sizeof(*newNode));
 	newNode->data = data;
 	newNode->func = func;
 
@@ -226,7 +227,8 @@ unsigned _starpu_attempt_to_submit_arbitered_data_request(unsigned request_from_
 						       void (*callback)(void *), void *argcb,
 						       struct _starpu_job *j, unsigned buffer_index)
 {
-	struct starpu_submit_arbitered_args* args = malloc(sizeof(*args));
+	struct starpu_submit_arbitered_args* args;
+	STARPU_MALLOC(args, sizeof(*args));
 	args->request_from_codelet = request_from_codelet;
 	args->handle = handle;
 	args->mode = mode;
@@ -361,7 +363,8 @@ static void __starpu_submit_job_enforce_arbitered_deps(void* inData)
 
 void _starpu_submit_job_enforce_arbitered_deps(struct _starpu_job *j, unsigned buf, unsigned nbuffers)
 {
-	struct starpu_enforce_arbitered_args* args = malloc(sizeof(*args));
+	struct starpu_enforce_arbitered_args* args;
+	STARPU_MALLOC(args, sizeof(*args));
 	starpu_data_handle_t handle = _STARPU_JOB_GET_ORDERED_BUFFER_HANDLE(j, buf);
 	args->j = j;
 	args->buf = buf;
@@ -665,7 +668,8 @@ void _starpu_notify_arbitered_dependencies(starpu_data_handle_t handle)
 
 starpu_arbiter_t starpu_arbiter_create(void)
 {
-	starpu_arbiter_t res = malloc(sizeof(*res));
+	starpu_arbiter_t res;
+	STARPU_MALLOC(res, sizeof(*res));
 
 #ifdef LOCK_OR_DELEGATE
 	res->dlTaskListHead = NULL;

+ 5 - 4
src/core/dependencies/implicit_data_deps.c

@@ -1,7 +1,7 @@
 /* StarPU --- Runtime system for heterogeneous multicore architectures.
  *
  * Copyright (C) 2010-2016  Université de Bordeaux
- * Copyright (C) 2010, 2011, 2013, 2015  CNRS
+ * Copyright (C) 2010, 2011, 2013, 2015, 2016  CNRS
  * Copyright (C) 2016  Inria
  *
  * StarPU is free software; you can redistribute it and/or modify
@@ -434,8 +434,8 @@ void _starpu_release_data_enforce_sequential_consistency(struct starpu_task *tas
 			{
 				/* Save the job id of the reader task in the ghost reader linked list list */
 				struct _starpu_job *ghost_reader_job = _starpu_get_job_associated_to_task(task);
-				struct _starpu_jobid_list *link = (struct _starpu_jobid_list *) malloc(sizeof(struct _starpu_jobid_list));
-				STARPU_ASSERT(link);
+				struct _starpu_jobid_list *link;
+				STARPU_MALLOC(link, sizeof(struct _starpu_jobid_list));
 				link->next = handle->last_submitted_ghost_accessors_id;
 				link->id = ghost_reader_job->job_id;
 				handle->last_submitted_ghost_accessors_id = link;
@@ -498,7 +498,8 @@ void _starpu_add_post_sync_tasks(struct starpu_task *post_sync_task, starpu_data
 	{
 		handle->post_sync_tasks_cnt++;
 
-		struct _starpu_task_wrapper_list *link = (struct _starpu_task_wrapper_list *) malloc(sizeof(struct _starpu_task_wrapper_list));
+		struct _starpu_task_wrapper_list *link;
+		STARPU_MALLOC(link, sizeof(struct _starpu_task_wrapper_list));
 		link->task = post_sync_task;
 		link->next = handle->post_sync_tasks;
 		handle->post_sync_tasks = link;

+ 7 - 9
src/core/dependencies/tags.c

@@ -1,7 +1,7 @@
 /* StarPU --- Runtime system for heterogeneous multicore architectures.
  *
  * Copyright (C) 2009-2013, 2016  Université de Bordeaux
- * Copyright (C) 2010, 2011, 2012, 2013  CNRS
+ * Copyright (C) 2010, 2011, 2012, 2013, 2016  CNRS
  * Copyright (C) 2016  Inria
  *
  * StarPU is free software; you can redistribute it and/or modify
@@ -44,8 +44,8 @@ static starpu_pthread_rwlock_t tag_global_rwlock;
 
 static struct _starpu_cg *create_cg_apps(unsigned ntags)
 {
-	struct _starpu_cg *cg = (struct _starpu_cg *) malloc(sizeof(struct _starpu_cg));
-	STARPU_ASSERT(cg);
+	struct _starpu_cg *cg;
+	STARPU_MALLOC(cg, sizeof(struct _starpu_cg));
 
 	cg->ntags = ntags;
 	cg->remaining = ntags;
@@ -60,8 +60,8 @@ static struct _starpu_cg *create_cg_apps(unsigned ntags)
 
 static struct _starpu_cg *create_cg_tag(unsigned ntags, struct _starpu_tag *tag)
 {
-	struct _starpu_cg *cg = (struct _starpu_cg *) malloc(sizeof(struct _starpu_cg));
-	STARPU_ASSERT(cg);
+	struct _starpu_cg *cg;
+	STARPU_MALLOC(cg, sizeof(struct _starpu_cg));
 
 	cg->ntags = ntags;
 	cg->remaining = ntags;
@@ -76,8 +76,7 @@ static struct _starpu_cg *create_cg_tag(unsigned ntags, struct _starpu_tag *tag)
 static struct _starpu_tag *_starpu_tag_init(starpu_tag_t id)
 {
 	struct _starpu_tag *tag;
-	tag = (struct _starpu_tag *) malloc(sizeof(struct _starpu_tag));
-	STARPU_ASSERT(tag);
+	STARPU_MALLOC(tag, sizeof(struct _starpu_tag));
 
 	tag->job = NULL;
 	tag->is_assigned = 0;
@@ -191,8 +190,7 @@ static struct _starpu_tag *_gettag_struct(starpu_tag_t id)
 		tag = _starpu_tag_init(id);
 
 		struct _starpu_tag_table *entry2;
-		entry2 = (struct _starpu_tag_table *) malloc(sizeof(*entry2));
-		STARPU_ASSERT(entry2 != NULL);
+		STARPU_MALLOC(entry2, sizeof(*entry2));
 		entry2->id = id;
 		entry2->tag = tag;
 

+ 3 - 3
src/core/dependencies/task_deps.c

@@ -1,7 +1,7 @@
 /* StarPU --- Runtime system for heterogeneous multicore architectures.
  *
  * Copyright (C) 2010-2016  Université de Bordeaux
- * Copyright (C) 2010, 2011, 2012, 2013, 2015  CNRS
+ * Copyright (C) 2010, 2011, 2012, 2013, 2015, 2016  CNRS
  * Copyright (C) 2014, 2016  INRIA
  *
  * StarPU is free software; you can redistribute it and/or modify
@@ -30,8 +30,8 @@
 
 static struct _starpu_cg *create_cg_task(unsigned ntags, struct _starpu_job *j)
 {
-	struct _starpu_cg *cg = (struct _starpu_cg *) malloc(sizeof(struct _starpu_cg));
-	STARPU_ASSERT(cg);
+	struct _starpu_cg *cg;
+	STARPU_MALLOC(cg, sizeof(struct _starpu_cg));
 
 	cg->ntags = ntags;
 	cg->remaining = ntags;

+ 7 - 7
src/core/detect_combined_workers.c

@@ -1,7 +1,7 @@
 /* StarPU --- Runtime system for heterogeneous multicore architectures.
  *
  * Copyright (C) 2010-2016  Université de Bordeaux
- * Copyright (C) 2011, 2012, 2013       CNRS
+ * Copyright (C) 2011, 2012, 2013, 2016       CNRS
  *
  * 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
@@ -254,14 +254,14 @@ static void find_and_assign_combinations_without_hwloc(int *workerids, int nwork
 	unsigned * nmics_table;
 	int * mic_id;
 	int ** mic_workers;
-	mic_id = malloc(sizeof(int)*nb_mics);
-	nmics_table = malloc(sizeof(unsigned)*nb_mics);
-	mic_workers = malloc(sizeof(int*)*nb_mics);
+	STARPU_MALLOC(mic_id, sizeof(int)*nb_mics);
+	STARPU_MALLOC(nmics_table, sizeof(unsigned)*nb_mics);
+	STARPU_MALLOC(mic_workers, sizeof(int*)*nb_mics);
 	for(j=0; j<nb_mics; j++)
 	{
 		mic_id[j] = -1;
 		nmics_table[j] = 0;
-		mic_workers[j] = malloc(sizeof(int)*STARPU_NMAXWORKERS);
+		STARPU_MALLOC(mic_workers[j], sizeof(int)*STARPU_NMAXWORKERS);
 	}
 #endif /* STARPU_USE_MIC */
 
@@ -279,7 +279,7 @@ static void find_and_assign_combinations_without_hwloc(int *workerids, int nwork
 			{
 				if(mic_id[j] == -1)
 				{
-					mic_id[j] = worker->devid;					
+					mic_id[j] = worker->devid;
 				}
 				mic_workers[j][nmics_table[j]++] = i;
 			}
@@ -295,7 +295,7 @@ static void find_and_assign_combinations_without_hwloc(int *workerids, int nwork
 	max = starpu_get_env_number("STARPU_MAX_WORKERSIZE");
 	if (max == -1 || max > (int) ncpus)
 		max = ncpus;
-	
+
 	assign_combinations_without_hwloc(workers,cpu_workers,ncpus,min,max);
 #ifdef STARPU_USE_MIC
 	mic_min = starpu_get_env_number("STARPU_MIN_WORKERSIZE");

+ 6 - 16
src/core/disk.c

@@ -1,7 +1,7 @@
 /* StarPU --- Runtime system for heterogeneous multicore architectures.
  *
  * Copyright (C) 2013  Corentin Salingue
- * Copyright (C) 2015  CNRS
+ * Copyright (C) 2015, 2016  CNRS
  *
  * 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
@@ -276,27 +276,17 @@ static void add_disk_in_list(unsigned node,  struct starpu_disk_ops *func, void
 	/* initialization */
 	if (disk_register_list == NULL)
 	{
-		disk_register_list = malloc(size_register_list*sizeof(struct disk_register *));
-		STARPU_ASSERT(disk_register_list != NULL);
+		STARPU_MALLOC(disk_register_list, size_register_list*sizeof(struct disk_register *));
 	}
 	/* small size -> new size  */
 	if ((disk_number+1) > size_register_list)
 	{
-		struct disk_register **ptr_realloc = realloc(disk_register_list, 2*size_register_list*sizeof(struct disk_register *));
-
-		if (ptr_realloc != NULL)
-		{
-			size_register_list *= 2;
-			disk_register_list = ptr_realloc;
-		}
-		else
-		{
-			STARPU_ASSERT(ptr_realloc != NULL);
-		}
+		size_register_list *= 2;
+		STARPU_REALLOC(disk_register_list, size_register_list*sizeof(struct disk_register *));
 	}
 
-	struct disk_register *dr = malloc(sizeof(struct disk_register));
-	STARPU_ASSERT(dr != NULL);
+	struct disk_register *dr;
+	STARPU_MALLOC(dr, sizeof(struct disk_register));
 	dr->node = node;
 	dr->base = base;
 	dr->flag = STARPU_DISK_ALL;

+ 13 - 13
src/core/disk_ops/disk_leveldb.cpp

@@ -50,13 +50,14 @@ struct starpu_leveldb_base
 static void *starpu_leveldb_alloc(void *base, size_t size STARPU_ATTRIBUTE_UNUSED)
 {
 	struct starpu_leveldb_base *base_tmp = (struct starpu_leveldb_base *) base;
-	struct starpu_leveldb_obj *obj = (struct starpu_leveldb_obj *) malloc(sizeof(struct starpu_leveldb_obj));
-	STARPU_ASSERT(obj != NULL);
+	struct starpu_leveldb_obj *obj;
+	STARPU_MALLOC(obj, sizeof(struct starpu_leveldb_obj));
 
         STARPU_PTHREAD_MUTEX_INIT(&obj->mutex, NULL);
 
 	size_t len = 6 + 1 + 2+sizeof(void*)*2 + 1;
-	char *key = (char *) malloc(len*sizeof(char));
+	char *key;
+	STARPU_MALLOC(key, len*sizeof(char));
 	snprintf(key, len, "STARPU-%p", obj);
 
 	/* create and add a key with a small memory */
@@ -87,8 +88,8 @@ static void starpu_leveldb_free(void *base , void *obj, size_t size STARPU_ATTRI
 /* open an existing memory on disk */
 static void *starpu_leveldb_open(void *base STARPU_ATTRIBUTE_UNUSED, void *pos, size_t size)
 {
-	struct starpu_leveldb_obj *obj = (struct starpu_leveldb_obj *) malloc(sizeof(struct starpu_leveldb_obj));
-	STARPU_ASSERT(obj != NULL);
+	struct starpu_leveldb_obj *obj;
+	STARPU_MALLOC(obj, sizeof(struct starpu_leveldb_obj));
 
         STARPU_PTHREAD_MUTEX_INIT(&obj->mutex, NULL);
 
@@ -148,7 +149,7 @@ static int starpu_leveldb_full_read(void *base, void *obj, void **ptr, size_t *s
 	STARPU_ASSERT(s.ok());
 
 	*size = value.length();
-	*ptr = malloc(*size);
+	STARPU_MALLOC(*ptr, *size);
 
 	/* use buffer */
 	memcpy(*ptr, value.c_str(), *size);
@@ -176,7 +177,7 @@ static int starpu_leveldb_write(void *base, void *obj, const void *buf, off_t of
 	else
 	{
 		uintptr_t buf_tmp = (uintptr_t) buf;
-		buffer = (void *) malloc((tmp->size > (offset + size)) ? tmp->size : (offset + size));
+		STARPU_MALLOC(buffer, (tmp->size > (offset + size)) ? tmp->size : (offset + size));
 
 		/* we read the data */
 		std::string value;
@@ -223,8 +224,8 @@ static int starpu_leveldb_full_write(void *base, void *obj, void *ptr, size_t si
 /* create a new copy of parameter == base */
 static void *starpu_leveldb_plug(void *parameter, starpu_ssize_t size STARPU_ATTRIBUTE_UNUSED)
 {
-	struct starpu_leveldb_base *tmp = (struct starpu_leveldb_base *) malloc(sizeof(struct starpu_leveldb_base));
-	STARPU_ASSERT(tmp != NULL);
+	struct starpu_leveldb_base *tmp;
+	STARPU_MALLOC(tmp, sizeof(struct starpu_leveldb_base));
 
 	leveldb::Status status;
 	leveldb::DB *db;
@@ -267,8 +268,8 @@ static int get_leveldb_bandwidth_between_disk_and_main_ram(unsigned node)
 	double end;
 
 	srand(time (NULL));
-	char *buf = (char *) malloc(SIZE_DISK_MIN*sizeof(char));
-	STARPU_ASSERT(buf != NULL);
+	char *buf;
+	STARPU_MALLOC(buf, SIZE_DISK_MIN*sizeof(char));
 
 	/* allocate memory */
 	void *mem = _starpu_disk_alloc(node, SIZE_DISK_MIN);
@@ -292,8 +293,7 @@ static int get_leveldb_bandwidth_between_disk_and_main_ram(unsigned node)
 	/* free memory */
 	free(buf);
 
-	buf = (char *) malloc(sizeof(char));
-	STARPU_ASSERT(buf != NULL);
+	STARPU_MALLOC(buf, sizeof(char));
 
 	/* Measure latency */
 	start = starpu_timing_now();

+ 6 - 6
src/core/disk_ops/disk_stdio.c

@@ -57,8 +57,8 @@ struct starpu_stdio_obj
 
 static struct starpu_stdio_obj *_starpu_stdio_init(int descriptor, char *path, size_t size)
 {
-	struct starpu_stdio_obj *obj = malloc(sizeof(struct starpu_stdio_obj));
-	STARPU_ASSERT(obj != NULL);
+	struct starpu_stdio_obj *obj;
+	STARPU_MALLOC(obj, sizeof(struct starpu_stdio_obj));
 
 	FILE *f = fdopen(descriptor,"rb+");
 	if (f == NULL)
@@ -173,8 +173,8 @@ static void *starpu_stdio_open(void *base, void *pos, size_t size)
 {
 	struct starpu_stdio_obj *obj;
 	/* create template */
-	char *baseCpy = malloc(strlen(base)+1+strlen(pos)+1);
-	STARPU_ASSERT(baseCpy != NULL);
+	char *baseCpy;
+	STARPU_MALLOC(baseCpy, strlen(base)+1+strlen(pos)+1);
 	strcpy(baseCpy,(char *) base);
 	strcat(baseCpy,(char *) "/");
 	strcat(baseCpy,(char *) pos);
@@ -319,8 +319,8 @@ static int starpu_stdio_full_write(void *base STARPU_ATTRIBUTE_UNUSED, void *obj
 /* create a new copy of parameter == base */
 static void *starpu_stdio_plug(void *parameter, starpu_ssize_t size STARPU_ATTRIBUTE_UNUSED)
 {
-	char *tmp = malloc(sizeof(char)*(strlen(parameter)+1));
-	STARPU_ASSERT(tmp != NULL);
+	char *tmp;
+	STARPU_MALLOC(tmp, sizeof(char)*(strlen(parameter)+1));
 	strcpy(tmp,(char *) parameter);
 
 	{

+ 4 - 4
src/core/disk_ops/disk_unistd.c

@@ -33,8 +33,8 @@
 /* allocation memory on disk */
 static void *starpu_unistd_alloc(void *base, size_t size)
 {
-        struct starpu_unistd_global_obj *obj = malloc(sizeof(struct starpu_unistd_global_obj));
-        STARPU_ASSERT(obj != NULL);
+        struct starpu_unistd_global_obj *obj;
+	STARPU_MALLOC(obj, sizeof(struct starpu_unistd_global_obj));
 	/* only flags change between unistd and unistd_o_direct */
 	obj->flags = O_RDWR | O_BINARY;
 	return starpu_unistd_global_alloc(obj, base, size);
@@ -43,8 +43,8 @@ static void *starpu_unistd_alloc(void *base, size_t size)
 /* open an existing memory on disk */
 static void *starpu_unistd_open(void *base, void *pos, size_t size)
 {
-	struct starpu_unistd_global_obj *obj = malloc(sizeof(struct starpu_unistd_global_obj));
-	STARPU_ASSERT(obj != NULL);
+	struct starpu_unistd_global_obj *obj;
+	STARPU_MALLOC(obj, sizeof(struct starpu_unistd_global_obj));
 	/* only flags change between unistd and unistd_o_direct */
 	obj->flags = O_RDWR | O_BINARY;
 	return starpu_unistd_global_open(obj, base, pos, size);

+ 4 - 4
src/core/disk_ops/disk_unistd_o_direct.c

@@ -33,8 +33,8 @@
 /* allocation memory on disk */
 static void *starpu_unistd_o_direct_alloc(void *base, size_t size)
 {
-        struct starpu_unistd_global_obj *obj = malloc(sizeof(struct starpu_unistd_global_obj));
-        STARPU_ASSERT(obj != NULL);
+        struct starpu_unistd_global_obj *obj;
+	STARPU_MALLOC(obj, sizeof(struct starpu_unistd_global_obj));
         /* only flags change between unistd and unistd_o_direct */
         obj->flags = O_RDWR | O_DIRECT | O_BINARY;
         return starpu_unistd_global_alloc (obj, base, size);
@@ -43,8 +43,8 @@ static void *starpu_unistd_o_direct_alloc(void *base, size_t size)
 /* open an existing memory on disk */
 static void *starpu_unistd_o_direct_open(void *base, void *pos, size_t size)
 {
-        struct starpu_unistd_global_obj * obj = malloc(sizeof(struct starpu_unistd_global_obj));
-        STARPU_ASSERT(obj != NULL);
+        struct starpu_unistd_global_obj *obj;
+	STARPU_MALLOC(obj, sizeof(struct starpu_unistd_global_obj));
         /* only flags change between unistd and unistd_o_direct */
         obj->flags = O_RDWR | O_DIRECT | O_BINARY;
         return starpu_unistd_global_open (obj, base, pos, size);

+ 8 - 6
src/core/disk_ops/unistd/disk_unistd_global.c

@@ -159,8 +159,8 @@ void starpu_unistd_global_free(void *base STARPU_ATTRIBUTE_UNUSED, void *obj, si
 void *starpu_unistd_global_open(struct starpu_unistd_global_obj *obj, void *base, void *pos, size_t size)
 {
 	/* create template */
-	char *baseCpy = malloc(strlen(base)+1+strlen(pos)+1);
-	STARPU_ASSERT(baseCpy != NULL);
+	char *baseCpy;
+	STARPU_MALLOC(baseCpy, strlen(base)+1+strlen(pos)+1);
 	strcpy(baseCpy,(char *) base);
 	strcat(baseCpy,(char *) "/");
 	strcat(baseCpy,(char *) pos);
@@ -226,7 +226,8 @@ int starpu_unistd_global_read(void *base STARPU_ATTRIBUTE_UNUSED, void *obj, voi
 void *starpu_unistd_global_async_read(void *base STARPU_ATTRIBUTE_UNUSED, void *obj, void *buf, off_t offset, size_t size)
 {
         struct starpu_unistd_global_obj *tmp = obj;
-        struct starpu_unistd_aiocb *starpu_aiocb = calloc(1,sizeof(*starpu_aiocb));
+        struct starpu_unistd_aiocb *starpu_aiocb;
+	STARPU_CALLOC(starpu_aiocb, 1,sizeof(*starpu_aiocb));
         struct aiocb *aiocb = &starpu_aiocb->aiocb;
         starpu_aiocb->obj = obj;
         int fd = tmp->descriptor;
@@ -315,7 +316,8 @@ int starpu_unistd_global_write(void *base STARPU_ATTRIBUTE_UNUSED, void *obj, co
 void *starpu_unistd_global_async_write(void *base STARPU_ATTRIBUTE_UNUSED, void *obj, void *buf, off_t offset, size_t size)
 {
         struct starpu_unistd_global_obj *tmp = obj;
-        struct starpu_unistd_aiocb *starpu_aiocb = calloc(1,sizeof(*starpu_aiocb));
+        struct starpu_unistd_aiocb *starpu_aiocb;
+	STARPU_CALLOC(starpu_aiocb, 1,sizeof(*starpu_aiocb));
         struct aiocb *aiocb = &starpu_aiocb->aiocb;
         starpu_aiocb->obj = obj;
         int fd = tmp->descriptor;
@@ -366,8 +368,8 @@ int starpu_unistd_global_full_write(void *base STARPU_ATTRIBUTE_UNUSED, void *ob
 /* create a new copy of parameter == base */
 void *starpu_unistd_global_plug(void *parameter, starpu_ssize_t size STARPU_ATTRIBUTE_UNUSED)
 {
-	char *tmp = malloc(sizeof(char)*(strlen(parameter)+1));
-	STARPU_ASSERT(tmp != NULL);
+	char *tmp;
+	STARPU_MALLOC(tmp, sizeof(char)*(strlen(parameter)+1));
 	strcpy(tmp,(char *) parameter);
 
 	{

+ 5 - 5
src/core/jobs.c

@@ -1,7 +1,7 @@
 /* StarPU --- Runtime system for heterogeneous multicore architectures.
  *
  * Copyright (C) 2009-2016  Université de Bordeaux
- * Copyright (C) 2010, 2011, 2012, 2013, 2014, 2015  CNRS
+ * Copyright (C) 2010, 2011, 2012, 2013, 2014, 2015, 2016  CNRS
  * Copyright (C) 2011  Télécom-SudParis
  * Copyright (C) 2011, 2014, 2016  INRIA
  *
@@ -72,7 +72,7 @@ struct _starpu_job* STARPU_ATTRIBUTE_MALLOC _starpu_job_create(struct starpu_tas
 	struct _starpu_job *job;
         _STARPU_LOG_IN();
 
-	job = malloc(sizeof(*job));
+	STARPU_MALLOC(job, sizeof(*job));
 
 	/* As most of the fields must be initialized at NULL, let's put 0
 	 * everywhere */
@@ -80,8 +80,8 @@ struct _starpu_job* STARPU_ATTRIBUTE_MALLOC _starpu_job_create(struct starpu_tas
 
 	if (task->dyn_handles)
 	{
-	     job->dyn_ordered_buffers = malloc(STARPU_TASK_GET_NBUFFERS(task) * sizeof(job->dyn_ordered_buffers[0]));
-	     job->dyn_dep_slots = calloc(STARPU_TASK_GET_NBUFFERS(task), sizeof(job->dyn_dep_slots[0]));
+		STARPU_MALLOC(job->dyn_ordered_buffers, STARPU_TASK_GET_NBUFFERS(task) * sizeof(job->dyn_ordered_buffers[0]));
+		STARPU_CALLOC(job->dyn_dep_slots, STARPU_TASK_GET_NBUFFERS(task), sizeof(job->dyn_dep_slots[0]));
 	}
 
 	job->task = task;
@@ -727,7 +727,7 @@ int _starpu_push_local_task(struct _starpu_worker *worker, struct starpu_task *t
 				alloc = 1;
 			while (alloc < needed)
 				alloc *= 2;
-			new = malloc(alloc * sizeof(*new));
+			STARPU_MALLOC(new, alloc * sizeof(*new));
 
 			if (worker->local_ordered_tasks_size)
 			{

+ 2 - 2
src/core/parallel_task.c

@@ -24,8 +24,8 @@
 
 struct starpu_task *starpu_task_dup(struct starpu_task *task)
 {
-	struct starpu_task *task_dup = (struct starpu_task *) malloc(sizeof(struct starpu_task));
-	STARPU_ASSERT(task_dup);
+	struct starpu_task *task_dup;
+	STARPU_MALLOC(task_dup, sizeof(struct starpu_task));
 
 	/* TODO perhaps this is a bit too much overhead and we should only copy
 	 * part of the structure ? */

+ 26 - 23
src/core/perfmodel/multiple_regression.c

@@ -1,7 +1,7 @@
 /* StarPU --- Runtime system for heterogeneous multicore architectures.
  *
  * Copyright (C) 2009, 2010, 2011, 2015-2016  Université de Bordeaux
- * Copyright (C) 2010, 2011  CNRS
+ * Copyright (C) 2010, 2011, 2016  CNRS
  * Copyright (C) 2016  Inria
  *
  * StarPU is free software; you can redistribute it and/or modify
@@ -104,7 +104,7 @@ static long find_long_list_size(struct starpu_perfmodel_history_list *list_histo
 
 #ifdef STARPU_MLR_MODEL
 int dgels_multiple_reg_coeff(double *mpar, double *my, long nn, unsigned ncoeff, unsigned nparameters, double *coeff, unsigned **combinations)
-{	
+{
  /*  Arguments */
 /*  ========= */
 
@@ -185,13 +185,15 @@ int dgels_multiple_reg_coeff(double *mpar, double *my, long nn, unsigned ncoeff,
 		_STARPU_DISP("Warning: This function is not intended for the use when number of parameters is larger than the number of observations. Check how your matrices A and B were allocated or simply add more benchmarks.\n Multiple linear regression model will not be written into perfmodel file.\n");
 		return 1;
 	}
-	
+
 	char trans = 'N';
 	integer m = nn;
 	integer n = ncoeff;
 	integer nrhs = 1; // number of columns of B and X (wich are vectors therefore nrhs=1)
-	doublereal *X = malloc(sizeof(double)*n*m); // (/!\ modified at the output) contain the model and the different values of pararmters
-	doublereal *Y = malloc(sizeof(double)*m);
+	doublereal *X;
+	STARPU_MALLOC(X, sizeof(double)*n*m); // (/!\ modified at the output) contain the model and the different values of pararmters
+	doublereal *Y;
+	STARPU_MALLOC(Y, sizeof(double)*m);
 
 	double coefficient;
 	int i;
@@ -206,17 +208,18 @@ int dgels_multiple_reg_coeff(double *mpar, double *my, long nn, unsigned ncoeff,
 			for(k=0; k < nparameters; k++)
 			{
 				coefficient *= pow(mpar[i*nparameters+k],combinations[j-1][k]);
-			}			
+			}
 			X[i*n+j] = coefficient;
 		}
 	}
 
-	integer lda = m; 
+	integer lda = m;
 	integer ldb = m; //
 	integer info = 0;
 
 	integer lwork = n*2;
-	doublereal *work = malloc(sizeof(double)*lwork); // (output)
+	doublereal *work; // (output)
+	STARPU_MALLOC(work, sizeof(double)*lwork);
 
 	/* // Running LAPACK dgels_ */
 	dgels_(&trans, &m, &n, &nrhs, X, &lda, Y, &ldb, work, &lwork, &info);
@@ -249,38 +252,38 @@ void validate(double *coeff, unsigned ncoeff, const char *codelet_name)
 	unsigned i;
 	if (coeff[0] < 0)
 		_STARPU_DISP("Warning: Constant computed by least square method is negative (%f). The model %s is likely to be inaccurate.\n", coeff[0], codelet_name);
-		
+
 	for(i=1; i<ncoeff; i++)
 		if(fabs(coeff[i]) < 1E-10)
 			_STARPU_DISP("Warning: Coefficient computed by least square method is extremelly small (%f). The model %s is likely to be inaccurate.\n", coeff[i], codelet_name);
 }
-	
+
 int _starpu_multiple_regression(struct starpu_perfmodel_history_list *ptr, double *coeff, unsigned ncoeff, unsigned nparameters, const char **parameters_names, unsigned **combinations, const char *codelet_name)
 {
         long i;
 	unsigned j;
-	
+
 	/* Computing number of rows */
 	long n=find_long_list_size(ptr);
 	STARPU_ASSERT(n);
-	
+
         /* Reading old calibrations if necessary */
 	FILE *f=NULL;
-	
+
 	char directory[300];
 	snprintf(directory, 300, "%s/.starpu/sampling/codelets/tmp", _starpu_get_home_path());
 	_starpu_mkpath_and_check(directory, S_IRWXU);
-	
+
 	char filepath[300];
 	snprintf(filepath, 300, "%s/%s.out", directory,codelet_name);
-	
+
 	long old_lines=0;
 	int calibrate = _starpu_get_calibrate_flag();
 	if (calibrate==1)
 	{
 		f = fopen(filepath, "a+");
 		STARPU_ASSERT_MSG(f, "Could not save performance model into the file %s\n", filepath);
-		
+
 		old_lines=count_file_lines(f);
 		/* If the program is run for the first time the old_lines will be 0 */
 		//STARPU_ASSERT(old_lines);
@@ -291,10 +294,10 @@ int _starpu_multiple_regression(struct starpu_perfmodel_history_list *ptr, doubl
 	}
 
 	/* Allocating X and Y matrices */
-	double *mpar = (double *) malloc(nparameters*n*sizeof(double));
-	STARPU_ASSERT(mpar);
-	double *my = (double *) malloc(n*sizeof(double));
-	STARPU_ASSERT(my);
+	double *mpar;
+	STARPU_MALLOC(mpar, nparameters*n*sizeof(double));
+	double *my;
+	STARPU_MALLOC(my, n*sizeof(double));
 
 	/* Loading old calibration */
 	if (calibrate==1)
@@ -310,7 +313,7 @@ int _starpu_multiple_regression(struct starpu_perfmodel_history_list *ptr, doubl
 		if(dgels_multiple_reg_coeff(mpar, my, n, ncoeff, nparameters, coeff, combinations))
 		{
 			free(mpar);
-			free(my);	
+			free(my);
 			return 1;
 		}
 		/* Basic validation of the model accuracy */
@@ -321,7 +324,7 @@ int _starpu_multiple_regression(struct starpu_perfmodel_history_list *ptr, doubl
 			coeff[i] = 0.;
 #endif //STARPU_MLR_MODEL
 	}
-	
+
 	/* Preparing new output calibration file */
 	if (calibrate==1 || calibrate==2)
 	{
@@ -344,7 +347,7 @@ int _starpu_multiple_regression(struct starpu_perfmodel_history_list *ptr, doubl
 			}
 		}
 	}
-	
+
 	/* Writing parameters to calibration file */
 	if (calibrate==1 || calibrate==2)
 	{

+ 4 - 4
src/core/perfmodel/perfmodel.c

@@ -452,10 +452,10 @@ static char *_perf_model_dir_debug = NULL;
 
 void _starpu_set_perf_model_dirs()
 {
-	_perf_model_dir = malloc(_PERF_MODEL_DIR_MAXLEN);
-	_perf_model_dir_codelet = malloc(_PERF_MODEL_DIR_MAXLEN);
-	_perf_model_dir_bus = malloc(_PERF_MODEL_DIR_MAXLEN);
-	_perf_model_dir_debug = malloc(_PERF_MODEL_DIR_MAXLEN);
+	STARPU_MALLOC(_perf_model_dir, _PERF_MODEL_DIR_MAXLEN);
+	STARPU_MALLOC(_perf_model_dir_codelet, _PERF_MODEL_DIR_MAXLEN);
+	STARPU_MALLOC(_perf_model_dir_bus, _PERF_MODEL_DIR_MAXLEN);
+	STARPU_MALLOC(_perf_model_dir_debug, _PERF_MODEL_DIR_MAXLEN);
 
 #ifdef STARPU_PERF_MODEL_DIR
 	/* use the directory specified at configure time */

+ 8 - 17
src/core/perfmodel/perfmodel_bus.c

@@ -391,8 +391,7 @@ static void measure_bandwidth_between_host_and_dev_on_cpu_with_opencl(int dev, i
 	_starpu_bind_thread_on_cpu(config, cpu, STARPU_NOWORKERID);
         /* Allocate a buffer on the host */
 	unsigned char *h_buffer;
-        h_buffer = (unsigned char *)malloc(size);
-	STARPU_ASSERT(h_buffer);
+	STARPU_MALLOC(h_buffer, size);
 
 	/* hack to avoid third party libs to rebind threads */
 	_starpu_bind_thread_on_cpu(config, cpu, STARPU_NOWORKERID);
@@ -535,20 +534,11 @@ static void measure_bandwidth_between_cpus_and_dev(int dev, struct dev_timing *d
 
 	if (!no_node_obj_was_found)
 	{
-		is_available_per_numa_node = (unsigned *)malloc(nnuma_nodes * sizeof(unsigned));
-		STARPU_ASSERT(is_available_per_numa_node);
-
-		dev_timing_htod_per_numa_node = (double *)malloc(nnuma_nodes * sizeof(double));
-		STARPU_ASSERT(dev_timing_htod_per_numa_node);
-		dev_latency_htod_per_numa_node = (double *)malloc(nnuma_nodes * sizeof(double));
-		STARPU_ASSERT(dev_latency_htod_per_numa_node);
-
-		dev_timing_dtoh_per_numa_node = (double *)malloc(nnuma_nodes * sizeof(double));
-		STARPU_ASSERT(dev_timing_dtoh_per_numa_node);
-		dev_latency_dtoh_per_numa_node = (double *)malloc(nnuma_nodes * sizeof(double));
-		STARPU_ASSERT(dev_latency_dtoh_per_numa_node);
-
-		memset(is_available_per_numa_node, 0, nnuma_nodes*sizeof(unsigned));
+		STARPU_CALLOC(is_available_per_numa_node, nnuma_nodes, sizeof(unsigned));
+		STARPU_MALLOC(dev_timing_htod_per_numa_node, nnuma_nodes * sizeof(double));
+		STARPU_MALLOC(dev_latency_htod_per_numa_node, nnuma_nodes * sizeof(double));
+		STARPU_MALLOC(dev_timing_dtoh_per_numa_node, nnuma_nodes * sizeof(double));
+		STARPU_MALLOC(dev_latency_dtoh_per_numa_node, nnuma_nodes * sizeof(double));
 	}
 #endif
 
@@ -1751,7 +1741,8 @@ static void allocate_userdata(hwloc_obj_t obj)
 	if (obj->userdata)
 		return;
 
-	data = obj->userdata = malloc(sizeof(*data));
+	STARPU_MALLOC(obj->userdata, sizeof(*data));
+	data = obj->userdata;
 	data->bw_up = 0.0;
 	data->bw_down = 0.0;
 	data->bw = 0.0;

+ 33 - 37
src/core/perfmodel/perfmodel_history.c

@@ -73,7 +73,7 @@ void _starpu_perfmodel_malloc_per_arch(struct starpu_perfmodel *model, int comb,
 {
 	int i;
 
-	model->state->per_arch[comb] = (struct starpu_perfmodel_per_arch*)malloc(nb_impl*sizeof(struct starpu_perfmodel_per_arch));
+	STARPU_MALLOC(model->state->per_arch[comb], nb_impl*sizeof(struct starpu_perfmodel_per_arch));
 	for(i = 0; i < nb_impl; i++)
 	{
 		memset(&model->state->per_arch[comb][i], 0, sizeof(struct starpu_perfmodel_per_arch));
@@ -85,7 +85,7 @@ void _starpu_perfmodel_malloc_per_arch_is_set(struct starpu_perfmodel *model, in
 {
 	int i;
 
-	model->state->per_arch_is_set[comb] = (int*)malloc(nb_impl*sizeof(int));
+	STARPU_MALLOC(model->state->per_arch_is_set[comb], nb_impl*sizeof(int));
 	for(i = 0; i < nb_impl; i++)
 	{
 		model->state->per_arch_is_set[comb][i] = 0;
@@ -145,10 +145,10 @@ int starpu_perfmodel_arch_comb_add(int ndevices, struct starpu_perfmodel_device*
 	{
 		// We need to allocate more arch_combs
 		nb_arch_combs = current_arch_comb+10;
-		arch_combs = (struct starpu_perfmodel_arch**) realloc(arch_combs, nb_arch_combs*sizeof(struct starpu_perfmodel_arch*));
+		STARPU_REALLOC(arch_combs, nb_arch_combs*sizeof(struct starpu_perfmodel_arch*));
 	}
-	arch_combs[current_arch_comb] = (struct starpu_perfmodel_arch*)malloc(sizeof(struct starpu_perfmodel_arch));
-	arch_combs[current_arch_comb]->devices = (struct starpu_perfmodel_device*)malloc(ndevices*sizeof(struct starpu_perfmodel_device));
+	STARPU_MALLOC(arch_combs[current_arch_comb], sizeof(struct starpu_perfmodel_arch));
+	STARPU_MALLOC(arch_combs[current_arch_comb]->devices, ndevices*sizeof(struct starpu_perfmodel_device));
 	arch_combs[current_arch_comb]->ndevices = ndevices;
 	int dev;
 	for(dev = 0; dev < ndevices; dev++)
@@ -223,7 +223,7 @@ static void insert_history_entry(struct starpu_perfmodel_history_entry *entry, s
 	struct starpu_perfmodel_history_list *link;
 	struct starpu_perfmodel_history_table *table;
 
-	link = (struct starpu_perfmodel_history_list *) malloc(sizeof(struct starpu_perfmodel_history_list));
+	STARPU_MALLOC(link, sizeof(struct starpu_perfmodel_history_list));
 	link->next = *list;
 	link->entry = entry;
 	*list = link;
@@ -232,8 +232,7 @@ static void insert_history_entry(struct starpu_perfmodel_history_entry *entry, s
 	//HASH_FIND_UINT32_T(*history_ptr, &entry->footprint, table);
 	//STARPU_ASSERT(table == NULL);
 
-	table = (struct starpu_perfmodel_history_table*) malloc(sizeof(*table));
-	STARPU_ASSERT(table != NULL);
+	STARPU_MALLOC(table, sizeof(*table));
 	table->footprint = entry->footprint;
 	table->history_entry = entry;
 	HASH_ADD_UINT32_T(*history_ptr, footprint, table);
@@ -298,7 +297,7 @@ static void dump_reg_model(FILE *f, struct starpu_perfmodel *model, int comb, in
 			reg_model->ncoeff = model->ncombinations + 1;
 		}
 
-		reg_model->coeff = (double *) malloc(reg_model->ncoeff*sizeof(double));
+		STARPU_MALLOC(reg_model->coeff,  reg_model->ncoeff*sizeof(double));
 		_starpu_multiple_regression(per_arch_model->list, reg_model->coeff, reg_model->ncoeff, model->nparameters, model->parameters_names, model->combinations, model->symbol);
 
 		fprintf(f, "# n\tintercept\t");
@@ -323,12 +322,12 @@ static void dump_reg_model(FILE *f, struct starpu_perfmodel *model, int comb, in
 								first=0;
 							else
 								fprintf(f, "*");
-						
+
 							if(model->parameters_names[j] != NULL)
 								fprintf(f, "%s", model->parameters_names[j]);
 							else
 								fprintf(f, "P%d", j);
-						
+
 							if (model->combinations[i][j] > 1)
 								fprintf(f, "^%d", model->combinations[i][j]);
 						}
@@ -398,7 +397,7 @@ static void scan_reg_model(FILE *f, struct starpu_perfmodel_regression_model *re
 		res = fscanf(f, "%u", &reg_model->ncoeff);
 		STARPU_ASSERT_MSG(res == 1, "Incorrect performance model file");
 
-		reg_model->coeff = malloc(reg_model->ncoeff*sizeof(double));
+		STARPU_MALLOC(reg_model->coeff, reg_model->ncoeff*sizeof(double));
 
 		unsigned multi_invalid = 0;
 		unsigned i;
@@ -485,8 +484,7 @@ static void parse_per_arch_model_file(FILE *f, struct starpu_perfmodel_per_arch
 		struct starpu_perfmodel_history_entry *entry = NULL;
 		if (scan_history)
 		{
-			entry = (struct starpu_perfmodel_history_entry *) malloc(sizeof(struct starpu_perfmodel_history_entry));
-			STARPU_ASSERT(entry);
+			STARPU_MALLOC(entry, sizeof(struct starpu_perfmodel_history_entry));
 
 			/* Tell  helgrind that we do not care about
 			 * racing access to the sampling, we only want a
@@ -749,11 +747,11 @@ void _starpu_perfmodel_realloc(struct starpu_perfmodel *model, int nb)
 #ifdef SSIZE_MAX
 	STARPU_ASSERT((size_t) nb < SSIZE_MAX / sizeof(struct starpu_perfmodel_per_arch*));
 #endif
-	model->state->per_arch = (struct starpu_perfmodel_per_arch**) realloc(model->state->per_arch, nb*sizeof(struct starpu_perfmodel_per_arch*));
-	model->state->per_arch_is_set = (int**) realloc(model->state->per_arch_is_set, nb*sizeof(int*));
-	model->state->nimpls = (int *)realloc(model->state->nimpls, nb*sizeof(int));
-	model->state->nimpls_set = (int *)realloc(model->state->nimpls_set, nb*sizeof(int));
-	model->state->combs = (int*)realloc(model->state->combs, nb*sizeof(int));
+	STARPU_REALLOC(model->state->per_arch, nb*sizeof(struct starpu_perfmodel_per_arch*));
+	STARPU_REALLOC(model->state->per_arch_is_set, nb*sizeof(int*));
+	STARPU_REALLOC(model->state->nimpls, nb*sizeof(int));
+	STARPU_REALLOC(model->state->nimpls_set, nb*sizeof(int));
+	STARPU_REALLOC(model->state->combs, nb*sizeof(int));
 	for(i = model->state->ncombs_set; i < nb; i++)
 	{
 		model->state->per_arch[i] = NULL;
@@ -789,17 +787,17 @@ void starpu_perfmodel_init(struct starpu_perfmodel *model)
 		return;
 	}
 
-	model->state = malloc(sizeof(struct _starpu_perfmodel_state));
+	STARPU_MALLOC(model->state, sizeof(struct _starpu_perfmodel_state));
 	STARPU_PTHREAD_RWLOCK_INIT(&model->state->model_rwlock, NULL);
 
 	STARPU_PTHREAD_RWLOCK_RDLOCK(&arch_combs_mutex);
 	model->state->ncombs_set = ncombs = nb_arch_combs;
 	STARPU_PTHREAD_RWLOCK_UNLOCK(&arch_combs_mutex);
-	model->state->per_arch = (struct starpu_perfmodel_per_arch**) malloc(ncombs*sizeof(struct starpu_perfmodel_per_arch*));
-	model->state->per_arch_is_set = (int**) malloc(ncombs*sizeof(int*));
-	model->state->nimpls = (int *)malloc(ncombs*sizeof(int));
-	model->state->nimpls_set = (int *)malloc(ncombs*sizeof(int));
-	model->state->combs = (int*)malloc(ncombs*sizeof(int));
+	STARPU_MALLOC(model->state->per_arch, ncombs*sizeof(struct starpu_perfmodel_per_arch*));
+	STARPU_MALLOC(model->state->per_arch_is_set, ncombs*sizeof(int*));
+	STARPU_MALLOC(model->state->nimpls, ncombs*sizeof(int));
+	STARPU_MALLOC(model->state->nimpls_set, ncombs*sizeof(int));
+	STARPU_MALLOC(model->state->combs, ncombs*sizeof(int));
 	model->state->ncombs = 0;
 
 	for(i = 0; i < ncombs; i++)
@@ -811,7 +809,8 @@ void starpu_perfmodel_init(struct starpu_perfmodel *model)
 	}
 
 	/* add the model to a linked list */
-	struct _starpu_perfmodel_list *node = (struct _starpu_perfmodel_list *) malloc(sizeof(struct _starpu_perfmodel_list));
+	struct _starpu_perfmodel_list *node;
+	STARPU_MALLOC(node, sizeof(struct _starpu_perfmodel_list));
 
 	node->model = model;
 	//model->debug_modelid = debug_modelid++;
@@ -916,7 +915,7 @@ void _starpu_initialize_registered_performance_models(void)
 	// We used to allocate 2**(ncores + ncuda + nopencl + nmic + nscc), this is too big
 	// We now allocate only 2*(ncores + ncuda + nopencl + nmic + nscc), and reallocate when necessary in starpu_perfmodel_arch_comb_add
 	nb_arch_combs = 2 * (ncores + ncuda + nopencl + nmic + nscc);
-	arch_combs = (struct starpu_perfmodel_arch**) malloc(nb_arch_combs*sizeof(struct starpu_perfmodel_arch*));
+	STARPU_MALLOC(arch_combs, nb_arch_combs*sizeof(struct starpu_perfmodel_arch*));
 	current_arch_comb = 0;
 	STARPU_PTHREAD_RWLOCK_INIT(&arch_combs_mutex, NULL);
 	historymaxerror = starpu_get_env_number_default("STARPU_HISTORY_MAX_ERROR", STARPU_HISTORYMAXERROR);
@@ -1340,10 +1339,10 @@ double _starpu_multiple_regression_based_job_expected_perf(struct starpu_perfmod
 	reg_model = &model->state->per_arch[comb][nimpl].regression;
 	if (reg_model->coeff == NULL)
 		goto docal;
-	
+
 	double parameter_value;
 	double *parameters;
-	parameters = (double *) malloc(model->nparameters*sizeof(double));
+	STARPU_MALLOC(parameters, model->nparameters*sizeof(double));
 	model->parameters(j->task, parameters);
 	expected_duration=reg_model->coeff[0];
 	unsigned i, k;
@@ -1503,8 +1502,7 @@ void _starpu_update_perfmodel_history(struct _starpu_job *j, struct starpu_perfm
 			if (!entry)
 			{
 				/* this is the first entry with such a footprint */
-				entry = (struct starpu_perfmodel_history_entry *) malloc(sizeof(struct starpu_perfmodel_history_entry));
-				STARPU_ASSERT(entry);
+				STARPU_MALLOC(entry, sizeof(struct starpu_perfmodel_history_entry));
 
 				/* Tell  helgrind that we do not care about
 				 * racing access to the sampling, we only want a
@@ -1619,16 +1617,14 @@ void _starpu_update_perfmodel_history(struct _starpu_job *j, struct starpu_perfm
 			struct starpu_perfmodel_history_list **list;
 			list = &per_arch_model->list;
 
-			entry = (struct starpu_perfmodel_history_entry *) calloc(1, sizeof(struct starpu_perfmodel_history_entry));
-			STARPU_ASSERT(entry);
-
-			entry->parameters = (double *) malloc(model->nparameters*sizeof(double));
+			STARPU_CALLOC(entry, 1, sizeof(struct starpu_perfmodel_history_entry));
+			STARPU_MALLOC(entry->parameters, model->nparameters*sizeof(double));
 			model->parameters(j->task, entry->parameters);
 			entry->tag = j->task->tag_id;
 			entry->duration = measured;
 
 			struct starpu_perfmodel_history_list *link;
-			link = (struct starpu_perfmodel_history_list *) malloc(sizeof(struct starpu_perfmodel_history_list));
+			STARPU_MALLOC(link, sizeof(struct starpu_perfmodel_history_list));
 			link->next = *list;
 			link->entry = entry;
 			*list = link;
@@ -1748,7 +1744,7 @@ struct starpu_perfmodel_per_arch *_starpu_perfmodel_get_model_per_devices(struct
 	va_end(varg_list_copy);
 
 	// We set the devices
-	arch.devices = (struct starpu_perfmodel_device*)malloc(arch.ndevices * sizeof(struct starpu_perfmodel_device));
+	STARPU_MALLOC(arch.devices, arch.ndevices * sizeof(struct starpu_perfmodel_device));
 	va_copy(varg_list_copy, varg_list);
 	for(i=0 ; i<arch.ndevices ; i++)
 	{

+ 5 - 5
src/core/perfmodel/perfmodel_print.c

@@ -1,7 +1,7 @@
 /* StarPU --- Runtime system for heterogeneous multicore architectures.
  *
  * Copyright (C) 2011, 2013-2016  Université de Bordeaux
- * Copyright (C) 2011, 2012, 2013, 2014, 2015  CNRS
+ * Copyright (C) 2011, 2012, 2013, 2014, 2015, 2016  CNRS
  * Copyright (C) 2011  Télécom-SudParis
  *
  * StarPU is free software; you can redistribute it and/or modify
@@ -190,7 +190,7 @@ int starpu_perfmodel_print_all(struct starpu_perfmodel *model, char *arch, char
 			int implid;
 			struct starpu_perfmodel_arch perf_arch;
 			perf_arch.ndevices = 1;
-			perf_arch.devices = (struct starpu_perfmodel_device*)malloc(sizeof(struct starpu_perfmodel_device));
+			STARPU_MALLOC(perf_arch.devices, sizeof(struct starpu_perfmodel_device));
 			perf_arch.devices[0].type = STARPU_CPU_WORKER;
 			perf_arch.devices[0].devid = 0;
 			perf_arch.devices[0].ncores = 1;
@@ -216,7 +216,7 @@ int starpu_perfmodel_print_all(struct starpu_perfmodel *model, char *arch, char
 			int implid;
 			struct starpu_perfmodel_arch perf_arch;
 			perf_arch.ndevices = 1;
-			perf_arch.devices = (struct starpu_perfmodel_device*)malloc(sizeof(struct starpu_perfmodel_device));
+			STARPU_MALLOC(perf_arch.devices, sizeof(struct starpu_perfmodel_device));
 			perf_arch.devices[0].type = STARPU_CPU_WORKER;
 			perf_arch.devices[0].devid = 0;
 			perf_arch.devices[0].ncores = k;
@@ -236,7 +236,7 @@ int starpu_perfmodel_print_all(struct starpu_perfmodel *model, char *arch, char
 			struct starpu_perfmodel_arch perf_arch;
 
 			perf_arch.ndevices = 1;
-			perf_arch.devices = (struct starpu_perfmodel_device*)malloc(sizeof(struct starpu_perfmodel_device));
+			STARPU_MALLOC(perf_arch.devices, sizeof(struct starpu_perfmodel_device));
 			perf_arch.devices[0].type = STARPU_CUDA_WORKER;
 			perf_arch.devices[0].ncores = 1;
 			int comb;
@@ -266,7 +266,7 @@ int starpu_perfmodel_print_all(struct starpu_perfmodel *model, char *arch, char
 		{
 			struct starpu_perfmodel_arch perf_arch;
 			perf_arch.ndevices = 1;
-			perf_arch.devices = (struct starpu_perfmodel_device*)malloc(sizeof(struct starpu_perfmodel_device));
+			STARPU_MALLOC(perf_arch.devices, sizeof(struct starpu_perfmodel_device));
 
 			perf_arch.devices[0].type = STARPU_CUDA_WORKER;
 			perf_arch.devices[0].devid = gpuid;

+ 5 - 4
src/core/perfmodel/regression.c

@@ -1,7 +1,7 @@
 /* StarPU --- Runtime system for heterogeneous multicore architectures.
  *
  * Copyright (C) 2009, 2010, 2011, 2015  Université de Bordeaux
- * Copyright (C) 2010, 2011  CNRS
+ * Copyright (C) 2010, 2011, 2016  CNRS
  *
  * 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
@@ -164,10 +164,11 @@ int _starpu_regression_non_linear_power(struct starpu_perfmodel_history_list *pt
 	unsigned n = find_list_size(ptr);
 	STARPU_ASSERT(n);
 
-	unsigned *x = (unsigned *) malloc(n*sizeof(unsigned));
-	STARPU_ASSERT(x);
+	unsigned *x;
+	STARPU_MALLOC(x, n*sizeof(unsigned));
 
-	double *y = (double *) malloc(n*sizeof(double));
+	double *y;
+	STARPU_MALLOC(y, n*sizeof(double));
 	STARPU_ASSERT(y);
 
 	dump_list(x, y, ptr);

+ 17 - 7
src/core/sched_ctx.c

@@ -247,7 +247,9 @@ static void _starpu_add_workers_to_sched_ctx(struct _starpu_sched_ctx *sched_ctx
 	{
 
 		if(sched_ctx->perf_arch.devices == NULL)
-			sched_ctx->perf_arch.devices = (struct starpu_perfmodel_device*)malloc(ndevices*sizeof(struct starpu_perfmodel_device));
+		{
+			STARPU_MALLOC(sched_ctx->perf_arch.devices, ndevices*sizeof(struct starpu_perfmodel_device));
+		}
 		else
 		{
 			int nfinal_devices = 0;
@@ -272,7 +274,7 @@ static void _starpu_add_workers_to_sched_ctx(struct _starpu_sched_ctx *sched_ctx
 
 
 			int nsize =  (sched_ctx->perf_arch.ndevices+nfinal_devices);
-			sched_ctx->perf_arch.devices  = (struct starpu_perfmodel_device*)realloc(sched_ctx->perf_arch.devices, nsize*sizeof(struct starpu_perfmodel_device));
+			STARPU_REALLOC(sched_ctx->perf_arch.devices, nsize*sizeof(struct starpu_perfmodel_device));
 
 		}
 
@@ -498,7 +500,14 @@ struct _starpu_sched_ctx* _starpu_create_sched_ctx(struct starpu_sched_policy *p
 
 	STARPU_PTHREAD_MUTEX_INIT(&sched_ctx->sched_ctx_list_mutex, NULL);
 
-	sched_ctx->sched_policy = policy ? (struct starpu_sched_policy*)malloc(sizeof(struct starpu_sched_policy)) : NULL;
+	if (policy)
+	{
+		STARPU_MALLOC(sched_ctx->sched_policy, sizeof(struct starpu_sched_policy));
+	}
+	else
+	{
+		sched_ctx->sched_policy = NULL;
+	}
 	sched_ctx->is_initial_sched = is_initial_sched;
 	sched_ctx->name = sched_ctx_name;
 	sched_ctx->inheritor = STARPU_NMAX_SCHED_CTXS;
@@ -1610,7 +1619,7 @@ void* starpu_sched_ctx_get_policy_data(unsigned sched_ctx_id)
 struct starpu_worker_collection* starpu_sched_ctx_create_worker_collection(unsigned sched_ctx_id, enum starpu_worker_collection_type  worker_collection_type)
 {
 	struct _starpu_sched_ctx *sched_ctx = _starpu_get_sched_ctx_struct(sched_ctx_id);
-	sched_ctx->workers = (struct starpu_worker_collection*)malloc(sizeof(struct starpu_worker_collection));
+	STARPU_MALLOC(sched_ctx->workers, sizeof(struct starpu_worker_collection));
 
 	switch(worker_collection_type)
 	{
@@ -1676,12 +1685,13 @@ unsigned starpu_sched_ctx_get_workers_list(unsigned sched_ctx_id, int **workerid
 {
 	struct _starpu_sched_ctx *sched_ctx = _starpu_get_sched_ctx_struct(sched_ctx_id);
 	struct starpu_worker_collection *workers = sched_ctx->workers;
-	if(!workers) return 0;
-	*workerids = (int*)malloc(workers->nworkers*sizeof(int));
 	int worker;
 	unsigned nworkers = 0;
 	struct starpu_sched_ctx_iterator it;
 
+	if(!workers) return 0;
+	STARPU_MALLOC(*workerids, workers->nworkers*sizeof(int));
+
 	workers->init_iterator(workers, &it);
 	while(workers->has_next(workers, &it))
 	{
@@ -2371,7 +2381,7 @@ void starpu_sched_ctx_get_available_cpuids(unsigned sched_ctx_id, int **cpuids,
 	int current_worker_id = starpu_worker_get_id();
 	struct _starpu_sched_ctx *sched_ctx = _starpu_get_sched_ctx_struct(sched_ctx_id);
 	struct starpu_worker_collection *workers = sched_ctx->workers;
-	(*cpuids) = (int*)malloc(workers->nworkers*sizeof(int));
+	STARPU_MALLOC((*cpuids), workers->nworkers*sizeof(int));
 	int w = 0;
 
 	struct starpu_sched_ctx_iterator it;

+ 5 - 3
src/core/sched_ctx_list.c

@@ -65,7 +65,8 @@ struct _starpu_sched_ctx_elt* _starpu_sched_ctx_elt_add_after(struct _starpu_sch
 							      unsigned sched_ctx)
 {
 	struct _starpu_sched_ctx_elt *head, *next;
-	struct _starpu_sched_ctx_elt *elt = (struct _starpu_sched_ctx_elt*)malloc(sizeof(struct _starpu_sched_ctx_elt));
+	struct _starpu_sched_ctx_elt *elt;
+	STARPU_MALLOC(elt, sizeof(struct _starpu_sched_ctx_elt));
 
 	_starpu_sched_ctx_elt_init(elt, sched_ctx);
 	elt->parent = list;
@@ -96,7 +97,8 @@ struct _starpu_sched_ctx_elt* _starpu_sched_ctx_elt_add_before(struct _starpu_sc
 							       unsigned sched_ctx)
 {
 	struct _starpu_sched_ctx_elt *head, *prev;
-	struct _starpu_sched_ctx_elt *elt = (struct _starpu_sched_ctx_elt*)malloc(sizeof(struct _starpu_sched_ctx_elt));
+	struct _starpu_sched_ctx_elt *elt;
+	STARPU_MALLOC(elt, sizeof(struct _starpu_sched_ctx_elt));
 
 	_starpu_sched_ctx_elt_init(elt, sched_ctx);
 	elt->parent = list;
@@ -194,7 +196,7 @@ struct _starpu_sched_ctx_elt* _starpu_sched_ctx_list_add_prio(struct _starpu_sch
 	}
 	else //l's priority is inferior or inexistant, add before
 	{
-		parent_list = (struct _starpu_sched_ctx_list*)malloc(sizeof(struct _starpu_sched_ctx_list));
+		STARPU_MALLOC(parent_list, sizeof(struct _starpu_sched_ctx_list));
 		parent_list->priority = prio;
 		parent_list->next = l;
 		parent_list->head = NULL;

+ 14 - 6
src/core/simgrid.c

@@ -2,6 +2,7 @@
  *
  * Copyright (C) 2012-2016  Université de Bordeaux
  * Copyright (C) 2016  	    Inria
+ * Copyright (C) 2016  	    CNRS
  *
  * 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
@@ -248,7 +249,8 @@ int main(int argc, char **argv)
 	start_simgrid(&argc, argv);
 
 	/* Create a simgrid process for main */
-	struct main_args *args = malloc(sizeof(*args));
+	struct main_args *args;
+	STARPU_MALLOC(args, sizeof(*args));
 	args->argc = argc;
 	args->argv = argv;
 	MSG_process_create_with_arguments("main", &do_starpu_main, calloc(MAX_TSD, sizeof(void*)), MSG_get_host_by_name("MAIN"), 0, (char**) args);
@@ -411,7 +413,8 @@ void _starpu_simgrid_submit_job(int workerid, struct _starpu_job *j, struct star
 	else
 	{
 		/* Asynchronous execution */
-		struct task *task = malloc(sizeof(*task));
+		struct task *task;
+		STARPU_MALLOC(task, sizeof(*task));
 		task->task = simgrid_task;
 		task->workerid = workerid;
 		task->finished = finished;
@@ -568,7 +571,7 @@ static void transfer_submit(struct transfer *transfer)
 			/* Make new wait for the old */
 			transfer->nwait++;
 			/* Make old wake the new */
-			old->wake = realloc(old->wake, (old->nwake + 1) * sizeof(old->wake));
+			STARPU_REALLOC(old->wake, (old->nwake + 1) * sizeof(old->wake));
 			old->wake[old->nwake] = transfer;
 			old->nwake++;
 		}
@@ -589,14 +592,19 @@ int _starpu_simgrid_transfer(size_t size, unsigned src_node, unsigned dst_node,
 	/* Simgrid does not like 0-bytes transfers */
 	if (!size)
 		return 0;
+
 	msg_task_t task;
-	msg_host_t *hosts = calloc(2, sizeof(*hosts));
-	double *computation = calloc(2, sizeof(*computation));
-	double *communication = calloc(4, sizeof(*communication));
+	msg_host_t *hosts;
+	double *computation;
+	double *communication;
 	starpu_pthread_mutex_t mutex;
 	starpu_pthread_cond_t cond;
 	unsigned finished;
 
+	STARPU_CALLOC(hosts, 2, sizeof(*hosts));
+	STARPU_CALLOC(computation, 2, sizeof(*computation));
+	STARPU_CALLOC(communication, 4, sizeof(*communication));
+
 	hosts[0] = _starpu_simgrid_memory_node_get_host(src_node);
 	hosts[1] = _starpu_simgrid_memory_node_get_host(dst_node);
 	STARPU_ASSERT(hosts[0] != hosts[1]);

+ 3 - 5
src/core/task.c

@@ -148,9 +148,7 @@ struct starpu_task * STARPU_ATTRIBUTE_MALLOC starpu_task_create(void)
 {
 	struct starpu_task *task;
 
-	task = (struct starpu_task *) malloc(sizeof(struct starpu_task));
-	STARPU_ASSERT(task);
-
+	STARPU_MALLOC(task, sizeof(struct starpu_task));
 	starpu_task_init(task);
 
 	/* Dynamically allocated tasks are destroyed by default */
@@ -311,7 +309,7 @@ int _starpu_submit_job(struct _starpu_job *j)
 	   && sched_ctx->perf_counters != NULL)
 	{
 		struct starpu_perfmodel_arch arch;
-		arch.devices = (struct starpu_perfmodel_device*)malloc(sizeof(struct starpu_perfmodel_device));
+		STARPU_MALLOC(arch.devices, sizeof(struct starpu_perfmodel_device));
 		arch.ndevices = 1;
 		arch.devices[0].type = STARPU_CPU_WORKER;
 		arch.devices[0].devid = 0;
@@ -552,7 +550,7 @@ static int _starpu_task_submit_head(struct starpu_task *task)
 
 		if (task->dyn_handles)
 		{
-			task->dyn_interfaces = malloc(nbuffers * sizeof(void *));
+			STARPU_MALLOC(task->dyn_interfaces, nbuffers * sizeof(void *));
 		}
 
 		for (i = 0; i < nbuffers; i++)

+ 6 - 8
src/core/task_bundle.c

@@ -28,8 +28,7 @@
 /* Initialize a task bundle */
 void starpu_task_bundle_create(starpu_task_bundle_t *bundle)
 {
-	*bundle = (starpu_task_bundle_t) malloc(sizeof(struct _starpu_task_bundle));
-	STARPU_ASSERT(*bundle);
+	STARPU_MALLOC(*bundle, sizeof(struct _starpu_task_bundle));
 
 	STARPU_PTHREAD_MUTEX_INIT(&(*bundle)->mutex, NULL);
 	/* Of course at the beginning a bundle is open,
@@ -62,8 +61,7 @@ int starpu_task_bundle_insert(starpu_task_bundle_t bundle, struct starpu_task *t
 
 	/* Insert a task at the end of the bundle */
 	struct _starpu_task_bundle_entry *entry;
-	entry = (struct _starpu_task_bundle_entry *) malloc(sizeof(struct _starpu_task_bundle_entry));
-	STARPU_ASSERT(entry);
+	STARPU_MALLOC(entry, sizeof(struct _starpu_task_bundle_entry));
 	entry->task = task;
 	entry->next = NULL;
 
@@ -198,8 +196,8 @@ void _insertion_handle_sorted(struct _starpu_handle_list **listp, starpu_data_ha
 	 * list, we insert it as first element */
 	if (!list || list->handle > handle)
 	{
-		struct _starpu_handle_list *link = (struct _starpu_handle_list *) malloc(sizeof(struct _starpu_handle_list));
-		STARPU_ASSERT(link);
+		struct _starpu_handle_list *link;
+		STARPU_MALLOC(link, sizeof(struct _starpu_handle_list));
 		link->handle = handle;
 		link->mode = mode;
 		link->next = list;
@@ -226,8 +224,8 @@ void _insertion_handle_sorted(struct _starpu_handle_list **listp, starpu_data_ha
 	{
 		/* The handle was not in the list, we insert it after 'prev', thus right before
 		 * 'list' which is the smallest following handle */
-		struct _starpu_handle_list *link = (struct _starpu_handle_list *) malloc(sizeof(struct _starpu_handle_list));
-		STARPU_ASSERT(link);
+		struct _starpu_handle_list *link;
+		STARPU_MALLOC(link, sizeof(struct _starpu_handle_list));
 		link->handle = handle;
 		link->mode = mode;
 		link->next = prev->next;

+ 9 - 11
src/core/topology.c

@@ -295,8 +295,7 @@ _starpu_initialize_workers_opencl_gpuid (struct _starpu_machine_config*config)
 			if (entry == NULL)
 			{
 				struct handle_entry *entry2;
-				entry2 = (struct handle_entry *) malloc(sizeof(*entry2));
-				STARPU_ASSERT(entry2 != NULL);
+				STARPU_MALLOC(entry2, sizeof(*entry2));
 				entry2->gpuid = devid;
 				HASH_ADD_INT(devices_already_used, gpuid,
 					     entry2);
@@ -811,7 +810,7 @@ _starpu_init_mic_config (struct _starpu_machine_config *config,
 		int worker_idx = topology->nworkers + miccore_id;
 		config->workers[worker_idx].set = &mic_worker_set[mic_idx];
 		config->workers[worker_idx].arch = STARPU_MIC_WORKER;
-		config->workers[worker_idx].perf_arch.devices = (struct starpu_perfmodel_device *) malloc(sizeof(struct starpu_perfmodel_device));
+		STARPU_MALLOC(config->workers[worker_idx].perf_arch.devices, sizeof(struct starpu_perfmodel_device));
 		config->workers[worker_idx].perf_arch.ndevices = 1;
 		config->workers[worker_idx].perf_arch.devices[0].type = STARPU_MIC_WORKER;
 		config->workers[worker_idx].perf_arch.devices[0].devid = mic_idx;
@@ -996,7 +995,7 @@ _starpu_init_machine_config(struct _starpu_machine_config *config, int no_mp_con
 
 			config->workers[worker_idx].set = &cuda_worker_set[devid];
 			config->workers[worker_idx].arch = STARPU_CUDA_WORKER;
-			config->workers[worker_idx].perf_arch.devices = (struct starpu_perfmodel_device*)malloc(sizeof(struct starpu_perfmodel_device));
+			STARPU_MALLOC(config->workers[worker_idx].perf_arch.devices, sizeof(struct starpu_perfmodel_device));
 			config->workers[worker_idx].perf_arch.ndevices = 1;
 			config->workers[worker_idx].perf_arch.devices[0].type = STARPU_CUDA_WORKER;
 			config->workers[worker_idx].perf_arch.devices[0].devid = devid;
@@ -1009,8 +1008,7 @@ _starpu_init_machine_config(struct _starpu_machine_config *config, int no_mp_con
 			config->worker_mask |= STARPU_CUDA;
 
 			struct handle_entry *entry;
-			entry = (struct handle_entry *) malloc(sizeof(*entry));
-			STARPU_ASSERT(entry != NULL);
+			STARPU_MALLOC(entry, sizeof(*entry));
 			entry->gpuid = devid;
 			HASH_ADD_INT(devices_using_cuda, gpuid, entry);
 		}
@@ -1075,7 +1073,7 @@ _starpu_init_machine_config(struct _starpu_machine_config *config, int no_mp_con
 			break;
 		}
 		config->workers[worker_idx].arch = STARPU_OPENCL_WORKER;
-		config->workers[worker_idx].perf_arch.devices = (struct starpu_perfmodel_device*)malloc(sizeof(struct starpu_perfmodel_device));
+		STARPU_MALLOC(config->workers[worker_idx].perf_arch.devices, sizeof(struct starpu_perfmodel_device));
 		config->workers[worker_idx].perf_arch.ndevices = 1;
 		config->workers[worker_idx].perf_arch.devices[0].type = STARPU_OPENCL_WORKER;
 		config->workers[worker_idx].perf_arch.devices[0].devid = devid;
@@ -1139,7 +1137,7 @@ _starpu_init_machine_config(struct _starpu_machine_config *config, int no_mp_con
 	{
 		config->workers[topology->nworkers + sccdev].arch = STARPU_SCC_WORKER;
 		int devid = _starpu_get_next_scc_deviceid(config);
-		config->workers[topology->nworkers + sccdev].perf_arch.devices = (struct starpu_perfmodel_device)malloc(sizeof(struct starpu_perfmodel_device));
+		STARPU_MALLOC(config->workers[topology->nworkers + sccdev].perf_arch.devices, sizeof(struct starpu_perfmodel_device));
 		config->workers[topology->nworkers + sccdev].perf_arch.ndevices = 1;
 
 		config->workers[topology->nworkers + sccdev].perf_arch.devices[0].type = STARPU_SCC_WORKER;
@@ -1208,7 +1206,7 @@ _starpu_init_machine_config(struct _starpu_machine_config *config, int no_mp_con
 	{
 		int worker_idx = topology->nworkers + cpu;
 		config->workers[worker_idx].arch = STARPU_CPU_WORKER;
-		config->workers[worker_idx].perf_arch.devices = (struct starpu_perfmodel_device*)malloc(sizeof(struct starpu_perfmodel_device));
+		STARPU_MALLOC(config->workers[worker_idx].perf_arch.devices,  sizeof(struct starpu_perfmodel_device));
 		config->workers[worker_idx].perf_arch.ndevices = 1;
 		config->workers[worker_idx].perf_arch.devices[0].type = STARPU_CPU_WORKER;
 		config->workers[worker_idx].perf_arch.devices[0].devid = 0;
@@ -1716,13 +1714,13 @@ _starpu_init_workers_binding (struct _starpu_machine_config *config, int no_mp_c
 					config->nbindid = STARPU_NMAXWORKERS;
 				else
 					config->nbindid = 2 * old_nbindid;
-				config->bindid_workers = realloc(config->bindid_workers, config->nbindid * sizeof(config->bindid_workers[0]));
+				STARPU_REALLOC(config->bindid_workers, config->nbindid * sizeof(config->bindid_workers[0]));
 				memset(&config->bindid_workers[old_nbindid], 0, (config->nbindid - old_nbindid) * sizeof(config->bindid_workers[0]));
 			}
 			/* Add slot for this worker */
 			/* Don't care about amortizing the cost, there are usually very few workers sharing the same bindid */
 			config->bindid_workers[bindid].nworkers++;
-			config->bindid_workers[bindid].workerids = realloc(config->bindid_workers[bindid].workerids, config->bindid_workers[bindid].nworkers * sizeof(config->bindid_workers[bindid].workerids[0]));
+			STARPU_REALLOC(config->bindid_workers[bindid].workerids, config->bindid_workers[bindid].nworkers * sizeof(config->bindid_workers[bindid].workerids[0]));
 			config->bindid_workers[bindid].workerids[config->bindid_workers[bindid].nworkers-1] = worker;
 		}
 	}

+ 1 - 1
src/core/tree.c

@@ -37,7 +37,7 @@ void starpu_tree_reset_visited(struct starpu_tree *tree, char *visited)
 
 void starpu_tree_prepare_children(unsigned arity, struct starpu_tree *father)
 {
-	father->nodes = (struct starpu_tree*)malloc(arity*sizeof(struct starpu_tree));
+	STARPU_MALLOC(father->nodes, arity*sizeof(struct starpu_tree));
 	father->arity = arity;
 }
 

+ 7 - 4
src/core/workers.c

@@ -1028,7 +1028,8 @@ static void _fill_tree(struct starpu_tree *tree, hwloc_obj_t curr_obj, unsigned
 static void _starpu_build_tree(void)
 {
 #ifdef STARPU_HAVE_HWLOC
-	struct starpu_tree* tree = (struct starpu_tree*)malloc(sizeof(struct starpu_tree));
+	struct starpu_tree *tree;
+	STARPU_MALLOC(tree, sizeof(struct starpu_tree));
 	_starpu_config.topology.tree = tree;
 
 	hwloc_obj_t root = hwloc_get_root_obj(_starpu_config.topology.hwtopology);
@@ -1173,14 +1174,16 @@ int starpu_initialize(struct starpu_conf *user_conf, int *argc, char ***argv)
 	if (_starpu_config.conf.n_cuda_opengl_interoperability)
 	{
 		size_t size = _starpu_config.conf.n_cuda_opengl_interoperability * sizeof(*_starpu_config.conf.cuda_opengl_interoperability);
-		unsigned *copy = malloc(size);
+		unsigned *copy;
+		STARPU_MALLOC(copy, size);
 		memcpy(copy, _starpu_config.conf.cuda_opengl_interoperability, size);
 		_starpu_config.conf.cuda_opengl_interoperability = copy;
 	}
 	if (_starpu_config.conf.n_not_launched_drivers)
 	{
 		size_t size = _starpu_config.conf.n_not_launched_drivers * sizeof(*_starpu_config.conf.not_launched_drivers);
-		struct starpu_driver *copy = malloc(size);
+		struct starpu_driver *copy;
+		STARPU_MALLOC(copy, size);
 		memcpy(copy, _starpu_config.conf.not_launched_drivers, size);
 		_starpu_config.conf.not_launched_drivers = copy;
 	}
@@ -2116,7 +2119,7 @@ unsigned starpu_worker_get_sched_ctx_list(int workerid, unsigned **sched_ctxs)
 {
 	unsigned s = 0;
 	unsigned nsched_ctxs = _starpu_worker_get_nsched_ctxs(workerid);
-	*sched_ctxs = (unsigned*)malloc(nsched_ctxs*sizeof(unsigned));
+	STARPU_MALLOC(*sched_ctxs, nsched_ctxs*sizeof(unsigned));
 	struct _starpu_worker *worker = _starpu_get_worker_struct(workerid);
 	struct _starpu_sched_ctx_elt *e = NULL;
 	struct _starpu_sched_ctx_list_iterator list_it;

+ 1 - 1
src/datawizard/coherency.c

@@ -1142,7 +1142,7 @@ void _starpu_fetch_nowhere_task_input(struct _starpu_job *j)
 		return;
 	}
 
-	wrapper = malloc(sizeof(*wrapper));
+	STARPU_MALLOC(wrapper, (sizeof(*wrapper)));
 	wrapper->j = j;
 	/* +1 for the call below */
 	wrapper->pending = nfetchbuffers + 1;

+ 3 - 3
src/datawizard/data_request.c

@@ -1,7 +1,7 @@
 /* StarPU --- Runtime system for heterogeneous multicore architectures.
  *
  * Copyright (C) 2009-2016  Université de Bordeaux
- * Copyright (C) 2010, 2011, 2012, 2013, 2014, 2015  CNRS
+ * Copyright (C) 2010, 2011, 2012, 2013, 2014, 2015, 2016  CNRS
  *
  * 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
@@ -319,8 +319,8 @@ void _starpu_data_request_append_callback(struct _starpu_data_request *r, void (
 
 	if (callback_func)
 	{
-		struct _starpu_callback_list *link = (struct _starpu_callback_list *) malloc(sizeof(struct _starpu_callback_list));
-		STARPU_ASSERT(link);
+		struct _starpu_callback_list *link;
+		STARPU_MALLOC(link, sizeof(struct _starpu_callback_list));
 
 		link->callback_func = callback_func;
 		link->callback_arg = callback_arg;

+ 7 - 5
src/datawizard/filters.c

@@ -172,8 +172,7 @@ static void _starpu_data_partition(starpu_data_handle_t initial_handle, starpu_d
 	/* allocate the children */
 	if (inherit_state)
 	{
-		initial_handle->children = (struct _starpu_data_state *) calloc(nparts, sizeof(struct _starpu_data_state));
-		STARPU_ASSERT(initial_handle->children);
+		STARPU_CALLOC(initial_handle->children, nparts, sizeof(struct _starpu_data_state));
 
 		/* this handle now has children */
 		initial_handle->nchildren = nparts;
@@ -556,13 +555,16 @@ void starpu_data_partition_plan(starpu_data_handle_t initial_handle, struct star
 		home_node = STARPU_MAIN_RAM;
 
 	for (i = 0; i < nparts; i++)
-		childrenp[i] = calloc(1, sizeof(struct _starpu_data_state));
+	{
+		STARPU_CALLOC(childrenp[i], 1, sizeof(struct _starpu_data_state));
+	}
 	_starpu_data_partition(initial_handle, childrenp, nparts, f, 0);
 
 	if (!cl)
 	{
 		/* Create a codelet that will make the coherency on the home node */
-		cl = initial_handle->switch_cl = calloc(1, sizeof(*initial_handle->switch_cl));
+		STARPU_CALLOC(initial_handle->switch_cl, 1, sizeof(*initial_handle->switch_cl));
+		cl = initial_handle->switch_cl;
 		cl->where = STARPU_NOWHERE;
 		cl->nbuffers = STARPU_VARIABLE_NBUFFERS;
 		cl->name = "data_partition_switch";
@@ -571,7 +573,7 @@ void starpu_data_partition_plan(starpu_data_handle_t initial_handle, struct star
 	if (initial_handle->switch_cl_nparts < nparts)
 	{
 		/* First initialization, or previous initialization was with fewer parts, enlarge it */
-		cl->dyn_nodes = realloc(cl->dyn_nodes, (nparts+1) * sizeof(*cl->dyn_nodes));
+		STARPU_REALLOC(cl->dyn_nodes, (nparts+1) * sizeof(*cl->dyn_nodes));
 		for (i = initial_handle->switch_cl_nparts; i < nparts+1; i++)
 			cl->dyn_nodes[i] = home_node;
 		initial_handle->switch_cl_nparts = nparts;

+ 6 - 9
src/datawizard/interfaces/data_interface.c

@@ -149,8 +149,7 @@ void _starpu_data_register_ram_pointer(starpu_data_handle_t handle, void *ptr)
 {
 	struct handle_entry *entry;
 
-	entry = (struct handle_entry *) malloc(sizeof(*entry));
-	STARPU_ASSERT(entry != NULL);
+	STARPU_MALLOC(entry, sizeof(*entry));
 
 	entry->pointer = ptr;
 	entry->handle = handle;
@@ -370,7 +369,7 @@ _starpu_data_initialize_per_worker(starpu_data_handle_t handle)
 
 	_starpu_spin_checklocked(&handle->header_lock);
 
-	handle->per_worker = calloc(nworkers, sizeof(*handle->per_worker));
+	STARPU_CALLOC(handle->per_worker, nworkers, sizeof(*handle->per_worker));
 
 	size_t interfacesize = handle->ops->interface_size;
 
@@ -396,8 +395,7 @@ _starpu_data_initialize_per_worker(starpu_data_handle_t handle)
 		replicate->initialized = 0;
 		replicate->memory_node = starpu_worker_get_memory_node(worker);
 
-		replicate->data_interface = calloc(1, interfacesize);
-		STARPU_ASSERT(replicate->data_interface);
+		STARPU_CALLOC(replicate->data_interface, 1, interfacesize);
 		/* duplicate  the content of the interface on node 0 */
 		memcpy(replicate->data_interface, handle->per_node[STARPU_MAIN_RAM].data_interface, interfacesize);
 	}
@@ -440,8 +438,7 @@ int _starpu_data_handle_init(starpu_data_handle_t handle, struct starpu_data_int
 
 		replicate->handle = handle;
 
-		replicate->data_interface = calloc(1, interfacesize);
-		STARPU_ASSERT(replicate->data_interface);
+		STARPU_CALLOC(replicate->data_interface, 1, interfacesize);
 	}
 
 	return 0;
@@ -450,8 +447,8 @@ int _starpu_data_handle_init(starpu_data_handle_t handle, struct starpu_data_int
 static
 starpu_data_handle_t _starpu_data_handle_allocate(struct starpu_data_interface_ops *interface_ops, unsigned int mf_node)
 {
-	starpu_data_handle_t handle = (starpu_data_handle_t) calloc(1, sizeof(struct _starpu_data_state));
-	STARPU_ASSERT(handle);
+	starpu_data_handle_t handle;
+	STARPU_CALLOC(handle, 1, sizeof(struct _starpu_data_state));
 	_starpu_data_handle_init(handle, interface_ops, mf_node);
 	return handle;
 }

+ 4 - 4
src/datawizard/interfaces/multiformat_interface.c

@@ -1,7 +1,7 @@
 /* StarPU --- Runtime system for heterogeneous multicore architectures.
  *
  * Copyright (C) 2011-2012  INRIA
- * Copyright (C) 2012, 2013, 2014       CNRS
+ * Copyright (C) 2012, 2013, 2014, 2016       CNRS
  *
  * 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
@@ -649,7 +649,7 @@ static int copy_opencl_to_ram_async(void *src_interface, unsigned src_node,
 	if (dst_multiformat->opencl_ptr == NULL)
 	{
 		/* XXX : it is weird that we might have to allocate memory here... */
-		dst_multiformat->opencl_ptr = malloc(dst_multiformat->nx * dst_multiformat->ops->opencl_elemsize);
+		STARPU_MALLOC(dst_multiformat->opencl_ptr, dst_multiformat->nx * dst_multiformat->ops->opencl_elemsize);
 	}
 	err = starpu_opencl_copy_opencl_to_ram((cl_mem)src_multiformat->opencl_ptr,
 					       src_node,
@@ -711,7 +711,7 @@ static int copy_mic_common_ram_to_mic(void *src_interface, unsigned src_node, vo
 		if (src_multiformat->mic_ptr == NULL)
 			return -ENOMEM;
 	}
-	
+
 	copy_func(src_multiformat->cpu_ptr, src_node, dst_multiformat->cpu_ptr, dst_node, size);
 
 	_STARPU_TRACE_DATA_COPY(src_node, dst_node, size);
@@ -728,7 +728,7 @@ static int copy_mic_common_mic_to_ram(void *src_interface, unsigned src_node, vo
 	STARPU_ASSERT(src_multiformat != NULL);
 	STARPU_ASSERT(dst_multiformat != NULL);
 	STARPU_ASSERT(dst_multiformat->ops != NULL);
-			
+
 	size_t size = src_multiformat->nx * src_multiformat->ops->mic_elemsize;
 	copy_func(src_multiformat->mic_ptr, src_node, dst_multiformat->mic_ptr, dst_node, size);
 

+ 2 - 2
src/datawizard/malloc.c

@@ -1,7 +1,7 @@
 /* StarPU --- Runtime system for heterogeneous multicore architectures.
  *
  * Copyright (C) 2009-2010, 2012-2016  Université de Bordeaux
- * Copyright (C) 2010, 2011, 2012, 2013, 2014, 2015  CNRS
+ * Copyright (C) 2010, 2011, 2012, 2013, 2014, 2015, 2016  CNRS
  *
  * 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
@@ -255,7 +255,7 @@ int starpu_malloc_flags(void **A, size_t dim, int flags)
 				}
 				unlink(name);
 				free(name);
-				dumb = calloc(1,_starpu_malloc_simulation_fold);
+				STARPU_CALLOC(dumb, 1,_starpu_malloc_simulation_fold);
 				write(bogusfile, dumb, _starpu_malloc_simulation_fold);
 				free(dumb);
 			}

+ 2 - 2
src/datawizard/memstats.c

@@ -1,7 +1,7 @@
 /* StarPU --- Runtime system for heterogeneous multicore architectures.
  *
  * Copyright (C) 2009, 2010, 2012  Université de Bordeaux
- * Copyright (C) 2010, 2011, 2012  CNRS
+ * Copyright (C) 2010, 2011, 2012, 2016  CNRS
  *
  * 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
@@ -23,7 +23,7 @@
 void _starpu_memory_stats_init(starpu_data_handle_t handle STARPU_ATTRIBUTE_UNUSED)
 {
 #ifdef STARPU_MEMORY_STATS
-	handle->memory_stats = calloc(1, sizeof(struct _starpu_memory_stats));
+	STARPU_CALLOC(handle->memory_stats, 1, sizeof(struct _starpu_memory_stats));
 #endif
 }
 

+ 2 - 2
src/datawizard/reduction.c

@@ -1,7 +1,7 @@
 /* StarPU --- Runtime system for heterogeneous multicore architectures.
  *
  * Copyright (C) 2010-2014, 2016  Université de Bordeaux
- * Copyright (C) 2011, 2012, 2013  CNRS
+ * Copyright (C) 2011, 2012, 2013, 2016  CNRS
  *
  * 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
@@ -160,7 +160,7 @@ void _starpu_data_end_reduction_mode(starpu_data_handle_t handle)
 	/* Register all valid per-worker replicates */
 	unsigned nworkers = starpu_worker_get_count();
 	STARPU_ASSERT(!handle->reduction_tmp_handles);
-	handle->reduction_tmp_handles = malloc(nworkers * sizeof(handle->reduction_tmp_handles[0]));
+	STARPU_MALLOC(handle->reduction_tmp_handles, nworkers*sizeof(handle->reduction_tmp_handles[0]));
 	for (worker = 0; worker < nworkers; worker++)
 	{
 		if (handle->per_worker[worker].initialized)

+ 5 - 4
src/datawizard/user_interactions.c

@@ -1,7 +1,7 @@
 /* StarPU --- Runtime system for heterogeneous multicore architectures.
  *
  * Copyright (C) 2009-2016  Université de Bordeaux
- * Copyright (C) 2010, 2011, 2012, 2013, 2015  CNRS
+ * Copyright (C) 2010, 2011, 2012, 2013, 2015, 2016  CNRS
  *
  * 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
@@ -126,8 +126,8 @@ int starpu_data_acquire_on_node_cb_sequential_consistency(starpu_data_handle_t h
 	STARPU_ASSERT_MSG(handle->nchildren == 0, "Acquiring a partitioned data (%p) is not possible", handle);
         _STARPU_LOG_IN();
 
-	struct user_interaction_wrapper *wrapper = (struct user_interaction_wrapper *) malloc(sizeof(struct user_interaction_wrapper));
-	STARPU_ASSERT(wrapper);
+	struct user_interaction_wrapper *wrapper;
+	STARPU_MALLOC(wrapper, sizeof(struct user_interaction_wrapper));
 
 	wrapper->handle = handle;
 	wrapper->node = node;
@@ -388,7 +388,8 @@ int _starpu_prefetch_data_on_node_with_mode(starpu_data_handle_t handle, unsigne
 	/* it is forbidden to call this function from a callback or a codelet */
 	STARPU_ASSERT_MSG(async || _starpu_worker_may_perform_blocking_calls(), "Synchronous prefetch is not possible from a task or a callback");
 
-	struct user_interaction_wrapper *wrapper = (struct user_interaction_wrapper *) malloc(sizeof(*wrapper));
+	struct user_interaction_wrapper *wrapper;
+	STARPU_MALLOC(wrapper, sizeof(*wrapper));
 
 	wrapper->handle = handle;
 	wrapper->node = node;

+ 3 - 2
src/debug/traces/anim.c

@@ -56,7 +56,8 @@ static struct component *fxt_component_root(void)
 
 void _starpu_fxt_component_new(uint64_t component, char *name)
 {
-	struct component *comp = malloc(sizeof(*comp));
+	struct component *comp;
+	STARPU_MALLOC(comp, sizeof(*comp));
 
 	if (!strncmp(name, "worker ", 7))
 	{
@@ -368,7 +369,7 @@ void _starpu_fxt_component_connect(uint64_t parent, uint64_t child)
 	STARPU_ASSERT(child_p);
 
 	n = ++parent_p->nchildren;
-	parent_p->children = realloc(parent_p->children, n * sizeof(*parent_p->children));
+	STARPU_REALLOC(parent_p->children, n * sizeof(*parent_p->children));
 	parent_p->children[n-1] = child_p;
 	if (!child_p->parent)
 		child_p->parent = parent_p;

+ 12 - 8
src/debug/traces/starpu_fxt.c

@@ -108,7 +108,7 @@ static struct task_info *get_task(unsigned long job_id, int mpi_rank)
 	HASH_FIND(hh, tasks_info, &job_id, sizeof(job_id), task);
 	if (!task)
 	{
-		task = malloc(sizeof(*task));
+		STARPU_MALLOC(task, sizeof(*task));
 		task->model_name = NULL;
 		task->name = NULL;
 		task->exclude_from_dag = 0;
@@ -396,7 +396,7 @@ static int register_thread(unsigned long tid, int workerid, int sync)
 	if (entry)
 		return 0;
 
-	entry = malloc(sizeof(*entry));
+	STARPU_MALLOC(entry, sizeof(*entry));
 	entry->tid = tid;
 	entry->workerid = workerid;
 	entry->sync = sync;
@@ -864,7 +864,7 @@ static void handle_worker_init_start(struct fxt_ev_64 *ev, struct starpu_fxt_opt
 	char *kindstr = "";
 	struct starpu_perfmodel_arch arch;
 	arch.ndevices = 1;
-	arch.devices = (struct starpu_perfmodel_device *)malloc(sizeof(struct starpu_perfmodel_device));
+	STARPU_MALLOC(arch.devices, sizeof(struct starpu_perfmodel_device));
 
 	switch (ev->param[0])
 	{
@@ -1034,7 +1034,7 @@ static void create_paje_state_if_not_found(char *name, struct starpu_fxt_options
 
 	/* it's the first time ... */
 	struct _starpu_symbol_name *entry = _starpu_symbol_name_new();
-	entry->name = malloc(strlen(name) + 1);
+	STARPU_MALLOC(entry->name, strlen(name) + 1);
 	strcpy(entry->name, name);
 
 	_starpu_symbol_name_list_push_front(&symbol_list, entry);
@@ -1235,7 +1235,9 @@ static void handle_codelet_data_handle(struct fxt_ev_64 *ev, struct starpu_fxt_o
 		}
 	}
 	if (alloc)
-		task->data = realloc(task->data, sizeof(*task->data) * alloc);
+	{
+		STARPU_REALLOC(task->data, sizeof(*task->data) * alloc);
+	}
 	task->data[task->ndata].handle = ev->param[1];
 	task->data[task->ndata].size = ev->param[2];
 	task->data[task->ndata].mode = ev->param[3];
@@ -1324,7 +1326,7 @@ static void handle_end_codelet_body(struct fxt_ev_64 *ev, struct starpu_fxt_opti
 	if (options->dumped_codelets)
 	{
 		dumped_codelets_count++;
-		dumped_codelets = realloc(dumped_codelets, dumped_codelets_count*sizeof(struct starpu_fxt_codelet_event));
+		STARPU_REALLOC(dumped_codelets, dumped_codelets_count*sizeof(struct starpu_fxt_codelet_event));
 
 		snprintf(dumped_codelets[dumped_codelets_count - 1].symbol, sizeof(dumped_codelets[dumped_codelets_count - 1].symbol)-1, "%s", _starpu_last_codelet_symbol[worker]);
 		dumped_codelets[dumped_codelets_count - 1].symbol[sizeof(dumped_codelets[dumped_codelets_count - 1].symbol)-1] = 0;
@@ -2004,7 +2006,9 @@ static void handle_task_deps(struct fxt_ev_64 *ev, struct starpu_fxt_options *op
 		}
 	}
 	if (alloc)
-		task->dependencies = realloc(task->dependencies, sizeof(*task->dependencies) * alloc);
+	{
+		STARPU_REALLOC(task->dependencies, sizeof(*task->dependencies) * alloc);
+	}
 	task->dependencies[task->ndeps++] = dep_prev;
 
 	/* There is a dependency between both job id : dep_prev -> dep_succ */
@@ -3391,7 +3395,7 @@ static void write_task(struct parse_task pt)
 	//fprintf(stderr, "%p %p %s\n", kernel, kernels, codelet_name);
 	if(kernel == NULL)
 	{
-		kernel = malloc(sizeof(*kernel));
+		STARPU_MALLOC(kernel, sizeof(*kernel));
 		kernel->name = strdup(codelet_name);
 		//fprintf(stderr, "%s\n", kernel->name);
 		kernel->file = fopen(codelet_name, "w+");

+ 3 - 3
src/debug/traces/starpu_fxt_mpi.c

@@ -1,7 +1,7 @@
 /* StarPU --- Runtime system for heterogeneous multicore architectures.
  *
  * Copyright (C) 2012-2013, 2016  Université Bordeaux
- * Copyright (C) 2010, 2011, 2014  CNRS
+ * Copyright (C) 2010, 2011, 2014, 2016  CNRS
  *
  * 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
@@ -132,7 +132,7 @@ void _starpu_fxt_mpi_add_send_transfer(int src, int dst STARPU_ATTRIBUTE_UNUSED,
 			mpi_sends_list_size[src] = 1;
 		}
 
-		mpi_sends[src] = realloc(mpi_sends[src], mpi_sends_list_size[src]*sizeof(struct mpi_transfer));
+		STARPU_REALLOC(mpi_sends[src], mpi_sends_list_size[src]*sizeof(struct mpi_transfer));
 	}
 
 	mpi_sends[src][slot].matched = 0;
@@ -157,7 +157,7 @@ void _starpu_fxt_mpi_add_recv_transfer(int src STARPU_ATTRIBUTE_UNUSED, int dst,
 			mpi_recvs_list_size[dst] = 1;
 		}
 
-		mpi_recvs[dst] = realloc(mpi_recvs[dst], mpi_recvs_list_size[dst]*sizeof(struct mpi_transfer));
+		STARPU_REALLOC(mpi_recvs[dst], mpi_recvs_list_size[dst]*sizeof(struct mpi_transfer));
 	}
 
 	mpi_recvs[dst][slot].matched = 0;

+ 10 - 11
src/drivers/mic/driver_mic_sink.c

@@ -52,9 +52,9 @@ void _starpu_mic_sink_init(struct _starpu_mp_node *node)
 
 	_starpu_mic_common_accept(&node->host_sink_dt_connection.mic_endpoint,
 									 STARPU_MIC_SOURCE_DT_PORT_NUMBER);
-	
+
 	node->nb_cores = COISysGetHardwareThreadCount() - COISysGetHardwareThreadCount() / COISysGetCoreCount();
-	node->thread_table = malloc(sizeof(starpu_pthread_t)*node->nb_cores);
+	STARPU_MALLOC(node->thread_table, sizeof(starpu_pthread_t)*node->nb_cores);
 
 #ifdef STARPU_DEVEL
 #warning rather use hwloc
@@ -75,7 +75,7 @@ void _starpu_mic_sink_init(struct _starpu_mp_node *node)
 	//	_starpu_mic_common_connect(&node->sink_sink_dt_connections[i].mic_endpoint,
 	//								STARPU_TO_MIC_ID(i),
 	//								NULL,
-	//								STARPU_MIC_SINK_SINK_DT_PORT_NUMBER(node->devid, i),	
+	//								STARPU_MIC_SINK_SINK_DT_PORT_NUMBER(node->devid, i),
 	//								STARPU_MIC_SINK_SINK_DT_PORT_NUMBER(i, node->devid));
 
 	//for (i = node->devid + 1; i < node->nb_mp_sinks; ++i)
@@ -92,7 +92,7 @@ void _starpu_mic_sink_launch_workers(struct _starpu_mp_node *node)
 	cpu_set_t cpuset;
 	starpu_pthread_attr_t attr;
 	starpu_pthread_t thread;
-	
+
 	/*for each core init the mutex, the task pointer and launch the thread */
 	for(i=0; i<node->nb_cores; i++)
 	{
@@ -106,10 +106,10 @@ void _starpu_mic_sink_launch_workers(struct _starpu_mp_node *node)
 		STARPU_ASSERT(ret == 0);
 
 		/*prepare the argument for the thread*/
-		arg= malloc(sizeof(struct arg_sink_thread));
+		STARPU_MALLOC(arg, sizeof(struct arg_sink_thread));
 		arg->coreid = i;
 		arg->node = node;
-		
+
 		ret = starpu_pthread_create(&thread, &attr, _starpu_sink_thread, arg);
 		STARPU_ASSERT(ret == 0);
 		((starpu_pthread_t *)node->thread_table)[i] = thread;
@@ -121,7 +121,7 @@ void _starpu_mic_sink_launch_workers(struct _starpu_mp_node *node)
  */
 void _starpu_mic_sink_deinit(struct _starpu_mp_node *node)
 {
-	
+
 	int i;
 	node->is_running = 0;
 	for(i=0; i<node->nb_cores; i++)
@@ -165,7 +165,7 @@ void _starpu_mic_sink_allocate(const struct _starpu_mp_node *mp_node, void *arg,
 
 	void *addr = NULL;
 	size_t size = *(size_t *)(arg);
-	
+
 	if (posix_memalign(&addr, STARPU_MIC_PAGE_SIZE, size) != 0)
 		_starpu_mp_common_send_command(mp_node, STARPU_ERROR_ALLOCATE, NULL, 0);
 
@@ -179,7 +179,7 @@ void _starpu_mic_sink_allocate(const struct _starpu_mp_node *mp_node, void *arg,
 		_starpu_mp_common_send_command(mp_node, STARPU_ERROR_ALLOCATE, NULL, 0);
 	}
 #endif
-	
+
 	_starpu_mp_common_send_command(mp_node, STARPU_ANSWER_ALLOCATE, &addr, sizeof(addr));
 }
 
@@ -189,7 +189,7 @@ void _starpu_mic_sink_free(const struct _starpu_mp_node *mp_node STARPU_ATTRIBUT
 	STARPU_ASSERT(arg_size == sizeof(struct _starpu_mic_free_command));
 
 	void *addr = ((struct _starpu_mic_free_command *)arg)->addr;
-	
+
 #ifndef STARPU_DISABLE_ASYNCHRONOUS_MIC_COPY
 	scif_epd_t epd = mp_node->host_sink_dt_connection.mic_endpoint;
 	size_t size = ((struct _starpu_mic_free_command *)arg)->size;
@@ -226,4 +226,3 @@ void (*_starpu_mic_sink_lookup (const struct _starpu_mp_node * node STARPU_ATTRI
 	void *dl_handle = dlopen(NULL, RTLD_NOW);
 	return dlsym(dl_handle, func_name);
 }
-

+ 4 - 4
src/drivers/mp_common/mp_common.c

@@ -109,7 +109,7 @@ _starpu_mp_common_node_create(enum _starpu_mp_node_kind node_kind,
 {
 	struct _starpu_mp_node *node;
 
-	node = (struct _starpu_mp_node *) malloc(sizeof(struct _starpu_mp_node));
+	STARPU_MALLOC(node, sizeof(struct _starpu_mp_node));
 
 	node->kind = node_kind;
 
@@ -235,7 +235,7 @@ _starpu_mp_common_node_create(enum _starpu_mp_node_kind node_kind,
 
 	/* Let's allocate the buffer, we want it to be big enough to contain
 	 * a command, an argument and the argument size */
-	node->buffer = (void *) malloc(BUFFER_SIZE);
+	STARPU_MALLOC(node->buffer, BUFFER_SIZE);
 
 	if (node->init)
 		node->init(node);
@@ -248,8 +248,8 @@ _starpu_mp_common_node_create(enum _starpu_mp_node_kind node_kind,
 	{
 		int i;
 		node->is_running = 1;
-		node->run_table = malloc(sizeof(struct mp_task *)*node->nb_cores);
-		node->sem_run_table = malloc(sizeof(sem_t)*node->nb_cores);
+		STARPU_MALLOC(node->run_table, sizeof(struct mp_task *)*node->nb_cores);
+		STARPU_MALLOC(node->sem_run_table, sizeof(sem_t)*node->nb_cores);
 
 		for(i=0; i<node->nb_cores; i++)
 		{

+ 12 - 9
src/drivers/mp_common/sink_common.c

@@ -87,7 +87,8 @@ void _starpu_sink_common_allocate(const struct _starpu_mp_node *mp_node,
 {
 	STARPU_ASSERT(arg_size == sizeof(size_t));
 
-	void *addr = malloc(*(size_t *)(arg));
+	void *addr;
+	STARPU_MALLOC(addr, *(size_t *)(arg));
 
 	/* If the allocation fail, let's send an error to the host.
 	 */
@@ -375,7 +376,7 @@ static void _starpu_sink_common_pre_execution_message(struct _starpu_mp_node *no
 	/* Init message to tell the sink that the execution has begun */
 	struct mp_message * message = mp_message_new();
 	message->type = STARPU_PRE_EXECUTION;
-	message->buffer = malloc(sizeof(int));
+	STARPU_MALLOC(message->buffer, sizeof(int));
 	*(int *) message->buffer = task->combined_workerid;
 	message->size = sizeof(int);
 
@@ -392,7 +393,7 @@ static void _starpu_sink_common_execution_completed_message(struct _starpu_mp_no
 	/* Init message to tell the sink that the execution is completed */
 	struct mp_message * message = mp_message_new();
 	message->type = STARPU_EXECUTION_COMPLETED;
-	message->buffer = malloc(sizeof(int));
+	STARPU_MALLOC(message->buffer, sizeof(int));
 	*(int*) message->buffer = task->coreid;
 	message->size = sizeof(int);
 
@@ -405,7 +406,8 @@ static void _starpu_sink_common_execution_completed_message(struct _starpu_mp_no
 static void _starpu_sink_common_bind_to_combined_worker(struct _starpu_mp_node *node, int coreid, struct _starpu_combined_worker * combined_worker)
 {
 	int i;
-	int * bind_set = malloc(sizeof(int)*combined_worker->worker_size);
+	int * bind_set;
+	STARPU_MALLOC(bind_set, sizeof(int)*combined_worker->worker_size);
 	for(i=0;i<combined_worker->worker_size;i++)
 		bind_set[i] = combined_worker->combined_workerid[i] - node->baseworkerid;
 	node->bind_thread(node, coreid, bind_set, combined_worker->worker_size);
@@ -567,8 +569,9 @@ void _starpu_sink_common_execute(struct _starpu_mp_node *node,
 	unsigned i;
 
 	uintptr_t arg_ptr = (uintptr_t) arg;
-	struct mp_task *task = malloc(sizeof(struct mp_task));
+	struct mp_task *task;
 
+	STARPU_MALLOC(task, sizeof(struct mp_task));
 	task->kernel = *(void(**)(void **, void *)) arg_ptr;
 	arg_ptr += sizeof(task->kernel);
 
@@ -592,7 +595,7 @@ void _starpu_sink_common_execute(struct _starpu_mp_node *node,
 	task->nb_interfaces = *(unsigned *) arg_ptr;
 	arg_ptr += sizeof(task->nb_interfaces);
 
-	task->interfaces = malloc(task->nb_interfaces * sizeof(*task->interfaces));
+	STARPU_MALLOC(task->interfaces, task->nb_interfaces * sizeof(*task->interfaces));
 
 	/* The function needs an array pointing to each interface it needs
 	 * during execution. As in sink-side there is no mean to know which
@@ -600,9 +603,9 @@ void _starpu_sink_common_execute(struct _starpu_mp_node *node,
 	 * interfaces, thus we expect the same size anyway */
 	for (i = 0; i < task->nb_interfaces; i++)
 	{
-		union _starpu_interface * interface = malloc(sizeof(union _starpu_interface));
-		memcpy(interface, (void*) arg_ptr,
-				sizeof(union _starpu_interface));
+		union _starpu_interface * interface;
+		STARPU_MALLOC(interface, sizeof(union _starpu_interface));
+		memcpy(interface, (void*) arg_ptr, sizeof(union _starpu_interface));
 		task->interfaces[i] = interface;
 		arg_ptr += sizeof(union _starpu_interface);
 	}

+ 5 - 3
src/drivers/mp_common/source_common.c

@@ -158,7 +158,7 @@ int _starpu_src_common_store_message(struct _starpu_mp_node *node,
 		case STARPU_PRE_EXECUTION:
 			message = mp_message_new();
 			message->type = answer;
-			message->buffer = malloc(arg_size);
+			STARPU_MALLOC(message->buffer, arg_size);
 			memcpy(message->buffer, arg, arg_size);
 			message->size = arg_size;
 
@@ -340,7 +340,7 @@ int _starpu_src_common_execute_kernel(struct _starpu_mp_node *node,
 	 * a pointer to the function (sink-side), core on which execute this
 	 * function (sink-side), number of interfaces we send,
 	 * an array of generic (union) interfaces and the value of cl_arg */
-	buffer = (void *) malloc(buffer_size);
+	STARPU_MALLOC(buffer, buffer_size);
 	buffer_ptr = (uintptr_t) buffer;
 
 	*(void(**)(void)) buffer = kernel;
@@ -677,7 +677,9 @@ void _starpu_src_common_worker(struct _starpu_worker_set * worker_set,
 		struct _starpu_mp_node * mp_node)
 {
 	unsigned memnode = worker_set->workers[0].memory_node;
-	struct starpu_task **tasks = malloc(sizeof(struct starpu_task *)*worker_set->nworkers);
+	struct starpu_task **tasks;
+
+	STARPU_MALLOC(tasks, sizeof(struct starpu_task *)*worker_set->nworkers);
 
 	_starpu_src_common_send_workers(mp_node, baseworkerid, worker_set->nworkers);
 

+ 13 - 11
src/drivers/opencl/driver_opencl_utils.c

@@ -53,14 +53,14 @@ int _starpu_opencl_locate_file(const char *source_file_name, char **located_file
 	_STARPU_DEBUG("Trying to locate <%s>\n", source_file_name);
 	if (access(source_file_name, R_OK) == 0)
 	{
-		*located_file_name = calloc(1, strlen(source_file_name)+1);
+		STARPU_CALLOC(*located_file_name, 1, strlen(source_file_name)+1);
 		sprintf(*located_file_name, "%s", source_file_name);
 		ret = EXIT_SUCCESS;
 	}
 
 	if (ret == EXIT_FAILURE && _starpu_opencl_program_dir)
 	{
-		*located_file_name = calloc(1, strlen(_starpu_opencl_program_dir)+1+strlen(source_file_name)+1);
+		STARPU_CALLOC(*located_file_name, 1, strlen(_starpu_opencl_program_dir)+1+strlen(source_file_name)+1);
 		sprintf(*located_file_name, "%s/%s", _starpu_opencl_program_dir, source_file_name);
 		_STARPU_DEBUG("Trying to locate <%s>\n", *located_file_name);
 		if (access(*located_file_name, R_OK) == 0)
@@ -69,7 +69,7 @@ int _starpu_opencl_locate_file(const char *source_file_name, char **located_file
 
 	if (ret == EXIT_FAILURE)
 	{
-		*located_file_name = calloc(1, strlen(STARPU_SRC_DIR)+1+strlen(source_file_name)+1);
+		STARPU_CALLOC(*located_file_name, 1, strlen(STARPU_SRC_DIR)+1+strlen(source_file_name)+1);
 		sprintf(*located_file_name, "%s/%s", STARPU_SRC_DIR, source_file_name);
 		_STARPU_DEBUG("Trying to locate <%s>\n", *located_file_name);
 		if (access(*located_file_name, R_OK) == 0)
@@ -78,7 +78,7 @@ int _starpu_opencl_locate_file(const char *source_file_name, char **located_file
 
 	if (ret == EXIT_FAILURE)
 	{
-		*located_file_name = calloc(1, strlen(_STARPU_STRINGIFY(STARPU_OPENCL_DATADIR))+1+strlen(source_file_name)+1);
+		STARPU_CALLOC(*located_file_name, 1, strlen(_STARPU_STRINGIFY(STARPU_OPENCL_DATADIR))+1+strlen(source_file_name)+1);
 		sprintf(*located_file_name, "%s/%s", _STARPU_STRINGIFY(STARPU_OPENCL_DATADIR), source_file_name);
 		_STARPU_DEBUG("Trying to locate <%s>\n", *located_file_name);
 		if (access(*located_file_name, R_OK) == 0)
@@ -95,12 +95,12 @@ int _starpu_opencl_locate_file(const char *source_file_name, char **located_file
 
 		if (!last)
 		{
-			*located_dir_name = calloc(2, sizeof(char));
+			STARPU_CALLOC(*located_dir_name, 2, sizeof(char));
 			sprintf(*located_dir_name, "%s", "");
 		}
 		else
 		{
-			*located_dir_name=calloc(1, 1+strlen(*located_file_name));
+			STARPU_CALLOC(*located_dir_name, 1, 1+strlen(*located_file_name));
 			sprintf(*located_dir_name, "%s", *located_file_name);
 			(*located_dir_name)[strlen(*located_file_name)-strlen(last)+1] = '\0';
 		}
@@ -161,7 +161,7 @@ char *_starpu_opencl_load_program_source(const char *filename)
 
 	err = stat(filename, &statbuf);
 	STARPU_ASSERT_MSG(err == 0, "could not open file %s\n", filename);
-	source = (char *) malloc(statbuf.st_size + 1);
+	STARPU_MALLOC(source, statbuf.st_size + 1);
 
 	for(c=(char)fgetc(fh), x=0 ; c != EOF ; c =(char)fgetc(fh), x++)
 	{
@@ -310,7 +310,8 @@ int _starpu_opencl_compile_or_load_opencl_from_string(const char *opencl_program
 			clGetProgramBuildInfo(program, device, CL_PROGRAM_BUILD_LOG, 0, NULL, &len);
 			if (len > 2)
 			{
-				char *buffer = malloc(len);
+				char *buffer;
+				STARPU_MALLOC(buffer, len);
 
 				clGetProgramBuildInfo(program, device, CL_PROGRAM_BUILD_LOG, len, buffer, &len);
 				_STARPU_DISP("Compilation output\n%s\n", buffer);
@@ -344,7 +345,7 @@ int _starpu_opencl_compile_or_load_opencl_from_string(const char *opencl_program
 
 			err = clGetProgramInfo(program, CL_PROGRAM_BINARY_SIZES, sizeof(size_t), &binary_len, NULL);
 			if (STARPU_UNLIKELY(err != CL_SUCCESS)) STARPU_OPENCL_REPORT_ERROR(err);
-			binary = malloc(binary_len);
+			STARPU_MALLOC(binary, binary_len);
 
 			err = clGetProgramInfo(program, CL_PROGRAM_BINARIES, sizeof(binary), &binary, NULL);
 			if (STARPU_UNLIKELY(err != CL_SUCCESS)) STARPU_OPENCL_REPORT_ERROR(err);
@@ -380,7 +381,7 @@ void starpu_opencl_load_program_source_malloc(const char *source_file_name, char
 	if(!source)
 		_STARPU_ERROR("Failed to load compute program from file <%s>!\n", *located_file_name);
 
-	*opencl_program_source = malloc(strlen(source)+1);
+	STARPU_MALLOC(*opencl_program_source, strlen(source)+1);
 	sprintf(*opencl_program_source, "%s", source);
 	free(source);
 }
@@ -522,7 +523,8 @@ int starpu_opencl_load_binary_opencl(const char *kernel_id, struct starpu_opencl
 			clGetProgramBuildInfo(program, device, CL_PROGRAM_BUILD_LOG, 0, NULL, &len);
 			if (len > 2)
 			{
-				char *buffer = malloc(len);
+				char *buffer;
+				STARPU_MALLOC(buffer, len);
 
 				clGetProgramBuildInfo(program, device, CL_PROGRAM_BUILD_LOG, len, buffer, &len);
 				_STARPU_DISP("Compilation output\n%s\n", buffer);

+ 9 - 8
src/profiling/bound.c

@@ -1,6 +1,6 @@
 /* StarPU --- Runtime system for heterogeneous multicore architectures.
  *
- * Copyright (C) 2010, 2011, 2012, 2013  CNRS
+ * Copyright (C) 2010, 2011, 2012, 2013, 2016  CNRS
  * Copyright (C) 2010-2016  Université de Bordeaux
  * Copyright (C) 2011  Télécom-SudParis
  *
@@ -192,7 +192,8 @@ static int good_job(struct _starpu_job *j)
 static double** initialize_arch_duration(int maxdevid, unsigned* maxncore_table)
 {
 	int devid, maxncore;
-	double ** arch_model = malloc(sizeof(*arch_model)*(maxdevid+1));
+	double ** arch_model;
+	STARPU_MALLOC(arch_model, sizeof(*arch_model)*(maxdevid+1));
 	arch_model[maxdevid] = NULL;
 	for(devid=0; devid<maxdevid; devid++)
 	{
@@ -200,7 +201,7 @@ static double** initialize_arch_duration(int maxdevid, unsigned* maxncore_table)
 			maxncore = maxncore_table[devid];
 		else
 			maxncore = 1;
-		arch_model[devid] = calloc(maxncore+1,sizeof(*arch_model[devid]));
+		STARPU_CALLOC(arch_model[devid], maxncore+1,sizeof(*arch_model[devid]));
 	}
 	return arch_model;
 }
@@ -234,7 +235,7 @@ static void new_task(struct _starpu_job *j)
 	if (j->bound_task)
 		return;
 
-	t = (struct bound_task *) malloc(sizeof(*t));
+	STARPU_MALLOC(t, sizeof(*t));
 	memset(t, 0, sizeof(*t));
 	t->id = j->job_id;
 	t->tag_id = j->task->tag_id;
@@ -286,7 +287,7 @@ void _starpu_bound_record(struct _starpu_job *j)
 
 		if (!tp)
 		{
-			tp = (struct bound_task_pool *) malloc(sizeof(*tp));
+			STARPU_MALLOC(tp, sizeof(*tp));
 			tp->cl = j->task->cl;
 			tp->footprint = j->footprint;
 			tp->n = 0;
@@ -317,7 +318,7 @@ void _starpu_bound_tag_dep(starpu_tag_t id, starpu_tag_t dep_id)
 		return;
 	}
 
-	td = (struct bound_tag_dep *) malloc(sizeof(*td));
+	STARPU_MALLOC(td, sizeof(*td));
 	td->tag = id;
 	td->dep_tag = dep_id;
 	td->next = tag_deps;
@@ -354,7 +355,7 @@ void _starpu_bound_task_dep(struct _starpu_job *j, struct _starpu_job *dep_j)
 	if (i == t->depsn)
 	{
 		/* Not already there, add */
-		t->deps = (struct task_dep *) realloc(t->deps, ++t->depsn * sizeof(t->deps[0]));
+		STARPU_REALLOC(t->deps, ++t->depsn * sizeof(t->deps[0]));
 		t->deps[t->depsn-1].dep = dep_j->bound_task;
 		t->deps[t->depsn-1].size = 0; /* We don't have data information in that case */
 	}
@@ -411,7 +412,7 @@ void _starpu_bound_job_id_dep(starpu_data_handle_t handle, struct _starpu_job *j
 	if (i == t->depsn)
 	{
 		/* Not already there, add */
-		t->deps = (struct task_dep *) realloc(t->deps, ++t->depsn * sizeof(t->deps[0]));
+		STARPU_REALLOC(t->deps, ++t->depsn * sizeof(t->deps[0]));
 		t->deps[t->depsn-1].dep = dep_t;
 		t->deps[t->depsn-1].size = _starpu_data_get_size(handle);
 	}

+ 2 - 3
src/profiling/profiling.c

@@ -1,7 +1,7 @@
 /* StarPU --- Runtime system for heterogeneous multicore architectures.
  *
  * Copyright (C) 2010-2013, 2016  Université de Bordeaux
- * Copyright (C) 2010, 2011, 2012, 2013  CNRS
+ * Copyright (C) 2010, 2011, 2012, 2013, 2016  CNRS
  *
  * 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
@@ -147,8 +147,7 @@ struct starpu_profiling_task_info *_starpu_allocate_profiling_info_if_needed(str
 	/* If we are benchmarking, we need room for the energy */
 	if (starpu_profiling_status_get() || (task->cl && task->cl->energy_model && (task->cl->energy_model->benchmarking || _starpu_get_calibrate_flag())))
 	{
-		info = (struct starpu_profiling_task_info *) calloc(1, sizeof(struct starpu_profiling_task_info));
-		STARPU_ASSERT(info);
+		STARPU_CALLOC(info, 1, sizeof(struct starpu_profiling_task_info));
 	}
 
 	return info;

+ 4 - 2
src/sched_policies/component_composed.c

@@ -36,7 +36,8 @@ struct starpu_sched_component_composed_recipe
 
 struct starpu_sched_component_composed_recipe * starpu_sched_component_composed_recipe_create(void)
 {
-	struct starpu_sched_component_composed_recipe * recipe = malloc(sizeof(*recipe));
+	struct starpu_sched_component_composed_recipe *recipe;
+	STARPU_MALLOC(recipe, sizeof(*recipe));
 	fun_create_component_list_init(&recipe->list);
 	return recipe;
 }
@@ -212,7 +213,8 @@ struct starpu_sched_component * starpu_sched_component_composed_component_create
 		return l->_head->create_component(tree, l->_head->arg);
 	struct starpu_sched_component * component = starpu_sched_component_create(tree, "composed");
 
-	struct composed_component * c = malloc(sizeof(struct composed_component));
+	struct composed_component *c;
+	STARPU_MALLOC(c, sizeof(struct composed_component));
 	*c = create_composed_component(tree, recipe
 #ifdef STARPU_HAVE_HWLOC
 				  ,component->obj

+ 3 - 2
src/sched_policies/component_fifo.c

@@ -240,8 +240,9 @@ int starpu_sched_component_is_fifo(struct starpu_sched_component * component)
 
 struct starpu_sched_component * starpu_sched_component_fifo_create(struct starpu_sched_tree *tree, struct starpu_sched_component_fifo_data * params)
 {
-	struct starpu_sched_component * component = starpu_sched_component_create(tree, "fifo");
-	struct _starpu_fifo_data * data = malloc(sizeof(*data));
+	struct starpu_sched_component *component = starpu_sched_component_create(tree, "fifo");
+	struct _starpu_fifo_data *data;
+	STARPU_MALLOC(data, sizeof(*data));
 	data->fifo = _starpu_create_fifo();
 	STARPU_PTHREAD_MUTEX_INIT(&data->mutex,NULL);
 	component->data = data;

+ 2 - 1
src/sched_policies/component_heft.c

@@ -230,7 +230,8 @@ struct starpu_sched_component * starpu_sched_component_heft_create(struct starpu
 {
 	struct starpu_sched_component * component = starpu_sched_component_create(tree, "heft");
 	struct _starpu_mct_data *mct_data = starpu_mct_init_parameters(params);
-	struct _starpu_heft_data *data = malloc(sizeof(*data));
+	struct _starpu_heft_data *data;
+	STARPU_MALLOC(data, sizeof(*data));
 
 	_starpu_prio_deque_init(&data->prio);
 	STARPU_PTHREAD_MUTEX_INIT(&data->mutex,NULL);

+ 5 - 3
src/sched_policies/component_perfmodel_select.c

@@ -40,7 +40,7 @@ static int perfmodel_select_push_task(struct starpu_sched_component * component,
 	struct _starpu_perfmodel_select_data * data = component->data;
 	double length;
 	int can_execute = starpu_sched_component_execute_preds(component,task,&length);
-	
+
 	if(can_execute)
 	{
 		if(isnan(length))
@@ -72,11 +72,13 @@ struct starpu_sched_component * starpu_sched_component_perfmodel_select_create(s
 	STARPU_ASSERT(params->calibrator_component && params->no_perfmodel_component && params->perfmodel_component);
 	struct starpu_sched_component * component = starpu_sched_component_create(tree, "perfmodel_selector");
 
-	struct _starpu_perfmodel_select_data * data = malloc(sizeof(*data));
+	struct _starpu_perfmodel_select_data *data;
+	STARPU_MALLOC(data, sizeof(*data));
+
 	data->calibrator_component = params->calibrator_component;
 	data->no_perfmodel_component = params->no_perfmodel_component;
 	data->perfmodel_component = params->perfmodel_component;
-	
+
 	component->data = data;
 	component->push_task = perfmodel_select_push_task;
 	component->deinit_data = perfmodel_select_component_deinit_data;

+ 2 - 1
src/sched_policies/component_prio.c

@@ -262,7 +262,8 @@ int starpu_sched_component_is_prio(struct starpu_sched_component * component)
 struct starpu_sched_component * starpu_sched_component_prio_create(struct starpu_sched_tree *tree, struct starpu_sched_component_prio_data * params)
 {
 	struct starpu_sched_component * component = starpu_sched_component_create(tree, "prio");
-	struct _starpu_prio_data * data = malloc(sizeof(*data));
+	struct _starpu_prio_data *data;
+	STARPU_MALLOC(data, sizeof(*data));
 	_starpu_prio_deque_init(&data->prio);
 	STARPU_PTHREAD_MUTEX_INIT(&data->mutex,NULL);
 	component->data = data;

+ 6 - 6
src/sched_policies/component_sched.c

@@ -411,8 +411,8 @@ struct starpu_sched_tree * starpu_sched_tree_create(unsigned sched_ctx_id)
 {
 	STARPU_ASSERT(sched_ctx_id < STARPU_NMAX_SCHED_CTXS);
 	STARPU_ASSERT(!trees[sched_ctx_id]);
-	struct starpu_sched_tree * t = malloc(sizeof(*t));
-	memset(t, 0, sizeof(*t));
+	struct starpu_sched_tree *t;
+	STARPU_CALLOC(t, 1, sizeof(*t));
 	t->sched_ctx_id = sched_ctx_id;
 	t->workers = starpu_bitmap_create();
 	STARPU_PTHREAD_MUTEX_INIT(&t->lock,NULL);
@@ -455,7 +455,7 @@ void starpu_sched_component_add_child(struct starpu_sched_component* component,
 		STARPU_ASSERT(component->children[i] != NULL);
 	}
 
-	component->children = realloc(component->children, sizeof(struct starpu_sched_component *) * (component->nchildren + 1));
+	STARPU_REALLOC(component->children, sizeof(struct starpu_sched_component *) * (component->nchildren + 1));
 	component->children[component->nchildren] = child;
 	component->nchildren++;
 }
@@ -482,7 +482,7 @@ static void starpu_sched_component_add_parent(struct starpu_sched_component* com
 		STARPU_ASSERT(component->parents[i] != NULL);
 	}
 
-	component->parents = realloc(component->parents, sizeof(struct starpu_sched_component *) * (component->nparents + 1));
+	STARPU_REALLOC(component->parents, sizeof(struct starpu_sched_component *) * (component->nparents + 1));
 	component->parents[component->nparents] = parent;
 	component->nparents++;
 }
@@ -588,8 +588,8 @@ static void take_component_and_does_nothing(struct starpu_sched_component * comp
 
 struct starpu_sched_component * starpu_sched_component_create(struct starpu_sched_tree *tree, const char *name)
 {
-	struct starpu_sched_component * component = malloc(sizeof(*component));
-	memset(component,0,sizeof(*component));
+	struct starpu_sched_component *component;
+	STARPU_CALLOC(component, 1, sizeof(*component));
 	component->tree = tree;
 	component->workers = starpu_bitmap_create();
 	component->workers_in_ctx = starpu_bitmap_create();

+ 10 - 8
src/sched_policies/component_work_stealing.c

@@ -1,7 +1,7 @@
 /* StarPU --- Runtime system for heterogeneous multicore architectures.
  *
  * Copyright (C) 2013  INRIA
- * Copyright (C) 2013  CNRS
+ * Copyright (C) 2013, 2016  CNRS
  * Copyright (C) 2013  Simon Archipoff
  *
  * StarPU is free software; you can redistribute it and/or modify
@@ -289,16 +289,18 @@ void _ws_add_child(struct starpu_sched_component * component, struct starpu_sche
 	if(wsd->size < component->nchildren)
 	{
 		STARPU_ASSERT(wsd->size == component->nchildren - 1);
-		wsd->fifos = realloc(wsd->fifos, component->nchildren * sizeof(*wsd->fifos));
-		wsd->mutexes = realloc(wsd->mutexes, component->nchildren * sizeof(*wsd->mutexes));
+		STARPU_REALLOC(wsd->fifos, component->nchildren * sizeof(*wsd->fifos));
+		STARPU_REALLOC(wsd->mutexes, component->nchildren * sizeof(*wsd->mutexes));
 		wsd->size = component->nchildren;
 	}
 
-	struct _starpu_prio_deque * fifo = malloc(sizeof(*fifo));
+	struct _starpu_prio_deque *fifo;
+	STARPU_MALLOC(fifo, sizeof(*fifo));
 	_starpu_prio_deque_init(fifo);
 	wsd->fifos[component->nchildren - 1] = fifo;
 
-	starpu_pthread_mutex_t * mutex = malloc(sizeof(*mutex));
+	starpu_pthread_mutex_t *mutex;
+	STARPU_MALLOC(mutex, sizeof(*mutex));
 	STARPU_PTHREAD_MUTEX_INIT(mutex,NULL);
 	wsd->mutexes[component->nchildren - 1] = mutex;
 }
@@ -347,9 +349,9 @@ int starpu_sched_component_is_work_stealing(struct starpu_sched_component * comp
 
 struct starpu_sched_component * starpu_sched_component_work_stealing_create(struct starpu_sched_tree *tree, void * arg STARPU_ATTRIBUTE_UNUSED)
 {
-	struct starpu_sched_component * component = starpu_sched_component_create(tree, "work_stealing");
-	struct _starpu_work_stealing_data * wsd = malloc(sizeof(*wsd));
-	memset(wsd, 0, sizeof(*wsd));
+	struct starpu_sched_component *component = starpu_sched_component_create(tree, "work_stealing");
+	struct _starpu_work_stealing_data *wsd;
+	STARPU_CALLOC(wsd, 1, sizeof(*wsd));
 	component->pull_task = pull_task;
 	component->push_task = push_task;
 	component->add_child = _ws_add_child;

+ 8 - 4
src/sched_policies/component_worker.c

@@ -133,7 +133,8 @@ static struct starpu_sched_component * _worker_components[STARPU_NMAX_SCHED_CTXS
 
 static struct _starpu_worker_task_list * _starpu_worker_task_list_create(void)
 {
-	struct _starpu_worker_task_list * l = malloc(sizeof(*l));
+	struct _starpu_worker_task_list *l;
+	STARPU_MALLOC(l, sizeof(*l));
 	memset(l, 0, sizeof(*l));
 	l->exp_len = 0.0;
 	l->exp_start = l->exp_end = starpu_timing_now();
@@ -143,7 +144,8 @@ static struct _starpu_worker_task_list * _starpu_worker_task_list_create(void)
 
 static struct _starpu_task_grid * _starpu_task_grid_create(void)
 {
-	struct _starpu_task_grid * t = malloc(sizeof(*t));
+	struct _starpu_task_grid *t;
+	STARPU_MALLOC(t, sizeof(*t));
 	memset(t, 0, sizeof(*t));
 	return t;
 }
@@ -589,7 +591,8 @@ static struct starpu_sched_component * starpu_sched_component_worker_create(stru
 	char name[32];
 	snprintf(name, sizeof(name), "worker %u", workerid);
 	struct starpu_sched_component * component = starpu_sched_component_create(tree, name);
-	struct _starpu_worker_component_data * data = malloc(sizeof(*data));
+	struct _starpu_worker_component_data *data;
+	STARPU_MALLOC(data, sizeof(*data));
 	memset(data, 0, sizeof(*data));
 
 	data->worker = worker;
@@ -772,7 +775,8 @@ static struct starpu_sched_component  * starpu_sched_component_combined_worker_c
 	if(combined_worker == NULL)
 		return NULL;
 	struct starpu_sched_component * component = starpu_sched_component_create(tree, "combined_worker");
-	struct _starpu_worker_component_data * data = malloc(sizeof(*data));
+	struct _starpu_worker_component_data *data;
+	STARPU_MALLOC(data, sizeof(*data));
 	memset(data, 0, sizeof(*data));
 	data->combined_worker = combined_worker;
 	data->status = COMPONENT_STATUS_SLEEPING;

+ 5 - 4
src/sched_policies/deque_modeling_policy_data_aware.c

@@ -916,8 +916,8 @@ static void dmda_add_workers(unsigned sched_ctx_id, int *workerids, unsigned nwo
 
 		if(dt->num_priorities != -1)
 		{
-			dt->queue_array[workerid]->exp_len_per_priority = (double*)malloc(dt->num_priorities*sizeof(double));
-			dt->queue_array[workerid]->ntasks_per_priority = (unsigned*)malloc(dt->num_priorities*sizeof(unsigned));
+			STARPU_MALLOC(dt->queue_array[workerid]->exp_len_per_priority, dt->num_priorities*sizeof(double));
+			STARPU_MALLOC(dt->queue_array[workerid]->ntasks_per_priority, dt->num_priorities*sizeof(unsigned));
 			int j;
 			for(j = 0; j < dt->num_priorities; j++)
 			{
@@ -953,11 +953,12 @@ static void dmda_remove_workers(unsigned sched_ctx_id, int *workerids, unsigned
 
 static void initialize_dmda_policy(unsigned sched_ctx_id)
 {
-	struct _starpu_dmda_data *dt = (struct _starpu_dmda_data*)calloc(1, sizeof(struct _starpu_dmda_data));
+	struct _starpu_dmda_data *dt;
+	STARPU_CALLOC(dt, 1, sizeof(struct _starpu_dmda_data));
 
 	starpu_sched_ctx_set_policy_data(sched_ctx_id, (void*)dt);
 
-	dt->queue_array = (struct _starpu_fifo_taskq**)malloc(STARPU_NMAXWORKERS*sizeof(struct _starpu_fifo_taskq*));
+	STARPU_MALLOC(dt->queue_array, STARPU_NMAXWORKERS*sizeof(struct _starpu_fifo_taskq*));
 
 	int i;
 	for(i = 0; i < STARPU_NMAXWORKERS; i++)

+ 2 - 1
src/sched_policies/eager_central_policy.c

@@ -36,7 +36,8 @@ struct _starpu_eager_center_policy_data
 
 static void initialize_eager_center_policy(unsigned sched_ctx_id)
 {
-	struct _starpu_eager_center_policy_data *data = (struct _starpu_eager_center_policy_data*)malloc(sizeof(struct _starpu_eager_center_policy_data));
+	struct _starpu_eager_center_policy_data *data;
+	STARPU_MALLOC(data, sizeof(struct _starpu_eager_center_policy_data));
 
 	_STARPU_DISP("Warning: you are running the default eager scheduler, which is not a very smart scheduler. Make sure to read the StarPU documentation about adding performance models in order to be able to use the dmda or dmdas scheduler instead.\n");
 

+ 5 - 4
src/sched_policies/eager_central_priority_policy.c

@@ -58,12 +58,12 @@ static struct _starpu_priority_taskq *_starpu_create_priority_taskq(int min_prio
 {
 	struct _starpu_priority_taskq *central_queue;
 
-	central_queue = (struct _starpu_priority_taskq *) malloc(sizeof(struct _starpu_priority_taskq));
+	STARPU_MALLOC(central_queue, sizeof(struct _starpu_priority_taskq));
 	central_queue->min_prio = min_prio;
 	central_queue->max_prio = max_prio;
 	central_queue->total_ntasks = 0;
-	central_queue->taskq = malloc((max_prio-min_prio+1) * sizeof(struct starpu_task_list));
-	central_queue->ntasks = malloc((max_prio-min_prio+1) * sizeof(unsigned));
+	STARPU_MALLOC(central_queue->taskq, (max_prio-min_prio+1) * sizeof(struct starpu_task_list));
+	STARPU_MALLOC(central_queue->ntasks, (max_prio-min_prio+1) * sizeof(unsigned));
 
 	int prio;
 	for (prio = 0; prio < (max_prio-min_prio+1); prio++)
@@ -84,7 +84,8 @@ static void _starpu_destroy_priority_taskq(struct _starpu_priority_taskq *priori
 
 static void initialize_eager_center_priority_policy(unsigned sched_ctx_id)
 {
-	struct _starpu_eager_central_prio_data *data = (struct _starpu_eager_central_prio_data*)malloc(sizeof(struct _starpu_eager_central_prio_data));
+	struct _starpu_eager_central_prio_data *data;
+	STARPU_MALLOC(data, sizeof(struct _starpu_eager_central_prio_data));
 
 	/* In this policy, we support more than two levels of priority. */
 

+ 2 - 2
src/sched_policies/fifo_queues.c

@@ -1,7 +1,7 @@
 /* StarPU --- Runtime system for heterogeneous multicore architectures.
  *
  * Copyright (C) 2010-2016  Université de Bordeaux
- * Copyright (C) 2010, 2011, 2013  CNRS
+ * Copyright (C) 2010, 2011, 2013, 2016  CNRS
  * Copyright (C) 2011  Télécom-SudParis
  *
  * StarPU is free software; you can redistribute it and/or modify
@@ -44,7 +44,7 @@ static int is_sorted_task_list(struct starpu_task * task)
 struct _starpu_fifo_taskq *_starpu_create_fifo(void)
 {
 	struct _starpu_fifo_taskq *fifo;
-	fifo = (struct _starpu_fifo_taskq *) malloc(sizeof(struct _starpu_fifo_taskq));
+	STARPU_MALLOC(fifo, sizeof(struct _starpu_fifo_taskq));
 
 	/* note that not all mechanisms (eg. the semaphore) have to be used */
 	starpu_task_list_init(&fifo->taskq);

+ 2 - 1
src/sched_policies/graph_test_policy.c

@@ -49,7 +49,8 @@ struct _starpu_graph_test_policy_data
 
 static void initialize_graph_test_policy(unsigned sched_ctx_id)
 {
-	struct _starpu_graph_test_policy_data *data = (struct _starpu_graph_test_policy_data*)malloc(sizeof(struct _starpu_graph_test_policy_data));
+	struct _starpu_graph_test_policy_data *data;
+	STARPU_MALLOC(data, sizeof(struct _starpu_graph_test_policy_data));
 
 	/* there is only a single queue in that trivial design */
 	data->fifo =  _starpu_create_fifo();

+ 2 - 1
src/sched_policies/helper_mct.c

@@ -50,7 +50,8 @@ static const float idle_power_maximum=10000.0;
 
 struct _starpu_mct_data *starpu_mct_init_parameters(struct starpu_sched_component_mct_data *params)
 {
-	struct _starpu_mct_data * data = malloc(sizeof(*data));
+	struct _starpu_mct_data *data;
+	STARPU_MALLOC(data, sizeof(*data));
 	if (params)
 	{
 		data->alpha = params->alpha;

+ 2 - 1
src/sched_policies/heteroprio.c

@@ -200,7 +200,8 @@ static inline void default_init_sched(unsigned sched_ctx_id)
 static void initialize_heteroprio_policy(unsigned sched_ctx_id)
 {
 	/* Alloc the scheduler data  */
-	struct _starpu_heteroprio_data *hp = (struct _starpu_heteroprio_data*)malloc(sizeof(struct _starpu_heteroprio_data));
+	struct _starpu_heteroprio_data *hp;
+	STARPU_MALLOC(hp, sizeof(struct _starpu_heteroprio_data));
 	memset(hp, 0, sizeof(*hp));
 
 	hp->waiters = starpu_bitmap_create();

+ 9 - 8
src/sched_policies/parallel_eager.c

@@ -30,7 +30,7 @@ struct _starpu_peager_data
         starpu_pthread_mutex_t policy_mutex;
 };
 
-#define STARPU_NMAXCOMBINED_WORKERS 520 
+#define STARPU_NMAXCOMBINED_WORKERS 520
 /* instead of STARPU_NMAXCOMBINED_WORKERS, we should use some "MAX combination .."*/
 static int possible_combinations_cnt[STARPU_NMAXWORKERS];
 static int possible_combinations[STARPU_NMAXWORKERS][STARPU_NMAXCOMBINED_WORKERS];
@@ -88,13 +88,13 @@ static void peager_add_workers(unsigned sched_ctx_id, int *workerids, unsigned n
 	for(i = 0; i < nworkers; i++)
 	{
 		workerid = workerids[i];
-		
+
 		/* slaves pick up tasks from their local queue, their master
 		 * will put tasks directly in that local list when a parallel
 		 * tasks comes. */
 		data->local_fifo[workerid] = _starpu_create_fifo();
 	}
-	
+
 #if 0
 	for(i = 0; i < nworkers; i++)
         {
@@ -120,7 +120,8 @@ static void peager_remove_workers(unsigned sched_ctx_id, int *workerids, unsigne
 
 static void initialize_peager_policy(unsigned sched_ctx_id)
 {
-	struct _starpu_peager_data *data = (struct _starpu_peager_data*)malloc(sizeof(struct _starpu_peager_data));
+	struct _starpu_peager_data *data;
+	STARPU_MALLOC(data, sizeof(struct _starpu_peager_data));
 	/* masters pick tasks from that queue */
 	data->fifo = _starpu_create_fifo();
 
@@ -145,9 +146,9 @@ static int push_task_peager_policy(struct starpu_task *task)
 {
 	unsigned sched_ctx_id = task->sched_ctx;
 	int ret_val = -1;
-	
+
 	struct _starpu_peager_data *data = (struct _starpu_peager_data*)starpu_sched_ctx_get_policy_data(sched_ctx_id);
-	
+
 	STARPU_PTHREAD_MUTEX_LOCK(&data->policy_mutex);
 	ret_val = _starpu_fifo_push_task(data->fifo, task);
 	starpu_push_task_end(task);
@@ -167,9 +168,9 @@ static int push_task_peager_policy(struct starpu_task *task)
 		worker = workers->get_next(workers, &it);
 		int master = data->master_id[worker];
 		/* If this is not a CPU or a MIC, then the worker simply grabs tasks from the fifo */
-		if ((!starpu_worker_is_combined_worker(worker) && 
+		if ((!starpu_worker_is_combined_worker(worker) &&
 		    starpu_worker_get_type(worker) != STARPU_MIC_WORKER &&
-		    starpu_worker_get_type(worker) != STARPU_CPU_WORKER)  
+		    starpu_worker_get_type(worker) != STARPU_CPU_WORKER)
 			|| (master == worker))
 			starpu_wake_worker(worker);
 	}

+ 2 - 1
src/sched_policies/parallel_heft.c

@@ -552,7 +552,8 @@ static void parallel_heft_add_workers(__attribute__((unused)) unsigned sched_ctx
 
 static void initialize_parallel_heft_policy(unsigned sched_ctx_id)
 {
-	struct _starpu_pheft_data *hd = (struct _starpu_pheft_data*)malloc(sizeof(struct _starpu_pheft_data));
+	struct _starpu_pheft_data *hd;
+	STARPU_MALLOC(hd, sizeof(struct _starpu_pheft_data));
 
 	if (starpu_sched_ctx_min_priority_is_set(sched_ctx_id) == 0)
 		starpu_sched_ctx_set_min_priority(sched_ctx_id, DEFAULT_MIN_PRIORITY);

+ 1 - 1
src/sched_policies/prio_deque.c

@@ -60,7 +60,7 @@ static struct starpu_task_list * get_prio(struct _starpu_prio_deque * pdeque, in
 				break;
 	}
 	pdeque->size_array++;
-	pdeque->array = realloc(pdeque->array, sizeof(struct _starpu_prio_list) * (pdeque->size_array));
+	STARPU_REALLOC(pdeque->array, sizeof(struct _starpu_prio_list) * (pdeque->size_array));
 	memmove(pdeque->array + i + 1,
 		pdeque->array + i,
 		(pdeque->size_array - i - 1) * sizeof(struct _starpu_prio_list));

+ 1 - 1
src/sched_policies/scheduler_maker.c

@@ -48,7 +48,7 @@ static void destroy_list(struct sched_component_list * list)
 }
 static void add_component(struct sched_component_list *list, struct starpu_sched_component * component)
 {
-	list->arr = realloc(list->arr,sizeof(*list->arr) * (list->size + 1));
+	STARPU_REALLOC(list->arr, sizeof(*list->arr) * (list->size + 1));
 	list->arr[list->size] = component;
 	list->size++;
 }

+ 5 - 4
src/sched_policies/work_stealing_policy.c

@@ -267,7 +267,7 @@ static void locality_pushed_task(struct _starpu_work_stealing_data *ws, struct s
 			HASH_FIND_PTR(data->queued_tasks_per_data, &handle, entry);
 			if (STARPU_LIKELY(!entry))
 			{
-				entry = malloc(sizeof(*entry));
+				STARPU_MALLOC(entry, sizeof(*entry));
 				entry->data = handle;
 				entry->task = task;
 				HASH_ADD_PTR(data->queued_tasks_per_data, data, entry);
@@ -681,7 +681,8 @@ static void ws_remove_workers(unsigned sched_ctx_id, int *workerids, unsigned nw
 
 static void initialize_ws_policy(unsigned sched_ctx_id)
 {
-	struct _starpu_work_stealing_data *ws = (struct _starpu_work_stealing_data*)malloc(sizeof(struct _starpu_work_stealing_data));
+	struct _starpu_work_stealing_data *ws;
+	STARPU_MALLOC(ws, sizeof(struct _starpu_work_stealing_data));
 	starpu_sched_ctx_set_policy_data(sched_ctx_id, (void*)ws);
 
 	ws->last_pop_worker = 0;
@@ -691,7 +692,7 @@ static void initialize_ws_policy(unsigned sched_ctx_id)
 	ws->select_victim = select_victim;
 
 	unsigned nw = starpu_worker_get_count();
-	ws->per_worker = calloc(nw, sizeof(struct _starpu_work_stealing_data_per_worker));
+	STARPU_CALLOC(ws->per_worker, nw, sizeof(struct _starpu_work_stealing_data_per_worker));
 }
 
 static void deinit_ws_policy(unsigned sched_ctx_id)
@@ -758,7 +759,7 @@ static void lws_add_workers(unsigned sched_ctx_id, int *workerids,
 	for (i = 0; i < nworkers; i++)
 	{
 		workerid = workerids[i];
-		ws->per_worker[workerid].proxlist = (int*)malloc(nworkers*sizeof(int));
+		STARPU_MALLOC(ws->per_worker[workerid].proxlist, nworkers*sizeof(int));
 		int bindid;
 
 		struct starpu_tree *neighbour = NULL;

+ 51 - 34
src/top/starpu_top.c

@@ -1,7 +1,7 @@
 /* StarPU --- Runtime system for heterogeneous multicore architectures.
  *
  * Copyright (C) 2011 William Braik, Yann Courtois, Jean-Marie Couteyen, Anthony Roy
- * Copyright (C) 2011, 2012, 2013 CNRS
+ * Copyright (C) 2011, 2012, 2013, 2016 CNRS
  *
  * 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
@@ -70,7 +70,7 @@ void copy_data_and_param(void)
 {
 	printf("%s:%d trace\n", __FILE__, __LINE__);
 	//copying datas
-	starpu_top_datas = (struct starpu_top_data **) malloc(starpu_top_data_cpt*sizeof(struct starpu_top_data*));
+	STARPU_MALLOC(starpu_top_datas, starpu_top_data_cpt*sizeof(struct starpu_top_data*));
 	struct starpu_top_data* cur = starpu_top_first_data;
 	unsigned int i = 0;
 	for(i = 0; i < starpu_top_data_cpt; i++)
@@ -79,7 +79,7 @@ void copy_data_and_param(void)
 		cur = cur->next;
 	}
 	//copying params
-	starpu_top_params = (struct starpu_top_param **) malloc(starpu_top_param_cpt*sizeof(struct starpu_top_param*));
+	STARPU_MALLOC(starpu_top_params, starpu_top_param_cpt*sizeof(struct starpu_top_param*));
 	struct starpu_top_param* cur2 = starpu_top_first_param;
 	for(i = 0; i < starpu_top_param_cpt; i++)
 	{
@@ -117,14 +117,15 @@ static void starpu_top_get_device_type(int id, char* type)
 
 static void starpu_top_send_devices_info(void)
 {
-	char* message=(char *)malloc(5*sizeof(char));
+	char *message;
+	STARPU_MALLOC(message, 5*sizeof(char));
 	snprintf(message,5,"DEV\n");
 	_starpu_top_message_add(_starpu_top_mt,message);
 
 	unsigned int i;
 	for(i=0;i<starpu_worker_get_count();i++)
 	{
-		message=(char *)malloc(sizeof(char)*128);
+		STARPU_MALLOC(message, sizeof(char)*128);
 		char dev_type[10];
 		char dev_name[64];
 		starpu_top_get_device_type(i,dev_type);
@@ -133,7 +134,7 @@ static void starpu_top_send_devices_info(void)
 		_starpu_top_message_add(_starpu_top_mt,message);
 	}
 
-	message=(char*)malloc(6*sizeof(char));
+	STARPU_MALLOC(message, 6*sizeof(char));
 	snprintf(message,6,"/DEV\n");
 	_starpu_top_message_add(_starpu_top_mt,message);
 }
@@ -158,22 +159,23 @@ void starpu_top_init_and_wait(const char* server_name)
 	_starpu_top_communications_threads_launcher();
 
 	//sending server information (report to protocol)
-	char* message = (char *) malloc(strlen("SERVERINFO\n")+1);
+	char *message;
+	STARPU_MALLOC(message, strlen("SERVERINFO\n")+1);
 	sprintf(message, "%s", "SERVERINFO\n");
 	_starpu_top_message_add(_starpu_top_mt,message);
-	message = (char *) malloc(strlen(server_name)+2);
+	STARPU_MALLOC(message, strlen(server_name)+2);
 	sprintf(message, "%s\n", server_name);
 	_starpu_top_message_add(_starpu_top_mt,message);
-	message = (char *) malloc(25);
+	STARPU_MALLOC(message, 25);
 	sprintf(message, "%llu\n", current_timestamp());
 	_starpu_top_message_add(_starpu_top_mt,message);
-	message = (char *) malloc(strlen("/SERVERINFO\n")+1);
+	STARPU_MALLOC(message, strlen("/SERVERINFO\n")+1);
 	sprintf(message,"%s", "/SERVERINFO\n");
 	_starpu_top_message_add(_starpu_top_mt,message);
 
 
 	//sending data list
-	message = (char *) malloc(strlen("DATA\n")+1);
+	STARPU_MALLOC(message, strlen("DATA\n")+1);
 	sprintf(message, "%s", "DATA\n");
 	_starpu_top_message_add(_starpu_top_mt,message);
 	struct starpu_top_data * cur_data = starpu_top_first_data;
@@ -182,12 +184,12 @@ void starpu_top_init_and_wait(const char* server_name)
 		_starpu_top_message_add(_starpu_top_mt,message_for_topdata_init(cur_data));
 		cur_data = cur_data->next;
 	}
-	message = (char *) malloc(strlen("/DATA\n")+1);
+	STARPU_MALLOC(message, strlen("/DATA\n")+1);
 	sprintf(message, "%s", "/DATA\n");
 	_starpu_top_message_add(_starpu_top_mt,message);
 
 	//sending parameter list
-	message = (char *) malloc(strlen("PARAMS\n")+1);
+	STARPU_MALLOC(message, strlen("PARAMS\n")+1);
 	sprintf(message, "%s", "PARAMS\n");
 	_starpu_top_message_add(_starpu_top_mt,message);
 	struct starpu_top_param * cur_param = starpu_top_first_param;
@@ -198,7 +200,7 @@ void starpu_top_init_and_wait(const char* server_name)
 	  cur_param = cur_param->next;
 	}
 	printf("%s:%d parameters sended\n", __FILE__, __LINE__);
-	message = (char *) malloc(strlen("/PARAMS\n")+1);
+	STARPU_MALLOC(message, strlen("/PARAMS\n")+1);
 	sprintf(message, "%s", "/PARAMS\n");
 	_starpu_top_message_add(_starpu_top_mt,message);
 
@@ -211,7 +213,7 @@ void starpu_top_init_and_wait(const char* server_name)
 	copy_data_and_param();
 
 	//sending READY message
-	message = (char *) malloc(strlen("READY\n")+1);
+	STARPU_MALLOC(message, strlen("READY\n")+1);
 	sprintf(message, "%s", "READY\n");
 	_starpu_top_message_add(_starpu_top_mt,message);
 
@@ -262,7 +264,8 @@ void starpu_top_enqueue_data(struct starpu_top_data * data)
 struct starpu_top_data * starpu_top_add_data_boolean(const char* data_name,
 						     int active)
 {
-	struct starpu_top_data * data = (struct starpu_top_data *) malloc(sizeof(struct starpu_top_data));
+	struct starpu_top_data *data;
+	STARPU_MALLOC(data, sizeof(struct starpu_top_data));
 	data->id = starpu_top_data_cpt++;
 	data->name = data_name;
 	data->type = STARPU_TOP_DATA_BOOLEAN;
@@ -279,7 +282,8 @@ struct starpu_top_data * starpu_top_add_data_integer(const char* data_name,
 						     int maximum_value,
 						     int active)
 {
-	struct starpu_top_data * data = (struct starpu_top_data *) malloc(sizeof(struct starpu_top_data));
+	struct starpu_top_data *data;
+	STARPU_MALLOC(data, sizeof(struct starpu_top_data));
 	data->id = starpu_top_data_cpt++;
 	data->name = data_name;
 	data->type = STARPU_TOP_DATA_INTEGER;
@@ -298,7 +302,8 @@ struct starpu_top_data* starpu_top_add_data_float(const char* data_name,
 						  double maximum_value,
 						  int active)
 {
-	struct starpu_top_data * data = (struct starpu_top_data *) malloc(sizeof(struct starpu_top_data));
+	struct starpu_top_data *data;
+	STARPU_MALLOC(data, sizeof(struct starpu_top_data));
 	data->id = starpu_top_data_cpt++;
 	data->name = data_name;
 	data->type = STARPU_TOP_DATA_FLOAT;
@@ -314,7 +319,8 @@ struct starpu_top_data* starpu_top_add_data_float(const char* data_name,
 
 char *message_for_topdata_init(struct starpu_top_data* data)
 {
-	char*message = (char *) malloc(256+strlen(data->name));
+	char *message;
+	STARPU_MALLOC(message, 256+strlen(data->name));
 	switch(data->type)
 	{
 		case STARPU_TOP_DATA_BOOLEAN:
@@ -354,7 +360,7 @@ char *message_for_topparam_init(struct starpu_top_param* param)
 	switch(param->type)
 	{
 	case STARPU_TOP_PARAM_BOOLEAN:
-		message = (char *) malloc(256);
+		STARPU_MALLOC(message, 256);
 		sprintf(message,
 				"BOOL;%u;%s;%d\n",
 				param->id,
@@ -362,7 +368,7 @@ char *message_for_topparam_init(struct starpu_top_param* param)
 				(*(int*)(param->value)) ? 1 : 0);
 		break;
 	case STARPU_TOP_PARAM_INTEGER:
-		message = (char *) malloc(256);
+		STARPU_MALLOC(message, 256);
 		sprintf(message,
 				"INT;%u;%s;%d;%d;%d\n",param->id,
 				param->name,
@@ -371,7 +377,7 @@ char *message_for_topparam_init(struct starpu_top_param* param)
 				*(int*)(param->value));
 		break;
 	case STARPU_TOP_PARAM_FLOAT:
-		message = (char *) malloc(256);
+		STARPU_MALLOC(message, 256);
 		sprintf(message,
 				"FLOAT;%u;%s;%f;%f;%f\n",
 				param->id,
@@ -386,7 +392,7 @@ char *message_for_topparam_init(struct starpu_top_param* param)
 		{
 			length += strlen(param->enum_values[i])+1;
 		}
-		message = (char *) malloc(256+length);
+		STARPU_MALLOC(message, 256+length);
 		sprintf(message,
 				"ENUM;%u;%s;",
 				param->id,
@@ -431,7 +437,8 @@ struct starpu_top_param* starpu_top_register_parameter_boolean(const char* param
 							       void (*callback)(struct starpu_top_param*))
 {
 	STARPU_ASSERT(!_starpu_top_status_get());
-	struct starpu_top_param * param = (struct starpu_top_param *) malloc(sizeof(struct starpu_top_param));
+	struct starpu_top_param *param;
+	STARPU_MALLOC(param, sizeof(struct starpu_top_param));
 	param->callback = callback;
 	param->name = param_name;
 	param->id = starpu_top_param_cpt++;
@@ -451,7 +458,8 @@ struct starpu_top_param* starpu_top_register_parameter_integer(const char* param
 							       void (*callback)(struct starpu_top_param*))
 {
 	STARPU_ASSERT(!_starpu_top_status_get());
-	struct starpu_top_param * param = (struct starpu_top_param *) malloc(sizeof(struct starpu_top_param));
+	struct starpu_top_param *param;
+	STARPU_MALLOC(param, sizeof(struct starpu_top_param));
 	param->callback = callback;
 	param->name = param_name;
 	param->id = starpu_top_param_cpt++;
@@ -473,7 +481,8 @@ struct starpu_top_param* starpu_top_register_parameter_float(const char* param_n
 							     void (*callback)(struct starpu_top_param*))
 {
 	STARPU_ASSERT(!_starpu_top_status_get());
-	struct starpu_top_param * param = (struct starpu_top_param *) malloc(sizeof(struct starpu_top_param));
+	struct starpu_top_param *param;
+	STARPU_MALLOC(param, sizeof(struct starpu_top_param));
 	param->callback = callback;
 	param->name = param_name;
 	param->id = starpu_top_param_cpt++;
@@ -495,7 +504,8 @@ struct starpu_top_param* starpu_top_register_parameter_enum(const char* param_na
 							    void (*callback)(struct starpu_top_param*))
 {
 	STARPU_ASSERT(!_starpu_top_status_get());
-	struct starpu_top_param * param = (struct starpu_top_param *) malloc(sizeof(struct starpu_top_param));
+	struct starpu_top_param *param;
+	STARPU_MALLOC(param, sizeof(struct starpu_top_param));
 	param->callback = callback;
 	param->name = param_name;
 	param->id = starpu_top_param_cpt++;
@@ -519,7 +529,8 @@ void starpu_top_update_data_boolean(const struct starpu_top_data* data, int valu
 		return;
 	if(data->active)
 	{
-		char*message = (char *) malloc(256+strlen(data->name));
+		char *message;
+		STARPU_MALLOC(message, 256+strlen(data->name));
 		sprintf(message,
 				"U;%u;%d;%llu\n",
 				data->id,
@@ -535,7 +546,8 @@ void starpu_top_update_data_integer(const struct starpu_top_data* data, int valu
 		return;
 	if(data->active)
 	{
-		char*message = (char *) malloc(256+strlen(data->name));
+		char *message;
+		STARPU_MALLOC(message, 256+strlen(data->name));
 		sprintf(message,
 				"U;%u;%d;%llu\n",
 				data->id,
@@ -551,7 +563,8 @@ void starpu_top_update_data_float(const struct starpu_top_data* data, double val
 		return;
 	if(data->active)
 	{
-		char*message = (char *) malloc(256+strlen(data->name));
+		char *message;
+		STARPU_MALLOC(message, 256+strlen(data->name));
 		sprintf(message,
 				"U;%u;%f;%llu\n",
 				data->id, value,
@@ -564,7 +577,8 @@ void starpu_top_update_parameter(const struct starpu_top_param* param)
 {
 	if (!_starpu_top_status_get())
 		return;
-	char*message = (char *) malloc(50);
+	char *message;
+	STARPU_MALLOC(message, 50);
 
 	switch(param->type)
 	{
@@ -599,7 +613,8 @@ void starpu_top_debug_log(const char* debug_message)
 	if(starpu_top_debug_on)
 	{
 		//length can be up to strlen*2, if message contains only unwanted chars
-		char * message = (char *) malloc(strlen(debug_message)*2+16);
+		char *message;
+		STARPU_MALLOC(message, strlen(debug_message)*2+16);
 		sprintf(message,"MESSAGE;");
 
 		//escape unwanted char : ; and \n
@@ -627,7 +642,8 @@ void starpu_top_debug_lock(const char* debug_message)
 {
 	if(starpu_top_debug_on)
 	{
-		char * message = (char *) malloc(strlen(debug_message)*2+16);
+		char *message;
+		STARPU_MALLOC(message, strlen(debug_message)*2+16);
 		sprintf(message,"LOCK;");
 		char* cur = message+5;
 		while(*debug_message!='\0')
@@ -761,7 +777,8 @@ void starpu_top_change_debug_mode(const char*message)
 		printf("%s:%d debug is now OFF\n", __FILE__, __LINE__);
 	}
 
-	char * m = (char *) malloc(strlen(message)+1);
+	char *m;
+	STARPU_MALLOC(m, strlen(message)+1);
 	sprintf(m,"%s",message);
 	_starpu_top_message_add(_starpu_top_mt,m);
 }

+ 2 - 6
src/top/starpu_top_message_queue.c

@@ -99,12 +99,8 @@ char* _starpu_top_message_remove(struct _starpu_top_message_queue* s)
 
 struct _starpu_top_message_queue* _starpu_top_message_queue_new(void)
 {
-	struct _starpu_top_message_queue* p = (struct _starpu_top_message_queue *) malloc( 1 * sizeof(*p));
-	if( NULL == p )
-	{
-		fprintf(stderr, "LINE: %d, malloc() failed\n", __LINE__);
-		return NULL;
-	}
+	struct _starpu_top_message_queue *p;
+	STARPU_MALLOC(p, sizeof(*p));
 
 	p->head = p->tail = NULL;
 	sem_init(&(p->semaphore),0,0);

+ 7 - 4
src/top/starpu_top_task.c

@@ -1,7 +1,7 @@
 /* StarPU --- Runtime system for heterogeneous multicore architectures.
  *
  * Copyright (C) 2011 William Braik, Yann Courtois, Jean-Marie Couteyen, Anthony Roy
- * Copyright (C) 2011, 2013 CNRS
+ * Copyright (C) 2011, 2013, 2016 CNRS
  *
  * 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
@@ -34,7 +34,8 @@ void _starpu_top_task_started(struct starpu_task *task,
 {
 	unsigned long long taskid = _starpu_get_job_associated_to_task(task)->job_id;
 	STARPU_ASSERT(_starpu_top_status_get());
-	char *str = (char *) malloc(sizeof(char)*64);
+	char *str;
+	STARPU_MALLOC(str, sizeof(char)*64);
 	snprintf(str, 64,
 				"START;%llu;%d;%llu\n",
 				taskid,
@@ -51,7 +52,8 @@ void _starpu_top_task_ended(struct starpu_task *task,
 	unsigned long long taskid = _starpu_get_job_associated_to_task(task)->job_id;
 	(void) devid; //unused
 	STARPU_ASSERT(_starpu_top_status_get());
-	char *str = (char *) malloc(sizeof(char)*64);
+	char *str;
+	STARPU_MALLOC(str, sizeof(char)*64);
 	snprintf(str, 64,
 				"END;%llu;%llu\n",
 				taskid,
@@ -83,7 +85,8 @@ void starpu_top_task_prevision(struct starpu_task *task,
 	STARPU_ASSERT(_starpu_top_status_get());
 	struct timespec now;
 	_starpu_clock_gettime(&now);
-	char * str= (char *)malloc(sizeof(char)*200);
+	char *str;
+	STARPU_MALLOC(str, sizeof(char)*200);
 	snprintf(str, 128,
 		 "PREV;%llu;%d;%llu;%llu;%llu\n",
 		 taskid,

+ 12 - 5
src/util/fstarpu.c

@@ -161,7 +161,8 @@ intptr_t fstarpu_get_constant(char *s)
 
 struct starpu_conf *fstarpu_conf_allocate(void)
 {
-	struct starpu_conf *conf = malloc(sizeof(*conf));
+	struct starpu_conf *conf;
+	STARPU_MALLOC(conf, sizeof(*conf));
 	starpu_conf_init(conf);
 	return conf;
 }
@@ -236,7 +237,8 @@ void fstarpu_topology_print(void)
 
 struct starpu_codelet *fstarpu_codelet_allocate(void)
 {
-	struct starpu_codelet *cl = malloc(sizeof(*cl));
+	struct starpu_codelet *cl;
+	STARPU_MALLOC(cl, sizeof(*cl));
 	starpu_codelet_init(cl);
 	return cl;
 }
@@ -542,7 +544,9 @@ void fstarpu_worker_get_type_as_string(intptr_t type, char *dst, size_t maxlen)
 
 struct starpu_data_handle *fstarpu_data_handle_array_alloc(int nb)
 {
-	return calloc((size_t)nb, sizeof(starpu_data_handle_t));
+	void *ptr;
+	STARPU_CALLOC(ptr, (size_t)nb, sizeof(starpu_data_handle_t));
+	return ptr;
 }
 
 void fstarpu_data_handle_array_free(starpu_data_handle_t *handles)
@@ -557,7 +561,9 @@ void fstarpu_data_handle_array_set(starpu_data_handle_t *handles, int i, starpu_
 
 struct starpu_data_descr *fstarpu_data_descr_array_alloc(int nb)
 {
-	return calloc((size_t)nb, sizeof(struct starpu_data_descr));
+	void *ptr;
+	STARPU_CALLOC(ptr, (size_t)nb, sizeof(struct starpu_data_descr));
+	return ptr;
 }
 
 struct starpu_data_descr *fstarpu_data_descr_alloc(void)
@@ -588,7 +594,8 @@ void fstarpu_data_descr_set(struct starpu_data_descr *descr, starpu_data_handle_
 
 struct starpu_data_filter *fstarpu_data_filter_allocate(void)
 {
-	struct starpu_data_filter *filter = calloc(1, sizeof(*filter));
+	struct starpu_data_filter *filter;
+	STARPU_CALLOC(filter, 1, sizeof(*filter));
 	return filter;
 }
 

+ 23 - 42
src/util/openmp_runtime_support.c

@@ -263,9 +263,7 @@ static struct starpu_omp_critical *create_omp_critical_struct(void)
 {
 	struct starpu_omp_critical *critical;
 
-	critical = calloc(1, sizeof(*critical));
-	if (!critical)
-		_STARPU_ERROR("memory allocation failed");
+	STARPU_CALLOC(critical, 1, sizeof(*critical));
 	_starpu_spin_init(&critical->lock);
 	return critical;
 }
@@ -283,9 +281,7 @@ static struct starpu_omp_device *create_omp_device_struct(void)
 {
 	struct starpu_omp_device *device;
 
-	device = calloc(1, sizeof(*device));
-	if (!device)
-		_STARPU_ERROR("memory allocation failed");
+	STARPU_CALLOC(device, 1, sizeof(*device));
 	_starpu_spin_init(&device->atomic_lock);
 	return device;
 }
@@ -318,9 +314,7 @@ static struct starpu_omp_region *create_omp_region_struct(struct starpu_omp_regi
 {
 	struct starpu_omp_region *region;
 
-	region = calloc(1, sizeof(*region));
-	if (!region)
-		_STARPU_ERROR("memory allocation failed");
+	STARPU_CALLOC(region, 1, sizeof(*region));
 	region->parent_region = parent_region;
 	region->owner_device = owner_device;
 	starpu_omp_thread_list_init(&region->thread_list);
@@ -474,9 +468,7 @@ static void starpu_omp_implicit_task_exec(void *buffers[], void *cl_arg)
 		task->starpu_cl_arg = cl_arg;
 		STARPU_ASSERT(task->stack == NULL);
 		STARPU_ASSERT(task->stacksize > 0);
-		task->stack = malloc(task->stacksize);
-		if (task->stack == NULL)
-			_STARPU_ERROR("memory allocation failed");
+		STARPU_MALLOC(task->stack, task->stacksize);
 		getcontext(&task->ctx);
 		/*
 		 * we do not use uc_link, starpu_omp_task_entry will handle
@@ -644,9 +636,7 @@ static void starpu_omp_explicit_task_exec(void *buffers[], void *cl_arg)
 		task->starpu_cl_arg = cl_arg;
 		STARPU_ASSERT(task->stack == NULL);
 		STARPU_ASSERT(task->stacksize > 0);
-		task->stack = malloc(task->stacksize);
-		if (task->stack == NULL)
-			_STARPU_ERROR("memory allocation failed");
+		STARPU_MALLOC(task->stack, task->stacksize);
 		getcontext(&task->ctx);
 		/*
 		 * we do not use uc_link, starpu_omp_task_entry will handle
@@ -741,7 +731,7 @@ static void omp_initial_thread_setup(void)
 	initial_thread->current_task = initial_task;
 	/* .owner_region already set in create_omp_thread_struct */
 	/* .initial_thread_stack */
-	initial_thread->initial_thread_stack = malloc(_STARPU_INITIAL_THREAD_STACKSIZE);
+	STARPU_MALLOC(initial_thread->initial_thread_stack, _STARPU_INITIAL_THREAD_STACKSIZE);
 	if (initial_thread->initial_thread_stack == NULL)
 		_STARPU_ERROR("memory allocation failed");
 	/* .ctx */
@@ -774,7 +764,7 @@ static void omp_initial_thread_setup(void)
 	_starpu_omp_set_task(initial_task);
 
 	_global_state.nb_starpu_cpu_workers = starpu_worker_get_count_by_type(STARPU_CPU_WORKER);
-	_global_state.starpu_cpu_worker_ids = malloc(_global_state.nb_starpu_cpu_workers * sizeof(int));
+	STARPU_MALLOC(_global_state.starpu_cpu_worker_ids, _global_state.nb_starpu_cpu_workers * sizeof(int));
 	if (_global_state.starpu_cpu_worker_ids == NULL)
 		_STARPU_ERROR("memory allocation failed");
 	ret = starpu_worker_get_ids_by_type(STARPU_CPU_WORKER, _global_state.starpu_cpu_worker_ids, _global_state.nb_starpu_cpu_workers);
@@ -844,7 +834,7 @@ static void omp_initial_region_setup(void)
 	_global_state.initial_region->icvs.nest_var = _starpu_omp_initial_icv_values->nest_var;
 	if (_starpu_omp_initial_icv_values->nthreads_var[1] != 0)
 	{
-		_global_state.initial_region->icvs.nthreads_var = malloc((1+max_active_levels-_global_state.initial_region->level) * sizeof(*_global_state.initial_region->icvs.nthreads_var));
+		STARPU_MALLOC(_global_state.initial_region->icvs.nthreads_var, (1+max_active_levels-_global_state.initial_region->level) * sizeof(*_global_state.initial_region->icvs.nthreads_var));
 		int i,j;
 		for (i = _global_state.initial_region->level, j = 0; i < max_active_levels; i++, j++)
 		{
@@ -854,14 +844,14 @@ static void omp_initial_region_setup(void)
 	}
 	else
 	{
-		_global_state.initial_region->icvs.nthreads_var = malloc(2 * sizeof(*_global_state.initial_region->icvs.nthreads_var));
+		STARPU_MALLOC(_global_state.initial_region->icvs.nthreads_var, 2 * sizeof(*_global_state.initial_region->icvs.nthreads_var));
 		_global_state.initial_region->icvs.nthreads_var[0] = _starpu_omp_initial_icv_values->nthreads_var[0];
 		_global_state.initial_region->icvs.nthreads_var[1] = 0;
 	}
 
 	if (_starpu_omp_initial_icv_values->bind_var[1] != starpu_omp_proc_bind_undefined)
 	{
-		_global_state.initial_region->icvs.bind_var = malloc((1+max_active_levels-_global_state.initial_region->level) * sizeof(*_global_state.initial_region->icvs.bind_var));
+		STARPU_MALLOC(_global_state.initial_region->icvs.bind_var, (1+max_active_levels-_global_state.initial_region->level) * sizeof(*_global_state.initial_region->icvs.bind_var));
 		int i,j;
 		for (i = _global_state.initial_region->level, j = 0; i < max_active_levels; i++, j++)
 		{
@@ -871,7 +861,7 @@ static void omp_initial_region_setup(void)
 	}
 	else
 	{
-		_global_state.initial_region->icvs.bind_var = malloc(2 * sizeof(*_global_state.initial_region->icvs.bind_var));
+		STARPU_MALLOC(_global_state.initial_region->icvs.bind_var, 2 * sizeof(*_global_state.initial_region->icvs.bind_var));
 		_global_state.initial_region->icvs.bind_var[0] = _starpu_omp_initial_icv_values->bind_var[0];
 		_global_state.initial_region->icvs.bind_var[1] = starpu_omp_proc_bind_undefined;
 	}
@@ -1048,7 +1038,7 @@ void starpu_omp_parallel_region(const struct starpu_omp_parallel_region_attr *at
 	{
 		if (generating_region->icvs.nthreads_var[1] != 0)
 		{
-			new_region->icvs.nthreads_var = malloc((1+max_active_levels-new_region->level) * sizeof(*new_region->icvs.nthreads_var));
+			STARPU_MALLOC(new_region->icvs.nthreads_var, (1+max_active_levels-new_region->level) * sizeof(*new_region->icvs.nthreads_var));
 			int i,j;
 			for (i = new_region->level, j = 0; i < max_active_levels; i++, j++)
 			{
@@ -1058,14 +1048,14 @@ void starpu_omp_parallel_region(const struct starpu_omp_parallel_region_attr *at
 		}
 		else
 		{
-			new_region->icvs.nthreads_var = malloc(2 * sizeof(*new_region->icvs.nthreads_var));
+			STARPU_MALLOC(new_region->icvs.nthreads_var, 2 * sizeof(*new_region->icvs.nthreads_var));
 			new_region->icvs.nthreads_var[0] = generating_region->icvs.nthreads_var[0];
 			new_region->icvs.nthreads_var[1] = 0;
 		}
 
 		if (generating_region->icvs.bind_var[1] != starpu_omp_proc_bind_undefined)
 		{
-			new_region->icvs.bind_var = malloc((1+max_active_levels-new_region->level) * sizeof(*new_region->icvs.bind_var));
+			STARPU_MALLOC(new_region->icvs.bind_var, (1+max_active_levels-new_region->level) * sizeof(*new_region->icvs.bind_var));
 			int i,j;
 			for (i = new_region->level, j = 0; i < max_active_levels; i++, j++)
 			{
@@ -1075,17 +1065,17 @@ void starpu_omp_parallel_region(const struct starpu_omp_parallel_region_attr *at
 		}
 		else
 		{
-			new_region->icvs.bind_var = malloc(2 * sizeof(*new_region->icvs.bind_var));
+			STARPU_MALLOC(new_region->icvs.bind_var, 2 * sizeof(*new_region->icvs.bind_var));
 			new_region->icvs.bind_var[0] = generating_region->icvs.bind_var[0];
 			new_region->icvs.bind_var[1] = starpu_omp_proc_bind_undefined;
 		}
 	}
 	else
 	{
-		new_region->icvs.nthreads_var = malloc(sizeof(*new_region->icvs.nthreads_var));
+		STARPU_MALLOC(new_region->icvs.nthreads_var, sizeof(*new_region->icvs.nthreads_var));
 		new_region->icvs.nthreads_var[0] = generating_region->icvs.nthreads_var[0];
 
-		new_region->icvs.bind_var = malloc(sizeof(*new_region->icvs.bind_var));
+		STARPU_MALLOC(new_region->icvs.bind_var, sizeof(*new_region->icvs.bind_var));
 		new_region->icvs.bind_var[0] = generating_region->icvs.bind_var[0];
 	}
 	new_region->icvs.thread_limit_var = generating_region->icvs.thread_limit_var;
@@ -1762,9 +1752,8 @@ void starpu_omp_taskgroup(void (*f)(void *arg), void *arg)
 void starpu_omp_taskgroup_inline_begin(void)
 {
 	struct starpu_omp_task *task = _starpu_omp_get_task();
-	struct starpu_omp_task_group *p_task_group = malloc(sizeof(*p_task_group));
-	if (p_task_group == NULL)
-		_STARPU_ERROR("memory allocation failed\n");
+	struct starpu_omp_task_group *p_task_group;
+	STARPU_MALLOC(p_task_group, sizeof(*p_task_group));
 	p_task_group->p_previous_task_group = task->task_group;
 	p_task_group->descendent_task_count = 0;
 	p_task_group->leader_task = task;
@@ -1947,9 +1936,7 @@ static inline struct starpu_omp_loop *_starpu_omp_for_loop_begin(struct starpu_o
 	loop = _starpu_omp_for_get_loop(parallel_region, task);
 	if (!loop)
 	{
-		loop = malloc(sizeof(*loop));
-		if (loop == NULL)
-			_STARPU_ERROR("memory allocation failed\n");
+		STARPU_MALLOC(loop, sizeof(*loop));
 		loop->id = task->loop_id;
 		loop->next_iteration = 0;
 		loop->nb_completed_threads = 0;
@@ -2126,9 +2113,7 @@ static inline struct starpu_omp_sections *_starpu_omp_sections_begin(struct star
 	sections = _starpu_omp_get_sections(parallel_region, task);
 	if (!sections)
 	{
-		sections = malloc(sizeof(*sections));
-		if (sections == NULL)
-			_STARPU_ERROR("memory allocation failed\n");
+		STARPU_MALLOC(sections, sizeof(*sections));
 		sections->id = task->sections_id;
 		sections->next_section_num = 0;
 		sections->nb_completed_threads = 0;
@@ -2222,9 +2207,7 @@ static void _starpu_omp_lock_init(void **_internal)
 {
 	struct _starpu_omp_lock_internal *_lock;
 
-	_lock = calloc(1, sizeof(*_lock));
-	if (!_lock)
-		_STARPU_ERROR("memory allocation failed");
+	STARPU_CALLOC(_lock, 1, sizeof(*_lock));
 	_starpu_spin_init(&_lock->lock);
 	condition_init(&_lock->cond);
 	*_internal = _lock;
@@ -2281,9 +2264,7 @@ static void _starpu_omp_nest_lock_init(void **_internal)
 {
 	struct _starpu_omp_nest_lock_internal *_nest_lock;
 
-	_nest_lock = calloc(1, sizeof(*_nest_lock));
-	if (!_nest_lock)
-		_STARPU_ERROR("memory allocation failed");
+	STARPU_CALLOC(_nest_lock, 1, sizeof(*_nest_lock));
 	_starpu_spin_init(&_nest_lock->lock);
 	condition_init(&_nest_lock->cond);
 	*_internal = _nest_lock;

+ 8 - 24
src/util/openmp_runtime_support_environment.c

@@ -402,16 +402,12 @@ static void read_a_places_list(const char *str, struct starpu_omp_place *places)
 						if (excluded_items_list_size == 0)
 						{
 							excluded_items_list_size = _STARPU_INITIAL_PLACE_ITEMS_LIST_SIZE;
-							excluded_items_list = malloc(excluded_items_list_size * sizeof(int));
-							if (excluded_items_list == NULL)
-								_STARPU_ERROR("memory allocation failed");
+							STARPU_MALLOC(excluded_items_list, excluded_items_list_size * sizeof(int));
 						}
 						else if (nb_excluded_items == excluded_items_list_size)
 						{
 							excluded_items_list_size *= 2;
-							excluded_items_list = realloc(excluded_items_list, excluded_items_list_size * sizeof(int));
-							if (excluded_items_list == NULL)
-								_STARPU_ERROR("memory allocation failed");
+							STARPU_REALLOC(excluded_items_list, excluded_items_list_size * sizeof(int));
 						}
 						excluded_items_list[nb_excluded_items] = v;
 						nb_excluded_items++;
@@ -421,16 +417,12 @@ static void read_a_places_list(const char *str, struct starpu_omp_place *places)
 						if (included_items_list_size == 0)
 						{
 							included_items_list_size = _STARPU_INITIAL_PLACE_ITEMS_LIST_SIZE;
-							included_items_list = malloc(included_items_list_size * sizeof(int));
-							if (included_items_list == NULL)
-								_STARPU_ERROR("memory allocation failed");
+							STARPU_MALLOC(included_items_list, included_items_list_size * sizeof(int));
 						}
 						else if (nb_included_items == included_items_list_size)
 						{
 							included_items_list_size *= 2;
-							included_items_list = realloc(included_items_list, included_items_list_size * sizeof(int));
-							if (included_items_list == NULL)
-								_STARPU_ERROR("memory allocation failed");
+							STARPU_REALLOC(included_items_list, included_items_list_size * sizeof(int));
 						}
 						included_items_list[nb_included_items] = v;
 						nb_included_items++;
@@ -462,16 +454,12 @@ static void read_a_places_list(const char *str, struct starpu_omp_place *places)
 				if (places_list_size == 0)
 				{
 					places_list_size = _STARPU_INITIAL_PLACES_LIST_SIZE;
-					places_list = malloc(places_list_size * sizeof(*places_list));
-					if (places_list == NULL)
-						_STARPU_ERROR("memory allocation failed");
+					STARPU_MALLOC(places_list, places_list_size * sizeof(*places_list));
 				}
 				else if (nb_places == places_list_size)
 				{
 					places_list_size *= 2;
-					places_list = realloc(places_list, places_list_size * sizeof(*places_list));
-					if (places_list == NULL)
-						_STARPU_ERROR("memory allocation failed");
+					STARPU_REALLOC(places_list, places_list_size * sizeof(*places_list));
 				}
 				places_list[nb_places].excluded_place = exclude_place_flag;
 				places_list[nb_places].included_numeric_items = included_items_list;
@@ -557,9 +545,7 @@ static void read_proc_bind_var()
 	int level = 0;
 	char *env;
 
-	bind_list = calloc(max_levels, sizeof(*bind_list));
-	if (!bind_list)
-		_STARPU_ERROR("memory allocation failed\n");
+	STARPU_CALLOC(bind_list, max_levels, sizeof(*bind_list));
 
 	env = starpu_getenv("OMP_PROC_BIND");
 	if (env)
@@ -589,9 +575,7 @@ static void read_num_threads_var()
 	int level = 0;
 	char *env;
 
-	num_threads_list = calloc(max_levels, sizeof(*num_threads_list));
-	if (!num_threads_list)
-		_STARPU_ERROR("memory allocation failed\n");
+	STARPU_CALLOC(num_threads_list, max_levels, sizeof(*num_threads_list));
 
 	env = starpu_getenv("OMP_NUM_THREADS");
 	if (env)

+ 15 - 15
src/util/starpu_clusters_create.c

@@ -96,9 +96,10 @@ struct starpu_cluster_machine *starpu_cluster_machine(hwloc_obj_type_t cluster_l
 	va_list varg_list;
 	int arg_type;
 	struct _starpu_cluster_parameters *params;
-	struct starpu_cluster_machine *machine = malloc(sizeof(struct starpu_cluster_machine));
+	struct starpu_cluster_machine *machine;
+	STARPU_MALLOC(machine, sizeof(struct starpu_cluster_machine));
 
-	machine->params = malloc(sizeof(struct _starpu_cluster_parameters));
+	STARPU_MALLOC(machine->params, sizeof(struct _starpu_cluster_parameters));
 	machine->id = STARPU_NMAX_SCHED_CTXS;
 	machine->groups = _starpu_cluster_group_list_new();
 	machine->nclusters = 0;
@@ -341,9 +342,8 @@ void _starpu_cluster_group_init(struct _starpu_cluster_group *group,
 	group->nclusters = 0;
 	group->clusters = _starpu_cluster_list_new();
 	group->father = father;
-	group->params = malloc(sizeof(struct _starpu_cluster_parameters));
-	_starpu_cluster_copy_parameters(group->params,
-					father->params);
+	STARPU_MALLOC(group->params, sizeof(struct _starpu_cluster_parameters));
+	_starpu_cluster_copy_parameters(group->params, father->params);
 	return;
 }
 
@@ -356,9 +356,8 @@ void _starpu_cluster_init(struct _starpu_cluster *cluster,
 	cluster->cores = NULL;
 	cluster->workerids = NULL;
 	cluster->father = father;
-	cluster->params = malloc(sizeof(struct _starpu_cluster_parameters));
-	_starpu_cluster_copy_parameters(cluster->params,
-					father->params);
+	STARPU_MALLOC(cluster->params, sizeof(struct _starpu_cluster_parameters));
+	_starpu_cluster_copy_parameters(cluster->params, father->params);
 }
 
 int _starpu_cluster_remove(struct _starpu_cluster_list *cluster_list,
@@ -540,7 +539,8 @@ int _starpu_cluster_topology(hwloc_obj_type_t cluster_level,
 	int nworkers = starpu_worker_get_count_by_type(STARPU_CPU_WORKER);
 	if (nworkers == 0)
 		return -ENODEV;
-	int *workers = (int*) malloc(sizeof(int) * nworkers);
+	int *workers;
+	STARPU_MALLOC(workers, sizeof(int) * nworkers);
 	starpu_worker_get_ids_by_type(STARPU_CPU_WORKER, workers, nworkers);
 
 	struct _starpu_machine_config *config = _starpu_get_machine_config();
@@ -628,8 +628,8 @@ void _starpu_cluster(struct _starpu_cluster_group *group)
 
 		if (cluster->ncores > 0)
 		{
-			cluster->cores = malloc(sizeof(int)*cluster->ncores);
-			cluster->workerids = malloc(sizeof(int)*cluster->ncores);
+			STARPU_MALLOC(cluster->cores, sizeof(int)*cluster->ncores);
+			STARPU_MALLOC(cluster->workerids, sizeof(int)*cluster->ncores);
 			avail_pus -= cluster->ncores;
 			npreset++;
 		}
@@ -660,8 +660,8 @@ void _starpu_cluster(struct _starpu_cluster_group *group)
 					avail_pus/(group->nclusters-i);
 		}
 		avail_pus -= cluster->ncores;
-		cluster->cores = malloc(sizeof(int)*cluster->ncores);
-		cluster->workerids = malloc(sizeof(int)*cluster->ncores);
+		STARPU_MALLOC(cluster->cores, sizeof(int)*cluster->ncores);
+		STARPU_MALLOC(cluster->workerids, sizeof(int)*cluster->ncores);
 
 		cluster = _starpu_cluster_list_next(cluster);
 	}
@@ -699,8 +699,8 @@ void _starpu_cluster(struct _starpu_cluster_group *group)
 				starpu_cluster_warned = 1;
 			}
 			cluster->ncores += size-1;
-			cluster->cores = realloc(cluster->cores, sizeof(int)*cluster->ncores);
-			cluster->workerids = realloc(cluster->workerids, sizeof(int)*cluster->ncores);
+			STARPU_REALLOC(cluster->cores, sizeof(int)*cluster->ncores);
+			STARPU_REALLOC(cluster->workerids, sizeof(int)*cluster->ncores);
 		}
 
 		/* grab workerid list and return first cpu */

+ 2 - 1
src/util/starpu_data_cpy.c

@@ -152,7 +152,8 @@ int _starpu_data_cpy(starpu_data_handle_t dst_handle, starpu_data_handle_t src_h
 
 	task->cl = &copy_cl;
 
-	unsigned *interface_id = malloc(sizeof(*interface_id));
+	unsigned *interface_id;
+	STARPU_MALLOC(interface_id, sizeof(*interface_id));
 	*interface_id = dst_handle->ops->interfaceid; 
 	task->cl_arg = interface_id;
 	task->cl_arg_size = sizeof(*interface_id);

+ 7 - 5
src/util/starpu_task_insert_utils.c

@@ -30,7 +30,7 @@ static void _starpu_pack_arguments(size_t *current_offset, size_t *arg_buffer_si
 			*arg_buffer_size_ = 128 + sizeof(ptr_size) + ptr_size;
 		else
 			*arg_buffer_size_ = 2 * *arg_buffer_size_ + sizeof(ptr_size) + ptr_size;
-		*arg_buffer_ = realloc(*arg_buffer_, *arg_buffer_size_);
+		STARPU_REALLOC(*arg_buffer_, *arg_buffer_size_);
 	}
 	memcpy(*arg_buffer_+*current_offset, (void *)&ptr_size, sizeof(ptr_size));
 	*current_offset += sizeof(ptr_size);
@@ -186,14 +186,14 @@ void _starpu_task_insert_check_nb_buffers(struct starpu_codelet *cl, struct star
 			int i;
 			struct starpu_codelet *cl2 = (*task)->cl;
 			*allocated_buffers = STARPU_NMAXBUFS * 2;
-			(*task)->dyn_handles = malloc(*allocated_buffers * sizeof(starpu_data_handle_t));
+			STARPU_MALLOC((*task)->dyn_handles, *allocated_buffers * sizeof(starpu_data_handle_t));
 			for(i=0 ; i<current_buffer ; i++)
 			{
 				(*task)->dyn_handles[i] = (*task)->handles[i];
 			}
 			if (cl2->nbuffers == STARPU_VARIABLE_NBUFFERS || !cl2->dyn_modes)
 			{
-				(*task)->dyn_modes = malloc(*allocated_buffers * sizeof(enum starpu_data_access_mode));
+				STARPU_MALLOC((*task)->dyn_modes, *allocated_buffers * sizeof(enum starpu_data_access_mode));
 				for(i=0 ; i<current_buffer ; i++)
 				{
 					(*task)->dyn_modes[i] = (*task)->modes[i];
@@ -203,9 +203,11 @@ void _starpu_task_insert_check_nb_buffers(struct starpu_codelet *cl, struct star
 		else if (current_buffer >= *allocated_buffers)
 		{
 			*allocated_buffers *= 2;
-			(*task)->dyn_handles = realloc((*task)->dyn_handles, *allocated_buffers * sizeof(starpu_data_handle_t));
+			STARPU_REALLOC((*task)->dyn_handles, *allocated_buffers * sizeof(starpu_data_handle_t));
 			if (cl->nbuffers == STARPU_VARIABLE_NBUFFERS || !cl->dyn_modes)
-				(*task)->dyn_modes = realloc((*task)->dyn_modes, *allocated_buffers * sizeof(enum starpu_data_access_mode));
+			{
+				STARPU_REALLOC((*task)->dyn_modes, *allocated_buffers * sizeof(enum starpu_data_access_mode));
+			}
 		}
 	}
 }

+ 7 - 3
src/worker_collection/worker_list.c

@@ -234,9 +234,13 @@ static void _init_workers(int *workerids)
 
 static void list_init(struct starpu_worker_collection *workers)
 {
-	int *workerids = (int*)malloc((STARPU_NMAXWORKERS+STARPU_NMAX_COMBINEDWORKERS) * sizeof(int));
-	int *unblocked_workers = (int*)malloc((STARPU_NMAXWORKERS+STARPU_NMAX_COMBINEDWORKERS) * sizeof(int));
-	int *masters = (int*)malloc((STARPU_NMAXWORKERS+STARPU_NMAX_COMBINEDWORKERS) * sizeof(int));
+	int *workerids;
+	int *unblocked_workers;
+	int *masters;
+
+	STARPU_MALLOC(workerids, (STARPU_NMAXWORKERS+STARPU_NMAX_COMBINEDWORKERS) * sizeof(int));
+	STARPU_MALLOC(unblocked_workers, (STARPU_NMAXWORKERS+STARPU_NMAX_COMBINEDWORKERS) * sizeof(int));
+	STARPU_MALLOC(masters, (STARPU_NMAXWORKERS+STARPU_NMAX_COMBINEDWORKERS) * sizeof(int));
 	_init_workers(workerids);
 	_init_workers(unblocked_workers);
 	_init_workers(masters);

+ 1 - 1
src/worker_collection/worker_tree.c

@@ -287,7 +287,7 @@ static int tree_remove(struct starpu_worker_collection *workers, int worker)
 
 static void tree_init(struct starpu_worker_collection *workers)
 {
-	workers->workerids = (int*)malloc((STARPU_NMAXWORKERS+STARPU_NMAX_COMBINEDWORKERS) * sizeof(int));
+	STARPU_MALLOC(workers->workerids, (STARPU_NMAXWORKERS+STARPU_NMAX_COMBINEDWORKERS) * sizeof(int));
 	workers->collection_private = (void*)starpu_workers_get_tree();
 	workers->nworkers = 0;