detect/flowbits: implement prefilter support

Allow for more efficient rules that 'prefilter' on flowbits with 'isset' logic.

This prefilter is enabled by default, which means that if no mpm is present or
no explicit prefilter is used, the flowbits prefilter will be set up for a rule.

flowbits 'isset' prefilter

For rules that have a 'flowbits:isset,<bit>' statement, a "regular" prefilter
facility is created. It means that the rules are removed from the normal
match list(s) and added to a prefilter engine that runs prior to the individual
rule inspection stage.

Implementation: the prefilter is implemented as an RB_TREE of flowbits, with the
rule id's they "enable" stored per tree node. The matching logic is walking the
list of bits set in the flow and looking each of them up in the RB_TREE, adding
the rule ids of each of the matching bits to the list of rule candidates.

The 'isset' prefilter has one important corner case, which is that bits can in
fact be set during the rule evaluation stage. This is different from all other
prefilter engines, that evaluate an immutable state (for the lifetime of the
packets inspection).

flowbits 'set' post-match prefilter

For flowbits 'set' action, special post-match 'prefilter' facilities deal with
this corner case. The high level logic is that these track which 'isset' sigs
depend on them, and add these dependencies to the candidates list when a 'set'
action occurs.

This is implemented in a few steps:

1. flowbits 'set' is flagged
2. when 'set' action occurs the flowbit is added to a "post rule
   match work queue"
3. when the rule evaluation ends, the post-match "prefilter" engine is run
   on each of the flowbits in the "post rule match work queue"
4. these engines ammend the candidates list with the rule id dependencies
   for the flowbit
5. the candidates list is sorted to make sure within the execution for that
   packet the inspection order is maintained

Ticket: #2486.
pull/12681/head
Victor Julien 4 years ago committed by Victor Julien
parent 749ee7788c
commit 03e395c3d9

@ -1,4 +1,4 @@
/* Copyright (C) 2016-2021 Open Information Security Foundation
/* Copyright (C) 2016-2025 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
@ -144,6 +144,35 @@ void DetectRunPrefilterTx(DetectEngineThreadCtx *det_ctx,
}
}
/** \brief invoke post-rule match "prefilter" engines
*
* Invoke prefilter engines that depend on a rule match to run.
* e.g. the flowbits:set prefilter that adds sids that depend on
* a flowbit "set" to the match array.
*/
void PrefilterPostRuleMatch(
DetectEngineThreadCtx *det_ctx, const SigGroupHead *sgh, Packet *p, Flow *f)
{
SCLogDebug("post-rule-match engines %p", sgh->post_rule_match_engines);
if (sgh->post_rule_match_engines) {
PrefilterEngine *engine = sgh->post_rule_match_engines;
do {
SCLogDebug("running post-rule-match engine");
PREFILTER_PROFILING_START(det_ctx);
engine->cb.PrefilterPostRule(det_ctx, engine->pectx, p, f);
PREFILTER_PROFILING_END(det_ctx, engine->gid);
if (engine->is_last)
break;
engine++;
} while (1);
if (det_ctx->pmq.rule_id_array_cnt > 1) {
QuickSortSigIntId(det_ctx->pmq.rule_id_array, det_ctx->pmq.rule_id_array_cnt);
}
}
}
void Prefilter(DetectEngineThreadCtx *det_ctx, const SigGroupHead *sgh, Packet *p,
const uint8_t flags, const SignatureMask mask)
{
@ -342,6 +371,39 @@ int PrefilterAppendFrameEngine(DetectEngineCtx *de_ctx, SigGroupHead *sgh,
return 0;
}
int PrefilterAppendPostRuleEngine(DetectEngineCtx *de_ctx, SigGroupHead *sgh,
void (*PrefilterPostRuleFunc)(
DetectEngineThreadCtx *det_ctx, const void *pectx, Packet *p, Flow *f),
void *pectx, void (*FreeFunc)(void *pectx), const char *name)
{
if (sgh == NULL || PrefilterPostRuleFunc == NULL || pectx == NULL)
return -1;
PrefilterEngineList *e = SCMallocAligned(sizeof(*e), CLS);
if (e == NULL)
return -1;
memset(e, 0x00, sizeof(*e));
e->PrefilterPostRule = PrefilterPostRuleFunc;
e->pectx = pectx;
e->Free = FreeFunc;
if (sgh->init->post_rule_match_engines == NULL) {
sgh->init->post_rule_match_engines = e;
} else {
PrefilterEngineList *t = sgh->init->post_rule_match_engines;
while (t->next != NULL) {
t = t->next;
}
t->next = e;
e->id = t->id + 1;
}
e->name = name;
e->gid = PrefilterStoreGetId(de_ctx, e->name, e->Free);
return 0;
}
static void PrefilterFreeEngineList(PrefilterEngineList *e)
{
if (e->Free && e->pectx) {
@ -396,6 +458,10 @@ void PrefilterCleanupRuleGroup(const DetectEngineCtx *de_ctx, SigGroupHead *sgh)
PrefilterFreeEngines(de_ctx, sgh->frame_engines);
sgh->frame_engines = NULL;
}
if (sgh->post_rule_match_engines) {
PrefilterFreeEngines(de_ctx, sgh->post_rule_match_engines);
sgh->post_rule_match_engines = NULL;
}
}
static int PrefilterSetupRuleGroupSortHelper(const void *a, const void *b)
@ -589,6 +655,30 @@ void PrefilterSetupRuleGroup(DetectEngineCtx *de_ctx, SigGroupHead *sgh)
e++;
}
}
if (sgh->init->post_rule_match_engines != NULL) {
uint32_t cnt = 0;
for (el = sgh->init->post_rule_match_engines; el != NULL; el = el->next) {
cnt++;
}
sgh->post_rule_match_engines = SCMallocAligned(cnt * sizeof(PrefilterEngine), CLS);
if (sgh->post_rule_match_engines == NULL) {
return;
}
memset(sgh->post_rule_match_engines, 0x00, (cnt * sizeof(PrefilterEngine)));
uint16_t local_id = 0;
PrefilterEngine *e = sgh->post_rule_match_engines;
for (el = sgh->init->post_rule_match_engines; el != NULL; el = el->next) {
e->local_id = local_id++;
e->cb.PrefilterPostRule = el->PrefilterPostRule;
e->pectx = el->pectx;
el->pectx = NULL; // e now owns the ctx
e->gid = el->gid;
e->is_last = (el->next == NULL);
e++;
}
SCLogDebug("sgh %p max local_id %u", sgh, local_id);
}
}
/* hash table for assigning a unique id to each engine type. */
@ -889,3 +979,30 @@ int PrefilterGenericMpmPktRegister(DetectEngineCtx *de_ctx, SigGroupHead *sgh, M
}
return r;
}
#define QUEUE_STEP 16
void PostRuleMatchWorkQueueAppend(
DetectEngineThreadCtx *det_ctx, const Signature *s, const int type, const uint32_t value)
{
if (det_ctx->post_rule_work_queue.q == NULL) {
det_ctx->post_rule_work_queue.q =
SCCalloc(1, sizeof(PostRuleMatchWorkQueueItem) * QUEUE_STEP);
BUG_ON(det_ctx->post_rule_work_queue.q == NULL);
det_ctx->post_rule_work_queue.size = QUEUE_STEP;
} else if (det_ctx->post_rule_work_queue.len == det_ctx->post_rule_work_queue.size) {
void *ptr = SCRealloc(
det_ctx->post_rule_work_queue.q, (det_ctx->post_rule_work_queue.size + QUEUE_STEP) *
sizeof(PostRuleMatchWorkQueueItem));
BUG_ON(ptr == NULL);
det_ctx->post_rule_work_queue.q = ptr;
det_ctx->post_rule_work_queue.size += QUEUE_STEP;
}
det_ctx->post_rule_work_queue.q[det_ctx->post_rule_work_queue.len].sm_type = type;
det_ctx->post_rule_work_queue.q[det_ctx->post_rule_work_queue.len].value = value;
#ifdef DEBUG
det_ctx->post_rule_work_queue.q[det_ctx->post_rule_work_queue.len].id = s->num;
#endif
det_ctx->post_rule_work_queue.len++;
SCLogDebug("det_ctx->post_rule_work_queue.len %u", det_ctx->post_rule_work_queue.len);
}

@ -1,4 +1,4 @@
/* Copyright (C) 2016 Open Information Security Foundation
/* Copyright (C) 2016-2025 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
@ -52,6 +52,10 @@ void Prefilter(DetectEngineThreadCtx *, const SigGroupHead *, Packet *p, const u
int PrefilterAppendEngine(DetectEngineCtx *de_ctx, SigGroupHead *sgh, PrefilterPktFn PrefilterFunc,
SignatureMask mask, void *pectx, void (*FreeFunc)(void *pectx), const char *name);
void PrefilterPostRuleMatch(
DetectEngineThreadCtx *det_ctx, const SigGroupHead *sgh, Packet *p, Flow *f);
int PrefilterAppendPayloadEngine(DetectEngineCtx *de_ctx, SigGroupHead *sgh,
PrefilterPktFn PrefilterFunc, void *pectx, void (*FreeFunc)(void *pectx), const char *name);
int PrefilterAppendTxEngine(DetectEngineCtx *de_ctx, SigGroupHead *sgh,
@ -60,6 +64,10 @@ int PrefilterAppendTxEngine(DetectEngineCtx *de_ctx, SigGroupHead *sgh,
int PrefilterAppendFrameEngine(DetectEngineCtx *de_ctx, SigGroupHead *sgh,
PrefilterFrameFn PrefilterFrameFunc, AppProto alproto, uint8_t frame_type, void *pectx,
void (*FreeFunc)(void *pectx), const char *name);
int PrefilterAppendPostRuleEngine(DetectEngineCtx *de_ctx, SigGroupHead *sgh,
void (*PrefilterPostRuleFunc)(
DetectEngineThreadCtx *det_ctx, const void *pectx, Packet *p, Flow *f),
void *pectx, void (*FreeFunc)(void *pectx), const char *name);
void DetectRunPrefilterTx(DetectEngineThreadCtx *det_ctx,
const SigGroupHead *sgh,
@ -91,4 +99,7 @@ int PrefilterMultiGenericMpmRegister(DetectEngineCtx *de_ctx, SigGroupHead *sgh,
int PrefilterGenericMpmPktRegister(DetectEngineCtx *de_ctx, SigGroupHead *sgh, MpmCtx *mpm_ctx,
const DetectBufferMpmRegistry *mpm_reg, int list_id);
void PostRuleMatchWorkQueueAppend(
DetectEngineThreadCtx *det_ctx, const Signature *s, const int type, const uint32_t value);
#endif

@ -81,6 +81,7 @@ void SigGroupHeadInitDataFree(SigGroupHeadInitData *sghid)
PrefilterFreeEnginesList(sghid->pkt_engines);
PrefilterFreeEnginesList(sghid->payload_engines);
PrefilterFreeEnginesList(sghid->frame_engines);
PrefilterFreeEnginesList(sghid->post_rule_match_engines);
SCFree(sghid);
}

@ -3564,6 +3564,9 @@ static void DetectEngineThreadCtxFree(DetectEngineThreadCtx *det_ctx)
AlertQueueFree(det_ctx);
if (det_ctx->post_rule_work_queue.q)
SCFree(det_ctx->post_rule_work_queue.q);
if (det_ctx->byte_values != NULL)
SCFree(det_ctx->byte_values);

@ -42,6 +42,9 @@
#include "detect-engine-mpm.h"
#include "detect-engine-state.h"
#include "detect-engine-build.h"
#include "detect-engine-prefilter.h"
#include "tree.h"
#include "util-var-name.h"
#include "util-unittest.h"
@ -61,6 +64,8 @@ void DetectFlowbitFree (DetectEngineCtx *, void *);
#ifdef UNITTESTS
void FlowBitsRegisterTests(void);
#endif
static bool PrefilterFlowbitIsPrefilterable(const Signature *s);
static int PrefilterSetupFlowbits(DetectEngineCtx *de_ctx, SigGroupHead *sgh);
void DetectFlowbitsRegister (void)
{
@ -76,6 +81,8 @@ void DetectFlowbitsRegister (void)
/* this is compatible to ip-only signatures */
sigmatch_table[DETECT_FLOWBITS].flags |= SIGMATCH_IPONLY_COMPAT;
sigmatch_table[DETECT_FLOWBITS].SupportsPrefilter = PrefilterFlowbitIsPrefilterable;
sigmatch_table[DETECT_FLOWBITS].SetupPrefilter = PrefilterSetupFlowbits;
DetectSetupParseRegexes(PARSE_REGEX, &parse_regex);
}
@ -132,11 +139,9 @@ static int FlowbitOrAddData(DetectEngineCtx *de_ctx, DetectFlowbitsData *cd, cha
static int DetectFlowbitMatchToggle (Packet *p, const DetectFlowbitsData *fd)
{
if (p->flow == NULL)
return 0;
FlowBitToggle(p->flow,fd->idx);
return -1;
return 1;
return FlowBitToggle(p->flow, fd->idx);
}
static int DetectFlowbitMatchUnset (Packet *p, const DetectFlowbitsData *fd)
@ -152,11 +157,11 @@ static int DetectFlowbitMatchUnset (Packet *p, const DetectFlowbitsData *fd)
static int DetectFlowbitMatchSet (Packet *p, const DetectFlowbitsData *fd)
{
if (p->flow == NULL)
return 0;
FlowBitSet(p->flow,fd->idx);
return -1;
return 1;
int r = FlowBitSet(p->flow, fd->idx);
SCLogDebug("set %u", fd->idx);
return r;
}
static int DetectFlowbitMatchIsset (Packet *p, const DetectFlowbitsData *fd)
@ -206,12 +211,25 @@ int DetectFlowbitMatch (DetectEngineThreadCtx *det_ctx, Packet *p,
return DetectFlowbitMatchIsset(p,fd);
case DETECT_FLOWBITS_CMD_ISNOTSET:
return DetectFlowbitMatchIsnotset(p,fd);
case DETECT_FLOWBITS_CMD_SET:
return DetectFlowbitMatchSet(p,fd);
case DETECT_FLOWBITS_CMD_SET: {
int r = DetectFlowbitMatchSet(p, fd);
/* only on a new "set" invoke the prefilter */
if (r == 1 && fd->post_rule_match_prefilter) {
SCLogDebug("flowbit set, appending to work queue");
PostRuleMatchWorkQueueAppend(det_ctx, s, DETECT_FLOWBITS, fd->idx);
}
return (r != -1);
}
case DETECT_FLOWBITS_CMD_UNSET:
return DetectFlowbitMatchUnset(p,fd);
case DETECT_FLOWBITS_CMD_TOGGLE:
return DetectFlowbitMatchToggle(p,fd);
case DETECT_FLOWBITS_CMD_TOGGLE: {
int r = DetectFlowbitMatchToggle(p, fd);
if (r == 1 && fd->post_rule_match_prefilter) {
SCLogDebug("flowbit set (by toggle), appending to work queue");
PostRuleMatchWorkQueueAppend(det_ctx, s, DETECT_FLOWBITS, fd->idx);
}
return (r != -1);
}
default:
SCLogError("unknown cmd %" PRIu32 "", fd->cmd);
return 0;
@ -388,6 +406,11 @@ void DetectFlowbitFree (DetectEngineCtx *de_ctx, void *ptr)
SCFree(fd);
}
struct FBAnalyzer {
struct FBAnalyze *array;
uint32_t array_size;
};
struct FBAnalyze {
uint16_t cnts[DETECT_FLOWBITS_CMD_MAX];
uint16_t state_cnts[DETECT_FLOWBITS_CMD_MAX];
@ -417,20 +440,143 @@ extern bool rule_engine_analysis_set;
static void DetectFlowbitsAnalyzeDump(const DetectEngineCtx *de_ctx,
struct FBAnalyze *array, uint32_t elements);
static void FBAnalyzerArrayFree(struct FBAnalyze *array, const uint32_t array_size)
{
if (array) {
for (uint32_t i = 0; i < array_size; i++) {
SCFree(array[i].set_sids);
SCFree(array[i].unset_sids);
SCFree(array[i].isset_sids);
SCFree(array[i].isnotset_sids);
SCFree(array[i].toggle_sids);
}
SCFree(array);
}
}
static void FBAnalyzerFree(struct FBAnalyzer *fba)
{
if (fba && fba->array) {
FBAnalyzerArrayFree(fba->array, fba->array_size);
fba->array = NULL;
fba->array_size = 0;
}
}
#define MAX_SIDS 8
static bool CheckExpand(const uint32_t sids_idx, uint32_t **sids, uint32_t *sids_size)
{
if (sids_idx >= *sids_size) {
const uint32_t old_size = *sids_size;
const uint32_t new_size = MAX(2 * old_size, MAX_SIDS);
void *ptr = SCRealloc(*sids, new_size * sizeof(uint32_t));
if (ptr == NULL)
return false;
*sids_size = new_size;
*sids = ptr;
}
return true;
}
static int DetectFlowbitsAnalyzeSignature(const Signature *s, struct FBAnalyzer *fba)
{
struct FBAnalyze *array = fba->array;
if (array == NULL)
return -1;
/* see if the signature uses stateful matching TODO is there not a flag? */
bool has_state = (s->init_data->buffer_index != 0);
for (const SigMatch *sm = s->init_data->smlists[DETECT_SM_LIST_MATCH]; sm != NULL;
sm = sm->next) {
if (sm->type != DETECT_FLOWBITS)
continue;
/* figure out the flowbit action */
const DetectFlowbitsData *fb = (DetectFlowbitsData *)sm->ctx;
// Handle flowbit array in case of ORed flowbits
for (uint8_t k = 0; k < fb->or_list_size; k++) {
struct FBAnalyze *fa = &array[fb->or_list[k]];
fa->cnts[fb->cmd]++;
fa->state_cnts[fb->cmd] += has_state;
if (fb->cmd == DETECT_FLOWBITS_CMD_ISSET) {
if (!CheckExpand(fa->isset_sids_idx, &fa->isset_sids, &fa->isset_sids_size))
return -1;
fa->isset_sids[fa->isset_sids_idx] = s->num;
fa->isset_sids_idx++;
} else if (fb->cmd == DETECT_FLOWBITS_CMD_ISNOTSET) {
if (!CheckExpand(
fa->isnotset_sids_idx, &fa->isnotset_sids, &fa->isnotset_sids_size))
return -1;
fa->isnotset_sids[fa->isnotset_sids_idx] = s->num;
fa->isnotset_sids_idx++;
}
}
if (fb->or_list_size == 0) {
struct FBAnalyze *fa = &array[fb->idx];
fa->cnts[fb->cmd]++;
fa->state_cnts[fb->cmd] += has_state;
if (fb->cmd == DETECT_FLOWBITS_CMD_ISSET) {
if (!CheckExpand(fa->isset_sids_idx, &fa->isset_sids, &fa->isset_sids_size))
return -1;
fa->isset_sids[fa->isset_sids_idx] = s->num;
fa->isset_sids_idx++;
} else if (fb->cmd == DETECT_FLOWBITS_CMD_ISNOTSET) {
if (!CheckExpand(
fa->isnotset_sids_idx, &fa->isnotset_sids, &fa->isnotset_sids_size))
return -1;
fa->isnotset_sids[fa->isnotset_sids_idx] = s->num;
fa->isnotset_sids_idx++;
}
}
}
for (const SigMatch *sm = s->init_data->smlists[DETECT_SM_LIST_POSTMATCH]; sm != NULL;
sm = sm->next) {
if (sm->type != DETECT_FLOWBITS)
continue;
/* figure out what flowbit action */
const DetectFlowbitsData *fb = (DetectFlowbitsData *)sm->ctx;
struct FBAnalyze *fa = &array[fb->idx];
fa->cnts[fb->cmd]++;
fa->state_cnts[fb->cmd] += has_state;
if (fb->cmd == DETECT_FLOWBITS_CMD_SET) {
if (!CheckExpand(fa->set_sids_idx, &fa->set_sids, &fa->set_sids_size))
return -1;
fa->set_sids[fa->set_sids_idx] = s->num;
fa->set_sids_idx++;
} else if (fb->cmd == DETECT_FLOWBITS_CMD_UNSET) {
if (!CheckExpand(fa->unset_sids_idx, &fa->unset_sids, &fa->unset_sids_size))
return -1;
fa->unset_sids[fa->unset_sids_idx] = s->num;
fa->unset_sids_idx++;
} else if (fb->cmd == DETECT_FLOWBITS_CMD_TOGGLE) {
if (!CheckExpand(fa->toggle_sids_idx, &fa->toggle_sids, &fa->toggle_sids_size))
return -1;
fa->toggle_sids[fa->toggle_sids_idx] = s->num;
fa->toggle_sids_idx++;
}
}
return 0;
}
int DetectFlowbitsAnalyze(DetectEngineCtx *de_ctx)
{
const uint32_t max_fb_id = de_ctx->max_fb_id;
if (max_fb_id == 0)
return 0;
#define MAX_SIDS 8
uint32_t array_size = max_fb_id + 1;
struct FBAnalyzer fba = { .array = NULL, .array_size = 0 };
const uint32_t array_size = max_fb_id + 1;
struct FBAnalyze *array = SCCalloc(array_size, sizeof(struct FBAnalyze));
if (array == NULL) {
SCLogError("Unable to allocate flowbit analyze array");
return -1;
}
fba.array = array;
fba.array_size = array_size;
SCLogDebug("fb analyzer array size: %"PRIu64,
(uint64_t)(array_size * sizeof(struct FBAnalyze)));
@ -439,143 +585,10 @@ int DetectFlowbitsAnalyze(DetectEngineCtx *de_ctx)
for (uint32_t i = 0; i < de_ctx->sig_array_len; i++) {
const Signature *s = de_ctx->sig_array[i];
/* see if the signature uses stateful matching */
bool has_state = (s->init_data->buffer_index != 0);
for (const SigMatch *sm = s->init_data->smlists[DETECT_SM_LIST_MATCH] ; sm != NULL; sm = sm->next) {
switch (sm->type) {
case DETECT_FLOWBITS:
{
/* figure out the flowbit action */
const DetectFlowbitsData *fb = (DetectFlowbitsData *)sm->ctx;
// Handle flowbit array in case of ORed flowbits
for (uint8_t k = 0; k < fb->or_list_size; k++) {
array[fb->or_list[k]].cnts[fb->cmd]++;
if (has_state)
array[fb->or_list[k]].state_cnts[fb->cmd]++;
if (fb->cmd == DETECT_FLOWBITS_CMD_ISSET) {
if (array[fb->or_list[k]].isset_sids_idx >= array[fb->or_list[k]].isset_sids_size) {
uint32_t old_size = array[fb->or_list[k]].isset_sids_size;
uint32_t new_size = MAX(2 * old_size, MAX_SIDS);
void *ptr = SCRealloc(array[fb->or_list[k]].isset_sids, new_size * sizeof(uint32_t));
if (ptr == NULL)
goto end;
array[fb->or_list[k]].isset_sids_size = new_size;
array[fb->or_list[k]].isset_sids = ptr;
}
array[fb->or_list[k]].isset_sids[array[fb->or_list[k]].isset_sids_idx] = s->num;
array[fb->or_list[k]].isset_sids_idx++;
} else if (fb->cmd == DETECT_FLOWBITS_CMD_ISNOTSET) {
if (array[fb->or_list[k]].isnotset_sids_idx >= array[fb->or_list[k]].isnotset_sids_size) {
uint32_t old_size = array[fb->or_list[k]].isnotset_sids_size;
uint32_t new_size = MAX(2 * old_size, MAX_SIDS);
void *ptr = SCRealloc(array[fb->or_list[k]].isnotset_sids, new_size * sizeof(uint32_t));
if (ptr == NULL)
goto end;
array[fb->or_list[k]].isnotset_sids_size = new_size;
array[fb->or_list[k]].isnotset_sids = ptr;
}
array[fb->or_list[k]].isnotset_sids[array[fb->or_list[k]].isnotset_sids_idx] = s->num;
array[fb->or_list[k]].isnotset_sids_idx++;
}
}
if (fb->or_list_size == 0) {
array[fb->idx].cnts[fb->cmd]++;
if (has_state)
array[fb->idx].state_cnts[fb->cmd]++;
if (fb->cmd == DETECT_FLOWBITS_CMD_ISSET) {
if (array[fb->idx].isset_sids_idx >= array[fb->idx].isset_sids_size) {
uint32_t old_size = array[fb->idx].isset_sids_size;
uint32_t new_size = MAX(2 * old_size, MAX_SIDS);
void *ptr = SCRealloc(array[fb->idx].isset_sids, new_size * sizeof(uint32_t));
if (ptr == NULL)
goto end;
array[fb->idx].isset_sids_size = new_size;
array[fb->idx].isset_sids = ptr;
}
array[fb->idx].isset_sids[array[fb->idx].isset_sids_idx] = s->num;
array[fb->idx].isset_sids_idx++;
} else if (fb->cmd == DETECT_FLOWBITS_CMD_ISNOTSET) {
if (array[fb->idx].isnotset_sids_idx >= array[fb->idx].isnotset_sids_size) {
uint32_t old_size = array[fb->idx].isnotset_sids_size;
uint32_t new_size = MAX(2 * old_size, MAX_SIDS);
void *ptr = SCRealloc(array[fb->idx].isnotset_sids, new_size * sizeof(uint32_t));
if (ptr == NULL)
goto end;
array[fb->idx].isnotset_sids_size = new_size;
array[fb->idx].isnotset_sids = ptr;
}
array[fb->idx].isnotset_sids[array[fb->idx].isnotset_sids_idx] = s->num;
array[fb->idx].isnotset_sids_idx++;
}
}
}
}
}
for (const SigMatch *sm = s->init_data->smlists[DETECT_SM_LIST_POSTMATCH] ; sm != NULL; sm = sm->next) {
switch (sm->type) {
case DETECT_FLOWBITS:
{
/* figure out what flowbit action */
const DetectFlowbitsData *fb = (DetectFlowbitsData *)sm->ctx;
array[fb->idx].cnts[fb->cmd]++;
if (has_state)
array[fb->idx].state_cnts[fb->cmd]++;
if (fb->cmd == DETECT_FLOWBITS_CMD_SET) {
if (array[fb->idx].set_sids_idx >= array[fb->idx].set_sids_size) {
uint32_t old_size = array[fb->idx].set_sids_size;
uint32_t new_size = MAX(2 * old_size, MAX_SIDS);
void *ptr = SCRealloc(array[fb->idx].set_sids, new_size * sizeof(uint32_t));
if (ptr == NULL)
goto end;
array[fb->idx].set_sids_size = new_size;
array[fb->idx].set_sids = ptr;
}
array[fb->idx].set_sids[array[fb->idx].set_sids_idx] = s->num;
array[fb->idx].set_sids_idx++;
}
else if (fb->cmd == DETECT_FLOWBITS_CMD_UNSET) {
if (array[fb->idx].unset_sids_idx >= array[fb->idx].unset_sids_size) {
uint32_t old_size = array[fb->idx].unset_sids_size;
uint32_t new_size = MAX(2 * old_size, MAX_SIDS);
void *ptr = SCRealloc(array[fb->idx].unset_sids, new_size * sizeof(uint32_t));
if (ptr == NULL)
goto end;
array[fb->idx].unset_sids_size = new_size;
array[fb->idx].unset_sids = ptr;
}
array[fb->idx].unset_sids[array[fb->idx].unset_sids_idx] = s->num;
array[fb->idx].unset_sids_idx++;
}
else if (fb->cmd == DETECT_FLOWBITS_CMD_TOGGLE) {
if (array[fb->idx].toggle_sids_idx >= array[fb->idx].toggle_sids_size) {
uint32_t old_size = array[fb->idx].toggle_sids_size;
uint32_t new_size = MAX(2 * old_size, MAX_SIDS);
void *ptr = SCRealloc(array[fb->idx].toggle_sids, new_size * sizeof(uint32_t));
if (ptr == NULL)
goto end;
array[fb->idx].toggle_sids_size = new_size;
array[fb->idx].toggle_sids = ptr;
}
array[fb->idx].toggle_sids[array[fb->idx].toggle_sids_idx] = s->num;
array[fb->idx].toggle_sids_idx++;
}
}
}
int r = DetectFlowbitsAnalyzeSignature(s, &fba);
if (r < 0) {
FBAnalyzerFree(&fba);
return -1;
}
}
@ -640,14 +653,14 @@ int DetectFlowbitsAnalyze(DetectEngineCtx *de_ctx)
SCCalloc(sids_array_size, sizeof(uint32_t));
if (s->init_data->rule_state_dependant_sids_array == NULL) {
SCLogError("Failed to allocate memory for rule_state_dependant_ids");
goto end;
goto error;
}
s->init_data->rule_state_flowbits_ids_size = 1;
s->init_data->rule_state_flowbits_ids_array =
SCCalloc(s->init_data->rule_state_flowbits_ids_size, sizeof(uint32_t));
if (s->init_data->rule_state_flowbits_ids_array == NULL) {
SCLogError("Failed to allocate memory for rule_state_variable_idx");
goto end;
goto error;
}
s->init_data->rule_state_dependant_sids_size = sids_array_size;
SCLogDebug("alloc'ed array for rule dependency and fbs idx array, sid %u, "
@ -661,7 +674,7 @@ int DetectFlowbitsAnalyze(DetectEngineCtx *de_ctx)
new_array_size * sizeof(uint32_t));
if (tmp_ptr == NULL) {
SCLogError("Failed to allocate memory for rule_state_variable_idx");
goto end;
goto error;
}
s->init_data->rule_state_dependant_sids_array = tmp_ptr;
s->init_data->rule_state_dependant_sids_size = new_array_size;
@ -673,7 +686,7 @@ int DetectFlowbitsAnalyze(DetectEngineCtx *de_ctx)
s->init_data->rule_state_flowbits_ids_array = tmp_fb_ptr;
if (s->init_data->rule_state_flowbits_ids_array == NULL) {
SCLogError("Failed to reallocate memory for rule_state_variable_idx");
goto end;
goto error;
}
SCLogDebug(
"realloc'ed array for flowbits ids, new size is %u", new_fb_array_size);
@ -704,17 +717,96 @@ int DetectFlowbitsAnalyze(DetectEngineCtx *de_ctx)
DetectFlowbitsAnalyzeDump(de_ctx, array, array_size);
}
end:
FBAnalyzerFree(&fba);
return 0;
error:
FBAnalyzerFree(&fba);
return -1;
}
// TODO misses IPOnly rules. IPOnly flowbit rules are set only though.
static struct FBAnalyzer DetectFlowbitsAnalyzeForGroup(
const DetectEngineCtx *de_ctx, SigGroupHead *sgh)
{
struct FBAnalyzer fba = { .array = NULL, .array_size = 0 };
const uint32_t max_fb_id = de_ctx->max_fb_id;
if (max_fb_id == 0)
return fba;
uint32_t array_size = max_fb_id + 1;
struct FBAnalyze *array = SCCalloc(array_size, sizeof(struct FBAnalyze));
if (array == NULL) {
SCLogError("Unable to allocate flowbit analyze array");
return fba;
}
SCLogDebug(
"fb analyzer array size: %" PRIu64, (uint64_t)(array_size * sizeof(struct FBAnalyze)));
fba.array = array;
fba.array_size = array_size;
/* fill flowbit array, updating counters per sig */
for (uint32_t i = 0; i < sgh->init->sig_cnt; i++) {
const Signature *s = sgh->init->match_array[i];
SCLogDebug("sgh %p: s->id %u", sgh, s->id);
int r = DetectFlowbitsAnalyzeSignature(s, &fba);
if (r < 0) {
FBAnalyzerFree(&fba);
return fba;
}
}
/* walk array to see if all bits make sense */
for (uint32_t i = 0; i < array_size; i++) {
SCFree(array[i].set_sids);
SCFree(array[i].unset_sids);
SCFree(array[i].isset_sids);
SCFree(array[i].isnotset_sids);
SCFree(array[i].toggle_sids);
const char *varname = VarNameStoreSetupLookup(i, VAR_TYPE_FLOW_BIT);
if (varname == NULL)
continue;
bool to_state = false;
if (array[i].state_cnts[DETECT_FLOWBITS_CMD_ISSET] &&
array[i].state_cnts[DETECT_FLOWBITS_CMD_SET] == 0) {
SCLogDebug("flowbit %s/%u: isset in state, set not in state", varname, i);
}
/* if signature depends on 'stateful' flowbits, then turn the
* sig into a stateful sig itself */
if (array[i].cnts[DETECT_FLOWBITS_CMD_ISSET] > 0 &&
array[i].state_cnts[DETECT_FLOWBITS_CMD_ISSET] == 0 &&
array[i].state_cnts[DETECT_FLOWBITS_CMD_SET]) {
SCLogDebug("flowbit %s/%u: isset not in state, set in state", varname, i);
to_state = true;
}
SCLogDebug("ALL flowbit %s/%u: sets %u toggles %u unsets %u isnotsets %u issets %u",
varname, i, array[i].cnts[DETECT_FLOWBITS_CMD_SET],
array[i].cnts[DETECT_FLOWBITS_CMD_TOGGLE], array[i].cnts[DETECT_FLOWBITS_CMD_UNSET],
array[i].cnts[DETECT_FLOWBITS_CMD_ISNOTSET],
array[i].cnts[DETECT_FLOWBITS_CMD_ISSET]);
SCLogDebug("STATE flowbit %s/%u: sets %u toggles %u unsets %u isnotsets %u issets %u",
varname, i, array[i].state_cnts[DETECT_FLOWBITS_CMD_SET],
array[i].state_cnts[DETECT_FLOWBITS_CMD_TOGGLE],
array[i].state_cnts[DETECT_FLOWBITS_CMD_UNSET],
array[i].state_cnts[DETECT_FLOWBITS_CMD_ISNOTSET],
array[i].state_cnts[DETECT_FLOWBITS_CMD_ISSET]);
for (uint32_t x = 0; x < array[i].set_sids_idx; x++) {
SCLogDebug("SET flowbit %s/%u: SID %u", varname, i,
de_ctx->sig_array[array[i].set_sids[x]]->id);
}
for (uint32_t x = 0; x < array[i].isset_sids_idx; x++) {
Signature *s = de_ctx->sig_array[array[i].isset_sids[x]];
SCLogDebug("GET flowbit %s/%u: SID %u", varname, i, s->id);
if (to_state) {
s->init_data->init_flags |= SIG_FLAG_INIT_STATE_MATCH;
SCLogDebug("made SID %u stateful because it depends on "
"stateful rules that set flowbit %s",
s->id, varname);
}
}
}
SCFree(array);
return 0;
return fba;
}
SCMutex g_flowbits_dump_write_m = SCMUTEX_INITIALIZER;
@ -809,6 +901,470 @@ static void DetectFlowbitsAnalyzeDump(const DetectEngineCtx *de_ctx,
jb_free(js);
}
static bool PrefilterFlowbitIsPrefilterable(const Signature *s)
{
SCLogDebug("sid:%u: checking", s->id);
for (const SigMatch *sm = s->init_data->smlists[DETECT_SM_LIST_MATCH]; sm != NULL;
sm = sm->next) {
switch (sm->type) {
case DETECT_FLOWBITS: {
const DetectFlowbitsData *fb = (DetectFlowbitsData *)sm->ctx;
if (fb->cmd == DETECT_FLOWBITS_CMD_ISSET) {
SCLogDebug("sid:%u: FLOWBITS ISSET can prefilter", s->id);
return true;
}
break;
}
}
}
SCLogDebug("sid:%u: no flowbit prefilter", s->id);
return false;
}
/** core flowbit data structure: map a flowbit id to the signatures that need inspecting after it is
* found. Part of a rb-tree. */
typedef struct PrefilterFlowbit {
uint32_t id; /**< flowbit id */
uint32_t rule_id_size; /**< size in elements of `rule_id` */
uint32_t rule_id_cnt; /**< usage in elements of `rule_id` */
uint32_t *rule_id; /**< array of signature iid that are part of this prefilter */
RB_ENTRY(PrefilterFlowbit) __attribute__((__packed__)) rb;
} __attribute__((__packed__)) PrefilterFlowbit;
static int PrefilterFlowbitCompare(const PrefilterFlowbit *a, const PrefilterFlowbit *b)
{
if (a->id > b->id)
return 1;
else if (a->id < b->id)
return -1;
else
return 0;
}
/** red-black tree prototype for PFB (Prefilter Flow Bits) */
RB_HEAD(PFB, PrefilterFlowbit);
RB_PROTOTYPE(PFB, PrefilterFlowbit, rb, PrefilterFlowbitCompare);
RB_GENERATE(PFB, PrefilterFlowbit, rb, PrefilterFlowbitCompare);
struct PrefilterEngineFlowbits {
struct PFB fb_tree;
};
static void PrefilterFlowbitFree(void *vctx)
{
struct PrefilterEngineFlowbits *ctx = vctx;
struct PrefilterFlowbit *rec, *safe = NULL;
RB_FOREACH_SAFE (rec, PFB, &ctx->fb_tree, safe) {
PFB_RB_REMOVE(&ctx->fb_tree, rec);
SCFree(rec->rule_id);
SCFree(rec);
}
SCFree(ctx);
}
static void PrefilterFlowbitMatch(DetectEngineThreadCtx *det_ctx, Packet *p, const void *pectx)
{
struct PrefilterEngineFlowbits *ctx = (struct PrefilterEngineFlowbits *)pectx;
SCLogDebug("%" PRIu64 ": ctx %p", p->pcap_cnt, ctx);
if (p->flow == NULL) {
SCReturn;
}
for (GenericVar *gv = p->flow->flowvar; gv != NULL; gv = gv->next) {
if (gv->type != DETECT_FLOWBITS)
continue;
PrefilterFlowbit lookup;
memset(&lookup, 0, sizeof(lookup));
lookup.id = gv->idx;
SCLogDebug("flowbit %u", gv->idx);
PrefilterFlowbit *b = PFB_RB_FIND(&ctx->fb_tree, &lookup);
if (b == NULL) {
SCLogDebug("flowbit %u not in the tree", lookup.id);
} else {
SCLogDebug("flowbit %u found in the tree: %u", lookup.id, b->id);
PrefilterAddSids(&det_ctx->pmq, b->rule_id, b->rule_id_cnt);
#ifdef DEBUG
for (uint32_t x = 0; x < b->rule_id_cnt; x++) {
const Signature *s = det_ctx->de_ctx->sig_array[b->rule_id[x]];
SCLogDebug("flowbit %u -> sig %u", gv->idx, s->id);
}
#endif
}
}
}
static void PrefilterFlowbitPostRuleMatch(
DetectEngineThreadCtx *det_ctx, const void *pectx, Packet *p, Flow *f)
{
struct PrefilterEngineFlowbits *ctx = (struct PrefilterEngineFlowbits *)pectx;
SCLogDebug("%" PRIu64 ": ctx %p", p->pcap_cnt, ctx);
if (p->flow == NULL) {
SCReturn;
}
for (uint32_t i = 0; i < det_ctx->post_rule_work_queue.len; i++) {
const PostRuleMatchWorkQueueItem *w = &det_ctx->post_rule_work_queue.q[i];
if (w->sm_type != DETECT_FLOWBITS)
continue;
PrefilterFlowbit lookup;
memset(&lookup, 0, sizeof(lookup));
lookup.id = w->value;
PrefilterFlowbit *b = PFB_RB_FIND(&ctx->fb_tree, &lookup);
if (b == NULL) {
SCLogDebug("flowbit %u not in the tree", lookup.id);
} else {
SCLogDebug("flowbit %u found in the tree: %u. Adding %u sids", lookup.id, b->id,
b->rule_id_cnt);
PrefilterAddSids(&det_ctx->pmq, b->rule_id, b->rule_id_cnt);
#ifdef DEBUG
// SCLogDebug("b %u", b->rule_id_cnt);
for (uint32_t x = 0; x < b->rule_id_cnt; x++) {
Signature *s = det_ctx->de_ctx->sig_array[b->rule_id[x]];
SCLogDebug("flowbit %u -> sig %u (triggered by %u)", w->value, s->id,
det_ctx->de_ctx->sig_array[w->id]->id);
}
#endif
}
}
}
#define BLOCK_SIZE 8
static int AddBitAndSid(
struct PrefilterEngineFlowbits *ctx, const Signature *s, const uint32_t flowbit_id)
{
PrefilterFlowbit x;
memset(&x, 0, sizeof(x));
x.id = flowbit_id;
PrefilterFlowbit *pfb = PFB_RB_FIND(&ctx->fb_tree, &x);
if (pfb == NULL) {
PrefilterFlowbit *add = SCCalloc(1, sizeof(*add));
if (add == NULL)
return -1;
add->id = flowbit_id;
add->rule_id = SCCalloc(1, BLOCK_SIZE * sizeof(uint32_t));
if (add->rule_id == NULL) {
SCFree(add);
return -1;
}
add->rule_id_size = BLOCK_SIZE;
add->rule_id_cnt = 1;
add->rule_id[0] = s->num;
PrefilterFlowbit *res = PFB_RB_INSERT(&ctx->fb_tree, add);
SCLogDebug("not found, so added (res %p)", res);
if (res != NULL) {
// duplicate, shouldn't be possible after the FIND above
BUG_ON(1);
return -1;
}
} else {
SCLogDebug("found! pfb %p id %u", pfb, pfb->id);
if (pfb->rule_id_cnt < pfb->rule_id_size) {
pfb->rule_id[pfb->rule_id_cnt++] = s->num;
} else {
uint32_t *ptr =
SCRealloc(pfb->rule_id, (pfb->rule_id_size + BLOCK_SIZE) * sizeof(uint32_t));
if (ptr == NULL) {
// memory stays in the tree
return -1;
}
pfb->rule_id = ptr;
pfb->rule_id_size += BLOCK_SIZE;
pfb->rule_id[pfb->rule_id_cnt++] = s->num;
}
}
return 0;
}
static int AddBitsAndSid(const DetectEngineCtx *de_ctx, struct PrefilterEngineFlowbits *ctx,
const DetectFlowbitsData *fb, const Signature *s)
{
if (fb->or_list_size == 0) {
if (AddBitAndSid(ctx, s, fb->idx) < 0) {
return -1;
}
} else {
for (uint8_t i = 0; i < fb->or_list_size; i++) {
SCLogDebug("flowbit OR: bit %u", fb->or_list[i]);
if (AddBitAndSid(ctx, s, fb->or_list[i]) < 0) {
return -1;
}
}
}
return 0;
}
static uint32_t NextMultiple(const uint32_t v, const uint32_t m)
{
return v + (m - v % m);
}
/** \internal
* \brief adds sids for 'isset' prefilter flowbits
* \retval int 1 if we added sid(s), 0 if we didn't, -1 on error */
// TODO skip sids that aren't set by this sgh
// TODO skip sids that doesn't have a isset in the same direction
static int AddIssetSidsForBit(const DetectEngineCtx *de_ctx, const struct FBAnalyzer *fba,
const DetectFlowbitsData *fb, PrefilterFlowbit *add)
{
int added = 0;
for (uint32_t i = 0; i < fba->array[fb->idx].isset_sids_idx; i++) {
const uint32_t sig_iid = fba->array[fb->idx].isset_sids[i];
const Signature *s = de_ctx->sig_array[sig_iid];
SCLogDebug("flowbit: %u => considering sid %u (iid:%u)", fb->idx, s->id, s->num);
/* Skip sids that aren't prefilter. These would just run all the time. */
if (s->init_data->prefilter_sm == NULL ||
s->init_data->prefilter_sm->type != DETECT_FLOWBITS) {
#ifdef DEBUG
const char *name = s->init_data->prefilter_sm
? sigmatch_table[s->init_data->prefilter_sm->type].name
: "none";
SCLogDebug("flowbit: %u => rejected sid %u (iid:%u). No prefilter or prefilter not "
"flowbits (%p, %s, %d)",
fb->idx, s->id, sig_iid, s->init_data->prefilter_sm, name,
s->init_data->prefilter_sm ? s->init_data->prefilter_sm->type : -1);
#endif
continue;
}
/* only add sids that match our bit */
const DetectFlowbitsData *fs_fb =
(const DetectFlowbitsData *)s->init_data->prefilter_sm->ctx;
if (fs_fb->idx != fb->idx) {
SCLogDebug(
"flowbit: %u => rejected sid %u (iid:%u). Sig prefilters on different bit %u",
fb->idx, s->id, sig_iid, fs_fb->idx);
continue;
}
bool dup = false;
for (uint32_t x = 0; x < add->rule_id_cnt; x++) {
if (add->rule_id[x] == sig_iid) {
dup = true;
}
}
if (!dup) {
if (add->rule_id_cnt < add->rule_id_size) {
add->rule_id[add->rule_id_cnt++] = sig_iid;
} else {
uint32_t *ptr = SCRealloc(
add->rule_id, (add->rule_id_size + BLOCK_SIZE) * sizeof(uint32_t));
if (ptr == NULL) {
return -1;
}
add->rule_id = ptr;
add->rule_id_size += BLOCK_SIZE;
add->rule_id[add->rule_id_cnt++] = sig_iid;
}
added = 1;
SCLogDebug("flowbit: %u => accepted sid %u (iid:%u)", fb->idx, s->id, sig_iid);
}
}
return added;
}
/* TODO shouldn't add sids for which Signature::num is < our num. Is this possible after sorting? */
/** \brief For set/toggle flowbits, build "set" post-rule-match engine
*
* For set/toggle flowbits, a special post-rule-match engine is constructed
* to update the running match array during rule matching.
*/
static int AddBitSetToggle(const DetectEngineCtx *de_ctx, struct FBAnalyzer *fba,
struct PrefilterEngineFlowbits *ctx, const DetectFlowbitsData *fb, const Signature *s)
{
PrefilterFlowbit x;
memset(&x, 0, sizeof(x));
x.id = fb->idx;
PrefilterFlowbit *pfb = PFB_RB_FIND(&ctx->fb_tree, &x);
if (pfb == NULL) {
PrefilterFlowbit *add = SCCalloc(1, sizeof(*add));
if (add == NULL)
return -1;
add->id = fb->idx;
add->rule_id_size = NextMultiple(fba->array[fb->idx].isset_sids_idx, BLOCK_SIZE);
add->rule_id = SCCalloc(1, add->rule_id_size * sizeof(uint32_t));
if (add->rule_id == NULL) {
SCFree(add);
return -1;
}
if (!AddIssetSidsForBit(de_ctx, fba, fb, add)) {
SCLogDebug("no sids added");
SCFree(add->rule_id);
SCFree(add);
return 0;
}
PrefilterFlowbit *res = PFB_RB_INSERT(&ctx->fb_tree, add);
SCLogDebug("not found, so added (res %p)", res);
BUG_ON(res != NULL); // TODO if res != NULL we have a duplicate which should be impossible
} else {
SCLogDebug("found! pfb %p id %u", pfb, pfb->id);
int r = AddIssetSidsForBit(de_ctx, fba, fb, pfb);
if (r < 0) {
return -1;
} else if (r == 0) {
SCLogDebug("no sids added");
return 0;
}
}
return 1;
}
/** \brief build flowbit prefilter state(s)
*
* Build "set" and "isset" states.
*
* For each flowbit "isset" in the sgh, we need to check:
* 1. is it supported
* 2. is prefilter enabled
* 3. does it match in the same dir or only opposing dir
*/
static int PrefilterSetupFlowbits(DetectEngineCtx *de_ctx, SigGroupHead *sgh)
{
if (sgh == NULL)
return 0;
SCLogDebug("sgh %p: setting up prefilter", sgh);
struct PrefilterEngineFlowbits *isset_ctx = NULL;
struct PrefilterEngineFlowbits *set_ctx = NULL;
struct FBAnalyzer fb_analysis = DetectFlowbitsAnalyzeForGroup(de_ctx, sgh);
if (fb_analysis.array == NULL)
goto error;
for (uint32_t i = 0; i < sgh->init->sig_cnt; i++) {
Signature *s = sgh->init->match_array[i];
if (s == NULL)
continue;
SCLogDebug("checking sid %u", s->id);
/* first build the 'set' state */
for (SigMatch *sm = s->init_data->smlists[DETECT_SM_LIST_POSTMATCH]; sm != NULL;
sm = sm->next) {
if (sm->type != DETECT_FLOWBITS) {
SCLogDebug("skip non flowbits sm");
continue;
}
DetectFlowbitsData *fb = (DetectFlowbitsData *)sm->ctx;
if (fb->cmd == DETECT_FLOWBITS_CMD_SET) {
SCLogDebug(
"DETECT_SM_LIST_POSTMATCH: sid %u DETECT_FLOWBITS set %u", s->id, fb->idx);
} else if (fb->cmd == DETECT_FLOWBITS_CMD_TOGGLE) {
SCLogDebug("DETECT_SM_LIST_POSTMATCH: sid %u DETECT_FLOWBITS toggle %u", s->id,
fb->idx);
} else {
SCLogDebug("unsupported flowbits setting");
continue;
}
if (fb_analysis.array[fb->idx].isnotset_sids_idx ||
fb_analysis.array[fb->idx].unset_sids_idx) {
SCLogDebug("flowbit %u not supported: unset in use", fb->idx);
continue;
}
if (set_ctx == NULL) {
set_ctx = SCCalloc(1, sizeof(*set_ctx));
if (set_ctx == NULL)
goto error;
}
SCLogDebug("setting up sets/toggles for sid %u", s->id);
if (AddBitSetToggle(de_ctx, &fb_analysis, set_ctx, fb, s) == 1) {
// flag the set/toggle to trigger the post-rule match logic
SCLogDebug("set up sets/toggles for sid %u", s->id);
fb->post_rule_match_prefilter = true;
}
// TODO don't add for sigs that don't have isset in this sgh. Reasoning:
// prefilter post match logic only makes sense in the same dir as otherwise
// the regular 'isset' logic can simply run with the regular prefilters
// before the rule loop
}
/* next, build the 'isset' state */
if (s->init_data->prefilter_sm == NULL ||
s->init_data->prefilter_sm->type != DETECT_FLOWBITS) {
SCLogDebug("no prefilter or prefilter not flowbits");
continue;
}
const DetectFlowbitsData *fb = (DetectFlowbitsData *)s->init_data->prefilter_sm->ctx;
if (fb_analysis.array[fb->idx].isnotset_sids_idx ||
fb_analysis.array[fb->idx].unset_sids_idx) {
SCLogDebug("flowbit %u not supported: toggle or unset in use", fb->idx);
s->init_data->prefilter_sm = NULL;
s->flags &= ~SIG_FLAG_PREFILTER;
continue;
}
SCLogDebug("isset: adding sid %u, flowbit %u", s->id, fb->idx);
if (isset_ctx == NULL) {
isset_ctx = SCCalloc(1, sizeof(*isset_ctx));
if (isset_ctx == NULL)
goto error;
}
if (AddBitsAndSid(de_ctx, isset_ctx, fb, s) < 0) {
goto error;
}
}
/* finally, register the states with their engines */
static const char *g_prefilter_flowbits_isset = "flowbits:isset";
if (isset_ctx != NULL) {
PrefilterAppendEngine(de_ctx, sgh, PrefilterFlowbitMatch, SIG_MASK_REQUIRE_FLOW, isset_ctx,
PrefilterFlowbitFree, g_prefilter_flowbits_isset);
SCLogDebug("isset: added prefilter engine");
if (set_ctx != NULL && !RB_EMPTY(&set_ctx->fb_tree)) {
static const char *g_prefilter_flowbits_set = "flowbits:set";
PrefilterAppendPostRuleEngine(de_ctx, sgh, PrefilterFlowbitPostRuleMatch, set_ctx,
PrefilterFlowbitFree, g_prefilter_flowbits_set);
SCLogDebug("set/toggle: added prefilter engine");
} else {
if (set_ctx) {
PrefilterFlowbitFree(set_ctx);
}
SCLogDebug("set/toggle: NO prefilter engine added");
}
} else if (set_ctx != NULL) {
PrefilterFlowbitFree(set_ctx);
}
FBAnalyzerFree(&fb_analysis);
return 0;
error:
if (set_ctx) {
PrefilterFlowbitFree(set_ctx);
}
if (isset_ctx) {
PrefilterFlowbitFree(isset_ctx);
}
FBAnalyzerFree(&fb_analysis);
return -1;
}
#ifdef UNITTESTS
static int FlowBitsTestParse01(void)

@ -1,4 +1,4 @@
/* Copyright (C) 2007-2010 Open Information Security Foundation
/* Copyright (C) 2007-2025 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
@ -36,6 +36,9 @@ typedef struct DetectFlowbitsData_ {
uint32_t idx;
uint8_t cmd;
uint8_t or_list_size;
/** Flag to trigger post rule match prefilter following a 'set' match. */
bool post_rule_match_prefilter; /**< set/toggle command should trigger post-rule-match
"prefilter" */
uint32_t *or_list;
} DetectFlowbitsData;

@ -98,6 +98,8 @@ static int DetectPrefilterSetup (DetectEngineCtx *de_ctx, Signature *s, const ch
}
s->init_data->prefilter_sm = sm;
SCLogDebug(
"sid %u: prefilter is on \"%s\" (%u)", s->id, sigmatch_table[sm->type].name, sm->type);
SCReturnInt(0);
}

@ -746,6 +746,15 @@ static bool IsOnlyTxInDirection(Flow *f, uint64_t txid, uint8_t dir)
return false;
}
static int SortHelper(const void *a, const void *b)
{
const Signature *sa = *(const Signature **)a;
const Signature *sb = *(const Signature **)b;
if (sa->num == sb->num)
return 0;
return sa->num > sb->num ? 1 : -1;
}
static inline void DetectRulePacketRules(
ThreadVars * const tv,
DetectEngineCtx * const de_ctx,
@ -793,7 +802,7 @@ static inline void DetectRulePacketRules(
}
const uint8_t s_proto_flags = s->proto.flags;
SCLogDebug("inspecting signature id %"PRIu32"", s->id);
SCLogDebug("packet %" PRIu64 ": inspecting signature id %" PRIu32 "", p->pcap_cnt, s->id);
if (s->app_inspect != NULL) {
goto next; // handle sig in DetectRunTx
@ -861,6 +870,51 @@ static inline void DetectRulePacketRules(
}
}
AlertQueueAppend(det_ctx, s, p, txid, alert_flags);
if (det_ctx->post_rule_work_queue.len > 0) {
/* run post match prefilter engines on work queue */
PrefilterPostRuleMatch(det_ctx, scratch->sgh, p, pflow);
if (det_ctx->pmq.rule_id_array_cnt > 0) {
/* undo "prefetch" */
if (next_s)
match_array--;
/* create temporary rule pointer array starting
* at where we are in the current match array */
const Signature *replace[de_ctx->sig_array_len]; // TODO heap?
SCLogDebug("sig_array_len %u det_ctx->pmq.rule_id_array_cnt %u",
de_ctx->sig_array_len, det_ctx->pmq.rule_id_array_cnt);
const Signature **r = replace;
for (uint32_t x = 0; x < match_cnt; x++) {
*r++ = match_array[x];
SCLogDebug("appended %u", match_array[x]->id);
}
/* append the prefilter results, then sort it */
for (uint32_t x = 0; x < det_ctx->pmq.rule_id_array_cnt; x++) {
// TODO what happens if a tx engine is added?
SCLogDebug("adding iid %u", det_ctx->pmq.rule_id_array[x]);
Signature *ts = de_ctx->sig_array[det_ctx->pmq.rule_id_array[x]];
SCLogDebug("adding id %u", ts->id);
if (ts->app_inspect == NULL) {
*r++ = ts;
match_cnt++;
}
}
// TODO if sid 12 is added twice, how do we dedup
qsort(replace, match_cnt, sizeof(Signature *), SortHelper);
/* rewrite match_array to include the new additions */
Signature **m = match_array;
for (uint32_t x = 0; x < match_cnt; x++) {
*m++ = (Signature *)replace[x];
}
/* prefetch next */
next_s = *match_array++;
next_sflags = next_s->flags;
SCLogDebug("%u rules added", det_ctx->pmq.rule_id_array_cnt);
det_ctx->post_rule_work_queue.len = 0;
PMQ_RESET(&det_ctx->pmq);
}
}
next:
DetectVarProcessList(det_ctx, pflow, p);
DetectReplaceFree(det_ctx);
@ -1640,6 +1694,38 @@ static void DetectRunTx(ThreadVars *tv,
}
DetectVarProcessList(det_ctx, p->flow, p);
RULE_PROFILING_END(det_ctx, s, r, p);
if (det_ctx->post_rule_work_queue.len > 0) {
SCLogDebug("%p/%" PRIu64 " post_rule_work_queue len %u", tx.tx_ptr, tx.tx_id,
det_ctx->post_rule_work_queue.len);
/* run post match prefilter engines on work queue */
PrefilterPostRuleMatch(det_ctx, scratch->sgh, p, f);
uint32_t prev_array_idx = array_idx;
for (uint32_t j = 0; j < det_ctx->pmq.rule_id_array_cnt; j++) {
const Signature *ts = de_ctx->sig_array[det_ctx->pmq.rule_id_array[j]];
if (ts->app_inspect != NULL) {
const SigIntId id = ts->num;
det_ctx->tx_candidates[array_idx].s = ts;
det_ctx->tx_candidates[array_idx].id = id;
det_ctx->tx_candidates[array_idx].flags = NULL;
det_ctx->tx_candidates[array_idx].stream_reset = 0;
array_idx++;
SCLogDebug("%p/%" PRIu64 " rule %u (%u) added from 'post match' prefilter",
tx.tx_ptr, tx.tx_id, ts->id, id);
}
}
SCLogDebug("%p/%" PRIu64 " rules added from 'post match' prefilter: %u", tx.tx_ptr,
tx.tx_id, array_idx - prev_array_idx);
if (prev_array_idx != array_idx) {
/* sort, but only part of array we're still going to process */
qsort(det_ctx->tx_candidates + i, array_idx - i, sizeof(RuleMatchCandidateTx),
DetectRunTxSortHelper);
}
det_ctx->post_rule_work_queue.len = 0;
PMQ_RESET(&det_ctx->pmq);
}
}
det_ctx->tx_id = 0;

@ -1103,6 +1103,25 @@ typedef struct RuleMatchCandidateTx {
const Signature *s; /**< ptr to sig */
} RuleMatchCandidateTx;
/** Stores a single u32 for a rule match of the type `sm_type`. Used by
* flowbits prefilter to register DETECT_FLOWBITS,<flowbit id> for post
* match handling. */
typedef struct PostRuleMatchWorkQueueItem {
int sm_type; /**< sigmatch type e.g. DETECT_FLOWBITS */
uint32_t value; /**< value to be interpreted by the sm_type
* implementation. E.g. flowbit id. */
#ifdef DEBUG
SigIntId id;
#endif
} PostRuleMatchWorkQueueItem;
/** Array of PostRuleMatchWorkQueueItem's. */
typedef struct PostRuleMatchWorkQueue {
PostRuleMatchWorkQueueItem *q; /**< array pointer */
uint32_t len; /**< number of array elements in use. */
uint32_t size; /**< allocation size in number of elements. */
} PostRuleMatchWorkQueue;
/**
* Detection engine thread data.
*/
@ -1212,6 +1231,9 @@ typedef struct DetectEngineThreadCtx_ {
uint32_t non_pf_store_cnt;
MpmThreadCtx mtc; /**< thread ctx for the mpm */
/* work queue for post-rule matching affecting prefilter */
PostRuleMatchWorkQueue post_rule_work_queue;
PrefilterRuleStore pmq;
/* string to replace */
@ -1403,6 +1425,8 @@ typedef struct PrefilterEngineList_ {
PrefilterPktFn Prefilter;
PrefilterTxFn PrefilterTx;
PrefilterFrameFn PrefilterFrame;
void (*PrefilterPostRule)(
DetectEngineThreadCtx *det_ctx, const void *pectx, Packet *p, Flow *f);
struct PrefilterEngineList_ *next;
@ -1436,6 +1460,8 @@ typedef struct PrefilterEngine_ {
PrefilterPktFn Prefilter;
PrefilterTxFn PrefilterTx;
PrefilterFrameFn PrefilterFrame;
void (*PrefilterPostRule)(
DetectEngineThreadCtx *det_ctx, const void *pectx, Packet *p, Flow *f);
} cb;
/* global id for this prefilter */
@ -1463,6 +1489,7 @@ typedef struct SigGroupHeadInitData_ {
PrefilterEngineList *payload_engines;
PrefilterEngineList *tx_engines;
PrefilterEngineList *frame_engines;
PrefilterEngineList *post_rule_match_engines;
/** number of sigs in this group */
SigIntId sig_cnt;
@ -1493,6 +1520,7 @@ typedef struct SigGroupHead_ {
PrefilterEngine *payload_engines;
PrefilterEngine *tx_engines;
PrefilterEngine *frame_engines;
PrefilterEngine *post_rule_match_engines; /**< engines to run after rules modified a state */
/* ptr to our init data we only use at... init :) */
SigGroupHeadInitData *init;

Loading…
Cancel
Save