af-packet: kernel bypass implementation

This patch implements bypass capability for af-packet.

The filter only bypass TCP and UDP in IPv4 and IPv6. It don't
don't bypass IPv6 with extended headers.

This patch also introduces a bypassed flow manager that takes
care of timeouting the bypassed flows. It uses a 60 sec
timeout on flow. As they are supposed to be active we can
try that. If they are not active then we don't care to get them
back in Suricata.
pull/3221/head
Eric Leblond 7 years ago
parent 91e1256b01
commit 06173267c6

@ -1,6 +1,6 @@
if BUILD_EBPF
all: lb.bpf filter.bpf
all: lb.bpf filter.bpf bypass_filter.bpf
%.bpf: %.c
${CC} -Wall -O2 -D__KERNEL__ -D__ASM_SYSREG_H -emit-llvm -c $< -o - | ${LLC} -march=bpf -filetype=obj -o $@

@ -0,0 +1,193 @@
//#include <bcc/proto.h>
#include <stdint.h>
#include <stddef.h>
#include <linux/bpf.h>
#include <linux/if_ether.h>
#include <linux/in.h>
#include <linux/ip.h>
#include <linux/in6.h>
#include <linux/ipv6.h>
#include <linux/filter.h>
#include "bpf_helpers.h"
#define LINUX_VERSION_CODE 263682
struct flowv4_keys {
__be32 src;
__be32 dst;
union {
__be32 ports;
__be16 port16[2];
};
__u32 ip_proto;
} __attribute__((__aligned__(8)));
struct flowv6_keys {
__be32 src[4];
__be32 dst[4];
union {
__be32 ports;
__be16 port16[2];
};
__u32 ip_proto;
} __attribute__((__aligned__(8)));
struct pair {
uint64_t time;
uint64_t packets;
uint64_t bytes;
} __attribute__((__aligned__(8)));
struct bpf_map_def SEC("maps") flow_table_v4 = {
.type = BPF_MAP_TYPE_HASH,
.key_size = sizeof(struct flowv4_keys),
.value_size = sizeof(struct pair),
.max_entries = 32768,
};
struct bpf_map_def SEC("maps") flow_table_v6 = {
.type = BPF_MAP_TYPE_HASH,
.key_size = sizeof(struct flowv6_keys),
.value_size = sizeof(struct pair),
.max_entries = 32768,
};
static __always_inline int ipv4_filter(struct __sk_buff *skb)
{
uint32_t nhoff, verlen;
struct flowv4_keys tuple;
struct pair *value;
uint16_t port;
nhoff = skb->cb[0];
tuple.ip_proto = load_byte(skb, nhoff + offsetof(struct iphdr, protocol));
/* only support TCP and UDP for now */
switch (tuple.ip_proto) {
case IPPROTO_TCP:
case IPPROTO_UDP:
break;
default:
return -1;
}
tuple.src = load_word(skb, nhoff + offsetof(struct iphdr, saddr));
tuple.dst = load_word(skb, nhoff + offsetof(struct iphdr, daddr));
verlen = load_byte(skb, nhoff + 0/*offsetof(struct iphdr, ihl)*/);
nhoff += (verlen & 0xF) << 2;
tuple.ports = load_word(skb, nhoff);
port = tuple.port16[1];
tuple.port16[1] = tuple.port16[0];
tuple.port16[0] = port;
#if 0
if ((tuple.port16[0] == 22) || (tuple.port16[1] == 22))
{
uint16_t sp = tuple.port16[0];
//uint16_t dp = tuple.port16[1];
char fmt[] = "Parsed SSH flow: %u %d -> %u\n";
bpf_trace_printk(fmt, sizeof(fmt), tuple.src, sp, tuple.dst);
}
#endif
/* Test if src is in hash */
value = bpf_map_lookup_elem(&flow_table_v4, &tuple);
if (value) {
#if 0
{
uint16_t sp = tuple.port16[0];
//uint16_t dp = tuple.port16[1];
char bfmt[] = "Found flow: %u %d -> %u\n";
bpf_trace_printk(bfmt, sizeof(bfmt), tuple.src, sp, tuple.dst);
}
#endif
__sync_fetch_and_add(&value->packets, 1);
__sync_fetch_and_add(&value->bytes, skb->len);
value->time = bpf_ktime_get_ns();
return 0;
}
return -1;
}
static __always_inline int ipv6_filter(struct __sk_buff *skb)
{
uint32_t nhoff;
uint8_t nhdr;
struct flowv6_keys tuple;
struct pair *value;
uint16_t port;
nhoff = skb->cb[0];
tuple.src[0] = load_word(skb, nhoff + offsetof(struct ipv6hdr, saddr));
tuple.src[1] = load_word(skb, nhoff + offsetof(struct ipv6hdr, saddr) + 4);
tuple.src[2] = load_word(skb, nhoff + offsetof(struct ipv6hdr, saddr) + 8);
tuple.src[3] = load_word(skb, nhoff + offsetof(struct ipv6hdr, saddr) + 12);
tuple.dst[0] = load_word(skb, nhoff + offsetof(struct ipv6hdr, daddr));
tuple.dst[1] = load_word(skb, nhoff + offsetof(struct ipv6hdr, daddr) + 4);
tuple.dst[2] = load_word(skb, nhoff + offsetof(struct ipv6hdr, daddr) + 8);
tuple.dst[3] = load_word(skb, nhoff + offsetof(struct ipv6hdr, daddr) + 12);
/* get next header */
nhdr = load_byte(skb, nhoff + offsetof(struct ipv6hdr, nexthdr));
/* only support direct TCP and UDP for now */
switch (nhdr) {
case IPPROTO_TCP:
case IPPROTO_UDP:
break;
default:
return -1;
}
/* Parse TCP */
tuple.ports = load_word(skb, nhoff + 40 /* IPV6_HEADER_LEN */);
port = tuple.port16[1];
tuple.port16[1] = tuple.port16[0];
tuple.port16[0] = port;
tuple.ip_proto = nhdr;
//char fmt[] = "Now Got IPv6 port %u and %u\n";
//bpf_trace_printk(fmt, sizeof(fmt), tuple.port16[0], tuple.port16[1]);
/* Test if src is in hash */
value = bpf_map_lookup_elem(&flow_table_v6, &tuple);
if (value) {
//char fmt[] = "Got a match IPv6: %u and %u\n";
//bpf_trace_printk(fmt, sizeof(fmt), tuple.port16[0], tuple.port16[1]);
__sync_fetch_and_add(&value->packets, 1);
__sync_fetch_and_add(&value->bytes, skb->len);
value->time = bpf_ktime_get_ns();
return 0;
}
return -1;
}
int SEC("filter") hashfilter(struct __sk_buff *skb) {
__u32 nhoff = BPF_LL_OFF + ETH_HLEN;
skb->cb[0] = nhoff;
switch (skb->protocol) {
case __constant_htons(ETH_P_IP):
return ipv4_filter(skb);
case __constant_htons(ETH_P_IPV6):
return ipv6_filter(skb);
default:
#if 0
{
char fmt[] = "Got proto %u\n";
bpf_trace_printk(fmt, sizeof(fmt), h_proto);
break;
}
#else
break;
#endif
}
return -1;
}
char __license[] SEC("license") = "GPL";
uint32_t __version SEC("version") = LINUX_VERSION_CODE;

@ -260,6 +260,7 @@ detect-xbits.c detect-xbits.h \
detect-cipservice.c detect-cipservice.h \
flow-bit.c flow-bit.h \
flow.c flow.h \
flow-bypass.c flow-bypass.h \
flow-hash.c flow-hash.h \
flow-manager.c flow-manager.h \
flow-queue.c flow-queue.h \

@ -0,0 +1,178 @@
/* Copyright (C) 2016 Open Information Security Foundation
*
* You can copy, redistribute or modify this Program under the terms of
* the GNU General Public License version 2 as published by the Free
* Software Foundation.
*
* This program is distributed in the hope that it will be useful,
* but WITHOUT ANY WARRANTY; without even the implied warranty of
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
* GNU General Public License for more details.
*
* You should have received a copy of the GNU General Public License
* version 2 along with this program; if not, write to the Free Software
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA
* 02110-1301, USA.
*/
/**
* \file
*
* \author Eric Leblond <eleblond@stamus-networks.com>
*/
#include "suricata-common.h"
#include "tm-threads.h"
#include "flow.h"
#include "flow-bypass.h"
#include "flow-private.h"
#include "util-ebpf.h"
#define BYPASSED_FLOW_TIMEOUT 60
#define FLOW_BYPASS_DELAY 10
typedef struct BypassedFlowManagerThreadData_ {
uint16_t flow_bypassed_cnt_clo;
uint16_t flow_bypassed_pkts;
uint16_t flow_bypassed_bytes;
} BypassedFlowManagerThreadData;
#ifdef HAVE_PACKET_EBPF
static int BypassedFlowV4Timeout(int fd, struct flowv4_keys *key, struct pair *value, void *data)
{
struct timespec *curtime = (struct timespec *)data;
SCLogDebug("Got curtime %" PRIu64 " and value %" PRIu64 " (sp:%d, dp:%d)",
curtime->tv_sec, value->time / 1000000000,
key->port16[0], key->port16[1]
);
if (curtime->tv_sec - value->time / 1000000000 > BYPASSED_FLOW_TIMEOUT) {
SCLogDebug("Got no packet for %d -> %d at %" PRIu64,
key->port16[0], key->port16[1], value->time);
EBPFDeleteKey(fd, key);
return 1;
}
return 0;
}
static int BypassedFlowV6Timeout(int fd, struct flowv6_keys *key, struct pair *value, void *data)
{
struct timespec *curtime = (struct timespec *)data;
SCLogDebug("Got curtime %" PRIu64 " and value %" PRIu64 " (sp:%d, dp:%d)",
curtime->tv_sec, value->time / 1000000000,
key->port16[0], key->port16[1]
);
if (curtime->tv_sec - value->time / 1000000000 > BYPASSED_FLOW_TIMEOUT) {
SCLogDebug("Got no packet for %d -> %d at %" PRIu64,
key->port16[0], key->port16[1], value->time);
EBPFDeleteKey(fd, key);
return 1;
}
return 0;
}
#endif
static TmEcode BypassedFlowManager(ThreadVars *th_v, void *thread_data)
{
#ifdef HAVE_PACKET_EBPF
int tcount = 0;
BypassedFlowManagerThreadData *ftd = thread_data;
while (1) {
SCLogDebug("Dumping the table");
struct timespec curtime;
struct flows_stats bypassstats = { 0, 0, 0};
if (clock_gettime(CLOCK_MONOTONIC, &curtime) != 0) {
SCLogWarning(SC_ERR_INVALID_VALUE, "Can't get time: %s (%d)",
strerror(errno), errno);
sleep(1);
continue;
}
/* TODO indirection here: AF_PACKET and NFQ should be able to give their iterate function */
tcount = EBPFForEachFlowV4Table("flow_table_v4", BypassedFlowV4Timeout, &bypassstats, &curtime);
if (tcount) {
StatsAddUI64(th_v, ftd->flow_bypassed_cnt_clo, (uint64_t)bypassstats.count);
StatsAddUI64(th_v, ftd->flow_bypassed_pkts, (uint64_t)bypassstats.packets);
StatsAddUI64(th_v, ftd->flow_bypassed_bytes, (uint64_t)bypassstats.bytes);
}
memset(&bypassstats, 0, sizeof(bypassstats));
/* TODO indirection here: AF_PACKET and NFQ should be able to give their iterate function */
tcount = EBPFForEachFlowV6Table("flow_table_v6", BypassedFlowV6Timeout, &bypassstats, &curtime);
if (tcount) {
StatsAddUI64(th_v, ftd->flow_bypassed_cnt_clo, (uint64_t)bypassstats.count);
StatsAddUI64(th_v, ftd->flow_bypassed_pkts, (uint64_t)bypassstats.packets);
StatsAddUI64(th_v, ftd->flow_bypassed_bytes, (uint64_t)bypassstats.bytes);
}
if (TmThreadsCheckFlag(th_v, THV_KILL)) {
StatsSyncCounters(th_v);
return TM_ECODE_OK;
}
sleep(FLOW_BYPASS_DELAY);
StatsSyncCountersIfSignalled(th_v);
}
#endif
return TM_ECODE_OK;
}
static TmEcode BypassedFlowManagerThreadInit(ThreadVars *t, const void *initdata, void **data)
{
BypassedFlowManagerThreadData *ftd = SCCalloc(1, sizeof(BypassedFlowManagerThreadData));
if (ftd == NULL)
return TM_ECODE_FAILED;
*data = ftd;
ftd->flow_bypassed_cnt_clo = StatsRegisterCounter("flow_bypassed.closed", t);
ftd->flow_bypassed_pkts = StatsRegisterCounter("flow_bypassed.pkts", t);
ftd->flow_bypassed_bytes = StatsRegisterCounter("flow_bypassed.bytes", t);
return TM_ECODE_OK;
}
static TmEcode BypassedFlowManagerThreadDeinit(ThreadVars *t, void *data)
{
if (data)
SCFree(data);
return TM_ECODE_OK;
}
/** \brief spawn the flow manager thread */
void BypassedFlowManagerThreadSpawn()
{
#ifdef AFLFUZZ_DISABLE_MGTTHREADS
return;
#endif
#ifdef HAVE_PACKET_EBPF
ThreadVars *tv_flowmgr = NULL;
tv_flowmgr = TmThreadCreateMgmtThreadByName("BypassedFlowManager",
"BypassedFlowManager", 0);
BUG_ON(tv_flowmgr == NULL);
if (tv_flowmgr == NULL) {
printf("ERROR: TmThreadsCreate failed\n");
exit(1);
}
if (TmThreadSpawn(tv_flowmgr) != TM_ECODE_OK) {
printf("ERROR: TmThreadSpawn failed\n");
exit(1);
}
#endif
}
void TmModuleBypassedFlowManagerRegister (void)
{
tmm_modules[TMM_BYPASSEDFLOWMANAGER].name = "BypassedFlowManager";
tmm_modules[TMM_BYPASSEDFLOWMANAGER].ThreadInit = BypassedFlowManagerThreadInit;
tmm_modules[TMM_BYPASSEDFLOWMANAGER].ThreadDeinit = BypassedFlowManagerThreadDeinit;
tmm_modules[TMM_BYPASSEDFLOWMANAGER].Management = BypassedFlowManager;
tmm_modules[TMM_BYPASSEDFLOWMANAGER].cap_flags = 0;
tmm_modules[TMM_BYPASSEDFLOWMANAGER].flags = TM_FLAG_MANAGEMENT_TM;
SCLogDebug("%s registered", tmm_modules[TMM_BYPASSEDFLOWMANAGER].name);
}

@ -0,0 +1,34 @@
/* Copyright (C) 2016 Open Information Security Foundation
*
* You can copy, redistribute or modify this Program under the terms of
* the GNU General Public License version 2 as published by the Free
* Software Foundation.
*
* This program is distributed in the hope that it will be useful,
* but WITHOUT ANY WARRANTY; without even the implied warranty of
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
* GNU General Public License for more details.
*
* You should have received a copy of the GNU General Public License
* version 2 along with this program; if not, write to the Free Software
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA
* 02110-1301, USA.
*/
/**
* \file
*
* \author Eric Leblond <eleblond@stamus-networks.com>
*/
#ifndef __FLOW_BYPASS_H__
#define __FLOW_BYPASS_H__
void FlowAddToBypassed(Flow *f);
void BypassedFlowManagerThreadSpawn(void);
void TmModuleBypassedFlowManagerRegister(void);
#endif

@ -376,18 +376,20 @@ static void *ParseAFPConfig(const char *iface)
aconf->ebpf_lb_file = ebpf_file;
}
#ifdef HAVE_PACKET_EBPF
/* One shot loading of the eBPF file */
if (aconf->ebpf_lb_file && cluster_type == PACKET_FANOUT_EBPF) {
#ifdef HAVE_PACKET_EBPF
int ret = EBPFLoadFile(aconf->ebpf_lb_file, "loadbalancer",
&aconf->ebpf_lb_fd);
if (ret != 0) {
SCLogWarning(SC_ERR_INVALID_VALUE, "Error when loading eBPF lb file");
}
}
#else
if (aconf->ebpf_lb_file) {
SCLogError(SC_ERR_UNIMPLEMENTED, "eBPF support is not build-in");
#endif
}
#endif
if (ConfGetChildValueWithDefault(if_root, if_default, "ebpf-filter-file", &ebpf_file) != 1) {
aconf->ebpf_filter_file = NULL;
@ -397,6 +399,12 @@ static void *ParseAFPConfig(const char *iface)
ebpf_file);
#endif
aconf->ebpf_filter_file = ebpf_file;
ConfGetChildValueBoolWithDefault(if_root, if_default, "bypass", &conf_val);
if (conf_val) {
SCLogConfig("Using bypass kernel functionality for AF_PACKET (iface %s)",
aconf->iface);
aconf->flags |= AFP_BYPASS;
}
}
/* One shot loading of the eBPF file */

@ -52,6 +52,7 @@
#include "tmqh-flow.h"
#include "flow-manager.h"
#include "flow-bypass.h"
#include "counters.h"
int debuglog_enabled = 0;
@ -380,6 +381,7 @@ void RunModeDispatch(int runmode, const char *custom_mode)
/* spawn management threads */
FlowManagerThreadSpawn();
FlowRecyclerThreadSpawn();
BypassedFlowManagerThreadSpawn();
StatsSpawnThreads();
}
}

@ -64,6 +64,12 @@
#include <sys/ioctl.h>
#endif
#ifdef HAVE_PACKET_EBPF
#include "util-ebpf.h"
#include <bpf/libbpf.h>
#include <bpf/bpf.h>
#endif
struct bpf_program {
unsigned int bf_len;
struct bpf_insn *bf_insns;
@ -192,6 +198,8 @@ union thdr {
void *raw;
};
static int AFPBypassCallback(Packet *p);
#define MAX_MAPS 32
/**
* \brief Structure to hold thread specific variables.
@ -606,6 +614,9 @@ static int AFPRead(AFPThreadVars *ptv)
SCReturnInt(AFP_FAILURE);
}
PKT_SET_SRC(p, PKT_SRC_WIRE);
if (ptv->flags & AFP_BYPASS) {
p->BypassPacketsFlow = AFPBypassCallback;
}
/* get timestamp of packet via ioctl */
if (ioctl(ptv->socket, SIOCGSTAMP, &p->ts) == -1) {
@ -875,6 +886,9 @@ static int AFPReadFromRing(AFPThreadVars *ptv)
SCReturnInt(AFP_FAILURE);
}
PKT_SET_SRC(p, PKT_SRC_WIRE);
if (ptv->flags & AFP_BYPASS) {
p->BypassPacketsFlow = AFPBypassCallback;
}
/* Suricata will treat packet so telling it is busy, this
* status will be reset to 0 (ie TP_STATUS_KERNEL) in the release
@ -987,6 +1001,9 @@ static inline int AFPParsePacketV3(AFPThreadVars *ptv, struct tpacket_block_desc
SCReturnInt(AFP_FAILURE);
}
PKT_SET_SRC(p, PKT_SRC_WIRE);
if (ptv->flags & AFP_BYPASS) {
p->BypassPacketsFlow = AFPBypassCallback;
}
ptv->pkts++;
p->livedev = ptv->livedev;
@ -2191,6 +2208,123 @@ TmEcode AFPSetBPFFilter(AFPThreadVars *ptv)
return TM_ECODE_OK;
}
#ifdef HAVE_PACKET_EBPF
/**
* Insert a half flow in the kernel bypass table
*
* \param mapfd file descriptor of the protocol bypass table
* \param key data to use as key in the table
* \param inittime time of creation of the entry (in monotonic clock)
*/
static int AFPInsertHalfFlow(int mapd, void *key, uint64_t inittime)
{
/* FIXME error handling */
struct pair value = {inittime, 0, 0};
SCLogDebug("Inserting element in eBPF mapping");
if (bpf_map_update_elem(mapd, key, &value, BPF_NOEXIST) != 0) {
switch (errno) {
case E2BIG:
case EEXIST:
return 0;
default:
SCLogError(SC_ERR_BPF, "Can't update eBPF map: %s (%d)",
strerror(errno),
errno);
return 0;
}
}
return 1;
}
#endif
static int AFPBypassCallback(Packet *p)
{
#ifdef HAVE_PACKET_EBPF
SCLogDebug("Calling af_packet callback function");
/* Only bypass TCP and UDP */
if (!(PKT_IS_TCP(p) || PKT_IS_UDP(p))) {
return 0;
}
/* Bypassing tunneled packets is currently not supported
* because we can't discard the inner packet only due to
* primitive parsing in eBPF */
if (IS_TUNNEL_PKT(p)) {
return 0;
}
struct timespec curtime;
uint64_t inittime = 0;
/* In eBPF, the function that we have use to get time return the
* monotonic clock (the time since start of the computer). So we
* can't use the timestamp of the packet. */
if (clock_gettime(CLOCK_MONOTONIC, &curtime) == 0) {
inittime = curtime.tv_sec * 1000000000;
}
if (PKT_IS_IPV4(p)) {
/* FIXME cache this and handle error at cache time*/
int mapd = EBPFGetMapFDByName("flow_table_v4");
if (mapd == -1) {
SCLogNotice("Can't find eBPF map fd for '%s'", "flow_table_v4");
return 0;
}
/* FIXME error handling */
struct flowv4_keys key = {};
key.src = htonl(GET_IPV4_SRC_ADDR_U32(p));
key.dst = htonl(GET_IPV4_DST_ADDR_U32(p));
key.port16[0] = GET_TCP_SRC_PORT(p);
key.port16[1] = GET_TCP_DST_PORT(p);
key.ip_proto = IPV4_GET_IPPROTO(p);
if (AFPInsertHalfFlow(mapd, &key, inittime) == 0) {
return 0;
}
key.src = htonl(GET_IPV4_DST_ADDR_U32(p));
key.dst = htonl(GET_IPV4_SRC_ADDR_U32(p));
key.port16[0] = GET_TCP_DST_PORT(p);
key.port16[1] = GET_TCP_SRC_PORT(p);
if (AFPInsertHalfFlow(mapd, &key, inittime) == 0) {
return 0;
}
return 1;
}
/* For IPv6 case we don't handle extended header in eBPF */
if (PKT_IS_IPV6(p) &&
((IPV6_GET_NH(p) == IPPROTO_TCP) || (IPV6_GET_NH(p) == IPPROTO_UDP))) {
/* FIXME cache this and handle error at cache time*/
int mapd = EBPFGetMapFDByName("flow_table_v6");
int i = 0;
if (mapd == -1) {
SCLogNotice("Can't find eBPF map fd for '%s'", "flow_table_v6");
return 0;
}
SCLogDebug("add an IPv6");
/* FIXME error handling */
/* FIXME filter out next hdr IPV6 packets */
struct flowv6_keys key = {};
for (i = 0; i < 4; i++) {
key.src[i] = ntohl(GET_IPV6_SRC_ADDR(p)[i]);
key.dst[i] = ntohl(GET_IPV6_DST_ADDR(p)[i]);
}
key.port16[0] = GET_TCP_SRC_PORT(p);
key.port16[1] = GET_TCP_DST_PORT(p);
key.ip_proto = IPV6_GET_NH(p);
if (AFPInsertHalfFlow(mapd, &key, inittime) == 0) {
return 0;
}
for (i = 0; i < 4; i++) {
key.src[i] = ntohl(GET_IPV6_DST_ADDR(p)[i]);
key.dst[i] = ntohl(GET_IPV6_SRC_ADDR(p)[i]);
}
key.port16[0] = GET_TCP_DST_PORT(p);
key.port16[1] = GET_TCP_SRC_PORT(p);
if (AFPInsertHalfFlow(mapd, &key, inittime) == 0) {
return 0;
}
return 1;
}
#endif
return 0;
}
/**
* \brief Init function for ReceiveAFP.
*

@ -51,6 +51,7 @@
#define AFP_TPACKET_V3 (1<<4)
#define AFP_VLAN_DISABLED (1<<5)
#define AFP_MMAP_LOCKED (1<<6)
#define AFP_BYPASS (1<<7)
#define AFP_COPY_MODE_NONE 0
#define AFP_COPY_MODE_TAP 1

@ -100,6 +100,7 @@
#include "flow.h"
#include "flow-timeout.h"
#include "flow-manager.h"
#include "flow-bypass.h"
#include "flow-var.h"
#include "flow-bit.h"
#include "pkt-var.h"
@ -855,6 +856,7 @@ void RegisterAllModules(void)
/* managers */
TmModuleFlowManagerRegister();
TmModuleFlowRecyclerRegister();
TmModuleBypassedFlowManagerRegister();
/* nfq */
TmModuleReceiveNFQRegister();
TmModuleVerdictNFQRegister();

@ -229,6 +229,7 @@ const char * TmModuleTmmIdToString(TmmId id)
CASE_CODE (TMM_STATSLOGGER);
CASE_CODE (TMM_FLOWMANAGER);
CASE_CODE (TMM_FLOWRECYCLER);
CASE_CODE (TMM_BYPASSEDFLOWMANAGER);
CASE_CODE (TMM_UNIXMANAGER);
CASE_CODE (TMM_DETECTLOADER);
CASE_CODE (TMM_RECEIVENETMAP);

@ -64,6 +64,7 @@ typedef enum {
TMM_FLOWMANAGER,
TMM_FLOWRECYCLER,
TMM_BYPASSEDFLOWMANAGER,
TMM_DETECTLOADER,
TMM_UNIXMANAGER,

@ -45,15 +45,6 @@
#define BPF_MAP_MAX_COUNT 16
#define MAX_ERRNO 4095
#define IS_ERR_VALUE(x) unlikely((x) >= (unsigned long)-MAX_ERRNO)
static inline long IS_ERR(const void *ptr)
{
return IS_ERR_VALUE((unsigned long)ptr);
}
struct bpf_map_item {
const char * name;
int fd;
@ -69,14 +60,19 @@ int EBPFGetMapFDByName(const char *name)
if (name == NULL)
return -1;
for (i = 0; i < BPF_MAP_MAX_COUNT; i++) {
if (!bpf_map_array[i].name)
continue;
if (!strcmp(bpf_map_array[i].name, name)) {
SCLogNotice("Got fd %d for eBPF map '%s'", bpf_map_array[i].fd, name);
SCLogDebug("Got fd %d for eBPF map '%s'", bpf_map_array[i].fd, name);
return bpf_map_array[i].fd;
}
}
return -1;
}
#define bpf__is_error(ee) ee
#define bpf__get_error(ee) 1
/**
* Load a section of an eBPF file
*
@ -104,10 +100,13 @@ int EBPFLoadFile(const char *path, const char * section, int *val)
bpfobj = bpf_object__open(path);
if (IS_ERR(bpfobj)) {
if (libbpf_get_error(bpfobj)) {
char err_buf[128];
libbpf_strerror(bpf__get_error(bpfobj), err_buf,
sizeof(err_buf));
SCLogError(SC_ERR_INVALID_VALUE,
"Unable to load eBPF objects in '%s'",
path);
"Unable to load eBPF objects in '%s': %s",
path, err_buf);
return -1;
}
@ -149,7 +148,7 @@ int EBPFLoadFile(const char *path, const char * section, int *val)
/* store the map in our array */
bpf_map__for_each(map, bpfobj) {
SCLogNotice("Got a map '%s' with fd '%d'", bpf_map__name(map), bpf_map__fd(map));
SCLogDebug("Got a map '%s' with fd '%d'", bpf_map__name(map), bpf_map__fd(map));
bpf_map_array[bpf_map_last].fd = bpf_map__fd(map);
bpf_map_array[bpf_map_last].name = SCStrdup(bpf_map__name(map));
if (!bpf_map_array[bpf_map_last].name) {
@ -174,4 +173,81 @@ int EBPFLoadFile(const char *path, const char * section, int *val)
return 0;
}
int EBPFForEachFlowV4Table(const char *name,
int (*FlowCallback)(int fd, struct flowv4_keys *key, struct pair *value, void *data),
struct flows_stats *flowstats,
void *data)
{
int mapfd = EBPFGetMapFDByName(name);
struct flowv4_keys key = {}, next_key;
struct pair value = {0, 0, 0};
int ret, found = 0;
if (bpf_map_get_next_key(mapfd, &key, &next_key) != 0) {
return found;
}
while (bpf_map_get_next_key(mapfd, &key, &next_key) == 0) {
bpf_map_lookup_elem(mapfd, &key, &value);
ret = FlowCallback(mapfd, &key, &value, data);
if (ret) {
flowstats->count++;
flowstats->packets += value.packets;
flowstats->bytes += value.bytes;
found = 1;
}
key = next_key;
}
bpf_map_lookup_elem(mapfd, &key, &value);
ret = FlowCallback(mapfd, &key, &value, data);
if (ret) {
flowstats->count++;
flowstats->packets += value.packets;
flowstats->bytes += value.bytes;
found = 1;
}
return found;
}
int EBPFForEachFlowV6Table(const char *name,
int (*FlowCallback)(int fd, struct flowv6_keys *key, struct pair *value, void *data),
struct flows_stats *flowstats,
void *data)
{
int mapfd = EBPFGetMapFDByName(name);
struct flowv6_keys key = {}, next_key;
struct pair value = {0, 0, 0};
int ret, found = 0;
if (bpf_map_get_next_key(mapfd, &key, &next_key) != 0) {
return found;
}
while (bpf_map_get_next_key(mapfd, &key, &next_key) == 0) {
bpf_map_lookup_elem(mapfd, &key, &value);
ret = FlowCallback(mapfd, &key, &value, data);
if (ret) {
flowstats->count++;
flowstats->packets += value.packets;
flowstats->bytes += value.bytes;
found = 1;
}
key = next_key;
}
bpf_map_lookup_elem(mapfd, &key, &value);
ret = FlowCallback(mapfd, &key, &value, data);
if (ret) {
flowstats->count++;
flowstats->packets += value.packets;
flowstats->bytes += value.bytes;
found = 1;
}
return found;
}
void EBPFDeleteKey(int fd, void *key)
{
bpf_map_delete_elem(fd, key);
}
#endif

@ -24,7 +24,49 @@
#ifndef __UTIL_EBPF_H__
#define __UTIL_EBPF_H__
struct flowv4_keys {
__be32 src;
__be32 dst;
union {
__be32 ports;
__be16 port16[2];
};
__u32 ip_proto;
};
struct flowv6_keys {
__be32 src[4];
__be32 dst[4];
union {
__be32 ports;
__be16 port16[2];
};
__u32 ip_proto;
};
struct pair {
uint64_t time;
uint64_t packets;
uint64_t bytes;
};
struct flows_stats {
uint64_t count;
uint64_t packets;
uint64_t bytes;
};
int EBPFGetMapFDByName(const char *name);
int EBPFLoadFile(const char *path, const char * section, int *val);
int EBPFForEachFlowV4Table(const char *name,
int (*FlowCallback)(int fd, struct flowv4_keys *key, struct pair *value, void *data),
struct flows_stats *flowstats,
void *data);
int EBPFForEachFlowV6Table(const char *name,
int (*FlowCallback)(int fd, struct flowv6_keys *key, struct pair *value, void *data),
struct flows_stats *flowstats,
void *data);
void EBPFDeleteKey(int fd, void *key);
#endif

@ -635,6 +635,9 @@ af-packet:
# eBPF file containing a 'filter' function that will be inserted into the
# kernel and used as packet filter function
#ebpf-filter-file: @e_sysconfdir@/ebpf/filter.bpf
# if the ebpf filter implements a bypass function, you can set 'bypass' to
# yes and benefit from these feature
#bypass: yes
# After Linux kernel 3.10 it is possible to activate the rollover option: if a socket is
# full then kernel will send the packet on the next socket with room available. This option
# can minimize packet drop and increase the treated bandwidth on single intensive flow.

Loading…
Cancel
Save