2
0
mirror of https://github.com/openvswitch/ovs synced 2025-09-01 14:55:18 +00:00

conntrack: Add rcu support.

For performance and code simplification reasons, add rcu support for
conntrack. The array of hmaps is replaced by a cmap as part of this
conversion.  Using a single map also simplifies the handling of NAT
and allows the removal of the nat_conn map and friends.  Per connection
entry locks are introduced, which are needed in a few code paths.

Signed-off-by: Darrell Ball <dlu998@gmail.com>
Signed-off-by: Ben Pfaff <blp@ovn.org>
This commit is contained in:
Darrell Ball
2019-05-09 08:15:07 -07:00
committed by Ben Pfaff
parent fe772f5395
commit 967bb5c5cd
7 changed files with 460 additions and 862 deletions

View File

@@ -1,5 +1,5 @@
/*
* Copyright (c) 2015, 2016 Nicira, Inc.
* Copyright (c) 2015-2019 Nicira, Inc.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
@@ -24,14 +24,14 @@
#include "conntrack-private.h"
#include "dp-packet.h"
enum icmp_state {
enum OVS_PACKED_ENUM icmp_state {
ICMPS_FIRST,
ICMPS_REPLY,
};
struct conn_icmp {
struct conn up;
enum icmp_state state;
enum icmp_state state; /* 'conn' lock protected. */
};
static const enum ct_timeout icmp_timeouts[] = {
@@ -46,16 +46,12 @@ conn_icmp_cast(const struct conn *conn)
}
static enum ct_update_res
icmp_conn_update(struct conn *conn_, struct conntrack_bucket *ctb,
icmp_conn_update(struct conntrack *ct, struct conn *conn_,
struct dp_packet *pkt OVS_UNUSED, bool reply, long long now)
{
struct conn_icmp *conn = conn_icmp_cast(conn_);
if (reply && conn->state != ICMPS_REPLY) {
conn->state = ICMPS_REPLY;
}
conn_update_expiration(ctb, &conn->up, icmp_timeouts[conn->state], now);
conn->state = reply ? ICMPS_REPLY : ICMPS_FIRST;
conn_update_expiration(ct, &conn->up, icmp_timeouts[conn->state], now);
return CT_UPDATE_VALID;
}
@@ -79,15 +75,12 @@ icmp6_valid_new(struct dp_packet *pkt)
}
static struct conn *
icmp_new_conn(struct conntrack_bucket *ctb, struct dp_packet *pkt OVS_UNUSED,
long long now)
icmp_new_conn(struct conntrack *ct, struct dp_packet *pkt OVS_UNUSED,
long long now)
{
struct conn_icmp *conn;
conn = xzalloc(sizeof *conn);
struct conn_icmp *conn = xzalloc(sizeof *conn);
conn->state = ICMPS_FIRST;
conn_init_expiration(ctb, &conn->up, icmp_timeouts[conn->state], now);
conn_init_expiration(ct, &conn->up, icmp_timeouts[conn->state], now);
return &conn->up;
}

View File

@@ -1,5 +1,5 @@
/*
* Copyright (c) 2015, 2016 Nicira, Inc.
* Copyright (c) 2015-2019 Nicira, Inc.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
@@ -19,7 +19,7 @@
#include "conntrack-private.h"
#include "dp-packet.h"
enum other_state {
enum OVS_PACKED_ENUM other_state {
OTHERS_FIRST,
OTHERS_MULTIPLE,
OTHERS_BIDIR,
@@ -27,7 +27,7 @@ enum other_state {
struct conn_other {
struct conn up;
enum other_state state;
enum other_state state; /* 'conn' lock protected. */
};
static const enum ct_timeout other_timeouts[] = {
@@ -43,7 +43,7 @@ conn_other_cast(const struct conn *conn)
}
static enum ct_update_res
other_conn_update(struct conn *conn_, struct conntrack_bucket *ctb,
other_conn_update(struct conntrack *ct, struct conn *conn_,
struct dp_packet *pkt OVS_UNUSED, bool reply, long long now)
{
struct conn_other *conn = conn_other_cast(conn_);
@@ -54,7 +54,7 @@ other_conn_update(struct conn *conn_, struct conntrack_bucket *ctb,
conn->state = OTHERS_MULTIPLE;
}
conn_update_expiration(ctb, &conn->up, other_timeouts[conn->state], now);
conn_update_expiration(ct, &conn->up, other_timeouts[conn->state], now);
return CT_UPDATE_VALID;
}
@@ -66,7 +66,7 @@ other_valid_new(struct dp_packet *pkt OVS_UNUSED)
}
static struct conn *
other_new_conn(struct conntrack_bucket *ctb, struct dp_packet *pkt OVS_UNUSED,
other_new_conn(struct conntrack *ct, struct dp_packet *pkt OVS_UNUSED,
long long now)
{
struct conn_other *conn;
@@ -74,7 +74,7 @@ other_new_conn(struct conntrack_bucket *ctb, struct dp_packet *pkt OVS_UNUSED,
conn = xzalloc(sizeof *conn);
conn->state = OTHERS_FIRST;
conn_init_expiration(ctb, &conn->up, other_timeouts[conn->state], now);
conn_init_expiration(ct, &conn->up, other_timeouts[conn->state], now);
return &conn->up;
}

View File

@@ -1,5 +1,5 @@
/*
* Copyright (c) 2015, 2016, 2017 Nicira, Inc.
* Copyright (c) 2015-2019 Nicira, Inc.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
@@ -21,8 +21,10 @@
#include <netinet/in.h>
#include <netinet/ip6.h>
#include "cmap.h"
#include "conntrack.h"
#include "ct-dpif.h"
#include "ipf.h"
#include "openvswitch/hmap.h"
#include "openvswitch/list.h"
#include "openvswitch/types.h"
@@ -57,12 +59,6 @@ struct conn_key {
uint8_t nw_proto;
};
struct nat_conn_key_node {
struct hmap_node node;
struct conn_key key;
struct conn_key value;
};
/* This is used for alg expectations; an expectation is a
* context created in preparation for establishing a data
* connection. The expectation is created by the control
@@ -87,25 +83,34 @@ struct alg_exp_node {
bool nat_rpl_dst;
};
enum OVS_PACKED_ENUM ct_conn_type {
CT_CONN_TYPE_DEFAULT,
CT_CONN_TYPE_UN_NAT,
};
struct conn {
/* Immutable data. */
struct conn_key key;
struct conn_key rev_key;
/* Only used for orig_tuple support. */
struct conn_key master_key;
long long expiration;
struct conn_key master_key; /* Only used for orig_tuple support. */
struct ovs_list exp_node;
struct hmap_node node;
ovs_u128 label;
/* XXX: consider flattening. */
struct cmap_node cm_node;
struct nat_action_info_t *nat_info;
char *alg;
int seq_skew;
struct conn *nat_conn; /* The NAT 'conn' context, if there is one. */
/* Mutable data. */
struct ovs_mutex lock; /* Guards all mutable fields. */
ovs_u128 label;
uint32_t mark;
uint8_t conn_type;
/* TCP sequence skew due to NATTing of FTP control messages. */
uint8_t seq_skew_dir;
/* True if alg data connection. */
uint8_t alg_related;
long long expiration;
int seq_skew;
bool seq_skew_dir; /* TCP sequence skew direction due to NATTing of FTP
* control messages; true if reply direction. */
/* Immutable data. */
bool alg_related; /* True if alg data connection. */
enum ct_conn_type conn_type;
};
enum ct_update_res {
@@ -114,78 +119,6 @@ enum ct_update_res {
CT_UPDATE_NEW,
};
enum ct_conn_type {
CT_CONN_TYPE_DEFAULT,
CT_CONN_TYPE_UN_NAT,
};
/* 'struct ct_lock' is a wrapper for an adaptive mutex. It's useful to try
* different types of locks (e.g. spinlocks) */
struct OVS_LOCKABLE ct_lock {
struct ovs_mutex lock;
};
static inline void ct_lock_init(struct ct_lock *lock)
{
ovs_mutex_init_adaptive(&lock->lock);
}
static inline void ct_lock_lock(struct ct_lock *lock)
OVS_ACQUIRES(lock)
OVS_NO_THREAD_SAFETY_ANALYSIS
{
ovs_mutex_lock(&lock->lock);
}
static inline void ct_lock_unlock(struct ct_lock *lock)
OVS_RELEASES(lock)
OVS_NO_THREAD_SAFETY_ANALYSIS
{
ovs_mutex_unlock(&lock->lock);
}
static inline void ct_lock_destroy(struct ct_lock *lock)
{
ovs_mutex_destroy(&lock->lock);
}
struct OVS_LOCKABLE ct_rwlock {
struct ovs_rwlock lock;
};
static inline void ct_rwlock_init(struct ct_rwlock *lock)
{
ovs_rwlock_init(&lock->lock);
}
static inline void ct_rwlock_wrlock(struct ct_rwlock *lock)
OVS_ACQ_WRLOCK(lock)
OVS_NO_THREAD_SAFETY_ANALYSIS
{
ovs_rwlock_wrlock(&lock->lock);
}
static inline void ct_rwlock_rdlock(struct ct_rwlock *lock)
OVS_ACQ_RDLOCK(lock)
OVS_NO_THREAD_SAFETY_ANALYSIS
{
ovs_rwlock_rdlock(&lock->lock);
}
static inline void ct_rwlock_unlock(struct ct_rwlock *lock)
OVS_RELEASES(lock)
OVS_NO_THREAD_SAFETY_ANALYSIS
{
ovs_rwlock_unlock(&lock->lock);
}
static inline void ct_rwlock_destroy(struct ct_rwlock *lock)
{
ovs_rwlock_destroy(&lock->lock);
}
/* Timeouts: all the possible timeout states passed to update_expiration()
* are listed here. The name will be prefix by CT_TM_ and the value is in
* milliseconds */
@@ -217,115 +150,81 @@ enum ct_timeout {
N_CT_TM
};
/* Locking:
*
* The connections are kept in different buckets, which are completely
* independent. The connection bucket is determined by the hash of its key.
*
* Each bucket has two locks. Acquisition order is, from outermost to
* innermost:
*
* cleanup_mutex
* lock
*
* */
struct conntrack_bucket {
/* Protects 'connections' and 'exp_lists'. Used in the fast path */
struct ct_lock lock;
/* Contains the connections in the bucket, indexed by 'struct conn_key' */
struct hmap connections OVS_GUARDED;
/* For each possible timeout we have a list of connections. When the
* timeout of a connection is updated, we move it to the back of the list.
* Since the connection in a list have the same relative timeout, the list
* will be ordered, with the oldest connections to the front. */
struct ovs_list exp_lists[N_CT_TM] OVS_GUARDED;
/* Protects 'next_cleanup'. Used to make sure that there's only one thread
* performing the cleanup. */
struct ovs_mutex cleanup_mutex;
long long next_cleanup OVS_GUARDED;
};
#define CONNTRACK_BUCKETS_SHIFT 8
#define CONNTRACK_BUCKETS (1 << CONNTRACK_BUCKETS_SHIFT)
struct conntrack {
/* Independent buckets containing the connections */
struct conntrack_bucket buckets[CONNTRACK_BUCKETS];
struct ovs_mutex ct_lock; /* Protects 2 following fields. */
struct cmap conns OVS_GUARDED;
struct ovs_list exp_lists[N_CT_TM] OVS_GUARDED;
uint32_t hash_basis; /* Salt for hashing a connection key. */
pthread_t clean_thread; /* Periodically cleans up connection tracker. */
struct latch clean_thread_exit; /* To destroy the 'clean_thread'. */
/* Salt for hashing a connection key. */
uint32_t hash_basis;
/* The thread performing periodic cleanup of the connection
* tracker. */
pthread_t clean_thread;
/* Latch to destroy the 'clean_thread' */
struct latch clean_thread_exit;
/* Counting connections. */
atomic_count n_conn; /* Number of connections currently tracked. */
atomic_uint n_conn_limit; /* Max connections tracked. */
/* Number of connections currently in the connection tracker. */
atomic_count n_conn;
/* Connections limit. When this limit is reached, no new connection
* will be accepted. */
atomic_uint n_conn_limit;
/* The following resources are referenced during nat connection
* creation and deletion. */
struct hmap nat_conn_keys OVS_GUARDED;
/* Hash table for alg expectations. Expectations are created
* by control connections to help create data connections. */
struct hmap alg_expectations OVS_GUARDED;
/* Used to lookup alg expectations from the control context. */
struct hindex alg_expectation_refs OVS_GUARDED;
/* Expiry list for alg expectations. */
struct ovs_list alg_exp_list OVS_GUARDED;
/* This lock is used during NAT connection creation and deletion;
* it is taken after a bucket lock and given back before that
* bucket unlock.
* This lock is similarly used to guard alg_expectations and
* alg_expectation_refs. If a bucket lock is also held during
* the normal code flow, then is must be taken first and released
* last.
*/
struct ct_rwlock resources_lock;
/* Expectations for application level gateways (created by control
* connections to help create data connections, e.g. for FTP). */
struct ovs_rwlock resources_lock; /* Protects fields below. */
struct hmap alg_expectations OVS_GUARDED; /* Holds struct
* alg_exp_nodes. */
struct hindex alg_expectation_refs OVS_GUARDED; /* For lookup from
* control context. */
/* Fragmentation handling context. */
struct ipf *ipf;
};
struct ct_l4_proto {
struct conn *(*new_conn)(struct conntrack_bucket *, struct dp_packet *pkt,
long long now);
bool (*valid_new)(struct dp_packet *pkt);
enum ct_update_res (*conn_update)(struct conn *conn,
struct conntrack_bucket *,
struct dp_packet *pkt, bool reply,
long long now);
void (*conn_get_protoinfo)(const struct conn *,
struct ct_dpif_protoinfo *);
};
/* Lock acquisition order:
* 1. 'ct_lock'
* 2. 'conn->lock'
* 3. 'resources_lock'
*/
extern struct ct_l4_proto ct_proto_tcp;
extern struct ct_l4_proto ct_proto_other;
extern struct ct_l4_proto ct_proto_icmp4;
extern struct ct_l4_proto ct_proto_icmp6;
struct ct_l4_proto {
struct conn *(*new_conn)(struct conntrack *ct, struct dp_packet *pkt,
long long now);
bool (*valid_new)(struct dp_packet *pkt);
enum ct_update_res (*conn_update)(struct conntrack *ct, struct conn *conn,
struct dp_packet *pkt, bool reply,
long long now);
void (*conn_get_protoinfo)(const struct conn *,
struct ct_dpif_protoinfo *);
};
extern long long ct_timeout_val[];
/* ct_lock must be held. */
static inline void
conn_init_expiration(struct conntrack_bucket *ctb, struct conn *conn,
enum ct_timeout tm, long long now)
conn_init_expiration(struct conntrack *ct, struct conn *conn,
enum ct_timeout tm, long long now)
{
conn->expiration = now + ct_timeout_val[tm];
ovs_list_push_back(&ctb->exp_lists[tm], &conn->exp_node);
ovs_list_push_back(&ct->exp_lists[tm], &conn->exp_node);
}
/* The conn entry lock must be held on entry and exit. */
static inline void
conn_update_expiration(struct conntrack_bucket *ctb, struct conn *conn,
conn_update_expiration(struct conntrack *ct, struct conn *conn,
enum ct_timeout tm, long long now)
OVS_NO_THREAD_SAFETY_ANALYSIS
{
ovs_mutex_unlock(&conn->lock);
ovs_mutex_lock(&ct->ct_lock);
ovs_mutex_lock(&conn->lock);
conn->expiration = now + ct_timeout_val[tm];
ovs_list_remove(&conn->exp_node);
conn_init_expiration(ctb, conn, tm, now);
ovs_list_push_back(&ct->exp_lists[tm], &conn->exp_node);
ovs_mutex_unlock(&conn->lock);
ovs_mutex_unlock(&ct->ct_lock);
ovs_mutex_lock(&conn->lock);
}
static inline uint32_t

View File

@@ -44,16 +44,16 @@
#include "util.h"
struct tcp_peer {
enum ct_dpif_tcp_state state;
uint32_t seqlo; /* Max sequence number sent */
uint32_t seqhi; /* Max the other end ACKd + win */
uint16_t max_win; /* largest window (pre scaling) */
uint8_t wscale; /* window scaling factor */
enum ct_dpif_tcp_state state;
};
struct conn_tcp {
struct conn up;
struct tcp_peer peer[2];
struct tcp_peer peer[2]; /* 'conn' lock protected. */
};
enum {
@@ -145,7 +145,7 @@ tcp_get_wscale(const struct tcp_header *tcp)
}
static enum ct_update_res
tcp_conn_update(struct conn *conn_, struct conntrack_bucket *ctb,
tcp_conn_update(struct conntrack *ct, struct conn *conn_,
struct dp_packet *pkt, bool reply, long long now)
{
struct conn_tcp *conn = conn_tcp_cast(conn_);
@@ -317,18 +317,18 @@ tcp_conn_update(struct conn *conn_, struct conntrack_bucket *ctb,
if (src->state >= CT_DPIF_TCPS_FIN_WAIT_2
&& dst->state >= CT_DPIF_TCPS_FIN_WAIT_2) {
conn_update_expiration(ctb, &conn->up, CT_TM_TCP_CLOSED, now);
conn_update_expiration(ct, &conn->up, CT_TM_TCP_CLOSED, now);
} else if (src->state >= CT_DPIF_TCPS_CLOSING
&& dst->state >= CT_DPIF_TCPS_CLOSING) {
conn_update_expiration(ctb, &conn->up, CT_TM_TCP_FIN_WAIT, now);
conn_update_expiration(ct, &conn->up, CT_TM_TCP_FIN_WAIT, now);
} else if (src->state < CT_DPIF_TCPS_ESTABLISHED
|| dst->state < CT_DPIF_TCPS_ESTABLISHED) {
conn_update_expiration(ctb, &conn->up, CT_TM_TCP_OPENING, now);
conn_update_expiration(ct, &conn->up, CT_TM_TCP_OPENING, now);
} else if (src->state >= CT_DPIF_TCPS_CLOSING
|| dst->state >= CT_DPIF_TCPS_CLOSING) {
conn_update_expiration(ctb, &conn->up, CT_TM_TCP_CLOSING, now);
conn_update_expiration(ct, &conn->up, CT_TM_TCP_CLOSING, now);
} else {
conn_update_expiration(ctb, &conn->up, CT_TM_TCP_ESTABLISHED, now);
conn_update_expiration(ct, &conn->up, CT_TM_TCP_ESTABLISHED, now);
}
} else if ((dst->state < CT_DPIF_TCPS_SYN_SENT
|| dst->state >= CT_DPIF_TCPS_FIN_WAIT_2
@@ -412,8 +412,7 @@ tcp_valid_new(struct dp_packet *pkt)
}
static struct conn *
tcp_new_conn(struct conntrack_bucket *ctb, struct dp_packet *pkt,
long long now)
tcp_new_conn(struct conntrack *ct, struct dp_packet *pkt, long long now)
{
struct conn_tcp* newconn = NULL;
struct tcp_header *tcp = dp_packet_l4(pkt);
@@ -449,8 +448,7 @@ tcp_new_conn(struct conntrack_bucket *ctb, struct dp_packet *pkt,
src->state = CT_DPIF_TCPS_SYN_SENT;
dst->state = CT_DPIF_TCPS_CLOSED;
conn_init_expiration(ctb, &newconn->up, CT_TM_TCP_FIRST_PACKET,
now);
conn_init_expiration(ct, &newconn->up, CT_TM_TCP_FIRST_PACKET, now);
return &newconn->up;
}

File diff suppressed because it is too large Load Diff

View File

@@ -19,6 +19,7 @@
#include <stdbool.h>
#include "cmap.h"
#include "latch.h"
#include "odp-netlink.h"
#include "openvswitch/hmap.h"
@@ -38,20 +39,25 @@
* Usage
* =====
*
* struct conntrack ct;
* struct conntrack *ct;
*
* Initialization:
*
* conntrack_init(&ct);
* ct = conntrack_init();
*
* To send a group of packets through the connection tracker:
*
* conntrack_execute(&ct, pkt_batch, ...);
* conntrack_execute(ct, pkt_batch, ...);
*
* Thread-safety
* =============
* Thread-safety:
*
* conntrack_execute() can be called by multiple threads simultaneoulsy.
*
* Shutdown:
*
* 1/ Shutdown packet input to the datapath
* 2/ Destroy PMD threads after quiescence.
* 3/ conntrack_destroy(ct);
*/
struct dp_packet_batch;
@@ -93,7 +99,7 @@ void conntrack_clear(struct dp_packet *packet);
struct conntrack_dump {
struct conntrack *ct;
unsigned bucket;
struct hmap_position bucket_pos;
struct cmap_position cm_pos;
bool filter_zone;
uint16_t zone;
};
@@ -114,5 +120,4 @@ int conntrack_get_maxconns(struct conntrack *ct, uint32_t *maxconns);
int conntrack_get_nconns(struct conntrack *ct, uint32_t *nconns);
struct ipf *conntrack_ipf_ctx(struct conntrack *ct);
#endif /* conntrack.h */

View File

@@ -73,7 +73,7 @@ struct ct_dpif_timestamp {
CT_DPIF_TCP_STATE(TIME_WAIT) \
CT_DPIF_TCP_STATE(MAX_NUM)
enum ct_dpif_tcp_state {
enum OVS_PACKED_ENUM ct_dpif_tcp_state {
#define CT_DPIF_TCP_STATE(STATE) CT_DPIF_TCPS_##STATE,
CT_DPIF_TCP_STATES
#undef CT_DPIF_TCP_STATE