This patch used userpsace tunneling mechanism for implementing
STT tunneling protocol.
For details about STT you can refer the draft:
https://tools.ietf.org/html/draft-davie-stt-07
Signed-off-by: Pravin B Shelar <pshe...@ovn.org>
---
 lib/automake.mk               |   2 +
 lib/netdev-native-stt.c       | 700 ++++++++++++++++++++++++++++++++++++++++++
 lib/netdev-native-stt.h       |  37 +++
 lib/netdev-vport.c            |   9 +-
 lib/odp-util.c                |  62 +++-
 lib/packets.h                 |  33 ++
 lib/timeval.h                 |   1 +
 tests/tunnel-push-pop-ipv6.at |  19 +-
 tests/tunnel-push-pop.at      |  27 ++
 9 files changed, 885 insertions(+), 5 deletions(-)
 create mode 100644 lib/netdev-native-stt.c
 create mode 100644 lib/netdev-native-stt.h

diff --git a/lib/automake.mk b/lib/automake.mk
index 7972392..a7a0911 100644
--- a/lib/automake.mk
+++ b/lib/automake.mk
@@ -252,6 +252,8 @@ lib_libopenvswitch_la_SOURCES = \
        lib/tnl-neigh-cache.h \
        lib/tnl-ports.c \
        lib/tnl-ports.h \
+       lib/netdev-native-stt.c \
+       lib/netdev-native-stt.h \
        lib/netdev-native-tnl.c \
        lib/netdev-native-tnl.h \
        lib/token-bucket.c \
diff --git a/lib/netdev-native-stt.c b/lib/netdev-native-stt.c
new file mode 100644
index 0000000..0a894ad
--- /dev/null
+++ b/lib/netdev-native-stt.c
@@ -0,0 +1,700 @@
+/*
+ * Copyright (c) 2010, 2011, 2012, 2013, 2014 Nicira, Inc.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at:
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include <config.h>
+
+#include <errno.h>
+#include <fcntl.h>
+#include <sys/socket.h>
+#include <net/if.h>
+#include <netinet/ip6.h>
+#include <sys/ioctl.h>
+
+#include <errno.h>
+#include <stdlib.h>
+#include <sys/time.h>
+
+#include "openvswitch/list.h"
+
+#include "byte-order.h"
+#include "csum.h"
+#include "daemon.h"
+#include "dirs.h"
+#include "dpif.h"
+#include "dp-packet.h"
+#include "entropy.h"
+#include "flow.h"
+#include "hash.h"
+#include "hmap.h"
+#include "id-pool.h"
+#include "netdev-provider.h"
+#include "netdev-vport.h"
+#include "netdev-vport-private.h"
+#include "odp-netlink.h"
+#include "dp-packet.h"
+#include "dp-packet-lso.h"
+#include "ovs-router.h"
+#include "packets.h"
+#include "poll-loop.h"
+#include "random.h"
+#include "route-table.h"
+#include "shash.h"
+#include "socket-util.h"
+#include "timeval.h"
+#include "netdev-native-stt.h"
+#include "netdev-native-tnl.h"
+#include "openvswitch/vlog.h"
+#include "unaligned.h"
+#include "unixctl.h"
+#include "util.h"
+
+VLOG_DEFINE_THIS_MODULE(native_stt);
+static struct vlog_rate_limit err_rl = VLOG_RATE_LIMIT_INIT(60, 5);
+
+/* The maximum amount of memory used to store packets waiting to be reassembled
+ * on a given CPU.  Once this threshold is exceeded we will begin freeing the
+ * least recently used fragments.
+ */
+#define REASM_HI_THRESH (4 * 1024 * 1024)
+/* The target for the high memory evictor.  Once we have exceeded
+ * REASM_HI_THRESH, we will continue freeing fragments until we hit
+ * this limit.
+ */
+#define REASM_LO_THRESH (3 * 1024 * 1024)
+/* The length of time a given packet has to be reassembled from the time the
+ * first fragment arrives.  Once this limit is exceeded it becomes available
+ * for cleaning.
+ */
+#define HZ 1000000000L
+
+#define FRAG_EXP_TIME  (30 * HZ)
+
+#define FRAG_HASH_SHIFT         8
+#define FRAG_HASH_ENTRIES       (1 << FRAG_HASH_SHIFT)
+#define FRAG_HASH_SEGS          ((sizeof(uint32_t) * 8) / FRAG_HASH_SHIFT)
+
+struct pkt_key {
+        struct in6_addr ipv6_src;
+        struct in6_addr ipv6_dst;
+        ovs_be32 pkt_seq;
+};
+
+struct pkt_frag {
+        struct dp_packet *pkts;
+        unsigned long timestamp;
+        struct ovs_list lru_node;
+        struct pkt_key key;
+};
+
+struct first_frag {
+        struct dp_packet *last_pkt;
+        unsigned int mem_used;
+        uint16_t tot_len;
+        uint16_t rcvd_len;
+        bool set_ecn_ce;
+};
+
+struct frag_packet_data {
+        uint16_t offset;
+       uint16_t pkt_size;
+        struct dp_packet *next;
+        /* Only valid for the first packet in the chain. */
+        struct first_frag first;
+};
+
+BUILD_ASSERT_DECL(DP_PACKET_CONTEXT_SIZE >= sizeof(struct frag_packet_data));
+
+#define FRAG_DATA(packet) ((struct frag_packet_data *)(packet)->data)
+#define STT_PACKET_DATA(pkt)   ((unsigned char *)dp_packet_l4(pkt) + \
+                                                  sizeof(struct tcp_header))
+
+struct stt_reassemble {
+        struct pkt_frag frag_hash[FRAG_HASH_ENTRIES];
+        struct ovs_list frag_lru;
+        unsigned int frag_mem_used;
+        uint32_t id;
+        uint32_t counter;
+};
+
+static struct ovs_mutex thread_is_lock;
+static struct id_pool *id_ppol;
+static uint32_t frag_hash_seed;
+static ovsthread_key_t per_thread_reasm_data;
+static void evict_frags(struct stt_reassemble *reasm, int mem_limit);
+
+static struct stt_reassemble *
+get_reasm(void)
+{
+    struct stt_reassemble *reasm;
+    bool res;
+
+    reasm = ovsthread_getspecific(per_thread_reasm_data);
+    if (OVS_LIKELY(reasm)) {
+        return reasm;
+    }
+
+    reasm = xzalloc_cacheline(sizeof(*reasm));
+    ovs_list_init(&reasm->frag_lru);
+    reasm->counter = random_uint32();
+
+    ovs_mutex_lock(&thread_is_lock);
+
+    res = id_pool_alloc_id(id_ppol, &reasm->id);
+
+    if (!res) {
+        OVS_NOT_REACHED();
+    }
+    ovs_mutex_unlock(&thread_is_lock);
+    ovsthread_setspecific(per_thread_reasm_data, reasm);
+    return reasm;
+}
+
+static void
+reasm_destructor(void *_reasm)
+{
+    struct stt_reassemble *reasm = _reasm;
+
+    evict_frags(reasm, 0);
+
+    ovs_mutex_lock(&thread_is_lock);
+    id_pool_free_id(id_ppol, reasm->id);
+    ovs_mutex_unlock(&thread_is_lock);
+    free(reasm);
+}
+
+static inline unsigned long long
+cycles_counter(void)
+{
+    struct timespec tm;
+
+    if (clock_gettime(CLOCK_REALTIME, &tm) == -1 ) {
+        return 0;
+    }
+    return tm.tv_sec * HZ + tm.tv_nsec;
+}
+
+void
+netdev_stt_class_init(void)
+{
+    static bool initialized = false;
+
+    if (initialized) {
+        return;
+    }
+
+    ovsthread_key_create(&per_thread_reasm_data, reasm_destructor);
+    frag_hash_seed = random();
+    ovs_mutex_init(&thread_is_lock);
+    id_ppol = id_pool_create(0, USHRT_MAX);
+    initialized = true;
+    return;
+}
+
+static bool pkt_key_match(const struct pkt_key *a, const struct pkt_key *b)
+{
+    return !memcmp(a, b, sizeof (*a));
+}
+
+static uint32_t pkt_key_hash(const struct pkt_key *key)
+{
+        return hash_2words(hash_bytes(key, offsetof(struct pkt_key, pkt_seq),
+                                      key->pkt_seq),
+                           frag_hash_seed);
+}
+
+static inline void list_packet_delete(struct dp_packet *pkt)
+{
+    do {
+        struct dp_packet *next = FRAG_DATA(pkt)->next;
+
+        dp_packet_delete(pkt);
+        pkt = next;
+    } while (pkt);
+}
+
+static void free_frag(struct stt_reassemble *reasm, struct pkt_frag *frag)
+{
+    reasm->frag_mem_used -= FRAG_DATA(frag->pkts)->first.mem_used;
+    list_packet_delete(frag->pkts);
+    frag->pkts = NULL;
+}
+
+static struct pkt_frag *
+pkt_frag_from_node(const struct ovs_list *node)
+{
+    return CONTAINER_OF(node, struct pkt_frag, lru_node);
+}
+
+static void evict_frags(struct stt_reassemble *reasm, int mem_limit)
+{
+    VLOG_DBG_RL(&err_rl, "STT reasm cache evist");
+    while (!ovs_list_is_empty(&reasm->frag_lru) &&
+           reasm->frag_mem_used > mem_limit) {
+        struct pkt_frag *frag;
+
+        frag = pkt_frag_from_node(ovs_list_pop_front(&reasm->frag_lru));
+        free_frag(reasm, frag);
+    }
+}
+
+static struct pkt_frag *
+lookup_frag(struct stt_reassemble *reasm,
+            const struct pkt_key *key, uint32_t hash)
+{
+    struct pkt_frag *frag, *victim_frag = NULL;
+    int i;
+
+    for (i = 0; i < FRAG_HASH_SEGS; i++) {
+        frag = &reasm->frag_hash[hash & (FRAG_HASH_ENTRIES - 1)];
+
+        if (frag->pkts &&
+            pkt_key_match(&frag->key, key)) {
+            if (time_before(cycles_counter(), frag->timestamp + 
FRAG_EXP_TIME)) {
+                return frag;
+            }
+            break;
+        }
+        if (!victim_frag ||
+            (victim_frag->pkts &&
+             (!frag->pkts ||
+              time_before(frag->timestamp, victim_frag->timestamp))))
+            victim_frag = frag;
+
+        hash >>= FRAG_HASH_SHIFT;
+    }
+
+    if (victim_frag->pkts)
+        free_frag(reasm, victim_frag);
+
+    return victim_frag;
+}
+
+static struct dp_packet *
+packet_merge(struct dp_packet *pkt)
+{
+    struct dp_packet *next;
+    struct dp_packet *m;
+
+    m = dp_packet_clone(pkt);
+    next = FRAG_DATA(pkt)->next;
+    dp_packet_delete(pkt);
+
+    pkt = next;
+    while (pkt) {
+        dp_packet_put(m, STT_PACKET_DATA(pkt), FRAG_DATA(pkt)->pkt_size);
+        next = FRAG_DATA(pkt)->next;
+        dp_packet_delete(pkt);
+        pkt = next;
+    }
+    return m;
+}
+
+static struct dp_packet *
+reassemble(struct dp_packet *packet)
+{
+    struct tcp_header *tcph = dp_packet_l4(packet);
+    int pkt_size = dp_packet_l4_size(packet) - sizeof(*tcph);
+    uint32_t seq = ntohl(get_16aligned_be32(&tcph->tcp_seq));
+    struct stt_reassemble *reasm = get_reasm();
+    struct dp_packet *last_pkt;
+    struct pkt_frag *frag;
+    struct pkt_key key;
+    uint32_t hash;
+    bool is_ipv6;
+    uint8_t tos;
+    int tot_len;
+
+    tot_len = seq >> STT_SEQ_LEN_SHIFT;
+    FRAG_DATA(packet)->offset = seq & STT_SEQ_OFFSET_MASK;
+    FRAG_DATA(packet)->next = NULL;
+    FRAG_DATA(packet)->pkt_size = pkt_size;
+
+    if (STT_BASE_HLEN > pkt_size) {
+        goto out_free;
+    }
+
+    if (FRAG_DATA(packet)->offset + pkt_size > tot_len) {
+        goto out_free;
+    }
+
+    if (tot_len == pkt_size) {
+        goto out;
+    }
+
+    is_ipv6 = is_header_ipv6(dp_packet_data(packet));
+
+    memset(&key, 0, sizeof(key));
+    if (is_ipv6) {
+        struct ovs_16aligned_ip6_hdr *ip6 = dp_packet_l3(packet);
+
+        memcpy(&key.ipv6_src, &ip6->ip6_src.be16, sizeof ip6->ip6_src);
+        memcpy(&key.ipv6_dst, &ip6->ip6_dst.be16, sizeof ip6->ip6_dst);
+    } else {
+        struct ip_header *iph = dp_packet_l3(packet);
+
+        in6_addr_set_mapped_ipv4(&key.ipv6_src, 
get_16aligned_be32(&iph->ip_src));
+        in6_addr_set_mapped_ipv4(&key.ipv6_dst, 
get_16aligned_be32(&iph->ip_dst));
+    }
+    key.pkt_seq = get_16aligned_be32(&tcph->tcp_ack);
+    hash = pkt_key_hash(&key);
+
+    frag = lookup_frag(reasm, &key, hash);
+    if (!frag->pkts) {
+        frag->pkts = packet;
+        frag->key = key;
+        frag->timestamp = cycles_counter();
+        FRAG_DATA(packet)->first.last_pkt = packet;
+        FRAG_DATA(packet)->first.mem_used = dp_packet_get_allocated(packet);
+        FRAG_DATA(packet)->first.tot_len = tot_len;
+        FRAG_DATA(packet)->first.rcvd_len = pkt_size;
+        FRAG_DATA(packet)->first.set_ecn_ce = false;
+        ovs_list_push_back(&reasm->frag_lru, &frag->lru_node);
+        reasm->frag_mem_used += dp_packet_get_allocated(packet);
+
+        packet = NULL;
+        goto out;
+    }
+    /* Optimize for the common case where fragments are received in-order
+     * and not overlapping.
+     */
+    last_pkt = FRAG_DATA(frag->pkts)->first.last_pkt;
+    if (FRAG_DATA(last_pkt)->offset + FRAG_DATA(last_pkt)->pkt_size ==
+           FRAG_DATA(packet)->offset) {
+        FRAG_DATA(last_pkt)->next = packet;
+        FRAG_DATA(frag->pkts)->first.last_pkt = packet;
+    } else {
+        struct dp_packet *prev = NULL, *next;
+
+        for (next = frag->pkts; next; next = FRAG_DATA(next)->next) {
+            if (FRAG_DATA(next)->offset >= FRAG_DATA(packet)->offset) {
+                break;
+            }
+            prev = next;
+        }
+
+        /* Overlapping fragments aren't allowed.  We shouldn't start
+         * before the end of the previous fragment.
+         */
+        if (prev &&
+            FRAG_DATA(prev)->offset + FRAG_DATA(prev)->pkt_size > 
FRAG_DATA(packet)->offset) {
+            goto out_free;
+        }
+
+        /* We also shouldn't end after the beginning of the next
+         * fragment.
+         */
+        if (next &&
+            FRAG_DATA(packet)->offset + pkt_size > FRAG_DATA(next)->offset) {
+            goto out_free;
+        }
+
+        if (prev) {
+            FRAG_DATA(prev)->next = packet;
+        } else {
+            FRAG_DATA(packet)->first = FRAG_DATA(frag->pkts)->first;
+            frag->pkts = packet;
+        }
+
+        if (next) {
+            FRAG_DATA(packet)->next = next;
+        } else {
+            FRAG_DATA(frag->pkts)->first.last_pkt = packet;
+        }
+    }
+
+    if (is_ipv6) {
+        struct ovs_16aligned_ip6_hdr *ip6 = dp_packet_l3(packet);
+
+        tos = (ntohl(get_16aligned_be32(&ip6->ip6_flow)) >> 20) & 0x0FF;
+    } else {
+        struct ip_header *iph = dp_packet_l3(packet);
+        tos = iph->ip_tos;
+    }
+
+    FRAG_DATA(frag->pkts)->first.set_ecn_ce |= IP_ECN_is_ce(tos);
+    FRAG_DATA(frag->pkts)->first.rcvd_len += pkt_size;
+    FRAG_DATA(frag->pkts)->first.mem_used += dp_packet_get_allocated(packet);
+    reasm->frag_mem_used += dp_packet_get_allocated(packet);
+
+    if (FRAG_DATA(frag->pkts)->first.tot_len ==
+        FRAG_DATA(frag->pkts)->first.rcvd_len) {
+        struct dp_packet *frag_head = frag->pkts;
+
+        if (FRAG_DATA(frag_head)->first.set_ecn_ce) {
+            IP_ECN_set_ce(frag_head, is_ipv6);
+        }
+
+        ovs_list_remove(&frag->lru_node);
+        reasm->frag_mem_used -= FRAG_DATA(frag_head)->first.mem_used;
+        frag->pkts = NULL;
+        packet = packet_merge(frag_head);
+    } else {
+        ovs_list_remove(&frag->lru_node);
+        ovs_list_push_back(&reasm->frag_lru, &frag->lru_node);
+        packet = NULL;
+    }
+
+out:
+    if (!packet) {
+        if (reasm->frag_mem_used > REASM_HI_THRESH) {
+            evict_frags(reasm, REASM_LO_THRESH);
+        }
+    }
+    return packet;
+out_free:
+    dp_packet_delete(packet);
+    return NULL;
+}
+
+static bool
+valid_tcp_checksum(struct dp_packet *packet)
+{
+    uint32_t csum;
+
+    if (is_header_ipv6(dp_packet_data(packet))) {
+        csum = packet_csum_pseudoheader6(dp_packet_l3(packet));
+    } else {
+        csum = packet_csum_pseudoheader(dp_packet_l3(packet));
+    }
+
+    csum = csum_continue(csum, dp_packet_l4(packet), 
dp_packet_l4_size(packet));
+    if (csum_finish(csum)) {
+        return false;
+    }
+    return true;
+}
+
+static void *
+tcp_extract_tnl_md(struct dp_packet *packet, struct flow_tnl *tnl,
+                   unsigned int *hlen)
+{
+    struct tcp_header *tcp;
+    int tcp_len;
+
+    tcp = ip_extract_tnl_md(packet, tnl, hlen);
+    if (!tcp) {
+        return NULL;
+    }
+
+    tcp_len = TCP_OFFSET(tcp->tcp_ctl) * 4;
+    tnl->flags |= FLOW_TNL_F_CSUM;
+    tnl->tp_src = tcp->tcp_src;
+    tnl->tp_dst = tcp->tcp_dst;
+    return (void *) ((const char *)tcp + tcp_len);
+}
+
+static int
+handle_lso_offloads(struct dp_packet *packet,
+                    struct stthdr *stth, uint8_t l4_offset)
+{
+    uint8_t proto_type, lso_type;
+    int l3_header_size;
+    int l4_header_size;
+
+    proto_type = stth->flags & STT_PROTO_TYPES;
+    if (proto_type == (STT_PROTO_IPV4 | STT_PROTO_TCP)) {
+       /* TCP/IPv4 */
+       l3_header_size = sizeof(struct ip_header);
+       l4_header_size = sizeof(struct tcp_header);
+       lso_type = DPBUF_LSO_TCPv4;
+    } else if (proto_type == STT_PROTO_TCP) {
+       /* TCP/IPv6 */
+       l3_header_size = sizeof(struct ovs_16aligned_ip6_hdr);
+       l4_header_size = sizeof(struct tcp_header);
+       lso_type = DPBUF_LSO_TCPv6;
+    } else if (proto_type == STT_PROTO_IPV4) {
+       /* UDP/IPv4 */
+       l3_header_size = sizeof(struct ip_header);
+       l4_header_size = sizeof(struct udp_header);
+       lso_type = DPBUF_LSO_UDPv4;
+    } else {
+       /* UDP/IPv6 */
+       l3_header_size = sizeof(struct ovs_16aligned_ip6_hdr);
+       l4_header_size = sizeof(struct udp_header);
+       lso_type = DPBUF_LSO_UDPv6;
+    }
+
+    if (l4_offset < ETH_HEADER_LEN + l3_header_size) {
+        return EINVAL;
+    }
+    if (dp_packet_size(packet) < l4_offset + l4_header_size) {
+        return EINVAL;
+    }
+    packet->lso.mss = ntohs(stth->mss);
+    packet->lso.type = lso_type;
+    return 0;
+}
+
+static int
+stt_extract_tnl_md(struct dp_packet *packet)
+{
+    struct pkt_metadata *md = &packet->md;
+    struct flow_tnl *tnl = &md->tunnel;
+    uint8_t flags, l4_offset;
+    struct stthdr *stth;
+    uint32_t hlen;
+
+    pkt_metadata_init_tnl(md);
+    stth = tcp_extract_tnl_md(packet, tnl, &hlen);
+    if (!stth) {
+        return EINVAL;
+    }
+
+    if (stth->version != 0) {
+        VLOG_WARN_RL(&err_rl, "invalid STT version = %d\n", stth->version);
+        return EINVAL;
+    }
+    flags = stth->flags;
+    l4_offset = stth->l4_offset;
+
+    tnl->tun_id = get_32aligned_be64(&stth->key);
+    tnl->flags |= FLOW_TNL_F_KEY;
+
+    dp_packet_reset_packet(packet, hlen + STT_HEADER_LEN);
+    if (stth->vlan_tci) {
+        eth_push_vlan(packet, htons(ETH_TYPE_VLAN), stth->vlan_tci);
+    }
+
+    if (!(flags & STT_CSUM_PARTIAL)) {
+        memset(&packet->lso, 0, sizeof packet->lso);
+        return 0;
+    }
+    return handle_lso_offloads(packet, stth, l4_offset);
+}
+
+struct dp_packet *
+netdev_stt_pop_header(struct dp_packet *packet)
+{
+    struct dp_packet *reasm_pkt;
+
+    if (!valid_tcp_checksum(packet)) {
+        goto err;
+    }
+
+    reasm_pkt = reassemble(packet);
+    if (reasm_pkt) {
+        if (stt_extract_tnl_md(reasm_pkt)) {
+            goto err;
+        }
+        return reasm_pkt;
+    }
+    return NULL;
+err:
+    dp_packet_delete(packet);
+    return NULL;
+}
+
+static void *
+tcp_build_header(struct netdev_tunnel_config *tnl_cfg,
+                 struct ovs_action_push_tnl *data,
+                 unsigned int *hlen)
+{
+    struct tcp_header *tcp;
+    bool is_ipv6;
+
+    tcp = ip_build_header(data, IPPROTO_TCP, hlen, &is_ipv6);
+    tcp->tcp_dst   = tnl_cfg->dst_port;
+    tcp->tcp_ctl   = TCP_CTL(TCP_ACK | TCP_PSH, sizeof(struct tcp_header) >> 
2);
+    tcp->tcp_winsz = htons(USHRT_MAX);
+    return tcp + 1;
+}
+
+int
+netdev_stt_build_header(const struct netdev *netdev,
+                        struct ovs_action_push_tnl *data,
+                        const struct flow *tnl_flow)
+{
+    struct netdev_vport *dev = netdev_vport_cast(netdev);
+    struct netdev_tunnel_config *tnl_cfg;
+    struct stthdr *stth;
+    unsigned int hlen;
+
+    /* XXX: RCUfy tnl_cfg. */
+    ovs_mutex_lock(&dev->mutex);
+    tnl_cfg = &dev->tnl_cfg;
+
+    stth = tcp_build_header(tnl_cfg, data, &hlen);
+
+    put_32aligned_be64(&stth->key, tnl_flow->tunnel.tun_id);
+
+    ovs_mutex_unlock(&dev->mutex);
+    data->header_len = hlen + STT_HEADER_LEN;
+    data->tnl_type = OVS_VPORT_TYPE_STT;
+    return 0;
+}
+
+static uint32_t ack_seq(void)
+{
+    struct stt_reassemble *reasm = get_reasm();
+    uint32_t ack;
+
+    ack = reasm->counter << 16 | reasm->id;
+    reasm->counter++;
+    return ack;
+}
+
+void
+netdev_stt_push_header(struct dp_packet *packet,
+                       const struct ovs_action_push_tnl *data)
+{
+    struct tcp_header *tcp;
+    uint32_t stt_len, csum;
+    int ip_tot_size;
+    struct stthdr *stth;
+
+    tcp = push_ip_header(packet, data->header, data->header_len, &ip_tot_size);
+
+    /* set tcp src port */
+    tcp->tcp_src = get_src_port(packet);
+
+    stt_len = ip_tot_size - sizeof(struct tcp_header);
+    put_16aligned_be32(&tcp->tcp_seq, htonl(stt_len << STT_SEQ_LEN_SHIFT));
+    put_16aligned_be32(&tcp->tcp_ack, htonl(ack_seq()));
+    if (is_header_ipv6(dp_packet_data(packet))) {
+        csum = packet_csum_pseudoheader6(ipv6_hdr(dp_packet_data(packet)));
+    } else {
+        csum = packet_csum_pseudoheader(ip_hdr(dp_packet_data(packet)));
+    }
+    /* TODO: Handle vlan header offload. */
+    stth = (void *) (tcp + 1);
+
+    if (packet->lso.type) {
+        tcp->tcp_csum = ~csum_finish(csum);
+
+        if (packet->lso.type == DPBUF_LSO_TCPv4) {
+            stth->flags = STT_PROTO_IPV4 | STT_PROTO_TCP;
+        } else if (packet->lso.type == DPBUF_LSO_UDPv4) {
+            stth->flags = STT_PROTO_IPV4;
+        } else if (packet->lso.type == DPBUF_LSO_TCPv6) {
+            stth->flags = STT_PROTO_TCP;
+        } else if (packet->lso.type == DPBUF_LSO_UDPv6) {
+            stth->flags = 0;
+        }
+        stth->flags |= STT_CSUM_PARTIAL;
+        stth->l4_offset = packet->l4_ofs;
+        stth->mss = htons(packet->lso.mss);
+
+        if (is_header_ipv6(dp_packet_data(packet))) {
+            packet->lso.type = DPBUF_LSO_TCPv6;
+        } else {
+            packet->lso.type = DPBUF_LSO_TCPv4;
+        }
+    } else {
+        /* Do not set any checksum flags, let the receiver verify it. */
+        csum = csum_continue(csum, tcp, ip_tot_size);
+        tcp->tcp_csum = csum_finish(csum);
+    }
+}
diff --git a/lib/netdev-native-stt.h b/lib/netdev-native-stt.h
new file mode 100644
index 0000000..e9d994f
--- /dev/null
+++ b/lib/netdev-native-stt.h
@@ -0,0 +1,37 @@
+/*
+ * Copyright (c) 2010, 2011, 2013, 2015 Nicira, Inc.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at:
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#ifndef NETDEV_VPORT_NATIVE_STT_H
+#define NETDEV_VPORT_NATIVE_STT_H 1
+
+#include <stdbool.h>
+#include <stddef.h>
+#include "compiler.h"
+#include "unixctl.h"
+#include "netdev-native-tnl.h"
+
+struct dp_packet *
+netdev_stt_pop_header(struct dp_packet *packet);
+int
+netdev_stt_build_header(const struct netdev *netdev,
+                        struct ovs_action_push_tnl *data,
+                        const struct flow *tnl_flow);
+void
+netdev_stt_push_header(struct dp_packet *packet,
+                       const struct ovs_action_push_tnl *data);
+void
+netdev_stt_class_init(void);
+#endif
diff --git a/lib/netdev-vport.c b/lib/netdev-vport.c
index a31667d..1d2638b 100644
--- a/lib/netdev-vport.c
+++ b/lib/netdev-vport.c
@@ -41,12 +41,14 @@
 #include "netdev-vport-private.h"
 #include "odp-netlink.h"
 #include "dp-packet.h"
+#include "dp-packet-lso.h"
 #include "ovs-router.h"
 #include "packets.h"
 #include "poll-loop.h"
 #include "route-table.h"
 #include "shash.h"
 #include "socket-util.h"
+#include "netdev-native-stt.h"
 #include "netdev-native-tnl.h"
 #include "openvswitch/vlog.h"
 #include "unaligned.h"
@@ -213,6 +215,9 @@ netdev_vport_construct(struct netdev *netdev_)
         dev->tnl_cfg.dst_port = htons(LISP_DST_PORT);
     } else if (!strcmp(type, "stt")) {
         dev->tnl_cfg.dst_port = htons(STT_DST_PORT);
+        netdev_->supported_lso_types = DPBUF_LSO_TCPv4 | DPBUF_LSO_TCPv6 |
+                                       DPBUF_LSO_UDPv4 | DPBUF_LSO_UDPv6;
+        netdev_stt_class_init();
     }
 
     dev->tnl_cfg.dont_fragment = true;
@@ -907,8 +912,10 @@ netdev_vport_tunnel_register(void)
         TUNNEL_CLASS("vxlan", "vxlan_sys", netdev_vxlan_build_header,
                                            push_udp_header,
                                            netdev_vxlan_pop_header),
+        TUNNEL_CLASS("stt", "stt_sys", netdev_stt_build_header,
+                                       netdev_stt_push_header,
+                                       netdev_stt_pop_header),
         TUNNEL_CLASS("lisp", "lisp_sys", NULL, NULL, NULL),
-        TUNNEL_CLASS("stt", "stt_sys", NULL, NULL, NULL),
     };
     static struct ovsthread_once once = OVSTHREAD_ONCE_INITIALIZER;
 
diff --git a/lib/odp-util.c b/lib/odp-util.c
index b4689cc..f518725 100644
--- a/lib/odp-util.c
+++ b/lib/odp-util.c
@@ -85,6 +85,8 @@ static void format_geneve_opts(const struct geneve_opt *opt,
 static struct nlattr *generate_all_wildcard_mask(const struct attr_len_tbl 
tbl[],
                                                  int max, struct ofpbuf *,
                                                  const struct nlattr *key);
+static void format_be64(struct ds *ds, const char *name, ovs_be64 key,
+                        const ovs_be64 *mask, bool verbose);
 static void format_u128(struct ds *ds, const ovs_u128 *value,
                         const ovs_u128 *mask, bool verbose);
 static int scan_u128(const char *s, ovs_u128 *value, ovs_u128 *mask);
@@ -435,13 +437,40 @@ format_udp_tnl_push_header(struct ds *ds, const struct 
udp_header *udp)
     return udp + 1;
 }
 
+static const void *
+format_tcp_tnl_push_header(struct ds *ds, const struct tcp_header *tcp)
+{
+    ds_put_format(ds, "tcp(src=%"PRIu16",dst=%"PRIu16,
+                  ntohs(tcp->tcp_src), ntohs(tcp->tcp_dst));
+
+    format_flags_masked(ds, ",flags", packet_tcp_flag_to_string,
+                        TCP_FLAGS(tcp->tcp_ctl), TCP_FLAGS(OVS_BE16_MAX),
+                        TCP_FLAGS(OVS_BE16_MAX));
+    if (get_16aligned_be32(&tcp->tcp_ack)) {
+        ds_put_format(ds, ",PacketID=0x%"PRIx32,
+                      ntohl(get_16aligned_be32(&tcp->tcp_ack)));
+    }
+    if (get_16aligned_be32(&tcp->tcp_seq)) {
+        ds_put_format(ds, ",len=%"PRIu16",offset=%"PRIu16,
+                      ntohl(get_16aligned_be32(&tcp->tcp_seq)) >> 
STT_SEQ_LEN_SHIFT,
+                      ntohl(get_16aligned_be32(&tcp->tcp_seq)) & 
STT_SEQ_OFFSET_MASK);
+    }
+    if (tcp->tcp_csum) {
+        ds_put_format(ds, ",csum=0x%"PRIx16, ntohs(tcp->tcp_csum));
+    }
+    if (tcp->tcp_urg) {
+        ds_put_format(ds, ",csum=0x%"PRIx16, ntohs(tcp->tcp_urg));
+    }
+    ds_put_char(ds, ')');
+    return tcp + 1;
+}
+
 static void
 format_odp_tnl_push_header(struct ds *ds, struct ovs_action_push_tnl *data)
 {
     const struct eth_header *eth;
     const void *l3;
     const void *l4;
-    const struct udp_header *udp;
 
     eth = (const struct eth_header *)data->header;
 
@@ -481,11 +510,12 @@ format_odp_tnl_push_header(struct ds *ds, struct 
ovs_action_push_tnl *data)
         l4 = (ip6 + 1);
     }
 
-    udp = (const struct udp_header *) l4;
 
     if (data->tnl_type == OVS_VPORT_TYPE_VXLAN) {
         const struct vxlanhdr *vxh;
+        const struct udp_header *udp;
 
+        udp = (const struct udp_header *) l4;
         vxh = format_udp_tnl_push_header(ds, udp);
 
         ds_put_format(ds, "vxlan(flags=0x%"PRIx32",vni=0x%"PRIx32")",
@@ -493,7 +523,9 @@ format_odp_tnl_push_header(struct ds *ds, struct 
ovs_action_push_tnl *data)
                       ntohl(get_16aligned_be32(&vxh->vx_vni)) >> 8);
     } else if (data->tnl_type == OVS_VPORT_TYPE_GENEVE) {
         const struct genevehdr *gnh;
+        const struct udp_header *udp;
 
+        udp = (const struct udp_header *) l4;
         gnh = format_udp_tnl_push_header(ds, udp);
 
         ds_put_format(ds, "geneve(%s%svni=0x%"PRIx32,
@@ -531,6 +563,30 @@ format_odp_tnl_push_header(struct ds *ds, struct 
ovs_action_push_tnl *data)
             options++;
         }
         ds_put_format(ds, ")");
+    } else if (data->tnl_type == OVS_VPORT_TYPE_STT) {
+        const struct tcp_header *tcp;
+        const struct stthdr *stth;
+
+        tcp = (const struct tcp_header *) l4;
+        stth = format_tcp_tnl_push_header(ds, tcp);
+        ds_put_format(ds, ",stt(ver=%"PRIx8, stth->version);
+        format_be64(ds, ",tun_id", get_32aligned_be64(&stth->key), NULL, 
false);
+        if (stth->flags) {
+            ds_put_format(ds, ",flags=0x%"PRIx8, stth->flags);
+        }
+        if (stth->l4_offset) {
+            ds_put_format(ds, ",l4_offset=%"PRIu8, stth->l4_offset);
+        }
+        if (stth->reserved) {
+            ds_put_format(ds, ",reserved=0x%"PRIx8, stth->reserved);
+        }
+        if (stth->mss) {
+            ds_put_format(ds, ",mss=%"PRIu16, stth->mss);
+        }
+        if (stth->vlan_tci) {
+            ds_put_format(ds, ",vlan_tci=%"PRIu16, ntohs(stth->vlan_tci));
+        }
+        ds_put_char(ds, ')');
     }
     ds_put_format(ds, ")");
 }
@@ -2124,7 +2180,6 @@ format_be64(struct ds *ds, const char *name, ovs_be64 key,
         if (!mask_full) { /* Partially masked. */
             ds_put_format(ds, "/%#"PRIx64, ntohll(*mask));
         }
-        ds_put_char(ds, ',');
     }
 }
 
@@ -2504,6 +2559,7 @@ format_odp_tun_attr(const struct nlattr *attr, const 
struct nlattr *mask_attr,
         case OVS_TUNNEL_KEY_ATTR_ID:
             format_be64(ds, "tun_id", nl_attr_get_be64(a),
                         ma ? nl_attr_get(ma) : NULL, verbose);
+            ds_put_char(ds, ',');
            flags |= FLOW_TNL_F_KEY;
             if (ma) {
                 mask_flags |= FLOW_TNL_F_KEY;
diff --git a/lib/packets.h b/lib/packets.h
index 8d627a5..59ac10b 100644
--- a/lib/packets.h
+++ b/lib/packets.h
@@ -1011,6 +1011,39 @@ struct vxlanhdr {
 
 #define VXLAN_FLAGS 0x08000000  /* struct vxlanhdr.vx_flags required value. */
 
+/* STT header */
+
+struct stthdr {
+    __u8                version;
+    __u8                flags;
+    __u8                l4_offset;
+    __u8                reserved;
+    ovs_be16            mss;
+    ovs_be16            vlan_tci;
+    ovs_32aligned_be64  key;
+};
+
+/* Padding after the end of the tunnel headers to provide alignment
+ * for inner packet IP header after 14 byte Ethernet header.
+ */
+#define STT_ETH_PAD 2
+
+#define STT_BASE_HLEN   (sizeof(struct stthdr) + STT_ETH_PAD)
+#define STT_HEADER_LEN (sizeof(struct tcp_header) + STT_BASE_HLEN)
+
+#define STT_CSUM_VERIFIED       (1 << 0)
+#define STT_CSUM_PARTIAL        (1 << 1)
+#define STT_PROTO_IPV4          (1 << 2)
+#define STT_PROTO_TCP           (1 << 3)
+#define STT_PROTO_TYPES         (STT_PROTO_IPV4 | STT_PROTO_TCP)
+
+/* The length and offset of a fragment are encoded in the sequence number.
+ * STT_SEQ_LEN_SHIFT is the left shift needed to store the length.
+ * STT_SEQ_OFFSET_MASK is the mask to extract the offset.
+ */
+#define STT_SEQ_LEN_SHIFT      16
+#define STT_SEQ_OFFSET_MASK    ((1 << STT_SEQ_LEN_SHIFT) - 1)
+
 void ipv6_format_addr(const struct in6_addr *addr, struct ds *);
 void ipv6_format_addr_bracket(const struct in6_addr *addr, struct ds *,
                               bool bracket);
diff --git a/lib/timeval.h b/lib/timeval.h
index 7957dad..956ba51 100644
--- a/lib/timeval.h
+++ b/lib/timeval.h
@@ -76,6 +76,7 @@ long long int time_boot_msec(void);
 
 void timewarp_run(void);
 
+#define time_before(a, b)  ((long)((a) - (b)) < 0)
 #ifdef  __cplusplus
 }
 #endif
diff --git a/tests/tunnel-push-pop-ipv6.at b/tests/tunnel-push-pop-ipv6.at
index ee8c48f..a1fbbc3 100644
--- a/tests/tunnel-push-pop-ipv6.at
+++ b/tests/tunnel-push-pop-ipv6.at
@@ -12,6 +12,8 @@ AT_CHECK([ovs-vsctl add-port int-br t2 -- set Interface t2 
type=vxlan \
                        options:remote_ip=2001:cafe::93 options:out_key=flow 
options:csum=true ofport_request=4\
                     -- add-port int-br t4 -- set Interface t4 type=geneve \
                        options:remote_ip=flow options:key=123 ofport_request=5\
+                    -- add-port int-br t5 -- set Interface t5 type=stt \
+                       options:remote_ip=2001:cafe::93 options:key=789 
ofport_request=6 options:csum=true\
                        ], [0])
 
 AT_CHECK([ovs-appctl dpif/show], [0], [dnl
@@ -25,6 +27,7 @@ dummy@ovs-dummy: hit:0 missed:0
                t2 2/4789: (vxlan: key=123, remote_ip=2001:cafe::92)
                t3 4/4789: (vxlan: csum=true, out_key=flow, 
remote_ip=2001:cafe::93)
                t4 5/6081: (geneve: key=123, remote_ip=flow)
+               t5 6/7471: (stt: csum=true, key=789, remote_ip=2001:cafe::93)
 ])
 
 dnl First setup dummy interface IP address, then add the route
@@ -63,10 +66,11 @@ IP                                            MAC           
      Bridge
 2001:cafe::93                                 f8:bc:12:44:34:b7   br0
 ])
 
-AT_CHECK([ovs-appctl tnl/ports/show |sort], [0], [dnl
+AT_CHECK([ovs-appctl tnl/ports/show|sort], [0], [dnl
 Listening ports:
 genev_sys_6081 (6081)
 gre_sys (3)
+stt_sys_7471 (7471)
 vxlan_sys_4789 (4789)
 ])
 
@@ -109,6 +113,19 @@ AT_CHECK([tail -1 stdout], [0],
   [Datapath actions: 
tnl_push(tnl_port(3),header(size=62,type=3,eth(dst=f8:bc:12:44:34:b6,src=aa:55:aa:55:00:00,dl_type=0x86dd),ipv6(src=2001:cafe::88,dst=2001:cafe::92,label=0,proto=47,tclass=0x0,hlimit=64),gre((flags=0x2000,proto=0x6558),key=0x1c8)),out_port(100))
 ])
 
+dnl Check STT tunnel push
+AT_CHECK([ovs-ofctl add-flow int-br action=6])
+AT_CHECK([ovs-appctl ofproto/trace ovs-dummy 
'in_port(2),eth_type(0x0800),ipv4(src=1.1.3.88,dst=1.1.3.112,proto=47,tos=0,ttl=64,frag=no)'],
 [0], [stdout])
+AT_CHECK([tail -1 stdout], [0],
+  [Datapath actions: 
tnl_push(tnl_port(7471),header(size=92,type=106,eth(dst=f8:bc:12:44:34:b7,src=aa:55:aa:55:00:00,dl_type=0x86dd),ipv6(src=2001:cafe::88,dst=2001:cafe::93,label=0,proto=6,tclass=0x0,hlimit=64),tcp(src=0,dst=7471,flags=psh|ack),stt(ver=0,tun_id=0x315)),out_port(100))
+])
+
+dnl Check STT tunnel pop
+AT_CHECK([ovs-appctl ofproto/trace ovs-dummy 
'in_port(1),eth(src=f8:bc:12:44:34:b6,dst=aa:55:aa:55:00:00),eth_type(0x86dd),ipv6(src=2001:cafe::93,dst=2001:cafe::88,label=0,proto=6,tclass=0x0,hlimit=64),tcp(src=31283,dst=7471)'],
 [0], [stdout])
+AT_CHECK([tail -1 stdout], [0],
+  [Datapath actions: tnl_pop(7471)
+])
+
 dnl Check Geneve tunnel push
 AT_CHECK([ovs-ofctl add-flow int-br 
"actions=set_field:2001:cafe::92->tun_ipv6_dst,5"])
 AT_CHECK([ovs-appctl ofproto/trace ovs-dummy 
'in_port(2),eth_type(0x0800),ipv4(src=1.1.3.88,dst=1.1.3.112,proto=47,tos=0,ttl=64,frag=no)'],
 [0], [stdout])
diff --git a/tests/tunnel-push-pop.at b/tests/tunnel-push-pop.at
index a7909d3..70e021f 100644
--- a/tests/tunnel-push-pop.at
+++ b/tests/tunnel-push-pop.at
@@ -12,6 +12,8 @@ AT_CHECK([ovs-vsctl add-port int-br t2 -- set Interface t2 
type=vxlan \
                        options:remote_ip=1.1.2.93 options:out_key=flow 
options:csum=true ofport_request=4\
                     -- add-port int-br t4 -- set Interface t4 type=geneve \
                        options:remote_ip=flow options:key=123 ofport_request=5\
+                    -- add-port int-br t5 -- set Interface t5 type=stt \
+                       options:remote_ip=1.1.2.92 options:key=789 
ofport_request=6 options:csum=true\
                        ], [0])
 
 AT_CHECK([ovs-appctl dpif/show], [0], [dnl
@@ -25,6 +27,7 @@ dummy@ovs-dummy: hit:0 missed:0
                t2 2/4789: (vxlan: key=123, remote_ip=1.1.2.92)
                t3 4/4789: (vxlan: csum=true, out_key=flow, remote_ip=1.1.2.93)
                t4 5/6081: (geneve: key=123, remote_ip=flow)
+               t5 6/7471: (stt: csum=true, key=789, remote_ip=1.1.2.92)
 ])
 
 dnl First setup dummy interface IP address, then add the route
@@ -67,6 +70,7 @@ AT_CHECK([ovs-appctl tnl/ports/show |sort], [0], [dnl
 Listening ports:
 genev_sys_6081 (6081)
 gre_sys (3)
+stt_sys_7471 (7471)
 vxlan_sys_4789 (4789)
 ])
 
@@ -87,6 +91,13 @@ AT_CHECK([ovs-appctl ofproto/trace ovs-dummy 
'in_port(1),eth(src=f8:bc:12:44:34:
 AT_CHECK([tail -1 stdout], [0],
   [Datapath actions: tnl_pop(6081)
 ])
+i
+
+dnl Check STT tunnel pop
+AT_CHECK([ovs-appctl ofproto/trace ovs-dummy 
'in_port(1),eth(src=f8:bc:12:44:34:b6,dst=aa:55:aa:55:00:00),eth_type(0x0800),ipv4(src=1.1.2.92,dst=1.1.2.88,proto=6,tos=0,ttl=64,frag=no),tcp(src=51283,dst=7471)'],
 [0], [stdout])
+AT_CHECK([tail -1 stdout], [0],
+  [Datapath actions: tnl_pop(7471)
+])
 
 dnl Check VXLAN tunnel push
 AT_CHECK([ovs-ofctl add-flow int-br action=2])
@@ -124,6 +135,13 @@ AT_CHECK([tail -1 stdout], [0],
   [Datapath actions: 
tnl_push(tnl_port(6081),header(size=58,type=5,eth(dst=f8:bc:12:44:34:b6,src=aa:55:aa:55:00:00,dl_type=0x0800),ipv4(src=1.1.2.88,dst=1.1.2.92,proto=17,tos=0,ttl=64,frag=0x4000),udp(src=0,dst=6081,csum=0x0),geneve(crit,vni=0x7b,options({class=0xffff,type=0x80,len=4,0xa}))),out_port(100))
 ])
 
+dnl Check STT tunnel push
+AT_CHECK([ovs-ofctl add-flow int-br action=6])
+AT_CHECK([ovs-appctl ofproto/trace ovs-dummy 
'in_port(2),eth_type(0x0800),ipv4(src=1.1.3.88,dst=1.1.3.112,proto=47,tos=0,ttl=64,frag=no)'],
 [0], [stdout])
+AT_CHECK([tail -1 stdout], [0],
+  [Datapath actions: 
tnl_push(tnl_port(7471),header(size=72,type=106,eth(dst=f8:bc:12:44:34:b6,src=aa:55:aa:55:00:00,dl_type=0x0800),ipv4(src=1.1.2.88,dst=1.1.2.92,proto=6,tos=0,ttl=64,frag=0x4000),tcp(src=0,dst=7471,flags=psh|ack),stt(ver=0,tun_id=0x315)),out_port(100))
+])
+
 dnl Check decapsulation of GRE packet
 AT_CHECK([ovs-appctl netdev-dummy/receive p0 
'aa55aa550000001b213cab6408004500007e79464000402fba550101025c0101025820006558000001c8fe71d883724fbeb6f4e1494a080045000054ba200000400184861e0000011e00000200004227e75400030af3195500000000f265010000000000101112131415161718191a1b1c1d1e1f202122232425262728292a2b2c2d2e2f3031323334353637'])
 ovs-appctl time/warp 1000
@@ -140,6 +158,15 @@ AT_CHECK([ovs-ofctl dump-ports int-br | grep 'port  3'], 
[0], [dnl
   port  3: rx pkts=1, bytes=98, drop=0, errs=0, frame=0, over=0, crc=0
 ])
 
+dnl Check STT decapsulation
+AT_CHECK([ovs-ofctl del-flows int-br])
+AT_CHECK([ovs-appctl netdev-dummy/receive p0 
'aa55aa550000001b213cab6408004500009c00004000400633a70101025c01010258204e1d2f007400000000001c5018ffff7cb8000000010000000000000000000000000315000066b2591a7347427c73ecf94a080045000054b15e400040017e950101045c010104580800a59658cb00018a7b8f560000000016f80a0000000000101112131415161718191a1b1c1d1e1f202122232425262728292a2b2c2d2e2f3031323334353637'])
+ovs-appctl time/warp 1000
+
+AT_CHECK([ovs-ofctl dump-ports int-br | grep 'port  6'], [0], [dnl
+  port  6: rx pkts=1, bytes=98, drop=0, errs=0, frame=0, over=0, crc=0
+])
+
 dnl Check decapsulation of Geneve packet with options
 AT_CAPTURE_FILE([ofctl_monitor.log])
 AT_CHECK([ovs-ofctl monitor int-br 65534 --detach --no-chdir --pidfile 2> 
ofctl_monitor.log])
-- 
2.5.5


_______________________________________________
dev mailing list
dev@openvswitch.org
http://openvswitch.org/mailman/listinfo/dev

Reply via email to