@@ -137,6 +137,7 @@ lib_libopenvswitch_la_SOURCES = \
lib/fatal-signal.h \
lib/flow.c \
lib/flow.h \
+ lib/flow_avx512.h \
lib/guarded-list.c \
lib/guarded-list.h \
lib/hash.c \
@@ -655,6 +655,7 @@ dpdk_get_cpu_has_isa(const char *arch, const char *feature)
#if __x86_64__
/* CPU flags only defined for the architecture that support it. */
CHECK_CPU_FEATURE(feature, "avx512f", RTE_CPUFLAG_AVX512F);
+ CHECK_CPU_FEATURE(feature, "avx512vbmi", RTE_CPUFLAG_AVX512VBMI);
CHECK_CPU_FEATURE(feature, "avx512vpopcntdq", RTE_CPUFLAG_AVX512VPOPCNTDQ);
CHECK_CPU_FEATURE(feature, "bmi2", RTE_CPUFLAG_BMI2);
#endif
@@ -35,6 +35,8 @@
#include "immintrin.h"
+#include "flow_avx512.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
* it's a bit awkward to maintain them in a performant way. This structure
@@ -68,15 +70,24 @@ dp_netdev_input_outer_avx512_probe(void)
return 0;
}
-int32_t
-dp_netdev_input_outer_avx512(struct dp_netdev_pmd_thread *pmd,
- struct dp_packet_batch *packets,
- odp_port_t in_port)
+/* Specialize DPIF based on enabled options, eg for DPCLS only. */
+static inline ALWAYS_INLINE int32_t
+dp_netdev_input_outer_avx512_impl(struct dp_netdev_pmd_thread *pmd,
+ struct dp_packet_batch *packets,
+ odp_port_t in_port,
+ uint32_t dpcls_only)
{
- /* Allocate DPIF userdata. */
if (OVS_UNLIKELY(!pmd->netdev_input_func_userdata)) {
pmd->netdev_input_func_userdata =
xmalloc_pagealign(sizeof(struct dpif_userdata));
+ /* TODO: Enable MFEX selector/autovalidator as done for DPCLS.
+ * This code shows the POC value, not final upstream code.
+ * As the code uses AVX512-VBMI, check for ISA at runtime.
+ */
+ int avx512vbmi = dpdk_get_cpu_has_isa("x86_64", "avx512vbmi");
+ if (avx512vbmi) {
+ pmd->mfex_func = mfex_avx512_ipv4_udp;
+ }
}
struct dpif_userdata *ud = pmd->netdev_input_func_userdata;
@@ -84,6 +95,14 @@ dp_netdev_input_outer_avx512(struct dp_netdev_pmd_thread *pmd,
struct netdev_flow_key **key_ptrs = ud->key_ptrs;
struct pkt_flow_meta *pkt_meta = ud->pkt_meta;
+ /* TODO: make runtime command to allow users to disable/enable.
+ * Not all users need TCP-flags or bytes per rule, and it costs performance
+ * to always calculate it. Enabling this costs ~6 cycles/pkt. It will be
+ * enabled by default for consistency & backwards compat, but disabling
+ * could be investigated by users if they so desire.
+ */
+ uint32_t do_pkt_meta = 1;
+
/* Stores the computed output: a rule pointer for each packet */
/* The AVX512 DPIF implementation handles rules in a way that is optimized
* for reducing data-movement between HWOL/EMC/SMC and DPCLS. This is
@@ -92,7 +111,8 @@ dp_netdev_input_outer_avx512(struct dp_netdev_pmd_thread *pmd,
* array. Later the two arrays are merged by AVX-512 expand instructions.
*/
struct dpcls_rule *rules[NETDEV_MAX_BURST];
- struct dpcls_rule *dpcls_rules[NETDEV_MAX_BURST];
+ struct dpcls_rule *dpcls_rules_impl[NETDEV_MAX_BURST];
+ struct dpcls_rule **dpcls_rules = dpcls_rules_impl;
uint32_t dpcls_key_idx = 0;
for (uint32_t i = 0; i < NETDEV_MAX_BURST; i += 8) {
@@ -100,12 +120,8 @@ dp_netdev_input_outer_avx512(struct dp_netdev_pmd_thread *pmd,
_mm512_storeu_si512(&dpcls_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);
+ if (dpcls_only) {
+ dpcls_rules = rules;
}
/* Check if EMC or SMC are enabled */
@@ -120,32 +136,41 @@ dp_netdev_input_outer_avx512(struct dp_netdev_pmd_thread *pmd,
uint32_t hwol_emc_smc_hitmask = 0;
/* Perform first packet interation */
+ const size_t batch_size = dp_packet_batch_size(packets);
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);
+
+ const uint32_t pf_ahead = 4;
+ int pf = batch_size < pf_ahead ? batch_size : pf_ahead;
+ for (int i = 0; i < pf; i++) {
+ struct dp_packet *packet = packets->packets[i];
+ char *pkt_data_ptr = dp_packet_data(packet);
+ OVS_PREFETCH(pkt_data_ptr);
+ pkt_metadata_prefetch_init(&packet->md);
+ }
+
+ for (int i = 0; i < batch_size; i++) {
+ if (i + pf < batch_size) {
+ struct dp_packet *pfm = packets->packets[i + pf];
+ char *pkt_data_ptr = dp_packet_data(pfm);
+ OVS_PREFETCH(pkt_data_ptr);
+ pkt_metadata_prefetch_init(&pfm->md);
+ }
/* Get packet pointer from bitmask and packet md */
struct dp_packet *packet = packets->packets[i];
pkt_metadata_init(&packet->md, in_port);
-
struct dp_netdev_flow *f = NULL;
/* Check for partial hardware offload mark */
uint32_t mark;
- if (dp_packet_has_flow_mark(packet, &mark)) {
+ if (!dpcls_only && dp_packet_has_flow_mark(packet, &mark)) {
f = mark_to_flow_find(pmd, mark);
if (f) {
rules[i] = &f->cr;
-
- /* This is nasty - instead of using the HWOL provided flow,
- * parse the packet data anyway to find the location of the TCP
- * header to extract the TCP flags for the rule.
- */
- pkt_meta[i].tcp_flags = parse_tcp_flags(packet);
-
- pkt_meta[i].bytes = dp_packet_size(packet);
+ if (do_pkt_meta) {
+ pkt_meta[i].tcp_flags = parse_tcp_flags(packet);
+ pkt_meta[i].bytes = dp_packet_size(packet);
+ }
hwol_emc_smc_hitmask |= (1 << i);
continue;
}
@@ -153,16 +178,29 @@ dp_netdev_input_outer_avx512(struct dp_netdev_pmd_thread *pmd,
/* Do miniflow extract into keys */
struct netdev_flow_key *key = &keys[i];
- miniflow_extract(packet, &key->mf);
- /* Cache TCP and byte values for all packets */
- pkt_meta[i].bytes = dp_packet_size(packet);
- pkt_meta[i].tcp_flags = miniflow_get_tcp_flags(&key->mf);
+ const struct pkt_metadata *md = &packet->md;
+ if (pmd->mfex_func) {
+ uint32_t match = pmd->mfex_func(packet,
+ (struct miniflow *)&key->mf,
+ md->in_port.odp_port);
+ if (!match) {
+ miniflow_extract(packet, &key->mf);
+ }
+ } else {
+ miniflow_extract(packet, &key->mf);
+ }
+
+ if (do_pkt_meta) {
+ /* Cache TCP and byte values for all packets */
+ pkt_meta[i].bytes = dp_packet_size(packet);
+ pkt_meta[i].tcp_flags = miniflow_get_tcp_flags(&key->mf);
+ }
key->len = netdev_flow_key_size(miniflow_n_values(&key->mf));
key->hash = dpif_netdev_packet_get_rss_hash_orig_pkt(packet, &key->mf);
- if (emc_enabled) {
+ if (!dpcls_only && emc_enabled) {
f = emc_lookup(&cache->emc_cache, key);
if (f) {
@@ -173,7 +211,7 @@ dp_netdev_input_outer_avx512(struct dp_netdev_pmd_thread *pmd,
}
};
- if (smc_enabled && !f) {
+ if (!dpcls_only && smc_enabled && !f) {
f = smc_lookup_single(pmd, packet, key);
if (f) {
rules[i] = &f->cr;
@@ -207,28 +245,29 @@ dp_netdev_input_outer_avx512(struct dp_netdev_pmd_thread *pmd,
return -1;
}
- /* Merge DPCLS rules and HWOL/EMC/SMC rules. */
- uint32_t dpcls_idx = 0;
- for (int i = 0; i < NETDEV_MAX_BURST; i += 8) {
- /* Indexing here is somewhat complicated due to DPCLS output rule
- * load index depending on the hitmask of HWOL/EMC/SMC. More
- * packets from HWOL/EMC/SMC bitmask means less DPCLS rules are
- * used.
- */
- __m512i v_cache_rules = _mm512_loadu_si512(&rules[i]);
- __m512i v_merged_rules =
- _mm512_mask_expandloadu_epi64(v_cache_rules,
+ if (!dpcls_only) {
+ /* Merge DPCLS rules and HWOL/EMC/SMC rules. */
+ uint32_t dpcls_idx = 0;
+ for (int i = 0; i < NETDEV_MAX_BURST; i += 8) {
+ /* Indexing here is somewhat complicated due to DPCLS output rule
+ * load index depending on the hitmask of HWOL/EMC/SMC. More
+ * packets from HWOL/EMC/SMC bitmask means less DPCLS rules are
+ * used.
+ */
+ __m512i v_cache_rules = _mm512_loadu_si512(&rules[i]);
+ __m512i v_merged_rules = _mm512_mask_expandloadu_epi64(v_cache_rules,
~hwol_emc_smc_hitmask,
&dpcls_rules[dpcls_idx]);
- _mm512_storeu_si512(&rules[i], v_merged_rules);
-
- /* Update DPCLS load index and bitmask for HWOL/EMC/SMC hits.
- * There are 8 output pointer per register, subtract the
- * HWOL/EMC/SMC lanes equals the number of DPCLS rules consumed.
- */
- uint32_t hitmask_FF = (hwol_emc_smc_hitmask & 0xFF);
- dpcls_idx += 8 - __builtin_popcountll(hitmask_FF);
- hwol_emc_smc_hitmask = (hwol_emc_smc_hitmask >> 8);
+ _mm512_storeu_si512(&rules[i], v_merged_rules);
+
+ /* Update DPCLS load index and bitmask for HWOL/EMC/SMC hits.
+ * There are 8 output pointer per register, subtract the
+ * HWOL/EMC/SMC lanes equals the number of DPCLS rules consumed.
+ */
+ uint32_t hitmask_FF = (hwol_emc_smc_hitmask & 0xFF);
+ dpcls_idx += 8 - __builtin_popcountll(hitmask_FF);
+ hwol_emc_smc_hitmask = (hwol_emc_smc_hitmask >> 8);
+ }
}
}
@@ -280,13 +319,17 @@ dp_netdev_input_outer_avx512(struct dp_netdev_pmd_thread *pmd,
*/
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;
+ /* Avoid this bitmasky/store-y work if possible */
+ if (do_pkt_meta) {
+ 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],
@@ -296,5 +339,24 @@ dp_netdev_input_outer_avx512(struct dp_netdev_pmd_thread *pmd,
return 0;
}
+/* Specialized DPIFs remove branches/complexity in DPCLS only case. */
+int32_t
+dpif_outer_avx512_wildcard(struct dp_netdev_pmd_thread *pmd,
+ struct dp_packet_batch *packets,
+ odp_port_t in_port)
+{
+ uint32_t dpcls_only = 0;
+ return dp_netdev_input_outer_avx512_impl(pmd, packets, in_port, dpcls_only);
+}
+
+int32_t
+dp_netdev_input_outer_avx512(struct dp_netdev_pmd_thread *pmd,
+ struct dp_packet_batch *packets,
+ odp_port_t in_port)
+{
+ uint32_t dpcls_only = 1;
+ return dp_netdev_input_outer_avx512_impl(pmd, packets, in_port, dpcls_only);
+}
+
#endif
#endif
@@ -71,9 +71,15 @@ dp_netdev_input(struct dp_netdev_pmd_thread *pmd,
/* AVX512 enabled DPIF implementation and probe functions */
int32_t
dp_netdev_input_outer_avx512_probe(void);
+
+/* Two specialized instances of the same DPIF impl. */
int32_t
dp_netdev_input_outer_avx512(struct dp_netdev_pmd_thread *pmd,
struct dp_packet_batch *packets,
odp_port_t in_port);
+int32_t
+dpif_outer_avx512_wildcard(struct dp_netdev_pmd_thread *pmd,
+ struct dp_packet_batch *packets,
+ odp_port_t in_port);
#endif /* netdev-private.h */
@@ -51,6 +51,14 @@ struct dp_netdev_pmd_thread_ctx {
bool smc_enable_db;
};
+struct miniflow;
+typedef uint32_t (*dp_netdev_mfex_func)(struct dp_packet *pkt,
+ struct miniflow *mf, uint32_t in_port);
+
+/* Prototype for traffic specific AVX512 MFEX */
+uint32_t mfex_avx512_ipv4_udp(struct dp_packet *pkt,
+ struct miniflow *mf, uint32_t in_port);
+
/* PMD: Poll modes drivers. PMD accesses devices via polling to eliminate
* the performance overhead of interrupt processing. Therefore netdev can
* not implement rx-wait for these devices. dpif-netdev needs to poll
@@ -110,6 +118,8 @@ struct dp_netdev_pmd_thread {
/* Pointer for per-DPIF implementation scratch space. */
void *netdev_input_func_userdata;
+ dp_netdev_mfex_func mfex_func;
+
struct seq *reload_seq;
uint64_t last_reload_seq;
new file mode 100644
@@ -0,0 +1,117 @@
+#pragma once
+
+#include <stdint.h>
+#include <immintrin.h>
+
+#include "dp-packet.h"
+
+/* This file contains optimized implementations of miniflow_extract()
+ * for specific common traffic patterns. The optimizations allow for
+ * quick probing of a specific packet type, and if a match with a specific
+ * type is found, a shuffle like proceedure builds up the required miniflow.
+ *
+ * The functionality here can be easily auto-validated and tested against the
+ * scalar miniflow_extract() function. As such, manual review of the code by
+ * the community (although welcome) is not required. Confidence in the
+ * correctness of the code can be had from the autovalidation.
+ */
+
+/* Generator for EtherType masks and values. */
+#define PATTERN_ETHERTYPE_GEN(type_b0, type_b1) \
+ 0, 0, 0, 0, 0, 0, /* Ether MAC DST */ \
+ 0, 0, 0, 0, 0, 0, /* Ether MAC SRC */ \
+ type_b0, type_b1, /* EtherType */
+
+#define PATTERN_ETHERTYPE_MASK PATTERN_ETHERTYPE_GEN(0xFF, 0xFF)
+#define PATTERN_ETHERTYPE_IPV4 PATTERN_ETHERTYPE_GEN(0x08, 0x00)
+
+/* Generator for checking IPv4 ver, ihl, and proto */
+#define PATTERN_IPV4_GEN(VER_IHL, FLAG_OFF_B0, FLAG_OFF_B1, PROTO) \
+ VER_IHL, /* Version and IHL */ \
+ 0, 0, 0, /* DSCP, ECN, Total Lenght */ \
+ 0, 0, /* Identification */ \
+ /* Flags/Fragment offset: don't match MoreFrag (MF) or FragOffset */ \
+ FLAG_OFF_B0, FLAG_OFF_B1, \
+ 0, /* TTL */ \
+ PROTO, /* Protocol */ \
+ 0, 0, /* Header checksum */ \
+ 0, 0, 0, 0, /* Src IP */ \
+ 0, 0, 0, 0, /* Dst IP */
+
+#define PATTERN_IPV4_MASK PATTERN_IPV4_GEN(0xFF, 0xFE, 0xFF, 0xFF)
+#define PATTERN_IPV4_UDP PATTERN_IPV4_GEN(0x45, 0, 0, 0x11)
+
+#define NU 0
+#define PATTERN_IPV4_UDP_SHUFFLE \
+ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, NU, NU, /* Ether */ \
+ 26, 27, 28, 29, 30, 31, 32, 33, NU, NU, NU, NU, 20, 15, 22, 23, /* IPv4 */ \
+ 34, 35, 36, 37, NU, NU, NU, NU, NU, NU, NU, NU, NU, NU, NU, NU, /* UDP */
+
+/* Masks for Ether()/IP()/UDP() traffic */
+static const uint8_t eth_ip_udp_mask[64] = {
+ PATTERN_ETHERTYPE_MASK PATTERN_IPV4_MASK
+};
+static const uint8_t eth_ip_udp_values[64] = {
+ PATTERN_ETHERTYPE_IPV4 PATTERN_IPV4_UDP
+};
+static const uint8_t eth_ip_udp_shuf[64] = {
+ PATTERN_IPV4_UDP_SHUFFLE
+};
+
+static inline void __attribute__((target("avx512vbmi")))
+avx512_ipv4_udp_store(const uint8_t *pkt, struct miniflow *mf, uint32_t in_port)
+{
+ int64_t u0b = 0x18a0000000000000;
+ int64_t u1b = 0x0000000000040401;
+ __m128i v_bits = {u0b, u1b};
+
+ /* Store mf Bits */
+ uint64_t *bits = (void *)&mf->map.bits[0];
+ uint64_t *blocks = miniflow_values(mf);
+ _mm_storeu_si128((__m128i*)bits, v_bits);
+
+ /* Load packet and shuffle */
+ __m512i v_pkt0 = _mm512_loadu_si512(&pkt[0]);
+ __m512i v_eth_ip_udp_shuf = _mm512_loadu_si512(eth_ip_udp_shuf);
+
+ /* Shuffle pkt and store blocks */
+ __mmask64 k_shufzero = 0b0000111111110000111111110011111111111111;
+ __m512i v_blk0 = _mm512_maskz_permutexvar_epi8(k_shufzero, v_eth_ip_udp_shuf, v_pkt0);
+ _mm512_storeu_si512(&blocks[2], v_blk0);
+
+ uint64_t inp = ((uint64_t)in_port) << 32;
+ blocks[0] = inp;
+}
+
+static inline uint32_t
+avx512_ipv4_udp_probe(const uint8_t *pkt, uint32_t len)
+{
+ /* Packet data is masked to known IPv4/UDP parse length. */
+ uint64_t klen = UINT64_MAX;
+ if (len < 64) {
+ klen = (1ULL << len) - 1;
+ }
+
+ __m512i v_pkt0 = _mm512_maskz_loadu_epi8(klen, &pkt[0]);
+ __m512i v_eth_ip_udp_mask = _mm512_loadu_si512(eth_ip_udp_mask);
+ __m512i v_eth_ip_udp_vals = _mm512_loadu_si512(eth_ip_udp_values);
+ __m512i v_pkt0_masked = _mm512_and_si512(v_pkt0, v_eth_ip_udp_mask);
+ __mmask64 k_cmp = _mm512_cmpeq_epi8_mask(v_pkt0_masked, v_eth_ip_udp_vals);
+
+ return (k_cmp == -1);
+}
+
+uint32_t __attribute__((target("avx512vbmi")))
+mfex_avx512_ipv4_udp(struct dp_packet *dp_pkt, struct miniflow *mf,
+ uint32_t in_port)
+{
+ const uint8_t *pkt = dp_packet_data(dp_pkt);
+ const uint32_t size = dp_packet_size(dp_pkt);
+
+ uint32_t match = avx512_ipv4_udp_probe(pkt, size);
+ if (match) {
+ avx512_ipv4_udp_store(pkt, mf, in_port);
+ return 1;
+ }
+ return 0;
+}
This is a POC patch, showing future DPIF and MFEX optimizations. The main optimization is doing MiniflowExtract in AVX512. This speeds up the specific protocol parsing a lot. Other optimizations for DPIF show value in removing complexity from the code by specialization. In particular if only DPCLS is enabled, we can avoid rebatching packets. Signed-off-by: Harry van Haaren <harry.van.haaren@intel.com> --- lib/automake.mk | 1 + lib/dpdk.c | 1 + lib/dpif-netdev-avx512.c | 178 +++++++++++++++++++++---------- lib/dpif-netdev-private-dpif.h | 6 ++ lib/dpif-netdev-private-thread.h | 10 ++ lib/flow_avx512.h | 117 ++++++++++++++++++++ 6 files changed, 255 insertions(+), 58 deletions(-) create mode 100644 lib/flow_avx512.h