[ovs-dev] [PATCH v8] datapath: Add support for VXLAN tunnels to Open vSwitch

Kyle Mestery kmestery at cisco.com
Wed Dec 5 21:06:46 UTC 2012


Add support for VXLAN tunnels to Open vSwitch. Add support
for setting the destination UDP port on a per-port basis.
This is done by adding a "dst_port" parameter to the port
configuration. This is only applicable currently to VXLAN
tunnels.

Please note this currently does not implement any sort of multicast
learning. With this patch, VXLAN tunnels must be configured similar
to GRE tunnels (e.g. point to point). A subsequent patch will implement
a VXLAN control plane in userspace to handle multicast learning.

This patch set is based on one posted by Ben Pfaff on Oct. 12, 2011
to the ovs-dev mailing list:

http://openvswitch.org/pipermail/dev/2011-October/012051.html

The patch has been maintained, updated, and freshened by me and a
version of it is available at the following github repository:

https://github.com/mestery/ovs-vxlan/tree/vxlan

I've tested this patch with multiple VXLAN tunnels between hosts
using different UDP port numbers. Performance is on par (though
slightly faster) than comparable GRE tunnels.

See the following IETF draft for additional information about VXLAN:
http://tools.ietf.org/html/draft-mahalingam-dutt-dcops-vxlan-02

Signed-off-by: Kyle Mestery <kmestery at cisco.com>
---
Note: v8 addresses additional comments from Jesse and fixes some
sparse warnings.

Note: v7 addresses comments from Jesse.

Note: v6 of this patch addresses additional comments from
Jesse. Specifically, I've removed the hash table of
port/socket mappings and moved to a linked list. I've also
cleaned up the code around this quite a bit.

Note: v5 of this patch addresses comments from Jesse
and Chris.

Note: v4 of this patch removes VXLAN over IPSEC support,
per an offline conversation with Jesse.
---
 NEWS                          |   3 +
 README                        |   2 +-
 datapath/Modules.mk           |   3 +-
 datapath/linux/.gitignore     |   1 +
 datapath/tunnel.c             |   8 +
 datapath/tunnel.h             |   2 +
 datapath/vport-vxlan.c        | 392 ++++++++++++++++++++++++++++++++++++++++++
 datapath/vport.c              |   1 +
 datapath/vport.h              |   1 +
 include/linux/openvswitch.h   |   1 +
 include/openflow/nicira-ext.h |   8 +-
 include/openvswitch/tunnel.h  |   1 +
 lib/netdev-vport.c            |  29 ++++
 vswitchd/vswitch.xml          |  39 +++--
 14 files changed, 473 insertions(+), 18 deletions(-)
 create mode 100644 datapath/vport-vxlan.c

diff --git a/NEWS b/NEWS
index bb80beb..d81126e 100644
--- a/NEWS
+++ b/NEWS
@@ -1,5 +1,7 @@
 post-v1.9.0
 --------------------
+    - New support for the VXLAN tunnel protocol (see the IETF draft here:
+      http://tools.ietf.org/html/draft-mahalingam-dutt-dcops-vxlan-02).
 
 
 v1.9.0 - xx xxx xxxx
@@ -68,6 +70,7 @@ v1.9.0 - xx xxx xxxx
     - The data in the RARP packets can now be matched in the same way as the
       data in ARP packets.
 
+
 v1.8.0 - xx xxx xxxx
 ------------------------
     - New FAQ.  Please send updates and additions!
diff --git a/README b/README
index b0e6d05..f5a9db8 100644
--- a/README
+++ b/README
@@ -24,7 +24,7 @@ vSwitch supports the following features:
     * NIC bonding with or without LACP on upstream switch
     * NetFlow, sFlow(R), and mirroring for increased visibility
     * QoS (Quality of Service) configuration, plus policing
-    * GRE, GRE over IPSEC, and CAPWAP tunneling
+    * GRE, GRE over IPSEC, CAPWAP, and VXLAN tunneling
     * 802.1ag connectivity fault management
     * OpenFlow 1.0 plus numerous extensions
     * Transactional configuration database with C and Python bindings
diff --git a/datapath/Modules.mk b/datapath/Modules.mk
index 24c1075..24e6559 100644
--- a/datapath/Modules.mk
+++ b/datapath/Modules.mk
@@ -26,7 +26,8 @@ openvswitch_sources = \
 	vport-gre.c \
 	vport-internal_dev.c \
 	vport-netdev.c \
-	vport-patch.c
+	vport-patch.c \
+	vport-vxlan.c
 
 openvswitch_headers = \
 	checksum.h \
diff --git a/datapath/linux/.gitignore b/datapath/linux/.gitignore
index d6de397..301c0a0 100644
--- a/datapath/linux/.gitignore
+++ b/datapath/linux/.gitignore
@@ -39,5 +39,6 @@
 /vport-internal_dev.c
 /vport-netdev.c
 /vport-patch.c
+/vport-vxlan.c
 /vport.c
 /workqueue.c
diff --git a/datapath/tunnel.c b/datapath/tunnel.c
index fb4854a..1db60d2 100644
--- a/datapath/tunnel.c
+++ b/datapath/tunnel.c
@@ -1042,6 +1042,7 @@ static const struct nla_policy tnl_policy[OVS_TUNNEL_ATTR_MAX + 1] = {
 	[OVS_TUNNEL_ATTR_IN_KEY]   = { .type = NLA_U64 },
 	[OVS_TUNNEL_ATTR_TOS]      = { .type = NLA_U8 },
 	[OVS_TUNNEL_ATTR_TTL]      = { .type = NLA_U8 },
+	[OVS_TUNNEL_ATTR_DST_PORT] = { .type = NLA_U16 },
 };
 
 /* Sets OVS_TUNNEL_ATTR_* fields in 'mutable', which must initially be
@@ -1087,6 +1088,10 @@ static int tnl_set_config(struct net *net, struct nlattr *options,
 	if (a[OVS_TUNNEL_ATTR_TTL])
 		mutable->ttl = nla_get_u8(a[OVS_TUNNEL_ATTR_TTL]);
 
+	if (a[OVS_TUNNEL_ATTR_DST_PORT])
+		mutable->dst_port =
+			htons(nla_get_u16(a[OVS_TUNNEL_ATTR_DST_PORT]));
+
 	if (!a[OVS_TUNNEL_ATTR_IN_KEY]) {
 		mutable->key.tunnel_type |= TNL_T_KEY_MATCH;
 		mutable->flags |= TNL_F_IN_KEY_MATCH;
@@ -1242,6 +1247,9 @@ int ovs_tnl_get_options(const struct vport *vport, struct sk_buff *skb)
 		goto nla_put_failure;
 	if (mutable->ttl && nla_put_u8(skb, OVS_TUNNEL_ATTR_TTL, mutable->ttl))
 		goto nla_put_failure;
+	if (mutable->dst_port && nla_put_u16(skb, OVS_TUNNEL_ATTR_DST_PORT,
+					     ntohs(mutable->dst_port)))
+		goto nla_put_failure;
 
 	return 0;
 
diff --git a/datapath/tunnel.h b/datapath/tunnel.h
index c268057..b7de7a9 100644
--- a/datapath/tunnel.h
+++ b/datapath/tunnel.h
@@ -42,6 +42,7 @@
 #define TNL_T_PROTO_GRE		0
 #define TNL_T_PROTO_GRE64	1
 #define TNL_T_PROTO_CAPWAP	2
+#define TNL_T_PROTO_VXLAN	3
 
 /* These flags are only needed when calling tnl_find_port(). */
 #define TNL_T_KEY_EXACT		(1 << 10)
@@ -116,6 +117,7 @@ struct tnl_mutable_config {
 	u32	flags;
 	u8	tos;
 	u8	ttl;
+	__be16	dst_port;
 
 	/* Multicast configuration. */
 	int	mlink;
diff --git a/datapath/vport-vxlan.c b/datapath/vport-vxlan.c
new file mode 100644
index 0000000..0b727bb
--- /dev/null
+++ b/datapath/vport-vxlan.c
@@ -0,0 +1,392 @@
+/*
+ * Copyright (c) 2011 Nicira, Inc.
+ * Copyright (c) 2012 Cisco Systems, Inc.
+ *
+ * This program is free software; you can redistribute it and/or
+ * modify it under the terms of version 2 of the GNU General Public
+ * License as published by the Free Software Foundation.
+ *
+ * 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 pr_fmt(fmt) KBUILD_MODNAME ": " fmt
+
+#include <linux/version.h>
+#if LINUX_VERSION_CODE >= KERNEL_VERSION(2,6,26)
+
+#include <linux/in.h>
+#include <linux/ip.h>
+#include <linux/list.h>
+#include <linux/net.h>
+#include <linux/udp.h>
+
+#include <net/icmp.h>
+#include <net/ip.h>
+#include <net/udp.h>
+
+#include "datapath.h"
+#include "tunnel.h"
+#include "vport.h"
+#include "vport-generic.h"
+
+#define VXLAN_FLAGS 0x08000000  /* struct vxlanhdr.vx_flags required value. */
+
+/**
+ * struct vxlanhdr - VXLAN header
+ * @vx_flags: Must have the exact value %VXLAN_FLAGS.
+ * @vx_vni: VXLAN Network Identifier (VNI) in top 24 bits, low 8 bits zeroed.
+ */
+struct vxlanhdr {
+	__be32 vx_flags;
+	__be32 vx_vni;
+};
+
+#define VXLAN_HLEN (sizeof(struct udphdr) + sizeof(struct vxlanhdr))
+
+static inline int vxlan_hdr_len(const struct tnl_mutable_config *mutable,
+				const struct ovs_key_ipv4_tunnel *tun_key)
+{
+	return VXLAN_HLEN;
+}
+
+/**
+ * struct vxlan_port - Keeps track of open UDP ports
+ * @list: list element.
+ * @port: The UDP port number in network byte order.
+ * @net: The net namespace this port is assosciated with.
+ * @socket: The socket created for this port number.
+ * @count: How many ports are using this socket/port.
+ */
+struct vxlan_port {
+	struct list_head list;
+	__be16 port;
+	struct socket *vxlan_rcv_socket;
+	int count;
+};
+
+static LIST_HEAD(vxlan_ports);
+
+static struct vxlan_port *vxlan_port_exists(struct net *net, __be16 port)
+{
+	struct vxlan_port *vxlan_port;
+
+	list_for_each_entry(vxlan_port, &vxlan_ports, list) {
+		if (vxlan_port->port == port &&
+			net_eq(sock_net(vxlan_port->vxlan_rcv_socket->sk), net))
+			return vxlan_port;
+	}
+
+	return NULL;
+}
+
+static inline struct vxlanhdr *vxlan_hdr(const struct sk_buff *skb)
+{
+	return (struct vxlanhdr *)(udp_hdr(skb) + 1);
+}
+
+/* Compute source port for outgoing packet.
+ * Currently we use the flow hash.
+ */
+static u16 get_src_port(struct sk_buff *skb)
+{
+	int low;
+	int high;
+	unsigned int range;
+	u32 hash = OVS_CB(skb)->flow->hash;
+
+        inet_get_local_port_range(&low, &high);
+        range = (high - low) + 1;
+	return (((u64) hash * range) >> 32) + low;
+}
+
+static struct sk_buff *vxlan_build_header(const struct vport *vport,
+					  const struct tnl_mutable_config *mutable,
+					  struct dst_entry *dst,
+					  struct sk_buff *skb,
+					  int tunnel_hlen)
+{
+	struct udphdr *udph = udp_hdr(skb);
+	struct vxlanhdr *vxh = (struct vxlanhdr *)(udph + 1);
+	const struct ovs_key_ipv4_tunnel *tun_key = OVS_CB(skb)->tun_key;
+	__be64 out_key;
+	u32 flags;
+
+	tnl_get_param(mutable, tun_key, &flags, &out_key);
+
+	udph->dest = mutable->dst_port;
+	udph->source = htons(get_src_port(skb));
+	udph->check = 0;
+	udph->len = htons(skb->len - skb_transport_offset(skb));
+
+	vxh->vx_flags = htonl(VXLAN_FLAGS);
+	vxh->vx_vni = htonl(be64_to_cpu(out_key) << 8);
+
+	/*
+	 * Allow our local IP stack to fragment the outer packet even if the
+	 * DF bit is set as a last resort.  We also need to force selection of
+	 * an IP ID here because Linux will otherwise leave it at 0 if the
+	 * packet originally had DF set.
+	 */
+	skb->local_df = 1;
+	__ip_select_ident(ip_hdr(skb), dst, 0);
+
+	return skb;
+}
+
+/* Called with rcu_read_lock and BH disabled. */
+static int vxlan_rcv(struct sock *sk, struct sk_buff *skb)
+{
+	struct vport *vport;
+	struct vxlanhdr *vxh;
+	const struct tnl_mutable_config *mutable;
+	struct iphdr *iph;
+	struct ovs_key_ipv4_tunnel tun_key;
+	__be64 key;
+	u32 tunnel_flags = 0;
+
+	if (unlikely(!pskb_may_pull(skb, VXLAN_HLEN + ETH_HLEN)))
+		goto error;
+
+	vxh = vxlan_hdr(skb);
+	if (unlikely(vxh->vx_flags != htonl(VXLAN_FLAGS) ||
+		     vxh->vx_vni & htonl(0xff)))
+		goto error;
+
+	__skb_pull(skb, VXLAN_HLEN);
+	skb_postpull_rcsum(skb, skb_transport_header(skb), VXLAN_HLEN + ETH_HLEN);
+
+	key = cpu_to_be64(ntohl(vxh->vx_vni) >> 8);
+
+	iph = ip_hdr(skb);
+	vport = ovs_tnl_find_port(dev_net(skb->dev), iph->daddr, iph->saddr,
+		key, TNL_T_PROTO_VXLAN, &mutable);
+	if (unlikely(!vport)) {
+		icmp_send(skb, ICMP_DEST_UNREACH, ICMP_PORT_UNREACH, 0);
+		goto error;
+	}
+
+	if (mutable->flags & TNL_F_IN_KEY_MATCH || !mutable->key.daddr)
+		tunnel_flags = OVS_TNL_F_KEY;
+	else
+		key = 0;
+
+	/* Save outer tunnel values */
+	tnl_tun_key_init(&tun_key, iph, key, tunnel_flags);
+	OVS_CB(skb)->tun_key = &tun_key;
+
+	ovs_tnl_rcv(vport, skb);
+	goto out;
+
+error:
+	kfree_skb(skb);
+out:
+	return 0;
+}
+
+/* Random value.  Irrelevant as long as it's not 0 since we set the handler. */
+#define UDP_ENCAP_VXLAN 1
+static int vxlan_socket_init(struct vxlan_port *vxlan_port, struct net *net)
+{
+	int err;
+	struct sockaddr_in sin;
+
+	err = sock_create_kern(AF_INET, SOCK_DGRAM, 0,
+			       &vxlan_port->vxlan_rcv_socket);
+	if (err)
+		goto error;
+
+	/* release net ref. */
+	sk_change_net(vxlan_port->vxlan_rcv_socket->sk, net);
+
+	sin.sin_family = AF_INET;
+	sin.sin_addr.s_addr = htonl(INADDR_ANY);
+	sin.sin_port = vxlan_port->port;
+
+	err = kernel_bind(vxlan_port->vxlan_rcv_socket, (struct sockaddr *)&sin,
+			  sizeof(struct sockaddr_in));
+	if (err)
+		goto error_sock;
+
+	udp_sk(vxlan_port->vxlan_rcv_socket->sk)->encap_type = UDP_ENCAP_VXLAN;
+	udp_sk(vxlan_port->vxlan_rcv_socket->sk)->encap_rcv = vxlan_rcv;
+
+	udp_encap_enable();
+
+	return 0;
+
+error_sock:
+	sock_release(vxlan_port->vxlan_rcv_socket);
+error:
+	pr_warn("cannot register vxlan protocol handler\n");
+	return err;
+}
+
+static void vxlan_tunnel_release(struct vxlan_port *vxlan_port)
+{
+	vxlan_port->count--;
+
+	if (vxlan_port->count == 0) {
+		/* Release old socket */
+		sock_release(vxlan_port->vxlan_rcv_socket);
+		list_del(&vxlan_port->list);
+		kfree(vxlan_port);
+	}
+}
+static int vxlan_tunnel_setup(struct net *net, const char *name,
+			      struct nlattr *options,
+			      struct vxlan_port **vxport)
+{
+	struct nlattr *a;
+	int err;
+	u16 dst_port;
+	struct vxlan_port *vxlan_port = NULL;
+
+	*vxport = NULL;
+
+	if (!options) {
+		err = -EINVAL;
+		goto out;
+	}
+
+	a = nla_find_nested(options, OVS_TUNNEL_ATTR_DST_PORT);
+	if (a && nla_len(a) == sizeof(u16)) {
+		dst_port = nla_get_u16(a);
+	} else {
+		/* Require destination port from userspace. */
+		err = -EINVAL;
+		goto out;
+	}
+
+	/* Verify if we already have a socket created for this port */
+	vxlan_port = vxlan_port_exists(net, htons(dst_port));
+	if (vxlan_port) {
+		vxlan_port->count++;
+		err = 0;
+		goto out;
+	}
+
+	/* Add a new socket for this port */
+	vxlan_port = kzalloc(sizeof(struct vxlan_port), GFP_KERNEL);
+	if (!vxlan_port) {
+		err = -ENOMEM;
+		goto error;
+	}
+
+	vxlan_port->port = htons(dst_port);
+	vxlan_port->count = 1;
+	list_add_tail(&vxlan_port->list, &vxlan_ports);
+
+	err = vxlan_socket_init(vxlan_port, net);
+	if (err)
+		goto error;
+
+	*vxport = vxlan_port;
+	goto out;
+
+error:
+	if (vxlan_port) {
+		list_del(&vxlan_port->list);
+		kfree(vxlan_port);
+	}
+out:
+	return err;
+}
+
+static int vxlan_set_options(struct vport *vport, struct nlattr *options)
+{
+	int err;
+	const char *vname = vport->ops->get_name(vport);
+	struct net *net = ovs_dp_get_net(vport->dp);
+	struct tnl_vport *tnl_vport = tnl_vport_priv(vport);
+	struct tnl_mutable_config *config;
+	struct vxlan_port *old_port = NULL;
+	struct vxlan_port *vxlan_port = NULL;
+
+	config = rtnl_dereference(tnl_vport->mutable);
+
+	old_port = vxlan_port_exists(net, config->dst_port);
+
+	err = vxlan_tunnel_setup(net, vname, options, &vxlan_port);
+	if (err)
+		goto out;
+
+	err = ovs_tnl_set_options(vport, options);
+
+	if (err)
+		vxlan_tunnel_release(vxlan_port);
+	else {
+		/* Release old socket */
+		vxlan_tunnel_release(old_port);
+	}
+out:
+	return err;
+}
+
+static const struct tnl_ops ovs_vxlan_tnl_ops = {
+	.tunnel_type	= TNL_T_PROTO_VXLAN,
+	.ipproto	= IPPROTO_UDP,
+	.hdr_len	= vxlan_hdr_len,
+	.build_header	= vxlan_build_header,
+};
+
+static void vxlan_tnl_destroy(struct vport *vport)
+{
+	struct vxlan_port *vxlan_port;
+	struct tnl_vport *tnl_vport = tnl_vport_priv(vport);
+	struct tnl_mutable_config *config;
+
+	config = rtnl_dereference(tnl_vport->mutable);
+
+	vxlan_port = vxlan_port_exists(ovs_dp_get_net(vport->dp),
+					 config->dst_port);
+
+	vxlan_tunnel_release(vxlan_port);
+
+	ovs_tnl_destroy(vport);
+}
+
+static struct vport *vxlan_tnl_create(const struct vport_parms *parms)
+{
+	int err;
+	struct vport *vport;
+	struct vxlan_port *vxlan_port = NULL;
+
+	err = vxlan_tunnel_setup(ovs_dp_get_net(parms->dp), parms->name,
+						parms->options, &vxlan_port);
+	if (err)
+		return ERR_PTR(err);
+
+	vport = ovs_tnl_create(parms, &ovs_vxlan_vport_ops, &ovs_vxlan_tnl_ops);
+
+	if (IS_ERR(vport))
+		vxlan_tunnel_release(vxlan_port);
+
+	return vport;
+}
+
+const struct vport_ops ovs_vxlan_vport_ops = {
+	.type		= OVS_VPORT_TYPE_VXLAN,
+	.flags		= VPORT_F_TUN_ID,
+	.create		= vxlan_tnl_create,
+	.destroy	= vxlan_tnl_destroy,
+	.set_addr	= ovs_tnl_set_addr,
+	.get_name	= ovs_tnl_get_name,
+	.get_addr	= ovs_tnl_get_addr,
+	.get_options	= ovs_tnl_get_options,
+	.set_options	= vxlan_set_options,
+	.get_dev_flags	= ovs_vport_gen_get_dev_flags,
+	.is_running	= ovs_vport_gen_is_running,
+	.get_operstate	= ovs_vport_gen_get_operstate,
+	.send		= ovs_tnl_send,
+};
+#else
+#warning VXLAN tunneling will not be available on kernels before 2.6.26
+#endif /* Linux kernel < 2.6.26 */
diff --git a/datapath/vport.c b/datapath/vport.c
index 4934ac1..a1c7542 100644
--- a/datapath/vport.c
+++ b/datapath/vport.c
@@ -45,6 +45,7 @@ static const struct vport_ops *base_vport_ops_list[] = {
 	&ovs_gre64_vport_ops,
 #if LINUX_VERSION_CODE >= KERNEL_VERSION(2,6,26)
 	&ovs_capwap_vport_ops,
+	&ovs_vxlan_vport_ops,
 #endif
 };
 
diff --git a/datapath/vport.h b/datapath/vport.h
index de65000..1a2e9ef 100644
--- a/datapath/vport.h
+++ b/datapath/vport.h
@@ -257,5 +257,6 @@ extern const struct vport_ops ovs_gre_vport_ops;
 extern const struct vport_ops ovs_gre_ft_vport_ops;
 extern const struct vport_ops ovs_gre64_vport_ops;
 extern const struct vport_ops ovs_capwap_vport_ops;
+extern const struct vport_ops ovs_vxlan_vport_ops;
 
 #endif /* vport.h */
diff --git a/include/linux/openvswitch.h b/include/linux/openvswitch.h
index e7d4b49..56b9785 100644
--- a/include/linux/openvswitch.h
+++ b/include/linux/openvswitch.h
@@ -183,6 +183,7 @@ enum ovs_vport_type {
 	OVS_VPORT_TYPE_NETDEV,   /* network device */
 	OVS_VPORT_TYPE_INTERNAL, /* network device implemented by datapath */
 	OVS_VPORT_TYPE_FT_GRE,	 /* Flow based GRE tunnel. */
+	OVS_VPORT_TYPE_VXLAN,    /* VXLAN tunnel */
 	OVS_VPORT_TYPE_PATCH = 100, /* virtual tunnel connecting two vports */
 	OVS_VPORT_TYPE_GRE,      /* GRE tunnel */
 	OVS_VPORT_TYPE_CAPWAP,   /* CAPWAP tunnel */
diff --git a/include/openflow/nicira-ext.h b/include/openflow/nicira-ext.h
index 88eba19..11b761d 100644
--- a/include/openflow/nicira-ext.h
+++ b/include/openflow/nicira-ext.h
@@ -1578,9 +1578,11 @@ OFP_ASSERT(sizeof(struct nx_action_output_reg) == 24);
 
 /* Tunnel ID.
  *
- * For a packet received via GRE tunnel including a (32-bit) key, the key is
- * stored in the low 32-bits and the high bits are zeroed.  For other packets,
- * the value is 0.
+ * For a packet received via a GRE or VXLAN tunnel including a (32-bit) key, the
+ * key is stored in the low 32-bits and the high bits are zeroed.  For other
+ * packets, the value is 0.
+ *
+ * All zero bits, for packets not received via a keyed tunnel.
  *
  * Prereqs: None.
  *
diff --git a/include/openvswitch/tunnel.h b/include/openvswitch/tunnel.h
index 42c3621..23d8ba7 100644
--- a/include/openvswitch/tunnel.h
+++ b/include/openvswitch/tunnel.h
@@ -57,6 +57,7 @@ enum {
 	OVS_TUNNEL_ATTR_IN_KEY,   /* __be64 key to match on input. */
 	OVS_TUNNEL_ATTR_TOS,      /* 8-bit TOS value. */
 	OVS_TUNNEL_ATTR_TTL,      /* 8-bit TTL value. */
+	OVS_TUNNEL_ATTR_DST_PORT, /* 16-bit UDP port, used by VXLAN. */
 	__OVS_TUNNEL_ATTR_MAX
 };
 
diff --git a/lib/netdev-vport.c b/lib/netdev-vport.c
index 5171171..eadba27 100644
--- a/lib/netdev-vport.c
+++ b/lib/netdev-vport.c
@@ -49,6 +49,9 @@
 
 VLOG_DEFINE_THIS_MODULE(netdev_vport);
 
+/* Default to the OTV port, per the VXLAN IETF draft. */
+#define VXLAN_DST_PORT 8472
+
 struct netdev_dev_vport {
     struct netdev_dev netdev_dev;
     struct ofpbuf *options;
@@ -173,6 +176,9 @@ netdev_vport_get_netdev_type(const struct dpif_linux_vport *vport)
     case OVS_VPORT_TYPE_CAPWAP:
         return "capwap";
 
+    case OVS_VPORT_TYPE_VXLAN:
+        return ("vxlan");
+
     case OVS_VPORT_TYPE_FT_GRE:
     case __OVS_VPORT_TYPE_MAX:
         break;
@@ -585,6 +591,8 @@ parse_tunnel_config(const char *name, const char *type,
 {
     bool is_gre = false;
     bool is_ipsec = false;
+    bool needs_dst_port = false;
+    bool found_dst_port = false;
     struct smap_node *node;
     bool ipsec_mech_set = false;
     ovs_be32 daddr = htonl(0);
@@ -602,6 +610,8 @@ parse_tunnel_config(const char *name, const char *type,
         is_gre = true;
         is_ipsec = true;
         flags |= TNL_F_IPSEC;
+    } else if (!strcmp(type, "vxlan")) {
+        needs_dst_port = true;
     }
 
     SMAP_FOR_EACH (node, args) {
@@ -638,6 +648,10 @@ parse_tunnel_config(const char *name, const char *type,
             } else {
                 nl_msg_put_u8(options, OVS_TUNNEL_ATTR_TTL, atoi(node->value));
             }
+        } else if (!strcmp(node->key, "dst_port") && needs_dst_port) {
+            nl_msg_put_u16(options, OVS_TUNNEL_ATTR_DST_PORT,
+                           atoi(node->value));
+            found_dst_port = true;
         } else if (!strcmp(node->key, "csum") && is_gre) {
             if (!strcmp(node->value, "true")) {
                 flags |= TNL_F_CSUM;
@@ -694,6 +708,11 @@ parse_tunnel_config(const char *name, const char *type,
         }
     }
 
+    /* Add a default destination port for VXLAN if none specified. */
+    if (needs_dst_port && !found_dst_port) {
+        nl_msg_put_u16(options, OVS_TUNNEL_ATTR_DST_PORT, VXLAN_DST_PORT);
+    }
+
     if (is_ipsec) {
         static pid_t pid = 0;
         if (pid <= 0) {
@@ -756,6 +775,7 @@ tnl_port_config_from_nlattr(const struct nlattr *options, size_t options_len,
         [OVS_TUNNEL_ATTR_OUT_KEY] = { .type = NL_A_BE64, .optional = true },
         [OVS_TUNNEL_ATTR_TOS] = { .type = NL_A_U8, .optional = true },
         [OVS_TUNNEL_ATTR_TTL] = { .type = NL_A_U8, .optional = true },
+        [OVS_TUNNEL_ATTR_DST_PORT] = { .type = NL_A_U16, .optional = true },
     };
     struct ofpbuf buf;
 
@@ -835,6 +855,11 @@ unparse_tunnel_config(const char *name OVS_UNUSED, const char *type OVS_UNUSED,
         smap_add_format(args, "tos", "0x%x", tos);
     }
 
+    if (a[OVS_TUNNEL_ATTR_DST_PORT]) {
+        uint16_t dst_port = nl_attr_get_u16(a[OVS_TUNNEL_ATTR_DST_PORT]);
+        smap_add_format(args, "dst_port", "%d", dst_port);
+    }
+
     if (flags & TNL_F_CSUM) {
         smap_add(args, "csum", "true");
     }
@@ -990,6 +1015,10 @@ netdev_vport_register(void)
           { "capwap", VPORT_FUNCTIONS(netdev_vport_get_drv_info) },
           parse_tunnel_config, unparse_tunnel_config },
 
+        { OVS_VPORT_TYPE_VXLAN,
+          { "vxlan", VPORT_FUNCTIONS(netdev_vport_get_drv_info) },
+          parse_tunnel_config, unparse_tunnel_config },
+
         { OVS_VPORT_TYPE_PATCH,
           { "patch", VPORT_FUNCTIONS(NULL) },
           parse_patch_config, unparse_patch_config }
diff --git a/vswitchd/vswitch.xml b/vswitchd/vswitch.xml
index c2786a5..31c15b4 100644
--- a/vswitchd/vswitch.xml
+++ b/vswitchd/vswitch.xml
@@ -1209,8 +1209,7 @@
           <dt><code>gre</code></dt>
           <dd>
             An Ethernet over RFC 2890 Generic Routing Encapsulation over IPv4
-            tunnel.  See <ref group="Tunnel Options"/> for information on
-            configuring GRE tunnels.
+            tunnel.
           </dd>
 
           <dt><code>ipsec_gre</code></dt>
@@ -1247,6 +1246,23 @@
             February 2013.
           </dd>
 
+          <dt><code>vxlan</code></dt>
+          <dd>
+	    <p>
+	      An Ethernet tunnel over the experimental, UDP-based VXLAN
+	      protocol described at
+	      <code>http://tools.ietf.org/html/draft-mahalingam-dutt-dcops-vxlan-02</code>.
+	      VXLAN is currently supported only with the Linux kernel datapath
+	      with kernel version 2.6.26 or later.
+	    </p>
+	    <p>
+	      As an experimental protocol, VXLAN has no officially assigned UDP
+	      port.  Open vSwitch currently uses UDP destination port 8472.
+	      The source port used for VXLAN traffic varies on a per-flow basis
+	      and is in the ephemeral port range.
+	    </p>
+          </dd>
+
           <dt><code>patch</code></dt>
           <dd>
             A pair of virtual devices that act as a patch cable.
@@ -1263,7 +1279,8 @@
       <p>
         These options apply to interfaces with <ref column="type"/> of
         <code>gre</code>, <code>ipsec_gre</code>, <code>gre64</code>,
-        <code>ipsec_gre64</code>, and <code>capwap</code>.
+        <code>ipsec_gre64</code>, <code>capwap</code>, and
+        <code>vxlan</code>.
       </p>
 
       <p>
@@ -1306,8 +1323,9 @@
             key="in_key"/> at all.
           </li>
           <li>
-            A positive 32-bit (for GRE) or 64-bit (for CAPWAP) number.  The
-            tunnel receives only packets with the specified key.
+            A positive 24-bit (for VXLAN), 32-bit (for GRE) or 64-bit (for
+            CAPWAP) number.  The tunnel receives only packets with the
+            specified key.
           </li>
           <li>
             The word <code>flow</code>.  The tunnel accepts packets with any
@@ -1332,8 +1350,9 @@
             key="out_key"/> at all.
           </li>
           <li>
-            A positive 32-bit (for GRE) or 64-bit (for CAPWAP) number.  Packets
-            sent through the tunnel will have the specified key.
+            A positive 24-bit (for VXLAN), 32-bit (for GRE) or 64-bit (for
+            CAPWAP) number.  Packets sent through the tunnel will have the
+            specified key.
           </li>
           <li>
             The word <code>flow</code>.  Packets sent through the tunnel will
@@ -1393,12 +1412,6 @@
         deprecated and will be removed soon.
       </column>
 
-      <group title="Tunnel Options: gre only">
-        <p>
-          Only <code>gre</code> interfaces support these options.
-        </p>
-      </group>
-
       <group title="Tunnel Options: gre and ipsec_gre only">
         <p>
           Only <code>gre</code> and <code>ipsec_gre</code> interfaces support
-- 
1.7.11.7




More information about the dev mailing list