All of lore.kernel.org
 help / color / mirror / Atom feed
* [RFC v2 net-next 00/10] Time based packet transmission
@ 2018-01-17 23:06 ` Jesus Sanchez-Palencia
  0 siblings, 0 replies; 86+ messages in thread
From: Jesus Sanchez-Palencia @ 2018-01-17 23:06 UTC (permalink / raw)
  To: netdev
  Cc: jhs, xiyou.wangcong, jiri, vinicius.gomes, richardcochran,
	intel-wired-lan, anna-maria, henrik, tglx, john.stultz,
	andre.guedes, ivan.briano, levi.pearson

This series is the v2 of the Time based packet transmission RFC, which was
originally proposed by Richard Cochran: https://lwn.net/Articles/733962/ .

It introduces a new socket option (SO_TXTIME), a new qdisc (tbs) and implements
support for hw offloading on the igb driver for the Intel i210 NIC. The tbs
qdisc also supports SW best effort that can be used as a fallback.

The main changes since v1 are:
 - the tstamp field from sk_buffs is now used;
 - ktime_t is the type used for the field added to struct sockcm_cookie instead
   of u64;
 - the tbs qdisc is introduced with SW best effort and hw offloading;
 - the igb implementation for HW offloading was re-written, allowing both tbs
   and cbs qdiscs to co-exist with proper driver support.

The tbs qdisc is designed so it buffers packets until a configurable time before
their deadline (tx times). It uses a rbtree internally, thus the buffered
packets are always 'ordered' by the earliest deadline.

The other configurable parameter from the tbs qdisc is the clockid to be used.
In order to provide that, this series adds a new API to pkt_sched.h (i.e.
qdisc_watchdog_init_clockid()).

As an usage example:

    $ tc qdisc replace dev enp2s0 parent root handle 100 mqprio num_tc 3 \
               map 2 2 1 0 2 2 2 2 2 2 2 2 2 2 2 2 queues 1@0 1@1 2@2 hw 0

    $ tc qdisc add dev enp2s0 parent 100:1 tbs delta 60000 clockid 11 \
               offload 1

In this example first the mqprio qdisc is setup, then the tbs qdisc is
configured onto the first hw Tx queue and tries to enable HW offloading (i.e.
offload 1). Also, it is configured so the timestamps on each packet are in
reference to the clockid '11' (CLOCK_TAI) and so packets are dequeued from
the qdisc 60000 nanoseconds before their transmission time.

The tbs qdisc will drop any packets with a transmission time in the past or
when a deadline is missed. Queueing packets in advance plus configuring the
delta parameter for the system correctly makes all the difference in reducing
the number of drops. Moreover, note that the delta parameter ends up defining
the Tx time when SW best effort is used given that the timestamps won't be used
by the NIC on this case.

For testing, we've followed a similar approach from the v1 testing:
  1. Prepared a PC and the Device Under Test (DUT) each with an Intel
     i210 card connected.
  2. The DUT was a Intel(R) Core(TM) i5-7600 CPU @ 3.50GHz running on top of
     kernel 4.15.0-rc8+ with about 50 usec maximum latency under cyclictest.
  3. Synchronized the DUT's PHC to the PC's PHC using ptp4l.
  4. Synchronized the DUT's system clock to its PHC using phc2sys.
  5. Measured the arrival time of the packets at the PC's PHC using
     hardware time stamping.

First, a baseline test was ran for 10 minutes with the plain kernel only:

|                 | plain kernel @ 1ms |
|-----------------+--------------------+
| min (ns):       |    +4.820000e+02   |
| max (ns):       |    +9.999300e+05   |
| pk-pk:          |    +9.994480e+05   |
| mean (ns):      |    +3.464421e+04   |
| stddev:         |    +1.305947e+05   |
| count:          |           600000   |


Tests were then ran for 10 minutes with a period of 1 millisecond using both
SW best effort and HW offloading. For last, we repeated the HW offloading test
with a 250 microsecond period. The measured offset from the expected period is
shown below, plus the tbs delta parameter that was used in each case.

|                 |    tbs SW @ 1ms   |  tbs HW @ 1ms  | tbs HW @ 250 us |
|-----------------+-------------------+----------------+-----------------|
| min (ns):       |    +1.510000e+02  |  +4.420000e+02 |   +4.260000e+02 |
| max (ns):       |    +9.977030e+05  |  +5.060000e+02 |   +5.060000e+02 |
| pk-pk:          |    +9.975520e+05  |  +6.400000e+01 |   +8.000000e+01 |
| mean (ns):      |    +1.416511e+04  |  +4.687228e+02 |   +4.600596e+02 |
| stddev:         |    +5.750639e+04  |  +9.868569e+00 |   +1.287626e+01 |
| count:          |           600000  |         600000 |         2400000 |
| dropped:        |                3  |              0 |               0 |
| tbs delta (ns): |           130000  |         130000 |          130000 |


The code used for testing is appended below. The wake_tx parameter (-d) used
for all tests was 600000 ns and the priority parameter was 90 (-p). The
baseline test (plain kernel) used a wake_tx parameter of 130000 ns.

Our main questions at this stage are related to the qdisc:
 - does the proposed design attend all use cases?
 - should the qdisc really drop packets that expired after being queued even
   for the SW best effort mode?
 - once one expired packet is found and dropped during a dequeue, should we
   traverse the rbtree and drop other expired packets if any, or should we
   keep deferring that to the next dequeue call?

For last, most of the To Dos we still have before a final patchset are related
to further testing the igb support:
 - testing with L2 only talkers + AF_PACKET sockets;
 - testing tbs in conjunction with cbs;

Thanks,
Jesus


Jesus Sanchez-Palencia (4):
  igb: Refactor igb_configure_cbs()
  igb: Only change Tx arbitration when CBS is on
  igb: Refactor igb_offload_cbs()
  igb: Add support for TBS offload

Richard Cochran (4):
  net: Add a new socket option for a future transmit time.
  net: ipv4: raw: Hook into time based transmission.
  net: ipv4: udp: Hook into time based transmission.
  net: packet: Hook into time based transmission.

Vinicius Costa Gomes (2):
  net/sched: Allow creating a Qdisc watchdog with other clocks
  net/sched: Introduce the TBS Qdisc

 arch/alpha/include/uapi/asm/socket.h           |   3 +
 arch/frv/include/uapi/asm/socket.h             |   3 +
 arch/ia64/include/uapi/asm/socket.h            |   3 +
 arch/m32r/include/uapi/asm/socket.h            |   3 +
 arch/mips/include/uapi/asm/socket.h            |   3 +
 arch/mn10300/include/uapi/asm/socket.h         |   3 +
 arch/parisc/include/uapi/asm/socket.h          |   3 +
 arch/s390/include/uapi/asm/socket.h            |   3 +
 arch/sparc/include/uapi/asm/socket.h           |   3 +
 arch/xtensa/include/uapi/asm/socket.h          |   3 +
 drivers/net/ethernet/intel/igb/e1000_defines.h |  16 +
 drivers/net/ethernet/intel/igb/igb.h           |   1 +
 drivers/net/ethernet/intel/igb/igb_main.c      | 239 +++++++++++----
 include/linux/netdevice.h                      |   1 +
 include/net/pkt_sched.h                        |   7 +
 include/net/sock.h                             |   2 +
 include/uapi/asm-generic/socket.h              |   3 +
 include/uapi/linux/pkt_sched.h                 |  17 ++
 net/core/sock.c                                |  16 +
 net/ipv4/raw.c                                 |   2 +
 net/ipv4/udp.c                                 |   5 +-
 net/packet/af_packet.c                         |   6 +
 net/sched/Kconfig                              |  11 +
 net/sched/Makefile                             |   1 +
 net/sched/sch_api.c                            |  11 +-
 net/sched/sch_tbs.c                            | 392 +++++++++++++++++++++++++
 26 files changed, 699 insertions(+), 61 deletions(-)
 create mode 100644 net/sched/sch_tbs.c

-- 
2.15.1

---8<---
/*
 * This program demonstrates transmission of UDP packets using the
 * system TAI timer.
 *
 * Copyright (C) 2017 linutronix GmbH
 *
 * Large portions taken from the linuxptp stack.
 * Copyright (C) 2011, 2012 Richard Cochran <richardcochran@gmail.com>
 *
 * Some portions taken from the sgd test program.
 * Copyright (C) 2015 linutronix GmbH
 *
 * This program is free software; you can redistribute it and/or modify
 * it under the terms of the GNU General Public License as published by
 * the Free Software Foundation; either version 2 of the License, or
 * (at your option) any later version.
 *
 * This program is distributed in the hope that it will be useful,
 * but WITHOUT ANY WARRANTY; without even the implied warranty of
 * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE.  See the
 * GNU General Public License for more details.
 *
 * You should have received a copy of the GNU General Public License along
 * with this program; if not, write to the Free Software Foundation, Inc.,
 * 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301 USA.
 */
#define _GNU_SOURCE /*for CPU_SET*/
#include <arpa/inet.h>
#include <errno.h>
#include <fcntl.h>
#include <ifaddrs.h>
#include <linux/ethtool.h>
#include <linux/net_tstamp.h>
#include <linux/sockios.h>
#include <net/if.h>
#include <netinet/in.h>
#include <poll.h>
#include <pthread.h>
#include <sched.h>
#include <signal.h>
#include <stdio.h>
#include <stdlib.h>
#include <string.h>
#include <sys/ioctl.h>
#include <sys/socket.h>
#include <sys/stat.h>
#include <sys/types.h>
#include <unistd.h>

#define DEFAULT_PERIOD	1000000
#define DEFAULT_DELAY	500000
#define MCAST_IPADDR	"239.1.1.1"
#define UDP_PORT	7788

#ifndef SO_TXTIME
#define SO_TXTIME	61
#endif

#define pr_err(s)	fprintf(stderr, s "\n")
#define pr_info(s)	fprintf(stdout, s "\n")

static int running = 1, use_so_txtime = 1;
static int period_nsec = DEFAULT_PERIOD;
static int waketx_delay = DEFAULT_DELAY;
static struct in_addr mcast_addr;

static int mcast_bind(int fd, int index)
{
	int err;
	struct ip_mreqn req;
	memset(&req, 0, sizeof(req));
	req.imr_ifindex = index;
	err = setsockopt(fd, IPPROTO_IP, IP_MULTICAST_IF, &req, sizeof(req));
	if (err) {
		pr_err("setsockopt IP_MULTICAST_IF failed: %m");
		return -1;
	}
	return 0;
}

static int mcast_join(int fd, int index, const struct sockaddr *grp,
		      socklen_t grplen)
{
	int err, off = 0;
	struct ip_mreqn req;
	struct sockaddr_in *sa = (struct sockaddr_in *) grp;

	memset(&req, 0, sizeof(req));
	memcpy(&req.imr_multiaddr, &sa->sin_addr, sizeof(struct in_addr));
	req.imr_ifindex = index;
	err = setsockopt(fd, IPPROTO_IP, IP_ADD_MEMBERSHIP, &req, sizeof(req));
	if (err) {
		pr_err("setsockopt IP_ADD_MEMBERSHIP failed: %m");
		return -1;
	}
	err = setsockopt(fd, IPPROTO_IP, IP_MULTICAST_LOOP, &off, sizeof(off));
	if (err) {
		pr_err("setsockopt IP_MULTICAST_LOOP failed: %m");
		return -1;
	}
	return 0;
}

static void normalize(struct timespec *ts)
{
	while (ts->tv_nsec > 999999999) {
		ts->tv_sec += 1;
		ts->tv_nsec -= 1000000000;
	}
}

static int sk_interface_index(int fd, const char *name)
{
	struct ifreq ifreq;
	int err;

	memset(&ifreq, 0, sizeof(ifreq));
	strncpy(ifreq.ifr_name, name, sizeof(ifreq.ifr_name) - 1);
	err = ioctl(fd, SIOCGIFINDEX, &ifreq);
	if (err < 0) {
		pr_err("ioctl SIOCGIFINDEX failed: %m");
		return err;
	}
	return ifreq.ifr_ifindex;
}

static int open_socket(const char *name, struct in_addr mc_addr, short port)
{
	struct sockaddr_in addr;
	int fd, index, on = 1;
	int priority = 3;

	memset(&addr, 0, sizeof(addr));
	addr.sin_family = AF_INET;
	addr.sin_addr.s_addr = htonl(INADDR_ANY);
	addr.sin_port = htons(port);

	fd = socket(PF_INET, SOCK_DGRAM, IPPROTO_UDP);
	if (fd < 0) {
		pr_err("socket failed: %m");
		goto no_socket;
	}
	index = sk_interface_index(fd, name);
	if (index < 0)
		goto no_option;

	if (setsockopt(fd, SOL_SOCKET, SO_PRIORITY, &priority, sizeof(priority))) {
		pr_err("Couldn't set priority");
		goto no_option;
	}
	if (setsockopt(fd, SOL_SOCKET, SO_REUSEADDR, &on, sizeof(on))) {
		pr_err("setsockopt SO_REUSEADDR failed: %m");
		goto no_option;
	}
	if (bind(fd, (struct sockaddr *) &addr, sizeof(addr))) {
		pr_err("bind failed: %m");
		goto no_option;
	}
	if (setsockopt(fd, SOL_SOCKET, SO_BINDTODEVICE, name, strlen(name))) {
		pr_err("setsockopt SO_BINDTODEVICE failed: %m");
		goto no_option;
	}
	addr.sin_addr = mc_addr;
	if (mcast_join(fd, index, (struct sockaddr *) &addr, sizeof(addr))) {
		pr_err("mcast_join failed");
		goto no_option;
	}
	if (mcast_bind(fd, index)) {
		goto no_option;
	}
	if (use_so_txtime && setsockopt(fd, SOL_SOCKET, SO_TXTIME, &on, sizeof(on))) {
		pr_err("setsockopt SO_TXTIME failed: %m");
		goto no_option;
	}

	return fd;
no_option:
	close(fd);
no_socket:
	return -1;
}

static int udp_open(const char *name)
{
	int fd;

	if (!inet_aton(MCAST_IPADDR, &mcast_addr))
		return -1;

	fd = open_socket(name, mcast_addr, UDP_PORT);

	return fd;
}

static int udp_send(int fd, void *buf, int len, __u64 txtime)
{
	union {
		char buf[CMSG_SPACE(sizeof(__u64))];
		struct cmsghdr align;
	} u;
	struct sockaddr_in sin;
	struct cmsghdr *cmsg;
	struct msghdr msg;
	struct iovec iov;
	ssize_t cnt;

	memset(&sin, 0, sizeof(sin));
	sin.sin_family = AF_INET;
	sin.sin_addr = mcast_addr;
	sin.sin_port = htons(UDP_PORT);

	iov.iov_base = buf;
	iov.iov_len = len;

	memset(&msg, 0, sizeof(msg));
	msg.msg_name = &sin;
	msg.msg_namelen = sizeof(sin);
	msg.msg_iov = &iov;
	msg.msg_iovlen = 1;

	/*
	 * We specify the transmission time in the CMSG.
	 */
	if (use_so_txtime) {
		msg.msg_control = u.buf;
		msg.msg_controllen = sizeof(u.buf);
		cmsg = CMSG_FIRSTHDR(&msg);
		cmsg->cmsg_level = SOL_SOCKET;
		cmsg->cmsg_type = SO_TXTIME;
		cmsg->cmsg_len = CMSG_LEN(sizeof(__u64));
		*((__u64 *) CMSG_DATA(cmsg)) = txtime;
	}
	cnt = sendmsg(fd, &msg, 0);
	if (cnt < 1) {
		pr_err("sendmsg failed: %m");
		return cnt;
	}
	return cnt;
}

static unsigned char tx_buffer[256];
static int marker;

static int run_nanosleep(clockid_t clkid, int fd)
{
	struct timespec ts;
	int cnt, err;
	__u64 txtime;

	clock_gettime(clkid, &ts);

	/* Start one to two seconds in the future. */
	ts.tv_sec += 1;
	ts.tv_nsec = 1000000000 - waketx_delay;
	normalize(&ts);

	txtime = ts.tv_sec * 1000000000ULL + ts.tv_nsec;
	txtime += waketx_delay;

	while (running) {
		err = clock_nanosleep(clkid, TIMER_ABSTIME, &ts, NULL);
		switch (err) {
		case 0:
			cnt = udp_send(fd, tx_buffer, sizeof(tx_buffer), txtime);
			if (cnt != sizeof(tx_buffer)) {
				pr_err("udp_send failed");
			}
			memset(tx_buffer, marker++, sizeof(tx_buffer));
			ts.tv_nsec += period_nsec;
			normalize(&ts);
			txtime += period_nsec;
			break;
		case EINTR:
			continue;
		default:
			fprintf(stderr, "clock_nanosleep returned %d: %s",
				err, strerror(err));
			return err;
		}
	}

	return 0;
}

static int set_realtime(pthread_t thread, int priority, int cpu)
{
	cpu_set_t cpuset;
	struct sched_param sp;
	int err, policy;

	int min = sched_get_priority_min(SCHED_FIFO);
	int max = sched_get_priority_max(SCHED_FIFO);

	fprintf(stderr, "min %d max %d\n", min, max);

	if (priority < 0) {
		return 0;
	}

	err = pthread_getschedparam(thread, &policy, &sp);
	if (err) {
		fprintf(stderr, "pthread_getschedparam: %s\n", strerror(err));
		return -1;
	}

	sp.sched_priority = priority;

	err = pthread_setschedparam(thread, SCHED_FIFO, &sp);
	if (err) {
		fprintf(stderr, "pthread_setschedparam: %s\n", strerror(err));
		return -1;
	}

	if (cpu < 0) {
		return 0;
	}
	CPU_ZERO(&cpuset);
	CPU_SET(cpu, &cpuset);
	err = pthread_setaffinity_np(thread, sizeof(cpu_set_t), &cpuset);
	if (err) {
		fprintf(stderr, "pthread_setaffinity_np: %s\n", strerror(err));
		return -1;
	}

	return 0;
}

static void usage(char *progname)
{
	fprintf(stderr,
		"\n"
		"usage: %s [options]\n"
		"\n"
		" -c [num]   run on CPU 'num'\n"
		" -d [num]   delay from wake up to transmission in nanoseconds (default %d)\n"
		" -h         prints this message and exits\n"
		" -i [name]  use network interface 'name'\n"
		" -p [num]   run with RT priorty 'num'\n"
		" -P [num]   period in nanoseconds (default %d)\n"
		" -u         do not use SO_TXTIME\n"
		"\n",
		progname, DEFAULT_DELAY, DEFAULT_PERIOD);
}

int main(int argc, char *argv[])
{
	int c, cpu = -1, err, fd, priority = -1;
	clockid_t clkid = CLOCK_TAI;
	char *iface = NULL, *progname;

	/* Process the command line arguments. */
	progname = strrchr(argv[0], '/');
	progname = progname ? 1 + progname : argv[0];
	while (EOF != (c = getopt(argc, argv, "c:d:hi:p:P:u"))) {
		switch (c) {
		case 'c':
			cpu = atoi(optarg);
			break;
		case 'd':
			waketx_delay = atoi(optarg);
			break;
		case 'h':
			usage(progname);
			return 0;
		case 'i':
			iface = optarg;
			break;
		case 'p':
			priority = atoi(optarg);
			break;
		case 'P':
			period_nsec = atoi(optarg);
			break;
		case 'u':
			use_so_txtime = 0;
			break;
		case '?':
			usage(progname);
			return -1;
		}
	}

	if (waketx_delay > 999999999 || waketx_delay < 0) {
		pr_err("Bad wake up to transmission delay.");
		usage(progname);
		return -1;
	}

	if (period_nsec < 1000) {
		pr_err("Bad period.");
		usage(progname);
		return -1;
	}

	if (!iface) {
		pr_err("Need a network interface.");
		usage(progname);
		return -1;
	}

	if (set_realtime(pthread_self(), priority, cpu)) {
		return -1;
	}

	fd = udp_open(iface);
	if (fd < 0) {
		return -1;
	}

	err = run_nanosleep(clkid, fd);

	close(fd);
	return err;
}

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

* [Intel-wired-lan] [RFC v2 net-next 00/10] Time based packet transmission
@ 2018-01-17 23:06 ` Jesus Sanchez-Palencia
  0 siblings, 0 replies; 86+ messages in thread
From: Jesus Sanchez-Palencia @ 2018-01-17 23:06 UTC (permalink / raw)
  To: intel-wired-lan

This series is the v2 of the Time based packet transmission RFC, which was
originally proposed by Richard Cochran: https://lwn.net/Articles/733962/ .

It introduces a new socket option (SO_TXTIME), a new qdisc (tbs) and implements
support for hw offloading on the igb driver for the Intel i210 NIC. The tbs
qdisc also supports SW best effort that can be used as a fallback.

The main changes since v1 are:
 - the tstamp field from sk_buffs is now used;
 - ktime_t is the type used for the field added to struct sockcm_cookie instead
   of u64;
 - the tbs qdisc is introduced with SW best effort and hw offloading;
 - the igb implementation for HW offloading was re-written, allowing both tbs
   and cbs qdiscs to co-exist with proper driver support.

The tbs qdisc is designed so it buffers packets until a configurable time before
their deadline (tx times). It uses a rbtree internally, thus the buffered
packets are always 'ordered' by the earliest deadline.

The other configurable parameter from the tbs qdisc is the clockid to be used.
In order to provide that, this series adds a new API to pkt_sched.h (i.e.
qdisc_watchdog_init_clockid()).

As an usage example:

    $ tc qdisc replace dev enp2s0 parent root handle 100 mqprio num_tc 3 \
               map 2 2 1 0 2 2 2 2 2 2 2 2 2 2 2 2 queues 1 at 0 1 at 1 2 at 2 hw 0

    $ tc qdisc add dev enp2s0 parent 100:1 tbs delta 60000 clockid 11 \
               offload 1

In this example first the mqprio qdisc is setup, then the tbs qdisc is
configured onto the first hw Tx queue and tries to enable HW offloading (i.e.
offload 1). Also, it is configured so the timestamps on each packet are in
reference to the clockid '11' (CLOCK_TAI) and so packets are dequeued from
the qdisc 60000 nanoseconds before their transmission time.

The tbs qdisc will drop any packets with a transmission time in the past or
when a deadline is missed. Queueing packets in advance plus configuring the
delta parameter for the system correctly makes all the difference in reducing
the number of drops. Moreover, note that the delta parameter ends up defining
the Tx time when SW best effort is used given that the timestamps won't be used
by the NIC on this case.

For testing, we've followed a similar approach from the v1 testing:
  1. Prepared a PC and the Device Under Test (DUT) each with an Intel
     i210 card connected.
  2. The DUT was a Intel(R) Core(TM) i5-7600 CPU @ 3.50GHz running on top of
     kernel 4.15.0-rc8+ with about 50 usec maximum latency under cyclictest.
  3. Synchronized the DUT's PHC to the PC's PHC using ptp4l.
  4. Synchronized the DUT's system clock to its PHC using phc2sys.
  5. Measured the arrival time of the packets at the PC's PHC using
     hardware time stamping.

First, a baseline test was ran for 10 minutes with the plain kernel only:

|                 | plain kernel @ 1ms |
|-----------------+--------------------+
| min (ns):       |    +4.820000e+02   |
| max (ns):       |    +9.999300e+05   |
| pk-pk:          |    +9.994480e+05   |
| mean (ns):      |    +3.464421e+04   |
| stddev:         |    +1.305947e+05   |
| count:          |           600000   |


Tests were then ran for 10 minutes with a period of 1 millisecond using both
SW best effort and HW offloading. For last, we repeated the HW offloading test
with a 250 microsecond period. The measured offset from the expected period is
shown below, plus the tbs delta parameter that was used in each case.

|                 |    tbs SW @ 1ms   |  tbs HW @ 1ms  | tbs HW @ 250 us |
|-----------------+-------------------+----------------+-----------------|
| min (ns):       |    +1.510000e+02  |  +4.420000e+02 |   +4.260000e+02 |
| max (ns):       |    +9.977030e+05  |  +5.060000e+02 |   +5.060000e+02 |
| pk-pk:          |    +9.975520e+05  |  +6.400000e+01 |   +8.000000e+01 |
| mean (ns):      |    +1.416511e+04  |  +4.687228e+02 |   +4.600596e+02 |
| stddev:         |    +5.750639e+04  |  +9.868569e+00 |   +1.287626e+01 |
| count:          |           600000  |         600000 |         2400000 |
| dropped:        |                3  |              0 |               0 |
| tbs delta (ns): |           130000  |         130000 |          130000 |


The code used for testing is appended below. The wake_tx parameter (-d) used
for all tests was 600000 ns and the priority parameter was 90 (-p). The
baseline test (plain kernel) used a wake_tx parameter of 130000 ns.

Our main questions at this stage are related to the qdisc:
 - does the proposed design attend all use cases?
 - should the qdisc really drop packets that expired after being queued even
   for the SW best effort mode?
 - once one expired packet is found and dropped during a dequeue, should we
   traverse the rbtree and drop other expired packets if any, or should we
   keep deferring that to the next dequeue call?

For last, most of the To Dos we still have before a final patchset are related
to further testing the igb support:
 - testing with L2 only talkers + AF_PACKET sockets;
 - testing tbs in conjunction with cbs;

Thanks,
Jesus


Jesus Sanchez-Palencia (4):
  igb: Refactor igb_configure_cbs()
  igb: Only change Tx arbitration when CBS is on
  igb: Refactor igb_offload_cbs()
  igb: Add support for TBS offload

Richard Cochran (4):
  net: Add a new socket option for a future transmit time.
  net: ipv4: raw: Hook into time based transmission.
  net: ipv4: udp: Hook into time based transmission.
  net: packet: Hook into time based transmission.

Vinicius Costa Gomes (2):
  net/sched: Allow creating a Qdisc watchdog with other clocks
  net/sched: Introduce the TBS Qdisc

 arch/alpha/include/uapi/asm/socket.h           |   3 +
 arch/frv/include/uapi/asm/socket.h             |   3 +
 arch/ia64/include/uapi/asm/socket.h            |   3 +
 arch/m32r/include/uapi/asm/socket.h            |   3 +
 arch/mips/include/uapi/asm/socket.h            |   3 +
 arch/mn10300/include/uapi/asm/socket.h         |   3 +
 arch/parisc/include/uapi/asm/socket.h          |   3 +
 arch/s390/include/uapi/asm/socket.h            |   3 +
 arch/sparc/include/uapi/asm/socket.h           |   3 +
 arch/xtensa/include/uapi/asm/socket.h          |   3 +
 drivers/net/ethernet/intel/igb/e1000_defines.h |  16 +
 drivers/net/ethernet/intel/igb/igb.h           |   1 +
 drivers/net/ethernet/intel/igb/igb_main.c      | 239 +++++++++++----
 include/linux/netdevice.h                      |   1 +
 include/net/pkt_sched.h                        |   7 +
 include/net/sock.h                             |   2 +
 include/uapi/asm-generic/socket.h              |   3 +
 include/uapi/linux/pkt_sched.h                 |  17 ++
 net/core/sock.c                                |  16 +
 net/ipv4/raw.c                                 |   2 +
 net/ipv4/udp.c                                 |   5 +-
 net/packet/af_packet.c                         |   6 +
 net/sched/Kconfig                              |  11 +
 net/sched/Makefile                             |   1 +
 net/sched/sch_api.c                            |  11 +-
 net/sched/sch_tbs.c                            | 392 +++++++++++++++++++++++++
 26 files changed, 699 insertions(+), 61 deletions(-)
 create mode 100644 net/sched/sch_tbs.c

-- 
2.15.1

---8<---
/*
 * This program demonstrates transmission of UDP packets using the
 * system TAI timer.
 *
 * Copyright (C) 2017 linutronix GmbH
 *
 * Large portions taken from the linuxptp stack.
 * Copyright (C) 2011, 2012 Richard Cochran <richardcochran@gmail.com>
 *
 * Some portions taken from the sgd test program.
 * Copyright (C) 2015 linutronix GmbH
 *
 * This program is free software; you can redistribute it and/or modify
 * it under the terms of the GNU General Public License as published by
 * the Free Software Foundation; either version 2 of the License, or
 * (at your option) any later version.
 *
 * This program is distributed in the hope that it will be useful,
 * but WITHOUT ANY WARRANTY; without even the implied warranty of
 * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE.  See the
 * GNU General Public License for more details.
 *
 * You should have received a copy of the GNU General Public License along
 * with this program; if not, write to the Free Software Foundation, Inc.,
 * 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301 USA.
 */
#define _GNU_SOURCE /*for CPU_SET*/
#include <arpa/inet.h>
#include <errno.h>
#include <fcntl.h>
#include <ifaddrs.h>
#include <linux/ethtool.h>
#include <linux/net_tstamp.h>
#include <linux/sockios.h>
#include <net/if.h>
#include <netinet/in.h>
#include <poll.h>
#include <pthread.h>
#include <sched.h>
#include <signal.h>
#include <stdio.h>
#include <stdlib.h>
#include <string.h>
#include <sys/ioctl.h>
#include <sys/socket.h>
#include <sys/stat.h>
#include <sys/types.h>
#include <unistd.h>

#define DEFAULT_PERIOD	1000000
#define DEFAULT_DELAY	500000
#define MCAST_IPADDR	"239.1.1.1"
#define UDP_PORT	7788

#ifndef SO_TXTIME
#define SO_TXTIME	61
#endif

#define pr_err(s)	fprintf(stderr, s "\n")
#define pr_info(s)	fprintf(stdout, s "\n")

static int running = 1, use_so_txtime = 1;
static int period_nsec = DEFAULT_PERIOD;
static int waketx_delay = DEFAULT_DELAY;
static struct in_addr mcast_addr;

static int mcast_bind(int fd, int index)
{
	int err;
	struct ip_mreqn req;
	memset(&req, 0, sizeof(req));
	req.imr_ifindex = index;
	err = setsockopt(fd, IPPROTO_IP, IP_MULTICAST_IF, &req, sizeof(req));
	if (err) {
		pr_err("setsockopt IP_MULTICAST_IF failed: %m");
		return -1;
	}
	return 0;
}

static int mcast_join(int fd, int index, const struct sockaddr *grp,
		      socklen_t grplen)
{
	int err, off = 0;
	struct ip_mreqn req;
	struct sockaddr_in *sa = (struct sockaddr_in *) grp;

	memset(&req, 0, sizeof(req));
	memcpy(&req.imr_multiaddr, &sa->sin_addr, sizeof(struct in_addr));
	req.imr_ifindex = index;
	err = setsockopt(fd, IPPROTO_IP, IP_ADD_MEMBERSHIP, &req, sizeof(req));
	if (err) {
		pr_err("setsockopt IP_ADD_MEMBERSHIP failed: %m");
		return -1;
	}
	err = setsockopt(fd, IPPROTO_IP, IP_MULTICAST_LOOP, &off, sizeof(off));
	if (err) {
		pr_err("setsockopt IP_MULTICAST_LOOP failed: %m");
		return -1;
	}
	return 0;
}

static void normalize(struct timespec *ts)
{
	while (ts->tv_nsec > 999999999) {
		ts->tv_sec += 1;
		ts->tv_nsec -= 1000000000;
	}
}

static int sk_interface_index(int fd, const char *name)
{
	struct ifreq ifreq;
	int err;

	memset(&ifreq, 0, sizeof(ifreq));
	strncpy(ifreq.ifr_name, name, sizeof(ifreq.ifr_name) - 1);
	err = ioctl(fd, SIOCGIFINDEX, &ifreq);
	if (err < 0) {
		pr_err("ioctl SIOCGIFINDEX failed: %m");
		return err;
	}
	return ifreq.ifr_ifindex;
}

static int open_socket(const char *name, struct in_addr mc_addr, short port)
{
	struct sockaddr_in addr;
	int fd, index, on = 1;
	int priority = 3;

	memset(&addr, 0, sizeof(addr));
	addr.sin_family = AF_INET;
	addr.sin_addr.s_addr = htonl(INADDR_ANY);
	addr.sin_port = htons(port);

	fd = socket(PF_INET, SOCK_DGRAM, IPPROTO_UDP);
	if (fd < 0) {
		pr_err("socket failed: %m");
		goto no_socket;
	}
	index = sk_interface_index(fd, name);
	if (index < 0)
		goto no_option;

	if (setsockopt(fd, SOL_SOCKET, SO_PRIORITY, &priority, sizeof(priority))) {
		pr_err("Couldn't set priority");
		goto no_option;
	}
	if (setsockopt(fd, SOL_SOCKET, SO_REUSEADDR, &on, sizeof(on))) {
		pr_err("setsockopt SO_REUSEADDR failed: %m");
		goto no_option;
	}
	if (bind(fd, (struct sockaddr *) &addr, sizeof(addr))) {
		pr_err("bind failed: %m");
		goto no_option;
	}
	if (setsockopt(fd, SOL_SOCKET, SO_BINDTODEVICE, name, strlen(name))) {
		pr_err("setsockopt SO_BINDTODEVICE failed: %m");
		goto no_option;
	}
	addr.sin_addr = mc_addr;
	if (mcast_join(fd, index, (struct sockaddr *) &addr, sizeof(addr))) {
		pr_err("mcast_join failed");
		goto no_option;
	}
	if (mcast_bind(fd, index)) {
		goto no_option;
	}
	if (use_so_txtime && setsockopt(fd, SOL_SOCKET, SO_TXTIME, &on, sizeof(on))) {
		pr_err("setsockopt SO_TXTIME failed: %m");
		goto no_option;
	}

	return fd;
no_option:
	close(fd);
no_socket:
	return -1;
}

static int udp_open(const char *name)
{
	int fd;

	if (!inet_aton(MCAST_IPADDR, &mcast_addr))
		return -1;

	fd = open_socket(name, mcast_addr, UDP_PORT);

	return fd;
}

static int udp_send(int fd, void *buf, int len, __u64 txtime)
{
	union {
		char buf[CMSG_SPACE(sizeof(__u64))];
		struct cmsghdr align;
	} u;
	struct sockaddr_in sin;
	struct cmsghdr *cmsg;
	struct msghdr msg;
	struct iovec iov;
	ssize_t cnt;

	memset(&sin, 0, sizeof(sin));
	sin.sin_family = AF_INET;
	sin.sin_addr = mcast_addr;
	sin.sin_port = htons(UDP_PORT);

	iov.iov_base = buf;
	iov.iov_len = len;

	memset(&msg, 0, sizeof(msg));
	msg.msg_name = &sin;
	msg.msg_namelen = sizeof(sin);
	msg.msg_iov = &iov;
	msg.msg_iovlen = 1;

	/*
	 * We specify the transmission time in the CMSG.
	 */
	if (use_so_txtime) {
		msg.msg_control = u.buf;
		msg.msg_controllen = sizeof(u.buf);
		cmsg = CMSG_FIRSTHDR(&msg);
		cmsg->cmsg_level = SOL_SOCKET;
		cmsg->cmsg_type = SO_TXTIME;
		cmsg->cmsg_len = CMSG_LEN(sizeof(__u64));
		*((__u64 *) CMSG_DATA(cmsg)) = txtime;
	}
	cnt = sendmsg(fd, &msg, 0);
	if (cnt < 1) {
		pr_err("sendmsg failed: %m");
		return cnt;
	}
	return cnt;
}

static unsigned char tx_buffer[256];
static int marker;

static int run_nanosleep(clockid_t clkid, int fd)
{
	struct timespec ts;
	int cnt, err;
	__u64 txtime;

	clock_gettime(clkid, &ts);

	/* Start one to two seconds in the future. */
	ts.tv_sec += 1;
	ts.tv_nsec = 1000000000 - waketx_delay;
	normalize(&ts);

	txtime = ts.tv_sec * 1000000000ULL + ts.tv_nsec;
	txtime += waketx_delay;

	while (running) {
		err = clock_nanosleep(clkid, TIMER_ABSTIME, &ts, NULL);
		switch (err) {
		case 0:
			cnt = udp_send(fd, tx_buffer, sizeof(tx_buffer), txtime);
			if (cnt != sizeof(tx_buffer)) {
				pr_err("udp_send failed");
			}
			memset(tx_buffer, marker++, sizeof(tx_buffer));
			ts.tv_nsec += period_nsec;
			normalize(&ts);
			txtime += period_nsec;
			break;
		case EINTR:
			continue;
		default:
			fprintf(stderr, "clock_nanosleep returned %d: %s",
				err, strerror(err));
			return err;
		}
	}

	return 0;
}

static int set_realtime(pthread_t thread, int priority, int cpu)
{
	cpu_set_t cpuset;
	struct sched_param sp;
	int err, policy;

	int min = sched_get_priority_min(SCHED_FIFO);
	int max = sched_get_priority_max(SCHED_FIFO);

	fprintf(stderr, "min %d max %d\n", min, max);

	if (priority < 0) {
		return 0;
	}

	err = pthread_getschedparam(thread, &policy, &sp);
	if (err) {
		fprintf(stderr, "pthread_getschedparam: %s\n", strerror(err));
		return -1;
	}

	sp.sched_priority = priority;

	err = pthread_setschedparam(thread, SCHED_FIFO, &sp);
	if (err) {
		fprintf(stderr, "pthread_setschedparam: %s\n", strerror(err));
		return -1;
	}

	if (cpu < 0) {
		return 0;
	}
	CPU_ZERO(&cpuset);
	CPU_SET(cpu, &cpuset);
	err = pthread_setaffinity_np(thread, sizeof(cpu_set_t), &cpuset);
	if (err) {
		fprintf(stderr, "pthread_setaffinity_np: %s\n", strerror(err));
		return -1;
	}

	return 0;
}

static void usage(char *progname)
{
	fprintf(stderr,
		"\n"
		"usage: %s [options]\n"
		"\n"
		" -c [num]   run on CPU 'num'\n"
		" -d [num]   delay from wake up to transmission in nanoseconds (default %d)\n"
		" -h         prints this message and exits\n"
		" -i [name]  use network interface 'name'\n"
		" -p [num]   run with RT priorty 'num'\n"
		" -P [num]   period in nanoseconds (default %d)\n"
		" -u         do not use SO_TXTIME\n"
		"\n",
		progname, DEFAULT_DELAY, DEFAULT_PERIOD);
}

int main(int argc, char *argv[])
{
	int c, cpu = -1, err, fd, priority = -1;
	clockid_t clkid = CLOCK_TAI;
	char *iface = NULL, *progname;

	/* Process the command line arguments. */
	progname = strrchr(argv[0], '/');
	progname = progname ? 1 + progname : argv[0];
	while (EOF != (c = getopt(argc, argv, "c:d:hi:p:P:u"))) {
		switch (c) {
		case 'c':
			cpu = atoi(optarg);
			break;
		case 'd':
			waketx_delay = atoi(optarg);
			break;
		case 'h':
			usage(progname);
			return 0;
		case 'i':
			iface = optarg;
			break;
		case 'p':
			priority = atoi(optarg);
			break;
		case 'P':
			period_nsec = atoi(optarg);
			break;
		case 'u':
			use_so_txtime = 0;
			break;
		case '?':
			usage(progname);
			return -1;
		}
	}

	if (waketx_delay > 999999999 || waketx_delay < 0) {
		pr_err("Bad wake up to transmission delay.");
		usage(progname);
		return -1;
	}

	if (period_nsec < 1000) {
		pr_err("Bad period.");
		usage(progname);
		return -1;
	}

	if (!iface) {
		pr_err("Need a network interface.");
		usage(progname);
		return -1;
	}

	if (set_realtime(pthread_self(), priority, cpu)) {
		return -1;
	}

	fd = udp_open(iface);
	if (fd < 0) {
		return -1;
	}

	err = run_nanosleep(clkid, fd);

	close(fd);
	return err;
}


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

* [RFC v2 net-next 01/10] net: Add a new socket option for a future transmit time.
  2018-01-17 23:06 ` [Intel-wired-lan] " Jesus Sanchez-Palencia
@ 2018-01-17 23:06   ` Jesus Sanchez-Palencia
  -1 siblings, 0 replies; 86+ messages in thread
From: Jesus Sanchez-Palencia @ 2018-01-17 23:06 UTC (permalink / raw)
  To: netdev
  Cc: jhs, xiyou.wangcong, jiri, vinicius.gomes, richardcochran,
	intel-wired-lan, anna-maria, henrik, tglx, john.stultz,
	andre.guedes, ivan.briano, levi.pearson, Richard Cochran,
	Jesus Sanchez-Palencia

From: Richard Cochran <rcochran@linutronix.de>

This patch introduces SO_TXTIME.  User space enables this option in
order to pass a desired future transmit time in a CMSG when calling
sendmsg(2).

A new field is added to struct sockcm_cookie, and the tstamp from
skbuffs will be used later on.

Signed-off-by: Richard Cochran <rcochran@linutronix.de>
Signed-off-by: Jesus Sanchez-Palencia <jesus.sanchez-palencia@intel.com>
---
 arch/alpha/include/uapi/asm/socket.h   |  3 +++
 arch/frv/include/uapi/asm/socket.h     |  3 +++
 arch/ia64/include/uapi/asm/socket.h    |  3 +++
 arch/m32r/include/uapi/asm/socket.h    |  3 +++
 arch/mips/include/uapi/asm/socket.h    |  3 +++
 arch/mn10300/include/uapi/asm/socket.h |  3 +++
 arch/parisc/include/uapi/asm/socket.h  |  3 +++
 arch/s390/include/uapi/asm/socket.h    |  3 +++
 arch/sparc/include/uapi/asm/socket.h   |  3 +++
 arch/xtensa/include/uapi/asm/socket.h  |  3 +++
 include/net/sock.h                     |  2 ++
 include/uapi/asm-generic/socket.h      |  3 +++
 net/core/sock.c                        | 16 ++++++++++++++++
 13 files changed, 51 insertions(+)

diff --git a/arch/alpha/include/uapi/asm/socket.h b/arch/alpha/include/uapi/asm/socket.h
index be14f16149d5..065fb372e355 100644
--- a/arch/alpha/include/uapi/asm/socket.h
+++ b/arch/alpha/include/uapi/asm/socket.h
@@ -112,4 +112,7 @@
 
 #define SO_ZEROCOPY		60
 
+#define SO_TXTIME		61
+#define SCM_TXTIME		SO_TXTIME
+
 #endif /* _UAPI_ASM_SOCKET_H */
diff --git a/arch/frv/include/uapi/asm/socket.h b/arch/frv/include/uapi/asm/socket.h
index 9168e78fa32a..0e95f45cd058 100644
--- a/arch/frv/include/uapi/asm/socket.h
+++ b/arch/frv/include/uapi/asm/socket.h
@@ -105,5 +105,8 @@
 
 #define SO_ZEROCOPY		60
 
+#define SO_TXTIME		61
+#define SCM_TXTIME		SO_TXTIME
+
 #endif /* _ASM_SOCKET_H */
 
diff --git a/arch/ia64/include/uapi/asm/socket.h b/arch/ia64/include/uapi/asm/socket.h
index 3efba40adc54..c872c4e6bafb 100644
--- a/arch/ia64/include/uapi/asm/socket.h
+++ b/arch/ia64/include/uapi/asm/socket.h
@@ -114,4 +114,7 @@
 
 #define SO_ZEROCOPY		60
 
+#define SO_TXTIME		61
+#define SCM_TXTIME		SO_TXTIME
+
 #endif /* _ASM_IA64_SOCKET_H */
diff --git a/arch/m32r/include/uapi/asm/socket.h b/arch/m32r/include/uapi/asm/socket.h
index cf5018e82c3d..65276c95b8df 100644
--- a/arch/m32r/include/uapi/asm/socket.h
+++ b/arch/m32r/include/uapi/asm/socket.h
@@ -105,4 +105,7 @@
 
 #define SO_ZEROCOPY		60
 
+#define SO_TXTIME		61
+#define SCM_TXTIME		SO_TXTIME
+
 #endif /* _ASM_M32R_SOCKET_H */
diff --git a/arch/mips/include/uapi/asm/socket.h b/arch/mips/include/uapi/asm/socket.h
index 49c3d4795963..71370fb3ceef 100644
--- a/arch/mips/include/uapi/asm/socket.h
+++ b/arch/mips/include/uapi/asm/socket.h
@@ -123,4 +123,7 @@
 
 #define SO_ZEROCOPY		60
 
+#define SO_TXTIME		61
+#define SCM_TXTIME		SO_TXTIME
+
 #endif /* _UAPI_ASM_SOCKET_H */
diff --git a/arch/mn10300/include/uapi/asm/socket.h b/arch/mn10300/include/uapi/asm/socket.h
index b35eee132142..d029a40b1b55 100644
--- a/arch/mn10300/include/uapi/asm/socket.h
+++ b/arch/mn10300/include/uapi/asm/socket.h
@@ -105,4 +105,7 @@
 
 #define SO_ZEROCOPY		60
 
+#define SO_TXTIME		61
+#define SCM_TXTIME		SO_TXTIME
+
 #endif /* _ASM_SOCKET_H */
diff --git a/arch/parisc/include/uapi/asm/socket.h b/arch/parisc/include/uapi/asm/socket.h
index 1d0fdc3b5d22..061b9cf2a779 100644
--- a/arch/parisc/include/uapi/asm/socket.h
+++ b/arch/parisc/include/uapi/asm/socket.h
@@ -104,4 +104,7 @@
 
 #define SO_ZEROCOPY		0x4035
 
+#define SO_TXTIME		0x4036
+#define SCM_TXTIME		SO_TXTIME
+
 #endif /* _UAPI_ASM_SOCKET_H */
diff --git a/arch/s390/include/uapi/asm/socket.h b/arch/s390/include/uapi/asm/socket.h
index 3510c0fd06f4..39d901476ee5 100644
--- a/arch/s390/include/uapi/asm/socket.h
+++ b/arch/s390/include/uapi/asm/socket.h
@@ -111,4 +111,7 @@
 
 #define SO_ZEROCOPY		60
 
+#define SO_TXTIME		61
+#define SCM_TXTIME		SO_TXTIME
+
 #endif /* _ASM_SOCKET_H */
diff --git a/arch/sparc/include/uapi/asm/socket.h b/arch/sparc/include/uapi/asm/socket.h
index d58520c2e6ff..7ea35e5601b6 100644
--- a/arch/sparc/include/uapi/asm/socket.h
+++ b/arch/sparc/include/uapi/asm/socket.h
@@ -101,6 +101,9 @@
 
 #define SO_ZEROCOPY		0x003e
 
+#define SO_TXTIME		0x003f
+#define SCM_TXTIME		SO_TXTIME
+
 /* Security levels - as per NRL IPv6 - don't actually do anything */
 #define SO_SECURITY_AUTHENTICATION		0x5001
 #define SO_SECURITY_ENCRYPTION_TRANSPORT	0x5002
diff --git a/arch/xtensa/include/uapi/asm/socket.h b/arch/xtensa/include/uapi/asm/socket.h
index 75a07b8119a9..1de07a7f7680 100644
--- a/arch/xtensa/include/uapi/asm/socket.h
+++ b/arch/xtensa/include/uapi/asm/socket.h
@@ -116,4 +116,7 @@
 
 #define SO_ZEROCOPY		60
 
+#define SO_TXTIME		61
+#define SCM_TXTIME		SO_TXTIME
+
 #endif	/* _XTENSA_SOCKET_H */
diff --git a/include/net/sock.h b/include/net/sock.h
index 73b7830b0bb8..927af34f3e2c 100644
--- a/include/net/sock.h
+++ b/include/net/sock.h
@@ -777,6 +777,7 @@ enum sock_flags {
 	SOCK_FILTER_LOCKED, /* Filter cannot be changed anymore */
 	SOCK_SELECT_ERR_QUEUE, /* Wake select on error queue */
 	SOCK_RCU_FREE, /* wait rcu grace period in sk_destruct() */
+	SOCK_TXTIME,
 };
 
 #define SK_FLAGS_TIMESTAMP ((1UL << SOCK_TIMESTAMP) | (1UL << SOCK_TIMESTAMPING_RX_SOFTWARE))
@@ -1567,6 +1568,7 @@ void sock_kzfree_s(struct sock *sk, void *mem, int size);
 void sk_send_sigurg(struct sock *sk);
 
 struct sockcm_cookie {
+	ktime_t transmit_time;
 	u32 mark;
 	u16 tsflags;
 };
diff --git a/include/uapi/asm-generic/socket.h b/include/uapi/asm-generic/socket.h
index 0ae758c90e54..a12692e5f7a8 100644
--- a/include/uapi/asm-generic/socket.h
+++ b/include/uapi/asm-generic/socket.h
@@ -107,4 +107,7 @@
 
 #define SO_ZEROCOPY		60
 
+#define SO_TXTIME		61
+#define SCM_TXTIME		SO_TXTIME
+
 #endif /* __ASM_GENERIC_SOCKET_H */
diff --git a/net/core/sock.c b/net/core/sock.c
index abf4cbff99b2..37ef4b33fd92 100644
--- a/net/core/sock.c
+++ b/net/core/sock.c
@@ -1061,6 +1061,13 @@ int sock_setsockopt(struct socket *sock, int level, int optname,
 			sock_valbool_flag(sk, SOCK_ZEROCOPY, valbool);
 		break;
 
+	case SO_TXTIME:
+		if (ns_capable(sock_net(sk)->user_ns, CAP_NET_ADMIN))
+			sock_valbool_flag(sk, SOCK_TXTIME, valbool);
+		else
+			ret = -EPERM;
+		break;
+
 	default:
 		ret = -ENOPROTOOPT;
 		break;
@@ -2130,6 +2137,15 @@ int __sock_cmsg_send(struct sock *sk, struct msghdr *msg, struct cmsghdr *cmsg,
 		sockc->tsflags &= ~SOF_TIMESTAMPING_TX_RECORD_MASK;
 		sockc->tsflags |= tsflags;
 		break;
+	case SO_TXTIME:
+		if (!ns_capable(sock_net(sk)->user_ns, CAP_NET_ADMIN))
+			return -EPERM;
+		if (!sock_flag(sk, SOCK_TXTIME))
+			return -EINVAL;
+		if (cmsg->cmsg_len != CMSG_LEN(sizeof(ktime_t)))
+			return -EINVAL;
+		sockc->transmit_time = *(ktime_t *)CMSG_DATA(cmsg);
+		break;
 	/* SCM_RIGHTS and SCM_CREDENTIALS are semantically in SOL_UNIX. */
 	case SCM_RIGHTS:
 	case SCM_CREDENTIALS:
-- 
2.15.1

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

* [Intel-wired-lan] [RFC v2 net-next 01/10] net: Add a new socket option for a future transmit time.
@ 2018-01-17 23:06   ` Jesus Sanchez-Palencia
  0 siblings, 0 replies; 86+ messages in thread
From: Jesus Sanchez-Palencia @ 2018-01-17 23:06 UTC (permalink / raw)
  To: intel-wired-lan

From: Richard Cochran <rcochran@linutronix.de>

This patch introduces SO_TXTIME.  User space enables this option in
order to pass a desired future transmit time in a CMSG when calling
sendmsg(2).

A new field is added to struct sockcm_cookie, and the tstamp from
skbuffs will be used later on.

Signed-off-by: Richard Cochran <rcochran@linutronix.de>
Signed-off-by: Jesus Sanchez-Palencia <jesus.sanchez-palencia@intel.com>
---
 arch/alpha/include/uapi/asm/socket.h   |  3 +++
 arch/frv/include/uapi/asm/socket.h     |  3 +++
 arch/ia64/include/uapi/asm/socket.h    |  3 +++
 arch/m32r/include/uapi/asm/socket.h    |  3 +++
 arch/mips/include/uapi/asm/socket.h    |  3 +++
 arch/mn10300/include/uapi/asm/socket.h |  3 +++
 arch/parisc/include/uapi/asm/socket.h  |  3 +++
 arch/s390/include/uapi/asm/socket.h    |  3 +++
 arch/sparc/include/uapi/asm/socket.h   |  3 +++
 arch/xtensa/include/uapi/asm/socket.h  |  3 +++
 include/net/sock.h                     |  2 ++
 include/uapi/asm-generic/socket.h      |  3 +++
 net/core/sock.c                        | 16 ++++++++++++++++
 13 files changed, 51 insertions(+)

diff --git a/arch/alpha/include/uapi/asm/socket.h b/arch/alpha/include/uapi/asm/socket.h
index be14f16149d5..065fb372e355 100644
--- a/arch/alpha/include/uapi/asm/socket.h
+++ b/arch/alpha/include/uapi/asm/socket.h
@@ -112,4 +112,7 @@
 
 #define SO_ZEROCOPY		60
 
+#define SO_TXTIME		61
+#define SCM_TXTIME		SO_TXTIME
+
 #endif /* _UAPI_ASM_SOCKET_H */
diff --git a/arch/frv/include/uapi/asm/socket.h b/arch/frv/include/uapi/asm/socket.h
index 9168e78fa32a..0e95f45cd058 100644
--- a/arch/frv/include/uapi/asm/socket.h
+++ b/arch/frv/include/uapi/asm/socket.h
@@ -105,5 +105,8 @@
 
 #define SO_ZEROCOPY		60
 
+#define SO_TXTIME		61
+#define SCM_TXTIME		SO_TXTIME
+
 #endif /* _ASM_SOCKET_H */
 
diff --git a/arch/ia64/include/uapi/asm/socket.h b/arch/ia64/include/uapi/asm/socket.h
index 3efba40adc54..c872c4e6bafb 100644
--- a/arch/ia64/include/uapi/asm/socket.h
+++ b/arch/ia64/include/uapi/asm/socket.h
@@ -114,4 +114,7 @@
 
 #define SO_ZEROCOPY		60
 
+#define SO_TXTIME		61
+#define SCM_TXTIME		SO_TXTIME
+
 #endif /* _ASM_IA64_SOCKET_H */
diff --git a/arch/m32r/include/uapi/asm/socket.h b/arch/m32r/include/uapi/asm/socket.h
index cf5018e82c3d..65276c95b8df 100644
--- a/arch/m32r/include/uapi/asm/socket.h
+++ b/arch/m32r/include/uapi/asm/socket.h
@@ -105,4 +105,7 @@
 
 #define SO_ZEROCOPY		60
 
+#define SO_TXTIME		61
+#define SCM_TXTIME		SO_TXTIME
+
 #endif /* _ASM_M32R_SOCKET_H */
diff --git a/arch/mips/include/uapi/asm/socket.h b/arch/mips/include/uapi/asm/socket.h
index 49c3d4795963..71370fb3ceef 100644
--- a/arch/mips/include/uapi/asm/socket.h
+++ b/arch/mips/include/uapi/asm/socket.h
@@ -123,4 +123,7 @@
 
 #define SO_ZEROCOPY		60
 
+#define SO_TXTIME		61
+#define SCM_TXTIME		SO_TXTIME
+
 #endif /* _UAPI_ASM_SOCKET_H */
diff --git a/arch/mn10300/include/uapi/asm/socket.h b/arch/mn10300/include/uapi/asm/socket.h
index b35eee132142..d029a40b1b55 100644
--- a/arch/mn10300/include/uapi/asm/socket.h
+++ b/arch/mn10300/include/uapi/asm/socket.h
@@ -105,4 +105,7 @@
 
 #define SO_ZEROCOPY		60
 
+#define SO_TXTIME		61
+#define SCM_TXTIME		SO_TXTIME
+
 #endif /* _ASM_SOCKET_H */
diff --git a/arch/parisc/include/uapi/asm/socket.h b/arch/parisc/include/uapi/asm/socket.h
index 1d0fdc3b5d22..061b9cf2a779 100644
--- a/arch/parisc/include/uapi/asm/socket.h
+++ b/arch/parisc/include/uapi/asm/socket.h
@@ -104,4 +104,7 @@
 
 #define SO_ZEROCOPY		0x4035
 
+#define SO_TXTIME		0x4036
+#define SCM_TXTIME		SO_TXTIME
+
 #endif /* _UAPI_ASM_SOCKET_H */
diff --git a/arch/s390/include/uapi/asm/socket.h b/arch/s390/include/uapi/asm/socket.h
index 3510c0fd06f4..39d901476ee5 100644
--- a/arch/s390/include/uapi/asm/socket.h
+++ b/arch/s390/include/uapi/asm/socket.h
@@ -111,4 +111,7 @@
 
 #define SO_ZEROCOPY		60
 
+#define SO_TXTIME		61
+#define SCM_TXTIME		SO_TXTIME
+
 #endif /* _ASM_SOCKET_H */
diff --git a/arch/sparc/include/uapi/asm/socket.h b/arch/sparc/include/uapi/asm/socket.h
index d58520c2e6ff..7ea35e5601b6 100644
--- a/arch/sparc/include/uapi/asm/socket.h
+++ b/arch/sparc/include/uapi/asm/socket.h
@@ -101,6 +101,9 @@
 
 #define SO_ZEROCOPY		0x003e
 
+#define SO_TXTIME		0x003f
+#define SCM_TXTIME		SO_TXTIME
+
 /* Security levels - as per NRL IPv6 - don't actually do anything */
 #define SO_SECURITY_AUTHENTICATION		0x5001
 #define SO_SECURITY_ENCRYPTION_TRANSPORT	0x5002
diff --git a/arch/xtensa/include/uapi/asm/socket.h b/arch/xtensa/include/uapi/asm/socket.h
index 75a07b8119a9..1de07a7f7680 100644
--- a/arch/xtensa/include/uapi/asm/socket.h
+++ b/arch/xtensa/include/uapi/asm/socket.h
@@ -116,4 +116,7 @@
 
 #define SO_ZEROCOPY		60
 
+#define SO_TXTIME		61
+#define SCM_TXTIME		SO_TXTIME
+
 #endif	/* _XTENSA_SOCKET_H */
diff --git a/include/net/sock.h b/include/net/sock.h
index 73b7830b0bb8..927af34f3e2c 100644
--- a/include/net/sock.h
+++ b/include/net/sock.h
@@ -777,6 +777,7 @@ enum sock_flags {
 	SOCK_FILTER_LOCKED, /* Filter cannot be changed anymore */
 	SOCK_SELECT_ERR_QUEUE, /* Wake select on error queue */
 	SOCK_RCU_FREE, /* wait rcu grace period in sk_destruct() */
+	SOCK_TXTIME,
 };
 
 #define SK_FLAGS_TIMESTAMP ((1UL << SOCK_TIMESTAMP) | (1UL << SOCK_TIMESTAMPING_RX_SOFTWARE))
@@ -1567,6 +1568,7 @@ void sock_kzfree_s(struct sock *sk, void *mem, int size);
 void sk_send_sigurg(struct sock *sk);
 
 struct sockcm_cookie {
+	ktime_t transmit_time;
 	u32 mark;
 	u16 tsflags;
 };
diff --git a/include/uapi/asm-generic/socket.h b/include/uapi/asm-generic/socket.h
index 0ae758c90e54..a12692e5f7a8 100644
--- a/include/uapi/asm-generic/socket.h
+++ b/include/uapi/asm-generic/socket.h
@@ -107,4 +107,7 @@
 
 #define SO_ZEROCOPY		60
 
+#define SO_TXTIME		61
+#define SCM_TXTIME		SO_TXTIME
+
 #endif /* __ASM_GENERIC_SOCKET_H */
diff --git a/net/core/sock.c b/net/core/sock.c
index abf4cbff99b2..37ef4b33fd92 100644
--- a/net/core/sock.c
+++ b/net/core/sock.c
@@ -1061,6 +1061,13 @@ int sock_setsockopt(struct socket *sock, int level, int optname,
 			sock_valbool_flag(sk, SOCK_ZEROCOPY, valbool);
 		break;
 
+	case SO_TXTIME:
+		if (ns_capable(sock_net(sk)->user_ns, CAP_NET_ADMIN))
+			sock_valbool_flag(sk, SOCK_TXTIME, valbool);
+		else
+			ret = -EPERM;
+		break;
+
 	default:
 		ret = -ENOPROTOOPT;
 		break;
@@ -2130,6 +2137,15 @@ int __sock_cmsg_send(struct sock *sk, struct msghdr *msg, struct cmsghdr *cmsg,
 		sockc->tsflags &= ~SOF_TIMESTAMPING_TX_RECORD_MASK;
 		sockc->tsflags |= tsflags;
 		break;
+	case SO_TXTIME:
+		if (!ns_capable(sock_net(sk)->user_ns, CAP_NET_ADMIN))
+			return -EPERM;
+		if (!sock_flag(sk, SOCK_TXTIME))
+			return -EINVAL;
+		if (cmsg->cmsg_len != CMSG_LEN(sizeof(ktime_t)))
+			return -EINVAL;
+		sockc->transmit_time = *(ktime_t *)CMSG_DATA(cmsg);
+		break;
 	/* SCM_RIGHTS and SCM_CREDENTIALS are semantically in SOL_UNIX. */
 	case SCM_RIGHTS:
 	case SCM_CREDENTIALS:
-- 
2.15.1


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

* [RFC v2 net-next 02/10] net: ipv4: raw: Hook into time based transmission.
  2018-01-17 23:06 ` [Intel-wired-lan] " Jesus Sanchez-Palencia
@ 2018-01-17 23:06   ` Jesus Sanchez-Palencia
  -1 siblings, 0 replies; 86+ messages in thread
From: Jesus Sanchez-Palencia @ 2018-01-17 23:06 UTC (permalink / raw)
  To: netdev
  Cc: jhs, xiyou.wangcong, jiri, vinicius.gomes, richardcochran,
	intel-wired-lan, anna-maria, henrik, tglx, john.stultz,
	andre.guedes, ivan.briano, levi.pearson, Richard Cochran,
	Jesus Sanchez-Palencia

From: Richard Cochran <rcochran@linutronix.de>

For raw packets, copy the desired future transmit time from the CMSG
cookie into the skb.

Signed-off-by: Richard Cochran <rcochran@linutronix.de>
Signed-off-by: Jesus Sanchez-Palencia <jesus.sanchez-palencia@intel.com>
---
 net/ipv4/raw.c | 2 ++
 1 file changed, 2 insertions(+)

diff --git a/net/ipv4/raw.c b/net/ipv4/raw.c
index 136544b36a46..e37ea8ab6a78 100644
--- a/net/ipv4/raw.c
+++ b/net/ipv4/raw.c
@@ -381,6 +381,7 @@ static int raw_send_hdrinc(struct sock *sk, struct flowi4 *fl4,
 
 	skb->priority = sk->sk_priority;
 	skb->mark = sk->sk_mark;
+	skb->tstamp = sockc->transmit_time;
 	skb_dst_set(skb, &rt->dst);
 	*rtp = NULL;
 
@@ -562,6 +563,7 @@ static int raw_sendmsg(struct sock *sk, struct msghdr *msg, size_t len)
 	}
 
 	ipc.sockc.tsflags = sk->sk_tsflags;
+	ipc.sockc.transmit_time = 0;
 	ipc.addr = inet->inet_saddr;
 	ipc.opt = NULL;
 	ipc.tx_flags = 0;
-- 
2.15.1

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

* [Intel-wired-lan] [RFC v2 net-next 02/10] net: ipv4: raw: Hook into time based transmission.
@ 2018-01-17 23:06   ` Jesus Sanchez-Palencia
  0 siblings, 0 replies; 86+ messages in thread
From: Jesus Sanchez-Palencia @ 2018-01-17 23:06 UTC (permalink / raw)
  To: intel-wired-lan

From: Richard Cochran <rcochran@linutronix.de>

For raw packets, copy the desired future transmit time from the CMSG
cookie into the skb.

Signed-off-by: Richard Cochran <rcochran@linutronix.de>
Signed-off-by: Jesus Sanchez-Palencia <jesus.sanchez-palencia@intel.com>
---
 net/ipv4/raw.c | 2 ++
 1 file changed, 2 insertions(+)

diff --git a/net/ipv4/raw.c b/net/ipv4/raw.c
index 136544b36a46..e37ea8ab6a78 100644
--- a/net/ipv4/raw.c
+++ b/net/ipv4/raw.c
@@ -381,6 +381,7 @@ static int raw_send_hdrinc(struct sock *sk, struct flowi4 *fl4,
 
 	skb->priority = sk->sk_priority;
 	skb->mark = sk->sk_mark;
+	skb->tstamp = sockc->transmit_time;
 	skb_dst_set(skb, &rt->dst);
 	*rtp = NULL;
 
@@ -562,6 +563,7 @@ static int raw_sendmsg(struct sock *sk, struct msghdr *msg, size_t len)
 	}
 
 	ipc.sockc.tsflags = sk->sk_tsflags;
+	ipc.sockc.transmit_time = 0;
 	ipc.addr = inet->inet_saddr;
 	ipc.opt = NULL;
 	ipc.tx_flags = 0;
-- 
2.15.1


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

* [RFC v2 net-next 03/10] net: ipv4: udp: Hook into time based transmission.
  2018-01-17 23:06 ` [Intel-wired-lan] " Jesus Sanchez-Palencia
@ 2018-01-17 23:06   ` Jesus Sanchez-Palencia
  -1 siblings, 0 replies; 86+ messages in thread
From: Jesus Sanchez-Palencia @ 2018-01-17 23:06 UTC (permalink / raw)
  To: netdev
  Cc: jhs, xiyou.wangcong, jiri, vinicius.gomes, richardcochran,
	intel-wired-lan, anna-maria, henrik, tglx, john.stultz,
	andre.guedes, ivan.briano, levi.pearson, Richard Cochran,
	Jesus Sanchez-Palencia

From: Richard Cochran <rcochran@linutronix.de>

For udp packets, copy the desired future transmit time from the CMSG
cookie into the skb.

Signed-off-by: Richard Cochran <rcochran@linutronix.de>
Signed-off-by: Jesus Sanchez-Palencia <jesus.sanchez-palencia@intel.com>
---
 net/ipv4/udp.c | 5 ++++-
 1 file changed, 4 insertions(+), 1 deletion(-)

diff --git a/net/ipv4/udp.c b/net/ipv4/udp.c
index 853321555a4e..89257422503a 100644
--- a/net/ipv4/udp.c
+++ b/net/ipv4/udp.c
@@ -926,6 +926,7 @@ int udp_sendmsg(struct sock *sk, struct msghdr *msg, size_t len)
 	}
 
 	ipc.sockc.tsflags = sk->sk_tsflags;
+	ipc.sockc.transmit_time = 0;
 	ipc.addr = inet->inet_saddr;
 	ipc.oif = sk->sk_bound_dev_if;
 
@@ -1027,8 +1028,10 @@ int udp_sendmsg(struct sock *sk, struct msghdr *msg, size_t len)
 				  sizeof(struct udphdr), &ipc, &rt,
 				  msg->msg_flags);
 		err = PTR_ERR(skb);
-		if (!IS_ERR_OR_NULL(skb))
+		if (!IS_ERR_OR_NULL(skb)) {
+			skb->tstamp = ipc.sockc.transmit_time;
 			err = udp_send_skb(skb, fl4);
+		}
 		goto out;
 	}
 
-- 
2.15.1

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

* [Intel-wired-lan] [RFC v2 net-next 03/10] net: ipv4: udp: Hook into time based transmission.
@ 2018-01-17 23:06   ` Jesus Sanchez-Palencia
  0 siblings, 0 replies; 86+ messages in thread
From: Jesus Sanchez-Palencia @ 2018-01-17 23:06 UTC (permalink / raw)
  To: intel-wired-lan

From: Richard Cochran <rcochran@linutronix.de>

For udp packets, copy the desired future transmit time from the CMSG
cookie into the skb.

Signed-off-by: Richard Cochran <rcochran@linutronix.de>
Signed-off-by: Jesus Sanchez-Palencia <jesus.sanchez-palencia@intel.com>
---
 net/ipv4/udp.c | 5 ++++-
 1 file changed, 4 insertions(+), 1 deletion(-)

diff --git a/net/ipv4/udp.c b/net/ipv4/udp.c
index 853321555a4e..89257422503a 100644
--- a/net/ipv4/udp.c
+++ b/net/ipv4/udp.c
@@ -926,6 +926,7 @@ int udp_sendmsg(struct sock *sk, struct msghdr *msg, size_t len)
 	}
 
 	ipc.sockc.tsflags = sk->sk_tsflags;
+	ipc.sockc.transmit_time = 0;
 	ipc.addr = inet->inet_saddr;
 	ipc.oif = sk->sk_bound_dev_if;
 
@@ -1027,8 +1028,10 @@ int udp_sendmsg(struct sock *sk, struct msghdr *msg, size_t len)
 				  sizeof(struct udphdr), &ipc, &rt,
 				  msg->msg_flags);
 		err = PTR_ERR(skb);
-		if (!IS_ERR_OR_NULL(skb))
+		if (!IS_ERR_OR_NULL(skb)) {
+			skb->tstamp = ipc.sockc.transmit_time;
 			err = udp_send_skb(skb, fl4);
+		}
 		goto out;
 	}
 
-- 
2.15.1


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

* [RFC v2 net-next 04/10] net: packet: Hook into time based transmission.
  2018-01-17 23:06 ` [Intel-wired-lan] " Jesus Sanchez-Palencia
@ 2018-01-17 23:06   ` Jesus Sanchez-Palencia
  -1 siblings, 0 replies; 86+ messages in thread
From: Jesus Sanchez-Palencia @ 2018-01-17 23:06 UTC (permalink / raw)
  To: netdev
  Cc: jhs, xiyou.wangcong, jiri, vinicius.gomes, richardcochran,
	intel-wired-lan, anna-maria, henrik, tglx, john.stultz,
	andre.guedes, ivan.briano, levi.pearson, Richard Cochran,
	Jesus Sanchez-Palencia

From: Richard Cochran <rcochran@linutronix.de>

For raw layer-2 packets, copy the desired future transmit time from
the CMSG cookie into the skb.

Signed-off-by: Richard Cochran <rcochran@linutronix.de>
Signed-off-by: Jesus Sanchez-Palencia <jesus.sanchez-palencia@intel.com>
---
 net/packet/af_packet.c | 6 ++++++
 1 file changed, 6 insertions(+)

diff --git a/net/packet/af_packet.c b/net/packet/af_packet.c
index 05d31864a34e..6af129a65e85 100644
--- a/net/packet/af_packet.c
+++ b/net/packet/af_packet.c
@@ -1976,6 +1976,7 @@ static int packet_sendmsg_spkt(struct socket *sock, struct msghdr *msg,
 		goto out_unlock;
 	}
 
+	sockc.transmit_time = 0;
 	sockc.tsflags = sk->sk_tsflags;
 	if (msg->msg_controllen) {
 		err = sock_cmsg_send(sk, msg, &sockc);
@@ -1987,6 +1988,7 @@ static int packet_sendmsg_spkt(struct socket *sock, struct msghdr *msg,
 	skb->dev = dev;
 	skb->priority = sk->sk_priority;
 	skb->mark = sk->sk_mark;
+	skb->tstamp = sockc.transmit_time;
 
 	sock_tx_timestamp(sk, sockc.tsflags, &skb_shinfo(skb)->tx_flags);
 
@@ -2484,6 +2486,7 @@ static int tpacket_fill_skb(struct packet_sock *po, struct sk_buff *skb,
 	skb->dev = dev;
 	skb->priority = po->sk.sk_priority;
 	skb->mark = po->sk.sk_mark;
+	skb->tstamp = sockc->transmit_time;
 	sock_tx_timestamp(&po->sk, sockc->tsflags, &skb_shinfo(skb)->tx_flags);
 	skb_shinfo(skb)->destructor_arg = ph.raw;
 
@@ -2660,6 +2663,7 @@ static int tpacket_snd(struct packet_sock *po, struct msghdr *msg)
 	if (unlikely(!(dev->flags & IFF_UP)))
 		goto out_put;
 
+	sockc.transmit_time = 0;
 	sockc.tsflags = po->sk.sk_tsflags;
 	if (msg->msg_controllen) {
 		err = sock_cmsg_send(&po->sk, msg, &sockc);
@@ -2856,6 +2860,7 @@ static int packet_snd(struct socket *sock, struct msghdr *msg, size_t len)
 	if (unlikely(!(dev->flags & IFF_UP)))
 		goto out_unlock;
 
+	sockc.transmit_time = 0;
 	sockc.tsflags = sk->sk_tsflags;
 	sockc.mark = sk->sk_mark;
 	if (msg->msg_controllen) {
@@ -2928,6 +2933,7 @@ static int packet_snd(struct socket *sock, struct msghdr *msg, size_t len)
 	skb->dev = dev;
 	skb->priority = sk->sk_priority;
 	skb->mark = sockc.mark;
+	skb->tstamp = sockc.transmit_time;
 
 	if (has_vnet_hdr) {
 		err = virtio_net_hdr_to_skb(skb, &vnet_hdr, vio_le());
-- 
2.15.1

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

* [Intel-wired-lan] [RFC v2 net-next 04/10] net: packet: Hook into time based transmission.
@ 2018-01-17 23:06   ` Jesus Sanchez-Palencia
  0 siblings, 0 replies; 86+ messages in thread
From: Jesus Sanchez-Palencia @ 2018-01-17 23:06 UTC (permalink / raw)
  To: intel-wired-lan

From: Richard Cochran <rcochran@linutronix.de>

For raw layer-2 packets, copy the desired future transmit time from
the CMSG cookie into the skb.

Signed-off-by: Richard Cochran <rcochran@linutronix.de>
Signed-off-by: Jesus Sanchez-Palencia <jesus.sanchez-palencia@intel.com>
---
 net/packet/af_packet.c | 6 ++++++
 1 file changed, 6 insertions(+)

diff --git a/net/packet/af_packet.c b/net/packet/af_packet.c
index 05d31864a34e..6af129a65e85 100644
--- a/net/packet/af_packet.c
+++ b/net/packet/af_packet.c
@@ -1976,6 +1976,7 @@ static int packet_sendmsg_spkt(struct socket *sock, struct msghdr *msg,
 		goto out_unlock;
 	}
 
+	sockc.transmit_time = 0;
 	sockc.tsflags = sk->sk_tsflags;
 	if (msg->msg_controllen) {
 		err = sock_cmsg_send(sk, msg, &sockc);
@@ -1987,6 +1988,7 @@ static int packet_sendmsg_spkt(struct socket *sock, struct msghdr *msg,
 	skb->dev = dev;
 	skb->priority = sk->sk_priority;
 	skb->mark = sk->sk_mark;
+	skb->tstamp = sockc.transmit_time;
 
 	sock_tx_timestamp(sk, sockc.tsflags, &skb_shinfo(skb)->tx_flags);
 
@@ -2484,6 +2486,7 @@ static int tpacket_fill_skb(struct packet_sock *po, struct sk_buff *skb,
 	skb->dev = dev;
 	skb->priority = po->sk.sk_priority;
 	skb->mark = po->sk.sk_mark;
+	skb->tstamp = sockc->transmit_time;
 	sock_tx_timestamp(&po->sk, sockc->tsflags, &skb_shinfo(skb)->tx_flags);
 	skb_shinfo(skb)->destructor_arg = ph.raw;
 
@@ -2660,6 +2663,7 @@ static int tpacket_snd(struct packet_sock *po, struct msghdr *msg)
 	if (unlikely(!(dev->flags & IFF_UP)))
 		goto out_put;
 
+	sockc.transmit_time = 0;
 	sockc.tsflags = po->sk.sk_tsflags;
 	if (msg->msg_controllen) {
 		err = sock_cmsg_send(&po->sk, msg, &sockc);
@@ -2856,6 +2860,7 @@ static int packet_snd(struct socket *sock, struct msghdr *msg, size_t len)
 	if (unlikely(!(dev->flags & IFF_UP)))
 		goto out_unlock;
 
+	sockc.transmit_time = 0;
 	sockc.tsflags = sk->sk_tsflags;
 	sockc.mark = sk->sk_mark;
 	if (msg->msg_controllen) {
@@ -2928,6 +2933,7 @@ static int packet_snd(struct socket *sock, struct msghdr *msg, size_t len)
 	skb->dev = dev;
 	skb->priority = sk->sk_priority;
 	skb->mark = sockc.mark;
+	skb->tstamp = sockc.transmit_time;
 
 	if (has_vnet_hdr) {
 		err = virtio_net_hdr_to_skb(skb, &vnet_hdr, vio_le());
-- 
2.15.1


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

* [RFC v2 net-next 05/10] net/sched: Allow creating a Qdisc watchdog with other clocks
  2018-01-17 23:06 ` [Intel-wired-lan] " Jesus Sanchez-Palencia
@ 2018-01-17 23:06   ` Jesus Sanchez-Palencia
  -1 siblings, 0 replies; 86+ messages in thread
From: Jesus Sanchez-Palencia @ 2018-01-17 23:06 UTC (permalink / raw)
  To: netdev
  Cc: jhs, xiyou.wangcong, jiri, vinicius.gomes, richardcochran,
	intel-wired-lan, anna-maria, henrik, tglx, john.stultz,
	andre.guedes, ivan.briano, levi.pearson

From: Vinicius Costa Gomes <vinicius.gomes@intel.com>

This adds 'qdisc_watchdog_init_clockid()' that allows a clockid to be
passed, this allows other time references to be used when scheduling
the Qdisc to run.

Signed-off-by: Vinicius Costa Gomes <vinicius.gomes@intel.com>
---
 include/net/pkt_sched.h |  2 ++
 net/sched/sch_api.c     | 11 +++++++++--
 2 files changed, 11 insertions(+), 2 deletions(-)

diff --git a/include/net/pkt_sched.h b/include/net/pkt_sched.h
index 815b92a23936..2466ea143d01 100644
--- a/include/net/pkt_sched.h
+++ b/include/net/pkt_sched.h
@@ -72,6 +72,8 @@ struct qdisc_watchdog {
 	struct Qdisc	*qdisc;
 };
 
+void qdisc_watchdog_init_clockid(struct qdisc_watchdog *wd, struct Qdisc *qdisc,
+				 clockid_t clockid);
 void qdisc_watchdog_init(struct qdisc_watchdog *wd, struct Qdisc *qdisc);
 void qdisc_watchdog_schedule_ns(struct qdisc_watchdog *wd, u64 expires);
 
diff --git a/net/sched/sch_api.c b/net/sched/sch_api.c
index d512f49ee83c..d6136920b5c1 100644
--- a/net/sched/sch_api.c
+++ b/net/sched/sch_api.c
@@ -596,12 +596,19 @@ static enum hrtimer_restart qdisc_watchdog(struct hrtimer *timer)
 	return HRTIMER_NORESTART;
 }
 
-void qdisc_watchdog_init(struct qdisc_watchdog *wd, struct Qdisc *qdisc)
+void qdisc_watchdog_init_clockid(struct qdisc_watchdog *wd, struct Qdisc *qdisc,
+				 clockid_t clockid)
 {
-	hrtimer_init(&wd->timer, CLOCK_MONOTONIC, HRTIMER_MODE_ABS_PINNED);
+	hrtimer_init(&wd->timer, clockid, HRTIMER_MODE_ABS_PINNED);
 	wd->timer.function = qdisc_watchdog;
 	wd->qdisc = qdisc;
 }
+EXPORT_SYMBOL(qdisc_watchdog_init_clockid);
+
+void qdisc_watchdog_init(struct qdisc_watchdog *wd, struct Qdisc *qdisc)
+{
+	qdisc_watchdog_init_clockid(wd, qdisc, CLOCK_MONOTONIC);
+}
 EXPORT_SYMBOL(qdisc_watchdog_init);
 
 void qdisc_watchdog_schedule_ns(struct qdisc_watchdog *wd, u64 expires)
-- 
2.15.1

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

* [Intel-wired-lan] [RFC v2 net-next 05/10] net/sched: Allow creating a Qdisc watchdog with other clocks
@ 2018-01-17 23:06   ` Jesus Sanchez-Palencia
  0 siblings, 0 replies; 86+ messages in thread
From: Jesus Sanchez-Palencia @ 2018-01-17 23:06 UTC (permalink / raw)
  To: intel-wired-lan

From: Vinicius Costa Gomes <vinicius.gomes@intel.com>

This adds 'qdisc_watchdog_init_clockid()' that allows a clockid to be
passed, this allows other time references to be used when scheduling
the Qdisc to run.

Signed-off-by: Vinicius Costa Gomes <vinicius.gomes@intel.com>
---
 include/net/pkt_sched.h |  2 ++
 net/sched/sch_api.c     | 11 +++++++++--
 2 files changed, 11 insertions(+), 2 deletions(-)

diff --git a/include/net/pkt_sched.h b/include/net/pkt_sched.h
index 815b92a23936..2466ea143d01 100644
--- a/include/net/pkt_sched.h
+++ b/include/net/pkt_sched.h
@@ -72,6 +72,8 @@ struct qdisc_watchdog {
 	struct Qdisc	*qdisc;
 };
 
+void qdisc_watchdog_init_clockid(struct qdisc_watchdog *wd, struct Qdisc *qdisc,
+				 clockid_t clockid);
 void qdisc_watchdog_init(struct qdisc_watchdog *wd, struct Qdisc *qdisc);
 void qdisc_watchdog_schedule_ns(struct qdisc_watchdog *wd, u64 expires);
 
diff --git a/net/sched/sch_api.c b/net/sched/sch_api.c
index d512f49ee83c..d6136920b5c1 100644
--- a/net/sched/sch_api.c
+++ b/net/sched/sch_api.c
@@ -596,12 +596,19 @@ static enum hrtimer_restart qdisc_watchdog(struct hrtimer *timer)
 	return HRTIMER_NORESTART;
 }
 
-void qdisc_watchdog_init(struct qdisc_watchdog *wd, struct Qdisc *qdisc)
+void qdisc_watchdog_init_clockid(struct qdisc_watchdog *wd, struct Qdisc *qdisc,
+				 clockid_t clockid)
 {
-	hrtimer_init(&wd->timer, CLOCK_MONOTONIC, HRTIMER_MODE_ABS_PINNED);
+	hrtimer_init(&wd->timer, clockid, HRTIMER_MODE_ABS_PINNED);
 	wd->timer.function = qdisc_watchdog;
 	wd->qdisc = qdisc;
 }
+EXPORT_SYMBOL(qdisc_watchdog_init_clockid);
+
+void qdisc_watchdog_init(struct qdisc_watchdog *wd, struct Qdisc *qdisc)
+{
+	qdisc_watchdog_init_clockid(wd, qdisc, CLOCK_MONOTONIC);
+}
 EXPORT_SYMBOL(qdisc_watchdog_init);
 
 void qdisc_watchdog_schedule_ns(struct qdisc_watchdog *wd, u64 expires)
-- 
2.15.1


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

* [RFC v2 net-next 06/10] net/sched: Introduce the TBS Qdisc
  2018-01-17 23:06 ` [Intel-wired-lan] " Jesus Sanchez-Palencia
@ 2018-01-17 23:06   ` Jesus Sanchez-Palencia
  -1 siblings, 0 replies; 86+ messages in thread
From: Jesus Sanchez-Palencia @ 2018-01-17 23:06 UTC (permalink / raw)
  To: netdev
  Cc: jhs, xiyou.wangcong, jiri, vinicius.gomes, richardcochran,
	intel-wired-lan, anna-maria, henrik, tglx, john.stultz,
	andre.guedes, ivan.briano, levi.pearson, Jesus Sanchez-Palencia

From: Vinicius Costa Gomes <vinicius.gomes@intel.com>

TBS (Time Based Scheduler) uses the information added earlier in this
series (the socket option SO_TXTIME and the new role of
sk_buff->tstamp) to schedule traffic transmission based on absolute
time.

For some workloads, just bandwidth enforcement is not enough, and
precise control of the transmission of packets is necessary.

Example:

$ tc qdisc replace dev enp2s0 parent root handle 100 mqprio num_tc 3 \
           map 2 2 1 0 2 2 2 2 2 2 2 2 2 2 2 2 queues 1@0 1@1 2@2 hw 0

$ tc qdisc add dev enp2s0 parent 100:1 tbs delta 60000 clockid 11 offload 1

In this example, the Qdisc will try to enable offloading (offload 1)
the control of the transmission time to the network adapter, the
time stamp in socket are in reference to the clockid '11' (CLOCK_TAI)
and packets leave the Qdisc "delta" (60000) nanoseconds before its
transmission time.

When offloading is disabled, the network adapter will ignore the
sk_buff time stamp, and so, the transmission time will be only "best
effort" from the Qdisc.

Signed-off-by: Vinicius Costa Gomes <vinicius.gomes@intel.com>
Signed-off-by: Jesus Sanchez-Palencia <jesus.sanchez-palencia@intel.com>
---
 include/linux/netdevice.h      |   1 +
 include/net/pkt_sched.h        |   5 +
 include/uapi/linux/pkt_sched.h |  17 ++
 net/sched/Kconfig              |  11 ++
 net/sched/Makefile             |   1 +
 net/sched/sch_tbs.c            | 392 +++++++++++++++++++++++++++++++++++++++++
 6 files changed, 427 insertions(+)
 create mode 100644 net/sched/sch_tbs.c

diff --git a/include/linux/netdevice.h b/include/linux/netdevice.h
index ed0799a12bf2..e87031bd108e 100644
--- a/include/linux/netdevice.h
+++ b/include/linux/netdevice.h
@@ -781,6 +781,7 @@ enum tc_setup_type {
 	TC_SETUP_QDISC_CBS,
 	TC_SETUP_QDISC_RED,
 	TC_SETUP_QDISC_PRIO,
+	TC_SETUP_QDISC_TBS,
 };
 
 /* These structures hold the attributes of bpf state that are being passed
diff --git a/include/net/pkt_sched.h b/include/net/pkt_sched.h
index 2466ea143d01..d042ffda7f21 100644
--- a/include/net/pkt_sched.h
+++ b/include/net/pkt_sched.h
@@ -155,4 +155,9 @@ struct tc_cbs_qopt_offload {
 	s32 sendslope;
 };
 
+struct tc_tbs_qopt_offload {
+	u8 enable;
+	s32 queue;
+};
+
 #endif
diff --git a/include/uapi/linux/pkt_sched.h b/include/uapi/linux/pkt_sched.h
index 37b5096ae97b..6bb39944ba32 100644
--- a/include/uapi/linux/pkt_sched.h
+++ b/include/uapi/linux/pkt_sched.h
@@ -934,4 +934,21 @@ enum {
 
 #define TCA_CBS_MAX (__TCA_CBS_MAX - 1)
 
+
+/* TBS */
+struct tc_tbs_qopt {
+	__u8 offload;
+	__u8 _pad[3];
+	__s32 delta;
+	__s32 clockid;
+};
+
+enum {
+	TCA_TBS_UNSPEC,
+	TCA_TBS_PARMS,
+	__TCA_TBS_MAX,
+};
+
+#define TCA_TBS_MAX (__TCA_TBS_MAX - 1)
+
 #endif
diff --git a/net/sched/Kconfig b/net/sched/Kconfig
index c03d86a7775e..7d54045995a3 100644
--- a/net/sched/Kconfig
+++ b/net/sched/Kconfig
@@ -183,6 +183,17 @@ config NET_SCH_CBS
 	  To compile this code as a module, choose M here: the
 	  module will be called sch_cbs.
 
+config NET_SCH_TBS
+	tristate "Time Based Scheduler (TBS)"
+	---help---
+	  Say Y here if you want to use the Time Based Scheduler (TBS) packet
+	  scheduling algorithm.
+
+	  See the top of <file:net/sched/sch_tbs.c> for more details.
+
+	  To compile this code as a module, choose M here: the
+	  module will be called sch_tbs.
+
 config NET_SCH_GRED
 	tristate "Generic Random Early Detection (GRED)"
 	---help---
diff --git a/net/sched/Makefile b/net/sched/Makefile
index 5b635447e3f8..0f7f29505c89 100644
--- a/net/sched/Makefile
+++ b/net/sched/Makefile
@@ -54,6 +54,7 @@ obj-$(CONFIG_NET_SCH_FQ)	+= sch_fq.o
 obj-$(CONFIG_NET_SCH_HHF)	+= sch_hhf.o
 obj-$(CONFIG_NET_SCH_PIE)	+= sch_pie.o
 obj-$(CONFIG_NET_SCH_CBS)	+= sch_cbs.o
+obj-$(CONFIG_NET_SCH_TBS)	+= sch_tbs.o
 
 obj-$(CONFIG_NET_CLS_U32)	+= cls_u32.o
 obj-$(CONFIG_NET_CLS_ROUTE4)	+= cls_route.o
diff --git a/net/sched/sch_tbs.c b/net/sched/sch_tbs.c
new file mode 100644
index 000000000000..300456063ac9
--- /dev/null
+++ b/net/sched/sch_tbs.c
@@ -0,0 +1,392 @@
+/*
+ * net/sched/sch_tbs.c	Time Based Shaper
+ *
+ *		This program is free software; you can redistribute it and/or
+ *		modify it under the terms of the GNU General Public License
+ *		as published by the Free Software Foundation; either version
+ *		2 of the License, or (at your option) any later version.
+ *
+ * Authors:	Vinicius Costa Gomes <vinicius.gomes@intel.com>
+ *		Jesus Sanchez-Palencia <jesus.sanchez-palencia@intel.com>
+ *
+ */
+
+#include <linux/module.h>
+#include <linux/types.h>
+#include <linux/kernel.h>
+#include <linux/string.h>
+#include <linux/errno.h>
+#include <linux/rbtree.h>
+#include <linux/skbuff.h>
+#include <net/netlink.h>
+#include <net/sch_generic.h>
+#include <net/pkt_sched.h>
+#include <net/pkt_sched.h>
+#include <net/sock.h>
+
+struct tbs_sched_data {
+	bool offload;
+	int clockid;
+	int queue;
+	s32 delta; /* in ns */
+	ktime_t last; /* The txtime of the last skb sent to the netdevice. */
+	struct rb_root head;
+	struct qdisc_watchdog watchdog;
+	struct Qdisc *qdisc;
+};
+
+static const struct nla_policy tbs_policy[TCA_TBS_MAX + 1] = {
+	[TCA_TBS_PARMS]	= { .len = sizeof(struct tc_tbs_qopt) },
+};
+
+typedef ktime_t (*get_time_func_t)(void);
+
+static const get_time_func_t clockid_to_get_time[MAX_CLOCKS] = {
+	[CLOCK_MONOTONIC] = ktime_get,
+	[CLOCK_REALTIME] = ktime_get_real,
+	[CLOCK_BOOTTIME] = ktime_get_boottime,
+	[CLOCK_TAI] = ktime_get_clocktai,
+};
+
+static ktime_t get_time_by_clockid(clockid_t clockid)
+{
+	get_time_func_t func = clockid_to_get_time[clockid];
+
+	if (!func)
+		return 0;
+
+	return func();
+}
+
+static struct sk_buff *tbs_peek(struct Qdisc *sch)
+{
+	struct tbs_sched_data *q = qdisc_priv(sch);
+	struct rb_node *p;
+
+	p = rb_first(&q->head);
+	if (!p)
+		return NULL;
+
+	return rb_to_skb(p);
+}
+
+static void reset_watchdog(struct Qdisc *sch)
+{
+	struct tbs_sched_data *q = qdisc_priv(sch);
+	struct sk_buff *skb = tbs_peek(sch);
+	ktime_t next;
+
+	if (!skb)
+		return;
+
+	next = ktime_sub_ns(skb->tstamp, q->delta);
+	qdisc_watchdog_schedule_ns(&q->watchdog, ktime_to_ns(next));
+}
+
+static int tbs_enqueue(struct sk_buff *nskb, struct Qdisc *sch,
+		       struct sk_buff **to_free)
+{
+	struct tbs_sched_data *q = qdisc_priv(sch);
+	struct rb_node **p = &q->head.rb_node, *parent = NULL;
+	ktime_t txtime = nskb->tstamp;
+	struct sock *sk = nskb->sk;
+	ktime_t now;
+
+	if (sk && !sock_flag(sk, SOCK_TXTIME))
+		goto drop;
+
+	now = get_time_by_clockid(q->clockid);
+
+	if (ktime_before(txtime, now) || ktime_before(txtime, q->last))
+		goto drop;
+
+	while (*p) {
+		struct sk_buff *skb;
+
+		parent = *p;
+		skb = rb_to_skb(parent);
+		if (ktime_after(txtime, skb->tstamp))
+			p = &parent->rb_right;
+		else
+			p = &parent->rb_left;
+	}
+	rb_link_node(&nskb->rbnode, parent, p);
+	rb_insert_color(&nskb->rbnode, &q->head);
+
+	qdisc_qstats_backlog_inc(sch, nskb);
+	sch->q.qlen++;
+
+	/* Now we may need to re-arm the qdisc watchdog for the next packet. */
+	reset_watchdog(sch);
+
+	return NET_XMIT_SUCCESS;
+
+drop:
+	return qdisc_drop(nskb, sch, to_free);
+}
+
+static struct sk_buff *timerqueue_erase(struct Qdisc *sch,
+					struct sk_buff *skb, bool drop)
+{
+	struct tbs_sched_data *q = qdisc_priv(sch);
+
+	rb_erase(&skb->rbnode, &q->head);
+
+	if (drop) {
+		struct sk_buff *to_free = NULL;
+
+		qdisc_drop(skb, sch, &to_free);
+		kfree_skb_list(to_free);
+	} else {
+		qdisc_qstats_backlog_dec(sch, skb);
+		qdisc_bstats_update(sch, skb);
+
+		q->last = skb->tstamp;
+	}
+
+	sch->q.qlen--;
+
+	/* The rbnode field in the skb re-uses these fields, now that
+	 * we are done with the rbnode, reset them.
+	 */
+	skb->next = NULL;
+	skb->prev = NULL;
+	skb->dev = qdisc_dev(sch);
+
+	return skb;
+}
+
+static struct sk_buff *tbs_dequeue(struct Qdisc *sch)
+{
+	struct tbs_sched_data *q = qdisc_priv(sch);
+	struct sk_buff *skb;
+	ktime_t now, next;
+
+	skb = tbs_peek(sch);
+	if (!skb)
+		return NULL;
+
+	now = get_time_by_clockid(q->clockid);
+
+	/* If packet has expired while in queue, drop it. */
+	if (ktime_before(skb->tstamp, now)) {
+		timerqueue_erase(sch, skb, true);
+		skb = NULL;
+		goto out;
+	}
+
+	next = ktime_sub_ns(skb->tstamp, q->delta);
+
+	/* Dequeue only if now is within the [txtime - delta, txtime] range. */
+	if (ktime_after(now, next))
+		timerqueue_erase(sch, skb, false);
+	else
+		skb = NULL;
+
+out:
+	/* Now we may need to re-arm the qdisc watchdog for the next packet. */
+	reset_watchdog(sch);
+
+	return skb;
+}
+
+static void tbs_disable_offload(struct net_device *dev,
+				struct tbs_sched_data *q)
+{
+	struct tc_tbs_qopt_offload tbs = { };
+	const struct net_device_ops *ops;
+	int err;
+
+	if (!q->offload)
+		return;
+
+	ops = dev->netdev_ops;
+	if (!ops->ndo_setup_tc)
+		return;
+
+	tbs.queue = q->queue;
+	tbs.enable = 0;
+
+	err = ops->ndo_setup_tc(dev, TC_SETUP_QDISC_TBS, &tbs);
+	if (err < 0)
+		pr_warn("Couldn't disable TBS offload for queue %d\n",
+			tbs.queue);
+}
+
+static int tbs_enable_offload(struct net_device *dev, struct tbs_sched_data *q,
+			      struct netlink_ext_ack *extack)
+{
+	const struct net_device_ops *ops = dev->netdev_ops;
+	struct tc_tbs_qopt_offload tbs = { };
+	int err;
+
+	if (q->offload)
+		return 0;
+
+	if (!ops->ndo_setup_tc) {
+		NL_SET_ERR_MSG(extack, "Specified device does not support TBS offload");
+		return -EOPNOTSUPP;
+	}
+
+	tbs.queue = q->queue;
+	tbs.enable = 1;
+
+	err = ops->ndo_setup_tc(dev, TC_SETUP_QDISC_TBS, &tbs);
+	if (err < 0) {
+		NL_SET_ERR_MSG(extack, "Specified device failed to setup TBS hardware offload");
+		return err;
+	}
+
+	return 0;
+}
+
+static int tbs_change(struct Qdisc *sch, struct nlattr *opt,
+		      struct netlink_ext_ack *extack)
+{
+	struct tbs_sched_data *q = qdisc_priv(sch);
+	struct net_device *dev = qdisc_dev(sch);
+	struct nlattr *tb[TCA_CBS_MAX + 1];
+	struct tc_tbs_qopt *qopt;
+	int err;
+
+	err = nla_parse_nested(tb, TCA_TBS_MAX, opt, tbs_policy, extack);
+	if (err < 0)
+		return err;
+
+	if (!tb[TCA_TBS_PARMS]) {
+		NL_SET_ERR_MSG(extack, "Missing mandatory TBS parameters");
+		return -EINVAL;
+	}
+
+	qopt = nla_data(tb[TCA_TBS_PARMS]);
+
+	if (qopt->clockid < 0 || qopt->clockid >= MAX_CLOCKS ||
+	    !clockid_to_get_time[qopt->clockid]) {
+		NL_SET_ERR_MSG(extack, "Invalid clockid");
+		return -EINVAL;
+	}
+
+	pr_debug("delta %d clockid %d offload %d\n",
+		 qopt->delta, qopt->clockid, qopt->offload);
+
+	if (!qopt->offload) {
+		tbs_disable_offload(dev, q);
+	} else {
+		err = tbs_enable_offload(dev, q, extack);
+		if (err < 0)
+			return err;
+	}
+
+	/* Everything went OK, save the parameters used. */
+	q->delta = qopt->delta;
+	q->clockid = qopt->clockid;
+	q->offload = qopt->offload;
+
+	qdisc_watchdog_init_clockid(&q->watchdog, sch, q->clockid);
+
+	return 0;
+}
+
+static int tbs_init(struct Qdisc *sch, struct nlattr *opt,
+		    struct netlink_ext_ack *extack)
+{
+	struct tbs_sched_data *q = qdisc_priv(sch);
+	struct net_device *dev = qdisc_dev(sch);
+
+	if (!opt) {
+		NL_SET_ERR_MSG(extack, "Missing TBS qdisc options which are mandatory");
+		return -EINVAL;
+	}
+
+	q->queue = sch->dev_queue - netdev_get_tx_queue(dev, 0);
+
+	return tbs_change(sch, opt, extack);
+}
+
+static void timerqueue_clear(struct Qdisc *sch)
+{
+	struct tbs_sched_data *q = qdisc_priv(sch);
+	struct rb_node *p = rb_first(&q->head);
+
+	while (p) {
+		struct sk_buff *skb = rb_to_skb(p);
+
+		p = rb_next(p);
+		rb_erase(&skb->rbnode, &q->head);
+		rtnl_kfree_skbs(skb, skb);
+	}
+}
+
+static void tbs_reset(struct Qdisc *sch)
+{
+	struct tbs_sched_data *q = qdisc_priv(sch);
+
+	qdisc_watchdog_cancel(&q->watchdog);
+	timerqueue_clear(sch);
+
+	sch->qstats.backlog = 0;
+	sch->q.qlen = 0;
+
+	q->last = 0;
+}
+
+static void tbs_destroy(struct Qdisc *sch)
+{
+	struct tbs_sched_data *q = qdisc_priv(sch);
+	struct net_device *dev = qdisc_dev(sch);
+
+	qdisc_watchdog_cancel(&q->watchdog);
+	timerqueue_clear(sch);
+	tbs_disable_offload(dev, q);
+}
+
+static int tbs_dump(struct Qdisc *sch, struct sk_buff *skb)
+{
+	struct tbs_sched_data *q = qdisc_priv(sch);
+	struct tc_tbs_qopt opt = { };
+	struct nlattr *nest;
+
+	nest = nla_nest_start(skb, TCA_OPTIONS);
+	if (!nest)
+		goto nla_put_failure;
+
+	opt.delta = q->delta;
+	opt.clockid = q->clockid;
+	opt.offload = q->offload;
+
+	if (nla_put(skb, TCA_TBS_PARMS, sizeof(opt), &opt))
+		goto nla_put_failure;
+
+	return nla_nest_end(skb, nest);
+
+nla_put_failure:
+	nla_nest_cancel(skb, nest);
+	return -1;
+}
+
+static struct Qdisc_ops tbs_qdisc_ops __read_mostly = {
+	.id		=	"tbs",
+	.priv_size	=	sizeof(struct tbs_sched_data),
+	.enqueue	=	tbs_enqueue,
+	.dequeue	=	tbs_dequeue,
+	.peek		=	tbs_peek,
+	.init		=	tbs_init,
+	.reset		=	tbs_reset,
+	.destroy	=	tbs_destroy,
+	.change		=	tbs_change,
+	.dump		=	tbs_dump,
+	.owner		=	THIS_MODULE,
+};
+
+static int __init tbs_module_init(void)
+{
+	return register_qdisc(&tbs_qdisc_ops);
+}
+
+static void __exit tbs_module_exit(void)
+{
+	unregister_qdisc(&tbs_qdisc_ops);
+}
+module_init(tbs_module_init)
+module_exit(tbs_module_exit)
+MODULE_LICENSE("GPL");
-- 
2.15.1

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

* [Intel-wired-lan] [RFC v2 net-next 06/10] net/sched: Introduce the TBS Qdisc
@ 2018-01-17 23:06   ` Jesus Sanchez-Palencia
  0 siblings, 0 replies; 86+ messages in thread
From: Jesus Sanchez-Palencia @ 2018-01-17 23:06 UTC (permalink / raw)
  To: intel-wired-lan

From: Vinicius Costa Gomes <vinicius.gomes@intel.com>

TBS (Time Based Scheduler) uses the information added earlier in this
series (the socket option SO_TXTIME and the new role of
sk_buff->tstamp) to schedule traffic transmission based on absolute
time.

For some workloads, just bandwidth enforcement is not enough, and
precise control of the transmission of packets is necessary.

Example:

$ tc qdisc replace dev enp2s0 parent root handle 100 mqprio num_tc 3 \
           map 2 2 1 0 2 2 2 2 2 2 2 2 2 2 2 2 queues 1 at 0 1 at 1 2 at 2 hw 0

$ tc qdisc add dev enp2s0 parent 100:1 tbs delta 60000 clockid 11 offload 1

In this example, the Qdisc will try to enable offloading (offload 1)
the control of the transmission time to the network adapter, the
time stamp in socket are in reference to the clockid '11' (CLOCK_TAI)
and packets leave the Qdisc "delta" (60000) nanoseconds before its
transmission time.

When offloading is disabled, the network adapter will ignore the
sk_buff time stamp, and so, the transmission time will be only "best
effort" from the Qdisc.

Signed-off-by: Vinicius Costa Gomes <vinicius.gomes@intel.com>
Signed-off-by: Jesus Sanchez-Palencia <jesus.sanchez-palencia@intel.com>
---
 include/linux/netdevice.h      |   1 +
 include/net/pkt_sched.h        |   5 +
 include/uapi/linux/pkt_sched.h |  17 ++
 net/sched/Kconfig              |  11 ++
 net/sched/Makefile             |   1 +
 net/sched/sch_tbs.c            | 392 +++++++++++++++++++++++++++++++++++++++++
 6 files changed, 427 insertions(+)
 create mode 100644 net/sched/sch_tbs.c

diff --git a/include/linux/netdevice.h b/include/linux/netdevice.h
index ed0799a12bf2..e87031bd108e 100644
--- a/include/linux/netdevice.h
+++ b/include/linux/netdevice.h
@@ -781,6 +781,7 @@ enum tc_setup_type {
 	TC_SETUP_QDISC_CBS,
 	TC_SETUP_QDISC_RED,
 	TC_SETUP_QDISC_PRIO,
+	TC_SETUP_QDISC_TBS,
 };
 
 /* These structures hold the attributes of bpf state that are being passed
diff --git a/include/net/pkt_sched.h b/include/net/pkt_sched.h
index 2466ea143d01..d042ffda7f21 100644
--- a/include/net/pkt_sched.h
+++ b/include/net/pkt_sched.h
@@ -155,4 +155,9 @@ struct tc_cbs_qopt_offload {
 	s32 sendslope;
 };
 
+struct tc_tbs_qopt_offload {
+	u8 enable;
+	s32 queue;
+};
+
 #endif
diff --git a/include/uapi/linux/pkt_sched.h b/include/uapi/linux/pkt_sched.h
index 37b5096ae97b..6bb39944ba32 100644
--- a/include/uapi/linux/pkt_sched.h
+++ b/include/uapi/linux/pkt_sched.h
@@ -934,4 +934,21 @@ enum {
 
 #define TCA_CBS_MAX (__TCA_CBS_MAX - 1)
 
+
+/* TBS */
+struct tc_tbs_qopt {
+	__u8 offload;
+	__u8 _pad[3];
+	__s32 delta;
+	__s32 clockid;
+};
+
+enum {
+	TCA_TBS_UNSPEC,
+	TCA_TBS_PARMS,
+	__TCA_TBS_MAX,
+};
+
+#define TCA_TBS_MAX (__TCA_TBS_MAX - 1)
+
 #endif
diff --git a/net/sched/Kconfig b/net/sched/Kconfig
index c03d86a7775e..7d54045995a3 100644
--- a/net/sched/Kconfig
+++ b/net/sched/Kconfig
@@ -183,6 +183,17 @@ config NET_SCH_CBS
 	  To compile this code as a module, choose M here: the
 	  module will be called sch_cbs.
 
+config NET_SCH_TBS
+	tristate "Time Based Scheduler (TBS)"
+	---help---
+	  Say Y here if you want to use the Time Based Scheduler (TBS) packet
+	  scheduling algorithm.
+
+	  See the top of <file:net/sched/sch_tbs.c> for more details.
+
+	  To compile this code as a module, choose M here: the
+	  module will be called sch_tbs.
+
 config NET_SCH_GRED
 	tristate "Generic Random Early Detection (GRED)"
 	---help---
diff --git a/net/sched/Makefile b/net/sched/Makefile
index 5b635447e3f8..0f7f29505c89 100644
--- a/net/sched/Makefile
+++ b/net/sched/Makefile
@@ -54,6 +54,7 @@ obj-$(CONFIG_NET_SCH_FQ)	+= sch_fq.o
 obj-$(CONFIG_NET_SCH_HHF)	+= sch_hhf.o
 obj-$(CONFIG_NET_SCH_PIE)	+= sch_pie.o
 obj-$(CONFIG_NET_SCH_CBS)	+= sch_cbs.o
+obj-$(CONFIG_NET_SCH_TBS)	+= sch_tbs.o
 
 obj-$(CONFIG_NET_CLS_U32)	+= cls_u32.o
 obj-$(CONFIG_NET_CLS_ROUTE4)	+= cls_route.o
diff --git a/net/sched/sch_tbs.c b/net/sched/sch_tbs.c
new file mode 100644
index 000000000000..300456063ac9
--- /dev/null
+++ b/net/sched/sch_tbs.c
@@ -0,0 +1,392 @@
+/*
+ * net/sched/sch_tbs.c	Time Based Shaper
+ *
+ *		This program is free software; you can redistribute it and/or
+ *		modify it under the terms of the GNU General Public License
+ *		as published by the Free Software Foundation; either version
+ *		2 of the License, or (at your option) any later version.
+ *
+ * Authors:	Vinicius Costa Gomes <vinicius.gomes@intel.com>
+ *		Jesus Sanchez-Palencia <jesus.sanchez-palencia@intel.com>
+ *
+ */
+
+#include <linux/module.h>
+#include <linux/types.h>
+#include <linux/kernel.h>
+#include <linux/string.h>
+#include <linux/errno.h>
+#include <linux/rbtree.h>
+#include <linux/skbuff.h>
+#include <net/netlink.h>
+#include <net/sch_generic.h>
+#include <net/pkt_sched.h>
+#include <net/pkt_sched.h>
+#include <net/sock.h>
+
+struct tbs_sched_data {
+	bool offload;
+	int clockid;
+	int queue;
+	s32 delta; /* in ns */
+	ktime_t last; /* The txtime of the last skb sent to the netdevice. */
+	struct rb_root head;
+	struct qdisc_watchdog watchdog;
+	struct Qdisc *qdisc;
+};
+
+static const struct nla_policy tbs_policy[TCA_TBS_MAX + 1] = {
+	[TCA_TBS_PARMS]	= { .len = sizeof(struct tc_tbs_qopt) },
+};
+
+typedef ktime_t (*get_time_func_t)(void);
+
+static const get_time_func_t clockid_to_get_time[MAX_CLOCKS] = {
+	[CLOCK_MONOTONIC] = ktime_get,
+	[CLOCK_REALTIME] = ktime_get_real,
+	[CLOCK_BOOTTIME] = ktime_get_boottime,
+	[CLOCK_TAI] = ktime_get_clocktai,
+};
+
+static ktime_t get_time_by_clockid(clockid_t clockid)
+{
+	get_time_func_t func = clockid_to_get_time[clockid];
+
+	if (!func)
+		return 0;
+
+	return func();
+}
+
+static struct sk_buff *tbs_peek(struct Qdisc *sch)
+{
+	struct tbs_sched_data *q = qdisc_priv(sch);
+	struct rb_node *p;
+
+	p = rb_first(&q->head);
+	if (!p)
+		return NULL;
+
+	return rb_to_skb(p);
+}
+
+static void reset_watchdog(struct Qdisc *sch)
+{
+	struct tbs_sched_data *q = qdisc_priv(sch);
+	struct sk_buff *skb = tbs_peek(sch);
+	ktime_t next;
+
+	if (!skb)
+		return;
+
+	next = ktime_sub_ns(skb->tstamp, q->delta);
+	qdisc_watchdog_schedule_ns(&q->watchdog, ktime_to_ns(next));
+}
+
+static int tbs_enqueue(struct sk_buff *nskb, struct Qdisc *sch,
+		       struct sk_buff **to_free)
+{
+	struct tbs_sched_data *q = qdisc_priv(sch);
+	struct rb_node **p = &q->head.rb_node, *parent = NULL;
+	ktime_t txtime = nskb->tstamp;
+	struct sock *sk = nskb->sk;
+	ktime_t now;
+
+	if (sk && !sock_flag(sk, SOCK_TXTIME))
+		goto drop;
+
+	now = get_time_by_clockid(q->clockid);
+
+	if (ktime_before(txtime, now) || ktime_before(txtime, q->last))
+		goto drop;
+
+	while (*p) {
+		struct sk_buff *skb;
+
+		parent = *p;
+		skb = rb_to_skb(parent);
+		if (ktime_after(txtime, skb->tstamp))
+			p = &parent->rb_right;
+		else
+			p = &parent->rb_left;
+	}
+	rb_link_node(&nskb->rbnode, parent, p);
+	rb_insert_color(&nskb->rbnode, &q->head);
+
+	qdisc_qstats_backlog_inc(sch, nskb);
+	sch->q.qlen++;
+
+	/* Now we may need to re-arm the qdisc watchdog for the next packet. */
+	reset_watchdog(sch);
+
+	return NET_XMIT_SUCCESS;
+
+drop:
+	return qdisc_drop(nskb, sch, to_free);
+}
+
+static struct sk_buff *timerqueue_erase(struct Qdisc *sch,
+					struct sk_buff *skb, bool drop)
+{
+	struct tbs_sched_data *q = qdisc_priv(sch);
+
+	rb_erase(&skb->rbnode, &q->head);
+
+	if (drop) {
+		struct sk_buff *to_free = NULL;
+
+		qdisc_drop(skb, sch, &to_free);
+		kfree_skb_list(to_free);
+	} else {
+		qdisc_qstats_backlog_dec(sch, skb);
+		qdisc_bstats_update(sch, skb);
+
+		q->last = skb->tstamp;
+	}
+
+	sch->q.qlen--;
+
+	/* The rbnode field in the skb re-uses these fields, now that
+	 * we are done with the rbnode, reset them.
+	 */
+	skb->next = NULL;
+	skb->prev = NULL;
+	skb->dev = qdisc_dev(sch);
+
+	return skb;
+}
+
+static struct sk_buff *tbs_dequeue(struct Qdisc *sch)
+{
+	struct tbs_sched_data *q = qdisc_priv(sch);
+	struct sk_buff *skb;
+	ktime_t now, next;
+
+	skb = tbs_peek(sch);
+	if (!skb)
+		return NULL;
+
+	now = get_time_by_clockid(q->clockid);
+
+	/* If packet has expired while in queue, drop it. */
+	if (ktime_before(skb->tstamp, now)) {
+		timerqueue_erase(sch, skb, true);
+		skb = NULL;
+		goto out;
+	}
+
+	next = ktime_sub_ns(skb->tstamp, q->delta);
+
+	/* Dequeue only if now is within the [txtime - delta, txtime] range. */
+	if (ktime_after(now, next))
+		timerqueue_erase(sch, skb, false);
+	else
+		skb = NULL;
+
+out:
+	/* Now we may need to re-arm the qdisc watchdog for the next packet. */
+	reset_watchdog(sch);
+
+	return skb;
+}
+
+static void tbs_disable_offload(struct net_device *dev,
+				struct tbs_sched_data *q)
+{
+	struct tc_tbs_qopt_offload tbs = { };
+	const struct net_device_ops *ops;
+	int err;
+
+	if (!q->offload)
+		return;
+
+	ops = dev->netdev_ops;
+	if (!ops->ndo_setup_tc)
+		return;
+
+	tbs.queue = q->queue;
+	tbs.enable = 0;
+
+	err = ops->ndo_setup_tc(dev, TC_SETUP_QDISC_TBS, &tbs);
+	if (err < 0)
+		pr_warn("Couldn't disable TBS offload for queue %d\n",
+			tbs.queue);
+}
+
+static int tbs_enable_offload(struct net_device *dev, struct tbs_sched_data *q,
+			      struct netlink_ext_ack *extack)
+{
+	const struct net_device_ops *ops = dev->netdev_ops;
+	struct tc_tbs_qopt_offload tbs = { };
+	int err;
+
+	if (q->offload)
+		return 0;
+
+	if (!ops->ndo_setup_tc) {
+		NL_SET_ERR_MSG(extack, "Specified device does not support TBS offload");
+		return -EOPNOTSUPP;
+	}
+
+	tbs.queue = q->queue;
+	tbs.enable = 1;
+
+	err = ops->ndo_setup_tc(dev, TC_SETUP_QDISC_TBS, &tbs);
+	if (err < 0) {
+		NL_SET_ERR_MSG(extack, "Specified device failed to setup TBS hardware offload");
+		return err;
+	}
+
+	return 0;
+}
+
+static int tbs_change(struct Qdisc *sch, struct nlattr *opt,
+		      struct netlink_ext_ack *extack)
+{
+	struct tbs_sched_data *q = qdisc_priv(sch);
+	struct net_device *dev = qdisc_dev(sch);
+	struct nlattr *tb[TCA_CBS_MAX + 1];
+	struct tc_tbs_qopt *qopt;
+	int err;
+
+	err = nla_parse_nested(tb, TCA_TBS_MAX, opt, tbs_policy, extack);
+	if (err < 0)
+		return err;
+
+	if (!tb[TCA_TBS_PARMS]) {
+		NL_SET_ERR_MSG(extack, "Missing mandatory TBS parameters");
+		return -EINVAL;
+	}
+
+	qopt = nla_data(tb[TCA_TBS_PARMS]);
+
+	if (qopt->clockid < 0 || qopt->clockid >= MAX_CLOCKS ||
+	    !clockid_to_get_time[qopt->clockid]) {
+		NL_SET_ERR_MSG(extack, "Invalid clockid");
+		return -EINVAL;
+	}
+
+	pr_debug("delta %d clockid %d offload %d\n",
+		 qopt->delta, qopt->clockid, qopt->offload);
+
+	if (!qopt->offload) {
+		tbs_disable_offload(dev, q);
+	} else {
+		err = tbs_enable_offload(dev, q, extack);
+		if (err < 0)
+			return err;
+	}
+
+	/* Everything went OK, save the parameters used. */
+	q->delta = qopt->delta;
+	q->clockid = qopt->clockid;
+	q->offload = qopt->offload;
+
+	qdisc_watchdog_init_clockid(&q->watchdog, sch, q->clockid);
+
+	return 0;
+}
+
+static int tbs_init(struct Qdisc *sch, struct nlattr *opt,
+		    struct netlink_ext_ack *extack)
+{
+	struct tbs_sched_data *q = qdisc_priv(sch);
+	struct net_device *dev = qdisc_dev(sch);
+
+	if (!opt) {
+		NL_SET_ERR_MSG(extack, "Missing TBS qdisc options which are mandatory");
+		return -EINVAL;
+	}
+
+	q->queue = sch->dev_queue - netdev_get_tx_queue(dev, 0);
+
+	return tbs_change(sch, opt, extack);
+}
+
+static void timerqueue_clear(struct Qdisc *sch)
+{
+	struct tbs_sched_data *q = qdisc_priv(sch);
+	struct rb_node *p = rb_first(&q->head);
+
+	while (p) {
+		struct sk_buff *skb = rb_to_skb(p);
+
+		p = rb_next(p);
+		rb_erase(&skb->rbnode, &q->head);
+		rtnl_kfree_skbs(skb, skb);
+	}
+}
+
+static void tbs_reset(struct Qdisc *sch)
+{
+	struct tbs_sched_data *q = qdisc_priv(sch);
+
+	qdisc_watchdog_cancel(&q->watchdog);
+	timerqueue_clear(sch);
+
+	sch->qstats.backlog = 0;
+	sch->q.qlen = 0;
+
+	q->last = 0;
+}
+
+static void tbs_destroy(struct Qdisc *sch)
+{
+	struct tbs_sched_data *q = qdisc_priv(sch);
+	struct net_device *dev = qdisc_dev(sch);
+
+	qdisc_watchdog_cancel(&q->watchdog);
+	timerqueue_clear(sch);
+	tbs_disable_offload(dev, q);
+}
+
+static int tbs_dump(struct Qdisc *sch, struct sk_buff *skb)
+{
+	struct tbs_sched_data *q = qdisc_priv(sch);
+	struct tc_tbs_qopt opt = { };
+	struct nlattr *nest;
+
+	nest = nla_nest_start(skb, TCA_OPTIONS);
+	if (!nest)
+		goto nla_put_failure;
+
+	opt.delta = q->delta;
+	opt.clockid = q->clockid;
+	opt.offload = q->offload;
+
+	if (nla_put(skb, TCA_TBS_PARMS, sizeof(opt), &opt))
+		goto nla_put_failure;
+
+	return nla_nest_end(skb, nest);
+
+nla_put_failure:
+	nla_nest_cancel(skb, nest);
+	return -1;
+}
+
+static struct Qdisc_ops tbs_qdisc_ops __read_mostly = {
+	.id		=	"tbs",
+	.priv_size	=	sizeof(struct tbs_sched_data),
+	.enqueue	=	tbs_enqueue,
+	.dequeue	=	tbs_dequeue,
+	.peek		=	tbs_peek,
+	.init		=	tbs_init,
+	.reset		=	tbs_reset,
+	.destroy	=	tbs_destroy,
+	.change		=	tbs_change,
+	.dump		=	tbs_dump,
+	.owner		=	THIS_MODULE,
+};
+
+static int __init tbs_module_init(void)
+{
+	return register_qdisc(&tbs_qdisc_ops);
+}
+
+static void __exit tbs_module_exit(void)
+{
+	unregister_qdisc(&tbs_qdisc_ops);
+}
+module_init(tbs_module_init)
+module_exit(tbs_module_exit)
+MODULE_LICENSE("GPL");
-- 
2.15.1


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

* [RFC v2 net-next 07/10] igb: Refactor igb_configure_cbs()
  2018-01-17 23:06 ` [Intel-wired-lan] " Jesus Sanchez-Palencia
@ 2018-01-17 23:06   ` Jesus Sanchez-Palencia
  -1 siblings, 0 replies; 86+ messages in thread
From: Jesus Sanchez-Palencia @ 2018-01-17 23:06 UTC (permalink / raw)
  To: netdev
  Cc: jhs, xiyou.wangcong, jiri, vinicius.gomes, richardcochran,
	intel-wired-lan, anna-maria, henrik, tglx, john.stultz,
	andre.guedes, ivan.briano, levi.pearson, Jesus Sanchez-Palencia

Make this function retrieve what it needs from the Tx ring being
addressed since it already relies on what had been saved on it before.
Also, since this function will be used by the upcoming Launchtime
patches rename it to better reflect its intention. Note that
Launchtime is not part of what 802.1Qav specifies, but the i210
datasheet refers to this set of functionality as "Qav Transmission
Mode".

Here we also perform a tiny refactor at is_any_cbs_enabled(), and add
further documentation to igb_setup_tx_mode().

Signed-off-by: Jesus Sanchez-Palencia <jesus.sanchez-palencia@intel.com>
---
 drivers/net/ethernet/intel/igb/igb_main.c | 54 ++++++++++++++-----------------
 1 file changed, 25 insertions(+), 29 deletions(-)

diff --git a/drivers/net/ethernet/intel/igb/igb_main.c b/drivers/net/ethernet/intel/igb/igb_main.c
index c208753ff5b7..dd19c325a85c 100644
--- a/drivers/net/ethernet/intel/igb/igb_main.c
+++ b/drivers/net/ethernet/intel/igb/igb_main.c
@@ -1673,23 +1673,17 @@ static void set_queue_mode(struct e1000_hw *hw, int queue, enum queue_mode mode)
 }
 
 /**
- *  igb_configure_cbs - Configure Credit-Based Shaper (CBS)
+ *  igb_config_tx_modes - Configure "Qav Tx mode" features on igb
  *  @adapter: pointer to adapter struct
  *  @queue: queue number
- *  @enable: true = enable CBS, false = disable CBS
- *  @idleslope: idleSlope in kbps
- *  @sendslope: sendSlope in kbps
- *  @hicredit: hiCredit in bytes
- *  @locredit: loCredit in bytes
  *
- *  Configure CBS for a given hardware queue. When disabling, idleslope,
- *  sendslope, hicredit, locredit arguments are ignored. Returns 0 if
- *  success. Negative otherwise.
+ *  Configure CBS for a given hardware queue. Parameters are retrieved
+ *  from the correct Tx ring, so igb_save_cbs_params() should be used
+ *  for setting those correctly prior to this function being called.
  **/
-static void igb_configure_cbs(struct igb_adapter *adapter, int queue,
-			      bool enable, int idleslope, int sendslope,
-			      int hicredit, int locredit)
+static void igb_config_tx_modes(struct igb_adapter *adapter, int queue)
 {
+	struct igb_ring *ring = adapter->tx_ring[queue];
 	struct net_device *netdev = adapter->netdev;
 	struct e1000_hw *hw = &adapter->hw;
 	u32 tqavcc;
@@ -1698,7 +1692,7 @@ static void igb_configure_cbs(struct igb_adapter *adapter, int queue,
 	WARN_ON(hw->mac.type != e1000_i210);
 	WARN_ON(queue < 0 || queue > 1);
 
-	if (enable) {
+	if (ring->cbs_enable) {
 		set_tx_desc_fetch_prio(hw, queue, TX_QUEUE_PRIO_HIGH);
 		set_queue_mode(hw, queue, QUEUE_MODE_STREAM_RESERVATION);
 
@@ -1745,14 +1739,15 @@ static void igb_configure_cbs(struct igb_adapter *adapter, int queue,
 		 *             -----------------                          (E6)
 		 *                  1000000
 		 */
-		value = DIV_ROUND_UP_ULL(idleslope * 61034ULL, 1000000);
+		value = DIV_ROUND_UP_ULL(ring->idleslope * 61034ULL, 1000000);
 
 		tqavcc = rd32(E1000_I210_TQAVCC(queue));
 		tqavcc &= ~E1000_TQAVCC_IDLESLOPE_MASK;
 		tqavcc |= value;
 		wr32(E1000_I210_TQAVCC(queue), tqavcc);
 
-		wr32(E1000_I210_TQAVHC(queue), 0x80000000 + hicredit * 0x7735);
+		wr32(E1000_I210_TQAVHC(queue),
+		     0x80000000 + ring->hicredit * 0x7735);
 	} else {
 		set_tx_desc_fetch_prio(hw, queue, TX_QUEUE_PRIO_LOW);
 		set_queue_mode(hw, queue, QUEUE_MODE_STRICT_PRIORITY);
@@ -1772,8 +1767,9 @@ static void igb_configure_cbs(struct igb_adapter *adapter, int queue,
 	 */
 
 	netdev_dbg(netdev, "CBS %s: queue %d idleslope %d sendslope %d hiCredit %d locredit %d\n",
-		   (enable) ? "enabled" : "disabled", queue,
-		   idleslope, sendslope, hicredit, locredit);
+		   (ring->cbs_enable) ? "enabled" : "disabled", queue,
+		   ring->idleslope, ring->sendslope, ring->hicredit,
+		   ring->locredit);
 }
 
 static int igb_save_cbs_params(struct igb_adapter *adapter, int queue,
@@ -1798,19 +1794,25 @@ static int igb_save_cbs_params(struct igb_adapter *adapter, int queue,
 
 static bool is_any_cbs_enabled(struct igb_adapter *adapter)
 {
-	struct igb_ring *ring;
 	int i;
 
 	for (i = 0; i < adapter->num_tx_queues; i++) {
-		ring = adapter->tx_ring[i];
-
-		if (ring->cbs_enable)
+		if (adapter->tx_ring[i]->cbs_enable)
 			return true;
 	}
 
 	return false;
 }
 
+/**
+ *  igb_setup_tx_mode - Switch to/from Qav Tx mode when applicable
+ *  @adapter: pointer to adapter struct
+ *
+ *  Configure TQAVCTRL register switching the controller's Tx mode
+ *  if FQTSS mode is enabled or disabled. Additionally, will issue
+ *  a call to igb_config_tx_modes() per queue so any previously saved
+ *  Tx parameters are applied.
+ **/
 static void igb_setup_tx_mode(struct igb_adapter *adapter)
 {
 	struct net_device *netdev = adapter->netdev;
@@ -1870,11 +1872,7 @@ static void igb_setup_tx_mode(struct igb_adapter *adapter)
 			    adapter->num_tx_queues : I210_SR_QUEUES_NUM;
 
 		for (i = 0; i < max_queue; i++) {
-			struct igb_ring *ring = adapter->tx_ring[i];
-
-			igb_configure_cbs(adapter, i, ring->cbs_enable,
-					  ring->idleslope, ring->sendslope,
-					  ring->hicredit, ring->locredit);
+			igb_config_tx_modes(adapter, i);
 		}
 	} else {
 		wr32(E1000_RXPBS, I210_RXPBSIZE_DEFAULT);
@@ -2468,9 +2466,7 @@ static int igb_offload_cbs(struct igb_adapter *adapter,
 		return err;
 
 	if (is_fqtss_enabled(adapter)) {
-		igb_configure_cbs(adapter, qopt->queue, qopt->enable,
-				  qopt->idleslope, qopt->sendslope,
-				  qopt->hicredit, qopt->locredit);
+		igb_config_tx_modes(adapter, qopt->queue);
 
 		if (!is_any_cbs_enabled(adapter))
 			enable_fqtss(adapter, false);
-- 
2.15.1

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

* [Intel-wired-lan] [RFC v2 net-next 07/10] igb: Refactor igb_configure_cbs()
@ 2018-01-17 23:06   ` Jesus Sanchez-Palencia
  0 siblings, 0 replies; 86+ messages in thread
From: Jesus Sanchez-Palencia @ 2018-01-17 23:06 UTC (permalink / raw)
  To: intel-wired-lan

Make this function retrieve what it needs from the Tx ring being
addressed since it already relies on what had been saved on it before.
Also, since this function will be used by the upcoming Launchtime
patches rename it to better reflect its intention. Note that
Launchtime is not part of what 802.1Qav specifies, but the i210
datasheet refers to this set of functionality as "Qav Transmission
Mode".

Here we also perform a tiny refactor at is_any_cbs_enabled(), and add
further documentation to igb_setup_tx_mode().

Signed-off-by: Jesus Sanchez-Palencia <jesus.sanchez-palencia@intel.com>
---
 drivers/net/ethernet/intel/igb/igb_main.c | 54 ++++++++++++++-----------------
 1 file changed, 25 insertions(+), 29 deletions(-)

diff --git a/drivers/net/ethernet/intel/igb/igb_main.c b/drivers/net/ethernet/intel/igb/igb_main.c
index c208753ff5b7..dd19c325a85c 100644
--- a/drivers/net/ethernet/intel/igb/igb_main.c
+++ b/drivers/net/ethernet/intel/igb/igb_main.c
@@ -1673,23 +1673,17 @@ static void set_queue_mode(struct e1000_hw *hw, int queue, enum queue_mode mode)
 }
 
 /**
- *  igb_configure_cbs - Configure Credit-Based Shaper (CBS)
+ *  igb_config_tx_modes - Configure "Qav Tx mode" features on igb
  *  @adapter: pointer to adapter struct
  *  @queue: queue number
- *  @enable: true = enable CBS, false = disable CBS
- *  @idleslope: idleSlope in kbps
- *  @sendslope: sendSlope in kbps
- *  @hicredit: hiCredit in bytes
- *  @locredit: loCredit in bytes
  *
- *  Configure CBS for a given hardware queue. When disabling, idleslope,
- *  sendslope, hicredit, locredit arguments are ignored. Returns 0 if
- *  success. Negative otherwise.
+ *  Configure CBS for a given hardware queue. Parameters are retrieved
+ *  from the correct Tx ring, so igb_save_cbs_params() should be used
+ *  for setting those correctly prior to this function being called.
  **/
-static void igb_configure_cbs(struct igb_adapter *adapter, int queue,
-			      bool enable, int idleslope, int sendslope,
-			      int hicredit, int locredit)
+static void igb_config_tx_modes(struct igb_adapter *adapter, int queue)
 {
+	struct igb_ring *ring = adapter->tx_ring[queue];
 	struct net_device *netdev = adapter->netdev;
 	struct e1000_hw *hw = &adapter->hw;
 	u32 tqavcc;
@@ -1698,7 +1692,7 @@ static void igb_configure_cbs(struct igb_adapter *adapter, int queue,
 	WARN_ON(hw->mac.type != e1000_i210);
 	WARN_ON(queue < 0 || queue > 1);
 
-	if (enable) {
+	if (ring->cbs_enable) {
 		set_tx_desc_fetch_prio(hw, queue, TX_QUEUE_PRIO_HIGH);
 		set_queue_mode(hw, queue, QUEUE_MODE_STREAM_RESERVATION);
 
@@ -1745,14 +1739,15 @@ static void igb_configure_cbs(struct igb_adapter *adapter, int queue,
 		 *             -----------------                          (E6)
 		 *                  1000000
 		 */
-		value = DIV_ROUND_UP_ULL(idleslope * 61034ULL, 1000000);
+		value = DIV_ROUND_UP_ULL(ring->idleslope * 61034ULL, 1000000);
 
 		tqavcc = rd32(E1000_I210_TQAVCC(queue));
 		tqavcc &= ~E1000_TQAVCC_IDLESLOPE_MASK;
 		tqavcc |= value;
 		wr32(E1000_I210_TQAVCC(queue), tqavcc);
 
-		wr32(E1000_I210_TQAVHC(queue), 0x80000000 + hicredit * 0x7735);
+		wr32(E1000_I210_TQAVHC(queue),
+		     0x80000000 + ring->hicredit * 0x7735);
 	} else {
 		set_tx_desc_fetch_prio(hw, queue, TX_QUEUE_PRIO_LOW);
 		set_queue_mode(hw, queue, QUEUE_MODE_STRICT_PRIORITY);
@@ -1772,8 +1767,9 @@ static void igb_configure_cbs(struct igb_adapter *adapter, int queue,
 	 */
 
 	netdev_dbg(netdev, "CBS %s: queue %d idleslope %d sendslope %d hiCredit %d locredit %d\n",
-		   (enable) ? "enabled" : "disabled", queue,
-		   idleslope, sendslope, hicredit, locredit);
+		   (ring->cbs_enable) ? "enabled" : "disabled", queue,
+		   ring->idleslope, ring->sendslope, ring->hicredit,
+		   ring->locredit);
 }
 
 static int igb_save_cbs_params(struct igb_adapter *adapter, int queue,
@@ -1798,19 +1794,25 @@ static int igb_save_cbs_params(struct igb_adapter *adapter, int queue,
 
 static bool is_any_cbs_enabled(struct igb_adapter *adapter)
 {
-	struct igb_ring *ring;
 	int i;
 
 	for (i = 0; i < adapter->num_tx_queues; i++) {
-		ring = adapter->tx_ring[i];
-
-		if (ring->cbs_enable)
+		if (adapter->tx_ring[i]->cbs_enable)
 			return true;
 	}
 
 	return false;
 }
 
+/**
+ *  igb_setup_tx_mode - Switch to/from Qav Tx mode when applicable
+ *  @adapter: pointer to adapter struct
+ *
+ *  Configure TQAVCTRL register switching the controller's Tx mode
+ *  if FQTSS mode is enabled or disabled. Additionally, will issue
+ *  a call to igb_config_tx_modes() per queue so any previously saved
+ *  Tx parameters are applied.
+ **/
 static void igb_setup_tx_mode(struct igb_adapter *adapter)
 {
 	struct net_device *netdev = adapter->netdev;
@@ -1870,11 +1872,7 @@ static void igb_setup_tx_mode(struct igb_adapter *adapter)
 			    adapter->num_tx_queues : I210_SR_QUEUES_NUM;
 
 		for (i = 0; i < max_queue; i++) {
-			struct igb_ring *ring = adapter->tx_ring[i];
-
-			igb_configure_cbs(adapter, i, ring->cbs_enable,
-					  ring->idleslope, ring->sendslope,
-					  ring->hicredit, ring->locredit);
+			igb_config_tx_modes(adapter, i);
 		}
 	} else {
 		wr32(E1000_RXPBS, I210_RXPBSIZE_DEFAULT);
@@ -2468,9 +2466,7 @@ static int igb_offload_cbs(struct igb_adapter *adapter,
 		return err;
 
 	if (is_fqtss_enabled(adapter)) {
-		igb_configure_cbs(adapter, qopt->queue, qopt->enable,
-				  qopt->idleslope, qopt->sendslope,
-				  qopt->hicredit, qopt->locredit);
+		igb_config_tx_modes(adapter, qopt->queue);
 
 		if (!is_any_cbs_enabled(adapter))
 			enable_fqtss(adapter, false);
-- 
2.15.1


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

* [RFC v2 net-next 08/10] igb: Only change Tx arbitration when CBS is on
  2018-01-17 23:06 ` [Intel-wired-lan] " Jesus Sanchez-Palencia
@ 2018-01-17 23:06   ` Jesus Sanchez-Palencia
  -1 siblings, 0 replies; 86+ messages in thread
From: Jesus Sanchez-Palencia @ 2018-01-17 23:06 UTC (permalink / raw)
  To: netdev
  Cc: jhs, xiyou.wangcong, jiri, vinicius.gomes, richardcochran,
	intel-wired-lan, anna-maria, henrik, tglx, john.stultz,
	andre.guedes, ivan.briano, levi.pearson, Jesus Sanchez-Palencia

Currently the data transmission arbitration algorithm - DataTranARB
field on TQAVCTRL reg - is always set to CBS when the Tx mode is
changed from legacy to 'Qav' mode.

Make that configuration a bit more granular in preparation for the
upcoming Launchtime enabling patches, since CBS and Launchtime can be
enabled separately. That is achieved by moving the DataTranARB setup
to igb_config_tx_modes() instead.

Similarly, when disabling CBS we must check if it has been disabled
for all queues, and clear the DataTranARB accordingly.

Signed-off-by: Jesus Sanchez-Palencia <jesus.sanchez-palencia@intel.com>
---
 drivers/net/ethernet/intel/igb/igb_main.c | 49 +++++++++++++++++++++----------
 1 file changed, 33 insertions(+), 16 deletions(-)

diff --git a/drivers/net/ethernet/intel/igb/igb_main.c b/drivers/net/ethernet/intel/igb/igb_main.c
index dd19c325a85c..a955e1723ed5 100644
--- a/drivers/net/ethernet/intel/igb/igb_main.c
+++ b/drivers/net/ethernet/intel/igb/igb_main.c
@@ -1672,6 +1672,18 @@ static void set_queue_mode(struct e1000_hw *hw, int queue, enum queue_mode mode)
 	wr32(E1000_I210_TQAVCC(queue), val);
 }
 
+static bool is_any_cbs_enabled(struct igb_adapter *adapter)
+{
+	int i;
+
+	for (i = 0; i < adapter->num_tx_queues; i++) {
+		if (adapter->tx_ring[i]->cbs_enable)
+			return true;
+	}
+
+	return false;
+}
+
 /**
  *  igb_config_tx_modes - Configure "Qav Tx mode" features on igb
  *  @adapter: pointer to adapter struct
@@ -1686,7 +1698,7 @@ static void igb_config_tx_modes(struct igb_adapter *adapter, int queue)
 	struct igb_ring *ring = adapter->tx_ring[queue];
 	struct net_device *netdev = adapter->netdev;
 	struct e1000_hw *hw = &adapter->hw;
-	u32 tqavcc;
+	u32 tqavcc, tqavctrl;
 	u16 value;
 
 	WARN_ON(hw->mac.type != e1000_i210);
@@ -1696,6 +1708,14 @@ static void igb_config_tx_modes(struct igb_adapter *adapter, int queue)
 		set_tx_desc_fetch_prio(hw, queue, TX_QUEUE_PRIO_HIGH);
 		set_queue_mode(hw, queue, QUEUE_MODE_STREAM_RESERVATION);
 
+		/* Always set data transfer arbitration to credit-based
+		 * shaper algorithm on TQAVCTRL if CBS is enabled for any of
+		 * the queues.
+		 */
+		tqavctrl = rd32(E1000_I210_TQAVCTRL);
+		tqavctrl |= E1000_TQAVCTRL_DATATRANARB;
+		wr32(E1000_I210_TQAVCTRL, tqavctrl);
+
 		/* According to i210 datasheet section 7.2.7.7, we should set
 		 * the 'idleSlope' field from TQAVCC register following the
 		 * equation:
@@ -1759,6 +1779,16 @@ static void igb_config_tx_modes(struct igb_adapter *adapter, int queue)
 
 		/* Set hiCredit to zero. */
 		wr32(E1000_I210_TQAVHC(queue), 0);
+
+		/* If CBS is not enabled for any queues anymore, then return to
+		 * the default state of Data Transmission Arbitration on
+		 * TQAVCTRL.
+		 */
+		if (!is_any_cbs_enabled(adapter)) {
+			tqavctrl = rd32(E1000_I210_TQAVCTRL);
+			tqavctrl &= ~E1000_TQAVCTRL_DATATRANARB;
+			wr32(E1000_I210_TQAVCTRL, tqavctrl);
+		}
 	}
 
 	/* XXX: In i210 controller the sendSlope and loCredit parameters from
@@ -1792,18 +1822,6 @@ static int igb_save_cbs_params(struct igb_adapter *adapter, int queue,
 	return 0;
 }
 
-static bool is_any_cbs_enabled(struct igb_adapter *adapter)
-{
-	int i;
-
-	for (i = 0; i < adapter->num_tx_queues; i++) {
-		if (adapter->tx_ring[i]->cbs_enable)
-			return true;
-	}
-
-	return false;
-}
-
 /**
  *  igb_setup_tx_mode - Switch to/from Qav Tx mode when applicable
  *  @adapter: pointer to adapter struct
@@ -1827,11 +1845,10 @@ static void igb_setup_tx_mode(struct igb_adapter *adapter)
 		int i, max_queue;
 
 		/* Configure TQAVCTRL register: set transmit mode to 'Qav',
-		 * set data fetch arbitration to 'round robin' and set data
-		 * transfer arbitration to 'credit shaper algorithm.
+		 * set data fetch arbitration to 'round robin'.
 		 */
 		val = rd32(E1000_I210_TQAVCTRL);
-		val |= E1000_TQAVCTRL_XMIT_MODE | E1000_TQAVCTRL_DATATRANARB;
+		val |= E1000_TQAVCTRL_XMIT_MODE;
 		val &= ~E1000_TQAVCTRL_DATAFETCHARB;
 		wr32(E1000_I210_TQAVCTRL, val);
 
-- 
2.15.1

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

* [Intel-wired-lan] [RFC v2 net-next 08/10] igb: Only change Tx arbitration when CBS is on
@ 2018-01-17 23:06   ` Jesus Sanchez-Palencia
  0 siblings, 0 replies; 86+ messages in thread
From: Jesus Sanchez-Palencia @ 2018-01-17 23:06 UTC (permalink / raw)
  To: intel-wired-lan

Currently the data transmission arbitration algorithm - DataTranARB
field on TQAVCTRL reg - is always set to CBS when the Tx mode is
changed from legacy to 'Qav' mode.

Make that configuration a bit more granular in preparation for the
upcoming Launchtime enabling patches, since CBS and Launchtime can be
enabled separately. That is achieved by moving the DataTranARB setup
to igb_config_tx_modes() instead.

Similarly, when disabling CBS we must check if it has been disabled
for all queues, and clear the DataTranARB accordingly.

Signed-off-by: Jesus Sanchez-Palencia <jesus.sanchez-palencia@intel.com>
---
 drivers/net/ethernet/intel/igb/igb_main.c | 49 +++++++++++++++++++++----------
 1 file changed, 33 insertions(+), 16 deletions(-)

diff --git a/drivers/net/ethernet/intel/igb/igb_main.c b/drivers/net/ethernet/intel/igb/igb_main.c
index dd19c325a85c..a955e1723ed5 100644
--- a/drivers/net/ethernet/intel/igb/igb_main.c
+++ b/drivers/net/ethernet/intel/igb/igb_main.c
@@ -1672,6 +1672,18 @@ static void set_queue_mode(struct e1000_hw *hw, int queue, enum queue_mode mode)
 	wr32(E1000_I210_TQAVCC(queue), val);
 }
 
+static bool is_any_cbs_enabled(struct igb_adapter *adapter)
+{
+	int i;
+
+	for (i = 0; i < adapter->num_tx_queues; i++) {
+		if (adapter->tx_ring[i]->cbs_enable)
+			return true;
+	}
+
+	return false;
+}
+
 /**
  *  igb_config_tx_modes - Configure "Qav Tx mode" features on igb
  *  @adapter: pointer to adapter struct
@@ -1686,7 +1698,7 @@ static void igb_config_tx_modes(struct igb_adapter *adapter, int queue)
 	struct igb_ring *ring = adapter->tx_ring[queue];
 	struct net_device *netdev = adapter->netdev;
 	struct e1000_hw *hw = &adapter->hw;
-	u32 tqavcc;
+	u32 tqavcc, tqavctrl;
 	u16 value;
 
 	WARN_ON(hw->mac.type != e1000_i210);
@@ -1696,6 +1708,14 @@ static void igb_config_tx_modes(struct igb_adapter *adapter, int queue)
 		set_tx_desc_fetch_prio(hw, queue, TX_QUEUE_PRIO_HIGH);
 		set_queue_mode(hw, queue, QUEUE_MODE_STREAM_RESERVATION);
 
+		/* Always set data transfer arbitration to credit-based
+		 * shaper algorithm on TQAVCTRL if CBS is enabled for any of
+		 * the queues.
+		 */
+		tqavctrl = rd32(E1000_I210_TQAVCTRL);
+		tqavctrl |= E1000_TQAVCTRL_DATATRANARB;
+		wr32(E1000_I210_TQAVCTRL, tqavctrl);
+
 		/* According to i210 datasheet section 7.2.7.7, we should set
 		 * the 'idleSlope' field from TQAVCC register following the
 		 * equation:
@@ -1759,6 +1779,16 @@ static void igb_config_tx_modes(struct igb_adapter *adapter, int queue)
 
 		/* Set hiCredit to zero. */
 		wr32(E1000_I210_TQAVHC(queue), 0);
+
+		/* If CBS is not enabled for any queues anymore, then return to
+		 * the default state of Data Transmission Arbitration on
+		 * TQAVCTRL.
+		 */
+		if (!is_any_cbs_enabled(adapter)) {
+			tqavctrl = rd32(E1000_I210_TQAVCTRL);
+			tqavctrl &= ~E1000_TQAVCTRL_DATATRANARB;
+			wr32(E1000_I210_TQAVCTRL, tqavctrl);
+		}
 	}
 
 	/* XXX: In i210 controller the sendSlope and loCredit parameters from
@@ -1792,18 +1822,6 @@ static int igb_save_cbs_params(struct igb_adapter *adapter, int queue,
 	return 0;
 }
 
-static bool is_any_cbs_enabled(struct igb_adapter *adapter)
-{
-	int i;
-
-	for (i = 0; i < adapter->num_tx_queues; i++) {
-		if (adapter->tx_ring[i]->cbs_enable)
-			return true;
-	}
-
-	return false;
-}
-
 /**
  *  igb_setup_tx_mode - Switch to/from Qav Tx mode when applicable
  *  @adapter: pointer to adapter struct
@@ -1827,11 +1845,10 @@ static void igb_setup_tx_mode(struct igb_adapter *adapter)
 		int i, max_queue;
 
 		/* Configure TQAVCTRL register: set transmit mode to 'Qav',
-		 * set data fetch arbitration to 'round robin' and set data
-		 * transfer arbitration to 'credit shaper algorithm.
+		 * set data fetch arbitration to 'round robin'.
 		 */
 		val = rd32(E1000_I210_TQAVCTRL);
-		val |= E1000_TQAVCTRL_XMIT_MODE | E1000_TQAVCTRL_DATATRANARB;
+		val |= E1000_TQAVCTRL_XMIT_MODE;
 		val &= ~E1000_TQAVCTRL_DATAFETCHARB;
 		wr32(E1000_I210_TQAVCTRL, val);
 
-- 
2.15.1


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

* [RFC v2 net-next 09/10] igb: Refactor igb_offload_cbs()
  2018-01-17 23:06 ` [Intel-wired-lan] " Jesus Sanchez-Palencia
@ 2018-01-17 23:06   ` Jesus Sanchez-Palencia
  -1 siblings, 0 replies; 86+ messages in thread
From: Jesus Sanchez-Palencia @ 2018-01-17 23:06 UTC (permalink / raw)
  To: netdev
  Cc: jhs, xiyou.wangcong, jiri, vinicius.gomes, richardcochran,
	intel-wired-lan, anna-maria, henrik, tglx, john.stultz,
	andre.guedes, ivan.briano, levi.pearson, Jesus Sanchez-Palencia

Split code into a separate function (igb_offload_apply()) that will be
used by TBS offload implementation.

Signed-off-by: Jesus Sanchez-Palencia <jesus.sanchez-palencia@intel.com>
---
 drivers/net/ethernet/intel/igb/igb_main.c | 23 ++++++++++++++---------
 1 file changed, 14 insertions(+), 9 deletions(-)

diff --git a/drivers/net/ethernet/intel/igb/igb_main.c b/drivers/net/ethernet/intel/igb/igb_main.c
index a955e1723ed5..fc2fc571b87b 100644
--- a/drivers/net/ethernet/intel/igb/igb_main.c
+++ b/drivers/net/ethernet/intel/igb/igb_main.c
@@ -2462,6 +2462,19 @@ igb_features_check(struct sk_buff *skb, struct net_device *dev,
 	return features;
 }
 
+static void igb_offload_apply(struct igb_adapter *adapter, s32 queue)
+{
+	if (!is_fqtss_enabled(adapter)) {
+		enable_fqtss(adapter, true);
+		return;
+	}
+
+	igb_config_tx_modes(adapter, queue);
+
+	if (!is_any_cbs_enabled(adapter))
+		enable_fqtss(adapter, false);
+}
+
 static int igb_offload_cbs(struct igb_adapter *adapter,
 			   struct tc_cbs_qopt_offload *qopt)
 {
@@ -2482,15 +2495,7 @@ static int igb_offload_cbs(struct igb_adapter *adapter,
 	if (err)
 		return err;
 
-	if (is_fqtss_enabled(adapter)) {
-		igb_config_tx_modes(adapter, qopt->queue);
-
-		if (!is_any_cbs_enabled(adapter))
-			enable_fqtss(adapter, false);
-
-	} else {
-		enable_fqtss(adapter, true);
-	}
+	igb_offload_apply(adapter, qopt->queue);
 
 	return 0;
 }
-- 
2.15.1

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

* [Intel-wired-lan] [RFC v2 net-next 09/10] igb: Refactor igb_offload_cbs()
@ 2018-01-17 23:06   ` Jesus Sanchez-Palencia
  0 siblings, 0 replies; 86+ messages in thread
From: Jesus Sanchez-Palencia @ 2018-01-17 23:06 UTC (permalink / raw)
  To: intel-wired-lan

Split code into a separate function (igb_offload_apply()) that will be
used by TBS offload implementation.

Signed-off-by: Jesus Sanchez-Palencia <jesus.sanchez-palencia@intel.com>
---
 drivers/net/ethernet/intel/igb/igb_main.c | 23 ++++++++++++++---------
 1 file changed, 14 insertions(+), 9 deletions(-)

diff --git a/drivers/net/ethernet/intel/igb/igb_main.c b/drivers/net/ethernet/intel/igb/igb_main.c
index a955e1723ed5..fc2fc571b87b 100644
--- a/drivers/net/ethernet/intel/igb/igb_main.c
+++ b/drivers/net/ethernet/intel/igb/igb_main.c
@@ -2462,6 +2462,19 @@ igb_features_check(struct sk_buff *skb, struct net_device *dev,
 	return features;
 }
 
+static void igb_offload_apply(struct igb_adapter *adapter, s32 queue)
+{
+	if (!is_fqtss_enabled(adapter)) {
+		enable_fqtss(adapter, true);
+		return;
+	}
+
+	igb_config_tx_modes(adapter, queue);
+
+	if (!is_any_cbs_enabled(adapter))
+		enable_fqtss(adapter, false);
+}
+
 static int igb_offload_cbs(struct igb_adapter *adapter,
 			   struct tc_cbs_qopt_offload *qopt)
 {
@@ -2482,15 +2495,7 @@ static int igb_offload_cbs(struct igb_adapter *adapter,
 	if (err)
 		return err;
 
-	if (is_fqtss_enabled(adapter)) {
-		igb_config_tx_modes(adapter, qopt->queue);
-
-		if (!is_any_cbs_enabled(adapter))
-			enable_fqtss(adapter, false);
-
-	} else {
-		enable_fqtss(adapter, true);
-	}
+	igb_offload_apply(adapter, qopt->queue);
 
 	return 0;
 }
-- 
2.15.1


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

* [RFC v2 net-next 10/10] igb: Add support for TBS offload
  2018-01-17 23:06 ` [Intel-wired-lan] " Jesus Sanchez-Palencia
@ 2018-01-17 23:06   ` Jesus Sanchez-Palencia
  -1 siblings, 0 replies; 86+ messages in thread
From: Jesus Sanchez-Palencia @ 2018-01-17 23:06 UTC (permalink / raw)
  To: netdev
  Cc: jhs, xiyou.wangcong, jiri, vinicius.gomes, richardcochran,
	intel-wired-lan, anna-maria, henrik, tglx, john.stultz,
	andre.guedes, ivan.briano, levi.pearson, Jesus Sanchez-Palencia

Implement HW offload support for SO_TXTIME through igb's Launchtime
feature. This is done by extending igb_setup_tc() so it supports
TC_SETUP_QDISC_TBS and configuring i210 so time based transmit
arbitration is enabled.

The FQTSS transmission mode added before is extended so strict
priority (SP) queues wait for stream reservation (SR) ones.
igb_config_tx_modes() is extended so it can support enabling/disabling
Launchtime following the previous approach used for the credit-based
shaper (CBS).

As the previous flow, FQTSS transmission mode is enabled automatically
by the driver once Launchtime (or CBS, as before) is enabled.
Similarly, it's automatically disabled when the feature is disabled
for the last queue that had it setup on.

The driver just consumes the transmit times from the skbuffs directly,
so no special handling is done in case an 'invalid' time is provided.
We assume this has been handled by the TBS qdisc already.

Signed-off-by: Jesus Sanchez-Palencia <jesus.sanchez-palencia@intel.com>
---
 drivers/net/ethernet/intel/igb/e1000_defines.h |  16 +++
 drivers/net/ethernet/intel/igb/igb.h           |   1 +
 drivers/net/ethernet/intel/igb/igb_main.c      | 135 ++++++++++++++++++++++---
 3 files changed, 137 insertions(+), 15 deletions(-)

diff --git a/drivers/net/ethernet/intel/igb/e1000_defines.h b/drivers/net/ethernet/intel/igb/e1000_defines.h
index 83cabff1e0ab..9e357848c550 100644
--- a/drivers/net/ethernet/intel/igb/e1000_defines.h
+++ b/drivers/net/ethernet/intel/igb/e1000_defines.h
@@ -1066,6 +1066,22 @@
 #define E1000_TQAVCTRL_XMIT_MODE	BIT(0)
 #define E1000_TQAVCTRL_DATAFETCHARB	BIT(4)
 #define E1000_TQAVCTRL_DATATRANARB	BIT(8)
+#define E1000_TQAVCTRL_DATATRANTIM	BIT(9)
+#define E1000_TQAVCTRL_SP_WAIT_SR	BIT(10)
+/* Fetch Time Delta - bits 31:16
+ *
+ * This field holds the value to be reduced from the launch time for
+ * fetch time decision. The FetchTimeDelta value is defined in 32 ns
+ * granularity.
+ *
+ * This field is 16 bits wide, and so the maximum value is:
+ *
+ * 65535 * 32 = 2097120 ~= 2.1 msec
+ *
+ * XXX: We are configuring the max value here since we couldn't come up
+ * with a reason for not doing so.
+ */
+#define E1000_TQAVCTRL_FETCHTIME_DELTA	(0xFFFF << 16)
 
 /* TX Qav Credit Control fields */
 #define E1000_TQAVCC_IDLESLOPE_MASK	0xFFFF
diff --git a/drivers/net/ethernet/intel/igb/igb.h b/drivers/net/ethernet/intel/igb/igb.h
index 92845692087a..ef2bbef054e8 100644
--- a/drivers/net/ethernet/intel/igb/igb.h
+++ b/drivers/net/ethernet/intel/igb/igb.h
@@ -281,6 +281,7 @@ struct igb_ring {
 	u16 count;			/* number of desc. in the ring */
 	u8 queue_index;			/* logical index of the ring*/
 	u8 reg_idx;			/* physical index of the ring */
+	bool launchtime_enable;		/* true if LaunchTime is enabled */
 	bool cbs_enable;		/* indicates if CBS is enabled */
 	s32 idleslope;			/* idleSlope in kbps */
 	s32 sendslope;			/* sendSlope in kbps */
diff --git a/drivers/net/ethernet/intel/igb/igb_main.c b/drivers/net/ethernet/intel/igb/igb_main.c
index fc2fc571b87b..30243b3250e4 100644
--- a/drivers/net/ethernet/intel/igb/igb_main.c
+++ b/drivers/net/ethernet/intel/igb/igb_main.c
@@ -1684,13 +1684,26 @@ static bool is_any_cbs_enabled(struct igb_adapter *adapter)
 	return false;
 }
 
+static bool is_any_txtime_enabled(struct igb_adapter *adapter)
+{
+	int i;
+
+	for (i = 0; i < adapter->num_tx_queues; i++) {
+		if (adapter->tx_ring[i]->launchtime_enable)
+			return true;
+	}
+
+	return false;
+}
+
 /**
  *  igb_config_tx_modes - Configure "Qav Tx mode" features on igb
  *  @adapter: pointer to adapter struct
  *  @queue: queue number
  *
- *  Configure CBS for a given hardware queue. Parameters are retrieved
- *  from the correct Tx ring, so igb_save_cbs_params() should be used
+ *  Configure CBS and Launchtime for a given hardware queue.
+ *  Parameters are retrieved from the correct Tx ring, so
+ *  igb_save_cbs_params() and igb_save_txtime_params() should be used
  *  for setting those correctly prior to this function being called.
  **/
 static void igb_config_tx_modes(struct igb_adapter *adapter, int queue)
@@ -1704,10 +1717,20 @@ static void igb_config_tx_modes(struct igb_adapter *adapter, int queue)
 	WARN_ON(hw->mac.type != e1000_i210);
 	WARN_ON(queue < 0 || queue > 1);
 
-	if (ring->cbs_enable) {
+	/* If any of the Qav features is enabled, configure queues as SR and
+	 * with HIGH PRIO. If none is, then configure them with LOW PRIO and
+	 * as SP.
+	 */
+	if (ring->cbs_enable || ring->launchtime_enable) {
 		set_tx_desc_fetch_prio(hw, queue, TX_QUEUE_PRIO_HIGH);
 		set_queue_mode(hw, queue, QUEUE_MODE_STREAM_RESERVATION);
+	} else {
+		set_tx_desc_fetch_prio(hw, queue, TX_QUEUE_PRIO_LOW);
+		set_queue_mode(hw, queue, QUEUE_MODE_STRICT_PRIORITY);
+	}
 
+	/* If CBS is enabled, set DataTranARB and config its parameters. */
+	if (ring->cbs_enable) {
 		/* Always set data transfer arbitration to credit-based
 		 * shaper algorithm on TQAVCTRL if CBS is enabled for any of
 		 * the queues.
@@ -1769,8 +1792,6 @@ static void igb_config_tx_modes(struct igb_adapter *adapter, int queue)
 		wr32(E1000_I210_TQAVHC(queue),
 		     0x80000000 + ring->hicredit * 0x7735);
 	} else {
-		set_tx_desc_fetch_prio(hw, queue, TX_QUEUE_PRIO_LOW);
-		set_queue_mode(hw, queue, QUEUE_MODE_STRICT_PRIORITY);
 
 		/* Set idleSlope to zero. */
 		tqavcc = rd32(E1000_I210_TQAVCC(queue));
@@ -1791,17 +1812,61 @@ static void igb_config_tx_modes(struct igb_adapter *adapter, int queue)
 		}
 	}
 
+	/* If LaunchTime is enabled, set DataTranTIM. */
+	if (ring->launchtime_enable) {
+		/* Always set DataTranTIM on TQAVCTRL if LaunchTime is enabled
+		 * for any of the SR queues, and configure fetchtime delta.
+		 * XXX NOTE:
+		 *     - LaunchTime will be enabled for all SR queues.
+		 *     - A fixed offset can be added relative to the launch
+		 *       time of all packets if configured at reg LAUNCH_OS0.
+		 *       We are keeping it as 0 for now (default value).
+		 */
+		tqavctrl = rd32(E1000_I210_TQAVCTRL);
+		tqavctrl |= E1000_TQAVCTRL_DATATRANTIM |
+		       E1000_TQAVCTRL_FETCHTIME_DELTA;
+		wr32(E1000_I210_TQAVCTRL, tqavctrl);
+	} else {
+		/* If Launchtime is not enabled for any SR queues anymore,
+		 * then clear DataTranTIM on TQAVCTRL and clear fetchtime delta,
+		 * effectively disabling Launchtime.
+		 */
+		if (!is_any_txtime_enabled(adapter)) {
+			tqavctrl = rd32(E1000_I210_TQAVCTRL);
+			tqavctrl &= ~E1000_TQAVCTRL_DATATRANTIM;
+			tqavctrl &= ~E1000_TQAVCTRL_FETCHTIME_DELTA;
+			wr32(E1000_I210_TQAVCTRL, tqavctrl);
+		}
+	}
+
 	/* XXX: In i210 controller the sendSlope and loCredit parameters from
 	 * CBS are not configurable by software so we don't do any 'controller
 	 * configuration' in respect to these parameters.
 	 */
 
-	netdev_dbg(netdev, "CBS %s: queue %d idleslope %d sendslope %d hiCredit %d locredit %d\n",
-		   (ring->cbs_enable) ? "enabled" : "disabled", queue,
+	netdev_dbg(netdev, "Qav Tx mode: cbs %s, launchtime %s, queue %d \
+			    idleslope %d sendslope %d hiCredit %d \
+			    locredit %d\n",
+		   (ring->cbs_enable) ? "enabled" : "disabled",
+		   (ring->launchtime_enable) ? "enabled" : "disabled", queue,
 		   ring->idleslope, ring->sendslope, ring->hicredit,
 		   ring->locredit);
 }
 
+static int igb_save_txtime_params(struct igb_adapter *adapter, int queue,
+				  bool enable)
+{
+	struct igb_ring *ring;
+
+	if (queue < 0 || queue > adapter->num_tx_queues)
+		return -EINVAL;
+
+	ring = adapter->tx_ring[queue];
+	ring->launchtime_enable = enable;
+
+	return 0;
+}
+
 static int igb_save_cbs_params(struct igb_adapter *adapter, int queue,
 			       bool enable, int idleslope, int sendslope,
 			       int hicredit, int locredit)
@@ -1845,10 +1910,11 @@ static void igb_setup_tx_mode(struct igb_adapter *adapter)
 		int i, max_queue;
 
 		/* Configure TQAVCTRL register: set transmit mode to 'Qav',
-		 * set data fetch arbitration to 'round robin'.
+		 * set data fetch arbitration to 'round robin', set SP_WAIT_SR
+		 * so SP queues wait for SR ones.
 		 */
 		val = rd32(E1000_I210_TQAVCTRL);
-		val |= E1000_TQAVCTRL_XMIT_MODE;
+		val |= E1000_TQAVCTRL_XMIT_MODE | E1000_TQAVCTRL_SP_WAIT_SR;
 		val &= ~E1000_TQAVCTRL_DATAFETCHARB;
 		wr32(E1000_I210_TQAVCTRL, val);
 
@@ -2471,7 +2537,7 @@ static void igb_offload_apply(struct igb_adapter *adapter, s32 queue)
 
 	igb_config_tx_modes(adapter, queue);
 
-	if (!is_any_cbs_enabled(adapter))
+	if (!is_any_cbs_enabled(adapter) && !is_any_txtime_enabled(adapter))
 		enable_fqtss(adapter, false);
 }
 
@@ -2500,6 +2566,30 @@ static int igb_offload_cbs(struct igb_adapter *adapter,
 	return 0;
 }
 
+static int igb_offload_txtime(struct igb_adapter *adapter,
+			      struct tc_tbs_qopt_offload *qopt)
+{
+	struct e1000_hw *hw = &adapter->hw;
+	int err;
+
+	/* Launchtime offloading is only supported by i210 controller. */
+	if (hw->mac.type != e1000_i210)
+		return -EOPNOTSUPP;
+
+	/* Launchtime offloading is only supported by queues 0 and 1. */
+	if (qopt->queue < 0 || qopt->queue > 1)
+		return -EINVAL;
+
+	err = igb_save_txtime_params(adapter, qopt->queue, qopt->enable);
+
+	if (err)
+		return err;
+
+	igb_offload_apply(adapter, qopt->queue);
+
+	return 0;
+}
+
 static int igb_setup_tc(struct net_device *dev, enum tc_setup_type type,
 			void *type_data)
 {
@@ -2508,6 +2598,8 @@ static int igb_setup_tc(struct net_device *dev, enum tc_setup_type type,
 	switch (type) {
 	case TC_SETUP_QDISC_CBS:
 		return igb_offload_cbs(adapter, type_data);
+	case TC_SETUP_QDISC_TBS:
+		return igb_offload_txtime(adapter, type_data);
 
 	default:
 		return -EOPNOTSUPP;
@@ -5313,11 +5405,14 @@ static void igb_set_itr(struct igb_q_vector *q_vector)
 	}
 }
 
-static void igb_tx_ctxtdesc(struct igb_ring *tx_ring, u32 vlan_macip_lens,
-			    u32 type_tucmd, u32 mss_l4len_idx)
+static void igb_tx_ctxtdesc(struct igb_ring *tx_ring,
+			    struct igb_tx_buffer *first,
+			    u32 vlan_macip_lens, u32 type_tucmd,
+			    u32 mss_l4len_idx)
 {
 	struct e1000_adv_tx_context_desc *context_desc;
 	u16 i = tx_ring->next_to_use;
+	struct timespec64 ts;
 
 	context_desc = IGB_TX_CTXTDESC(tx_ring, i);
 
@@ -5332,9 +5427,18 @@ static void igb_tx_ctxtdesc(struct igb_ring *tx_ring, u32 vlan_macip_lens,
 		mss_l4len_idx |= tx_ring->reg_idx << 4;
 
 	context_desc->vlan_macip_lens	= cpu_to_le32(vlan_macip_lens);
-	context_desc->seqnum_seed	= 0;
 	context_desc->type_tucmd_mlhl	= cpu_to_le32(type_tucmd);
 	context_desc->mss_l4len_idx	= cpu_to_le32(mss_l4len_idx);
+
+	/* We assume there is always a valid tx time available. Invalid times
+	 * should have been handled by the upper layers.
+	 */
+	if (tx_ring->launchtime_enable) {
+		ts = ns_to_timespec64(first->skb->tstamp);
+		context_desc->seqnum_seed = cpu_to_le32(ts.tv_nsec / 32);
+	} else {
+		context_desc->seqnum_seed = 0;
+	}
 }
 
 static int igb_tso(struct igb_ring *tx_ring,
@@ -5417,7 +5521,8 @@ static int igb_tso(struct igb_ring *tx_ring,
 	vlan_macip_lens |= (ip.hdr - skb->data) << E1000_ADVTXD_MACLEN_SHIFT;
 	vlan_macip_lens |= first->tx_flags & IGB_TX_FLAGS_VLAN_MASK;
 
-	igb_tx_ctxtdesc(tx_ring, vlan_macip_lens, type_tucmd, mss_l4len_idx);
+	igb_tx_ctxtdesc(tx_ring, first, vlan_macip_lens,
+			type_tucmd, mss_l4len_idx);
 
 	return 1;
 }
@@ -5472,7 +5577,7 @@ static void igb_tx_csum(struct igb_ring *tx_ring, struct igb_tx_buffer *first)
 	vlan_macip_lens |= skb_network_offset(skb) << E1000_ADVTXD_MACLEN_SHIFT;
 	vlan_macip_lens |= first->tx_flags & IGB_TX_FLAGS_VLAN_MASK;
 
-	igb_tx_ctxtdesc(tx_ring, vlan_macip_lens, type_tucmd, 0);
+	igb_tx_ctxtdesc(tx_ring, first, vlan_macip_lens, type_tucmd, 0);
 }
 
 #define IGB_SET_FLAG(_input, _flag, _result) \
-- 
2.15.1

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

* [Intel-wired-lan] [RFC v2 net-next 10/10] igb: Add support for TBS offload
@ 2018-01-17 23:06   ` Jesus Sanchez-Palencia
  0 siblings, 0 replies; 86+ messages in thread
From: Jesus Sanchez-Palencia @ 2018-01-17 23:06 UTC (permalink / raw)
  To: intel-wired-lan

Implement HW offload support for SO_TXTIME through igb's Launchtime
feature. This is done by extending igb_setup_tc() so it supports
TC_SETUP_QDISC_TBS and configuring i210 so time based transmit
arbitration is enabled.

The FQTSS transmission mode added before is extended so strict
priority (SP) queues wait for stream reservation (SR) ones.
igb_config_tx_modes() is extended so it can support enabling/disabling
Launchtime following the previous approach used for the credit-based
shaper (CBS).

As the previous flow, FQTSS transmission mode is enabled automatically
by the driver once Launchtime (or CBS, as before) is enabled.
Similarly, it's automatically disabled when the feature is disabled
for the last queue that had it setup on.

The driver just consumes the transmit times from the skbuffs directly,
so no special handling is done in case an 'invalid' time is provided.
We assume this has been handled by the TBS qdisc already.

Signed-off-by: Jesus Sanchez-Palencia <jesus.sanchez-palencia@intel.com>
---
 drivers/net/ethernet/intel/igb/e1000_defines.h |  16 +++
 drivers/net/ethernet/intel/igb/igb.h           |   1 +
 drivers/net/ethernet/intel/igb/igb_main.c      | 135 ++++++++++++++++++++++---
 3 files changed, 137 insertions(+), 15 deletions(-)

diff --git a/drivers/net/ethernet/intel/igb/e1000_defines.h b/drivers/net/ethernet/intel/igb/e1000_defines.h
index 83cabff1e0ab..9e357848c550 100644
--- a/drivers/net/ethernet/intel/igb/e1000_defines.h
+++ b/drivers/net/ethernet/intel/igb/e1000_defines.h
@@ -1066,6 +1066,22 @@
 #define E1000_TQAVCTRL_XMIT_MODE	BIT(0)
 #define E1000_TQAVCTRL_DATAFETCHARB	BIT(4)
 #define E1000_TQAVCTRL_DATATRANARB	BIT(8)
+#define E1000_TQAVCTRL_DATATRANTIM	BIT(9)
+#define E1000_TQAVCTRL_SP_WAIT_SR	BIT(10)
+/* Fetch Time Delta - bits 31:16
+ *
+ * This field holds the value to be reduced from the launch time for
+ * fetch time decision. The FetchTimeDelta value is defined in 32 ns
+ * granularity.
+ *
+ * This field is 16 bits wide, and so the maximum value is:
+ *
+ * 65535 * 32 = 2097120 ~= 2.1 msec
+ *
+ * XXX: We are configuring the max value here since we couldn't come up
+ * with a reason for not doing so.
+ */
+#define E1000_TQAVCTRL_FETCHTIME_DELTA	(0xFFFF << 16)
 
 /* TX Qav Credit Control fields */
 #define E1000_TQAVCC_IDLESLOPE_MASK	0xFFFF
diff --git a/drivers/net/ethernet/intel/igb/igb.h b/drivers/net/ethernet/intel/igb/igb.h
index 92845692087a..ef2bbef054e8 100644
--- a/drivers/net/ethernet/intel/igb/igb.h
+++ b/drivers/net/ethernet/intel/igb/igb.h
@@ -281,6 +281,7 @@ struct igb_ring {
 	u16 count;			/* number of desc. in the ring */
 	u8 queue_index;			/* logical index of the ring*/
 	u8 reg_idx;			/* physical index of the ring */
+	bool launchtime_enable;		/* true if LaunchTime is enabled */
 	bool cbs_enable;		/* indicates if CBS is enabled */
 	s32 idleslope;			/* idleSlope in kbps */
 	s32 sendslope;			/* sendSlope in kbps */
diff --git a/drivers/net/ethernet/intel/igb/igb_main.c b/drivers/net/ethernet/intel/igb/igb_main.c
index fc2fc571b87b..30243b3250e4 100644
--- a/drivers/net/ethernet/intel/igb/igb_main.c
+++ b/drivers/net/ethernet/intel/igb/igb_main.c
@@ -1684,13 +1684,26 @@ static bool is_any_cbs_enabled(struct igb_adapter *adapter)
 	return false;
 }
 
+static bool is_any_txtime_enabled(struct igb_adapter *adapter)
+{
+	int i;
+
+	for (i = 0; i < adapter->num_tx_queues; i++) {
+		if (adapter->tx_ring[i]->launchtime_enable)
+			return true;
+	}
+
+	return false;
+}
+
 /**
  *  igb_config_tx_modes - Configure "Qav Tx mode" features on igb
  *  @adapter: pointer to adapter struct
  *  @queue: queue number
  *
- *  Configure CBS for a given hardware queue. Parameters are retrieved
- *  from the correct Tx ring, so igb_save_cbs_params() should be used
+ *  Configure CBS and Launchtime for a given hardware queue.
+ *  Parameters are retrieved from the correct Tx ring, so
+ *  igb_save_cbs_params() and igb_save_txtime_params() should be used
  *  for setting those correctly prior to this function being called.
  **/
 static void igb_config_tx_modes(struct igb_adapter *adapter, int queue)
@@ -1704,10 +1717,20 @@ static void igb_config_tx_modes(struct igb_adapter *adapter, int queue)
 	WARN_ON(hw->mac.type != e1000_i210);
 	WARN_ON(queue < 0 || queue > 1);
 
-	if (ring->cbs_enable) {
+	/* If any of the Qav features is enabled, configure queues as SR and
+	 * with HIGH PRIO. If none is, then configure them with LOW PRIO and
+	 * as SP.
+	 */
+	if (ring->cbs_enable || ring->launchtime_enable) {
 		set_tx_desc_fetch_prio(hw, queue, TX_QUEUE_PRIO_HIGH);
 		set_queue_mode(hw, queue, QUEUE_MODE_STREAM_RESERVATION);
+	} else {
+		set_tx_desc_fetch_prio(hw, queue, TX_QUEUE_PRIO_LOW);
+		set_queue_mode(hw, queue, QUEUE_MODE_STRICT_PRIORITY);
+	}
 
+	/* If CBS is enabled, set DataTranARB and config its parameters. */
+	if (ring->cbs_enable) {
 		/* Always set data transfer arbitration to credit-based
 		 * shaper algorithm on TQAVCTRL if CBS is enabled for any of
 		 * the queues.
@@ -1769,8 +1792,6 @@ static void igb_config_tx_modes(struct igb_adapter *adapter, int queue)
 		wr32(E1000_I210_TQAVHC(queue),
 		     0x80000000 + ring->hicredit * 0x7735);
 	} else {
-		set_tx_desc_fetch_prio(hw, queue, TX_QUEUE_PRIO_LOW);
-		set_queue_mode(hw, queue, QUEUE_MODE_STRICT_PRIORITY);
 
 		/* Set idleSlope to zero. */
 		tqavcc = rd32(E1000_I210_TQAVCC(queue));
@@ -1791,17 +1812,61 @@ static void igb_config_tx_modes(struct igb_adapter *adapter, int queue)
 		}
 	}
 
+	/* If LaunchTime is enabled, set DataTranTIM. */
+	if (ring->launchtime_enable) {
+		/* Always set DataTranTIM on TQAVCTRL if LaunchTime is enabled
+		 * for any of the SR queues, and configure fetchtime delta.
+		 * XXX NOTE:
+		 *     - LaunchTime will be enabled for all SR queues.
+		 *     - A fixed offset can be added relative to the launch
+		 *       time of all packets if configured at reg LAUNCH_OS0.
+		 *       We are keeping it as 0 for now (default value).
+		 */
+		tqavctrl = rd32(E1000_I210_TQAVCTRL);
+		tqavctrl |= E1000_TQAVCTRL_DATATRANTIM |
+		       E1000_TQAVCTRL_FETCHTIME_DELTA;
+		wr32(E1000_I210_TQAVCTRL, tqavctrl);
+	} else {
+		/* If Launchtime is not enabled for any SR queues anymore,
+		 * then clear DataTranTIM on TQAVCTRL and clear fetchtime delta,
+		 * effectively disabling Launchtime.
+		 */
+		if (!is_any_txtime_enabled(adapter)) {
+			tqavctrl = rd32(E1000_I210_TQAVCTRL);
+			tqavctrl &= ~E1000_TQAVCTRL_DATATRANTIM;
+			tqavctrl &= ~E1000_TQAVCTRL_FETCHTIME_DELTA;
+			wr32(E1000_I210_TQAVCTRL, tqavctrl);
+		}
+	}
+
 	/* XXX: In i210 controller the sendSlope and loCredit parameters from
 	 * CBS are not configurable by software so we don't do any 'controller
 	 * configuration' in respect to these parameters.
 	 */
 
-	netdev_dbg(netdev, "CBS %s: queue %d idleslope %d sendslope %d hiCredit %d locredit %d\n",
-		   (ring->cbs_enable) ? "enabled" : "disabled", queue,
+	netdev_dbg(netdev, "Qav Tx mode: cbs %s, launchtime %s, queue %d \
+			    idleslope %d sendslope %d hiCredit %d \
+			    locredit %d\n",
+		   (ring->cbs_enable) ? "enabled" : "disabled",
+		   (ring->launchtime_enable) ? "enabled" : "disabled", queue,
 		   ring->idleslope, ring->sendslope, ring->hicredit,
 		   ring->locredit);
 }
 
+static int igb_save_txtime_params(struct igb_adapter *adapter, int queue,
+				  bool enable)
+{
+	struct igb_ring *ring;
+
+	if (queue < 0 || queue > adapter->num_tx_queues)
+		return -EINVAL;
+
+	ring = adapter->tx_ring[queue];
+	ring->launchtime_enable = enable;
+
+	return 0;
+}
+
 static int igb_save_cbs_params(struct igb_adapter *adapter, int queue,
 			       bool enable, int idleslope, int sendslope,
 			       int hicredit, int locredit)
@@ -1845,10 +1910,11 @@ static void igb_setup_tx_mode(struct igb_adapter *adapter)
 		int i, max_queue;
 
 		/* Configure TQAVCTRL register: set transmit mode to 'Qav',
-		 * set data fetch arbitration to 'round robin'.
+		 * set data fetch arbitration to 'round robin', set SP_WAIT_SR
+		 * so SP queues wait for SR ones.
 		 */
 		val = rd32(E1000_I210_TQAVCTRL);
-		val |= E1000_TQAVCTRL_XMIT_MODE;
+		val |= E1000_TQAVCTRL_XMIT_MODE | E1000_TQAVCTRL_SP_WAIT_SR;
 		val &= ~E1000_TQAVCTRL_DATAFETCHARB;
 		wr32(E1000_I210_TQAVCTRL, val);
 
@@ -2471,7 +2537,7 @@ static void igb_offload_apply(struct igb_adapter *adapter, s32 queue)
 
 	igb_config_tx_modes(adapter, queue);
 
-	if (!is_any_cbs_enabled(adapter))
+	if (!is_any_cbs_enabled(adapter) && !is_any_txtime_enabled(adapter))
 		enable_fqtss(adapter, false);
 }
 
@@ -2500,6 +2566,30 @@ static int igb_offload_cbs(struct igb_adapter *adapter,
 	return 0;
 }
 
+static int igb_offload_txtime(struct igb_adapter *adapter,
+			      struct tc_tbs_qopt_offload *qopt)
+{
+	struct e1000_hw *hw = &adapter->hw;
+	int err;
+
+	/* Launchtime offloading is only supported by i210 controller. */
+	if (hw->mac.type != e1000_i210)
+		return -EOPNOTSUPP;
+
+	/* Launchtime offloading is only supported by queues 0 and 1. */
+	if (qopt->queue < 0 || qopt->queue > 1)
+		return -EINVAL;
+
+	err = igb_save_txtime_params(adapter, qopt->queue, qopt->enable);
+
+	if (err)
+		return err;
+
+	igb_offload_apply(adapter, qopt->queue);
+
+	return 0;
+}
+
 static int igb_setup_tc(struct net_device *dev, enum tc_setup_type type,
 			void *type_data)
 {
@@ -2508,6 +2598,8 @@ static int igb_setup_tc(struct net_device *dev, enum tc_setup_type type,
 	switch (type) {
 	case TC_SETUP_QDISC_CBS:
 		return igb_offload_cbs(adapter, type_data);
+	case TC_SETUP_QDISC_TBS:
+		return igb_offload_txtime(adapter, type_data);
 
 	default:
 		return -EOPNOTSUPP;
@@ -5313,11 +5405,14 @@ static void igb_set_itr(struct igb_q_vector *q_vector)
 	}
 }
 
-static void igb_tx_ctxtdesc(struct igb_ring *tx_ring, u32 vlan_macip_lens,
-			    u32 type_tucmd, u32 mss_l4len_idx)
+static void igb_tx_ctxtdesc(struct igb_ring *tx_ring,
+			    struct igb_tx_buffer *first,
+			    u32 vlan_macip_lens, u32 type_tucmd,
+			    u32 mss_l4len_idx)
 {
 	struct e1000_adv_tx_context_desc *context_desc;
 	u16 i = tx_ring->next_to_use;
+	struct timespec64 ts;
 
 	context_desc = IGB_TX_CTXTDESC(tx_ring, i);
 
@@ -5332,9 +5427,18 @@ static void igb_tx_ctxtdesc(struct igb_ring *tx_ring, u32 vlan_macip_lens,
 		mss_l4len_idx |= tx_ring->reg_idx << 4;
 
 	context_desc->vlan_macip_lens	= cpu_to_le32(vlan_macip_lens);
-	context_desc->seqnum_seed	= 0;
 	context_desc->type_tucmd_mlhl	= cpu_to_le32(type_tucmd);
 	context_desc->mss_l4len_idx	= cpu_to_le32(mss_l4len_idx);
+
+	/* We assume there is always a valid tx time available. Invalid times
+	 * should have been handled by the upper layers.
+	 */
+	if (tx_ring->launchtime_enable) {
+		ts = ns_to_timespec64(first->skb->tstamp);
+		context_desc->seqnum_seed = cpu_to_le32(ts.tv_nsec / 32);
+	} else {
+		context_desc->seqnum_seed = 0;
+	}
 }
 
 static int igb_tso(struct igb_ring *tx_ring,
@@ -5417,7 +5521,8 @@ static int igb_tso(struct igb_ring *tx_ring,
 	vlan_macip_lens |= (ip.hdr - skb->data) << E1000_ADVTXD_MACLEN_SHIFT;
 	vlan_macip_lens |= first->tx_flags & IGB_TX_FLAGS_VLAN_MASK;
 
-	igb_tx_ctxtdesc(tx_ring, vlan_macip_lens, type_tucmd, mss_l4len_idx);
+	igb_tx_ctxtdesc(tx_ring, first, vlan_macip_lens,
+			type_tucmd, mss_l4len_idx);
 
 	return 1;
 }
@@ -5472,7 +5577,7 @@ static void igb_tx_csum(struct igb_ring *tx_ring, struct igb_tx_buffer *first)
 	vlan_macip_lens |= skb_network_offset(skb) << E1000_ADVTXD_MACLEN_SHIFT;
 	vlan_macip_lens |= first->tx_flags & IGB_TX_FLAGS_VLAN_MASK;
 
-	igb_tx_ctxtdesc(tx_ring, vlan_macip_lens, type_tucmd, 0);
+	igb_tx_ctxtdesc(tx_ring, first, vlan_macip_lens, type_tucmd, 0);
 }
 
 #define IGB_SET_FLAG(_input, _flag, _result) \
-- 
2.15.1


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

* Re: [RFC v2 net-next 02/10] net: ipv4: raw: Hook into time based transmission.
  2018-01-17 23:06   ` [Intel-wired-lan] " Jesus Sanchez-Palencia
@ 2018-01-18  0:28     ` Eric Dumazet
  -1 siblings, 0 replies; 86+ messages in thread
From: Eric Dumazet @ 2018-01-18  0:28 UTC (permalink / raw)
  To: Jesus Sanchez-Palencia, netdev
  Cc: jhs, xiyou.wangcong, jiri, vinicius.gomes, richardcochran,
	intel-wired-lan, anna-maria, henrik, tglx, john.stultz,
	andre.guedes, ivan.briano, levi.pearson, Richard Cochran

On Wed, 2018-01-17 at 15:06 -0800, Jesus Sanchez-Palencia wrote:
> From: Richard Cochran <rcochran@linutronix.de>
> 
> For raw packets, copy the desired future transmit time from the CMSG
> cookie into the skb.
> 
> Signed-off-by: Richard Cochran <rcochran@linutronix.de>
> Signed-off-by: Jesus Sanchez-Palencia <jesus.sanchez-palencia@intel.com>
> ---
>  net/ipv4/raw.c | 2 ++
>  1 file changed, 2 insertions(+)
> 
> diff --git a/net/ipv4/raw.c b/net/ipv4/raw.c
> index 136544b36a46..e37ea8ab6a78 100644
> --- a/net/ipv4/raw.c
> +++ b/net/ipv4/raw.c
> @@ -381,6 +381,7 @@ static int raw_send_hdrinc(struct sock *sk, struct flowi4 *fl4,
>  
>  	skb->priority = sk->sk_priority;
>  	skb->mark = sk->sk_mark;
> +	skb->tstamp = sockc->transmit_time;
>  	skb_dst_set(skb, &rt->dst);
>  	*rtp = NULL;
>  
> @@ -562,6 +563,7 @@ static int raw_sendmsg(struct sock *sk, struct msghdr *msg, size_t len)
>  	}
>  
>  	ipc.sockc.tsflags = sk->sk_tsflags;
> +	ipc.sockc.transmit_time = 0;
>  	ipc.addr = inet->inet_saddr;
>  	ipc.opt = NULL;
>  	ipc.tx_flags = 0;


It seems that skb_scrub_packet() will clear skb->tstamp, meaning  that
going through a tunnel will break your feature.

Maybe we need to remove skb->tstamp clear from skb_scrub_packet() and
do the cleaning only in forwarding path.

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

* [Intel-wired-lan] [RFC v2 net-next 02/10] net: ipv4: raw: Hook into time based transmission.
@ 2018-01-18  0:28     ` Eric Dumazet
  0 siblings, 0 replies; 86+ messages in thread
From: Eric Dumazet @ 2018-01-18  0:28 UTC (permalink / raw)
  To: intel-wired-lan

On Wed, 2018-01-17 at 15:06 -0800, Jesus Sanchez-Palencia wrote:
> From: Richard Cochran <rcochran@linutronix.de>
> 
> For raw packets, copy the desired future transmit time from the CMSG
> cookie into the skb.
> 
> Signed-off-by: Richard Cochran <rcochran@linutronix.de>
> Signed-off-by: Jesus Sanchez-Palencia <jesus.sanchez-palencia@intel.com>
> ---
>  net/ipv4/raw.c | 2 ++
>  1 file changed, 2 insertions(+)
> 
> diff --git a/net/ipv4/raw.c b/net/ipv4/raw.c
> index 136544b36a46..e37ea8ab6a78 100644
> --- a/net/ipv4/raw.c
> +++ b/net/ipv4/raw.c
> @@ -381,6 +381,7 @@ static int raw_send_hdrinc(struct sock *sk, struct flowi4 *fl4,
>  
>  	skb->priority = sk->sk_priority;
>  	skb->mark = sk->sk_mark;
> +	skb->tstamp = sockc->transmit_time;
>  	skb_dst_set(skb, &rt->dst);
>  	*rtp = NULL;
>  
> @@ -562,6 +563,7 @@ static int raw_sendmsg(struct sock *sk, struct msghdr *msg, size_t len)
>  	}
>  
>  	ipc.sockc.tsflags = sk->sk_tsflags;
> +	ipc.sockc.transmit_time = 0;
>  	ipc.addr = inet->inet_saddr;
>  	ipc.opt = NULL;
>  	ipc.tx_flags = 0;


It seems that skb_scrub_packet() will clear skb->tstamp, meaning  that
going through a tunnel will break your feature.

Maybe we need to remove skb->tstamp clear from skb_scrub_packet() and
do the cleaning only in forwarding path.



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

* Re: [Intel-wired-lan] [RFC v2 net-next 01/10] net: Add a new socket option for a future transmit time.
  2018-01-17 23:06   ` [Intel-wired-lan] " Jesus Sanchez-Palencia
@ 2018-01-18  8:42     ` Miroslav Lichvar
  -1 siblings, 0 replies; 86+ messages in thread
From: Miroslav Lichvar @ 2018-01-18  8:42 UTC (permalink / raw)
  To: Jesus Sanchez-Palencia
  Cc: netdev, john.stultz, Richard Cochran, jiri, ivan.briano,
	richardcochran, henrik, jhs, levi.pearson, intel-wired-lan,
	xiyou.wangcong, tglx, anna-maria

On Wed, Jan 17, 2018 at 03:06:12PM -0800, Jesus Sanchez-Palencia wrote:
> From: Richard Cochran <rcochran@linutronix.de>
> 
> This patch introduces SO_TXTIME.  User space enables this option in
> order to pass a desired future transmit time in a CMSG when calling
> sendmsg(2).
> 
> A new field is added to struct sockcm_cookie, and the tstamp from
> skbuffs will be used later on.

In the discussion about the v1 patchset, there was a question if the
cmsg should include a clockid_t. Without that, how can an application
prevent the packet from being sent using an incorrect clock, e.g.
the system clock when it expects it to be a PHC, or a different PHC
when the socket is not bound to a specific interface?

At least in some applications it would be preferred to not sent a
packet at all instead of sending it at a wrong time.

Please keep in mind that the PHCs and the system clock don't have to
be synchronized to each other. If I understand the rest of the series
correctly, there is an assumption that the PHCs are keeping time in
TAI and CLOCK_TAI can be used as a fallback.

-- 
Miroslav Lichvar

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

* [Intel-wired-lan] [RFC v2 net-next 01/10] net: Add a new socket option for a future transmit time.
@ 2018-01-18  8:42     ` Miroslav Lichvar
  0 siblings, 0 replies; 86+ messages in thread
From: Miroslav Lichvar @ 2018-01-18  8:42 UTC (permalink / raw)
  To: intel-wired-lan

On Wed, Jan 17, 2018 at 03:06:12PM -0800, Jesus Sanchez-Palencia wrote:
> From: Richard Cochran <rcochran@linutronix.de>
> 
> This patch introduces SO_TXTIME.  User space enables this option in
> order to pass a desired future transmit time in a CMSG when calling
> sendmsg(2).
> 
> A new field is added to struct sockcm_cookie, and the tstamp from
> skbuffs will be used later on.

In the discussion about the v1 patchset, there was a question if the
cmsg should include a clockid_t. Without that, how can an application
prevent the packet from being sent using an incorrect clock, e.g.
the system clock when it expects it to be a PHC, or a different PHC
when the socket is not bound to a specific interface?

At least in some applications it would be preferred to not sent a
packet at all instead of sending it at a wrong time.

Please keep in mind that the PHCs and the system clock don't have to
be synchronized to each other. If I understand the rest of the series
correctly, there is an assumption that the PHCs are keeping time in
TAI and CLOCK_TAI can be used as a fallback.

-- 
Miroslav Lichvar

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

* Re: [RFC v2 net-next 06/10] net/sched: Introduce the TBS Qdisc
  2018-01-17 23:06   ` [Intel-wired-lan] " Jesus Sanchez-Palencia
@ 2018-01-18 13:35     ` Jamal Hadi Salim
  -1 siblings, 0 replies; 86+ messages in thread
From: Jamal Hadi Salim @ 2018-01-18 13:35 UTC (permalink / raw)
  To: Jesus Sanchez-Palencia, netdev
  Cc: xiyou.wangcong, jiri, vinicius.gomes, richardcochran,
	intel-wired-lan, anna-maria, henrik, tglx, john.stultz,
	andre.guedes, ivan.briano, levi.pearson

On 18-01-17 06:06 PM, Jesus Sanchez-Palencia wrote:
> From: Vinicius Costa Gomes <vinicius.gomes@intel.com>
> 
> TBS (Time Based Scheduler) uses the information added earlier in this
> series (the socket option SO_TXTIME and the new role of
> sk_buff->tstamp) to schedule traffic transmission based on absolute
> time.
> 
> For some workloads, just bandwidth enforcement is not enough, and
> precise control of the transmission of packets is necessary.
> 
> Example:
> 
> $ tc qdisc replace dev enp2s0 parent root handle 100 mqprio num_tc 3 \

handle 100:0 ?

>             map 2 2 1 0 2 2 2 2 2 2 2 2 2 2 2 2 queues 1@0 1@1 2@2 hw 0
> 
> $ tc qdisc add dev enp2s0 parent 100:1 tbs delta 60000 clockid 11 offload 1
>
>
> In this example, the Qdisc will try to enable offloading (offload 1)
> the control of the transmission time to the network adapter, the
> time stamp in socket are in reference to the clockid '11' (CLOCK_TAI)
> and packets leave the Qdisc "delta" (60000) nanoseconds before its
> transmission time.
>


> When offloading is disabled, the network adapter will ignore the
> sk_buff time stamp, and so, the transmission time will be only "best
> effort" from the Qdisc.
>

General comments:
1) iproute2: Avoid magic numbers like 1 or 11 please; "offload"
(without 1) and "TAI" will be more human friendly.

2) Experience shows that adding padding fields in the control structs
implies they will _never ever_ be used. That was not design intent
for netlink but over years shit like that has happened.
Maybe look at using a 32 bitmap? It is more "future proof".
You seem to only have 2-3 flags but it gives you opportunity
to add more changes later. If you are 100% sure youll never need
it - then maybe just move the tc_tbs_qopt::offload to the end of
of the struct.

3)It would be helpful for debugging to increment some stats other
than drop counters on enqueu/dequeue obsolete packet drop. Maybe use
overlimits for the dequeu drops (in addition)?

4) I may be misreading things - but did you need to reset the
watchdog on dequeue? It is already being kicked for every incoming packet.

cheers,
jamal

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

* [Intel-wired-lan] [RFC v2 net-next 06/10] net/sched: Introduce the TBS Qdisc
@ 2018-01-18 13:35     ` Jamal Hadi Salim
  0 siblings, 0 replies; 86+ messages in thread
From: Jamal Hadi Salim @ 2018-01-18 13:35 UTC (permalink / raw)
  To: intel-wired-lan

On 18-01-17 06:06 PM, Jesus Sanchez-Palencia wrote:
> From: Vinicius Costa Gomes <vinicius.gomes@intel.com>
> 
> TBS (Time Based Scheduler) uses the information added earlier in this
> series (the socket option SO_TXTIME and the new role of
> sk_buff->tstamp) to schedule traffic transmission based on absolute
> time.
> 
> For some workloads, just bandwidth enforcement is not enough, and
> precise control of the transmission of packets is necessary.
> 
> Example:
> 
> $ tc qdisc replace dev enp2s0 parent root handle 100 mqprio num_tc 3 \

handle 100:0 ?

>             map 2 2 1 0 2 2 2 2 2 2 2 2 2 2 2 2 queues 1 at 0 1 at 1 2 at 2 hw 0
> 
> $ tc qdisc add dev enp2s0 parent 100:1 tbs delta 60000 clockid 11 offload 1
>
>
> In this example, the Qdisc will try to enable offloading (offload 1)
> the control of the transmission time to the network adapter, the
> time stamp in socket are in reference to the clockid '11' (CLOCK_TAI)
> and packets leave the Qdisc "delta" (60000) nanoseconds before its
> transmission time.
>


> When offloading is disabled, the network adapter will ignore the
> sk_buff time stamp, and so, the transmission time will be only "best
> effort" from the Qdisc.
>

General comments:
1) iproute2: Avoid magic numbers like 1 or 11 please; "offload"
(without 1) and "TAI" will be more human friendly.

2) Experience shows that adding padding fields in the control structs
implies they will _never ever_ be used. That was not design intent
for netlink but over years shit like that has happened.
Maybe look at using a 32 bitmap? It is more "future proof".
You seem to only have 2-3 flags but it gives you opportunity
to add more changes later. If you are 100% sure youll never need
it - then maybe just move the tc_tbs_qopt::offload to the end of
of the struct.

3)It would be helpful for debugging to increment some stats other
than drop counters on enqueu/dequeue obsolete packet drop. Maybe use
overlimits for the dequeu drops (in addition)?

4) I may be misreading things - but did you need to reset the
watchdog on dequeue? It is already being kicked for every incoming packet.

cheers,
jamal

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

* Re: [RFC v2 net-next 06/10] net/sched: Introduce the TBS Qdisc
  2018-01-18 13:35     ` [Intel-wired-lan] " Jamal Hadi Salim
@ 2018-01-18 13:44       ` Jamal Hadi Salim
  -1 siblings, 0 replies; 86+ messages in thread
From: Jamal Hadi Salim @ 2018-01-18 13:44 UTC (permalink / raw)
  To: Jesus Sanchez-Palencia, netdev
  Cc: xiyou.wangcong, jiri, vinicius.gomes, richardcochran,
	intel-wired-lan, anna-maria, henrik, tglx, john.stultz,
	andre.guedes, ivan.briano, levi.pearson

One more comment:
Probably try to run a test with a very small delta with
no offload (probably using something like prio as the root qdisc)
and dump the stats.
My gut feeling is your accounting of the backlog in particular is off.

cheers,
jamal

On 18-01-18 08:35 AM, Jamal Hadi Salim wrote:
> On 18-01-17 06:06 PM, Jesus Sanchez-Palencia wrote:
>> From: Vinicius Costa Gomes <vinicius.gomes@intel.com>
>>
>> TBS (Time Based Scheduler) uses the information added earlier in this
>> series (the socket option SO_TXTIME and the new role of
>> sk_buff->tstamp) to schedule traffic transmission based on absolute
>> time.
>>
>> For some workloads, just bandwidth enforcement is not enough, and
>> precise control of the transmission of packets is necessary.
>>
>> Example:
>>
>> $ tc qdisc replace dev enp2s0 parent root handle 100 mqprio num_tc 3 \
> 
> handle 100:0 ?
> 
>>             map 2 2 1 0 2 2 2 2 2 2 2 2 2 2 2 2 queues 1@0 1@1 2@2 hw 0
>>
>> $ tc qdisc add dev enp2s0 parent 100:1 tbs delta 60000 clockid 11 
>> offload 1
>>
>>
>> In this example, the Qdisc will try to enable offloading (offload 1)
>> the control of the transmission time to the network adapter, the
>> time stamp in socket are in reference to the clockid '11' (CLOCK_TAI)
>> and packets leave the Qdisc "delta" (60000) nanoseconds before its
>> transmission time.
>>
> 
> 
>> When offloading is disabled, the network adapter will ignore the
>> sk_buff time stamp, and so, the transmission time will be only "best
>> effort" from the Qdisc.
>>
> 
> General comments:
> 1) iproute2: Avoid magic numbers like 1 or 11 please; "offload"
> (without 1) and "TAI" will be more human friendly.
> 
> 2) Experience shows that adding padding fields in the control structs
> implies they will _never ever_ be used. That was not design intent
> for netlink but over years shit like that has happened.
> Maybe look at using a 32 bitmap? It is more "future proof".
> You seem to only have 2-3 flags but it gives you opportunity
> to add more changes later. If you are 100% sure youll never need
> it - then maybe just move the tc_tbs_qopt::offload to the end of
> of the struct.
> 
> 3)It would be helpful for debugging to increment some stats other
> than drop counters on enqueu/dequeue obsolete packet drop. Maybe use
> overlimits for the dequeu drops (in addition)?
> 
> 4) I may be misreading things - but did you need to reset the
> watchdog on dequeue? It is already being kicked for every incoming packet.
> 
> cheers,
> jamal

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

* [Intel-wired-lan] [RFC v2 net-next 06/10] net/sched: Introduce the TBS Qdisc
@ 2018-01-18 13:44       ` Jamal Hadi Salim
  0 siblings, 0 replies; 86+ messages in thread
From: Jamal Hadi Salim @ 2018-01-18 13:44 UTC (permalink / raw)
  To: intel-wired-lan

One more comment:
Probably try to run a test with a very small delta with
no offload (probably using something like prio as the root qdisc)
and dump the stats.
My gut feeling is your accounting of the backlog in particular is off.

cheers,
jamal

On 18-01-18 08:35 AM, Jamal Hadi Salim wrote:
> On 18-01-17 06:06 PM, Jesus Sanchez-Palencia wrote:
>> From: Vinicius Costa Gomes <vinicius.gomes@intel.com>
>>
>> TBS (Time Based Scheduler) uses the information added earlier in this
>> series (the socket option SO_TXTIME and the new role of
>> sk_buff->tstamp) to schedule traffic transmission based on absolute
>> time.
>>
>> For some workloads, just bandwidth enforcement is not enough, and
>> precise control of the transmission of packets is necessary.
>>
>> Example:
>>
>> $ tc qdisc replace dev enp2s0 parent root handle 100 mqprio num_tc 3 \
> 
> handle 100:0 ?
> 
>> ??????????? map 2 2 1 0 2 2 2 2 2 2 2 2 2 2 2 2 queues 1 at 0 1 at 1 2 at 2 hw 0
>>
>> $ tc qdisc add dev enp2s0 parent 100:1 tbs delta 60000 clockid 11 
>> offload 1
>>
>>
>> In this example, the Qdisc will try to enable offloading (offload 1)
>> the control of the transmission time to the network adapter, the
>> time stamp in socket are in reference to the clockid '11' (CLOCK_TAI)
>> and packets leave the Qdisc "delta" (60000) nanoseconds before its
>> transmission time.
>>
> 
> 
>> When offloading is disabled, the network adapter will ignore the
>> sk_buff time stamp, and so, the transmission time will be only "best
>> effort" from the Qdisc.
>>
> 
> General comments:
> 1) iproute2: Avoid magic numbers like 1 or 11 please; "offload"
> (without 1) and "TAI" will be more human friendly.
> 
> 2) Experience shows that adding padding fields in the control structs
> implies they will _never ever_ be used. That was not design intent
> for netlink but over years shit like that has happened.
> Maybe look at using a 32 bitmap? It is more "future proof".
> You seem to only have 2-3 flags but it gives you opportunity
> to add more changes later. If you are 100% sure youll never need
> it - then maybe just move the tc_tbs_qopt::offload to the end of
> of the struct.
> 
> 3)It would be helpful for debugging to increment some stats other
> than drop counters on enqueu/dequeue obsolete packet drop. Maybe use
> overlimits for the dequeu drops (in addition)?
> 
> 4) I may be misreading things - but did you need to reset the
> watchdog on dequeue? It is already being kicked for every incoming packet.
> 
> cheers,
> jamal


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

* Re: [RFC v2 net-next 01/10] net: Add a new socket option for a future transmit time.
  2018-01-17 23:06   ` [Intel-wired-lan] " Jesus Sanchez-Palencia
@ 2018-01-18 17:11     ` Richard Cochran
  -1 siblings, 0 replies; 86+ messages in thread
From: Richard Cochran @ 2018-01-18 17:11 UTC (permalink / raw)
  To: Jesus Sanchez-Palencia
  Cc: netdev, jhs, xiyou.wangcong, jiri, vinicius.gomes,
	intel-wired-lan, anna-maria, henrik, tglx, john.stultz,
	andre.guedes, ivan.briano, levi.pearson, Richard Cochran

On Wed, Jan 17, 2018 at 03:06:12PM -0800, Jesus Sanchez-Palencia wrote:
> @@ -2130,6 +2137,15 @@ int __sock_cmsg_send(struct sock *sk, struct msghdr *msg, struct cmsghdr *cmsg,
>  		sockc->tsflags &= ~SOF_TIMESTAMPING_TX_RECORD_MASK;
>  		sockc->tsflags |= tsflags;
>  		break;
> +	case SO_TXTIME:
> +		if (!ns_capable(sock_net(sk)->user_ns, CAP_NET_ADMIN))
> +			return -EPERM;
> +		if (!sock_flag(sk, SOCK_TXTIME))
> +			return -EINVAL;
> +		if (cmsg->cmsg_len != CMSG_LEN(sizeof(ktime_t)))
> +			return -EINVAL;
> +		sockc->transmit_time = *(ktime_t *)CMSG_DATA(cmsg);

As pointed out in the first series' review:

  No guarantee the CMSG is properly aligned on arches that might trap
  on unaligned access.

Thanks,
Richard

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

* [Intel-wired-lan] [RFC v2 net-next 01/10] net: Add a new socket option for a future transmit time.
@ 2018-01-18 17:11     ` Richard Cochran
  0 siblings, 0 replies; 86+ messages in thread
From: Richard Cochran @ 2018-01-18 17:11 UTC (permalink / raw)
  To: intel-wired-lan

On Wed, Jan 17, 2018 at 03:06:12PM -0800, Jesus Sanchez-Palencia wrote:
> @@ -2130,6 +2137,15 @@ int __sock_cmsg_send(struct sock *sk, struct msghdr *msg, struct cmsghdr *cmsg,
>  		sockc->tsflags &= ~SOF_TIMESTAMPING_TX_RECORD_MASK;
>  		sockc->tsflags |= tsflags;
>  		break;
> +	case SO_TXTIME:
> +		if (!ns_capable(sock_net(sk)->user_ns, CAP_NET_ADMIN))
> +			return -EPERM;
> +		if (!sock_flag(sk, SOCK_TXTIME))
> +			return -EINVAL;
> +		if (cmsg->cmsg_len != CMSG_LEN(sizeof(ktime_t)))
> +			return -EINVAL;
> +		sockc->transmit_time = *(ktime_t *)CMSG_DATA(cmsg);

As pointed out in the first series' review:

  No guarantee the CMSG is properly aligned on arches that might trap
  on unaligned access.

Thanks,
Richard

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

* Re: [Intel-wired-lan] [RFC v2 net-next 01/10] net: Add a new socket option for a future transmit time.
  2018-01-18  8:42     ` Miroslav Lichvar
@ 2018-01-18 17:13       ` Richard Cochran
  -1 siblings, 0 replies; 86+ messages in thread
From: Richard Cochran @ 2018-01-18 17:13 UTC (permalink / raw)
  To: Miroslav Lichvar
  Cc: Jesus Sanchez-Palencia, netdev, john.stultz, Richard Cochran,
	jiri, ivan.briano, henrik, jhs, levi.pearson, intel-wired-lan,
	xiyou.wangcong, tglx, anna-maria

On Thu, Jan 18, 2018 at 09:42:27AM +0100, Miroslav Lichvar wrote:
> In the discussion about the v1 patchset, there was a question if the
> cmsg should include a clockid_t. Without that, how can an application
> prevent the packet from being sent using an incorrect clock, e.g.
> the system clock when it expects it to be a PHC, or a different PHC
> when the socket is not bound to a specific interface?

Right, the clockid_t should be passed in through the CMSG along with
the time.
 
Thanks,
Richard

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

* [Intel-wired-lan] [RFC v2 net-next 01/10] net: Add a new socket option for a future transmit time.
@ 2018-01-18 17:13       ` Richard Cochran
  0 siblings, 0 replies; 86+ messages in thread
From: Richard Cochran @ 2018-01-18 17:13 UTC (permalink / raw)
  To: intel-wired-lan

On Thu, Jan 18, 2018 at 09:42:27AM +0100, Miroslav Lichvar wrote:
> In the discussion about the v1 patchset, there was a question if the
> cmsg should include a clockid_t. Without that, how can an application
> prevent the packet from being sent using an incorrect clock, e.g.
> the system clock when it expects it to be a PHC, or a different PHC
> when the socket is not bound to a specific interface?

Right, the clockid_t should be passed in through the CMSG along with
the time.
 
Thanks,
Richard

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

* Re: [RFC v2 net-next 06/10] net/sched: Introduce the TBS Qdisc
  2018-01-18 13:35     ` [Intel-wired-lan] " Jamal Hadi Salim
@ 2018-01-18 17:18       ` Richard Cochran
  -1 siblings, 0 replies; 86+ messages in thread
From: Richard Cochran @ 2018-01-18 17:18 UTC (permalink / raw)
  To: Jamal Hadi Salim
  Cc: Jesus Sanchez-Palencia, netdev, xiyou.wangcong, jiri,
	vinicius.gomes, intel-wired-lan, anna-maria, henrik, tglx,
	john.stultz, andre.guedes, ivan.briano, levi.pearson

On Thu, Jan 18, 2018 at 08:35:18AM -0500, Jamal Hadi Salim wrote:
> 1) iproute2: Avoid magic numbers like 1 or 11 please; "offload"
> (without 1) and "TAI" will be more human friendly.

Yes, and for the clockid, the program should accept CLOCK_REALTIME or
CLOCK_TAI for the hard coded SYS-V IDs or /dev/ptp0 for dynamic IDs.

Thanks,
Richard

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

* [Intel-wired-lan] [RFC v2 net-next 06/10] net/sched: Introduce the TBS Qdisc
@ 2018-01-18 17:18       ` Richard Cochran
  0 siblings, 0 replies; 86+ messages in thread
From: Richard Cochran @ 2018-01-18 17:18 UTC (permalink / raw)
  To: intel-wired-lan

On Thu, Jan 18, 2018 at 08:35:18AM -0500, Jamal Hadi Salim wrote:
> 1) iproute2: Avoid magic numbers like 1 or 11 please; "offload"
> (without 1) and "TAI" will be more human friendly.

Yes, and for the clockid, the program should accept CLOCK_REALTIME or
CLOCK_TAI for the hard coded SYS-V IDs or /dev/ptp0 for dynamic IDs.

Thanks,
Richard

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

* Re: [RFC v2 net-next 01/10] net: Add a new socket option for a future transmit time.
  2018-01-17 23:06   ` [Intel-wired-lan] " Jesus Sanchez-Palencia
@ 2018-01-19 21:15     ` Willem de Bruijn
  -1 siblings, 0 replies; 86+ messages in thread
From: Willem de Bruijn @ 2018-01-19 21:15 UTC (permalink / raw)
  To: Jesus Sanchez-Palencia
  Cc: Network Development, Jamal Hadi Salim, Cong Wang,
	Jiří Pírko, vinicius.gomes, Richard Cochran,
	intel-wired-lan, anna-maria, henrik, Thomas Gleixner,
	John Stultz, andre.guedes, ivan.briano, levi.pearson,
	Richard Cochran

On Wed, Jan 17, 2018 at 6:06 PM, Jesus Sanchez-Palencia
<jesus.sanchez-palencia@intel.com> wrote:
> From: Richard Cochran <rcochran@linutronix.de>
>
> This patch introduces SO_TXTIME.  User space enables this option in
> order to pass a desired future transmit time in a CMSG when calling
> sendmsg(2).
>
> A new field is added to struct sockcm_cookie, and the tstamp from
> skbuffs will be used later on.
>
> Signed-off-by: Richard Cochran <rcochran@linutronix.de>
> Signed-off-by: Jesus Sanchez-Palencia <jesus.sanchez-palencia@intel.com>
> ---

> diff --git a/net/core/sock.c b/net/core/sock.c
> index abf4cbff99b2..37ef4b33fd92 100644
> --- a/net/core/sock.c
> +++ b/net/core/sock.c
> @@ -1061,6 +1061,13 @@ int sock_setsockopt(struct socket *sock, int level, int optname,
>                         sock_valbool_flag(sk, SOCK_ZEROCOPY, valbool);
>                 break;
>
> +       case SO_TXTIME:
> +               if (ns_capable(sock_net(sk)->user_ns, CAP_NET_ADMIN))
> +                       sock_valbool_flag(sk, SOCK_TXTIME, valbool);
> +               else
> +                       ret = -EPERM;
> +               break;
> +
>         default:
>                 ret = -ENOPROTOOPT;
>                 break;

Please add getsockopt alongside setsockopt.

I would also restrict input to [0, 1] exactly to allow for future extensions.

If using ns_capable, skb->tstamp must continue to be scrubbed when traversing
network namespaces.

> @@ -2130,6 +2137,15 @@ int __sock_cmsg_send(struct sock *sk, struct msghdr *msg, struct cmsghdr *cmsg,
>                 sockc->tsflags &= ~SOF_TIMESTAMPING_TX_RECORD_MASK;
>                 sockc->tsflags |= tsflags;
>                 break;
> +       case SO_TXTIME:
> +               if (!ns_capable(sock_net(sk)->user_ns, CAP_NET_ADMIN))
> +                       return -EPERM;
> +               if (!sock_flag(sk, SOCK_TXTIME))
> +                       return -EINVAL;

No need for ns_capable check on each packet when already required to
toggle socket option.

> +               if (cmsg->cmsg_len != CMSG_LEN(sizeof(ktime_t)))
> +                       return -EINVAL;

I don't see any existing reference to ktime_t in include/uapi. Just use a s64?

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

* [Intel-wired-lan] [RFC v2 net-next 01/10] net: Add a new socket option for a future transmit time.
@ 2018-01-19 21:15     ` Willem de Bruijn
  0 siblings, 0 replies; 86+ messages in thread
From: Willem de Bruijn @ 2018-01-19 21:15 UTC (permalink / raw)
  To: intel-wired-lan

On Wed, Jan 17, 2018 at 6:06 PM, Jesus Sanchez-Palencia
<jesus.sanchez-palencia@intel.com> wrote:
> From: Richard Cochran <rcochran@linutronix.de>
>
> This patch introduces SO_TXTIME.  User space enables this option in
> order to pass a desired future transmit time in a CMSG when calling
> sendmsg(2).
>
> A new field is added to struct sockcm_cookie, and the tstamp from
> skbuffs will be used later on.
>
> Signed-off-by: Richard Cochran <rcochran@linutronix.de>
> Signed-off-by: Jesus Sanchez-Palencia <jesus.sanchez-palencia@intel.com>
> ---

> diff --git a/net/core/sock.c b/net/core/sock.c
> index abf4cbff99b2..37ef4b33fd92 100644
> --- a/net/core/sock.c
> +++ b/net/core/sock.c
> @@ -1061,6 +1061,13 @@ int sock_setsockopt(struct socket *sock, int level, int optname,
>                         sock_valbool_flag(sk, SOCK_ZEROCOPY, valbool);
>                 break;
>
> +       case SO_TXTIME:
> +               if (ns_capable(sock_net(sk)->user_ns, CAP_NET_ADMIN))
> +                       sock_valbool_flag(sk, SOCK_TXTIME, valbool);
> +               else
> +                       ret = -EPERM;
> +               break;
> +
>         default:
>                 ret = -ENOPROTOOPT;
>                 break;

Please add getsockopt alongside setsockopt.

I would also restrict input to [0, 1] exactly to allow for future extensions.

If using ns_capable, skb->tstamp must continue to be scrubbed when traversing
network namespaces.

> @@ -2130,6 +2137,15 @@ int __sock_cmsg_send(struct sock *sk, struct msghdr *msg, struct cmsghdr *cmsg,
>                 sockc->tsflags &= ~SOF_TIMESTAMPING_TX_RECORD_MASK;
>                 sockc->tsflags |= tsflags;
>                 break;
> +       case SO_TXTIME:
> +               if (!ns_capable(sock_net(sk)->user_ns, CAP_NET_ADMIN))
> +                       return -EPERM;
> +               if (!sock_flag(sk, SOCK_TXTIME))
> +                       return -EINVAL;

No need for ns_capable check on each packet when already required to
toggle socket option.

> +               if (cmsg->cmsg_len != CMSG_LEN(sizeof(ktime_t)))
> +                       return -EINVAL;

I don't see any existing reference to ktime_t in include/uapi. Just use a s64?

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

* Re: [RFC v2 net-next 06/10] net/sched: Introduce the TBS Qdisc
  2018-01-17 23:06   ` [Intel-wired-lan] " Jesus Sanchez-Palencia
@ 2018-01-19 21:18     ` Willem de Bruijn
  -1 siblings, 0 replies; 86+ messages in thread
From: Willem de Bruijn @ 2018-01-19 21:18 UTC (permalink / raw)
  To: Jesus Sanchez-Palencia
  Cc: Network Development, Jamal Hadi Salim, Cong Wang,
	Jiří Pírko, vinicius.gomes, Richard Cochran,
	intel-wired-lan, anna-maria, henrik, Thomas Gleixner,
	John Stultz, andre.guedes, ivan.briano, levi.pearson

On Wed, Jan 17, 2018 at 6:06 PM, Jesus Sanchez-Palencia
<jesus.sanchez-palencia@intel.com> wrote:
> From: Vinicius Costa Gomes <vinicius.gomes@intel.com>
>
> TBS (Time Based Scheduler) uses the information added earlier in this
> series (the socket option SO_TXTIME and the new role of
> sk_buff->tstamp) to schedule traffic transmission based on absolute
> time.
>
> For some workloads, just bandwidth enforcement is not enough, and
> precise control of the transmission of packets is necessary.
>
> Example:
>
> $ tc qdisc replace dev enp2s0 parent root handle 100 mqprio num_tc 3 \
>            map 2 2 1 0 2 2 2 2 2 2 2 2 2 2 2 2 queues 1@0 1@1 2@2 hw 0
>
> $ tc qdisc add dev enp2s0 parent 100:1 tbs delta 60000 clockid 11 offload 1
>
> In this example, the Qdisc will try to enable offloading (offload 1)
> the control of the transmission time to the network adapter, the
> time stamp in socket are in reference to the clockid '11' (CLOCK_TAI)
> and packets leave the Qdisc "delta" (60000) nanoseconds before its
> transmission time.
>
> When offloading is disabled, the network adapter will ignore the
> sk_buff time stamp, and so, the transmission time will be only "best
> effort" from the Qdisc.
>
> Signed-off-by: Vinicius Costa Gomes <vinicius.gomes@intel.com>
> Signed-off-by: Jesus Sanchez-Palencia <jesus.sanchez-palencia@intel.com>
> ---

> +static struct sk_buff *timerqueue_erase(struct Qdisc *sch,
> +                                       struct sk_buff *skb, bool drop)
> +{
> +       struct tbs_sched_data *q = qdisc_priv(sch);
> +
> +       rb_erase(&skb->rbnode, &q->head);
> +
> +       if (drop) {
> +               struct sk_buff *to_free = NULL;
> +
> +               qdisc_drop(skb, sch, &to_free);
> +               kfree_skb_list(to_free);
> +       } else {
> +               qdisc_qstats_backlog_dec(sch, skb);
> +               qdisc_bstats_update(sch, skb);
> +
> +               q->last = skb->tstamp;
> +       }
> +
> +       sch->q.qlen--;
> +
> +       /* The rbnode field in the skb re-uses these fields, now that
> +        * we are done with the rbnode, reset them.
> +        */
> +       skb->next = NULL;
> +       skb->prev = NULL;
> +       skb->dev = qdisc_dev(sch);
> +
> +       return skb;
> +}

Return value is not used in either caller.

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

* [Intel-wired-lan] [RFC v2 net-next 06/10] net/sched: Introduce the TBS Qdisc
@ 2018-01-19 21:18     ` Willem de Bruijn
  0 siblings, 0 replies; 86+ messages in thread
From: Willem de Bruijn @ 2018-01-19 21:18 UTC (permalink / raw)
  To: intel-wired-lan

On Wed, Jan 17, 2018 at 6:06 PM, Jesus Sanchez-Palencia
<jesus.sanchez-palencia@intel.com> wrote:
> From: Vinicius Costa Gomes <vinicius.gomes@intel.com>
>
> TBS (Time Based Scheduler) uses the information added earlier in this
> series (the socket option SO_TXTIME and the new role of
> sk_buff->tstamp) to schedule traffic transmission based on absolute
> time.
>
> For some workloads, just bandwidth enforcement is not enough, and
> precise control of the transmission of packets is necessary.
>
> Example:
>
> $ tc qdisc replace dev enp2s0 parent root handle 100 mqprio num_tc 3 \
>            map 2 2 1 0 2 2 2 2 2 2 2 2 2 2 2 2 queues 1 at 0 1 at 1 2 at 2 hw 0
>
> $ tc qdisc add dev enp2s0 parent 100:1 tbs delta 60000 clockid 11 offload 1
>
> In this example, the Qdisc will try to enable offloading (offload 1)
> the control of the transmission time to the network adapter, the
> time stamp in socket are in reference to the clockid '11' (CLOCK_TAI)
> and packets leave the Qdisc "delta" (60000) nanoseconds before its
> transmission time.
>
> When offloading is disabled, the network adapter will ignore the
> sk_buff time stamp, and so, the transmission time will be only "best
> effort" from the Qdisc.
>
> Signed-off-by: Vinicius Costa Gomes <vinicius.gomes@intel.com>
> Signed-off-by: Jesus Sanchez-Palencia <jesus.sanchez-palencia@intel.com>
> ---

> +static struct sk_buff *timerqueue_erase(struct Qdisc *sch,
> +                                       struct sk_buff *skb, bool drop)
> +{
> +       struct tbs_sched_data *q = qdisc_priv(sch);
> +
> +       rb_erase(&skb->rbnode, &q->head);
> +
> +       if (drop) {
> +               struct sk_buff *to_free = NULL;
> +
> +               qdisc_drop(skb, sch, &to_free);
> +               kfree_skb_list(to_free);
> +       } else {
> +               qdisc_qstats_backlog_dec(sch, skb);
> +               qdisc_bstats_update(sch, skb);
> +
> +               q->last = skb->tstamp;
> +       }
> +
> +       sch->q.qlen--;
> +
> +       /* The rbnode field in the skb re-uses these fields, now that
> +        * we are done with the rbnode, reset them.
> +        */
> +       skb->next = NULL;
> +       skb->prev = NULL;
> +       skb->dev = qdisc_dev(sch);
> +
> +       return skb;
> +}

Return value is not used in either caller.

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

* Re: [RFC v2 net-next 01/10] net: Add a new socket option for a future transmit time.
  2018-01-19 21:15     ` [Intel-wired-lan] " Willem de Bruijn
@ 2018-01-20  2:09       ` Richard Cochran
  -1 siblings, 0 replies; 86+ messages in thread
From: Richard Cochran @ 2018-01-20  2:09 UTC (permalink / raw)
  To: Willem de Bruijn
  Cc: Jesus Sanchez-Palencia, Network Development, Jamal Hadi Salim,
	Cong Wang, Jiří Pírko, vinicius.gomes,
	intel-wired-lan, anna-maria, henrik, Thomas Gleixner,
	John Stultz, andre.guedes, ivan.briano, levi.pearson,
	Richard Cochran

On Fri, Jan 19, 2018 at 04:15:46PM -0500, Willem de Bruijn wrote:
> > +               if (cmsg->cmsg_len != CMSG_LEN(sizeof(ktime_t)))
> > +                       return -EINVAL;
> 
> I don't see any existing reference to ktime_t in include/uapi. Just use a s64?

Agreed.  I didn't see the point of switching to ktime, either.

Thanks,
Richard

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

* [Intel-wired-lan] [RFC v2 net-next 01/10] net: Add a new socket option for a future transmit time.
@ 2018-01-20  2:09       ` Richard Cochran
  0 siblings, 0 replies; 86+ messages in thread
From: Richard Cochran @ 2018-01-20  2:09 UTC (permalink / raw)
  To: intel-wired-lan

On Fri, Jan 19, 2018 at 04:15:46PM -0500, Willem de Bruijn wrote:
> > +               if (cmsg->cmsg_len != CMSG_LEN(sizeof(ktime_t)))
> > +                       return -EINVAL;
> 
> I don't see any existing reference to ktime_t in include/uapi. Just use a s64?

Agreed.  I didn't see the point of switching to ktime, either.

Thanks,
Richard

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

* Re: [RFC v2 net-next 00/10] Time based packet transmission
  2018-01-17 23:06 ` [Intel-wired-lan] " Jesus Sanchez-Palencia
@ 2018-01-23  5:23   ` Richard Cochran
  -1 siblings, 0 replies; 86+ messages in thread
From: Richard Cochran @ 2018-01-23  5:23 UTC (permalink / raw)
  To: Jesus Sanchez-Palencia
  Cc: netdev, jhs, xiyou.wangcong, jiri, vinicius.gomes,
	intel-wired-lan, anna-maria, henrik, tglx, john.stultz,
	andre.guedes, ivan.briano, levi.pearson

On Wed, Jan 17, 2018 at 03:06:11PM -0800, Jesus Sanchez-Palencia wrote:
> First, a baseline test was ran for 10 minutes with the plain kernel only:
> 
> |                 | plain kernel @ 1ms |
> |-----------------+--------------------+
> | min (ns):       |    +4.820000e+02   |
> | max (ns):       |    +9.999300e+05   |
> | pk-pk:          |    +9.994480e+05   |

...

> |                 |    tbs SW @ 1ms   |  tbs HW @ 1ms  | tbs HW @ 250 us |
> |-----------------+-------------------+----------------+-----------------|
> | min (ns):       |    +1.510000e+02  |  +4.420000e+02 |   +4.260000e+02 |
> | max (ns):       |    +9.977030e+05  |  +5.060000e+02 |   +5.060000e+02 |
> | pk-pk:          |    +9.975520e+05  |  +6.400000e+01 |   +8.000000e+01 |

I wonder about these worst case measurements of 999 and 998
milliseconds.  It almost looks like you missed one entire period.
Could this simply be a bug in the test setup?

Thanks,
Richard

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

* [Intel-wired-lan] [RFC v2 net-next 00/10] Time based packet transmission
@ 2018-01-23  5:23   ` Richard Cochran
  0 siblings, 0 replies; 86+ messages in thread
From: Richard Cochran @ 2018-01-23  5:23 UTC (permalink / raw)
  To: intel-wired-lan

On Wed, Jan 17, 2018 at 03:06:11PM -0800, Jesus Sanchez-Palencia wrote:
> First, a baseline test was ran for 10 minutes with the plain kernel only:
> 
> |                 | plain kernel @ 1ms |
> |-----------------+--------------------+
> | min (ns):       |    +4.820000e+02   |
> | max (ns):       |    +9.999300e+05   |
> | pk-pk:          |    +9.994480e+05   |

...

> |                 |    tbs SW @ 1ms   |  tbs HW @ 1ms  | tbs HW @ 250 us |
> |-----------------+-------------------+----------------+-----------------|
> | min (ns):       |    +1.510000e+02  |  +4.420000e+02 |   +4.260000e+02 |
> | max (ns):       |    +9.977030e+05  |  +5.060000e+02 |   +5.060000e+02 |
> | pk-pk:          |    +9.975520e+05  |  +6.400000e+01 |   +8.000000e+01 |

I wonder about these worst case measurements of 999 and 998
milliseconds.  It almost looks like you missed one entire period.
Could this simply be a bug in the test setup?

Thanks,
Richard

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

* Re: [RFC v2 net-next 00/10] Time based packet transmission
  2018-01-23  5:23   ` [Intel-wired-lan] " Richard Cochran
@ 2018-01-23  5:26     ` Richard Cochran
  -1 siblings, 0 replies; 86+ messages in thread
From: Richard Cochran @ 2018-01-23  5:26 UTC (permalink / raw)
  To: Jesus Sanchez-Palencia
  Cc: netdev, jhs, xiyou.wangcong, jiri, vinicius.gomes,
	intel-wired-lan, anna-maria, henrik, tglx, john.stultz,
	andre.guedes, ivan.briano, levi.pearson

On Mon, Jan 22, 2018 at 09:23:27PM -0800, Richard Cochran wrote:
> On Wed, Jan 17, 2018 at 03:06:11PM -0800, Jesus Sanchez-Palencia wrote:
> > First, a baseline test was ran for 10 minutes with the plain kernel only:
> > 
> > |                 | plain kernel @ 1ms |
> > |-----------------+--------------------+
> > | min (ns):       |    +4.820000e+02   |
> > | max (ns):       |    +9.999300e+05   |
> > | pk-pk:          |    +9.994480e+05   |
> 
> ...
> 
> > |                 |    tbs SW @ 1ms   |  tbs HW @ 1ms  | tbs HW @ 250 us |
> > |-----------------+-------------------+----------------+-----------------|
> > | min (ns):       |    +1.510000e+02  |  +4.420000e+02 |   +4.260000e+02 |
> > | max (ns):       |    +9.977030e+05  |  +5.060000e+02 |   +5.060000e+02 |
> > | pk-pk:          |    +9.975520e+05  |  +6.400000e+01 |   +8.000000e+01 |
> 
> I wonder about these worst case measurements of 999 and 998
> milliseconds.  It almost looks like you missed one entire period.
  ^^^^
microseconds

> Could this simply be a bug in the test setup?
> 
> Thanks,
> Richard

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

* [Intel-wired-lan] [RFC v2 net-next 00/10] Time based packet transmission
@ 2018-01-23  5:26     ` Richard Cochran
  0 siblings, 0 replies; 86+ messages in thread
From: Richard Cochran @ 2018-01-23  5:26 UTC (permalink / raw)
  To: intel-wired-lan

On Mon, Jan 22, 2018 at 09:23:27PM -0800, Richard Cochran wrote:
> On Wed, Jan 17, 2018 at 03:06:11PM -0800, Jesus Sanchez-Palencia wrote:
> > First, a baseline test was ran for 10 minutes with the plain kernel only:
> > 
> > |                 | plain kernel @ 1ms |
> > |-----------------+--------------------+
> > | min (ns):       |    +4.820000e+02   |
> > | max (ns):       |    +9.999300e+05   |
> > | pk-pk:          |    +9.994480e+05   |
> 
> ...
> 
> > |                 |    tbs SW @ 1ms   |  tbs HW @ 1ms  | tbs HW @ 250 us |
> > |-----------------+-------------------+----------------+-----------------|
> > | min (ns):       |    +1.510000e+02  |  +4.420000e+02 |   +4.260000e+02 |
> > | max (ns):       |    +9.977030e+05  |  +5.060000e+02 |   +5.060000e+02 |
> > | pk-pk:          |    +9.975520e+05  |  +6.400000e+01 |   +8.000000e+01 |
> 
> I wonder about these worst case measurements of 999 and 998
> milliseconds.  It almost looks like you missed one entire period.
  ^^^^
microseconds

> Could this simply be a bug in the test setup?
> 
> Thanks,
> Richard

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

* Re: [RFC v2 net-next 00/10] Time based packet transmission
  2018-01-23  5:26     ` [Intel-wired-lan] " Richard Cochran
@ 2018-01-23 18:07       ` Jesus Sanchez-Palencia
  -1 siblings, 0 replies; 86+ messages in thread
From: Jesus Sanchez-Palencia @ 2018-01-23 18:07 UTC (permalink / raw)
  To: Richard Cochran
  Cc: netdev, jhs, xiyou.wangcong, jiri, vinicius.gomes,
	intel-wired-lan, anna-maria, henrik, tglx, john.stultz,
	andre.guedes, ivan.briano, levi.pearson

Hi,


On 01/22/2018 09:26 PM, Richard Cochran wrote:
> On Mon, Jan 22, 2018 at 09:23:27PM -0800, Richard Cochran wrote:
>> On Wed, Jan 17, 2018 at 03:06:11PM -0800, Jesus Sanchez-Palencia wrote:
>>> First, a baseline test was ran for 10 minutes with the plain kernel only:
>>>
>>> |                 | plain kernel @ 1ms |
>>> |-----------------+--------------------+
>>> | min (ns):       |    +4.820000e+02   |
>>> | max (ns):       |    +9.999300e+05   |
>>> | pk-pk:          |    +9.994480e+05   |
>>
>> ...
>>
>>> |                 |    tbs SW @ 1ms   |  tbs HW @ 1ms  | tbs HW @ 250 us |
>>> |-----------------+-------------------+----------------+-----------------|
>>> | min (ns):       |    +1.510000e+02  |  +4.420000e+02 |   +4.260000e+02 |
>>> | max (ns):       |    +9.977030e+05  |  +5.060000e+02 |   +5.060000e+02 |
>>> | pk-pk:          |    +9.975520e+05  |  +6.400000e+01 |   +8.000000e+01 |
>>
>> I wonder about these worst case measurements of 999 and 998
>> milliseconds.  It almost looks like you missed one entire period.
>   ^^^^
> microseconds
> 
>> Could this simply be a bug in the test setup?


Yes. From the data set of the tbs SW:

offset |      timestamp
-------+---------------------
(...)  |
10639  | 1516117448.058010639
9503   | 1516117448.059009503
10167  | 1516117448.060010167
9823   | 1516117448.061009823
9567   | 1516117448.062009567
997703 | 1516117448.062997703 ****
911719 | 1516117448.063911719
12655  | 1516117448.065012655
12399  | 1516117448.066012399
(...)

Since the period was 1ms, the highlighted entry should have arrived within the
[1516117448.063000000, 1516117448.063999999] range, so in this case it was
early. For the next runs, I will modify the test setup so the txtime is sent as
part of the packet payload and later taken into account by the post-processing
script that is calculating the offsets.


Thanks,
Jesus


>>
>> Thanks,
>> Richard

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

* [Intel-wired-lan] [RFC v2 net-next 00/10] Time based packet transmission
@ 2018-01-23 18:07       ` Jesus Sanchez-Palencia
  0 siblings, 0 replies; 86+ messages in thread
From: Jesus Sanchez-Palencia @ 2018-01-23 18:07 UTC (permalink / raw)
  To: intel-wired-lan

Hi,


On 01/22/2018 09:26 PM, Richard Cochran wrote:
> On Mon, Jan 22, 2018 at 09:23:27PM -0800, Richard Cochran wrote:
>> On Wed, Jan 17, 2018 at 03:06:11PM -0800, Jesus Sanchez-Palencia wrote:
>>> First, a baseline test was ran for 10 minutes with the plain kernel only:
>>>
>>> |                 | plain kernel @ 1ms |
>>> |-----------------+--------------------+
>>> | min (ns):       |    +4.820000e+02   |
>>> | max (ns):       |    +9.999300e+05   |
>>> | pk-pk:          |    +9.994480e+05   |
>>
>> ...
>>
>>> |                 |    tbs SW @ 1ms   |  tbs HW @ 1ms  | tbs HW @ 250 us |
>>> |-----------------+-------------------+----------------+-----------------|
>>> | min (ns):       |    +1.510000e+02  |  +4.420000e+02 |   +4.260000e+02 |
>>> | max (ns):       |    +9.977030e+05  |  +5.060000e+02 |   +5.060000e+02 |
>>> | pk-pk:          |    +9.975520e+05  |  +6.400000e+01 |   +8.000000e+01 |
>>
>> I wonder about these worst case measurements of 999 and 998
>> milliseconds.  It almost looks like you missed one entire period.
>   ^^^^
> microseconds
> 
>> Could this simply be a bug in the test setup?


Yes. From the data set of the tbs SW:

offset |      timestamp
-------+---------------------
(...)  |
10639  | 1516117448.058010639
9503   | 1516117448.059009503
10167  | 1516117448.060010167
9823   | 1516117448.061009823
9567   | 1516117448.062009567
997703 | 1516117448.062997703 ****
911719 | 1516117448.063911719
12655  | 1516117448.065012655
12399  | 1516117448.066012399
(...)

Since the period was 1ms, the highlighted entry should have arrived within the
[1516117448.063000000, 1516117448.063999999] range, so in this case it was
early. For the next runs, I will modify the test setup so the txtime is sent as
part of the packet payload and later taken into account by the post-processing
script that is calculating the offsets.


Thanks,
Jesus


>>
>> Thanks,
>> Richard

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

* Re: [RFC v2 net-next 01/10] net: Add a new socket option for a future transmit time.
  2018-01-18 17:11     ` [Intel-wired-lan] " Richard Cochran
@ 2018-01-23 18:12       ` Jesus Sanchez-Palencia
  -1 siblings, 0 replies; 86+ messages in thread
From: Jesus Sanchez-Palencia @ 2018-01-23 18:12 UTC (permalink / raw)
  To: Richard Cochran
  Cc: netdev, jhs, xiyou.wangcong, jiri, vinicius.gomes,
	intel-wired-lan, anna-maria, henrik, tglx, john.stultz,
	andre.guedes, ivan.briano, levi.pearson, Richard Cochran



On 01/18/2018 09:11 AM, Richard Cochran wrote:
> On Wed, Jan 17, 2018 at 03:06:12PM -0800, Jesus Sanchez-Palencia wrote:
>> @@ -2130,6 +2137,15 @@ int __sock_cmsg_send(struct sock *sk, struct msghdr *msg, struct cmsghdr *cmsg,
>>  		sockc->tsflags &= ~SOF_TIMESTAMPING_TX_RECORD_MASK;
>>  		sockc->tsflags |= tsflags;
>>  		break;
>> +	case SO_TXTIME:
>> +		if (!ns_capable(sock_net(sk)->user_ns, CAP_NET_ADMIN))
>> +			return -EPERM;
>> +		if (!sock_flag(sk, SOCK_TXTIME))
>> +			return -EINVAL;
>> +		if (cmsg->cmsg_len != CMSG_LEN(sizeof(ktime_t)))
>> +			return -EINVAL;
>> +		sockc->transmit_time = *(ktime_t *)CMSG_DATA(cmsg);
> 
> As pointed out in the first series' review:
> 
>   No guarantee the CMSG is properly aligned on arches that might trap
>   on unaligned access.


Yes, it will be fixed on the next version. We should probably fix the other
cases on this function as well then.

Thanks,
Jesus

> 
> Thanks,
> Richard
> 

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

* [Intel-wired-lan] [RFC v2 net-next 01/10] net: Add a new socket option for a future transmit time.
@ 2018-01-23 18:12       ` Jesus Sanchez-Palencia
  0 siblings, 0 replies; 86+ messages in thread
From: Jesus Sanchez-Palencia @ 2018-01-23 18:12 UTC (permalink / raw)
  To: intel-wired-lan



On 01/18/2018 09:11 AM, Richard Cochran wrote:
> On Wed, Jan 17, 2018 at 03:06:12PM -0800, Jesus Sanchez-Palencia wrote:
>> @@ -2130,6 +2137,15 @@ int __sock_cmsg_send(struct sock *sk, struct msghdr *msg, struct cmsghdr *cmsg,
>>  		sockc->tsflags &= ~SOF_TIMESTAMPING_TX_RECORD_MASK;
>>  		sockc->tsflags |= tsflags;
>>  		break;
>> +	case SO_TXTIME:
>> +		if (!ns_capable(sock_net(sk)->user_ns, CAP_NET_ADMIN))
>> +			return -EPERM;
>> +		if (!sock_flag(sk, SOCK_TXTIME))
>> +			return -EINVAL;
>> +		if (cmsg->cmsg_len != CMSG_LEN(sizeof(ktime_t)))
>> +			return -EINVAL;
>> +		sockc->transmit_time = *(ktime_t *)CMSG_DATA(cmsg);
> 
> As pointed out in the first series' review:
> 
>   No guarantee the CMSG is properly aligned on arches that might trap
>   on unaligned access.


Yes, it will be fixed on the next version. We should probably fix the other
cases on this function as well then.

Thanks,
Jesus

> 
> Thanks,
> Richard
> 

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

* Re: [RFC v2 net-next 01/10] net: Add a new socket option for a future transmit time.
  2018-01-19 21:15     ` [Intel-wired-lan] " Willem de Bruijn
@ 2018-01-23 18:24       ` Jesus Sanchez-Palencia
  -1 siblings, 0 replies; 86+ messages in thread
From: Jesus Sanchez-Palencia @ 2018-01-23 18:24 UTC (permalink / raw)
  To: Willem de Bruijn
  Cc: Network Development, Jamal Hadi Salim, Cong Wang,
	Jiří Pírko, vinicius.gomes, Richard Cochran,
	intel-wired-lan, anna-maria, henrik, Thomas Gleixner,
	John Stultz, andre.guedes, ivan.briano, levi.pearson,
	Richard Cochran

Hi,


On 01/19/2018 01:15 PM, Willem de Bruijn wrote:
> On Wed, Jan 17, 2018 at 6:06 PM, Jesus Sanchez-Palencia
> <jesus.sanchez-palencia@intel.com> wrote:
>> From: Richard Cochran <rcochran@linutronix.de>
>>
>> This patch introduces SO_TXTIME.  User space enables this option in
>> order to pass a desired future transmit time in a CMSG when calling
>> sendmsg(2).
>>
>> A new field is added to struct sockcm_cookie, and the tstamp from
>> skbuffs will be used later on.
>>
>> Signed-off-by: Richard Cochran <rcochran@linutronix.de>
>> Signed-off-by: Jesus Sanchez-Palencia <jesus.sanchez-palencia@intel.com>
>> ---
> 
>> diff --git a/net/core/sock.c b/net/core/sock.c
>> index abf4cbff99b2..37ef4b33fd92 100644
>> --- a/net/core/sock.c
>> +++ b/net/core/sock.c
>> @@ -1061,6 +1061,13 @@ int sock_setsockopt(struct socket *sock, int level, int optname,
>>                         sock_valbool_flag(sk, SOCK_ZEROCOPY, valbool);
>>                 break;
>>
>> +       case SO_TXTIME:
>> +               if (ns_capable(sock_net(sk)->user_ns, CAP_NET_ADMIN))
>> +                       sock_valbool_flag(sk, SOCK_TXTIME, valbool);
>> +               else
>> +                       ret = -EPERM;
>> +               break;
>> +
>>         default:
>>                 ret = -ENOPROTOOPT;
>>                 break;
> 
> Please add getsockopt alongside setsockopt.
> 
> I would also restrict input to [0, 1] exactly to allow for future extensions.


Ok, will do.


> 
> If using ns_capable, skb->tstamp must continue to be scrubbed when traversing
> network namespaces.


I was planning to follow Eric's suggestion and move the tstamp scrubbing out of
skb_scrub_packet() into ____dev_forward_skb() instead. Would that break when
traversing namespaces?



> 
>> @@ -2130,6 +2137,15 @@ int __sock_cmsg_send(struct sock *sk, struct msghdr *msg, struct cmsghdr *cmsg,
>>                 sockc->tsflags &= ~SOF_TIMESTAMPING_TX_RECORD_MASK;
>>                 sockc->tsflags |= tsflags;
>>                 break;
>> +       case SO_TXTIME:
>> +               if (!ns_capable(sock_net(sk)->user_ns, CAP_NET_ADMIN))
>> +                       return -EPERM;
>> +               if (!sock_flag(sk, SOCK_TXTIME))
>> +                       return -EINVAL;
> 
> No need for ns_capable check on each packet when already required to
> toggle socket option.


Ok. SO_MARK is doing the same so it might have "mis-inspired" me. I should
probably fix both.



> 
>> +               if (cmsg->cmsg_len != CMSG_LEN(sizeof(ktime_t)))
>> +                       return -EINVAL;
> 
> I don't see any existing reference to ktime_t in include/uapi. Just use a s64?


Sure, will fix.


Thanks,
Jesus

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

* [Intel-wired-lan] [RFC v2 net-next 01/10] net: Add a new socket option for a future transmit time.
@ 2018-01-23 18:24       ` Jesus Sanchez-Palencia
  0 siblings, 0 replies; 86+ messages in thread
From: Jesus Sanchez-Palencia @ 2018-01-23 18:24 UTC (permalink / raw)
  To: intel-wired-lan

Hi,


On 01/19/2018 01:15 PM, Willem de Bruijn wrote:
> On Wed, Jan 17, 2018 at 6:06 PM, Jesus Sanchez-Palencia
> <jesus.sanchez-palencia@intel.com> wrote:
>> From: Richard Cochran <rcochran@linutronix.de>
>>
>> This patch introduces SO_TXTIME.  User space enables this option in
>> order to pass a desired future transmit time in a CMSG when calling
>> sendmsg(2).
>>
>> A new field is added to struct sockcm_cookie, and the tstamp from
>> skbuffs will be used later on.
>>
>> Signed-off-by: Richard Cochran <rcochran@linutronix.de>
>> Signed-off-by: Jesus Sanchez-Palencia <jesus.sanchez-palencia@intel.com>
>> ---
> 
>> diff --git a/net/core/sock.c b/net/core/sock.c
>> index abf4cbff99b2..37ef4b33fd92 100644
>> --- a/net/core/sock.c
>> +++ b/net/core/sock.c
>> @@ -1061,6 +1061,13 @@ int sock_setsockopt(struct socket *sock, int level, int optname,
>>                         sock_valbool_flag(sk, SOCK_ZEROCOPY, valbool);
>>                 break;
>>
>> +       case SO_TXTIME:
>> +               if (ns_capable(sock_net(sk)->user_ns, CAP_NET_ADMIN))
>> +                       sock_valbool_flag(sk, SOCK_TXTIME, valbool);
>> +               else
>> +                       ret = -EPERM;
>> +               break;
>> +
>>         default:
>>                 ret = -ENOPROTOOPT;
>>                 break;
> 
> Please add getsockopt alongside setsockopt.
> 
> I would also restrict input to [0, 1] exactly to allow for future extensions.


Ok, will do.


> 
> If using ns_capable, skb->tstamp must continue to be scrubbed when traversing
> network namespaces.


I was planning to follow Eric's suggestion and move the tstamp scrubbing out of
skb_scrub_packet() into ____dev_forward_skb() instead. Would that break when
traversing namespaces?



> 
>> @@ -2130,6 +2137,15 @@ int __sock_cmsg_send(struct sock *sk, struct msghdr *msg, struct cmsghdr *cmsg,
>>                 sockc->tsflags &= ~SOF_TIMESTAMPING_TX_RECORD_MASK;
>>                 sockc->tsflags |= tsflags;
>>                 break;
>> +       case SO_TXTIME:
>> +               if (!ns_capable(sock_net(sk)->user_ns, CAP_NET_ADMIN))
>> +                       return -EPERM;
>> +               if (!sock_flag(sk, SOCK_TXTIME))
>> +                       return -EINVAL;
> 
> No need for ns_capable check on each packet when already required to
> toggle socket option.


Ok. SO_MARK is doing the same so it might have "mis-inspired" me. I should
probably fix both.



> 
>> +               if (cmsg->cmsg_len != CMSG_LEN(sizeof(ktime_t)))
>> +                       return -EINVAL;
> 
> I don't see any existing reference to ktime_t in include/uapi. Just use a s64?


Sure, will fix.


Thanks,
Jesus

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

* Re: [RFC v2 net-next 01/10] net: Add a new socket option for a future transmit time.
  2018-01-23 18:24       ` [Intel-wired-lan] " Jesus Sanchez-Palencia
@ 2018-01-23 20:02         ` Willem de Bruijn
  -1 siblings, 0 replies; 86+ messages in thread
From: Willem de Bruijn @ 2018-01-23 20:02 UTC (permalink / raw)
  To: Jesus Sanchez-Palencia
  Cc: Network Development, Jamal Hadi Salim, Cong Wang,
	Jiří Pírko, Vinicius Gomes, Richard Cochran,
	intel-wired-lan, anna-maria, Henrik Austad, Thomas Gleixner,
	John Stultz, andre.guedes, Ivan Briano, Levi Pearson,
	Richard Cochran

>> If using ns_capable, skb->tstamp must continue to be scrubbed when traversing
>> network namespaces.
>
>
> I was planning to follow Eric's suggestion and move the tstamp scrubbing out of
> skb_scrub_packet() into ____dev_forward_skb() instead. Would that break when
> traversing namespaces?

That implies namespace traversal, so sounds perfect for this purpose.

>>
>>> @@ -2130,6 +2137,15 @@ int __sock_cmsg_send(struct sock *sk, struct msghdr *msg, struct cmsghdr *cmsg,
>>>                 sockc->tsflags &= ~SOF_TIMESTAMPING_TX_RECORD_MASK;
>>>                 sockc->tsflags |= tsflags;
>>>                 break;
>>> +       case SO_TXTIME:
>>> +               if (!ns_capable(sock_net(sk)->user_ns, CAP_NET_ADMIN))
>>> +                       return -EPERM;
>>> +               if (!sock_flag(sk, SOCK_TXTIME))
>>> +                       return -EINVAL;
>>
>> No need for ns_capable check on each packet when already required to
>> toggle socket option.
>
>
> Ok. SO_MARK is doing the same so it might have "mis-inspired" me. I should
> probably fix both.

The SO_MARK cmsg does need a check on each invocation,
because it is not conditional on a sock_flag like SO_TXTIME.

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

* [Intel-wired-lan] [RFC v2 net-next 01/10] net: Add a new socket option for a future transmit time.
@ 2018-01-23 20:02         ` Willem de Bruijn
  0 siblings, 0 replies; 86+ messages in thread
From: Willem de Bruijn @ 2018-01-23 20:02 UTC (permalink / raw)
  To: intel-wired-lan

>> If using ns_capable, skb->tstamp must continue to be scrubbed when traversing
>> network namespaces.
>
>
> I was planning to follow Eric's suggestion and move the tstamp scrubbing out of
> skb_scrub_packet() into ____dev_forward_skb() instead. Would that break when
> traversing namespaces?

That implies namespace traversal, so sounds perfect for this purpose.

>>
>>> @@ -2130,6 +2137,15 @@ int __sock_cmsg_send(struct sock *sk, struct msghdr *msg, struct cmsghdr *cmsg,
>>>                 sockc->tsflags &= ~SOF_TIMESTAMPING_TX_RECORD_MASK;
>>>                 sockc->tsflags |= tsflags;
>>>                 break;
>>> +       case SO_TXTIME:
>>> +               if (!ns_capable(sock_net(sk)->user_ns, CAP_NET_ADMIN))
>>> +                       return -EPERM;
>>> +               if (!sock_flag(sk, SOCK_TXTIME))
>>> +                       return -EINVAL;
>>
>> No need for ns_capable check on each packet when already required to
>> toggle socket option.
>
>
> Ok. SO_MARK is doing the same so it might have "mis-inspired" me. I should
> probably fix both.

The SO_MARK cmsg does need a check on each invocation,
because it is not conditional on a sock_flag like SO_TXTIME.

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

* Re: [Intel-wired-lan] [RFC v2 net-next 01/10] net: Add a new socket option for a future transmit time.
  2018-01-18  8:42     ` Miroslav Lichvar
@ 2018-01-23 21:22       ` Vinicius Costa Gomes
  -1 siblings, 0 replies; 86+ messages in thread
From: Vinicius Costa Gomes @ 2018-01-23 21:22 UTC (permalink / raw)
  To: Miroslav Lichvar, Jesus Sanchez-Palencia
  Cc: netdev, john.stultz, Richard Cochran, jiri, ivan.briano,
	richardcochran, henrik, jhs, levi.pearson, intel-wired-lan,
	xiyou.wangcong, tglx, anna-maria

Hi,

Miroslav Lichvar <mlichvar@redhat.com> writes:

> On Wed, Jan 17, 2018 at 03:06:12PM -0800, Jesus Sanchez-Palencia wrote:
>> From: Richard Cochran <rcochran@linutronix.de>
>> 
>> This patch introduces SO_TXTIME.  User space enables this option in
>> order to pass a desired future transmit time in a CMSG when calling
>> sendmsg(2).
>> 
>> A new field is added to struct sockcm_cookie, and the tstamp from
>> skbuffs will be used later on.
>
> In the discussion about the v1 patchset, there was a question if the
> cmsg should include a clockid_t. Without that, how can an application
> prevent the packet from being sent using an incorrect clock, e.g.
> the system clock when it expects it to be a PHC, or a different PHC
> when the socket is not bound to a specific interface?
>
> At least in some applications it would be preferred to not sent a
> packet at all instead of sending it at a wrong time.

Including the clockid in a CMSG field does make sense. Will add it in
the next version of this series.

What I think would be the ideal scenario would be if the clockid
parameter to the TBS Qdisc would not be necessary (if offload was
enabled), but that's not quite possible right now, because there's no
support for using the hrtimer infrastructure with dynamic clocks
(/dev/ptp*).

What I am thinking is to keep the clockid parameter for the Qdisc (and
add support for expressing the clockid in friendlier ways, as requested
later in this thread), but I can't think of a way to add support for
using /dev/ptp* clocks without first having hrtimer support them.

And the behavior would be to drop any packets with a clockid not
matching the Qdisc clockid.

How does this sound?


>
> Please keep in mind that the PHCs and the system clock don't have to
> be synchronized to each other. If I understand the rest of the series
> correctly, there is an assumption that the PHCs are keeping time in
> TAI and CLOCK_TAI can be used as a fallback.

You understand correctly, that's because of whole hrtimer issue.

>
> -- 
> Miroslav Lichvar


Cheers,

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

* [Intel-wired-lan] [RFC v2 net-next 01/10] net: Add a new socket option for a future transmit time.
@ 2018-01-23 21:22       ` Vinicius Costa Gomes
  0 siblings, 0 replies; 86+ messages in thread
From: Vinicius Costa Gomes @ 2018-01-23 21:22 UTC (permalink / raw)
  To: intel-wired-lan

Hi,

Miroslav Lichvar <mlichvar@redhat.com> writes:

> On Wed, Jan 17, 2018 at 03:06:12PM -0800, Jesus Sanchez-Palencia wrote:
>> From: Richard Cochran <rcochran@linutronix.de>
>> 
>> This patch introduces SO_TXTIME.  User space enables this option in
>> order to pass a desired future transmit time in a CMSG when calling
>> sendmsg(2).
>> 
>> A new field is added to struct sockcm_cookie, and the tstamp from
>> skbuffs will be used later on.
>
> In the discussion about the v1 patchset, there was a question if the
> cmsg should include a clockid_t. Without that, how can an application
> prevent the packet from being sent using an incorrect clock, e.g.
> the system clock when it expects it to be a PHC, or a different PHC
> when the socket is not bound to a specific interface?
>
> At least in some applications it would be preferred to not sent a
> packet at all instead of sending it at a wrong time.

Including the clockid in a CMSG field does make sense. Will add it in
the next version of this series.

What I think would be the ideal scenario would be if the clockid
parameter to the TBS Qdisc would not be necessary (if offload was
enabled), but that's not quite possible right now, because there's no
support for using the hrtimer infrastructure with dynamic clocks
(/dev/ptp*).

What I am thinking is to keep the clockid parameter for the Qdisc (and
add support for expressing the clockid in friendlier ways, as requested
later in this thread), but I can't think of a way to add support for
using /dev/ptp* clocks without first having hrtimer support them.

And the behavior would be to drop any packets with a clockid not
matching the Qdisc clockid.

How does this sound?


>
> Please keep in mind that the PHCs and the system clock don't have to
> be synchronized to each other. If I understand the rest of the series
> correctly, there is an assumption that the PHCs are keeping time in
> TAI and CLOCK_TAI can be used as a fallback.

You understand correctly, that's because of whole hrtimer issue.

>
> -- 
> Miroslav Lichvar


Cheers,
--
Vinicius

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

* Re: [RFC v2 net-next 06/10] net/sched: Introduce the TBS Qdisc
  2018-01-18 13:44       ` [Intel-wired-lan] " Jamal Hadi Salim
@ 2018-01-23 21:45         ` Jesus Sanchez-Palencia
  -1 siblings, 0 replies; 86+ messages in thread
From: Jesus Sanchez-Palencia @ 2018-01-23 21:45 UTC (permalink / raw)
  To: Jamal Hadi Salim, netdev
  Cc: xiyou.wangcong, jiri, vinicius.gomes, richardcochran,
	intel-wired-lan, anna-maria, henrik, tglx, john.stultz,
	andre.guedes, ivan.briano, levi.pearson

Hi,


On 01/18/2018 05:44 AM, Jamal Hadi Salim wrote:
> One more comment:
> Probably try to run a test with a very small delta with
> no offload (probably using something like prio as the root qdisc)
> and dump the stats.
> My gut feeling is your accounting of the backlog in particular is off.


You were right, thanks. It'll be fixed on our next version.

Regards,
Jesus


> 
> cheers,
> jamal
> 
> On 18-01-18 08:35 AM, Jamal Hadi Salim wrote:
>> On 18-01-17 06:06 PM, Jesus Sanchez-Palencia wrote:
>>> From: Vinicius Costa Gomes <vinicius.gomes@intel.com>
>>>
>>> TBS (Time Based Scheduler) uses the information added earlier in this
>>> series (the socket option SO_TXTIME and the new role of
>>> sk_buff->tstamp) to schedule traffic transmission based on absolute
>>> time.
>>>
>>> For some workloads, just bandwidth enforcement is not enough, and
>>> precise control of the transmission of packets is necessary.
>>>
>>> Example:
>>>
>>> $ tc qdisc replace dev enp2s0 parent root handle 100 mqprio num_tc 3 \
>>
>> handle 100:0 ?
>>
>>>             map 2 2 1 0 2 2 2 2 2 2 2 2 2 2 2 2 queues 1@0 1@1 2@2 hw 0
>>>
>>> $ tc qdisc add dev enp2s0 parent 100:1 tbs delta 60000 clockid 11 offload 1
>>>
>>>
>>> In this example, the Qdisc will try to enable offloading (offload 1)
>>> the control of the transmission time to the network adapter, the
>>> time stamp in socket are in reference to the clockid '11' (CLOCK_TAI)
>>> and packets leave the Qdisc "delta" (60000) nanoseconds before its
>>> transmission time.
>>>
>>
>>
>>> When offloading is disabled, the network adapter will ignore the
>>> sk_buff time stamp, and so, the transmission time will be only "best
>>> effort" from the Qdisc.
>>>
>>
>> General comments:
>> 1) iproute2: Avoid magic numbers like 1 or 11 please; "offload"
>> (without 1) and "TAI" will be more human friendly.
>>
>> 2) Experience shows that adding padding fields in the control structs
>> implies they will _never ever_ be used. That was not design intent
>> for netlink but over years shit like that has happened.
>> Maybe look at using a 32 bitmap? It is more "future proof".
>> You seem to only have 2-3 flags but it gives you opportunity
>> to add more changes later. If you are 100% sure youll never need
>> it - then maybe just move the tc_tbs_qopt::offload to the end of
>> of the struct.
>>
>> 3)It would be helpful for debugging to increment some stats other
>> than drop counters on enqueu/dequeue obsolete packet drop. Maybe use
>> overlimits for the dequeu drops (in addition)?
>>
>> 4) I may be misreading things - but did you need to reset the
>> watchdog on dequeue? It is already being kicked for every incoming packet.
>>
>> cheers,
>> jamal
> 

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

* [Intel-wired-lan] [RFC v2 net-next 06/10] net/sched: Introduce the TBS Qdisc
@ 2018-01-23 21:45         ` Jesus Sanchez-Palencia
  0 siblings, 0 replies; 86+ messages in thread
From: Jesus Sanchez-Palencia @ 2018-01-23 21:45 UTC (permalink / raw)
  To: intel-wired-lan

Hi,


On 01/18/2018 05:44 AM, Jamal Hadi Salim wrote:
> One more comment:
> Probably try to run a test with a very small delta with
> no offload (probably using something like prio as the root qdisc)
> and dump the stats.
> My gut feeling is your accounting of the backlog in particular is off.


You were right, thanks. It'll be fixed on our next version.

Regards,
Jesus


> 
> cheers,
> jamal
> 
> On 18-01-18 08:35 AM, Jamal Hadi Salim wrote:
>> On 18-01-17 06:06 PM, Jesus Sanchez-Palencia wrote:
>>> From: Vinicius Costa Gomes <vinicius.gomes@intel.com>
>>>
>>> TBS (Time Based Scheduler) uses the information added earlier in this
>>> series (the socket option SO_TXTIME and the new role of
>>> sk_buff->tstamp) to schedule traffic transmission based on absolute
>>> time.
>>>
>>> For some workloads, just bandwidth enforcement is not enough, and
>>> precise control of the transmission of packets is necessary.
>>>
>>> Example:
>>>
>>> $ tc qdisc replace dev enp2s0 parent root handle 100 mqprio num_tc 3 \
>>
>> handle 100:0 ?
>>
>>> ??????????? map 2 2 1 0 2 2 2 2 2 2 2 2 2 2 2 2 queues 1 at 0 1 at 1 2 at 2 hw 0
>>>
>>> $ tc qdisc add dev enp2s0 parent 100:1 tbs delta 60000 clockid 11 offload 1
>>>
>>>
>>> In this example, the Qdisc will try to enable offloading (offload 1)
>>> the control of the transmission time to the network adapter, the
>>> time stamp in socket are in reference to the clockid '11' (CLOCK_TAI)
>>> and packets leave the Qdisc "delta" (60000) nanoseconds before its
>>> transmission time.
>>>
>>
>>
>>> When offloading is disabled, the network adapter will ignore the
>>> sk_buff time stamp, and so, the transmission time will be only "best
>>> effort" from the Qdisc.
>>>
>>
>> General comments:
>> 1) iproute2: Avoid magic numbers like 1 or 11 please; "offload"
>> (without 1) and "TAI" will be more human friendly.
>>
>> 2) Experience shows that adding padding fields in the control structs
>> implies they will _never ever_ be used. That was not design intent
>> for netlink but over years shit like that has happened.
>> Maybe look at using a 32 bitmap? It is more "future proof".
>> You seem to only have 2-3 flags but it gives you opportunity
>> to add more changes later. If you are 100% sure youll never need
>> it - then maybe just move the tc_tbs_qopt::offload to the end of
>> of the struct.
>>
>> 3)It would be helpful for debugging to increment some stats other
>> than drop counters on enqueu/dequeue obsolete packet drop. Maybe use
>> overlimits for the dequeu drops (in addition)?
>>
>> 4) I may be misreading things - but did you need to reset the
>> watchdog on dequeue? It is already being kicked for every incoming packet.
>>
>> cheers,
>> jamal
> 

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

* Re: [RFC v2 net-next 06/10] net/sched: Introduce the TBS Qdisc
  2018-01-18 13:35     ` [Intel-wired-lan] " Jamal Hadi Salim
@ 2018-01-23 22:01       ` Jesus Sanchez-Palencia
  -1 siblings, 0 replies; 86+ messages in thread
From: Jesus Sanchez-Palencia @ 2018-01-23 22:01 UTC (permalink / raw)
  To: Jamal Hadi Salim, netdev
  Cc: xiyou.wangcong, jiri, vinicius.gomes, richardcochran,
	intel-wired-lan, anna-maria, henrik, tglx, john.stultz,
	andre.guedes, ivan.briano, levi.pearson

Hi,


On 01/18/2018 05:35 AM, Jamal Hadi Salim wrote:
> On 18-01-17 06:06 PM, Jesus Sanchez-Palencia wrote:
>> From: Vinicius Costa Gomes <vinicius.gomes@intel.com>
>>
>> TBS (Time Based Scheduler) uses the information added earlier in this
>> series (the socket option SO_TXTIME and the new role of
>> sk_buff->tstamp) to schedule traffic transmission based on absolute
>> time.
>>
>> For some workloads, just bandwidth enforcement is not enough, and
>> precise control of the transmission of packets is necessary.
>>
>> Example:
>>
>> $ tc qdisc replace dev enp2s0 parent root handle 100 mqprio num_tc 3 \
> 
> handle 100:0 ?
> 
>>             map 2 2 1 0 2 2 2 2 2 2 2 2 2 2 2 2 queues 1@0 1@1 2@2 hw 0
>>
>> $ tc qdisc add dev enp2s0 parent 100:1 tbs delta 60000 clockid 11 offload 1
>>
>>
>> In this example, the Qdisc will try to enable offloading (offload 1)
>> the control of the transmission time to the network adapter, the
>> time stamp in socket are in reference to the clockid '11' (CLOCK_TAI)
>> and packets leave the Qdisc "delta" (60000) nanoseconds before its
>> transmission time.
>>
> 
> 
>> When offloading is disabled, the network adapter will ignore the
>> sk_buff time stamp, and so, the transmission time will be only "best
>> effort" from the Qdisc.
>>
> 
> General comments:
> 1) iproute2: Avoid magic numbers like 1 or 11 please; "offload"
> (without 1) and "TAI" will be more human friendly.


Sure, we'll change both parameters.


> 
> 2) Experience shows that adding padding fields in the control structs
> implies they will _never ever_ be used. That was not design intent
> for netlink but over years shit like that has happened.
> Maybe look at using a 32 bitmap? It is more "future proof".
> You seem to only have 2-3 flags but it gives you opportunity
> to add more changes later. If you are 100% sure youll never need
> it - then maybe just move the tc_tbs_qopt::offload to the end of
> of the struct.


Ok, we are looking into it.


> 
> 3)It would be helpful for debugging to increment some stats other
> than drop counters on enqueu/dequeue obsolete packet drop. Maybe use
> overlimits for the dequeu drops (in addition)?


Yes, sure, that's a good idea.


> 
> 4) I may be misreading things - but did you need to reset the
> watchdog on dequeue? It is already being kicked for every incoming packet.


The watchdog timer must always be set to the next deadline. Since both enqueue()
and dequeue() modify the tree structure and may have it rebalanced, we need to
make sure the watchdog has been 're-set' to the next deadline. i.e. After a
dequeue, we need to re-arm the timer for the next head of the queue, so that is
why. Does that make sense?

I'm now thinking that naming that helper as reset_watchdog() can be misleading,
so we are considering naming it as rearm_watchdog() instead.


Thanks,
Jesus


> 
> cheers,
> jamal

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

* [Intel-wired-lan] [RFC v2 net-next 06/10] net/sched: Introduce the TBS Qdisc
@ 2018-01-23 22:01       ` Jesus Sanchez-Palencia
  0 siblings, 0 replies; 86+ messages in thread
From: Jesus Sanchez-Palencia @ 2018-01-23 22:01 UTC (permalink / raw)
  To: intel-wired-lan

Hi,


On 01/18/2018 05:35 AM, Jamal Hadi Salim wrote:
> On 18-01-17 06:06 PM, Jesus Sanchez-Palencia wrote:
>> From: Vinicius Costa Gomes <vinicius.gomes@intel.com>
>>
>> TBS (Time Based Scheduler) uses the information added earlier in this
>> series (the socket option SO_TXTIME and the new role of
>> sk_buff->tstamp) to schedule traffic transmission based on absolute
>> time.
>>
>> For some workloads, just bandwidth enforcement is not enough, and
>> precise control of the transmission of packets is necessary.
>>
>> Example:
>>
>> $ tc qdisc replace dev enp2s0 parent root handle 100 mqprio num_tc 3 \
> 
> handle 100:0 ?
> 
>> ??????????? map 2 2 1 0 2 2 2 2 2 2 2 2 2 2 2 2 queues 1 at 0 1 at 1 2 at 2 hw 0
>>
>> $ tc qdisc add dev enp2s0 parent 100:1 tbs delta 60000 clockid 11 offload 1
>>
>>
>> In this example, the Qdisc will try to enable offloading (offload 1)
>> the control of the transmission time to the network adapter, the
>> time stamp in socket are in reference to the clockid '11' (CLOCK_TAI)
>> and packets leave the Qdisc "delta" (60000) nanoseconds before its
>> transmission time.
>>
> 
> 
>> When offloading is disabled, the network adapter will ignore the
>> sk_buff time stamp, and so, the transmission time will be only "best
>> effort" from the Qdisc.
>>
> 
> General comments:
> 1) iproute2: Avoid magic numbers like 1 or 11 please; "offload"
> (without 1) and "TAI" will be more human friendly.


Sure, we'll change both parameters.


> 
> 2) Experience shows that adding padding fields in the control structs
> implies they will _never ever_ be used. That was not design intent
> for netlink but over years shit like that has happened.
> Maybe look at using a 32 bitmap? It is more "future proof".
> You seem to only have 2-3 flags but it gives you opportunity
> to add more changes later. If you are 100% sure youll never need
> it - then maybe just move the tc_tbs_qopt::offload to the end of
> of the struct.


Ok, we are looking into it.


> 
> 3)It would be helpful for debugging to increment some stats other
> than drop counters on enqueu/dequeue obsolete packet drop. Maybe use
> overlimits for the dequeu drops (in addition)?


Yes, sure, that's a good idea.


> 
> 4) I may be misreading things - but did you need to reset the
> watchdog on dequeue? It is already being kicked for every incoming packet.


The watchdog timer must always be set to the next deadline. Since both enqueue()
and dequeue() modify the tree structure and may have it rebalanced, we need to
make sure the watchdog has been 're-set' to the next deadline. i.e. After a
dequeue, we need to re-arm the timer for the next head of the queue, so that is
why. Does that make sense?

I'm now thinking that naming that helper as reset_watchdog() can be misleading,
so we are considering naming it as rearm_watchdog() instead.


Thanks,
Jesus


> 
> cheers,
> jamal

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

* Re: [RFC v2 net-next 00/10] Time based packet transmission
  2018-01-17 23:06 ` [Intel-wired-lan] " Jesus Sanchez-Palencia
@ 2018-01-24  1:43   ` Levi Pearson
  -1 siblings, 0 replies; 86+ messages in thread
From: Levi Pearson @ 2018-01-24  1:43 UTC (permalink / raw)
  To: Jesus Sanchez-Palencia
  Cc: Linux Kernel Network Developers, Jamal Hadi Salim, Cong Wang,
	Jiri Pirko, Vinicius Costa Gomes, Richard Cochran,
	intel-wired-lan, anna-maria, Henrik Austad, tglx, John Stultz,
	andre.guedes, Ivan Briano

On Wed, Jan 17, 2018 at 4:06 PM, Jesus Sanchez-Palencia
<jesus.sanchez-palencia@intel.com> wrote:
> This series is the v2 of the Time based packet transmission RFC, which was
> originally proposed by Richard Cochran: https://lwn.net/Articles/733962/ .

Great to see you carrying on with this!

> Our main questions at this stage are related to the qdisc:
>  - does the proposed design attend all use cases?
>  - should the qdisc really drop packets that expired after being queued even
>    for the SW best effort mode?

I don't think that being "expired" is necessarily cause for dropping.
The semantic of a launch time is "launch no earlier than this point"
after all, not a deadline. To keep the hardware working, we must only
enforce the invariant that we never queue a packet with an earlier
timestamp than one we previously enqueued that has not launched yet.
Just checking for expiration is going to rule out some potential uses
and also won't necessarily prevent enqueuing out-of-order packets.
Here is an example:

A group of applications enqueue packets to be sent at 1 second
intervals, and share a 5ms window in which they can send them. Due to
scheduling variation, they may finish executing in a different order
per interval, and occasionally some may not finish preparing their
packet before the window opens, although they always will present
their packet before the window closes.

If they all pick different times within the launch window, it is
possible that two of them might pick times very close to one another.
If they present their frames out-of-order to the qdisc, but close
enough to the launch time that the qdisc doesn't hold on to them (i.e.
in the [txtime - delta, txtime] range mentioned in tbs_dequeue), then
they will get enqueued out of order and the invariant will be
violated.  Reordering within some time window only works if all frames
for that window are scheduled well in advance of the first launch
time, and that's not great for applications that need to to calculate
right up to the time they need to send their data.

If they each schedule their packet for the beginning of the window, on
the other hand, everything will be fine for those that complete before
the window opens; there's no longer any order requirement that needs
to be maintained between those in the group, since they all use the
same timestamp. But those that finish after the window opens will be
"late" and have their frames dropped in the current scheme. There's no
technical reason to regard them as late until sending would exceed the
window bounds, but we also don't want to delay any of them once the
window opens.

To maintain the hardware ordering invariant, you need to keep track of
the most recent timestamp you have enqueued in the hardware. Anything
that hits tbs_enqueue with a timestamp earlier than that must be
either dropped or have its timestamp adjusted.

The one remaining question is how late can a timestamped frame be
before it should be dropped instead of enqueued, assuming it is to be
allowed at all? The qdisc could track the allowed window based on user
configuration. I believe the i210 hardware will launch any frame at
the head of queue with a launch time set at or before the present
time, but not so far before that it wraps and interprets the time as a
future time. The qdisc would need to be able query the driver about
how large that window is if it wants to pass in-the-past timestamps
through as-is, but it could also just update timestamps still within
the user-configured window to be set at the current time.

My understanding of reservations for industrial TSN use cases is that
applications will present their working period and their scheduling
accuracy to the central manager, which will take into account the
worst case timing bounds when creating the window that the application
will use on the network. It will then give back an assignment for a
start time offset from the period base time (UTC_time values that are
multiples of interval_time) at which the application's transmit window
starts, and it will remain open long enough to account for the
scheduling jitter in the application.

I think putting the window concept in the qdisc makes for a nice
mapping to how the TSN scheduling works as well as resolving some of
the tricky details around ensuring that you don't jam the hardware
with out-of-order timestamps or unnecessarily delay scheduling packets
to reorder them.


--Levi

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

* [Intel-wired-lan] [RFC v2 net-next 00/10] Time based packet transmission
@ 2018-01-24  1:43   ` Levi Pearson
  0 siblings, 0 replies; 86+ messages in thread
From: Levi Pearson @ 2018-01-24  1:43 UTC (permalink / raw)
  To: intel-wired-lan

On Wed, Jan 17, 2018 at 4:06 PM, Jesus Sanchez-Palencia
<jesus.sanchez-palencia@intel.com> wrote:
> This series is the v2 of the Time based packet transmission RFC, which was
> originally proposed by Richard Cochran: https://lwn.net/Articles/733962/ .

Great to see you carrying on with this!

> Our main questions at this stage are related to the qdisc:
>  - does the proposed design attend all use cases?
>  - should the qdisc really drop packets that expired after being queued even
>    for the SW best effort mode?

I don't think that being "expired" is necessarily cause for dropping.
The semantic of a launch time is "launch no earlier than this point"
after all, not a deadline. To keep the hardware working, we must only
enforce the invariant that we never queue a packet with an earlier
timestamp than one we previously enqueued that has not launched yet.
Just checking for expiration is going to rule out some potential uses
and also won't necessarily prevent enqueuing out-of-order packets.
Here is an example:

A group of applications enqueue packets to be sent at 1 second
intervals, and share a 5ms window in which they can send them. Due to
scheduling variation, they may finish executing in a different order
per interval, and occasionally some may not finish preparing their
packet before the window opens, although they always will present
their packet before the window closes.

If they all pick different times within the launch window, it is
possible that two of them might pick times very close to one another.
If they present their frames out-of-order to the qdisc, but close
enough to the launch time that the qdisc doesn't hold on to them (i.e.
in the [txtime - delta, txtime] range mentioned in tbs_dequeue), then
they will get enqueued out of order and the invariant will be
violated.  Reordering within some time window only works if all frames
for that window are scheduled well in advance of the first launch
time, and that's not great for applications that need to to calculate
right up to the time they need to send their data.

If they each schedule their packet for the beginning of the window, on
the other hand, everything will be fine for those that complete before
the window opens; there's no longer any order requirement that needs
to be maintained between those in the group, since they all use the
same timestamp. But those that finish after the window opens will be
"late" and have their frames dropped in the current scheme. There's no
technical reason to regard them as late until sending would exceed the
window bounds, but we also don't want to delay any of them once the
window opens.

To maintain the hardware ordering invariant, you need to keep track of
the most recent timestamp you have enqueued in the hardware. Anything
that hits tbs_enqueue with a timestamp earlier than that must be
either dropped or have its timestamp adjusted.

The one remaining question is how late can a timestamped frame be
before it should be dropped instead of enqueued, assuming it is to be
allowed at all? The qdisc could track the allowed window based on user
configuration. I believe the i210 hardware will launch any frame at
the head of queue with a launch time set at or before the present
time, but not so far before that it wraps and interprets the time as a
future time. The qdisc would need to be able query the driver about
how large that window is if it wants to pass in-the-past timestamps
through as-is, but it could also just update timestamps still within
the user-configured window to be set at the current time.

My understanding of reservations for industrial TSN use cases is that
applications will present their working period and their scheduling
accuracy to the central manager, which will take into account the
worst case timing bounds when creating the window that the application
will use on the network. It will then give back an assignment for a
start time offset from the period base time (UTC_time values that are
multiples of interval_time) at which the application's transmit window
starts, and it will remain open long enough to account for the
scheduling jitter in the application.

I think putting the window concept in the qdisc makes for a nice
mapping to how the TSN scheduling works as well as resolving some of
the tricky details around ensuring that you don't jam the hardware
with out-of-order timestamps or unnecessarily delay scheduling packets
to reorder them.


--Levi

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

* Re: [Intel-wired-lan] [RFC v2 net-next 01/10] net: Add a new socket option for a future transmit time.
  2018-01-23 21:22       ` Vinicius Costa Gomes
@ 2018-01-24  3:04         ` Richard Cochran
  -1 siblings, 0 replies; 86+ messages in thread
From: Richard Cochran @ 2018-01-24  3:04 UTC (permalink / raw)
  To: Vinicius Costa Gomes
  Cc: Miroslav Lichvar, Jesus Sanchez-Palencia, netdev, john.stultz,
	Richard Cochran, jiri, ivan.briano, henrik, jhs, levi.pearson,
	intel-wired-lan, xiyou.wangcong, tglx, anna-maria

On Tue, Jan 23, 2018 at 01:22:37PM -0800, Vinicius Costa Gomes wrote:
> What I think would be the ideal scenario would be if the clockid
> parameter to the TBS Qdisc would not be necessary (if offload was
> enabled), but that's not quite possible right now, because there's no
> support for using the hrtimer infrastructure with dynamic clocks
> (/dev/ptp*).

We don't need hrtimer for HW offloading.  Just enqueue the packets.  I
thought we agreed that user space get the ordering correct.  In fact,
davem insisted on it, IIRC.

Thanks,
Richard

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

* [Intel-wired-lan] [RFC v2 net-next 01/10] net: Add a new socket option for a future transmit time.
@ 2018-01-24  3:04         ` Richard Cochran
  0 siblings, 0 replies; 86+ messages in thread
From: Richard Cochran @ 2018-01-24  3:04 UTC (permalink / raw)
  To: intel-wired-lan

On Tue, Jan 23, 2018 at 01:22:37PM -0800, Vinicius Costa Gomes wrote:
> What I think would be the ideal scenario would be if the clockid
> parameter to the TBS Qdisc would not be necessary (if offload was
> enabled), but that's not quite possible right now, because there's no
> support for using the hrtimer infrastructure with dynamic clocks
> (/dev/ptp*).

We don't need hrtimer for HW offloading.  Just enqueue the packets.  I
thought we agreed that user space get the ordering correct.  In fact,
davem insisted on it, IIRC.

Thanks,
Richard

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

* Re: [Intel-wired-lan] [RFC v2 net-next 01/10] net: Add a new socket option for a future transmit time.
  2018-01-24  3:04         ` Richard Cochran
@ 2018-01-24 22:46           ` Vinicius Costa Gomes
  -1 siblings, 0 replies; 86+ messages in thread
From: Vinicius Costa Gomes @ 2018-01-24 22:46 UTC (permalink / raw)
  To: Richard Cochran
  Cc: Miroslav Lichvar, Jesus Sanchez-Palencia, netdev, john.stultz,
	Richard Cochran, jiri, ivan.briano, henrik, jhs, levi.pearson,
	intel-wired-lan, xiyou.wangcong, tglx, anna-maria

Hi Richard,

Richard Cochran <richardcochran@gmail.com> writes:

> On Tue, Jan 23, 2018 at 01:22:37PM -0800, Vinicius Costa Gomes wrote:
>> What I think would be the ideal scenario would be if the clockid
>> parameter to the TBS Qdisc would not be necessary (if offload was
>> enabled), but that's not quite possible right now, because there's no
>> support for using the hrtimer infrastructure with dynamic clocks
>> (/dev/ptp*).
>
> We don't need hrtimer for HW offloading.  Just enqueue the packets.  I
> thought we agreed that user space get the ordering correct.  In fact,
> davem insisted on it, IIRC.

About the ordering of packets, From here [1], there are 3 clear points
(in my understanding):

1. Re-ordering of TX descriptors on the device queue should/must not
   happen;

2. Out of order requests are an error;

3. Timestamps in the past are an error;

The only robust way that we could think of about keeping the the packets
in order for the device queue is re-ordering packets in the Qdisc.

We tried to reach out for confirmation [2] of this understanding but
didn't receive any word.

Even if we reach a decision that the Qdisc should not re-order packets
(we wouldn't have any dependency on hrtimers in the offload case, as you
pointed out), we still need hrtimers for the software implementation.

So, I guess, the problem remains, if it's possible for the user to
express a /dev/ptp* clock, what should we do? 

>
> Thanks,
> Richard

Cheers,

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

* [Intel-wired-lan] [RFC v2 net-next 01/10] net: Add a new socket option for a future transmit time.
@ 2018-01-24 22:46           ` Vinicius Costa Gomes
  0 siblings, 0 replies; 86+ messages in thread
From: Vinicius Costa Gomes @ 2018-01-24 22:46 UTC (permalink / raw)
  To: intel-wired-lan

Hi Richard,

Richard Cochran <richardcochran@gmail.com> writes:

> On Tue, Jan 23, 2018 at 01:22:37PM -0800, Vinicius Costa Gomes wrote:
>> What I think would be the ideal scenario would be if the clockid
>> parameter to the TBS Qdisc would not be necessary (if offload was
>> enabled), but that's not quite possible right now, because there's no
>> support for using the hrtimer infrastructure with dynamic clocks
>> (/dev/ptp*).
>
> We don't need hrtimer for HW offloading.  Just enqueue the packets.  I
> thought we agreed that user space get the ordering correct.  In fact,
> davem insisted on it, IIRC.

About the ordering of packets, From here [1], there are 3 clear points
(in my understanding):

1. Re-ordering of TX descriptors on the device queue should/must not
   happen;

2. Out of order requests are an error;

3. Timestamps in the past are an error;

The only robust way that we could think of about keeping the the packets
in order for the device queue is re-ordering packets in the Qdisc.

We tried to reach out for confirmation [2] of this understanding but
didn't receive any word.

Even if we reach a decision that the Qdisc should not re-order packets
(we wouldn't have any dependency on hrtimers in the offload case, as you
pointed out), we still need hrtimers for the software implementation.

So, I guess, the problem remains, if it's possible for the user to
express a /dev/ptp* clock, what should we do? 

>
> Thanks,
> Richard

Cheers,
--
Vinicius


[1] https://patchwork.ozlabs.org/comment/1770302/

[2] https://patchwork.ozlabs.org/comment/1816492/q

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

* Re: [Intel-wired-lan] [RFC v2 net-next 01/10] net: Add a new socket option for a future transmit time.
  2018-01-20  2:09       ` [Intel-wired-lan] " Richard Cochran
@ 2018-01-25  9:12         ` Miroslav Lichvar
  -1 siblings, 0 replies; 86+ messages in thread
From: Miroslav Lichvar @ 2018-01-25  9:12 UTC (permalink / raw)
  To: Richard Cochran
  Cc: Willem de Bruijn, John Stultz, Richard Cochran,
	Jiří Pírko, ivan.briano, Network Development,
	henrik, Jamal Hadi Salim, levi.pearson, intel-wired-lan,
	Cong Wang, Thomas Gleixner, anna-maria, Jesus Sanchez-Palencia

On Fri, Jan 19, 2018 at 06:09:15PM -0800, Richard Cochran wrote:
> On Fri, Jan 19, 2018 at 04:15:46PM -0500, Willem de Bruijn wrote:
> > > +               if (cmsg->cmsg_len != CMSG_LEN(sizeof(ktime_t)))
> > > +                       return -EINVAL;
> > 
> > I don't see any existing reference to ktime_t in include/uapi. Just use a s64?
> 
> Agreed.  I didn't see the point of switching to ktime, either.

Do I understand it correctly that no other interface is using
nanoseconds since 1970? We probably don't have to worry about year
2262 yet, but wouldn't it be better to make it consistent with the
timestamping API using timespec? Or is it just better to avoid the
64/32-bit mess of time_t?

-- 
Miroslav Lichvar

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

* [Intel-wired-lan] [RFC v2 net-next 01/10] net: Add a new socket option for a future transmit time.
@ 2018-01-25  9:12         ` Miroslav Lichvar
  0 siblings, 0 replies; 86+ messages in thread
From: Miroslav Lichvar @ 2018-01-25  9:12 UTC (permalink / raw)
  To: intel-wired-lan

On Fri, Jan 19, 2018 at 06:09:15PM -0800, Richard Cochran wrote:
> On Fri, Jan 19, 2018 at 04:15:46PM -0500, Willem de Bruijn wrote:
> > > +               if (cmsg->cmsg_len != CMSG_LEN(sizeof(ktime_t)))
> > > +                       return -EINVAL;
> > 
> > I don't see any existing reference to ktime_t in include/uapi. Just use a s64?
> 
> Agreed.  I didn't see the point of switching to ktime, either.

Do I understand it correctly that no other interface is using
nanoseconds since 1970? We probably don't have to worry about year
2262 yet, but wouldn't it be better to make it consistent with the
timestamping API using timespec? Or is it just better to avoid the
64/32-bit mess of time_t?

-- 
Miroslav Lichvar

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

* Re: [Intel-wired-lan] [RFC v2 net-next 01/10] net: Add a new socket option for a future transmit time.
  2018-01-25  9:12         ` Miroslav Lichvar
@ 2018-01-25 16:52           ` Richard Cochran
  -1 siblings, 0 replies; 86+ messages in thread
From: Richard Cochran @ 2018-01-25 16:52 UTC (permalink / raw)
  To: Miroslav Lichvar
  Cc: Willem de Bruijn, John Stultz, Richard Cochran,
	Jiří Pírko, ivan.briano, Network Development,
	henrik, Jamal Hadi Salim, levi.pearson, intel-wired-lan,
	Cong Wang, Thomas Gleixner, anna-maria, Jesus Sanchez-Palencia

On Thu, Jan 25, 2018 at 10:12:25AM +0100, Miroslav Lichvar wrote:
> Do I understand it correctly that no other interface is using
> nanoseconds since 1970? We probably don't have to worry about year
> 2262 yet, but wouldn't it be better to make it consistent with the
> timestamping API using timespec? Or is it just better to avoid the
> 64/32-bit mess of time_t?

I prefer a single 64 bit nanoseconds field:

- Applications won't have to convert to timespec.

- Avoids the time_t issue.

Thanks,
Richard

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

* [Intel-wired-lan] [RFC v2 net-next 01/10] net: Add a new socket option for a future transmit time.
@ 2018-01-25 16:52           ` Richard Cochran
  0 siblings, 0 replies; 86+ messages in thread
From: Richard Cochran @ 2018-01-25 16:52 UTC (permalink / raw)
  To: intel-wired-lan

On Thu, Jan 25, 2018 at 10:12:25AM +0100, Miroslav Lichvar wrote:
> Do I understand it correctly that no other interface is using
> nanoseconds since 1970? We probably don't have to worry about year
> 2262 yet, but wouldn't it be better to make it consistent with the
> timestamping API using timespec? Or is it just better to avoid the
> 64/32-bit mess of time_t?

I prefer a single 64 bit nanoseconds field:

- Applications won't have to convert to timespec.

- Avoids the time_t issue.

Thanks,
Richard

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

* Re: [Intel-wired-lan] [RFC v2 net-next 01/10] net: Add a new socket option for a future transmit time.
  2018-01-24 22:46           ` Vinicius Costa Gomes
@ 2018-01-26  2:12             ` Richard Cochran
  -1 siblings, 0 replies; 86+ messages in thread
From: Richard Cochran @ 2018-01-26  2:12 UTC (permalink / raw)
  To: Vinicius Costa Gomes
  Cc: Miroslav Lichvar, Jesus Sanchez-Palencia, netdev, john.stultz,
	Richard Cochran, jiri, ivan.briano, henrik, jhs, levi.pearson,
	intel-wired-lan, xiyou.wangcong, tglx, anna-maria

On Wed, Jan 24, 2018 at 02:46:24PM -0800, Vinicius Costa Gomes wrote:
> The only robust way that we could think of about keeping the the packets
> in order for the device queue is re-ordering packets in the Qdisc.

Right, but you cannot afford the overhead of the timerqueue when using
HW offload, when the HW device sits on a PCIe bus.  Many serious TSN
applications (like industrial controls) will want to have just one
packet queued, readying the next one just in time for the next
deadline.  The control loops are sensitive to the feedback interval.
 
> Even if we reach a decision that the Qdisc should not re-order packets
> (we wouldn't have any dependency on hrtimers in the offload case, as you
> pointed out), we still need hrtimers for the software implementation.

Fine.
 
> So, I guess, the problem remains, if it's possible for the user to
> express a /dev/ptp* clock, what should we do? 

Thinking a bit more, it doesn't make sense to have a user choice for
the HW offloading case.  The clock should implicitly be the device
clock, always.  Using any other clock would make no sense.

Thanks,
Richard

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

* [Intel-wired-lan] [RFC v2 net-next 01/10] net: Add a new socket option for a future transmit time.
@ 2018-01-26  2:12             ` Richard Cochran
  0 siblings, 0 replies; 86+ messages in thread
From: Richard Cochran @ 2018-01-26  2:12 UTC (permalink / raw)
  To: intel-wired-lan

On Wed, Jan 24, 2018 at 02:46:24PM -0800, Vinicius Costa Gomes wrote:
> The only robust way that we could think of about keeping the the packets
> in order for the device queue is re-ordering packets in the Qdisc.

Right, but you cannot afford the overhead of the timerqueue when using
HW offload, when the HW device sits on a PCIe bus.  Many serious TSN
applications (like industrial controls) will want to have just one
packet queued, readying the next one just in time for the next
deadline.  The control loops are sensitive to the feedback interval.
 
> Even if we reach a decision that the Qdisc should not re-order packets
> (we wouldn't have any dependency on hrtimers in the offload case, as you
> pointed out), we still need hrtimers for the software implementation.

Fine.
 
> So, I guess, the problem remains, if it's possible for the user to
> express a /dev/ptp* clock, what should we do? 

Thinking a bit more, it doesn't make sense to have a user choice for
the HW offloading case.  The clock should implicitly be the device
clock, always.  Using any other clock would make no sense.

Thanks,
Richard

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

* Re: [RFC v2 net-next 00/10] Time based packet transmission
  2018-01-24  1:43   ` [Intel-wired-lan] " Levi Pearson
@ 2018-01-27  0:04     ` Jesus Sanchez-Palencia
  -1 siblings, 0 replies; 86+ messages in thread
From: Jesus Sanchez-Palencia @ 2018-01-27  0:04 UTC (permalink / raw)
  To: Levi Pearson
  Cc: Linux Kernel Network Developers, Jamal Hadi Salim, Cong Wang,
	Jiri Pirko, Vinicius Costa Gomes, Richard Cochran,
	intel-wired-lan, anna-maria, Henrik Austad, tglx, John Stultz,
	andre.guedes, Ivan Briano

Hi Levi,

On 01/23/2018 05:43 PM, Levi Pearson wrote:
> On Wed, Jan 17, 2018 at 4:06 PM, Jesus Sanchez-Palencia
> <jesus.sanchez-palencia@intel.com> wrote:
>> This series is the v2 of the Time based packet transmission RFC, which was
>> originally proposed by Richard Cochran: https://lwn.net/Articles/733962/ .
> 
> Great to see you carrying on with this!
> 
>> Our main questions at this stage are related to the qdisc:
>>  - does the proposed design attend all use cases?
>>  - should the qdisc really drop packets that expired after being queued even
>>    for the SW best effort mode?
> 
> I don't think that being "expired" is necessarily cause for dropping.
> The semantic of a launch time is "launch no earlier than this point"
> after all, not a deadline. To keep the hardware working, we must only
> enforce the invariant that we never queue a packet with an earlier
> timestamp than one we previously enqueued that has not launched yet.
> Just checking for expiration is going to rule out some potential uses
> and also won't necessarily prevent enqueuing out-of-order packets.


Let me just split this a bit to make sure we don’t mix things up.

Currently, as discussed during the RFC v1 thread, on tbs_enqueue() we drop
packets if they are expired or if they have an earlier timestamp than the last
dequeued packet.

On tbs_dequeue(), we drop packets if they have expired while sitting at our
timerqueue. That is done because our current semantic for txtime is “no later
than this point”.  Are you suggesting that we change that to “no earlier than
this point” instead? The delta parameter would then be defining how early is
acceptable for dequeuing a packet, but we’ll need another parameter that can
define how late it should be when we decide to drop it.



> Here is an example:
> 
> A group of applications enqueue packets to be sent at 1 second
> intervals, and share a 5ms window in which they can send them. Due to
> scheduling variation, they may finish executing in a different order
> per interval, and occasionally some may not finish preparing their
> packet before the window opens, although they always will present
> their packet before the window closes.
> 
> If they all pick different times within the launch window, it is
> possible that two of them might pick times very close to one another.
> If they present their frames out-of-order to the qdisc, but close
> enough to the launch time that the qdisc doesn't hold on to them (i.e.
> in the [txtime - delta, txtime] range mentioned in tbs_dequeue), then
> they will get enqueued out of order and the invariant will be
> violated.  Reordering within some time window only works if all frames
> for that window are scheduled well in advance of the first launch
> time, and that's not great for applications that need to to calculate
> right up to the time they need to send their data.


I like the example, but due to the data structure that we use internally,
tbs_enqueue() will always enqueue packets onto their correct position, i.e. the
rbtree will always be ‘sorted’. If a dequeue() happens before the next enqueue,
then yes we may get to the situation you are describing, but that will always be
true regardless of the applications that are running, right? If that can’t be
fixed in userspace, then I’m afraid that either using a per-packet txtime is not
the right strategy for this system or tbs might not be the correct qdisc for it.

(...)


> 
> To maintain the hardware ordering invariant, you need to keep track of
> the most recent timestamp you have enqueued in the hardware. Anything
> that hits tbs_enqueue with a timestamp earlier than that must be
> either dropped or have its timestamp adjusted.


Yes, and we currently drop them there (that’s what the ktime_before(txtime, q->last)
check is doing). Adjusting timestamps is a can-of-worms, in my opinion, and I
don’t think we should go down that route.



> 
> The one remaining question is how late can a timestamped frame be
> before it should be dropped instead of enqueued, assuming it is to be
> allowed at all? The qdisc could track the allowed window based on user
> configuration. I believe the i210 hardware will launch any frame at
> the head of queue with a launch time set at or before the present
> time, but not so far before that it wraps and interprets the time as a
> future time. The qdisc would need to be able query the driver about
> how large that window is if it wants to pass in-the-past timestamps
> through as-is, but it could also just update timestamps still within
> the user-configured window to be set at the current time.


I believe I have tackled the question here already. For the rest, we don’t think
a qdisc should fetch any information from the driver. The information flow
should be kept as is, from qdisc to the driver, not the other way around.



> 
> My understanding of reservations for industrial TSN use cases is that
> applications will present their working period and their scheduling
> accuracy to the central manager, which will take into account the
> worst case timing bounds when creating the window that the application
> will use on the network. It will then give back an assignment for a
> start time offset from the period base time (UTC_time values that are
> multiples of interval_time) at which the application's transmit window
> starts, and it will remain open long enough to account for the
> scheduling jitter in the application.
> 
> I think putting the window concept in the qdisc makes for a nice
> mapping to how the TSN scheduling works as well as resolving some of
> the tricky details around ensuring that you don't jam the hardware
> with out-of-order timestamps or unnecessarily delay scheduling packets
> to reorder them.


I follow what you are saying, but I think we will either have those fixed at
userspace or through a separate mechanism. The tbs qdisc should be kept simple
and concise:

 * it provides a per-packet tx time;

 * it holds packets (in order) until at least their txtime minus a configurable
delta;

 * it drops packets if they are expired (or perhaps if they are after a
configurable threshold);

 * it avoids out-of-order timestamps and it allows you to turn a hw feature on
to increase accuracy for per-packet time based transmission.

Even though the concept of per-queue tx windows (with periods and offsets)
should be implementable on top of tbs, our opinion is that it might better fit
under yet another qdisc that should be designed specifically for that. We have
shared some ideas before, but that is definitely out-of-scope here.


Thanks for the feedback so far.

Regards,
Jesus

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

* [Intel-wired-lan] [RFC v2 net-next 00/10] Time based packet transmission
@ 2018-01-27  0:04     ` Jesus Sanchez-Palencia
  0 siblings, 0 replies; 86+ messages in thread
From: Jesus Sanchez-Palencia @ 2018-01-27  0:04 UTC (permalink / raw)
  To: intel-wired-lan

Hi Levi,

On 01/23/2018 05:43 PM, Levi Pearson wrote:
> On Wed, Jan 17, 2018 at 4:06 PM, Jesus Sanchez-Palencia
> <jesus.sanchez-palencia@intel.com> wrote:
>> This series is the v2 of the Time based packet transmission RFC, which was
>> originally proposed by Richard Cochran: https://lwn.net/Articles/733962/ .
> 
> Great to see you carrying on with this!
> 
>> Our main questions at this stage are related to the qdisc:
>>  - does the proposed design attend all use cases?
>>  - should the qdisc really drop packets that expired after being queued even
>>    for the SW best effort mode?
> 
> I don't think that being "expired" is necessarily cause for dropping.
> The semantic of a launch time is "launch no earlier than this point"
> after all, not a deadline. To keep the hardware working, we must only
> enforce the invariant that we never queue a packet with an earlier
> timestamp than one we previously enqueued that has not launched yet.
> Just checking for expiration is going to rule out some potential uses
> and also won't necessarily prevent enqueuing out-of-order packets.


Let me just split this a bit to make sure we don?t mix things up.

Currently, as discussed during the RFC v1 thread, on tbs_enqueue() we drop
packets if they are expired or if they have an earlier timestamp than the last
dequeued packet.

On tbs_dequeue(), we drop packets if they have expired while sitting at our
timerqueue. That is done because our current semantic for txtime is ?no later
than this point?.  Are you suggesting that we change that to ?no earlier than
this point? instead? The delta parameter would then be defining how early is
acceptable for dequeuing a packet, but we?ll need another parameter that can
define how late it should be when we decide to drop it.



> Here is an example:
> 
> A group of applications enqueue packets to be sent at 1 second
> intervals, and share a 5ms window in which they can send them. Due to
> scheduling variation, they may finish executing in a different order
> per interval, and occasionally some may not finish preparing their
> packet before the window opens, although they always will present
> their packet before the window closes.
> 
> If they all pick different times within the launch window, it is
> possible that two of them might pick times very close to one another.
> If they present their frames out-of-order to the qdisc, but close
> enough to the launch time that the qdisc doesn't hold on to them (i.e.
> in the [txtime - delta, txtime] range mentioned in tbs_dequeue), then
> they will get enqueued out of order and the invariant will be
> violated.  Reordering within some time window only works if all frames
> for that window are scheduled well in advance of the first launch
> time, and that's not great for applications that need to to calculate
> right up to the time they need to send their data.


I like the example, but due to the data structure that we use internally,
tbs_enqueue() will always enqueue packets onto their correct position, i.e. the
rbtree will always be ?sorted?. If a dequeue() happens before the next enqueue,
then yes we may get to the situation you are describing, but that will always be
true regardless of the applications that are running, right? If that can?t be
fixed in userspace, then I?m afraid that either using a per-packet txtime is not
the right strategy for this system or tbs might not be the correct qdisc for it.

(...)


> 
> To maintain the hardware ordering invariant, you need to keep track of
> the most recent timestamp you have enqueued in the hardware. Anything
> that hits tbs_enqueue with a timestamp earlier than that must be
> either dropped or have its timestamp adjusted.


Yes, and we currently drop them there (that?s what the ktime_before(txtime, q->last)
check is doing). Adjusting timestamps is a can-of-worms, in my opinion, and I
don?t think we should go down that route.



> 
> The one remaining question is how late can a timestamped frame be
> before it should be dropped instead of enqueued, assuming it is to be
> allowed at all? The qdisc could track the allowed window based on user
> configuration. I believe the i210 hardware will launch any frame at
> the head of queue with a launch time set at or before the present
> time, but not so far before that it wraps and interprets the time as a
> future time. The qdisc would need to be able query the driver about
> how large that window is if it wants to pass in-the-past timestamps
> through as-is, but it could also just update timestamps still within
> the user-configured window to be set at the current time.


I believe I have tackled the question here already. For the rest, we don?t think
a qdisc should fetch any information from the driver. The information flow
should be kept as is, from qdisc to the driver, not the other way around.



> 
> My understanding of reservations for industrial TSN use cases is that
> applications will present their working period and their scheduling
> accuracy to the central manager, which will take into account the
> worst case timing bounds when creating the window that the application
> will use on the network. It will then give back an assignment for a
> start time offset from the period base time (UTC_time values that are
> multiples of interval_time) at which the application's transmit window
> starts, and it will remain open long enough to account for the
> scheduling jitter in the application.
> 
> I think putting the window concept in the qdisc makes for a nice
> mapping to how the TSN scheduling works as well as resolving some of
> the tricky details around ensuring that you don't jam the hardware
> with out-of-order timestamps or unnecessarily delay scheduling packets
> to reorder them.


I follow what you are saying, but I think we will either have those fixed at
userspace or through a separate mechanism. The tbs qdisc should be kept simple
and concise:

 * it provides a per-packet tx time;

 * it holds packets (in order) until at least their txtime minus a configurable
delta;

 * it drops packets if they are expired (or perhaps if they are after a
configurable threshold);

 * it avoids out-of-order timestamps and it allows you to turn a hw feature on
to increase accuracy for per-packet time based transmission.

Even though the concept of per-queue tx windows (with periods and offsets)
should be implementable on top of tbs, our opinion is that it might better fit
under yet another qdisc that should be designed specifically for that. We have
shared some ideas before, but that is definitely out-of-scope here.


Thanks for the feedback so far.

Regards,
Jesus

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

* Re: [Intel-wired-lan] [RFC v2 net-next 01/10] net: Add a new socket option for a future transmit time.
  2018-01-18 17:13       ` Richard Cochran
@ 2018-02-01  0:49         ` Jesus Sanchez-Palencia
  -1 siblings, 0 replies; 86+ messages in thread
From: Jesus Sanchez-Palencia @ 2018-02-01  0:49 UTC (permalink / raw)
  To: Richard Cochran, Miroslav Lichvar
  Cc: netdev, john.stultz, Richard Cochran, jiri, ivan.briano, henrik,
	jhs, levi.pearson, intel-wired-lan, xiyou.wangcong, tglx,
	anna-maria

Hi,


On 01/18/2018 09:13 AM, Richard Cochran wrote:
> On Thu, Jan 18, 2018 at 09:42:27AM +0100, Miroslav Lichvar wrote:
>> In the discussion about the v1 patchset, there was a question if the
>> cmsg should include a clockid_t. Without that, how can an application
>> prevent the packet from being sent using an incorrect clock, e.g.
>> the system clock when it expects it to be a PHC, or a different PHC
>> when the socket is not bound to a specific interface?
> 
> Right, the clockid_t should be passed in through the CMSG along with
> the time.

While implementing this today it crossed my mind that why don't we have the
clockid_t set per socket (e.g. as an argument to SO_TXTIME) instead of per packet?

The only use-case that we could think of that would be 'blocked' was using
sendmmsg() to send a packet to different interfaces with a single syscall, but
I'm not sure how common that is.

What do you think?

Thanks,
Jesus


>  
> Thanks,
> Richard
> 

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

* [Intel-wired-lan] [RFC v2 net-next 01/10] net: Add a new socket option for a future transmit time.
@ 2018-02-01  0:49         ` Jesus Sanchez-Palencia
  0 siblings, 0 replies; 86+ messages in thread
From: Jesus Sanchez-Palencia @ 2018-02-01  0:49 UTC (permalink / raw)
  To: intel-wired-lan

Hi,


On 01/18/2018 09:13 AM, Richard Cochran wrote:
> On Thu, Jan 18, 2018 at 09:42:27AM +0100, Miroslav Lichvar wrote:
>> In the discussion about the v1 patchset, there was a question if the
>> cmsg should include a clockid_t. Without that, how can an application
>> prevent the packet from being sent using an incorrect clock, e.g.
>> the system clock when it expects it to be a PHC, or a different PHC
>> when the socket is not bound to a specific interface?
> 
> Right, the clockid_t should be passed in through the CMSG along with
> the time.

While implementing this today it crossed my mind that why don't we have the
clockid_t set per socket (e.g. as an argument to SO_TXTIME) instead of per packet?

The only use-case that we could think of that would be 'blocked' was using
sendmmsg() to send a packet to different interfaces with a single syscall, but
I'm not sure how common that is.

What do you think?

Thanks,
Jesus


>  
> Thanks,
> Richard
> 

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

* Re: [Intel-wired-lan] [RFC v2 net-next 01/10] net: Add a new socket option for a future transmit time.
  2018-02-01  0:49         ` Jesus Sanchez-Palencia
@ 2018-02-01  4:16           ` Richard Cochran
  -1 siblings, 0 replies; 86+ messages in thread
From: Richard Cochran @ 2018-02-01  4:16 UTC (permalink / raw)
  To: Jesus Sanchez-Palencia
  Cc: Miroslav Lichvar, netdev, john.stultz, Richard Cochran, jiri,
	ivan.briano, henrik, jhs, levi.pearson, intel-wired-lan,
	xiyou.wangcong, tglx, anna-maria

On Wed, Jan 31, 2018 at 04:49:36PM -0800, Jesus Sanchez-Palencia wrote:
> While implementing this today it crossed my mind that why don't we have the
> clockid_t set per socket (e.g. as an argument to SO_TXTIME) instead of per packet?

Sounds good to me.

Thanks,
Richard

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

* [Intel-wired-lan] [RFC v2 net-next 01/10] net: Add a new socket option for a future transmit time.
@ 2018-02-01  4:16           ` Richard Cochran
  0 siblings, 0 replies; 86+ messages in thread
From: Richard Cochran @ 2018-02-01  4:16 UTC (permalink / raw)
  To: intel-wired-lan

On Wed, Jan 31, 2018 at 04:49:36PM -0800, Jesus Sanchez-Palencia wrote:
> While implementing this today it crossed my mind that why don't we have the
> clockid_t set per socket (e.g. as an argument to SO_TXTIME) instead of per packet?

Sounds good to me.

Thanks,
Richard

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

* Re: [Intel-wired-lan] [RFC v2 net-next 01/10] net: Add a new socket option for a future transmit time.
  2018-02-01  0:49         ` Jesus Sanchez-Palencia
@ 2018-02-01  9:27           ` Miroslav Lichvar
  -1 siblings, 0 replies; 86+ messages in thread
From: Miroslav Lichvar @ 2018-02-01  9:27 UTC (permalink / raw)
  To: Jesus Sanchez-Palencia
  Cc: Richard Cochran, netdev, john.stultz, Richard Cochran, jiri,
	ivan.briano, henrik, jhs, levi.pearson, intel-wired-lan,
	xiyou.wangcong, tglx, anna-maria

On Wed, Jan 31, 2018 at 04:49:36PM -0800, Jesus Sanchez-Palencia wrote:
> On 01/18/2018 09:13 AM, Richard Cochran wrote:
> > Right, the clockid_t should be passed in through the CMSG along with
> > the time.
> 
> While implementing this today it crossed my mind that why don't we have the
> clockid_t set per socket (e.g. as an argument to SO_TXTIME) instead of per packet?

I suspect that might have an impact on the performance. Even if the
application doesn't use sendmmsg(), it would possibly have to call
setsockopt() before each sendmsg() to change the clockid_t, right?

If clockid_t could be set per packet, a special value could be used
to allow sending on interfaces that don't support it.

> The only use-case that we could think of that would be 'blocked' was using
> sendmmsg() to send a packet to different interfaces with a single syscall, but
> I'm not sure how common that is.

The SO_TXTIME option will make sendmmsg() useful in applications where
it wasn't before. For instance, an NTP server will be able to batch
multiple responses as their transmit timestamps can be set accurately
in advance and it's no longer necessary to send the responses as soon
as they are assembled.

I think it would be nice the sendmmsg() calls didn't have to be split
by clockid_t.

-- 
Miroslav Lichvar

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

* [Intel-wired-lan] [RFC v2 net-next 01/10] net: Add a new socket option for a future transmit time.
@ 2018-02-01  9:27           ` Miroslav Lichvar
  0 siblings, 0 replies; 86+ messages in thread
From: Miroslav Lichvar @ 2018-02-01  9:27 UTC (permalink / raw)
  To: intel-wired-lan

On Wed, Jan 31, 2018 at 04:49:36PM -0800, Jesus Sanchez-Palencia wrote:
> On 01/18/2018 09:13 AM, Richard Cochran wrote:
> > Right, the clockid_t should be passed in through the CMSG along with
> > the time.
> 
> While implementing this today it crossed my mind that why don't we have the
> clockid_t set per socket (e.g. as an argument to SO_TXTIME) instead of per packet?

I suspect that might have an impact on the performance. Even if the
application doesn't use sendmmsg(), it would possibly have to call
setsockopt() before each sendmsg() to change the clockid_t, right?

If clockid_t could be set per packet, a special value could be used
to allow sending on interfaces that don't support it.

> The only use-case that we could think of that would be 'blocked' was using
> sendmmsg() to send a packet to different interfaces with a single syscall, but
> I'm not sure how common that is.

The SO_TXTIME option will make sendmmsg() useful in applications where
it wasn't before. For instance, an NTP server will be able to batch
multiple responses as their transmit timestamps can be set accurately
in advance and it's no longer necessary to send the responses as soon
as they are assembled.

I think it would be nice the sendmmsg() calls didn't have to be split
by clockid_t.

-- 
Miroslav Lichvar

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

* Re: [Intel-wired-lan] [RFC v2 net-next 01/10] net: Add a new socket option for a future transmit time.
  2018-02-01  9:27           ` Miroslav Lichvar
@ 2018-02-01 20:55             ` Jesus Sanchez-Palencia
  -1 siblings, 0 replies; 86+ messages in thread
From: Jesus Sanchez-Palencia @ 2018-02-01 20:55 UTC (permalink / raw)
  To: Miroslav Lichvar
  Cc: Richard Cochran, netdev, john.stultz, Richard Cochran, jiri,
	ivan.briano, henrik, jhs, levi.pearson, intel-wired-lan,
	xiyou.wangcong, tglx, anna-maria

Hi,


On 02/01/2018 01:27 AM, Miroslav Lichvar wrote:
> On Wed, Jan 31, 2018 at 04:49:36PM -0800, Jesus Sanchez-Palencia wrote:
>> On 01/18/2018 09:13 AM, Richard Cochran wrote:
>>> Right, the clockid_t should be passed in through the CMSG along with
>>> the time.
>>
>> While implementing this today it crossed my mind that why don't we have the
>> clockid_t set per socket (e.g. as an argument to SO_TXTIME) instead of per packet?
> 
> I suspect that might have an impact on the performance. Even if the
> application doesn't use sendmmsg(), it would possibly have to call
> setsockopt() before each sendmsg() to change the clockid_t, right?


Yes. On the other hand, for applications that will be using only 1 clockid_t,
keeping it per packet will also have an impact as we'll be copying the same
value from the cmsg cookie into sk_buffs over and over.


> 
> If clockid_t could be set per packet, a special value could be used
> to allow sending on interfaces that don't support it.
> 
>> The only use-case that we could think of that would be 'blocked' was using
>> sendmmsg() to send a packet to different interfaces with a single syscall, but
>> I'm not sure how common that is.
> 
> The SO_TXTIME option will make sendmmsg() useful in applications where
> it wasn't before. For instance, an NTP server will be able to batch
> multiple responses as their transmit timestamps can be set accurately
> in advance and it's no longer necessary to send the responses as soon
> as they are assembled.
> 
> I think it would be nice the sendmmsg() calls didn't have to be split
> by clockid_t.


OK, fair enough. I will keep it per-packet for now as initially agreed and we
can revisit this later if needed.


Thanks,
Jesus

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

* [Intel-wired-lan] [RFC v2 net-next 01/10] net: Add a new socket option for a future transmit time.
@ 2018-02-01 20:55             ` Jesus Sanchez-Palencia
  0 siblings, 0 replies; 86+ messages in thread
From: Jesus Sanchez-Palencia @ 2018-02-01 20:55 UTC (permalink / raw)
  To: intel-wired-lan

Hi,


On 02/01/2018 01:27 AM, Miroslav Lichvar wrote:
> On Wed, Jan 31, 2018 at 04:49:36PM -0800, Jesus Sanchez-Palencia wrote:
>> On 01/18/2018 09:13 AM, Richard Cochran wrote:
>>> Right, the clockid_t should be passed in through the CMSG along with
>>> the time.
>>
>> While implementing this today it crossed my mind that why don't we have the
>> clockid_t set per socket (e.g. as an argument to SO_TXTIME) instead of per packet?
> 
> I suspect that might have an impact on the performance. Even if the
> application doesn't use sendmmsg(), it would possibly have to call
> setsockopt() before each sendmsg() to change the clockid_t, right?


Yes. On the other hand, for applications that will be using only 1 clockid_t,
keeping it per packet will also have an impact as we'll be copying the same
value from the cmsg cookie into sk_buffs over and over.


> 
> If clockid_t could be set per packet, a special value could be used
> to allow sending on interfaces that don't support it.
> 
>> The only use-case that we could think of that would be 'blocked' was using
>> sendmmsg() to send a packet to different interfaces with a single syscall, but
>> I'm not sure how common that is.
> 
> The SO_TXTIME option will make sendmmsg() useful in applications where
> it wasn't before. For instance, an NTP server will be able to batch
> multiple responses as their transmit timestamps can be set accurately
> in advance and it's no longer necessary to send the responses as soon
> as they are assembled.
> 
> I think it would be nice the sendmmsg() calls didn't have to be split
> by clockid_t.


OK, fair enough. I will keep it per-packet for now as initially agreed and we
can revisit this later if needed.


Thanks,
Jesus

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

* Re: [Intel-wired-lan] [RFC v2 net-next 01/10] net: Add a new socket option for a future transmit time.
  2018-01-18  8:42     ` Miroslav Lichvar
@ 2018-02-12 22:39       ` Jesus Sanchez-Palencia
  -1 siblings, 0 replies; 86+ messages in thread
From: Jesus Sanchez-Palencia @ 2018-02-12 22:39 UTC (permalink / raw)
  To: Miroslav Lichvar
  Cc: netdev, john.stultz, Richard Cochran, jiri, ivan.briano,
	richardcochran, henrik, jhs, levi.pearson, intel-wired-lan,
	xiyou.wangcong, tglx, anna-maria

Hi,


On 01/18/2018 12:42 AM, Miroslav Lichvar wrote:
> On Wed, Jan 17, 2018 at 03:06:12PM -0800, Jesus Sanchez-Palencia wrote:
>> From: Richard Cochran <rcochran@linutronix.de>
>>
>> This patch introduces SO_TXTIME.  User space enables this option in
>> order to pass a desired future transmit time in a CMSG when calling
>> sendmsg(2).
>>
>> A new field is added to struct sockcm_cookie, and the tstamp from
>> skbuffs will be used later on.
> 
> In the discussion about the v1 patchset, there was a question if the
> cmsg should include a clockid_t. Without that, how can an application
> prevent the packet from being sent using an incorrect clock, e.g.
> the system clock when it expects it to be a PHC, or a different PHC
> when the socket is not bound to a specific interface?
> 
> At least in some applications it would be preferred to not sent a
> packet at all instead of sending it at a wrong time.
> 
> Please keep in mind that the PHCs and the system clock don't have to
> be synchronized to each other. If I understand the rest of the series
> correctly, there is an assumption that the PHCs are keeping time in
> TAI and CLOCK_TAI can be used as a fallback.


Just to double-check, imagine that I've configured the qdisc for
SW best-effort and with clockid CLOCK_REALTIME. When it receives a
packet with the clockid of a /dev/ptpX, the qdisc should just drop that
packet, right?

Or would this block any use-cases that I couldn't think of ?

Thanks,
Jesus

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

* [Intel-wired-lan] [RFC v2 net-next 01/10] net: Add a new socket option for a future transmit time.
@ 2018-02-12 22:39       ` Jesus Sanchez-Palencia
  0 siblings, 0 replies; 86+ messages in thread
From: Jesus Sanchez-Palencia @ 2018-02-12 22:39 UTC (permalink / raw)
  To: intel-wired-lan

Hi,


On 01/18/2018 12:42 AM, Miroslav Lichvar wrote:
> On Wed, Jan 17, 2018 at 03:06:12PM -0800, Jesus Sanchez-Palencia wrote:
>> From: Richard Cochran <rcochran@linutronix.de>
>>
>> This patch introduces SO_TXTIME.  User space enables this option in
>> order to pass a desired future transmit time in a CMSG when calling
>> sendmsg(2).
>>
>> A new field is added to struct sockcm_cookie, and the tstamp from
>> skbuffs will be used later on.
> 
> In the discussion about the v1 patchset, there was a question if the
> cmsg should include a clockid_t. Without that, how can an application
> prevent the packet from being sent using an incorrect clock, e.g.
> the system clock when it expects it to be a PHC, or a different PHC
> when the socket is not bound to a specific interface?
> 
> At least in some applications it would be preferred to not sent a
> packet at all instead of sending it at a wrong time.
> 
> Please keep in mind that the PHCs and the system clock don't have to
> be synchronized to each other. If I understand the rest of the series
> correctly, there is an assumption that the PHCs are keeping time in
> TAI and CLOCK_TAI can be used as a fallback.


Just to double-check, imagine that I've configured the qdisc for
SW best-effort and with clockid CLOCK_REALTIME. When it receives a
packet with the clockid of a /dev/ptpX, the qdisc should just drop that
packet, right?

Or would this block any use-cases that I couldn't think of ?

Thanks,
Jesus

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

* Re: [Intel-wired-lan] [RFC v2 net-next 01/10] net: Add a new socket option for a future transmit time.
  2018-02-12 22:39       ` Jesus Sanchez-Palencia
@ 2018-02-13  9:56         ` Miroslav Lichvar
  -1 siblings, 0 replies; 86+ messages in thread
From: Miroslav Lichvar @ 2018-02-13  9:56 UTC (permalink / raw)
  To: Jesus Sanchez-Palencia
  Cc: netdev, john.stultz, Richard Cochran, jiri, ivan.briano,
	richardcochran, henrik, jhs, levi.pearson, intel-wired-lan,
	xiyou.wangcong, tglx, anna-maria

On Mon, Feb 12, 2018 at 02:39:06PM -0800, Jesus Sanchez-Palencia wrote:
> On 01/18/2018 12:42 AM, Miroslav Lichvar wrote:
> > Please keep in mind that the PHCs and the system clock don't have to
> > be synchronized to each other. If I understand the rest of the series
> > correctly, there is an assumption that the PHCs are keeping time in
> > TAI and CLOCK_TAI can be used as a fallback.
> 
> Just to double-check, imagine that I've configured the qdisc for
> SW best-effort and with clockid CLOCK_REALTIME. When it receives a
> packet with the clockid of a /dev/ptpX, the qdisc should just drop that
> packet, right?

Yes, I think it should drop it. The kernel does not know the offset
between the two clocks (they don't even have to be synchronized), so
it cannot convert a PHC-based TX time to the system time.

-- 
Miroslav Lichvar

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

* [Intel-wired-lan] [RFC v2 net-next 01/10] net: Add a new socket option for a future transmit time.
@ 2018-02-13  9:56         ` Miroslav Lichvar
  0 siblings, 0 replies; 86+ messages in thread
From: Miroslav Lichvar @ 2018-02-13  9:56 UTC (permalink / raw)
  To: intel-wired-lan

On Mon, Feb 12, 2018 at 02:39:06PM -0800, Jesus Sanchez-Palencia wrote:
> On 01/18/2018 12:42 AM, Miroslav Lichvar wrote:
> > Please keep in mind that the PHCs and the system clock don't have to
> > be synchronized to each other. If I understand the rest of the series
> > correctly, there is an assumption that the PHCs are keeping time in
> > TAI and CLOCK_TAI can be used as a fallback.
> 
> Just to double-check, imagine that I've configured the qdisc for
> SW best-effort and with clockid CLOCK_REALTIME. When it receives a
> packet with the clockid of a /dev/ptpX, the qdisc should just drop that
> packet, right?

Yes, I think it should drop it. The kernel does not know the offset
between the two clocks (they don't even have to be synchronized), so
it cannot convert a PHC-based TX time to the system time.

-- 
Miroslav Lichvar

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

end of thread, other threads:[~2018-02-13  9:56 UTC | newest]

Thread overview: 86+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2018-01-17 23:06 [RFC v2 net-next 00/10] Time based packet transmission Jesus Sanchez-Palencia
2018-01-17 23:06 ` [Intel-wired-lan] " Jesus Sanchez-Palencia
2018-01-17 23:06 ` [RFC v2 net-next 01/10] net: Add a new socket option for a future transmit time Jesus Sanchez-Palencia
2018-01-17 23:06   ` [Intel-wired-lan] " Jesus Sanchez-Palencia
2018-01-18  8:42   ` Miroslav Lichvar
2018-01-18  8:42     ` Miroslav Lichvar
2018-01-18 17:13     ` Richard Cochran
2018-01-18 17:13       ` Richard Cochran
2018-02-01  0:49       ` Jesus Sanchez-Palencia
2018-02-01  0:49         ` Jesus Sanchez-Palencia
2018-02-01  4:16         ` Richard Cochran
2018-02-01  4:16           ` Richard Cochran
2018-02-01  9:27         ` Miroslav Lichvar
2018-02-01  9:27           ` Miroslav Lichvar
2018-02-01 20:55           ` Jesus Sanchez-Palencia
2018-02-01 20:55             ` Jesus Sanchez-Palencia
2018-01-23 21:22     ` Vinicius Costa Gomes
2018-01-23 21:22       ` Vinicius Costa Gomes
2018-01-24  3:04       ` Richard Cochran
2018-01-24  3:04         ` Richard Cochran
2018-01-24 22:46         ` Vinicius Costa Gomes
2018-01-24 22:46           ` Vinicius Costa Gomes
2018-01-26  2:12           ` Richard Cochran
2018-01-26  2:12             ` Richard Cochran
2018-02-12 22:39     ` Jesus Sanchez-Palencia
2018-02-12 22:39       ` Jesus Sanchez-Palencia
2018-02-13  9:56       ` Miroslav Lichvar
2018-02-13  9:56         ` Miroslav Lichvar
2018-01-18 17:11   ` Richard Cochran
2018-01-18 17:11     ` [Intel-wired-lan] " Richard Cochran
2018-01-23 18:12     ` Jesus Sanchez-Palencia
2018-01-23 18:12       ` [Intel-wired-lan] " Jesus Sanchez-Palencia
2018-01-19 21:15   ` Willem de Bruijn
2018-01-19 21:15     ` [Intel-wired-lan] " Willem de Bruijn
2018-01-20  2:09     ` Richard Cochran
2018-01-20  2:09       ` [Intel-wired-lan] " Richard Cochran
2018-01-25  9:12       ` Miroslav Lichvar
2018-01-25  9:12         ` Miroslav Lichvar
2018-01-25 16:52         ` Richard Cochran
2018-01-25 16:52           ` Richard Cochran
2018-01-23 18:24     ` Jesus Sanchez-Palencia
2018-01-23 18:24       ` [Intel-wired-lan] " Jesus Sanchez-Palencia
2018-01-23 20:02       ` Willem de Bruijn
2018-01-23 20:02         ` [Intel-wired-lan] " Willem de Bruijn
2018-01-17 23:06 ` [RFC v2 net-next 02/10] net: ipv4: raw: Hook into time based transmission Jesus Sanchez-Palencia
2018-01-17 23:06   ` [Intel-wired-lan] " Jesus Sanchez-Palencia
2018-01-18  0:28   ` Eric Dumazet
2018-01-18  0:28     ` [Intel-wired-lan] " Eric Dumazet
2018-01-17 23:06 ` [RFC v2 net-next 03/10] net: ipv4: udp: " Jesus Sanchez-Palencia
2018-01-17 23:06   ` [Intel-wired-lan] " Jesus Sanchez-Palencia
2018-01-17 23:06 ` [RFC v2 net-next 04/10] net: packet: " Jesus Sanchez-Palencia
2018-01-17 23:06   ` [Intel-wired-lan] " Jesus Sanchez-Palencia
2018-01-17 23:06 ` [RFC v2 net-next 05/10] net/sched: Allow creating a Qdisc watchdog with other clocks Jesus Sanchez-Palencia
2018-01-17 23:06   ` [Intel-wired-lan] " Jesus Sanchez-Palencia
2018-01-17 23:06 ` [RFC v2 net-next 06/10] net/sched: Introduce the TBS Qdisc Jesus Sanchez-Palencia
2018-01-17 23:06   ` [Intel-wired-lan] " Jesus Sanchez-Palencia
2018-01-18 13:35   ` Jamal Hadi Salim
2018-01-18 13:35     ` [Intel-wired-lan] " Jamal Hadi Salim
2018-01-18 13:44     ` Jamal Hadi Salim
2018-01-18 13:44       ` [Intel-wired-lan] " Jamal Hadi Salim
2018-01-23 21:45       ` Jesus Sanchez-Palencia
2018-01-23 21:45         ` [Intel-wired-lan] " Jesus Sanchez-Palencia
2018-01-18 17:18     ` Richard Cochran
2018-01-18 17:18       ` [Intel-wired-lan] " Richard Cochran
2018-01-23 22:01     ` Jesus Sanchez-Palencia
2018-01-23 22:01       ` [Intel-wired-lan] " Jesus Sanchez-Palencia
2018-01-19 21:18   ` Willem de Bruijn
2018-01-19 21:18     ` [Intel-wired-lan] " Willem de Bruijn
2018-01-17 23:06 ` [RFC v2 net-next 07/10] igb: Refactor igb_configure_cbs() Jesus Sanchez-Palencia
2018-01-17 23:06   ` [Intel-wired-lan] " Jesus Sanchez-Palencia
2018-01-17 23:06 ` [RFC v2 net-next 08/10] igb: Only change Tx arbitration when CBS is on Jesus Sanchez-Palencia
2018-01-17 23:06   ` [Intel-wired-lan] " Jesus Sanchez-Palencia
2018-01-17 23:06 ` [RFC v2 net-next 09/10] igb: Refactor igb_offload_cbs() Jesus Sanchez-Palencia
2018-01-17 23:06   ` [Intel-wired-lan] " Jesus Sanchez-Palencia
2018-01-17 23:06 ` [RFC v2 net-next 10/10] igb: Add support for TBS offload Jesus Sanchez-Palencia
2018-01-17 23:06   ` [Intel-wired-lan] " Jesus Sanchez-Palencia
2018-01-23  5:23 ` [RFC v2 net-next 00/10] Time based packet transmission Richard Cochran
2018-01-23  5:23   ` [Intel-wired-lan] " Richard Cochran
2018-01-23  5:26   ` Richard Cochran
2018-01-23  5:26     ` [Intel-wired-lan] " Richard Cochran
2018-01-23 18:07     ` Jesus Sanchez-Palencia
2018-01-23 18:07       ` [Intel-wired-lan] " Jesus Sanchez-Palencia
2018-01-24  1:43 ` Levi Pearson
2018-01-24  1:43   ` [Intel-wired-lan] " Levi Pearson
2018-01-27  0:04   ` Jesus Sanchez-Palencia
2018-01-27  0:04     ` [Intel-wired-lan] " Jesus Sanchez-Palencia

This is an external index of several public inboxes,
see mirroring instructions on how to clone and mirror
all data and code used by this external index.