]> git.proxmox.com Git - mirror_ovs.git/blob - lib/dpif-netdev.c
cirrus: Use FreeBSD 12.2.
[mirror_ovs.git] / lib / dpif-netdev.c
1 /*
2 * Copyright (c) 2009-2014, 2016-2018 Nicira, Inc.
3 *
4 * Licensed under the Apache License, Version 2.0 (the "License");
5 * you may not use this file except in compliance with the License.
6 * You may obtain a copy of the License at:
7 *
8 * http://www.apache.org/licenses/LICENSE-2.0
9 *
10 * Unless required by applicable law or agreed to in writing, software
11 * distributed under the License is distributed on an "AS IS" BASIS,
12 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 * See the License for the specific language governing permissions and
14 * limitations under the License.
15 */
16
17 #include <config.h>
18 #include "dpif-netdev.h"
19 #include "dpif-netdev-private.h"
20
21 #include <ctype.h>
22 #include <errno.h>
23 #include <fcntl.h>
24 #include <inttypes.h>
25 #include <net/if.h>
26 #include <sys/types.h>
27 #include <netinet/in.h>
28 #include <stdint.h>
29 #include <stdlib.h>
30 #include <string.h>
31 #include <sys/ioctl.h>
32 #include <sys/socket.h>
33 #include <sys/stat.h>
34 #include <unistd.h>
35
36 #include "bitmap.h"
37 #include "cmap.h"
38 #include "conntrack.h"
39 #include "conntrack-tp.h"
40 #include "coverage.h"
41 #include "ct-dpif.h"
42 #include "csum.h"
43 #include "dp-packet.h"
44 #include "dpif.h"
45 #include "dpif-netdev-lookup.h"
46 #include "dpif-netdev-perf.h"
47 #include "dpif-provider.h"
48 #include "dummy.h"
49 #include "fat-rwlock.h"
50 #include "flow.h"
51 #include "hmapx.h"
52 #include "id-pool.h"
53 #include "ipf.h"
54 #include "netdev.h"
55 #include "netdev-offload.h"
56 #include "netdev-provider.h"
57 #include "netdev-vport.h"
58 #include "netlink.h"
59 #include "odp-execute.h"
60 #include "odp-util.h"
61 #include "openvswitch/dynamic-string.h"
62 #include "openvswitch/list.h"
63 #include "openvswitch/match.h"
64 #include "openvswitch/ofp-parse.h"
65 #include "openvswitch/ofp-print.h"
66 #include "openvswitch/ofpbuf.h"
67 #include "openvswitch/shash.h"
68 #include "openvswitch/vlog.h"
69 #include "ovs-numa.h"
70 #include "ovs-rcu.h"
71 #include "packets.h"
72 #include "openvswitch/poll-loop.h"
73 #include "pvector.h"
74 #include "random.h"
75 #include "seq.h"
76 #include "smap.h"
77 #include "sset.h"
78 #include "timeval.h"
79 #include "tnl-neigh-cache.h"
80 #include "tnl-ports.h"
81 #include "unixctl.h"
82 #include "util.h"
83 #include "uuid.h"
84
85 VLOG_DEFINE_THIS_MODULE(dpif_netdev);
86
87 /* Auto Load Balancing Defaults */
88 #define ALB_IMPROVEMENT_THRESHOLD 25
89 #define ALB_LOAD_THRESHOLD 95
90 #define ALB_REBALANCE_INTERVAL 1 /* 1 Min */
91 #define MIN_TO_MSEC 60000
92
93 #define FLOW_DUMP_MAX_BATCH 50
94 /* Use per thread recirc_depth to prevent recirculation loop. */
95 #define MAX_RECIRC_DEPTH 6
96 DEFINE_STATIC_PER_THREAD_DATA(uint32_t, recirc_depth, 0)
97
98 /* Use instant packet send by default. */
99 #define DEFAULT_TX_FLUSH_INTERVAL 0
100
101 /* Configuration parameters. */
102 enum { MAX_METERS = 65536 }; /* Maximum number of meters. */
103 enum { MAX_BANDS = 8 }; /* Maximum number of bands / meter. */
104 enum { N_METER_LOCKS = 64 }; /* Maximum number of meters. */
105
106 COVERAGE_DEFINE(datapath_drop_meter);
107 COVERAGE_DEFINE(datapath_drop_upcall_error);
108 COVERAGE_DEFINE(datapath_drop_lock_error);
109 COVERAGE_DEFINE(datapath_drop_userspace_action_error);
110 COVERAGE_DEFINE(datapath_drop_tunnel_push_error);
111 COVERAGE_DEFINE(datapath_drop_tunnel_pop_error);
112 COVERAGE_DEFINE(datapath_drop_recirc_error);
113 COVERAGE_DEFINE(datapath_drop_invalid_port);
114 COVERAGE_DEFINE(datapath_drop_invalid_bond);
115 COVERAGE_DEFINE(datapath_drop_invalid_tnl_port);
116 COVERAGE_DEFINE(datapath_drop_rx_invalid_packet);
117
118 /* Protects against changes to 'dp_netdevs'. */
119 static struct ovs_mutex dp_netdev_mutex = OVS_MUTEX_INITIALIZER;
120
121 /* Contains all 'struct dp_netdev's. */
122 static struct shash dp_netdevs OVS_GUARDED_BY(dp_netdev_mutex)
123 = SHASH_INITIALIZER(&dp_netdevs);
124
125 static struct vlog_rate_limit upcall_rl = VLOG_RATE_LIMIT_INIT(600, 600);
126
127 #define DP_NETDEV_CS_SUPPORTED_MASK (CS_NEW | CS_ESTABLISHED | CS_RELATED \
128 | CS_INVALID | CS_REPLY_DIR | CS_TRACKED \
129 | CS_SRC_NAT | CS_DST_NAT)
130 #define DP_NETDEV_CS_UNSUPPORTED_MASK (~(uint32_t)DP_NETDEV_CS_SUPPORTED_MASK)
131
132 static struct odp_support dp_netdev_support = {
133 .max_vlan_headers = SIZE_MAX,
134 .max_mpls_depth = SIZE_MAX,
135 .recirc = true,
136 .ct_state = true,
137 .ct_zone = true,
138 .ct_mark = true,
139 .ct_label = true,
140 .ct_state_nat = true,
141 .ct_orig_tuple = true,
142 .ct_orig_tuple6 = true,
143 };
144
145 /* EMC cache and SMC cache compose the datapath flow cache (DFC)
146 *
147 * Exact match cache for frequently used flows
148 *
149 * The cache uses a 32-bit hash of the packet (which can be the RSS hash) to
150 * search its entries for a miniflow that matches exactly the miniflow of the
151 * packet. It stores the 'dpcls_rule' (rule) that matches the miniflow.
152 *
153 * A cache entry holds a reference to its 'dp_netdev_flow'.
154 *
155 * A miniflow with a given hash can be in one of EM_FLOW_HASH_SEGS different
156 * entries. The 32-bit hash is split into EM_FLOW_HASH_SEGS values (each of
157 * them is EM_FLOW_HASH_SHIFT bits wide and the remainder is thrown away). Each
158 * value is the index of a cache entry where the miniflow could be.
159 *
160 *
161 * Signature match cache (SMC)
162 *
163 * This cache stores a 16-bit signature for each flow without storing keys, and
164 * stores the corresponding 16-bit flow_table index to the 'dp_netdev_flow'.
165 * Each flow thus occupies 32bit which is much more memory efficient than EMC.
166 * SMC uses a set-associative design that each bucket contains
167 * SMC_ENTRY_PER_BUCKET number of entries.
168 * Since 16-bit flow_table index is used, if there are more than 2^16
169 * dp_netdev_flow, SMC will miss them that cannot be indexed by a 16-bit value.
170 *
171 *
172 * Thread-safety
173 * =============
174 *
175 * Each pmd_thread has its own private exact match cache.
176 * If dp_netdev_input is not called from a pmd thread, a mutex is used.
177 */
178
179 #define EM_FLOW_HASH_SHIFT 13
180 #define EM_FLOW_HASH_ENTRIES (1u << EM_FLOW_HASH_SHIFT)
181 #define EM_FLOW_HASH_MASK (EM_FLOW_HASH_ENTRIES - 1)
182 #define EM_FLOW_HASH_SEGS 2
183
184 /* SMC uses a set-associative design. A bucket contains a set of entries that
185 * a flow item can occupy. For now, it uses one hash function rather than two
186 * as for the EMC design. */
187 #define SMC_ENTRY_PER_BUCKET 4
188 #define SMC_ENTRIES (1u << 20)
189 #define SMC_BUCKET_CNT (SMC_ENTRIES / SMC_ENTRY_PER_BUCKET)
190 #define SMC_MASK (SMC_BUCKET_CNT - 1)
191
192 /* Default EMC insert probability is 1 / DEFAULT_EM_FLOW_INSERT_INV_PROB */
193 #define DEFAULT_EM_FLOW_INSERT_INV_PROB 100
194 #define DEFAULT_EM_FLOW_INSERT_MIN (UINT32_MAX / \
195 DEFAULT_EM_FLOW_INSERT_INV_PROB)
196
197 struct emc_entry {
198 struct dp_netdev_flow *flow;
199 struct netdev_flow_key key; /* key.hash used for emc hash value. */
200 };
201
202 struct emc_cache {
203 struct emc_entry entries[EM_FLOW_HASH_ENTRIES];
204 int sweep_idx; /* For emc_cache_slow_sweep(). */
205 };
206
207 struct smc_bucket {
208 uint16_t sig[SMC_ENTRY_PER_BUCKET];
209 uint16_t flow_idx[SMC_ENTRY_PER_BUCKET];
210 };
211
212 /* Signature match cache, differentiate from EMC cache */
213 struct smc_cache {
214 struct smc_bucket buckets[SMC_BUCKET_CNT];
215 };
216
217 struct dfc_cache {
218 struct emc_cache emc_cache;
219 struct smc_cache smc_cache;
220 };
221
222 /* Iterate in the exact match cache through every entry that might contain a
223 * miniflow with hash 'HASH'. */
224 #define EMC_FOR_EACH_POS_WITH_HASH(EMC, CURRENT_ENTRY, HASH) \
225 for (uint32_t i__ = 0, srch_hash__ = (HASH); \
226 (CURRENT_ENTRY) = &(EMC)->entries[srch_hash__ & EM_FLOW_HASH_MASK], \
227 i__ < EM_FLOW_HASH_SEGS; \
228 i__++, srch_hash__ >>= EM_FLOW_HASH_SHIFT)
229 \f
230 /* Simple non-wildcarding single-priority classifier. */
231
232 /* Time in microseconds between successive optimizations of the dpcls
233 * subtable vector */
234 #define DPCLS_OPTIMIZATION_INTERVAL 1000000LL
235
236 /* Time in microseconds of the interval in which rxq processing cycles used
237 * in rxq to pmd assignments is measured and stored. */
238 #define PMD_RXQ_INTERVAL_LEN 10000000LL
239
240 /* Number of intervals for which cycles are stored
241 * and used during rxq to pmd assignment. */
242 #define PMD_RXQ_INTERVAL_MAX 6
243
244 /* Time in microseconds to try RCU quiescing. */
245 #define PMD_RCU_QUIESCE_INTERVAL 10000LL
246
247 struct dpcls {
248 struct cmap_node node; /* Within dp_netdev_pmd_thread.classifiers */
249 odp_port_t in_port;
250 struct cmap subtables_map;
251 struct pvector subtables;
252 };
253
254 /* Data structure to keep packet order till fastpath processing. */
255 struct dp_packet_flow_map {
256 struct dp_packet *packet;
257 struct dp_netdev_flow *flow;
258 uint16_t tcp_flags;
259 };
260
261 static void dpcls_init(struct dpcls *);
262 static void dpcls_destroy(struct dpcls *);
263 static void dpcls_sort_subtable_vector(struct dpcls *);
264 static uint32_t dpcls_subtable_lookup_reprobe(struct dpcls *cls);
265 static void dpcls_insert(struct dpcls *, struct dpcls_rule *,
266 const struct netdev_flow_key *mask);
267 static void dpcls_remove(struct dpcls *, struct dpcls_rule *);
268 static bool dpcls_lookup(struct dpcls *cls,
269 const struct netdev_flow_key *keys[],
270 struct dpcls_rule **rules, size_t cnt,
271 int *num_lookups_p);
272
273 /* Set of supported meter flags */
274 #define DP_SUPPORTED_METER_FLAGS_MASK \
275 (OFPMF13_STATS | OFPMF13_PKTPS | OFPMF13_KBPS | OFPMF13_BURST)
276
277 /* Set of supported meter band types */
278 #define DP_SUPPORTED_METER_BAND_TYPES \
279 ( 1 << OFPMBT13_DROP )
280
281 struct dp_meter_band {
282 struct ofputil_meter_band up; /* type, prec_level, pad, rate, burst_size */
283 uint32_t bucket; /* In 1/1000 packets (for PKTPS), or in bits (for KBPS) */
284 uint64_t packet_count;
285 uint64_t byte_count;
286 };
287
288 struct dp_meter {
289 uint16_t flags;
290 uint16_t n_bands;
291 uint32_t max_delta_t;
292 uint64_t used;
293 uint64_t packet_count;
294 uint64_t byte_count;
295 struct dp_meter_band bands[];
296 };
297
298 struct pmd_auto_lb {
299 bool auto_lb_requested; /* Auto load balancing requested by user. */
300 bool is_enabled; /* Current status of Auto load balancing. */
301 uint64_t rebalance_intvl;
302 uint64_t rebalance_poll_timer;
303 uint8_t rebalance_improve_thresh;
304 atomic_uint8_t rebalance_load_thresh;
305 };
306
307 /* Datapath based on the network device interface from netdev.h.
308 *
309 *
310 * Thread-safety
311 * =============
312 *
313 * Some members, marked 'const', are immutable. Accessing other members
314 * requires synchronization, as noted in more detail below.
315 *
316 * Acquisition order is, from outermost to innermost:
317 *
318 * dp_netdev_mutex (global)
319 * port_mutex
320 * bond_mutex
321 * non_pmd_mutex
322 */
323 struct dp_netdev {
324 const struct dpif_class *const class;
325 const char *const name;
326 struct ovs_refcount ref_cnt;
327 atomic_flag destroyed;
328
329 /* Ports.
330 *
331 * Any lookup into 'ports' or any access to the dp_netdev_ports found
332 * through 'ports' requires taking 'port_mutex'. */
333 struct ovs_mutex port_mutex;
334 struct hmap ports;
335 struct seq *port_seq; /* Incremented whenever a port changes. */
336
337 /* The time that a packet can wait in output batch for sending. */
338 atomic_uint32_t tx_flush_interval;
339
340 /* Meters. */
341 struct ovs_mutex meter_locks[N_METER_LOCKS];
342 struct dp_meter *meters[MAX_METERS]; /* Meter bands. */
343
344 /* Probability of EMC insertions is a factor of 'emc_insert_min'.*/
345 OVS_ALIGNED_VAR(CACHE_LINE_SIZE) atomic_uint32_t emc_insert_min;
346 /* Enable collection of PMD performance metrics. */
347 atomic_bool pmd_perf_metrics;
348 /* Enable the SMC cache from ovsdb config */
349 atomic_bool smc_enable_db;
350
351 /* Protects access to ofproto-dpif-upcall interface during revalidator
352 * thread synchronization. */
353 struct fat_rwlock upcall_rwlock;
354 upcall_callback *upcall_cb; /* Callback function for executing upcalls. */
355 void *upcall_aux;
356
357 /* Callback function for notifying the purging of dp flows (during
358 * reseting pmd deletion). */
359 dp_purge_callback *dp_purge_cb;
360 void *dp_purge_aux;
361
362 /* Stores all 'struct dp_netdev_pmd_thread's. */
363 struct cmap poll_threads;
364 /* id pool for per thread static_tx_qid. */
365 struct id_pool *tx_qid_pool;
366 struct ovs_mutex tx_qid_pool_mutex;
367 /* Use measured cycles for rxq to pmd assignment. */
368 bool pmd_rxq_assign_cyc;
369
370 /* Protects the access of the 'struct dp_netdev_pmd_thread'
371 * instance for non-pmd thread. */
372 struct ovs_mutex non_pmd_mutex;
373
374 /* Each pmd thread will store its pointer to
375 * 'struct dp_netdev_pmd_thread' in 'per_pmd_key'. */
376 ovsthread_key_t per_pmd_key;
377
378 struct seq *reconfigure_seq;
379 uint64_t last_reconfigure_seq;
380
381 /* Cpu mask for pin of pmd threads. */
382 char *pmd_cmask;
383
384 uint64_t last_tnl_conf_seq;
385
386 struct conntrack *conntrack;
387 struct pmd_auto_lb pmd_alb;
388
389 /* Bonds. */
390 struct ovs_mutex bond_mutex; /* Protects updates of 'tx_bonds'. */
391 struct cmap tx_bonds; /* Contains 'struct tx_bond'. */
392 };
393
394 static void meter_lock(const struct dp_netdev *dp, uint32_t meter_id)
395 OVS_ACQUIRES(dp->meter_locks[meter_id % N_METER_LOCKS])
396 {
397 ovs_mutex_lock(&dp->meter_locks[meter_id % N_METER_LOCKS]);
398 }
399
400 static void meter_unlock(const struct dp_netdev *dp, uint32_t meter_id)
401 OVS_RELEASES(dp->meter_locks[meter_id % N_METER_LOCKS])
402 {
403 ovs_mutex_unlock(&dp->meter_locks[meter_id % N_METER_LOCKS]);
404 }
405
406
407 static struct dp_netdev_port *dp_netdev_lookup_port(const struct dp_netdev *dp,
408 odp_port_t)
409 OVS_REQUIRES(dp->port_mutex);
410
411 enum rxq_cycles_counter_type {
412 RXQ_CYCLES_PROC_CURR, /* Cycles spent successfully polling and
413 processing packets during the current
414 interval. */
415 RXQ_CYCLES_PROC_HIST, /* Total cycles of all intervals that are used
416 during rxq to pmd assignment. */
417 RXQ_N_CYCLES
418 };
419
420 enum {
421 DP_NETDEV_FLOW_OFFLOAD_OP_ADD,
422 DP_NETDEV_FLOW_OFFLOAD_OP_MOD,
423 DP_NETDEV_FLOW_OFFLOAD_OP_DEL,
424 };
425
426 struct dp_flow_offload_item {
427 struct dp_netdev_pmd_thread *pmd;
428 struct dp_netdev_flow *flow;
429 int op;
430 struct match match;
431 struct nlattr *actions;
432 size_t actions_len;
433
434 struct ovs_list node;
435 };
436
437 struct dp_flow_offload {
438 struct ovs_mutex mutex;
439 struct ovs_list list;
440 pthread_cond_t cond;
441 };
442
443 static struct dp_flow_offload dp_flow_offload = {
444 .mutex = OVS_MUTEX_INITIALIZER,
445 .list = OVS_LIST_INITIALIZER(&dp_flow_offload.list),
446 };
447
448 static struct ovsthread_once offload_thread_once
449 = OVSTHREAD_ONCE_INITIALIZER;
450
451 #define XPS_TIMEOUT 500000LL /* In microseconds. */
452
453 /* Contained by struct dp_netdev_port's 'rxqs' member. */
454 struct dp_netdev_rxq {
455 struct dp_netdev_port *port;
456 struct netdev_rxq *rx;
457 unsigned core_id; /* Core to which this queue should be
458 pinned. OVS_CORE_UNSPEC if the
459 queue doesn't need to be pinned to a
460 particular core. */
461 unsigned intrvl_idx; /* Write index for 'cycles_intrvl'. */
462 struct dp_netdev_pmd_thread *pmd; /* pmd thread that polls this queue. */
463 bool is_vhost; /* Is rxq of a vhost port. */
464
465 /* Counters of cycles spent successfully polling and processing pkts. */
466 atomic_ullong cycles[RXQ_N_CYCLES];
467 /* We store PMD_RXQ_INTERVAL_MAX intervals of data for an rxq and then
468 sum them to yield the cycles used for an rxq. */
469 atomic_ullong cycles_intrvl[PMD_RXQ_INTERVAL_MAX];
470 };
471
472 /* A port in a netdev-based datapath. */
473 struct dp_netdev_port {
474 odp_port_t port_no;
475 bool dynamic_txqs; /* If true XPS will be used. */
476 bool need_reconfigure; /* True if we should reconfigure netdev. */
477 struct netdev *netdev;
478 struct hmap_node node; /* Node in dp_netdev's 'ports'. */
479 struct netdev_saved_flags *sf;
480 struct dp_netdev_rxq *rxqs;
481 unsigned n_rxq; /* Number of elements in 'rxqs' */
482 unsigned *txq_used; /* Number of threads that use each tx queue. */
483 struct ovs_mutex txq_used_mutex;
484 bool emc_enabled; /* If true EMC will be used. */
485 char *type; /* Port type as requested by user. */
486 char *rxq_affinity_list; /* Requested affinity of rx queues. */
487 };
488
489 /* Contained by struct dp_netdev_flow's 'stats' member. */
490 struct dp_netdev_flow_stats {
491 atomic_llong used; /* Last used time, in monotonic msecs. */
492 atomic_ullong packet_count; /* Number of packets matched. */
493 atomic_ullong byte_count; /* Number of bytes matched. */
494 atomic_uint16_t tcp_flags; /* Bitwise-OR of seen tcp_flags values. */
495 };
496
497 /* Contained by struct dp_netdev_flow's 'last_attrs' member. */
498 struct dp_netdev_flow_attrs {
499 atomic_bool offloaded; /* True if flow is offloaded to HW. */
500 ATOMIC(const char *) dp_layer; /* DP layer the flow is handled in. */
501 };
502
503 /* A flow in 'dp_netdev_pmd_thread's 'flow_table'.
504 *
505 *
506 * Thread-safety
507 * =============
508 *
509 * Except near the beginning or ending of its lifespan, rule 'rule' belongs to
510 * its pmd thread's classifier. The text below calls this classifier 'cls'.
511 *
512 * Motivation
513 * ----------
514 *
515 * The thread safety rules described here for "struct dp_netdev_flow" are
516 * motivated by two goals:
517 *
518 * - Prevent threads that read members of "struct dp_netdev_flow" from
519 * reading bad data due to changes by some thread concurrently modifying
520 * those members.
521 *
522 * - Prevent two threads making changes to members of a given "struct
523 * dp_netdev_flow" from interfering with each other.
524 *
525 *
526 * Rules
527 * -----
528 *
529 * A flow 'flow' may be accessed without a risk of being freed during an RCU
530 * grace period. Code that needs to hold onto a flow for a while
531 * should try incrementing 'flow->ref_cnt' with dp_netdev_flow_ref().
532 *
533 * 'flow->ref_cnt' protects 'flow' from being freed. It doesn't protect the
534 * flow from being deleted from 'cls' and it doesn't protect members of 'flow'
535 * from modification.
536 *
537 * Some members, marked 'const', are immutable. Accessing other members
538 * requires synchronization, as noted in more detail below.
539 */
540 struct dp_netdev_flow {
541 const struct flow flow; /* Unmasked flow that created this entry. */
542 /* Hash table index by unmasked flow. */
543 const struct cmap_node node; /* In owning dp_netdev_pmd_thread's */
544 /* 'flow_table'. */
545 const struct cmap_node mark_node; /* In owning flow_mark's mark_to_flow */
546 const ovs_u128 ufid; /* Unique flow identifier. */
547 const ovs_u128 mega_ufid; /* Unique mega flow identifier. */
548 const unsigned pmd_id; /* The 'core_id' of pmd thread owning this */
549 /* flow. */
550
551 /* Number of references.
552 * The classifier owns one reference.
553 * Any thread trying to keep a rule from being freed should hold its own
554 * reference. */
555 struct ovs_refcount ref_cnt;
556
557 bool dead;
558 uint32_t mark; /* Unique flow mark assigned to a flow */
559
560 /* Statistics. */
561 struct dp_netdev_flow_stats stats;
562
563 /* Statistics and attributes received from the netdev offload provider. */
564 atomic_int netdev_flow_get_result;
565 struct dp_netdev_flow_stats last_stats;
566 struct dp_netdev_flow_attrs last_attrs;
567
568 /* Actions. */
569 OVSRCU_TYPE(struct dp_netdev_actions *) actions;
570
571 /* While processing a group of input packets, the datapath uses the next
572 * member to store a pointer to the output batch for the flow. It is
573 * reset after the batch has been sent out (See dp_netdev_queue_batches(),
574 * packet_batch_per_flow_init() and packet_batch_per_flow_execute()). */
575 struct packet_batch_per_flow *batch;
576
577 /* Packet classification. */
578 char *dp_extra_info; /* String to return in a flow dump/get. */
579 struct dpcls_rule cr; /* In owning dp_netdev's 'cls'. */
580 /* 'cr' must be the last member. */
581 };
582
583 static void dp_netdev_flow_unref(struct dp_netdev_flow *);
584 static bool dp_netdev_flow_ref(struct dp_netdev_flow *);
585 static int dpif_netdev_flow_from_nlattrs(const struct nlattr *, uint32_t,
586 struct flow *, bool);
587
588 /* A set of datapath actions within a "struct dp_netdev_flow".
589 *
590 *
591 * Thread-safety
592 * =============
593 *
594 * A struct dp_netdev_actions 'actions' is protected with RCU. */
595 struct dp_netdev_actions {
596 /* These members are immutable: they do not change during the struct's
597 * lifetime. */
598 unsigned int size; /* Size of 'actions', in bytes. */
599 struct nlattr actions[]; /* Sequence of OVS_ACTION_ATTR_* attributes. */
600 };
601
602 struct dp_netdev_actions *dp_netdev_actions_create(const struct nlattr *,
603 size_t);
604 struct dp_netdev_actions *dp_netdev_flow_get_actions(
605 const struct dp_netdev_flow *);
606 static void dp_netdev_actions_free(struct dp_netdev_actions *);
607
608 struct polled_queue {
609 struct dp_netdev_rxq *rxq;
610 odp_port_t port_no;
611 bool emc_enabled;
612 bool rxq_enabled;
613 uint64_t change_seq;
614 };
615
616 /* Contained by struct dp_netdev_pmd_thread's 'poll_list' member. */
617 struct rxq_poll {
618 struct dp_netdev_rxq *rxq;
619 struct hmap_node node;
620 };
621
622 /* Contained by struct dp_netdev_pmd_thread's 'send_port_cache',
623 * 'tnl_port_cache' or 'tx_ports'. */
624 struct tx_port {
625 struct dp_netdev_port *port;
626 int qid;
627 long long last_used;
628 struct hmap_node node;
629 long long flush_time;
630 struct dp_packet_batch output_pkts;
631 struct dp_netdev_rxq *output_pkts_rxqs[NETDEV_MAX_BURST];
632 };
633
634 /* Contained by struct tx_bond 'member_buckets'. */
635 struct member_entry {
636 odp_port_t member_id;
637 atomic_ullong n_packets;
638 atomic_ullong n_bytes;
639 };
640
641 /* Contained by struct dp_netdev_pmd_thread's 'tx_bonds'. */
642 struct tx_bond {
643 struct cmap_node node;
644 uint32_t bond_id;
645 struct member_entry member_buckets[BOND_BUCKETS];
646 };
647
648 /* A set of properties for the current processing loop that is not directly
649 * associated with the pmd thread itself, but with the packets being
650 * processed or the short-term system configuration (for example, time).
651 * Contained by struct dp_netdev_pmd_thread's 'ctx' member. */
652 struct dp_netdev_pmd_thread_ctx {
653 /* Latest measured time. See 'pmd_thread_ctx_time_update()'. */
654 long long now;
655 /* RX queue from which last packet was received. */
656 struct dp_netdev_rxq *last_rxq;
657 /* EMC insertion probability context for the current processing cycle. */
658 uint32_t emc_insert_min;
659 };
660
661 /* PMD: Poll modes drivers. PMD accesses devices via polling to eliminate
662 * the performance overhead of interrupt processing. Therefore netdev can
663 * not implement rx-wait for these devices. dpif-netdev needs to poll
664 * these device to check for recv buffer. pmd-thread does polling for
665 * devices assigned to itself.
666 *
667 * DPDK used PMD for accessing NIC.
668 *
669 * Note, instance with cpu core id NON_PMD_CORE_ID will be reserved for
670 * I/O of all non-pmd threads. There will be no actual thread created
671 * for the instance.
672 *
673 * Each struct has its own flow cache and classifier per managed ingress port.
674 * For packets received on ingress port, a look up is done on corresponding PMD
675 * thread's flow cache and in case of a miss, lookup is performed in the
676 * corresponding classifier of port. Packets are executed with the found
677 * actions in either case.
678 * */
679 struct dp_netdev_pmd_thread {
680 struct dp_netdev *dp;
681 struct ovs_refcount ref_cnt; /* Every reference must be refcount'ed. */
682 struct cmap_node node; /* In 'dp->poll_threads'. */
683
684 /* Per thread exact-match cache. Note, the instance for cpu core
685 * NON_PMD_CORE_ID can be accessed by multiple threads, and thusly
686 * need to be protected by 'non_pmd_mutex'. Every other instance
687 * will only be accessed by its own pmd thread. */
688 OVS_ALIGNED_VAR(CACHE_LINE_SIZE) struct dfc_cache flow_cache;
689
690 /* Flow-Table and classifiers
691 *
692 * Writers of 'flow_table' must take the 'flow_mutex'. Corresponding
693 * changes to 'classifiers' must be made while still holding the
694 * 'flow_mutex'.
695 */
696 struct ovs_mutex flow_mutex;
697 struct cmap flow_table OVS_GUARDED; /* Flow table. */
698
699 /* One classifier per in_port polled by the pmd */
700 struct cmap classifiers;
701 /* Periodically sort subtable vectors according to hit frequencies */
702 long long int next_optimization;
703 /* End of the next time interval for which processing cycles
704 are stored for each polled rxq. */
705 long long int rxq_next_cycle_store;
706
707 /* Last interval timestamp. */
708 uint64_t intrvl_tsc_prev;
709 /* Last interval cycles. */
710 atomic_ullong intrvl_cycles;
711
712 /* Current context of the PMD thread. */
713 struct dp_netdev_pmd_thread_ctx ctx;
714
715 struct seq *reload_seq;
716 uint64_t last_reload_seq;
717
718 /* These are atomic variables used as a synchronization and configuration
719 * points for thread reload/exit.
720 *
721 * 'reload' atomic is the main one and it's used as a memory
722 * synchronization point for all other knobs and data.
723 *
724 * For a thread that requests PMD reload:
725 *
726 * * All changes that should be visible to the PMD thread must be made
727 * before setting the 'reload'. These changes could use any memory
728 * ordering model including 'relaxed'.
729 * * Setting the 'reload' atomic should occur in the same thread where
730 * all other PMD configuration options updated.
731 * * Setting the 'reload' atomic should be done with 'release' memory
732 * ordering model or stricter. This will guarantee that all previous
733 * changes (including non-atomic and 'relaxed') will be visible to
734 * the PMD thread.
735 * * To check that reload is done, thread should poll the 'reload' atomic
736 * to become 'false'. Polling should be done with 'acquire' memory
737 * ordering model or stricter. This ensures that PMD thread completed
738 * the reload process.
739 *
740 * For the PMD thread:
741 *
742 * * PMD thread should read 'reload' atomic with 'acquire' memory
743 * ordering model or stricter. This will guarantee that all changes
744 * made before setting the 'reload' in the requesting thread will be
745 * visible to the PMD thread.
746 * * All other configuration data could be read with any memory
747 * ordering model (including non-atomic and 'relaxed') but *only after*
748 * reading the 'reload' atomic set to 'true'.
749 * * When the PMD reload done, PMD should (optionally) set all the below
750 * knobs except the 'reload' to their default ('false') values and
751 * (mandatory), as the last step, set the 'reload' to 'false' using
752 * 'release' memory ordering model or stricter. This will inform the
753 * requesting thread that PMD has completed a reload cycle.
754 */
755 atomic_bool reload; /* Do we need to reload ports? */
756 atomic_bool wait_for_reload; /* Can we busy wait for the next reload? */
757 atomic_bool reload_tx_qid; /* Do we need to reload static_tx_qid? */
758 atomic_bool exit; /* For terminating the pmd thread. */
759
760 pthread_t thread;
761 unsigned core_id; /* CPU core id of this pmd thread. */
762 int numa_id; /* numa node id of this pmd thread. */
763 bool isolated;
764
765 /* Queue id used by this pmd thread to send packets on all netdevs if
766 * XPS disabled for this netdev. All static_tx_qid's are unique and less
767 * than 'cmap_count(dp->poll_threads)'. */
768 uint32_t static_tx_qid;
769
770 /* Number of filled output batches. */
771 int n_output_batches;
772
773 struct ovs_mutex port_mutex; /* Mutex for 'poll_list' and 'tx_ports'. */
774 /* List of rx queues to poll. */
775 struct hmap poll_list OVS_GUARDED;
776 /* Map of 'tx_port's used for transmission. Written by the main thread,
777 * read by the pmd thread. */
778 struct hmap tx_ports OVS_GUARDED;
779
780 struct ovs_mutex bond_mutex; /* Protects updates of 'tx_bonds'. */
781 /* Map of 'tx_bond's used for transmission. Written by the main thread
782 * and read by the pmd thread. */
783 struct cmap tx_bonds;
784
785 /* These are thread-local copies of 'tx_ports'. One contains only tunnel
786 * ports (that support push_tunnel/pop_tunnel), the other contains ports
787 * with at least one txq (that support send). A port can be in both.
788 *
789 * There are two separate maps to make sure that we don't try to execute
790 * OUTPUT on a device which has 0 txqs or PUSH/POP on a non-tunnel device.
791 *
792 * The instances for cpu core NON_PMD_CORE_ID can be accessed by multiple
793 * threads, and thusly need to be protected by 'non_pmd_mutex'. Every
794 * other instance will only be accessed by its own pmd thread. */
795 struct hmap tnl_port_cache;
796 struct hmap send_port_cache;
797
798 /* Keep track of detailed PMD performance statistics. */
799 struct pmd_perf_stats perf_stats;
800
801 /* Stats from previous iteration used by automatic pmd
802 * load balance logic. */
803 uint64_t prev_stats[PMD_N_STATS];
804 atomic_count pmd_overloaded;
805
806 /* Set to true if the pmd thread needs to be reloaded. */
807 bool need_reload;
808
809 /* Next time when PMD should try RCU quiescing. */
810 long long next_rcu_quiesce;
811 };
812
813 /* Interface to netdev-based datapath. */
814 struct dpif_netdev {
815 struct dpif dpif;
816 struct dp_netdev *dp;
817 uint64_t last_port_seq;
818 };
819
820 static int get_port_by_number(struct dp_netdev *dp, odp_port_t port_no,
821 struct dp_netdev_port **portp)
822 OVS_REQUIRES(dp->port_mutex);
823 static int get_port_by_name(struct dp_netdev *dp, const char *devname,
824 struct dp_netdev_port **portp)
825 OVS_REQUIRES(dp->port_mutex);
826 static void dp_netdev_free(struct dp_netdev *)
827 OVS_REQUIRES(dp_netdev_mutex);
828 static int do_add_port(struct dp_netdev *dp, const char *devname,
829 const char *type, odp_port_t port_no)
830 OVS_REQUIRES(dp->port_mutex);
831 static void do_del_port(struct dp_netdev *dp, struct dp_netdev_port *)
832 OVS_REQUIRES(dp->port_mutex);
833 static int dpif_netdev_open(const struct dpif_class *, const char *name,
834 bool create, struct dpif **);
835 static void dp_netdev_execute_actions(struct dp_netdev_pmd_thread *pmd,
836 struct dp_packet_batch *,
837 bool should_steal,
838 const struct flow *flow,
839 const struct nlattr *actions,
840 size_t actions_len);
841 static void dp_netdev_input(struct dp_netdev_pmd_thread *,
842 struct dp_packet_batch *, odp_port_t port_no);
843 static void dp_netdev_recirculate(struct dp_netdev_pmd_thread *,
844 struct dp_packet_batch *);
845
846 static void dp_netdev_disable_upcall(struct dp_netdev *);
847 static void dp_netdev_pmd_reload_done(struct dp_netdev_pmd_thread *pmd);
848 static void dp_netdev_configure_pmd(struct dp_netdev_pmd_thread *pmd,
849 struct dp_netdev *dp, unsigned core_id,
850 int numa_id);
851 static void dp_netdev_destroy_pmd(struct dp_netdev_pmd_thread *pmd);
852 static void dp_netdev_set_nonpmd(struct dp_netdev *dp)
853 OVS_REQUIRES(dp->port_mutex);
854
855 static void *pmd_thread_main(void *);
856 static struct dp_netdev_pmd_thread *dp_netdev_get_pmd(struct dp_netdev *dp,
857 unsigned core_id);
858 static struct dp_netdev_pmd_thread *
859 dp_netdev_pmd_get_next(struct dp_netdev *dp, struct cmap_position *pos);
860 static void dp_netdev_del_pmd(struct dp_netdev *dp,
861 struct dp_netdev_pmd_thread *pmd);
862 static void dp_netdev_destroy_all_pmds(struct dp_netdev *dp, bool non_pmd);
863 static void dp_netdev_pmd_clear_ports(struct dp_netdev_pmd_thread *pmd);
864 static void dp_netdev_add_port_tx_to_pmd(struct dp_netdev_pmd_thread *pmd,
865 struct dp_netdev_port *port)
866 OVS_REQUIRES(pmd->port_mutex);
867 static void dp_netdev_del_port_tx_from_pmd(struct dp_netdev_pmd_thread *pmd,
868 struct tx_port *tx)
869 OVS_REQUIRES(pmd->port_mutex);
870 static void dp_netdev_add_rxq_to_pmd(struct dp_netdev_pmd_thread *pmd,
871 struct dp_netdev_rxq *rxq)
872 OVS_REQUIRES(pmd->port_mutex);
873 static void dp_netdev_del_rxq_from_pmd(struct dp_netdev_pmd_thread *pmd,
874 struct rxq_poll *poll)
875 OVS_REQUIRES(pmd->port_mutex);
876 static int
877 dp_netdev_pmd_flush_output_packets(struct dp_netdev_pmd_thread *pmd,
878 bool force);
879 static void dp_netdev_add_bond_tx_to_pmd(struct dp_netdev_pmd_thread *pmd,
880 struct tx_bond *bond, bool update)
881 OVS_EXCLUDED(pmd->bond_mutex);
882 static void dp_netdev_del_bond_tx_from_pmd(struct dp_netdev_pmd_thread *pmd,
883 uint32_t bond_id)
884 OVS_EXCLUDED(pmd->bond_mutex);
885
886 static void reconfigure_datapath(struct dp_netdev *dp)
887 OVS_REQUIRES(dp->port_mutex);
888 static bool dp_netdev_pmd_try_ref(struct dp_netdev_pmd_thread *pmd);
889 static void dp_netdev_pmd_unref(struct dp_netdev_pmd_thread *pmd);
890 static void dp_netdev_pmd_flow_flush(struct dp_netdev_pmd_thread *pmd);
891 static void pmd_load_cached_ports(struct dp_netdev_pmd_thread *pmd)
892 OVS_REQUIRES(pmd->port_mutex);
893 static inline void
894 dp_netdev_pmd_try_optimize(struct dp_netdev_pmd_thread *pmd,
895 struct polled_queue *poll_list, int poll_cnt);
896 static void
897 dp_netdev_rxq_set_cycles(struct dp_netdev_rxq *rx,
898 enum rxq_cycles_counter_type type,
899 unsigned long long cycles);
900 static uint64_t
901 dp_netdev_rxq_get_cycles(struct dp_netdev_rxq *rx,
902 enum rxq_cycles_counter_type type);
903 static void
904 dp_netdev_rxq_set_intrvl_cycles(struct dp_netdev_rxq *rx,
905 unsigned long long cycles);
906 static uint64_t
907 dp_netdev_rxq_get_intrvl_cycles(struct dp_netdev_rxq *rx, unsigned idx);
908 static void
909 dpif_netdev_xps_revalidate_pmd(const struct dp_netdev_pmd_thread *pmd,
910 bool purge);
911 static int dpif_netdev_xps_get_tx_qid(const struct dp_netdev_pmd_thread *pmd,
912 struct tx_port *tx);
913 static inline struct dpcls *
914 dp_netdev_pmd_lookup_dpcls(struct dp_netdev_pmd_thread *pmd,
915 odp_port_t in_port);
916
917 static inline bool emc_entry_alive(struct emc_entry *ce);
918 static void emc_clear_entry(struct emc_entry *ce);
919 static void smc_clear_entry(struct smc_bucket *b, int idx);
920
921 static void dp_netdev_request_reconfigure(struct dp_netdev *dp);
922 static inline bool
923 pmd_perf_metrics_enabled(const struct dp_netdev_pmd_thread *pmd);
924 static void queue_netdev_flow_del(struct dp_netdev_pmd_thread *pmd,
925 struct dp_netdev_flow *flow);
926
927 static void
928 emc_cache_init(struct emc_cache *flow_cache)
929 {
930 int i;
931
932 flow_cache->sweep_idx = 0;
933 for (i = 0; i < ARRAY_SIZE(flow_cache->entries); i++) {
934 flow_cache->entries[i].flow = NULL;
935 flow_cache->entries[i].key.hash = 0;
936 flow_cache->entries[i].key.len = sizeof(struct miniflow);
937 flowmap_init(&flow_cache->entries[i].key.mf.map);
938 }
939 }
940
941 static void
942 smc_cache_init(struct smc_cache *smc_cache)
943 {
944 int i, j;
945 for (i = 0; i < SMC_BUCKET_CNT; i++) {
946 for (j = 0; j < SMC_ENTRY_PER_BUCKET; j++) {
947 smc_cache->buckets[i].flow_idx[j] = UINT16_MAX;
948 }
949 }
950 }
951
952 static void
953 dfc_cache_init(struct dfc_cache *flow_cache)
954 {
955 emc_cache_init(&flow_cache->emc_cache);
956 smc_cache_init(&flow_cache->smc_cache);
957 }
958
959 static void
960 emc_cache_uninit(struct emc_cache *flow_cache)
961 {
962 int i;
963
964 for (i = 0; i < ARRAY_SIZE(flow_cache->entries); i++) {
965 emc_clear_entry(&flow_cache->entries[i]);
966 }
967 }
968
969 static void
970 smc_cache_uninit(struct smc_cache *smc)
971 {
972 int i, j;
973
974 for (i = 0; i < SMC_BUCKET_CNT; i++) {
975 for (j = 0; j < SMC_ENTRY_PER_BUCKET; j++) {
976 smc_clear_entry(&(smc->buckets[i]), j);
977 }
978 }
979 }
980
981 static void
982 dfc_cache_uninit(struct dfc_cache *flow_cache)
983 {
984 smc_cache_uninit(&flow_cache->smc_cache);
985 emc_cache_uninit(&flow_cache->emc_cache);
986 }
987
988 /* Check and clear dead flow references slowly (one entry at each
989 * invocation). */
990 static void
991 emc_cache_slow_sweep(struct emc_cache *flow_cache)
992 {
993 struct emc_entry *entry = &flow_cache->entries[flow_cache->sweep_idx];
994
995 if (!emc_entry_alive(entry)) {
996 emc_clear_entry(entry);
997 }
998 flow_cache->sweep_idx = (flow_cache->sweep_idx + 1) & EM_FLOW_HASH_MASK;
999 }
1000
1001 /* Updates the time in PMD threads context and should be called in three cases:
1002 *
1003 * 1. PMD structure initialization:
1004 * - dp_netdev_configure_pmd()
1005 *
1006 * 2. Before processing of the new packet batch:
1007 * - dpif_netdev_execute()
1008 * - dp_netdev_process_rxq_port()
1009 *
1010 * 3. At least once per polling iteration in main polling threads if no
1011 * packets received on current iteration:
1012 * - dpif_netdev_run()
1013 * - pmd_thread_main()
1014 *
1015 * 'pmd->ctx.now' should be used without update in all other cases if possible.
1016 */
1017 static inline void
1018 pmd_thread_ctx_time_update(struct dp_netdev_pmd_thread *pmd)
1019 {
1020 pmd->ctx.now = time_usec();
1021 }
1022
1023 /* Returns true if 'dpif' is a netdev or dummy dpif, false otherwise. */
1024 bool
1025 dpif_is_netdev(const struct dpif *dpif)
1026 {
1027 return dpif->dpif_class->open == dpif_netdev_open;
1028 }
1029
1030 static struct dpif_netdev *
1031 dpif_netdev_cast(const struct dpif *dpif)
1032 {
1033 ovs_assert(dpif_is_netdev(dpif));
1034 return CONTAINER_OF(dpif, struct dpif_netdev, dpif);
1035 }
1036
1037 static struct dp_netdev *
1038 get_dp_netdev(const struct dpif *dpif)
1039 {
1040 return dpif_netdev_cast(dpif)->dp;
1041 }
1042 \f
1043 enum pmd_info_type {
1044 PMD_INFO_SHOW_STATS, /* Show how cpu cycles are spent. */
1045 PMD_INFO_CLEAR_STATS, /* Set the cycles count to 0. */
1046 PMD_INFO_SHOW_RXQ, /* Show poll lists of pmd threads. */
1047 PMD_INFO_PERF_SHOW, /* Show pmd performance details. */
1048 };
1049
1050 static void
1051 format_pmd_thread(struct ds *reply, struct dp_netdev_pmd_thread *pmd)
1052 {
1053 ds_put_cstr(reply, (pmd->core_id == NON_PMD_CORE_ID)
1054 ? "main thread" : "pmd thread");
1055 if (pmd->numa_id != OVS_NUMA_UNSPEC) {
1056 ds_put_format(reply, " numa_id %d", pmd->numa_id);
1057 }
1058 if (pmd->core_id != OVS_CORE_UNSPEC && pmd->core_id != NON_PMD_CORE_ID) {
1059 ds_put_format(reply, " core_id %u", pmd->core_id);
1060 }
1061 ds_put_cstr(reply, ":\n");
1062 }
1063
1064 static void
1065 pmd_info_show_stats(struct ds *reply,
1066 struct dp_netdev_pmd_thread *pmd)
1067 {
1068 uint64_t stats[PMD_N_STATS];
1069 uint64_t total_cycles, total_packets;
1070 double passes_per_pkt = 0;
1071 double lookups_per_hit = 0;
1072 double packets_per_batch = 0;
1073
1074 pmd_perf_read_counters(&pmd->perf_stats, stats);
1075 total_cycles = stats[PMD_CYCLES_ITER_IDLE]
1076 + stats[PMD_CYCLES_ITER_BUSY];
1077 total_packets = stats[PMD_STAT_RECV];
1078
1079 format_pmd_thread(reply, pmd);
1080
1081 if (total_packets > 0) {
1082 passes_per_pkt = (total_packets + stats[PMD_STAT_RECIRC])
1083 / (double) total_packets;
1084 }
1085 if (stats[PMD_STAT_MASKED_HIT] > 0) {
1086 lookups_per_hit = stats[PMD_STAT_MASKED_LOOKUP]
1087 / (double) stats[PMD_STAT_MASKED_HIT];
1088 }
1089 if (stats[PMD_STAT_SENT_BATCHES] > 0) {
1090 packets_per_batch = stats[PMD_STAT_SENT_PKTS]
1091 / (double) stats[PMD_STAT_SENT_BATCHES];
1092 }
1093
1094 ds_put_format(reply,
1095 " packets received: %"PRIu64"\n"
1096 " packet recirculations: %"PRIu64"\n"
1097 " avg. datapath passes per packet: %.02f\n"
1098 " emc hits: %"PRIu64"\n"
1099 " smc hits: %"PRIu64"\n"
1100 " megaflow hits: %"PRIu64"\n"
1101 " avg. subtable lookups per megaflow hit: %.02f\n"
1102 " miss with success upcall: %"PRIu64"\n"
1103 " miss with failed upcall: %"PRIu64"\n"
1104 " avg. packets per output batch: %.02f\n",
1105 total_packets, stats[PMD_STAT_RECIRC],
1106 passes_per_pkt, stats[PMD_STAT_EXACT_HIT],
1107 stats[PMD_STAT_SMC_HIT],
1108 stats[PMD_STAT_MASKED_HIT], lookups_per_hit,
1109 stats[PMD_STAT_MISS], stats[PMD_STAT_LOST],
1110 packets_per_batch);
1111
1112 if (total_cycles == 0) {
1113 return;
1114 }
1115
1116 ds_put_format(reply,
1117 " idle cycles: %"PRIu64" (%.02f%%)\n"
1118 " processing cycles: %"PRIu64" (%.02f%%)\n",
1119 stats[PMD_CYCLES_ITER_IDLE],
1120 stats[PMD_CYCLES_ITER_IDLE] / (double) total_cycles * 100,
1121 stats[PMD_CYCLES_ITER_BUSY],
1122 stats[PMD_CYCLES_ITER_BUSY] / (double) total_cycles * 100);
1123
1124 if (total_packets == 0) {
1125 return;
1126 }
1127
1128 ds_put_format(reply,
1129 " avg cycles per packet: %.02f (%"PRIu64"/%"PRIu64")\n",
1130 total_cycles / (double) total_packets,
1131 total_cycles, total_packets);
1132
1133 ds_put_format(reply,
1134 " avg processing cycles per packet: "
1135 "%.02f (%"PRIu64"/%"PRIu64")\n",
1136 stats[PMD_CYCLES_ITER_BUSY] / (double) total_packets,
1137 stats[PMD_CYCLES_ITER_BUSY], total_packets);
1138 }
1139
1140 static void
1141 pmd_info_show_perf(struct ds *reply,
1142 struct dp_netdev_pmd_thread *pmd,
1143 struct pmd_perf_params *par)
1144 {
1145 if (pmd->core_id != NON_PMD_CORE_ID) {
1146 char *time_str =
1147 xastrftime_msec("%H:%M:%S.###", time_wall_msec(), true);
1148 long long now = time_msec();
1149 double duration = (now - pmd->perf_stats.start_ms) / 1000.0;
1150
1151 ds_put_cstr(reply, "\n");
1152 ds_put_format(reply, "Time: %s\n", time_str);
1153 ds_put_format(reply, "Measurement duration: %.3f s\n", duration);
1154 ds_put_cstr(reply, "\n");
1155 format_pmd_thread(reply, pmd);
1156 ds_put_cstr(reply, "\n");
1157 pmd_perf_format_overall_stats(reply, &pmd->perf_stats, duration);
1158 if (pmd_perf_metrics_enabled(pmd)) {
1159 /* Prevent parallel clearing of perf metrics. */
1160 ovs_mutex_lock(&pmd->perf_stats.clear_mutex);
1161 if (par->histograms) {
1162 ds_put_cstr(reply, "\n");
1163 pmd_perf_format_histograms(reply, &pmd->perf_stats);
1164 }
1165 if (par->iter_hist_len > 0) {
1166 ds_put_cstr(reply, "\n");
1167 pmd_perf_format_iteration_history(reply, &pmd->perf_stats,
1168 par->iter_hist_len);
1169 }
1170 if (par->ms_hist_len > 0) {
1171 ds_put_cstr(reply, "\n");
1172 pmd_perf_format_ms_history(reply, &pmd->perf_stats,
1173 par->ms_hist_len);
1174 }
1175 ovs_mutex_unlock(&pmd->perf_stats.clear_mutex);
1176 }
1177 free(time_str);
1178 }
1179 }
1180
1181 static int
1182 compare_poll_list(const void *a_, const void *b_)
1183 {
1184 const struct rxq_poll *a = a_;
1185 const struct rxq_poll *b = b_;
1186
1187 const char *namea = netdev_rxq_get_name(a->rxq->rx);
1188 const char *nameb = netdev_rxq_get_name(b->rxq->rx);
1189
1190 int cmp = strcmp(namea, nameb);
1191 if (!cmp) {
1192 return netdev_rxq_get_queue_id(a->rxq->rx)
1193 - netdev_rxq_get_queue_id(b->rxq->rx);
1194 } else {
1195 return cmp;
1196 }
1197 }
1198
1199 static void
1200 sorted_poll_list(struct dp_netdev_pmd_thread *pmd, struct rxq_poll **list,
1201 size_t *n)
1202 OVS_REQUIRES(pmd->port_mutex)
1203 {
1204 struct rxq_poll *ret, *poll;
1205 size_t i;
1206
1207 *n = hmap_count(&pmd->poll_list);
1208 if (!*n) {
1209 ret = NULL;
1210 } else {
1211 ret = xcalloc(*n, sizeof *ret);
1212 i = 0;
1213 HMAP_FOR_EACH (poll, node, &pmd->poll_list) {
1214 ret[i] = *poll;
1215 i++;
1216 }
1217 ovs_assert(i == *n);
1218 qsort(ret, *n, sizeof *ret, compare_poll_list);
1219 }
1220
1221 *list = ret;
1222 }
1223
1224 static void
1225 pmd_info_show_rxq(struct ds *reply, struct dp_netdev_pmd_thread *pmd)
1226 {
1227 if (pmd->core_id != NON_PMD_CORE_ID) {
1228 struct rxq_poll *list;
1229 size_t n_rxq;
1230 uint64_t total_cycles = 0;
1231
1232 ds_put_format(reply,
1233 "pmd thread numa_id %d core_id %u:\n isolated : %s\n",
1234 pmd->numa_id, pmd->core_id, (pmd->isolated)
1235 ? "true" : "false");
1236
1237 ovs_mutex_lock(&pmd->port_mutex);
1238 sorted_poll_list(pmd, &list, &n_rxq);
1239
1240 /* Get the total pmd cycles for an interval. */
1241 atomic_read_relaxed(&pmd->intrvl_cycles, &total_cycles);
1242 /* Estimate the cycles to cover all intervals. */
1243 total_cycles *= PMD_RXQ_INTERVAL_MAX;
1244
1245 for (int i = 0; i < n_rxq; i++) {
1246 struct dp_netdev_rxq *rxq = list[i].rxq;
1247 const char *name = netdev_rxq_get_name(rxq->rx);
1248 uint64_t proc_cycles = 0;
1249
1250 for (int j = 0; j < PMD_RXQ_INTERVAL_MAX; j++) {
1251 proc_cycles += dp_netdev_rxq_get_intrvl_cycles(rxq, j);
1252 }
1253 ds_put_format(reply, " port: %-16s queue-id: %2d", name,
1254 netdev_rxq_get_queue_id(list[i].rxq->rx));
1255 ds_put_format(reply, " %s", netdev_rxq_enabled(list[i].rxq->rx)
1256 ? "(enabled) " : "(disabled)");
1257 ds_put_format(reply, " pmd usage: ");
1258 if (total_cycles) {
1259 ds_put_format(reply, "%2"PRIu64"",
1260 proc_cycles * 100 / total_cycles);
1261 ds_put_cstr(reply, " %");
1262 } else {
1263 ds_put_format(reply, "%s", "NOT AVAIL");
1264 }
1265 ds_put_cstr(reply, "\n");
1266 }
1267 ovs_mutex_unlock(&pmd->port_mutex);
1268 free(list);
1269 }
1270 }
1271
1272 static int
1273 compare_poll_thread_list(const void *a_, const void *b_)
1274 {
1275 const struct dp_netdev_pmd_thread *a, *b;
1276
1277 a = *(struct dp_netdev_pmd_thread **)a_;
1278 b = *(struct dp_netdev_pmd_thread **)b_;
1279
1280 if (a->core_id < b->core_id) {
1281 return -1;
1282 }
1283 if (a->core_id > b->core_id) {
1284 return 1;
1285 }
1286 return 0;
1287 }
1288
1289 /* Create a sorted list of pmd's from the dp->poll_threads cmap. We can use
1290 * this list, as long as we do not go to quiescent state. */
1291 static void
1292 sorted_poll_thread_list(struct dp_netdev *dp,
1293 struct dp_netdev_pmd_thread ***list,
1294 size_t *n)
1295 {
1296 struct dp_netdev_pmd_thread *pmd;
1297 struct dp_netdev_pmd_thread **pmd_list;
1298 size_t k = 0, n_pmds;
1299
1300 n_pmds = cmap_count(&dp->poll_threads);
1301 pmd_list = xcalloc(n_pmds, sizeof *pmd_list);
1302
1303 CMAP_FOR_EACH (pmd, node, &dp->poll_threads) {
1304 if (k >= n_pmds) {
1305 break;
1306 }
1307 pmd_list[k++] = pmd;
1308 }
1309
1310 qsort(pmd_list, k, sizeof *pmd_list, compare_poll_thread_list);
1311
1312 *list = pmd_list;
1313 *n = k;
1314 }
1315
1316 static void
1317 dpif_netdev_subtable_lookup_get(struct unixctl_conn *conn, int argc OVS_UNUSED,
1318 const char *argv[] OVS_UNUSED,
1319 void *aux OVS_UNUSED)
1320 {
1321 /* Get a list of all lookup functions. */
1322 struct dpcls_subtable_lookup_info_t *lookup_funcs = NULL;
1323 int32_t count = dpcls_subtable_lookup_info_get(&lookup_funcs);
1324 if (count < 0) {
1325 unixctl_command_reply_error(conn, "error getting lookup names");
1326 return;
1327 }
1328
1329 /* Add all lookup functions to reply string. */
1330 struct ds reply = DS_EMPTY_INITIALIZER;
1331 ds_put_cstr(&reply, "Available lookup functions (priority : name)\n");
1332 for (int i = 0; i < count; i++) {
1333 ds_put_format(&reply, " %d : %s\n", lookup_funcs[i].prio,
1334 lookup_funcs[i].name);
1335 }
1336 unixctl_command_reply(conn, ds_cstr(&reply));
1337 ds_destroy(&reply);
1338 }
1339
1340 static void
1341 dpif_netdev_subtable_lookup_set(struct unixctl_conn *conn, int argc,
1342 const char *argv[], void *aux OVS_UNUSED)
1343 {
1344 /* This function requires 2 parameters (argv[1] and argv[2]) to execute.
1345 * argv[1] is subtable name
1346 * argv[2] is priority
1347 * argv[3] is the datapath name (optional if only 1 datapath exists)
1348 */
1349 const char *func_name = argv[1];
1350
1351 errno = 0;
1352 char *err_char;
1353 uint32_t new_prio = strtoul(argv[2], &err_char, 10);
1354 if (errno != 0 || new_prio > UINT8_MAX) {
1355 unixctl_command_reply_error(conn,
1356 "error converting priority, use integer in range 0-255\n");
1357 return;
1358 }
1359
1360 int32_t err = dpcls_subtable_set_prio(func_name, new_prio);
1361 if (err) {
1362 unixctl_command_reply_error(conn,
1363 "error, subtable lookup function not found\n");
1364 return;
1365 }
1366
1367 /* argv[3] is optional datapath instance. If no datapath name is provided
1368 * and only one datapath exists, the one existing datapath is reprobed.
1369 */
1370 ovs_mutex_lock(&dp_netdev_mutex);
1371 struct dp_netdev *dp = NULL;
1372
1373 if (argc == 4) {
1374 dp = shash_find_data(&dp_netdevs, argv[3]);
1375 } else if (shash_count(&dp_netdevs) == 1) {
1376 dp = shash_first(&dp_netdevs)->data;
1377 }
1378
1379 if (!dp) {
1380 ovs_mutex_unlock(&dp_netdev_mutex);
1381 unixctl_command_reply_error(conn,
1382 "please specify an existing datapath");
1383 return;
1384 }
1385
1386 /* Get PMD threads list, required to get DPCLS instances. */
1387 size_t n;
1388 uint32_t lookup_dpcls_changed = 0;
1389 uint32_t lookup_subtable_changed = 0;
1390 struct dp_netdev_pmd_thread **pmd_list;
1391 sorted_poll_thread_list(dp, &pmd_list, &n);
1392
1393 /* take port mutex as HMAP iters over them. */
1394 ovs_mutex_lock(&dp->port_mutex);
1395
1396 for (size_t i = 0; i < n; i++) {
1397 struct dp_netdev_pmd_thread *pmd = pmd_list[i];
1398 if (pmd->core_id == NON_PMD_CORE_ID) {
1399 continue;
1400 }
1401
1402 struct dp_netdev_port *port = NULL;
1403 HMAP_FOR_EACH (port, node, &dp->ports) {
1404 odp_port_t in_port = port->port_no;
1405 struct dpcls *cls = dp_netdev_pmd_lookup_dpcls(pmd, in_port);
1406 if (!cls) {
1407 continue;
1408 }
1409 uint32_t subtbl_changes = dpcls_subtable_lookup_reprobe(cls);
1410 if (subtbl_changes) {
1411 lookup_dpcls_changed++;
1412 lookup_subtable_changed += subtbl_changes;
1413 }
1414 }
1415 }
1416
1417 /* release port mutex before netdev mutex. */
1418 ovs_mutex_unlock(&dp->port_mutex);
1419 ovs_mutex_unlock(&dp_netdev_mutex);
1420
1421 struct ds reply = DS_EMPTY_INITIALIZER;
1422 ds_put_format(&reply,
1423 "Lookup priority change affected %d dpcls ports and %d subtables.\n",
1424 lookup_dpcls_changed, lookup_subtable_changed);
1425 const char *reply_str = ds_cstr(&reply);
1426 unixctl_command_reply(conn, reply_str);
1427 VLOG_INFO("%s", reply_str);
1428 ds_destroy(&reply);
1429 }
1430
1431 static void
1432 dpif_netdev_pmd_rebalance(struct unixctl_conn *conn, int argc,
1433 const char *argv[], void *aux OVS_UNUSED)
1434 {
1435 struct ds reply = DS_EMPTY_INITIALIZER;
1436 struct dp_netdev *dp = NULL;
1437
1438 ovs_mutex_lock(&dp_netdev_mutex);
1439
1440 if (argc == 2) {
1441 dp = shash_find_data(&dp_netdevs, argv[1]);
1442 } else if (shash_count(&dp_netdevs) == 1) {
1443 /* There's only one datapath */
1444 dp = shash_first(&dp_netdevs)->data;
1445 }
1446
1447 if (!dp) {
1448 ovs_mutex_unlock(&dp_netdev_mutex);
1449 unixctl_command_reply_error(conn,
1450 "please specify an existing datapath");
1451 return;
1452 }
1453
1454 dp_netdev_request_reconfigure(dp);
1455 ovs_mutex_unlock(&dp_netdev_mutex);
1456 ds_put_cstr(&reply, "pmd rxq rebalance requested.\n");
1457 unixctl_command_reply(conn, ds_cstr(&reply));
1458 ds_destroy(&reply);
1459 }
1460
1461 static void
1462 dpif_netdev_pmd_info(struct unixctl_conn *conn, int argc, const char *argv[],
1463 void *aux)
1464 {
1465 struct ds reply = DS_EMPTY_INITIALIZER;
1466 struct dp_netdev_pmd_thread **pmd_list;
1467 struct dp_netdev *dp = NULL;
1468 enum pmd_info_type type = *(enum pmd_info_type *) aux;
1469 unsigned int core_id;
1470 bool filter_on_pmd = false;
1471 size_t n;
1472
1473 ovs_mutex_lock(&dp_netdev_mutex);
1474
1475 while (argc > 1) {
1476 if (!strcmp(argv[1], "-pmd") && argc > 2) {
1477 if (str_to_uint(argv[2], 10, &core_id)) {
1478 filter_on_pmd = true;
1479 }
1480 argc -= 2;
1481 argv += 2;
1482 } else {
1483 dp = shash_find_data(&dp_netdevs, argv[1]);
1484 argc -= 1;
1485 argv += 1;
1486 }
1487 }
1488
1489 if (!dp) {
1490 if (shash_count(&dp_netdevs) == 1) {
1491 /* There's only one datapath */
1492 dp = shash_first(&dp_netdevs)->data;
1493 } else {
1494 ovs_mutex_unlock(&dp_netdev_mutex);
1495 unixctl_command_reply_error(conn,
1496 "please specify an existing datapath");
1497 return;
1498 }
1499 }
1500
1501 sorted_poll_thread_list(dp, &pmd_list, &n);
1502 for (size_t i = 0; i < n; i++) {
1503 struct dp_netdev_pmd_thread *pmd = pmd_list[i];
1504 if (!pmd) {
1505 break;
1506 }
1507 if (filter_on_pmd && pmd->core_id != core_id) {
1508 continue;
1509 }
1510 if (type == PMD_INFO_SHOW_RXQ) {
1511 pmd_info_show_rxq(&reply, pmd);
1512 } else if (type == PMD_INFO_CLEAR_STATS) {
1513 pmd_perf_stats_clear(&pmd->perf_stats);
1514 } else if (type == PMD_INFO_SHOW_STATS) {
1515 pmd_info_show_stats(&reply, pmd);
1516 } else if (type == PMD_INFO_PERF_SHOW) {
1517 pmd_info_show_perf(&reply, pmd, (struct pmd_perf_params *)aux);
1518 }
1519 }
1520 free(pmd_list);
1521
1522 ovs_mutex_unlock(&dp_netdev_mutex);
1523
1524 unixctl_command_reply(conn, ds_cstr(&reply));
1525 ds_destroy(&reply);
1526 }
1527
1528 static void
1529 pmd_perf_show_cmd(struct unixctl_conn *conn, int argc,
1530 const char *argv[],
1531 void *aux OVS_UNUSED)
1532 {
1533 struct pmd_perf_params par;
1534 long int it_hist = 0, ms_hist = 0;
1535 par.histograms = true;
1536
1537 while (argc > 1) {
1538 if (!strcmp(argv[1], "-nh")) {
1539 par.histograms = false;
1540 argc -= 1;
1541 argv += 1;
1542 } else if (!strcmp(argv[1], "-it") && argc > 2) {
1543 it_hist = strtol(argv[2], NULL, 10);
1544 if (it_hist < 0) {
1545 it_hist = 0;
1546 } else if (it_hist > HISTORY_LEN) {
1547 it_hist = HISTORY_LEN;
1548 }
1549 argc -= 2;
1550 argv += 2;
1551 } else if (!strcmp(argv[1], "-ms") && argc > 2) {
1552 ms_hist = strtol(argv[2], NULL, 10);
1553 if (ms_hist < 0) {
1554 ms_hist = 0;
1555 } else if (ms_hist > HISTORY_LEN) {
1556 ms_hist = HISTORY_LEN;
1557 }
1558 argc -= 2;
1559 argv += 2;
1560 } else {
1561 break;
1562 }
1563 }
1564 par.iter_hist_len = it_hist;
1565 par.ms_hist_len = ms_hist;
1566 par.command_type = PMD_INFO_PERF_SHOW;
1567 dpif_netdev_pmd_info(conn, argc, argv, &par);
1568 }
1569
1570 static void
1571 dpif_netdev_bond_show(struct unixctl_conn *conn, int argc,
1572 const char *argv[], void *aux OVS_UNUSED)
1573 {
1574 struct ds reply = DS_EMPTY_INITIALIZER;
1575 struct dp_netdev *dp = NULL;
1576
1577 ovs_mutex_lock(&dp_netdev_mutex);
1578 if (argc == 2) {
1579 dp = shash_find_data(&dp_netdevs, argv[1]);
1580 } else if (shash_count(&dp_netdevs) == 1) {
1581 /* There's only one datapath. */
1582 dp = shash_first(&dp_netdevs)->data;
1583 }
1584 if (!dp) {
1585 ovs_mutex_unlock(&dp_netdev_mutex);
1586 unixctl_command_reply_error(conn,
1587 "please specify an existing datapath");
1588 return;
1589 }
1590
1591 if (cmap_count(&dp->tx_bonds) > 0) {
1592 struct tx_bond *dp_bond_entry;
1593
1594 ds_put_cstr(&reply, "Bonds:\n");
1595 CMAP_FOR_EACH (dp_bond_entry, node, &dp->tx_bonds) {
1596 ds_put_format(&reply, " bond-id %"PRIu32":\n",
1597 dp_bond_entry->bond_id);
1598 for (int bucket = 0; bucket < BOND_BUCKETS; bucket++) {
1599 uint32_t member_id = odp_to_u32(
1600 dp_bond_entry->member_buckets[bucket].member_id);
1601 ds_put_format(&reply,
1602 " bucket %d - member %"PRIu32"\n",
1603 bucket, member_id);
1604 }
1605 }
1606 }
1607 ovs_mutex_unlock(&dp_netdev_mutex);
1608 unixctl_command_reply(conn, ds_cstr(&reply));
1609 ds_destroy(&reply);
1610 }
1611
1612 \f
1613 static int
1614 dpif_netdev_init(void)
1615 {
1616 static enum pmd_info_type show_aux = PMD_INFO_SHOW_STATS,
1617 clear_aux = PMD_INFO_CLEAR_STATS,
1618 poll_aux = PMD_INFO_SHOW_RXQ;
1619
1620 unixctl_command_register("dpif-netdev/pmd-stats-show", "[-pmd core] [dp]",
1621 0, 3, dpif_netdev_pmd_info,
1622 (void *)&show_aux);
1623 unixctl_command_register("dpif-netdev/pmd-stats-clear", "[-pmd core] [dp]",
1624 0, 3, dpif_netdev_pmd_info,
1625 (void *)&clear_aux);
1626 unixctl_command_register("dpif-netdev/pmd-rxq-show", "[-pmd core] [dp]",
1627 0, 3, dpif_netdev_pmd_info,
1628 (void *)&poll_aux);
1629 unixctl_command_register("dpif-netdev/pmd-perf-show",
1630 "[-nh] [-it iter-history-len]"
1631 " [-ms ms-history-len]"
1632 " [-pmd core] [dp]",
1633 0, 8, pmd_perf_show_cmd,
1634 NULL);
1635 unixctl_command_register("dpif-netdev/pmd-rxq-rebalance", "[dp]",
1636 0, 1, dpif_netdev_pmd_rebalance,
1637 NULL);
1638 unixctl_command_register("dpif-netdev/pmd-perf-log-set",
1639 "on|off [-b before] [-a after] [-e|-ne] "
1640 "[-us usec] [-q qlen]",
1641 0, 10, pmd_perf_log_set_cmd,
1642 NULL);
1643 unixctl_command_register("dpif-netdev/bond-show", "[dp]",
1644 0, 1, dpif_netdev_bond_show,
1645 NULL);
1646 unixctl_command_register("dpif-netdev/subtable-lookup-prio-set",
1647 "[lookup_func] [prio] [dp]",
1648 2, 3, dpif_netdev_subtable_lookup_set,
1649 NULL);
1650 unixctl_command_register("dpif-netdev/subtable-lookup-prio-get", "",
1651 0, 0, dpif_netdev_subtable_lookup_get,
1652 NULL);
1653 return 0;
1654 }
1655
1656 static int
1657 dpif_netdev_enumerate(struct sset *all_dps,
1658 const struct dpif_class *dpif_class)
1659 {
1660 struct shash_node *node;
1661
1662 ovs_mutex_lock(&dp_netdev_mutex);
1663 SHASH_FOR_EACH(node, &dp_netdevs) {
1664 struct dp_netdev *dp = node->data;
1665 if (dpif_class != dp->class) {
1666 /* 'dp_netdevs' contains both "netdev" and "dummy" dpifs.
1667 * If the class doesn't match, skip this dpif. */
1668 continue;
1669 }
1670 sset_add(all_dps, node->name);
1671 }
1672 ovs_mutex_unlock(&dp_netdev_mutex);
1673
1674 return 0;
1675 }
1676
1677 static bool
1678 dpif_netdev_class_is_dummy(const struct dpif_class *class)
1679 {
1680 return class != &dpif_netdev_class;
1681 }
1682
1683 static const char *
1684 dpif_netdev_port_open_type(const struct dpif_class *class, const char *type)
1685 {
1686 return strcmp(type, "internal") ? type
1687 : dpif_netdev_class_is_dummy(class) ? "dummy-internal"
1688 : "tap";
1689 }
1690
1691 static struct dpif *
1692 create_dpif_netdev(struct dp_netdev *dp)
1693 {
1694 uint16_t netflow_id = hash_string(dp->name, 0);
1695 struct dpif_netdev *dpif;
1696
1697 ovs_refcount_ref(&dp->ref_cnt);
1698
1699 dpif = xmalloc(sizeof *dpif);
1700 dpif_init(&dpif->dpif, dp->class, dp->name, netflow_id >> 8, netflow_id);
1701 dpif->dp = dp;
1702 dpif->last_port_seq = seq_read(dp->port_seq);
1703
1704 return &dpif->dpif;
1705 }
1706
1707 /* Choose an unused, non-zero port number and return it on success.
1708 * Return ODPP_NONE on failure. */
1709 static odp_port_t
1710 choose_port(struct dp_netdev *dp, const char *name)
1711 OVS_REQUIRES(dp->port_mutex)
1712 {
1713 uint32_t port_no;
1714
1715 if (dp->class != &dpif_netdev_class) {
1716 const char *p;
1717 int start_no = 0;
1718
1719 /* If the port name begins with "br", start the number search at
1720 * 100 to make writing tests easier. */
1721 if (!strncmp(name, "br", 2)) {
1722 start_no = 100;
1723 }
1724
1725 /* If the port name contains a number, try to assign that port number.
1726 * This can make writing unit tests easier because port numbers are
1727 * predictable. */
1728 for (p = name; *p != '\0'; p++) {
1729 if (isdigit((unsigned char) *p)) {
1730 port_no = start_no + strtol(p, NULL, 10);
1731 if (port_no > 0 && port_no != odp_to_u32(ODPP_NONE)
1732 && !dp_netdev_lookup_port(dp, u32_to_odp(port_no))) {
1733 return u32_to_odp(port_no);
1734 }
1735 break;
1736 }
1737 }
1738 }
1739
1740 for (port_no = 1; port_no <= UINT16_MAX; port_no++) {
1741 if (!dp_netdev_lookup_port(dp, u32_to_odp(port_no))) {
1742 return u32_to_odp(port_no);
1743 }
1744 }
1745
1746 return ODPP_NONE;
1747 }
1748
1749 static int
1750 create_dp_netdev(const char *name, const struct dpif_class *class,
1751 struct dp_netdev **dpp)
1752 OVS_REQUIRES(dp_netdev_mutex)
1753 {
1754 static struct ovsthread_once tsc_freq_check = OVSTHREAD_ONCE_INITIALIZER;
1755 struct dp_netdev *dp;
1756 int error;
1757
1758 /* Avoid estimating TSC frequency for dummy datapath to not slow down
1759 * unit tests. */
1760 if (!dpif_netdev_class_is_dummy(class)
1761 && ovsthread_once_start(&tsc_freq_check)) {
1762 pmd_perf_estimate_tsc_frequency();
1763 ovsthread_once_done(&tsc_freq_check);
1764 }
1765
1766 dp = xzalloc(sizeof *dp);
1767 shash_add(&dp_netdevs, name, dp);
1768
1769 *CONST_CAST(const struct dpif_class **, &dp->class) = class;
1770 *CONST_CAST(const char **, &dp->name) = xstrdup(name);
1771 ovs_refcount_init(&dp->ref_cnt);
1772 atomic_flag_clear(&dp->destroyed);
1773
1774 ovs_mutex_init_recursive(&dp->port_mutex);
1775 hmap_init(&dp->ports);
1776 dp->port_seq = seq_create();
1777 ovs_mutex_init(&dp->bond_mutex);
1778 cmap_init(&dp->tx_bonds);
1779
1780 fat_rwlock_init(&dp->upcall_rwlock);
1781
1782 dp->reconfigure_seq = seq_create();
1783 dp->last_reconfigure_seq = seq_read(dp->reconfigure_seq);
1784
1785 for (int i = 0; i < N_METER_LOCKS; ++i) {
1786 ovs_mutex_init_adaptive(&dp->meter_locks[i]);
1787 }
1788
1789 /* Disable upcalls by default. */
1790 dp_netdev_disable_upcall(dp);
1791 dp->upcall_aux = NULL;
1792 dp->upcall_cb = NULL;
1793
1794 dp->conntrack = conntrack_init();
1795
1796 atomic_init(&dp->emc_insert_min, DEFAULT_EM_FLOW_INSERT_MIN);
1797 atomic_init(&dp->tx_flush_interval, DEFAULT_TX_FLUSH_INTERVAL);
1798
1799 cmap_init(&dp->poll_threads);
1800 dp->pmd_rxq_assign_cyc = true;
1801
1802 ovs_mutex_init(&dp->tx_qid_pool_mutex);
1803 /* We need 1 Tx queue for each possible core + 1 for non-PMD threads. */
1804 dp->tx_qid_pool = id_pool_create(0, ovs_numa_get_n_cores() + 1);
1805
1806 ovs_mutex_init_recursive(&dp->non_pmd_mutex);
1807 ovsthread_key_create(&dp->per_pmd_key, NULL);
1808
1809 ovs_mutex_lock(&dp->port_mutex);
1810 /* non-PMD will be created before all other threads and will
1811 * allocate static_tx_qid = 0. */
1812 dp_netdev_set_nonpmd(dp);
1813
1814 error = do_add_port(dp, name, dpif_netdev_port_open_type(dp->class,
1815 "internal"),
1816 ODPP_LOCAL);
1817 ovs_mutex_unlock(&dp->port_mutex);
1818 if (error) {
1819 dp_netdev_free(dp);
1820 return error;
1821 }
1822
1823 dp->last_tnl_conf_seq = seq_read(tnl_conf_seq);
1824 *dpp = dp;
1825 return 0;
1826 }
1827
1828 static void
1829 dp_netdev_request_reconfigure(struct dp_netdev *dp)
1830 {
1831 seq_change(dp->reconfigure_seq);
1832 }
1833
1834 static bool
1835 dp_netdev_is_reconf_required(struct dp_netdev *dp)
1836 {
1837 return seq_read(dp->reconfigure_seq) != dp->last_reconfigure_seq;
1838 }
1839
1840 static int
1841 dpif_netdev_open(const struct dpif_class *class, const char *name,
1842 bool create, struct dpif **dpifp)
1843 {
1844 struct dp_netdev *dp;
1845 int error;
1846
1847 ovs_mutex_lock(&dp_netdev_mutex);
1848 dp = shash_find_data(&dp_netdevs, name);
1849 if (!dp) {
1850 error = create ? create_dp_netdev(name, class, &dp) : ENODEV;
1851 } else {
1852 error = (dp->class != class ? EINVAL
1853 : create ? EEXIST
1854 : 0);
1855 }
1856 if (!error) {
1857 *dpifp = create_dpif_netdev(dp);
1858 }
1859 ovs_mutex_unlock(&dp_netdev_mutex);
1860
1861 return error;
1862 }
1863
1864 static void
1865 dp_netdev_destroy_upcall_lock(struct dp_netdev *dp)
1866 OVS_NO_THREAD_SAFETY_ANALYSIS
1867 {
1868 /* Check that upcalls are disabled, i.e. that the rwlock is taken */
1869 ovs_assert(fat_rwlock_tryrdlock(&dp->upcall_rwlock));
1870
1871 /* Before freeing a lock we should release it */
1872 fat_rwlock_unlock(&dp->upcall_rwlock);
1873 fat_rwlock_destroy(&dp->upcall_rwlock);
1874 }
1875
1876 static void
1877 dp_delete_meter(struct dp_netdev *dp, uint32_t meter_id)
1878 OVS_REQUIRES(dp->meter_locks[meter_id % N_METER_LOCKS])
1879 {
1880 if (dp->meters[meter_id]) {
1881 free(dp->meters[meter_id]);
1882 dp->meters[meter_id] = NULL;
1883 }
1884 }
1885
1886 static uint32_t
1887 hash_bond_id(uint32_t bond_id)
1888 {
1889 return hash_int(bond_id, 0);
1890 }
1891
1892 /* Requires dp_netdev_mutex so that we can't get a new reference to 'dp'
1893 * through the 'dp_netdevs' shash while freeing 'dp'. */
1894 static void
1895 dp_netdev_free(struct dp_netdev *dp)
1896 OVS_REQUIRES(dp_netdev_mutex)
1897 {
1898 struct dp_netdev_port *port, *next;
1899 struct tx_bond *bond;
1900
1901 shash_find_and_delete(&dp_netdevs, dp->name);
1902
1903 ovs_mutex_lock(&dp->port_mutex);
1904 HMAP_FOR_EACH_SAFE (port, next, node, &dp->ports) {
1905 do_del_port(dp, port);
1906 }
1907 ovs_mutex_unlock(&dp->port_mutex);
1908
1909 ovs_mutex_lock(&dp->bond_mutex);
1910 CMAP_FOR_EACH (bond, node, &dp->tx_bonds) {
1911 cmap_remove(&dp->tx_bonds, &bond->node, hash_bond_id(bond->bond_id));
1912 ovsrcu_postpone(free, bond);
1913 }
1914 ovs_mutex_unlock(&dp->bond_mutex);
1915
1916 dp_netdev_destroy_all_pmds(dp, true);
1917 cmap_destroy(&dp->poll_threads);
1918
1919 ovs_mutex_destroy(&dp->tx_qid_pool_mutex);
1920 id_pool_destroy(dp->tx_qid_pool);
1921
1922 ovs_mutex_destroy(&dp->non_pmd_mutex);
1923 ovsthread_key_delete(dp->per_pmd_key);
1924
1925 conntrack_destroy(dp->conntrack);
1926
1927
1928 seq_destroy(dp->reconfigure_seq);
1929
1930 seq_destroy(dp->port_seq);
1931 hmap_destroy(&dp->ports);
1932 ovs_mutex_destroy(&dp->port_mutex);
1933
1934 cmap_destroy(&dp->tx_bonds);
1935 ovs_mutex_destroy(&dp->bond_mutex);
1936
1937 /* Upcalls must be disabled at this point */
1938 dp_netdev_destroy_upcall_lock(dp);
1939
1940 int i;
1941
1942 for (i = 0; i < MAX_METERS; ++i) {
1943 meter_lock(dp, i);
1944 dp_delete_meter(dp, i);
1945 meter_unlock(dp, i);
1946 }
1947 for (i = 0; i < N_METER_LOCKS; ++i) {
1948 ovs_mutex_destroy(&dp->meter_locks[i]);
1949 }
1950
1951 free(dp->pmd_cmask);
1952 free(CONST_CAST(char *, dp->name));
1953 free(dp);
1954 }
1955
1956 static void
1957 dp_netdev_unref(struct dp_netdev *dp)
1958 {
1959 if (dp) {
1960 /* Take dp_netdev_mutex so that, if dp->ref_cnt falls to zero, we can't
1961 * get a new reference to 'dp' through the 'dp_netdevs' shash. */
1962 ovs_mutex_lock(&dp_netdev_mutex);
1963 if (ovs_refcount_unref_relaxed(&dp->ref_cnt) == 1) {
1964 dp_netdev_free(dp);
1965 }
1966 ovs_mutex_unlock(&dp_netdev_mutex);
1967 }
1968 }
1969
1970 static void
1971 dpif_netdev_close(struct dpif *dpif)
1972 {
1973 struct dp_netdev *dp = get_dp_netdev(dpif);
1974
1975 dp_netdev_unref(dp);
1976 free(dpif);
1977 }
1978
1979 static int
1980 dpif_netdev_destroy(struct dpif *dpif)
1981 {
1982 struct dp_netdev *dp = get_dp_netdev(dpif);
1983
1984 if (!atomic_flag_test_and_set(&dp->destroyed)) {
1985 if (ovs_refcount_unref_relaxed(&dp->ref_cnt) == 1) {
1986 /* Can't happen: 'dpif' still owns a reference to 'dp'. */
1987 OVS_NOT_REACHED();
1988 }
1989 }
1990
1991 return 0;
1992 }
1993
1994 /* Add 'n' to the atomic variable 'var' non-atomically and using relaxed
1995 * load/store semantics. While the increment is not atomic, the load and
1996 * store operations are, making it impossible to read inconsistent values.
1997 *
1998 * This is used to update thread local stats counters. */
1999 static void
2000 non_atomic_ullong_add(atomic_ullong *var, unsigned long long n)
2001 {
2002 unsigned long long tmp;
2003
2004 atomic_read_relaxed(var, &tmp);
2005 tmp += n;
2006 atomic_store_relaxed(var, tmp);
2007 }
2008
2009 static int
2010 dpif_netdev_get_stats(const struct dpif *dpif, struct dpif_dp_stats *stats)
2011 {
2012 struct dp_netdev *dp = get_dp_netdev(dpif);
2013 struct dp_netdev_pmd_thread *pmd;
2014 uint64_t pmd_stats[PMD_N_STATS];
2015
2016 stats->n_flows = stats->n_hit = stats->n_missed = stats->n_lost = 0;
2017 CMAP_FOR_EACH (pmd, node, &dp->poll_threads) {
2018 stats->n_flows += cmap_count(&pmd->flow_table);
2019 pmd_perf_read_counters(&pmd->perf_stats, pmd_stats);
2020 stats->n_hit += pmd_stats[PMD_STAT_EXACT_HIT];
2021 stats->n_hit += pmd_stats[PMD_STAT_SMC_HIT];
2022 stats->n_hit += pmd_stats[PMD_STAT_MASKED_HIT];
2023 stats->n_missed += pmd_stats[PMD_STAT_MISS];
2024 stats->n_lost += pmd_stats[PMD_STAT_LOST];
2025 }
2026 stats->n_masks = UINT32_MAX;
2027 stats->n_mask_hit = UINT64_MAX;
2028
2029 return 0;
2030 }
2031
2032 static void
2033 dp_netdev_reload_pmd__(struct dp_netdev_pmd_thread *pmd)
2034 {
2035 if (pmd->core_id == NON_PMD_CORE_ID) {
2036 ovs_mutex_lock(&pmd->dp->non_pmd_mutex);
2037 ovs_mutex_lock(&pmd->port_mutex);
2038 pmd_load_cached_ports(pmd);
2039 ovs_mutex_unlock(&pmd->port_mutex);
2040 ovs_mutex_unlock(&pmd->dp->non_pmd_mutex);
2041 return;
2042 }
2043
2044 seq_change(pmd->reload_seq);
2045 atomic_store_explicit(&pmd->reload, true, memory_order_release);
2046 }
2047
2048 static uint32_t
2049 hash_port_no(odp_port_t port_no)
2050 {
2051 return hash_int(odp_to_u32(port_no), 0);
2052 }
2053
2054 static int
2055 port_create(const char *devname, const char *type,
2056 odp_port_t port_no, struct dp_netdev_port **portp)
2057 {
2058 struct dp_netdev_port *port;
2059 enum netdev_flags flags;
2060 struct netdev *netdev;
2061 int error;
2062
2063 *portp = NULL;
2064
2065 /* Open and validate network device. */
2066 error = netdev_open(devname, type, &netdev);
2067 if (error) {
2068 return error;
2069 }
2070 /* XXX reject non-Ethernet devices */
2071
2072 netdev_get_flags(netdev, &flags);
2073 if (flags & NETDEV_LOOPBACK) {
2074 VLOG_ERR("%s: cannot add a loopback device", devname);
2075 error = EINVAL;
2076 goto out;
2077 }
2078
2079 port = xzalloc(sizeof *port);
2080 port->port_no = port_no;
2081 port->netdev = netdev;
2082 port->type = xstrdup(type);
2083 port->sf = NULL;
2084 port->emc_enabled = true;
2085 port->need_reconfigure = true;
2086 ovs_mutex_init(&port->txq_used_mutex);
2087
2088 *portp = port;
2089
2090 return 0;
2091
2092 out:
2093 netdev_close(netdev);
2094 return error;
2095 }
2096
2097 static int
2098 do_add_port(struct dp_netdev *dp, const char *devname, const char *type,
2099 odp_port_t port_no)
2100 OVS_REQUIRES(dp->port_mutex)
2101 {
2102 struct netdev_saved_flags *sf;
2103 struct dp_netdev_port *port;
2104 int error;
2105
2106 /* Reject devices already in 'dp'. */
2107 if (!get_port_by_name(dp, devname, &port)) {
2108 return EEXIST;
2109 }
2110
2111 error = port_create(devname, type, port_no, &port);
2112 if (error) {
2113 return error;
2114 }
2115
2116 hmap_insert(&dp->ports, &port->node, hash_port_no(port_no));
2117 seq_change(dp->port_seq);
2118
2119 reconfigure_datapath(dp);
2120
2121 /* Check that port was successfully configured. */
2122 if (!dp_netdev_lookup_port(dp, port_no)) {
2123 return EINVAL;
2124 }
2125
2126 /* Updating device flags triggers an if_notifier, which triggers a bridge
2127 * reconfiguration and another attempt to add this port, leading to an
2128 * infinite loop if the device is configured incorrectly and cannot be
2129 * added. Setting the promisc mode after a successful reconfiguration,
2130 * since we already know that the device is somehow properly configured. */
2131 error = netdev_turn_flags_on(port->netdev, NETDEV_PROMISC, &sf);
2132 if (error) {
2133 VLOG_ERR("%s: cannot set promisc flag", devname);
2134 do_del_port(dp, port);
2135 return error;
2136 }
2137 port->sf = sf;
2138
2139 return 0;
2140 }
2141
2142 static int
2143 dpif_netdev_port_add(struct dpif *dpif, struct netdev *netdev,
2144 odp_port_t *port_nop)
2145 {
2146 struct dp_netdev *dp = get_dp_netdev(dpif);
2147 char namebuf[NETDEV_VPORT_NAME_BUFSIZE];
2148 const char *dpif_port;
2149 odp_port_t port_no;
2150 int error;
2151
2152 ovs_mutex_lock(&dp->port_mutex);
2153 dpif_port = netdev_vport_get_dpif_port(netdev, namebuf, sizeof namebuf);
2154 if (*port_nop != ODPP_NONE) {
2155 port_no = *port_nop;
2156 error = dp_netdev_lookup_port(dp, *port_nop) ? EBUSY : 0;
2157 } else {
2158 port_no = choose_port(dp, dpif_port);
2159 error = port_no == ODPP_NONE ? EFBIG : 0;
2160 }
2161 if (!error) {
2162 *port_nop = port_no;
2163 error = do_add_port(dp, dpif_port, netdev_get_type(netdev), port_no);
2164 }
2165 ovs_mutex_unlock(&dp->port_mutex);
2166
2167 return error;
2168 }
2169
2170 static int
2171 dpif_netdev_port_del(struct dpif *dpif, odp_port_t port_no)
2172 {
2173 struct dp_netdev *dp = get_dp_netdev(dpif);
2174 int error;
2175
2176 ovs_mutex_lock(&dp->port_mutex);
2177 if (port_no == ODPP_LOCAL) {
2178 error = EINVAL;
2179 } else {
2180 struct dp_netdev_port *port;
2181
2182 error = get_port_by_number(dp, port_no, &port);
2183 if (!error) {
2184 do_del_port(dp, port);
2185 }
2186 }
2187 ovs_mutex_unlock(&dp->port_mutex);
2188
2189 return error;
2190 }
2191
2192 static bool
2193 is_valid_port_number(odp_port_t port_no)
2194 {
2195 return port_no != ODPP_NONE;
2196 }
2197
2198 static struct dp_netdev_port *
2199 dp_netdev_lookup_port(const struct dp_netdev *dp, odp_port_t port_no)
2200 OVS_REQUIRES(dp->port_mutex)
2201 {
2202 struct dp_netdev_port *port;
2203
2204 HMAP_FOR_EACH_WITH_HASH (port, node, hash_port_no(port_no), &dp->ports) {
2205 if (port->port_no == port_no) {
2206 return port;
2207 }
2208 }
2209 return NULL;
2210 }
2211
2212 static int
2213 get_port_by_number(struct dp_netdev *dp,
2214 odp_port_t port_no, struct dp_netdev_port **portp)
2215 OVS_REQUIRES(dp->port_mutex)
2216 {
2217 if (!is_valid_port_number(port_no)) {
2218 *portp = NULL;
2219 return EINVAL;
2220 } else {
2221 *portp = dp_netdev_lookup_port(dp, port_no);
2222 return *portp ? 0 : ENODEV;
2223 }
2224 }
2225
2226 static void
2227 port_destroy(struct dp_netdev_port *port)
2228 {
2229 if (!port) {
2230 return;
2231 }
2232
2233 netdev_close(port->netdev);
2234 netdev_restore_flags(port->sf);
2235
2236 for (unsigned i = 0; i < port->n_rxq; i++) {
2237 netdev_rxq_close(port->rxqs[i].rx);
2238 }
2239 ovs_mutex_destroy(&port->txq_used_mutex);
2240 free(port->rxq_affinity_list);
2241 free(port->txq_used);
2242 free(port->rxqs);
2243 free(port->type);
2244 free(port);
2245 }
2246
2247 static int
2248 get_port_by_name(struct dp_netdev *dp,
2249 const char *devname, struct dp_netdev_port **portp)
2250 OVS_REQUIRES(dp->port_mutex)
2251 {
2252 struct dp_netdev_port *port;
2253
2254 HMAP_FOR_EACH (port, node, &dp->ports) {
2255 if (!strcmp(netdev_get_name(port->netdev), devname)) {
2256 *portp = port;
2257 return 0;
2258 }
2259 }
2260
2261 /* Callers of dpif_netdev_port_query_by_name() expect ENODEV for a non
2262 * existing port. */
2263 return ENODEV;
2264 }
2265
2266 /* Returns 'true' if there is a port with pmd netdev. */
2267 static bool
2268 has_pmd_port(struct dp_netdev *dp)
2269 OVS_REQUIRES(dp->port_mutex)
2270 {
2271 struct dp_netdev_port *port;
2272
2273 HMAP_FOR_EACH (port, node, &dp->ports) {
2274 if (netdev_is_pmd(port->netdev)) {
2275 return true;
2276 }
2277 }
2278
2279 return false;
2280 }
2281
2282 static void
2283 do_del_port(struct dp_netdev *dp, struct dp_netdev_port *port)
2284 OVS_REQUIRES(dp->port_mutex)
2285 {
2286 netdev_flow_flush(port->netdev);
2287 netdev_uninit_flow_api(port->netdev);
2288 hmap_remove(&dp->ports, &port->node);
2289 seq_change(dp->port_seq);
2290
2291 reconfigure_datapath(dp);
2292
2293 port_destroy(port);
2294 }
2295
2296 static void
2297 answer_port_query(const struct dp_netdev_port *port,
2298 struct dpif_port *dpif_port)
2299 {
2300 dpif_port->name = xstrdup(netdev_get_name(port->netdev));
2301 dpif_port->type = xstrdup(port->type);
2302 dpif_port->port_no = port->port_no;
2303 }
2304
2305 static int
2306 dpif_netdev_port_query_by_number(const struct dpif *dpif, odp_port_t port_no,
2307 struct dpif_port *dpif_port)
2308 {
2309 struct dp_netdev *dp = get_dp_netdev(dpif);
2310 struct dp_netdev_port *port;
2311 int error;
2312
2313 ovs_mutex_lock(&dp->port_mutex);
2314 error = get_port_by_number(dp, port_no, &port);
2315 if (!error && dpif_port) {
2316 answer_port_query(port, dpif_port);
2317 }
2318 ovs_mutex_unlock(&dp->port_mutex);
2319
2320 return error;
2321 }
2322
2323 static int
2324 dpif_netdev_port_query_by_name(const struct dpif *dpif, const char *devname,
2325 struct dpif_port *dpif_port)
2326 {
2327 struct dp_netdev *dp = get_dp_netdev(dpif);
2328 struct dp_netdev_port *port;
2329 int error;
2330
2331 ovs_mutex_lock(&dp->port_mutex);
2332 error = get_port_by_name(dp, devname, &port);
2333 if (!error && dpif_port) {
2334 answer_port_query(port, dpif_port);
2335 }
2336 ovs_mutex_unlock(&dp->port_mutex);
2337
2338 return error;
2339 }
2340
2341 static void
2342 dp_netdev_flow_free(struct dp_netdev_flow *flow)
2343 {
2344 dp_netdev_actions_free(dp_netdev_flow_get_actions(flow));
2345 free(flow->dp_extra_info);
2346 free(flow);
2347 }
2348
2349 static void dp_netdev_flow_unref(struct dp_netdev_flow *flow)
2350 {
2351 if (ovs_refcount_unref_relaxed(&flow->ref_cnt) == 1) {
2352 ovsrcu_postpone(dp_netdev_flow_free, flow);
2353 }
2354 }
2355
2356 static uint32_t
2357 dp_netdev_flow_hash(const ovs_u128 *ufid)
2358 {
2359 return ufid->u32[0];
2360 }
2361
2362 static inline struct dpcls *
2363 dp_netdev_pmd_lookup_dpcls(struct dp_netdev_pmd_thread *pmd,
2364 odp_port_t in_port)
2365 {
2366 struct dpcls *cls;
2367 uint32_t hash = hash_port_no(in_port);
2368 CMAP_FOR_EACH_WITH_HASH (cls, node, hash, &pmd->classifiers) {
2369 if (cls->in_port == in_port) {
2370 /* Port classifier exists already */
2371 return cls;
2372 }
2373 }
2374 return NULL;
2375 }
2376
2377 static inline struct dpcls *
2378 dp_netdev_pmd_find_dpcls(struct dp_netdev_pmd_thread *pmd,
2379 odp_port_t in_port)
2380 OVS_REQUIRES(pmd->flow_mutex)
2381 {
2382 struct dpcls *cls = dp_netdev_pmd_lookup_dpcls(pmd, in_port);
2383 uint32_t hash = hash_port_no(in_port);
2384
2385 if (!cls) {
2386 /* Create new classifier for in_port */
2387 cls = xmalloc(sizeof(*cls));
2388 dpcls_init(cls);
2389 cls->in_port = in_port;
2390 cmap_insert(&pmd->classifiers, &cls->node, hash);
2391 VLOG_DBG("Creating dpcls %p for in_port %d", cls, in_port);
2392 }
2393 return cls;
2394 }
2395
2396 #define MAX_FLOW_MARK (UINT32_MAX - 1)
2397 #define INVALID_FLOW_MARK 0
2398 /* Zero flow mark is used to indicate the HW to remove the mark. A packet
2399 * marked with zero mark is received in SW without a mark at all, so it
2400 * cannot be used as a valid mark.
2401 */
2402
2403 struct megaflow_to_mark_data {
2404 const struct cmap_node node;
2405 ovs_u128 mega_ufid;
2406 uint32_t mark;
2407 };
2408
2409 struct flow_mark {
2410 struct cmap megaflow_to_mark;
2411 struct cmap mark_to_flow;
2412 struct id_pool *pool;
2413 };
2414
2415 static struct flow_mark flow_mark = {
2416 .megaflow_to_mark = CMAP_INITIALIZER,
2417 .mark_to_flow = CMAP_INITIALIZER,
2418 };
2419
2420 static uint32_t
2421 flow_mark_alloc(void)
2422 {
2423 uint32_t mark;
2424
2425 if (!flow_mark.pool) {
2426 /* Haven't initiated yet, do it here */
2427 flow_mark.pool = id_pool_create(1, MAX_FLOW_MARK);
2428 }
2429
2430 if (id_pool_alloc_id(flow_mark.pool, &mark)) {
2431 return mark;
2432 }
2433
2434 return INVALID_FLOW_MARK;
2435 }
2436
2437 static void
2438 flow_mark_free(uint32_t mark)
2439 {
2440 id_pool_free_id(flow_mark.pool, mark);
2441 }
2442
2443 /* associate megaflow with a mark, which is a 1:1 mapping */
2444 static void
2445 megaflow_to_mark_associate(const ovs_u128 *mega_ufid, uint32_t mark)
2446 {
2447 size_t hash = dp_netdev_flow_hash(mega_ufid);
2448 struct megaflow_to_mark_data *data = xzalloc(sizeof(*data));
2449
2450 data->mega_ufid = *mega_ufid;
2451 data->mark = mark;
2452
2453 cmap_insert(&flow_mark.megaflow_to_mark,
2454 CONST_CAST(struct cmap_node *, &data->node), hash);
2455 }
2456
2457 /* disassociate meagaflow with a mark */
2458 static void
2459 megaflow_to_mark_disassociate(const ovs_u128 *mega_ufid)
2460 {
2461 size_t hash = dp_netdev_flow_hash(mega_ufid);
2462 struct megaflow_to_mark_data *data;
2463
2464 CMAP_FOR_EACH_WITH_HASH (data, node, hash, &flow_mark.megaflow_to_mark) {
2465 if (ovs_u128_equals(*mega_ufid, data->mega_ufid)) {
2466 cmap_remove(&flow_mark.megaflow_to_mark,
2467 CONST_CAST(struct cmap_node *, &data->node), hash);
2468 ovsrcu_postpone(free, data);
2469 return;
2470 }
2471 }
2472
2473 VLOG_WARN("Masked ufid "UUID_FMT" is not associated with a mark?\n",
2474 UUID_ARGS((struct uuid *)mega_ufid));
2475 }
2476
2477 static inline uint32_t
2478 megaflow_to_mark_find(const ovs_u128 *mega_ufid)
2479 {
2480 size_t hash = dp_netdev_flow_hash(mega_ufid);
2481 struct megaflow_to_mark_data *data;
2482
2483 CMAP_FOR_EACH_WITH_HASH (data, node, hash, &flow_mark.megaflow_to_mark) {
2484 if (ovs_u128_equals(*mega_ufid, data->mega_ufid)) {
2485 return data->mark;
2486 }
2487 }
2488
2489 VLOG_DBG("Mark id for ufid "UUID_FMT" was not found\n",
2490 UUID_ARGS((struct uuid *)mega_ufid));
2491 return INVALID_FLOW_MARK;
2492 }
2493
2494 /* associate mark with a flow, which is 1:N mapping */
2495 static void
2496 mark_to_flow_associate(const uint32_t mark, struct dp_netdev_flow *flow)
2497 {
2498 dp_netdev_flow_ref(flow);
2499
2500 cmap_insert(&flow_mark.mark_to_flow,
2501 CONST_CAST(struct cmap_node *, &flow->mark_node),
2502 hash_int(mark, 0));
2503 flow->mark = mark;
2504
2505 VLOG_DBG("Associated dp_netdev flow %p with mark %u mega_ufid "UUID_FMT,
2506 flow, mark, UUID_ARGS((struct uuid *) &flow->mega_ufid));
2507 }
2508
2509 static bool
2510 flow_mark_has_no_ref(uint32_t mark)
2511 {
2512 struct dp_netdev_flow *flow;
2513
2514 CMAP_FOR_EACH_WITH_HASH (flow, mark_node, hash_int(mark, 0),
2515 &flow_mark.mark_to_flow) {
2516 if (flow->mark == mark) {
2517 return false;
2518 }
2519 }
2520
2521 return true;
2522 }
2523
2524 static int
2525 mark_to_flow_disassociate(struct dp_netdev_pmd_thread *pmd,
2526 struct dp_netdev_flow *flow)
2527 {
2528 const char *dpif_type_str = dpif_normalize_type(pmd->dp->class->type);
2529 struct cmap_node *mark_node = CONST_CAST(struct cmap_node *,
2530 &flow->mark_node);
2531 uint32_t mark = flow->mark;
2532 int ret = 0;
2533
2534 /* INVALID_FLOW_MARK may mean that the flow has been disassociated or
2535 * never associated. */
2536 if (OVS_UNLIKELY(mark == INVALID_FLOW_MARK)) {
2537 return EINVAL;
2538 }
2539
2540 cmap_remove(&flow_mark.mark_to_flow, mark_node, hash_int(mark, 0));
2541 flow->mark = INVALID_FLOW_MARK;
2542
2543 /*
2544 * no flow is referencing the mark any more? If so, let's
2545 * remove the flow from hardware and free the mark.
2546 */
2547 if (flow_mark_has_no_ref(mark)) {
2548 struct netdev *port;
2549 odp_port_t in_port = flow->flow.in_port.odp_port;
2550
2551 port = netdev_ports_get(in_port, dpif_type_str);
2552 if (port) {
2553 /* Taking a global 'port_mutex' to fulfill thread safety
2554 * restrictions for the netdev-offload-dpdk module. */
2555 ovs_mutex_lock(&pmd->dp->port_mutex);
2556 ret = netdev_flow_del(port, &flow->mega_ufid, NULL);
2557 ovs_mutex_unlock(&pmd->dp->port_mutex);
2558 netdev_close(port);
2559 }
2560
2561 flow_mark_free(mark);
2562 VLOG_DBG("Freed flow mark %u mega_ufid "UUID_FMT, mark,
2563 UUID_ARGS((struct uuid *) &flow->mega_ufid));
2564
2565 megaflow_to_mark_disassociate(&flow->mega_ufid);
2566 }
2567 dp_netdev_flow_unref(flow);
2568
2569 return ret;
2570 }
2571
2572 static void
2573 flow_mark_flush(struct dp_netdev_pmd_thread *pmd)
2574 {
2575 struct dp_netdev_flow *flow;
2576
2577 CMAP_FOR_EACH (flow, mark_node, &flow_mark.mark_to_flow) {
2578 if (flow->pmd_id == pmd->core_id) {
2579 queue_netdev_flow_del(pmd, flow);
2580 }
2581 }
2582 }
2583
2584 static struct dp_netdev_flow *
2585 mark_to_flow_find(const struct dp_netdev_pmd_thread *pmd,
2586 const uint32_t mark)
2587 {
2588 struct dp_netdev_flow *flow;
2589
2590 CMAP_FOR_EACH_WITH_HASH (flow, mark_node, hash_int(mark, 0),
2591 &flow_mark.mark_to_flow) {
2592 if (flow->mark == mark && flow->pmd_id == pmd->core_id &&
2593 flow->dead == false) {
2594 return flow;
2595 }
2596 }
2597
2598 return NULL;
2599 }
2600
2601 static struct dp_flow_offload_item *
2602 dp_netdev_alloc_flow_offload(struct dp_netdev_pmd_thread *pmd,
2603 struct dp_netdev_flow *flow,
2604 int op)
2605 {
2606 struct dp_flow_offload_item *offload;
2607
2608 offload = xzalloc(sizeof(*offload));
2609 offload->pmd = pmd;
2610 offload->flow = flow;
2611 offload->op = op;
2612
2613 dp_netdev_flow_ref(flow);
2614 dp_netdev_pmd_try_ref(pmd);
2615
2616 return offload;
2617 }
2618
2619 static void
2620 dp_netdev_free_flow_offload(struct dp_flow_offload_item *offload)
2621 {
2622 dp_netdev_pmd_unref(offload->pmd);
2623 dp_netdev_flow_unref(offload->flow);
2624
2625 free(offload->actions);
2626 free(offload);
2627 }
2628
2629 static void
2630 dp_netdev_append_flow_offload(struct dp_flow_offload_item *offload)
2631 {
2632 ovs_mutex_lock(&dp_flow_offload.mutex);
2633 ovs_list_push_back(&dp_flow_offload.list, &offload->node);
2634 xpthread_cond_signal(&dp_flow_offload.cond);
2635 ovs_mutex_unlock(&dp_flow_offload.mutex);
2636 }
2637
2638 static int
2639 dp_netdev_flow_offload_del(struct dp_flow_offload_item *offload)
2640 {
2641 return mark_to_flow_disassociate(offload->pmd, offload->flow);
2642 }
2643
2644 /*
2645 * There are two flow offload operations here: addition and modification.
2646 *
2647 * For flow addition, this function does:
2648 * - allocate a new flow mark id
2649 * - perform hardware flow offload
2650 * - associate the flow mark with flow and mega flow
2651 *
2652 * For flow modification, both flow mark and the associations are still
2653 * valid, thus only item 2 needed.
2654 */
2655 static int
2656 dp_netdev_flow_offload_put(struct dp_flow_offload_item *offload)
2657 {
2658 struct dp_netdev_pmd_thread *pmd = offload->pmd;
2659 struct dp_netdev_flow *flow = offload->flow;
2660 odp_port_t in_port = flow->flow.in_port.odp_port;
2661 const char *dpif_type_str = dpif_normalize_type(pmd->dp->class->type);
2662 bool modification = offload->op == DP_NETDEV_FLOW_OFFLOAD_OP_MOD;
2663 struct offload_info info;
2664 struct netdev *port;
2665 uint32_t mark;
2666 int ret;
2667
2668 if (flow->dead) {
2669 return -1;
2670 }
2671
2672 if (modification) {
2673 mark = flow->mark;
2674 ovs_assert(mark != INVALID_FLOW_MARK);
2675 } else {
2676 /*
2677 * If a mega flow has already been offloaded (from other PMD
2678 * instances), do not offload it again.
2679 */
2680 mark = megaflow_to_mark_find(&flow->mega_ufid);
2681 if (mark != INVALID_FLOW_MARK) {
2682 VLOG_DBG("Flow has already been offloaded with mark %u\n", mark);
2683 if (flow->mark != INVALID_FLOW_MARK) {
2684 ovs_assert(flow->mark == mark);
2685 } else {
2686 mark_to_flow_associate(mark, flow);
2687 }
2688 return 0;
2689 }
2690
2691 mark = flow_mark_alloc();
2692 if (mark == INVALID_FLOW_MARK) {
2693 VLOG_ERR("Failed to allocate flow mark!\n");
2694 return -1;
2695 }
2696 }
2697 info.flow_mark = mark;
2698
2699 port = netdev_ports_get(in_port, dpif_type_str);
2700 if (!port || netdev_vport_is_vport_class(port->netdev_class)) {
2701 netdev_close(port);
2702 goto err_free;
2703 }
2704 /* Taking a global 'port_mutex' to fulfill thread safety restrictions for
2705 * the netdev-offload-dpdk module. */
2706 ovs_mutex_lock(&pmd->dp->port_mutex);
2707 ret = netdev_flow_put(port, &offload->match,
2708 CONST_CAST(struct nlattr *, offload->actions),
2709 offload->actions_len, &flow->mega_ufid, &info,
2710 NULL);
2711 ovs_mutex_unlock(&pmd->dp->port_mutex);
2712 netdev_close(port);
2713
2714 if (ret) {
2715 goto err_free;
2716 }
2717
2718 if (!modification) {
2719 megaflow_to_mark_associate(&flow->mega_ufid, mark);
2720 mark_to_flow_associate(mark, flow);
2721 }
2722 return 0;
2723
2724 err_free:
2725 if (!modification) {
2726 flow_mark_free(mark);
2727 } else {
2728 mark_to_flow_disassociate(pmd, flow);
2729 }
2730 return -1;
2731 }
2732
2733 static void *
2734 dp_netdev_flow_offload_main(void *data OVS_UNUSED)
2735 {
2736 struct dp_flow_offload_item *offload;
2737 struct ovs_list *list;
2738 const char *op;
2739 int ret;
2740
2741 for (;;) {
2742 ovs_mutex_lock(&dp_flow_offload.mutex);
2743 if (ovs_list_is_empty(&dp_flow_offload.list)) {
2744 ovsrcu_quiesce_start();
2745 ovs_mutex_cond_wait(&dp_flow_offload.cond,
2746 &dp_flow_offload.mutex);
2747 ovsrcu_quiesce_end();
2748 }
2749 list = ovs_list_pop_front(&dp_flow_offload.list);
2750 offload = CONTAINER_OF(list, struct dp_flow_offload_item, node);
2751 ovs_mutex_unlock(&dp_flow_offload.mutex);
2752
2753 switch (offload->op) {
2754 case DP_NETDEV_FLOW_OFFLOAD_OP_ADD:
2755 op = "add";
2756 ret = dp_netdev_flow_offload_put(offload);
2757 break;
2758 case DP_NETDEV_FLOW_OFFLOAD_OP_MOD:
2759 op = "modify";
2760 ret = dp_netdev_flow_offload_put(offload);
2761 break;
2762 case DP_NETDEV_FLOW_OFFLOAD_OP_DEL:
2763 op = "delete";
2764 ret = dp_netdev_flow_offload_del(offload);
2765 break;
2766 default:
2767 OVS_NOT_REACHED();
2768 }
2769
2770 VLOG_DBG("%s to %s netdev flow "UUID_FMT,
2771 ret == 0 ? "succeed" : "failed", op,
2772 UUID_ARGS((struct uuid *) &offload->flow->mega_ufid));
2773 dp_netdev_free_flow_offload(offload);
2774 ovsrcu_quiesce();
2775 }
2776
2777 return NULL;
2778 }
2779
2780 static void
2781 queue_netdev_flow_del(struct dp_netdev_pmd_thread *pmd,
2782 struct dp_netdev_flow *flow)
2783 {
2784 struct dp_flow_offload_item *offload;
2785
2786 if (ovsthread_once_start(&offload_thread_once)) {
2787 xpthread_cond_init(&dp_flow_offload.cond, NULL);
2788 ovs_thread_create("dp_netdev_flow_offload",
2789 dp_netdev_flow_offload_main, NULL);
2790 ovsthread_once_done(&offload_thread_once);
2791 }
2792
2793 offload = dp_netdev_alloc_flow_offload(pmd, flow,
2794 DP_NETDEV_FLOW_OFFLOAD_OP_DEL);
2795 dp_netdev_append_flow_offload(offload);
2796 }
2797
2798 static void
2799 queue_netdev_flow_put(struct dp_netdev_pmd_thread *pmd,
2800 struct dp_netdev_flow *flow, struct match *match,
2801 const struct nlattr *actions, size_t actions_len)
2802 {
2803 struct dp_flow_offload_item *offload;
2804 int op;
2805
2806 if (!netdev_is_flow_api_enabled()) {
2807 return;
2808 }
2809
2810 if (ovsthread_once_start(&offload_thread_once)) {
2811 xpthread_cond_init(&dp_flow_offload.cond, NULL);
2812 ovs_thread_create("dp_netdev_flow_offload",
2813 dp_netdev_flow_offload_main, NULL);
2814 ovsthread_once_done(&offload_thread_once);
2815 }
2816
2817 if (flow->mark != INVALID_FLOW_MARK) {
2818 op = DP_NETDEV_FLOW_OFFLOAD_OP_MOD;
2819 } else {
2820 op = DP_NETDEV_FLOW_OFFLOAD_OP_ADD;
2821 }
2822 offload = dp_netdev_alloc_flow_offload(pmd, flow, op);
2823 offload->match = *match;
2824 offload->actions = xmalloc(actions_len);
2825 memcpy(offload->actions, actions, actions_len);
2826 offload->actions_len = actions_len;
2827
2828 dp_netdev_append_flow_offload(offload);
2829 }
2830
2831 static void
2832 dp_netdev_pmd_remove_flow(struct dp_netdev_pmd_thread *pmd,
2833 struct dp_netdev_flow *flow)
2834 OVS_REQUIRES(pmd->flow_mutex)
2835 {
2836 struct cmap_node *node = CONST_CAST(struct cmap_node *, &flow->node);
2837 struct dpcls *cls;
2838 odp_port_t in_port = flow->flow.in_port.odp_port;
2839
2840 cls = dp_netdev_pmd_lookup_dpcls(pmd, in_port);
2841 ovs_assert(cls != NULL);
2842 dpcls_remove(cls, &flow->cr);
2843 cmap_remove(&pmd->flow_table, node, dp_netdev_flow_hash(&flow->ufid));
2844 if (flow->mark != INVALID_FLOW_MARK) {
2845 queue_netdev_flow_del(pmd, flow);
2846 }
2847 flow->dead = true;
2848
2849 dp_netdev_flow_unref(flow);
2850 }
2851
2852 static void
2853 dp_netdev_pmd_flow_flush(struct dp_netdev_pmd_thread *pmd)
2854 {
2855 struct dp_netdev_flow *netdev_flow;
2856
2857 ovs_mutex_lock(&pmd->flow_mutex);
2858 CMAP_FOR_EACH (netdev_flow, node, &pmd->flow_table) {
2859 dp_netdev_pmd_remove_flow(pmd, netdev_flow);
2860 }
2861 ovs_mutex_unlock(&pmd->flow_mutex);
2862 }
2863
2864 static int
2865 dpif_netdev_flow_flush(struct dpif *dpif)
2866 {
2867 struct dp_netdev *dp = get_dp_netdev(dpif);
2868 struct dp_netdev_pmd_thread *pmd;
2869
2870 CMAP_FOR_EACH (pmd, node, &dp->poll_threads) {
2871 dp_netdev_pmd_flow_flush(pmd);
2872 }
2873
2874 return 0;
2875 }
2876
2877 struct dp_netdev_port_state {
2878 struct hmap_position position;
2879 char *name;
2880 };
2881
2882 static int
2883 dpif_netdev_port_dump_start(const struct dpif *dpif OVS_UNUSED, void **statep)
2884 {
2885 *statep = xzalloc(sizeof(struct dp_netdev_port_state));
2886 return 0;
2887 }
2888
2889 static int
2890 dpif_netdev_port_dump_next(const struct dpif *dpif, void *state_,
2891 struct dpif_port *dpif_port)
2892 {
2893 struct dp_netdev_port_state *state = state_;
2894 struct dp_netdev *dp = get_dp_netdev(dpif);
2895 struct hmap_node *node;
2896 int retval;
2897
2898 ovs_mutex_lock(&dp->port_mutex);
2899 node = hmap_at_position(&dp->ports, &state->position);
2900 if (node) {
2901 struct dp_netdev_port *port;
2902
2903 port = CONTAINER_OF(node, struct dp_netdev_port, node);
2904
2905 free(state->name);
2906 state->name = xstrdup(netdev_get_name(port->netdev));
2907 dpif_port->name = state->name;
2908 dpif_port->type = port->type;
2909 dpif_port->port_no = port->port_no;
2910
2911 retval = 0;
2912 } else {
2913 retval = EOF;
2914 }
2915 ovs_mutex_unlock(&dp->port_mutex);
2916
2917 return retval;
2918 }
2919
2920 static int
2921 dpif_netdev_port_dump_done(const struct dpif *dpif OVS_UNUSED, void *state_)
2922 {
2923 struct dp_netdev_port_state *state = state_;
2924 free(state->name);
2925 free(state);
2926 return 0;
2927 }
2928
2929 static int
2930 dpif_netdev_port_poll(const struct dpif *dpif_, char **devnamep OVS_UNUSED)
2931 {
2932 struct dpif_netdev *dpif = dpif_netdev_cast(dpif_);
2933 uint64_t new_port_seq;
2934 int error;
2935
2936 new_port_seq = seq_read(dpif->dp->port_seq);
2937 if (dpif->last_port_seq != new_port_seq) {
2938 dpif->last_port_seq = new_port_seq;
2939 error = ENOBUFS;
2940 } else {
2941 error = EAGAIN;
2942 }
2943
2944 return error;
2945 }
2946
2947 static void
2948 dpif_netdev_port_poll_wait(const struct dpif *dpif_)
2949 {
2950 struct dpif_netdev *dpif = dpif_netdev_cast(dpif_);
2951
2952 seq_wait(dpif->dp->port_seq, dpif->last_port_seq);
2953 }
2954
2955 static struct dp_netdev_flow *
2956 dp_netdev_flow_cast(const struct dpcls_rule *cr)
2957 {
2958 return cr ? CONTAINER_OF(cr, struct dp_netdev_flow, cr) : NULL;
2959 }
2960
2961 static bool dp_netdev_flow_ref(struct dp_netdev_flow *flow)
2962 {
2963 return ovs_refcount_try_ref_rcu(&flow->ref_cnt);
2964 }
2965
2966 /* netdev_flow_key utilities.
2967 *
2968 * netdev_flow_key is basically a miniflow. We use these functions
2969 * (netdev_flow_key_clone, netdev_flow_key_equal, ...) instead of the miniflow
2970 * functions (miniflow_clone_inline, miniflow_equal, ...), because:
2971 *
2972 * - Since we are dealing exclusively with miniflows created by
2973 * miniflow_extract(), if the map is different the miniflow is different.
2974 * Therefore we can be faster by comparing the map and the miniflow in a
2975 * single memcmp().
2976 * - These functions can be inlined by the compiler. */
2977
2978 /* Given the number of bits set in miniflow's maps, returns the size of the
2979 * 'netdev_flow_key.mf' */
2980 static inline size_t
2981 netdev_flow_key_size(size_t flow_u64s)
2982 {
2983 return sizeof(struct miniflow) + MINIFLOW_VALUES_SIZE(flow_u64s);
2984 }
2985
2986 static inline bool
2987 netdev_flow_key_equal(const struct netdev_flow_key *a,
2988 const struct netdev_flow_key *b)
2989 {
2990 /* 'b->len' may be not set yet. */
2991 return a->hash == b->hash && !memcmp(&a->mf, &b->mf, a->len);
2992 }
2993
2994 /* Used to compare 'netdev_flow_key' in the exact match cache to a miniflow.
2995 * The maps are compared bitwise, so both 'key->mf' and 'mf' must have been
2996 * generated by miniflow_extract. */
2997 static inline bool
2998 netdev_flow_key_equal_mf(const struct netdev_flow_key *key,
2999 const struct miniflow *mf)
3000 {
3001 return !memcmp(&key->mf, mf, key->len);
3002 }
3003
3004 static inline void
3005 netdev_flow_key_clone(struct netdev_flow_key *dst,
3006 const struct netdev_flow_key *src)
3007 {
3008 memcpy(dst, src,
3009 offsetof(struct netdev_flow_key, mf) + src->len);
3010 }
3011
3012 /* Initialize a netdev_flow_key 'mask' from 'match'. */
3013 static inline void
3014 netdev_flow_mask_init(struct netdev_flow_key *mask,
3015 const struct match *match)
3016 {
3017 uint64_t *dst = miniflow_values(&mask->mf);
3018 struct flowmap fmap;
3019 uint32_t hash = 0;
3020 size_t idx;
3021
3022 /* Only check masks that make sense for the flow. */
3023 flow_wc_map(&match->flow, &fmap);
3024 flowmap_init(&mask->mf.map);
3025
3026 FLOWMAP_FOR_EACH_INDEX(idx, fmap) {
3027 uint64_t mask_u64 = flow_u64_value(&match->wc.masks, idx);
3028
3029 if (mask_u64) {
3030 flowmap_set(&mask->mf.map, idx, 1);
3031 *dst++ = mask_u64;
3032 hash = hash_add64(hash, mask_u64);
3033 }
3034 }
3035
3036 map_t map;
3037
3038 FLOWMAP_FOR_EACH_MAP (map, mask->mf.map) {
3039 hash = hash_add64(hash, map);
3040 }
3041
3042 size_t n = dst - miniflow_get_values(&mask->mf);
3043
3044 mask->hash = hash_finish(hash, n * 8);
3045 mask->len = netdev_flow_key_size(n);
3046 }
3047
3048 /* Initializes 'dst' as a copy of 'flow' masked with 'mask'. */
3049 static inline void
3050 netdev_flow_key_init_masked(struct netdev_flow_key *dst,
3051 const struct flow *flow,
3052 const struct netdev_flow_key *mask)
3053 {
3054 uint64_t *dst_u64 = miniflow_values(&dst->mf);
3055 const uint64_t *mask_u64 = miniflow_get_values(&mask->mf);
3056 uint32_t hash = 0;
3057 uint64_t value;
3058
3059 dst->len = mask->len;
3060 dst->mf = mask->mf; /* Copy maps. */
3061
3062 FLOW_FOR_EACH_IN_MAPS(value, flow, mask->mf.map) {
3063 *dst_u64 = value & *mask_u64++;
3064 hash = hash_add64(hash, *dst_u64++);
3065 }
3066 dst->hash = hash_finish(hash,
3067 (dst_u64 - miniflow_get_values(&dst->mf)) * 8);
3068 }
3069
3070 static inline bool
3071 emc_entry_alive(struct emc_entry *ce)
3072 {
3073 return ce->flow && !ce->flow->dead;
3074 }
3075
3076 static void
3077 emc_clear_entry(struct emc_entry *ce)
3078 {
3079 if (ce->flow) {
3080 dp_netdev_flow_unref(ce->flow);
3081 ce->flow = NULL;
3082 }
3083 }
3084
3085 static inline void
3086 emc_change_entry(struct emc_entry *ce, struct dp_netdev_flow *flow,
3087 const struct netdev_flow_key *key)
3088 {
3089 if (ce->flow != flow) {
3090 if (ce->flow) {
3091 dp_netdev_flow_unref(ce->flow);
3092 }
3093
3094 if (dp_netdev_flow_ref(flow)) {
3095 ce->flow = flow;
3096 } else {
3097 ce->flow = NULL;
3098 }
3099 }
3100 if (key) {
3101 netdev_flow_key_clone(&ce->key, key);
3102 }
3103 }
3104
3105 static inline void
3106 emc_insert(struct emc_cache *cache, const struct netdev_flow_key *key,
3107 struct dp_netdev_flow *flow)
3108 {
3109 struct emc_entry *to_be_replaced = NULL;
3110 struct emc_entry *current_entry;
3111
3112 EMC_FOR_EACH_POS_WITH_HASH(cache, current_entry, key->hash) {
3113 if (netdev_flow_key_equal(&current_entry->key, key)) {
3114 /* We found the entry with the 'mf' miniflow */
3115 emc_change_entry(current_entry, flow, NULL);
3116 return;
3117 }
3118
3119 /* Replacement policy: put the flow in an empty (not alive) entry, or
3120 * in the first entry where it can be */
3121 if (!to_be_replaced
3122 || (emc_entry_alive(to_be_replaced)
3123 && !emc_entry_alive(current_entry))
3124 || current_entry->key.hash < to_be_replaced->key.hash) {
3125 to_be_replaced = current_entry;
3126 }
3127 }
3128 /* We didn't find the miniflow in the cache.
3129 * The 'to_be_replaced' entry is where the new flow will be stored */
3130
3131 emc_change_entry(to_be_replaced, flow, key);
3132 }
3133
3134 static inline void
3135 emc_probabilistic_insert(struct dp_netdev_pmd_thread *pmd,
3136 const struct netdev_flow_key *key,
3137 struct dp_netdev_flow *flow)
3138 {
3139 /* Insert an entry into the EMC based on probability value 'min'. By
3140 * default the value is UINT32_MAX / 100 which yields an insertion
3141 * probability of 1/100 ie. 1% */
3142
3143 uint32_t min = pmd->ctx.emc_insert_min;
3144
3145 if (min && random_uint32() <= min) {
3146 emc_insert(&(pmd->flow_cache).emc_cache, key, flow);
3147 }
3148 }
3149
3150 static inline struct dp_netdev_flow *
3151 emc_lookup(struct emc_cache *cache, const struct netdev_flow_key *key)
3152 {
3153 struct emc_entry *current_entry;
3154
3155 EMC_FOR_EACH_POS_WITH_HASH(cache, current_entry, key->hash) {
3156 if (current_entry->key.hash == key->hash
3157 && emc_entry_alive(current_entry)
3158 && netdev_flow_key_equal_mf(&current_entry->key, &key->mf)) {
3159
3160 /* We found the entry with the 'key->mf' miniflow */
3161 return current_entry->flow;
3162 }
3163 }
3164
3165 return NULL;
3166 }
3167
3168 static inline const struct cmap_node *
3169 smc_entry_get(struct dp_netdev_pmd_thread *pmd, const uint32_t hash)
3170 {
3171 struct smc_cache *cache = &(pmd->flow_cache).smc_cache;
3172 struct smc_bucket *bucket = &cache->buckets[hash & SMC_MASK];
3173 uint16_t sig = hash >> 16;
3174 uint16_t index = UINT16_MAX;
3175
3176 for (int i = 0; i < SMC_ENTRY_PER_BUCKET; i++) {
3177 if (bucket->sig[i] == sig) {
3178 index = bucket->flow_idx[i];
3179 break;
3180 }
3181 }
3182 if (index != UINT16_MAX) {
3183 return cmap_find_by_index(&pmd->flow_table, index);
3184 }
3185 return NULL;
3186 }
3187
3188 static void
3189 smc_clear_entry(struct smc_bucket *b, int idx)
3190 {
3191 b->flow_idx[idx] = UINT16_MAX;
3192 }
3193
3194 /* Insert the flow_table index into SMC. Insertion may fail when 1) SMC is
3195 * turned off, 2) the flow_table index is larger than uint16_t can handle.
3196 * If there is already an SMC entry having same signature, the index will be
3197 * updated. If there is no existing entry, but an empty entry is available,
3198 * the empty entry will be taken. If no empty entry or existing same signature,
3199 * a random entry from the hashed bucket will be picked. */
3200 static inline void
3201 smc_insert(struct dp_netdev_pmd_thread *pmd,
3202 const struct netdev_flow_key *key,
3203 uint32_t hash)
3204 {
3205 struct smc_cache *smc_cache = &(pmd->flow_cache).smc_cache;
3206 struct smc_bucket *bucket = &smc_cache->buckets[key->hash & SMC_MASK];
3207 uint16_t index;
3208 uint32_t cmap_index;
3209 bool smc_enable_db;
3210 int i;
3211
3212 atomic_read_relaxed(&pmd->dp->smc_enable_db, &smc_enable_db);
3213 if (!smc_enable_db) {
3214 return;
3215 }
3216
3217 cmap_index = cmap_find_index(&pmd->flow_table, hash);
3218 index = (cmap_index >= UINT16_MAX) ? UINT16_MAX : (uint16_t)cmap_index;
3219
3220 /* If the index is larger than SMC can handle (uint16_t), we don't
3221 * insert */
3222 if (index == UINT16_MAX) {
3223 return;
3224 }
3225
3226 /* If an entry with same signature already exists, update the index */
3227 uint16_t sig = key->hash >> 16;
3228 for (i = 0; i < SMC_ENTRY_PER_BUCKET; i++) {
3229 if (bucket->sig[i] == sig) {
3230 bucket->flow_idx[i] = index;
3231 return;
3232 }
3233 }
3234 /* If there is an empty entry, occupy it. */
3235 for (i = 0; i < SMC_ENTRY_PER_BUCKET; i++) {
3236 if (bucket->flow_idx[i] == UINT16_MAX) {
3237 bucket->sig[i] = sig;
3238 bucket->flow_idx[i] = index;
3239 return;
3240 }
3241 }
3242 /* Otherwise, pick a random entry. */
3243 i = random_uint32() % SMC_ENTRY_PER_BUCKET;
3244 bucket->sig[i] = sig;
3245 bucket->flow_idx[i] = index;
3246 }
3247
3248 static struct dp_netdev_flow *
3249 dp_netdev_pmd_lookup_flow(struct dp_netdev_pmd_thread *pmd,
3250 const struct netdev_flow_key *key,
3251 int *lookup_num_p)
3252 {
3253 struct dpcls *cls;
3254 struct dpcls_rule *rule;
3255 odp_port_t in_port = u32_to_odp(MINIFLOW_GET_U32(&key->mf,
3256 in_port.odp_port));
3257 struct dp_netdev_flow *netdev_flow = NULL;
3258
3259 cls = dp_netdev_pmd_lookup_dpcls(pmd, in_port);
3260 if (OVS_LIKELY(cls)) {
3261 dpcls_lookup(cls, &key, &rule, 1, lookup_num_p);
3262 netdev_flow = dp_netdev_flow_cast(rule);
3263 }
3264 return netdev_flow;
3265 }
3266
3267 static struct dp_netdev_flow *
3268 dp_netdev_pmd_find_flow(const struct dp_netdev_pmd_thread *pmd,
3269 const ovs_u128 *ufidp, const struct nlattr *key,
3270 size_t key_len)
3271 {
3272 struct dp_netdev_flow *netdev_flow;
3273 struct flow flow;
3274 ovs_u128 ufid;
3275
3276 /* If a UFID is not provided, determine one based on the key. */
3277 if (!ufidp && key && key_len
3278 && !dpif_netdev_flow_from_nlattrs(key, key_len, &flow, false)) {
3279 odp_flow_key_hash(&flow, sizeof flow, &ufid);
3280 ufidp = &ufid;
3281 }
3282
3283 if (ufidp) {
3284 CMAP_FOR_EACH_WITH_HASH (netdev_flow, node, dp_netdev_flow_hash(ufidp),
3285 &pmd->flow_table) {
3286 if (ovs_u128_equals(netdev_flow->ufid, *ufidp)) {
3287 return netdev_flow;
3288 }
3289 }
3290 }
3291
3292 return NULL;
3293 }
3294
3295 static void
3296 dp_netdev_flow_set_last_stats_attrs(struct dp_netdev_flow *netdev_flow,
3297 const struct dpif_flow_stats *stats,
3298 const struct dpif_flow_attrs *attrs,
3299 int result)
3300 {
3301 struct dp_netdev_flow_stats *last_stats = &netdev_flow->last_stats;
3302 struct dp_netdev_flow_attrs *last_attrs = &netdev_flow->last_attrs;
3303
3304 atomic_store_relaxed(&netdev_flow->netdev_flow_get_result, result);
3305 if (result) {
3306 return;
3307 }
3308
3309 atomic_store_relaxed(&last_stats->used, stats->used);
3310 atomic_store_relaxed(&last_stats->packet_count, stats->n_packets);
3311 atomic_store_relaxed(&last_stats->byte_count, stats->n_bytes);
3312 atomic_store_relaxed(&last_stats->tcp_flags, stats->tcp_flags);
3313
3314 atomic_store_relaxed(&last_attrs->offloaded, attrs->offloaded);
3315 atomic_store_relaxed(&last_attrs->dp_layer, attrs->dp_layer);
3316
3317 }
3318
3319 static void
3320 dp_netdev_flow_get_last_stats_attrs(struct dp_netdev_flow *netdev_flow,
3321 struct dpif_flow_stats *stats,
3322 struct dpif_flow_attrs *attrs,
3323 int *result)
3324 {
3325 struct dp_netdev_flow_stats *last_stats = &netdev_flow->last_stats;
3326 struct dp_netdev_flow_attrs *last_attrs = &netdev_flow->last_attrs;
3327
3328 atomic_read_relaxed(&netdev_flow->netdev_flow_get_result, result);
3329 if (*result) {
3330 return;
3331 }
3332
3333 atomic_read_relaxed(&last_stats->used, &stats->used);
3334 atomic_read_relaxed(&last_stats->packet_count, &stats->n_packets);
3335 atomic_read_relaxed(&last_stats->byte_count, &stats->n_bytes);
3336 atomic_read_relaxed(&last_stats->tcp_flags, &stats->tcp_flags);
3337
3338 atomic_read_relaxed(&last_attrs->offloaded, &attrs->offloaded);
3339 atomic_read_relaxed(&last_attrs->dp_layer, &attrs->dp_layer);
3340 }
3341
3342 static bool
3343 dpif_netdev_get_flow_offload_status(const struct dp_netdev *dp,
3344 struct dp_netdev_flow *netdev_flow,
3345 struct dpif_flow_stats *stats,
3346 struct dpif_flow_attrs *attrs)
3347 {
3348 uint64_t act_buf[1024 / 8];
3349 struct nlattr *actions;
3350 struct netdev *netdev;
3351 struct match match;
3352 struct ofpbuf buf;
3353
3354 int ret = 0;
3355
3356 if (!netdev_is_flow_api_enabled()) {
3357 return false;
3358 }
3359
3360 netdev = netdev_ports_get(netdev_flow->flow.in_port.odp_port,
3361 dpif_normalize_type(dp->class->type));
3362 if (!netdev) {
3363 return false;
3364 }
3365 ofpbuf_use_stack(&buf, &act_buf, sizeof act_buf);
3366 /* Taking a global 'port_mutex' to fulfill thread safety
3367 * restrictions for the netdev-offload-dpdk module.
3368 *
3369 * XXX: Main thread will try to pause/stop all revalidators during datapath
3370 * reconfiguration via datapath purge callback (dp_purge_cb) while
3371 * holding 'dp->port_mutex'. So we're not waiting for mutex here.
3372 * Otherwise, deadlock is possible, bcause revalidators might sleep
3373 * waiting for the main thread to release the lock and main thread
3374 * will wait for them to stop processing.
3375 * This workaround might make statistics less accurate. Especially
3376 * for flow deletion case, since there will be no other attempt. */
3377 if (!ovs_mutex_trylock(&dp->port_mutex)) {
3378 ret = netdev_flow_get(netdev, &match, &actions,
3379 &netdev_flow->mega_ufid, stats, attrs, &buf);
3380 /* Storing statistics and attributes from the last request for
3381 * later use on mutex contention. */
3382 dp_netdev_flow_set_last_stats_attrs(netdev_flow, stats, attrs, ret);
3383 ovs_mutex_unlock(&dp->port_mutex);
3384 } else {
3385 dp_netdev_flow_get_last_stats_attrs(netdev_flow, stats, attrs, &ret);
3386 if (!ret && !attrs->dp_layer) {
3387 /* Flow was never reported as 'offloaded' so it's harmless
3388 * to continue to think so. */
3389 ret = EAGAIN;
3390 }
3391 }
3392 netdev_close(netdev);
3393 if (ret) {
3394 return false;
3395 }
3396
3397 return true;
3398 }
3399
3400 static void
3401 get_dpif_flow_status(const struct dp_netdev *dp,
3402 const struct dp_netdev_flow *netdev_flow_,
3403 struct dpif_flow_stats *stats,
3404 struct dpif_flow_attrs *attrs)
3405 {
3406 struct dpif_flow_stats offload_stats;
3407 struct dpif_flow_attrs offload_attrs;
3408 struct dp_netdev_flow *netdev_flow;
3409 unsigned long long n;
3410 long long used;
3411 uint16_t flags;
3412
3413 netdev_flow = CONST_CAST(struct dp_netdev_flow *, netdev_flow_);
3414
3415 atomic_read_relaxed(&netdev_flow->stats.packet_count, &n);
3416 stats->n_packets = n;
3417 atomic_read_relaxed(&netdev_flow->stats.byte_count, &n);
3418 stats->n_bytes = n;
3419 atomic_read_relaxed(&netdev_flow->stats.used, &used);
3420 stats->used = used;
3421 atomic_read_relaxed(&netdev_flow->stats.tcp_flags, &flags);
3422 stats->tcp_flags = flags;
3423
3424 if (dpif_netdev_get_flow_offload_status(dp, netdev_flow,
3425 &offload_stats, &offload_attrs)) {
3426 stats->n_packets += offload_stats.n_packets;
3427 stats->n_bytes += offload_stats.n_bytes;
3428 stats->used = MAX(stats->used, offload_stats.used);
3429 stats->tcp_flags |= offload_stats.tcp_flags;
3430 if (attrs) {
3431 attrs->offloaded = offload_attrs.offloaded;
3432 attrs->dp_layer = offload_attrs.dp_layer;
3433 }
3434 } else if (attrs) {
3435 attrs->offloaded = false;
3436 attrs->dp_layer = "ovs";
3437 }
3438 }
3439
3440 /* Converts to the dpif_flow format, using 'key_buf' and 'mask_buf' for
3441 * storing the netlink-formatted key/mask. 'key_buf' may be the same as
3442 * 'mask_buf'. Actions will be returned without copying, by relying on RCU to
3443 * protect them. */
3444 static void
3445 dp_netdev_flow_to_dpif_flow(const struct dp_netdev *dp,
3446 const struct dp_netdev_flow *netdev_flow,
3447 struct ofpbuf *key_buf, struct ofpbuf *mask_buf,
3448 struct dpif_flow *flow, bool terse)
3449 {
3450 if (terse) {
3451 memset(flow, 0, sizeof *flow);
3452 } else {
3453 struct flow_wildcards wc;
3454 struct dp_netdev_actions *actions;
3455 size_t offset;
3456 struct odp_flow_key_parms odp_parms = {
3457 .flow = &netdev_flow->flow,
3458 .mask = &wc.masks,
3459 .support = dp_netdev_support,
3460 };
3461
3462 miniflow_expand(&netdev_flow->cr.mask->mf, &wc.masks);
3463 /* in_port is exact matched, but we have left it out from the mask for
3464 * optimnization reasons. Add in_port back to the mask. */
3465 wc.masks.in_port.odp_port = ODPP_NONE;
3466
3467 /* Key */
3468 offset = key_buf->size;
3469 flow->key = ofpbuf_tail(key_buf);
3470 odp_flow_key_from_flow(&odp_parms, key_buf);
3471 flow->key_len = key_buf->size - offset;
3472
3473 /* Mask */
3474 offset = mask_buf->size;
3475 flow->mask = ofpbuf_tail(mask_buf);
3476 odp_parms.key_buf = key_buf;
3477 odp_flow_key_from_mask(&odp_parms, mask_buf);
3478 flow->mask_len = mask_buf->size - offset;
3479
3480 /* Actions */
3481 actions = dp_netdev_flow_get_actions(netdev_flow);
3482 flow->actions = actions->actions;
3483 flow->actions_len = actions->size;
3484 }
3485
3486 flow->ufid = netdev_flow->ufid;
3487 flow->ufid_present = true;
3488 flow->pmd_id = netdev_flow->pmd_id;
3489
3490 get_dpif_flow_status(dp, netdev_flow, &flow->stats, &flow->attrs);
3491 flow->attrs.dp_extra_info = netdev_flow->dp_extra_info;
3492 }
3493
3494 static int
3495 dpif_netdev_mask_from_nlattrs(const struct nlattr *key, uint32_t key_len,
3496 const struct nlattr *mask_key,
3497 uint32_t mask_key_len, const struct flow *flow,
3498 struct flow_wildcards *wc, bool probe)
3499 {
3500 enum odp_key_fitness fitness;
3501
3502 fitness = odp_flow_key_to_mask(mask_key, mask_key_len, wc, flow, NULL);
3503 if (fitness) {
3504 if (!probe) {
3505 /* This should not happen: it indicates that
3506 * odp_flow_key_from_mask() and odp_flow_key_to_mask()
3507 * disagree on the acceptable form of a mask. Log the problem
3508 * as an error, with enough details to enable debugging. */
3509 static struct vlog_rate_limit rl = VLOG_RATE_LIMIT_INIT(1, 5);
3510
3511 if (!VLOG_DROP_ERR(&rl)) {
3512 struct ds s;
3513
3514 ds_init(&s);
3515 odp_flow_format(key, key_len, mask_key, mask_key_len, NULL, &s,
3516 true);
3517 VLOG_ERR("internal error parsing flow mask %s (%s)",
3518 ds_cstr(&s), odp_key_fitness_to_string(fitness));
3519 ds_destroy(&s);
3520 }
3521 }
3522
3523 return EINVAL;
3524 }
3525
3526 return 0;
3527 }
3528
3529 static int
3530 dpif_netdev_flow_from_nlattrs(const struct nlattr *key, uint32_t key_len,
3531 struct flow *flow, bool probe)
3532 {
3533 if (odp_flow_key_to_flow(key, key_len, flow, NULL)) {
3534 if (!probe) {
3535 /* This should not happen: it indicates that
3536 * odp_flow_key_from_flow() and odp_flow_key_to_flow() disagree on
3537 * the acceptable form of a flow. Log the problem as an error,
3538 * with enough details to enable debugging. */
3539 static struct vlog_rate_limit rl = VLOG_RATE_LIMIT_INIT(1, 5);
3540
3541 if (!VLOG_DROP_ERR(&rl)) {
3542 struct ds s;
3543
3544 ds_init(&s);
3545 odp_flow_format(key, key_len, NULL, 0, NULL, &s, true);
3546 VLOG_ERR("internal error parsing flow key %s", ds_cstr(&s));
3547 ds_destroy(&s);
3548 }
3549 }
3550
3551 return EINVAL;
3552 }
3553
3554 if (flow->ct_state & DP_NETDEV_CS_UNSUPPORTED_MASK) {
3555 return EINVAL;
3556 }
3557
3558 return 0;
3559 }
3560
3561 static int
3562 dpif_netdev_flow_get(const struct dpif *dpif, const struct dpif_flow_get *get)
3563 {
3564 struct dp_netdev *dp = get_dp_netdev(dpif);
3565 struct dp_netdev_flow *netdev_flow;
3566 struct dp_netdev_pmd_thread *pmd;
3567 struct hmapx to_find = HMAPX_INITIALIZER(&to_find);
3568 struct hmapx_node *node;
3569 int error = EINVAL;
3570
3571 if (get->pmd_id == PMD_ID_NULL) {
3572 CMAP_FOR_EACH (pmd, node, &dp->poll_threads) {
3573 if (dp_netdev_pmd_try_ref(pmd) && !hmapx_add(&to_find, pmd)) {
3574 dp_netdev_pmd_unref(pmd);
3575 }
3576 }
3577 } else {
3578 pmd = dp_netdev_get_pmd(dp, get->pmd_id);
3579 if (!pmd) {
3580 goto out;
3581 }
3582 hmapx_add(&to_find, pmd);
3583 }
3584
3585 if (!hmapx_count(&to_find)) {
3586 goto out;
3587 }
3588
3589 HMAPX_FOR_EACH (node, &to_find) {
3590 pmd = (struct dp_netdev_pmd_thread *) node->data;
3591 netdev_flow = dp_netdev_pmd_find_flow(pmd, get->ufid, get->key,
3592 get->key_len);
3593 if (netdev_flow) {
3594 dp_netdev_flow_to_dpif_flow(dp, netdev_flow, get->buffer,
3595 get->buffer, get->flow, false);
3596 error = 0;
3597 break;
3598 } else {
3599 error = ENOENT;
3600 }
3601 }
3602
3603 HMAPX_FOR_EACH (node, &to_find) {
3604 pmd = (struct dp_netdev_pmd_thread *) node->data;
3605 dp_netdev_pmd_unref(pmd);
3606 }
3607 out:
3608 hmapx_destroy(&to_find);
3609 return error;
3610 }
3611
3612 static void
3613 dp_netdev_get_mega_ufid(const struct match *match, ovs_u128 *mega_ufid)
3614 {
3615 struct flow masked_flow;
3616 size_t i;
3617
3618 for (i = 0; i < sizeof(struct flow); i++) {
3619 ((uint8_t *)&masked_flow)[i] = ((uint8_t *)&match->flow)[i] &
3620 ((uint8_t *)&match->wc)[i];
3621 }
3622 odp_flow_key_hash(&masked_flow, sizeof masked_flow, mega_ufid);
3623 }
3624
3625 static struct dp_netdev_flow *
3626 dp_netdev_flow_add(struct dp_netdev_pmd_thread *pmd,
3627 struct match *match, const ovs_u128 *ufid,
3628 const struct nlattr *actions, size_t actions_len)
3629 OVS_REQUIRES(pmd->flow_mutex)
3630 {
3631 struct ds extra_info = DS_EMPTY_INITIALIZER;
3632 struct dp_netdev_flow *flow;
3633 struct netdev_flow_key mask;
3634 struct dpcls *cls;
3635 size_t unit;
3636
3637 /* Make sure in_port is exact matched before we read it. */
3638 ovs_assert(match->wc.masks.in_port.odp_port == ODPP_NONE);
3639 odp_port_t in_port = match->flow.in_port.odp_port;
3640
3641 /* As we select the dpcls based on the port number, each netdev flow
3642 * belonging to the same dpcls will have the same odp_port value.
3643 * For performance reasons we wildcard odp_port here in the mask. In the
3644 * typical case dp_hash is also wildcarded, and the resulting 8-byte
3645 * chunk {dp_hash, in_port} will be ignored by netdev_flow_mask_init() and
3646 * will not be part of the subtable mask.
3647 * This will speed up the hash computation during dpcls_lookup() because
3648 * there is one less call to hash_add64() in this case. */
3649 match->wc.masks.in_port.odp_port = 0;
3650 netdev_flow_mask_init(&mask, match);
3651 match->wc.masks.in_port.odp_port = ODPP_NONE;
3652
3653 /* Make sure wc does not have metadata. */
3654 ovs_assert(!FLOWMAP_HAS_FIELD(&mask.mf.map, metadata)
3655 && !FLOWMAP_HAS_FIELD(&mask.mf.map, regs));
3656
3657 /* Do not allocate extra space. */
3658 flow = xmalloc(sizeof *flow - sizeof flow->cr.flow.mf + mask.len);
3659 memset(&flow->stats, 0, sizeof flow->stats);
3660 atomic_init(&flow->netdev_flow_get_result, 0);
3661 memset(&flow->last_stats, 0, sizeof flow->last_stats);
3662 memset(&flow->last_attrs, 0, sizeof flow->last_attrs);
3663 flow->dead = false;
3664 flow->batch = NULL;
3665 flow->mark = INVALID_FLOW_MARK;
3666 *CONST_CAST(unsigned *, &flow->pmd_id) = pmd->core_id;
3667 *CONST_CAST(struct flow *, &flow->flow) = match->flow;
3668 *CONST_CAST(ovs_u128 *, &flow->ufid) = *ufid;
3669 ovs_refcount_init(&flow->ref_cnt);
3670 ovsrcu_set(&flow->actions, dp_netdev_actions_create(actions, actions_len));
3671
3672 dp_netdev_get_mega_ufid(match, CONST_CAST(ovs_u128 *, &flow->mega_ufid));
3673 netdev_flow_key_init_masked(&flow->cr.flow, &match->flow, &mask);
3674
3675 /* Select dpcls for in_port. Relies on in_port to be exact match. */
3676 cls = dp_netdev_pmd_find_dpcls(pmd, in_port);
3677 dpcls_insert(cls, &flow->cr, &mask);
3678
3679 ds_put_cstr(&extra_info, "miniflow_bits(");
3680 FLOWMAP_FOR_EACH_UNIT (unit) {
3681 if (unit) {
3682 ds_put_char(&extra_info, ',');
3683 }
3684 ds_put_format(&extra_info, "%d",
3685 count_1bits(flow->cr.mask->mf.map.bits[unit]));
3686 }
3687 ds_put_char(&extra_info, ')');
3688 flow->dp_extra_info = ds_steal_cstr(&extra_info);
3689 ds_destroy(&extra_info);
3690
3691 cmap_insert(&pmd->flow_table, CONST_CAST(struct cmap_node *, &flow->node),
3692 dp_netdev_flow_hash(&flow->ufid));
3693
3694 queue_netdev_flow_put(pmd, flow, match, actions, actions_len);
3695
3696 if (OVS_UNLIKELY(!VLOG_DROP_DBG((&upcall_rl)))) {
3697 struct ds ds = DS_EMPTY_INITIALIZER;
3698 struct ofpbuf key_buf, mask_buf;
3699 struct odp_flow_key_parms odp_parms = {
3700 .flow = &match->flow,
3701 .mask = &match->wc.masks,
3702 .support = dp_netdev_support,
3703 };
3704
3705 ofpbuf_init(&key_buf, 0);
3706 ofpbuf_init(&mask_buf, 0);
3707
3708 odp_flow_key_from_flow(&odp_parms, &key_buf);
3709 odp_parms.key_buf = &key_buf;
3710 odp_flow_key_from_mask(&odp_parms, &mask_buf);
3711
3712 ds_put_cstr(&ds, "flow_add: ");
3713 odp_format_ufid(ufid, &ds);
3714 ds_put_cstr(&ds, " mega_");
3715 odp_format_ufid(&flow->mega_ufid, &ds);
3716 ds_put_cstr(&ds, " ");
3717 odp_flow_format(key_buf.data, key_buf.size,
3718 mask_buf.data, mask_buf.size,
3719 NULL, &ds, false);
3720 ds_put_cstr(&ds, ", actions:");
3721 format_odp_actions(&ds, actions, actions_len, NULL);
3722
3723 VLOG_DBG("%s", ds_cstr(&ds));
3724
3725 ofpbuf_uninit(&key_buf);
3726 ofpbuf_uninit(&mask_buf);
3727
3728 /* Add a printout of the actual match installed. */
3729 struct match m;
3730 ds_clear(&ds);
3731 ds_put_cstr(&ds, "flow match: ");
3732 miniflow_expand(&flow->cr.flow.mf, &m.flow);
3733 miniflow_expand(&flow->cr.mask->mf, &m.wc.masks);
3734 memset(&m.tun_md, 0, sizeof m.tun_md);
3735 match_format(&m, NULL, &ds, OFP_DEFAULT_PRIORITY);
3736
3737 VLOG_DBG("%s", ds_cstr(&ds));
3738
3739 ds_destroy(&ds);
3740 }
3741
3742 return flow;
3743 }
3744
3745 static int
3746 flow_put_on_pmd(struct dp_netdev_pmd_thread *pmd,
3747 struct netdev_flow_key *key,
3748 struct match *match,
3749 ovs_u128 *ufid,
3750 const struct dpif_flow_put *put,
3751 struct dpif_flow_stats *stats)
3752 {
3753 struct dp_netdev_flow *netdev_flow;
3754 int error = 0;
3755
3756 if (stats) {
3757 memset(stats, 0, sizeof *stats);
3758 }
3759
3760 ovs_mutex_lock(&pmd->flow_mutex);
3761 netdev_flow = dp_netdev_pmd_lookup_flow(pmd, key, NULL);
3762 if (!netdev_flow) {
3763 if (put->flags & DPIF_FP_CREATE) {
3764 dp_netdev_flow_add(pmd, match, ufid, put->actions,
3765 put->actions_len);
3766 } else {
3767 error = ENOENT;
3768 }
3769 } else {
3770 if (put->flags & DPIF_FP_MODIFY) {
3771 struct dp_netdev_actions *new_actions;
3772 struct dp_netdev_actions *old_actions;
3773
3774 new_actions = dp_netdev_actions_create(put->actions,
3775 put->actions_len);
3776
3777 old_actions = dp_netdev_flow_get_actions(netdev_flow);
3778 ovsrcu_set(&netdev_flow->actions, new_actions);
3779
3780 queue_netdev_flow_put(pmd, netdev_flow, match,
3781 put->actions, put->actions_len);
3782
3783 if (stats) {
3784 get_dpif_flow_status(pmd->dp, netdev_flow, stats, NULL);
3785 }
3786 if (put->flags & DPIF_FP_ZERO_STATS) {
3787 /* XXX: The userspace datapath uses thread local statistics
3788 * (for flows), which should be updated only by the owning
3789 * thread. Since we cannot write on stats memory here,
3790 * we choose not to support this flag. Please note:
3791 * - This feature is currently used only by dpctl commands with
3792 * option --clear.
3793 * - Should the need arise, this operation can be implemented
3794 * by keeping a base value (to be update here) for each
3795 * counter, and subtracting it before outputting the stats */
3796 error = EOPNOTSUPP;
3797 }
3798
3799 ovsrcu_postpone(dp_netdev_actions_free, old_actions);
3800 } else if (put->flags & DPIF_FP_CREATE) {
3801 error = EEXIST;
3802 } else {
3803 /* Overlapping flow. */
3804 error = EINVAL;
3805 }
3806 }
3807 ovs_mutex_unlock(&pmd->flow_mutex);
3808 return error;
3809 }
3810
3811 static int
3812 dpif_netdev_flow_put(struct dpif *dpif, const struct dpif_flow_put *put)
3813 {
3814 struct dp_netdev *dp = get_dp_netdev(dpif);
3815 struct netdev_flow_key key, mask;
3816 struct dp_netdev_pmd_thread *pmd;
3817 struct match match;
3818 ovs_u128 ufid;
3819 int error;
3820 bool probe = put->flags & DPIF_FP_PROBE;
3821
3822 if (put->stats) {
3823 memset(put->stats, 0, sizeof *put->stats);
3824 }
3825 error = dpif_netdev_flow_from_nlattrs(put->key, put->key_len, &match.flow,
3826 probe);
3827 if (error) {
3828 return error;
3829 }
3830 error = dpif_netdev_mask_from_nlattrs(put->key, put->key_len,
3831 put->mask, put->mask_len,
3832 &match.flow, &match.wc, probe);
3833 if (error) {
3834 return error;
3835 }
3836
3837 if (put->ufid) {
3838 ufid = *put->ufid;
3839 } else {
3840 odp_flow_key_hash(&match.flow, sizeof match.flow, &ufid);
3841 }
3842
3843 /* The Netlink encoding of datapath flow keys cannot express
3844 * wildcarding the presence of a VLAN tag. Instead, a missing VLAN
3845 * tag is interpreted as exact match on the fact that there is no
3846 * VLAN. Unless we refactor a lot of code that translates between
3847 * Netlink and struct flow representations, we have to do the same
3848 * here. This must be in sync with 'match' in handle_packet_upcall(). */
3849 if (!match.wc.masks.vlans[0].tci) {
3850 match.wc.masks.vlans[0].tci = htons(0xffff);
3851 }
3852
3853 /* Must produce a netdev_flow_key for lookup.
3854 * Use the same method as employed to create the key when adding
3855 * the flow to the dplcs to make sure they match. */
3856 netdev_flow_mask_init(&mask, &match);
3857 netdev_flow_key_init_masked(&key, &match.flow, &mask);
3858
3859 if (put->pmd_id == PMD_ID_NULL) {
3860 if (cmap_count(&dp->poll_threads) == 0) {
3861 return EINVAL;
3862 }
3863 CMAP_FOR_EACH (pmd, node, &dp->poll_threads) {
3864 struct dpif_flow_stats pmd_stats;
3865 int pmd_error;
3866
3867 pmd_error = flow_put_on_pmd(pmd, &key, &match, &ufid, put,
3868 &pmd_stats);
3869 if (pmd_error) {
3870 error = pmd_error;
3871 } else if (put->stats) {
3872 put->stats->n_packets += pmd_stats.n_packets;
3873 put->stats->n_bytes += pmd_stats.n_bytes;
3874 put->stats->used = MAX(put->stats->used, pmd_stats.used);
3875 put->stats->tcp_flags |= pmd_stats.tcp_flags;
3876 }
3877 }
3878 } else {
3879 pmd = dp_netdev_get_pmd(dp, put->pmd_id);
3880 if (!pmd) {
3881 return EINVAL;
3882 }
3883 error = flow_put_on_pmd(pmd, &key, &match, &ufid, put, put->stats);
3884 dp_netdev_pmd_unref(pmd);
3885 }
3886
3887 return error;
3888 }
3889
3890 static int
3891 flow_del_on_pmd(struct dp_netdev_pmd_thread *pmd,
3892 struct dpif_flow_stats *stats,
3893 const struct dpif_flow_del *del)
3894 {
3895 struct dp_netdev_flow *netdev_flow;
3896 int error = 0;
3897
3898 ovs_mutex_lock(&pmd->flow_mutex);
3899 netdev_flow = dp_netdev_pmd_find_flow(pmd, del->ufid, del->key,
3900 del->key_len);
3901 if (netdev_flow) {
3902 if (stats) {
3903 get_dpif_flow_status(pmd->dp, netdev_flow, stats, NULL);
3904 }
3905 dp_netdev_pmd_remove_flow(pmd, netdev_flow);
3906 } else {
3907 error = ENOENT;
3908 }
3909 ovs_mutex_unlock(&pmd->flow_mutex);
3910
3911 return error;
3912 }
3913
3914 static int
3915 dpif_netdev_flow_del(struct dpif *dpif, const struct dpif_flow_del *del)
3916 {
3917 struct dp_netdev *dp = get_dp_netdev(dpif);
3918 struct dp_netdev_pmd_thread *pmd;
3919 int error = 0;
3920
3921 if (del->stats) {
3922 memset(del->stats, 0, sizeof *del->stats);
3923 }
3924
3925 if (del->pmd_id == PMD_ID_NULL) {
3926 if (cmap_count(&dp->poll_threads) == 0) {
3927 return EINVAL;
3928 }
3929 CMAP_FOR_EACH (pmd, node, &dp->poll_threads) {
3930 struct dpif_flow_stats pmd_stats;
3931 int pmd_error;
3932
3933 pmd_error = flow_del_on_pmd(pmd, &pmd_stats, del);
3934 if (pmd_error) {
3935 error = pmd_error;
3936 } else if (del->stats) {
3937 del->stats->n_packets += pmd_stats.n_packets;
3938 del->stats->n_bytes += pmd_stats.n_bytes;
3939 del->stats->used = MAX(del->stats->used, pmd_stats.used);
3940 del->stats->tcp_flags |= pmd_stats.tcp_flags;
3941 }
3942 }
3943 } else {
3944 pmd = dp_netdev_get_pmd(dp, del->pmd_id);
3945 if (!pmd) {
3946 return EINVAL;
3947 }
3948 error = flow_del_on_pmd(pmd, del->stats, del);
3949 dp_netdev_pmd_unref(pmd);
3950 }
3951
3952
3953 return error;
3954 }
3955
3956 struct dpif_netdev_flow_dump {
3957 struct dpif_flow_dump up;
3958 struct cmap_position poll_thread_pos;
3959 struct cmap_position flow_pos;
3960 struct dp_netdev_pmd_thread *cur_pmd;
3961 int status;
3962 struct ovs_mutex mutex;
3963 };
3964
3965 static struct dpif_netdev_flow_dump *
3966 dpif_netdev_flow_dump_cast(struct dpif_flow_dump *dump)
3967 {
3968 return CONTAINER_OF(dump, struct dpif_netdev_flow_dump, up);
3969 }
3970
3971 static struct dpif_flow_dump *
3972 dpif_netdev_flow_dump_create(const struct dpif *dpif_, bool terse,
3973 struct dpif_flow_dump_types *types OVS_UNUSED)
3974 {
3975 struct dpif_netdev_flow_dump *dump;
3976
3977 dump = xzalloc(sizeof *dump);
3978 dpif_flow_dump_init(&dump->up, dpif_);
3979 dump->up.terse = terse;
3980 ovs_mutex_init(&dump->mutex);
3981
3982 return &dump->up;
3983 }
3984
3985 static int
3986 dpif_netdev_flow_dump_destroy(struct dpif_flow_dump *dump_)
3987 {
3988 struct dpif_netdev_flow_dump *dump = dpif_netdev_flow_dump_cast(dump_);
3989
3990 ovs_mutex_destroy(&dump->mutex);
3991 free(dump);
3992 return 0;
3993 }
3994
3995 struct dpif_netdev_flow_dump_thread {
3996 struct dpif_flow_dump_thread up;
3997 struct dpif_netdev_flow_dump *dump;
3998 struct odputil_keybuf keybuf[FLOW_DUMP_MAX_BATCH];
3999 struct odputil_keybuf maskbuf[FLOW_DUMP_MAX_BATCH];
4000 };
4001
4002 static struct dpif_netdev_flow_dump_thread *
4003 dpif_netdev_flow_dump_thread_cast(struct dpif_flow_dump_thread *thread)
4004 {
4005 return CONTAINER_OF(thread, struct dpif_netdev_flow_dump_thread, up);
4006 }
4007
4008 static struct dpif_flow_dump_thread *
4009 dpif_netdev_flow_dump_thread_create(struct dpif_flow_dump *dump_)
4010 {
4011 struct dpif_netdev_flow_dump *dump = dpif_netdev_flow_dump_cast(dump_);
4012 struct dpif_netdev_flow_dump_thread *thread;
4013
4014 thread = xmalloc(sizeof *thread);
4015 dpif_flow_dump_thread_init(&thread->up, &dump->up);
4016 thread->dump = dump;
4017 return &thread->up;
4018 }
4019
4020 static void
4021 dpif_netdev_flow_dump_thread_destroy(struct dpif_flow_dump_thread *thread_)
4022 {
4023 struct dpif_netdev_flow_dump_thread *thread
4024 = dpif_netdev_flow_dump_thread_cast(thread_);
4025
4026 free(thread);
4027 }
4028
4029 static int
4030 dpif_netdev_flow_dump_next(struct dpif_flow_dump_thread *thread_,
4031 struct dpif_flow *flows, int max_flows)
4032 {
4033 struct dpif_netdev_flow_dump_thread *thread
4034 = dpif_netdev_flow_dump_thread_cast(thread_);
4035 struct dpif_netdev_flow_dump *dump = thread->dump;
4036 struct dp_netdev_flow *netdev_flows[FLOW_DUMP_MAX_BATCH];
4037 struct dpif_netdev *dpif = dpif_netdev_cast(thread->up.dpif);
4038 struct dp_netdev *dp = get_dp_netdev(&dpif->dpif);
4039 int n_flows = 0;
4040 int i;
4041
4042 ovs_mutex_lock(&dump->mutex);
4043 if (!dump->status) {
4044 struct dp_netdev_pmd_thread *pmd = dump->cur_pmd;
4045 int flow_limit = MIN(max_flows, FLOW_DUMP_MAX_BATCH);
4046
4047 /* First call to dump_next(), extracts the first pmd thread.
4048 * If there is no pmd thread, returns immediately. */
4049 if (!pmd) {
4050 pmd = dp_netdev_pmd_get_next(dp, &dump->poll_thread_pos);
4051 if (!pmd) {
4052 ovs_mutex_unlock(&dump->mutex);
4053 return n_flows;
4054
4055 }
4056 }
4057
4058 do {
4059 for (n_flows = 0; n_flows < flow_limit; n_flows++) {
4060 struct cmap_node *node;
4061
4062 node = cmap_next_position(&pmd->flow_table, &dump->flow_pos);
4063 if (!node) {
4064 break;
4065 }
4066 netdev_flows[n_flows] = CONTAINER_OF(node,
4067 struct dp_netdev_flow,
4068 node);
4069 }
4070 /* When finishing dumping the current pmd thread, moves to
4071 * the next. */
4072 if (n_flows < flow_limit) {
4073 memset(&dump->flow_pos, 0, sizeof dump->flow_pos);
4074 dp_netdev_pmd_unref(pmd);
4075 pmd = dp_netdev_pmd_get_next(dp, &dump->poll_thread_pos);
4076 if (!pmd) {
4077 dump->status = EOF;
4078 break;
4079 }
4080 }
4081 /* Keeps the reference to next caller. */
4082 dump->cur_pmd = pmd;
4083
4084 /* If the current dump is empty, do not exit the loop, since the
4085 * remaining pmds could have flows to be dumped. Just dumps again
4086 * on the new 'pmd'. */
4087 } while (!n_flows);
4088 }
4089 ovs_mutex_unlock(&dump->mutex);
4090
4091 for (i = 0; i < n_flows; i++) {
4092 struct odputil_keybuf *maskbuf = &thread->maskbuf[i];
4093 struct odputil_keybuf *keybuf = &thread->keybuf[i];
4094 struct dp_netdev_flow *netdev_flow = netdev_flows[i];
4095 struct dpif_flow *f = &flows[i];
4096 struct ofpbuf key, mask;
4097
4098 ofpbuf_use_stack(&key, keybuf, sizeof *keybuf);
4099 ofpbuf_use_stack(&mask, maskbuf, sizeof *maskbuf);
4100 dp_netdev_flow_to_dpif_flow(dp, netdev_flow, &key, &mask, f,
4101 dump->up.terse);
4102 }
4103
4104 return n_flows;
4105 }
4106
4107 static int
4108 dpif_netdev_execute(struct dpif *dpif, struct dpif_execute *execute)
4109 OVS_NO_THREAD_SAFETY_ANALYSIS
4110 {
4111 struct dp_netdev *dp = get_dp_netdev(dpif);
4112 struct dp_netdev_pmd_thread *pmd;
4113 struct dp_packet_batch pp;
4114
4115 if (dp_packet_size(execute->packet) < ETH_HEADER_LEN ||
4116 dp_packet_size(execute->packet) > UINT16_MAX) {
4117 return EINVAL;
4118 }
4119
4120 /* Tries finding the 'pmd'. If NULL is returned, that means
4121 * the current thread is a non-pmd thread and should use
4122 * dp_netdev_get_pmd(dp, NON_PMD_CORE_ID). */
4123 pmd = ovsthread_getspecific(dp->per_pmd_key);
4124 if (!pmd) {
4125 pmd = dp_netdev_get_pmd(dp, NON_PMD_CORE_ID);
4126 if (!pmd) {
4127 return EBUSY;
4128 }
4129 }
4130
4131 if (execute->probe) {
4132 /* If this is part of a probe, Drop the packet, since executing
4133 * the action may actually cause spurious packets be sent into
4134 * the network. */
4135 if (pmd->core_id == NON_PMD_CORE_ID) {
4136 dp_netdev_pmd_unref(pmd);
4137 }
4138 return 0;
4139 }
4140
4141 /* If the current thread is non-pmd thread, acquires
4142 * the 'non_pmd_mutex'. */
4143 if (pmd->core_id == NON_PMD_CORE_ID) {
4144 ovs_mutex_lock(&dp->non_pmd_mutex);
4145 }
4146
4147 /* Update current time in PMD context. We don't care about EMC insertion
4148 * probability, because we are on a slow path. */
4149 pmd_thread_ctx_time_update(pmd);
4150
4151 /* The action processing expects the RSS hash to be valid, because
4152 * it's always initialized at the beginning of datapath processing.
4153 * In this case, though, 'execute->packet' may not have gone through
4154 * the datapath at all, it may have been generated by the upper layer
4155 * (OpenFlow packet-out, BFD frame, ...). */
4156 if (!dp_packet_rss_valid(execute->packet)) {
4157 dp_packet_set_rss_hash(execute->packet,
4158 flow_hash_5tuple(execute->flow, 0));
4159 }
4160
4161 dp_packet_batch_init_packet(&pp, execute->packet);
4162 pp.do_not_steal = true;
4163 dp_netdev_execute_actions(pmd, &pp, false, execute->flow,
4164 execute->actions, execute->actions_len);
4165 dp_netdev_pmd_flush_output_packets(pmd, true);
4166
4167 if (pmd->core_id == NON_PMD_CORE_ID) {
4168 ovs_mutex_unlock(&dp->non_pmd_mutex);
4169 dp_netdev_pmd_unref(pmd);
4170 }
4171
4172 return 0;
4173 }
4174
4175 static void
4176 dpif_netdev_operate(struct dpif *dpif, struct dpif_op **ops, size_t n_ops,
4177 enum dpif_offload_type offload_type OVS_UNUSED)
4178 {
4179 size_t i;
4180
4181 for (i = 0; i < n_ops; i++) {
4182 struct dpif_op *op = ops[i];
4183
4184 switch (op->type) {
4185 case DPIF_OP_FLOW_PUT:
4186 op->error = dpif_netdev_flow_put(dpif, &op->flow_put);
4187 break;
4188
4189 case DPIF_OP_FLOW_DEL:
4190 op->error = dpif_netdev_flow_del(dpif, &op->flow_del);
4191 break;
4192
4193 case DPIF_OP_EXECUTE:
4194 op->error = dpif_netdev_execute(dpif, &op->execute);
4195 break;
4196
4197 case DPIF_OP_FLOW_GET:
4198 op->error = dpif_netdev_flow_get(dpif, &op->flow_get);
4199 break;
4200 }
4201 }
4202 }
4203
4204 /* Enable or Disable PMD auto load balancing. */
4205 static void
4206 set_pmd_auto_lb(struct dp_netdev *dp, bool always_log)
4207 {
4208 unsigned int cnt = 0;
4209 struct dp_netdev_pmd_thread *pmd;
4210 struct pmd_auto_lb *pmd_alb = &dp->pmd_alb;
4211 uint8_t rebalance_load_thresh;
4212
4213 bool enable_alb = false;
4214 bool multi_rxq = false;
4215 bool pmd_rxq_assign_cyc = dp->pmd_rxq_assign_cyc;
4216
4217 /* Ensure that there is at least 2 non-isolated PMDs and
4218 * one of them is polling more than one rxq. */
4219 CMAP_FOR_EACH (pmd, node, &dp->poll_threads) {
4220 if (pmd->core_id == NON_PMD_CORE_ID || pmd->isolated) {
4221 continue;
4222 }
4223
4224 if (hmap_count(&pmd->poll_list) > 1) {
4225 multi_rxq = true;
4226 }
4227 if (cnt && multi_rxq) {
4228 enable_alb = true;
4229 break;
4230 }
4231 cnt++;
4232 }
4233
4234 /* Enable auto LB if it is requested and cycle based assignment is true. */
4235 enable_alb = enable_alb && pmd_rxq_assign_cyc &&
4236 pmd_alb->auto_lb_requested;
4237
4238 if (pmd_alb->is_enabled != enable_alb || always_log) {
4239 pmd_alb->is_enabled = enable_alb;
4240 if (pmd_alb->is_enabled) {
4241 atomic_read_relaxed(&pmd_alb->rebalance_load_thresh,
4242 &rebalance_load_thresh);
4243 VLOG_INFO("PMD auto load balance is enabled "
4244 "interval %"PRIu64" mins, "
4245 "pmd load threshold %"PRIu8"%%, "
4246 "improvement threshold %"PRIu8"%%",
4247 pmd_alb->rebalance_intvl / MIN_TO_MSEC,
4248 rebalance_load_thresh,
4249 pmd_alb->rebalance_improve_thresh);
4250
4251 } else {
4252 pmd_alb->rebalance_poll_timer = 0;
4253 VLOG_INFO("PMD auto load balance is disabled");
4254 }
4255 }
4256 }
4257
4258 /* Applies datapath configuration from the database. Some of the changes are
4259 * actually applied in dpif_netdev_run(). */
4260 static int
4261 dpif_netdev_set_config(struct dpif *dpif, const struct smap *other_config)
4262 {
4263 struct dp_netdev *dp = get_dp_netdev(dpif);
4264 const char *cmask = smap_get(other_config, "pmd-cpu-mask");
4265 const char *pmd_rxq_assign = smap_get_def(other_config, "pmd-rxq-assign",
4266 "cycles");
4267 unsigned long long insert_prob =
4268 smap_get_ullong(other_config, "emc-insert-inv-prob",
4269 DEFAULT_EM_FLOW_INSERT_INV_PROB);
4270 uint32_t insert_min, cur_min;
4271 uint32_t tx_flush_interval, cur_tx_flush_interval;
4272 uint64_t rebalance_intvl;
4273 uint8_t rebalance_load, cur_rebalance_load;
4274 uint8_t rebalance_improve;
4275 bool log_autolb = false;
4276
4277 tx_flush_interval = smap_get_int(other_config, "tx-flush-interval",
4278 DEFAULT_TX_FLUSH_INTERVAL);
4279 atomic_read_relaxed(&dp->tx_flush_interval, &cur_tx_flush_interval);
4280 if (tx_flush_interval != cur_tx_flush_interval) {
4281 atomic_store_relaxed(&dp->tx_flush_interval, tx_flush_interval);
4282 VLOG_INFO("Flushing interval for tx queues set to %"PRIu32" us",
4283 tx_flush_interval);
4284 }
4285
4286 if (!nullable_string_is_equal(dp->pmd_cmask, cmask)) {
4287 free(dp->pmd_cmask);
4288 dp->pmd_cmask = nullable_xstrdup(cmask);
4289 dp_netdev_request_reconfigure(dp);
4290 }
4291
4292 atomic_read_relaxed(&dp->emc_insert_min, &cur_min);
4293 if (insert_prob <= UINT32_MAX) {
4294 insert_min = insert_prob == 0 ? 0 : UINT32_MAX / insert_prob;
4295 } else {
4296 insert_min = DEFAULT_EM_FLOW_INSERT_MIN;
4297 insert_prob = DEFAULT_EM_FLOW_INSERT_INV_PROB;
4298 }
4299
4300 if (insert_min != cur_min) {
4301 atomic_store_relaxed(&dp->emc_insert_min, insert_min);
4302 if (insert_min == 0) {
4303 VLOG_INFO("EMC insertion probability changed to zero");
4304 } else {
4305 VLOG_INFO("EMC insertion probability changed to 1/%llu (~%.2f%%)",
4306 insert_prob, (100 / (float)insert_prob));
4307 }
4308 }
4309
4310 bool perf_enabled = smap_get_bool(other_config, "pmd-perf-metrics", false);
4311 bool cur_perf_enabled;
4312 atomic_read_relaxed(&dp->pmd_perf_metrics, &cur_perf_enabled);
4313 if (perf_enabled != cur_perf_enabled) {
4314 atomic_store_relaxed(&dp->pmd_perf_metrics, perf_enabled);
4315 if (perf_enabled) {
4316 VLOG_INFO("PMD performance metrics collection enabled");
4317 } else {
4318 VLOG_INFO("PMD performance metrics collection disabled");
4319 }
4320 }
4321
4322 bool smc_enable = smap_get_bool(other_config, "smc-enable", false);
4323 bool cur_smc;
4324 atomic_read_relaxed(&dp->smc_enable_db, &cur_smc);
4325 if (smc_enable != cur_smc) {
4326 atomic_store_relaxed(&dp->smc_enable_db, smc_enable);
4327 if (smc_enable) {
4328 VLOG_INFO("SMC cache is enabled");
4329 } else {
4330 VLOG_INFO("SMC cache is disabled");
4331 }
4332 }
4333
4334 bool pmd_rxq_assign_cyc = !strcmp(pmd_rxq_assign, "cycles");
4335 if (!pmd_rxq_assign_cyc && strcmp(pmd_rxq_assign, "roundrobin")) {
4336 VLOG_WARN("Unsupported Rxq to PMD assignment mode in pmd-rxq-assign. "
4337 "Defaulting to 'cycles'.");
4338 pmd_rxq_assign_cyc = true;
4339 pmd_rxq_assign = "cycles";
4340 }
4341 if (dp->pmd_rxq_assign_cyc != pmd_rxq_assign_cyc) {
4342 dp->pmd_rxq_assign_cyc = pmd_rxq_assign_cyc;
4343 VLOG_INFO("Rxq to PMD assignment mode changed to: \'%s\'.",
4344 pmd_rxq_assign);
4345 dp_netdev_request_reconfigure(dp);
4346 }
4347
4348 struct pmd_auto_lb *pmd_alb = &dp->pmd_alb;
4349 pmd_alb->auto_lb_requested = smap_get_bool(other_config, "pmd-auto-lb",
4350 false);
4351
4352 rebalance_intvl = smap_get_int(other_config, "pmd-auto-lb-rebal-interval",
4353 ALB_REBALANCE_INTERVAL);
4354
4355 /* Input is in min, convert it to msec. */
4356 rebalance_intvl =
4357 rebalance_intvl ? rebalance_intvl * MIN_TO_MSEC : MIN_TO_MSEC;
4358
4359 if (pmd_alb->rebalance_intvl != rebalance_intvl) {
4360 pmd_alb->rebalance_intvl = rebalance_intvl;
4361 VLOG_INFO("PMD auto load balance interval set to "
4362 "%"PRIu64" mins\n", rebalance_intvl / MIN_TO_MSEC);
4363 log_autolb = true;
4364 }
4365
4366 rebalance_improve = smap_get_int(other_config,
4367 "pmd-auto-lb-improvement-threshold",
4368 ALB_IMPROVEMENT_THRESHOLD);
4369 if (rebalance_improve > 100) {
4370 rebalance_improve = ALB_IMPROVEMENT_THRESHOLD;
4371 }
4372 if (rebalance_improve != pmd_alb->rebalance_improve_thresh) {
4373 pmd_alb->rebalance_improve_thresh = rebalance_improve;
4374 VLOG_INFO("PMD auto load balance improvement threshold set to "
4375 "%"PRIu8"%%", rebalance_improve);
4376 log_autolb = true;
4377 }
4378
4379 rebalance_load = smap_get_int(other_config, "pmd-auto-lb-load-threshold",
4380 ALB_LOAD_THRESHOLD);
4381 if (rebalance_load > 100) {
4382 rebalance_load = ALB_LOAD_THRESHOLD;
4383 }
4384 atomic_read_relaxed(&pmd_alb->rebalance_load_thresh, &cur_rebalance_load);
4385 if (rebalance_load != cur_rebalance_load) {
4386 atomic_store_relaxed(&pmd_alb->rebalance_load_thresh,
4387 rebalance_load);
4388 VLOG_INFO("PMD auto load balance load threshold set to %"PRIu8"%%",
4389 rebalance_load);
4390 log_autolb = true;
4391 }
4392 set_pmd_auto_lb(dp, log_autolb);
4393 return 0;
4394 }
4395
4396 /* Parses affinity list and returns result in 'core_ids'. */
4397 static int
4398 parse_affinity_list(const char *affinity_list, unsigned *core_ids, int n_rxq)
4399 {
4400 unsigned i;
4401 char *list, *copy, *key, *value;
4402 int error = 0;
4403
4404 for (i = 0; i < n_rxq; i++) {
4405 core_ids[i] = OVS_CORE_UNSPEC;
4406 }
4407
4408 if (!affinity_list) {
4409 return 0;
4410 }
4411
4412 list = copy = xstrdup(affinity_list);
4413
4414 while (ofputil_parse_key_value(&list, &key, &value)) {
4415 int rxq_id, core_id;
4416
4417 if (!str_to_int(key, 0, &rxq_id) || rxq_id < 0
4418 || !str_to_int(value, 0, &core_id) || core_id < 0) {
4419 error = EINVAL;
4420 break;
4421 }
4422
4423 if (rxq_id < n_rxq) {
4424 core_ids[rxq_id] = core_id;
4425 }
4426 }
4427
4428 free(copy);
4429 return error;
4430 }
4431
4432 /* Parses 'affinity_list' and applies configuration if it is valid. */
4433 static int
4434 dpif_netdev_port_set_rxq_affinity(struct dp_netdev_port *port,
4435 const char *affinity_list)
4436 {
4437 unsigned *core_ids, i;
4438 int error = 0;
4439
4440 core_ids = xmalloc(port->n_rxq * sizeof *core_ids);
4441 if (parse_affinity_list(affinity_list, core_ids, port->n_rxq)) {
4442 error = EINVAL;
4443 goto exit;
4444 }
4445
4446 for (i = 0; i < port->n_rxq; i++) {
4447 port->rxqs[i].core_id = core_ids[i];
4448 }
4449
4450 exit:
4451 free(core_ids);
4452 return error;
4453 }
4454
4455 /* Returns 'true' if one of the 'port's RX queues exists in 'poll_list'
4456 * of given PMD thread. */
4457 static bool
4458 dpif_netdev_pmd_polls_port(struct dp_netdev_pmd_thread *pmd,
4459 struct dp_netdev_port *port)
4460 OVS_EXCLUDED(pmd->port_mutex)
4461 {
4462 struct rxq_poll *poll;
4463 bool found = false;
4464
4465 ovs_mutex_lock(&pmd->port_mutex);
4466 HMAP_FOR_EACH (poll, node, &pmd->poll_list) {
4467 if (port == poll->rxq->port) {
4468 found = true;
4469 break;
4470 }
4471 }
4472 ovs_mutex_unlock(&pmd->port_mutex);
4473 return found;
4474 }
4475
4476 /* Updates port configuration from the database. The changes are actually
4477 * applied in dpif_netdev_run(). */
4478 static int
4479 dpif_netdev_port_set_config(struct dpif *dpif, odp_port_t port_no,
4480 const struct smap *cfg)
4481 {
4482 struct dp_netdev *dp = get_dp_netdev(dpif);
4483 struct dp_netdev_port *port;
4484 int error = 0;
4485 const char *affinity_list = smap_get(cfg, "pmd-rxq-affinity");
4486 bool emc_enabled = smap_get_bool(cfg, "emc-enable", true);
4487
4488 ovs_mutex_lock(&dp->port_mutex);
4489 error = get_port_by_number(dp, port_no, &port);
4490 if (error) {
4491 goto unlock;
4492 }
4493
4494 if (emc_enabled != port->emc_enabled) {
4495 struct dp_netdev_pmd_thread *pmd;
4496 struct ds ds = DS_EMPTY_INITIALIZER;
4497 uint32_t cur_min, insert_prob;
4498
4499 port->emc_enabled = emc_enabled;
4500 /* Mark for reload all the threads that polls this port and request
4501 * for reconfiguration for the actual reloading of threads. */
4502 CMAP_FOR_EACH (pmd, node, &dp->poll_threads) {
4503 if (dpif_netdev_pmd_polls_port(pmd, port)) {
4504 pmd->need_reload = true;
4505 }
4506 }
4507 dp_netdev_request_reconfigure(dp);
4508
4509 ds_put_format(&ds, "%s: EMC has been %s.",
4510 netdev_get_name(port->netdev),
4511 (emc_enabled) ? "enabled" : "disabled");
4512 if (emc_enabled) {
4513 ds_put_cstr(&ds, " Current insertion probability is ");
4514 atomic_read_relaxed(&dp->emc_insert_min, &cur_min);
4515 if (!cur_min) {
4516 ds_put_cstr(&ds, "zero.");
4517 } else {
4518 insert_prob = UINT32_MAX / cur_min;
4519 ds_put_format(&ds, "1/%"PRIu32" (~%.2f%%).",
4520 insert_prob, 100 / (float) insert_prob);
4521 }
4522 }
4523 VLOG_INFO("%s", ds_cstr(&ds));
4524 ds_destroy(&ds);
4525 }
4526
4527 /* Checking for RXq affinity changes. */
4528 if (!netdev_is_pmd(port->netdev)
4529 || nullable_string_is_equal(affinity_list, port->rxq_affinity_list)) {
4530 goto unlock;
4531 }
4532
4533 error = dpif_netdev_port_set_rxq_affinity(port, affinity_list);
4534 if (error) {
4535 goto unlock;
4536 }
4537 free(port->rxq_affinity_list);
4538 port->rxq_affinity_list = nullable_xstrdup(affinity_list);
4539
4540 dp_netdev_request_reconfigure(dp);
4541 unlock:
4542 ovs_mutex_unlock(&dp->port_mutex);
4543 return error;
4544 }
4545
4546 static int
4547 dpif_netdev_queue_to_priority(const struct dpif *dpif OVS_UNUSED,
4548 uint32_t queue_id, uint32_t *priority)
4549 {
4550 *priority = queue_id;
4551 return 0;
4552 }
4553
4554 \f
4555 /* Creates and returns a new 'struct dp_netdev_actions', whose actions are
4556 * a copy of the 'size' bytes of 'actions' input parameters. */
4557 struct dp_netdev_actions *
4558 dp_netdev_actions_create(const struct nlattr *actions, size_t size)
4559 {
4560 struct dp_netdev_actions *netdev_actions;
4561
4562 netdev_actions = xmalloc(sizeof *netdev_actions + size);
4563 memcpy(netdev_actions->actions, actions, size);
4564 netdev_actions->size = size;
4565
4566 return netdev_actions;
4567 }
4568
4569 struct dp_netdev_actions *
4570 dp_netdev_flow_get_actions(const struct dp_netdev_flow *flow)
4571 {
4572 return ovsrcu_get(struct dp_netdev_actions *, &flow->actions);
4573 }
4574
4575 static void
4576 dp_netdev_actions_free(struct dp_netdev_actions *actions)
4577 {
4578 free(actions);
4579 }
4580 \f
4581 static void
4582 dp_netdev_rxq_set_cycles(struct dp_netdev_rxq *rx,
4583 enum rxq_cycles_counter_type type,
4584 unsigned long long cycles)
4585 {
4586 atomic_store_relaxed(&rx->cycles[type], cycles);
4587 }
4588
4589 static void
4590 dp_netdev_rxq_add_cycles(struct dp_netdev_rxq *rx,
4591 enum rxq_cycles_counter_type type,
4592 unsigned long long cycles)
4593 {
4594 non_atomic_ullong_add(&rx->cycles[type], cycles);
4595 }
4596
4597 static uint64_t
4598 dp_netdev_rxq_get_cycles(struct dp_netdev_rxq *rx,
4599 enum rxq_cycles_counter_type type)
4600 {
4601 unsigned long long processing_cycles;
4602 atomic_read_relaxed(&rx->cycles[type], &processing_cycles);
4603 return processing_cycles;
4604 }
4605
4606 static void
4607 dp_netdev_rxq_set_intrvl_cycles(struct dp_netdev_rxq *rx,
4608 unsigned long long cycles)
4609 {
4610 unsigned int idx = rx->intrvl_idx++ % PMD_RXQ_INTERVAL_MAX;
4611 atomic_store_relaxed(&rx->cycles_intrvl[idx], cycles);
4612 }
4613
4614 static uint64_t
4615 dp_netdev_rxq_get_intrvl_cycles(struct dp_netdev_rxq *rx, unsigned idx)
4616 {
4617 unsigned long long processing_cycles;
4618 atomic_read_relaxed(&rx->cycles_intrvl[idx], &processing_cycles);
4619 return processing_cycles;
4620 }
4621
4622 #if ATOMIC_ALWAYS_LOCK_FREE_8B
4623 static inline bool
4624 pmd_perf_metrics_enabled(const struct dp_netdev_pmd_thread *pmd)
4625 {
4626 bool pmd_perf_enabled;
4627 atomic_read_relaxed(&pmd->dp->pmd_perf_metrics, &pmd_perf_enabled);
4628 return pmd_perf_enabled;
4629 }
4630 #else
4631 /* If stores and reads of 64-bit integers are not atomic, the full PMD
4632 * performance metrics are not available as locked access to 64 bit
4633 * integers would be prohibitively expensive. */
4634 static inline bool
4635 pmd_perf_metrics_enabled(const struct dp_netdev_pmd_thread *pmd OVS_UNUSED)
4636 {
4637 return false;
4638 }
4639 #endif
4640
4641 static int
4642 dp_netdev_pmd_flush_output_on_port(struct dp_netdev_pmd_thread *pmd,
4643 struct tx_port *p)
4644 {
4645 int i;
4646 int tx_qid;
4647 int output_cnt;
4648 bool dynamic_txqs;
4649 struct cycle_timer timer;
4650 uint64_t cycles;
4651 uint32_t tx_flush_interval;
4652
4653 cycle_timer_start(&pmd->perf_stats, &timer);
4654
4655 dynamic_txqs = p->port->dynamic_txqs;
4656 if (dynamic_txqs) {
4657 tx_qid = dpif_netdev_xps_get_tx_qid(pmd, p);
4658 } else {
4659 tx_qid = pmd->static_tx_qid;
4660 }
4661
4662 output_cnt = dp_packet_batch_size(&p->output_pkts);
4663 ovs_assert(output_cnt > 0);
4664
4665 netdev_send(p->port->netdev, tx_qid, &p->output_pkts, dynamic_txqs);
4666 dp_packet_batch_init(&p->output_pkts);
4667
4668 /* Update time of the next flush. */
4669 atomic_read_relaxed(&pmd->dp->tx_flush_interval, &tx_flush_interval);
4670 p->flush_time = pmd->ctx.now + tx_flush_interval;
4671
4672 ovs_assert(pmd->n_output_batches > 0);
4673 pmd->n_output_batches--;
4674
4675 pmd_perf_update_counter(&pmd->perf_stats, PMD_STAT_SENT_PKTS, output_cnt);
4676 pmd_perf_update_counter(&pmd->perf_stats, PMD_STAT_SENT_BATCHES, 1);
4677
4678 /* Distribute send cycles evenly among transmitted packets and assign to
4679 * their respective rx queues. */
4680 cycles = cycle_timer_stop(&pmd->perf_stats, &timer) / output_cnt;
4681 for (i = 0; i < output_cnt; i++) {
4682 if (p->output_pkts_rxqs[i]) {
4683 dp_netdev_rxq_add_cycles(p->output_pkts_rxqs[i],
4684 RXQ_CYCLES_PROC_CURR, cycles);
4685 }
4686 }
4687
4688 return output_cnt;
4689 }
4690
4691 static int
4692 dp_netdev_pmd_flush_output_packets(struct dp_netdev_pmd_thread *pmd,
4693 bool force)
4694 {
4695 struct tx_port *p;
4696 int output_cnt = 0;
4697
4698 if (!pmd->n_output_batches) {
4699 return 0;
4700 }
4701
4702 HMAP_FOR_EACH (p, node, &pmd->send_port_cache) {
4703 if (!dp_packet_batch_is_empty(&p->output_pkts)
4704 && (force || pmd->ctx.now >= p->flush_time)) {
4705 output_cnt += dp_netdev_pmd_flush_output_on_port(pmd, p);
4706 }
4707 }
4708 return output_cnt;
4709 }
4710
4711 static int
4712 dp_netdev_process_rxq_port(struct dp_netdev_pmd_thread *pmd,
4713 struct dp_netdev_rxq *rxq,
4714 odp_port_t port_no)
4715 {
4716 struct pmd_perf_stats *s = &pmd->perf_stats;
4717 struct dp_packet_batch batch;
4718 struct cycle_timer timer;
4719 int error;
4720 int batch_cnt = 0;
4721 int rem_qlen = 0, *qlen_p = NULL;
4722 uint64_t cycles;
4723
4724 /* Measure duration for polling and processing rx burst. */
4725 cycle_timer_start(&pmd->perf_stats, &timer);
4726
4727 pmd->ctx.last_rxq = rxq;
4728 dp_packet_batch_init(&batch);
4729
4730 /* Fetch the rx queue length only for vhostuser ports. */
4731 if (pmd_perf_metrics_enabled(pmd) && rxq->is_vhost) {
4732 qlen_p = &rem_qlen;
4733 }
4734
4735 error = netdev_rxq_recv(rxq->rx, &batch, qlen_p);
4736 if (!error) {
4737 /* At least one packet received. */
4738 *recirc_depth_get() = 0;
4739 pmd_thread_ctx_time_update(pmd);
4740 batch_cnt = dp_packet_batch_size(&batch);
4741 if (pmd_perf_metrics_enabled(pmd)) {
4742 /* Update batch histogram. */
4743 s->current.batches++;
4744 histogram_add_sample(&s->pkts_per_batch, batch_cnt);
4745 /* Update the maximum vhost rx queue fill level. */
4746 if (rxq->is_vhost && rem_qlen >= 0) {
4747 uint32_t qfill = batch_cnt + rem_qlen;
4748 if (qfill > s->current.max_vhost_qfill) {
4749 s->current.max_vhost_qfill = qfill;
4750 }
4751 }
4752 }
4753 /* Process packet batch. */
4754 dp_netdev_input(pmd, &batch, port_no);
4755
4756 /* Assign processing cycles to rx queue. */
4757 cycles = cycle_timer_stop(&pmd->perf_stats, &timer);
4758 dp_netdev_rxq_add_cycles(rxq, RXQ_CYCLES_PROC_CURR, cycles);
4759
4760 dp_netdev_pmd_flush_output_packets(pmd, false);
4761 } else {
4762 /* Discard cycles. */
4763 cycle_timer_stop(&pmd->perf_stats, &timer);
4764 if (error != EAGAIN && error != EOPNOTSUPP) {
4765 static struct vlog_rate_limit rl = VLOG_RATE_LIMIT_INIT(1, 5);
4766
4767 VLOG_ERR_RL(&rl, "error receiving data from %s: %s",
4768 netdev_rxq_get_name(rxq->rx), ovs_strerror(error));
4769 }
4770 }
4771
4772 pmd->ctx.last_rxq = NULL;
4773
4774 return batch_cnt;
4775 }
4776
4777 static struct tx_port *
4778 tx_port_lookup(const struct hmap *hmap, odp_port_t port_no)
4779 {
4780 struct tx_port *tx;
4781
4782 HMAP_FOR_EACH_IN_BUCKET (tx, node, hash_port_no(port_no), hmap) {
4783 if (tx->port->port_no == port_no) {
4784 return tx;
4785 }
4786 }
4787
4788 return NULL;
4789 }
4790
4791 static struct tx_bond *
4792 tx_bond_lookup(const struct cmap *tx_bonds, uint32_t bond_id)
4793 {
4794 uint32_t hash = hash_bond_id(bond_id);
4795 struct tx_bond *tx;
4796
4797 CMAP_FOR_EACH_WITH_HASH (tx, node, hash, tx_bonds) {
4798 if (tx->bond_id == bond_id) {
4799 return tx;
4800 }
4801 }
4802 return NULL;
4803 }
4804
4805 static int
4806 port_reconfigure(struct dp_netdev_port *port)
4807 {
4808 struct netdev *netdev = port->netdev;
4809 int i, err;
4810
4811 /* Closes the existing 'rxq's. */
4812 for (i = 0; i < port->n_rxq; i++) {
4813 netdev_rxq_close(port->rxqs[i].rx);
4814 port->rxqs[i].rx = NULL;
4815 }
4816 unsigned last_nrxq = port->n_rxq;
4817 port->n_rxq = 0;
4818
4819 /* Allows 'netdev' to apply the pending configuration changes. */
4820 if (netdev_is_reconf_required(netdev) || port->need_reconfigure) {
4821 err = netdev_reconfigure(netdev);
4822 if (err && (err != EOPNOTSUPP)) {
4823 VLOG_ERR("Failed to set interface %s new configuration",
4824 netdev_get_name(netdev));
4825 return err;
4826 }
4827 }
4828 /* If the netdev_reconfigure() above succeeds, reopens the 'rxq's. */
4829 port->rxqs = xrealloc(port->rxqs,
4830 sizeof *port->rxqs * netdev_n_rxq(netdev));
4831 /* Realloc 'used' counters for tx queues. */
4832 free(port->txq_used);
4833 port->txq_used = xcalloc(netdev_n_txq(netdev), sizeof *port->txq_used);
4834
4835 for (i = 0; i < netdev_n_rxq(netdev); i++) {
4836 bool new_queue = i >= last_nrxq;
4837 if (new_queue) {
4838 memset(&port->rxqs[i], 0, sizeof port->rxqs[i]);
4839 }
4840
4841 port->rxqs[i].port = port;
4842 port->rxqs[i].is_vhost = !strncmp(port->type, "dpdkvhost", 9);
4843
4844 err = netdev_rxq_open(netdev, &port->rxqs[i].rx, i);
4845 if (err) {
4846 return err;
4847 }
4848 port->n_rxq++;
4849 }
4850
4851 /* Parse affinity list to apply configuration for new queues. */
4852 dpif_netdev_port_set_rxq_affinity(port, port->rxq_affinity_list);
4853
4854 /* If reconfiguration was successful mark it as such, so we can use it */
4855 port->need_reconfigure = false;
4856
4857 return 0;
4858 }
4859
4860 struct rr_numa_list {
4861 struct hmap numas; /* Contains 'struct rr_numa' */
4862 };
4863
4864 struct rr_numa {
4865 struct hmap_node node;
4866
4867 int numa_id;
4868
4869 /* Non isolated pmds on numa node 'numa_id' */
4870 struct dp_netdev_pmd_thread **pmds;
4871 int n_pmds;
4872
4873 int cur_index;
4874 bool idx_inc;
4875 };
4876
4877 static struct rr_numa *
4878 rr_numa_list_lookup(struct rr_numa_list *rr, int numa_id)
4879 {
4880 struct rr_numa *numa;
4881
4882 HMAP_FOR_EACH_WITH_HASH (numa, node, hash_int(numa_id, 0), &rr->numas) {
4883 if (numa->numa_id == numa_id) {
4884 return numa;
4885 }
4886 }
4887
4888 return NULL;
4889 }
4890
4891 /* Returns the next node in numa list following 'numa' in round-robin fashion.
4892 * Returns first node if 'numa' is a null pointer or the last node in 'rr'.
4893 * Returns NULL if 'rr' numa list is empty. */
4894 static struct rr_numa *
4895 rr_numa_list_next(struct rr_numa_list *rr, const struct rr_numa *numa)
4896 {
4897 struct hmap_node *node = NULL;
4898
4899 if (numa) {
4900 node = hmap_next(&rr->numas, &numa->node);
4901 }
4902 if (!node) {
4903 node = hmap_first(&rr->numas);
4904 }
4905
4906 return (node) ? CONTAINER_OF(node, struct rr_numa, node) : NULL;
4907 }
4908
4909 static void
4910 rr_numa_list_populate(struct dp_netdev *dp, struct rr_numa_list *rr)
4911 {
4912 struct dp_netdev_pmd_thread *pmd;
4913 struct rr_numa *numa;
4914
4915 hmap_init(&rr->numas);
4916
4917 CMAP_FOR_EACH (pmd, node, &dp->poll_threads) {
4918 if (pmd->core_id == NON_PMD_CORE_ID || pmd->isolated) {
4919 continue;
4920 }
4921
4922 numa = rr_numa_list_lookup(rr, pmd->numa_id);
4923 if (!numa) {
4924 numa = xzalloc(sizeof *numa);
4925 numa->numa_id = pmd->numa_id;
4926 hmap_insert(&rr->numas, &numa->node, hash_int(pmd->numa_id, 0));
4927 }
4928 numa->n_pmds++;
4929 numa->pmds = xrealloc(numa->pmds, numa->n_pmds * sizeof *numa->pmds);
4930 numa->pmds[numa->n_pmds - 1] = pmd;
4931 /* At least one pmd so initialise curr_idx and idx_inc. */
4932 numa->cur_index = 0;
4933 numa->idx_inc = true;
4934 }
4935 }
4936
4937 /*
4938 * Returns the next pmd from the numa node.
4939 *
4940 * If 'updown' is 'true' it will alternate between selecting the next pmd in
4941 * either an up or down walk, switching between up/down when the first or last
4942 * core is reached. e.g. 1,2,3,3,2,1,1,2...
4943 *
4944 * If 'updown' is 'false' it will select the next pmd wrapping around when last
4945 * core reached. e.g. 1,2,3,1,2,3,1,2...
4946 */
4947 static struct dp_netdev_pmd_thread *
4948 rr_numa_get_pmd(struct rr_numa *numa, bool updown)
4949 {
4950 int numa_idx = numa->cur_index;
4951
4952 if (numa->idx_inc == true) {
4953 /* Incrementing through list of pmds. */
4954 if (numa->cur_index == numa->n_pmds-1) {
4955 /* Reached the last pmd. */
4956 if (updown) {
4957 numa->idx_inc = false;
4958 } else {
4959 numa->cur_index = 0;
4960 }
4961 } else {
4962 numa->cur_index++;
4963 }
4964 } else {
4965 /* Decrementing through list of pmds. */
4966 if (numa->cur_index == 0) {
4967 /* Reached the first pmd. */
4968 numa->idx_inc = true;
4969 } else {
4970 numa->cur_index--;
4971 }
4972 }
4973 return numa->pmds[numa_idx];
4974 }
4975
4976 static void
4977 rr_numa_list_destroy(struct rr_numa_list *rr)
4978 {
4979 struct rr_numa *numa;
4980
4981 HMAP_FOR_EACH_POP (numa, node, &rr->numas) {
4982 free(numa->pmds);
4983 free(numa);
4984 }
4985 hmap_destroy(&rr->numas);
4986 }
4987
4988 /* Sort Rx Queues by the processing cycles they are consuming. */
4989 static int
4990 compare_rxq_cycles(const void *a, const void *b)
4991 {
4992 struct dp_netdev_rxq *qa;
4993 struct dp_netdev_rxq *qb;
4994 uint64_t cycles_qa, cycles_qb;
4995
4996 qa = *(struct dp_netdev_rxq **) a;
4997 qb = *(struct dp_netdev_rxq **) b;
4998
4999 cycles_qa = dp_netdev_rxq_get_cycles(qa, RXQ_CYCLES_PROC_HIST);
5000 cycles_qb = dp_netdev_rxq_get_cycles(qb, RXQ_CYCLES_PROC_HIST);
5001
5002 if (cycles_qa != cycles_qb) {
5003 return (cycles_qa < cycles_qb) ? 1 : -1;
5004 } else {
5005 /* Cycles are the same so tiebreak on port/queue id.
5006 * Tiebreaking (as opposed to return 0) ensures consistent
5007 * sort results across multiple OS's. */
5008 uint32_t port_qa = odp_to_u32(qa->port->port_no);
5009 uint32_t port_qb = odp_to_u32(qb->port->port_no);
5010 if (port_qa != port_qb) {
5011 return port_qa > port_qb ? 1 : -1;
5012 } else {
5013 return netdev_rxq_get_queue_id(qa->rx)
5014 - netdev_rxq_get_queue_id(qb->rx);
5015 }
5016 }
5017 }
5018
5019 /* Assign pmds to queues. If 'pinned' is true, assign pmds to pinned
5020 * queues and marks the pmds as isolated. Otherwise, assign non isolated
5021 * pmds to unpinned queues.
5022 *
5023 * The function doesn't touch the pmd threads, it just stores the assignment
5024 * in the 'pmd' member of each rxq. */
5025 static void
5026 rxq_scheduling(struct dp_netdev *dp, bool pinned) OVS_REQUIRES(dp->port_mutex)
5027 {
5028 struct dp_netdev_port *port;
5029 struct rr_numa_list rr;
5030 struct rr_numa *non_local_numa = NULL;
5031 struct dp_netdev_rxq ** rxqs = NULL;
5032 int n_rxqs = 0;
5033 struct rr_numa *numa = NULL;
5034 int numa_id;
5035 bool assign_cyc = dp->pmd_rxq_assign_cyc;
5036
5037 HMAP_FOR_EACH (port, node, &dp->ports) {
5038 if (!netdev_is_pmd(port->netdev)) {
5039 continue;
5040 }
5041
5042 for (int qid = 0; qid < port->n_rxq; qid++) {
5043 struct dp_netdev_rxq *q = &port->rxqs[qid];
5044
5045 if (pinned && q->core_id != OVS_CORE_UNSPEC) {
5046 struct dp_netdev_pmd_thread *pmd;
5047
5048 pmd = dp_netdev_get_pmd(dp, q->core_id);
5049 if (!pmd) {
5050 VLOG_WARN("There is no PMD thread on core %d. Queue "
5051 "%d on port \'%s\' will not be polled.",
5052 q->core_id, qid, netdev_get_name(port->netdev));
5053 } else {
5054 q->pmd = pmd;
5055 pmd->isolated = true;
5056 VLOG_INFO("Core %d on numa node %d assigned port \'%s\' "
5057 "rx queue %d.", pmd->core_id, pmd->numa_id,
5058 netdev_rxq_get_name(q->rx),
5059 netdev_rxq_get_queue_id(q->rx));
5060 dp_netdev_pmd_unref(pmd);
5061 }
5062 } else if (!pinned && q->core_id == OVS_CORE_UNSPEC) {
5063 uint64_t cycle_hist = 0;
5064
5065 if (n_rxqs == 0) {
5066 rxqs = xmalloc(sizeof *rxqs);
5067 } else {
5068 rxqs = xrealloc(rxqs, sizeof *rxqs * (n_rxqs + 1));
5069 }
5070
5071 if (assign_cyc) {
5072 /* Sum the queue intervals and store the cycle history. */
5073 for (unsigned i = 0; i < PMD_RXQ_INTERVAL_MAX; i++) {
5074 cycle_hist += dp_netdev_rxq_get_intrvl_cycles(q, i);
5075 }
5076 dp_netdev_rxq_set_cycles(q, RXQ_CYCLES_PROC_HIST,
5077 cycle_hist);
5078 }
5079 /* Store the queue. */
5080 rxqs[n_rxqs++] = q;
5081 }
5082 }
5083 }
5084
5085 if (n_rxqs > 1 && assign_cyc) {
5086 /* Sort the queues in order of the processing cycles
5087 * they consumed during their last pmd interval. */
5088 qsort(rxqs, n_rxqs, sizeof *rxqs, compare_rxq_cycles);
5089 }
5090
5091 rr_numa_list_populate(dp, &rr);
5092 /* Assign the sorted queues to pmds in round robin. */
5093 for (int i = 0; i < n_rxqs; i++) {
5094 numa_id = netdev_get_numa_id(rxqs[i]->port->netdev);
5095 numa = rr_numa_list_lookup(&rr, numa_id);
5096 if (!numa) {
5097 /* There are no pmds on the queue's local NUMA node.
5098 Round robin on the NUMA nodes that do have pmds. */
5099 non_local_numa = rr_numa_list_next(&rr, non_local_numa);
5100 if (!non_local_numa) {
5101 VLOG_ERR("There is no available (non-isolated) pmd "
5102 "thread for port \'%s\' queue %d. This queue "
5103 "will not be polled. Is pmd-cpu-mask set to "
5104 "zero? Or are all PMDs isolated to other "
5105 "queues?", netdev_rxq_get_name(rxqs[i]->rx),
5106 netdev_rxq_get_queue_id(rxqs[i]->rx));
5107 continue;
5108 }
5109 rxqs[i]->pmd = rr_numa_get_pmd(non_local_numa, assign_cyc);
5110 VLOG_WARN("There's no available (non-isolated) pmd thread "
5111 "on numa node %d. Queue %d on port \'%s\' will "
5112 "be assigned to the pmd on core %d "
5113 "(numa node %d). Expect reduced performance.",
5114 numa_id, netdev_rxq_get_queue_id(rxqs[i]->rx),
5115 netdev_rxq_get_name(rxqs[i]->rx),
5116 rxqs[i]->pmd->core_id, rxqs[i]->pmd->numa_id);
5117 } else {
5118 rxqs[i]->pmd = rr_numa_get_pmd(numa, assign_cyc);
5119 if (assign_cyc) {
5120 VLOG_INFO("Core %d on numa node %d assigned port \'%s\' "
5121 "rx queue %d "
5122 "(measured processing cycles %"PRIu64").",
5123 rxqs[i]->pmd->core_id, numa_id,
5124 netdev_rxq_get_name(rxqs[i]->rx),
5125 netdev_rxq_get_queue_id(rxqs[i]->rx),
5126 dp_netdev_rxq_get_cycles(rxqs[i],
5127 RXQ_CYCLES_PROC_HIST));
5128 } else {
5129 VLOG_INFO("Core %d on numa node %d assigned port \'%s\' "
5130 "rx queue %d.", rxqs[i]->pmd->core_id, numa_id,
5131 netdev_rxq_get_name(rxqs[i]->rx),
5132 netdev_rxq_get_queue_id(rxqs[i]->rx));
5133 }
5134 }
5135 }
5136
5137 rr_numa_list_destroy(&rr);
5138 free(rxqs);
5139 }
5140
5141 static void
5142 reload_affected_pmds(struct dp_netdev *dp)
5143 {
5144 struct dp_netdev_pmd_thread *pmd;
5145
5146 CMAP_FOR_EACH (pmd, node, &dp->poll_threads) {
5147 if (pmd->need_reload) {
5148 flow_mark_flush(pmd);
5149 dp_netdev_reload_pmd__(pmd);
5150 }
5151 }
5152
5153 CMAP_FOR_EACH (pmd, node, &dp->poll_threads) {
5154 if (pmd->need_reload) {
5155 if (pmd->core_id != NON_PMD_CORE_ID) {
5156 bool reload;
5157
5158 do {
5159 atomic_read_explicit(&pmd->reload, &reload,
5160 memory_order_acquire);
5161 } while (reload);
5162 }
5163 pmd->need_reload = false;
5164 }
5165 }
5166 }
5167
5168 static void
5169 reconfigure_pmd_threads(struct dp_netdev *dp)
5170 OVS_REQUIRES(dp->port_mutex)
5171 {
5172 struct dp_netdev_pmd_thread *pmd;
5173 struct ovs_numa_dump *pmd_cores;
5174 struct ovs_numa_info_core *core;
5175 struct hmapx to_delete = HMAPX_INITIALIZER(&to_delete);
5176 struct hmapx_node *node;
5177 bool changed = false;
5178 bool need_to_adjust_static_tx_qids = false;
5179
5180 /* The pmd threads should be started only if there's a pmd port in the
5181 * datapath. If the user didn't provide any "pmd-cpu-mask", we start
5182 * NR_PMD_THREADS per numa node. */
5183 if (!has_pmd_port(dp)) {
5184 pmd_cores = ovs_numa_dump_n_cores_per_numa(0);
5185 } else if (dp->pmd_cmask && dp->pmd_cmask[0]) {
5186 pmd_cores = ovs_numa_dump_cores_with_cmask(dp->pmd_cmask);
5187 } else {
5188 pmd_cores = ovs_numa_dump_n_cores_per_numa(NR_PMD_THREADS);
5189 }
5190
5191 /* We need to adjust 'static_tx_qid's only if we're reducing number of
5192 * PMD threads. Otherwise, new threads will allocate all the freed ids. */
5193 if (ovs_numa_dump_count(pmd_cores) < cmap_count(&dp->poll_threads) - 1) {
5194 /* Adjustment is required to keep 'static_tx_qid's sequential and
5195 * avoid possible issues, for example, imbalanced tx queue usage
5196 * and unnecessary locking caused by remapping on netdev level. */
5197 need_to_adjust_static_tx_qids = true;
5198 }
5199
5200 /* Check for unwanted pmd threads */
5201 CMAP_FOR_EACH (pmd, node, &dp->poll_threads) {
5202 if (pmd->core_id == NON_PMD_CORE_ID) {
5203 continue;
5204 }
5205 if (!ovs_numa_dump_contains_core(pmd_cores, pmd->numa_id,
5206 pmd->core_id)) {
5207 hmapx_add(&to_delete, pmd);
5208 } else if (need_to_adjust_static_tx_qids) {
5209 atomic_store_relaxed(&pmd->reload_tx_qid, true);
5210 pmd->need_reload = true;
5211 }
5212 }
5213
5214 HMAPX_FOR_EACH (node, &to_delete) {
5215 pmd = (struct dp_netdev_pmd_thread *) node->data;
5216 VLOG_INFO("PMD thread on numa_id: %d, core id: %2d destroyed.",
5217 pmd->numa_id, pmd->core_id);
5218 dp_netdev_del_pmd(dp, pmd);
5219 }
5220 changed = !hmapx_is_empty(&to_delete);
5221 hmapx_destroy(&to_delete);
5222
5223 if (need_to_adjust_static_tx_qids) {
5224 /* 'static_tx_qid's are not sequential now.
5225 * Reload remaining threads to fix this. */
5226 reload_affected_pmds(dp);
5227 }
5228
5229 /* Check for required new pmd threads */
5230 FOR_EACH_CORE_ON_DUMP(core, pmd_cores) {
5231 pmd = dp_netdev_get_pmd(dp, core->core_id);
5232 if (!pmd) {
5233 struct ds name = DS_EMPTY_INITIALIZER;
5234
5235 pmd = xzalloc(sizeof *pmd);
5236 dp_netdev_configure_pmd(pmd, dp, core->core_id, core->numa_id);
5237
5238 ds_put_format(&name, "pmd-c%02d/id:", core->core_id);
5239 pmd->thread = ovs_thread_create(ds_cstr(&name),
5240 pmd_thread_main, pmd);
5241 ds_destroy(&name);
5242
5243 VLOG_INFO("PMD thread on numa_id: %d, core id: %2d created.",
5244 pmd->numa_id, pmd->core_id);
5245 changed = true;
5246 } else {
5247 dp_netdev_pmd_unref(pmd);
5248 }
5249 }
5250
5251 if (changed) {
5252 struct ovs_numa_info_numa *numa;
5253
5254 /* Log the number of pmd threads per numa node. */
5255 FOR_EACH_NUMA_ON_DUMP (numa, pmd_cores) {
5256 VLOG_INFO("There are %"PRIuSIZE" pmd threads on numa node %d",
5257 numa->n_cores, numa->numa_id);
5258 }
5259 }
5260
5261 ovs_numa_dump_destroy(pmd_cores);
5262 }
5263
5264 static void
5265 pmd_remove_stale_ports(struct dp_netdev *dp,
5266 struct dp_netdev_pmd_thread *pmd)
5267 OVS_EXCLUDED(pmd->port_mutex)
5268 OVS_REQUIRES(dp->port_mutex)
5269 {
5270 struct rxq_poll *poll, *poll_next;
5271 struct tx_port *tx, *tx_next;
5272
5273 ovs_mutex_lock(&pmd->port_mutex);
5274 HMAP_FOR_EACH_SAFE (poll, poll_next, node, &pmd->poll_list) {
5275 struct dp_netdev_port *port = poll->rxq->port;
5276
5277 if (port->need_reconfigure
5278 || !hmap_contains(&dp->ports, &port->node)) {
5279 dp_netdev_del_rxq_from_pmd(pmd, poll);
5280 }
5281 }
5282 HMAP_FOR_EACH_SAFE (tx, tx_next, node, &pmd->tx_ports) {
5283 struct dp_netdev_port *port = tx->port;
5284
5285 if (port->need_reconfigure
5286 || !hmap_contains(&dp->ports, &port->node)) {
5287 dp_netdev_del_port_tx_from_pmd(pmd, tx);
5288 }
5289 }
5290 ovs_mutex_unlock(&pmd->port_mutex);
5291 }
5292
5293 /* Must be called each time a port is added/removed or the cmask changes.
5294 * This creates and destroys pmd threads, reconfigures ports, opens their
5295 * rxqs and assigns all rxqs/txqs to pmd threads. */
5296 static void
5297 reconfigure_datapath(struct dp_netdev *dp)
5298 OVS_REQUIRES(dp->port_mutex)
5299 {
5300 struct hmapx busy_threads = HMAPX_INITIALIZER(&busy_threads);
5301 struct dp_netdev_pmd_thread *pmd;
5302 struct dp_netdev_port *port;
5303 int wanted_txqs;
5304
5305 dp->last_reconfigure_seq = seq_read(dp->reconfigure_seq);
5306
5307 /* Step 1: Adjust the pmd threads based on the datapath ports, the cores
5308 * on the system and the user configuration. */
5309 reconfigure_pmd_threads(dp);
5310
5311 wanted_txqs = cmap_count(&dp->poll_threads);
5312
5313 /* The number of pmd threads might have changed, or a port can be new:
5314 * adjust the txqs. */
5315 HMAP_FOR_EACH (port, node, &dp->ports) {
5316 netdev_set_tx_multiq(port->netdev, wanted_txqs);
5317 }
5318
5319 /* Step 2: Remove from the pmd threads ports that have been removed or
5320 * need reconfiguration. */
5321
5322 /* Check for all the ports that need reconfiguration. We cache this in
5323 * 'port->need_reconfigure', because netdev_is_reconf_required() can
5324 * change at any time.
5325 * Also mark for reconfiguration all ports which will likely change their
5326 * 'dynamic_txqs' parameter. It's required to stop using them before
5327 * changing this setting and it's simpler to mark ports here and allow
5328 * 'pmd_remove_stale_ports' to remove them from threads. There will be
5329 * no actual reconfiguration in 'port_reconfigure' because it's
5330 * unnecessary. */
5331 HMAP_FOR_EACH (port, node, &dp->ports) {
5332 if (netdev_is_reconf_required(port->netdev)
5333 || (port->dynamic_txqs
5334 != (netdev_n_txq(port->netdev) < wanted_txqs))) {
5335 port->need_reconfigure = true;
5336 }
5337 }
5338
5339 /* Remove from the pmd threads all the ports that have been deleted or
5340 * need reconfiguration. */
5341 CMAP_FOR_EACH (pmd, node, &dp->poll_threads) {
5342 pmd_remove_stale_ports(dp, pmd);
5343 }
5344
5345 /* Reload affected pmd threads. We must wait for the pmd threads before
5346 * reconfiguring the ports, because a port cannot be reconfigured while
5347 * it's being used. */
5348 reload_affected_pmds(dp);
5349
5350 /* Step 3: Reconfigure ports. */
5351
5352 /* We only reconfigure the ports that we determined above, because they're
5353 * not being used by any pmd thread at the moment. If a port fails to
5354 * reconfigure we remove it from the datapath. */
5355 struct dp_netdev_port *next_port;
5356 HMAP_FOR_EACH_SAFE (port, next_port, node, &dp->ports) {
5357 int err;
5358
5359 if (!port->need_reconfigure) {
5360 continue;
5361 }
5362
5363 err = port_reconfigure(port);
5364 if (err) {
5365 hmap_remove(&dp->ports, &port->node);
5366 seq_change(dp->port_seq);
5367 port_destroy(port);
5368 } else {
5369 port->dynamic_txqs = netdev_n_txq(port->netdev) < wanted_txqs;
5370 }
5371 }
5372
5373 /* Step 4: Compute new rxq scheduling. We don't touch the pmd threads
5374 * for now, we just update the 'pmd' pointer in each rxq to point to the
5375 * wanted thread according to the scheduling policy. */
5376
5377 /* Reset all the pmd threads to non isolated. */
5378 CMAP_FOR_EACH (pmd, node, &dp->poll_threads) {
5379 pmd->isolated = false;
5380 }
5381
5382 /* Reset all the queues to unassigned */
5383 HMAP_FOR_EACH (port, node, &dp->ports) {
5384 for (int i = 0; i < port->n_rxq; i++) {
5385 port->rxqs[i].pmd = NULL;
5386 }
5387 }
5388
5389 /* Add pinned queues and mark pmd threads isolated. */
5390 rxq_scheduling(dp, true);
5391
5392 /* Add non-pinned queues. */
5393 rxq_scheduling(dp, false);
5394
5395 /* Step 5: Remove queues not compliant with new scheduling. */
5396
5397 /* Count all the threads that will have at least one queue to poll. */
5398 HMAP_FOR_EACH (port, node, &dp->ports) {
5399 for (int qid = 0; qid < port->n_rxq; qid++) {
5400 struct dp_netdev_rxq *q = &port->rxqs[qid];
5401
5402 if (q->pmd) {
5403 hmapx_add(&busy_threads, q->pmd);
5404 }
5405 }
5406 }
5407
5408 CMAP_FOR_EACH (pmd, node, &dp->poll_threads) {
5409 struct rxq_poll *poll, *poll_next;
5410
5411 ovs_mutex_lock(&pmd->port_mutex);
5412 HMAP_FOR_EACH_SAFE (poll, poll_next, node, &pmd->poll_list) {
5413 if (poll->rxq->pmd != pmd) {
5414 dp_netdev_del_rxq_from_pmd(pmd, poll);
5415
5416 /* This pmd might sleep after this step if it has no rxq
5417 * remaining. Tell it to busy wait for new assignment if it
5418 * has at least one scheduled queue. */
5419 if (hmap_count(&pmd->poll_list) == 0 &&
5420 hmapx_contains(&busy_threads, pmd)) {
5421 atomic_store_relaxed(&pmd->wait_for_reload, true);
5422 }
5423 }
5424 }
5425 ovs_mutex_unlock(&pmd->port_mutex);
5426 }
5427
5428 hmapx_destroy(&busy_threads);
5429
5430 /* Reload affected pmd threads. We must wait for the pmd threads to remove
5431 * the old queues before readding them, otherwise a queue can be polled by
5432 * two threads at the same time. */
5433 reload_affected_pmds(dp);
5434
5435 /* Step 6: Add queues from scheduling, if they're not there already. */
5436 HMAP_FOR_EACH (port, node, &dp->ports) {
5437 if (!netdev_is_pmd(port->netdev)) {
5438 continue;
5439 }
5440
5441 for (int qid = 0; qid < port->n_rxq; qid++) {
5442 struct dp_netdev_rxq *q = &port->rxqs[qid];
5443
5444 if (q->pmd) {
5445 ovs_mutex_lock(&q->pmd->port_mutex);
5446 dp_netdev_add_rxq_to_pmd(q->pmd, q);
5447 ovs_mutex_unlock(&q->pmd->port_mutex);
5448 }
5449 }
5450 }
5451
5452 /* Add every port and bond to the tx port and bond caches of
5453 * every pmd thread, if it's not there already and if this pmd
5454 * has at least one rxq to poll.
5455 */
5456 CMAP_FOR_EACH (pmd, node, &dp->poll_threads) {
5457 ovs_mutex_lock(&pmd->port_mutex);
5458 if (hmap_count(&pmd->poll_list) || pmd->core_id == NON_PMD_CORE_ID) {
5459 struct tx_bond *bond;
5460
5461 HMAP_FOR_EACH (port, node, &dp->ports) {
5462 dp_netdev_add_port_tx_to_pmd(pmd, port);
5463 }
5464
5465 CMAP_FOR_EACH (bond, node, &dp->tx_bonds) {
5466 dp_netdev_add_bond_tx_to_pmd(pmd, bond, false);
5467 }
5468 }
5469 ovs_mutex_unlock(&pmd->port_mutex);
5470 }
5471
5472 /* Reload affected pmd threads. */
5473 reload_affected_pmds(dp);
5474
5475 /* Check if PMD Auto LB is to be enabled */
5476 set_pmd_auto_lb(dp, false);
5477 }
5478
5479 /* Returns true if one of the netdevs in 'dp' requires a reconfiguration */
5480 static bool
5481 ports_require_restart(const struct dp_netdev *dp)
5482 OVS_REQUIRES(dp->port_mutex)
5483 {
5484 struct dp_netdev_port *port;
5485
5486 HMAP_FOR_EACH (port, node, &dp->ports) {
5487 if (netdev_is_reconf_required(port->netdev)) {
5488 return true;
5489 }
5490 }
5491
5492 return false;
5493 }
5494
5495 /* Calculates variance in the values stored in array 'a'. 'n' is the number
5496 * of elements in array to be considered for calculating vairance.
5497 * Usage example: data array 'a' contains the processing load of each pmd and
5498 * 'n' is the number of PMDs. It returns the variance in processing load of
5499 * PMDs*/
5500 static uint64_t
5501 variance(uint64_t a[], int n)
5502 {
5503 /* Compute mean (average of elements). */
5504 uint64_t sum = 0;
5505 uint64_t mean = 0;
5506 uint64_t sqDiff = 0;
5507
5508 if (!n) {
5509 return 0;
5510 }
5511
5512 for (int i = 0; i < n; i++) {
5513 sum += a[i];
5514 }
5515
5516 if (sum) {
5517 mean = sum / n;
5518
5519 /* Compute sum squared differences with mean. */
5520 for (int i = 0; i < n; i++) {
5521 sqDiff += (a[i] - mean)*(a[i] - mean);
5522 }
5523 }
5524 return (sqDiff ? (sqDiff / n) : 0);
5525 }
5526
5527
5528 /* Returns the variance in the PMDs usage as part of dry run of rxqs
5529 * assignment to PMDs. */
5530 static bool
5531 get_dry_run_variance(struct dp_netdev *dp, uint32_t *core_list,
5532 uint32_t num_pmds, uint64_t *predicted_variance)
5533 OVS_REQUIRES(dp->port_mutex)
5534 {
5535 struct dp_netdev_port *port;
5536 struct dp_netdev_pmd_thread *pmd;
5537 struct dp_netdev_rxq **rxqs = NULL;
5538 struct rr_numa *numa = NULL;
5539 struct rr_numa_list rr;
5540 int n_rxqs = 0;
5541 bool ret = false;
5542 uint64_t *pmd_usage;
5543
5544 if (!predicted_variance) {
5545 return ret;
5546 }
5547
5548 pmd_usage = xcalloc(num_pmds, sizeof(uint64_t));
5549
5550 HMAP_FOR_EACH (port, node, &dp->ports) {
5551 if (!netdev_is_pmd(port->netdev)) {
5552 continue;
5553 }
5554
5555 for (int qid = 0; qid < port->n_rxq; qid++) {
5556 struct dp_netdev_rxq *q = &port->rxqs[qid];
5557 uint64_t cycle_hist = 0;
5558
5559 if (q->pmd->isolated) {
5560 continue;
5561 }
5562
5563 if (n_rxqs == 0) {
5564 rxqs = xmalloc(sizeof *rxqs);
5565 } else {
5566 rxqs = xrealloc(rxqs, sizeof *rxqs * (n_rxqs + 1));
5567 }
5568
5569 /* Sum the queue intervals and store the cycle history. */
5570 for (unsigned i = 0; i < PMD_RXQ_INTERVAL_MAX; i++) {
5571 cycle_hist += dp_netdev_rxq_get_intrvl_cycles(q, i);
5572 }
5573 dp_netdev_rxq_set_cycles(q, RXQ_CYCLES_PROC_HIST,
5574 cycle_hist);
5575 /* Store the queue. */
5576 rxqs[n_rxqs++] = q;
5577 }
5578 }
5579 if (n_rxqs > 1) {
5580 /* Sort the queues in order of the processing cycles
5581 * they consumed during their last pmd interval. */
5582 qsort(rxqs, n_rxqs, sizeof *rxqs, compare_rxq_cycles);
5583 }
5584 rr_numa_list_populate(dp, &rr);
5585
5586 for (int i = 0; i < n_rxqs; i++) {
5587 int numa_id = netdev_get_numa_id(rxqs[i]->port->netdev);
5588 numa = rr_numa_list_lookup(&rr, numa_id);
5589 if (!numa) {
5590 /* Abort if cross NUMA polling. */
5591 VLOG_DBG("PMD auto lb dry run."
5592 " Aborting due to cross-numa polling.");
5593 goto cleanup;
5594 }
5595
5596 pmd = rr_numa_get_pmd(numa, true);
5597 VLOG_DBG("PMD auto lb dry run. Predicted: Core %d on numa node %d "
5598 "to be assigned port \'%s\' rx queue %d "
5599 "(measured processing cycles %"PRIu64").",
5600 pmd->core_id, numa_id,
5601 netdev_rxq_get_name(rxqs[i]->rx),
5602 netdev_rxq_get_queue_id(rxqs[i]->rx),
5603 dp_netdev_rxq_get_cycles(rxqs[i], RXQ_CYCLES_PROC_HIST));
5604
5605 for (int id = 0; id < num_pmds; id++) {
5606 if (pmd->core_id == core_list[id]) {
5607 /* Add the processing cycles of rxq to pmd polling it. */
5608 pmd_usage[id] += dp_netdev_rxq_get_cycles(rxqs[i],
5609 RXQ_CYCLES_PROC_HIST);
5610 }
5611 }
5612 }
5613
5614 CMAP_FOR_EACH (pmd, node, &dp->poll_threads) {
5615 uint64_t total_cycles = 0;
5616
5617 if ((pmd->core_id == NON_PMD_CORE_ID) || pmd->isolated) {
5618 continue;
5619 }
5620
5621 /* Get the total pmd cycles for an interval. */
5622 atomic_read_relaxed(&pmd->intrvl_cycles, &total_cycles);
5623 /* Estimate the cycles to cover all intervals. */
5624 total_cycles *= PMD_RXQ_INTERVAL_MAX;
5625 for (int id = 0; id < num_pmds; id++) {
5626 if (pmd->core_id == core_list[id]) {
5627 if (pmd_usage[id]) {
5628 pmd_usage[id] = (pmd_usage[id] * 100) / total_cycles;
5629 }
5630 VLOG_DBG("PMD auto lb dry run. Predicted: Core %d, "
5631 "usage %"PRIu64"", pmd->core_id, pmd_usage[id]);
5632 }
5633 }
5634 }
5635 *predicted_variance = variance(pmd_usage, num_pmds);
5636 ret = true;
5637
5638 cleanup:
5639 rr_numa_list_destroy(&rr);
5640 free(rxqs);
5641 free(pmd_usage);
5642 return ret;
5643 }
5644
5645 /* Does the dry run of Rxq assignment to PMDs and returns true if it gives
5646 * better distribution of load on PMDs. */
5647 static bool
5648 pmd_rebalance_dry_run(struct dp_netdev *dp)
5649 OVS_REQUIRES(dp->port_mutex)
5650 {
5651 struct dp_netdev_pmd_thread *pmd;
5652 uint64_t *curr_pmd_usage;
5653
5654 uint64_t curr_variance;
5655 uint64_t new_variance;
5656 uint64_t improvement = 0;
5657 uint32_t num_pmds;
5658 uint32_t *pmd_corelist;
5659 struct rxq_poll *poll;
5660 bool ret;
5661
5662 num_pmds = cmap_count(&dp->poll_threads);
5663
5664 if (num_pmds > 1) {
5665 curr_pmd_usage = xcalloc(num_pmds, sizeof(uint64_t));
5666 pmd_corelist = xcalloc(num_pmds, sizeof(uint32_t));
5667 } else {
5668 return false;
5669 }
5670
5671 num_pmds = 0;
5672 CMAP_FOR_EACH (pmd, node, &dp->poll_threads) {
5673 uint64_t total_cycles = 0;
5674 uint64_t total_proc = 0;
5675
5676 if ((pmd->core_id == NON_PMD_CORE_ID) || pmd->isolated) {
5677 continue;
5678 }
5679
5680 /* Get the total pmd cycles for an interval. */
5681 atomic_read_relaxed(&pmd->intrvl_cycles, &total_cycles);
5682 /* Estimate the cycles to cover all intervals. */
5683 total_cycles *= PMD_RXQ_INTERVAL_MAX;
5684
5685 ovs_mutex_lock(&pmd->port_mutex);
5686 HMAP_FOR_EACH (poll, node, &pmd->poll_list) {
5687 for (unsigned i = 0; i < PMD_RXQ_INTERVAL_MAX; i++) {
5688 total_proc += dp_netdev_rxq_get_intrvl_cycles(poll->rxq, i);
5689 }
5690 }
5691 ovs_mutex_unlock(&pmd->port_mutex);
5692
5693 if (total_proc) {
5694 curr_pmd_usage[num_pmds] = (total_proc * 100) / total_cycles;
5695 }
5696
5697 VLOG_DBG("PMD auto lb dry run. Current: Core %d, usage %"PRIu64"",
5698 pmd->core_id, curr_pmd_usage[num_pmds]);
5699
5700 if (atomic_count_get(&pmd->pmd_overloaded)) {
5701 atomic_count_set(&pmd->pmd_overloaded, 0);
5702 }
5703
5704 pmd_corelist[num_pmds] = pmd->core_id;
5705 num_pmds++;
5706 }
5707
5708 curr_variance = variance(curr_pmd_usage, num_pmds);
5709 ret = get_dry_run_variance(dp, pmd_corelist, num_pmds, &new_variance);
5710
5711 if (ret) {
5712 VLOG_DBG("PMD auto lb dry run. Current PMD variance: %"PRIu64","
5713 " Predicted PMD variance: %"PRIu64"",
5714 curr_variance, new_variance);
5715
5716 if (new_variance < curr_variance) {
5717 improvement =
5718 ((curr_variance - new_variance) * 100) / curr_variance;
5719 }
5720 if (improvement < dp->pmd_alb.rebalance_improve_thresh) {
5721 ret = false;
5722 }
5723 }
5724
5725 free(curr_pmd_usage);
5726 free(pmd_corelist);
5727 return ret;
5728 }
5729
5730
5731 /* Return true if needs to revalidate datapath flows. */
5732 static bool
5733 dpif_netdev_run(struct dpif *dpif)
5734 {
5735 struct dp_netdev_port *port;
5736 struct dp_netdev *dp = get_dp_netdev(dpif);
5737 struct dp_netdev_pmd_thread *non_pmd;
5738 uint64_t new_tnl_seq;
5739 bool need_to_flush = true;
5740 bool pmd_rebalance = false;
5741 long long int now = time_msec();
5742 struct dp_netdev_pmd_thread *pmd;
5743
5744 ovs_mutex_lock(&dp->port_mutex);
5745 non_pmd = dp_netdev_get_pmd(dp, NON_PMD_CORE_ID);
5746 if (non_pmd) {
5747 ovs_mutex_lock(&dp->non_pmd_mutex);
5748 HMAP_FOR_EACH (port, node, &dp->ports) {
5749 if (!netdev_is_pmd(port->netdev)) {
5750 int i;
5751
5752 if (port->emc_enabled) {
5753 atomic_read_relaxed(&dp->emc_insert_min,
5754 &non_pmd->ctx.emc_insert_min);
5755 } else {
5756 non_pmd->ctx.emc_insert_min = 0;
5757 }
5758
5759 for (i = 0; i < port->n_rxq; i++) {
5760
5761 if (!netdev_rxq_enabled(port->rxqs[i].rx)) {
5762 continue;
5763 }
5764
5765 if (dp_netdev_process_rxq_port(non_pmd,
5766 &port->rxqs[i],
5767 port->port_no)) {
5768 need_to_flush = false;
5769 }
5770 }
5771 }
5772 }
5773 if (need_to_flush) {
5774 /* We didn't receive anything in the process loop.
5775 * Check if we need to send something.
5776 * There was no time updates on current iteration. */
5777 pmd_thread_ctx_time_update(non_pmd);
5778 dp_netdev_pmd_flush_output_packets(non_pmd, false);
5779 }
5780
5781 dpif_netdev_xps_revalidate_pmd(non_pmd, false);
5782 ovs_mutex_unlock(&dp->non_pmd_mutex);
5783
5784 dp_netdev_pmd_unref(non_pmd);
5785 }
5786
5787 struct pmd_auto_lb *pmd_alb = &dp->pmd_alb;
5788 if (pmd_alb->is_enabled) {
5789 if (!pmd_alb->rebalance_poll_timer) {
5790 pmd_alb->rebalance_poll_timer = now;
5791 } else if ((pmd_alb->rebalance_poll_timer +
5792 pmd_alb->rebalance_intvl) < now) {
5793 pmd_alb->rebalance_poll_timer = now;
5794 CMAP_FOR_EACH (pmd, node, &dp->poll_threads) {
5795 if (atomic_count_get(&pmd->pmd_overloaded) >=
5796 PMD_RXQ_INTERVAL_MAX) {
5797 pmd_rebalance = true;
5798 break;
5799 }
5800 }
5801
5802 if (pmd_rebalance &&
5803 !dp_netdev_is_reconf_required(dp) &&
5804 !ports_require_restart(dp) &&
5805 pmd_rebalance_dry_run(dp)) {
5806 VLOG_INFO("PMD auto lb dry run."
5807 " requesting datapath reconfigure.");
5808 dp_netdev_request_reconfigure(dp);
5809 }
5810 }
5811 }
5812
5813 if (dp_netdev_is_reconf_required(dp) || ports_require_restart(dp)) {
5814 reconfigure_datapath(dp);
5815 }
5816 ovs_mutex_unlock(&dp->port_mutex);
5817
5818 tnl_neigh_cache_run();
5819 tnl_port_map_run();
5820 new_tnl_seq = seq_read(tnl_conf_seq);
5821
5822 if (dp->last_tnl_conf_seq != new_tnl_seq) {
5823 dp->last_tnl_conf_seq = new_tnl_seq;
5824 return true;
5825 }
5826 return false;
5827 }
5828
5829 static void
5830 dpif_netdev_wait(struct dpif *dpif)
5831 {
5832 struct dp_netdev_port *port;
5833 struct dp_netdev *dp = get_dp_netdev(dpif);
5834
5835 ovs_mutex_lock(&dp_netdev_mutex);
5836 ovs_mutex_lock(&dp->port_mutex);
5837 HMAP_FOR_EACH (port, node, &dp->ports) {
5838 netdev_wait_reconf_required(port->netdev);
5839 if (!netdev_is_pmd(port->netdev)) {
5840 int i;
5841
5842 for (i = 0; i < port->n_rxq; i++) {
5843 netdev_rxq_wait(port->rxqs[i].rx);
5844 }
5845 }
5846 }
5847 ovs_mutex_unlock(&dp->port_mutex);
5848 ovs_mutex_unlock(&dp_netdev_mutex);
5849 seq_wait(tnl_conf_seq, dp->last_tnl_conf_seq);
5850 }
5851
5852 static void
5853 pmd_free_cached_ports(struct dp_netdev_pmd_thread *pmd)
5854 {
5855 struct tx_port *tx_port_cached;
5856
5857 /* Flush all the queued packets. */
5858 dp_netdev_pmd_flush_output_packets(pmd, true);
5859 /* Free all used tx queue ids. */
5860 dpif_netdev_xps_revalidate_pmd(pmd, true);
5861
5862 HMAP_FOR_EACH_POP (tx_port_cached, node, &pmd->tnl_port_cache) {
5863 free(tx_port_cached);
5864 }
5865 HMAP_FOR_EACH_POP (tx_port_cached, node, &pmd->send_port_cache) {
5866 free(tx_port_cached);
5867 }
5868 }
5869
5870 /* Copies ports from 'pmd->tx_ports' (shared with the main thread) to
5871 * thread-local copies. Copy to 'pmd->tnl_port_cache' if it is a tunnel
5872 * device, otherwise to 'pmd->send_port_cache' if the port has at least
5873 * one txq. */
5874 static void
5875 pmd_load_cached_ports(struct dp_netdev_pmd_thread *pmd)
5876 OVS_REQUIRES(pmd->port_mutex)
5877 {
5878 struct tx_port *tx_port, *tx_port_cached;
5879
5880 pmd_free_cached_ports(pmd);
5881 hmap_shrink(&pmd->send_port_cache);
5882 hmap_shrink(&pmd->tnl_port_cache);
5883
5884 HMAP_FOR_EACH (tx_port, node, &pmd->tx_ports) {
5885 if (netdev_has_tunnel_push_pop(tx_port->port->netdev)) {
5886 tx_port_cached = xmemdup(tx_port, sizeof *tx_port_cached);
5887 hmap_insert(&pmd->tnl_port_cache, &tx_port_cached->node,
5888 hash_port_no(tx_port_cached->port->port_no));
5889 }
5890
5891 if (netdev_n_txq(tx_port->port->netdev)) {
5892 tx_port_cached = xmemdup(tx_port, sizeof *tx_port_cached);
5893 hmap_insert(&pmd->send_port_cache, &tx_port_cached->node,
5894 hash_port_no(tx_port_cached->port->port_no));
5895 }
5896 }
5897 }
5898
5899 static void
5900 pmd_alloc_static_tx_qid(struct dp_netdev_pmd_thread *pmd)
5901 {
5902 ovs_mutex_lock(&pmd->dp->tx_qid_pool_mutex);
5903 if (!id_pool_alloc_id(pmd->dp->tx_qid_pool, &pmd->static_tx_qid)) {
5904 VLOG_ABORT("static_tx_qid allocation failed for PMD on core %2d"
5905 ", numa_id %d.", pmd->core_id, pmd->numa_id);
5906 }
5907 ovs_mutex_unlock(&pmd->dp->tx_qid_pool_mutex);
5908
5909 VLOG_DBG("static_tx_qid = %d allocated for PMD thread on core %2d"
5910 ", numa_id %d.", pmd->static_tx_qid, pmd->core_id, pmd->numa_id);
5911 }
5912
5913 static void
5914 pmd_free_static_tx_qid(struct dp_netdev_pmd_thread *pmd)
5915 {
5916 ovs_mutex_lock(&pmd->dp->tx_qid_pool_mutex);
5917 id_pool_free_id(pmd->dp->tx_qid_pool, pmd->static_tx_qid);
5918 ovs_mutex_unlock(&pmd->dp->tx_qid_pool_mutex);
5919 }
5920
5921 static int
5922 pmd_load_queues_and_ports(struct dp_netdev_pmd_thread *pmd,
5923 struct polled_queue **ppoll_list)
5924 {
5925 struct polled_queue *poll_list = *ppoll_list;
5926 struct rxq_poll *poll;
5927 int i;
5928
5929 ovs_mutex_lock(&pmd->port_mutex);
5930 poll_list = xrealloc(poll_list, hmap_count(&pmd->poll_list)
5931 * sizeof *poll_list);
5932
5933 i = 0;
5934 HMAP_FOR_EACH (poll, node, &pmd->poll_list) {
5935 poll_list[i].rxq = poll->rxq;
5936 poll_list[i].port_no = poll->rxq->port->port_no;
5937 poll_list[i].emc_enabled = poll->rxq->port->emc_enabled;
5938 poll_list[i].rxq_enabled = netdev_rxq_enabled(poll->rxq->rx);
5939 poll_list[i].change_seq =
5940 netdev_get_change_seq(poll->rxq->port->netdev);
5941 i++;
5942 }
5943
5944 pmd_load_cached_ports(pmd);
5945
5946 ovs_mutex_unlock(&pmd->port_mutex);
5947
5948 *ppoll_list = poll_list;
5949 return i;
5950 }
5951
5952 static void *
5953 pmd_thread_main(void *f_)
5954 {
5955 struct dp_netdev_pmd_thread *pmd = f_;
5956 struct pmd_perf_stats *s = &pmd->perf_stats;
5957 unsigned int lc = 0;
5958 struct polled_queue *poll_list;
5959 bool wait_for_reload = false;
5960 bool reload_tx_qid;
5961 bool exiting;
5962 bool reload;
5963 int poll_cnt;
5964 int i;
5965 int process_packets = 0;
5966
5967 poll_list = NULL;
5968
5969 /* Stores the pmd thread's 'pmd' to 'per_pmd_key'. */
5970 ovsthread_setspecific(pmd->dp->per_pmd_key, pmd);
5971 ovs_numa_thread_setaffinity_core(pmd->core_id);
5972 dpdk_set_lcore_id(pmd->core_id);
5973 poll_cnt = pmd_load_queues_and_ports(pmd, &poll_list);
5974 dfc_cache_init(&pmd->flow_cache);
5975 pmd_alloc_static_tx_qid(pmd);
5976
5977 reload:
5978 atomic_count_init(&pmd->pmd_overloaded, 0);
5979
5980 /* List port/core affinity */
5981 for (i = 0; i < poll_cnt; i++) {
5982 VLOG_DBG("Core %d processing port \'%s\' with queue-id %d\n",
5983 pmd->core_id, netdev_rxq_get_name(poll_list[i].rxq->rx),
5984 netdev_rxq_get_queue_id(poll_list[i].rxq->rx));
5985 /* Reset the rxq current cycles counter. */
5986 dp_netdev_rxq_set_cycles(poll_list[i].rxq, RXQ_CYCLES_PROC_CURR, 0);
5987 }
5988
5989 if (!poll_cnt) {
5990 if (wait_for_reload) {
5991 /* Don't sleep, control thread will ask for a reload shortly. */
5992 do {
5993 atomic_read_explicit(&pmd->reload, &reload,
5994 memory_order_acquire);
5995 } while (!reload);
5996 } else {
5997 while (seq_read(pmd->reload_seq) == pmd->last_reload_seq) {
5998 seq_wait(pmd->reload_seq, pmd->last_reload_seq);
5999 poll_block();
6000 }
6001 }
6002 }
6003
6004 pmd->intrvl_tsc_prev = 0;
6005 atomic_store_relaxed(&pmd->intrvl_cycles, 0);
6006 cycles_counter_update(s);
6007
6008 pmd->next_rcu_quiesce = pmd->ctx.now + PMD_RCU_QUIESCE_INTERVAL;
6009
6010 /* Protect pmd stats from external clearing while polling. */
6011 ovs_mutex_lock(&pmd->perf_stats.stats_mutex);
6012 for (;;) {
6013 uint64_t rx_packets = 0, tx_packets = 0;
6014
6015 pmd_perf_start_iteration(s);
6016
6017 for (i = 0; i < poll_cnt; i++) {
6018
6019 if (!poll_list[i].rxq_enabled) {
6020 continue;
6021 }
6022
6023 if (poll_list[i].emc_enabled) {
6024 atomic_read_relaxed(&pmd->dp->emc_insert_min,
6025 &pmd->ctx.emc_insert_min);
6026 } else {
6027 pmd->ctx.emc_insert_min = 0;
6028 }
6029
6030 process_packets =
6031 dp_netdev_process_rxq_port(pmd, poll_list[i].rxq,
6032 poll_list[i].port_no);
6033 rx_packets += process_packets;
6034 }
6035
6036 if (!rx_packets) {
6037 /* We didn't receive anything in the process loop.
6038 * Check if we need to send something.
6039 * There was no time updates on current iteration. */
6040 pmd_thread_ctx_time_update(pmd);
6041 tx_packets = dp_netdev_pmd_flush_output_packets(pmd, false);
6042 }
6043
6044 /* Do RCU synchronization at fixed interval. This ensures that
6045 * synchronization would not be delayed long even at high load of
6046 * packet processing. */
6047 if (pmd->ctx.now > pmd->next_rcu_quiesce) {
6048 if (!ovsrcu_try_quiesce()) {
6049 pmd->next_rcu_quiesce =
6050 pmd->ctx.now + PMD_RCU_QUIESCE_INTERVAL;
6051 }
6052 }
6053
6054 if (lc++ > 1024) {
6055 lc = 0;
6056
6057 coverage_try_clear();
6058 dp_netdev_pmd_try_optimize(pmd, poll_list, poll_cnt);
6059 if (!ovsrcu_try_quiesce()) {
6060 emc_cache_slow_sweep(&((pmd->flow_cache).emc_cache));
6061 pmd->next_rcu_quiesce =
6062 pmd->ctx.now + PMD_RCU_QUIESCE_INTERVAL;
6063 }
6064
6065 for (i = 0; i < poll_cnt; i++) {
6066 uint64_t current_seq =
6067 netdev_get_change_seq(poll_list[i].rxq->port->netdev);
6068 if (poll_list[i].change_seq != current_seq) {
6069 poll_list[i].change_seq = current_seq;
6070 poll_list[i].rxq_enabled =
6071 netdev_rxq_enabled(poll_list[i].rxq->rx);
6072 }
6073 }
6074 }
6075
6076 atomic_read_explicit(&pmd->reload, &reload, memory_order_acquire);
6077 if (OVS_UNLIKELY(reload)) {
6078 break;
6079 }
6080
6081 pmd_perf_end_iteration(s, rx_packets, tx_packets,
6082 pmd_perf_metrics_enabled(pmd));
6083 }
6084 ovs_mutex_unlock(&pmd->perf_stats.stats_mutex);
6085
6086 poll_cnt = pmd_load_queues_and_ports(pmd, &poll_list);
6087 atomic_read_relaxed(&pmd->wait_for_reload, &wait_for_reload);
6088 atomic_read_relaxed(&pmd->reload_tx_qid, &reload_tx_qid);
6089 atomic_read_relaxed(&pmd->exit, &exiting);
6090 /* Signal here to make sure the pmd finishes
6091 * reloading the updated configuration. */
6092 dp_netdev_pmd_reload_done(pmd);
6093
6094 if (reload_tx_qid) {
6095 pmd_free_static_tx_qid(pmd);
6096 pmd_alloc_static_tx_qid(pmd);
6097 }
6098
6099 if (!exiting) {
6100 goto reload;
6101 }
6102
6103 pmd_free_static_tx_qid(pmd);
6104 dfc_cache_uninit(&pmd->flow_cache);
6105 free(poll_list);
6106 pmd_free_cached_ports(pmd);
6107 return NULL;
6108 }
6109
6110 static void
6111 dp_netdev_disable_upcall(struct dp_netdev *dp)
6112 OVS_ACQUIRES(dp->upcall_rwlock)
6113 {
6114 fat_rwlock_wrlock(&dp->upcall_rwlock);
6115 }
6116
6117 \f
6118 /* Meters */
6119 static void
6120 dpif_netdev_meter_get_features(const struct dpif * dpif OVS_UNUSED,
6121 struct ofputil_meter_features *features)
6122 {
6123 features->max_meters = MAX_METERS;
6124 features->band_types = DP_SUPPORTED_METER_BAND_TYPES;
6125 features->capabilities = DP_SUPPORTED_METER_FLAGS_MASK;
6126 features->max_bands = MAX_BANDS;
6127 features->max_color = 0;
6128 }
6129
6130 /* Applies the meter identified by 'meter_id' to 'packets_'. Packets
6131 * that exceed a band are dropped in-place. */
6132 static void
6133 dp_netdev_run_meter(struct dp_netdev *dp, struct dp_packet_batch *packets_,
6134 uint32_t meter_id, long long int now)
6135 {
6136 struct dp_meter *meter;
6137 struct dp_meter_band *band;
6138 struct dp_packet *packet;
6139 long long int long_delta_t; /* msec */
6140 uint32_t delta_t; /* msec */
6141 const size_t cnt = dp_packet_batch_size(packets_);
6142 uint32_t bytes, volume;
6143 int exceeded_band[NETDEV_MAX_BURST];
6144 uint32_t exceeded_rate[NETDEV_MAX_BURST];
6145 int exceeded_pkt = cnt; /* First packet that exceeded a band rate. */
6146
6147 if (meter_id >= MAX_METERS) {
6148 return;
6149 }
6150
6151 meter_lock(dp, meter_id);
6152 meter = dp->meters[meter_id];
6153 if (!meter) {
6154 goto out;
6155 }
6156
6157 /* Initialize as negative values. */
6158 memset(exceeded_band, 0xff, cnt * sizeof *exceeded_band);
6159 /* Initialize as zeroes. */
6160 memset(exceeded_rate, 0, cnt * sizeof *exceeded_rate);
6161
6162 /* All packets will hit the meter at the same time. */
6163 long_delta_t = now / 1000 - meter->used / 1000; /* msec */
6164
6165 if (long_delta_t < 0) {
6166 /* This condition means that we have several threads fighting for a
6167 meter lock, and the one who received the packets a bit later wins.
6168 Assuming that all racing threads received packets at the same time
6169 to avoid overflow. */
6170 long_delta_t = 0;
6171 }
6172
6173 /* Make sure delta_t will not be too large, so that bucket will not
6174 * wrap around below. */
6175 delta_t = (long_delta_t > (long long int)meter->max_delta_t)
6176 ? meter->max_delta_t : (uint32_t)long_delta_t;
6177
6178 /* Update meter stats. */
6179 meter->used = now;
6180 meter->packet_count += cnt;
6181 bytes = 0;
6182 DP_PACKET_BATCH_FOR_EACH (i, packet, packets_) {
6183 bytes += dp_packet_size(packet);
6184 }
6185 meter->byte_count += bytes;
6186
6187 /* Meters can operate in terms of packets per second or kilobits per
6188 * second. */
6189 if (meter->flags & OFPMF13_PKTPS) {
6190 /* Rate in packets/second, bucket 1/1000 packets. */
6191 /* msec * packets/sec = 1/1000 packets. */
6192 volume = cnt * 1000; /* Take 'cnt' packets from the bucket. */
6193 } else {
6194 /* Rate in kbps, bucket in bits. */
6195 /* msec * kbps = bits */
6196 volume = bytes * 8;
6197 }
6198
6199 /* Update all bands and find the one hit with the highest rate for each
6200 * packet (if any). */
6201 for (int m = 0; m < meter->n_bands; ++m) {
6202 band = &meter->bands[m];
6203
6204 /* Update band's bucket. */
6205 band->bucket += delta_t * band->up.rate;
6206 if (band->bucket > band->up.burst_size) {
6207 band->bucket = band->up.burst_size;
6208 }
6209
6210 /* Drain the bucket for all the packets, if possible. */
6211 if (band->bucket >= volume) {
6212 band->bucket -= volume;
6213 } else {
6214 int band_exceeded_pkt;
6215
6216 /* Band limit hit, must process packet-by-packet. */
6217 if (meter->flags & OFPMF13_PKTPS) {
6218 band_exceeded_pkt = band->bucket / 1000;
6219 band->bucket %= 1000; /* Remainder stays in bucket. */
6220
6221 /* Update the exceeding band for each exceeding packet.
6222 * (Only one band will be fired by a packet, and that
6223 * can be different for each packet.) */
6224 for (int i = band_exceeded_pkt; i < cnt; i++) {
6225 if (band->up.rate > exceeded_rate[i]) {
6226 exceeded_rate[i] = band->up.rate;
6227 exceeded_band[i] = m;
6228 }
6229 }
6230 } else {
6231 /* Packet sizes differ, must process one-by-one. */
6232 band_exceeded_pkt = cnt;
6233 DP_PACKET_BATCH_FOR_EACH (i, packet, packets_) {
6234 uint32_t bits = dp_packet_size(packet) * 8;
6235
6236 if (band->bucket >= bits) {
6237 band->bucket -= bits;
6238 } else {
6239 if (i < band_exceeded_pkt) {
6240 band_exceeded_pkt = i;
6241 }
6242 /* Update the exceeding band for the exceeding packet.
6243 * (Only one band will be fired by a packet, and that
6244 * can be different for each packet.) */
6245 if (band->up.rate > exceeded_rate[i]) {
6246 exceeded_rate[i] = band->up.rate;
6247 exceeded_band[i] = m;
6248 }
6249 }
6250 }
6251 }
6252 /* Remember the first exceeding packet. */
6253 if (exceeded_pkt > band_exceeded_pkt) {
6254 exceeded_pkt = band_exceeded_pkt;
6255 }
6256 }
6257 }
6258
6259 /* Fire the highest rate band exceeded by each packet, and drop
6260 * packets if needed. */
6261 size_t j;
6262 DP_PACKET_BATCH_REFILL_FOR_EACH (j, cnt, packet, packets_) {
6263 if (exceeded_band[j] >= 0) {
6264 /* Meter drop packet. */
6265 band = &meter->bands[exceeded_band[j]];
6266 band->packet_count += 1;
6267 band->byte_count += dp_packet_size(packet);
6268 COVERAGE_INC(datapath_drop_meter);
6269 dp_packet_delete(packet);
6270 } else {
6271 /* Meter accepts packet. */
6272 dp_packet_batch_refill(packets_, packet, j);
6273 }
6274 }
6275 out:
6276 meter_unlock(dp, meter_id);
6277 }
6278
6279 /* Meter set/get/del processing is still single-threaded. */
6280 static int
6281 dpif_netdev_meter_set(struct dpif *dpif, ofproto_meter_id meter_id,
6282 struct ofputil_meter_config *config)
6283 {
6284 struct dp_netdev *dp = get_dp_netdev(dpif);
6285 uint32_t mid = meter_id.uint32;
6286 struct dp_meter *meter;
6287 int i;
6288
6289 if (mid >= MAX_METERS) {
6290 return EFBIG; /* Meter_id out of range. */
6291 }
6292
6293 if (config->flags & ~DP_SUPPORTED_METER_FLAGS_MASK) {
6294 return EBADF; /* Unsupported flags set */
6295 }
6296
6297 if (config->n_bands > MAX_BANDS) {
6298 return EINVAL;
6299 }
6300
6301 for (i = 0; i < config->n_bands; ++i) {
6302 switch (config->bands[i].type) {
6303 case OFPMBT13_DROP:
6304 break;
6305 default:
6306 return ENODEV; /* Unsupported band type */
6307 }
6308 }
6309
6310 /* Allocate meter */
6311 meter = xzalloc(sizeof *meter
6312 + config->n_bands * sizeof(struct dp_meter_band));
6313
6314 meter->flags = config->flags;
6315 meter->n_bands = config->n_bands;
6316 meter->max_delta_t = 0;
6317 meter->used = time_usec();
6318
6319 /* set up bands */
6320 for (i = 0; i < config->n_bands; ++i) {
6321 uint32_t band_max_delta_t;
6322
6323 /* Set burst size to a workable value if none specified. */
6324 if (config->bands[i].burst_size == 0) {
6325 config->bands[i].burst_size = config->bands[i].rate;
6326 }
6327
6328 meter->bands[i].up = config->bands[i];
6329 /* Convert burst size to the bucket units: */
6330 /* pkts => 1/1000 packets, kilobits => bits. */
6331 meter->bands[i].up.burst_size *= 1000;
6332 /* Initialize bucket to empty. */
6333 meter->bands[i].bucket = 0;
6334
6335 /* Figure out max delta_t that is enough to fill any bucket. */
6336 band_max_delta_t
6337 = meter->bands[i].up.burst_size / meter->bands[i].up.rate;
6338 if (band_max_delta_t > meter->max_delta_t) {
6339 meter->max_delta_t = band_max_delta_t;
6340 }
6341 }
6342
6343 meter_lock(dp, mid);
6344 dp_delete_meter(dp, mid); /* Free existing meter, if any */
6345 dp->meters[mid] = meter;
6346 meter_unlock(dp, mid);
6347
6348 return 0;
6349 }
6350
6351 static int
6352 dpif_netdev_meter_get(const struct dpif *dpif,
6353 ofproto_meter_id meter_id_,
6354 struct ofputil_meter_stats *stats, uint16_t n_bands)
6355 {
6356 const struct dp_netdev *dp = get_dp_netdev(dpif);
6357 uint32_t meter_id = meter_id_.uint32;
6358 int retval = 0;
6359
6360 if (meter_id >= MAX_METERS) {
6361 return EFBIG;
6362 }
6363
6364 meter_lock(dp, meter_id);
6365 const struct dp_meter *meter = dp->meters[meter_id];
6366 if (!meter) {
6367 retval = ENOENT;
6368 goto done;
6369 }
6370 if (stats) {
6371 int i = 0;
6372
6373 stats->packet_in_count = meter->packet_count;
6374 stats->byte_in_count = meter->byte_count;
6375
6376 for (i = 0; i < n_bands && i < meter->n_bands; ++i) {
6377 stats->bands[i].packet_count = meter->bands[i].packet_count;
6378 stats->bands[i].byte_count = meter->bands[i].byte_count;
6379 }
6380
6381 stats->n_bands = i;
6382 }
6383
6384 done:
6385 meter_unlock(dp, meter_id);
6386 return retval;
6387 }
6388
6389 static int
6390 dpif_netdev_meter_del(struct dpif *dpif,
6391 ofproto_meter_id meter_id_,
6392 struct ofputil_meter_stats *stats, uint16_t n_bands)
6393 {
6394 struct dp_netdev *dp = get_dp_netdev(dpif);
6395 int error;
6396
6397 error = dpif_netdev_meter_get(dpif, meter_id_, stats, n_bands);
6398 if (!error) {
6399 uint32_t meter_id = meter_id_.uint32;
6400
6401 meter_lock(dp, meter_id);
6402 dp_delete_meter(dp, meter_id);
6403 meter_unlock(dp, meter_id);
6404 }
6405 return error;
6406 }
6407
6408 \f
6409 static void
6410 dpif_netdev_disable_upcall(struct dpif *dpif)
6411 OVS_NO_THREAD_SAFETY_ANALYSIS
6412 {
6413 struct dp_netdev *dp = get_dp_netdev(dpif);
6414 dp_netdev_disable_upcall(dp);
6415 }
6416
6417 static void
6418 dp_netdev_enable_upcall(struct dp_netdev *dp)
6419 OVS_RELEASES(dp->upcall_rwlock)
6420 {
6421 fat_rwlock_unlock(&dp->upcall_rwlock);
6422 }
6423
6424 static void
6425 dpif_netdev_enable_upcall(struct dpif *dpif)
6426 OVS_NO_THREAD_SAFETY_ANALYSIS
6427 {
6428 struct dp_netdev *dp = get_dp_netdev(dpif);
6429 dp_netdev_enable_upcall(dp);
6430 }
6431
6432 static void
6433 dp_netdev_pmd_reload_done(struct dp_netdev_pmd_thread *pmd)
6434 {
6435 atomic_store_relaxed(&pmd->wait_for_reload, false);
6436 atomic_store_relaxed(&pmd->reload_tx_qid, false);
6437 pmd->last_reload_seq = seq_read(pmd->reload_seq);
6438 atomic_store_explicit(&pmd->reload, false, memory_order_release);
6439 }
6440
6441 /* Finds and refs the dp_netdev_pmd_thread on core 'core_id'. Returns
6442 * the pointer if succeeds, otherwise, NULL (it can return NULL even if
6443 * 'core_id' is NON_PMD_CORE_ID).
6444 *
6445 * Caller must unrefs the returned reference. */
6446 static struct dp_netdev_pmd_thread *
6447 dp_netdev_get_pmd(struct dp_netdev *dp, unsigned core_id)
6448 {
6449 struct dp_netdev_pmd_thread *pmd;
6450 const struct cmap_node *pnode;
6451
6452 pnode = cmap_find(&dp->poll_threads, hash_int(core_id, 0));
6453 if (!pnode) {
6454 return NULL;
6455 }
6456 pmd = CONTAINER_OF(pnode, struct dp_netdev_pmd_thread, node);
6457
6458 return dp_netdev_pmd_try_ref(pmd) ? pmd : NULL;
6459 }
6460
6461 /* Sets the 'struct dp_netdev_pmd_thread' for non-pmd threads. */
6462 static void
6463 dp_netdev_set_nonpmd(struct dp_netdev *dp)
6464 OVS_REQUIRES(dp->port_mutex)
6465 {
6466 struct dp_netdev_pmd_thread *non_pmd;
6467
6468 non_pmd = xzalloc(sizeof *non_pmd);
6469 dp_netdev_configure_pmd(non_pmd, dp, NON_PMD_CORE_ID, OVS_NUMA_UNSPEC);
6470 }
6471
6472 /* Caller must have valid pointer to 'pmd'. */
6473 static bool
6474 dp_netdev_pmd_try_ref(struct dp_netdev_pmd_thread *pmd)
6475 {
6476 return ovs_refcount_try_ref_rcu(&pmd->ref_cnt);
6477 }
6478
6479 static void
6480 dp_netdev_pmd_unref(struct dp_netdev_pmd_thread *pmd)
6481 {
6482 if (pmd && ovs_refcount_unref(&pmd->ref_cnt) == 1) {
6483 ovsrcu_postpone(dp_netdev_destroy_pmd, pmd);
6484 }
6485 }
6486
6487 /* Given cmap position 'pos', tries to ref the next node. If try_ref()
6488 * fails, keeps checking for next node until reaching the end of cmap.
6489 *
6490 * Caller must unrefs the returned reference. */
6491 static struct dp_netdev_pmd_thread *
6492 dp_netdev_pmd_get_next(struct dp_netdev *dp, struct cmap_position *pos)
6493 {
6494 struct dp_netdev_pmd_thread *next;
6495
6496 do {
6497 struct cmap_node *node;
6498
6499 node = cmap_next_position(&dp->poll_threads, pos);
6500 next = node ? CONTAINER_OF(node, struct dp_netdev_pmd_thread, node)
6501 : NULL;
6502 } while (next && !dp_netdev_pmd_try_ref(next));
6503
6504 return next;
6505 }
6506
6507 /* Configures the 'pmd' based on the input argument. */
6508 static void
6509 dp_netdev_configure_pmd(struct dp_netdev_pmd_thread *pmd, struct dp_netdev *dp,
6510 unsigned core_id, int numa_id)
6511 {
6512 pmd->dp = dp;
6513 pmd->core_id = core_id;
6514 pmd->numa_id = numa_id;
6515 pmd->need_reload = false;
6516 pmd->n_output_batches = 0;
6517
6518 ovs_refcount_init(&pmd->ref_cnt);
6519 atomic_init(&pmd->exit, false);
6520 pmd->reload_seq = seq_create();
6521 pmd->last_reload_seq = seq_read(pmd->reload_seq);
6522 atomic_init(&pmd->reload, false);
6523 ovs_mutex_init(&pmd->flow_mutex);
6524 ovs_mutex_init(&pmd->port_mutex);
6525 ovs_mutex_init(&pmd->bond_mutex);
6526 cmap_init(&pmd->flow_table);
6527 cmap_init(&pmd->classifiers);
6528 pmd->ctx.last_rxq = NULL;
6529 pmd_thread_ctx_time_update(pmd);
6530 pmd->next_optimization = pmd->ctx.now + DPCLS_OPTIMIZATION_INTERVAL;
6531 pmd->next_rcu_quiesce = pmd->ctx.now + PMD_RCU_QUIESCE_INTERVAL;
6532 pmd->rxq_next_cycle_store = pmd->ctx.now + PMD_RXQ_INTERVAL_LEN;
6533 hmap_init(&pmd->poll_list);
6534 hmap_init(&pmd->tx_ports);
6535 hmap_init(&pmd->tnl_port_cache);
6536 hmap_init(&pmd->send_port_cache);
6537 cmap_init(&pmd->tx_bonds);
6538 /* init the 'flow_cache' since there is no
6539 * actual thread created for NON_PMD_CORE_ID. */
6540 if (core_id == NON_PMD_CORE_ID) {
6541 dfc_cache_init(&pmd->flow_cache);
6542 pmd_alloc_static_tx_qid(pmd);
6543 }
6544 pmd_perf_stats_init(&pmd->perf_stats);
6545 cmap_insert(&dp->poll_threads, CONST_CAST(struct cmap_node *, &pmd->node),
6546 hash_int(core_id, 0));
6547 }
6548
6549 static void
6550 dp_netdev_destroy_pmd(struct dp_netdev_pmd_thread *pmd)
6551 {
6552 struct dpcls *cls;
6553
6554 dp_netdev_pmd_flow_flush(pmd);
6555 hmap_destroy(&pmd->send_port_cache);
6556 hmap_destroy(&pmd->tnl_port_cache);
6557 hmap_destroy(&pmd->tx_ports);
6558 cmap_destroy(&pmd->tx_bonds);
6559 hmap_destroy(&pmd->poll_list);
6560 /* All flows (including their dpcls_rules) have been deleted already */
6561 CMAP_FOR_EACH (cls, node, &pmd->classifiers) {
6562 dpcls_destroy(cls);
6563 ovsrcu_postpone(free, cls);
6564 }
6565 cmap_destroy(&pmd->classifiers);
6566 cmap_destroy(&pmd->flow_table);
6567 ovs_mutex_destroy(&pmd->flow_mutex);
6568 seq_destroy(pmd->reload_seq);
6569 ovs_mutex_destroy(&pmd->port_mutex);
6570 ovs_mutex_destroy(&pmd->bond_mutex);
6571 free(pmd);
6572 }
6573
6574 /* Stops the pmd thread, removes it from the 'dp->poll_threads',
6575 * and unrefs the struct. */
6576 static void
6577 dp_netdev_del_pmd(struct dp_netdev *dp, struct dp_netdev_pmd_thread *pmd)
6578 {
6579 /* NON_PMD_CORE_ID doesn't have a thread, so we don't have to synchronize,
6580 * but extra cleanup is necessary */
6581 if (pmd->core_id == NON_PMD_CORE_ID) {
6582 ovs_mutex_lock(&dp->non_pmd_mutex);
6583 dfc_cache_uninit(&pmd->flow_cache);
6584 pmd_free_cached_ports(pmd);
6585 pmd_free_static_tx_qid(pmd);
6586 ovs_mutex_unlock(&dp->non_pmd_mutex);
6587 } else {
6588 atomic_store_relaxed(&pmd->exit, true);
6589 dp_netdev_reload_pmd__(pmd);
6590 xpthread_join(pmd->thread, NULL);
6591 }
6592
6593 dp_netdev_pmd_clear_ports(pmd);
6594
6595 /* Purges the 'pmd''s flows after stopping the thread, but before
6596 * destroying the flows, so that the flow stats can be collected. */
6597 if (dp->dp_purge_cb) {
6598 dp->dp_purge_cb(dp->dp_purge_aux, pmd->core_id);
6599 }
6600 cmap_remove(&pmd->dp->poll_threads, &pmd->node, hash_int(pmd->core_id, 0));
6601 dp_netdev_pmd_unref(pmd);
6602 }
6603
6604 /* Destroys all pmd threads. If 'non_pmd' is true it also destroys the non pmd
6605 * thread. */
6606 static void
6607 dp_netdev_destroy_all_pmds(struct dp_netdev *dp, bool non_pmd)
6608 {
6609 struct dp_netdev_pmd_thread *pmd;
6610 struct dp_netdev_pmd_thread **pmd_list;
6611 size_t k = 0, n_pmds;
6612
6613 n_pmds = cmap_count(&dp->poll_threads);
6614 pmd_list = xcalloc(n_pmds, sizeof *pmd_list);
6615
6616 CMAP_FOR_EACH (pmd, node, &dp->poll_threads) {
6617 if (!non_pmd && pmd->core_id == NON_PMD_CORE_ID) {
6618 continue;
6619 }
6620 /* We cannot call dp_netdev_del_pmd(), since it alters
6621 * 'dp->poll_threads' (while we're iterating it) and it
6622 * might quiesce. */
6623 ovs_assert(k < n_pmds);
6624 pmd_list[k++] = pmd;
6625 }
6626
6627 for (size_t i = 0; i < k; i++) {
6628 dp_netdev_del_pmd(dp, pmd_list[i]);
6629 }
6630 free(pmd_list);
6631 }
6632
6633 /* Deletes all rx queues from pmd->poll_list and all the ports from
6634 * pmd->tx_ports. */
6635 static void
6636 dp_netdev_pmd_clear_ports(struct dp_netdev_pmd_thread *pmd)
6637 {
6638 struct rxq_poll *poll;
6639 struct tx_port *port;
6640 struct tx_bond *tx;
6641
6642 ovs_mutex_lock(&pmd->port_mutex);
6643 HMAP_FOR_EACH_POP (poll, node, &pmd->poll_list) {
6644 free(poll);
6645 }
6646 HMAP_FOR_EACH_POP (port, node, &pmd->tx_ports) {
6647 free(port);
6648 }
6649 ovs_mutex_unlock(&pmd->port_mutex);
6650
6651 ovs_mutex_lock(&pmd->bond_mutex);
6652 CMAP_FOR_EACH (tx, node, &pmd->tx_bonds) {
6653 cmap_remove(&pmd->tx_bonds, &tx->node, hash_bond_id(tx->bond_id));
6654 ovsrcu_postpone(free, tx);
6655 }
6656 ovs_mutex_unlock(&pmd->bond_mutex);
6657 }
6658
6659 /* Adds rx queue to poll_list of PMD thread, if it's not there already. */
6660 static void
6661 dp_netdev_add_rxq_to_pmd(struct dp_netdev_pmd_thread *pmd,
6662 struct dp_netdev_rxq *rxq)
6663 OVS_REQUIRES(pmd->port_mutex)
6664 {
6665 int qid = netdev_rxq_get_queue_id(rxq->rx);
6666 uint32_t hash = hash_2words(odp_to_u32(rxq->port->port_no), qid);
6667 struct rxq_poll *poll;
6668
6669 HMAP_FOR_EACH_WITH_HASH (poll, node, hash, &pmd->poll_list) {
6670 if (poll->rxq == rxq) {
6671 /* 'rxq' is already polled by this thread. Do nothing. */
6672 return;
6673 }
6674 }
6675
6676 poll = xmalloc(sizeof *poll);
6677 poll->rxq = rxq;
6678 hmap_insert(&pmd->poll_list, &poll->node, hash);
6679
6680 pmd->need_reload = true;
6681 }
6682
6683 /* Delete 'poll' from poll_list of PMD thread. */
6684 static void
6685 dp_netdev_del_rxq_from_pmd(struct dp_netdev_pmd_thread *pmd,
6686 struct rxq_poll *poll)
6687 OVS_REQUIRES(pmd->port_mutex)
6688 {
6689 hmap_remove(&pmd->poll_list, &poll->node);
6690 free(poll);
6691
6692 pmd->need_reload = true;
6693 }
6694
6695 /* Add 'port' to the tx port cache of 'pmd', which must be reloaded for the
6696 * changes to take effect. */
6697 static void
6698 dp_netdev_add_port_tx_to_pmd(struct dp_netdev_pmd_thread *pmd,
6699 struct dp_netdev_port *port)
6700 OVS_REQUIRES(pmd->port_mutex)
6701 {
6702 struct tx_port *tx;
6703
6704 tx = tx_port_lookup(&pmd->tx_ports, port->port_no);
6705 if (tx) {
6706 /* 'port' is already on this thread tx cache. Do nothing. */
6707 return;
6708 }
6709
6710 tx = xzalloc(sizeof *tx);
6711
6712 tx->port = port;
6713 tx->qid = -1;
6714 tx->flush_time = 0LL;
6715 dp_packet_batch_init(&tx->output_pkts);
6716
6717 hmap_insert(&pmd->tx_ports, &tx->node, hash_port_no(tx->port->port_no));
6718 pmd->need_reload = true;
6719 }
6720
6721 /* Del 'tx' from the tx port cache of 'pmd', which must be reloaded for the
6722 * changes to take effect. */
6723 static void
6724 dp_netdev_del_port_tx_from_pmd(struct dp_netdev_pmd_thread *pmd,
6725 struct tx_port *tx)
6726 OVS_REQUIRES(pmd->port_mutex)
6727 {
6728 hmap_remove(&pmd->tx_ports, &tx->node);
6729 free(tx);
6730 pmd->need_reload = true;
6731 }
6732
6733 /* Add bond to the tx bond cmap of 'pmd'. */
6734 static void
6735 dp_netdev_add_bond_tx_to_pmd(struct dp_netdev_pmd_thread *pmd,
6736 struct tx_bond *bond, bool update)
6737 OVS_EXCLUDED(pmd->bond_mutex)
6738 {
6739 struct tx_bond *tx;
6740
6741 ovs_mutex_lock(&pmd->bond_mutex);
6742 tx = tx_bond_lookup(&pmd->tx_bonds, bond->bond_id);
6743
6744 if (tx && !update) {
6745 /* It's not an update and the entry already exists. Do nothing. */
6746 goto unlock;
6747 }
6748
6749 if (tx) {
6750 struct tx_bond *new_tx = xmemdup(bond, sizeof *bond);
6751
6752 /* Copy the stats for each bucket. */
6753 for (int i = 0; i < BOND_BUCKETS; i++) {
6754 uint64_t n_packets, n_bytes;
6755
6756 atomic_read_relaxed(&tx->member_buckets[i].n_packets, &n_packets);
6757 atomic_read_relaxed(&tx->member_buckets[i].n_bytes, &n_bytes);
6758 atomic_init(&new_tx->member_buckets[i].n_packets, n_packets);
6759 atomic_init(&new_tx->member_buckets[i].n_bytes, n_bytes);
6760 }
6761 cmap_replace(&pmd->tx_bonds, &tx->node, &new_tx->node,
6762 hash_bond_id(bond->bond_id));
6763 ovsrcu_postpone(free, tx);
6764 } else {
6765 tx = xmemdup(bond, sizeof *bond);
6766 cmap_insert(&pmd->tx_bonds, &tx->node, hash_bond_id(bond->bond_id));
6767 }
6768 unlock:
6769 ovs_mutex_unlock(&pmd->bond_mutex);
6770 }
6771
6772 /* Delete bond from the tx bond cmap of 'pmd'. */
6773 static void
6774 dp_netdev_del_bond_tx_from_pmd(struct dp_netdev_pmd_thread *pmd,
6775 uint32_t bond_id)
6776 OVS_EXCLUDED(pmd->bond_mutex)
6777 {
6778 struct tx_bond *tx;
6779
6780 ovs_mutex_lock(&pmd->bond_mutex);
6781 tx = tx_bond_lookup(&pmd->tx_bonds, bond_id);
6782 if (tx) {
6783 cmap_remove(&pmd->tx_bonds, &tx->node, hash_bond_id(tx->bond_id));
6784 ovsrcu_postpone(free, tx);
6785 }
6786 ovs_mutex_unlock(&pmd->bond_mutex);
6787 }
6788 \f
6789 static char *
6790 dpif_netdev_get_datapath_version(void)
6791 {
6792 return xstrdup("<built-in>");
6793 }
6794
6795 static void
6796 dp_netdev_flow_used(struct dp_netdev_flow *netdev_flow, int cnt, int size,
6797 uint16_t tcp_flags, long long now)
6798 {
6799 uint16_t flags;
6800
6801 atomic_store_relaxed(&netdev_flow->stats.used, now);
6802 non_atomic_ullong_add(&netdev_flow->stats.packet_count, cnt);
6803 non_atomic_ullong_add(&netdev_flow->stats.byte_count, size);
6804 atomic_read_relaxed(&netdev_flow->stats.tcp_flags, &flags);
6805 flags |= tcp_flags;
6806 atomic_store_relaxed(&netdev_flow->stats.tcp_flags, flags);
6807 }
6808
6809 static int
6810 dp_netdev_upcall(struct dp_netdev_pmd_thread *pmd, struct dp_packet *packet_,
6811 struct flow *flow, struct flow_wildcards *wc, ovs_u128 *ufid,
6812 enum dpif_upcall_type type, const struct nlattr *userdata,
6813 struct ofpbuf *actions, struct ofpbuf *put_actions)
6814 {
6815 struct dp_netdev *dp = pmd->dp;
6816
6817 if (OVS_UNLIKELY(!dp->upcall_cb)) {
6818 return ENODEV;
6819 }
6820
6821 if (OVS_UNLIKELY(!VLOG_DROP_DBG(&upcall_rl))) {
6822 struct ds ds = DS_EMPTY_INITIALIZER;
6823 char *packet_str;
6824 struct ofpbuf key;
6825 struct odp_flow_key_parms odp_parms = {
6826 .flow = flow,
6827 .mask = wc ? &wc->masks : NULL,
6828 .support = dp_netdev_support,
6829 };
6830
6831 ofpbuf_init(&key, 0);
6832 odp_flow_key_from_flow(&odp_parms, &key);
6833 packet_str = ofp_dp_packet_to_string(packet_);
6834
6835 odp_flow_key_format(key.data, key.size, &ds);
6836
6837 VLOG_DBG("%s: %s upcall:\n%s\n%s", dp->name,
6838 dpif_upcall_type_to_string(type), ds_cstr(&ds), packet_str);
6839
6840 ofpbuf_uninit(&key);
6841 free(packet_str);
6842
6843 ds_destroy(&ds);
6844 }
6845
6846 return dp->upcall_cb(packet_, flow, ufid, pmd->core_id, type, userdata,
6847 actions, wc, put_actions, dp->upcall_aux);
6848 }
6849
6850 static inline uint32_t
6851 dpif_netdev_packet_get_rss_hash_orig_pkt(struct dp_packet *packet,
6852 const struct miniflow *mf)
6853 {
6854 uint32_t hash;
6855
6856 if (OVS_LIKELY(dp_packet_rss_valid(packet))) {
6857 hash = dp_packet_get_rss_hash(packet);
6858 } else {
6859 hash = miniflow_hash_5tuple(mf, 0);
6860 dp_packet_set_rss_hash(packet, hash);
6861 }
6862
6863 return hash;
6864 }
6865
6866 static inline uint32_t
6867 dpif_netdev_packet_get_rss_hash(struct dp_packet *packet,
6868 const struct miniflow *mf)
6869 {
6870 uint32_t hash, recirc_depth;
6871
6872 if (OVS_LIKELY(dp_packet_rss_valid(packet))) {
6873 hash = dp_packet_get_rss_hash(packet);
6874 } else {
6875 hash = miniflow_hash_5tuple(mf, 0);
6876 dp_packet_set_rss_hash(packet, hash);
6877 }
6878
6879 /* The RSS hash must account for the recirculation depth to avoid
6880 * collisions in the exact match cache */
6881 recirc_depth = *recirc_depth_get_unsafe();
6882 if (OVS_UNLIKELY(recirc_depth)) {
6883 hash = hash_finish(hash, recirc_depth);
6884 }
6885 return hash;
6886 }
6887
6888 struct packet_batch_per_flow {
6889 unsigned int byte_count;
6890 uint16_t tcp_flags;
6891 struct dp_netdev_flow *flow;
6892
6893 struct dp_packet_batch array;
6894 };
6895
6896 static inline void
6897 packet_batch_per_flow_update(struct packet_batch_per_flow *batch,
6898 struct dp_packet *packet,
6899 uint16_t tcp_flags)
6900 {
6901 batch->byte_count += dp_packet_size(packet);
6902 batch->tcp_flags |= tcp_flags;
6903 dp_packet_batch_add(&batch->array, packet);
6904 }
6905
6906 static inline void
6907 packet_batch_per_flow_init(struct packet_batch_per_flow *batch,
6908 struct dp_netdev_flow *flow)
6909 {
6910 flow->batch = batch;
6911
6912 batch->flow = flow;
6913 dp_packet_batch_init(&batch->array);
6914 batch->byte_count = 0;
6915 batch->tcp_flags = 0;
6916 }
6917
6918 static inline void
6919 packet_batch_per_flow_execute(struct packet_batch_per_flow *batch,
6920 struct dp_netdev_pmd_thread *pmd)
6921 {
6922 struct dp_netdev_actions *actions;
6923 struct dp_netdev_flow *flow = batch->flow;
6924
6925 dp_netdev_flow_used(flow, dp_packet_batch_size(&batch->array),
6926 batch->byte_count,
6927 batch->tcp_flags, pmd->ctx.now / 1000);
6928
6929 actions = dp_netdev_flow_get_actions(flow);
6930
6931 dp_netdev_execute_actions(pmd, &batch->array, true, &flow->flow,
6932 actions->actions, actions->size);
6933 }
6934
6935 static inline void
6936 dp_netdev_queue_batches(struct dp_packet *pkt,
6937 struct dp_netdev_flow *flow, uint16_t tcp_flags,
6938 struct packet_batch_per_flow *batches,
6939 size_t *n_batches)
6940 {
6941 struct packet_batch_per_flow *batch = flow->batch;
6942
6943 if (OVS_UNLIKELY(!batch)) {
6944 batch = &batches[(*n_batches)++];
6945 packet_batch_per_flow_init(batch, flow);
6946 }
6947
6948 packet_batch_per_flow_update(batch, pkt, tcp_flags);
6949 }
6950
6951 static inline void
6952 packet_enqueue_to_flow_map(struct dp_packet *packet,
6953 struct dp_netdev_flow *flow,
6954 uint16_t tcp_flags,
6955 struct dp_packet_flow_map *flow_map,
6956 size_t index)
6957 {
6958 struct dp_packet_flow_map *map = &flow_map[index];
6959 map->flow = flow;
6960 map->packet = packet;
6961 map->tcp_flags = tcp_flags;
6962 }
6963
6964 /* SMC lookup function for a batch of packets.
6965 * By doing batching SMC lookup, we can use prefetch
6966 * to hide memory access latency.
6967 */
6968 static inline void
6969 smc_lookup_batch(struct dp_netdev_pmd_thread *pmd,
6970 struct netdev_flow_key *keys,
6971 struct netdev_flow_key **missed_keys,
6972 struct dp_packet_batch *packets_,
6973 const int cnt,
6974 struct dp_packet_flow_map *flow_map,
6975 uint8_t *index_map)
6976 {
6977 int i;
6978 struct dp_packet *packet;
6979 size_t n_smc_hit = 0, n_missed = 0;
6980 struct dfc_cache *cache = &pmd->flow_cache;
6981 struct smc_cache *smc_cache = &cache->smc_cache;
6982 const struct cmap_node *flow_node;
6983 int recv_idx;
6984 uint16_t tcp_flags;
6985
6986 /* Prefetch buckets for all packets */
6987 for (i = 0; i < cnt; i++) {
6988 OVS_PREFETCH(&smc_cache->buckets[keys[i].hash & SMC_MASK]);
6989 }
6990
6991 DP_PACKET_BATCH_REFILL_FOR_EACH (i, cnt, packet, packets_) {
6992 struct dp_netdev_flow *flow = NULL;
6993 flow_node = smc_entry_get(pmd, keys[i].hash);
6994 bool hit = false;
6995 /* Get the original order of this packet in received batch. */
6996 recv_idx = index_map[i];
6997
6998 if (OVS_LIKELY(flow_node != NULL)) {
6999 CMAP_NODE_FOR_EACH (flow, node, flow_node) {
7000 /* Since we dont have per-port megaflow to check the port
7001 * number, we need to verify that the input ports match. */
7002 if (OVS_LIKELY(dpcls_rule_matches_key(&flow->cr, &keys[i]) &&
7003 flow->flow.in_port.odp_port == packet->md.in_port.odp_port)) {
7004 tcp_flags = miniflow_get_tcp_flags(&keys[i].mf);
7005
7006 /* SMC hit and emc miss, we insert into EMC */
7007 keys[i].len =
7008 netdev_flow_key_size(miniflow_n_values(&keys[i].mf));
7009 emc_probabilistic_insert(pmd, &keys[i], flow);
7010 /* Add these packets into the flow map in the same order
7011 * as received.
7012 */
7013 packet_enqueue_to_flow_map(packet, flow, tcp_flags,
7014 flow_map, recv_idx);
7015 n_smc_hit++;
7016 hit = true;
7017 break;
7018 }
7019 }
7020 if (hit) {
7021 continue;
7022 }
7023 }
7024
7025 /* SMC missed. Group missed packets together at
7026 * the beginning of the 'packets' array. */
7027 dp_packet_batch_refill(packets_, packet, i);
7028
7029 /* Preserve the order of packet for flow batching. */
7030 index_map[n_missed] = recv_idx;
7031
7032 /* Put missed keys to the pointer arrays return to the caller */
7033 missed_keys[n_missed++] = &keys[i];
7034 }
7035
7036 pmd_perf_update_counter(&pmd->perf_stats, PMD_STAT_SMC_HIT, n_smc_hit);
7037 }
7038
7039 /* Try to process all ('cnt') the 'packets' using only the datapath flow cache
7040 * 'pmd->flow_cache'. If a flow is not found for a packet 'packets[i]', the
7041 * miniflow is copied into 'keys' and the packet pointer is moved at the
7042 * beginning of the 'packets' array. The pointers of missed keys are put in the
7043 * missed_keys pointer array for future processing.
7044 *
7045 * The function returns the number of packets that needs to be processed in the
7046 * 'packets' array (they have been moved to the beginning of the vector).
7047 *
7048 * For performance reasons a caller may choose not to initialize the metadata
7049 * in 'packets_'. If 'md_is_valid' is false, the metadata in 'packets'
7050 * is not valid and must be initialized by this function using 'port_no'.
7051 * If 'md_is_valid' is true, the metadata is already valid and 'port_no'
7052 * will be ignored.
7053 */
7054 static inline size_t
7055 dfc_processing(struct dp_netdev_pmd_thread *pmd,
7056 struct dp_packet_batch *packets_,
7057 struct netdev_flow_key *keys,
7058 struct netdev_flow_key **missed_keys,
7059 struct packet_batch_per_flow batches[], size_t *n_batches,
7060 struct dp_packet_flow_map *flow_map,
7061 size_t *n_flows, uint8_t *index_map,
7062 bool md_is_valid, odp_port_t port_no)
7063 {
7064 struct netdev_flow_key *key = &keys[0];
7065 size_t n_missed = 0, n_emc_hit = 0;
7066 struct dfc_cache *cache = &pmd->flow_cache;
7067 struct dp_packet *packet;
7068 const size_t cnt = dp_packet_batch_size(packets_);
7069 uint32_t cur_min = pmd->ctx.emc_insert_min;
7070 int i;
7071 uint16_t tcp_flags;
7072 bool smc_enable_db;
7073 size_t map_cnt = 0;
7074 bool batch_enable = true;
7075
7076 atomic_read_relaxed(&pmd->dp->smc_enable_db, &smc_enable_db);
7077 pmd_perf_update_counter(&pmd->perf_stats,
7078 md_is_valid ? PMD_STAT_RECIRC : PMD_STAT_RECV,
7079 cnt);
7080
7081 DP_PACKET_BATCH_REFILL_FOR_EACH (i, cnt, packet, packets_) {
7082 struct dp_netdev_flow *flow;
7083 uint32_t mark;
7084
7085 if (OVS_UNLIKELY(dp_packet_size(packet) < ETH_HEADER_LEN)) {
7086 dp_packet_delete(packet);
7087 COVERAGE_INC(datapath_drop_rx_invalid_packet);
7088 continue;
7089 }
7090
7091 if (i != cnt - 1) {
7092 struct dp_packet **packets = packets_->packets;
7093 /* Prefetch next packet data and metadata. */
7094 OVS_PREFETCH(dp_packet_data(packets[i+1]));
7095 pkt_metadata_prefetch_init(&packets[i+1]->md);
7096 }
7097
7098 if (!md_is_valid) {
7099 pkt_metadata_init(&packet->md, port_no);
7100 }
7101
7102 if ((*recirc_depth_get() == 0) &&
7103 dp_packet_has_flow_mark(packet, &mark)) {
7104 flow = mark_to_flow_find(pmd, mark);
7105 if (OVS_LIKELY(flow)) {
7106 tcp_flags = parse_tcp_flags(packet);
7107 if (OVS_LIKELY(batch_enable)) {
7108 dp_netdev_queue_batches(packet, flow, tcp_flags, batches,
7109 n_batches);
7110 } else {
7111 /* Flow batching should be performed only after fast-path
7112 * processing is also completed for packets with emc miss
7113 * or else it will result in reordering of packets with
7114 * same datapath flows. */
7115 packet_enqueue_to_flow_map(packet, flow, tcp_flags,
7116 flow_map, map_cnt++);
7117 }
7118 continue;
7119 }
7120 }
7121
7122 miniflow_extract(packet, &key->mf);
7123 key->len = 0; /* Not computed yet. */
7124 key->hash =
7125 (md_is_valid == false)
7126 ? dpif_netdev_packet_get_rss_hash_orig_pkt(packet, &key->mf)
7127 : dpif_netdev_packet_get_rss_hash(packet, &key->mf);
7128
7129 /* If EMC is disabled skip emc_lookup */
7130 flow = (cur_min != 0) ? emc_lookup(&cache->emc_cache, key) : NULL;
7131 if (OVS_LIKELY(flow)) {
7132 tcp_flags = miniflow_get_tcp_flags(&key->mf);
7133 n_emc_hit++;
7134 if (OVS_LIKELY(batch_enable)) {
7135 dp_netdev_queue_batches(packet, flow, tcp_flags, batches,
7136 n_batches);
7137 } else {
7138 /* Flow batching should be performed only after fast-path
7139 * processing is also completed for packets with emc miss
7140 * or else it will result in reordering of packets with
7141 * same datapath flows. */
7142 packet_enqueue_to_flow_map(packet, flow, tcp_flags,
7143 flow_map, map_cnt++);
7144 }
7145 } else {
7146 /* Exact match cache missed. Group missed packets together at
7147 * the beginning of the 'packets' array. */
7148 dp_packet_batch_refill(packets_, packet, i);
7149
7150 /* Preserve the order of packet for flow batching. */
7151 index_map[n_missed] = map_cnt;
7152 flow_map[map_cnt++].flow = NULL;
7153
7154 /* 'key[n_missed]' contains the key of the current packet and it
7155 * will be passed to SMC lookup. The next key should be extracted
7156 * to 'keys[n_missed + 1]'.
7157 * We also maintain a pointer array to keys missed both SMC and EMC
7158 * which will be returned to the caller for future processing. */
7159 missed_keys[n_missed] = key;
7160 key = &keys[++n_missed];
7161
7162 /* Skip batching for subsequent packets to avoid reordering. */
7163 batch_enable = false;
7164 }
7165 }
7166 /* Count of packets which are not flow batched. */
7167 *n_flows = map_cnt;
7168
7169 pmd_perf_update_counter(&pmd->perf_stats, PMD_STAT_EXACT_HIT, n_emc_hit);
7170
7171 if (!smc_enable_db) {
7172 return dp_packet_batch_size(packets_);
7173 }
7174
7175 /* Packets miss EMC will do a batch lookup in SMC if enabled */
7176 smc_lookup_batch(pmd, keys, missed_keys, packets_,
7177 n_missed, flow_map, index_map);
7178
7179 return dp_packet_batch_size(packets_);
7180 }
7181
7182 static inline int
7183 handle_packet_upcall(struct dp_netdev_pmd_thread *pmd,
7184 struct dp_packet *packet,
7185 const struct netdev_flow_key *key,
7186 struct ofpbuf *actions, struct ofpbuf *put_actions)
7187 {
7188 struct ofpbuf *add_actions;
7189 struct dp_packet_batch b;
7190 struct match match;
7191 ovs_u128 ufid;
7192 int error;
7193 uint64_t cycles = cycles_counter_update(&pmd->perf_stats);
7194
7195 match.tun_md.valid = false;
7196 miniflow_expand(&key->mf, &match.flow);
7197 memset(&match.wc, 0, sizeof match.wc);
7198
7199 ofpbuf_clear(actions);
7200 ofpbuf_clear(put_actions);
7201
7202 odp_flow_key_hash(&match.flow, sizeof match.flow, &ufid);
7203 error = dp_netdev_upcall(pmd, packet, &match.flow, &match.wc,
7204 &ufid, DPIF_UC_MISS, NULL, actions,
7205 put_actions);
7206 if (OVS_UNLIKELY(error && error != ENOSPC)) {
7207 dp_packet_delete(packet);
7208 COVERAGE_INC(datapath_drop_upcall_error);
7209 return error;
7210 }
7211
7212 /* The Netlink encoding of datapath flow keys cannot express
7213 * wildcarding the presence of a VLAN tag. Instead, a missing VLAN
7214 * tag is interpreted as exact match on the fact that there is no
7215 * VLAN. Unless we refactor a lot of code that translates between
7216 * Netlink and struct flow representations, we have to do the same
7217 * here. This must be in sync with 'match' in dpif_netdev_flow_put(). */
7218 if (!match.wc.masks.vlans[0].tci) {
7219 match.wc.masks.vlans[0].tci = htons(0xffff);
7220 }
7221
7222 /* We can't allow the packet batching in the next loop to execute
7223 * the actions. Otherwise, if there are any slow path actions,
7224 * we'll send the packet up twice. */
7225 dp_packet_batch_init_packet(&b, packet);
7226 dp_netdev_execute_actions(pmd, &b, true, &match.flow,
7227 actions->data, actions->size);
7228
7229 add_actions = put_actions->size ? put_actions : actions;
7230 if (OVS_LIKELY(error != ENOSPC)) {
7231 struct dp_netdev_flow *netdev_flow;
7232
7233 /* XXX: There's a race window where a flow covering this packet
7234 * could have already been installed since we last did the flow
7235 * lookup before upcall. This could be solved by moving the
7236 * mutex lock outside the loop, but that's an awful long time
7237 * to be locking revalidators out of making flow modifications. */
7238 ovs_mutex_lock(&pmd->flow_mutex);
7239 netdev_flow = dp_netdev_pmd_lookup_flow(pmd, key, NULL);
7240 if (OVS_LIKELY(!netdev_flow)) {
7241 netdev_flow = dp_netdev_flow_add(pmd, &match, &ufid,
7242 add_actions->data,
7243 add_actions->size);
7244 }
7245 ovs_mutex_unlock(&pmd->flow_mutex);
7246 uint32_t hash = dp_netdev_flow_hash(&netdev_flow->ufid);
7247 smc_insert(pmd, key, hash);
7248 emc_probabilistic_insert(pmd, key, netdev_flow);
7249 }
7250 if (pmd_perf_metrics_enabled(pmd)) {
7251 /* Update upcall stats. */
7252 cycles = cycles_counter_update(&pmd->perf_stats) - cycles;
7253 struct pmd_perf_stats *s = &pmd->perf_stats;
7254 s->current.upcalls++;
7255 s->current.upcall_cycles += cycles;
7256 histogram_add_sample(&s->cycles_per_upcall, cycles);
7257 }
7258 return error;
7259 }
7260
7261 static inline void
7262 fast_path_processing(struct dp_netdev_pmd_thread *pmd,
7263 struct dp_packet_batch *packets_,
7264 struct netdev_flow_key **keys,
7265 struct dp_packet_flow_map *flow_map,
7266 uint8_t *index_map,
7267 odp_port_t in_port)
7268 {
7269 const size_t cnt = dp_packet_batch_size(packets_);
7270 #if !defined(__CHECKER__) && !defined(_WIN32)
7271 const size_t PKT_ARRAY_SIZE = cnt;
7272 #else
7273 /* Sparse or MSVC doesn't like variable length array. */
7274 enum { PKT_ARRAY_SIZE = NETDEV_MAX_BURST };
7275 #endif
7276 struct dp_packet *packet;
7277 struct dpcls *cls;
7278 struct dpcls_rule *rules[PKT_ARRAY_SIZE];
7279 struct dp_netdev *dp = pmd->dp;
7280 int upcall_ok_cnt = 0, upcall_fail_cnt = 0;
7281 int lookup_cnt = 0, add_lookup_cnt;
7282 bool any_miss;
7283
7284 for (size_t i = 0; i < cnt; i++) {
7285 /* Key length is needed in all the cases, hash computed on demand. */
7286 keys[i]->len = netdev_flow_key_size(miniflow_n_values(&keys[i]->mf));
7287 }
7288 /* Get the classifier for the in_port */
7289 cls = dp_netdev_pmd_lookup_dpcls(pmd, in_port);
7290 if (OVS_LIKELY(cls)) {
7291 any_miss = !dpcls_lookup(cls, (const struct netdev_flow_key **)keys,
7292 rules, cnt, &lookup_cnt);
7293 } else {
7294 any_miss = true;
7295 memset(rules, 0, sizeof(rules));
7296 }
7297 if (OVS_UNLIKELY(any_miss) && !fat_rwlock_tryrdlock(&dp->upcall_rwlock)) {
7298 uint64_t actions_stub[512 / 8], slow_stub[512 / 8];
7299 struct ofpbuf actions, put_actions;
7300
7301 ofpbuf_use_stub(&actions, actions_stub, sizeof actions_stub);
7302 ofpbuf_use_stub(&put_actions, slow_stub, sizeof slow_stub);
7303
7304 DP_PACKET_BATCH_FOR_EACH (i, packet, packets_) {
7305 struct dp_netdev_flow *netdev_flow;
7306
7307 if (OVS_LIKELY(rules[i])) {
7308 continue;
7309 }
7310
7311 /* It's possible that an earlier slow path execution installed
7312 * a rule covering this flow. In this case, it's a lot cheaper
7313 * to catch it here than execute a miss. */
7314 netdev_flow = dp_netdev_pmd_lookup_flow(pmd, keys[i],
7315 &add_lookup_cnt);
7316 if (netdev_flow) {
7317 lookup_cnt += add_lookup_cnt;
7318 rules[i] = &netdev_flow->cr;
7319 continue;
7320 }
7321
7322 int error = handle_packet_upcall(pmd, packet, keys[i],
7323 &actions, &put_actions);
7324
7325 if (OVS_UNLIKELY(error)) {
7326 upcall_fail_cnt++;
7327 } else {
7328 upcall_ok_cnt++;
7329 }
7330 }
7331
7332 ofpbuf_uninit(&actions);
7333 ofpbuf_uninit(&put_actions);
7334 fat_rwlock_unlock(&dp->upcall_rwlock);
7335 } else if (OVS_UNLIKELY(any_miss)) {
7336 DP_PACKET_BATCH_FOR_EACH (i, packet, packets_) {
7337 if (OVS_UNLIKELY(!rules[i])) {
7338 dp_packet_delete(packet);
7339 COVERAGE_INC(datapath_drop_lock_error);
7340 upcall_fail_cnt++;
7341 }
7342 }
7343 }
7344
7345 DP_PACKET_BATCH_FOR_EACH (i, packet, packets_) {
7346 struct dp_netdev_flow *flow;
7347 /* Get the original order of this packet in received batch. */
7348 int recv_idx = index_map[i];
7349 uint16_t tcp_flags;
7350
7351 if (OVS_UNLIKELY(!rules[i])) {
7352 continue;
7353 }
7354
7355 flow = dp_netdev_flow_cast(rules[i]);
7356 uint32_t hash = dp_netdev_flow_hash(&flow->ufid);
7357 smc_insert(pmd, keys[i], hash);
7358
7359 emc_probabilistic_insert(pmd, keys[i], flow);
7360 /* Add these packets into the flow map in the same order
7361 * as received.
7362 */
7363 tcp_flags = miniflow_get_tcp_flags(&keys[i]->mf);
7364 packet_enqueue_to_flow_map(packet, flow, tcp_flags,
7365 flow_map, recv_idx);
7366 }
7367
7368 pmd_perf_update_counter(&pmd->perf_stats, PMD_STAT_MASKED_HIT,
7369 cnt - upcall_ok_cnt - upcall_fail_cnt);
7370 pmd_perf_update_counter(&pmd->perf_stats, PMD_STAT_MASKED_LOOKUP,
7371 lookup_cnt);
7372 pmd_perf_update_counter(&pmd->perf_stats, PMD_STAT_MISS,
7373 upcall_ok_cnt);
7374 pmd_perf_update_counter(&pmd->perf_stats, PMD_STAT_LOST,
7375 upcall_fail_cnt);
7376 }
7377
7378 /* Packets enter the datapath from a port (or from recirculation) here.
7379 *
7380 * When 'md_is_valid' is true the metadata in 'packets' are already valid.
7381 * When false the metadata in 'packets' need to be initialized. */
7382 static void
7383 dp_netdev_input__(struct dp_netdev_pmd_thread *pmd,
7384 struct dp_packet_batch *packets,
7385 bool md_is_valid, odp_port_t port_no)
7386 {
7387 #if !defined(__CHECKER__) && !defined(_WIN32)
7388 const size_t PKT_ARRAY_SIZE = dp_packet_batch_size(packets);
7389 #else
7390 /* Sparse or MSVC doesn't like variable length array. */
7391 enum { PKT_ARRAY_SIZE = NETDEV_MAX_BURST };
7392 #endif
7393 OVS_ALIGNED_VAR(CACHE_LINE_SIZE)
7394 struct netdev_flow_key keys[PKT_ARRAY_SIZE];
7395 struct netdev_flow_key *missed_keys[PKT_ARRAY_SIZE];
7396 struct packet_batch_per_flow batches[PKT_ARRAY_SIZE];
7397 size_t n_batches;
7398 struct dp_packet_flow_map flow_map[PKT_ARRAY_SIZE];
7399 uint8_t index_map[PKT_ARRAY_SIZE];
7400 size_t n_flows, i;
7401
7402 odp_port_t in_port;
7403
7404 n_batches = 0;
7405 dfc_processing(pmd, packets, keys, missed_keys, batches, &n_batches,
7406 flow_map, &n_flows, index_map, md_is_valid, port_no);
7407
7408 if (!dp_packet_batch_is_empty(packets)) {
7409 /* Get ingress port from first packet's metadata. */
7410 in_port = packets->packets[0]->md.in_port.odp_port;
7411 fast_path_processing(pmd, packets, missed_keys,
7412 flow_map, index_map, in_port);
7413 }
7414
7415 /* Batch rest of packets which are in flow map. */
7416 for (i = 0; i < n_flows; i++) {
7417 struct dp_packet_flow_map *map = &flow_map[i];
7418
7419 if (OVS_UNLIKELY(!map->flow)) {
7420 continue;
7421 }
7422 dp_netdev_queue_batches(map->packet, map->flow, map->tcp_flags,
7423 batches, &n_batches);
7424 }
7425
7426 /* All the flow batches need to be reset before any call to
7427 * packet_batch_per_flow_execute() as it could potentially trigger
7428 * recirculation. When a packet matching flow ‘j’ happens to be
7429 * recirculated, the nested call to dp_netdev_input__() could potentially
7430 * classify the packet as matching another flow - say 'k'. It could happen
7431 * that in the previous call to dp_netdev_input__() that same flow 'k' had
7432 * already its own batches[k] still waiting to be served. So if its
7433 * ‘batch’ member is not reset, the recirculated packet would be wrongly
7434 * appended to batches[k] of the 1st call to dp_netdev_input__(). */
7435 for (i = 0; i < n_batches; i++) {
7436 batches[i].flow->batch = NULL;
7437 }
7438
7439 for (i = 0; i < n_batches; i++) {
7440 packet_batch_per_flow_execute(&batches[i], pmd);
7441 }
7442 }
7443
7444 static void
7445 dp_netdev_input(struct dp_netdev_pmd_thread *pmd,
7446 struct dp_packet_batch *packets,
7447 odp_port_t port_no)
7448 {
7449 dp_netdev_input__(pmd, packets, false, port_no);
7450 }
7451
7452 static void
7453 dp_netdev_recirculate(struct dp_netdev_pmd_thread *pmd,
7454 struct dp_packet_batch *packets)
7455 {
7456 dp_netdev_input__(pmd, packets, true, 0);
7457 }
7458
7459 struct dp_netdev_execute_aux {
7460 struct dp_netdev_pmd_thread *pmd;
7461 const struct flow *flow;
7462 };
7463
7464 static void
7465 dpif_netdev_register_dp_purge_cb(struct dpif *dpif, dp_purge_callback *cb,
7466 void *aux)
7467 {
7468 struct dp_netdev *dp = get_dp_netdev(dpif);
7469 dp->dp_purge_aux = aux;
7470 dp->dp_purge_cb = cb;
7471 }
7472
7473 static void
7474 dpif_netdev_register_upcall_cb(struct dpif *dpif, upcall_callback *cb,
7475 void *aux)
7476 {
7477 struct dp_netdev *dp = get_dp_netdev(dpif);
7478 dp->upcall_aux = aux;
7479 dp->upcall_cb = cb;
7480 }
7481
7482 static void
7483 dpif_netdev_xps_revalidate_pmd(const struct dp_netdev_pmd_thread *pmd,
7484 bool purge)
7485 {
7486 struct tx_port *tx;
7487 struct dp_netdev_port *port;
7488 long long interval;
7489
7490 HMAP_FOR_EACH (tx, node, &pmd->send_port_cache) {
7491 if (!tx->port->dynamic_txqs) {
7492 continue;
7493 }
7494 interval = pmd->ctx.now - tx->last_used;
7495 if (tx->qid >= 0 && (purge || interval >= XPS_TIMEOUT)) {
7496 port = tx->port;
7497 ovs_mutex_lock(&port->txq_used_mutex);
7498 port->txq_used[tx->qid]--;
7499 ovs_mutex_unlock(&port->txq_used_mutex);
7500 tx->qid = -1;
7501 }
7502 }
7503 }
7504
7505 static int
7506 dpif_netdev_xps_get_tx_qid(const struct dp_netdev_pmd_thread *pmd,
7507 struct tx_port *tx)
7508 {
7509 struct dp_netdev_port *port;
7510 long long interval;
7511 int i, min_cnt, min_qid;
7512
7513 interval = pmd->ctx.now - tx->last_used;
7514 tx->last_used = pmd->ctx.now;
7515
7516 if (OVS_LIKELY(tx->qid >= 0 && interval < XPS_TIMEOUT)) {
7517 return tx->qid;
7518 }
7519
7520 port = tx->port;
7521
7522 ovs_mutex_lock(&port->txq_used_mutex);
7523 if (tx->qid >= 0) {
7524 port->txq_used[tx->qid]--;
7525 tx->qid = -1;
7526 }
7527
7528 min_cnt = -1;
7529 min_qid = 0;
7530 for (i = 0; i < netdev_n_txq(port->netdev); i++) {
7531 if (port->txq_used[i] < min_cnt || min_cnt == -1) {
7532 min_cnt = port->txq_used[i];
7533 min_qid = i;
7534 }
7535 }
7536
7537 port->txq_used[min_qid]++;
7538 tx->qid = min_qid;
7539
7540 ovs_mutex_unlock(&port->txq_used_mutex);
7541
7542 dpif_netdev_xps_revalidate_pmd(pmd, false);
7543
7544 VLOG_DBG("Core %d: New TX queue ID %d for port \'%s\'.",
7545 pmd->core_id, tx->qid, netdev_get_name(tx->port->netdev));
7546 return min_qid;
7547 }
7548
7549 static struct tx_port *
7550 pmd_tnl_port_cache_lookup(const struct dp_netdev_pmd_thread *pmd,
7551 odp_port_t port_no)
7552 {
7553 return tx_port_lookup(&pmd->tnl_port_cache, port_no);
7554 }
7555
7556 static struct tx_port *
7557 pmd_send_port_cache_lookup(const struct dp_netdev_pmd_thread *pmd,
7558 odp_port_t port_no)
7559 {
7560 return tx_port_lookup(&pmd->send_port_cache, port_no);
7561 }
7562
7563 static int
7564 push_tnl_action(const struct dp_netdev_pmd_thread *pmd,
7565 const struct nlattr *attr,
7566 struct dp_packet_batch *batch)
7567 {
7568 struct tx_port *tun_port;
7569 const struct ovs_action_push_tnl *data;
7570 int err;
7571
7572 data = nl_attr_get(attr);
7573
7574 tun_port = pmd_tnl_port_cache_lookup(pmd, data->tnl_port);
7575 if (!tun_port) {
7576 err = -EINVAL;
7577 goto error;
7578 }
7579 err = netdev_push_header(tun_port->port->netdev, batch, data);
7580 if (!err) {
7581 return 0;
7582 }
7583 error:
7584 dp_packet_delete_batch(batch, true);
7585 return err;
7586 }
7587
7588 static void
7589 dp_execute_userspace_action(struct dp_netdev_pmd_thread *pmd,
7590 struct dp_packet *packet, bool should_steal,
7591 struct flow *flow, ovs_u128 *ufid,
7592 struct ofpbuf *actions,
7593 const struct nlattr *userdata)
7594 {
7595 struct dp_packet_batch b;
7596 int error;
7597
7598 ofpbuf_clear(actions);
7599
7600 error = dp_netdev_upcall(pmd, packet, flow, NULL, ufid,
7601 DPIF_UC_ACTION, userdata, actions,
7602 NULL);
7603 if (!error || error == ENOSPC) {
7604 dp_packet_batch_init_packet(&b, packet);
7605 dp_netdev_execute_actions(pmd, &b, should_steal, flow,
7606 actions->data, actions->size);
7607 } else if (should_steal) {
7608 dp_packet_delete(packet);
7609 COVERAGE_INC(datapath_drop_userspace_action_error);
7610 }
7611 }
7612
7613 static bool
7614 dp_execute_output_action(struct dp_netdev_pmd_thread *pmd,
7615 struct dp_packet_batch *packets_,
7616 bool should_steal, odp_port_t port_no)
7617 {
7618 struct tx_port *p = pmd_send_port_cache_lookup(pmd, port_no);
7619 struct dp_packet_batch out;
7620
7621 if (!OVS_LIKELY(p)) {
7622 COVERAGE_ADD(datapath_drop_invalid_port,
7623 dp_packet_batch_size(packets_));
7624 dp_packet_delete_batch(packets_, should_steal);
7625 return false;
7626 }
7627 if (!should_steal) {
7628 dp_packet_batch_clone(&out, packets_);
7629 dp_packet_batch_reset_cutlen(packets_);
7630 packets_ = &out;
7631 }
7632 dp_packet_batch_apply_cutlen(packets_);
7633 #ifdef DPDK_NETDEV
7634 if (OVS_UNLIKELY(!dp_packet_batch_is_empty(&p->output_pkts)
7635 && packets_->packets[0]->source
7636 != p->output_pkts.packets[0]->source)) {
7637 /* XXX: netdev-dpdk assumes that all packets in a single
7638 * output batch has the same source. Flush here to
7639 * avoid memory access issues. */
7640 dp_netdev_pmd_flush_output_on_port(pmd, p);
7641 }
7642 #endif
7643 if (dp_packet_batch_size(&p->output_pkts)
7644 + dp_packet_batch_size(packets_) > NETDEV_MAX_BURST) {
7645 /* Flush here to avoid overflow. */
7646 dp_netdev_pmd_flush_output_on_port(pmd, p);
7647 }
7648 if (dp_packet_batch_is_empty(&p->output_pkts)) {
7649 pmd->n_output_batches++;
7650 }
7651
7652 struct dp_packet *packet;
7653 DP_PACKET_BATCH_FOR_EACH (i, packet, packets_) {
7654 p->output_pkts_rxqs[dp_packet_batch_size(&p->output_pkts)] =
7655 pmd->ctx.last_rxq;
7656 dp_packet_batch_add(&p->output_pkts, packet);
7657 }
7658 return true;
7659 }
7660
7661 static void
7662 dp_execute_lb_output_action(struct dp_netdev_pmd_thread *pmd,
7663 struct dp_packet_batch *packets_,
7664 bool should_steal, uint32_t bond)
7665 {
7666 struct tx_bond *p_bond = tx_bond_lookup(&pmd->tx_bonds, bond);
7667 struct dp_packet_batch out;
7668 struct dp_packet *packet;
7669
7670 if (!p_bond) {
7671 COVERAGE_ADD(datapath_drop_invalid_bond,
7672 dp_packet_batch_size(packets_));
7673 dp_packet_delete_batch(packets_, should_steal);
7674 return;
7675 }
7676 if (!should_steal) {
7677 dp_packet_batch_clone(&out, packets_);
7678 dp_packet_batch_reset_cutlen(packets_);
7679 packets_ = &out;
7680 }
7681 dp_packet_batch_apply_cutlen(packets_);
7682
7683 DP_PACKET_BATCH_FOR_EACH (i, packet, packets_) {
7684 /*
7685 * Lookup the bond-hash table using hash to get the member.
7686 */
7687 uint32_t hash = dp_packet_get_rss_hash(packet);
7688 struct member_entry *s_entry
7689 = &p_bond->member_buckets[hash & BOND_MASK];
7690 odp_port_t bond_member = s_entry->member_id;
7691 uint32_t size = dp_packet_size(packet);
7692 struct dp_packet_batch output_pkt;
7693
7694 dp_packet_batch_init_packet(&output_pkt, packet);
7695 if (OVS_LIKELY(dp_execute_output_action(pmd, &output_pkt, true,
7696 bond_member))) {
7697 /* Update member stats. */
7698 non_atomic_ullong_add(&s_entry->n_packets, 1);
7699 non_atomic_ullong_add(&s_entry->n_bytes, size);
7700 }
7701 }
7702 }
7703
7704 static void
7705 dp_execute_cb(void *aux_, struct dp_packet_batch *packets_,
7706 const struct nlattr *a, bool should_steal)
7707 OVS_NO_THREAD_SAFETY_ANALYSIS
7708 {
7709 struct dp_netdev_execute_aux *aux = aux_;
7710 uint32_t *depth = recirc_depth_get();
7711 struct dp_netdev_pmd_thread *pmd = aux->pmd;
7712 struct dp_netdev *dp = pmd->dp;
7713 int type = nl_attr_type(a);
7714 struct tx_port *p;
7715 uint32_t packet_count, packets_dropped;
7716
7717 switch ((enum ovs_action_attr)type) {
7718 case OVS_ACTION_ATTR_OUTPUT:
7719 dp_execute_output_action(pmd, packets_, should_steal,
7720 nl_attr_get_odp_port(a));
7721 return;
7722
7723 case OVS_ACTION_ATTR_LB_OUTPUT:
7724 dp_execute_lb_output_action(pmd, packets_, should_steal,
7725 nl_attr_get_u32(a));
7726 return;
7727
7728 case OVS_ACTION_ATTR_TUNNEL_PUSH:
7729 if (should_steal) {
7730 /* We're requested to push tunnel header, but also we need to take
7731 * the ownership of these packets. Thus, we can avoid performing
7732 * the action, because the caller will not use the result anyway.
7733 * Just break to free the batch. */
7734 break;
7735 }
7736 dp_packet_batch_apply_cutlen(packets_);
7737 packet_count = dp_packet_batch_size(packets_);
7738 if (push_tnl_action(pmd, a, packets_)) {
7739 COVERAGE_ADD(datapath_drop_tunnel_push_error,
7740 packet_count);
7741 }
7742 return;
7743
7744 case OVS_ACTION_ATTR_TUNNEL_POP:
7745 if (*depth < MAX_RECIRC_DEPTH) {
7746 struct dp_packet_batch *orig_packets_ = packets_;
7747 odp_port_t portno = nl_attr_get_odp_port(a);
7748
7749 p = pmd_tnl_port_cache_lookup(pmd, portno);
7750 if (p) {
7751 struct dp_packet_batch tnl_pkt;
7752
7753 if (!should_steal) {
7754 dp_packet_batch_clone(&tnl_pkt, packets_);
7755 packets_ = &tnl_pkt;
7756 dp_packet_batch_reset_cutlen(orig_packets_);
7757 }
7758
7759 dp_packet_batch_apply_cutlen(packets_);
7760
7761 packet_count = dp_packet_batch_size(packets_);
7762 netdev_pop_header(p->port->netdev, packets_);
7763 packets_dropped =
7764 packet_count - dp_packet_batch_size(packets_);
7765 if (packets_dropped) {
7766 COVERAGE_ADD(datapath_drop_tunnel_pop_error,
7767 packets_dropped);
7768 }
7769 if (dp_packet_batch_is_empty(packets_)) {
7770 return;
7771 }
7772
7773 struct dp_packet *packet;
7774 DP_PACKET_BATCH_FOR_EACH (i, packet, packets_) {
7775 packet->md.in_port.odp_port = portno;
7776 }
7777
7778 (*depth)++;
7779 dp_netdev_recirculate(pmd, packets_);
7780 (*depth)--;
7781 return;
7782 }
7783 COVERAGE_ADD(datapath_drop_invalid_tnl_port,
7784 dp_packet_batch_size(packets_));
7785 } else {
7786 COVERAGE_ADD(datapath_drop_recirc_error,
7787 dp_packet_batch_size(packets_));
7788 }
7789 break;
7790
7791 case OVS_ACTION_ATTR_USERSPACE:
7792 if (!fat_rwlock_tryrdlock(&dp->upcall_rwlock)) {
7793 struct dp_packet_batch *orig_packets_ = packets_;
7794 const struct nlattr *userdata;
7795 struct dp_packet_batch usr_pkt;
7796 struct ofpbuf actions;
7797 struct flow flow;
7798 ovs_u128 ufid;
7799 bool clone = false;
7800
7801 userdata = nl_attr_find_nested(a, OVS_USERSPACE_ATTR_USERDATA);
7802 ofpbuf_init(&actions, 0);
7803
7804 if (packets_->trunc) {
7805 if (!should_steal) {
7806 dp_packet_batch_clone(&usr_pkt, packets_);
7807 packets_ = &usr_pkt;
7808 clone = true;
7809 dp_packet_batch_reset_cutlen(orig_packets_);
7810 }
7811
7812 dp_packet_batch_apply_cutlen(packets_);
7813 }
7814
7815 struct dp_packet *packet;
7816 DP_PACKET_BATCH_FOR_EACH (i, packet, packets_) {
7817 flow_extract(packet, &flow);
7818 odp_flow_key_hash(&flow, sizeof flow, &ufid);
7819 dp_execute_userspace_action(pmd, packet, should_steal, &flow,
7820 &ufid, &actions, userdata);
7821 }
7822
7823 if (clone) {
7824 dp_packet_delete_batch(packets_, true);
7825 }
7826
7827 ofpbuf_uninit(&actions);
7828 fat_rwlock_unlock(&dp->upcall_rwlock);
7829
7830 return;
7831 }
7832 COVERAGE_ADD(datapath_drop_lock_error,
7833 dp_packet_batch_size(packets_));
7834 break;
7835
7836 case OVS_ACTION_ATTR_RECIRC:
7837 if (*depth < MAX_RECIRC_DEPTH) {
7838 struct dp_packet_batch recirc_pkts;
7839
7840 if (!should_steal) {
7841 dp_packet_batch_clone(&recirc_pkts, packets_);
7842 packets_ = &recirc_pkts;
7843 }
7844
7845 struct dp_packet *packet;
7846 DP_PACKET_BATCH_FOR_EACH (i, packet, packets_) {
7847 packet->md.recirc_id = nl_attr_get_u32(a);
7848 }
7849
7850 (*depth)++;
7851 dp_netdev_recirculate(pmd, packets_);
7852 (*depth)--;
7853
7854 return;
7855 }
7856
7857 COVERAGE_ADD(datapath_drop_recirc_error,
7858 dp_packet_batch_size(packets_));
7859 VLOG_WARN("Packet dropped. Max recirculation depth exceeded.");
7860 break;
7861
7862 case OVS_ACTION_ATTR_CT: {
7863 const struct nlattr *b;
7864 bool force = false;
7865 bool commit = false;
7866 unsigned int left;
7867 uint16_t zone = 0;
7868 uint32_t tp_id = 0;
7869 const char *helper = NULL;
7870 const uint32_t *setmark = NULL;
7871 const struct ovs_key_ct_labels *setlabel = NULL;
7872 struct nat_action_info_t nat_action_info;
7873 struct nat_action_info_t *nat_action_info_ref = NULL;
7874 bool nat_config = false;
7875
7876 NL_ATTR_FOR_EACH_UNSAFE (b, left, nl_attr_get(a),
7877 nl_attr_get_size(a)) {
7878 enum ovs_ct_attr sub_type = nl_attr_type(b);
7879
7880 switch(sub_type) {
7881 case OVS_CT_ATTR_FORCE_COMMIT:
7882 force = true;
7883 /* fall through. */
7884 case OVS_CT_ATTR_COMMIT:
7885 commit = true;
7886 break;
7887 case OVS_CT_ATTR_ZONE:
7888 zone = nl_attr_get_u16(b);
7889 break;
7890 case OVS_CT_ATTR_HELPER:
7891 helper = nl_attr_get_string(b);
7892 break;
7893 case OVS_CT_ATTR_MARK:
7894 setmark = nl_attr_get(b);
7895 break;
7896 case OVS_CT_ATTR_LABELS:
7897 setlabel = nl_attr_get(b);
7898 break;
7899 case OVS_CT_ATTR_EVENTMASK:
7900 /* Silently ignored, as userspace datapath does not generate
7901 * netlink events. */
7902 break;
7903 case OVS_CT_ATTR_TIMEOUT:
7904 if (!str_to_uint(nl_attr_get_string(b), 10, &tp_id)) {
7905 VLOG_WARN("Invalid Timeout Policy ID: %s.",
7906 nl_attr_get_string(b));
7907 tp_id = DEFAULT_TP_ID;
7908 }
7909 break;
7910 case OVS_CT_ATTR_NAT: {
7911 const struct nlattr *b_nest;
7912 unsigned int left_nest;
7913 bool ip_min_specified = false;
7914 bool proto_num_min_specified = false;
7915 bool ip_max_specified = false;
7916 bool proto_num_max_specified = false;
7917 memset(&nat_action_info, 0, sizeof nat_action_info);
7918 nat_action_info_ref = &nat_action_info;
7919
7920 NL_NESTED_FOR_EACH_UNSAFE (b_nest, left_nest, b) {
7921 enum ovs_nat_attr sub_type_nest = nl_attr_type(b_nest);
7922
7923 switch (sub_type_nest) {
7924 case OVS_NAT_ATTR_SRC:
7925 case OVS_NAT_ATTR_DST:
7926 nat_config = true;
7927 nat_action_info.nat_action |=
7928 ((sub_type_nest == OVS_NAT_ATTR_SRC)
7929 ? NAT_ACTION_SRC : NAT_ACTION_DST);
7930 break;
7931 case OVS_NAT_ATTR_IP_MIN:
7932 memcpy(&nat_action_info.min_addr,
7933 nl_attr_get(b_nest),
7934 nl_attr_get_size(b_nest));
7935 ip_min_specified = true;
7936 break;
7937 case OVS_NAT_ATTR_IP_MAX:
7938 memcpy(&nat_action_info.max_addr,
7939 nl_attr_get(b_nest),
7940 nl_attr_get_size(b_nest));
7941 ip_max_specified = true;
7942 break;
7943 case OVS_NAT_ATTR_PROTO_MIN:
7944 nat_action_info.min_port =
7945 nl_attr_get_u16(b_nest);
7946 proto_num_min_specified = true;
7947 break;
7948 case OVS_NAT_ATTR_PROTO_MAX:
7949 nat_action_info.max_port =
7950 nl_attr_get_u16(b_nest);
7951 proto_num_max_specified = true;
7952 break;
7953 case OVS_NAT_ATTR_PERSISTENT:
7954 case OVS_NAT_ATTR_PROTO_HASH:
7955 case OVS_NAT_ATTR_PROTO_RANDOM:
7956 break;
7957 case OVS_NAT_ATTR_UNSPEC:
7958 case __OVS_NAT_ATTR_MAX:
7959 OVS_NOT_REACHED();
7960 }
7961 }
7962
7963 if (ip_min_specified && !ip_max_specified) {
7964 nat_action_info.max_addr = nat_action_info.min_addr;
7965 }
7966 if (proto_num_min_specified && !proto_num_max_specified) {
7967 nat_action_info.max_port = nat_action_info.min_port;
7968 }
7969 if (proto_num_min_specified || proto_num_max_specified) {
7970 if (nat_action_info.nat_action & NAT_ACTION_SRC) {
7971 nat_action_info.nat_action |= NAT_ACTION_SRC_PORT;
7972 } else if (nat_action_info.nat_action & NAT_ACTION_DST) {
7973 nat_action_info.nat_action |= NAT_ACTION_DST_PORT;
7974 }
7975 }
7976 break;
7977 }
7978 case OVS_CT_ATTR_UNSPEC:
7979 case __OVS_CT_ATTR_MAX:
7980 OVS_NOT_REACHED();
7981 }
7982 }
7983
7984 /* We won't be able to function properly in this case, hence
7985 * complain loudly. */
7986 if (nat_config && !commit) {
7987 static struct vlog_rate_limit rl = VLOG_RATE_LIMIT_INIT(5, 5);
7988 VLOG_WARN_RL(&rl, "NAT specified without commit.");
7989 }
7990
7991 conntrack_execute(dp->conntrack, packets_, aux->flow->dl_type, force,
7992 commit, zone, setmark, setlabel, aux->flow->tp_src,
7993 aux->flow->tp_dst, helper, nat_action_info_ref,
7994 pmd->ctx.now / 1000, tp_id);
7995 break;
7996 }
7997
7998 case OVS_ACTION_ATTR_METER:
7999 dp_netdev_run_meter(pmd->dp, packets_, nl_attr_get_u32(a),
8000 pmd->ctx.now);
8001 break;
8002
8003 case OVS_ACTION_ATTR_PUSH_VLAN:
8004 case OVS_ACTION_ATTR_POP_VLAN:
8005 case OVS_ACTION_ATTR_PUSH_MPLS:
8006 case OVS_ACTION_ATTR_POP_MPLS:
8007 case OVS_ACTION_ATTR_SET:
8008 case OVS_ACTION_ATTR_SET_MASKED:
8009 case OVS_ACTION_ATTR_SAMPLE:
8010 case OVS_ACTION_ATTR_HASH:
8011 case OVS_ACTION_ATTR_UNSPEC:
8012 case OVS_ACTION_ATTR_TRUNC:
8013 case OVS_ACTION_ATTR_PUSH_ETH:
8014 case OVS_ACTION_ATTR_POP_ETH:
8015 case OVS_ACTION_ATTR_CLONE:
8016 case OVS_ACTION_ATTR_PUSH_NSH:
8017 case OVS_ACTION_ATTR_POP_NSH:
8018 case OVS_ACTION_ATTR_CT_CLEAR:
8019 case OVS_ACTION_ATTR_CHECK_PKT_LEN:
8020 case OVS_ACTION_ATTR_DROP:
8021 case __OVS_ACTION_ATTR_MAX:
8022 OVS_NOT_REACHED();
8023 }
8024
8025 dp_packet_delete_batch(packets_, should_steal);
8026 }
8027
8028 static void
8029 dp_netdev_execute_actions(struct dp_netdev_pmd_thread *pmd,
8030 struct dp_packet_batch *packets,
8031 bool should_steal, const struct flow *flow,
8032 const struct nlattr *actions, size_t actions_len)
8033 {
8034 struct dp_netdev_execute_aux aux = { pmd, flow };
8035
8036 odp_execute_actions(&aux, packets, should_steal, actions,
8037 actions_len, dp_execute_cb);
8038 }
8039
8040 struct dp_netdev_ct_dump {
8041 struct ct_dpif_dump_state up;
8042 struct conntrack_dump dump;
8043 struct conntrack *ct;
8044 struct dp_netdev *dp;
8045 };
8046
8047 static int
8048 dpif_netdev_ct_dump_start(struct dpif *dpif, struct ct_dpif_dump_state **dump_,
8049 const uint16_t *pzone, int *ptot_bkts)
8050 {
8051 struct dp_netdev *dp = get_dp_netdev(dpif);
8052 struct dp_netdev_ct_dump *dump;
8053
8054 dump = xzalloc(sizeof *dump);
8055 dump->dp = dp;
8056 dump->ct = dp->conntrack;
8057
8058 conntrack_dump_start(dp->conntrack, &dump->dump, pzone, ptot_bkts);
8059
8060 *dump_ = &dump->up;
8061
8062 return 0;
8063 }
8064
8065 static int
8066 dpif_netdev_ct_dump_next(struct dpif *dpif OVS_UNUSED,
8067 struct ct_dpif_dump_state *dump_,
8068 struct ct_dpif_entry *entry)
8069 {
8070 struct dp_netdev_ct_dump *dump;
8071
8072 INIT_CONTAINER(dump, dump_, up);
8073
8074 return conntrack_dump_next(&dump->dump, entry);
8075 }
8076
8077 static int
8078 dpif_netdev_ct_dump_done(struct dpif *dpif OVS_UNUSED,
8079 struct ct_dpif_dump_state *dump_)
8080 {
8081 struct dp_netdev_ct_dump *dump;
8082 int err;
8083
8084 INIT_CONTAINER(dump, dump_, up);
8085
8086 err = conntrack_dump_done(&dump->dump);
8087
8088 free(dump);
8089
8090 return err;
8091 }
8092
8093 static int
8094 dpif_netdev_ct_flush(struct dpif *dpif, const uint16_t *zone,
8095 const struct ct_dpif_tuple *tuple)
8096 {
8097 struct dp_netdev *dp = get_dp_netdev(dpif);
8098
8099 if (tuple) {
8100 return conntrack_flush_tuple(dp->conntrack, tuple, zone ? *zone : 0);
8101 }
8102 return conntrack_flush(dp->conntrack, zone);
8103 }
8104
8105 static int
8106 dpif_netdev_ct_set_maxconns(struct dpif *dpif, uint32_t maxconns)
8107 {
8108 struct dp_netdev *dp = get_dp_netdev(dpif);
8109
8110 return conntrack_set_maxconns(dp->conntrack, maxconns);
8111 }
8112
8113 static int
8114 dpif_netdev_ct_get_maxconns(struct dpif *dpif, uint32_t *maxconns)
8115 {
8116 struct dp_netdev *dp = get_dp_netdev(dpif);
8117
8118 return conntrack_get_maxconns(dp->conntrack, maxconns);
8119 }
8120
8121 static int
8122 dpif_netdev_ct_get_nconns(struct dpif *dpif, uint32_t *nconns)
8123 {
8124 struct dp_netdev *dp = get_dp_netdev(dpif);
8125
8126 return conntrack_get_nconns(dp->conntrack, nconns);
8127 }
8128
8129 static int
8130 dpif_netdev_ct_set_tcp_seq_chk(struct dpif *dpif, bool enabled)
8131 {
8132 struct dp_netdev *dp = get_dp_netdev(dpif);
8133
8134 return conntrack_set_tcp_seq_chk(dp->conntrack, enabled);
8135 }
8136
8137 static int
8138 dpif_netdev_ct_get_tcp_seq_chk(struct dpif *dpif, bool *enabled)
8139 {
8140 struct dp_netdev *dp = get_dp_netdev(dpif);
8141 *enabled = conntrack_get_tcp_seq_chk(dp->conntrack);
8142 return 0;
8143 }
8144
8145 static int
8146 dpif_netdev_ct_set_limits(struct dpif *dpif OVS_UNUSED,
8147 const uint32_t *default_limits,
8148 const struct ovs_list *zone_limits)
8149 {
8150 int err = 0;
8151 struct dp_netdev *dp = get_dp_netdev(dpif);
8152 if (default_limits) {
8153 err = zone_limit_update(dp->conntrack, DEFAULT_ZONE, *default_limits);
8154 if (err != 0) {
8155 return err;
8156 }
8157 }
8158
8159 struct ct_dpif_zone_limit *zone_limit;
8160 LIST_FOR_EACH (zone_limit, node, zone_limits) {
8161 err = zone_limit_update(dp->conntrack, zone_limit->zone,
8162 zone_limit->limit);
8163 if (err != 0) {
8164 break;
8165 }
8166 }
8167 return err;
8168 }
8169
8170 static int
8171 dpif_netdev_ct_get_limits(struct dpif *dpif OVS_UNUSED,
8172 uint32_t *default_limit,
8173 const struct ovs_list *zone_limits_request,
8174 struct ovs_list *zone_limits_reply)
8175 {
8176 struct dp_netdev *dp = get_dp_netdev(dpif);
8177 struct conntrack_zone_limit czl;
8178
8179 czl = zone_limit_get(dp->conntrack, DEFAULT_ZONE);
8180 if (czl.zone == DEFAULT_ZONE) {
8181 *default_limit = czl.limit;
8182 } else {
8183 return EINVAL;
8184 }
8185
8186 if (!ovs_list_is_empty(zone_limits_request)) {
8187 struct ct_dpif_zone_limit *zone_limit;
8188 LIST_FOR_EACH (zone_limit, node, zone_limits_request) {
8189 czl = zone_limit_get(dp->conntrack, zone_limit->zone);
8190 if (czl.zone == zone_limit->zone || czl.zone == DEFAULT_ZONE) {
8191 ct_dpif_push_zone_limit(zone_limits_reply, zone_limit->zone,
8192 czl.limit, czl.count);
8193 } else {
8194 return EINVAL;
8195 }
8196 }
8197 } else {
8198 for (int z = MIN_ZONE; z <= MAX_ZONE; z++) {
8199 czl = zone_limit_get(dp->conntrack, z);
8200 if (czl.zone == z) {
8201 ct_dpif_push_zone_limit(zone_limits_reply, z, czl.limit,
8202 czl.count);
8203 }
8204 }
8205 }
8206
8207 return 0;
8208 }
8209
8210 static int
8211 dpif_netdev_ct_del_limits(struct dpif *dpif OVS_UNUSED,
8212 const struct ovs_list *zone_limits)
8213 {
8214 int err = 0;
8215 struct dp_netdev *dp = get_dp_netdev(dpif);
8216 struct ct_dpif_zone_limit *zone_limit;
8217 LIST_FOR_EACH (zone_limit, node, zone_limits) {
8218 err = zone_limit_delete(dp->conntrack, zone_limit->zone);
8219 if (err != 0) {
8220 break;
8221 }
8222 }
8223
8224 return err;
8225 }
8226
8227 static int
8228 dpif_netdev_ct_set_timeout_policy(struct dpif *dpif,
8229 const struct ct_dpif_timeout_policy *dpif_tp)
8230 {
8231 struct timeout_policy tp;
8232 struct dp_netdev *dp;
8233
8234 dp = get_dp_netdev(dpif);
8235 memcpy(&tp.policy, dpif_tp, sizeof tp.policy);
8236 return timeout_policy_update(dp->conntrack, &tp);
8237 }
8238
8239 static int
8240 dpif_netdev_ct_get_timeout_policy(struct dpif *dpif, uint32_t tp_id,
8241 struct ct_dpif_timeout_policy *dpif_tp)
8242 {
8243 struct timeout_policy *tp;
8244 struct dp_netdev *dp;
8245 int err = 0;
8246
8247 dp = get_dp_netdev(dpif);
8248 tp = timeout_policy_get(dp->conntrack, tp_id);
8249 if (!tp) {
8250 return ENOENT;
8251 }
8252 memcpy(dpif_tp, &tp->policy, sizeof tp->policy);
8253 return err;
8254 }
8255
8256 static int
8257 dpif_netdev_ct_del_timeout_policy(struct dpif *dpif,
8258 uint32_t tp_id)
8259 {
8260 struct dp_netdev *dp;
8261 int err = 0;
8262
8263 dp = get_dp_netdev(dpif);
8264 err = timeout_policy_delete(dp->conntrack, tp_id);
8265 return err;
8266 }
8267
8268 static int
8269 dpif_netdev_ct_get_timeout_policy_name(struct dpif *dpif OVS_UNUSED,
8270 uint32_t tp_id,
8271 uint16_t dl_type OVS_UNUSED,
8272 uint8_t nw_proto OVS_UNUSED,
8273 char **tp_name, bool *is_generic)
8274 {
8275 struct ds ds = DS_EMPTY_INITIALIZER;
8276
8277 ds_put_format(&ds, "%"PRIu32, tp_id);
8278 *tp_name = ds_steal_cstr(&ds);
8279 *is_generic = true;
8280 return 0;
8281 }
8282
8283 static int
8284 dpif_netdev_ipf_set_enabled(struct dpif *dpif, bool v6, bool enable)
8285 {
8286 struct dp_netdev *dp = get_dp_netdev(dpif);
8287 return ipf_set_enabled(conntrack_ipf_ctx(dp->conntrack), v6, enable);
8288 }
8289
8290 static int
8291 dpif_netdev_ipf_set_min_frag(struct dpif *dpif, bool v6, uint32_t min_frag)
8292 {
8293 struct dp_netdev *dp = get_dp_netdev(dpif);
8294 return ipf_set_min_frag(conntrack_ipf_ctx(dp->conntrack), v6, min_frag);
8295 }
8296
8297 static int
8298 dpif_netdev_ipf_set_max_nfrags(struct dpif *dpif, uint32_t max_frags)
8299 {
8300 struct dp_netdev *dp = get_dp_netdev(dpif);
8301 return ipf_set_max_nfrags(conntrack_ipf_ctx(dp->conntrack), max_frags);
8302 }
8303
8304 /* Adjust this function if 'dpif_ipf_status' and 'ipf_status' were to
8305 * diverge. */
8306 static int
8307 dpif_netdev_ipf_get_status(struct dpif *dpif,
8308 struct dpif_ipf_status *dpif_ipf_status)
8309 {
8310 struct dp_netdev *dp = get_dp_netdev(dpif);
8311 ipf_get_status(conntrack_ipf_ctx(dp->conntrack),
8312 (struct ipf_status *) dpif_ipf_status);
8313 return 0;
8314 }
8315
8316 static int
8317 dpif_netdev_ipf_dump_start(struct dpif *dpif OVS_UNUSED,
8318 struct ipf_dump_ctx **ipf_dump_ctx)
8319 {
8320 return ipf_dump_start(ipf_dump_ctx);
8321 }
8322
8323 static int
8324 dpif_netdev_ipf_dump_next(struct dpif *dpif, void *ipf_dump_ctx, char **dump)
8325 {
8326 struct dp_netdev *dp = get_dp_netdev(dpif);
8327 return ipf_dump_next(conntrack_ipf_ctx(dp->conntrack), ipf_dump_ctx,
8328 dump);
8329 }
8330
8331 static int
8332 dpif_netdev_ipf_dump_done(struct dpif *dpif OVS_UNUSED, void *ipf_dump_ctx)
8333 {
8334 return ipf_dump_done(ipf_dump_ctx);
8335
8336 }
8337
8338 static int
8339 dpif_netdev_bond_add(struct dpif *dpif, uint32_t bond_id,
8340 odp_port_t *member_map)
8341 {
8342 struct tx_bond *new_tx = xzalloc(sizeof *new_tx);
8343 struct dp_netdev *dp = get_dp_netdev(dpif);
8344 struct dp_netdev_pmd_thread *pmd;
8345
8346 /* Prepare new bond mapping. */
8347 new_tx->bond_id = bond_id;
8348 for (int bucket = 0; bucket < BOND_BUCKETS; bucket++) {
8349 new_tx->member_buckets[bucket].member_id = member_map[bucket];
8350 }
8351
8352 ovs_mutex_lock(&dp->bond_mutex);
8353 /* Check if bond already existed. */
8354 struct tx_bond *old_tx = tx_bond_lookup(&dp->tx_bonds, bond_id);
8355 if (old_tx) {
8356 cmap_replace(&dp->tx_bonds, &old_tx->node, &new_tx->node,
8357 hash_bond_id(bond_id));
8358 ovsrcu_postpone(free, old_tx);
8359 } else {
8360 cmap_insert(&dp->tx_bonds, &new_tx->node, hash_bond_id(bond_id));
8361 }
8362 ovs_mutex_unlock(&dp->bond_mutex);
8363
8364 /* Update all PMDs with new bond mapping. */
8365 CMAP_FOR_EACH (pmd, node, &dp->poll_threads) {
8366 dp_netdev_add_bond_tx_to_pmd(pmd, new_tx, true);
8367 }
8368 return 0;
8369 }
8370
8371 static int
8372 dpif_netdev_bond_del(struct dpif *dpif, uint32_t bond_id)
8373 {
8374 struct dp_netdev *dp = get_dp_netdev(dpif);
8375 struct dp_netdev_pmd_thread *pmd;
8376 struct tx_bond *tx;
8377
8378 ovs_mutex_lock(&dp->bond_mutex);
8379 /* Check if bond existed. */
8380 tx = tx_bond_lookup(&dp->tx_bonds, bond_id);
8381 if (tx) {
8382 cmap_remove(&dp->tx_bonds, &tx->node, hash_bond_id(bond_id));
8383 ovsrcu_postpone(free, tx);
8384 } else {
8385 /* Bond is not present. */
8386 ovs_mutex_unlock(&dp->bond_mutex);
8387 return ENOENT;
8388 }
8389 ovs_mutex_unlock(&dp->bond_mutex);
8390
8391 /* Remove the bond map in all pmds. */
8392 CMAP_FOR_EACH (pmd, node, &dp->poll_threads) {
8393 dp_netdev_del_bond_tx_from_pmd(pmd, bond_id);
8394 }
8395 return 0;
8396 }
8397
8398 static int
8399 dpif_netdev_bond_stats_get(struct dpif *dpif, uint32_t bond_id,
8400 uint64_t *n_bytes)
8401 {
8402 struct dp_netdev *dp = get_dp_netdev(dpif);
8403 struct dp_netdev_pmd_thread *pmd;
8404
8405 if (!tx_bond_lookup(&dp->tx_bonds, bond_id)) {
8406 return ENOENT;
8407 }
8408
8409 /* Search the bond in all PMDs. */
8410 CMAP_FOR_EACH (pmd, node, &dp->poll_threads) {
8411 struct tx_bond *pmd_bond_entry
8412 = tx_bond_lookup(&pmd->tx_bonds, bond_id);
8413
8414 if (!pmd_bond_entry) {
8415 continue;
8416 }
8417
8418 /* Read bond stats. */
8419 for (int i = 0; i < BOND_BUCKETS; i++) {
8420 uint64_t pmd_n_bytes;
8421
8422 atomic_read_relaxed(&pmd_bond_entry->member_buckets[i].n_bytes,
8423 &pmd_n_bytes);
8424 n_bytes[i] += pmd_n_bytes;
8425 }
8426 }
8427 return 0;
8428 }
8429
8430 const struct dpif_class dpif_netdev_class = {
8431 "netdev",
8432 true, /* cleanup_required */
8433 dpif_netdev_init,
8434 dpif_netdev_enumerate,
8435 dpif_netdev_port_open_type,
8436 dpif_netdev_open,
8437 dpif_netdev_close,
8438 dpif_netdev_destroy,
8439 dpif_netdev_run,
8440 dpif_netdev_wait,
8441 dpif_netdev_get_stats,
8442 NULL, /* set_features */
8443 dpif_netdev_port_add,
8444 dpif_netdev_port_del,
8445 dpif_netdev_port_set_config,
8446 dpif_netdev_port_query_by_number,
8447 dpif_netdev_port_query_by_name,
8448 NULL, /* port_get_pid */
8449 dpif_netdev_port_dump_start,
8450 dpif_netdev_port_dump_next,
8451 dpif_netdev_port_dump_done,
8452 dpif_netdev_port_poll,
8453 dpif_netdev_port_poll_wait,
8454 dpif_netdev_flow_flush,
8455 dpif_netdev_flow_dump_create,
8456 dpif_netdev_flow_dump_destroy,
8457 dpif_netdev_flow_dump_thread_create,
8458 dpif_netdev_flow_dump_thread_destroy,
8459 dpif_netdev_flow_dump_next,
8460 dpif_netdev_operate,
8461 NULL, /* recv_set */
8462 NULL, /* handlers_set */
8463 dpif_netdev_set_config,
8464 dpif_netdev_queue_to_priority,
8465 NULL, /* recv */
8466 NULL, /* recv_wait */
8467 NULL, /* recv_purge */
8468 dpif_netdev_register_dp_purge_cb,
8469 dpif_netdev_register_upcall_cb,
8470 dpif_netdev_enable_upcall,
8471 dpif_netdev_disable_upcall,
8472 dpif_netdev_get_datapath_version,
8473 dpif_netdev_ct_dump_start,
8474 dpif_netdev_ct_dump_next,
8475 dpif_netdev_ct_dump_done,
8476 dpif_netdev_ct_flush,
8477 dpif_netdev_ct_set_maxconns,
8478 dpif_netdev_ct_get_maxconns,
8479 dpif_netdev_ct_get_nconns,
8480 dpif_netdev_ct_set_tcp_seq_chk,
8481 dpif_netdev_ct_get_tcp_seq_chk,
8482 dpif_netdev_ct_set_limits,
8483 dpif_netdev_ct_get_limits,
8484 dpif_netdev_ct_del_limits,
8485 dpif_netdev_ct_set_timeout_policy,
8486 dpif_netdev_ct_get_timeout_policy,
8487 dpif_netdev_ct_del_timeout_policy,
8488 NULL, /* ct_timeout_policy_dump_start */
8489 NULL, /* ct_timeout_policy_dump_next */
8490 NULL, /* ct_timeout_policy_dump_done */
8491 dpif_netdev_ct_get_timeout_policy_name,
8492 dpif_netdev_ipf_set_enabled,
8493 dpif_netdev_ipf_set_min_frag,
8494 dpif_netdev_ipf_set_max_nfrags,
8495 dpif_netdev_ipf_get_status,
8496 dpif_netdev_ipf_dump_start,
8497 dpif_netdev_ipf_dump_next,
8498 dpif_netdev_ipf_dump_done,
8499 dpif_netdev_meter_get_features,
8500 dpif_netdev_meter_set,
8501 dpif_netdev_meter_get,
8502 dpif_netdev_meter_del,
8503 dpif_netdev_bond_add,
8504 dpif_netdev_bond_del,
8505 dpif_netdev_bond_stats_get,
8506 };
8507
8508 static void
8509 dpif_dummy_change_port_number(struct unixctl_conn *conn, int argc OVS_UNUSED,
8510 const char *argv[], void *aux OVS_UNUSED)
8511 {
8512 struct dp_netdev_port *port;
8513 struct dp_netdev *dp;
8514 odp_port_t port_no;
8515
8516 ovs_mutex_lock(&dp_netdev_mutex);
8517 dp = shash_find_data(&dp_netdevs, argv[1]);
8518 if (!dp || !dpif_netdev_class_is_dummy(dp->class)) {
8519 ovs_mutex_unlock(&dp_netdev_mutex);
8520 unixctl_command_reply_error(conn, "unknown datapath or not a dummy");
8521 return;
8522 }
8523 ovs_refcount_ref(&dp->ref_cnt);
8524 ovs_mutex_unlock(&dp_netdev_mutex);
8525
8526 ovs_mutex_lock(&dp->port_mutex);
8527 if (get_port_by_name(dp, argv[2], &port)) {
8528 unixctl_command_reply_error(conn, "unknown port");
8529 goto exit;
8530 }
8531
8532 port_no = u32_to_odp(atoi(argv[3]));
8533 if (!port_no || port_no == ODPP_NONE) {
8534 unixctl_command_reply_error(conn, "bad port number");
8535 goto exit;
8536 }
8537 if (dp_netdev_lookup_port(dp, port_no)) {
8538 unixctl_command_reply_error(conn, "port number already in use");
8539 goto exit;
8540 }
8541
8542 /* Remove port. */
8543 hmap_remove(&dp->ports, &port->node);
8544 reconfigure_datapath(dp);
8545
8546 /* Reinsert with new port number. */
8547 port->port_no = port_no;
8548 hmap_insert(&dp->ports, &port->node, hash_port_no(port_no));
8549 reconfigure_datapath(dp);
8550
8551 seq_change(dp->port_seq);
8552 unixctl_command_reply(conn, NULL);
8553
8554 exit:
8555 ovs_mutex_unlock(&dp->port_mutex);
8556 dp_netdev_unref(dp);
8557 }
8558
8559 static void
8560 dpif_dummy_register__(const char *type)
8561 {
8562 struct dpif_class *class;
8563
8564 class = xmalloc(sizeof *class);
8565 *class = dpif_netdev_class;
8566 class->type = xstrdup(type);
8567 dp_register_provider(class);
8568 }
8569
8570 static void
8571 dpif_dummy_override(const char *type)
8572 {
8573 int error;
8574
8575 /*
8576 * Ignore EAFNOSUPPORT to allow --enable-dummy=system with
8577 * a userland-only build. It's useful for testsuite.
8578 */
8579 error = dp_unregister_provider(type);
8580 if (error == 0 || error == EAFNOSUPPORT) {
8581 dpif_dummy_register__(type);
8582 }
8583 }
8584
8585 void
8586 dpif_dummy_register(enum dummy_level level)
8587 {
8588 if (level == DUMMY_OVERRIDE_ALL) {
8589 struct sset types;
8590 const char *type;
8591
8592 sset_init(&types);
8593 dp_enumerate_types(&types);
8594 SSET_FOR_EACH (type, &types) {
8595 dpif_dummy_override(type);
8596 }
8597 sset_destroy(&types);
8598 } else if (level == DUMMY_OVERRIDE_SYSTEM) {
8599 dpif_dummy_override("system");
8600 }
8601
8602 dpif_dummy_register__("dummy");
8603
8604 unixctl_command_register("dpif-dummy/change-port-number",
8605 "dp port new-number",
8606 3, 3, dpif_dummy_change_port_number, NULL);
8607 }
8608 \f
8609 /* Datapath Classifier. */
8610
8611 static void
8612 dpcls_subtable_destroy_cb(struct dpcls_subtable *subtable)
8613 {
8614 cmap_destroy(&subtable->rules);
8615 ovsrcu_postpone(free, subtable->mf_masks);
8616 ovsrcu_postpone(free, subtable);
8617 }
8618
8619 /* Initializes 'cls' as a classifier that initially contains no classification
8620 * rules. */
8621 static void
8622 dpcls_init(struct dpcls *cls)
8623 {
8624 cmap_init(&cls->subtables_map);
8625 pvector_init(&cls->subtables);
8626 }
8627
8628 static void
8629 dpcls_destroy_subtable(struct dpcls *cls, struct dpcls_subtable *subtable)
8630 {
8631 VLOG_DBG("Destroying subtable %p for in_port %d", subtable, cls->in_port);
8632 pvector_remove(&cls->subtables, subtable);
8633 cmap_remove(&cls->subtables_map, &subtable->cmap_node,
8634 subtable->mask.hash);
8635 ovsrcu_postpone(dpcls_subtable_destroy_cb, subtable);
8636 }
8637
8638 /* Destroys 'cls'. Rules within 'cls', if any, are not freed; this is the
8639 * caller's responsibility.
8640 * May only be called after all the readers have been terminated. */
8641 static void
8642 dpcls_destroy(struct dpcls *cls)
8643 {
8644 if (cls) {
8645 struct dpcls_subtable *subtable;
8646
8647 CMAP_FOR_EACH (subtable, cmap_node, &cls->subtables_map) {
8648 ovs_assert(cmap_count(&subtable->rules) == 0);
8649 dpcls_destroy_subtable(cls, subtable);
8650 }
8651 cmap_destroy(&cls->subtables_map);
8652 pvector_destroy(&cls->subtables);
8653 }
8654 }
8655
8656 static struct dpcls_subtable *
8657 dpcls_create_subtable(struct dpcls *cls, const struct netdev_flow_key *mask)
8658 {
8659 struct dpcls_subtable *subtable;
8660
8661 /* Need to add one. */
8662 subtable = xmalloc(sizeof *subtable
8663 - sizeof subtable->mask.mf + mask->len);
8664 cmap_init(&subtable->rules);
8665 subtable->hit_cnt = 0;
8666 netdev_flow_key_clone(&subtable->mask, mask);
8667
8668 /* The count of bits in the mask defines the space required for masks.
8669 * Then call gen_masks() to create the appropriate masks, avoiding the cost
8670 * of doing runtime calculations. */
8671 uint32_t unit0 = count_1bits(mask->mf.map.bits[0]);
8672 uint32_t unit1 = count_1bits(mask->mf.map.bits[1]);
8673 subtable->mf_bits_set_unit0 = unit0;
8674 subtable->mf_bits_set_unit1 = unit1;
8675 subtable->mf_masks = xmalloc(sizeof(uint64_t) * (unit0 + unit1));
8676 netdev_flow_key_gen_masks(mask, subtable->mf_masks, unit0, unit1);
8677
8678 /* Get the preferred subtable search function for this (u0,u1) subtable.
8679 * The function is guaranteed to always return a valid implementation, and
8680 * possibly an ISA optimized, and/or specialized implementation.
8681 */
8682 subtable->lookup_func = dpcls_subtable_get_best_impl(unit0, unit1);
8683
8684 cmap_insert(&cls->subtables_map, &subtable->cmap_node, mask->hash);
8685 /* Add the new subtable at the end of the pvector (with no hits yet) */
8686 pvector_insert(&cls->subtables, subtable, 0);
8687 VLOG_DBG("Creating %"PRIuSIZE". subtable %p for in_port %d",
8688 cmap_count(&cls->subtables_map), subtable, cls->in_port);
8689 pvector_publish(&cls->subtables);
8690
8691 return subtable;
8692 }
8693
8694 static inline struct dpcls_subtable *
8695 dpcls_find_subtable(struct dpcls *cls, const struct netdev_flow_key *mask)
8696 {
8697 struct dpcls_subtable *subtable;
8698
8699 CMAP_FOR_EACH_WITH_HASH (subtable, cmap_node, mask->hash,
8700 &cls->subtables_map) {
8701 if (netdev_flow_key_equal(&subtable->mask, mask)) {
8702 return subtable;
8703 }
8704 }
8705 return dpcls_create_subtable(cls, mask);
8706 }
8707
8708 /* Checks for the best available implementation for each subtable lookup
8709 * function, and assigns it as the lookup function pointer for each subtable.
8710 * Returns the number of subtables that have changed lookup implementation.
8711 */
8712 static uint32_t
8713 dpcls_subtable_lookup_reprobe(struct dpcls *cls)
8714 {
8715 struct pvector *pvec = &cls->subtables;
8716 uint32_t subtables_changed = 0;
8717 struct dpcls_subtable *subtable = NULL;
8718
8719 PVECTOR_FOR_EACH (subtable, pvec) {
8720 uint32_t u0_bits = subtable->mf_bits_set_unit0;
8721 uint32_t u1_bits = subtable->mf_bits_set_unit1;
8722 void *old_func = subtable->lookup_func;
8723 subtable->lookup_func = dpcls_subtable_get_best_impl(u0_bits, u1_bits);
8724 subtables_changed += (old_func != subtable->lookup_func);
8725 }
8726 pvector_publish(pvec);
8727
8728 return subtables_changed;
8729 }
8730
8731 /* Periodically sort the dpcls subtable vectors according to hit counts */
8732 static void
8733 dpcls_sort_subtable_vector(struct dpcls *cls)
8734 {
8735 struct pvector *pvec = &cls->subtables;
8736 struct dpcls_subtable *subtable;
8737
8738 PVECTOR_FOR_EACH (subtable, pvec) {
8739 pvector_change_priority(pvec, subtable, subtable->hit_cnt);
8740 subtable->hit_cnt = 0;
8741 }
8742 pvector_publish(pvec);
8743 }
8744
8745 static inline void
8746 dp_netdev_pmd_try_optimize(struct dp_netdev_pmd_thread *pmd,
8747 struct polled_queue *poll_list, int poll_cnt)
8748 {
8749 struct dpcls *cls;
8750 uint64_t tot_idle = 0, tot_proc = 0;
8751 unsigned int pmd_load = 0;
8752
8753 if (pmd->ctx.now > pmd->rxq_next_cycle_store) {
8754 uint64_t curr_tsc;
8755 uint8_t rebalance_load_trigger;
8756 struct pmd_auto_lb *pmd_alb = &pmd->dp->pmd_alb;
8757 if (pmd_alb->is_enabled && !pmd->isolated
8758 && (pmd->perf_stats.counters.n[PMD_CYCLES_ITER_IDLE] >=
8759 pmd->prev_stats[PMD_CYCLES_ITER_IDLE])
8760 && (pmd->perf_stats.counters.n[PMD_CYCLES_ITER_BUSY] >=
8761 pmd->prev_stats[PMD_CYCLES_ITER_BUSY]))
8762 {
8763 tot_idle = pmd->perf_stats.counters.n[PMD_CYCLES_ITER_IDLE] -
8764 pmd->prev_stats[PMD_CYCLES_ITER_IDLE];
8765 tot_proc = pmd->perf_stats.counters.n[PMD_CYCLES_ITER_BUSY] -
8766 pmd->prev_stats[PMD_CYCLES_ITER_BUSY];
8767
8768 if (tot_proc) {
8769 pmd_load = ((tot_proc * 100) / (tot_idle + tot_proc));
8770 }
8771
8772 atomic_read_relaxed(&pmd_alb->rebalance_load_thresh,
8773 &rebalance_load_trigger);
8774 if (pmd_load >= rebalance_load_trigger) {
8775 atomic_count_inc(&pmd->pmd_overloaded);
8776 } else {
8777 atomic_count_set(&pmd->pmd_overloaded, 0);
8778 }
8779 }
8780
8781 pmd->prev_stats[PMD_CYCLES_ITER_IDLE] =
8782 pmd->perf_stats.counters.n[PMD_CYCLES_ITER_IDLE];
8783 pmd->prev_stats[PMD_CYCLES_ITER_BUSY] =
8784 pmd->perf_stats.counters.n[PMD_CYCLES_ITER_BUSY];
8785
8786 /* Get the cycles that were used to process each queue and store. */
8787 for (unsigned i = 0; i < poll_cnt; i++) {
8788 uint64_t rxq_cyc_curr = dp_netdev_rxq_get_cycles(poll_list[i].rxq,
8789 RXQ_CYCLES_PROC_CURR);
8790 dp_netdev_rxq_set_intrvl_cycles(poll_list[i].rxq, rxq_cyc_curr);
8791 dp_netdev_rxq_set_cycles(poll_list[i].rxq, RXQ_CYCLES_PROC_CURR,
8792 0);
8793 }
8794 curr_tsc = cycles_counter_update(&pmd->perf_stats);
8795 if (pmd->intrvl_tsc_prev) {
8796 /* There is a prev timestamp, store a new intrvl cycle count. */
8797 atomic_store_relaxed(&pmd->intrvl_cycles,
8798 curr_tsc - pmd->intrvl_tsc_prev);
8799 }
8800 pmd->intrvl_tsc_prev = curr_tsc;
8801 /* Start new measuring interval */
8802 pmd->rxq_next_cycle_store = pmd->ctx.now + PMD_RXQ_INTERVAL_LEN;
8803 }
8804
8805 if (pmd->ctx.now > pmd->next_optimization) {
8806 /* Try to obtain the flow lock to block out revalidator threads.
8807 * If not possible, just try next time. */
8808 if (!ovs_mutex_trylock(&pmd->flow_mutex)) {
8809 /* Optimize each classifier */
8810 CMAP_FOR_EACH (cls, node, &pmd->classifiers) {
8811 dpcls_sort_subtable_vector(cls);
8812 }
8813 ovs_mutex_unlock(&pmd->flow_mutex);
8814 /* Start new measuring interval */
8815 pmd->next_optimization = pmd->ctx.now
8816 + DPCLS_OPTIMIZATION_INTERVAL;
8817 }
8818 }
8819 }
8820
8821 /* Insert 'rule' into 'cls'. */
8822 static void
8823 dpcls_insert(struct dpcls *cls, struct dpcls_rule *rule,
8824 const struct netdev_flow_key *mask)
8825 {
8826 struct dpcls_subtable *subtable = dpcls_find_subtable(cls, mask);
8827
8828 /* Refer to subtable's mask, also for later removal. */
8829 rule->mask = &subtable->mask;
8830 cmap_insert(&subtable->rules, &rule->cmap_node, rule->flow.hash);
8831 }
8832
8833 /* Removes 'rule' from 'cls', also destructing the 'rule'. */
8834 static void
8835 dpcls_remove(struct dpcls *cls, struct dpcls_rule *rule)
8836 {
8837 struct dpcls_subtable *subtable;
8838
8839 ovs_assert(rule->mask);
8840
8841 /* Get subtable from reference in rule->mask. */
8842 INIT_CONTAINER(subtable, rule->mask, mask);
8843 if (cmap_remove(&subtable->rules, &rule->cmap_node, rule->flow.hash)
8844 == 0) {
8845 /* Delete empty subtable. */
8846 dpcls_destroy_subtable(cls, subtable);
8847 pvector_publish(&cls->subtables);
8848 }
8849 }
8850
8851 /* Inner loop for mask generation of a unit, see netdev_flow_key_gen_masks. */
8852 static inline void
8853 netdev_flow_key_gen_mask_unit(uint64_t iter,
8854 const uint64_t count,
8855 uint64_t *mf_masks)
8856 {
8857 int i;
8858 for (i = 0; i < count; i++) {
8859 uint64_t lowest_bit = (iter & -iter);
8860 iter &= ~lowest_bit;
8861 mf_masks[i] = (lowest_bit - 1);
8862 }
8863 /* Checks that count has covered all bits in the iter bitmap. */
8864 ovs_assert(iter == 0);
8865 }
8866
8867 /* Generate a mask for each block in the miniflow, based on the bits set. This
8868 * allows easily masking packets with the generated array here, without
8869 * calculations. This replaces runtime-calculating the masks.
8870 * @param key The table to generate the mf_masks for
8871 * @param mf_masks Pointer to a u64 array of at least *mf_bits* in size
8872 * @param mf_bits_total Number of bits set in the whole miniflow (both units)
8873 * @param mf_bits_unit0 Number of bits set in unit0 of the miniflow
8874 */
8875 void
8876 netdev_flow_key_gen_masks(const struct netdev_flow_key *tbl,
8877 uint64_t *mf_masks,
8878 const uint32_t mf_bits_u0,
8879 const uint32_t mf_bits_u1)
8880 {
8881 uint64_t iter_u0 = tbl->mf.map.bits[0];
8882 uint64_t iter_u1 = tbl->mf.map.bits[1];
8883
8884 netdev_flow_key_gen_mask_unit(iter_u0, mf_bits_u0, &mf_masks[0]);
8885 netdev_flow_key_gen_mask_unit(iter_u1, mf_bits_u1, &mf_masks[mf_bits_u0]);
8886 }
8887
8888 /* Returns true if 'target' satisfies 'key' in 'mask', that is, if each 1-bit
8889 * in 'mask' the values in 'key' and 'target' are the same. */
8890 bool
8891 dpcls_rule_matches_key(const struct dpcls_rule *rule,
8892 const struct netdev_flow_key *target)
8893 {
8894 const uint64_t *keyp = miniflow_get_values(&rule->flow.mf);
8895 const uint64_t *maskp = miniflow_get_values(&rule->mask->mf);
8896 uint64_t value;
8897
8898 NETDEV_FLOW_KEY_FOR_EACH_IN_FLOWMAP(value, target, rule->flow.mf.map) {
8899 if (OVS_UNLIKELY((value & *maskp++) != *keyp++)) {
8900 return false;
8901 }
8902 }
8903 return true;
8904 }
8905
8906 /* For each miniflow in 'keys' performs a classifier lookup writing the result
8907 * into the corresponding slot in 'rules'. If a particular entry in 'keys' is
8908 * NULL it is skipped.
8909 *
8910 * This function is optimized for use in the userspace datapath and therefore
8911 * does not implement a lot of features available in the standard
8912 * classifier_lookup() function. Specifically, it does not implement
8913 * priorities, instead returning any rule which matches the flow.
8914 *
8915 * Returns true if all miniflows found a corresponding rule. */
8916 static bool
8917 dpcls_lookup(struct dpcls *cls, const struct netdev_flow_key *keys[],
8918 struct dpcls_rule **rules, const size_t cnt,
8919 int *num_lookups_p)
8920 {
8921 /* The received 'cnt' miniflows are the search-keys that will be processed
8922 * to find a matching entry into the available subtables.
8923 * The number of bits in map_type is equal to NETDEV_MAX_BURST. */
8924 #define MAP_BITS (sizeof(uint32_t) * CHAR_BIT)
8925 BUILD_ASSERT_DECL(MAP_BITS >= NETDEV_MAX_BURST);
8926
8927 struct dpcls_subtable *subtable;
8928 uint32_t keys_map = TYPE_MAXIMUM(uint32_t); /* Set all bits. */
8929
8930 if (cnt != MAP_BITS) {
8931 keys_map >>= MAP_BITS - cnt; /* Clear extra bits. */
8932 }
8933 memset(rules, 0, cnt * sizeof *rules);
8934
8935 int lookups_match = 0, subtable_pos = 1;
8936 uint32_t found_map;
8937
8938 /* The Datapath classifier - aka dpcls - is composed of subtables.
8939 * Subtables are dynamically created as needed when new rules are inserted.
8940 * Each subtable collects rules with matches on a specific subset of packet
8941 * fields as defined by the subtable's mask. We proceed to process every
8942 * search-key against each subtable, but when a match is found for a
8943 * search-key, the search for that key can stop because the rules are
8944 * non-overlapping. */
8945 PVECTOR_FOR_EACH (subtable, &cls->subtables) {
8946 /* Call the subtable specific lookup function. */
8947 found_map = subtable->lookup_func(subtable, keys_map, keys, rules);
8948
8949 /* Count the number of subtables searched for this packet match. This
8950 * estimates the "spread" of subtables looked at per matched packet. */
8951 uint32_t pkts_matched = count_1bits(found_map);
8952 lookups_match += pkts_matched * subtable_pos;
8953
8954 /* Clear the found rules, and return early if all packets are found. */
8955 keys_map &= ~found_map;
8956 if (!keys_map) {
8957 if (num_lookups_p) {
8958 *num_lookups_p = lookups_match;
8959 }
8960 return true;
8961 }
8962 subtable_pos++;
8963 }
8964
8965 if (num_lookups_p) {
8966 *num_lookups_p = lookups_match;
8967 }
8968 return false;
8969 }