linux-kernel.vger.kernel.org archive mirror
 help / color / mirror / Atom feed
* [PATCH RFC] select_idle_sibling experiments
@ 2016-04-05 18:08 Chris Mason
  2016-04-05 18:43 ` Bastien Bastien Philbert
                   ` (4 more replies)
  0 siblings, 5 replies; 80+ messages in thread
From: Chris Mason @ 2016-04-05 18:08 UTC (permalink / raw)
  To: Peter Zijlstra, Ingo Molnar, Matt Fleming, Mike Galbraith, linux-kernel

[-- Attachment #1: Type: text/plain, Size: 21047 bytes --]

Hi everyone,

We're porting the fb kernel up to 4.5, and one of our last few out-of-tree
patches is a hack to try harder to find idle cpus when waking up tasks.
This helps in pretty much every workload we run, mostly because they all
get tuned with a similar setup:

1) find the load where latencies stop being acceptable
2) Run the server at just a little less than that

Usually this means our CPUs are just a little bit idle, and a poor
scheduler decision to place a task on a busy CPU instead of an idle CPU
ends up impacting our p99 latencies.

Mike helped us with this last year, fixing up wake_wide() to improve
things.  But we still ended up having to go back to the old hack.

I started with a small-ish program to benchmark wakeup latencies.  The
basic idea is a bunch of worker threads who sit around and burn CPU.
Every once and a while they send a message to a message thread.

The message thread records the time he woke up the worker, and the
worker records the delta between that time and the time he actually got
the CPU again.  At the end it spits out a latency histogram.  The only
thing we record is the wakeup latency, there's no measurement of 'work
done' or any of the normal things you'd expect in a benchmark.

It has knobs for cpu think time, and for how long the messenger thread
waits before replying.  Here's how I'm running it with my patch:

./schbench -c 30000 -s 30000 -m 6 -t 24 -r 30
Latency percentiles (usec)
        50.0000th: 50
        75.0000th: 62
        90.0000th: 73
        95.0000th: 79
        *99.0000th: 99
        99.5000th: 761
        99.9000th: 10160
        Over=0, min=0, max=14659

This translates to cputime of 30ms, sleep time of 30ms, 6 messenger
threads, 24 workers per messenger and a run time of 30 seconds.  My box
has two sockets, 24 cores each.  Mainline varies a bit, but numbers like
this are typical:

 ./schbench -c 30000 -s 30000 -m 6 -t 24 -r 30
Latency percentiles (usec)
        50.0000th: 50
        75.0000th: 63
        90.0000th: 76
        95.0000th: 85
        *99.0000th: 4680
        99.5000th: 10192
        99.9000th: 10928
        Over=0, min=0, max=21816

A high p99 in real application performance will block a new kernel for
us.  p99.5 and p99.9 are included just to show how long the tail really
is.

I've inlined schbench.c below and attached as a .gz file just in case
exchange manages to munge it.

Now, on to the patch.  I pushed some code around and narrowed the
problem down to select_idle_sibling()   We have cores going into and out
of idle fast enough that even this cut our latencies in half:

static int select_idle_sibling(struct task_struct *p, int target)
                                goto next;
 
                        for_each_cpu(i, sched_group_cpus(sg)) {
-                               if (i == target || !idle_cpu(i))
+                               if (!idle_cpu(i))
                                        goto next;
                        }
 
IOW, by the time we get down to for_each_cpu(), the idle_cpu() check
done at the top of the function is no longer valid.

I tried a few variations on select_idle_sibling() that preserved the
underlying goal of returning idle cores before idle SMT threads.  They
were all horrible in different ways, and none of them were fast.

The patch below just makes select_idle_sibling pick the first idle
thread it can find.  When I ran it through production workloads here, it
was faster than the patch we've been carrying around for the last few
years.


diff --git a/kernel/sched/fair.c b/kernel/sched/fair.c
index 56b7d4b..c41baa6 100644
--- a/kernel/sched/fair.c
+++ b/kernel/sched/fair.c
@@ -4974,7 +4974,6 @@ find_idlest_cpu(struct sched_group *group, struct task_struct *p, int this_cpu)
 static int select_idle_sibling(struct task_struct *p, int target)
 {
 	struct sched_domain *sd;
-	struct sched_group *sg;
 	int i = task_cpu(p);
 
 	if (idle_cpu(target))
@@ -4990,24 +4989,14 @@ static int select_idle_sibling(struct task_struct *p, int target)
 	 * Otherwise, iterate the domains and find an elegible idle cpu.
 	 */
 	sd = rcu_dereference(per_cpu(sd_llc, target));
-	for_each_lower_domain(sd) {
-		sg = sd->groups;
-		do {
-			if (!cpumask_intersects(sched_group_cpus(sg),
-						tsk_cpus_allowed(p)))
-				goto next;
-
-			for_each_cpu(i, sched_group_cpus(sg)) {
-				if (i == target || !idle_cpu(i))
-					goto next;
-			}
+	if (!sd)
+		goto done;
 
-			target = cpumask_first_and(sched_group_cpus(sg),
-					tsk_cpus_allowed(p));
+	for_each_cpu_and(i, sched_domain_span(sd), &p->cpus_allowed) {
+		if (cpu_active(i) && idle_cpu(i)) {
+			target = i;
 			goto done;
-next:
-			sg = sg->next;
-		} while (sg != sd->groups);
+		}
 	}
 done:
 	return target;

--------------------------------------------

/*
 * schbench.c
 *
 * Copyright (C) 2016 Facebook
 * Chris Mason <clm@fb.com>
 *
 * GPLv2, portions copied from the kernel and from Jens Axboe's fio
 *
 * gcc -Wall -O0 -W schbench.c -o schbench -lpthread
 */
#include <stdio.h>
#include <stdlib.h>
#include <pthread.h>
#include <fcntl.h>
#include <unistd.h>
#include <errno.h>
#include <getopt.h>
#include <sys/time.h>
#include <time.h>
#include <string.h>
#include <linux/futex.h>
#include <sys/syscall.h>

#define PLAT_BITS	8
#define PLAT_VAL	(1 << PLAT_BITS)
#define PLAT_GROUP_NR	19
#define PLAT_NR		(PLAT_GROUP_NR * PLAT_VAL)
#define PLAT_LIST_MAX	20

/* -m number of message threads */
static int message_threads = 2;
/* -t  number of workers per message thread */
static int worker_threads = 16;
/* -r  seconds */
static int runtime = 30;
/* -s  usec */
static int sleeptime = 30000;
/* -c  usec */
static unsigned long long cputime = 30000;
/* -a, bool */
static int autobench = 0;

/* the latency histogram uses this to pitch outliers */
static unsigned int max_us = 50000;

/* main() sets this to the time when we should all stop doing work */
static struct timeval global_stop;

/* the message threads flip this to true when they decide runtime is up */
static unsigned long stopping = 0;


/*
 * one stat struct per thread data, when the workers sleep this records the
 * latency between when they are woken up and when they actually get the
 * CPU again.  The message threads sum up the stats of all the workers and
 * then bubble them up to main() for printing
 */
struct stats {
	unsigned int plat[PLAT_NR];
	unsigned int nr_samples;
	unsigned int max;
	unsigned int min;
	unsigned int over;
};

/* this defines which latency profiles get printed */
#define PLIST_P99 4
static double plist[PLAT_LIST_MAX] = { 50.0, 75.0, 90.0, 95.0, 99.0, 99.5, 99.9 };

enum {
	HELP_LONG_OPT = 1,
};

char *option_string = "am:t:s:c:r:";
static struct option long_options[] = {
	{"auto", no_argument, 0, 'a'},
	{"message-threads", required_argument, 0, 'm'},
	{"threads", required_argument, 0, 't'},
	{"runtime", required_argument, 0, 'r'},
	{"sleeptime", required_argument, 0, 's'},
	{"cputime", required_argument, 0, 'c'},
	{"help", no_argument, 0, HELP_LONG_OPT},
	{0, 0, 0, 0}
};

static void print_usage(void)
{
	fprintf(stderr, "schbench usage:\n"
		"\t-d (--dispatch-threads): number of message threads (def: 2)\n"
		"\t-t (--threads): worker threads per message thread (def: 16)\n"
		"\t-r (--runtime): How long to run before exiting (seconds, def: 30)\n"
		"\t-s (--sleeptime): Message thread latency (usec, def: 10000\n"
		"\t-c (--cputime): How long to think during loop (usec, def: 10000\n"
	       );
	exit(1);
}

static void parse_options(int ac, char **av)
{
	int c;

	while (1) {
		int option_index = 0;

		c = getopt_long(ac, av, option_string,
				long_options, &option_index);

		if (c == -1)
			break;

		switch(c) {
		case 'a':
			autobench = 1;
			break;
		case 's':
			sleeptime = atoi(optarg);
			break;
		case 'c':
			cputime = atoi(optarg);
			break;
		case 'd':
			message_threads = atoi(optarg);
			break;
		case 't':
			worker_threads = atoi(optarg);
			break;
		case 'r':
			runtime = atoi(optarg);
			break;
		case '?':
		case HELP_LONG_OPT:
			print_usage();
			break;
		default:
			break;
		}
	}

	if (optind < ac) {
		fprintf(stderr, "Error Extra arguments '%s'\n", av[optind]);
		exit(1);
	}
}

void tvsub(struct timeval * tdiff, struct timeval * t1, struct timeval * t0)
{
	tdiff->tv_sec = t1->tv_sec - t0->tv_sec;
	tdiff->tv_usec = t1->tv_usec - t0->tv_usec;
	if (tdiff->tv_usec < 0 && tdiff->tv_sec > 0) {
		tdiff->tv_sec--;
		tdiff->tv_usec += 1000000;
		if (tdiff->tv_usec < 0) {
			fprintf(stderr, "lat_fs: tvsub shows test time ran backwards!\n");
			exit(1);
		}
	}

	/* time shouldn't go backwards!!! */
	if (tdiff->tv_usec < 0 || t1->tv_sec < t0->tv_sec) {
		tdiff->tv_sec = 0;
		tdiff->tv_usec = 0;
	}
}

/*
 * returns the difference between start and stop in usecs.  Negative values
 * are turned into 0
 */
unsigned long long tvdelta(struct timeval *start, struct timeval *stop)
{
	struct timeval td;
	unsigned long long usecs;

	tvsub(&td, stop, start);
	usecs = td.tv_sec;
	usecs *= 1000000;
	usecs += td.tv_usec;
	return (usecs);
}

/* mr axboe's magic latency histogram */
static unsigned int plat_val_to_idx(unsigned int val)
{
	unsigned int msb, error_bits, base, offset;

	/* Find MSB starting from bit 0 */
	if (val == 0)
		msb = 0;
	else
		msb = sizeof(val)*8 - __builtin_clz(val) - 1;

	/*
	 * MSB <= (PLAT_BITS-1), cannot be rounded off. Use
	 * all bits of the sample as index
	 */
	if (msb <= PLAT_BITS)
		return val;

	/* Compute the number of error bits to discard*/
	error_bits = msb - PLAT_BITS;

	/* Compute the number of buckets before the group */
	base = (error_bits + 1) << PLAT_BITS;

	/*
	 * Discard the error bits and apply the mask to find the
	 * index for the buckets in the group
	 */
	offset = (PLAT_VAL - 1) & (val >> error_bits);

	/* Make sure the index does not exceed (array size - 1) */
	return (base + offset) < (PLAT_NR - 1) ?
		(base + offset) : (PLAT_NR - 1);
}

/*
 * Convert the given index of the bucket array to the value
 * represented by the bucket
 */
static unsigned int plat_idx_to_val(unsigned int idx)
{
	unsigned int error_bits, k, base;

	if (idx >= PLAT_NR) {
		fprintf(stderr, "idx %u is too large\n", idx);
		exit(1);
	}

	/* MSB <= (PLAT_BITS-1), cannot be rounded off. Use
	 * all bits of the sample as index */
	if (idx < (PLAT_VAL << 1))
		return idx;

	/* Find the group and compute the minimum value of that group */
	error_bits = (idx >> PLAT_BITS) - 1;
	base = 1 << (error_bits + PLAT_BITS);

	/* Find its bucket number of the group */
	k = idx % PLAT_VAL;

	/* Return the mean of the range of the bucket */
	return base + ((k + 0.5) * (1 << error_bits));
}


static unsigned int calc_percentiles(unsigned int *io_u_plat, unsigned long nr,
				     unsigned int **output)
{
	unsigned long sum = 0;
	unsigned int len, i, j = 0;
	unsigned int oval_len = 0;
	unsigned int *ovals = NULL;
	int is_last;

	len = 0;
	while (len < PLAT_LIST_MAX && plist[len] != 0.0)
		len++;

	if (!len)
		return 0;

	/*
	 * Calculate bucket values, note down max and min values
	 */
	is_last = 0;
	for (i = 0; i < PLAT_NR && !is_last; i++) {
		sum += io_u_plat[i];
		while (sum >= (plist[j] / 100.0 * nr)) {
			if (j == oval_len) {
				oval_len += 100;
				ovals = realloc(ovals, oval_len * sizeof(unsigned int));
			}

			ovals[j] = plat_idx_to_val(i);
			is_last = (j == len - 1);
			if (is_last)
				break;

			j++;
		}
	}

	*output = ovals;
	return len;
}

static int calc_p99(struct stats *s)
{
	unsigned int *ovals = NULL;
	int ret = 0;
	int len;

	len = calc_percentiles(s->plat, s->nr_samples, &ovals);
	if (len && len > PLIST_P99)
		ret = ovals[PLIST_P99];
	if (ovals)
		free(ovals);
	return ret;
}

static void show_latencies(struct stats *s)
{
	unsigned int *ovals = NULL;
	unsigned int len, i;

	len = calc_percentiles(s->plat, s->nr_samples, &ovals);
	if (len) {
		fprintf(stderr, "Latency percentiles (usec)\n");
		for (i = 0; i < len; i++)
			fprintf(stderr, "\t%s%2.4fth: %u\n",
				i == PLIST_P99 ? "*" : "",
				plist[i], ovals[i]);
	}

	if (ovals)
		free(ovals);

	fprintf(stderr, "\tOver=%u, min=%u, max=%u\n", s->over, s->min, s->max);
}

/* fold latency info from s into d */
void combine_stats(struct stats *d, struct stats *s)
{
	int i;
	for (i = 0; i < PLAT_NR; i++)
		d->plat[i] += s->plat[i];
	d->nr_samples += s->nr_samples;
	d->over += s->over;
	if (s->max > d->max)
		d->max = s->max;
	if (s->min < d->min)
		d->min = s->min;
}

/* record a latency result into the histogram */
static void add_lat(struct stats *s, unsigned int us)
{
	int lat_index = 0;

	if (us > s->max)
		s->max = us;
	if (us < s->min)
		s->min = us;

	if (us > max_us) {
		fprintf(stderr, "latency=%u usec\n", us);
		s->over++;
	}

	lat_index = plat_val_to_idx(us);
	__sync_fetch_and_add(&s->plat[lat_index], 1);
	__sync_fetch_and_add(&s->nr_samples, 1);
}

/*
 * every thread has one of these, it comes out to about 19K thanks to the
 * giant stats struct
 */
struct thread_data {
	pthread_t tid;
	/* ->next is for placing us on the msg_thread's list for waking */
	struct thread_data *next;

	/* our parent thread and messaging partner */
	struct thread_data *msg_thread;

	/*
	 * the msg thread stuffs gtod in here before waking us, so we can
	 * measure scheduler latency
	 */
	struct timeval wake_time;

	/* keep the futex and the wake_time in the same cacheline */
	int futex;

	/* mr axboe's magic latency histogram */
	struct stats stats;
};

/* we're so fancy we make our own futex wrappers */
#define FUTEX_BLOCKED 0
#define FUTEX_RUNNING 1

static int futex(int *uaddr, int futex_op, int val,
		 const struct timespec *timeout, int *uaddr2, int val3)
{
	return syscall(SYS_futex, uaddr, futex_op, val, timeout, uaddr2, val3);
}

/*
 * wakeup a process waiting on a futex, making sure they are really waiting
 * first
 */
static void fpost(int *futexp)
{
	int s;

	if (__sync_bool_compare_and_swap(futexp, FUTEX_BLOCKED,
					 FUTEX_RUNNING)) {
		s = futex(futexp, FUTEX_WAKE, 1, NULL, NULL, 0);
		if (s  == -1) {
			perror("FUTEX_WAKE");
			exit(1);
		}
	}
}

/*
 * wait on a futex, with an optional timeout.  Make sure to set
 * the futex to FUTEX_BLOCKED beforehand.
 *
 * This will return zero if all went well, or return -ETIMEDOUT if you
 * hit the timeout without getting posted
 */
static int fwait(int *futexp, struct timespec *timeout)
{
	int s;
	while (1) {
		/* Is the futex available? */
		if (__sync_bool_compare_and_swap(futexp, FUTEX_RUNNING,
						 FUTEX_BLOCKED)) {
			break;      /* Yes */
		}
		/* Futex is not available; wait */
		s = futex(futexp, FUTEX_WAIT, FUTEX_BLOCKED, timeout, NULL, 0);
		if (s == -1 && errno != EAGAIN) {
			if (errno == ETIMEDOUT)
				return -ETIMEDOUT;
			perror("futex-FUTEX_WAIT");
			exit(1);
		}
	}
	return 0;
}

/*
 * cmpxchg based list prepend
 */
static void xlist_add(struct thread_data *head, struct thread_data *add)
{
	struct thread_data *old;
	struct thread_data *ret;

	while (1) {
		old = head->next;
		add->next = old;
		ret = __sync_val_compare_and_swap(&head->next, old, add);
		if (ret == old)
			break;
	}
}

/*
 * xchg based list splicing.  This returns the entire list and
 * replaces the head->next with NULL
 */
static struct thread_data *xlist_splice(struct thread_data *head)
{
	struct thread_data *old;
	struct thread_data *ret;

	while (1) {
		old = head->next;
		ret = __sync_val_compare_and_swap(&head->next, old, NULL);
		if (ret == old)
			break;
	}
	return ret;
}

/*
 * Wake everyone currently waiting on the message list, filling in their
 * thread_data->wake_time with the current time.
 *
 * It's not exactly the current time, it's really the time at the start of
 * the list run.  We want to detect when the scheduler is just preempting the
 * waker and giving away the rest of its timeslice.  So we gtod once at
 * the start of the loop and use that for all the threads we wake.
 */
static void xlist_wake_all(struct thread_data *td)
{
	struct thread_data *list;
	struct thread_data *next;
	struct timeval now;

	list = xlist_splice(td);
	gettimeofday(&now, NULL);
	while (list) {
		next = list->next;
		list->next = NULL;
		memcpy(&list->wake_time, &now, sizeof(now));
		fpost(&list->futex);
		list = next;
	}
}

/*
 * called by worker threads to send a message and wait for the answer.
 * In reality we're just trading one cacheline with the gtod and futex in
 * it, but that's good enough.  We gtod after waking and use that to
 * record scheduler latency.
 */
static void msg_and_wait(struct thread_data *td)
{
	struct timeval now;
	unsigned long long delta;
	struct timespec timeout;

	timeout.tv_sec = 0;
	timeout.tv_nsec = 5000 * 1000;

	/* set ourselves to blocked */
	td->futex = FUTEX_BLOCKED;
	gettimeofday(&td->wake_time, NULL);

	/* add us to the list */
	xlist_add(td->msg_thread, td);

	fpost(&td->msg_thread->futex);

	/*
	 * don't wait if the main threads are shutting down,
	 * they will never kick us fpost has a full barrier, so as long
	 * as the message thread walks his list after setting stopping,
	 * we shouldn't miss the wakeup
	 */
	if (!stopping) {
		/* if he hasn't already woken us up, wait */
		fwait(&td->futex, NULL);
	}

	gettimeofday(&now, NULL);
	delta = tvdelta(&td->wake_time, &now);
	if (delta > 0)
		add_lat(&td->stats, delta);
}

/*
 * once the message thread starts all his children, this is where he
 * loops until our runtime is up.  Basically this sits around waiting
 * for posting by the worker threads, replying to their messages after
 * a delay of 'sleeptime' + some jitter.
 */
static void run_msg_thread(struct thread_data *td)
{
	struct timeval now;
	struct timespec timeout;
	unsigned int seed = pthread_self();
	int max_jitter = sleeptime / 4;
	int jitter;

	jitter = rand_r(&seed) % max_jitter;
	timeout.tv_sec = 0;
	timeout.tv_nsec = (sleeptime + jitter) * 1000;

	while (1) {
		td->futex = FUTEX_BLOCKED;
		xlist_wake_all(td);

		gettimeofday(&now, NULL);
		if (now.tv_sec > global_stop.tv_sec) {
			stopping = 1;
			__sync_synchronize();
			xlist_wake_all(td);
			break;
		}
		fwait(&td->futex, &timeout);

		/*
		 * messages shouldn't be instant, sleep a little to make them
		 * wait
		 */
		jitter = rand_r(&seed) % max_jitter;
		usleep(sleeptime + jitter);
	}
}

#define nop __asm__ __volatile__("rep;nop": : :"memory")

static void usec_spin(unsigned long spin_time)
{
	struct timeval now;
	struct timeval start;
	unsigned long long delta;

	gettimeofday(&start, NULL);
	while (1) {
		gettimeofday(&now, NULL);
		delta = tvdelta(&start, &now);
		if (delta > spin_time)
			return;
		nop;
	}
}

/*
 * the worker thread is pretty simple, it just does a single spin and
 * then waits on a message from the message thread
 */
void *worker_thread(void *arg)
{
	struct thread_data *td = arg;

	while(1) {
		if (stopping)
			break;

		usec_spin(cputime);
		msg_and_wait(td);
	}
	return NULL;
}

/*
 * the message thread starts his own gaggle of workers and then sits around
 * replying when they post him.  He collects latency stats as all the threads
 * exit
 */
void *message_thread(void *arg)
{
	struct thread_data *td = arg;
	struct thread_data *worker_threads_mem = NULL;
	int i;
	int ret;

	worker_threads_mem = calloc(worker_threads, sizeof(struct thread_data));

	if (!worker_threads_mem) {
		perror("unable to allocate ram");
		pthread_exit((void *)-ENOMEM);
	}

	for (i = 0; i < worker_threads; i++) {
		pthread_t tid;

		worker_threads_mem[i].msg_thread = td;
		ret = pthread_create(&tid, NULL, worker_thread,
				     worker_threads_mem + i);
		if (ret) {
			fprintf(stderr, "error %d from pthread_create\n", ret);
			exit(1);
		}
		worker_threads_mem[i].tid = tid;
	}

	run_msg_thread(td);

	for (i = 0; i < worker_threads; i++) {
		pthread_join(worker_threads_mem[i].tid, NULL);
		combine_stats(&td->stats, &worker_threads_mem[i].stats);
	}
	free(worker_threads_mem);

	return NULL;
}

int main(int ac, char **av)
{
	int i;
	int ret;
	struct thread_data *message_threads_mem = NULL;
	struct stats stats;

	parse_options(ac, av);
again:
	stopping = 0;
	memset(&stats, 0, sizeof(stats));

	message_threads_mem = calloc(message_threads,
				      sizeof(struct thread_data));


	if (!message_threads_mem) {
		perror("unable to allocate ram");
		exit(1);
	}
	gettimeofday(&global_stop, NULL);
	global_stop.tv_sec += runtime;

	/* start our message threads, each one starts its own workers */
	for (i = 0; i < message_threads; i++) {
		pthread_t tid;
		ret = pthread_create(&tid, NULL, message_thread,
				     message_threads_mem + i);
		if (ret) {
			fprintf(stderr, "error %d from pthread_create\n", ret);
			exit(1);
		}
		message_threads_mem[i].tid = tid;
	}
	for (i = 0; i < message_threads; i++) {
		pthread_join(message_threads_mem[i].tid, NULL);
		combine_stats(&stats, &message_threads_mem[i].stats);
	}

	free(message_threads_mem);

	/*
	 * in auto bench mode, keep adding workers until our latencies get
	 * horrible
	 */
	if (autobench) {
		int p99 = calc_p99(&stats);
		fprintf(stderr, "cputime %Lu threads %d p99 %d\n",
			cputime, worker_threads, p99);
		if (p99 < 2000) {
			worker_threads++;
			goto again;
		}
	}

	show_latencies(&stats);

	return 0;
}

[-- Attachment #2: schbench.c.gz --]
[-- Type: application/octet-stream, Size: 5518 bytes --]

^ permalink raw reply related	[flat|nested] 80+ messages in thread

* Re: [PATCH RFC] select_idle_sibling experiments
  2016-04-05 18:08 [PATCH RFC] select_idle_sibling experiments Chris Mason
@ 2016-04-05 18:43 ` Bastien Bastien Philbert
  2016-04-05 19:28   ` Chris Mason
  2016-04-05 20:03 ` Matt Fleming
                   ` (3 subsequent siblings)
  4 siblings, 1 reply; 80+ messages in thread
From: Bastien Bastien Philbert @ 2016-04-05 18:43 UTC (permalink / raw)
  To: Chris Mason, Peter Zijlstra, Ingo Molnar, Matt Fleming,
	Mike Galbraith, linux-kernel

On Tue, Apr 5, 2016 at 2:08 PM, Chris Mason <clm@fb.com> wrote:
> Hi everyone,
>
> We're porting the fb kernel up to 4.5, and one of our last few out-of-tree
> patches is a hack to try harder to find idle cpus when waking up tasks.
> This helps in pretty much every workload we run, mostly because they all
> get tuned with a similar setup:
>
> 1) find the load where latencies stop being acceptable
> 2) Run the server at just a little less than that
>
> Usually this means our CPUs are just a little bit idle, and a poor
> scheduler decision to place a task on a busy CPU instead of an idle CPU
> ends up impacting our p99 latencies.
>
> Mike helped us with this last year, fixing up wake_wide() to improve
> things.  But we still ended up having to go back to the old hack.
>
> I started with a small-ish program to benchmark wakeup latencies.  The
> basic idea is a bunch of worker threads who sit around and burn CPU.
> Every once and a while they send a message to a message thread.
>
> The message thread records the time he woke up the worker, and the
> worker records the delta between that time and the time he actually got
> the CPU again.  At the end it spits out a latency histogram.  The only
> thing we record is the wakeup latency, there's no measurement of 'work
> done' or any of the normal things you'd expect in a benchmark.
>
> It has knobs for cpu think time, and for how long the messenger thread
> waits before replying.  Here's how I'm running it with my patch:
>
> ./schbench -c 30000 -s 30000 -m 6 -t 24 -r 30
> Latency percentiles (usec)
>         50.0000th: 50
>         75.0000th: 62
>         90.0000th: 73
>         95.0000th: 79
>         *99.0000th: 99
>         99.5000th: 761
>         99.9000th: 10160
>         Over=0, min=0, max=14659
>
> This translates to cputime of 30ms, sleep time of 30ms, 6 messenger
> threads, 24 workers per messenger and a run time of 30 seconds.  My box
> has two sockets, 24 cores each.  Mainline varies a bit, but numbers like
> this are typical:
>
>  ./schbench -c 30000 -s 30000 -m 6 -t 24 -r 30
> Latency percentiles (usec)
>         50.0000th: 50
>         75.0000th: 63
>         90.0000th: 76
>         95.0000th: 85
>         *99.0000th: 4680
>         99.5000th: 10192
>         99.9000th: 10928
>         Over=0, min=0, max=21816
>
> A high p99 in real application performance will block a new kernel for
> us.  p99.5 and p99.9 are included just to show how long the tail really
> is.
>
> I've inlined schbench.c below and attached as a .gz file just in case
> exchange manages to munge it.
>
> Now, on to the patch.  I pushed some code around and narrowed the
> problem down to select_idle_sibling()   We have cores going into and out
> of idle fast enough that even this cut our latencies in half:
>
> static int select_idle_sibling(struct task_struct *p, int target)
>                                 goto next;
>
>                         for_each_cpu(i, sched_group_cpus(sg)) {
> -                               if (i == target || !idle_cpu(i))
> +                               if (!idle_cpu(i))
>                                         goto next;
>                         }
>
> IOW, by the time we get down to for_each_cpu(), the idle_cpu() check
> done at the top of the function is no longer valid.
>
> I tried a few variations on select_idle_sibling() that preserved the
> underlying goal of returning idle cores before idle SMT threads.  They
> were all horrible in different ways, and none of them were fast.
>
> The patch below just makes select_idle_sibling pick the first idle
> thread it can find.  When I ran it through production workloads here, it
> was faster than the patch we've been carrying around for the last few
> years.
>
>
> diff --git a/kernel/sched/fair.c b/kernel/sched/fair.c
> index 56b7d4b..c41baa6 100644
> --- a/kernel/sched/fair.c
> +++ b/kernel/sched/fair.c
> @@ -4974,7 +4974,6 @@ find_idlest_cpu(struct sched_group *group, struct task_struct *p, int this_cpu)
>  static int select_idle_sibling(struct task_struct *p, int target)
>  {
>         struct sched_domain *sd;
> -       struct sched_group *sg;
>         int i = task_cpu(p);
>
>         if (idle_cpu(target))
> @@ -4990,24 +4989,14 @@ static int select_idle_sibling(struct task_struct *p, int target)
>          * Otherwise, iterate the domains and find an elegible idle cpu.
>          */
>         sd = rcu_dereference(per_cpu(sd_llc, target));
> -       for_each_lower_domain(sd) {
> -               sg = sd->groups;
> -               do {
> -                       if (!cpumask_intersects(sched_group_cpus(sg),
> -                                               tsk_cpus_allowed(p)))
> -                               goto next;
> -
> -                       for_each_cpu(i, sched_group_cpus(sg)) {
> -                               if (i == target || !idle_cpu(i))
> -                                       goto next;
> -                       }
> +       if (!sd)
> +               goto done;
>
> -                       target = cpumask_first_and(sched_group_cpus(sg),
> -                                       tsk_cpus_allowed(p));
> +       for_each_cpu_and(i, sched_domain_span(sd), &p->cpus_allowed) {
> +               if (cpu_active(i) && idle_cpu(i)) {
> +                       target = i;
>                         goto done;
> -next:
> -                       sg = sg->next;
> -               } while (sg != sd->groups);
> +               }
>         }
>  done:
>         return target;
>
> --------------------------------------------
>
Here is my concern, do you test this on standard scheduler workloads
or was this just written for Facebook's
internal workloads. I am going to test this later because frankly this
may cause a regression on my system
which has only 4  cores so a idle CPU is probably less common for a
small amount of time. I am wondering
however if Ingo has any complains before I test this to see if it
causes a regression or a bug on my system.
Ingo do you have any thoughts on this or would you like me to just test this?
Bastien
> /*
>  * schbench.c
>  *
>  * Copyright (C) 2016 Facebook
>  * Chris Mason <clm@fb.com>
>  *
>  * GPLv2, portions copied from the kernel and from Jens Axboe's fio
>  *
>  * gcc -Wall -O0 -W schbench.c -o schbench -lpthread
>  */
> #include <stdio.h>
> #include <stdlib.h>
> #include <pthread.h>
> #include <fcntl.h>
> #include <unistd.h>
> #include <errno.h>
> #include <getopt.h>
> #include <sys/time.h>
> #include <time.h>
> #include <string.h>
> #include <linux/futex.h>
> #include <sys/syscall.h>
>
> #define PLAT_BITS       8
> #define PLAT_VAL        (1 << PLAT_BITS)
> #define PLAT_GROUP_NR   19
> #define PLAT_NR         (PLAT_GROUP_NR * PLAT_VAL)
> #define PLAT_LIST_MAX   20
>
> /* -m number of message threads */
> static int message_threads = 2;
> /* -t  number of workers per message thread */
> static int worker_threads = 16;
> /* -r  seconds */
> static int runtime = 30;
> /* -s  usec */
> static int sleeptime = 30000;
> /* -c  usec */
> static unsigned long long cputime = 30000;
> /* -a, bool */
> static int autobench = 0;
>
> /* the latency histogram uses this to pitch outliers */
> static unsigned int max_us = 50000;
>
> /* main() sets this to the time when we should all stop doing work */
> static struct timeval global_stop;
>
> /* the message threads flip this to true when they decide runtime is up */
> static unsigned long stopping = 0;
>
>
> /*
>  * one stat struct per thread data, when the workers sleep this records the
>  * latency between when they are woken up and when they actually get the
>  * CPU again.  The message threads sum up the stats of all the workers and
>  * then bubble them up to main() for printing
>  */
> struct stats {
>         unsigned int plat[PLAT_NR];
>         unsigned int nr_samples;
>         unsigned int max;
>         unsigned int min;
>         unsigned int over;
> };
>
> /* this defines which latency profiles get printed */
> #define PLIST_P99 4
> static double plist[PLAT_LIST_MAX] = { 50.0, 75.0, 90.0, 95.0, 99.0, 99.5, 99.9 };
>
> enum {
>         HELP_LONG_OPT = 1,
> };
>
> char *option_string = "am:t:s:c:r:";
> static struct option long_options[] = {
>         {"auto", no_argument, 0, 'a'},
>         {"message-threads", required_argument, 0, 'm'},
>         {"threads", required_argument, 0, 't'},
>         {"runtime", required_argument, 0, 'r'},
>         {"sleeptime", required_argument, 0, 's'},
>         {"cputime", required_argument, 0, 'c'},
>         {"help", no_argument, 0, HELP_LONG_OPT},
>         {0, 0, 0, 0}
> };
>
> static void print_usage(void)
> {
>         fprintf(stderr, "schbench usage:\n"
>                 "\t-d (--dispatch-threads): number of message threads (def: 2)\n"
>                 "\t-t (--threads): worker threads per message thread (def: 16)\n"
>                 "\t-r (--runtime): How long to run before exiting (seconds, def: 30)\n"
>                 "\t-s (--sleeptime): Message thread latency (usec, def: 10000\n"
>                 "\t-c (--cputime): How long to think during loop (usec, def: 10000\n"
>                );
>         exit(1);
> }
>
> static void parse_options(int ac, char **av)
> {
>         int c;
>
>         while (1) {
>                 int option_index = 0;
>
>                 c = getopt_long(ac, av, option_string,
>                                 long_options, &option_index);
>
>                 if (c == -1)
>                         break;
>
>                 switch(c) {
>                 case 'a':
>                         autobench = 1;
>                         break;
>                 case 's':
>                         sleeptime = atoi(optarg);
>                         break;
>                 case 'c':
>                         cputime = atoi(optarg);
>                         break;
>                 case 'd':
>                         message_threads = atoi(optarg);
>                         break;
>                 case 't':
>                         worker_threads = atoi(optarg);
>                         break;
>                 case 'r':
>                         runtime = atoi(optarg);
>                         break;
>                 case '?':
>                 case HELP_LONG_OPT:
>                         print_usage();
>                         break;
>                 default:
>                         break;
>                 }
>         }
>
>         if (optind < ac) {
>                 fprintf(stderr, "Error Extra arguments '%s'\n", av[optind]);
>                 exit(1);
>         }
> }
>
> void tvsub(struct timeval * tdiff, struct timeval * t1, struct timeval * t0)
> {
>         tdiff->tv_sec = t1->tv_sec - t0->tv_sec;
>         tdiff->tv_usec = t1->tv_usec - t0->tv_usec;
>         if (tdiff->tv_usec < 0 && tdiff->tv_sec > 0) {
>                 tdiff->tv_sec--;
>                 tdiff->tv_usec += 1000000;
>                 if (tdiff->tv_usec < 0) {
>                         fprintf(stderr, "lat_fs: tvsub shows test time ran backwards!\n");
>                         exit(1);
>                 }
>         }
>
>         /* time shouldn't go backwards!!! */
>         if (tdiff->tv_usec < 0 || t1->tv_sec < t0->tv_sec) {
>                 tdiff->tv_sec = 0;
>                 tdiff->tv_usec = 0;
>         }
> }
>
> /*
>  * returns the difference between start and stop in usecs.  Negative values
>  * are turned into 0
>  */
> unsigned long long tvdelta(struct timeval *start, struct timeval *stop)
> {
>         struct timeval td;
>         unsigned long long usecs;
>
>         tvsub(&td, stop, start);
>         usecs = td.tv_sec;
>         usecs *= 1000000;
>         usecs += td.tv_usec;
>         return (usecs);
> }
>
> /* mr axboe's magic latency histogram */
> static unsigned int plat_val_to_idx(unsigned int val)
> {
>         unsigned int msb, error_bits, base, offset;
>
>         /* Find MSB starting from bit 0 */
>         if (val == 0)
>                 msb = 0;
>         else
>                 msb = sizeof(val)*8 - __builtin_clz(val) - 1;
>
>         /*
>          * MSB <= (PLAT_BITS-1), cannot be rounded off. Use
>          * all bits of the sample as index
>          */
>         if (msb <= PLAT_BITS)
>                 return val;
>
>         /* Compute the number of error bits to discard*/
>         error_bits = msb - PLAT_BITS;
>
>         /* Compute the number of buckets before the group */
>         base = (error_bits + 1) << PLAT_BITS;
>
>         /*
>          * Discard the error bits and apply the mask to find the
>          * index for the buckets in the group
>          */
>         offset = (PLAT_VAL - 1) & (val >> error_bits);
>
>         /* Make sure the index does not exceed (array size - 1) */
>         return (base + offset) < (PLAT_NR - 1) ?
>                 (base + offset) : (PLAT_NR - 1);
> }
>
> /*
>  * Convert the given index of the bucket array to the value
>  * represented by the bucket
>  */
> static unsigned int plat_idx_to_val(unsigned int idx)
> {
>         unsigned int error_bits, k, base;
>
>         if (idx >= PLAT_NR) {
>                 fprintf(stderr, "idx %u is too large\n", idx);
>                 exit(1);
>         }
>
>         /* MSB <= (PLAT_BITS-1), cannot be rounded off. Use
>          * all bits of the sample as index */
>         if (idx < (PLAT_VAL << 1))
>                 return idx;
>
>         /* Find the group and compute the minimum value of that group */
>         error_bits = (idx >> PLAT_BITS) - 1;
>         base = 1 << (error_bits + PLAT_BITS);
>
>         /* Find its bucket number of the group */
>         k = idx % PLAT_VAL;
>
>         /* Return the mean of the range of the bucket */
>         return base + ((k + 0.5) * (1 << error_bits));
> }
>
>
> static unsigned int calc_percentiles(unsigned int *io_u_plat, unsigned long nr,
>                                      unsigned int **output)
> {
>         unsigned long sum = 0;
>         unsigned int len, i, j = 0;
>         unsigned int oval_len = 0;
>         unsigned int *ovals = NULL;
>         int is_last;
>
>         len = 0;
>         while (len < PLAT_LIST_MAX && plist[len] != 0.0)
>                 len++;
>
>         if (!len)
>                 return 0;
>
>         /*
>          * Calculate bucket values, note down max and min values
>          */
>         is_last = 0;
>         for (i = 0; i < PLAT_NR && !is_last; i++) {
>                 sum += io_u_plat[i];
>                 while (sum >= (plist[j] / 100.0 * nr)) {
>                         if (j == oval_len) {
>                                 oval_len += 100;
>                                 ovals = realloc(ovals, oval_len * sizeof(unsigned int));
>                         }
>
>                         ovals[j] = plat_idx_to_val(i);
>                         is_last = (j == len - 1);
>                         if (is_last)
>                                 break;
>
>                         j++;
>                 }
>         }
>
>         *output = ovals;
>         return len;
> }
>
> static int calc_p99(struct stats *s)
> {
>         unsigned int *ovals = NULL;
>         int ret = 0;
>         int len;
>
>         len = calc_percentiles(s->plat, s->nr_samples, &ovals);
>         if (len && len > PLIST_P99)
>                 ret = ovals[PLIST_P99];
>         if (ovals)
>                 free(ovals);
>         return ret;
> }
>
> static void show_latencies(struct stats *s)
> {
>         unsigned int *ovals = NULL;
>         unsigned int len, i;
>
>         len = calc_percentiles(s->plat, s->nr_samples, &ovals);
>         if (len) {
>                 fprintf(stderr, "Latency percentiles (usec)\n");
>                 for (i = 0; i < len; i++)
>                         fprintf(stderr, "\t%s%2.4fth: %u\n",
>                                 i == PLIST_P99 ? "*" : "",
>                                 plist[i], ovals[i]);
>         }
>
>         if (ovals)
>                 free(ovals);
>
>         fprintf(stderr, "\tOver=%u, min=%u, max=%u\n", s->over, s->min, s->max);
> }
>
> /* fold latency info from s into d */
> void combine_stats(struct stats *d, struct stats *s)
> {
>         int i;
>         for (i = 0; i < PLAT_NR; i++)
>                 d->plat[i] += s->plat[i];
>         d->nr_samples += s->nr_samples;
>         d->over += s->over;
>         if (s->max > d->max)
>                 d->max = s->max;
>         if (s->min < d->min)
>                 d->min = s->min;
> }
>
> /* record a latency result into the histogram */
> static void add_lat(struct stats *s, unsigned int us)
> {
>         int lat_index = 0;
>
>         if (us > s->max)
>                 s->max = us;
>         if (us < s->min)
>                 s->min = us;
>
>         if (us > max_us) {
>                 fprintf(stderr, "latency=%u usec\n", us);
>                 s->over++;
>         }
>
>         lat_index = plat_val_to_idx(us);
>         __sync_fetch_and_add(&s->plat[lat_index], 1);
>         __sync_fetch_and_add(&s->nr_samples, 1);
> }
>
> /*
>  * every thread has one of these, it comes out to about 19K thanks to the
>  * giant stats struct
>  */
> struct thread_data {
>         pthread_t tid;
>         /* ->next is for placing us on the msg_thread's list for waking */
>         struct thread_data *next;
>
>         /* our parent thread and messaging partner */
>         struct thread_data *msg_thread;
>
>         /*
>          * the msg thread stuffs gtod in here before waking us, so we can
>          * measure scheduler latency
>          */
>         struct timeval wake_time;
>
>         /* keep the futex and the wake_time in the same cacheline */
>         int futex;
>
>         /* mr axboe's magic latency histogram */
>         struct stats stats;
> };
>
> /* we're so fancy we make our own futex wrappers */
> #define FUTEX_BLOCKED 0
> #define FUTEX_RUNNING 1
>
> static int futex(int *uaddr, int futex_op, int val,
>                  const struct timespec *timeout, int *uaddr2, int val3)
> {
>         return syscall(SYS_futex, uaddr, futex_op, val, timeout, uaddr2, val3);
> }
>
> /*
>  * wakeup a process waiting on a futex, making sure they are really waiting
>  * first
>  */
> static void fpost(int *futexp)
> {
>         int s;
>
>         if (__sync_bool_compare_and_swap(futexp, FUTEX_BLOCKED,
>                                          FUTEX_RUNNING)) {
>                 s = futex(futexp, FUTEX_WAKE, 1, NULL, NULL, 0);
>                 if (s  == -1) {
>                         perror("FUTEX_WAKE");
>                         exit(1);
>                 }
>         }
> }
>
> /*
>  * wait on a futex, with an optional timeout.  Make sure to set
>  * the futex to FUTEX_BLOCKED beforehand.
>  *
>  * This will return zero if all went well, or return -ETIMEDOUT if you
>  * hit the timeout without getting posted
>  */
> static int fwait(int *futexp, struct timespec *timeout)
> {
>         int s;
>         while (1) {
>                 /* Is the futex available? */
>                 if (__sync_bool_compare_and_swap(futexp, FUTEX_RUNNING,
>                                                  FUTEX_BLOCKED)) {
>                         break;      /* Yes */
>                 }
>                 /* Futex is not available; wait */
>                 s = futex(futexp, FUTEX_WAIT, FUTEX_BLOCKED, timeout, NULL, 0);
>                 if (s == -1 && errno != EAGAIN) {
>                         if (errno == ETIMEDOUT)
>                                 return -ETIMEDOUT;
>                         perror("futex-FUTEX_WAIT");
>                         exit(1);
>                 }
>         }
>         return 0;
> }
>
> /*
>  * cmpxchg based list prepend
>  */
> static void xlist_add(struct thread_data *head, struct thread_data *add)
> {
>         struct thread_data *old;
>         struct thread_data *ret;
>
>         while (1) {
>                 old = head->next;
>                 add->next = old;
>                 ret = __sync_val_compare_and_swap(&head->next, old, add);
>                 if (ret == old)
>                         break;
>         }
> }
>
> /*
>  * xchg based list splicing.  This returns the entire list and
>  * replaces the head->next with NULL
>  */
> static struct thread_data *xlist_splice(struct thread_data *head)
> {
>         struct thread_data *old;
>         struct thread_data *ret;
>
>         while (1) {
>                 old = head->next;
>                 ret = __sync_val_compare_and_swap(&head->next, old, NULL);
>                 if (ret == old)
>                         break;
>         }
>         return ret;
> }
>
> /*
>  * Wake everyone currently waiting on the message list, filling in their
>  * thread_data->wake_time with the current time.
>  *
>  * It's not exactly the current time, it's really the time at the start of
>  * the list run.  We want to detect when the scheduler is just preempting the
>  * waker and giving away the rest of its timeslice.  So we gtod once at
>  * the start of the loop and use that for all the threads we wake.
>  */
> static void xlist_wake_all(struct thread_data *td)
> {
>         struct thread_data *list;
>         struct thread_data *next;
>         struct timeval now;
>
>         list = xlist_splice(td);
>         gettimeofday(&now, NULL);
>         while (list) {
>                 next = list->next;
>                 list->next = NULL;
>                 memcpy(&list->wake_time, &now, sizeof(now));
>                 fpost(&list->futex);
>                 list = next;
>         }
> }
>
> /*
>  * called by worker threads to send a message and wait for the answer.
>  * In reality we're just trading one cacheline with the gtod and futex in
>  * it, but that's good enough.  We gtod after waking and use that to
>  * record scheduler latency.
>  */
> static void msg_and_wait(struct thread_data *td)
> {
>         struct timeval now;
>         unsigned long long delta;
>         struct timespec timeout;
>
>         timeout.tv_sec = 0;
>         timeout.tv_nsec = 5000 * 1000;
>
>         /* set ourselves to blocked */
>         td->futex = FUTEX_BLOCKED;
>         gettimeofday(&td->wake_time, NULL);
>
>         /* add us to the list */
>         xlist_add(td->msg_thread, td);
>
>         fpost(&td->msg_thread->futex);
>
>         /*
>          * don't wait if the main threads are shutting down,
>          * they will never kick us fpost has a full barrier, so as long
>          * as the message thread walks his list after setting stopping,
>          * we shouldn't miss the wakeup
>          */
>         if (!stopping) {
>                 /* if he hasn't already woken us up, wait */
>                 fwait(&td->futex, NULL);
>         }
>
>         gettimeofday(&now, NULL);
>         delta = tvdelta(&td->wake_time, &now);
>         if (delta > 0)
>                 add_lat(&td->stats, delta);
> }
>
> /*
>  * once the message thread starts all his children, this is where he
>  * loops until our runtime is up.  Basically this sits around waiting
>  * for posting by the worker threads, replying to their messages after
>  * a delay of 'sleeptime' + some jitter.
>  */
> static void run_msg_thread(struct thread_data *td)
> {
>         struct timeval now;
>         struct timespec timeout;
>         unsigned int seed = pthread_self();
>         int max_jitter = sleeptime / 4;
>         int jitter;
>
>         jitter = rand_r(&seed) % max_jitter;
>         timeout.tv_sec = 0;
>         timeout.tv_nsec = (sleeptime + jitter) * 1000;
>
>         while (1) {
>                 td->futex = FUTEX_BLOCKED;
>                 xlist_wake_all(td);
>
>                 gettimeofday(&now, NULL);
>                 if (now.tv_sec > global_stop.tv_sec) {
>                         stopping = 1;
>                         __sync_synchronize();
>                         xlist_wake_all(td);
>                         break;
>                 }
>                 fwait(&td->futex, &timeout);
>
>                 /*
>                  * messages shouldn't be instant, sleep a little to make them
>                  * wait
>                  */
>                 jitter = rand_r(&seed) % max_jitter;
>                 usleep(sleeptime + jitter);
>         }
> }
>
> #define nop __asm__ __volatile__("rep;nop": : :"memory")
>
> static void usec_spin(unsigned long spin_time)
> {
>         struct timeval now;
>         struct timeval start;
>         unsigned long long delta;
>
>         gettimeofday(&start, NULL);
>         while (1) {
>                 gettimeofday(&now, NULL);
>                 delta = tvdelta(&start, &now);
>                 if (delta > spin_time)
>                         return;
>                 nop;
>         }
> }
>
> /*
>  * the worker thread is pretty simple, it just does a single spin and
>  * then waits on a message from the message thread
>  */
> void *worker_thread(void *arg)
> {
>         struct thread_data *td = arg;
>
>         while(1) {
>                 if (stopping)
>                         break;
>
>                 usec_spin(cputime);
>                 msg_and_wait(td);
>         }
>         return NULL;
> }
>
> /*
>  * the message thread starts his own gaggle of workers and then sits around
>  * replying when they post him.  He collects latency stats as all the threads
>  * exit
>  */
> void *message_thread(void *arg)
> {
>         struct thread_data *td = arg;
>         struct thread_data *worker_threads_mem = NULL;
>         int i;
>         int ret;
>
>         worker_threads_mem = calloc(worker_threads, sizeof(struct thread_data));
>
>         if (!worker_threads_mem) {
>                 perror("unable to allocate ram");
>                 pthread_exit((void *)-ENOMEM);
>         }
>
>         for (i = 0; i < worker_threads; i++) {
>                 pthread_t tid;
>
>                 worker_threads_mem[i].msg_thread = td;
>                 ret = pthread_create(&tid, NULL, worker_thread,
>                                      worker_threads_mem + i);
>                 if (ret) {
>                         fprintf(stderr, "error %d from pthread_create\n", ret);
>                         exit(1);
>                 }
>                 worker_threads_mem[i].tid = tid;
>         }
>
>         run_msg_thread(td);
>
>         for (i = 0; i < worker_threads; i++) {
>                 pthread_join(worker_threads_mem[i].tid, NULL);
>                 combine_stats(&td->stats, &worker_threads_mem[i].stats);
>         }
>         free(worker_threads_mem);
>
>         return NULL;
> }
>
> int main(int ac, char **av)
> {
>         int i;
>         int ret;
>         struct thread_data *message_threads_mem = NULL;
>         struct stats stats;
>
>         parse_options(ac, av);
> again:
>         stopping = 0;
>         memset(&stats, 0, sizeof(stats));
>
>         message_threads_mem = calloc(message_threads,
>                                       sizeof(struct thread_data));
>
>
>         if (!message_threads_mem) {
>                 perror("unable to allocate ram");
>                 exit(1);
>         }
>         gettimeofday(&global_stop, NULL);
>         global_stop.tv_sec += runtime;
>
>         /* start our message threads, each one starts its own workers */
>         for (i = 0; i < message_threads; i++) {
>                 pthread_t tid;
>                 ret = pthread_create(&tid, NULL, message_thread,
>                                      message_threads_mem + i);
>                 if (ret) {
>                         fprintf(stderr, "error %d from pthread_create\n", ret);
>                         exit(1);
>                 }
>                 message_threads_mem[i].tid = tid;
>         }
>         for (i = 0; i < message_threads; i++) {
>                 pthread_join(message_threads_mem[i].tid, NULL);
>                 combine_stats(&stats, &message_threads_mem[i].stats);
>         }
>
>         free(message_threads_mem);
>
>         /*
>          * in auto bench mode, keep adding workers until our latencies get
>          * horrible
>          */
>         if (autobench) {
>                 int p99 = calc_p99(&stats);
>                 fprintf(stderr, "cputime %Lu threads %d p99 %d\n",
>                         cputime, worker_threads, p99);
>                 if (p99 < 2000) {
>                         worker_threads++;
>                         goto again;
>                 }
>         }
>
>         show_latencies(&stats);
>
>         return 0;
> }

^ permalink raw reply	[flat|nested] 80+ messages in thread

* Re: [PATCH RFC] select_idle_sibling experiments
  2016-04-05 18:43 ` Bastien Bastien Philbert
@ 2016-04-05 19:28   ` Chris Mason
  0 siblings, 0 replies; 80+ messages in thread
From: Chris Mason @ 2016-04-05 19:28 UTC (permalink / raw)
  To: Bastien Bastien Philbert
  Cc: Peter Zijlstra, Ingo Molnar, Matt Fleming, Mike Galbraith, linux-kernel

On Tue, Apr 05, 2016 at 02:43:09PM -0400, Bastien Bastien Philbert wrote:
> On Tue, Apr 5, 2016 at 2:08 PM, Chris Mason <clm@fb.com> wrote:

[ ... ]

> >
> > I tried a few variations on select_idle_sibling() that preserved the
> > underlying goal of returning idle cores before idle SMT threads.  They
> > were all horrible in different ways, and none of them were fast.
> >
> > The patch below just makes select_idle_sibling pick the first idle
> > thread it can find.  When I ran it through production workloads here, it
> > was faster than the patch we've been carrying around for the last few
> > years.

[ ... ]

> >
> Here is my concern, do you test this on standard scheduler workloads
> or was this just written for Facebook's internal workloads. I am going
> to test this later because frankly this may cause a regression on my
> system which has only 4  cores so a idle CPU is probably less common
> for a small amount of time. I am wondering however if Ingo has any
> complains before I test this to see if it causes a regression or a bug
> on my system.  Ingo do you have any thoughts on this or would you like
> me to just test this?  Bastien

Pretty much every commit to select_idle_sibling over the last few years
was somehow trying to preserve or improve the select-idle-cores-first
functionality I just ripped out.  So, it's safe to assume it'll break
something ;)

-chris

^ permalink raw reply	[flat|nested] 80+ messages in thread

* Re: [PATCH RFC] select_idle_sibling experiments
  2016-04-05 18:08 [PATCH RFC] select_idle_sibling experiments Chris Mason
  2016-04-05 18:43 ` Bastien Bastien Philbert
@ 2016-04-05 20:03 ` Matt Fleming
  2016-04-05 21:05   ` Bastien Philbert
  2016-04-06  0:44   ` Chris Mason
  2016-04-06  7:27 ` Mike Galbraith
                   ` (2 subsequent siblings)
  4 siblings, 2 replies; 80+ messages in thread
From: Matt Fleming @ 2016-04-05 20:03 UTC (permalink / raw)
  To: Chris Mason, Peter Zijlstra, Ingo Molnar, Mike Galbraith, linux-kernel
  Cc: Mel Gorman

On Tue, 05 Apr, at 02:08:22PM, Chris Mason wrote:
> 
> I started with a small-ish program to benchmark wakeup latencies.  The
> basic idea is a bunch of worker threads who sit around and burn CPU.
> Every once and a while they send a message to a message thread.
 
This reminds me of something I've been looking at recently; a similar
workload in Mel's mmtests based on pgbench with 1-client that also has
this problem of cpu_idle() being false at an inconvenient time in
select_idle_sibling(), so we move the task off the cpu and the cpu
then immediately goes idle.

This leads to tasks bouncing around the socket as we search for idle
cpus.

> It has knobs for cpu think time, and for how long the messenger thread
> waits before replying.  Here's how I'm running it with my patch:
 
[...]

Cool, I'll go have a play with this.

> Now, on to the patch.  I pushed some code around and narrowed the
> problem down to select_idle_sibling()   We have cores going into and out
> of idle fast enough that even this cut our latencies in half:
> 
> static int select_idle_sibling(struct task_struct *p, int target)
>                                 goto next;
>  
>                         for_each_cpu(i, sched_group_cpus(sg)) {
> -                               if (i == target || !idle_cpu(i))
> +                               if (!idle_cpu(i))
>                                         goto next;
>                         }
>  
> IOW, by the time we get down to for_each_cpu(), the idle_cpu() check
> done at the top of the function is no longer valid.
 
Yeah. The problem is that because we're racing with the cpu going in
and out of idle, and since you're exploiting that race condition, this
is highly tuned to your specific workload.

Which is a roundabout way of saying, this is probably going to
negatively impact other workloads.

> I tried a few variations on select_idle_sibling() that preserved the
> underlying goal of returning idle cores before idle SMT threads.  They
> were all horrible in different ways, and none of them were fast.
 
I toyed with ignoring cpu_idle() in select_idle_sibling() for my
workload. That actually was faster ;)

> The patch below just makes select_idle_sibling pick the first idle
> thread it can find.  When I ran it through production workloads here, it
> was faster than the patch we've been carrying around for the last few
> years.

It would be really nice if we had a lightweight way to gauge the
"idleness" of a cpu, and whether we expect it to be idle again soon.

Failing that, could we just force the task onto 'target' when it makes
sense and skip the idle search (and the race) altogether?

^ permalink raw reply	[flat|nested] 80+ messages in thread

* Re: [PATCH RFC] select_idle_sibling experiments
  2016-04-05 20:03 ` Matt Fleming
@ 2016-04-05 21:05   ` Bastien Philbert
  2016-04-06  0:44   ` Chris Mason
  1 sibling, 0 replies; 80+ messages in thread
From: Bastien Philbert @ 2016-04-05 21:05 UTC (permalink / raw)
  To: Matt Fleming, Chris Mason, Peter Zijlstra, Ingo Molnar,
	Mike Galbraith, linux-kernel
  Cc: Mel Gorman



On 2016-04-05 04:03 PM, Matt Fleming wrote:
> On Tue, 05 Apr, at 02:08:22PM, Chris Mason wrote:
>>
>> I started with a small-ish program to benchmark wakeup latencies.  The
>> basic idea is a bunch of worker threads who sit around and burn CPU.
>> Every once and a while they send a message to a message thread.
>  
> This reminds me of something I've been looking at recently; a similar
> workload in Mel's mmtests based on pgbench with 1-client that also has
> this problem of cpu_idle() being false at an inconvenient time in
> select_idle_sibling(), so we move the task off the cpu and the cpu
> then immediately goes idle.
> 
> This leads to tasks bouncing around the socket as we search for idle
> cpus.
> 
>> It has knobs for cpu think time, and for how long the messenger thread
>> waits before replying.  Here's how I'm running it with my patch:
>  
> [...]
> 
> Cool, I'll go have a play with this.
> 
>> Now, on to the patch.  I pushed some code around and narrowed the
>> problem down to select_idle_sibling()   We have cores going into and out
>> of idle fast enough that even this cut our latencies in half:
>>
>> static int select_idle_sibling(struct task_struct *p, int target)
>>                                 goto next;
>>  
>>                         for_each_cpu(i, sched_group_cpus(sg)) {
>> -                               if (i == target || !idle_cpu(i))
>> +                               if (!idle_cpu(i))
>>                                         goto next;
>>                         }
>>  
>> IOW, by the time we get down to for_each_cpu(), the idle_cpu() check
>> done at the top of the function is no longer valid.
>  
> Yeah. The problem is that because we're racing with the cpu going in
> and out of idle, and since you're exploiting that race condition, this
> is highly tuned to your specific workload.
> 
> Which is a roundabout way of saying, this is probably going to
> negatively impact other workloads.
> 
>> I tried a few variations on select_idle_sibling() that preserved the
>> underlying goal of returning idle cores before idle SMT threads.  They
>> were all horrible in different ways, and none of them were fast.
>  
> I toyed with ignoring cpu_idle() in select_idle_sibling() for my
> workload. That actually was faster ;)
> 
>> The patch below just makes select_idle_sibling pick the first idle
>> thread it can find.  When I ran it through production workloads here, it
>> was faster than the patch we've been carrying around for the last few
>> years.
> 
> It would be really nice if we had a lightweight way to gauge the
> "idleness" of a cpu, and whether we expect it to be idle again soon.
> 
The best way to do this is either embed it in a already used structure to
allow us to check it quickly. Otherwise I am curious if writing a marco
may prove useful for this. Seems that idleness checking needs to accounted
for when scheduling, in order to make this lightweight enough to avoid using
it during a context switch, the challenge however is to make the reference
counting lightweight enough to out weight it being done during current scheduling
functions.
> Failing that, could we just force the task onto 'target' when it makes
> sense and skip the idle search (and the race) altogether?
> 
Doesn't this possibly cause a context switch or even a extensive move to another
CPU instruction(s) on certain architectures. Seems we need to add reference counting
or tracking of idle CPUS somewhere.
Bastien 

^ permalink raw reply	[flat|nested] 80+ messages in thread

* Re: [PATCH RFC] select_idle_sibling experiments
  2016-04-05 20:03 ` Matt Fleming
  2016-04-05 21:05   ` Bastien Philbert
@ 2016-04-06  0:44   ` Chris Mason
  1 sibling, 0 replies; 80+ messages in thread
From: Chris Mason @ 2016-04-06  0:44 UTC (permalink / raw)
  To: Matt Fleming
  Cc: Peter Zijlstra, Ingo Molnar, Mike Galbraith, linux-kernel, Mel Gorman

On Tue, Apr 05, 2016 at 09:03:02PM +0100, Matt Fleming wrote:
> On Tue, 05 Apr, at 02:08:22PM, Chris Mason wrote:
> > 
> > I started with a small-ish program to benchmark wakeup latencies.  The
> > basic idea is a bunch of worker threads who sit around and burn CPU.
> > Every once and a while they send a message to a message thread.
>  
> This reminds me of something I've been looking at recently; a similar
> workload in Mel's mmtests based on pgbench with 1-client that also has
> this problem of cpu_idle() being false at an inconvenient time in
> select_idle_sibling(), so we move the task off the cpu and the cpu
> then immediately goes idle.
> 
> This leads to tasks bouncing around the socket as we search for idle
> cpus.

It might be worth making a way to claim the idle cpu.  If there are lots
of them, we'll fan out properly instead of piling up into the first one.
If there are very few of them, we'll figure it out much faster.

> 
> > It has knobs for cpu think time, and for how long the messenger thread
> > waits before replying.  Here's how I'm running it with my patch:
>  
> [...]
> 
> Cool, I'll go have a play with this.

I'm more than open to ways to improve it, and I'll send to Mel or put in
a git tree if people find it useful.

> 
> > Now, on to the patch.  I pushed some code around and narrowed the
> > problem down to select_idle_sibling()   We have cores going into and out
> > of idle fast enough that even this cut our latencies in half:
> > 
> > static int select_idle_sibling(struct task_struct *p, int target)
> >                                 goto next;
> >  
> >                         for_each_cpu(i, sched_group_cpus(sg)) {
> > -                               if (i == target || !idle_cpu(i))
> > +                               if (!idle_cpu(i))
> >                                         goto next;
> >                         }
> >  
> > IOW, by the time we get down to for_each_cpu(), the idle_cpu() check
> > done at the top of the function is no longer valid.
>  
> Yeah. The problem is that because we're racing with the cpu going in
> and out of idle, and since you're exploiting that race condition, this
> is highly tuned to your specific workload.
> 
> Which is a roundabout way of saying, this is probably going to
> negatively impact other workloads.
> 
> > I tried a few variations on select_idle_sibling() that preserved the
> > underlying goal of returning idle cores before idle SMT threads.  They
> > were all horrible in different ways, and none of them were fast.
>  
> I toyed with ignoring cpu_idle() in select_idle_sibling() for my
> workload. That actually was faster ;)
> 
> > The patch below just makes select_idle_sibling pick the first idle
> > thread it can find.  When I ran it through production workloads here, it
> > was faster than the patch we've been carrying around for the last few
> > years.
> 
> It would be really nice if we had a lightweight way to gauge the
> "idleness" of a cpu, and whether we expect it to be idle again soon.
> 
> Failing that, could we just force the task onto 'target' when it makes
> sense and skip the idle search (and the race) altogether?

To me it feels like the search for a full free core is impossible.  The boxes
are intentionally loaded to the point where a full core is never going
to be free.  So we need this loop to quickly pick a good candidate and
move on.

The benchmark is using ~30ms of CPU time in each worker thread, so
picking a CPU with a busy worker thread is going to have a pretty big
penalty.  Just grabbing any CPU and hoping it'll be idle soon isn't
likely to work well, and if it does that's probably a bug in my
benchmark ;)

You can see this in action by adding one (or at most two) more threads
to the command line.  The p99 jumps quickly to 2ms or more.

-chris

^ permalink raw reply	[flat|nested] 80+ messages in thread

* Re: [PATCH RFC] select_idle_sibling experiments
  2016-04-05 18:08 [PATCH RFC] select_idle_sibling experiments Chris Mason
  2016-04-05 18:43 ` Bastien Bastien Philbert
  2016-04-05 20:03 ` Matt Fleming
@ 2016-04-06  7:27 ` Mike Galbraith
  2016-04-06 13:36   ` Chris Mason
                     ` (2 more replies)
  2016-04-07 15:17 ` Chris Mason
  2016-04-09 19:05 ` sched: tweak select_idle_sibling to look for idle threads Chris Mason
  4 siblings, 3 replies; 80+ messages in thread
From: Mike Galbraith @ 2016-04-06  7:27 UTC (permalink / raw)
  To: Chris Mason, Peter Zijlstra, Ingo Molnar, Matt Fleming, linux-kernel

[-- Attachment #1: Type: text/plain, Size: 3515 bytes --]

> On Tue, 2016-04-05 at 14:08 -0400, Chris Mason wrote:

> Now, on to the patch.  I pushed some code around and narrowed the
> problem down to select_idle_sibling()   We have cores going into and out
> of idle fast enough that even this cut our latencies in half:

Are you using NO_HZ?  If so, you may want to try the attached.

> static int select_idle_sibling(struct task_struct *p, int target)
>                                 goto next;
>  
>                         for_each_cpu(i, sched_group_cpus(sg)) {
> -                               if (i == target || !idle_cpu(i))
> +                               if (!idle_cpu(i))
>                                         goto next;
>                         }
>  
> IOW, by the time we get down to for_each_cpu(), the idle_cpu() check
> done at the top of the function is no longer valid.

Ok, that's only an optimization, could go if it's causing trouble.

> I tried a few variations on select_idle_sibling() that preserved the
> underlying goal of returning idle cores before idle SMT threads.  They
> were all horrible in different ways, and none of them were fast.
> 
> The patch below just makes select_idle_sibling pick the first idle
> thread it can find.  When I ran it through production workloads here, it
> was faster than the patch we've been carrying around for the last few
> years.
> 
> 
> diff --git a/kernel/sched/fair.c b/kernel/sched/fair.c
> index 56b7d4b..c41baa6 100644
> --- a/kernel/sched/fair.c
> +++ b/kernel/sched/fair.c
> @@ -4974,7 +4974,6 @@ find_idlest_cpu(struct sched_group *group,
> struct task_struct *p, int this_cpu)
>  static int select_idle_sibling(struct task_struct *p, int target)
>  {
>  	struct sched_domain *sd;
> -	struct sched_group *sg;
>  	int i = task_cpu(p);
>  
>  	if (idle_cpu(target))
> @@ -4990,24 +4989,14 @@ static int select_idle_sibling(struct
> task_struct *p, int target)
>  	 * Otherwise, iterate the domains and find an elegible idle
> cpu.
>  	 */
>  	sd = rcu_dereference(per_cpu(sd_llc, target));
> -	for_each_lower_domain(sd) {
> -		sg = sd->groups;
> -		do {
> -			if
> (!cpumask_intersects(sched_group_cpus(sg),
> -						tsk_cpus_allowed(p))
> )
> -				goto next;
> -
> -			for_each_cpu(i, sched_group_cpus(sg)) {
> -				if (i == target || !idle_cpu(i))
> -					goto next;
> -			}
> +	if (!sd)
> +		goto done;
>  
> -			target =
> cpumask_first_and(sched_group_cpus(sg),
> -					tsk_cpus_allowed(p));
> +	for_each_cpu_and(i, sched_domain_span(sd), &p->cpus_allowed)
> {
> +		if (cpu_active(i) && idle_cpu(i)) {
> +			target = i;
>  			goto done;
> -next:
> -			sg = sg->next;
> -		} while (sg != sd->groups);
> +		}
>  	}
>  done:
>  	return target;
> 

Ew.  That may improve your latency is everything load, but worst case
package walk will hurt like hell on CPUs with insane number of threads.
 That full search also turns the evil face of two-faced little
select_idle_sibling() into it's only face, the one that bounces tasks
about much more than they appreciate.

Looking for an idle core first delivers the most throughput boost, and
only looking at target's threads if you don't find one keeps the bounce
and traverse pain down to a dull roar, while at least trying to get
that latency win.  To me, your patch looks like it trades harm to many,
for good for a few.

A behavior switch would be better.  It can't get any dumber, but trying
to make it smarter makes it too damn fat.  As it sits, it's aiming in
the general direction of the bullseye.. and occasionally hits the wall.

	-Mike

[-- Attachment #2: sched-throttle-nohz.patch --]
[-- Type: text/x-patch, Size: 1535 bytes --]

sched: ratelimit nohz

Entering nohz code on every micro-idle is too expensive to bear.

Signed-off-by: Mike Galbraith <efault@gmx.de>
---
 include/linux/sched.h    |    5 +++++
 kernel/sched/core.c      |    8 ++++++++
 kernel/time/tick-sched.c |    2 +-
 3 files changed, 14 insertions(+), 1 deletion(-)

--- a/include/linux/sched.h
+++ b/include/linux/sched.h
@@ -2286,6 +2286,11 @@ static inline int set_cpus_allowed_ptr(s
 #ifdef CONFIG_NO_HZ_COMMON
 void calc_load_enter_idle(void);
 void calc_load_exit_idle(void);
+#ifdef CONFIG_SMP
+extern int sched_needs_cpu(int cpu);
+#else
+static inline int sched_needs_cpu(int cpu) { return 0; }
+#endif
 #else
 static inline void calc_load_enter_idle(void) { }
 static inline void calc_load_exit_idle(void) { }
--- a/kernel/sched/core.c
+++ b/kernel/sched/core.c
@@ -577,6 +577,14 @@ static inline bool got_nohz_idle_kick(vo
 	return false;
 }
 
+int sched_needs_cpu(int cpu)
+{
+	if (tick_nohz_full_cpu(cpu))
+		return 0;
+
+	return  cpu_rq(cpu)->avg_idle < sysctl_sched_migration_cost;
+}
+
 #else /* CONFIG_NO_HZ_COMMON */
 
 static inline bool got_nohz_idle_kick(void)
--- a/kernel/time/tick-sched.c
+++ b/kernel/time/tick-sched.c
@@ -676,7 +676,7 @@ static ktime_t tick_nohz_stop_sched_tick
 	} while (read_seqretry(&jiffies_lock, seq));
 	ts->last_jiffies = basejiff;
 
-	if (rcu_needs_cpu(basemono, &next_rcu) ||
+	if (sched_needs_cpu(cpu) || rcu_needs_cpu(basemono, &next_rcu) ||
 	    arch_needs_cpu() || irq_work_needs_cpu()) {
 		next_tick = basemono + TICK_NSEC;
 	} else {

^ permalink raw reply	[flat|nested] 80+ messages in thread

* Re: [PATCH RFC] select_idle_sibling experiments
  2016-04-06  7:27 ` Mike Galbraith
@ 2016-04-06 13:36   ` Chris Mason
  2016-04-09 17:30   ` Chris Mason
  2016-04-28 12:00   ` Peter Zijlstra
  2 siblings, 0 replies; 80+ messages in thread
From: Chris Mason @ 2016-04-06 13:36 UTC (permalink / raw)
  To: Mike Galbraith; +Cc: Peter Zijlstra, Ingo Molnar, Matt Fleming, linux-kernel

On Wed, Apr 06, 2016 at 09:27:24AM +0200, Mike Galbraith wrote:
> > On Tue, 2016-04-05 at 14:08 -0400, Chris Mason wrote:
> 
> > Now, on to the patch.  I pushed some code around and narrowed the
> > problem down to select_idle_sibling()   We have cores going into and out
> > of idle fast enough that even this cut our latencies in half:
> 
> Are you using NO_HZ?  If so, you may want to try the attached.

I'll definitely give it a shot.  When I tried using the nohz idle bitmap
(Peter's idea) instead of the for_each_cpu() walks, it came out slower.
It feels like the cpus aren't getting all the way down into the idle
loop before more work comes, but I'll have to check.

> 
> > static int select_idle_sibling(struct task_struct *p, int target)
> >                                 goto next;
> >  
> >                         for_each_cpu(i, sched_group_cpus(sg)) {
> > -                               if (i == target || !idle_cpu(i))
> > +                               if (!idle_cpu(i))
> >                                         goto next;
> >                         }
> >  
> > IOW, by the time we get down to for_each_cpu(), the idle_cpu() check
> > done at the top of the function is no longer valid.
> 
> Ok, that's only an optimization, could go if it's causing trouble.

It's more an indication of how long we're spending in the current scan.
Long enough for the tests we're currently doing to be inaccurate.

[ my beautiful patch ]

> Ew.  That may improve your latency is everything load, but worst case
> package walk will hurt like hell on CPUs with insane number of threads.
>
>  That full search also turns the evil face of two-faced little
> select_idle_sibling() into it's only face, the one that bounces tasks
> about much more than they appreciate.
> 
> Looking for an idle core first delivers the most throughput boost, and
> only looking at target's threads if you don't find one keeps the bounce
> and traverse pain down to a dull roar, while at least trying to get
> that latency win.  To me, your patch looks like it trades harm to many,
> for good for a few.

Yes, I'm tossing an important optimization.  The goal wasn't to get rid
of that at all, but instead to find a way to get both.  I just ran out
of ideas ;)

> 
> A behavior switch would be better.  It can't get any dumber, but trying
> to make it smarter makes it too damn fat.  As it sits, it's aiming in
> the general direction of the bullseye.. and occasionally hits the wall.
> 
> 	-Mike
>
> sched: ratelimit nohz
> 
> Entering nohz code on every micro-idle is too expensive to bear.

This I really like.  I'll setup a benchmark in production with it and
come back with results.

-chris

^ permalink raw reply	[flat|nested] 80+ messages in thread

* Re: [PATCH RFC] select_idle_sibling experiments
  2016-04-05 18:08 [PATCH RFC] select_idle_sibling experiments Chris Mason
                   ` (2 preceding siblings ...)
  2016-04-06  7:27 ` Mike Galbraith
@ 2016-04-07 15:17 ` Chris Mason
  2016-04-09 19:05 ` sched: tweak select_idle_sibling to look for idle threads Chris Mason
  4 siblings, 0 replies; 80+ messages in thread
From: Chris Mason @ 2016-04-07 15:17 UTC (permalink / raw)
  To: Peter Zijlstra, Ingo Molnar, Matt Fleming, Mike Galbraith, linux-kernel

On Tue, Apr 05, 2016 at 02:08:22PM -0400, Chris Mason wrote:
> Hi everyone,
> 
> We're porting the fb kernel up to 4.5, and one of our last few out-of-tree
> patches is a hack to try harder to find idle cpus when waking up tasks.
> This helps in pretty much every workload we run, mostly because they all
> get tuned with a similar setup:
> 
> 1) find the load where latencies stop being acceptable
> 2) Run the server at just a little less than that
> 
> Usually this means our CPUs are just a little bit idle, and a poor
> scheduler decision to place a task on a busy CPU instead of an idle CPU
> ends up impacting our p99 latencies.
> 
> Mike helped us with this last year, fixing up wake_wide() to improve
> things.  But we still ended up having to go back to the old hack.
> 
> I started with a small-ish program to benchmark wakeup latencies.  The
> basic idea is a bunch of worker threads who sit around and burn CPU.
> Every once and a while they send a message to a message thread.
> 
> The message thread records the time he woke up the worker, and the
> worker records the delta between that time and the time he actually got
> the CPU again.  At the end it spits out a latency histogram.  The only
> thing we record is the wakeup latency, there's no measurement of 'work
> done' or any of the normal things you'd expect in a benchmark.
> 
> It has knobs for cpu think time, and for how long the messenger thread
> waits before replying.  Here's how I'm running it with my patch:
> 
> ./schbench -c 30000 -s 30000 -m 6 -t 24 -r 30

FYI, I changed schbench around a bit, and fixed a bug with -m (it was
ignored and always forced to 2).

The new code is here:

https://git.kernel.org/cgit/linux/kernel/git/mason/schbench.git/

I added a pipe simulation mode too, since I wanted wakeup latencies for
a raw tput test as well as my original workload.  -p takes the size of
the transfer you want to simulate.  There's no pipe involved, it's just
doing memsets on pages and waking the other thread with futexes.  The
latency is still only the latency of the worker thread wakeup:

# taskset -c 0 ./schbench -p 4 -m 1 -t 1 -r 20
Latency percentiles (usec)
        50.0000th: 1
        75.0000th: 2
        90.0000th: 2
        95.0000th: 2
        *99.0000th: 2
        99.5000th: 2
        99.9000th: 6
        Over=0, min=0, max=43
avg worker transfer: 372311.38 ops/sec 1.42MB/s

# taskset -c 0 perf bench sched pipe -l 5000000
# Running 'sched/pipe' benchmark:
# Executed 5000000 pipe operations between two processes

     Total time: 20.359 [sec]

       4.071851 usecs/op
         245588 ops/sec

I'm taking another stab at fixing the regression for picking an idle
core in my first patch, and I'll get some benchmark's with Mike's nohz
patch going as well.

-chris

^ permalink raw reply	[flat|nested] 80+ messages in thread

* Re: [PATCH RFC] select_idle_sibling experiments
  2016-04-06  7:27 ` Mike Galbraith
  2016-04-06 13:36   ` Chris Mason
@ 2016-04-09 17:30   ` Chris Mason
  2016-04-12 21:45     ` Matt Fleming
  2016-04-28 12:00   ` Peter Zijlstra
  2 siblings, 1 reply; 80+ messages in thread
From: Chris Mason @ 2016-04-09 17:30 UTC (permalink / raw)
  To: Mike Galbraith; +Cc: Peter Zijlstra, Ingo Molnar, Matt Fleming, linux-kernel

On Wed, Apr 06, 2016 at 09:27:24AM +0200, Mike Galbraith wrote:
> > On Tue, 2016-04-05 at 14:08 -0400, Chris Mason wrote:
> 
> > Now, on to the patch.  I pushed some code around and narrowed the
> > problem down to select_idle_sibling()   We have cores going into and out
> > of idle fast enough that even this cut our latencies in half:
> 
> Are you using NO_HZ?  If so, you may want to try the attached.

[ nohz throttling patch ]

I tested the nohz throttle two different ways, first with schbench's
pipe simulation, it's easily 8% faster with messages bouncing between
cpus.

In production it's hard to pick a single number because the benchmarks
produce latency curves as the workload scales up in RPS.  The benefits
range from 2-9% depending on the metric.  It's a nice win, and I'd love to
see it go in.

-chris

^ permalink raw reply	[flat|nested] 80+ messages in thread

* sched: tweak select_idle_sibling to look for idle threads
  2016-04-05 18:08 [PATCH RFC] select_idle_sibling experiments Chris Mason
                   ` (3 preceding siblings ...)
  2016-04-07 15:17 ` Chris Mason
@ 2016-04-09 19:05 ` Chris Mason
  2016-04-10 10:04   ` Mike Galbraith
  2016-04-30 12:47   ` Peter Zijlstra
  4 siblings, 2 replies; 80+ messages in thread
From: Chris Mason @ 2016-04-09 19:05 UTC (permalink / raw)
  To: Peter Zijlstra, Ingo Molnar, Matt Fleming, Mike Galbraith, linux-kernel

select_task_rq_fair() can leave cpu utilization a little lumpy,
especially as the workload ramps up to the maximum capacity of the
machine.  The end result can be high p99 response times as apps
wait to get scheduled, even when boxes are mostly idle.

I wrote schbench to try and measure this:

git://git.kernel.org/pub/scm/linux/kernel/git/mason/schbench.git

The basic idea is to record the latency between when a thread is kicked
and when it actually gets the CPU.  For this patch I used a simple model
where a thread thinks for a while and then waits for data from another
thread.  The command line below will start two messenger threads with 18
workers per messenger:

./schbench -m 2 -t 18 -s 30000 -c 30000 -r 30
Latency percentiles (usec)
    50.0000th: 52
    75.0000th: 63
    90.0000th: 74
    95.0000th: 80
    *99.0000th: 118
    99.5000th: 707
    99.9000th: 5016
    Over=0, min=0, max=12941

p99 numbers here are acceptable, but you can see the curve starting.
if I use 19 workers per messenger, p99 goes through the roof.  This
machine has two sockets, 10 cores each, so with HT on, this commandline
has one pthread on each CPU:

./schbench -m 2 -t 19 -s 30000 -c 30000 -r 30
Latency percentiles (usec)
    50.0000th: 51
    75.0000th: 63
    90.0000th: 76
    95.0000th: 89
    *99.0000th: 2132
    99.5000th: 6920
    99.9000th: 12752
    Over=0, min=0, max=17079

This commit tries to solve things by doing an extra scan in
select_idle_sibling().  If it can find an idle sibling in any core in the
package, it will return that:

./schbench -m2 -t 19 -c 30000 -s 30000 -r 30
Latency percentiles (usec)
    50.0000th: 65
    75.0000th: 104
    90.0000th: 115
    95.0000th: 118
    *99.0000th: 124
    99.5000th: 127
    99.9000th: 262
    Over=0, min=0, max=12987

This basically means the whole fleet can have one more pthread per socket
and still maintain acceptable latencies.  I can actually go up to -t 20,
but it's not as consistent:

./schbench -m2 -t 20 -c 30000 -s 30000 -r 30
Latency percentiles (usec)
    50.0000th: 50
    75.0000th: 63
    90.0000th: 75
    95.0000th: 81
    *99.0000th: 111
    99.5000th: 975
    99.9000th: 12464
    Over=0, min=0, max=18317

This does preserve the existing logic to prefer idle cores over idle
CPU threads, and includes some tests to try and avoid the idle scan when we're
actually better off sharing a non-idle CPU with someone else.

Benchmarks in production show overall capacity going up between 2-5%
depending on the metric.

Credits to Arun Sharma <asharma@fb.com> for initial versions of this
patch.

Signed-off-by: Chris Mason <clm@fb.com>

diff --git a/kernel/sched/fair.c b/kernel/sched/fair.c
index 56b7d4b..2c47240 100644
--- a/kernel/sched/fair.c
+++ b/kernel/sched/fair.c
@@ -4969,11 +4969,34 @@ find_idlest_cpu(struct sched_group *group, struct task_struct *p, int this_cpu)
 }
 
 /*
+ * helper for select_idle_sibling to decide if it should look for idle
+ * threads
+ */
+static int bounce_to_target(struct task_struct *p, int cpu)
+{
+	s64 delta;
+
+	/*
+	 * as the run queue gets bigger, its more and more likely that
+	 * balance will have distributed things for us, and less likely
+	 * that scanning all our CPUs for an idle one will find one.
+	 * So, if nr_running > 1, just call this CPU good enough
+	 */
+	if (cpu_rq(cpu)->cfs.nr_running > 1)
+		return 1;
+
+	/* taken from task_hot() */
+	delta = rq_clock_task(task_rq(p)) - p->se.exec_start;
+	return delta < (s64)sysctl_sched_migration_cost;
+}
+
+/*
  * Try and locate an idle CPU in the sched_domain.
  */
 static int select_idle_sibling(struct task_struct *p, int target)
 {
 	struct sched_domain *sd;
+	struct sched_domain *package_sd;
 	struct sched_group *sg;
 	int i = task_cpu(p);
 
@@ -4989,7 +5012,8 @@ static int select_idle_sibling(struct task_struct *p, int target)
 	/*
 	 * Otherwise, iterate the domains and find an elegible idle cpu.
 	 */
-	sd = rcu_dereference(per_cpu(sd_llc, target));
+	package_sd = rcu_dereference(per_cpu(sd_llc, target));
+	sd = package_sd;
 	for_each_lower_domain(sd) {
 		sg = sd->groups;
 		do {
@@ -4998,7 +5022,12 @@ static int select_idle_sibling(struct task_struct *p, int target)
 				goto next;
 
 			for_each_cpu(i, sched_group_cpus(sg)) {
-				if (i == target || !idle_cpu(i))
+				/*
+				 * we tested target for idle up above,
+				 * but don't skip it here because it might
+				 * have raced to idle while we were scanning
+				 */
+				if (!idle_cpu(i))
 					goto next;
 			}
 
@@ -5009,6 +5038,24 @@ next:
 			sg = sg->next;
 		} while (sg != sd->groups);
 	}
+
+	/*
+	 * we're here because we didn't find an idle core, or an idle sibling
+	 * in the target core.  For message bouncing workloads, we want to
+	 * just stick with the target suggestion from the caller, but
+	 * otherwise we'd rather have an idle CPU from anywhere else in
+	 * the package.
+	 */
+	if (package_sd && !bounce_to_target(p, target)) {
+		for_each_cpu_and(i, sched_domain_span(package_sd),
+				 tsk_cpus_allowed(p)) {
+			if (idle_cpu(i)) {
+				target = i;
+				break;
+			}
+
+		}
+	}
 done:
 	return target;
 }

^ permalink raw reply related	[flat|nested] 80+ messages in thread

* Re: sched: tweak select_idle_sibling to look for idle threads
  2016-04-09 19:05 ` sched: tweak select_idle_sibling to look for idle threads Chris Mason
@ 2016-04-10 10:04   ` Mike Galbraith
  2016-04-10 12:35     ` Chris Mason
  2016-04-10 19:55     ` Chris Mason
  2016-04-30 12:47   ` Peter Zijlstra
  1 sibling, 2 replies; 80+ messages in thread
From: Mike Galbraith @ 2016-04-10 10:04 UTC (permalink / raw)
  To: Chris Mason, Peter Zijlstra, Ingo Molnar, Matt Fleming, linux-kernel

On Sat, 2016-04-09 at 15:05 -0400, Chris Mason wrote:

> This does preserve the existing logic to prefer idle cores over idle
> CPU threads, and includes some tests to try and avoid the idle scan when we're
> actually better off sharing a non-idle CPU with someone else.

My box says the "oh nevermind" checks aren't selective enough, tbench
dropped 4% at clients=cores, and 2% at clients=threads.

> Benchmarks in production show overall capacity going up between 2-5%
> depending on the metric.

Latency rules all loads certainly exist, and clearly want some love,
but the bigger the socket, and the more threads/core, the more that
traverse is gonna hurt the others, so seems either we need a better
filter, or a (yeah yeah, yet another damn) tweakable.

Oh, and bounce_to_target() seems an odd way to say full_traverse.

	-Mike

^ permalink raw reply	[flat|nested] 80+ messages in thread

* Re: sched: tweak select_idle_sibling to look for idle threads
  2016-04-10 10:04   ` Mike Galbraith
@ 2016-04-10 12:35     ` Chris Mason
  2016-04-10 12:46       ` Mike Galbraith
  2016-04-10 19:55     ` Chris Mason
  1 sibling, 1 reply; 80+ messages in thread
From: Chris Mason @ 2016-04-10 12:35 UTC (permalink / raw)
  To: Mike Galbraith; +Cc: Peter Zijlstra, Ingo Molnar, Matt Fleming, linux-kernel

On Sun, Apr 10, 2016 at 12:04:21PM +0200, Mike Galbraith wrote:
> On Sat, 2016-04-09 at 15:05 -0400, Chris Mason wrote:
> 
> > This does preserve the existing logic to prefer idle cores over idle
> > CPU threads, and includes some tests to try and avoid the idle scan when we're
> > actually better off sharing a non-idle CPU with someone else.
> 
> My box says the "oh nevermind" checks aren't selective enough, tbench
> dropped 4% at clients=cores, and 2% at clients=threads.

I knew this part would need more experimentation, so I kept v1 as simple
as possible.  On my box, tbench clients=cores is 5% faster,
clients=threads is 4% faster.  bounce_to_target() is a small version of
task_hot(), I did get more accurate decisions by using the full
task_hot(), so I can try that again.

I'm testing on one of these:

Intel(R) Xeon(R) CPU E5-2660 v2 @ 2.20GHz, which has two sockets and 10
cores per socket.

What are you testing with?  If it's two sockets or less I may be able to
find one to reproduce with.

> 
> > Benchmarks in production show overall capacity going up between 2-5%
> > depending on the metric.
> 
> Latency rules all loads certainly exist, and clearly want some love,
> but the bigger the socket, and the more threads/core, the more that
> traverse is gonna hurt the others, so seems either we need a better
> filter, or a (yeah yeah, yet another damn) tweakable.
> 
> Oh, and bounce_to_target() seems an odd way to say full_traverse.

Sure, I can rename it.

-chris

^ permalink raw reply	[flat|nested] 80+ messages in thread

* Re: sched: tweak select_idle_sibling to look for idle threads
  2016-04-10 12:35     ` Chris Mason
@ 2016-04-10 12:46       ` Mike Galbraith
  0 siblings, 0 replies; 80+ messages in thread
From: Mike Galbraith @ 2016-04-10 12:46 UTC (permalink / raw)
  To: Chris Mason; +Cc: Peter Zijlstra, Ingo Molnar, Matt Fleming, linux-kernel

On Sun, 2016-04-10 at 08:35 -0400, Chris Mason wrote:

> What are you testing with?  If it's two sockets or less I may be able to
> find one to reproduce with.

i4790 desktop box.

	-Mike

^ permalink raw reply	[flat|nested] 80+ messages in thread

* Re: sched: tweak select_idle_sibling to look for idle threads
  2016-04-10 10:04   ` Mike Galbraith
  2016-04-10 12:35     ` Chris Mason
@ 2016-04-10 19:55     ` Chris Mason
  2016-04-11  4:54       ` Mike Galbraith
  1 sibling, 1 reply; 80+ messages in thread
From: Chris Mason @ 2016-04-10 19:55 UTC (permalink / raw)
  To: Mike Galbraith; +Cc: Peter Zijlstra, Ingo Molnar, Matt Fleming, linux-kernel

On Sun, Apr 10, 2016 at 12:04:21PM +0200, Mike Galbraith wrote:
> On Sat, 2016-04-09 at 15:05 -0400, Chris Mason wrote:
> 
> > This does preserve the existing logic to prefer idle cores over idle
> > CPU threads, and includes some tests to try and avoid the idle scan when we're
> > actually better off sharing a non-idle CPU with someone else.
> 
> My box says the "oh nevermind" checks aren't selective enough, tbench
> dropped 4% at clients=cores, and 2% at clients=threads.

Ok, I was able to reproduce this by stuffing tbench_srv and tbench onto
just socket 0.  Version 2 below fixes things for me, but I'm hoping
someone can suggest a way to get task_hot() buddy checks without the rq
lock.

I haven't run this on production loads yet, but our 4.0 patch for this
uses task_hot(), so I'd expect it to be on par.  If this doesn't fix it
for you, I'll dig up a similar machine on Monday.

-chris

>From 306f7a2019b341d11c9713be83f41b2261994f44 Mon Sep 17 00:00:00 2001
From: Chris Mason <clm@fb.com>
Date: Fri, 8 Apr 2016 13:18:20 -0700
Subject: [PATCH v2] sched: tweak select_idle_sibling to look for idle threads

select_task_rq_fair() can leave cpu utilization a little lumpy,
especially as the workload ramps up to the maximum capacity of the
machine.  The end result can be high p99 response times as apps
wait to get scheduled, even when boxes are mostly idle.

I wrote schbench to try and measure this:

git://git.kernel.org/pub/scm/linux/kernel/git/mason/schbench.git

The basic idea is to record the latency between when a thread is kicked
and when it actually gets the CPU.  For this patch I used a simple model
where a thread thinks for a while and then waits for data from another
thread.  The command line below will start two messenger threads with 18
workers per messenger:

 ./schbench -m 2 -t 18 -s 30000 -c 30000 -r 30
Latency percentiles (usec)
        50.0000th: 52
        75.0000th: 63
        90.0000th: 74
        95.0000th: 80
        *99.0000th: 118
        99.5000th: 707
        99.9000th: 5016
        Over=0, min=0, max=12941

p99 numbers here are acceptable.  But if I use 19 workers per messenger,
p99 goes through the roof.  This machine has two sockets, 10 cores each,
so with HT on, this commandline has one pthread on each CPU:

 ./schbench -m 2 -t 19 -s 30000 -c 30000 -r 30
Latency percentiles (usec)
        50.0000th: 51
        75.0000th: 63
        90.0000th: 76
        95.0000th: 89
        *99.0000th: 2132
        99.5000th: 6920
        99.9000th: 12752
        Over=0, min=0, max=17079

This commit tries to solve things by doing an extra scan in
select_idle_sibling().select_idle_sibling  If it can find an idle
sibling in any core in the package, it will return that:

./schbench -m2 -t 19 -c 30000 -s 30000 -r 30
Latency percentiles (usec)
        50.0000th: 65
        75.0000th: 104
        90.0000th: 115
        95.0000th: 118
        *99.0000th: 124
        99.5000th: 127
        99.9000th: 262
        Over=0, min=0, max=12987

This basically means the whole fleet can have one more pthread per socket
and still maintain acceptable latencies.  I can actually go up to -t 20,
but the curve starts getting steep.

./schbench -m2 -t 20 -c 30000 -s 30000 -r 30
Latency percentiles (usec)
        50.0000th: 50
        75.0000th: 63
        90.0000th: 75
        95.0000th: 81
        *99.0000th: 111
        99.5000th: 975
        99.9000th: 12464
        Over=0, min=0, max=18317

This does preserve the existing logic to prefer idle cores over idle
CPU threads, and includes some tests to try and avoid the idle scan when we're
actually better off sharing a non-idle CPU with someone else.

Benchmarks in production show average duration of a request is 13%
faster, with overall capacity going up between 2-5% depending on the
metric.

Credits to Arun Sharma <asharma@fb.com> for initial versions of this
patch.

Signed-off-by: Chris Mason <clm@fb.com>
---
 v1 -> v2 call task_hot to decide if we should scan for idle cpus
 kernel/sched/fair.c | 59 +++++++++++++++++++++++++++++++++++++++++++++++++++--
 1 file changed, 57 insertions(+), 2 deletions(-)

diff --git a/kernel/sched/fair.c b/kernel/sched/fair.c
index 56b7d4b..0c76505 100644
--- a/kernel/sched/fair.c
+++ b/kernel/sched/fair.c
@@ -658,6 +658,7 @@ static u64 sched_vslice(struct cfs_rq *cfs_rq, struct sched_entity *se)
 #ifdef CONFIG_SMP
 static int select_idle_sibling(struct task_struct *p, int cpu);
 static unsigned long task_h_load(struct task_struct *p);
+static int should_scan_idle(struct task_struct *p, int cpu);
 
 /*
  * We choose a half-life close to 1 scheduling period.
@@ -4974,6 +4975,7 @@ find_idlest_cpu(struct sched_group *group, struct task_struct *p, int this_cpu)
 static int select_idle_sibling(struct task_struct *p, int target)
 {
 	struct sched_domain *sd;
+	struct sched_domain *package_sd;
 	struct sched_group *sg;
 	int i = task_cpu(p);
 
@@ -4989,7 +4991,8 @@ static int select_idle_sibling(struct task_struct *p, int target)
 	/*
 	 * Otherwise, iterate the domains and find an elegible idle cpu.
 	 */
-	sd = rcu_dereference(per_cpu(sd_llc, target));
+	package_sd = rcu_dereference(per_cpu(sd_llc, target));
+	sd = package_sd;
 	for_each_lower_domain(sd) {
 		sg = sd->groups;
 		do {
@@ -4998,7 +5001,12 @@ static int select_idle_sibling(struct task_struct *p, int target)
 				goto next;
 
 			for_each_cpu(i, sched_group_cpus(sg)) {
-				if (i == target || !idle_cpu(i))
+				/*
+				 * we tested target for idle up above,
+				 * but don't skip it here because it might
+				 * have raced to idle while we were scanning
+				 */
+				if (!idle_cpu(i))
 					goto next;
 			}
 
@@ -5009,6 +5017,24 @@ next:
 			sg = sg->next;
 		} while (sg != sd->groups);
 	}
+
+	/*
+	 * we're here because we didn't find an idle core, or an idle sibling
+	 * in the target core.  For message bouncing workloads, we want to
+	 * just stick with the target suggestion from the caller, but
+	 * otherwise we'd rather have an idle CPU from anywhere else in
+	 * the package.
+	 */
+	if (package_sd && should_scan_idle(p, target)) {
+		for_each_cpu_and(i, sched_domain_span(package_sd),
+				 tsk_cpus_allowed(p)) {
+			if (idle_cpu(i)) {
+				target = i;
+				break;
+			}
+
+		}
+	}
 done:
 	return target;
 }
@@ -5714,6 +5740,35 @@ static int task_hot(struct task_struct *p, struct lb_env *env)
 	return delta < (s64)sysctl_sched_migration_cost;
 }
 
+/*
+ * helper for select_idle_sibling to decide if it should look for idle
+ * threads
+ */
+static int should_scan_idle(struct task_struct *p, int cpu)
+{
+	unsigned long flags;
+	struct lb_env env;
+	int hot;
+
+	/*
+	 * as the run queue gets bigger, its more and more likely that
+	 * balance will have distributed things for us, and less likely
+	 * that scanning all our CPUs for an idle one will find one.
+	 * So, if nr_running > 1, just call this CPU good enough
+	 */
+	if (cpu_rq(cpu)->cfs.nr_running > 1)
+		return 0;
+
+	env.src_rq = task_rq(p);
+	env.dst_rq = cpu_rq(cpu);
+	raw_spin_lock_irqsave(&env.src_rq->lock, flags);
+	hot = task_hot(p, &env);
+	raw_spin_unlock_irqrestore(&env.src_rq->lock, flags);
+
+	return hot == 0;
+}
+
+
 #ifdef CONFIG_NUMA_BALANCING
 /*
  * Returns 1, if task migration degrades locality
-- 
2.8.0.rc2

^ permalink raw reply related	[flat|nested] 80+ messages in thread

* Re: sched: tweak select_idle_sibling to look for idle threads
  2016-04-10 19:55     ` Chris Mason
@ 2016-04-11  4:54       ` Mike Galbraith
  2016-04-12  0:30         ` Chris Mason
  0 siblings, 1 reply; 80+ messages in thread
From: Mike Galbraith @ 2016-04-11  4:54 UTC (permalink / raw)
  To: Chris Mason; +Cc: Peter Zijlstra, Ingo Molnar, Matt Fleming, linux-kernel

On Sun, 2016-04-10 at 15:55 -0400, Chris Mason wrote:
> On Sun, Apr 10, 2016 at 12:04:21PM +0200, Mike Galbraith wrote:
> > On Sat, 2016-04-09 at 15:05 -0400, Chris Mason wrote:
> > 
> > > This does preserve the existing logic to prefer idle cores over idle
> > > CPU threads, and includes some tests to try and avoid the idle scan when we're
> > > actually better off sharing a non-idle CPU with someone else.
> > 
> > My box says the "oh nevermind" checks aren't selective enough, tbench
> > dropped 4% at clients=cores, and 2% at clients=threads.
> 
> Ok, I was able to reproduce this by stuffing tbench_srv and tbench onto
> just socket 0.  Version 2 below fixes things for me, but I'm hoping
> someone can suggest a way to get task_hot() buddy checks without the rq
> lock.
> 
> I haven't run this on production loads yet, but our 4.0 patch for this
> uses task_hot(), so I'd expect it to be on par.  If this doesn't fix it
> for you, I'll dig up a similar machine on Monday.

My box stopped caring.  I personally would be reluctant to apply it
without a "you asked for it" button or a large pile of benchmark
results.  Lock banging or not, full scan existing makes me nervous.

	-Mike

^ permalink raw reply	[flat|nested] 80+ messages in thread

* Re: sched: tweak select_idle_sibling to look for idle threads
  2016-04-11  4:54       ` Mike Galbraith
@ 2016-04-12  0:30         ` Chris Mason
  2016-04-12  4:44           ` Mike Galbraith
  0 siblings, 1 reply; 80+ messages in thread
From: Chris Mason @ 2016-04-12  0:30 UTC (permalink / raw)
  To: Mike Galbraith; +Cc: Peter Zijlstra, Ingo Molnar, Matt Fleming, linux-kernel

On Mon, Apr 11, 2016 at 06:54:21AM +0200, Mike Galbraith wrote:
> On Sun, 2016-04-10 at 15:55 -0400, Chris Mason wrote:
> > On Sun, Apr 10, 2016 at 12:04:21PM +0200, Mike Galbraith wrote:
> > > On Sat, 2016-04-09 at 15:05 -0400, Chris Mason wrote:
> > > 
> > > > This does preserve the existing logic to prefer idle cores over idle
> > > > CPU threads, and includes some tests to try and avoid the idle scan when we're
> > > > actually better off sharing a non-idle CPU with someone else.
> > > 
> > > My box says the "oh nevermind" checks aren't selective enough, tbench
> > > dropped 4% at clients=cores, and 2% at clients=threads.
> > 
> > Ok, I was able to reproduce this by stuffing tbench_srv and tbench onto
> > just socket 0.  Version 2 below fixes things for me, but I'm hoping
> > someone can suggest a way to get task_hot() buddy checks without the rq
> > lock.
> > 
> > I haven't run this on production loads yet, but our 4.0 patch for this
> > uses task_hot(), so I'd expect it to be on par.  If this doesn't fix it
> > for you, I'll dig up a similar machine on Monday.
> 
> My box stopped caring.  I personally would be reluctant to apply it
> without a "you asked for it" button or a large pile of benchmark
> results.  Lock banging or not, full scan existing makes me nervous.


We can use a bitmap at the socket level to keep track of which cpus are
idle.  I'm sure there are better places for the array and better ways to
allocate, this is just a rough cut to make sure the idle tracking works.

-chris

diff --git a/include/linux/sched.h b/include/linux/sched.h
index a10494a..1c3b5e4 100644
--- a/include/linux/sched.h
+++ b/include/linux/sched.h
@@ -1055,6 +1055,8 @@ struct sched_domain {
 	unsigned int balance_interval;	/* initialise to 1. units in ms. */
 	unsigned int nr_balance_failed; /* initialise to 0 */
 
+	cpumask_var_t idle_cpus_mask;
+
 	/* idle_balance() stats */
 	u64 max_newidle_lb_cost;
 	unsigned long next_decay_max_lb_cost;
diff --git a/kernel/sched/core.c b/kernel/sched/core.c
index 41f6b22..237d645 100644
--- a/kernel/sched/core.c
+++ b/kernel/sched/core.c
@@ -3204,6 +3204,7 @@ again:
 static void __sched notrace __schedule(bool preempt)
 {
 	struct task_struct *prev, *next;
+	struct sched_domain *package_sd;
 	unsigned long *switch_count;
 	struct rq *rq;
 	int cpu;
@@ -3270,11 +3270,19 @@ static void __sched notrace __schedule(bool preempt)
 		update_rq_clock(rq);
 
 	next = pick_next_task(rq, prev);
+
 	clear_tsk_need_resched(prev);
 	clear_preempt_need_resched();
 	rq->clock_skip_update = 0;
 
 	if (likely(prev != next)) {
+		package_sd = rcu_dereference(per_cpu(sd_llc, cpu));
+		if (package_sd) {
+			if (prev->policy == SCHED_IDLE && next->policy != SCHED_IDLE)
+				cpumask_clear_cpu(cpu, package_sd->idle_cpus_mask);
+			else if (next->policy == SCHED_IDLE)
+				cpumask_set_cpu(cpu, package_sd->idle_cpus_mask);
+		}
 		rq->nr_switches++;
 		rq->curr = next;
 		++*switch_count;
@@ -6599,7 +6607,6 @@ sd_init(struct sched_domain_topology_level *tl, int cpu)
 		sd->imbalance_pct = 117;
 		sd->cache_nice_tries = 1;
 		sd->busy_idx = 2;
-
 #ifdef CONFIG_NUMA
 	} else if (sd->flags & SD_NUMA) {
 		sd->cache_nice_tries = 2;
@@ -7041,6 +7048,8 @@ struct sched_domain *build_sched_domain(struct sched_domain_topology_level *tl,
 		return child;
 
 	cpumask_and(sched_domain_span(sd), cpu_map, tl->mask(cpu));
+	zalloc_cpumask_var(&sd->idle_cpus_mask, GFP_NOWAIT);
+	cpumask_and(sd->idle_cpus_mask, cpu_map, tl->mask(cpu));
 	if (child) {
 		sd->level = child->level + 1;
 		sched_domain_level_max = max(sched_domain_level_max, sd->level);
diff --git a/kernel/sched/fair.c b/kernel/sched/fair.c
index 0c76505..cae6bd7 100644
--- a/kernel/sched/fair.c
+++ b/kernel/sched/fair.c
@@ -5026,7 +5026,7 @@ next:
 	 * the package.
 	 */
 	if (package_sd && should_scan_idle(p, target)) {
-		for_each_cpu_and(i, sched_domain_span(package_sd),
+		for_each_cpu_and(i, package_sd->idle_cpus_mask,
 				 tsk_cpus_allowed(p)) {
 			if (idle_cpu(i)) {
 				target = i;
diff --git a/kernel/sched/idle.c b/kernel/sched/idle.c
index 544a713..7e34b42 100644
--- a/kernel/sched/idle.c
+++ b/kernel/sched/idle.c
@@ -202,6 +202,9 @@ DEFINE_PER_CPU(bool, cpu_dead_idle);
  */
 static void cpu_idle_loop(void)
 {
+	int cpu;
+	struct sched_domain *package_sd;
+
 	while (1) {
 		/*
 		 * If the arch has a polling bit, we maintain an invariant:
@@ -212,10 +215,19 @@ static void cpu_idle_loop(void)
 		 * guaranteed to cause the cpu to reschedule.
 		 */
 
+
 		__current_set_polling();
 		quiet_vmstat();
 		tick_nohz_idle_enter();
 
+		preempt_disable();
+		cpu = smp_processor_id();
+		package_sd = rcu_dereference(per_cpu(sd_llc, cpu));
+		if (package_sd) {
+			cpumask_set_cpu(cpu, package_sd->idle_cpus_mask);
+		}
+		preempt_enable();
+
 		while (!need_resched()) {
 			check_pgt_cache();
 			rmb();
-- 
2.8.0.rc2

^ permalink raw reply related	[flat|nested] 80+ messages in thread

* Re: sched: tweak select_idle_sibling to look for idle threads
  2016-04-12  0:30         ` Chris Mason
@ 2016-04-12  4:44           ` Mike Galbraith
  2016-04-12 13:27             ` Chris Mason
  0 siblings, 1 reply; 80+ messages in thread
From: Mike Galbraith @ 2016-04-12  4:44 UTC (permalink / raw)
  To: Chris Mason; +Cc: Peter Zijlstra, Ingo Molnar, Matt Fleming, linux-kernel

On Mon, 2016-04-11 at 20:30 -0400, Chris Mason wrote:
> On Mon, Apr 11, 2016 at 06:54:21AM +0200, Mike Galbraith wrote:

> > > Ok, I was able to reproduce this by stuffing tbench_srv and tbench onto
> > > just socket 0.  Version 2 below fixes things for me, but I'm hoping
> > > someone can suggest a way to get task_hot() buddy checks without the rq
> > > lock.
> > > 
> > > I haven't run this on production loads yet, but our 4.0 patch for this
> > > uses task_hot(), so I'd expect it to be on par.  If this doesn't fix it
> > > for you, I'll dig up a similar machine on Monday.
> > 
> > My box stopped caring.  I personally would be reluctant to apply it
> > without a "you asked for it" button or a large pile of benchmark
> > results.  Lock banging or not, full scan existing makes me nervous.
> 
> 
> We can use a bitmap at the socket level to keep track of which cpus are
> idle.  I'm sure there are better places for the array and better ways to
> allocate, this is just a rough cut to make sure the idle tracking works.

See e0a79f529d5b:

      pre   15.22 MB/sec 1 procs
      post 252.01 MB/sec 1 procs

You can make traverse cycles go away, but those cycles, while precious,
are not the most costly cycles.  The above was 1 tbench pair in an
otherwise idle box.. ie it wasn't traverse cycles that demolished it.

	-Mike

(p.s. SCHED_IDLE is dinky bandwidth fair class)

^ permalink raw reply	[flat|nested] 80+ messages in thread

* Re: sched: tweak select_idle_sibling to look for idle threads
  2016-04-12  4:44           ` Mike Galbraith
@ 2016-04-12 13:27             ` Chris Mason
  2016-04-12 18:16               ` Mike Galbraith
  0 siblings, 1 reply; 80+ messages in thread
From: Chris Mason @ 2016-04-12 13:27 UTC (permalink / raw)
  To: Mike Galbraith; +Cc: Peter Zijlstra, Ingo Molnar, Matt Fleming, linux-kernel

On Tue, Apr 12, 2016 at 06:44:08AM +0200, Mike Galbraith wrote:
> On Mon, 2016-04-11 at 20:30 -0400, Chris Mason wrote:
> > On Mon, Apr 11, 2016 at 06:54:21AM +0200, Mike Galbraith wrote:
> 
> > > > Ok, I was able to reproduce this by stuffing tbench_srv and tbench onto
> > > > just socket 0.  Version 2 below fixes things for me, but I'm hoping
> > > > someone can suggest a way to get task_hot() buddy checks without the rq
> > > > lock.
> > > > 
> > > > I haven't run this on production loads yet, but our 4.0 patch for this
> > > > uses task_hot(), so I'd expect it to be on par.  If this doesn't fix it
> > > > for you, I'll dig up a similar machine on Monday.
> > > 
> > > My box stopped caring.  I personally would be reluctant to apply it
> > > without a "you asked for it" button or a large pile of benchmark
> > > results.  Lock banging or not, full scan existing makes me nervous.
> > 
> > 
> > We can use a bitmap at the socket level to keep track of which cpus are
> > idle.  I'm sure there are better places for the array and better ways to
> > allocate, this is just a rough cut to make sure the idle tracking works.
> 
> See e0a79f529d5b:
> 
>       pre   15.22 MB/sec 1 procs
>       post 252.01 MB/sec 1 procs
> 
> You can make traverse cycles go away, but those cycles, while precious,
> are not the most costly cycles.  The above was 1 tbench pair in an
> otherwise idle box.. ie it wasn't traverse cycles that demolished it.

Agreed, this is why the decision not to scan is so important.  But while
I've been describing this patch in terms of latency, latency is really
the symptom instead of the goal.  Without these patches, workloads that
do want to fully utilize the hardware are basically getting one fewer
core of utilization.  It's true that we define 'fully utilize' with an
upper bound on application response time, but we're not talking high
frequency trading here.

It clearly shows up in our graphs.  CPU idle is higher (the lost core),
CPU user time is lower, average system load is higher (procs waiting on
a fewer number of core).

We measure this internally with scheduling latency because that's the
easiest way to talk about it across a wide variety of hardware.

> 
> 	-Mike
> 
> (p.s. SCHED_IDLE is dinky bandwidth fair class)

Ugh, not my best quick patch, but you get the idea I was going for.  I
can always add the tunable to flip things on/off but I'd prefer that we
find a good set of defaults, mostly so the FB production runtime is the
common config instead of the special snowflake.

-chris

^ permalink raw reply	[flat|nested] 80+ messages in thread

* Re: sched: tweak select_idle_sibling to look for idle threads
  2016-04-12 13:27             ` Chris Mason
@ 2016-04-12 18:16               ` Mike Galbraith
  2016-04-12 20:07                 ` Chris Mason
  0 siblings, 1 reply; 80+ messages in thread
From: Mike Galbraith @ 2016-04-12 18:16 UTC (permalink / raw)
  To: Chris Mason; +Cc: Peter Zijlstra, Ingo Molnar, Matt Fleming, linux-kernel

On Tue, 2016-04-12 at 09:27 -0400, Chris Mason wrote:
> I
> can always add the tunable to flip things on/off but I'd prefer that we
> find a good set of defaults, mostly so the FB production runtime is the
> common config instead of the special snowflake.

Yeah, generic has a much better chance to actually get merged, just
need a very solid chain on the lurking beast from hell.  Hm...

The last time we went through this, the problem child was the waker of
many in your load.  With tiny twiddle to wake_wide(), all was allegedly
well, or at least that's the impression I was left with.  That leads me
to a pseudo-random thought: iff that waker of many is still at the
root, you could try using wake_wide() as the full search trigger, which
should shrink the attack surface available to the horror-from-hell
quite a lot.  Just a thought.

	-Mike

^ permalink raw reply	[flat|nested] 80+ messages in thread

* Re: sched: tweak select_idle_sibling to look for idle threads
  2016-04-12 18:16               ` Mike Galbraith
@ 2016-04-12 20:07                 ` Chris Mason
  2016-04-13  3:18                   ` Mike Galbraith
  0 siblings, 1 reply; 80+ messages in thread
From: Chris Mason @ 2016-04-12 20:07 UTC (permalink / raw)
  To: Mike Galbraith; +Cc: Peter Zijlstra, Ingo Molnar, Matt Fleming, linux-kernel

On Tue, Apr 12, 2016 at 08:16:17PM +0200, Mike Galbraith wrote:
> On Tue, 2016-04-12 at 09:27 -0400, Chris Mason wrote:
> > I
> > can always add the tunable to flip things on/off but I'd prefer that we
> > find a good set of defaults, mostly so the FB production runtime is the
> > common config instead of the special snowflake.
> 
> Yeah, generic has a much better chance to actually get merged, just
> need a very solid chain on the lurking beast from hell.  Hm...
> 
> The last time we went through this, the problem child was the waker of
> many in your load.  With tiny twiddle to wake_wide(), all was allegedly
> well, or at least that's the impression I was left with.  That leads me
> to a pseudo-random thought: iff that waker of many is still at the
> root, you could try using wake_wide() as the full search trigger, which
> should shrink the attack surface available to the horror-from-hell
> quite a lot.  Just a thought.

I think that if we're worried about the cost of the idle scan for this
workload, find_idlest_group() is either going to hurt much more, or not
search enough CPUs to find the idle one.

But I'm happy to try patches or other ideas, I have a fixed version of
the bitmap one going through production benchmarks now.

-chris

^ permalink raw reply	[flat|nested] 80+ messages in thread

* Re: [PATCH RFC] select_idle_sibling experiments
  2016-04-09 17:30   ` Chris Mason
@ 2016-04-12 21:45     ` Matt Fleming
  2016-04-13  3:40       ` Mike Galbraith
  0 siblings, 1 reply; 80+ messages in thread
From: Matt Fleming @ 2016-04-12 21:45 UTC (permalink / raw)
  To: Chris Mason, Mike Galbraith, Peter Zijlstra, Ingo Molnar, linux-kernel

On Sat, 09 Apr, at 01:30:34PM, Chris Mason wrote:
> 
> [ nohz throttling patch ]
> 
> I tested the nohz throttle two different ways, first with schbench's
> pipe simulation, it's easily 8% faster with messages bouncing between
> cpus.
> 
> In production it's hard to pick a single number because the benchmarks
> produce latency curves as the workload scales up in RPS.  The benefits
> range from 2-9% depending on the metric.  It's a nice win, and I'd love to
> see it go in.

Do we have any idea what the tradeoff is against power consumption for
throttling nohz?

^ permalink raw reply	[flat|nested] 80+ messages in thread

* Re: sched: tweak select_idle_sibling to look for idle threads
  2016-04-12 20:07                 ` Chris Mason
@ 2016-04-13  3:18                   ` Mike Galbraith
  2016-04-13 13:44                     ` Chris Mason
  0 siblings, 1 reply; 80+ messages in thread
From: Mike Galbraith @ 2016-04-13  3:18 UTC (permalink / raw)
  To: Chris Mason; +Cc: Peter Zijlstra, Ingo Molnar, Matt Fleming, linux-kernel

On Tue, 2016-04-12 at 16:07 -0400, Chris Mason wrote:

> I think that if we're worried about the cost of the idle scan for this
> workload, find_idlest_group() is either going to hurt much more, or not
> search enough CPUs to find the idle one.

find_idlest_group()?  No no no, that's not what I mean at all.

wake_wide() identifies loads that really want to spread out, thus turns
off affine wakeups.  We still call select_idle_sibling(), only
difference being that target is the original cpu, not the waking cpu. 
 Given making that wide connection bidirectional helped FB's load, it
seems reasonable that passing wide information to select_idle_sibling()
would have a good chance of hitting the candidate that stands to gain
from a full socket scan, while also keeping that cache scrambling scan
far away from the rest. 

> But I'm happy to try patches or other ideas, I have a fixed version of
> the bitmap one going through production benchmarks now.

Making that wide/full search cheap is still good, because wake_wide()
also identifies interrupt sources that are waking many, so cheap wide
search should increase utilization there as well.  The thought was to
just make the wide thing have a tad wider effect on what it already
does affect.. and hope that doesn't demolish anything.

	-Mike

^ permalink raw reply	[flat|nested] 80+ messages in thread

* Re: [PATCH RFC] select_idle_sibling experiments
  2016-04-12 21:45     ` Matt Fleming
@ 2016-04-13  3:40       ` Mike Galbraith
  2016-04-13 15:54         ` Chris Mason
  0 siblings, 1 reply; 80+ messages in thread
From: Mike Galbraith @ 2016-04-13  3:40 UTC (permalink / raw)
  To: Matt Fleming, Chris Mason, Peter Zijlstra, Ingo Molnar, linux-kernel

On Tue, 2016-04-12 at 22:45 +0100, Matt Fleming wrote:
> On Sat, 09 Apr, at 01:30:34PM, Chris Mason wrote:
> > 
> > [ nohz throttling patch ]
> > 
> > I tested the nohz throttle two different ways, first with schbench's
> > pipe simulation, it's easily 8% faster with messages bouncing between
> > cpus.
> > 
> > In production it's hard to pick a single number because the benchmarks
> > produce latency curves as the workload scales up in RPS.  The benefits
> > range from 2-9% depending on the metric.  It's a nice win, and I'd love to
> > see it go in.
> 
> Do we have any idea what the tradeoff is against power consumption for
> throttling nohz?

That's measurable with the built in super duper watt meter gizmo
(turbostat).  It should be dinky but existent, could be given an off
button for particularly attentive laptop drivers to poke.  Servers
drivers are unlikely to care given the performance win.

	-Mike

^ permalink raw reply	[flat|nested] 80+ messages in thread

* Re: sched: tweak select_idle_sibling to look for idle threads
  2016-04-13  3:18                   ` Mike Galbraith
@ 2016-04-13 13:44                     ` Chris Mason
  2016-04-13 14:22                       ` Mike Galbraith
  0 siblings, 1 reply; 80+ messages in thread
From: Chris Mason @ 2016-04-13 13:44 UTC (permalink / raw)
  To: Mike Galbraith; +Cc: Peter Zijlstra, Ingo Molnar, Matt Fleming, linux-kernel

On Wed, Apr 13, 2016 at 05:18:51AM +0200, Mike Galbraith wrote:
> On Tue, 2016-04-12 at 16:07 -0400, Chris Mason wrote:
> 
> > I think that if we're worried about the cost of the idle scan for this
> > workload, find_idlest_group() is either going to hurt much more, or not
> > search enough CPUs to find the idle one.
> 
> find_idlest_group()?  No no no, that's not what I mean at all.
> 
> wake_wide() identifies loads that really want to spread out, thus turns
> off affine wakeups.  We still call select_idle_sibling(), only
> difference being that target is the original cpu, not the waking cpu. 

Ah ok, I see what you mean now.

>  Given making that wide connection bidirectional helped FB's load, it
> seems reasonable that passing wide information to select_idle_sibling()
> would have a good chance of hitting the candidate that stands to gain
> from a full socket scan, while also keeping that cache scrambling scan
> far away from the rest. 
> 
> > But I'm happy to try patches or other ideas, I have a fixed version of
> > the bitmap one going through production benchmarks now.

[ benchmarks say it needs more fixing, ick ]

> 
> Making that wide/full search cheap is still good, because wake_wide()
> also identifies interrupt sources that are waking many, so cheap wide
> search should increase utilization there as well.  The thought was to
> just make the wide thing have a tad wider effect on what it already
> does affect.. and hope that doesn't demolish anything.

So you're interested in numbers where we pass the wake_wide decision
into select_idle_sibling(), and then use that instead of (or in addition
to?) my should_scan_idle() function?

I agree we may need to tweak wake_wide, since most of our wakeups now
are failed affine wakeups.  But, the differences are in p99, so I'll
probably need to get some better metrics.

-chris

^ permalink raw reply	[flat|nested] 80+ messages in thread

* Re: sched: tweak select_idle_sibling to look for idle threads
  2016-04-13 13:44                     ` Chris Mason
@ 2016-04-13 14:22                       ` Mike Galbraith
  2016-04-13 14:36                         ` Chris Mason
  0 siblings, 1 reply; 80+ messages in thread
From: Mike Galbraith @ 2016-04-13 14:22 UTC (permalink / raw)
  To: Chris Mason; +Cc: Peter Zijlstra, Ingo Molnar, Matt Fleming, linux-kernel

On Wed, 2016-04-13 at 09:44 -0400, Chris Mason wrote:

> So you're interested in numbers where we pass the wake_wide decision
> into select_idle_sibling(), and then use that instead of (or in addition
> to?) my should_scan_idle() function?

Yeah, I was thinking instead of, and hoping that would be enough.

> I agree we may need to tweak wake_wide, since most of our wakeups now
> are failed affine wakeups.

What exactly do you mean by failed affine wakeups?  Failed because
wake_wide() said we don't want one, or because wake_affine() said we
can't have one?  If the later, my thought bubble may have just burst,
but it still "feels" right.

	-Mike

^ permalink raw reply	[flat|nested] 80+ messages in thread

* Re: sched: tweak select_idle_sibling to look for idle threads
  2016-04-13 14:22                       ` Mike Galbraith
@ 2016-04-13 14:36                         ` Chris Mason
  2016-04-13 15:05                           ` Mike Galbraith
  0 siblings, 1 reply; 80+ messages in thread
From: Chris Mason @ 2016-04-13 14:36 UTC (permalink / raw)
  To: Mike Galbraith; +Cc: Peter Zijlstra, Ingo Molnar, Matt Fleming, linux-kernel

On Wed, Apr 13, 2016 at 04:22:58PM +0200, Mike Galbraith wrote:
> On Wed, 2016-04-13 at 09:44 -0400, Chris Mason wrote:
> 
> > So you're interested in numbers where we pass the wake_wide decision
> > into select_idle_sibling(), and then use that instead of (or in addition
> > to?) my should_scan_idle() function?
> 
> Yeah, I was thinking instead of, and hoping that would be enough.

I'm definitely up for experimenting with different tests to decide when
to scan idle.  I'll have to wait until after lsf/vault, but I can layout
a bunch of tests.

> 
> > I agree we may need to tweak wake_wide, since most of our wakeups now
> > are failed affine wakeups.
> 
> What exactly do you mean by failed affine wakeups?  Failed because
> wake_wide() said we don't want one, or because wake_affine() said we
> can't have one?  If the later, my thought bubble may have just burst,
> but it still "feels" right.

I mean this number:

schedstat_inc(p, se.statistics.nr_wakeups_affine_attempts);

Is much much much higher than this number:

schedstat_inc(p, se.statistics.nr_wakeups_affine);

So, wake_affine said we can't have one.  I made a script to sum it up
across all the threads of the webserver workload.

-chris

^ permalink raw reply	[flat|nested] 80+ messages in thread

* Re: sched: tweak select_idle_sibling to look for idle threads
  2016-04-13 14:36                         ` Chris Mason
@ 2016-04-13 15:05                           ` Mike Galbraith
  2016-04-13 15:34                             ` Mike Galbraith
  0 siblings, 1 reply; 80+ messages in thread
From: Mike Galbraith @ 2016-04-13 15:05 UTC (permalink / raw)
  To: Chris Mason; +Cc: Peter Zijlstra, Ingo Molnar, Matt Fleming, linux-kernel

On Wed, 2016-04-13 at 10:36 -0400, Chris Mason wrote:
> On Wed, Apr 13, 2016 at 04:22:58PM +0200, Mike Galbraith wrote:

> > What exactly do you mean by failed affine wakeups?  Failed because
> > wake_wide() said we don't want one, or because wake_affine() said we
> > can't have one?  If the later, my thought bubble may have just burst,
> > but it still "feels" right.
> 
> I mean this number:
> 
> schedstat_inc(p, se.statistics.nr_wakeups_affine_attempts);
> 
> Is much much much higher than this number:
> 
> schedstat_inc(p, se.statistics.nr_wakeups_affine);
> 
> So, wake_affine said we can't have one.  I made a script to sum it up
> across all the threads of the webserver workload.

Hm, ok, that doesn't really tell us more than there's more to the load
than the 1:N bits that wake_wide() apparently did identify fairly well
last go, so targeting them still might help.

	-Mike

^ permalink raw reply	[flat|nested] 80+ messages in thread

* Re: sched: tweak select_idle_sibling to look for idle threads
  2016-04-13 15:05                           ` Mike Galbraith
@ 2016-04-13 15:34                             ` Mike Galbraith
  0 siblings, 0 replies; 80+ messages in thread
From: Mike Galbraith @ 2016-04-13 15:34 UTC (permalink / raw)
  To: Chris Mason; +Cc: Peter Zijlstra, Ingo Molnar, Matt Fleming, linux-kernel

Another thing you could try is looking at your avg_idle, and twiddling
sched_migration_cost_ns to crank up idle balancing a bit.

	-Mike

^ permalink raw reply	[flat|nested] 80+ messages in thread

* Re: [PATCH RFC] select_idle_sibling experiments
  2016-04-13  3:40       ` Mike Galbraith
@ 2016-04-13 15:54         ` Chris Mason
  0 siblings, 0 replies; 80+ messages in thread
From: Chris Mason @ 2016-04-13 15:54 UTC (permalink / raw)
  To: Mike Galbraith; +Cc: Matt Fleming, Peter Zijlstra, Ingo Molnar, linux-kernel

On Wed, Apr 13, 2016 at 05:40:20AM +0200, Mike Galbraith wrote:
> On Tue, 2016-04-12 at 22:45 +0100, Matt Fleming wrote:
> > On Sat, 09 Apr, at 01:30:34PM, Chris Mason wrote:
> > > 
> > > [ nohz throttling patch ]
> > > 
> > > I tested the nohz throttle two different ways, first with schbench's
> > > pipe simulation, it's easily 8% faster with messages bouncing between
> > > cpus.
> > > 
> > > In production it's hard to pick a single number because the benchmarks
> > > produce latency curves as the workload scales up in RPS.  The benefits
> > > range from 2-9% depending on the metric.  It's a nice win, and I'd love to
> > > see it go in.
> > 
> > Do we have any idea what the tradeoff is against power consumption for
> > throttling nohz?
> 
> That's measurable with the built in super duper watt meter gizmo
> (turbostat).  It should be dinky but existent, could be given an off
> button for particularly attentive laptop drivers to poke.  Servers
> drivers are unlikely to care given the performance win.

Our power sensors show its basically a wash during the production
benchmark runs.  Which makes sense because its really only blinking
on/off at very high frequency.

-chris

^ permalink raw reply	[flat|nested] 80+ messages in thread

* Re: [PATCH RFC] select_idle_sibling experiments
  2016-04-06  7:27 ` Mike Galbraith
  2016-04-06 13:36   ` Chris Mason
  2016-04-09 17:30   ` Chris Mason
@ 2016-04-28 12:00   ` Peter Zijlstra
  2016-04-28 13:17     ` Mike Galbraith
  2016-05-02  5:35     ` Mike Galbraith
  2 siblings, 2 replies; 80+ messages in thread
From: Peter Zijlstra @ 2016-04-28 12:00 UTC (permalink / raw)
  To: Mike Galbraith; +Cc: Chris Mason, Ingo Molnar, Matt Fleming, linux-kernel

On Wed, Apr 06, 2016 at 09:27:24AM +0200, Mike Galbraith wrote:
> sched: ratelimit nohz
> 
> Entering nohz code on every micro-idle is too expensive to bear.
> 
> Signed-off-by: Mike Galbraith <efault@gmx.de>

> +int sched_needs_cpu(int cpu)
> +{
> +	if (tick_nohz_full_cpu(cpu))
> +		return 0;
> +
> +	return  cpu_rq(cpu)->avg_idle < sysctl_sched_migration_cost;

So the only problem I have with this patch is the choice of limit. This
isn't at all tied to the migration cost.

And some people are already twiddling with the migration_cost knob to
affect the idle_balance() behaviour -- making it much more agressive by
dialing it down. When you do that you also loose the effectiveness of
this proposed usage, even though those same people would probably want
this.

Failing a spot of inspiration for a runtime limit on this; we might have
to introduce yet another knob :/

^ permalink raw reply	[flat|nested] 80+ messages in thread

* Re: [PATCH RFC] select_idle_sibling experiments
  2016-04-28 12:00   ` Peter Zijlstra
@ 2016-04-28 13:17     ` Mike Galbraith
  2016-05-02  5:35     ` Mike Galbraith
  1 sibling, 0 replies; 80+ messages in thread
From: Mike Galbraith @ 2016-04-28 13:17 UTC (permalink / raw)
  To: Peter Zijlstra; +Cc: Chris Mason, Ingo Molnar, Matt Fleming, linux-kernel

On Thu, 2016-04-28 at 14:00 +0200, Peter Zijlstra wrote:
> On Wed, Apr 06, 2016 at 09:27:24AM +0200, Mike Galbraith wrote:
> > sched: ratelimit nohz
> > 
> > Entering nohz code on every micro-idle is too expensive to bear.
> > 
> > Signed-off-by: Mike Galbraith <efault@gmx.de>
> 
> > +int sched_needs_cpu(int cpu)
> > +{
> > +> > 	> > if (tick_nohz_full_cpu(cpu))
> > +> > 	> > 	> > return 0;
> > +
> > +> > 	> > return  cpu_rq(cpu)->avg_idle < sysctl_sched_migration_cost;
> 
> So the only problem I have with this patch is the choice of limit. This
> isn't at all tied to the migration cost.

Yup.

> And some people are already twiddling with the migration_cost knob to
> affect the idle_balance() behaviour -- making it much more agressive by
> dialing it down. When you do that you also loose the effectiveness of
> this proposed usage, even though those same people would probably want
> this.
> 
> Failing a spot of inspiration for a runtime limit on this; we might have
> to introduce yet another knob :/

I'll roll one with a yet another of it's very own.

	-Mike

^ permalink raw reply	[flat|nested] 80+ messages in thread

* Re: sched: tweak select_idle_sibling to look for idle threads
  2016-04-09 19:05 ` sched: tweak select_idle_sibling to look for idle threads Chris Mason
  2016-04-10 10:04   ` Mike Galbraith
@ 2016-04-30 12:47   ` Peter Zijlstra
  2016-05-01  7:12     ` Mike Galbraith
  1 sibling, 1 reply; 80+ messages in thread
From: Peter Zijlstra @ 2016-04-30 12:47 UTC (permalink / raw)
  To: Chris Mason, Ingo Molnar, Matt Fleming, Mike Galbraith, linux-kernel

On Sat, Apr 09, 2016 at 03:05:54PM -0400, Chris Mason wrote:
> select_task_rq_fair() can leave cpu utilization a little lumpy,
> especially as the workload ramps up to the maximum capacity of the
> machine.  The end result can be high p99 response times as apps
> wait to get scheduled, even when boxes are mostly idle.
> 
> I wrote schbench to try and measure this:
> 
> git://git.kernel.org/pub/scm/linux/kernel/git/mason/schbench.git

Can you guys have a play with this; I think one and two node tbench are
good, but I seem to be getting significant run to run variance on that,
so maybe I'm not doing it right.

schbench numbers with: ./schbench -m2 -t 20 -c 30000 -s 30000 -r 30
on my ivb-ep (2 sockets, 10 cores/socket, 2 threads/core) appear to be
decent.

I've also not ran anything other than schbench/tbench so maybe I
completely wrecked something else (as per usual..).

I've not thought about that bounce_to_target() thing much.. I'll go give
that a ponder.

---
 kernel/sched/fair.c      | 180 +++++++++++++++++++++++++++++++++++------------
 kernel/sched/features.h  |   1 +
 kernel/sched/idle_task.c |   4 +-
 kernel/sched/sched.h     |   1 +
 kernel/time/tick-sched.c |  10 +--
 5 files changed, 146 insertions(+), 50 deletions(-)

diff --git a/kernel/sched/fair.c b/kernel/sched/fair.c
index b8a33abce650..b9d8d1dc5183 100644
--- a/kernel/sched/fair.c
+++ b/kernel/sched/fair.c
@@ -1501,8 +1501,10 @@ static void task_numa_compare(struct task_numa_env *env,
 	 * One idle CPU per node is evaluated for a task numa move.
 	 * Call select_idle_sibling to maybe find a better one.
 	 */
-	if (!cur)
+	if (!cur) {
+		// XXX borken
 		env->dst_cpu = select_idle_sibling(env->p, env->dst_cpu);
+	}
 
 assign:
 	assigned = true;
@@ -4491,6 +4493,17 @@ static void dequeue_task_fair(struct rq *rq, struct task_struct *p, int flags)
 }
 
 #ifdef CONFIG_SMP
+
+/*
+ * Working cpumask for:
+ *   load_balance,
+ *   load_balance_newidle,
+ *   select_idle_core.
+ *
+ * Assumes softirqs are disabled when in use.
+ */
+DEFINE_PER_CPU(cpumask_var_t, load_balance_mask);
+
 #ifdef CONFIG_NO_HZ_COMMON
 /*
  * per rq 'load' arrray crap; XXX kill this.
@@ -5162,65 +5175,147 @@ find_idlest_cpu(struct sched_group *group, struct task_struct *p, int this_cpu)
 	return shallowest_idle_cpu != -1 ? shallowest_idle_cpu : least_loaded_cpu;
 }
 
+#ifdef CONFIG_SCHED_SMT
+
+static inline void clear_idle_cores(int cpu)
+{
+	struct sched_domain *sd = rcu_dereference(per_cpu(sd_busy, cpu));
+	if (!sd)
+		return;
+
+	WRITE_ONCE(sd->groups->sgc->has_idle_cores, 0);
+}
+
+static inline void set_idle_cores(int cpu)
+{
+	struct sched_domain *sd = rcu_dereference(per_cpu(sd_busy, cpu));
+	if (!sd)
+		return;
+
+	WRITE_ONCE(sd->groups->sgc->has_idle_cores, 1);
+}
+
+static inline bool test_idle_cores(int cpu)
+{
+	struct sched_domain *sd = rcu_dereference(per_cpu(sd_busy, cpu));
+	if (!sd)
+		return false;
+
+	// XXX static key for !SMT topologies
+
+	return READ_ONCE(sd->groups->sgc->has_idle_cores);
+}
+
+void update_idle_core(struct rq *rq)
+{
+	int core = cpu_of(rq);
+	int cpu;
+
+	rcu_read_lock();
+	if (test_idle_cores(core))
+		goto unlock;
+
+	for_each_cpu(cpu, cpu_smt_mask(core)) {
+		if (cpu == core)
+			continue;
+
+		if (!idle_cpu(cpu))
+			goto unlock;
+	}
+
+	set_idle_cores(core);
+unlock:
+	rcu_read_unlock();
+}
+
+static int select_idle_core(struct task_struct *p, int target)
+{
+	struct cpumask *cpus = this_cpu_cpumask_var_ptr(load_balance_mask);
+	struct sched_domain *sd;
+	int core, cpu;
+
+	sd = rcu_dereference(per_cpu(sd_llc, target));
+	cpumask_and(cpus, sched_domain_span(sd), tsk_cpus_allowed(p));
+	for_each_cpu(core, cpus) {
+		bool idle = true;
+
+		for_each_cpu(cpu, cpu_smt_mask(core)) {
+			cpumask_clear_cpu(cpu, cpus);
+			if (!idle_cpu(cpu))
+				idle = false;
+		}
+
+		if (idle)
+			break;
+	}
+
+	return core;
+}
+
+#else /* CONFIG_SCHED_SMT */
+
+static inline void clear_idle_cores(int cpu) { }
+static inline void set_idle_cores(int cpu) { }
+
+static inline bool test_idle_cores(int cpu)
+{
+	return false;
+}
+
+void update_idle_core(struct rq *rq) { }
+
+static inline int select_idle_core(struct task_struct *p, int target)
+{
+	return -1;
+}
+
+#endif /* CONFIG_SCHED_SMT */
+
 /*
- * Try and locate an idle CPU in the sched_domain.
+ * Try and locate an idle core/thread in the LLC cache domain.
  */
 static int select_idle_sibling(struct task_struct *p, int target)
 {
 	struct sched_domain *sd;
-	struct sched_group *sg;
 	int i = task_cpu(p);
 
 	if (idle_cpu(target))
 		return target;
 
 	/*
-	 * If the prevous cpu is cache affine and idle, don't be stupid.
+	 * If the previous cpu is cache affine and idle, don't be stupid.
 	 */
 	if (i != target && cpus_share_cache(i, target) && idle_cpu(i))
 		return i;
 
+	sd = rcu_dereference(per_cpu(sd_llc, target));
+	if (!sd)
+		return target;
+
 	/*
-	 * Otherwise, iterate the domains and find an eligible idle cpu.
-	 *
-	 * A completely idle sched group at higher domains is more
-	 * desirable than an idle group at a lower level, because lower
-	 * domains have smaller groups and usually share hardware
-	 * resources which causes tasks to contend on them, e.g. x86
-	 * hyperthread siblings in the lowest domain (SMT) can contend
-	 * on the shared cpu pipeline.
-	 *
-	 * However, while we prefer idle groups at higher domains
-	 * finding an idle cpu at the lowest domain is still better than
-	 * returning 'target', which we've already established, isn't
-	 * idle.
+	 * If there are idle cores to be had, go find one.
 	 */
-	sd = rcu_dereference(per_cpu(sd_llc, target));
-	for_each_lower_domain(sd) {
-		sg = sd->groups;
-		do {
-			if (!cpumask_intersects(sched_group_cpus(sg),
-						tsk_cpus_allowed(p)))
-				goto next;
-
-			/* Ensure the entire group is idle */
-			for_each_cpu(i, sched_group_cpus(sg)) {
-				if (i == target || !idle_cpu(i))
-					goto next;
-			}
+	if (sched_feat(IDLE_CORE) && test_idle_cores(target)) {
+		i = select_idle_core(p, target);
+		if ((unsigned)i < nr_cpumask_bits)
+			return i;
 
-			/*
-			 * It doesn't matter which cpu we pick, the
-			 * whole group is idle.
-			 */
-			target = cpumask_first_and(sched_group_cpus(sg),
-					tsk_cpus_allowed(p));
-			goto done;
-next:
-			sg = sg->next;
-		} while (sg != sd->groups);
+		/*
+		 * Failed to find an idle core; stop looking for one.
+		 */
+		clear_idle_cores(target);
 	}
-done:
+
+	/*
+	 * Otherwise, settle for anything idle in this cache domain.
+	 */
+	for_each_cpu(i, sched_domain_span(sd)) {
+		if (!cpumask_test_cpu(i, tsk_cpus_allowed(p)))
+			continue;
+		if (idle_cpu(i))
+			return i;
+	}
+
 	return target;
 }
 
@@ -7229,9 +7324,6 @@ static struct rq *find_busiest_queue(struct lb_env *env,
  */
 #define MAX_PINNED_INTERVAL	512
 
-/* Working cpumask for load_balance and load_balance_newidle. */
-DEFINE_PER_CPU(cpumask_var_t, load_balance_mask);
-
 static int need_active_balance(struct lb_env *env)
 {
 	struct sched_domain *sd = env->sd;
diff --git a/kernel/sched/features.h b/kernel/sched/features.h
index 69631fa46c2f..76bb8814649a 100644
--- a/kernel/sched/features.h
+++ b/kernel/sched/features.h
@@ -69,3 +69,4 @@ SCHED_FEAT(RT_RUNTIME_SHARE, true)
 SCHED_FEAT(LB_MIN, false)
 SCHED_FEAT(ATTACH_AGE_LOAD, true)
 
+SCHED_FEAT(IDLE_CORE, true)
diff --git a/kernel/sched/idle_task.c b/kernel/sched/idle_task.c
index 47ce94931f1b..cb394db407e4 100644
--- a/kernel/sched/idle_task.c
+++ b/kernel/sched/idle_task.c
@@ -23,11 +23,13 @@ static void check_preempt_curr_idle(struct rq *rq, struct task_struct *p, int fl
 	resched_curr(rq);
 }
 
+extern void update_idle_core(struct rq *rq);
+
 static struct task_struct *
 pick_next_task_idle(struct rq *rq, struct task_struct *prev)
 {
 	put_prev_task(rq, prev);
-
+	update_idle_core(rq);
 	schedstat_inc(rq, sched_goidle);
 	return rq->idle;
 }
diff --git a/kernel/sched/sched.h b/kernel/sched/sched.h
index 69da6fcaa0e8..5994794bfc85 100644
--- a/kernel/sched/sched.h
+++ b/kernel/sched/sched.h
@@ -866,6 +866,7 @@ struct sched_group_capacity {
 	 * Number of busy cpus in this group.
 	 */
 	atomic_t nr_busy_cpus;
+	int	has_idle_cores;
 
 	unsigned long cpumask[0]; /* iteration mask */
 };
diff --git a/kernel/time/tick-sched.c b/kernel/time/tick-sched.c
index 31872bc53bc4..6e42cd218ba5 100644
--- a/kernel/time/tick-sched.c
+++ b/kernel/time/tick-sched.c
@@ -933,11 +933,11 @@ void tick_nohz_idle_enter(void)
 	WARN_ON_ONCE(irqs_disabled());
 
 	/*
- 	 * Update the idle state in the scheduler domain hierarchy
- 	 * when tick_nohz_stop_sched_tick() is called from the idle loop.
- 	 * State will be updated to busy during the first busy tick after
- 	 * exiting idle.
- 	 */
+	 * Update the idle state in the scheduler domain hierarchy
+	 * when tick_nohz_stop_sched_tick() is called from the idle loop.
+	 * State will be updated to busy during the first busy tick after
+	 * exiting idle.
+	 */
 	set_cpu_sd_state_idle();
 
 	local_irq_disable();

^ permalink raw reply related	[flat|nested] 80+ messages in thread

* Re: sched: tweak select_idle_sibling to look for idle threads
  2016-04-30 12:47   ` Peter Zijlstra
@ 2016-05-01  7:12     ` Mike Galbraith
  2016-05-01  8:53       ` Peter Zijlstra
  2016-05-02  8:46       ` Peter Zijlstra
  0 siblings, 2 replies; 80+ messages in thread
From: Mike Galbraith @ 2016-05-01  7:12 UTC (permalink / raw)
  To: Peter Zijlstra, Chris Mason, Ingo Molnar, Matt Fleming, linux-kernel

On Sat, 2016-04-30 at 14:47 +0200, Peter Zijlstra wrote:
> On Sat, Apr 09, 2016 at 03:05:54PM -0400, Chris Mason wrote:
> > select_task_rq_fair() can leave cpu utilization a little lumpy,
> > especially as the workload ramps up to the maximum capacity of the
> > machine.  The end result can be high p99 response times as apps
> > wait to get scheduled, even when boxes are mostly idle.
> > 
> > I wrote schbench to try and measure this:
> > 
> > git://git.kernel.org/pub/scm/linux/kernel/git/mason/schbench.git
> 
> Can you guys have a play with this; I think one and two node tbench are
> good, but I seem to be getting significant run to run variance on that,
> so maybe I'm not doing it right.

Nah, tbench is just variance prone.  It got dinged up at clients=cores
on my desktop box, on 4 sockets the high end got seriously dinged up.

tbench

1 x i4790
master                          avg
1     714       684     688     695   1.000
2    1260      1234    1284    1259   1.000
4    2238      2301    2286    2275   1.000
8    3388      3418    3396    3400   1.000

masterx
1     690       701     701     697   1.002
2    1287      1332    1235    1284   1.019
4    2014      2006    1999    2006   0.881
8    3388      3385    3404    3392   0.997

4 x E7-8890
master                          avg
1     524      524      523     523   1.000
2    1049     1053     1045    1049   1.000
4    2064     2081     2091    2078   1.000
8    3737     3813     3746    3765   1.000
16   7129     7028     7082    7079   1.000
32  13718    13730    13578   13675   1.000
64  21397    21435    21519   21450   1.000
128 39846    38397    39026   39089   1.000
256 59509    59797    59344   59550   1.000

masterx                         avg
1     505      507      501     504   0.963   1.000
2    1036     1027     1039    1034   0.985   1.000
4    1977     2001     1992    1990   0.957   1.000
8    3734     3802     3778    3771   1.001   1.000
16   7124     7079     7071    7091   1.001   1.000
32  13549    13758    13364   13557   0.991   1.000
64  21975    22161    22100   22078   1.029   1.000
128 23066    23044    23028   23046   0.589   1.000
256 29905    29630    30753   30096   0.505   1.000

masterx NO_IDLE_CORE            avg
1     500      521      502     507   0.969   1.005
2    1012      996     1043    1017   0.969   0.983
4    1988     1992     1995    1991   0.958   1.000
8    3834     3758     3671    3754   0.997   0.995
16   7160     7206     7168    7178   1.013   1.012
32  13788    13773    13672   13744   1.005   1.013
64  21771    21845    21826   21814   1.016   0.988
128 23248    23136    23133   23172   0.592   1.005
256 28683    30013    31850   30182   0.506   1.002

^ permalink raw reply	[flat|nested] 80+ messages in thread

* Re: sched: tweak select_idle_sibling to look for idle threads
  2016-05-01  7:12     ` Mike Galbraith
@ 2016-05-01  8:53       ` Peter Zijlstra
  2016-05-01  9:20         ` Mike Galbraith
  2016-05-02  8:46       ` Peter Zijlstra
  1 sibling, 1 reply; 80+ messages in thread
From: Peter Zijlstra @ 2016-05-01  8:53 UTC (permalink / raw)
  To: Mike Galbraith; +Cc: Chris Mason, Ingo Molnar, Matt Fleming, linux-kernel

On Sun, May 01, 2016 at 09:12:33AM +0200, Mike Galbraith wrote:
> On Sat, 2016-04-30 at 14:47 +0200, Peter Zijlstra wrote:

> > Can you guys have a play with this; I think one and two node tbench are
> > good, but I seem to be getting significant run to run variance on that,
> > so maybe I'm not doing it right.
> 
> Nah, tbench is just variance prone.  It got dinged up at clients=cores
> on my desktop box, on 4 sockets the high end got seriously dinged up.

Ouch, yeah, big hurt. Lets try that again... :-)

^ permalink raw reply	[flat|nested] 80+ messages in thread

* Re: sched: tweak select_idle_sibling to look for idle threads
  2016-05-01  8:53       ` Peter Zijlstra
@ 2016-05-01  9:20         ` Mike Galbraith
  2016-05-07  1:24           ` Yuyang Du
  0 siblings, 1 reply; 80+ messages in thread
From: Mike Galbraith @ 2016-05-01  9:20 UTC (permalink / raw)
  To: Peter Zijlstra; +Cc: Chris Mason, Ingo Molnar, Matt Fleming, linux-kernel

On Sun, 2016-05-01 at 10:53 +0200, Peter Zijlstra wrote:
> On Sun, May 01, 2016 at 09:12:33AM +0200, Mike Galbraith wrote:
> > On Sat, 2016-04-30 at 14:47 +0200, Peter Zijlstra wrote:
> 
> > > Can you guys have a play with this; I think one and two node tbench are
> > > good, but I seem to be getting significant run to run variance on that,
> > > so maybe I'm not doing it right.
> > 
> > Nah, tbench is just variance prone.  It got dinged up at clients=cores
> > on my desktop box, on 4 sockets the high end got seriously dinged up.
> 
> Ouch, yeah, big hurt. Lets try that again... :-)

Yeah, box could use a little bandaid and a hug :)

Playing with Chris' benchmark, seems the biggest problem is that we
don't buddy up waker of many and it's wakees in a node.. ie the wake
wide thing isn't necessarily our friend when there are multiple wakers
of many.  If I run an instance per node with one mother of all work in
autobench mode, it works exactly as you'd expect, game over is when
wakees = socket size. It never get's near that point if I let things
wander, it beats itself up well before we get there.

	-Mike

^ permalink raw reply	[flat|nested] 80+ messages in thread

* Re: [PATCH RFC] select_idle_sibling experiments
  2016-04-28 12:00   ` Peter Zijlstra
  2016-04-28 13:17     ` Mike Galbraith
@ 2016-05-02  5:35     ` Mike Galbraith
  1 sibling, 0 replies; 80+ messages in thread
From: Mike Galbraith @ 2016-05-02  5:35 UTC (permalink / raw)
  To: Peter Zijlstra; +Cc: Chris Mason, Ingo Molnar, Matt Fleming, linux-kernel

On Thu, 2016-04-28 at 14:00 +0200, Peter Zijlstra wrote:
> On Wed, Apr 06, 2016 at 09:27:24AM +0200, Mike Galbraith wrote:
> > sched: ratelimit nohz
> > 
> > Entering nohz code on every micro-idle is too expensive to bear.
> > 
> > Signed-off-by: Mike Galbraith <efault@gmx.de>
> 
> > +int sched_needs_cpu(int cpu)
> > +{
> > +> > 	> > if (tick_nohz_full_cpu(cpu))
> > +> > 	> > 	> > return 0;
> > +
> > +> > 	> > return  cpu_rq(cpu)->avg_idle < sysctl_sched_migration_cost;
> 
> So the only problem I have with this patch is the choice of limit. This
> isn't at all tied to the migration cost.
> 
> And some people are already twiddling with the migration_cost knob to
> affect the idle_balance() behaviour -- making it much more agressive by
> dialing it down. When you do that you also loose the effectiveness of
> this proposed usage, even though those same people would probably want
> this.
> 
> Failing a spot of inspiration for a runtime limit on this; we might have
> to introduce yet another knob :/

sched: ratelimit nohz tick shutdown/restart

Tick shutdown/restart overhead can be substantial when CPUs
enter/exit the idle loop at high frequency.  Ratelimit based
upon rq->avg_idle, and provide an adjustment knob.

Signed-off-by: Mike Galbraith <mgalbraith@suse.de>
---
 include/linux/sched.h        |    5 +++++
 include/linux/sched/sysctl.h |    4 ++++
 kernel/sched/core.c          |   10 ++++++++++
 kernel/sysctl.c              |    9 +++++++++
 kernel/time/tick-sched.c     |    2 +-
 5 files changed, 29 insertions(+), 1 deletion(-)

--- a/include/linux/sched.h
+++ b/include/linux/sched.h
@@ -2286,6 +2286,11 @@ static inline int set_cpus_allowed_ptr(s
 #ifdef CONFIG_NO_HZ_COMMON
 void calc_load_enter_idle(void);
 void calc_load_exit_idle(void);
+#ifdef CONFIG_SMP
+extern int sched_needs_cpu(int cpu);
+#else
+static inline int sched_needs_cpu(int cpu) { return 0; }
+#endif
 #else
 static inline void calc_load_enter_idle(void) { }
 static inline void calc_load_exit_idle(void) { }
--- a/include/linux/sched/sysctl.h
+++ b/include/linux/sched/sysctl.h
@@ -19,6 +19,10 @@ extern unsigned int sysctl_sched_min_gra
 extern unsigned int sysctl_sched_wakeup_granularity;
 extern unsigned int sysctl_sched_child_runs_first;
 
+#if defined(CONFIG_NO_HZ_COMMON) && defined(CONFIG_SMP)
+extern unsigned int sysctl_sched_nohz_throttle;
+#endif
+
 enum sched_tunable_scaling {
 	SCHED_TUNABLESCALING_NONE,
 	SCHED_TUNABLESCALING_LOG,
--- a/kernel/sched/core.c
+++ b/kernel/sched/core.c
@@ -577,6 +577,16 @@ static inline bool got_nohz_idle_kick(vo
 	return false;
 }
 
+unsigned int sysctl_sched_nohz_throttle = 500000UL;
+
+int sched_needs_cpu(int cpu)
+{
+	if (tick_nohz_full_cpu(cpu))
+		return 0;
+
+	return  cpu_rq(cpu)->avg_idle < sysctl_sched_nohz_throttle;
+}
+
 #else /* CONFIG_NO_HZ_COMMON */
 
 static inline bool got_nohz_idle_kick(void)
--- a/kernel/sysctl.c
+++ b/kernel/sysctl.c
@@ -351,6 +351,15 @@ static struct ctl_table kern_table[] = {
 		.mode		= 0644,
 		.proc_handler	= proc_dointvec,
 	},
+#ifdef CONFIG_NO_HZ_COMMON
+	{
+		.procname	= "sched_nohz_throttle_ns",
+		.data		= &sysctl_sched_nohz_throttle,
+		.maxlen		= sizeof(unsigned int),
+		.mode		= 0644,
+		.proc_handler	= proc_dointvec,
+	},
+#endif
 #ifdef CONFIG_SCHEDSTATS
 	{
 		.procname	= "sched_schedstats",
--- a/kernel/time/tick-sched.c
+++ b/kernel/time/tick-sched.c
@@ -676,7 +676,7 @@ static ktime_t tick_nohz_stop_sched_tick
 	} while (read_seqretry(&jiffies_lock, seq));
 	ts->last_jiffies = basejiff;
 
-	if (rcu_needs_cpu(basemono, &next_rcu) ||
+	if (sched_needs_cpu(cpu) || rcu_needs_cpu(basemono, &next_rcu) ||
 	    arch_needs_cpu() || irq_work_needs_cpu()) {
 		next_tick = basemono + TICK_NSEC;
 	} else {

^ permalink raw reply	[flat|nested] 80+ messages in thread

* Re: sched: tweak select_idle_sibling to look for idle threads
  2016-05-01  7:12     ` Mike Galbraith
  2016-05-01  8:53       ` Peter Zijlstra
@ 2016-05-02  8:46       ` Peter Zijlstra
  2016-05-02 14:50         ` Mike Galbraith
  1 sibling, 1 reply; 80+ messages in thread
From: Peter Zijlstra @ 2016-05-02  8:46 UTC (permalink / raw)
  To: Mike Galbraith; +Cc: Chris Mason, Ingo Molnar, Matt Fleming, linux-kernel

On Sun, May 01, 2016 at 09:12:33AM +0200, Mike Galbraith wrote:

> Nah, tbench is just variance prone.  It got dinged up at clients=cores
> on my desktop box, on 4 sockets the high end got seriously dinged up.


Ha!, check this:

root@ivb-ep:~# echo OLD_IDLE > /debug/sched_features ; echo
NO_ORDER_IDLE > /debug/sched_features ; echo IDLE_CORE >
/debug/sched_features ; echo NO_FORCE_CORE > /debug/sched_features ;
tbench 20 -t 10

Throughput 5956.32 MB/sec  20 clients  20 procs  max_latency=0.126 ms


root@ivb-ep:~# echo OLD_IDLE > /debug/sched_features ; echo ORDER_IDLE >
/debug/sched_features ; echo IDLE_CORE > /debug/sched_features ; echo
NO_FORCE_CORE > /debug/sched_features ; tbench 20 -t 10

Throughput 5011.86 MB/sec  20 clients  20 procs  max_latency=0.116 ms



That little ORDER_IDLE thing hurts silly. That's a little patch I had
lying about because some people complained that tasks hop around the
cache domain, instead of being stuck to a CPU.

I suspect what happens is that by all CPUs starting to look for idle at
the same place (the first cpu in the domain) they all find the same idle
cpu and things pile up.

The old behaviour, where they all start iterating from where they were
avoids some of that, at the cost of making tasks hop around.

Lets see if I can get the same behaviour out of the cpumask iteration
code..





---
 kernel/sched/fair.c      | 218 +++++++++++++++++++++++++++++++++++++----------
 kernel/sched/features.h  |   4 +
 kernel/sched/idle_task.c |   4 +-
 kernel/sched/sched.h     |   1 +
 kernel/time/tick-sched.c |  10 +--
 5 files changed, 188 insertions(+), 49 deletions(-)

diff --git a/kernel/sched/fair.c b/kernel/sched/fair.c
index b8a33ab..b7626a4 100644
--- a/kernel/sched/fair.c
+++ b/kernel/sched/fair.c
@@ -1501,8 +1501,10 @@ balance:
 	 * One idle CPU per node is evaluated for a task numa move.
 	 * Call select_idle_sibling to maybe find a better one.
 	 */
-	if (!cur)
+	if (!cur) {
+		// XXX borken
 		env->dst_cpu = select_idle_sibling(env->p, env->dst_cpu);
+	}
 
 assign:
 	assigned = true;
@@ -4491,6 +4493,17 @@ static void dequeue_task_fair(struct rq *rq, struct task_struct *p, int flags)
 }
 
 #ifdef CONFIG_SMP
+
+/* 
+ * Working cpumask for:
+ *   load_balance, 
+ *   load_balance_newidle,
+ *   select_idle_core.
+ *
+ * Assumes softirqs are disabled when in use.
+ */
+DEFINE_PER_CPU(cpumask_var_t, load_balance_mask);
+
 #ifdef CONFIG_NO_HZ_COMMON
 /*
  * per rq 'load' arrray crap; XXX kill this.
@@ -5162,65 +5175,187 @@ find_idlest_cpu(struct sched_group *group, struct task_struct *p, int this_cpu)
 	return shallowest_idle_cpu != -1 ? shallowest_idle_cpu : least_loaded_cpu;
 }
 
+#ifdef CONFIG_SCHED_SMT
+
+static inline void clear_idle_cores(int cpu)
+{
+	struct sched_domain *sd = rcu_dereference(per_cpu(sd_busy, cpu));
+	if (!sd)
+		return;
+
+	WRITE_ONCE(sd->groups->sgc->has_idle_cores, 0);
+}
+
+static inline void set_idle_cores(int cpu)
+{
+	struct sched_domain *sd = rcu_dereference(per_cpu(sd_busy, cpu));
+	if (!sd)
+		return;
+
+	WRITE_ONCE(sd->groups->sgc->has_idle_cores, 1);
+}
+
+static inline bool test_idle_cores(int cpu)
+{
+	struct sched_domain *sd = rcu_dereference(per_cpu(sd_busy, cpu));
+	if (!sd)
+		return false;
+
+	// XXX static key for !SMT topologies
+
+	return READ_ONCE(sd->groups->sgc->has_idle_cores);
+}
+
+void update_idle_core(struct rq *rq)
+{
+	int core = cpu_of(rq);
+	int cpu;
+
+	rcu_read_lock();
+	if (test_idle_cores(core))
+		goto unlock;
+
+	for_each_cpu(cpu, cpu_smt_mask(core)) {
+		if (cpu == core)
+			continue;
+
+		if (!idle_cpu(cpu))
+			goto unlock;
+	}
+
+	set_idle_cores(core);
+unlock:
+	rcu_read_unlock();
+}
+
+static int select_idle_core(struct task_struct *p, int target)
+{
+	struct cpumask *cpus = this_cpu_cpumask_var_ptr(load_balance_mask);
+	struct sched_domain *sd;
+	int core, cpu;
+
+	sd = rcu_dereference(per_cpu(sd_llc, target));
+	cpumask_and(cpus, sched_domain_span(sd), tsk_cpus_allowed(p));
+	for_each_cpu(core, cpus) {
+		bool idle = true;
+
+		for_each_cpu(cpu, cpu_smt_mask(core)) {
+			cpumask_clear_cpu(cpu, cpus);
+			if (!idle_cpu(cpu))
+				idle = false;
+		}
+
+		if (idle)
+			break;
+	}
+
+	return core;
+}
+
+#else /* CONFIG_SCHED_SMT */
+
+static inline void clear_idle_cores(int cpu) { }
+static inline void set_idle_cores(int cpu) { }
+
+static inline bool test_idle_cores(int cpu)
+{
+	return false;
+}
+
+void update_idle_core(struct rq *rq) { }
+
+static inline int select_idle_core(struct task_struct *p, int target)
+{
+	return -1;
+}
+
+#endif /* CONFIG_SCHED_SMT */
+
 /*
- * Try and locate an idle CPU in the sched_domain.
+ * Try and locate an idle core/thread in the LLC cache domain.
  */
 static int select_idle_sibling(struct task_struct *p, int target)
 {
 	struct sched_domain *sd;
-	struct sched_group *sg;
 	int i = task_cpu(p);
 
 	if (idle_cpu(target))
 		return target;
 
 	/*
-	 * If the prevous cpu is cache affine and idle, don't be stupid.
+	 * If the previous cpu is cache affine and idle, don't be stupid.
 	 */
 	if (i != target && cpus_share_cache(i, target) && idle_cpu(i))
 		return i;
 
-	/*
-	 * Otherwise, iterate the domains and find an eligible idle cpu.
-	 *
-	 * A completely idle sched group at higher domains is more
-	 * desirable than an idle group at a lower level, because lower
-	 * domains have smaller groups and usually share hardware
-	 * resources which causes tasks to contend on them, e.g. x86
-	 * hyperthread siblings in the lowest domain (SMT) can contend
-	 * on the shared cpu pipeline.
-	 *
-	 * However, while we prefer idle groups at higher domains
-	 * finding an idle cpu at the lowest domain is still better than
-	 * returning 'target', which we've already established, isn't
-	 * idle.
-	 */
-	sd = rcu_dereference(per_cpu(sd_llc, target));
-	for_each_lower_domain(sd) {
-		sg = sd->groups;
-		do {
-			if (!cpumask_intersects(sched_group_cpus(sg),
-						tsk_cpus_allowed(p)))
-				goto next;
+	i = target;
+	if (sched_feat(ORDER_IDLE))
+		i = per_cpu(sd_llc_id, target); /* first cpu in llc domain */
+	sd = rcu_dereference(per_cpu(sd_llc, i));
+	if (!sd)
+		return target;
+
+	if (sched_feat(OLD_IDLE)) {
+		struct sched_group *sg;
 
-			/* Ensure the entire group is idle */
-			for_each_cpu(i, sched_group_cpus(sg)) {
-				if (i == target || !idle_cpu(i))
+		for_each_lower_domain(sd) {
+			sg = sd->groups;
+			do {
+				if (!cpumask_intersects(sched_group_cpus(sg),
+							tsk_cpus_allowed(p)))
 					goto next;
-			}
 
-			/*
-			 * It doesn't matter which cpu we pick, the
-			 * whole group is idle.
-			 */
-			target = cpumask_first_and(sched_group_cpus(sg),
-					tsk_cpus_allowed(p));
-			goto done;
+				/* Ensure the entire group is idle */
+				for_each_cpu(i, sched_group_cpus(sg)) {
+					if (i == target || !idle_cpu(i))
+						goto next;
+				}
+
+				/*
+				 * It doesn't matter which cpu we pick, the
+				 * whole group is idle.
+				 */
+				target = cpumask_first_and(sched_group_cpus(sg),
+						tsk_cpus_allowed(p));
+				goto done;
 next:
-			sg = sg->next;
-		} while (sg != sd->groups);
-	}
+				sg = sg->next;
+			} while (sg != sd->groups);
+		}
 done:
+		return target;
+	}
+
+	/*
+	 * If there are idle cores to be had, go find one.
+	 */
+	if (sched_feat(IDLE_CORE) && test_idle_cores(target)) {
+		i = select_idle_core(p, target);
+		if ((unsigned)i < nr_cpumask_bits)
+			return i;
+
+		/*
+		 * Failed to find an idle core; stop looking for one.
+		 */
+		clear_idle_cores(target);
+	}
+
+	if (sched_feat(FORCE_CORE)) {
+		i = select_idle_core(p, target);
+		if ((unsigned)i < nr_cpumask_bits)
+			return i;
+	}
+
+	/*
+	 * Otherwise, settle for anything idle in this cache domain.
+	 */
+	for_each_cpu(i, sched_domain_span(sd)) {
+		if (!cpumask_test_cpu(i, tsk_cpus_allowed(p)))
+			continue;
+		if (idle_cpu(i))
+			return i;
+	}
+
 	return target;
 }
 
@@ -7229,9 +7364,6 @@ static struct rq *find_busiest_queue(struct lb_env *env,
  */
 #define MAX_PINNED_INTERVAL	512
 
-/* Working cpumask for load_balance and load_balance_newidle. */
-DEFINE_PER_CPU(cpumask_var_t, load_balance_mask);
-
 static int need_active_balance(struct lb_env *env)
 {
 	struct sched_domain *sd = env->sd;
diff --git a/kernel/sched/features.h b/kernel/sched/features.h
index 69631fa..11ab7ab 100644
--- a/kernel/sched/features.h
+++ b/kernel/sched/features.h
@@ -69,3 +69,7 @@ SCHED_FEAT(RT_RUNTIME_SHARE, true)
 SCHED_FEAT(LB_MIN, false)
 SCHED_FEAT(ATTACH_AGE_LOAD, true)
 
+SCHED_FEAT(OLD_IDLE, false)
+SCHED_FEAT(ORDER_IDLE, false)
+SCHED_FEAT(IDLE_CORE, true)
+SCHED_FEAT(FORCE_CORE, false)
diff --git a/kernel/sched/idle_task.c b/kernel/sched/idle_task.c
index 47ce949..cb394db 100644
--- a/kernel/sched/idle_task.c
+++ b/kernel/sched/idle_task.c
@@ -23,11 +23,13 @@ static void check_preempt_curr_idle(struct rq *rq, struct task_struct *p, int fl
 	resched_curr(rq);
 }
 
+extern void update_idle_core(struct rq *rq);
+
 static struct task_struct *
 pick_next_task_idle(struct rq *rq, struct task_struct *prev)
 {
 	put_prev_task(rq, prev);
-
+	update_idle_core(rq);
 	schedstat_inc(rq, sched_goidle);
 	return rq->idle;
 }
diff --git a/kernel/sched/sched.h b/kernel/sched/sched.h
index 69da6fc..5994794 100644
--- a/kernel/sched/sched.h
+++ b/kernel/sched/sched.h
@@ -866,6 +866,7 @@ struct sched_group_capacity {
 	 * Number of busy cpus in this group.
 	 */
 	atomic_t nr_busy_cpus;
+	int	has_idle_cores;
 
 	unsigned long cpumask[0]; /* iteration mask */
 };
diff --git a/kernel/time/tick-sched.c b/kernel/time/tick-sched.c
index 31872bc..6e42cd2 100644
--- a/kernel/time/tick-sched.c
+++ b/kernel/time/tick-sched.c
@@ -933,11 +933,11 @@ void tick_nohz_idle_enter(void)
 	WARN_ON_ONCE(irqs_disabled());
 
 	/*
- 	 * Update the idle state in the scheduler domain hierarchy
- 	 * when tick_nohz_stop_sched_tick() is called from the idle loop.
- 	 * State will be updated to busy during the first busy tick after
- 	 * exiting idle.
- 	 */
+	 * Update the idle state in the scheduler domain hierarchy
+	 * when tick_nohz_stop_sched_tick() is called from the idle loop.
+	 * State will be updated to busy during the first busy tick after
+	 * exiting idle.
+	 */
 	set_cpu_sd_state_idle();
 
 	local_irq_disable();

^ permalink raw reply related	[flat|nested] 80+ messages in thread

* Re: sched: tweak select_idle_sibling to look for idle threads
  2016-05-02  8:46       ` Peter Zijlstra
@ 2016-05-02 14:50         ` Mike Galbraith
  2016-05-02 14:58           ` Peter Zijlstra
                             ` (2 more replies)
  0 siblings, 3 replies; 80+ messages in thread
From: Mike Galbraith @ 2016-05-02 14:50 UTC (permalink / raw)
  To: Peter Zijlstra; +Cc: Chris Mason, Ingo Molnar, Matt Fleming, linux-kernel

On Mon, 2016-05-02 at 10:46 +0200, Peter Zijlstra wrote:
> On Sun, May 01, 2016 at 09:12:33AM +0200, Mike Galbraith wrote:
> 
> > Nah, tbench is just variance prone.  It got dinged up at clients=cores
> > on my desktop box, on 4 sockets the high end got seriously dinged up.
> 
> 
> Ha!, check this:
> 
> root@ivb-ep:~# echo OLD_IDLE > /debug/sched_features ; echo
> NO_ORDER_IDLE > /debug/sched_features ; echo IDLE_CORE >
> /debug/sched_features ; echo NO_FORCE_CORE > /debug/sched_features ;
> tbench 20 -t 10
> 
> Throughput 5956.32 MB/sec  20 clients  20 procs  max_latency=0.126 ms
> 
> 
> root@ivb-ep:~# echo OLD_IDLE > /debug/sched_features ; echo ORDER_IDLE >
> /debug/sched_features ; echo IDLE_CORE > /debug/sched_features ; echo
> NO_FORCE_CORE > /debug/sched_features ; tbench 20 -t 10
> 
> Throughput 5011.86 MB/sec  20 clients  20 procs  max_latency=0.116 ms
> 
> 
> 
> That little ORDER_IDLE thing hurts silly. That's a little patch I had
> lying about because some people complained that tasks hop around the
> cache domain, instead of being stuck to a CPU.
> 
> I suspect what happens is that by all CPUs starting to look for idle at
> the same place (the first cpu in the domain) they all find the same idle
> cpu and things pile up.
> 
> The old behaviour, where they all start iterating from where they were
> avoids some of that, at the cost of making tasks hop around.
> 
> Lets see if I can get the same behaviour out of the cpumask iteration
> code..

Order is one thing, but what the old behavior does first and foremost
is when the box starts getting really busy, only looking at target's
sibling shuts select_idle_sibling() down instead of letting it wreck
things.  Once cores are moving, there are no large piles of anything
left to collect other than pain.

We really need a good way to know we're not gonna turn the box into a
shredder.  The wake_wide() thing might help some, likely wants some
twiddling, in_interrupt() might be another time to try hard.

Anyway, the has_idle_cores business seems to shut select_idle_sibling()
down rather nicely when the the box gets busy.  Forcing either core,
target's sibling or go fish turned in a top end win on 48 rq/socket.

Oh btw, did you know single socket boxen have no sd_busy?  That doesn't
look right.

fromm:~/:[0]# for i in 1 2 4 8 16 32 64 128 256; do tbench.sh $i 30 2>&1| grep Throughput; done
Throughput 511.016 MB/sec  1 clients  1 procs  max_latency=0.113 ms
Throughput 1042.03 MB/sec  2 clients  2 procs  max_latency=0.098 ms
Throughput 1953.12 MB/sec  4 clients  4 procs  max_latency=0.236 ms
Throughput 3694.99 MB/sec  8 clients  8 procs  max_latency=0.308 ms
Throughput 7080.95 MB/sec  16 clients  16 procs  max_latency=0.442 ms
Throughput 13444.7 MB/sec  32 clients  32 procs  max_latency=1.417 ms
Throughput 20191.3 MB/sec  64 clients  64 procs  max_latency=4.554 ms
Throughput 41115.4 MB/sec  128 clients  128 procs  max_latency=13.414 ms
Throughput 66844.4 MB/sec  256 clients  256 procs  max_latency=50.069 ms

5226         /*
5227          * If there are idle cores to be had, go find one.
5228          */
5229         if (sched_feat(IDLE_CORE) && test_idle_cores(target)) {
5230                 i = select_idle_core(p, target);
5231                 if ((unsigned)i < nr_cpumask_bits)
5232                         return i;
5233  
5234                 /*
5235                  * Failed to find an idle core; stop looking for one.
5236                  */
5237                 clear_idle_cores(target);
5238         }
5239 #if 1
5240         for_each_cpu(i, cpu_smt_mask(target)) {
5241                 if (idle_cpu(i))
5242                         return i;
5243         }
5244  
5245         return target;
5246 #endif
5247  
5248         if (sched_feat(FORCE_CORE)) {

^ permalink raw reply	[flat|nested] 80+ messages in thread

* Re: sched: tweak select_idle_sibling to look for idle threads
  2016-05-02 14:50         ` Mike Galbraith
@ 2016-05-02 14:58           ` Peter Zijlstra
  2016-05-02 15:47             ` Chris Mason
  2016-05-02 17:30             ` Mike Galbraith
  2016-05-02 15:01           ` Peter Zijlstra
  2016-05-02 15:10           ` Peter Zijlstra
  2 siblings, 2 replies; 80+ messages in thread
From: Peter Zijlstra @ 2016-05-02 14:58 UTC (permalink / raw)
  To: Mike Galbraith; +Cc: Chris Mason, Ingo Molnar, Matt Fleming, linux-kernel

On Mon, May 02, 2016 at 04:50:04PM +0200, Mike Galbraith wrote:
> Oh btw, did you know single socket boxen have no sd_busy?  That doesn't
> look right.

I suspected; didn't bother looking at yet. The 'problem' is that the LLC
domain is the top-most, so it doesn't have a parent domain. I'm sure we
can come up with something if we can get this all working right.

And yes, I can get gains on various workloads with various options, I
can even break all workloads, but I've so far completely failed on
getting a win for everyone :/

In particular low count sysbench-psql (oltp test) vs tbench
client==nr_cores is having me flummoxed for a bit.

^ permalink raw reply	[flat|nested] 80+ messages in thread

* Re: sched: tweak select_idle_sibling to look for idle threads
  2016-05-02 14:50         ` Mike Galbraith
  2016-05-02 14:58           ` Peter Zijlstra
@ 2016-05-02 15:01           ` Peter Zijlstra
  2016-05-02 16:04             ` Ingo Molnar
  2016-05-02 15:10           ` Peter Zijlstra
  2 siblings, 1 reply; 80+ messages in thread
From: Peter Zijlstra @ 2016-05-02 15:01 UTC (permalink / raw)
  To: Mike Galbraith; +Cc: Chris Mason, Ingo Molnar, Matt Fleming, linux-kernel

On Mon, May 02, 2016 at 04:50:04PM +0200, Mike Galbraith wrote:
> On Mon, 2016-05-02 at 10:46 +0200, Peter Zijlstra wrote:
> 5226         /*
> 5227          * If there are idle cores to be had, go find one.
> 5228          */
> 5229         if (sched_feat(IDLE_CORE) && test_idle_cores(target)) {
> 5230                 i = select_idle_core(p, target);
> 5231                 if ((unsigned)i < nr_cpumask_bits)
> 5232                         return i;
> 5233  
> 5234                 /*
> 5235                  * Failed to find an idle core; stop looking for one.
> 5236                  */
> 5237                 clear_idle_cores(target);
> 5238         }
> 5239 #if 1
> 5240         for_each_cpu(i, cpu_smt_mask(target)) {
> 5241                 if (idle_cpu(i))
> 5242                         return i;
> 5243         }
> 5244  
> 5245         return target;
> 5246 #endif

And yes, I have a variant of that, that does indeed work way better than
scanning the whole LLC domain for idle threads.

If you want a laugh, modify select_idle_core() to remember the last idle
thread it encounters and have it return that when it fails to find an
idle core.. I'm still stumped to explain why it behaves the way it does.

^ permalink raw reply	[flat|nested] 80+ messages in thread

* Re: sched: tweak select_idle_sibling to look for idle threads
  2016-05-02 14:50         ` Mike Galbraith
  2016-05-02 14:58           ` Peter Zijlstra
  2016-05-02 15:01           ` Peter Zijlstra
@ 2016-05-02 15:10           ` Peter Zijlstra
  2 siblings, 0 replies; 80+ messages in thread
From: Peter Zijlstra @ 2016-05-02 15:10 UTC (permalink / raw)
  To: Mike Galbraith; +Cc: Chris Mason, Ingo Molnar, Matt Fleming, linux-kernel

On Mon, May 02, 2016 at 04:50:04PM +0200, Mike Galbraith wrote:

> Order is one thing, but what the old behavior does first and foremost
> is when the box starts getting really busy, only looking at target's
> sibling shuts select_idle_sibling() down instead of letting it wreck
> things.  Once cores are moving, there are no large piles of anything
> left to collect other than pain.

> Anyway, the has_idle_cores business seems to shut select_idle_sibling()
> down rather nicely when the the box gets busy.  Forcing either core,
> target's sibling or go fish turned in a top end win on 48 rq/socket.

FWIW making the select_idle_core() thing iterate in the old style (start
at target and wrap around) did bring an improvement, even in the face of
has_idle_cores; it shrank the hole between OLD_IDLE and
IDLE_CORES+IDLE_SMT (my variant of your #if 1 thing), but did not
completely eliminate it (for sysbench-psql-oltp).

^ permalink raw reply	[flat|nested] 80+ messages in thread

* Re: sched: tweak select_idle_sibling to look for idle threads
  2016-05-02 14:58           ` Peter Zijlstra
@ 2016-05-02 15:47             ` Chris Mason
  2016-05-03 14:32               ` Peter Zijlstra
  2016-05-02 17:30             ` Mike Galbraith
  1 sibling, 1 reply; 80+ messages in thread
From: Chris Mason @ 2016-05-02 15:47 UTC (permalink / raw)
  To: Peter Zijlstra; +Cc: Mike Galbraith, Ingo Molnar, Matt Fleming, linux-kernel

On Mon, May 02, 2016 at 04:58:17PM +0200, Peter Zijlstra wrote:
> On Mon, May 02, 2016 at 04:50:04PM +0200, Mike Galbraith wrote:
> > Oh btw, did you know single socket boxen have no sd_busy?  That doesn't
> > look right.
> 
> I suspected; didn't bother looking at yet. The 'problem' is that the LLC
> domain is the top-most, so it doesn't have a parent domain. I'm sure we
> can come up with something if we can get this all working right.
> 
> And yes, I can get gains on various workloads with various options, I
> can even break all workloads, but I've so far completely failed on
> getting a win for everyone :/

Adding in the task_hot() check to decide if scanning idle was a good
idea ended up being really important.  I'm happy to try a few variations
here as well, do you have a more recent patch?

-chris

^ permalink raw reply	[flat|nested] 80+ messages in thread

* Re: sched: tweak select_idle_sibling to look for idle threads
  2016-05-02 15:01           ` Peter Zijlstra
@ 2016-05-02 16:04             ` Ingo Molnar
  2016-05-03 11:31               ` Peter Zijlstra
  0 siblings, 1 reply; 80+ messages in thread
From: Ingo Molnar @ 2016-05-02 16:04 UTC (permalink / raw)
  To: Peter Zijlstra; +Cc: Mike Galbraith, Chris Mason, Matt Fleming, linux-kernel


* Peter Zijlstra <peterz@infradead.org> wrote:

> On Mon, May 02, 2016 at 04:50:04PM +0200, Mike Galbraith wrote:
> > On Mon, 2016-05-02 at 10:46 +0200, Peter Zijlstra wrote:
> > 5226         /*
> > 5227          * If there are idle cores to be had, go find one.
> > 5228          */
> > 5229         if (sched_feat(IDLE_CORE) && test_idle_cores(target)) {
> > 5230                 i = select_idle_core(p, target);
> > 5231                 if ((unsigned)i < nr_cpumask_bits)
> > 5232                         return i;
> > 5233  
> > 5234                 /*
> > 5235                  * Failed to find an idle core; stop looking for one.
> > 5236                  */
> > 5237                 clear_idle_cores(target);
> > 5238         }
> > 5239 #if 1
> > 5240         for_each_cpu(i, cpu_smt_mask(target)) {
> > 5241                 if (idle_cpu(i))
> > 5242                         return i;
> > 5243         }
> > 5244  
> > 5245         return target;
> > 5246 #endif
> 
> And yes, I have a variant of that, that does indeed work way better than
> scanning the whole LLC domain for idle threads.
> 
> If you want a laugh, modify select_idle_core() to remember the last idle
> thread it encounters and have it return that when it fails to find an
> idle core.. I'm still stumped to explain why it behaves the way it does.

Assuming by 'behaving the way it does' means it improves things, such a dynamic 
with history/memory could be disrupting escalating feedback loops. Only guessing 
though.

Thanks,

	Ingo

^ permalink raw reply	[flat|nested] 80+ messages in thread

* Re: sched: tweak select_idle_sibling to look for idle threads
  2016-05-02 14:58           ` Peter Zijlstra
  2016-05-02 15:47             ` Chris Mason
@ 2016-05-02 17:30             ` Mike Galbraith
  1 sibling, 0 replies; 80+ messages in thread
From: Mike Galbraith @ 2016-05-02 17:30 UTC (permalink / raw)
  To: Peter Zijlstra; +Cc: Chris Mason, Ingo Molnar, Matt Fleming, linux-kernel

On Mon, 2016-05-02 at 16:58 +0200, Peter Zijlstra wrote:
> On Mon, May 02, 2016 at 04:50:04PM +0200, Mike Galbraith wrote:
> > Oh btw, did you know single socket boxen have no sd_busy?  That
> > doesn't
> > look right.
> 
> I suspected; didn't bother looking at yet. The 'problem' is that the
> LLC
> domain is the top-most, so it doesn't have a parent domain. I'm sure
> we
> can come up with something if we can get this all working right.
> 
> And yes, I can get gains on various workloads with various options, I
> can even break all workloads, but I've so far completely failed on
> getting a win for everyone :/
> 
> In particular low count sysbench-psql (oltp test) vs tbench
> client==nr_cores is having me flummoxed for a bit.

When I was doing a lot of that, there was always considerable overlap
to reclaim with db stuff, and often other players.  With loalhost
tbench only a wee bit of overlap and two players, it's pretty close to
really being the synchronous load it advertises.  For tbench (and ilk),
the best thing you can do is completely kill select_idle_sibling() once
past ramp.  Hell, if you watch TCP_RR, pipe-test, whatever with much
less overlap, you'll ask yourself why the hell select_idle_sibling()
even exists for anything other than the original shared L2 case it was
born to serve. 

Unfortunately, the real world spoiler is that a sync wakeup means jack
diddly spit, a waker may very well suddenly be awakened on it's way to
sleepy land.. so much for sync wakeup, oh crap, we need to scale.  The
problem ain't simple, but the scheduler has to be to perform.  Catch22.

	-Mike

^ permalink raw reply	[flat|nested] 80+ messages in thread

* Re: sched: tweak select_idle_sibling to look for idle threads
  2016-05-02 16:04             ` Ingo Molnar
@ 2016-05-03 11:31               ` Peter Zijlstra
  2016-05-03 18:22                 ` Peter Zijlstra
  0 siblings, 1 reply; 80+ messages in thread
From: Peter Zijlstra @ 2016-05-03 11:31 UTC (permalink / raw)
  To: Ingo Molnar; +Cc: Mike Galbraith, Chris Mason, Matt Fleming, linux-kernel

On Mon, May 02, 2016 at 06:04:04PM +0200, Ingo Molnar wrote:
> * Peter Zijlstra <peterz@infradead.org> wrote:

> > If you want a laugh, modify select_idle_core() to remember the last idle
> > thread it encounters and have it return that when it fails to find an
> > idle core.. I'm still stumped to explain why it behaves the way it does.
> 
> Assuming by 'behaving the way it does' means it improves things, such a dynamic 
> with history/memory could be disrupting escalating feedback loops. Only guessing 
> though.

ha! no :-)


mainline-like:

root@ivb-ep:~/bench/sysbench# for i in OLD_IDLE NO_ORDER_IDLE IDLE_CORE NO_FORCE_CORE IDLE IDLE_SMT NO_IDLE_LAST NO_IDLE_FIRST ; do echo $i > /debug/sched_features ; done ; ./doit-psql 30 2 5 10 20 40 80
  2: [30 secs]     transactions:                        52336  (1744.47 per sec.)
  5: [30 secs]     transactions:                        121971 (4065.59 per sec.)
 10: [30 secs]     transactions:                        242741 (8091.05 per sec.)
 20: [30 secs]     transactions:                        382357 (12744.58 per sec.)
 40: [30 secs]     transactions:                        537705 (17922.33 per sec.)
 80: [30 secs]     transactions:                        544193 (18137.97 per sec.)


new code with settings that make sense (and aren't that far from current
mainline):

root@ivb-ep:~/bench/sysbench# for i in NO_OLD_IDLE NO_ORDER_IDLE IDLE_CORE NO_FORCE_CORE IDLE IDLE_SMT NO_IDLE_LAST NO_IDLE_FIRST ; do echo $i > /debug/sched_features ; done ; ./doit-psql 30 2 5 10 20 40 80
  2: [30 secs]     transactions:                        53663  (1788.71 per sec.)
  5: [30 secs]     transactions:                        122529 (4084.16 per sec.)
 10: [30 secs]     transactions:                        239607 (7986.60 per sec.)
 20: [30 secs]     transactions:                        379112 (12636.43 per sec.)
 40: [30 secs]     transactions:                        539161 (17970.83 per sec.)
 80: [30 secs]     transactions:                        544907 (18161.74 per sec.)


Then flip on the last_idle tracking in select_idle_core():

root@ivb-ep:~/bench/sysbench# for i in NO_OLD_IDLE NO_ORDER_IDLE IDLE_CORE NO_FORCE_CORE IDLE IDLE_SMT IDLE_LAST NO_IDLE_FIRST ; do echo $i > /debug/sched_features ; done ; ./doit-psql 30 2 5 10 20 40 80
  2: [30 secs]     transactions:                        54355  (1811.78 per sec.)
  5: [30 secs]     transactions:                        122609 (4086.81 per sec.)
 10: [30 secs]     transactions:                        238738 (7957.66 per sec.)
 20: [30 secs]     transactions:                        354693 (11822.49 per sec.)
 40: [30 secs]     transactions:                        421807 (14059.32 per sec.)
 80: [30 secs]     transactions:                        427088 (14234.25 per sec.)


And see the top end collapse..

The idea was that the whole has_idle_cores thing would switch off
select_idle_core() under 'high' load, and therefore last_idle tracking
would not affect that.

Clearly something is not quite working out :-)


Also, I think the current sharing of the load_balance_mask is borken.
While all users have BH disabled, the wakeup can be from IRQ context and
hence trample on the LB mask, lemme go fix that by allocating more masks
or so.

---
 kernel/sched/fair.c      | 269 +++++++++++++++++++++++++++++++++++++++--------
 kernel/sched/features.h  |   8 ++
 kernel/sched/idle_task.c |   4 +-
 kernel/sched/sched.h     |   1 +
 kernel/time/tick-sched.c |  10 +-
 5 files changed, 243 insertions(+), 49 deletions(-)

diff --git a/kernel/sched/fair.c b/kernel/sched/fair.c
index b8a33ab..98c2904 100644
--- a/kernel/sched/fair.c
+++ b/kernel/sched/fair.c
@@ -1501,8 +1501,10 @@ balance:
 	 * One idle CPU per node is evaluated for a task numa move.
 	 * Call select_idle_sibling to maybe find a better one.
 	 */
-	if (!cur)
+	if (!cur) {
+		// XXX borken
 		env->dst_cpu = select_idle_sibling(env->p, env->dst_cpu);
+	}
 
 assign:
 	assigned = true;
@@ -4491,6 +4493,17 @@ static void dequeue_task_fair(struct rq *rq, struct task_struct *p, int flags)
 }
 
 #ifdef CONFIG_SMP
+
+/* 
+ * Working cpumask for:
+ *   load_balance, 
+ *   load_balance_newidle,
+ *   select_idle_core.
+ *
+ * Assumes softirqs are disabled when in use.
+ */
+DEFINE_PER_CPU(cpumask_var_t, load_balance_mask);
+
 #ifdef CONFIG_NO_HZ_COMMON
 /*
  * per rq 'load' arrray crap; XXX kill this.
@@ -5162,65 +5175,238 @@ find_idlest_cpu(struct sched_group *group, struct task_struct *p, int this_cpu)
 	return shallowest_idle_cpu != -1 ? shallowest_idle_cpu : least_loaded_cpu;
 }
 
+static int cpumask_next_wrap(int n, const struct cpumask *mask, int start, int *wrapped)
+{
+	int next;
+
+again:
+	next = find_next_bit(cpumask_bits(mask), nr_cpumask_bits, n+1);
+
+	if (*wrapped) {
+		if (next >= start)
+			return nr_cpumask_bits;
+	} else {
+		if (next >= nr_cpumask_bits) {
+			*wrapped = 1;
+			n = -1;
+			goto again;
+		}
+	}
+
+	return next;
+}
+
+#define for_each_cpu_wrap(cpu, mask, start, wrap)				\
+	for ((wrap) = 0, (cpu) = (start)-1;					\
+		(cpu) = cpumask_next_wrap((cpu), (mask), (start), &(wrap)),	\
+		(cpu) < nr_cpumask_bits; )
+
+#ifdef CONFIG_SCHED_SMT
+
+static inline void clear_idle_cores(int cpu)
+{
+	struct sched_domain *sd = rcu_dereference(per_cpu(sd_busy, cpu));
+	if (!sd)
+		return;
+
+	WRITE_ONCE(sd->groups->sgc->has_idle_cores, 0);
+}
+
+static inline void set_idle_cores(int cpu)
+{
+	struct sched_domain *sd = rcu_dereference(per_cpu(sd_busy, cpu));
+	if (!sd)
+		return;
+
+	WRITE_ONCE(sd->groups->sgc->has_idle_cores, 1);
+}
+
+static inline bool test_idle_cores(int cpu)
+{
+	if (sched_feat(FORCE_CORE)) {
+		return true;
+	} else {
+		struct sched_domain *sd = rcu_dereference(per_cpu(sd_busy, cpu));
+		if (!sd)
+			return false;
+
+		// XXX static key for !SMT topologies
+
+		return READ_ONCE(sd->groups->sgc->has_idle_cores);
+	}
+}
+
+void update_idle_core(struct rq *rq)
+{
+	int core = cpu_of(rq);
+	int cpu;
+
+	rcu_read_lock();
+	if (test_idle_cores(core))
+		goto unlock;
+
+	for_each_cpu(cpu, cpu_smt_mask(core)) {
+		if (cpu == core)
+			continue;
+
+		if (!idle_cpu(cpu))
+			goto unlock;
+	}
+
+	set_idle_cores(core);
+unlock:
+	rcu_read_unlock();
+}
+
+static int select_idle_core(struct task_struct *p, int target)
+{
+	struct cpumask *cpus = this_cpu_cpumask_var_ptr(load_balance_mask);
+	int core, cpu, wrap, last_idle = -1, first_idle = -1;
+	struct sched_domain *sd;
+
+	sd = rcu_dereference(per_cpu(sd_llc, target));
+	cpumask_and(cpus, sched_domain_span(sd), tsk_cpus_allowed(p));
+
+	for_each_cpu_wrap(core, cpus, target, wrap) {
+		bool idle = true;
+
+		for_each_cpu(cpu, cpu_smt_mask(core)) {
+			if (cpumask_test_and_clear_cpu(cpu, cpus)) {
+				if (sched_feat(IDLE_LAST))
+					last_idle = cpu;
+				if (sched_feat(IDLE_FIRST) && first_idle == -1)
+					first_idle = cpu;
+			}
+			if (!idle_cpu(cpu))
+				idle = false;
+		}
+
+		if (idle)
+			break;
+	}
+
+	if (sched_feat(IDLE_LAST) && ((unsigned)core >= nr_cpumask_bits))
+		return last_idle;
+
+	if (sched_feat(IDLE_FIRST) && ((unsigned)core >= nr_cpumask_bits))
+		return first_idle;
+
+	return core;
+}
+
+#else /* CONFIG_SCHED_SMT */
+
+static inline void clear_idle_cores(int cpu) { }
+static inline void set_idle_cores(int cpu) { }
+
+static inline bool test_idle_cores(int cpu)
+{
+	return false;
+}
+
+void update_idle_core(struct rq *rq) { }
+
+static inline int select_idle_core(struct task_struct *p, int target)
+{
+	return -1;
+}
+
+#endif /* CONFIG_SCHED_SMT */
+
 /*
- * Try and locate an idle CPU in the sched_domain.
+ * Try and locate an idle core/thread in the LLC cache domain.
  */
 static int select_idle_sibling(struct task_struct *p, int target)
 {
 	struct sched_domain *sd;
-	struct sched_group *sg;
-	int i = task_cpu(p);
+	int wrap, start, i = task_cpu(p);
 
 	if (idle_cpu(target))
 		return target;
 
 	/*
-	 * If the prevous cpu is cache affine and idle, don't be stupid.
+	 * If the previous cpu is cache affine and idle, don't be stupid.
 	 */
 	if (i != target && cpus_share_cache(i, target) && idle_cpu(i))
 		return i;
 
-	/*
-	 * Otherwise, iterate the domains and find an eligible idle cpu.
-	 *
-	 * A completely idle sched group at higher domains is more
-	 * desirable than an idle group at a lower level, because lower
-	 * domains have smaller groups and usually share hardware
-	 * resources which causes tasks to contend on them, e.g. x86
-	 * hyperthread siblings in the lowest domain (SMT) can contend
-	 * on the shared cpu pipeline.
-	 *
-	 * However, while we prefer idle groups at higher domains
-	 * finding an idle cpu at the lowest domain is still better than
-	 * returning 'target', which we've already established, isn't
-	 * idle.
-	 */
-	sd = rcu_dereference(per_cpu(sd_llc, target));
-	for_each_lower_domain(sd) {
-		sg = sd->groups;
-		do {
-			if (!cpumask_intersects(sched_group_cpus(sg),
-						tsk_cpus_allowed(p)))
-				goto next;
+	start = target;
+	if (sched_feat(ORDER_IDLE))
+		start = per_cpu(sd_llc_id, target); /* first cpu in llc domain */
 
-			/* Ensure the entire group is idle */
-			for_each_cpu(i, sched_group_cpus(sg)) {
-				if (i == target || !idle_cpu(i))
+	sd = rcu_dereference(per_cpu(sd_llc, start));
+	if (!sd)
+		return target;
+
+	if (sched_feat(OLD_IDLE)) {
+		struct sched_group *sg;
+
+		for_each_lower_domain(sd) {
+			sg = sd->groups;
+			do {
+				if (!cpumask_intersects(sched_group_cpus(sg),
+							tsk_cpus_allowed(p)))
 					goto next;
-			}
+
+				/* Ensure the entire group is idle */
+				for_each_cpu(i, sched_group_cpus(sg)) {
+					if (i == target || !idle_cpu(i))
+						goto next;
+				}
+
+				/*
+				 * It doesn't matter which cpu we pick, the
+				 * whole group is idle.
+				 */
+				target = cpumask_first_and(sched_group_cpus(sg),
+						tsk_cpus_allowed(p));
+				goto done;
+next:
+				sg = sg->next;
+			} while (sg != sd->groups);
+		}
+done:
+		return target;
+	}
+
+	/*
+	 * If there are idle cores to be had, go find one.
+	 */
+	if (sched_feat(IDLE_CORE)) {
+		if (test_idle_cores(target)) {
+			i = select_idle_core(p, start);
+			if ((unsigned)i < nr_cpumask_bits)
+				return i;
 
 			/*
-			 * It doesn't matter which cpu we pick, the
-			 * whole group is idle.
+			 * Failed to find an idle core; stop looking for one.
 			 */
-			target = cpumask_first_and(sched_group_cpus(sg),
-					tsk_cpus_allowed(p));
-			goto done;
-next:
-			sg = sg->next;
-		} while (sg != sd->groups);
+			clear_idle_cores(target);
+		}
 	}
-done:
+
+	if (!sched_feat(IDLE))
+		return target;
+
+	/*
+	 * Otherwise, settle for anything idle in this cache domain.
+	 */
+	if (!sched_feat(IDLE_SMT)) {
+		for_each_cpu_wrap(i, sched_domain_span(sd), start, wrap) {
+			if (!cpumask_test_cpu(i, tsk_cpus_allowed(p)))
+				continue;
+			if (idle_cpu(i))
+				return i;
+		}
+	} else {
+		for_each_cpu(i, cpu_smt_mask(target)) {
+			if (!cpumask_test_cpu(i, tsk_cpus_allowed(p)))
+				continue;
+			if (idle_cpu(i))
+				return i;
+		}
+	}
+
 	return target;
 }
 
@@ -7229,9 +7415,6 @@ static struct rq *find_busiest_queue(struct lb_env *env,
  */
 #define MAX_PINNED_INTERVAL	512
 
-/* Working cpumask for load_balance and load_balance_newidle. */
-DEFINE_PER_CPU(cpumask_var_t, load_balance_mask);
-
 static int need_active_balance(struct lb_env *env)
 {
 	struct sched_domain *sd = env->sd;
diff --git a/kernel/sched/features.h b/kernel/sched/features.h
index 69631fa..347f6fe 100644
--- a/kernel/sched/features.h
+++ b/kernel/sched/features.h
@@ -69,3 +69,11 @@ SCHED_FEAT(RT_RUNTIME_SHARE, true)
 SCHED_FEAT(LB_MIN, false)
 SCHED_FEAT(ATTACH_AGE_LOAD, true)
 
+SCHED_FEAT(OLD_IDLE, false)
+SCHED_FEAT(ORDER_IDLE, false)
+SCHED_FEAT(IDLE_CORE, true)
+SCHED_FEAT(FORCE_CORE, false)
+SCHED_FEAT(IDLE_SMT, false)
+SCHED_FEAT(IDLE, true)
+SCHED_FEAT(IDLE_LAST, false)
+SCHED_FEAT(IDLE_FIRST, false)
diff --git a/kernel/sched/idle_task.c b/kernel/sched/idle_task.c
index 47ce949..cb394db 100644
--- a/kernel/sched/idle_task.c
+++ b/kernel/sched/idle_task.c
@@ -23,11 +23,13 @@ static void check_preempt_curr_idle(struct rq *rq, struct task_struct *p, int fl
 	resched_curr(rq);
 }
 
+extern void update_idle_core(struct rq *rq);
+
 static struct task_struct *
 pick_next_task_idle(struct rq *rq, struct task_struct *prev)
 {
 	put_prev_task(rq, prev);
-
+	update_idle_core(rq);
 	schedstat_inc(rq, sched_goidle);
 	return rq->idle;
 }
diff --git a/kernel/sched/sched.h b/kernel/sched/sched.h
index 69da6fc..5994794 100644
--- a/kernel/sched/sched.h
+++ b/kernel/sched/sched.h
@@ -866,6 +866,7 @@ struct sched_group_capacity {
 	 * Number of busy cpus in this group.
 	 */
 	atomic_t nr_busy_cpus;
+	int	has_idle_cores;
 
 	unsigned long cpumask[0]; /* iteration mask */
 };
diff --git a/kernel/time/tick-sched.c b/kernel/time/tick-sched.c
index 31872bc..6e42cd2 100644
--- a/kernel/time/tick-sched.c
+++ b/kernel/time/tick-sched.c
@@ -933,11 +933,11 @@ void tick_nohz_idle_enter(void)
 	WARN_ON_ONCE(irqs_disabled());
 
 	/*
- 	 * Update the idle state in the scheduler domain hierarchy
- 	 * when tick_nohz_stop_sched_tick() is called from the idle loop.
- 	 * State will be updated to busy during the first busy tick after
- 	 * exiting idle.
- 	 */
+	 * Update the idle state in the scheduler domain hierarchy
+	 * when tick_nohz_stop_sched_tick() is called from the idle loop.
+	 * State will be updated to busy during the first busy tick after
+	 * exiting idle.
+	 */
 	set_cpu_sd_state_idle();
 
 	local_irq_disable();

^ permalink raw reply related	[flat|nested] 80+ messages in thread

* Re: sched: tweak select_idle_sibling to look for idle threads
  2016-05-02 15:47             ` Chris Mason
@ 2016-05-03 14:32               ` Peter Zijlstra
  2016-05-03 15:11                 ` Chris Mason
  0 siblings, 1 reply; 80+ messages in thread
From: Peter Zijlstra @ 2016-05-03 14:32 UTC (permalink / raw)
  To: Chris Mason, Mike Galbraith, Ingo Molnar, Matt Fleming, linux-kernel

On Mon, May 02, 2016 at 11:47:25AM -0400, Chris Mason wrote:
> On Mon, May 02, 2016 at 04:58:17PM +0200, Peter Zijlstra wrote:
> > On Mon, May 02, 2016 at 04:50:04PM +0200, Mike Galbraith wrote:
> > > Oh btw, did you know single socket boxen have no sd_busy?  That doesn't
> > > look right.
> > 
> > I suspected; didn't bother looking at yet. The 'problem' is that the LLC
> > domain is the top-most, so it doesn't have a parent domain. I'm sure we
> > can come up with something if we can get this all working right.
> > 
> > And yes, I can get gains on various workloads with various options, I
> > can even break all workloads, but I've so far completely failed on
> > getting a win for everyone :/
> 
> Adding in the task_hot() check to decide if scanning idle was a good
> idea ended up being really important

So I'm conflicted on this patch:

+static int bounce_to_target(struct task_struct *p, int cpu)
+{
+       s64 delta;
+
+       /*
+        * as the run queue gets bigger, its more and more likely that
+        * balance will have distributed things for us, and less likely
+        * that scanning all our CPUs for an idle one will find one.
+        * So, if nr_running > 1, just call this CPU good enough
+        */
+       if (cpu_rq(cpu)->cfs.nr_running > 1)
+               return 1;
+
+       /* taken from task_hot() */
+       delta = rq_clock_task(task_rq(p)) - p->se.exec_start;
+       return delta < (s64)sysctl_sched_migration_cost;
+}

This will work for you schbench workload because it sleep for 30ms while
the migration_cost thingy is 500us, therefore you'll trigger the full
LLC scan.

_However_, the migration_cost is supposed the model the cost of leaving
the LLC, so testing against that here seems wrong.

Let me go play with something that measures the cost of doing that LLC
scan and compares that against the sleepy time -- of course, now need to
go figure out how to do this clock thing without rq-lock pain.



+       if (package_sd && !bounce_to_target(p, target)) {
+               for_each_cpu_and(i, sched_domain_span(package_sd), tsk_cpus_allowed(p)) {
+                       if (idle_cpu(i)) {
+                               target = i;
+                               break;
+                       }
+
+               }
+       }

Also note your s/sd/package_sd/ rename is, strictly speaking, wrong.
Sure, on your current Intel system the LLC is the entire package, but
this is not true in general.

Take for instance the Intel Core2Quad and AMD Bulldozer thingies, they
had two dies in one package, and correspondingly two LLC domains in one
package.

(also, the Intel cluster-on-die thing can split the thing in two)

There were also the old P6 era SMP boards which had external LLC, where
you could have an LLC shared across multiple packages -- although I'm
thinking we'll never see that again, due to off package being far
toooooo slooooooow these days.

^ permalink raw reply	[flat|nested] 80+ messages in thread

* Re: sched: tweak select_idle_sibling to look for idle threads
  2016-05-03 14:32               ` Peter Zijlstra
@ 2016-05-03 15:11                 ` Chris Mason
  2016-05-04 10:37                   ` Peter Zijlstra
                                     ` (2 more replies)
  0 siblings, 3 replies; 80+ messages in thread
From: Chris Mason @ 2016-05-03 15:11 UTC (permalink / raw)
  To: Peter Zijlstra; +Cc: Mike Galbraith, Ingo Molnar, Matt Fleming, linux-kernel

On Tue, May 03, 2016 at 04:32:25PM +0200, Peter Zijlstra wrote:
> On Mon, May 02, 2016 at 11:47:25AM -0400, Chris Mason wrote:
> > On Mon, May 02, 2016 at 04:58:17PM +0200, Peter Zijlstra wrote:
> > > On Mon, May 02, 2016 at 04:50:04PM +0200, Mike Galbraith wrote:
> > > > Oh btw, did you know single socket boxen have no sd_busy?  That doesn't
> > > > look right.
> > > 
> > > I suspected; didn't bother looking at yet. The 'problem' is that the LLC
> > > domain is the top-most, so it doesn't have a parent domain. I'm sure we
> > > can come up with something if we can get this all working right.
> > > 
> > > And yes, I can get gains on various workloads with various options, I
> > > can even break all workloads, but I've so far completely failed on
> > > getting a win for everyone :/
> > 
> > Adding in the task_hot() check to decide if scanning idle was a good
> > idea ended up being really important
> 
> So I'm conflicted on this patch:
> 
> +static int bounce_to_target(struct task_struct *p, int cpu)
> +{
> +       s64 delta;
> +
> +       /*
> +        * as the run queue gets bigger, its more and more likely that
> +        * balance will have distributed things for us, and less likely
> +        * that scanning all our CPUs for an idle one will find one.
> +        * So, if nr_running > 1, just call this CPU good enough
> +        */
> +       if (cpu_rq(cpu)->cfs.nr_running > 1)
> +               return 1;

The nr_running check is interesting.  It is supposed to give the same
benefit as your "do we have anything idle?" variable, but without having
to constantly update a variable somewhere.  I'll have to do a few runs
to verify (maybe a idle_scan_failed counter).

> +
> +       /* taken from task_hot() */
> +       delta = rq_clock_task(task_rq(p)) - p->se.exec_start;
> +       return delta < (s64)sysctl_sched_migration_cost;
> +}
> 
> This will work for you schbench workload because it sleep for 30ms while
> the migration_cost thingy is 500us, therefore you'll trigger the full
> LLC scan.

The task_hot checks don't do much for the sleeping schbench runs, but
they help a lot for this:

# pick a single core, in my case cpus 0,20 are the same core
# cpu_hog is any program that spins
#
taskset -c 20 cpu_hog &

# schbench -p 4 means message passing mode with 4 byte messages (like
# pipe test), no sleeps, just bouncing as fast as it can.
#
# make the scheduler choose between the sibling of the hog and cpu 1
#
taskset -c 0,1 schbench -p 4 -m 1 -t 1

Current mainline will stuff both schbench threads onto CPU 1, leaving
CPU 0 100% idle.  My first patch with the minimal task_hot() checks
would sometimes pick CPU 0.  My second patch that just directly calls
task_hot sticks to cpu1, which is ~3x faster than spreading it.

The full task_hot() checks also really help tbench.

> 
> _However_, the migration_cost is supposed the model the cost of leaving
> the LLC, so testing against that here seems wrong.
> 
> Let me go play with something that measures the cost of doing that LLC
> scan and compares that against the sleepy time -- of course, now need to
> go figure out how to do this clock thing without rq-lock pain.
> 
> 
> 
> +       if (package_sd && !bounce_to_target(p, target)) {
> +               for_each_cpu_and(i, sched_domain_span(package_sd), tsk_cpus_allowed(p)) {
> +                       if (idle_cpu(i)) {
> +                               target = i;
> +                               break;
> +                       }
> +
> +               }
> +       }
> 
> Also note your s/sd/package_sd/ rename is, strictly speaking, wrong.
> Sure, on your current Intel system the LLC is the entire package, but
> this is not true in general.
> 
> Take for instance the Intel Core2Quad and AMD Bulldozer thingies, they
> had two dies in one package, and correspondingly two LLC domains in one
> package.
> 
> (also, the Intel cluster-on-die thing can split the thing in two)
> 
> There were also the old P6 era SMP boards which had external LLC, where
> you could have an LLC shared across multiple packages -- although I'm
> thinking we'll never see that again, due to off package being far
> toooooo slooooooow these days.

Gotcha, makes sense.  I'll switch to llc_sd ;)

-chris

^ permalink raw reply	[flat|nested] 80+ messages in thread

* Re: sched: tweak select_idle_sibling to look for idle threads
  2016-05-03 11:31               ` Peter Zijlstra
@ 2016-05-03 18:22                 ` Peter Zijlstra
  0 siblings, 0 replies; 80+ messages in thread
From: Peter Zijlstra @ 2016-05-03 18:22 UTC (permalink / raw)
  To: Ingo Molnar; +Cc: Mike Galbraith, Chris Mason, Matt Fleming, linux-kernel

On Tue, May 03, 2016 at 01:31:31PM +0200, Peter Zijlstra wrote:
> Then flip on the last_idle tracking in select_idle_core():
> 
> root@ivb-ep:~/bench/sysbench# for i in NO_OLD_IDLE NO_ORDER_IDLE IDLE_CORE NO_FORCE_CORE IDLE IDLE_SMT IDLE_LAST NO_IDLE_FIRST ; do echo $i > /debug/sched_features ; done ; ./doit-psql 30 2 5 10 20 40 80
>   2: [30 secs]     transactions:                        54355  (1811.78 per sec.)
>   5: [30 secs]     transactions:                        122609 (4086.81 per sec.)
>  10: [30 secs]     transactions:                        238738 (7957.66 per sec.)
>  20: [30 secs]     transactions:                        354693 (11822.49 per sec.)
>  40: [30 secs]     transactions:                        421807 (14059.32 per sec.)
>  80: [30 secs]     transactions:                        427088 (14234.25 per sec.)
> 
> 
> And see the top end collapse..

> +	if (sched_feat(IDLE_CORE)) {
> +		if (test_idle_cores(target)) {
> +			i = select_idle_core(p, start);
> +			if ((unsigned)i < nr_cpumask_bits)
> +				return i;
>  
>  			/*
> +			 * Failed to find an idle core; stop looking for one.
>  			 */
> +			clear_idle_cores(target);
> +		}
>  	}

'Obvious' bug there; if we start returning idle threads; we'll not get
to clear_idle_cores() and the thing remains active and badness happens.

^ permalink raw reply	[flat|nested] 80+ messages in thread

* Re: sched: tweak select_idle_sibling to look for idle threads
  2016-05-03 15:11                 ` Chris Mason
@ 2016-05-04 10:37                   ` Peter Zijlstra
  2016-05-04 15:31                     ` Peter Zijlstra
  2016-05-05 22:03                     ` Matt Fleming
  2016-05-04 15:45                   ` Peter Zijlstra
  2016-05-06  7:25                   ` Peter Zijlstra
  2 siblings, 2 replies; 80+ messages in thread
From: Peter Zijlstra @ 2016-05-04 10:37 UTC (permalink / raw)
  To: Chris Mason, Mike Galbraith, Ingo Molnar, Matt Fleming, linux-kernel

On Tue, May 03, 2016 at 11:11:53AM -0400, Chris Mason wrote:

> > +       if (cpu_rq(cpu)->cfs.nr_running > 1)
> > +               return 1;
> 
> The nr_running check is interesting.  It is supposed to give the same
> benefit as your "do we have anything idle?" variable, but without having
> to constantly update a variable somewhere.  I'll have to do a few runs
> to verify (maybe a idle_scan_failed counter).

Right; I got that.. I tried it and it doesn't seem to work as well. But
yeah, its better than nothing.

The reason I'm not too worried about the update_idle_core() thing is
that SMT threads share L1 so touching their sibling state isn't
typically expensive.

But yes, I'd need to find someone with SMT8 or something daft like that
to try this.

> The task_hot checks don't do much for the sleeping schbench runs, but
> they help a lot for this:
> 
> # pick a single core, in my case cpus 0,20 are the same core
> # cpu_hog is any program that spins
> #
> taskset -c 20 cpu_hog &
> 
> # schbench -p 4 means message passing mode with 4 byte messages (like
> # pipe test), no sleeps, just bouncing as fast as it can.
> #
> # make the scheduler choose between the sibling of the hog and cpu 1
> #
> taskset -c 0,1 schbench -p 4 -m 1 -t 1
> 
> Current mainline will stuff both schbench threads onto CPU 1, leaving
> CPU 0 100% idle.  My first patch with the minimal task_hot() checks
> would sometimes pick CPU 0.  My second patch that just directly calls
> task_hot sticks to cpu1, which is ~3x faster than spreading it.

Urgh, another benchmark to play with ;-)

> The full task_hot() checks also really help tbench.

tbench wants select_idle_siblings() to just not exist; it goes happy
when you just return target.

tbench:

old (mainline like):

Throughput 875.822 MB/sec  2 clients  2 procs  max_latency=0.117 ms
Throughput 2017.57 MB/sec  5 clients  5 procs  max_latency=0.057 ms
Throughput 3954.66 MB/sec  10 clients  10 procs  max_latency=0.094 ms
Throughput 5886.11 MB/sec  20 clients  20 procs  max_latency=0.088 ms
Throughput 9095.57 MB/sec  40 clients  40 procs  max_latency=0.864 ms

new:

Throughput 876.794 MB/sec  2 clients  2 procs  max_latency=0.102 ms
Throughput 2048.73 MB/sec  5 clients  5 procs  max_latency=0.095 ms
Throughput 3802.69 MB/sec  10 clients  10 procs  max_latency=0.113 ms
Throughput 5521.81 MB/sec  20 clients  20 procs  max_latency=0.091 ms
Throughput 10331.8 MB/sec  40 clients  40 procs  max_latency=0.444 ms

nothing:

Throughput 759.532 MB/sec  2 clients  2 procs  max_latency=0.210 ms
Throughput 1884.01 MB/sec  5 clients  5 procs  max_latency=0.094 ms
Throughput 3931.31 MB/sec  10 clients  10 procs  max_latency=0.091 ms
Throughput 6478.81 MB/sec  20 clients  20 procs  max_latency=0.110 ms
Throughput 10001 MB/sec  40 clients  40 procs  max_latency=0.148 ms


See the 20 client have a happy moment ;-) [ivb-ep: 2*10*2]


I've not quite figured out how to make the new bits switch off aggressive
enough to make tbench happy without hurting the others. More numbers:


sysbench-oltp-psql:

old (mainline like):

  2: [30 secs]     transactions:                        53556  (1785.19 per sec.)
  5: [30 secs]     transactions:                        118957 (3965.08 per sec.)
 10: [30 secs]     transactions:                        241126 (8037.22 per sec.)
 20: [30 secs]     transactions:                        383256 (12774.63 per sec.)
 40: [30 secs]     transactions:                        539705 (17989.05 per sec.)
 80: [30 secs]     transactions:                        541833 (18059.16 per sec.)

new:

  2: [30 secs]     transactions:                        53012  (1767.03 per sec.)
  5: [30 secs]     transactions:                        122057 (4068.49 per sec.)
 10: [30 secs]     transactions:                        235781 (7859.09 per sec.)
 20: [30 secs]     transactions:                        355967 (11864.99 per sec.)
 40: [30 secs]     transactions:                        537327 (17909.80 per sec.)
 80: [30 secs]     transactions:                        546017 (18198.82 per sec.)



schbench -m2 -t 20 -c 30000 -s 30000 -r 30:

old (mainline like):

Latency percentiles (usec)
        50.0000th: 102
        75.0000th: 109
        90.0000th: 115
        95.0000th: 118
        *99.0000th: 5352
        99.5000th: 12112
        99.9000th: 13008
        Over=0, min=0, max=27238

new:

Latency percentiles (usec)
        50.0000th: 103
        75.0000th: 109
        90.0000th: 114
        95.0000th: 116
        *99.0000th: 120
        99.5000th: 121
        99.9000th: 124
        Over=0, min=0, max=12939


---
 include/linux/sched.h    |   2 +
 kernel/sched/core.c      |   3 +
 kernel/sched/fair.c      | 267 +++++++++++++++++++++++++++++++++++++++--------
 kernel/sched/features.h  |   9 ++
 kernel/sched/idle_task.c |   4 +-
 kernel/sched/sched.h     |   1 +
 kernel/time/tick-sched.c |  10 +-
 7 files changed, 246 insertions(+), 50 deletions(-)

diff --git a/include/linux/sched.h b/include/linux/sched.h
index ad9454d..e7ce1a0 100644
--- a/include/linux/sched.h
+++ b/include/linux/sched.h
@@ -1068,6 +1068,8 @@ struct sched_domain {
 	u64 max_newidle_lb_cost;
 	unsigned long next_decay_max_lb_cost;
 
+	u64 avg_scan_cost;		/* select_idle_sibling */
+
 #ifdef CONFIG_SCHEDSTATS
 	/* load_balance() stats */
 	unsigned int lb_count[CPU_MAX_IDLE_TYPES];
diff --git a/kernel/sched/core.c b/kernel/sched/core.c
index c82ca6e..280e73e 100644
--- a/kernel/sched/core.c
+++ b/kernel/sched/core.c
@@ -7278,6 +7278,7 @@ static struct kmem_cache *task_group_cache __read_mostly;
 #endif
 
 DECLARE_PER_CPU(cpumask_var_t, load_balance_mask);
+DECLARE_PER_CPU(cpumask_var_t, select_idle_mask);
 
 void __init sched_init(void)
 {
@@ -7314,6 +7315,8 @@ void __init sched_init(void)
 	for_each_possible_cpu(i) {
 		per_cpu(load_balance_mask, i) = (cpumask_var_t)kzalloc_node(
 			cpumask_size(), GFP_KERNEL, cpu_to_node(i));
+		per_cpu(select_idle_mask, i) = (cpumask_var_t)kzalloc_node(
+			cpumask_size(), GFP_KERNEL, cpu_to_node(i));
 	}
 #endif /* CONFIG_CPUMASK_OFFSTACK */
 
diff --git a/kernel/sched/fair.c b/kernel/sched/fair.c
index b8a33ab..9290fc8 100644
--- a/kernel/sched/fair.c
+++ b/kernel/sched/fair.c
@@ -1501,8 +1501,10 @@ balance:
 	 * One idle CPU per node is evaluated for a task numa move.
 	 * Call select_idle_sibling to maybe find a better one.
 	 */
-	if (!cur)
+	if (!cur) {
+		// XXX borken
 		env->dst_cpu = select_idle_sibling(env->p, env->dst_cpu);
+	}
 
 assign:
 	assigned = true;
@@ -4491,6 +4493,11 @@ static void dequeue_task_fair(struct rq *rq, struct task_struct *p, int flags)
 }
 
 #ifdef CONFIG_SMP
+
+/* Working cpumask for: load_balance, load_balance_newidle. */
+DEFINE_PER_CPU(cpumask_var_t, load_balance_mask);
+DEFINE_PER_CPU(cpumask_var_t, select_idle_mask);
+
 #ifdef CONFIG_NO_HZ_COMMON
 /*
  * per rq 'load' arrray crap; XXX kill this.
@@ -5162,65 +5169,240 @@ find_idlest_cpu(struct sched_group *group, struct task_struct *p, int this_cpu)
 	return shallowest_idle_cpu != -1 ? shallowest_idle_cpu : least_loaded_cpu;
 }
 
+static int cpumask_next_wrap(int n, const struct cpumask *mask, int start, int *wrapped)
+{
+	int next;
+
+again:
+	next = find_next_bit(cpumask_bits(mask), nr_cpumask_bits, n+1);
+
+	if (*wrapped) {
+		if (next >= start)
+			return nr_cpumask_bits;
+	} else {
+		if (next >= nr_cpumask_bits) {
+			*wrapped = 1;
+			n = -1;
+			goto again;
+		}
+	}
+
+	return next;
+}
+
+#define for_each_cpu_wrap(cpu, mask, start, wrap)				\
+	for ((wrap) = 0, (cpu) = (start)-1;					\
+		(cpu) = cpumask_next_wrap((cpu), (mask), (start), &(wrap)),	\
+		(cpu) < nr_cpumask_bits; )
+
+#ifdef CONFIG_SCHED_SMT
+
+static inline void clear_idle_cores(int cpu)
+{
+	struct sched_domain *sd = rcu_dereference(per_cpu(sd_busy, cpu));
+	if (!sd)
+		return;
+
+	WRITE_ONCE(sd->groups->sgc->has_idle_cores, 0);
+}
+
+static inline void set_idle_cores(int cpu)
+{
+	struct sched_domain *sd = rcu_dereference(per_cpu(sd_busy, cpu));
+	if (!sd)
+		return;
+
+	WRITE_ONCE(sd->groups->sgc->has_idle_cores, 1);
+}
+
+static inline bool test_idle_cores(int cpu)
+{
+	struct sched_domain *sd = rcu_dereference(per_cpu(sd_busy, cpu));
+	if (!sd)
+		return false;
+
+	// XXX static key for !SMT topologies
+
+	return READ_ONCE(sd->groups->sgc->has_idle_cores);
+}
+
+void update_idle_core(struct rq *rq)
+{
+	int core = cpu_of(rq);
+	int cpu;
+
+	rcu_read_lock();
+	if (test_idle_cores(core))
+		goto unlock;
+
+	for_each_cpu(cpu, cpu_smt_mask(core)) {
+		if (cpu == core)
+			continue;
+
+		if (!idle_cpu(cpu))
+			goto unlock;
+	}
+
+	set_idle_cores(core);
+unlock:
+	rcu_read_unlock();
+}
+
+static int select_idle_core(struct task_struct *p, struct sched_domain *sd, int target)
+{
+	struct cpumask *cpus = this_cpu_cpumask_var_ptr(select_idle_mask);
+	int core, cpu, wrap;
+
+	if (!test_idle_cores(target))
+		return -1;
+
+	cpumask_and(cpus, sched_domain_span(sd), tsk_cpus_allowed(p));
+
+	for_each_cpu_wrap(core, cpus, target, wrap) {
+		bool idle = true;
+
+		for_each_cpu(cpu, cpu_smt_mask(core)) {
+			cpumask_clear_cpu(cpu, cpus);
+			if (!idle_cpu(cpu))
+				idle = false;
+		}
+
+		if (idle)
+			return core;
+	}
+
+	/*
+	 * Failed to find an idle core; stop looking for one.
+	 */
+	clear_idle_cores(target);
+
+	return -1;
+}
+
+#else /* CONFIG_SCHED_SMT */
+
+void update_idle_core(struct rq *rq) { }
+
+static inline int select_idle_core(struct task_struct *p, struct sched_domain *sd, int target)
+{
+	return -1;
+}
+
+#endif /* CONFIG_SCHED_SMT */
+
+static int select_idle_cpu(struct task_struct *p, struct sched_domain *sd, int target)
+{
+	struct sched_domain *this_sd = rcu_dereference(*this_cpu_ptr(&sd_llc));
+	u64 time, cost;
+	s64 delta;
+	int cpu, wrap;
+
+	if (sched_feat(AVG_CPU)) {
+		u64 avg_idle = this_rq()->avg_idle;
+		u64 avg_cost = this_sd->avg_scan_cost;
+
+		if (sched_feat(PRINT_AVG))
+			trace_printk("idle: %Ld cost: %Ld\n", avg_idle, avg_cost);
+
+		if (avg_idle / 32 < avg_cost)
+			return -1;
+	}
+
+	time = local_clock();
+
+	for_each_cpu_wrap(cpu, sched_domain_span(sd), target, wrap) {
+		if (!cpumask_test_cpu(cpu, tsk_cpus_allowed(p)))
+			continue;
+		if (idle_cpu(cpu))
+			break;
+	}
+
+	time = local_clock() - time;
+	cost = this_sd->avg_scan_cost;
+	delta = (s64)(time - cost) / 8;
+	/* trace_printk("time: %Ld cost: %Ld delta: %Ld\n", time, cost, delta); */
+	this_sd->avg_scan_cost += delta;
+
+	return cpu;
+}
+
 /*
- * Try and locate an idle CPU in the sched_domain.
+ * Try and locate an idle core/thread in the LLC cache domain.
  */
 static int select_idle_sibling(struct task_struct *p, int target)
 {
 	struct sched_domain *sd;
-	struct sched_group *sg;
-	int i = task_cpu(p);
+	int start, i = task_cpu(p);
 
 	if (idle_cpu(target))
 		return target;
 
 	/*
-	 * If the prevous cpu is cache affine and idle, don't be stupid.
+	 * If the previous cpu is cache affine and idle, don't be stupid.
 	 */
 	if (i != target && cpus_share_cache(i, target) && idle_cpu(i))
 		return i;
 
-	/*
-	 * Otherwise, iterate the domains and find an eligible idle cpu.
-	 *
-	 * A completely idle sched group at higher domains is more
-	 * desirable than an idle group at a lower level, because lower
-	 * domains have smaller groups and usually share hardware
-	 * resources which causes tasks to contend on them, e.g. x86
-	 * hyperthread siblings in the lowest domain (SMT) can contend
-	 * on the shared cpu pipeline.
-	 *
-	 * However, while we prefer idle groups at higher domains
-	 * finding an idle cpu at the lowest domain is still better than
-	 * returning 'target', which we've already established, isn't
-	 * idle.
-	 */
-	sd = rcu_dereference(per_cpu(sd_llc, target));
-	for_each_lower_domain(sd) {
-		sg = sd->groups;
-		do {
-			if (!cpumask_intersects(sched_group_cpus(sg),
-						tsk_cpus_allowed(p)))
-				goto next;
+	start = target;
+	if (sched_feat(ORDER_IDLE))
+		start = per_cpu(sd_llc_id, target); /* first cpu in llc domain */
 
-			/* Ensure the entire group is idle */
-			for_each_cpu(i, sched_group_cpus(sg)) {
-				if (i == target || !idle_cpu(i))
+	sd = rcu_dereference(per_cpu(sd_llc, start));
+	if (!sd)
+		return target;
+
+	if (sched_feat(OLD_IDLE)) {
+		struct sched_group *sg;
+
+		for_each_lower_domain(sd) {
+			sg = sd->groups;
+			do {
+				if (!cpumask_intersects(sched_group_cpus(sg),
+							tsk_cpus_allowed(p)))
 					goto next;
-			}
 
-			/*
-			 * It doesn't matter which cpu we pick, the
-			 * whole group is idle.
-			 */
-			target = cpumask_first_and(sched_group_cpus(sg),
-					tsk_cpus_allowed(p));
-			goto done;
+				/* Ensure the entire group is idle */
+				for_each_cpu(i, sched_group_cpus(sg)) {
+					if (i == target || !idle_cpu(i))
+						goto next;
+				}
+
+				/*
+				 * It doesn't matter which cpu we pick, the
+				 * whole group is idle.
+				 */
+				target = cpumask_first_and(sched_group_cpus(sg),
+						tsk_cpus_allowed(p));
+				goto done;
 next:
-			sg = sg->next;
-		} while (sg != sd->groups);
-	}
+				sg = sg->next;
+			} while (sg != sd->groups);
+		}
 done:
+		return target;
+	}
+
+	if (sched_feat(IDLE_CORE)) {
+		i = select_idle_core(p, sd, start);
+		if ((unsigned)i < nr_cpumask_bits)
+			return i;
+	}
+
+	if (sched_feat(IDLE_CPU)) {
+		i = select_idle_cpu(p, sd, start);
+		if ((unsigned)i < nr_cpumask_bits)
+			return i;
+	}
+
+	if (sched_feat(IDLE_SMT)) {
+		for_each_cpu(i, cpu_smt_mask(target)) {
+			if (!cpumask_test_cpu(i, tsk_cpus_allowed(p)))
+				continue;
+			if (idle_cpu(i))
+				return i;
+		}
+	}
+
 	return target;
 }
 
@@ -7229,9 +7411,6 @@ static struct rq *find_busiest_queue(struct lb_env *env,
  */
 #define MAX_PINNED_INTERVAL	512
 
-/* Working cpumask for load_balance and load_balance_newidle. */
-DEFINE_PER_CPU(cpumask_var_t, load_balance_mask);
-
 static int need_active_balance(struct lb_env *env)
 {
 	struct sched_domain *sd = env->sd;
diff --git a/kernel/sched/features.h b/kernel/sched/features.h
index 69631fa..5dd10ec 100644
--- a/kernel/sched/features.h
+++ b/kernel/sched/features.h
@@ -69,3 +69,12 @@ SCHED_FEAT(RT_RUNTIME_SHARE, true)
 SCHED_FEAT(LB_MIN, false)
 SCHED_FEAT(ATTACH_AGE_LOAD, true)
 
+SCHED_FEAT(OLD_IDLE, false)
+SCHED_FEAT(ORDER_IDLE, false)
+
+SCHED_FEAT(IDLE_CORE, true)
+SCHED_FEAT(IDLE_CPU, true)
+SCHED_FEAT(AVG_CPU, true)
+SCHED_FEAT(PRINT_AVG, false)
+
+SCHED_FEAT(IDLE_SMT, false)
diff --git a/kernel/sched/idle_task.c b/kernel/sched/idle_task.c
index 47ce949..cb394db 100644
--- a/kernel/sched/idle_task.c
+++ b/kernel/sched/idle_task.c
@@ -23,11 +23,13 @@ static void check_preempt_curr_idle(struct rq *rq, struct task_struct *p, int fl
 	resched_curr(rq);
 }
 
+extern void update_idle_core(struct rq *rq);
+
 static struct task_struct *
 pick_next_task_idle(struct rq *rq, struct task_struct *prev)
 {
 	put_prev_task(rq, prev);
-
+	update_idle_core(rq);
 	schedstat_inc(rq, sched_goidle);
 	return rq->idle;
 }
diff --git a/kernel/sched/sched.h b/kernel/sched/sched.h
index 69da6fc..5994794 100644
--- a/kernel/sched/sched.h
+++ b/kernel/sched/sched.h
@@ -866,6 +866,7 @@ struct sched_group_capacity {
 	 * Number of busy cpus in this group.
 	 */
 	atomic_t nr_busy_cpus;
+	int	has_idle_cores;
 
 	unsigned long cpumask[0]; /* iteration mask */
 };
diff --git a/kernel/time/tick-sched.c b/kernel/time/tick-sched.c
index 31872bc..6e42cd2 100644
--- a/kernel/time/tick-sched.c
+++ b/kernel/time/tick-sched.c
@@ -933,11 +933,11 @@ void tick_nohz_idle_enter(void)
 	WARN_ON_ONCE(irqs_disabled());
 
 	/*
- 	 * Update the idle state in the scheduler domain hierarchy
- 	 * when tick_nohz_stop_sched_tick() is called from the idle loop.
- 	 * State will be updated to busy during the first busy tick after
- 	 * exiting idle.
- 	 */
+	 * Update the idle state in the scheduler domain hierarchy
+	 * when tick_nohz_stop_sched_tick() is called from the idle loop.
+	 * State will be updated to busy during the first busy tick after
+	 * exiting idle.
+	 */
 	set_cpu_sd_state_idle();
 
 	local_irq_disable();

^ permalink raw reply related	[flat|nested] 80+ messages in thread

* Re: sched: tweak select_idle_sibling to look for idle threads
  2016-05-04 10:37                   ` Peter Zijlstra
@ 2016-05-04 15:31                     ` Peter Zijlstra
  2016-05-05 22:03                     ` Matt Fleming
  1 sibling, 0 replies; 80+ messages in thread
From: Peter Zijlstra @ 2016-05-04 15:31 UTC (permalink / raw)
  To: Chris Mason, Mike Galbraith, Ingo Molnar, Matt Fleming, linux-kernel

On Wed, May 04, 2016 at 12:37:01PM +0200, Peter Zijlstra wrote:

> +static int select_idle_cpu(struct task_struct *p, struct sched_domain *sd, int target)
> +{
> +	struct sched_domain *this_sd = rcu_dereference(*this_cpu_ptr(&sd_llc));
> +	u64 time, cost;
> +	s64 delta;
> +	int cpu, wrap;
> +
> +	if (sched_feat(AVG_CPU)) {
> +		u64 avg_idle = this_rq()->avg_idle;
> +		u64 avg_cost = this_sd->avg_scan_cost;
> +
> +		if (sched_feat(PRINT_AVG))
> +			trace_printk("idle: %Ld cost: %Ld\n", avg_idle, avg_cost);
> +
> +		if (avg_idle / 32 < avg_cost)

s/32/512/ + IDLE_SMT fixes a hackbench regression

hackbench, like tbench, doesn't like IDLE_CPU to trigger, but apparently
needs IDLE_SMT.

Bah, I could sort of explain 32 away, but 512 is firmly in the magic
value range :/

> +			return -1;
> +	}
> +
> +	time = local_clock();
> +
> +	for_each_cpu_wrap(cpu, sched_domain_span(sd), target, wrap) {
> +		if (!cpumask_test_cpu(cpu, tsk_cpus_allowed(p)))
> +			continue;
> +		if (idle_cpu(cpu))
> +			break;
> +	}
> +
> +	time = local_clock() - time;
> +	cost = this_sd->avg_scan_cost;
> +	delta = (s64)(time - cost) / 8;
> +	/* trace_printk("time: %Ld cost: %Ld delta: %Ld\n", time, cost, delta); */
> +	this_sd->avg_scan_cost += delta;
> +
> +	return cpu;
> +}

^ permalink raw reply	[flat|nested] 80+ messages in thread

* Re: sched: tweak select_idle_sibling to look for idle threads
  2016-05-03 15:11                 ` Chris Mason
  2016-05-04 10:37                   ` Peter Zijlstra
@ 2016-05-04 15:45                   ` Peter Zijlstra
  2016-05-04 17:46                     ` Chris Mason
  2016-05-06  7:25                   ` Peter Zijlstra
  2 siblings, 1 reply; 80+ messages in thread
From: Peter Zijlstra @ 2016-05-04 15:45 UTC (permalink / raw)
  To: Chris Mason, Mike Galbraith, Ingo Molnar, Matt Fleming, linux-kernel

On Tue, May 03, 2016 at 11:11:53AM -0400, Chris Mason wrote:
> # pick a single core, in my case cpus 0,20 are the same core
> # cpu_hog is any program that spins
> #
> taskset -c 20 cpu_hog &
> 
> # schbench -p 4 means message passing mode with 4 byte messages (like
> # pipe test), no sleeps, just bouncing as fast as it can.
> #
> # make the scheduler choose between the sibling of the hog and cpu 1
> #
> taskset -c 0,1 schbench -p 4 -m 1 -t 1

Will that schbench thingy print something? Mine doesn't seem to output
anything, not actually exit, although it stops consuming CPU cycles at
some point.

^ permalink raw reply	[flat|nested] 80+ messages in thread

* Re: sched: tweak select_idle_sibling to look for idle threads
  2016-05-04 15:45                   ` Peter Zijlstra
@ 2016-05-04 17:46                     ` Chris Mason
  2016-05-05  9:33                       ` Peter Zijlstra
  0 siblings, 1 reply; 80+ messages in thread
From: Chris Mason @ 2016-05-04 17:46 UTC (permalink / raw)
  To: Peter Zijlstra; +Cc: Mike Galbraith, Ingo Molnar, Matt Fleming, linux-kernel

On Wed, May 04, 2016 at 05:45:10PM +0200, Peter Zijlstra wrote:
> On Tue, May 03, 2016 at 11:11:53AM -0400, Chris Mason wrote:
> > # pick a single core, in my case cpus 0,20 are the same core
> > # cpu_hog is any program that spins
> > #
> > taskset -c 20 cpu_hog &
> > 
> > # schbench -p 4 means message passing mode with 4 byte messages (like
> > # pipe test), no sleeps, just bouncing as fast as it can.
> > #
> > # make the scheduler choose between the sibling of the hog and cpu 1
> > #
> > taskset -c 0,1 schbench -p 4 -m 1 -t 1
> 
> Will that schbench thingy print something? Mine doesn't seem to output
> anything, not actually exit, although it stops consuming CPU cycles at
> some point.
> 
> 

It should, make sure you're at the top commit in git.

git://git.kernel.org/pub/scm/linux/kernel/git/mason/schbench.git

It's not recent so I'd be surprised if you weren't already there.  The
default runtime is 30 seconds, but you can use -r to specify something
shorter.

It's possible I'm missing a wakeup to shut the whole thing down, but I
thought I fixed that.

 ./schbench -p 4 -m 1 -t 1
Latency percentiles (usec)
        50.0000th: 5
        75.0000th: 5
        90.0000th: 5
        95.0000th: 5
        *99.0000th: 8
        99.5000th: 15
        99.9000th: 17
        Over=0, min=0, max=652
avg worker transfer: 113768.27 ops/sec 444.41KB/s

-chris

^ permalink raw reply	[flat|nested] 80+ messages in thread

* Re: sched: tweak select_idle_sibling to look for idle threads
  2016-05-04 17:46                     ` Chris Mason
@ 2016-05-05  9:33                       ` Peter Zijlstra
  2016-05-05 13:58                         ` Chris Mason
  0 siblings, 1 reply; 80+ messages in thread
From: Peter Zijlstra @ 2016-05-05  9:33 UTC (permalink / raw)
  To: Chris Mason, Mike Galbraith, Ingo Molnar, Matt Fleming, linux-kernel

On Wed, May 04, 2016 at 01:46:16PM -0400, Chris Mason wrote:
> It should, make sure you're at the top commit in git.
> 
> git://git.kernel.org/pub/scm/linux/kernel/git/mason/schbench.git

I did double check; I am on the top commit of that. I refetched and
rebuild just to make tripple sure.

> It's not recent so I'd be surprised if you weren't already there.  The
> default runtime is 30 seconds, but you can use -r to specify something
> shorter.
> 
> It's possible I'm missing a wakeup to shut the whole thing down, but I
> thought I fixed that.

Seems to still be missing, because:

>  ./schbench -p 4 -m 1 -t 1
> Latency percentiles (usec)
>         50.0000th: 5
>         75.0000th: 5
>         90.0000th: 5
>         95.0000th: 5
>         *99.0000th: 8
>         99.5000th: 15
>         99.9000th: 17
>         Over=0, min=0, max=652
> avg worker transfer: 113768.27 ops/sec 444.41KB/s

is not what mine does. I get ~25sec of cpu time and then it stalls
forever.

I'll try and have a prod at the program itself if you have no pending
changes on your end.

^ permalink raw reply	[flat|nested] 80+ messages in thread

* Re: sched: tweak select_idle_sibling to look for idle threads
  2016-05-05  9:33                       ` Peter Zijlstra
@ 2016-05-05 13:58                         ` Chris Mason
  2016-05-06  7:12                           ` Peter Zijlstra
  0 siblings, 1 reply; 80+ messages in thread
From: Chris Mason @ 2016-05-05 13:58 UTC (permalink / raw)
  To: Peter Zijlstra; +Cc: Mike Galbraith, Ingo Molnar, Matt Fleming, linux-kernel

On Thu, May 05, 2016 at 11:33:38AM +0200, Peter Zijlstra wrote:
> On Wed, May 04, 2016 at 01:46:16PM -0400, Chris Mason wrote:
> > It should, make sure you're at the top commit in git.
> > 
> > git://git.kernel.org/pub/scm/linux/kernel/git/mason/schbench.git
> 
> I did double check; I am on the top commit of that. I refetched and
> rebuild just to make tripple sure.
> 
> > It's not recent so I'd be surprised if you weren't already there.  The
> > default runtime is 30 seconds, but you can use -r to specify something
> > shorter.
> > 
> > It's possible I'm missing a wakeup to shut the whole thing down, but I
> > thought I fixed that.
> 
> Seems to still be missing, because:
> 
> >  ./schbench -p 4 -m 1 -t 1
> > Latency percentiles (usec)
> >         50.0000th: 5
> >         75.0000th: 5
> >         90.0000th: 5
> >         95.0000th: 5
> >         *99.0000th: 8
> >         99.5000th: 15
> >         99.9000th: 17
> >         Over=0, min=0, max=652
> > avg worker transfer: 113768.27 ops/sec 444.41KB/s
> 
> is not what mine does. I get ~25sec of cpu time and then it stalls
> forever.
> 
> I'll try and have a prod at the program itself if you have no pending
> changes on your end.

Sorry, I don't.  Look at sleep_for_runtime() and how I test/set the
global stopping variable in different places.  I've almost certainly got
someone waiting on a wakeup that'll never come.

If all else fails, run_msg_thread() can pass a timeout to fwait() for a
less error prone setup. I was just hoping to avoid the timers kernel side.

-chris

^ permalink raw reply	[flat|nested] 80+ messages in thread

* Re: sched: tweak select_idle_sibling to look for idle threads
  2016-05-04 10:37                   ` Peter Zijlstra
  2016-05-04 15:31                     ` Peter Zijlstra
@ 2016-05-05 22:03                     ` Matt Fleming
  2016-05-06 18:54                       ` Mike Galbraith
  1 sibling, 1 reply; 80+ messages in thread
From: Matt Fleming @ 2016-05-05 22:03 UTC (permalink / raw)
  To: Peter Zijlstra; +Cc: Chris Mason, Mike Galbraith, Ingo Molnar, linux-kernel

On Wed, 04 May, at 12:37:01PM, Peter Zijlstra wrote:
> 
> tbench wants select_idle_siblings() to just not exist; it goes happy
> when you just return target.

I've been playing with this patch a little bit by hitting it with
tbench on a Xeon, 12 cores with HT enabled, 2 sockets (48 cpus).

I see a throughput improvement for 16, 32, 64, 128 and 256 clients
when compared against mainline, so that's,

  OLD_IDLE, ORDER_IDLE, NO_IDLE_CORE, NO_IDLE_CPU, NO_IDLE_SMT

  vs.

  NO_OLD_IDLE, NO_ORDER_IDLE, IDLE_CORE, IDLE_CPU, IDLE_SMT

See,


 [OLD] Throughput 5345.6 MB/sec   16 clients  16 procs  max_latency=0.277 ms avg_latency=0.211853 ms
 [NEW] Throughput 5514.52 MB/sec  16 clients  16 procs  max_latency=0.493 ms avg_latency=0.176441 ms
 
 [OLD] Throughput 7401.76 MB/sec  32 clients  32 procs  max_latency=1.804 ms avg_latency=0.451147 ms
 [NEW] Throughput 10044.9 MB/sec  32 clients  32 procs  max_latency=3.421 ms avg_latency=0.582529 ms
 
 [OLD] Throughput 13265.9 MB/sec  64 clients  64 procs  max_latency=7.395 ms avg_latency=0.927147 ms
 [NEW] Throughput 13929.6 MB/sec  64 clients  64 procs  max_latency=7.022 ms avg_latency=1.017059 ms
 
 [OLD] Throughput 12827.8 MB/sec  128 clients  128 procs  max_latency=16.256 ms avg_latency=2.763706 ms
 [NEW] Throughput 13364.2 MB/sec  128 clients  128 procs  max_latency=16.630 ms avg_latency=3.002971 ms
 
 [OLD] Throughput 12653.1 MB/sec  256 clients  256 procs  max_latency=44.722 ms avg_latency=5.741647 ms
 [NEW] Throughput 12965.7 MB/sec  256 clients  256 procs  max_latency=59.061 ms avg_latency=8.699118 ms


For throughput changes to 1, 2, 4 and 8 clients it's more of a mixture
with sometimes the old config winning and sometimes losing.


 [OLD] Throughput 488.819 MB/sec  1 clients  1 procs  max_latency=0.191 ms avg_latency=0.058794 ms
 [NEW] Throughput 486.106 MB/sec  1 clients  1 procs  max_latency=0.085 ms avg_latency=0.045794 ms
 
 [OLD] Throughput 925.987 MB/sec  2 clients  2 procs  max_latency=0.201 ms avg_latency=0.090882 ms
 [NEW] Throughput 954.944 MB/sec  2 clients  2 procs  max_latency=0.199 ms avg_latency=0.064294 ms
 
 [OLD] Throughput 1764.02 MB/sec  4 clients  4 procs  max_latency=0.160 ms avg_latency=0.075206 ms
 [NEW] Throughput 1756.8 MB/sec   4 clients  4 procs  max_latency=0.105 ms avg_latency=0.062382 ms
 
 [OLD] Throughput 3384.22 MB/sec  8 clients  8 procs  max_latency=0.276 ms avg_latency=0.099441 ms
 [NEW] Throughput 3375.47 MB/sec  8 clients  8 procs  max_latency=0.103 ms avg_latency=0.064176 ms


Looking at latency, the new code consistently performs worse at the
top end for 256 clients. Admittedly at that point the machine is
pretty overloaded. Things are much better at the lower end.

One thing I haven't yet done is twiddled the bits individually to see
what the best combination is. Have you settled on the right settings
yet?

^ permalink raw reply	[flat|nested] 80+ messages in thread

* Re: sched: tweak select_idle_sibling to look for idle threads
  2016-05-05 13:58                         ` Chris Mason
@ 2016-05-06  7:12                           ` Peter Zijlstra
  2016-05-06 17:27                             ` Chris Mason
  0 siblings, 1 reply; 80+ messages in thread
From: Peter Zijlstra @ 2016-05-06  7:12 UTC (permalink / raw)
  To: Chris Mason, Mike Galbraith, Ingo Molnar, Matt Fleming, linux-kernel

On Thu, May 05, 2016 at 09:58:44AM -0400, Chris Mason wrote:
> > I'll try and have a prod at the program itself if you have no pending
> > changes on your end.
> 
> Sorry, I don't.  Look at sleep_for_runtime() and how I test/set the
> global stopping variable in different places.  I've almost certainly got
> someone waiting on a wakeup that'll never come.

The below makes it go..

---
 schbench.c | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)

diff --git a/schbench.c b/schbench.c
index a0e9f7e..f299959 100644
--- a/schbench.c
+++ b/schbench.c
@@ -49,7 +49,7 @@ static int pipe_test = 0;
 static unsigned int max_us = 50000;
 
 /* the message threads flip this to true when they decide runtime is up */
-static unsigned long stopping = 0;
+static volatile unsigned long stopping = 0;
 
 
 /*
@@ -746,8 +746,8 @@ static void sleep_for_runtime()
 		else
 			break;
 	}
-	stopping = 1;
 	__sync_synchronize();
+	stopping = 1;
 }
 
 int main(int ac, char **av)

^ permalink raw reply related	[flat|nested] 80+ messages in thread

* Re: sched: tweak select_idle_sibling to look for idle threads
  2016-05-03 15:11                 ` Chris Mason
  2016-05-04 10:37                   ` Peter Zijlstra
  2016-05-04 15:45                   ` Peter Zijlstra
@ 2016-05-06  7:25                   ` Peter Zijlstra
  2 siblings, 0 replies; 80+ messages in thread
From: Peter Zijlstra @ 2016-05-06  7:25 UTC (permalink / raw)
  To: Chris Mason, Mike Galbraith, Ingo Molnar, Matt Fleming, linux-kernel

On Tue, May 03, 2016 at 11:11:53AM -0400, Chris Mason wrote:
> # pick a single core, in my case cpus 0,20 are the same core
> # cpu_hog is any program that spins
> #
> taskset -c 20 cpu_hog &
> 
> # schbench -p 4 means message passing mode with 4 byte messages (like
> # pipe test), no sleeps, just bouncing as fast as it can.
> #
> # make the scheduler choose between the sibling of the hog and cpu 1
> #
> taskset -c 0,1 schbench -p 4 -m 1 -t 1
> 
> Current mainline will stuff both schbench threads onto CPU 1, leaving
> CPU 0 100% idle.  My first patch with the minimal task_hot() checks
> would sometimes pick CPU 0.  My second patch that just directly calls
> task_hot sticks to cpu1, which is ~3x faster than spreading it.

Ok, with the thing fixed, my current patch seems to DTRT. If I trace
sched_migrate_task() I get:

$ grep schbench trace

 doit-schbench-2-4042  [004] d..3 144541.309747: sched_migrate_task: comm=doit-schbench-2 pid=4042 prio=120 orig_cpu=4 dest_cpu=4
 doit-schbench-2-4042  [004] d..2 144541.309772: sched_migrate_task: comm=doit-schbench-2 pid=4043 prio=120 orig_cpu=4 dest_cpu=11
 doit-schbench-2-4042  [004] d..3 144541.309855: sched_migrate_task: comm=doit-schbench-2 pid=4042 prio=120 orig_cpu=4 dest_cpu=4
 doit-schbench-2-4042  [004] d..2 144541.309882: sched_migrate_task: comm=doit-schbench-2 pid=4044 prio=120 orig_cpu=4 dest_cpu=5
    migration/11-77    [011] d..4 144541.309974: sched_migrate_task: comm=doit-schbench-2 pid=4043 prio=120 orig_cpu=11 dest_cpu=12
     migration/5-40    [005] d..4 144541.310013: sched_migrate_task: comm=doit-schbench-2 pid=4044 prio=120 orig_cpu=5 dest_cpu=6
        schbench-4044  [001] d..3 144541.310995: sched_migrate_task: comm=schbench pid=4044 prio=120 orig_cpu=1 dest_cpu=1
        schbench-4044  [001] d..2 144541.310999: sched_migrate_task: comm=schbench pid=4045 prio=120 orig_cpu=1 dest_cpu=1
        schbench-4045  [001] d..3 144541.311232: sched_migrate_task: comm=schbench pid=4045 prio=120 orig_cpu=1 dest_cpu=1
        schbench-4045  [001] d..2 144541.311234: sched_migrate_task: comm=schbench pid=4046 prio=120 orig_cpu=1 dest_cpu=1

So the thing gets put on cpu1 and never leaves.

^ permalink raw reply	[flat|nested] 80+ messages in thread

* Re: sched: tweak select_idle_sibling to look for idle threads
  2016-05-06  7:12                           ` Peter Zijlstra
@ 2016-05-06 17:27                             ` Chris Mason
  0 siblings, 0 replies; 80+ messages in thread
From: Chris Mason @ 2016-05-06 17:27 UTC (permalink / raw)
  To: Peter Zijlstra; +Cc: Mike Galbraith, Ingo Molnar, Matt Fleming, linux-kernel

On Fri, May 06, 2016 at 09:12:51AM +0200, Peter Zijlstra wrote:
> ontent-Length: 973
> 
> On Thu, May 05, 2016 at 09:58:44AM -0400, Chris Mason wrote:
> > > I'll try and have a prod at the program itself if you have no pending
> > > changes on your end.
> > 
> > Sorry, I don't.  Look at sleep_for_runtime() and how I test/set the
> > global stopping variable in different places.  I've almost certainly got
> > someone waiting on a wakeup that'll never come.
> 
> The below makes it go..

Thanks Peter, pushed to git.

-chris

^ permalink raw reply	[flat|nested] 80+ messages in thread

* Re: sched: tweak select_idle_sibling to look for idle threads
  2016-05-05 22:03                     ` Matt Fleming
@ 2016-05-06 18:54                       ` Mike Galbraith
  2016-05-09  8:33                         ` Peter Zijlstra
  0 siblings, 1 reply; 80+ messages in thread
From: Mike Galbraith @ 2016-05-06 18:54 UTC (permalink / raw)
  To: Matt Fleming, Peter Zijlstra; +Cc: Chris Mason, Ingo Molnar, linux-kernel

On Thu, 2016-05-05 at 23:03 +0100, Matt Fleming wrote:

> One thing I haven't yet done is twiddled the bits individually to see
> what the best combination is. Have you settled on the right settings
> yet?

Lighter configs, revert sched/fair: Fix fairness issue on migration,
twiddle knobs.  Added an IDLE_SIBLING knob to ~virgin master.. only
sorta virgin because I always throttle nohz.

1 x i4790
master
for i in 1 2 4 8; do tbench.sh $i 30 2>&1|grep Throughput; done
Throughput 871.785 MB/sec  1 clients  1 procs  max_latency=0.324 ms
Throughput 1514.5 MB/sec  2 clients  2 procs  max_latency=0.411 ms
Throughput 2722.43 MB/sec  4 clients  4 procs  max_latency=2.400 ms
Throughput 4334.46 MB/sec  8 clients  8 procs  max_latency=3.561 ms

echo NO_IDLE_SIBLING > /sys/kernel/debug/sched_features
Throughput 1078.69 MB/sec  1 clients  1 procs  max_latency=2.274 ms
Throughput 2130.33 MB/sec  2 clients  2 procs  max_latency=1.451 ms
Throughput 3484.18 MB/sec  4 clients  4 procs  max_latency=3.430 ms
Throughput 4423.69 MB/sec  8 clients  8 procs  max_latency=5.363 ms


masterx
for i in 1 2 4 8; do tbench.sh $i 30 2>&1|grep Throughput; done
Throughput 707.673 MB/sec  1 clients  1 procs  max_latency=2.279 ms
Throughput 1503.55 MB/sec  2 clients  2 procs  max_latency=0.695 ms
Throughput 2527.73 MB/sec  4 clients  4 procs  max_latency=2.321 ms
Throughput 4291.26 MB/sec  8 clients  8 procs  max_latency=3.815 ms

echo NO_IDLE_CPU > /sys/kernel/debug/sched_features
homer:~ # for i in 1 2 4 8; do tbench.sh $i 30 2>&1|grep Throughput; done
Throughput 865.936 MB/sec  1 clients  1 procs  max_latency=0.411 ms
Throughput 1586.41 MB/sec  2 clients  2 procs  max_latency=2.293 ms
Throughput 2638.39 MB/sec  4 clients  4 procs  max_latency=2.037 ms
Throughput 4405.43 MB/sec  8 clients  8 procs  max_latency=3.581 ms

+ echo NO_AVG_CPU > /sys/kernel/debug/sched_features
+ echo IDLE_SMT > /sys/kernel/debug/sched_features
Throughput 697.126 MB/sec  1 clients  1 procs  max_latency=2.220 ms
Throughput 1562.82 MB/sec  2 clients  2 procs  max_latency=0.526 ms
Throughput 2620.62 MB/sec  4 clients  4 procs  max_latency=6.460 ms
Throughput 4345.13 MB/sec  8 clients  8 procs  max_latency=27.921 ms


4 x E7-8890
master
for i in 1 2 4 8 16 32 64 128 256; do tbench.sh $i 30 2>&1| grep Throughput; done
Throughput 615.663 MB/sec  1 clients  1 procs  max_latency=0.087 ms
Throughput 1171.53 MB/sec  2 clients  2 procs  max_latency=0.087 ms
Throughput 2251.22 MB/sec  4 clients  4 procs  max_latency=0.078 ms
Throughput 4090.76 MB/sec  8 clients  8 procs  max_latency=0.801 ms
Throughput 7695.92 MB/sec  16 clients  16 procs  max_latency=0.235 ms
Throughput 15152 MB/sec  32 clients  32 procs  max_latency=0.693 ms
Throughput 21628.2 MB/sec  64 clients  64 procs  max_latency=4.666 ms
Throughput 43185.7 MB/sec  128 clients  128 procs  max_latency=7.280 ms
Throughput 72144.5 MB/sec  256 clients  256 procs  max_latency=8.194 ms

echo NO_IDLE_SIBLING > /sys/kernel/debug/sched_features
Throughput 954.593 MB/sec  1 clients  1 procs  max_latency=0.185 ms
Throughput 1882.65 MB/sec  2 clients  2 procs  max_latency=0.278 ms
Throughput 3457.03 MB/sec  4 clients  4 procs  max_latency=0.431 ms
Throughput 6279.38 MB/sec  8 clients  8 procs  max_latency=0.730 ms
Throughput 11170.4 MB/sec  16 clients  16 procs  max_latency=0.500 ms
Throughput 21940.9 MB/sec  32 clients  32 procs  max_latency=0.475 ms
Throughput 41738.8 MB/sec  64 clients  64 procs  max_latency=3.669 ms
Throughput 67634.6 MB/sec  128 clients  128 procs  max_latency=6.676 ms
Throughput 76299.7 MB/sec  256 clients  256 procs  max_latency=7.878 ms

masterx
for i in 1 2 4 8 16 32 64 128 256; do tbench.sh $i 30 2>&1| grep Throughput; done
Throughput 587.956 MB/sec  1 clients  1 procs  max_latency=0.124 ms
Throughput 1140.16 MB/sec  2 clients  2 procs  max_latency=0.476 ms
Throughput 2296.03 MB/sec  4 clients  4 procs  max_latency=0.142 ms
Throughput 4116.65 MB/sec  8 clients  8 procs  max_latency=0.464 ms
Throughput 7820.27 MB/sec  16 clients  16 procs  max_latency=0.238 ms
Throughput 14899.2 MB/sec  32 clients  32 procs  max_latency=0.321 ms
Throughput 21909.8 MB/sec  64 clients  64 procs  max_latency=0.905 ms
Throughput 35495.2 MB/sec  128 clients  128 procs  max_latency=6.158 ms
Throughput 75863.2 MB/sec  256 clients  256 procs  max_latency=7.650 ms

echo NO_IDLE_CPU > /sys/kernel/debug/sched_features
Throughput 555.15 MB/sec  1 clients  1 procs  max_latency=0.096 ms
Throughput 1195.12 MB/sec  2 clients  2 procs  max_latency=0.131 ms
Throughput 2276.97 MB/sec  4 clients  4 procs  max_latency=0.105 ms
Throughput 4248.14 MB/sec  8 clients  8 procs  max_latency=0.131 ms
Throughput 7860.86 MB/sec  16 clients  16 procs  max_latency=0.210 ms
Throughput 15178.6 MB/sec  32 clients  32 procs  max_latency=0.229 ms
Throughput 21523.9 MB/sec  64 clients  64 procs  max_latency=0.842 ms
Throughput 31082.1 MB/sec  128 clients  128 procs  max_latency=7.311 ms
Throughput 75887.9 MB/sec  256 clients  256 procs  max_latency=7.764 ms

+ echo NO_AVG_CPU > /sys/kernel/debug/sched_features
Throughput 598.063 MB/sec  1 clients  1 procs  max_latency=0.131 ms
Throughput 1140.2 MB/sec  2 clients  2 procs  max_latency=0.092 ms
Throughput 2268.68 MB/sec  4 clients  4 procs  max_latency=0.170 ms
Throughput 4259.7 MB/sec  8 clients  8 procs  max_latency=0.212 ms
Throughput 7904.15 MB/sec  16 clients  16 procs  max_latency=0.191 ms
Throughput 14840 MB/sec  32 clients  32 procs  max_latency=0.279 ms
Throughput 21701.5 MB/sec  64 clients  64 procs  max_latency=0.856 ms
Throughput 38945 MB/sec  128 clients  128 procs  max_latency=7.501 ms
Throughput 75669.4 MB/sec  256 clients  256 procs  max_latency=14.984 ms

+ echo IDLE_SMT > /sys/kernel/debug/sched_features
Throughput 592.799 MB/sec  1 clients  1 procs  max_latency=0.120 ms
Throughput 1208.28 MB/sec  2 clients  2 procs  max_latency=0.078 ms
Throughput 2319.22 MB/sec  4 clients  4 procs  max_latency=0.141 ms
Throughput 4196.64 MB/sec  8 clients  8 procs  max_latency=0.253 ms
Throughput 7816.47 MB/sec  16 clients  16 procs  max_latency=0.117 ms
Throughput 14990.8 MB/sec  32 clients  32 procs  max_latency=0.189 ms
Throughput 21809.4 MB/sec  64 clients  64 procs  max_latency=0.832 ms
Throughput 44813 MB/sec  128 clients  128 procs  max_latency=7.930 ms
Throughput 75978.1 MB/sec  256 clients  256 procs  max_latency=7.337 ms

^ permalink raw reply	[flat|nested] 80+ messages in thread

* Re: sched: tweak select_idle_sibling to look for idle threads
  2016-05-01  9:20         ` Mike Galbraith
@ 2016-05-07  1:24           ` Yuyang Du
  2016-05-08  8:08             ` Mike Galbraith
  0 siblings, 1 reply; 80+ messages in thread
From: Yuyang Du @ 2016-05-07  1:24 UTC (permalink / raw)
  To: Mike Galbraith
  Cc: Peter Zijlstra, Chris Mason, Ingo Molnar, Matt Fleming, linux-kernel

On Sun, May 01, 2016 at 11:20:25AM +0200, Mike Galbraith wrote:
> On Sun, 2016-05-01 at 10:53 +0200, Peter Zijlstra wrote:
> > On Sun, May 01, 2016 at 09:12:33AM +0200, Mike Galbraith wrote:
> > > On Sat, 2016-04-30 at 14:47 +0200, Peter Zijlstra wrote:
> > 
> > > > Can you guys have a play with this; I think one and two node tbench are
> > > > good, but I seem to be getting significant run to run variance on that,
> > > > so maybe I'm not doing it right.
> > > 
> > > Nah, tbench is just variance prone.  It got dinged up at clients=cores
> > > on my desktop box, on 4 sockets the high end got seriously dinged up.
> > 
> > Ouch, yeah, big hurt. Lets try that again... :-)
> 
> Yeah, box could use a little bandaid and a hug :)
> 
> Playing with Chris' benchmark, seems the biggest problem is that we
> don't buddy up waker of many and it's wakees in a node.. ie the wake
> wide thing isn't necessarily our friend when there are multiple wakers
> of many.  If I run an instance per node with one mother of all work in
> autobench mode, it works exactly as you'd expect, game over is when
> wakees = socket size. It never get's near that point if I let things
> wander, it beats itself up well before we get there.

Maybe give the criteria a bit margin, not just wakees tend to equal llc_size,
but the numbers are so wild to easily break the fragile condition, like:

if (master * 100 < slave * factor * 110)
        return 0;

And since you accumulate wakee number (and decay at HZ), this check tends to
not satisfy ever?

if (slave < factor)
	return 0;

^ permalink raw reply	[flat|nested] 80+ messages in thread

* Re: sched: tweak select_idle_sibling to look for idle threads
  2016-05-07  1:24           ` Yuyang Du
@ 2016-05-08  8:08             ` Mike Galbraith
  2016-05-08 18:57               ` Yuyang Du
  0 siblings, 1 reply; 80+ messages in thread
From: Mike Galbraith @ 2016-05-08  8:08 UTC (permalink / raw)
  To: Yuyang Du
  Cc: Peter Zijlstra, Chris Mason, Ingo Molnar, Matt Fleming, linux-kernel

On Sat, 2016-05-07 at 09:24 +0800, Yuyang Du wrote:
> On Sun, May 01, 2016 at 11:20:25AM +0200, Mike Galbraith wrote:

> > Playing with Chris' benchmark, seems the biggest problem is that we
> > don't buddy up waker of many and it's wakees in a node.. ie the wake
> > wide thing isn't necessarily our friend when there are multiple wakers
> > of many.  If I run an instance per node with one mother of all work in
> > autobench mode, it works exactly as you'd expect, game over is when
> > wakees = socket size. It never get's near that point if I let things
> > wander, it beats itself up well before we get there.
> 
> Maybe give the criteria a bit margin, not just wakees tend to equal llc_size,
> but the numbers are so wild to easily break the fragile condition, like:

Seems lockless traversal and averages just lets multiple CPUs select
the same spot.  An atomic reservation (feature) when looking for an
idle spot (also for fork) might fix it up.  Run the thing as RT,
push/pull ensures that it reaches box saturation regardless of the
number of messaging threads, whereas with fair class, any number > 1
will certainly stack tasks before the box is saturated.

	-Mike

^ permalink raw reply	[flat|nested] 80+ messages in thread

* Re: sched: tweak select_idle_sibling to look for idle threads
  2016-05-08  8:08             ` Mike Galbraith
@ 2016-05-08 18:57               ` Yuyang Du
  2016-05-09  3:45                 ` Mike Galbraith
  2016-05-09  3:52                 ` Mike Galbraith
  0 siblings, 2 replies; 80+ messages in thread
From: Yuyang Du @ 2016-05-08 18:57 UTC (permalink / raw)
  To: Mike Galbraith
  Cc: Peter Zijlstra, Chris Mason, Ingo Molnar, Matt Fleming, linux-kernel

On Sun, May 08, 2016 at 10:08:55AM +0200, Mike Galbraith wrote:
> > Maybe give the criteria a bit margin, not just wakees tend to equal llc_size,
> > but the numbers are so wild to easily break the fragile condition, like:
> 
> Seems lockless traversal and averages just lets multiple CPUs select
> the same spot.  An atomic reservation (feature) when looking for an
> idle spot (also for fork) might fix it up.  Run the thing as RT,
> push/pull ensures that it reaches box saturation regardless of the
> number of messaging threads, whereas with fair class, any number > 1
> will certainly stack tasks before the box is saturated.

Yes, good idea, bringing order to the race to grab idle CPU is absolutely
helpful.

In addition, I would argue maybe beefing up idle balancing is a more
productive way to spread load, as work-stealing just does what needs
to be done. And seems it has been (sub-unconsciously) neglected in this
case, :)

Regarding wake_wide(), it seems the M:N is 1:24, not 6:6*24, if so,
the slave will be 0 forever (as last_wakee is never flipped).

Basically whenever a waker has more than 1 wakee, the wakee_flips
will comfortably grow very large (with last_wakee alternating),
whereas when a waker has 0 or 1 wakee, the wakee_flips will just be 0.

So recording only the last_wakee seems not right unless you have other
good reason. If not the latter, counting waking wakee times should be
better, and then allow the statistics to happily play.

^ permalink raw reply	[flat|nested] 80+ messages in thread

* Re: sched: tweak select_idle_sibling to look for idle threads
  2016-05-09  3:45                 ` Mike Galbraith
@ 2016-05-08 20:22                   ` Yuyang Du
  2016-05-09  7:44                     ` Mike Galbraith
  0 siblings, 1 reply; 80+ messages in thread
From: Yuyang Du @ 2016-05-08 20:22 UTC (permalink / raw)
  To: Mike Galbraith
  Cc: Peter Zijlstra, Chris Mason, Ingo Molnar, Matt Fleming, linux-kernel

On Mon, May 09, 2016 at 05:45:40AM +0200, Mike Galbraith wrote:
> On Mon, 2016-05-09 at 02:57 +0800, Yuyang Du wrote:
> > On Sun, May 08, 2016 at 10:08:55AM +0200, Mike Galbraith wrote:
> > > > Maybe give the criteria a bit margin, not just wakees tend to equal llc_size,
> > > > but the numbers are so wild to easily break the fragile condition, like:
> > > 
> > > Seems lockless traversal and averages just lets multiple CPUs select
> > > the same spot.  An atomic reservation (feature) when looking for an
> > > idle spot (also for fork) might fix it up.  Run the thing as RT,
> > > push/pull ensures that it reaches box saturation regardless of the
> > > number of messaging threads, whereas with fair class, any number > 1
> > > will certainly stack tasks before the box is saturated.
> > 
> > Yes, good idea, bringing order to the race to grab idle CPU is absolutely
> > helpful.
> 
> Well, good ideas work, as yet this one helps jack diddly spit.

Then a valid question is whether it is this selection screwed up in case
like this, as it should necessarily always be asked.
 
> > In addition, I would argue maybe beefing up idle balancing is a more
> > productive way to spread load, as work-stealing just does what needs
> > to be done. And seems it has been (sub-unconsciously) neglected in this
> > case, :)
> > 
> > Regarding wake_wide(), it seems the M:N is 1:24, not 6:6*24, if so,
> > the slave will be 0 forever (as last_wakee is never flipped).
> 
> Yeah, it's irrelevant here, this load is all about instantaneous state.
>  I could use a bit more of that, reserving on the wakeup side won't
> help this benchmark until everything else cares.  One stack, and it's
> game over.  It could help generic utilization and latency some.. but it
> seems kinda unlikely it'll be worth the cycle expenditure.

Yes and no, it depends on how efficient work-stealing is, compared to
selection, but remember, at the end of the day, the wakee CPU measures the
latency, that CPU does not care it is selected or it steals.
 
> > Basically whenever a waker has more than 1 wakee, the wakee_flips
> > will comfortably grow very large (with last_wakee alternating),
> > whereas when a waker has 0 or 1 wakee, the wakee_flips will just be 0.
> 
> Yup, it is a heuristic, and like all of those, imperfect.  I've watched
> it improving utilization in the wild though, so won't mind that until I
> catch it doing really bad things.
 
> > So recording only the last_wakee seems not right unless you have other
> > good reason. If not the latter, counting waking wakee times should be
> > better, and then allow the statistics to happily play.

En... should we try remove recording last_wakee?

^ permalink raw reply	[flat|nested] 80+ messages in thread

* Re: sched: tweak select_idle_sibling to look for idle threads
  2016-05-09  3:52                 ` Mike Galbraith
@ 2016-05-08 20:31                   ` Yuyang Du
  0 siblings, 0 replies; 80+ messages in thread
From: Yuyang Du @ 2016-05-08 20:31 UTC (permalink / raw)
  To: Mike Galbraith
  Cc: Peter Zijlstra, Chris Mason, Ingo Molnar, Matt Fleming, linux-kernel

On Mon, May 09, 2016 at 05:52:51AM +0200, Mike Galbraith wrote:
> On Mon, 2016-05-09 at 02:57 +0800, Yuyang Du wrote:
> 
> > In addition, I would argue maybe beefing up idle balancing is a more
> > productive way to spread load, as work-stealing just does what needs
> > to be done. And seems it has been (sub-unconsciously) neglected in this
> > case, :)
> 
> P.S. Nope, I'm dinging up multiple spots ;-)

You bet, :)

^ permalink raw reply	[flat|nested] 80+ messages in thread

* Re: sched: tweak select_idle_sibling to look for idle threads
  2016-05-09  7:44                     ` Mike Galbraith
@ 2016-05-09  1:13                       ` Yuyang Du
  2016-05-09  9:39                         ` Mike Galbraith
  0 siblings, 1 reply; 80+ messages in thread
From: Yuyang Du @ 2016-05-09  1:13 UTC (permalink / raw)
  To: Mike Galbraith
  Cc: Peter Zijlstra, Chris Mason, Ingo Molnar, Matt Fleming, linux-kernel

On Mon, May 09, 2016 at 09:44:13AM +0200, Mike Galbraith wrote:
> > Then a valid question is whether it is this selection screwed up in case
> > like this, as it should necessarily always be asked.
> 
> That's a given, it's just a question of how to do a bit better cheaply.
>  
> > > > Regarding wake_wide(), it seems the M:N is 1:24, not 6:6*24, if so,
> > > > the slave will be 0 forever (as last_wakee is never flipped).
> > > 
> > > Yeah, it's irrelevant here, this load is all about instantaneous state.
> > >  I could use a bit more of that, reserving on the wakeup side won't
> > > help this benchmark until everything else cares.  One stack, and it's
> > > game over.  It could help generic utilization and latency some.. but it
> > > seems kinda unlikely it'll be worth the cycle expenditure.
> > 
> > Yes and no, it depends on how efficient work-stealing is, compared to
> > selection, but remember, at the end of the day, the wakee CPU measures the
> > latency, that CPU does not care it is selected or it steals.
> 
> In a perfect world, running only Chris' benchmark on an otherwise idle
> box, there would never _be_ any work to steal. 

What is the perfect world like? I don't get what you mean.

> In the real world, we
> smooth utilization, optimistically peek at this/that, and intentionally
> throttle idle balancing (etc etc), which adds up to an imperfect world
> for this (based on real world load) benchmark.
 
So, is this a shout-out: these parts should be coordinated better?

> > En... should we try remove recording last_wakee?
> 
> The more the merrier, go for it! :)
 
Nuh, really, this heuristic is too heuristic, :) 
The totality of all possible cases is scary.

Just for a general M:N two-way waker-wakee relationship, not recording
last_wakee may work well generally. E.g., currently, on a 2-socket (24-thread
per socket) 1:24 and 1:48 can't really be differentiated, whereas 1:24 and
2:48 are completely different.

Am I understanding correctly?

^ permalink raw reply	[flat|nested] 80+ messages in thread

* Re: sched: tweak select_idle_sibling to look for idle threads
  2016-05-08 18:57               ` Yuyang Du
@ 2016-05-09  3:45                 ` Mike Galbraith
  2016-05-08 20:22                   ` Yuyang Du
  2016-05-09  3:52                 ` Mike Galbraith
  1 sibling, 1 reply; 80+ messages in thread
From: Mike Galbraith @ 2016-05-09  3:45 UTC (permalink / raw)
  To: Yuyang Du
  Cc: Peter Zijlstra, Chris Mason, Ingo Molnar, Matt Fleming, linux-kernel

On Mon, 2016-05-09 at 02:57 +0800, Yuyang Du wrote:
> On Sun, May 08, 2016 at 10:08:55AM +0200, Mike Galbraith wrote:
> > > Maybe give the criteria a bit margin, not just wakees tend to equal llc_size,
> > > but the numbers are so wild to easily break the fragile condition, like:
> > 
> > Seems lockless traversal and averages just lets multiple CPUs select
> > the same spot.  An atomic reservation (feature) when looking for an
> > idle spot (also for fork) might fix it up.  Run the thing as RT,
> > push/pull ensures that it reaches box saturation regardless of the
> > number of messaging threads, whereas with fair class, any number > 1
> > will certainly stack tasks before the box is saturated.
> 
> Yes, good idea, bringing order to the race to grab idle CPU is absolutely
> helpful.

Well, good ideas work, as yet this one helps jack diddly spit.

> In addition, I would argue maybe beefing up idle balancing is a more
> productive way to spread load, as work-stealing just does what needs
> to be done. And seems it has been (sub-unconsciously) neglected in this
> case, :)
> 
> Regarding wake_wide(), it seems the M:N is 1:24, not 6:6*24, if so,
> the slave will be 0 forever (as last_wakee is never flipped).

Yeah, it's irrelevant here, this load is all about instantaneous state.
 I could use a bit more of that, reserving on the wakeup side won't
help this benchmark until everything else cares.  One stack, and it's
game over.  It could help generic utilization and latency some.. but it
seems kinda unlikely it'll be worth the cycle expenditure.

> Basically whenever a waker has more than 1 wakee, the wakee_flips
> will comfortably grow very large (with last_wakee alternating),
> whereas when a waker has 0 or 1 wakee, the wakee_flips will just be 0.

Yup, it is a heuristic, and like all of those, imperfect.  I've watched
it improving utilization in the wild though, so won't mind that until I
catch it doing really bad things.

> So recording only the last_wakee seems not right unless you have other
> good reason. If not the latter, counting waking wakee times should be
> better, and then allow the statistics to happily play.

^ permalink raw reply	[flat|nested] 80+ messages in thread

* Re: sched: tweak select_idle_sibling to look for idle threads
  2016-05-08 18:57               ` Yuyang Du
  2016-05-09  3:45                 ` Mike Galbraith
@ 2016-05-09  3:52                 ` Mike Galbraith
  2016-05-08 20:31                   ` Yuyang Du
  1 sibling, 1 reply; 80+ messages in thread
From: Mike Galbraith @ 2016-05-09  3:52 UTC (permalink / raw)
  To: Yuyang Du
  Cc: Peter Zijlstra, Chris Mason, Ingo Molnar, Matt Fleming, linux-kernel

On Mon, 2016-05-09 at 02:57 +0800, Yuyang Du wrote:

> In addition, I would argue maybe beefing up idle balancing is a more
> productive way to spread load, as work-stealing just does what needs
> to be done. And seems it has been (sub-unconsciously) neglected in this
> case, :)

P.S. Nope, I'm dinging up multiple spots ;-)

^ permalink raw reply	[flat|nested] 80+ messages in thread

* Re: sched: tweak select_idle_sibling to look for idle threads
  2016-05-08 20:22                   ` Yuyang Du
@ 2016-05-09  7:44                     ` Mike Galbraith
  2016-05-09  1:13                       ` Yuyang Du
  0 siblings, 1 reply; 80+ messages in thread
From: Mike Galbraith @ 2016-05-09  7:44 UTC (permalink / raw)
  To: Yuyang Du
  Cc: Peter Zijlstra, Chris Mason, Ingo Molnar, Matt Fleming, linux-kernel

On Mon, 2016-05-09 at 04:22 +0800, Yuyang Du wrote:
> On Mon, May 09, 2016 at 05:45:40AM +0200, Mike Galbraith wrote:
> > On Mon, 2016-05-09 at 02:57 +0800, Yuyang Du wrote:
> > > On Sun, May 08, 2016 at 10:08:55AM +0200, Mike Galbraith wrote:
> > > > > Maybe give the criteria a bit margin, not just wakees tend to equal llc_size,
> > > > > but the numbers are so wild to easily break the fragile condition, like:
> > > > 
> > > > Seems lockless traversal and averages just lets multiple CPUs select
> > > > the same spot.  An atomic reservation (feature) when looking for an
> > > > idle spot (also for fork) might fix it up.  Run the thing as RT,
> > > > push/pull ensures that it reaches box saturation regardless of the
> > > > number of messaging threads, whereas with fair class, any number > 1
> > > > will certainly stack tasks before the box is saturated.
> > > 
> > > Yes, good idea, bringing order to the race to grab idle CPU is absolutely
> > > helpful.
> > 
> > Well, good ideas work, as yet this one helps jack diddly spit.
> 
> Then a valid question is whether it is this selection screwed up in case
> like this, as it should necessarily always be asked.

That's a given, it's just a question of how to do a bit better cheaply.
 
> > > Regarding wake_wide(), it seems the M:N is 1:24, not 6:6*24, if so,
> > > the slave will be 0 forever (as last_wakee is never flipped).
> > 
> > Yeah, it's irrelevant here, this load is all about instantaneous state.
> >  I could use a bit more of that, reserving on the wakeup side won't
> > help this benchmark until everything else cares.  One stack, and it's
> > game over.  It could help generic utilization and latency some.. but it
> > seems kinda unlikely it'll be worth the cycle expenditure.
> 
> Yes and no, it depends on how efficient work-stealing is, compared to
> selection, but remember, at the end of the day, the wakee CPU measures the
> latency, that CPU does not care it is selected or it steals.

In a perfect world, running only Chris' benchmark on an otherwise idle
box, there would never _be_ any work to steal.  In the real world, we
smooth utilization, optimistically peek at this/that, and intentionally
throttle idle balancing (etc etc), which adds up to an imperfect world
for this (based on real world load) benchmark.

> En... should we try remove recording last_wakee?

The more the merrier, go for it! :)

	-Mike

^ permalink raw reply	[flat|nested] 80+ messages in thread

* Re: sched: tweak select_idle_sibling to look for idle threads
  2016-05-06 18:54                       ` Mike Galbraith
@ 2016-05-09  8:33                         ` Peter Zijlstra
  2016-05-09  8:56                           ` Mike Galbraith
  0 siblings, 1 reply; 80+ messages in thread
From: Peter Zijlstra @ 2016-05-09  8:33 UTC (permalink / raw)
  To: Mike Galbraith; +Cc: Matt Fleming, Chris Mason, Ingo Molnar, linux-kernel

On Fri, May 06, 2016 at 08:54:38PM +0200, Mike Galbraith wrote:
> master
> Throughput 2722.43 MB/sec  4 clients  4 procs  max_latency=2.400 ms

> echo NO_IDLE_SIBLING > /sys/kernel/debug/sched_features
> Throughput 3484.18 MB/sec  4 clients  4 procs  max_latency=3.430 ms

Yeah, I know about that bump, I just haven't managed to find a way to
preserve that and keep all the other benchmarks ticking along :/

^ permalink raw reply	[flat|nested] 80+ messages in thread

* Re: sched: tweak select_idle_sibling to look for idle threads
  2016-05-09  8:33                         ` Peter Zijlstra
@ 2016-05-09  8:56                           ` Mike Galbraith
  0 siblings, 0 replies; 80+ messages in thread
From: Mike Galbraith @ 2016-05-09  8:56 UTC (permalink / raw)
  To: Peter Zijlstra; +Cc: Matt Fleming, Chris Mason, Ingo Molnar, linux-kernel

On Mon, 2016-05-09 at 10:33 +0200, Peter Zijlstra wrote:
> On Fri, May 06, 2016 at 08:54:38PM +0200, Mike Galbraith wrote:
> > master
> > Throughput 2722.43 MB/sec  4 clients  4 procs  max_latency=2.400 ms
> 
> > echo NO_IDLE_SIBLING > /sys/kernel/debug/sched_features
> > Throughput 3484.18 MB/sec  4 clients  4 procs  max_latency=3.430 ms
> 
> Yeah, I know about that bump, I just haven't managed to find a way to
> preserve that and keep all the other benchmarks ticking along :/

Yup, L3 ain't L2.  I haven't come up with a good metric either. Poo. 
 Until that comes along, microbenchmarks can bugger off, real boxen
don't just play high speed ping-pong with themselves for a living ;-)

	-Mike

^ permalink raw reply	[flat|nested] 80+ messages in thread

* Re: sched: tweak select_idle_sibling to look for idle threads
  2016-05-09  1:13                       ` Yuyang Du
@ 2016-05-09  9:39                         ` Mike Galbraith
  2016-05-09 23:26                           ` Yuyang Du
  0 siblings, 1 reply; 80+ messages in thread
From: Mike Galbraith @ 2016-05-09  9:39 UTC (permalink / raw)
  To: Yuyang Du
  Cc: Peter Zijlstra, Chris Mason, Ingo Molnar, Matt Fleming, linux-kernel

On Mon, 2016-05-09 at 09:13 +0800, Yuyang Du wrote:
> On Mon, May 09, 2016 at 09:44:13AM +0200, Mike Galbraith wrote:

> > In a perfect world, running only Chris' benchmark on an otherwise idle
> > box, there would never _be_ any work to steal. 
> 
> What is the perfect world like? I don't get what you mean.

In a perfect world from this benchmark's perspective, when you fork or
wake while box is underutilized, wakee/child lands on an idle CPU.  To
this benchmark, anything else is broken.
 
> > In the real world, we
> > smooth utilization, optimistically peek at this/that, and intentionally
> > throttle idle balancing (etc etc), which adds up to an imperfect world
> > for this (based on real world load) benchmark.
>  
> So, is this a shout-out: these parts should be coordinated better?

Switching to instantaneous load along with the cpu reservation hackery
made Chris's benchmark a happy camper.  Is that the answer?  Nope, just
verification of the where the problem lives.

> > > En... should we try remove recording last_wakee?
> > 
> > The more the merrier, go for it! :)
>  
> Nuh, really, this heuristic is too heuristic, :) 
> The totality of all possible cases is scary.

Well, make it better.  The author provided evidence when it was born.

	-Mike

^ permalink raw reply	[flat|nested] 80+ messages in thread

* Re: sched: tweak select_idle_sibling to look for idle threads
  2016-05-09  9:39                         ` Mike Galbraith
@ 2016-05-09 23:26                           ` Yuyang Du
  2016-05-10  7:49                             ` Mike Galbraith
  0 siblings, 1 reply; 80+ messages in thread
From: Yuyang Du @ 2016-05-09 23:26 UTC (permalink / raw)
  To: Mike Galbraith
  Cc: Peter Zijlstra, Chris Mason, Ingo Molnar, Matt Fleming, linux-kernel

On Mon, May 09, 2016 at 11:39:05AM +0200, Mike Galbraith wrote:
> On Mon, 2016-05-09 at 09:13 +0800, Yuyang Du wrote:
> > On Mon, May 09, 2016 at 09:44:13AM +0200, Mike Galbraith wrote:
> 
> > > In a perfect world, running only Chris' benchmark on an otherwise idle
> > > box, there would never _be_ any work to steal. 
> > 
> > What is the perfect world like? I don't get what you mean.
> 
> In a perfect world from this benchmark's perspective, when you fork or
> wake while box is underutilized, wakee/child lands on an idle CPU.  To
> this benchmark, anything else is broken.
>  
> > > In the real world, we
> > > smooth utilization, optimistically peek at this/that, and intentionally
> > > throttle idle balancing (etc etc), which adds up to an imperfect world
> > > for this (based on real world load) benchmark.
> >  
> > So, is this a shout-out: these parts should be coordinated better?
> 
> Switching to instantaneous load along with the cpu reservation hackery
> made Chris's benchmark a happy camper.  Is that the answer?  Nope, just
> verification of the where the problem lives.

By cpu reservation, you mean the various averages in select_task_rq_fair?
It does seem a lot of cleanup should be done.
 
> > > > En... should we try remove recording last_wakee?
> > > 
> > > The more the merrier, go for it! :)
> >  
> > Nuh, really, this heuristic is too heuristic, :) 
> > The totality of all possible cases is scary.
> 
> Well, make it better.  The author provided evidence when it was born.

I have to think this through, hot-potato. Maybe even droping it does not
sound outrageous.

^ permalink raw reply	[flat|nested] 80+ messages in thread

* Re: sched: tweak select_idle_sibling to look for idle threads
  2016-05-09 23:26                           ` Yuyang Du
@ 2016-05-10  7:49                             ` Mike Galbraith
  2016-05-10 15:26                               ` Mike Galbraith
  0 siblings, 1 reply; 80+ messages in thread
From: Mike Galbraith @ 2016-05-10  7:49 UTC (permalink / raw)
  To: Yuyang Du
  Cc: Peter Zijlstra, Chris Mason, Ingo Molnar, Matt Fleming, linux-kernel

On Tue, 2016-05-10 at 07:26 +0800, Yuyang Du wrote:

> By cpu reservation, you mean the various averages in select_task_rq_fair?
> It does seem a lot of cleanup should be done.

Nah, I meant claiming an idle cpu with cmpxchg().  It's mostly the
average load business that leads to premature stacking though, the
reservation thingy more or less just wastes cycles.  Only whacking
cfs_rq_runnable_load_avg() with a rock makes schbench -m <sockets> -t
<near socket size> -a work well.  'Course a rock in its gearbox also
rendered load balancing fairly busted for the general case :)

	-Mike

^ permalink raw reply	[flat|nested] 80+ messages in thread

* Re: sched: tweak select_idle_sibling to look for idle threads
  2016-05-10  7:49                             ` Mike Galbraith
@ 2016-05-10 15:26                               ` Mike Galbraith
  2016-05-10 19:16                                 ` Yuyang Du
  0 siblings, 1 reply; 80+ messages in thread
From: Mike Galbraith @ 2016-05-10 15:26 UTC (permalink / raw)
  To: Yuyang Du
  Cc: Peter Zijlstra, Chris Mason, Ingo Molnar, Matt Fleming, linux-kernel

On Tue, 2016-05-10 at 09:49 +0200, Mike Galbraith wrote:

>  Only whacking
> cfs_rq_runnable_load_avg() with a rock makes schbench -m <sockets> -t
> <near socket size> -a work well.  'Course a rock in its gearbox also
> rendered load balancing fairly busted for the general case :)

Smaller rock doesn't injure heavy tbench, but more importantly, still
demonstrates the issue when you want full spread.

schbench -m4 -t38 -a

cputime 30000 threads 38 p99 177
cputime 30000 threads 39 p99 10160

LB_TIP_AVG_HIGH
cputime 30000 threads 38 p99 193
cputime 30000 threads 39 p99 184
cputime 30000 threads 40 p99 203
cputime 30000 threads 41 p99 202
cputime 30000 threads 42 p99 205
cputime 30000 threads 43 p99 218
cputime 30000 threads 44 p99 237
cputime 30000 threads 45 p99 245
cputime 30000 threads 46 p99 262
cputime 30000 threads 47 p99 296
cputime 30000 threads 48 p99 3308

47*4+4=nr_cpus yay

---
 kernel/sched/fair.c     |    3 +++
 kernel/sched/features.h |    1 +
 2 files changed, 4 insertions(+)

--- a/kernel/sched/fair.c
+++ b/kernel/sched/fair.c
@@ -3027,6 +3027,9 @@ void remove_entity_load_avg(struct sched
 
 static inline unsigned long cfs_rq_runnable_load_avg(struct cfs_rq *cfs_rq)
 {
+	if (sched_feat(LB_TIP_AVG_HIGH) && cfs_rq->load.weight > cfs_rq->runnable_load_avg*2)
+		return cfs_rq->runnable_load_avg + min_t(unsigned long, NICE_0_LOAD,
+							 cfs_rq->load.weight/2);
 	return cfs_rq->runnable_load_avg;
 }
 
--- a/kernel/sched/features.h
+++ b/kernel/sched/features.h
@@ -67,6 +67,7 @@ SCHED_FEAT(RT_PUSH_IPI, true)
 SCHED_FEAT(FORCE_SD_OVERLAP, false)
 SCHED_FEAT(RT_RUNTIME_SHARE, true)
 SCHED_FEAT(LB_MIN, false)
+SCHED_FEAT(LB_TIP_AVG_HIGH, false)
 SCHED_FEAT(ATTACH_AGE_LOAD, true)
 
 SCHED_FEAT(OLD_IDLE, false)

^ permalink raw reply	[flat|nested] 80+ messages in thread

* Re: sched: tweak select_idle_sibling to look for idle threads
  2016-05-10 15:26                               ` Mike Galbraith
@ 2016-05-10 19:16                                 ` Yuyang Du
  2016-05-11  4:17                                   ` Mike Galbraith
  2016-05-18  6:41                                   ` Mike Galbraith
  0 siblings, 2 replies; 80+ messages in thread
From: Yuyang Du @ 2016-05-10 19:16 UTC (permalink / raw)
  To: Mike Galbraith
  Cc: Peter Zijlstra, Chris Mason, Ingo Molnar, Matt Fleming, linux-kernel

On Tue, May 10, 2016 at 05:26:05PM +0200, Mike Galbraith wrote:
> On Tue, 2016-05-10 at 09:49 +0200, Mike Galbraith wrote:
> 
> >  Only whacking
> > cfs_rq_runnable_load_avg() with a rock makes schbench -m <sockets> -t
> > <near socket size> -a work well.  'Course a rock in its gearbox also
> > rendered load balancing fairly busted for the general case :)
> 
> Smaller rock doesn't injure heavy tbench, but more importantly, still
> demonstrates the issue when you want full spread.
> 
> schbench -m4 -t38 -a
> 
> cputime 30000 threads 38 p99 177
> cputime 30000 threads 39 p99 10160
> 
> LB_TIP_AVG_HIGH
> cputime 30000 threads 38 p99 193
> cputime 30000 threads 39 p99 184
> cputime 30000 threads 40 p99 203
> cputime 30000 threads 41 p99 202
> cputime 30000 threads 42 p99 205
> cputime 30000 threads 43 p99 218
> cputime 30000 threads 44 p99 237
> cputime 30000 threads 45 p99 245
> cputime 30000 threads 46 p99 262
> cputime 30000 threads 47 p99 296
> cputime 30000 threads 48 p99 3308
> 
> 47*4+4=nr_cpus yay
 
yay... and haha, "a perfect world"...

> ---
>  kernel/sched/fair.c     |    3 +++
>  kernel/sched/features.h |    1 +
>  2 files changed, 4 insertions(+)
> 
> --- a/kernel/sched/fair.c
> +++ b/kernel/sched/fair.c
> @@ -3027,6 +3027,9 @@ void remove_entity_load_avg(struct sched
>  
>  static inline unsigned long cfs_rq_runnable_load_avg(struct cfs_rq *cfs_rq)
>  {
> +	if (sched_feat(LB_TIP_AVG_HIGH) && cfs_rq->load.weight > cfs_rq->runnable_load_avg*2)
> +		return cfs_rq->runnable_load_avg + min_t(unsigned long, NICE_0_LOAD,
> +							 cfs_rq->load.weight/2);
>  	return cfs_rq->runnable_load_avg;
>  }
  
cfs_rq->runnable_load_avg is for sure no greater than (in this case much less
than, maybe 1/2 of) load.weight, whereas load_avg is not necessarily a rock
in gearbox that only impedes speed up, but also speed down.

But I really don't know the load references in select_task_rq() should be
what kind. So maybe the real issue is a mix of them, i.e., conflated balancing
and just wanting an idle cpu. ?

^ permalink raw reply	[flat|nested] 80+ messages in thread

* Re: sched: tweak select_idle_sibling to look for idle threads
  2016-05-11  4:17                                   ` Mike Galbraith
@ 2016-05-11  1:23                                     ` Yuyang Du
  2016-05-11  9:56                                       ` Mike Galbraith
  0 siblings, 1 reply; 80+ messages in thread
From: Yuyang Du @ 2016-05-11  1:23 UTC (permalink / raw)
  To: Mike Galbraith
  Cc: Peter Zijlstra, Chris Mason, Ingo Molnar, Matt Fleming, linux-kernel

On Wed, May 11, 2016 at 06:17:51AM +0200, Mike Galbraith wrote:
> > >  static inline unsigned long cfs_rq_runnable_load_avg(struct cfs_rq *cfs_rq)
> > >  {
> > > +> > 	> > if (sched_feat(LB_TIP_AVG_HIGH) && cfs_rq->load.weight > cfs_rq->runnable_load_avg*2)
> > > +> > 	> > 	> > return cfs_rq->runnable_load_avg + min_t(unsigned long, NICE_0_LOAD,
> > > +> > 	> > 	> > 	> > 	> > 	> > 	> > 	> >  cfs_rq->load.weight/2);
> > >  > > 	> > return cfs_rq->runnable_load_avg;
> > >  }
> >   
> > cfs_rq->runnable_load_avg is for sure no greater than (in this case much less
> > than, maybe 1/2 of) load.weight, whereas load_avg is not necessarily a rock
> > in gearbox that only impedes speed up, but also speed down.
> 
> Yeah, just like everything else, it'll cuts both ways (why you can't
> win the sched game).  If I can believe tbench, at tasks=cpus, reducing
> lag increased utilization and reduced latency a wee bit, as did the
> reserve thing once a booboo got fixed up.

Ok, so you have a secret IDLE_RESERVE? Good luck and show it, ;)

> Makes sense, robbing Peter
> to pay Paul should work out better for Paul.
> 
> NO_LB_TIP_AVG_HIGH
> Throughput 27132.9 MB/sec  96 clients  96 procs  max_latency=7.656 ms
> Throughput 28464.1 MB/sec  96 clients  96 procs  max_latency=9.905 ms
> Throughput 25369.8 MB/sec  96 clients  96 procs  max_latency=7.192 ms
> Throughput 25670.3 MB/sec  96 clients  96 procs  max_latency=5.874 ms
> Throughput 29309.3 MB/sec  96 clients  96 procs  max_latency=1.331 ms
> avg        27189   1.000                                     6.391   1.000
> 
> NO_LB_TIP_AVG_HIGH IDLE_RESERVE
> Throughput 24437.5 MB/sec  96 clients  96 procs  max_latency=1.837 ms
> Throughput 29464.7 MB/sec  96 clients  96 procs  max_latency=1.594 ms
> Throughput 28023.6 MB/sec  96 clients  96 procs  max_latency=1.494 ms
> Throughput 28299.0 MB/sec  96 clients  96 procs  max_latency=10.404 ms
> Throughput 29072.1 MB/sec  96 clients  96 procs  max_latency=5.575 ms
> avg        27859   1.024                                     4.180   0.654
> 
> LB_TIP_AVG_HIGH NO_IDLE_RESERVE
> Throughput 29068.1 MB/sec  96 clients  96 procs  max_latency=5.599 ms
> Throughput 26435.6 MB/sec  96 clients  96 procs  max_latency=3.703 ms
> Throughput 23930.0 MB/sec  96 clients  96 procs  max_latency=7.742 ms
> Throughput 29464.2 MB/sec  96 clients  96 procs  max_latency=1.549 ms
> Throughput 24250.9 MB/sec  96 clients  96 procs  max_latency=1.518 ms
> avg        26629   0.979                                     4.022   0.629
> 
> LB_TIP_AVG_HIGH IDLE_RESERVE
> Throughput 30340.1 MB/sec  96 clients  96 procs  max_latency=1.465 ms
> Throughput 29042.9 MB/sec  96 clients  96 procs  max_latency=4.515 ms
> Throughput 26718.7 MB/sec  96 clients  96 procs  max_latency=1.822 ms
> Throughput 28694.4 MB/sec  96 clients  96 procs  max_latency=1.503 ms
> Throughput 28918.2 MB/sec  96 clients  96 procs  max_latency=7.599 ms
> avg        28742   1.057                                     3.380   0.528
> 
> > But I really don't know the load references in select_task_rq() should be
> > what kind. So maybe the real issue is a mix of them, i.e., conflated balancing
> > and just wanting an idle cpu. ?
> 
> Depends on the goal.  For both, load lagging reality means the high
> frequency component is squelched, meaning less migration cost, but also
> higher latency due to stacking.  It's a tradeoff where Chris' latency
> is everything" benchmark, and _maybe_ the real world load it's based
> upon is on Peter's end of the rob Peter to pay Paul transaction.  The
> benchmark says it definitely is, the real world load may have already
> been fixed up by the select_idle_sibling() rewrite.
 
Obviously, load avgs are good at balancing in a larger scale in a timeframe,
so they should be used in comparing/balancing sd's not cpus. However, this
is not the case currently: avgs are mixed with idle cpu/core selection, so
I think better job can be done before and after select_idle_sibling().

For example, I don't know what the complex wake_affine() is really doing for
what. Am i missing something, you think?

Kudos to select_idle_sibling() rewrite, like Peter said, a second step and
an even third step scans are really helping, in addition to many cleanups
and refactors.

^ permalink raw reply	[flat|nested] 80+ messages in thread

* Re: sched: tweak select_idle_sibling to look for idle threads
  2016-05-10 19:16                                 ` Yuyang Du
@ 2016-05-11  4:17                                   ` Mike Galbraith
  2016-05-11  1:23                                     ` Yuyang Du
  2016-05-18  6:41                                   ` Mike Galbraith
  1 sibling, 1 reply; 80+ messages in thread
From: Mike Galbraith @ 2016-05-11  4:17 UTC (permalink / raw)
  To: Yuyang Du
  Cc: Peter Zijlstra, Chris Mason, Ingo Molnar, Matt Fleming, linux-kernel

On Wed, 2016-05-11 at 03:16 +0800, Yuyang Du wrote:
> On Tue, May 10, 2016 at 05:26:05PM +0200, Mike Galbraith wrote:
> > On Tue, 2016-05-10 at 09:49 +0200, Mike Galbraith wrote:
> > 
> > >  Only whacking
> > > cfs_rq_runnable_load_avg() with a rock makes schbench -m  -t
> > >  -a work well.  'Course a rock in its gearbox also
> > > rendered load balancing fairly busted for the general case :)
> > 
> > Smaller rock doesn't injure heavy tbench, but more importantly, still
> > demonstrates the issue when you want full spread.
> > 
> > schbench -m4 -t38 -a
> > 
> > cputime 30000 threads 38 p99 177
> > cputime 30000 threads 39 p99 10160
> > 
> > LB_TIP_AVG_HIGH
> > cputime 30000 threads 38 p99 193
> > cputime 30000 threads 39 p99 184
> > cputime 30000 threads 40 p99 203
> > cputime 30000 threads 41 p99 202
> > cputime 30000 threads 42 p99 205
> > cputime 30000 threads 43 p99 218
> > cputime 30000 threads 44 p99 237
> > cputime 30000 threads 45 p99 245
> > cputime 30000 threads 46 p99 262
> > cputime 30000 threads 47 p99 296
> > cputime 30000 threads 48 p99 3308
> > 
> > 47*4+4=nr_cpus yay
>  
> yay... and haha, "a perfect world"...

Yup.. for this load.

> > ---
> >  kernel/sched/fair.c     |    3 +++
> >  kernel/sched/features.h |    1 +
> >  2 files changed, 4 insertions(+)
> > 
> > --- a/kernel/sched/fair.c
> > +++ b/kernel/sched/fair.c
> > @@ -3027,6 +3027,9 @@ void remove_entity_load_avg(struct sched
> >  
> >  static inline unsigned long cfs_rq_runnable_load_avg(struct cfs_rq *cfs_rq)
> >  {
> > +> > 	> > if (sched_feat(LB_TIP_AVG_HIGH) && cfs_rq->load.weight > cfs_rq->runnable_load_avg*2)
> > +> > 	> > 	> > return cfs_rq->runnable_load_avg + min_t(unsigned long, NICE_0_LOAD,
> > +> > 	> > 	> > 	> > 	> > 	> > 	> > 	> >  cfs_rq->load.weight/2);
> >  > > 	> > return cfs_rq->runnable_load_avg;
> >  }
>   
> cfs_rq->runnable_load_avg is for sure no greater than (in this case much less
> than, maybe 1/2 of) load.weight, whereas load_avg is not necessarily a rock
> in gearbox that only impedes speed up, but also speed down.

Yeah, just like everything else, it'll cuts both ways (why you can't
win the sched game).  If I can believe tbench, at tasks=cpus, reducing
lag increased utilization and reduced latency a wee bit, as did the
reserve thing once a booboo got fixed up.  Makes sense, robbing Peter
to pay Paul should work out better for Paul.

NO_LB_TIP_AVG_HIGH
Throughput 27132.9 MB/sec  96 clients  96 procs  max_latency=7.656 ms
Throughput 28464.1 MB/sec  96 clients  96 procs  max_latency=9.905 ms
Throughput 25369.8 MB/sec  96 clients  96 procs  max_latency=7.192 ms
Throughput 25670.3 MB/sec  96 clients  96 procs  max_latency=5.874 ms
Throughput 29309.3 MB/sec  96 clients  96 procs  max_latency=1.331 ms
avg        27189   1.000                                     6.391   1.000

NO_LB_TIP_AVG_HIGH IDLE_RESERVE
Throughput 24437.5 MB/sec  96 clients  96 procs  max_latency=1.837 ms
Throughput 29464.7 MB/sec  96 clients  96 procs  max_latency=1.594 ms
Throughput 28023.6 MB/sec  96 clients  96 procs  max_latency=1.494 ms
Throughput 28299.0 MB/sec  96 clients  96 procs  max_latency=10.404 ms
Throughput 29072.1 MB/sec  96 clients  96 procs  max_latency=5.575 ms
avg        27859   1.024                                     4.180   0.654

LB_TIP_AVG_HIGH NO_IDLE_RESERVE
Throughput 29068.1 MB/sec  96 clients  96 procs  max_latency=5.599 ms
Throughput 26435.6 MB/sec  96 clients  96 procs  max_latency=3.703 ms
Throughput 23930.0 MB/sec  96 clients  96 procs  max_latency=7.742 ms
Throughput 29464.2 MB/sec  96 clients  96 procs  max_latency=1.549 ms
Throughput 24250.9 MB/sec  96 clients  96 procs  max_latency=1.518 ms
avg        26629   0.979                                     4.022   0.629

LB_TIP_AVG_HIGH IDLE_RESERVE
Throughput 30340.1 MB/sec  96 clients  96 procs  max_latency=1.465 ms
Throughput 29042.9 MB/sec  96 clients  96 procs  max_latency=4.515 ms
Throughput 26718.7 MB/sec  96 clients  96 procs  max_latency=1.822 ms
Throughput 28694.4 MB/sec  96 clients  96 procs  max_latency=1.503 ms
Throughput 28918.2 MB/sec  96 clients  96 procs  max_latency=7.599 ms
avg        28742   1.057                                     3.380   0.528

> But I really don't know the load references in select_task_rq() should be
> what kind. So maybe the real issue is a mix of them, i.e., conflated balancing
> and just wanting an idle cpu. ?

Depends on the goal.  For both, load lagging reality means the high
frequency component is squelched, meaning less migration cost, but also
higher latency due to stacking.  It's a tradeoff where Chris' latency
is everything" benchmark, and _maybe_ the real world load it's based
upon is on Peter's end of the rob Peter to pay Paul transaction.  The
benchmark says it definitely is, the real world load may have already
been fixed up by the select_idle_sibling() rewrite.

	-Mike

^ permalink raw reply	[flat|nested] 80+ messages in thread

* Re: sched: tweak select_idle_sibling to look for idle threads
  2016-05-11  1:23                                     ` Yuyang Du
@ 2016-05-11  9:56                                       ` Mike Galbraith
  0 siblings, 0 replies; 80+ messages in thread
From: Mike Galbraith @ 2016-05-11  9:56 UTC (permalink / raw)
  To: Yuyang Du
  Cc: Peter Zijlstra, Chris Mason, Ingo Molnar, Matt Fleming, linux-kernel

On Wed, 2016-05-11 at 09:23 +0800, Yuyang Du wrote:

> > Yeah, just like everything else, it'll cuts both ways (why you can't
> > win the sched game).  If I can believe tbench, at tasks=cpus, reducing
> > lag increased utilization and reduced latency a wee bit, as did the
> > reserve thing once a booboo got fixed up.
> 
> Ok, so you have a secret IDLE_RESERVE? Good luck and show it, ;)

Nothing sexy, just cpmxchg(), with the obvious test/set/clear spots.

cmpxchg(&cpu_rq(cpu)->idle_latch, cpu, nr_cpu_ids)

> Depends on the goal.  For both, load lagging reality means the high
> > frequency component is squelched, meaning less migration cost, but also
> > higher latency due to stacking.  It's a tradeoff where Chris' latency
> > is everything" benchmark, and _maybe_ the real world load it's based
> > upon is on Peter's end of the rob Peter to pay Paul transaction.  The
> > benchmark says it definitely is, the real world load may have already
> > been fixed up by the select_idle_sibling() rewrite.
>  
> Obviously, load avgs are good at balancing in a larger scale in a timeframe,
> so they should be used in comparing/balancing sd's not cpus. However, this
> is not the case currently: avgs are mixed with idle cpu/core selection, so
> I think better job can be done before and after select_idle_sibling().
> 
> For example, I don't know what the complex wake_affine() is really doing for
> what. Am i missing something, you think?

wake_affine() just says no to keep us from pulling the whole load to
one cache, starting massive tug-o-war with LB and nuking throughput. 
 Everybody wants hot data, but they can't all have it and scale.

	-Mike

^ permalink raw reply	[flat|nested] 80+ messages in thread

* Re: sched: tweak select_idle_sibling to look for idle threads
  2016-05-10 19:16                                 ` Yuyang Du
  2016-05-11  4:17                                   ` Mike Galbraith
@ 2016-05-18  6:41                                   ` Mike Galbraith
  1 sibling, 0 replies; 80+ messages in thread
From: Mike Galbraith @ 2016-05-18  6:41 UTC (permalink / raw)
  To: Yuyang Du
  Cc: Peter Zijlstra, Chris Mason, Ingo Molnar, Matt Fleming, linux-kernel

On Wed, 2016-05-11 at 03:16 +0800, Yuyang Du wrote:

> > --- a/kernel/sched/fair.c
> > +++ b/kernel/sched/fair.c
> > @@ -3027,6 +3027,9 @@ void remove_entity_load_avg(struct sched
> >  
> >  static inline unsigned long cfs_rq_runnable_load_avg(struct cfs_rq *cfs_rq)
> >  {
> > +> > 	> > if (sched_feat(LB_TIP_AVG_HIGH) && cfs_rq->load.weight > cfs_rq->runnable_load_avg*2)
> > +> > 	> > 	> > return cfs_rq->runnable_load_avg + min_t(unsigned long, NICE_0_LOAD,
> > +> > 	> > 	> > 	> > 	> > 	> > 	> > 	> >  cfs_rq->load.weight/2);
> >  > > 	> > return cfs_rq->runnable_load_avg;
> >  }
>   
> cfs_rq->runnable_load_avg is for sure no greater than (in this case much less
> than, maybe 1/2 of) load.weight, whereas load_avg is not necessarily a rock
> in gearbox that only impedes speed up, but also speed down.

BTW, the reason hack helped is that the long (30ms) sleep/run cycle of
the benchmark's default settings causes large amplitude sawtooth of
load numbers (~300 - ~700 range), dinging up load delta resolvability.

	-Mike

^ permalink raw reply	[flat|nested] 80+ messages in thread

end of thread, other threads:[~2016-05-18  6:41 UTC | newest]

Thread overview: 80+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2016-04-05 18:08 [PATCH RFC] select_idle_sibling experiments Chris Mason
2016-04-05 18:43 ` Bastien Bastien Philbert
2016-04-05 19:28   ` Chris Mason
2016-04-05 20:03 ` Matt Fleming
2016-04-05 21:05   ` Bastien Philbert
2016-04-06  0:44   ` Chris Mason
2016-04-06  7:27 ` Mike Galbraith
2016-04-06 13:36   ` Chris Mason
2016-04-09 17:30   ` Chris Mason
2016-04-12 21:45     ` Matt Fleming
2016-04-13  3:40       ` Mike Galbraith
2016-04-13 15:54         ` Chris Mason
2016-04-28 12:00   ` Peter Zijlstra
2016-04-28 13:17     ` Mike Galbraith
2016-05-02  5:35     ` Mike Galbraith
2016-04-07 15:17 ` Chris Mason
2016-04-09 19:05 ` sched: tweak select_idle_sibling to look for idle threads Chris Mason
2016-04-10 10:04   ` Mike Galbraith
2016-04-10 12:35     ` Chris Mason
2016-04-10 12:46       ` Mike Galbraith
2016-04-10 19:55     ` Chris Mason
2016-04-11  4:54       ` Mike Galbraith
2016-04-12  0:30         ` Chris Mason
2016-04-12  4:44           ` Mike Galbraith
2016-04-12 13:27             ` Chris Mason
2016-04-12 18:16               ` Mike Galbraith
2016-04-12 20:07                 ` Chris Mason
2016-04-13  3:18                   ` Mike Galbraith
2016-04-13 13:44                     ` Chris Mason
2016-04-13 14:22                       ` Mike Galbraith
2016-04-13 14:36                         ` Chris Mason
2016-04-13 15:05                           ` Mike Galbraith
2016-04-13 15:34                             ` Mike Galbraith
2016-04-30 12:47   ` Peter Zijlstra
2016-05-01  7:12     ` Mike Galbraith
2016-05-01  8:53       ` Peter Zijlstra
2016-05-01  9:20         ` Mike Galbraith
2016-05-07  1:24           ` Yuyang Du
2016-05-08  8:08             ` Mike Galbraith
2016-05-08 18:57               ` Yuyang Du
2016-05-09  3:45                 ` Mike Galbraith
2016-05-08 20:22                   ` Yuyang Du
2016-05-09  7:44                     ` Mike Galbraith
2016-05-09  1:13                       ` Yuyang Du
2016-05-09  9:39                         ` Mike Galbraith
2016-05-09 23:26                           ` Yuyang Du
2016-05-10  7:49                             ` Mike Galbraith
2016-05-10 15:26                               ` Mike Galbraith
2016-05-10 19:16                                 ` Yuyang Du
2016-05-11  4:17                                   ` Mike Galbraith
2016-05-11  1:23                                     ` Yuyang Du
2016-05-11  9:56                                       ` Mike Galbraith
2016-05-18  6:41                                   ` Mike Galbraith
2016-05-09  3:52                 ` Mike Galbraith
2016-05-08 20:31                   ` Yuyang Du
2016-05-02  8:46       ` Peter Zijlstra
2016-05-02 14:50         ` Mike Galbraith
2016-05-02 14:58           ` Peter Zijlstra
2016-05-02 15:47             ` Chris Mason
2016-05-03 14:32               ` Peter Zijlstra
2016-05-03 15:11                 ` Chris Mason
2016-05-04 10:37                   ` Peter Zijlstra
2016-05-04 15:31                     ` Peter Zijlstra
2016-05-05 22:03                     ` Matt Fleming
2016-05-06 18:54                       ` Mike Galbraith
2016-05-09  8:33                         ` Peter Zijlstra
2016-05-09  8:56                           ` Mike Galbraith
2016-05-04 15:45                   ` Peter Zijlstra
2016-05-04 17:46                     ` Chris Mason
2016-05-05  9:33                       ` Peter Zijlstra
2016-05-05 13:58                         ` Chris Mason
2016-05-06  7:12                           ` Peter Zijlstra
2016-05-06 17:27                             ` Chris Mason
2016-05-06  7:25                   ` Peter Zijlstra
2016-05-02 17:30             ` Mike Galbraith
2016-05-02 15:01           ` Peter Zijlstra
2016-05-02 16:04             ` Ingo Molnar
2016-05-03 11:31               ` Peter Zijlstra
2016-05-03 18:22                 ` Peter Zijlstra
2016-05-02 15:10           ` Peter Zijlstra

This is a public inbox, see mirroring instructions
for how to clone and mirror all data and code used for this inbox;
as well as URLs for NNTP newsgroup(s).