Kaynağa Gözat

Merge branch 'master' into starpurm

Olivier Aumage 7 yıl önce
ebeveyn
işleme
9b411391e9

+ 31 - 1
doc/doxygen/chapters/210_check_list_performance.doxy

@@ -2,7 +2,7 @@
  *
  * Copyright (C) 2011-2013,2015,2017                      Inria
  * Copyright (C) 2010-2018                                CNRS
- * Copyright (C) 2009-2011,2013-2017                      Université de Bordeaux
+ * Copyright (C) 2009-2011,2013-2018                      Université de Bordeaux
  *
  * 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
@@ -26,6 +26,26 @@ performance, we give below a list of features which should be checked.
 For a start, you can use \ref OfflinePerformanceTools to get a Gantt chart which
 will show roughly where time is spent, and focus correspondingly.
 
+\section CheckTaskSize Check Task Size
+
+Make sure that your tasks are not too small, because the StarPU runtime overhead
+is not completely zero. You can run the tasks_size_overhead.sh script to get an
+idea of the scalability of tasks depending on their duration (in µs), on your
+own system.
+
+Typically, 10µs-ish tasks are definitely too small, the CUDA overhead itself is
+much bigger than this.
+
+1ms-ish tasks may be a good start, but will not necessarily scale to many dozens
+of cores, so it's better to try to get 10ms-ish tasks.
+
+Tasks durations can easily be observed when performance models are defined (see
+\ref PerformanceModelExample) by using the <c>starpu_perfmodel_plot</c> or
+<c>starpu_perfmodel_display</c> tool (see \ref PerformanceOfCodelets)
+
+When using parallel tasks, the problem is even worse since StarPU has to
+synchronize the execution of tasks.
+
 \section ConfigurationImprovePerformance Configuration Which May Improve Performance
 
 The \ref enable-fast "--enable-fast" configuration option disables all
@@ -116,6 +136,16 @@ enabled by setting the environment variable \ref STARPU_NWORKER_PER_CUDA to the
 number of kernels to execute concurrently.  This is useful when kernels are
 small and do not feed the whole GPU with threads to run.
 
+Concerning memory allocation, you should really not use cudaMalloc/cudaFree
+within the kernel, since cudaFree introduces a awfully lot of synchronizations
+within CUDA itself. You should instead add a parameter to the codelet with the
+STARPU_SCRATCH mode access. You can then pass to the task a handle registered
+with the desired size but with the NULL pointer, that handle can even be the
+shared between tasks, StarPU will allocate per-task data on the fly before task
+execution, and reuse the allocated data between tasks.
+
+See <c>examples/pi/pi_redux.c</c> for an example of use.
+
 \section OpenCL-specificOptimizations OpenCL-specific Optimizations
 
 If the kernel can be made to only use the StarPU-provided command queue or other self-allocated

+ 9 - 1
doc/doxygen/chapters/501_environment_variables.doxy

@@ -1,7 +1,7 @@
 /* StarPU --- Runtime system for heterogeneous multicore architectures.
  *
  * Copyright (C) 2011-2013,2015-2017                      Inria
- * Copyright (C) 2010-2017                                CNRS
+ * Copyright (C) 2010-2018                                CNRS
  * Copyright (C) 2009-2011,2013-2017                      Université de Bordeaux
  * Copyright (C) 2016                                     Uppsala University
  *
@@ -1173,6 +1173,14 @@ If StarPU doesn't find any NUMA node after these step, STARPU_MAIN_MEMORY is the
 discovered by StarPU.
 </dd>
 
+<dt>STARPU_IDLE_FILE</dt>
+<dd>
+\anchor STARPU_IDLE_FILE
+\addindex __env__STARPU_IDLE_FILE
+If the environment variable STARPU_IDLE_FILE is defined, a file named after its contents will be created at the end of the execution.
+The file will contain the sum of the idle times of all the workers.
+</dd>
+
 </dl>
 
 \section ConfiguringTheHypervisor Configuring The Hypervisor

+ 1 - 1
src/common/fxt.h

@@ -620,7 +620,7 @@ do {									\
 		}							\
 		const size_t __job_size = _starpu_job_get_data_size((job)->task->cl?(job)->task->cl->model:NULL, perf_arch, nimpl, (job));	\
 		const uint32_t __job_hash = _starpu_compute_buffers_footprint((job)->task->cl?(job)->task->cl->model:NULL, perf_arch, nimpl, (job));\
-		FUT_DO_PROBE7(_STARPU_FUT_CODELET_DETAILS, ((job)->task)->sched_ctx, __job_size, __job_hash, (job)->task->flops / 1000, (job)->task->tag_id, workerid, ((job)->job_id)); \
+		FUT_DO_PROBE7(_STARPU_FUT_CODELET_DETAILS, ((job)->task)->sched_ctx, __job_size, __job_hash, (job)->task->flops / 1000 / ((job)->task->cl && job->task->cl->type != STARPU_SEQ ? j->task_size : 1), (job)->task->tag_id, workerid, ((job)->job_id)); \
 	}								\
 } while(0);
 

+ 6 - 1
src/common/prio_list.h

@@ -167,7 +167,12 @@
 	{ \
 		/* Sort by decreasing order */ \
 		const struct ENAME##_prio_list_stage *e2 = ENAME##_node_to_list_stage_const(node); \
-		return (e2->prio - prio); \
+		if (e2->prio < prio) \
+			return -1; \
+		if (e2->prio == prio) \
+			return 0; \
+		/* e2->prio > prio */ \
+		return 1; \
 	} \
 	PRIO_LIST_INLINE struct ENAME##_prio_list_stage *ENAME##_prio_list_add(struct ENAME##_prio_list *priolist, int prio) \
 	{ \

+ 1 - 1
src/core/perfmodel/perfmodel_history.c

@@ -1765,7 +1765,7 @@ void _starpu_update_perfmodel_history(struct _starpu_job *j, struct starpu_perfm
 
 					unsigned n = entry->nsample;
 					entry->mean = entry->sum / n;
-					entry->deviation = sqrt((fabs(entry->sum2 - (entry->sum*entry->sum))/n)/n);
+					entry->deviation = sqrt((fabs(entry->sum2 - (entry->sum*entry->sum)/n))/n);
 				}
 
 				if (j->task->flops != 0.)

+ 2 - 0
src/core/sched_policy.c

@@ -397,6 +397,7 @@ static int _starpu_push_task_on_specific_worker(struct starpu_task *task, int wo
 			struct starpu_task *alias = starpu_task_dup(task);
 			alias->destroy = 1;
 
+			_STARPU_TRACE_JOB_PUSH(alias, alias->priority > 0);
 			worker = _starpu_get_worker_struct(combined_workerid[j]);
 			ret |= _starpu_push_local_task(worker, alias, 0);
 		}
@@ -581,6 +582,7 @@ int _starpu_push_task_to_workers(struct starpu_task *task)
 					if (job->task_size > 1)
 					{
 						alias = starpu_task_dup(task);
+						_STARPU_TRACE_JOB_PUSH(alias, alias->priority > 0);
 						alias->destroy = 1;
 					}
 					else

+ 11 - 1
src/drivers/cpu/driver_cpu.c

@@ -107,12 +107,22 @@ static int execute_job_on_cpu(struct _starpu_job *j, struct starpu_task *worker_
 			/* rebind to single CPU */
 			_starpu_bind_thread_on_cpu(cpu_args->bindid, cpu_args->workerid);
 	}
+	else
+	{
+		_STARPU_TRACE_START_EXECUTING();
+	}
+
+	if (is_parallel_task)
+	{
+		STARPU_PTHREAD_BARRIER_WAIT(&j->after_work_barrier);
+		if (rank != 0)
+			_STARPU_TRACE_END_EXECUTING();
+	}
 
 	_starpu_driver_end_job(cpu_args, j, perf_arch, rank, profiling);
 
 	if (is_parallel_task)
 	{
-		STARPU_PTHREAD_BARRIER_WAIT(&j->after_work_barrier);
 #ifdef STARPU_SIMGRID
 		if (rank == 0)
 		{

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

@@ -62,6 +62,7 @@ static nvmlDevice_t nvmlDev[STARPU_MAXCUDADEVS];
 int _starpu_cuda_bus_ids[STARPU_MAXCUDADEVS+STARPU_MAXNUMANODES][STARPU_MAXCUDADEVS+STARPU_MAXNUMANODES];
 #ifdef STARPU_USE_CUDA
 static cudaStream_t streams[STARPU_NMAXWORKERS];
+static char used_stream[STARPU_NMAXWORKERS];
 static cudaStream_t out_transfer_streams[STARPU_MAXCUDADEVS];
 static cudaStream_t in_transfer_streams[STARPU_MAXCUDADEVS];
 /* Note: streams are not thread-safe, so we define them for each CUDA worker
@@ -224,6 +225,7 @@ cudaStream_t starpu_cuda_get_local_stream(void)
 {
 	int worker = starpu_worker_get_id_check();
 
+	used_stream[worker] = 1;
 	return streams[worker];
 }
 
@@ -613,6 +615,14 @@ static void execute_job_on_cuda(struct starpu_task *task, struct _starpu_worker
 		}
 	}
 
+#ifndef STARPU_SIMGRID
+	if (!used_stream[workerid])
+	{
+		used_stream[workerid] = 1;
+		_STARPU_DISP("Warning: starpu_cuda_get_local_stream() was not used to submit kernel to CUDA on worker %d. CUDA will thus introduce a lot of useless synchronizations, which will prevent proper overlapping of data transfers and kernel execution. See the CUDA-specific part of the 'Check List When Performance Are Not There' of the StarPU handbook\n", workerid);
+	}
+#endif
+
 	if (task->cl->cuda_flags[j->nimpl] & STARPU_CUDA_ASYNC)
 	{
 		if (worker->pipeline_length == 0)

+ 3 - 1
src/sched_policies/component_worker.c

@@ -2,7 +2,7 @@
  *
  * Copyright (C) 2011-2014,2017                           Inria
  * Copyright (C) 2010-2012,2014-2017                      CNRS
- * Copyright (C) 2010-2017                                Université de Bordeaux
+ * Copyright (C) 2010-2018                                Université de Bordeaux
  * Copyright (C) 2011                                     Télécom-SudParis
  * Copyright (C) 2013                                     Simon Archipoff
  *
@@ -631,6 +631,7 @@ static int combined_worker_push_task(struct starpu_sched_component * component,
 	task_alias[0]->task->destroy = 1;
 	task_alias[0]->left = NULL;
 	task_alias[0]->ntasks = combined_worker->worker_size;
+	_STARPU_TRACE_JOB_PUSH(task_alias[0]->task, task_alias[0]->task->priority > 0);
 	int i;
 	for(i = 1; i < combined_worker->worker_size; i++)
 	{
@@ -641,6 +642,7 @@ static int combined_worker_push_task(struct starpu_sched_component * component,
 		task_alias[i]->left = task_alias[i-1];
 		task_alias[i - 1]->right = task_alias[i];
 		task_alias[i]->pntasks = &(task_alias[0]->ntasks);
+		_STARPU_TRACE_JOB_PUSH(task_alias[i]->task, task_alias[i]->task->priority > 0);
 	}
 
 	starpu_pthread_mutex_t * mutex_to_unlock = NULL;

+ 3 - 0
src/sched_policies/parallel_eager.c

@@ -342,6 +342,7 @@ static struct starpu_task *pop_task_peager_policy(unsigned sched_ctx_id)
 		struct starpu_task *alias = starpu_task_dup(task);
 		int local_worker = combined_workerid[i];
 		alias->destroy = 1;
+		_STARPU_TRACE_JOB_PUSH(alias, alias->priority > 0);
 		_starpu_fifo_push_task(data->local_fifo[local_worker], alias);
 	}
 
@@ -352,6 +353,8 @@ static struct starpu_task *pop_task_peager_policy(unsigned sched_ctx_id)
 
 	STARPU_PTHREAD_MUTEX_UNLOCK(&data->policy_mutex);
 
+	_STARPU_TRACE_JOB_PUSH(master_alias, master_alias->priority > 0);
+
 	for (i = 1; i < worker_size; i++)
 	{
 		int local_worker = combined_workerid[i];

+ 2 - 1
src/sched_policies/parallel_heft.c

@@ -1,7 +1,7 @@
 /* StarPU --- Runtime system for heterogeneous multicore architectures.
  *
  * Copyright (C) 2011-2013,2015,2017                      Inria
- * Copyright (C) 2010-2017                                Université de Bordeaux
+ * Copyright (C) 2010-2018                                Université de Bordeaux
  * Copyright (C) 2011-2017                                CNRS
  * Copyright (C) 2011                                     Télécom-SudParis
  *
@@ -175,6 +175,7 @@ static int push_task_on_best_worker(struct starpu_task *task, int best_workerid,
 			ntasks[local_combined_workerid]++;
 			_starpu_worker_unlock(local_combined_workerid);
 
+			_STARPU_TRACE_JOB_PUSH(alias, alias->priority > 0);
 			ret |= starpu_push_local_task(local_combined_workerid, alias, prio);
 		}