dp-packet: Rename 'dp_hash' in 'rss_hash'.
[cascardo/ovs.git] / lib / dpif-netdev.c
index a161e98..78ed486 100644 (file)
@@ -220,12 +220,19 @@ static struct dp_netdev_port *dp_netdev_lookup_port(const struct dp_netdev *dp,
                                                     odp_port_t);
 
 enum dp_stat_type {
-    DP_STAT_HIT,                /* Packets that matched in the flow table. */
+    DP_STAT_EXACT_HIT,          /* Packets that had an exact match (emc). */
+    DP_STAT_MASKED_HIT,         /* Packets that matched in the flow table. */
     DP_STAT_MISS,               /* Packets that did not match. */
     DP_STAT_LOST,               /* Packets not passed up to the client. */
     DP_N_STATS
 };
 
+enum pmd_cycles_counter_type {
+    PMD_CYCLES_POLLING,         /* Cycles spent polling NICs. */
+    PMD_CYCLES_PROCESSING,      /* Cycles spent processing packets */
+    PMD_N_CYCLES
+};
+
 /* A port in a netdev-based datapath. */
 struct dp_netdev_port {
     struct cmap_node node;      /* Node in dp_netdev's 'ports'. */
@@ -325,8 +332,8 @@ static int dpif_netdev_flow_from_nlattrs(const struct nlattr *, uint32_t,
 struct dp_netdev_actions {
     /* These members are immutable: they do not change during the struct's
      * lifetime.  */
-    struct nlattr *actions;     /* Sequence of OVS_ACTION_ATTR_* attributes. */
     unsigned int size;          /* Size of 'actions', in bytes. */
+    struct nlattr actions[];    /* Sequence of OVS_ACTION_ATTR_* attributes. */
 };
 
 struct dp_netdev_actions *dp_netdev_actions_create(const struct nlattr *,
@@ -341,6 +348,12 @@ struct dp_netdev_pmd_stats {
     atomic_ullong n[DP_N_STATS];
 };
 
+/* Contained by struct dp_netdev_pmd_thread's 'cycle' member.  */
+struct dp_netdev_pmd_cycles {
+    /* Indexed by PMD_CYCLES_*. */
+    atomic_ullong n[PMD_N_CYCLES];
+};
+
 /* PMD: Poll modes drivers.  PMD accesses devices via polling to eliminate
  * the performance overhead of interrupt processing.  Therefore netdev can
  * not implement rx-wait for these devices.  dpif-netdev needs to poll
@@ -383,6 +396,12 @@ struct dp_netdev_pmd_thread {
     /* Statistics. */
     struct dp_netdev_pmd_stats stats;
 
+    /* Cycles counters */
+    struct dp_netdev_pmd_cycles cycles;
+
+    /* Used to count cicles. See 'cycles_counter_end()' */
+    unsigned long long last_cycles;
+
     struct latch exit_latch;        /* For terminating the pmd thread. */
     atomic_uint change_seq;         /* For reloading pmd ports. */
     pthread_t thread;
@@ -390,6 +409,13 @@ struct dp_netdev_pmd_thread {
                                     /* threads on same numa node. */
     int core_id;                    /* CPU core id of this pmd thread. */
     int numa_id;                    /* numa node id of this pmd thread. */
+
+    /* Only a pmd thread can write on its own 'cycles' and 'stats'.
+     * The main thread keeps 'stats_zero' and 'cycles_zero' as base
+     * values and subtracts them from 'stats' and 'cycles' before
+     * reporting to the user */
+    unsigned long long stats_zero[DP_N_STATS];
+    uint64_t cycles_zero[PMD_N_CYCLES];
 };
 
 #define PMD_INITIAL_SEQ 1
@@ -497,6 +523,182 @@ get_dp_netdev(const struct dpif *dpif)
 {
     return dpif_netdev_cast(dpif)->dp;
 }
+\f
+enum pmd_info_type {
+    PMD_INFO_SHOW_STATS,  /* show how cpu cycles are spent */
+    PMD_INFO_CLEAR_STATS  /* set the cycles count to 0 */
+};
+
+static void
+pmd_info_show_stats(struct ds *reply,
+                    struct dp_netdev_pmd_thread *pmd,
+                    unsigned long long stats[DP_N_STATS],
+                    uint64_t cycles[PMD_N_CYCLES])
+{
+    unsigned long long total_packets = 0;
+    uint64_t total_cycles = 0;
+    int i;
+
+    /* These loops subtracts reference values ('*_zero') from the counters.
+     * Since loads and stores are relaxed, it might be possible for a '*_zero'
+     * value to be more recent than the current value we're reading from the
+     * counter.  This is not a big problem, since these numbers are not
+     * supposed to be too accurate, but we should at least make sure that
+     * the result is not negative. */
+    for (i = 0; i < DP_N_STATS; i++) {
+        if (stats[i] > pmd->stats_zero[i]) {
+            stats[i] -= pmd->stats_zero[i];
+        } else {
+            stats[i] = 0;
+        }
+
+        if (i != DP_STAT_LOST) {
+            /* Lost packets are already included in DP_STAT_MISS */
+            total_packets += stats[i];
+        }
+    }
+
+    for (i = 0; i < PMD_N_CYCLES; i++) {
+        if (cycles[i] > pmd->cycles_zero[i]) {
+           cycles[i] -= pmd->cycles_zero[i];
+        } else {
+            cycles[i] = 0;
+        }
+
+        total_cycles += cycles[i];
+    }
+
+    ds_put_cstr(reply, (pmd->core_id == NON_PMD_CORE_ID)
+                        ? "main thread" : "pmd thread");
+
+    if (pmd->numa_id != OVS_NUMA_UNSPEC) {
+        ds_put_format(reply, " numa_id %d", pmd->numa_id);
+    }
+    if (pmd->core_id != OVS_CORE_UNSPEC) {
+        ds_put_format(reply, " core_id %d", pmd->core_id);
+    }
+    ds_put_cstr(reply, ":\n");
+
+    ds_put_format(reply,
+                  "\temc hits:%llu\n\tmegaflow hits:%llu\n"
+                  "\tmiss:%llu\n\tlost:%llu\n",
+                  stats[DP_STAT_EXACT_HIT], stats[DP_STAT_MASKED_HIT],
+                  stats[DP_STAT_MISS], stats[DP_STAT_LOST]);
+
+    if (total_cycles == 0) {
+        return;
+    }
+
+    ds_put_format(reply,
+                  "\tpolling cycles:%"PRIu64" (%.02f%%)\n"
+                  "\tprocessing cycles:%"PRIu64" (%.02f%%)\n",
+                  cycles[PMD_CYCLES_POLLING],
+                  cycles[PMD_CYCLES_POLLING] / (double)total_cycles * 100,
+                  cycles[PMD_CYCLES_PROCESSING],
+                  cycles[PMD_CYCLES_PROCESSING] / (double)total_cycles * 100);
+
+    if (total_packets == 0) {
+        return;
+    }
+
+    ds_put_format(reply,
+                  "\tavg cycles per packet: %.02f (%"PRIu64"/%llu)\n",
+                  total_cycles / (double)total_packets,
+                  total_cycles, total_packets);
+
+    ds_put_format(reply,
+                  "\tavg processing cycles per packet: "
+                  "%.02f (%"PRIu64"/%llu)\n",
+                  cycles[PMD_CYCLES_PROCESSING] / (double)total_packets,
+                  cycles[PMD_CYCLES_PROCESSING], total_packets);
+}
+
+static void
+pmd_info_clear_stats(struct ds *reply OVS_UNUSED,
+                    struct dp_netdev_pmd_thread *pmd,
+                    unsigned long long stats[DP_N_STATS],
+                    uint64_t cycles[PMD_N_CYCLES])
+{
+    int i;
+
+    /* We cannot write 'stats' and 'cycles' (because they're written by other
+     * threads) and we shouldn't change 'stats' (because they're used to count
+     * datapath stats, which must not be cleared here).  Instead, we save the
+     * current values and subtract them from the values to be displayed in the
+     * future */
+    for (i = 0; i < DP_N_STATS; i++) {
+        pmd->stats_zero[i] = stats[i];
+    }
+    for (i = 0; i < PMD_N_CYCLES; i++) {
+        pmd->cycles_zero[i] = cycles[i];
+    }
+}
+
+static void
+dpif_netdev_pmd_info(struct unixctl_conn *conn, int argc, const char *argv[],
+                     void *aux)
+{
+    struct ds reply = DS_EMPTY_INITIALIZER;
+    struct dp_netdev_pmd_thread *pmd;
+    struct dp_netdev *dp = NULL;
+    enum pmd_info_type type = *(enum pmd_info_type *) aux;
+
+    ovs_mutex_lock(&dp_netdev_mutex);
+
+    if (argc == 2) {
+        dp = shash_find_data(&dp_netdevs, argv[1]);
+    } else if (shash_count(&dp_netdevs) == 1) {
+        /* There's only one datapath */
+        dp = shash_first(&dp_netdevs)->data;
+    }
+
+    if (!dp) {
+        ovs_mutex_unlock(&dp_netdev_mutex);
+        unixctl_command_reply_error(conn,
+                                    "please specify an existing datapath");
+        return;
+    }
+
+    CMAP_FOR_EACH (pmd, node, &dp->poll_threads) {
+        unsigned long long stats[DP_N_STATS];
+        uint64_t cycles[PMD_N_CYCLES];
+        int i;
+
+        /* Read current stats and cycle counters */
+        for (i = 0; i < ARRAY_SIZE(stats); i++) {
+            atomic_read_relaxed(&pmd->stats.n[i], &stats[i]);
+        }
+        for (i = 0; i < ARRAY_SIZE(cycles); i++) {
+            atomic_read_relaxed(&pmd->cycles.n[i], &cycles[i]);
+        }
+
+        if (type == PMD_INFO_CLEAR_STATS) {
+            pmd_info_clear_stats(&reply, pmd, stats, cycles);
+        } else if (type == PMD_INFO_SHOW_STATS) {
+            pmd_info_show_stats(&reply, pmd, stats, cycles);
+        }
+    }
+
+    ovs_mutex_unlock(&dp_netdev_mutex);
+
+    unixctl_command_reply(conn, ds_cstr(&reply));
+    ds_destroy(&reply);
+}
+\f
+static int
+dpif_netdev_init(void)
+{
+    static enum pmd_info_type show_aux = PMD_INFO_SHOW_STATS,
+                              clear_aux = PMD_INFO_CLEAR_STATS;
+
+    unixctl_command_register("dpif-netdev/pmd-stats-show", "[dp]",
+                             0, 1, dpif_netdev_pmd_info,
+                             (void *)&show_aux);
+    unixctl_command_register("dpif-netdev/pmd-stats-clear", "[dp]",
+                             0, 1, dpif_netdev_pmd_info,
+                             (void *)&clear_aux);
+    return 0;
+}
 
 static int
 dpif_netdev_enumerate(struct sset *all_dps,
@@ -772,7 +974,9 @@ dpif_netdev_get_stats(const struct dpif *dpif, struct dpif_dp_stats *stats)
         unsigned long long n;
         stats->n_flows += cmap_count(&pmd->flow_table);
 
-        atomic_read_relaxed(&pmd->stats.n[DP_STAT_HIT], &n);
+        atomic_read_relaxed(&pmd->stats.n[DP_STAT_MASKED_HIT], &n);
+        stats->n_hit += n;
+        atomic_read_relaxed(&pmd->stats.n[DP_STAT_EXACT_HIT], &n);
         stats->n_hit += n;
         atomic_read_relaxed(&pmd->stats.n[DP_STAT_MISS], &n);
         stats->n_missed += n;
@@ -831,7 +1035,10 @@ do_add_port(struct dp_netdev *dp, const char *devname, const char *type,
     int error;
     int i;
 
-    /* XXX reject devices already in some dp_netdev. */
+    /* Reject devices already in 'dp'. */
+    if (!get_port_by_name(dp, devname, &port)) {
+        return EEXIST;
+    }
 
     /* Open and validate network device. */
     open_type = dpif_netdev_port_open_type(dp->class, type);
@@ -2228,16 +2435,15 @@ dpif_netdev_queue_to_priority(const struct dpif *dpif OVS_UNUSED,
 }
 
 \f
-/* Creates and returns a new 'struct dp_netdev_actions', with a reference count
- * of 1, whose actions are a copy of from the 'ofpacts_len' bytes of
- * 'ofpacts'. */
+/* Creates and returns a new 'struct dp_netdev_actions', whose actions are
+ * a copy of the 'ofpacts_len' bytes of 'ofpacts'. */
 struct dp_netdev_actions *
 dp_netdev_actions_create(const struct nlattr *actions, size_t size)
 {
     struct dp_netdev_actions *netdev_actions;
 
-    netdev_actions = xmalloc(sizeof *netdev_actions);
-    netdev_actions->actions = xmemdup(actions, size);
+    netdev_actions = xmalloc(sizeof *netdev_actions + size);
+    memcpy(netdev_actions->actions, actions, size);
     netdev_actions->size = size;
 
     return netdev_actions;
@@ -2252,10 +2458,42 @@ dp_netdev_flow_get_actions(const struct dp_netdev_flow *flow)
 static void
 dp_netdev_actions_free(struct dp_netdev_actions *actions)
 {
-    free(actions->actions);
     free(actions);
 }
 \f
+static inline unsigned long long
+cycles_counter(void)
+{
+#ifdef DPDK_NETDEV
+    return rte_get_tsc_cycles();
+#else
+    return 0;
+#endif
+}
+
+/* Fake mutex to make sure that the calls to cycles_count_* are balanced */
+extern struct ovs_mutex cycles_counter_fake_mutex;
+
+/* Start counting cycles.  Must be followed by 'cycles_count_end()' */
+static inline void
+cycles_count_start(struct dp_netdev_pmd_thread *pmd)
+    OVS_ACQUIRES(&cycles_counter_fake_mutex)
+    OVS_NO_THREAD_SAFETY_ANALYSIS
+{
+    pmd->last_cycles = cycles_counter();
+}
+
+/* Stop counting cycles and add them to the counter 'type' */
+static inline void
+cycles_count_end(struct dp_netdev_pmd_thread *pmd,
+                 enum pmd_cycles_counter_type type)
+    OVS_RELEASES(&cycles_counter_fake_mutex)
+    OVS_NO_THREAD_SAFETY_ANALYSIS
+{
+    unsigned long long interval = cycles_counter() - pmd->last_cycles;
+
+    non_atomic_ullong_add(&pmd->cycles.n[type], interval);
+}
 
 static void
 dp_netdev_process_rxq_port(struct dp_netdev_pmd_thread *pmd,
@@ -2265,7 +2503,9 @@ dp_netdev_process_rxq_port(struct dp_netdev_pmd_thread *pmd,
     struct dp_packet *packets[NETDEV_MAX_RX_BATCH];
     int error, cnt;
 
+    cycles_count_start(pmd);
     error = netdev_rxq_recv(rxq, packets, &cnt);
+    cycles_count_end(pmd, PMD_CYCLES_POLLING);
     if (!error) {
         int i;
 
@@ -2275,7 +2515,9 @@ dp_netdev_process_rxq_port(struct dp_netdev_pmd_thread *pmd,
         for (i = 0; i < cnt; i++) {
             packets[i]->md = PKT_METADATA_INITIALIZER(port->port_no);
         }
+        cycles_count_start(pmd);
         dp_netdev_input(pmd, packets, cnt);
+        cycles_count_end(pmd, PMD_CYCLES_PROCESSING);
     } else if (error != EAGAIN && error != EOPNOTSUPP) {
         static struct vlog_rate_limit rl = VLOG_RATE_LIMIT_INIT(1, 5);
 
@@ -2707,9 +2949,8 @@ dpif_netdev_get_datapath_version(void)
 
 static void
 dp_netdev_flow_used(struct dp_netdev_flow *netdev_flow, int cnt, int size,
-                    uint16_t tcp_flags)
+                    uint16_t tcp_flags, long long now)
 {
-    long long now = time_msec();
     uint16_t flags;
 
     atomic_store_relaxed(&netdev_flow->stats.used, now);
@@ -2771,10 +3012,10 @@ dpif_netdev_packet_get_dp_hash(struct dp_packet *packet,
 {
     uint32_t hash;
 
-    hash = dp_packet_get_dp_hash(packet);
+    hash = dp_packet_get_rss_hash(packet);
     if (OVS_UNLIKELY(!hash)) {
         hash = miniflow_hash_5tuple(mf, 0);
-        dp_packet_set_dp_hash(packet, hash);
+        dp_packet_set_rss_hash(packet, hash);
     }
     return hash;
 }
@@ -2810,20 +3051,22 @@ packet_batch_init(struct packet_batch *batch, struct dp_netdev_flow *flow)
 
 static inline void
 packet_batch_execute(struct packet_batch *batch,
-                     struct dp_netdev_pmd_thread *pmd)
+                     struct dp_netdev_pmd_thread *pmd,
+                     enum dp_stat_type hit_type,
+                     long long now)
 {
     struct dp_netdev_actions *actions;
     struct dp_netdev_flow *flow = batch->flow;
 
     dp_netdev_flow_used(batch->flow, batch->packet_count, batch->byte_count,
-                        batch->tcp_flags);
+                        batch->tcp_flags, now);
 
     actions = dp_netdev_flow_get_actions(flow);
 
     dp_netdev_execute_actions(pmd, batch->packets, batch->packet_count, true,
                               actions->actions, actions->size);
 
-    dp_netdev_count_packet(pmd, DP_STAT_HIT, batch->packet_count);
+    dp_netdev_count_packet(pmd, hit_type, batch->packet_count);
 }
 
 static inline bool
@@ -2879,7 +3122,7 @@ dp_packet_swap(struct dp_packet **a, struct dp_packet **b)
  */
 static inline size_t
 emc_processing(struct dp_netdev_pmd_thread *pmd, struct dp_packet **packets,
-               size_t cnt, struct netdev_flow_key *keys)
+               size_t cnt, struct netdev_flow_key *keys, long long now)
 {
     struct netdev_flow_key key;
     struct packet_batch batches[4];
@@ -2914,7 +3157,7 @@ emc_processing(struct dp_netdev_pmd_thread *pmd, struct dp_packet **packets,
     }
 
     for (i = 0; i < n_batches; i++) {
-        packet_batch_execute(&batches[i], pmd);
+        packet_batch_execute(&batches[i], pmd, DP_STAT_EXACT_HIT, now);
     }
 
     return notfound_cnt;
@@ -2923,7 +3166,7 @@ emc_processing(struct dp_netdev_pmd_thread *pmd, struct dp_packet **packets,
 static inline void
 fast_path_processing(struct dp_netdev_pmd_thread *pmd,
                      struct dp_packet **packets, size_t cnt,
-                     struct netdev_flow_key *keys)
+                     struct netdev_flow_key *keys, long long now)
 {
 #if !defined(__CHECKER__) && !defined(_WIN32)
     const size_t PKT_ARRAY_SIZE = cnt;
@@ -3051,7 +3294,7 @@ fast_path_processing(struct dp_netdev_pmd_thread *pmd,
     }
 
     for (i = 0; i < n_batches; i++) {
-        packet_batch_execute(&batches[i], pmd);
+        packet_batch_execute(&batches[i], pmd, DP_STAT_MASKED_HIT, now);
     }
 }
 
@@ -3066,11 +3309,12 @@ dp_netdev_input(struct dp_netdev_pmd_thread *pmd,
     enum { PKT_ARRAY_SIZE = NETDEV_MAX_RX_BATCH };
 #endif
     struct netdev_flow_key keys[PKT_ARRAY_SIZE];
+    long long now = time_msec();
     size_t newcnt;
 
-    newcnt = emc_processing(pmd, packets, cnt, keys);
+    newcnt = emc_processing(pmd, packets, cnt, keys, now);
     if (OVS_UNLIKELY(newcnt)) {
-        fast_path_processing(pmd, packets, newcnt, keys);
+        fast_path_processing(pmd, packets, newcnt, keys, now);
     }
 }
 
@@ -3251,9 +3495,6 @@ dp_execute_cb(void *aux_, struct dp_packet **packets, int cnt,
 
                 recirc_pkt->md.recirc_id = nl_attr_get_u32(a);
 
-                /* Hash is private to each packet */
-                recirc_pkt->md.dp_hash = dp_packet_get_dp_hash(packets[i]);
-
                 dp_netdev_input(pmd, &recirc_pkt, 1);
             }
             (*depth)--;
@@ -3294,6 +3535,7 @@ dp_netdev_execute_actions(struct dp_netdev_pmd_thread *pmd,
 
 const struct dpif_class dpif_netdev_class = {
     "netdev",
+    dpif_netdev_init,
     dpif_netdev_enumerate,
     dpif_netdev_port_open_type,
     dpif_netdev_open,