/*
* Distributed under the terms of the GNU GPL version 2.
- * Copyright (c) 2007, 2008, 2009, 2010 Nicira Networks.
+ * Copyright (c) 2007, 2008, 2009, 2010, 2011 Nicira Networks.
*
* Significant portions of this file may be copied from parts of the Linux
* kernel, by Linus Torvalds and others.
if (!skb)
return -ENOMEM;
- upcall.type = _ODPL_ACTION_NR;
+ upcall.cmd = ODP_PACKET_CMD_ACTION;
upcall.key = key;
upcall.userdata = arg;
upcall.sample_pool = 0;
if (unlikely(!nskb))
return;
- upcall.type = _ODPL_SFLOW_NR;
+ upcall.cmd = ODP_PACKET_CMD_SAMPLE;
upcall.key = key;
upcall.userdata = 0;
upcall.sample_pool = atomic_read(&p->sflow_pool);
#include <linux/if_vlan.h>
#include <linux/in.h>
#include <linux/ip.h>
+#include <linux/jhash.h>
#include <linux/delay.h>
#include <linux/time.h>
#include <linux/etherdevice.h>
static void destroy_dp_rcu(struct rcu_head *rcu)
{
struct datapath *dp = container_of(rcu, struct datapath, rcu);
- int i;
-
- for (i = 0; i < DP_N_QUEUES; i++)
- skb_queue_purge(&dp->queues[i]);
tbl_destroy((struct tbl __force *)dp->table, flow_free_tbl);
free_percpu(dp->stats_percpu);
if (unlikely(!flow_node)) {
struct dp_upcall_info upcall;
- upcall.type = _ODPL_MISS_NR;
+ upcall.cmd = ODP_PACKET_CMD_MISS;
upcall.key = &key;
upcall.userdata = 0;
upcall.sample_pool = 0;
*(__sum16 *)(to + csum_start + csum_offset) = csum_fold(csum);
}
-/* Append each packet in 'skb' list to 'queue'. There will be only one packet
- * unless we broke up a GSO packet. */
+static struct genl_family dp_packet_genl_family;
+#define PACKET_N_MC_GROUPS 16
+
+static int packet_mc_group(struct datapath *dp, u8 cmd)
+{
+ BUILD_BUG_ON_NOT_POWER_OF_2(PACKET_N_MC_GROUPS);
+ return jhash_2words(dp->dp_idx, cmd, 0) & (PACKET_N_MC_GROUPS - 1);
+}
+
+/* Send each packet in the 'skb' list to userspace for 'dp' as directed by
+ * 'upcall_info'. There will be only one packet unless we broke up a GSO
+ * packet.
+ */
static int queue_control_packets(struct datapath *dp, struct sk_buff *skb,
const struct dp_upcall_info *upcall_info)
{
+ u32 group = packet_mc_group(dp, upcall_info->cmd);
struct sk_buff *nskb;
int port_no;
int err;
port_no = ODPP_LOCAL;
do {
- struct odp_packet *upcall;
+ struct odp_header *upcall;
struct sk_buff *user_skb; /* to be queued to userspace */
struct nlattr *nla;
unsigned int len;
nskb = skb->next;
skb->next = NULL;
- len = sizeof(struct odp_packet);
+ len = sizeof(struct odp_header);
len += nla_total_size(4); /* ODP_PACKET_ATTR_TYPE. */
len += nla_total_size(skb->len);
len += nla_total_size(FLOW_BUFSIZE);
if (upcall_info->actions_len)
len += nla_total_size(upcall_info->actions_len);
- user_skb = alloc_skb(len, GFP_ATOMIC);
- if (!user_skb)
+ user_skb = genlmsg_new(len, GFP_ATOMIC);
+ if (!user_skb) {
+ netlink_set_err(INIT_NET_GENL_SOCK, 0, group, -ENOBUFS);
goto err_kfree_skbs;
+ }
- upcall = (struct odp_packet *)__skb_put(user_skb, sizeof(*upcall));
+ upcall = genlmsg_put(user_skb, 0, 0, &dp_packet_genl_family, 0, upcall_info->cmd);
upcall->dp_idx = dp->dp_idx;
- nla_put_u32(user_skb, ODP_PACKET_ATTR_TYPE, upcall_info->type);
-
nla = nla_nest_start(user_skb, ODP_PACKET_ATTR_KEY);
flow_to_nlattrs(upcall_info->key, user_skb);
nla_nest_end(user_skb, nla);
else
skb_copy_bits(skb, 0, nla_data(nla), skb->len);
- upcall->len = user_skb->len;
- skb_queue_tail(&dp->queues[upcall_info->type], user_skb);
+ err = genlmsg_multicast(user_skb, 0, group, GFP_ATOMIC);
+ if (err)
+ goto err_kfree_skbs;
kfree_skb(skb);
skb = nskb;
return err;
}
+/* Generic Netlink multicast groups for upcalls.
+ *
+ * We really want three unique multicast groups per datapath, but we can't even
+ * get one, because genl_register_mc_group() takes genl_lock, which is also
+ * held during Generic Netlink message processing, so trying to acquire
+ * multicast groups during ODP_DP_NEW processing deadlocks. Instead, we
+ * preallocate a few groups and use them round-robin for datapaths. Collision
+ * isn't fatal--multicast listeners should check that the family is the one
+ * that they want and discard others--but it wastes time and memory to receive
+ * unwanted messages.
+ */
+static struct genl_multicast_group packet_mc_groups[PACKET_N_MC_GROUPS];
+
+static struct genl_family dp_packet_genl_family = {
+ .id = GENL_ID_GENERATE,
+ .hdrsize = sizeof(struct odp_header),
+ .name = ODP_PACKET_FAMILY,
+ .version = 1,
+ .maxattr = ODP_PACKET_ATTR_MAX
+};
+
+static int packet_register_mc_groups(void)
+{
+ int i;
+
+ for (i = 0; i < PACKET_N_MC_GROUPS; i++) {
+ struct genl_multicast_group *group = &packet_mc_groups[i];
+ int error;
+
+ sprintf(group->name, "packet%d", i);
+ error = genl_register_mc_group(&dp_packet_genl_family, group);
+ if (error)
+ return error;
+ }
+ return 0;
+}
+
int dp_upcall(struct datapath *dp, struct sk_buff *skb, const struct dp_upcall_info *upcall_info)
{
struct dp_stats_percpu *stats;
- struct sk_buff_head *queue;
int err;
WARN_ON_ONCE(skb_shared(skb));
- BUG_ON(upcall_info->type >= DP_N_QUEUES);
-
- queue = &dp->queues[upcall_info->type];
- err = -ENOBUFS;
- if (skb_queue_len(queue) >= DP_MAX_QUEUE_LEN)
- goto err_kfree_skb;
forward_ip_summed(skb);
}
}
- err = queue_control_packets(dp, skb, upcall_info);
- wake_up_interruptible(&dp->waitqueue);
- return err;
+ return queue_control_packets(dp, skb, upcall_info);
err_kfree_skb:
kfree_skb(skb);
return 0;
}
-static const struct nla_policy execute_policy[ODP_PACKET_ATTR_MAX + 1] = {
- [ODP_PACKET_ATTR_PACKET] = { .type = NLA_UNSPEC },
- [ODP_PACKET_ATTR_ACTIONS] = { .type = NLA_NESTED },
-};
-
-static int execute_packet(const struct odp_packet __user *uodp_packet)
+static int odp_packet_cmd_execute(struct sk_buff *skb, struct genl_info *info)
{
- struct nlattr *a[ODP_PACKET_ATTR_MAX + 1];
- struct odp_packet *odp_packet;
- struct sk_buff *skb, *packet;
+ struct odp_header *odp_header = info->userhdr;
+ struct nlattr **a = info->attrs;
+ struct sk_buff *packet;
unsigned int actions_len;
struct nlattr *actions;
struct sw_flow_key key;
struct datapath *dp;
struct ethhdr *eth;
bool is_frag;
- u32 len;
int err;
- if (get_user(len, &uodp_packet->len))
- return -EFAULT;
- if (len < sizeof(struct odp_packet))
- return -EINVAL;
-
- skb = alloc_skb(len, GFP_KERNEL);
- if (!skb)
- return -ENOMEM;
-
- err = -EFAULT;
- if (copy_from_user(__skb_put(skb, len), uodp_packet, len))
- goto exit_free_skb;
-
- odp_packet = (struct odp_packet *)skb->data;
- err = -EINVAL;
- if (odp_packet->len != len)
- goto exit_free_skb;
-
- __skb_pull(skb, sizeof(struct odp_packet));
- err = nla_parse(a, ODP_PACKET_ATTR_MAX, (struct nlattr *)skb->data,
- skb->len, execute_policy);
- if (err)
- goto exit_free_skb;
-
err = -EINVAL;
if (!a[ODP_PACKET_ATTR_PACKET] || !a[ODP_PACKET_ATTR_ACTIONS] ||
nla_len(a[ODP_PACKET_ATTR_PACKET]) < ETH_HLEN)
- goto exit_free_skb;
+ goto exit;
actions = nla_data(a[ODP_PACKET_ATTR_ACTIONS]);
actions_len = nla_len(a[ODP_PACKET_ATTR_ACTIONS]);
err = validate_actions(actions, actions_len);
if (err)
- goto exit_free_skb;
+ goto exit;
packet = skb_clone(skb, GFP_KERNEL);
err = -ENOMEM;
if (!packet)
- goto exit_free_skb;
+ goto exit;
packet->data = nla_data(a[ODP_PACKET_ATTR_PACKET]);
packet->len = nla_len(a[ODP_PACKET_ATTR_PACKET]);
err = flow_extract(packet, -1, &key, &is_frag);
if (err)
- goto exit_free_skb;
+ goto exit;
rcu_read_lock();
- dp = get_dp(odp_packet->dp_idx);
+ dp = get_dp(odp_header->dp_idx);
err = -ENODEV;
if (dp)
err = execute_actions(dp, packet, &key, actions, actions_len);
rcu_read_unlock();
-exit_free_skb:
- kfree_skb(skb);
+exit:
return err;
}
+static const struct nla_policy packet_policy[ODP_PACKET_ATTR_MAX + 1] = {
+ [ODP_PACKET_ATTR_PACKET] = { .type = NLA_UNSPEC },
+ [ODP_PACKET_ATTR_ACTIONS] = { .type = NLA_NESTED },
+};
+
+static struct genl_ops dp_packet_genl_ops[] = {
+ { .cmd = ODP_PACKET_CMD_EXECUTE,
+ .flags = GENL_ADMIN_PERM, /* Requires CAP_NET_ADMIN privilege. */
+ .policy = packet_policy,
+ .doit = odp_packet_cmd_execute
+ }
+};
+
static void get_dp_stats(struct datapath *dp, struct odp_stats *stats)
{
int i;
}
}
-static int get_listen_mask(const struct file *f)
-{
- return (long)f->private_data;
-}
-
-static void set_listen_mask(struct file *f, int listen_mask)
-{
- f->private_data = (void*)(long)listen_mask;
-}
-
static const struct nla_policy flow_policy[ODP_FLOW_ATTR_MAX + 1] = {
[ODP_FLOW_ATTR_KEY] = { .type = NLA_NESTED },
[ODP_FLOW_ATTR_ACTIONS] = { .type = NLA_NESTED },
if (dp->sflow_probability)
NLA_PUT_U32(skb, ODP_DP_ATTR_SAMPLING, dp->sflow_probability);
+ nla = nla_nest_start(skb, ODP_DP_ATTR_MCGROUPS);
+ if (!nla)
+ goto nla_put_failure;
+ NLA_PUT_U32(skb, ODP_PACKET_CMD_MISS, packet_mc_group(dp, ODP_PACKET_CMD_MISS));
+ NLA_PUT_U32(skb, ODP_PACKET_CMD_ACTION, packet_mc_group(dp, ODP_PACKET_CMD_ACTION));
+ NLA_PUT_U32(skb, ODP_PACKET_CMD_SAMPLE, packet_mc_group(dp, ODP_PACKET_CMD_SAMPLE));
+ nla_nest_end(skb, nla);
+
if (skb->len > total_len)
goto nla_put_failure;
struct vport *vport;
int dp_idx;
int err;
- int i;
skb = copy_datapath_from_user(uodp_datapath, a);
err = PTR_ERR(skb);
goto err_put_module;
INIT_LIST_HEAD(&dp->port_list);
dp->dp_idx = dp_idx;
- for (i = 0; i < DP_N_QUEUES; i++)
- skb_queue_head_init(&dp->queues[i]);
- init_waitqueue_head(&dp->waitqueue);
/* Initialize kobject for bridge. This will be added as
* /sys/class/net/<devname>/brif later, if sysfs is enabled. */
static long openvswitch_ioctl(struct file *f, unsigned int cmd,
unsigned long argp)
{
- int listeners;
int err;
genl_lock();
-
- /* Handle commands with special locking requirements up front. */
switch (cmd) {
case ODP_DP_NEW:
err = new_datapath((struct odp_datapath __user *)argp);
err = dump_flow((struct odp_flow __user *)argp);
goto exit;
- case ODP_EXECUTE:
- err = execute_packet((struct odp_packet __user *)argp);
- goto exit;
- }
-
- switch (cmd) {
- case ODP_GET_LISTEN_MASK:
- err = put_user(get_listen_mask(f), (int __user *)argp);
- break;
-
- case ODP_SET_LISTEN_MASK:
- err = get_user(listeners, (int __user *)argp);
- if (err)
- break;
- err = -EINVAL;
- if (listeners & ~ODPL_ALL)
- break;
- err = 0;
- set_listen_mask(f, listeners);
- break;
-
default:
err = -ENOIOCTLCMD;
break;
return err;
}
-static int dp_has_packet_of_interest(struct datapath *dp, int listeners)
-{
- int i;
- for (i = 0; i < DP_N_QUEUES; i++) {
- if (listeners & (1 << i) && !skb_queue_empty(&dp->queues[i]))
- return 1;
- }
- return 0;
-}
-
#ifdef CONFIG_COMPAT
static long openvswitch_compat_ioctl(struct file *f, unsigned int cmd, unsigned long argp)
{
case ODP_FLOW_GET:
case ODP_FLOW_SET:
case ODP_FLOW_DUMP:
- case ODP_SET_LISTEN_MASK:
- case ODP_GET_LISTEN_MASK:
- case ODP_EXECUTE:
/* Ioctls that just need their pointer argument extended. */
return openvswitch_ioctl(f, cmd, (unsigned long)compat_ptr(argp));
}
#endif
-static struct sk_buff *openvswitch_try_read(struct file *f, struct datapath *dp)
-{
- int listeners = get_listen_mask(f);
- int i;
+static struct file_operations openvswitch_fops = {
+ .owner = THIS_MODULE,
+ .unlocked_ioctl = openvswitch_ioctl,
+#ifdef CONFIG_COMPAT
+ .compat_ioctl = openvswitch_compat_ioctl,
+#endif
+};
- for (i = 0; i < DP_N_QUEUES; i++) {
- if (listeners & (1 << i)) {
- struct sk_buff *skb = skb_dequeue(&dp->queues[i]);
- if (skb)
- return skb;
- }
- }
+static int major;
- if (f->f_flags & O_NONBLOCK)
- return ERR_PTR(-EAGAIN);
+struct genl_family_and_ops {
+ struct genl_family *family;
+ struct genl_ops *ops;
+ int n_ops;
+ struct genl_multicast_group *group;
+};
- wait_event_interruptible(dp->waitqueue,
- dp_has_packet_of_interest(dp, listeners));
+static const struct genl_family_and_ops dp_genl_families[] = {
+ { &dp_packet_genl_family,
+ dp_packet_genl_ops, ARRAY_SIZE(dp_packet_genl_ops),
+ NULL },
+};
- if (signal_pending(current))
- return ERR_PTR(-ERESTARTSYS);
+static void dp_unregister_genl(int n_families)
+{
+ int i;
- return NULL;
+ for (i = 0; i < n_families; i++) {
+ genl_unregister_family(dp_genl_families[i].family);
+ }
}
-static ssize_t openvswitch_read(struct file *f, char __user *buf,
- size_t nbytes, loff_t *ppos)
+static int dp_register_genl(void)
{
- int dp_idx = iminor(f->f_dentry->d_inode);
- struct datapath *dp;
- struct sk_buff *skb;
- struct iovec iov;
- int retval;
-
- genl_lock();
-
- dp = get_dp(dp_idx);
- retval = -ENODEV;
- if (!dp)
- goto error;
-
- retval = 0;
- if (nbytes == 0 || !get_listen_mask(f))
- goto error;
+ int n_registered;
+ int err;
+ int i;
- do {
- skb = openvswitch_try_read(f, dp);
- } while (!skb);
+ n_registered = 0;
+ for (i = 0; i < ARRAY_SIZE(dp_genl_families); i++) {
+ const struct genl_family_and_ops *f = &dp_genl_families[i];
- genl_unlock();
- if (IS_ERR(skb))
- return PTR_ERR(skb);
+ err = genl_register_family_with_ops(f->family, f->ops,
+ f->n_ops);
+ if (err)
+ goto error;
+ n_registered++;
- iov.iov_base = buf;
- iov.iov_len = min_t(size_t, skb->len, nbytes);
- retval = skb_copy_datagram_iovec(skb, 0, &iov, iov.iov_len);
- if (!retval)
- retval = skb->len;
+ if (f->group) {
+ err = genl_register_mc_group(f->family, f->group);
+ if (err)
+ goto error;
+ }
+ }
- kfree_skb(skb);
- return retval;
+ err = packet_register_mc_groups();
+ if (err)
+ goto error;
+ return 0;
error:
- genl_unlock();
- return retval;
-}
-
-static unsigned int openvswitch_poll(struct file *file, poll_table *wait)
-{
- int dp_idx = iminor(file->f_dentry->d_inode);
- struct datapath *dp;
- unsigned int mask;
-
- genl_lock();
- dp = get_dp(dp_idx);
- if (dp) {
- mask = 0;
- poll_wait(file, &dp->waitqueue, wait);
- if (dp_has_packet_of_interest(dp, get_listen_mask(file)))
- mask |= POLLIN | POLLRDNORM;
- } else {
- mask = POLLIN | POLLRDNORM | POLLHUP;
- }
- genl_unlock();
-
- return mask;
+ dp_unregister_genl(n_registered);
+ return err;
}
-static struct file_operations openvswitch_fops = {
- .owner = THIS_MODULE,
- .read = openvswitch_read,
- .poll = openvswitch_poll,
- .unlocked_ioctl = openvswitch_ioctl,
-#ifdef CONFIG_COMPAT
- .compat_ioctl = openvswitch_compat_ioctl,
-#endif
-};
-
-static int major;
-
static int __init dp_init(void)
{
struct sk_buff *dummy_skb;
if (err < 0)
goto error_unreg_notifier;
+ err = dp_register_genl();
+ if (err < 0)
+ goto error_unreg_chrdev;
+
return 0;
+error_unreg_chrdev:
+ unregister_chrdev(major, "openvswitch");
error_unreg_notifier:
unregister_netdevice_notifier(&dp_device_notifier);
error_vport_exit:
static void dp_cleanup(void)
{
rcu_barrier();
+ dp_unregister_genl(ARRAY_SIZE(dp_genl_families));
unregister_chrdev(major, "openvswitch");
unregister_netdevice_notifier(&dp_device_notifier);
vport_exit();
#include <linux/kernel.h>
#include <linux/mutex.h>
#include <linux/netdevice.h>
-#include <linux/workqueue.h>
#include <linux/seqlock.h>
#include <linux/skbuff.h>
#include <linux/version.h>
#define DP_MAX_PORTS 1024
-#define DP_N_QUEUES 3
-#define DP_MAX_QUEUE_LEN 100
-
/**
* struct dp_stats_percpu - per-cpu packet processing statistics for a given
* datapath.
* @dp_idx: Datapath number (index into the dps[] array in datapath.c).
* @ifobj: Represents /sys/class/net/<devname>/brif. Protected by RTNL.
* @drop_frags: Drop all IP fragments if nonzero.
- * @queues: %DP_N_QUEUES sets of queued packets for userspace to handle.
- * @waitqueue: Waitqueue, for waiting for new packets in @queues.
* @n_flows: Number of flows currently in flow table.
* @table: Current flow table. Protected by genl_lock and RCU.
* @ports: Map from port number to &struct vport. %ODPP_LOCAL port
* to iterate or modify.
* @stats_percpu: Per-CPU datapath statistics.
* @sflow_probability: Number of packets out of UINT_MAX to sample to the
- * %ODPL_SFLOW queue, e.g. (@sflow_probability/UINT_MAX) is the probability of
- * sampling a given packet.
+ * %ODP_PACKET_CMD_SAMPLE multicast group, e.g. (@sflow_probability/UINT_MAX)
+ * is the probability of sampling a given packet.
*
* Context: See the comment on locking at the top of datapath.c for additional
* locking information.
int drop_frags;
- /* Queued data. */
- struct sk_buff_head queues[DP_N_QUEUES];
- wait_queue_head_t waitqueue;
-
/* Flow table. */
struct tbl __rcu *table;
/**
* struct dp_upcall - metadata to include with a packet to send to userspace
- * @type: One of %_ODPL_*_NR.
+ * @cmd: One of %ODP_PACKET_CMD_*.
* @key: Becomes %ODP_PACKET_ATTR_KEY. Must be nonnull.
* @userdata: Becomes %ODP_PACKET_ATTR_USERDATA if nonzero.
* @sample_pool: Becomes %ODP_PACKET_ATTR_SAMPLE_POOL if nonzero.
* @actions_len: Number of bytes in @actions.
*/
struct dp_upcall_info {
- u32 type;
+ u8 cmd;
const struct sw_flow_key *key;
u64 userdata;
u32 sample_pool;
linux-2.6/compat-2.6/include/net/dst.h \
linux-2.6/compat-2.6/include/net/genetlink.h \
linux-2.6/compat-2.6/include/net/ip.h \
+ linux-2.6/compat-2.6/include/net/net_namespace.h \
linux-2.6/compat-2.6/include/net/netlink.h \
linux-2.6/compat-2.6/include/net/protocol.h \
linux-2.6/compat-2.6/include/net/route.h \
return err;
}
#endif
+
+#if LINUX_VERSION_CODE < KERNEL_VERSION(2,6,30)
+/* This function wasn't exported before 2.6.30. Lose! */
+void netlink_set_err(struct sock *ssk, u32 pid, u32 group, int code)
+{
+}
+#endif
#define pr_warn pr_warning
#endif
+#ifndef BUILD_BUG_ON_NOT_POWER_OF_2
+/* Force a compilation error if a constant expression is not a power of 2 */
+#define BUILD_BUG_ON_NOT_POWER_OF_2(n) \
+ BUILD_BUG_ON((n) == 0 || (((n) & ((n) - 1)) != 0))
+#endif
+
#if defined(CONFIG_PREEMPT) && LINUX_VERSION_CODE < KERNEL_VERSION(2,6,21)
#error "CONFIG_PREEMPT is broken before 2.6.21--see commit 4498121ca3, \"[NET]: Handle disabled preemption in gfp_any()\""
#endif
#include <linux/netlink.h>
#include_next <net/genetlink.h>
+#include <net/net_namespace.h>
#include <linux/version.h>
#if LINUX_VERSION_CODE < KERNEL_VERSION(2,6,23)
--- /dev/null
+#ifndef __NET_NET_NAMESPACE_WRAPPER_H
+#define __NET_NET_NAMESPACE_WRAPPER_H 1
+
+#if LINUX_VERSION_CODE >= KERNEL_VERSION(2,6,24)
+/* <net/net_namespace.h> exists, go ahead and include it. */
+#include_next <net/net_namespace.h>
+#endif
+
+#if LINUX_VERSION_CODE >= KERNEL_VERSION(2,6,32)
+#define INIT_NET_GENL_SOCK init_net.genl_sock
+#else
+#define INIT_NET_GENL_SOCK genl_sock
+#endif
+
+#endif /* net/net_namespace.h wrapper */
#define ODP_DP_SET _IOWR('O', 3, struct odp_datapath)
#define ODP_DP_DUMP _IOWR('O', 4, struct odp_datapath)
-#define ODP_GET_LISTEN_MASK _IOW('O', 5, int)
-#define ODP_SET_LISTEN_MASK _IOR('O', 6, int)
-
#define ODP_VPORT_NEW _IOR('O', 7, struct odp_vport)
#define ODP_VPORT_DEL _IOR('O', 8, struct odp_vport)
#define ODP_VPORT_GET _IOWR('O', 9, struct odp_vport)
#define ODP_FLOW_DUMP _IOWR('O', 17, struct odp_flow)
#define ODP_FLOW_FLUSH _IO('O', 19)
-#define ODP_EXECUTE _IOR('O', 18, struct odp_packet)
-
+/**
+ * struct odp_header - header for ODP Generic Netlink messages.
+ * @dp_idx: Number of datapath to which the packet belongs.
+ *
+ * Attributes following the header are specific to a particular ODP Generic
+ * Netlink family, but all of the ODP families use this header.
+ */
+struct odp_header {
+ uint32_t dp_idx;
+};
+\f
/**
* struct odp_datapath - header with basic information about a datapath.
* @dp_idx: Datapath index (-1 to make a request not specific to a datapath).
ODP_DP_ATTR_STATS, /* struct odp_stats */
ODP_DP_ATTR_IPV4_FRAGS, /* 32-bit enum odp_frag_handling */
ODP_DP_ATTR_SAMPLING, /* 32-bit fraction of packets to sample. */
+ ODP_DP_ATTR_MCGROUPS, /* Nested attributes with multicast groups. */
__ODP_DP_ATTR_MAX
};
/* Logical ports. */
#define ODPP_LOCAL ((uint16_t)0)
+\f
+#define ODP_PACKET_FAMILY "odp_packet"
-/* Listening channels. */
-#define _ODPL_MISS_NR 0 /* Packet missed in flow table. */
-#define ODPL_MISS (1 << _ODPL_MISS_NR)
-#define _ODPL_ACTION_NR 1 /* Packet output to ODPP_CONTROLLER. */
-#define ODPL_ACTION (1 << _ODPL_ACTION_NR)
-#define _ODPL_SFLOW_NR 2 /* sFlow samples. */
-#define ODPL_SFLOW (1 << _ODPL_SFLOW_NR)
-#define ODPL_ALL (ODPL_MISS | ODPL_ACTION | ODPL_SFLOW)
+enum odp_packet_cmd {
+ ODP_PACKET_CMD_UNSPEC,
-enum odp_packet_type {
+ /* Kernel-to-user notifications. */
+ ODP_PACKET_CMD_MISS, /* Flow table miss. */
+ ODP_PACKET_CMD_ACTION, /* ODPAT_CONTROLLER action. */
+ ODP_PACKET_CMD_SAMPLE, /* Sampled packet. */
+
+ /* User commands. */
+ ODP_PACKET_CMD_EXECUTE /* Apply actions to a packet. */
+};
+
+/**
+ * enum odp_packet_attr - attributes for %ODP_PACKET_* commands.
+ * @ODP_PACKET_ATTR_PACKET: Present for all notifications. Contains the entire
+ * packet as received, from the start of the Ethernet header onward. For
+ * %ODP_PACKET_CMD_ACTION, %ODP_PACKET_ATTR_PACKET reflects changes made by
+ * actions preceding %ODPAT_CONTROLLER, but %ODP_PACKET_ATTR_KEY is the flow
+ * key extracted from the packet as originally received.
+ * @ODP_PACKET_ATTR_KEY: Present for all notifications. Contains the flow key
+ * extracted from the packet as nested %ODP_KEY_ATTR_* attributes. This allows
+ * userspace to adapt its flow setup strategy by comparing its notion of the
+ * flow key against the kernel's.
+ * @ODP_PACKET_ATTR_USERDATA: Present for an %ODP_PACKET_CMD_ACTION
+ * notification if the %ODPAT_CONTROLLER action's argument was nonzero.
+ * @ODP_PACKET_ATTR_SAMPLE_POOL: Present for %ODP_PACKET_CMD_SAMPLE. Contains
+ * the number of packets processed so far that were candidates for sampling.
+ * @ODP_PACKET_ATTR_ACTIONS: Present for %ODP_PACKET_CMD_SAMPLE. Contains a
+ * copy of the actions applied to the packet, as nested %ODPAT_* attributes.
+ *
+ * These attributes follow the &struct odp_header within the Generic Netlink
+ * payload for %ODP_PACKET_* commands.
+ *
+ * The %ODP_PACKET_ATTR_TYPE, %ODP_PACKET_ATTR_PACKET and %ODP_PACKET_ATTR_KEY
+ * attributes are present for all notifications. For %ODP_PACKET_CMD_ACTION,
+ * the %ODP_PACKET_ATTR_USERDATA attribute is included if it would be nonzero.
+ * For %ODP_PACKET_CMD_SAMPLE, the %ODP_PACKET_ATTR_SAMPLE_POOL and
+ * %ODP_PACKET_ATTR_ACTIONS attributes are included.
+ */
+enum odp_packet_attr {
ODP_PACKET_ATTR_UNSPEC,
- ODP_PACKET_ATTR_TYPE, /* 32-bit enum, one of _ODP_*_NR. */
ODP_PACKET_ATTR_PACKET, /* Packet data. */
ODP_PACKET_ATTR_KEY, /* Nested ODP_KEY_ATTR_* attributes. */
ODP_PACKET_ATTR_USERDATA, /* 64-bit data from ODPAT_CONTROLLER. */
};
#define ODP_PACKET_ATTR_MAX (__ODP_PACKET_ATTR_MAX - 1)
-
-/**
- * struct odp_packet - header for packets passed up between kernel and
- * userspace.
- * @dp_idx: Number of datapath to which the packet belongs.
- * @len: Length of complete message, including this header.
- *
- * The header is followed by a sequence of Netlink attributes. The
- * %ODP_PACKET_ATTR_TYPE, %ODP_PACKET_ATTR_PACKET, and %ODP_PACKET_ATTR_KEY
- * attributes are always present. When @type == %_ODPL_ACTION_NR, the
- * %ODP_PACKET_ATTR_USERDATA attribute is included if it would be nonzero.
- * When @type == %_ODPL_SFLOW_NR, the %ODP_PACKET_ATTR_SAMPLE_POOL and
- * %ODP_PACKET_ATTR_ACTIONS attributes are included.
- *
- * For @type of %_ODPL_ACTION_NR, %ODP_PACKET_ATTR_PACKET reflects changes made
- * by actions preceding %ODPAT_CONTROLLER, but %ODP_PACKET_ATTR_KEY is the flow
- * key extracted from the packet as originally received.
- */
-struct odp_packet {
- uint32_t dp_idx;
- uint32_t len;
-};
-
+\f
enum odp_vport_type {
ODP_VPORT_TYPE_UNSPEC,
ODP_VPORT_TYPE_NETDEV, /* network device */
#include "dpif-provider.h"
#include "netdev.h"
#include "netdev-vport.h"
+#include "netlink-socket.h"
#include "netlink.h"
#include "odp-util.h"
#include "ofpbuf.h"
struct odp_stats stats; /* ODP_DP_ATTR_STATS. */
enum odp_frag_handling ipv4_frags; /* ODP_DP_ATTR_IPV4_FRAGS. */
const uint32_t *sampling; /* ODP_DP_ATTR_SAMPLING. */
+ uint32_t mcgroups[DPIF_N_UC_TYPES]; /* ODP_DP_ATTR_MCGROUPS. */
};
static void dpif_linux_dp_init(struct dpif_linux_dp *);
struct dpif dpif;
int fd;
+ /* Multicast group messages. */
+ struct nl_sock *mc_sock;
+ uint32_t mcgroups[DPIF_N_UC_TYPES];
+ unsigned int listen_mask;
+
/* Used by dpif_linux_get_all_names(). */
char *local_ifname;
int minor;
static struct vlog_rate_limit error_rl = VLOG_RATE_LIMIT_INIT(9999, 5);
-static int do_ioctl(const struct dpif *, int cmd, const void *arg);
-static int open_dpif(const struct dpif_linux_vport *local_vport,
+/* Generic Netlink family numbers for ODP. */
+static int odp_packet_family;
+
+/* Generic Netlink socket. */
+static struct nl_sock *genl_sock;
+
+static int dpif_linux_init(void);
+static int open_dpif(const struct dpif_linux_dp *,
+ const struct dpif_linux_vport *local_vport,
struct dpif **);
static int get_openvswitch_major(void);
static int open_minor(int minor, int *fdp);
{
uint32_t dp_idx;
int major;
+ int err;
+
+ err = dpif_linux_init();
+ if (err) {
+ return err;
+ }
/* Check that the Open vSwitch module is loaded. */
major = get_openvswitch_major();
dpif_linux_open(const struct dpif_class *class OVS_UNUSED, const char *name,
bool create, struct dpif **dpifp)
{
- struct dpif_linux_vport request, reply;
+ struct dpif_linux_vport vport_request, vport;
+ struct dpif_linux_dp dp_request, dp;
struct ofpbuf *buf;
int minor;
int error;
+ error = dpif_linux_init();
+ if (error) {
+ return error;
+ }
+
minor = !strncmp(name, "dp", 2)
&& isdigit((unsigned char)name[2]) ? atoi(name + 2) : -1;
- if (create) {
- struct dpif_linux_dp request, reply;
- struct ofpbuf *buf;
- int error;
- dpif_linux_dp_init(&request);
- request.cmd = ODP_DP_NEW;
- request.dp_idx = minor;
- request.name = name;
- error = dpif_linux_dp_transact(&request, &reply, &buf);
- if (error) {
- return error;
- }
- minor = reply.dp_idx;
- ofpbuf_delete(buf);
- }
-
- dpif_linux_vport_init(&request);
- request.cmd = ODP_VPORT_GET;
- request.port_no = ODPP_LOCAL;
- if (minor >= 0) {
- request.dp_idx = minor;
- } else {
- request.name = name;
+ /* Create or look up datapath. */
+ dpif_linux_dp_init(&dp_request);
+ dp_request.cmd = create ? ODP_DP_NEW : ODP_DP_GET;
+ dp_request.dp_idx = minor;
+ dp_request.name = minor < 0 ? name : NULL;
+ error = dpif_linux_dp_transact(&dp_request, &dp, &buf);
+ if (error) {
+ return error;
}
+ ofpbuf_delete(buf); /* Pointers inside 'dp' are now invalid! */
- error = dpif_linux_vport_transact(&request, &reply, &buf);
+ /* Look up local port. */
+ dpif_linux_vport_init(&vport_request);
+ vport_request.cmd = ODP_VPORT_GET;
+ vport_request.dp_idx = dp.dp_idx;
+ vport_request.port_no = ODPP_LOCAL;
+ vport_request.name = minor < 0 ? name : NULL;
+ error = dpif_linux_vport_transact(&vport_request, &vport, &buf);
if (error) {
return error;
- } else if (reply.port_no != ODPP_LOCAL) {
+ } else if (vport.port_no != ODPP_LOCAL) {
/* This is an Open vSwitch device but not the local port. We
* intentionally support only using the name of the local port as the
* name of a datapath; otherwise, it would be too difficult to
* enumerate all the names of a datapath. */
error = EOPNOTSUPP;
} else {
- error = open_dpif(&reply, dpifp);
+ error = open_dpif(&dp, &vport, dpifp);
}
-
ofpbuf_delete(buf);
return error;
}
static int
-open_dpif(const struct dpif_linux_vport *local_vport, struct dpif **dpifp)
+open_dpif(const struct dpif_linux_dp *dp,
+ const struct dpif_linux_vport *local_vport, struct dpif **dpifp)
{
int dp_idx = local_vport->dp_idx;
struct dpif_linux *dpif;
char *name;
int error;
int fd;
+ int i;
error = open_minor(dp_idx, &fd);
if (error) {
free(name);
dpif->fd = fd;
+ dpif->mc_sock = NULL;
+ for (i = 0; i < DPIF_N_UC_TYPES; i++) {
+ dpif->mcgroups[i] = dp->mcgroups[i];
+ }
+ dpif->listen_mask = 0;
dpif->local_ifname = xstrdup(local_vport->name);
dpif->local_ifindex = local_vport->ifindex;
dpif->minor = dp_idx;
return error;
}
-
struct dpif_linux_flow_state {
struct dpif_linux_flow flow;
struct ofpbuf *buf;
const struct ofpbuf *packet)
{
struct dpif_linux *dpif = dpif_linux_cast(dpif_);
- struct odp_packet *execute;
+ struct odp_header *execute;
struct ofpbuf *buf;
int error;
buf = ofpbuf_new(128 + actions_len + packet->size);
- ofpbuf_reserve(buf, sizeof *execute);
- nl_msg_put_unspec(buf, ODP_PACKET_ATTR_PACKET, packet->data, packet->size);
- nl_msg_put_unspec(buf, ODP_PACKET_ATTR_ACTIONS, actions, actions_len);
+ nl_msg_put_genlmsghdr(buf, 0, odp_packet_family, NLM_F_REQUEST,
+ ODP_PACKET_CMD_EXECUTE, 1);
- execute = ofpbuf_push_uninit(buf, sizeof *execute);
+ execute = ofpbuf_put_uninit(buf, sizeof *execute);
execute->dp_idx = dpif->minor;
- execute->len = buf->size;
- error = do_ioctl(dpif_, ODP_EXECUTE, buf->data);
+ nl_msg_put_unspec(buf, ODP_PACKET_ATTR_PACKET, packet->data, packet->size);
+ nl_msg_put_unspec(buf, ODP_PACKET_ATTR_ACTIONS, actions, actions_len);
+ error = nl_sock_transact(genl_sock, buf, NULL);
ofpbuf_delete(buf);
return error;
}
static int
dpif_linux_recv_get_mask(const struct dpif *dpif_, int *listen_mask)
{
- return do_ioctl(dpif_, ODP_GET_LISTEN_MASK, listen_mask);
+ struct dpif_linux *dpif = dpif_linux_cast(dpif_);
+ *listen_mask = dpif->listen_mask;
+ return 0;
}
static int
dpif_linux_recv_set_mask(struct dpif *dpif_, int listen_mask)
{
- return do_ioctl(dpif_, ODP_SET_LISTEN_MASK, &listen_mask);
+ struct dpif_linux *dpif = dpif_linux_cast(dpif_);
+ int error;
+ int i;
+
+ if (listen_mask == dpif->listen_mask) {
+ return 0;
+ } else if (!listen_mask) {
+ nl_sock_destroy(dpif->mc_sock);
+ dpif->mc_sock = NULL;
+ dpif->listen_mask = 0;
+ return 0;
+ } else if (!dpif->mc_sock) {
+ error = nl_sock_create(NETLINK_GENERIC, &dpif->mc_sock);
+ if (error) {
+ return error;
+ }
+ }
+
+ /* Unsubscribe from old groups. */
+ for (i = 0; i < DPIF_N_UC_TYPES; i++) {
+ if (dpif->listen_mask & (1u << i)) {
+ nl_sock_leave_mcgroup(dpif->mc_sock, dpif->mcgroups[i]);
+ }
+ }
+
+ /* Update listen_mask. */
+ dpif->listen_mask = listen_mask;
+
+ /* Subscribe to new groups. */
+ error = 0;
+ for (i = 0; i < DPIF_N_UC_TYPES; i++) {
+ if (dpif->listen_mask & (1u << i)) {
+ int retval;
+
+ retval = nl_sock_join_mcgroup(dpif->mc_sock, dpif->mcgroups[i]);
+ if (retval) {
+ error = retval;
+ }
+ }
+ }
+ return error;
}
static int
}
static int
-parse_odp_packet(struct ofpbuf *buf, struct dpif_upcall *upcall)
+parse_odp_packet(struct ofpbuf *buf, struct dpif_upcall *upcall,
+ uint32_t *dp_idx)
{
static const struct nl_policy odp_packet_policy[] = {
/* Always present. */
- [ODP_PACKET_ATTR_TYPE] = { .type = NL_A_U32 },
[ODP_PACKET_ATTR_PACKET] = { .type = NL_A_UNSPEC,
.min_len = ETH_HEADER_LEN },
[ODP_PACKET_ATTR_KEY] = { .type = NL_A_NESTED },
- /* _ODPL_ACTION_NR only. */
+ /* ODP_PACKET_CMD_ACTION only. */
[ODP_PACKET_ATTR_USERDATA] = { .type = NL_A_U64, .optional = true },
- /* _ODPL_SFLOW_NR only. */
+ /* ODP_PACKET_CMD_SAMPLE only. */
[ODP_PACKET_ATTR_SAMPLE_POOL] = { .type = NL_A_U32, .optional = true },
[ODP_PACKET_ATTR_ACTIONS] = { .type = NL_A_NESTED, .optional = true },
};
- struct odp_packet *odp_packet = buf->data;
+ struct odp_header *odp_header;
struct nlattr *a[ARRAY_SIZE(odp_packet_policy)];
- uint32_t type;
-
- if (!nl_policy_parse(buf, sizeof *odp_packet, odp_packet_policy,
- a, ARRAY_SIZE(odp_packet_policy))) {
+ struct nlmsghdr *nlmsg;
+ struct genlmsghdr *genl;
+ struct ofpbuf b;
+
+ ofpbuf_use_const(&b, buf->data, buf->size);
+
+ nlmsg = ofpbuf_try_pull(&b, sizeof *nlmsg);
+ genl = ofpbuf_try_pull(&b, sizeof *genl);
+ odp_header = ofpbuf_try_pull(&b, sizeof *odp_header);
+ if (!nlmsg || !genl || !odp_header
+ || !nl_policy_parse(&b, 0, odp_packet_policy, a,
+ ARRAY_SIZE(odp_packet_policy))) {
return EINVAL;
}
memset(upcall, 0, sizeof *upcall);
- type = nl_attr_get_u32(a[ODP_PACKET_ATTR_TYPE]);
- upcall->type = (type == _ODPL_MISS_NR ? DPIF_UC_MISS
- : type == _ODPL_ACTION_NR ? DPIF_UC_ACTION
- : type == _ODPL_SFLOW_NR ? DPIF_UC_SAMPLE
+ upcall->type = (genl->cmd == ODP_PACKET_CMD_MISS ? DPIF_UC_MISS
+ : genl->cmd == ODP_PACKET_CMD_ACTION ? DPIF_UC_ACTION
+ : genl->cmd == ODP_PACKET_CMD_SAMPLE ? DPIF_UC_SAMPLE
: -1);
upcall->packet = buf;
upcall->actions_len = nl_attr_get_size(a[ODP_PACKET_ATTR_ACTIONS]);
}
+ *dp_idx = odp_header->dp_idx;
+
return 0;
}
{
struct dpif_linux *dpif = dpif_linux_cast(dpif_);
struct ofpbuf *buf;
- int retval;
int error;
+ int i;
- buf = ofpbuf_new(65536);
- retval = read(dpif->fd, ofpbuf_tail(buf), ofpbuf_tailroom(buf));
- if (retval < 0) {
- error = errno;
- if (error != EAGAIN) {
- VLOG_WARN_RL(&error_rl, "%s: read failed: %s",
- dpif_name(dpif_), strerror(error));
+ if (!dpif->mc_sock) {
+ return EAGAIN;
+ }
+
+ for (i = 0; i < 50; i++) {
+ uint32_t dp_idx;
+
+ error = nl_sock_recv(dpif->mc_sock, &buf, false);
+ if (error) {
+ return error;
}
- } else if (retval >= sizeof(struct odp_packet)) {
- struct odp_packet *odp_packet = buf->data;
- buf->size += retval;
- if (odp_packet->len <= retval) {
- error = parse_odp_packet(buf, upcall);
- } else {
- VLOG_WARN_RL(&error_rl, "%s: discarding message truncated "
- "from %"PRIu32" bytes to %d",
- dpif_name(dpif_), odp_packet->len, retval);
- error = ERANGE;
+ error = parse_odp_packet(buf, upcall, &dp_idx);
+ if (!error
+ && dp_idx == dpif->minor
+ && dpif->listen_mask & (1u << upcall->type)) {
+ return 0;
}
- } else if (!retval) {
- VLOG_WARN_RL(&error_rl, "%s: unexpected end of file", dpif_name(dpif_));
- error = EPROTO;
- } else {
- VLOG_WARN_RL(&error_rl, "%s: discarding too-short message (%d bytes)",
- dpif_name(dpif_), retval);
- error = ERANGE;
- }
- if (error) {
ofpbuf_delete(buf);
+ if (error) {
+ return error;
+ }
}
- return error;
+
+ return EAGAIN;
}
static void
dpif_linux_recv_wait(struct dpif *dpif_)
{
struct dpif_linux *dpif = dpif_linux_cast(dpif_);
- poll_fd_wait(dpif->fd, POLLIN);
+ if (dpif->mc_sock) {
+ nl_sock_wait(dpif->mc_sock, POLLIN);
+ }
}
static void
dpif_linux_recv_purge(struct dpif *dpif_)
{
struct dpif_linux *dpif = dpif_linux_cast(dpif_);
- int i;
-
- /* This is somewhat bogus because it assumes that the following macros have
- * fixed values, but it's going to go away later. */
-#define DP_N_QUEUES 3
-#define DP_MAX_QUEUE_LEN 100
- for (i = 0; i < DP_N_QUEUES * DP_MAX_QUEUE_LEN; i++) {
- /* Reading even 1 byte discards a whole datagram and saves time. */
- char buffer;
- if (read(dpif->fd, &buffer, 1) != 1) {
- break;
- }
+ if (dpif->mc_sock) {
+ nl_sock_drain(dpif->mc_sock);
}
}
const struct dpif_class dpif_linux_class = {
"system",
- NULL,
- NULL,
+ NULL, /* run */
+ NULL, /* wait */
dpif_linux_enumerate,
dpif_linux_open,
dpif_linux_close,
dpif_linux_recv_purge,
};
\f
-static int get_openvswitch_major(void);
static int get_major(const char *target);
static int
-do_ioctl(const struct dpif *dpif_, int cmd, const void *arg)
+dpif_linux_init(void)
{
- struct dpif_linux *dpif = dpif_linux_cast(dpif_);
- return ioctl(dpif->fd, cmd, arg) ? errno : 0;
+ static int error = -1;
+
+ if (error < 0) {
+ error = nl_lookup_genl_family(ODP_PACKET_FAMILY, &odp_packet_family);
+ if (!error) {
+ error = nl_sock_create(NETLINK_GENERIC, &genl_sock);
+ }
+ }
+
+ return error;
}
bool
.optional = true },
[ODP_DP_ATTR_IPV4_FRAGS] = { .type = NL_A_U32, .optional = true },
[ODP_DP_ATTR_SAMPLING] = { .type = NL_A_U32, .optional = true },
+ [ODP_DP_ATTR_MCGROUPS] = { .type = NL_A_NESTED, .optional = true },
};
struct odp_datapath *odp_dp;
if (a[ODP_DP_ATTR_SAMPLING]) {
dp->sampling = nl_attr_get(a[ODP_DP_ATTR_SAMPLING]);
}
+
+ if (a[ODP_DP_ATTR_MCGROUPS]) {
+ static const struct nl_policy odp_mcgroup_policy[] = {
+ [ODP_PACKET_CMD_MISS] = { .type = NL_A_U32, .optional = true },
+ [ODP_PACKET_CMD_ACTION] = { .type = NL_A_U32, .optional = true },
+ [ODP_PACKET_CMD_SAMPLE] = { .type = NL_A_U32, .optional = true },
+ };
+
+ struct nlattr *mcgroups[ARRAY_SIZE(odp_mcgroup_policy)];
+
+ if (!nl_parse_nested(a[ODP_DP_ATTR_MCGROUPS], odp_mcgroup_policy,
+ mcgroups, ARRAY_SIZE(odp_mcgroup_policy))) {
+ return EINVAL;
+ }
+
+ if (mcgroups[ODP_PACKET_CMD_MISS]) {
+ dp->mcgroups[DPIF_UC_MISS]
+ = nl_attr_get_u32(mcgroups[ODP_PACKET_CMD_MISS]);
+ }
+ if (mcgroups[ODP_PACKET_CMD_ACTION]) {
+ dp->mcgroups[DPIF_UC_ACTION]
+ = nl_attr_get_u32(mcgroups[ODP_PACKET_CMD_ACTION]);
+ }
+ if (mcgroups[ODP_PACKET_CMD_SAMPLE]) {
+ dp->mcgroups[DPIF_UC_SAMPLE]
+ = nl_attr_get_u32(mcgroups[ODP_PACKET_CMD_SAMPLE]);
+ }
+ }
+
return 0;
}
enum dpif_upcall_type {
DPIF_UC_MISS, /* Miss in flow table. */
DPIF_UC_ACTION, /* ODPAT_CONTROLLER action. */
- DPIF_UC_SAMPLE /* Packet sampling. */
+ DPIF_UC_SAMPLE, /* Packet sampling. */
+ DPIF_N_UC_TYPES
};
/* A packet passed up from the datapath to userspace.
handle_miss_upcall(p, upcall);
break;
+ case DPIF_N_UC_TYPES:
default:
VLOG_WARN_RL(&rl, "upcall has unexpected type %"PRIu32, upcall->type);
break;