[ovs-dev] [PATCH v2 08/11] dpif-avx512: add ISA implementation of dpif

Harry van Haaren harry.van.haaren at intel.com
Fri Oct 30 19:06:44 UTC 2020


This commit adds the AVX512 implementation of DPIF functionality,
specifically the dp_netdev_input_outer_avx512 function. This function
only handles outer (no re-circulations), and is optimized to use the
AVX512 ISA for packet batching and other DPIF work.

Sparse is not able to handle the AVX512 intrinsics, causing compile
time failures, so it is disabled for this file.

Signed-off-by: Harry van Haaren <harry.van.haaren at intel.com>

---

v2:
- Add BMI flag for _blsr_u64() instruction
- Add sparse avoidance to fix compilation with --enable-sparse
---
 lib/automake.mk                  |   5 +-
 lib/dpif-netdev-avx512.c         | 187 +++++++++++++++++++++++++++++++
 lib/dpif-netdev-private-dpcls.h  |  16 +++
 lib/dpif-netdev-private-dpif.h   |  33 ++++++
 lib/dpif-netdev-private-thread.h |   9 +-
 lib/dpif-netdev-private.h        |  25 +++++
 lib/dpif-netdev.c                |  60 +++++-----
 7 files changed, 302 insertions(+), 33 deletions(-)
 create mode 100644 lib/dpif-netdev-avx512.c
 create mode 100644 lib/dpif-netdev-private-dpif.h

diff --git a/lib/automake.mk b/lib/automake.mk
index eccfaf3e3..650207940 100644
--- a/lib/automake.mk
+++ b/lib/automake.mk
@@ -33,11 +33,13 @@ lib_libopenvswitchavx512_la_CFLAGS = \
 	-mavx512f \
 	-mavx512bw \
 	-mavx512dq \
+	-mbmi \
 	-mbmi2 \
 	-fPIC \
 	$(AM_CFLAGS)
 lib_libopenvswitchavx512_la_SOURCES = \
-	lib/dpif-netdev-lookup-avx512-gather.c
+	lib/dpif-netdev-lookup-avx512-gather.c \
+	lib/dpif-netdev-avx512.c
 lib_libopenvswitchavx512_la_LDFLAGS = \
 	-static
 endif
@@ -113,6 +115,7 @@ lib_libopenvswitch_la_SOURCES = \
 	lib/dpif-netdev.h \
 	lib/dpif-netdev-private-dfc.h \
 	lib/dpif-netdev-private-dpcls.h \
+	lib/dpif-netdev-private-dpif.h \
 	lib/dpif-netdev-private-flow.h \
 	lib/dpif-netdev-private-hwol.h \
 	lib/dpif-netdev-private-thread.h \
diff --git a/lib/dpif-netdev-avx512.c b/lib/dpif-netdev-avx512.c
new file mode 100644
index 000000000..552b50952
--- /dev/null
+++ b/lib/dpif-netdev-avx512.c
@@ -0,0 +1,187 @@
+/*
+ * Copyright (c) 2020 Intel.
+ *
+ * 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.
+ */
+
+#ifdef __x86_64__
+/* Sparse cannot handle the AVX512 instructions */
+#if !defined(__CHECKER__)
+
+#include <config.h>
+
+#include "dpif-netdev.h"
+#include "dpif-netdev-perf.h"
+
+#include "dpif-netdev-private.h"
+#include "dpif-netdev-private-dpcls.h"
+#include "dpif-netdev-private-flow.h"
+#include "dpif-netdev-private-thread.h"
+
+#include "dp-packet.h"
+#include "netdev.h"
+
+#include "immintrin.h"
+
+
+/* Structure to contain per-packet metadata that must be attributed to the
+ * dp netdev flow. This is unfortunate to have to track per packet, however
+ * its a bit difficult awkward to maintain them in a performant way. This
+ * structure helps to keep two variables on a single cache line per packet.
+ */
+struct pkt_flow_meta {
+    uint16_t bytes;
+    uint16_t tcp_flags;
+};
+
+int32_t
+dp_netdev_input_outer_avx512(struct dp_netdev_pmd_thread *pmd,
+                             struct dp_packet_batch *packets,
+                             odp_port_t in_port)
+{
+    OVS_ALIGNED_VAR(CACHE_LINE_SIZE)struct netdev_flow_key keys_impl[NETDEV_MAX_BURST+1];
+    OVS_ALIGNED_VAR(CACHE_LINE_SIZE)struct netdev_flow_key *key_ptrs[NETDEV_MAX_BURST];
+    OVS_ALIGNED_VAR(CACHE_LINE_SIZE)struct pkt_flow_meta pkt_meta[NETDEV_MAX_BURST];
+
+    /* Temporary pointers to the above stack allocated arrays due to the
+     * dpcls_lookup() function signature taking pointers, not linear flow_keys.
+     */
+    ssize_t blocks_offset = offsetof(struct netdev_flow_key, buf);
+    struct netdev_flow_key *keys = (void *)(((char *)keys_impl) + (64-blocks_offset));
+    for (int i = 0; i < NETDEV_MAX_BURST; i++) {
+         key_ptrs[i] = &keys[i];
+    }
+
+    /* Stores the computed output: a rule pointer for each packet */
+    struct dpcls_rule *rules[NETDEV_MAX_BURST];
+    for (uint32_t i = 0; i < NETDEV_MAX_BURST; i += 8) {
+        _mm512_storeu_si512(&rules[i], _mm512_setzero_si512());
+    }
+
+    /* Prefetch each packet's metadata */
+    const size_t batch_size = dp_packet_batch_size(packets);
+    for (int i = 0; i < batch_size; i++) {
+        struct dp_packet *packet = packets->packets[i];
+        OVS_PREFETCH(dp_packet_data(packet));
+        pkt_metadata_prefetch_init(&packet->md);
+    }
+
+    /* Check if EMC or SMC are enabled */
+    struct dfc_cache *cache = &pmd->flow_cache;
+    const uint32_t emc_enabled = pmd->ctx.emc_insert_min != 0;
+    uint32_t emc_hits = 0;
+
+    /* Perform first packet interation */
+    uint32_t lookup_pkts_bitmask = (1ULL << batch_size) - 1;
+    uint32_t iter = lookup_pkts_bitmask;
+    while (iter) {
+        uint32_t i = __builtin_ctz(iter);
+        iter = _blsr_u64(iter);
+
+        /* Initialize packet md and do miniflow extract */
+        struct dp_packet *packet = packets->packets[i];
+        pkt_metadata_init(&packet->md, in_port);
+        struct netdev_flow_key *key = &keys[i];
+        miniflow_extract(packet, &key->mf);
+        key->len = count_1bits(key->mf.map.bits[0] + key->mf.map.bits[1]);
+        key->hash = dpif_netdev_packet_get_rss_hash_orig_pkt(packet, &key->mf);
+
+        if (emc_enabled) {
+           struct dp_netdev_flow *f = emc_lookup(&cache->emc_cache, key);
+           if (f) {
+               rules[i] = &f->cr;
+               emc_hits++;
+               // TODO: remove this EMC hit from the dpcls lookup bitmask
+           }
+        };
+
+        /* Cache TCP and byte values for packets */
+        pkt_meta[i].bytes = dp_packet_size(packet);
+        pkt_meta[i].tcp_flags = miniflow_get_tcp_flags(&key->mf);
+    }
+
+    struct dpcls *cls = dp_netdev_pmd_lookup_dpcls(pmd, in_port);
+    if (OVS_UNLIKELY(!cls)) {
+        return -1;
+    }
+
+    int any_miss = !dpcls_lookup(cls, (const struct netdev_flow_key **)key_ptrs,
+                                rules, batch_size, NULL);
+    if (OVS_UNLIKELY(any_miss)) {
+        return -1;
+    }
+
+    /* At this point we don't return error anymore, so commit stats here */
+    pmd_perf_update_counter(&pmd->perf_stats, PMD_STAT_EXACT_HIT, emc_hits);
+    pmd_perf_update_counter(&pmd->perf_stats, PMD_STAT_RECV, batch_size);
+
+    uint32_t wild_hit = batch_size - emc_hits;
+    pmd_perf_update_counter(&pmd->perf_stats, PMD_STAT_MASKED_HIT, wild_hit);
+    pmd_perf_update_counter(&pmd->perf_stats, PMD_STAT_MASKED_LOOKUP, wild_hit);
+
+    /* Initialize the "Action Batch" for each flow handled below */
+    struct dp_packet_batch action_batch;
+    action_batch.trunc = 0;
+    action_batch.do_not_steal = false;
+
+    while (lookup_pkts_bitmask) {
+        uint32_t rule_pkt_idx = __builtin_ctz(lookup_pkts_bitmask);
+        uint64_t needle = (uintptr_t)rules[rule_pkt_idx];
+
+        /* Parallel compare 8 flow* 's to the needle, create a bitmask */
+        __mmask32 batch_bitmask = 0;
+        for(uint32_t j = 0; j < NETDEV_MAX_BURST; j += 8) {
+            /* Pre-calculate store addr */
+            uint32_t num_pkts_in_batch = __builtin_popcountll(batch_bitmask);
+            void *store_addr = &action_batch.packets[num_pkts_in_batch];
+
+            /* Search for identical flow* in burst, update bitmask */
+            __m512i v_needle = _mm512_maskz_set1_epi64(-1, needle);
+            __m512i v_hay = _mm512_loadu_si512(&rules[j]);
+            uint16_t cmp_bits = _mm512_cmpeq_epi64_mask(v_needle, v_hay);
+            batch_bitmask |= cmp_bits << j;
+
+            /* Compress & Store the batched packets */
+            struct dp_packet **packets_ptrs = &packets->packets[j];
+            __m512i v_pkt_ptrs = _mm512_loadu_si512(packets_ptrs);
+            _mm512_mask_compressstoreu_epi64(store_addr, cmp_bits, v_pkt_ptrs);
+        }
+
+        /* Strip all packets in this batch from the lookup_pkts_bitmask */
+        lookup_pkts_bitmask &= (~batch_bitmask);
+        action_batch.count = __builtin_popcountll(batch_bitmask);
+
+        /* Loop over all packets in this batch, to gather the byte and tcp_flag
+         * values, and pass them to the execute function. It would be nice to
+         * optimize this away, however it is not easy to refactor in dpif.
+         */
+        uint32_t bytes = 0;
+        uint16_t tcp_flags = 0;
+        uint32_t bitmask_iter = batch_bitmask;
+        for(int i = 0; i < action_batch.count; i++) {
+            uint32_t idx = __builtin_ctzll(bitmask_iter);
+            bitmask_iter = _blsr_u64(bitmask_iter);
+
+            bytes += pkt_meta[idx].bytes;
+            tcp_flags |= pkt_meta[idx].tcp_flags;
+        }
+
+        dp_netdev_batch_execute(pmd, &action_batch, rules[rule_pkt_idx],
+                                bytes, tcp_flags);
+    }
+
+    return 0;
+}
+
+#endif
+#endif
diff --git a/lib/dpif-netdev-private-dpcls.h b/lib/dpif-netdev-private-dpcls.h
index eec52217e..5bc579bba 100644
--- a/lib/dpif-netdev-private-dpcls.h
+++ b/lib/dpif-netdev-private-dpcls.h
@@ -106,6 +106,22 @@ netdev_flow_key_gen_masks(const struct netdev_flow_key *tbl,
 bool dpcls_rule_matches_key(const struct dpcls_rule *rule,
                             const struct netdev_flow_key *target);
 
+static inline uint32_t
+dpif_netdev_packet_get_rss_hash_orig_pkt(struct dp_packet *packet,
+                                const struct miniflow *mf)
+{
+    uint32_t hash;
+
+    if (OVS_LIKELY(dp_packet_rss_valid(packet))) {
+        hash = dp_packet_get_rss_hash(packet);
+    } else {
+        hash = miniflow_hash_5tuple(mf, 0);
+        dp_packet_set_rss_hash(packet, hash);
+    }
+
+    return hash;
+}
+
 #ifdef  __cplusplus
 }
 #endif
diff --git a/lib/dpif-netdev-private-dpif.h b/lib/dpif-netdev-private-dpif.h
new file mode 100644
index 000000000..9e64646d7
--- /dev/null
+++ b/lib/dpif-netdev-private-dpif.h
@@ -0,0 +1,33 @@
+/*
+ * Copyright (c) 2020 Intel Corperation.
+ *
+ * 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 DPIF_NETDEV_PRIVATE_DPIF_H
+#define DPIF_NETDEV_PRIVATE_DPIF_H 1
+
+#include "openvswitch/types.h"
+
+/* Forward declarations to avoid including files */
+struct dp_netdev_pmd_thread;
+struct dp_packet_batch;
+
+/* Available implementations for dpif work */
+// TODO make this register/selectable just like DPCLS
+int32_t
+dp_netdev_input_outer_avx512(struct dp_netdev_pmd_thread *pmd,
+                             struct dp_packet_batch *packets,
+                             odp_port_t in_port);
+
+#endif /* netdev-private.h */
diff --git a/lib/dpif-netdev-private-thread.h b/lib/dpif-netdev-private-thread.h
index 44267afe4..fa6c37a56 100644
--- a/lib/dpif-netdev-private-thread.h
+++ b/lib/dpif-netdev-private-thread.h
@@ -50,9 +50,12 @@ struct dp_netdev_pmd_thread_ctx {
 /* Foward declaration for typedef */
 struct dp_netdev_pmd_thread;
 
-typedef void (*dp_netdev_input_func)(struct dp_netdev_pmd_thread *pmd,
-                                     struct dp_packet_batch *packets,
-                                     odp_port_t port_no);
+/* Typedef for DPIF functions.
+ * Returns a bitmask of packets to handle, possibly including upcall/misses.
+ */
+typedef int32_t (*dp_netdev_input_func)(struct dp_netdev_pmd_thread *pmd,
+                                        struct dp_packet_batch *packets,
+                                        odp_port_t port_no);
 
 /* PMD: Poll modes drivers.  PMD accesses devices via polling to eliminate
  * the performance overhead of interrupt processing.  Therefore netdev can
diff --git a/lib/dpif-netdev-private.h b/lib/dpif-netdev-private.h
index 5a21dc1d7..3ae046ed4 100644
--- a/lib/dpif-netdev-private.h
+++ b/lib/dpif-netdev-private.h
@@ -31,4 +31,29 @@
 #include "dpif-netdev-private-dfc.h"
 #include "dpif-netdev-private-thread.h"
 
+/* Allow other implementations to lookup the DPCLS instances */
+struct dpcls *
+dp_netdev_pmd_lookup_dpcls(struct dp_netdev_pmd_thread *pmd,
+                           odp_port_t in_port);
+
+/* Allow other implementations to call dpcls_lookup() for subtable search */
+bool
+dpcls_lookup(struct dpcls *cls, const struct netdev_flow_key *keys[],
+             struct dpcls_rule **rules, const size_t cnt,
+             int *num_lookups_p);
+
+/* Allow other implementations to execute actions on a batch */
+void
+dp_netdev_batch_execute(struct dp_netdev_pmd_thread *pmd,
+                        struct dp_packet_batch *packets,
+                        struct dpcls_rule *rule,
+                        uint32_t bytes,
+                        uint16_t tcp_flags);
+
+/* Available implementations for dpif work */
+int32_t
+dp_netdev_input_outer_avx512(struct dp_netdev_pmd_thread *pmd,
+                             struct dp_packet_batch *packets,
+                             odp_port_t in_port);
+
 #endif /* netdev-private.h */
diff --git a/lib/dpif-netdev.c b/lib/dpif-netdev.c
index 1173aa60b..b296797a0 100644
--- a/lib/dpif-netdev.c
+++ b/lib/dpif-netdev.c
@@ -185,10 +185,6 @@ static uint32_t dpcls_subtable_lookup_reprobe(struct dpcls *cls);
 static void dpcls_insert(struct dpcls *, struct dpcls_rule *,
                          const struct netdev_flow_key *mask);
 static void dpcls_remove(struct dpcls *, struct dpcls_rule *);
-static bool dpcls_lookup(struct dpcls *cls,
-                         const struct netdev_flow_key *keys[],
-                         struct dpcls_rule **rules, size_t cnt,
-                         int *num_lookups_p);
 
 /* Set of supported meter flags */
 #define DP_SUPPORTED_METER_FLAGS_MASK \
@@ -482,7 +478,7 @@ static void dp_netdev_execute_actions(struct dp_netdev_pmd_thread *pmd,
                                       const struct flow *flow,
                                       const struct nlattr *actions,
                                       size_t actions_len);
-static void dp_netdev_input(struct dp_netdev_pmd_thread *,
+static int32_t dp_netdev_input(struct dp_netdev_pmd_thread *,
                             struct dp_packet_batch *, odp_port_t port_no);
 static void dp_netdev_recirculate(struct dp_netdev_pmd_thread *,
                                   struct dp_packet_batch *);
@@ -554,7 +550,7 @@ dpif_netdev_xps_revalidate_pmd(const struct dp_netdev_pmd_thread *pmd,
                                bool purge);
 static int dpif_netdev_xps_get_tx_qid(const struct dp_netdev_pmd_thread *pmd,
                                       struct tx_port *tx);
-static inline struct dpcls *
+inline struct dpcls *
 dp_netdev_pmd_lookup_dpcls(struct dp_netdev_pmd_thread *pmd,
                            odp_port_t in_port);
 
@@ -1917,7 +1913,7 @@ void dp_netdev_flow_unref(struct dp_netdev_flow *flow)
     }
 }
 
-static inline struct dpcls *
+inline struct dpcls *
 dp_netdev_pmd_lookup_dpcls(struct dp_netdev_pmd_thread *pmd,
                            odp_port_t in_port)
 {
@@ -2725,7 +2721,7 @@ dp_netdev_pmd_lookup_flow(struct dp_netdev_pmd_thread *pmd,
                           int *lookup_num_p)
 {
     struct dpcls *cls;
-    struct dpcls_rule *rule;
+    struct dpcls_rule *rule = NULL;
     odp_port_t in_port = u32_to_odp(MINIFLOW_GET_U32(&key->mf,
                                                      in_port.odp_port));
     struct dp_netdev_flow *netdev_flow = NULL;
@@ -4187,7 +4183,10 @@ dp_netdev_process_rxq_port(struct dp_netdev_pmd_thread *pmd,
         }
 
         /* Process packet batch. */
-        pmd->netdev_input_func(pmd, &batch, port_no);
+        int32_t ret = pmd->netdev_input_func(pmd, &batch, port_no);
+        if (ret) {
+            dp_netdev_input(pmd, &batch, port_no);
+        }
 
         /* Assign processing cycles to rx queue. */
         cycles = cycle_timer_stop(&pmd->perf_stats, &timer);
@@ -6360,22 +6359,6 @@ dp_netdev_upcall(struct dp_netdev_pmd_thread *pmd, struct dp_packet *packet_,
                          actions, wc, put_actions, dp->upcall_aux);
 }
 
-static inline uint32_t
-dpif_netdev_packet_get_rss_hash_orig_pkt(struct dp_packet *packet,
-                                const struct miniflow *mf)
-{
-    uint32_t hash;
-
-    if (OVS_LIKELY(dp_packet_rss_valid(packet))) {
-        hash = dp_packet_get_rss_hash(packet);
-    } else {
-        hash = miniflow_hash_5tuple(mf, 0);
-        dp_packet_set_rss_hash(packet, hash);
-    }
-
-    return hash;
-}
-
 static inline uint32_t
 dpif_netdev_packet_get_rss_hash(struct dp_packet *packet,
                                 const struct miniflow *mf)
@@ -6445,6 +6428,24 @@ packet_batch_per_flow_execute(struct packet_batch_per_flow *batch,
                               actions->actions, actions->size);
 }
 
+void
+dp_netdev_batch_execute(struct dp_netdev_pmd_thread *pmd,
+                        struct dp_packet_batch *packets,
+                        struct dpcls_rule *rule,
+                        uint32_t bytes,
+                        uint16_t tcp_flags)
+{
+    /* Gets action* from the rule */
+    struct dp_netdev_flow *flow = dp_netdev_flow_cast(rule);
+    struct dp_netdev_actions *actions = dp_netdev_flow_get_actions(flow);
+
+    dp_netdev_flow_used(flow, dp_packet_batch_size(packets), bytes,
+                        tcp_flags, pmd->ctx.now / 1000);
+    const uint32_t steal = 1;
+    dp_netdev_execute_actions(pmd, packets, steal, &flow->flow,
+                              actions->actions, actions->size);
+}
+
 static inline void
 dp_netdev_queue_batches(struct dp_packet *pkt,
                         struct dp_netdev_flow *flow, uint16_t tcp_flags,
@@ -6800,12 +6801,12 @@ fast_path_processing(struct dp_netdev_pmd_thread *pmd,
     }
     /* Get the classifier for the in_port */
     cls = dp_netdev_pmd_lookup_dpcls(pmd, in_port);
+    memset(rules, 0, sizeof(rules));
     if (OVS_LIKELY(cls)) {
         any_miss = !dpcls_lookup(cls, (const struct netdev_flow_key **)keys,
                                 rules, cnt, &lookup_cnt);
     } else {
         any_miss = true;
-        memset(rules, 0, sizeof(rules));
     }
     if (OVS_UNLIKELY(any_miss) && !fat_rwlock_tryrdlock(&dp->upcall_rwlock)) {
         uint64_t actions_stub[512 / 8], slow_stub[512 / 8];
@@ -6954,12 +6955,13 @@ dp_netdev_input__(struct dp_netdev_pmd_thread *pmd,
     }
 }
 
-static void
+static int32_t
 dp_netdev_input(struct dp_netdev_pmd_thread *pmd,
                 struct dp_packet_batch *packets,
                 odp_port_t port_no)
 {
     dp_netdev_input__(pmd, packets, false, port_no);
+    return 0;
 }
 
 static void
@@ -8363,7 +8365,7 @@ dpcls_rule_matches_key(const struct dpcls_rule *rule,
  * priorities, instead returning any rule which matches the flow.
  *
  * Returns true if all miniflows found a corresponding rule. */
-static bool
+bool
 dpcls_lookup(struct dpcls *cls, const struct netdev_flow_key *keys[],
              struct dpcls_rule **rules, const size_t cnt,
              int *num_lookups_p)
@@ -8380,7 +8382,7 @@ dpcls_lookup(struct dpcls *cls, const struct netdev_flow_key *keys[],
     if (cnt != MAP_BITS) {
         keys_map >>= MAP_BITS - cnt; /* Clear extra bits. */
     }
-    memset(rules, 0, cnt * sizeof *rules);
+    //memset(rules, 0, cnt * sizeof *rules);
 
     int lookups_match = 0, subtable_pos = 1;
     uint32_t found_map;
-- 
2.25.1



More information about the dev mailing list