Commit 4eaafe5a authored by Martin KaFai Lau's avatar Martin KaFai Lau Committed by Alexei Starovoitov

Merge branch 'bpf: tcp: Support arbitrary SYN Cookie at TC.'

Kuniyuki Iwashima says:

====================
Under SYN Flood, the TCP stack generates SYN Cookie to remain stateless
for the connection request until a valid ACK is responded to the SYN+ACK.

The cookie contains two kinds of host-specific bits, a timestamp and
secrets, so only can it be validated by the generator.  It means SYN
Cookie consumes network resources between the client and the server;
intermediate nodes must remember which nodes to route ACK for the cookie.

SYN Proxy reduces such unwanted resource allocation by handling 3WHS at
the edge network.  After SYN Proxy completes 3WHS, it forwards SYN to the
backend server and completes another 3WHS.  However, since the server's
ISN differs from the cookie, the proxy must manage the ISN mappings and
fix up SEQ/ACK numbers in every packet for each connection.  If a proxy
node goes down, all the connections through it are terminated.  Keeping
a state at proxy is painful from that perspective.

At AWS, we use a dirty hack to build truly stateless SYN Proxy at scale.
Our SYN Proxy consists of the front proxy layer and the backend kernel
module.  (See slides of LPC2023 [0], p37 - p48)

The cookie that SYN Proxy generates differs from the kernel's cookie in
that it contains a secret (called rolling salt) (i) shared by all the proxy
nodes so that any node can validate ACK and (ii) updated periodically so
that old cookies cannot be validated and we need not encode a timestamp for
the cookie.  Also, ISN contains WScale, SACK, and ECN, not in TS val.  This
is not to sacrifice any connection quality, where some customers turn off
TCP timestamps option due to retro CVE.

After 3WHS, the proxy restores SYN, encapsulates ACK into SYN, and forward
the TCP-in-TCP packet to the backend server.  Our kernel module works at
Netfilter input/output hooks and first feeds SYN to the TCP stack to
initiate 3WHS.  When the module is triggered for SYN+ACK, it looks up the
corresponding request socket and overwrites tcp_rsk(req)->snt_isn with the
proxy's cookie.  Then, the module can complete 3WHS with the original ACK
as is.

This way, our SYN Proxy does not manage the ISN mappings nor wait for
SYN+ACK from the backend thus can remain stateless.  It's working very
well for high-bandwidth services like multiple Tbps, but we are looking
for a way to drop the dirty hack and further optimise the sequences.

If we could validate an arbitrary SYN Cookie on the backend server with
BPF, the proxy would need not restore SYN nor pass it.  After validating
ACK, the proxy node just needs to forward it, and then the server can do
the lightweight validation (e.g. check if ACK came from proxy nodes, etc)
and create a connection from the ACK.

This series allows us to create a full sk from an arbitrary SYN Cookie,
which is done in 3 steps.

  1) At tc, BPF prog calls a new kfunc to create a reqsk and configure
     it based on the argument populated from SYN Cookie.  The reqsk has
     its listener as req->rsk_listener and is passed to the TCP stack as
     skb->sk.

  2) During TCP socket lookup for the skb, skb_steal_sock() returns a
     listener in the reuseport group that inet_reqsk(skb->sk)->rsk_listener
     belongs to.

  3) In cookie_v[46]_check(), the reqsk (skb->sk) is fully initialised and
     a full sk is created.

The kfunc usage is as follows:

    struct bpf_tcp_req_attrs attrs = {
        .mss = mss,
        .wscale_ok = wscale_ok,
        .rcv_wscale = rcv_wscale, /* Server's WScale < 15 */
        .snd_wscale = snd_wscale, /* Client's WScale < 15 */
        .tstamp_ok = tstamp_ok,
        .rcv_tsval = tsval,
        .rcv_tsecr = tsecr, /* Server's Initial TSval */
        .usec_ts_ok = usec_ts_ok,
        .sack_ok = sack_ok,
        .ecn_ok = ecn_ok,
    }

    skc = bpf_skc_lookup_tcp(...);
    sk = (struct sock *)bpf_skc_to_tcp_sock(skc);
    bpf_sk_assign_tcp_reqsk(skb, sk, attrs, sizeof(attrs));
    bpf_sk_release(skc);

[0]: https://lpc.events/event/17/contributions/1645/attachments/1350/2701/SYN_Proxy_at_Scale_with_BPF.pdf

Changes:
  v8
    * Rebase on Yonghong's cpuv4 fix
    * Patch 5
      * Fill the trailing 3-bytes padding in struct bpf_tcp_req_attrs
        and test it as null
    * Patch 6
      * Remove unused IPPROTP_MPTCP definition

  v7: https://lore.kernel.org/bpf/20231221012806.37137-1-kuniyu@amazon.com/
    * Patch 5 & 6
      * Drop MPTCP support

  v6: https://lore.kernel.org/bpf/20231214155424.67136-1-kuniyu@amazon.com/
    * Patch 5 & 6
      * /struct /s/tcp_cookie_attributes/bpf_tcp_req_attrs/
      * Don't reuse struct tcp_options_received and use u8 for each attrs
    * Patch 6
      * Check retval of test__start_subtest()

  v5: https://lore.kernel.org/netdev/20231211073650.90819-1-kuniyu@amazon.com/
    * Split patch 1-3
    * Patch 3
      * Clear req->rsk_listener in skb_steal_sock()
    * Patch 4 & 5
      * Move sysctl validation and tsoff init from cookie_bpf_check() to kfunc
    * Patch 5
      * Do not increment LINUX_MIB_SYNCOOKIES(RECV|FAILED)
    * Patch 6
      * Remove __always_inline
      * Test if tcp_handle_{syn,ack}() is executed
      * Move some definition to bpf_tracing_net.h
      * s/BPF_F_CURRENT_NETNS/-1/

  v4: https://lore.kernel.org/bpf/20231205013420.88067-1-kuniyu@amazon.com/
    * Patch 1 & 2
      * s/CONFIG_SYN_COOKIE/CONFIG_SYN_COOKIES/
    * Patch 1
      * Don't set rcv_wscale for BPF SYN Cookie case.
    * Patch 2
      * Add test for tcp_opt.{unused,rcv_wscale} in kfunc
      * Modify skb_steal_sock() to avoid resetting skb-sk
      * Support SO_REUSEPORT lookup
    * Patch 3
      * Add CONFIG_SYN_COOKIES to Kconfig for CI
      * Define BPF_F_CURRENT_NETNS

  v3: https://lore.kernel.org/netdev/20231121184245.69569-1-kuniyu@amazon.com/
    * Guard kfunc and req->syncookie part in inet6?_steal_sock() with
      CONFIG_SYN_COOKIE

  v2: https://lore.kernel.org/netdev/20231120222341.54776-1-kuniyu@amazon.com/
    * Drop SOCK_OPS and move SYN Cookie validation logic to TC with kfunc.
    * Add cleanup patches to reduce discrepancy between cookie_v[46]_check()

  v1: https://lore.kernel.org/bpf/20231013220433.70792-1-kuniyu@amazon.com/
====================
Signed-off-by: default avatarMartin KaFai Lau <martin.lau@kernel.org>
Signed-off-by: default avatarAlexei Starovoitov <ast@kernel.org>
parents d177c1be a7471224
......@@ -83,6 +83,45 @@ static inline struct sock *req_to_sk(struct request_sock *req)
return (struct sock *)req;
}
/**
* skb_steal_sock - steal a socket from an sk_buff
* @skb: sk_buff to steal the socket from
* @refcounted: is set to true if the socket is reference-counted
* @prefetched: is set to true if the socket was assigned from bpf
*/
static inline struct sock *skb_steal_sock(struct sk_buff *skb,
bool *refcounted, bool *prefetched)
{
struct sock *sk = skb->sk;
if (!sk) {
*prefetched = false;
*refcounted = false;
return NULL;
}
*prefetched = skb_sk_is_prefetched(skb);
if (*prefetched) {
#if IS_ENABLED(CONFIG_SYN_COOKIES)
if (sk->sk_state == TCP_NEW_SYN_RECV && inet_reqsk(sk)->syncookie) {
struct request_sock *req = inet_reqsk(sk);
*refcounted = false;
sk = req->rsk_listener;
req->rsk_listener = NULL;
return sk;
}
#endif
*refcounted = sk_is_refcounted(sk);
} else {
*refcounted = true;
}
skb->destructor = NULL;
skb->sk = NULL;
return sk;
}
static inline struct request_sock *
reqsk_alloc(const struct request_sock_ops *ops, struct sock *sk_listener,
bool attach_listener)
......
......@@ -2814,31 +2814,6 @@ sk_is_refcounted(struct sock *sk)
return !sk_fullsock(sk) || !sock_flag(sk, SOCK_RCU_FREE);
}
/**
* skb_steal_sock - steal a socket from an sk_buff
* @skb: sk_buff to steal the socket from
* @refcounted: is set to true if the socket is reference-counted
* @prefetched: is set to true if the socket was assigned from bpf
*/
static inline struct sock *
skb_steal_sock(struct sk_buff *skb, bool *refcounted, bool *prefetched)
{
if (skb->sk) {
struct sock *sk = skb->sk;
*refcounted = true;
*prefetched = skb_sk_is_prefetched(skb);
if (*prefetched)
*refcounted = sk_is_refcounted(sk);
skb->destructor = NULL;
skb->sk = NULL;
return sk;
}
*prefetched = false;
*refcounted = false;
return NULL;
}
/* Checks if this SKB belongs to an HW offloaded socket
* and whether any SW fallbacks are required based on dev.
* Check decrypted mark in case skb_orphan() cleared socket.
......
......@@ -577,6 +577,15 @@ static inline u32 tcp_cookie_time(void)
return val;
}
/* Convert one nsec 64bit timestamp to ts (ms or usec resolution) */
static inline u64 tcp_ns_to_ts(bool usec_ts, u64 val)
{
if (usec_ts)
return div_u64(val, NSEC_PER_USEC);
return div_u64(val, NSEC_PER_MSEC);
}
u32 __cookie_v4_init_sequence(const struct iphdr *iph, const struct tcphdr *th,
u16 *mssp);
__u32 cookie_v4_init_sequence(const struct sk_buff *skb, __u16 *mss);
......@@ -590,6 +599,40 @@ static inline bool cookie_ecn_ok(const struct net *net, const struct dst_entry *
dst_feature(dst, RTAX_FEATURE_ECN);
}
#if IS_ENABLED(CONFIG_BPF)
struct bpf_tcp_req_attrs {
u32 rcv_tsval;
u32 rcv_tsecr;
u16 mss;
u8 rcv_wscale;
u8 snd_wscale;
u8 ecn_ok;
u8 wscale_ok;
u8 sack_ok;
u8 tstamp_ok;
u8 usec_ts_ok;
u8 reserved[3];
};
static inline bool cookie_bpf_ok(struct sk_buff *skb)
{
return skb->sk;
}
struct request_sock *cookie_bpf_check(struct sock *sk, struct sk_buff *skb);
#else
static inline bool cookie_bpf_ok(struct sk_buff *skb)
{
return false;
}
static inline struct request_sock *cookie_bpf_check(struct net *net, struct sock *sk,
struct sk_buff *skb)
{
return NULL;
}
#endif
/* From net/ipv6/syncookies.c */
int __cookie_v6_check(const struct ipv6hdr *iph, const struct tcphdr *th);
struct sock *cookie_v6_check(struct sock *sk, struct sk_buff *skb);
......
......@@ -11837,6 +11837,103 @@ __bpf_kfunc int bpf_sock_addr_set_sun_path(struct bpf_sock_addr_kern *sa_kern,
return 0;
}
__bpf_kfunc int bpf_sk_assign_tcp_reqsk(struct sk_buff *skb, struct sock *sk,
struct bpf_tcp_req_attrs *attrs, int attrs__sz)
{
#if IS_ENABLED(CONFIG_SYN_COOKIES)
const struct request_sock_ops *ops;
struct inet_request_sock *ireq;
struct tcp_request_sock *treq;
struct request_sock *req;
struct net *net;
__u16 min_mss;
u32 tsoff = 0;
if (attrs__sz != sizeof(*attrs) ||
attrs->reserved[0] || attrs->reserved[1] || attrs->reserved[2])
return -EINVAL;
if (!skb_at_tc_ingress(skb))
return -EINVAL;
net = dev_net(skb->dev);
if (net != sock_net(sk))
return -ENETUNREACH;
switch (skb->protocol) {
case htons(ETH_P_IP):
ops = &tcp_request_sock_ops;
min_mss = 536;
break;
#if IS_BUILTIN(CONFIG_IPV6)
case htons(ETH_P_IPV6):
ops = &tcp6_request_sock_ops;
min_mss = IPV6_MIN_MTU - 60;
break;
#endif
default:
return -EINVAL;
}
if (sk->sk_type != SOCK_STREAM || sk->sk_state != TCP_LISTEN ||
sk_is_mptcp(sk))
return -EINVAL;
if (attrs->mss < min_mss)
return -EINVAL;
if (attrs->wscale_ok) {
if (!READ_ONCE(net->ipv4.sysctl_tcp_window_scaling))
return -EINVAL;
if (attrs->snd_wscale > TCP_MAX_WSCALE ||
attrs->rcv_wscale > TCP_MAX_WSCALE)
return -EINVAL;
}
if (attrs->sack_ok && !READ_ONCE(net->ipv4.sysctl_tcp_sack))
return -EINVAL;
if (attrs->tstamp_ok) {
if (!READ_ONCE(net->ipv4.sysctl_tcp_timestamps))
return -EINVAL;
tsoff = attrs->rcv_tsecr - tcp_ns_to_ts(attrs->usec_ts_ok, tcp_clock_ns());
}
req = inet_reqsk_alloc(ops, sk, false);
if (!req)
return -ENOMEM;
ireq = inet_rsk(req);
treq = tcp_rsk(req);
req->rsk_listener = sk;
req->syncookie = 1;
req->mss = attrs->mss;
req->ts_recent = attrs->rcv_tsval;
ireq->snd_wscale = attrs->snd_wscale;
ireq->rcv_wscale = attrs->rcv_wscale;
ireq->tstamp_ok = !!attrs->tstamp_ok;
ireq->sack_ok = !!attrs->sack_ok;
ireq->wscale_ok = !!attrs->wscale_ok;
ireq->ecn_ok = !!attrs->ecn_ok;
treq->req_usec_ts = !!attrs->usec_ts_ok;
treq->ts_off = tsoff;
skb_orphan(skb);
skb->sk = req_to_sk(req);
skb->destructor = sock_pfree;
return 0;
#else
return -EOPNOTSUPP;
#endif
}
__bpf_kfunc_end_defs();
int bpf_dynptr_from_skb_rdonly(struct sk_buff *skb, u64 flags,
......@@ -11865,6 +11962,10 @@ BTF_SET8_START(bpf_kfunc_check_set_sock_addr)
BTF_ID_FLAGS(func, bpf_sock_addr_set_sun_path)
BTF_SET8_END(bpf_kfunc_check_set_sock_addr)
BTF_SET8_START(bpf_kfunc_check_set_tcp_reqsk)
BTF_ID_FLAGS(func, bpf_sk_assign_tcp_reqsk, KF_TRUSTED_ARGS)
BTF_SET8_END(bpf_kfunc_check_set_tcp_reqsk)
static const struct btf_kfunc_id_set bpf_kfunc_set_skb = {
.owner = THIS_MODULE,
.set = &bpf_kfunc_check_set_skb,
......@@ -11880,6 +11981,11 @@ static const struct btf_kfunc_id_set bpf_kfunc_set_sock_addr = {
.set = &bpf_kfunc_check_set_sock_addr,
};
static const struct btf_kfunc_id_set bpf_kfunc_set_tcp_reqsk = {
.owner = THIS_MODULE,
.set = &bpf_kfunc_check_set_tcp_reqsk,
};
static int __init bpf_kfunc_init(void)
{
int ret;
......@@ -11895,8 +12001,9 @@ static int __init bpf_kfunc_init(void)
ret = ret ?: register_btf_kfunc_id_set(BPF_PROG_TYPE_LWT_SEG6LOCAL, &bpf_kfunc_set_skb);
ret = ret ?: register_btf_kfunc_id_set(BPF_PROG_TYPE_NETFILTER, &bpf_kfunc_set_skb);
ret = ret ?: register_btf_kfunc_id_set(BPF_PROG_TYPE_XDP, &bpf_kfunc_set_xdp);
return ret ?: register_btf_kfunc_id_set(BPF_PROG_TYPE_CGROUP_SOCK_ADDR,
&bpf_kfunc_set_sock_addr);
ret = ret ?: register_btf_kfunc_id_set(BPF_PROG_TYPE_CGROUP_SOCK_ADDR,
&bpf_kfunc_set_sock_addr);
return ret ?: register_btf_kfunc_id_set(BPF_PROG_TYPE_SCHED_CLS, &bpf_kfunc_set_tcp_reqsk);
}
late_initcall(bpf_kfunc_init);
......
......@@ -2582,8 +2582,18 @@ EXPORT_SYMBOL(sock_efree);
#ifdef CONFIG_INET
void sock_pfree(struct sk_buff *skb)
{
if (sk_is_refcounted(skb->sk))
sock_gen_put(skb->sk);
struct sock *sk = skb->sk;
if (!sk_is_refcounted(sk))
return;
if (sk->sk_state == TCP_NEW_SYN_RECV && inet_reqsk(sk)->syncookie) {
inet_reqsk(sk)->rsk_listener = NULL;
reqsk_free(inet_reqsk(sk));
return;
}
sock_gen_put(sk);
}
EXPORT_SYMBOL(sock_pfree);
#endif /* CONFIG_INET */
......
......@@ -51,15 +51,6 @@ static u32 cookie_hash(__be32 saddr, __be32 daddr, __be16 sport, __be16 dport,
count, &syncookie_secret[c]);
}
/* Convert one nsec 64bit timestamp to ts (ms or usec resolution) */
static u64 tcp_ns_to_ts(bool usec_ts, u64 val)
{
if (usec_ts)
return div_u64(val, NSEC_PER_USEC);
return div_u64(val, NSEC_PER_MSEC);
}
/*
* when syncookies are in effect and tcp timestamps are enabled we encode
* tcp options in the lower bits of the timestamp value that will be
......@@ -304,6 +295,24 @@ static int cookie_tcp_reqsk_init(struct sock *sk, struct sk_buff *skb,
return 0;
}
#if IS_ENABLED(CONFIG_BPF)
struct request_sock *cookie_bpf_check(struct sock *sk, struct sk_buff *skb)
{
struct request_sock *req = inet_reqsk(skb->sk);
skb->sk = NULL;
skb->destructor = NULL;
if (cookie_tcp_reqsk_init(sk, skb, req)) {
reqsk_free(req);
req = NULL;
}
return req;
}
EXPORT_SYMBOL_GPL(cookie_bpf_check);
#endif
struct request_sock *cookie_tcp_reqsk_alloc(const struct request_sock_ops *ops,
struct sock *sk, struct sk_buff *skb,
struct tcp_options_received *tcp_opt,
......@@ -404,9 +413,13 @@ struct sock *cookie_v4_check(struct sock *sk, struct sk_buff *skb)
!th->ack || th->rst)
goto out;
req = cookie_tcp_check(net, sk, skb);
if (IS_ERR(req))
goto out;
if (cookie_bpf_ok(skb)) {
req = cookie_bpf_check(sk, skb);
} else {
req = cookie_tcp_check(net, sk, skb);
if (IS_ERR(req))
goto out;
}
if (!req)
goto out_drop;
......@@ -454,7 +467,8 @@ struct sock *cookie_v4_check(struct sock *sk, struct sk_buff *skb)
ireq->wscale_ok, &rcv_wscale,
dst_metric(&rt->dst, RTAX_INITRWND));
ireq->rcv_wscale = rcv_wscale;
if (!req->syncookie)
ireq->rcv_wscale = rcv_wscale;
ireq->ecn_ok &= cookie_ecn_ok(net, &rt->dst);
ret = tcp_get_cookie_sock(sk, skb, req, &rt->dst);
......
......@@ -182,9 +182,13 @@ struct sock *cookie_v6_check(struct sock *sk, struct sk_buff *skb)
!th->ack || th->rst)
goto out;
req = cookie_tcp_check(net, sk, skb);
if (IS_ERR(req))
goto out;
if (cookie_bpf_ok(skb)) {
req = cookie_bpf_check(sk, skb);
} else {
req = cookie_tcp_check(net, sk, skb);
if (IS_ERR(req))
goto out;
}
if (!req)
goto out_drop;
......@@ -247,7 +251,8 @@ struct sock *cookie_v6_check(struct sock *sk, struct sk_buff *skb)
ireq->wscale_ok, &rcv_wscale,
dst_metric(dst, RTAX_INITRWND));
ireq->rcv_wscale = rcv_wscale;
if (!req->syncookie)
ireq->rcv_wscale = rcv_wscale;
ireq->ecn_ok &= cookie_ecn_ok(net, dst);
ret = tcp_get_cookie_sock(sk, skb, req, dst);
......
......@@ -51,6 +51,16 @@ extern int bpf_dynptr_clone(const struct bpf_dynptr *ptr, struct bpf_dynptr *clo
extern int bpf_sock_addr_set_sun_path(struct bpf_sock_addr_kern *sa_kern,
const __u8 *sun_path, __u32 sun_path__sz) __ksym;
/* Description
* Allocate and configure a reqsk and link it with a listener and skb.
* Returns
* Error code
*/
struct sock;
struct bpf_tcp_req_attrs;
extern int bpf_sk_assign_tcp_reqsk(struct __sk_buff *skb, struct sock *sk,
struct bpf_tcp_req_attrs *attrs, int attrs__sz) __ksym;
void *bpf_cast_to_kern_ctx(void *) __ksym;
void *bpf_rdonly_cast(void *obj, __u32 btf_id) __ksym;
......
......@@ -81,6 +81,7 @@ CONFIG_NF_NAT=y
CONFIG_RC_CORE=y
CONFIG_SECURITY=y
CONFIG_SECURITYFS=y
CONFIG_SYN_COOKIES=y
CONFIG_TEST_BPF=m
CONFIG_USERFAULTFD=y
CONFIG_VSOCKETS=y
......
// SPDX-License-Identifier: GPL-2.0
/* Copyright Amazon.com Inc. or its affiliates. */
#define _GNU_SOURCE
#include <sched.h>
#include <stdlib.h>
#include <net/if.h>
#include "test_progs.h"
#include "cgroup_helpers.h"
#include "network_helpers.h"
#include "test_tcp_custom_syncookie.skel.h"
static struct test_tcp_custom_syncookie_case {
int family, type;
char addr[16];
char name[10];
} test_cases[] = {
{
.name = "IPv4 TCP",
.family = AF_INET,
.type = SOCK_STREAM,
.addr = "127.0.0.1",
},
{
.name = "IPv6 TCP",
.family = AF_INET6,
.type = SOCK_STREAM,
.addr = "::1",
},
};
static int setup_netns(void)
{
if (!ASSERT_OK(unshare(CLONE_NEWNET), "create netns"))
return -1;
if (!ASSERT_OK(system("ip link set dev lo up"), "ip"))
goto err;
if (!ASSERT_OK(write_sysctl("/proc/sys/net/ipv4/tcp_ecn", "1"),
"write_sysctl"))
goto err;
return 0;
err:
return -1;
}
static int setup_tc(struct test_tcp_custom_syncookie *skel)
{
LIBBPF_OPTS(bpf_tc_hook, qdisc_lo, .attach_point = BPF_TC_INGRESS);
LIBBPF_OPTS(bpf_tc_opts, tc_attach,
.prog_fd = bpf_program__fd(skel->progs.tcp_custom_syncookie));
qdisc_lo.ifindex = if_nametoindex("lo");
if (!ASSERT_OK(bpf_tc_hook_create(&qdisc_lo), "qdisc add dev lo clsact"))
goto err;
if (!ASSERT_OK(bpf_tc_attach(&qdisc_lo, &tc_attach),
"filter add dev lo ingress"))
goto err;
return 0;
err:
return -1;
}
#define msg "Hello World"
#define msglen 11
static void transfer_message(int sender, int receiver)
{
char buf[msglen];
int ret;
ret = send(sender, msg, msglen, 0);
if (!ASSERT_EQ(ret, msglen, "send"))
return;
memset(buf, 0, sizeof(buf));
ret = recv(receiver, buf, msglen, 0);
if (!ASSERT_EQ(ret, msglen, "recv"))
return;
ret = strncmp(buf, msg, msglen);
if (!ASSERT_EQ(ret, 0, "strncmp"))
return;
}
static void create_connection(struct test_tcp_custom_syncookie_case *test_case)
{
int server, client, child;
server = start_server(test_case->family, test_case->type, test_case->addr, 0, 0);
if (!ASSERT_NEQ(server, -1, "start_server"))
return;
client = connect_to_fd(server, 0);
if (!ASSERT_NEQ(client, -1, "connect_to_fd"))
goto close_server;
child = accept(server, NULL, 0);
if (!ASSERT_NEQ(child, -1, "accept"))
goto close_client;
transfer_message(client, child);
transfer_message(child, client);
close(child);
close_client:
close(client);
close_server:
close(server);
}
void test_tcp_custom_syncookie(void)
{
struct test_tcp_custom_syncookie *skel;
int i;
if (setup_netns())
return;
skel = test_tcp_custom_syncookie__open_and_load();
if (!ASSERT_OK_PTR(skel, "open_and_load"))
return;
if (setup_tc(skel))
goto destroy_skel;
for (i = 0; i < ARRAY_SIZE(test_cases); i++) {
if (!test__start_subtest(test_cases[i].name))
continue;
skel->bss->handled_syn = false;
skel->bss->handled_ack = false;
create_connection(&test_cases[i]);
ASSERT_EQ(skel->bss->handled_syn, true, "SYN is not handled at tc.");
ASSERT_EQ(skel->bss->handled_ack, true, "ACK is not handled at tc");
}
destroy_skel:
system("tc qdisc del dev lo clsact");
test_tcp_custom_syncookie__destroy(skel);
}
......@@ -51,9 +51,25 @@
#define ICSK_TIME_LOSS_PROBE 5
#define ICSK_TIME_REO_TIMEOUT 6
#define ETH_ALEN 6
#define ETH_HLEN 14
#define ETH_P_IP 0x0800
#define ETH_P_IPV6 0x86DD
#define NEXTHDR_TCP 6
#define TCPOPT_NOP 1
#define TCPOPT_EOL 0
#define TCPOPT_MSS 2
#define TCPOPT_WINDOW 3
#define TCPOPT_TIMESTAMP 8
#define TCPOPT_SACK_PERM 4
#define TCPOLEN_MSS 4
#define TCPOLEN_WINDOW 3
#define TCPOLEN_TIMESTAMP 10
#define TCPOLEN_SACK_PERM 2
#define CHECKSUM_NONE 0
#define CHECKSUM_PARTIAL 3
......
// SPDX-License-Identifier: GPL-2.0
/* Copyright Amazon.com Inc. or its affiliates. */
#ifndef _TEST_SIPHASH_H
#define _TEST_SIPHASH_H
/* include/linux/bitops.h */
static inline u64 rol64(u64 word, unsigned int shift)
{
return (word << (shift & 63)) | (word >> ((-shift) & 63));
}
/* include/linux/siphash.h */
#define SIPHASH_PERMUTATION(a, b, c, d) ( \
(a) += (b), (b) = rol64((b), 13), (b) ^= (a), (a) = rol64((a), 32), \
(c) += (d), (d) = rol64((d), 16), (d) ^= (c), \
(a) += (d), (d) = rol64((d), 21), (d) ^= (a), \
(c) += (b), (b) = rol64((b), 17), (b) ^= (c), (c) = rol64((c), 32))
#define SIPHASH_CONST_0 0x736f6d6570736575ULL
#define SIPHASH_CONST_1 0x646f72616e646f6dULL
#define SIPHASH_CONST_2 0x6c7967656e657261ULL
#define SIPHASH_CONST_3 0x7465646279746573ULL
/* lib/siphash.c */
#define SIPROUND SIPHASH_PERMUTATION(v0, v1, v2, v3)
#define PREAMBLE(len) \
u64 v0 = SIPHASH_CONST_0; \
u64 v1 = SIPHASH_CONST_1; \
u64 v2 = SIPHASH_CONST_2; \
u64 v3 = SIPHASH_CONST_3; \
u64 b = ((u64)(len)) << 56; \
v3 ^= key->key[1]; \
v2 ^= key->key[0]; \
v1 ^= key->key[1]; \
v0 ^= key->key[0];
#define POSTAMBLE \
v3 ^= b; \
SIPROUND; \
SIPROUND; \
v0 ^= b; \
v2 ^= 0xff; \
SIPROUND; \
SIPROUND; \
SIPROUND; \
SIPROUND; \
return (v0 ^ v1) ^ (v2 ^ v3);
static inline u64 siphash_2u64(const u64 first, const u64 second, const siphash_key_t *key)
{
PREAMBLE(16)
v3 ^= first;
SIPROUND;
SIPROUND;
v0 ^= first;
v3 ^= second;
SIPROUND;
SIPROUND;
v0 ^= second;
POSTAMBLE
}
#endif
This diff is collapsed.
// SPDX-License-Identifier: GPL-2.0
/* Copyright Amazon.com Inc. or its affiliates. */
#ifndef _TEST_TCP_SYNCOOKIE_H
#define _TEST_TCP_SYNCOOKIE_H
#define __packed __attribute__((__packed__))
#define __force
#define ARRAY_SIZE(arr) (sizeof(arr) / sizeof((arr)[0]))
#define swap(a, b) \
do { \
typeof(a) __tmp = (a); \
(a) = (b); \
(b) = __tmp; \
} while (0)
#define swap_array(a, b) \
do { \
typeof(a) __tmp[sizeof(a)]; \
__builtin_memcpy(__tmp, a, sizeof(a)); \
__builtin_memcpy(a, b, sizeof(a)); \
__builtin_memcpy(b, __tmp, sizeof(a)); \
} while (0)
/* asm-generic/unaligned.h */
#define __get_unaligned_t(type, ptr) ({ \
const struct { type x; } __packed * __pptr = (typeof(__pptr))(ptr); \
__pptr->x; \
})
#define get_unaligned(ptr) __get_unaligned_t(typeof(*(ptr)), (ptr))
static inline u16 get_unaligned_be16(const void *p)
{
return bpf_ntohs(__get_unaligned_t(__be16, p));
}
static inline u32 get_unaligned_be32(const void *p)
{
return bpf_ntohl(__get_unaligned_t(__be32, p));
}
/* lib/checksum.c */
static inline u32 from64to32(u64 x)
{
/* add up 32-bit and 32-bit for 32+c bit */
x = (x & 0xffffffff) + (x >> 32);
/* add up carry.. */
x = (x & 0xffffffff) + (x >> 32);
return (u32)x;
}
static inline __wsum csum_tcpudp_nofold(__be32 saddr, __be32 daddr,
__u32 len, __u8 proto, __wsum sum)
{
unsigned long long s = (__force u32)sum;
s += (__force u32)saddr;
s += (__force u32)daddr;
#ifdef __BIG_ENDIAN
s += proto + len;
#else
s += (proto + len) << 8;
#endif
return (__force __wsum)from64to32(s);
}
/* asm-generic/checksum.h */
static inline __sum16 csum_fold(__wsum csum)
{
u32 sum = (__force u32)csum;
sum = (sum & 0xffff) + (sum >> 16);
sum = (sum & 0xffff) + (sum >> 16);
return (__force __sum16)~sum;
}
static inline __sum16 csum_tcpudp_magic(__be32 saddr, __be32 daddr, __u32 len,
__u8 proto, __wsum sum)
{
return csum_fold(csum_tcpudp_nofold(saddr, daddr, len, proto, sum));
}
/* net/ipv6/ip6_checksum.c */
static inline __sum16 csum_ipv6_magic(const struct in6_addr *saddr,
const struct in6_addr *daddr,
__u32 len, __u8 proto, __wsum csum)
{
int carry;
__u32 ulen;
__u32 uproto;
__u32 sum = (__force u32)csum;
sum += (__force u32)saddr->in6_u.u6_addr32[0];
carry = (sum < (__force u32)saddr->in6_u.u6_addr32[0]);
sum += carry;
sum += (__force u32)saddr->in6_u.u6_addr32[1];
carry = (sum < (__force u32)saddr->in6_u.u6_addr32[1]);
sum += carry;
sum += (__force u32)saddr->in6_u.u6_addr32[2];
carry = (sum < (__force u32)saddr->in6_u.u6_addr32[2]);
sum += carry;
sum += (__force u32)saddr->in6_u.u6_addr32[3];
carry = (sum < (__force u32)saddr->in6_u.u6_addr32[3]);
sum += carry;
sum += (__force u32)daddr->in6_u.u6_addr32[0];
carry = (sum < (__force u32)daddr->in6_u.u6_addr32[0]);
sum += carry;
sum += (__force u32)daddr->in6_u.u6_addr32[1];
carry = (sum < (__force u32)daddr->in6_u.u6_addr32[1]);
sum += carry;
sum += (__force u32)daddr->in6_u.u6_addr32[2];
carry = (sum < (__force u32)daddr->in6_u.u6_addr32[2]);
sum += carry;
sum += (__force u32)daddr->in6_u.u6_addr32[3];
carry = (sum < (__force u32)daddr->in6_u.u6_addr32[3]);
sum += carry;
ulen = (__force u32)bpf_htonl((__u32)len);
sum += ulen;
carry = (sum < ulen);
sum += carry;
uproto = (__force u32)bpf_htonl(proto);
sum += uproto;
carry = (sum < uproto);
sum += carry;
return csum_fold((__force __wsum)sum);
}
#endif
Markdown is supported
0%
or
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment