Where to track the rule matches. When using by_src/by_dst the tracking is
done per IP-address. The Host table is used for storage. When using by_rule
it's done globally for the rule.
+Option by_both used to track per IP pair of source and destination. Packets
+going to opposite directions between same addresses tracked as the same pair.
count
^^^^^
#include "host.h"
#include "host-storage.h"
+#include "ippair.h"
+#include "ippair-storage.h"
+
#include "detect-parse.h"
#include "detect-engine-sigorder.h"
#include "util-var-name.h"
#include "tm-threads.h"
-static int threshold_id = -1; /**< host storage id for thresholds */
+static int host_threshold_id = -1; /**< host storage id for thresholds */
+static int ippair_threshold_id = -1; /**< ip pair storage id for thresholds */
int ThresholdHostStorageId(void)
{
- return threshold_id;
+ return host_threshold_id;
}
void ThresholdInit(void)
{
- threshold_id = HostStorageRegister("threshold", sizeof(void *), NULL, ThresholdListFree);
- if (threshold_id == -1) {
+ host_threshold_id = HostStorageRegister("threshold", sizeof(void *), NULL, ThresholdListFree);
+ if (host_threshold_id == -1) {
SCLogError(SC_ERR_HOST_INIT, "Can't initiate host storage for thresholding");
exit(EXIT_FAILURE);
}
+ ippair_threshold_id = IPPairStorageRegister("threshold", sizeof(void *), NULL, ThresholdListFree);
+ if (ippair_threshold_id == -1) {
+ SCLogError(SC_ERR_HOST_INIT, "Can't initiate IP pair storage for thresholding");
+ exit(EXIT_FAILURE);
+ }
}
int ThresholdHostHasThreshold(Host *host)
{
- return HostGetStorageById(host, threshold_id) ? 1 : 0;
+ return HostGetStorageById(host, host_threshold_id) ? 1 : 0;
+}
+
+int ThresholdIPPairHasThreshold(IPPair *pair)
+{
+ return IPPairGetStorageById(pair, ippair_threshold_id) ? 1 : 0;
}
/**
/**
* \brief Remove timeout threshold hash elements
*
- * \param de_ctx Dectection Context
+ * \param head Current head element of storage
+ * \param tv Current time
+ *
+ * \retval DetectThresholdEntry Return new head element or NULL if all expired
*
*/
-int ThresholdTimeoutCheck(Host *host, struct timeval *tv)
+static DetectThresholdEntry* ThresholdTimeoutCheck(DetectThresholdEntry *head, struct timeval *tv)
{
- DetectThresholdEntry *tde = NULL;
- DetectThresholdEntry *tmp = NULL;
+ DetectThresholdEntry *tmp = head;
DetectThresholdEntry *prev = NULL;
- int retval = 1;
-
- tmp = HostGetStorageById(host, threshold_id);
- if (tmp == NULL)
- return 1;
+ DetectThresholdEntry *new_head = head;
- prev = NULL;
while (tmp != NULL) {
if ((tv->tv_sec - tmp->tv_sec1) <= tmp->seconds) {
prev = tmp;
tmp = tmp->next;
- retval = 0;
continue;
}
/* timed out */
+ DetectThresholdEntry *tde = tmp;
if (prev != NULL) {
prev->next = tmp->next;
+ }
+ else {
+ new_head = tmp->next;
+ }
+ tmp = tde->next;
+ SCFree(tde);
+ }
- tde = tmp;
- tmp = tde->next;
-
- SCFree(tde);
- } else {
- HostSetStorageById(host, threshold_id, tmp->next);
- tde = tmp;
- tmp = tde->next;
+ return new_head;
+}
- SCFree(tde);
- }
+int ThresholdHostTimeoutCheck(Host *host, struct timeval *tv)
+{
+ DetectThresholdEntry* head = HostGetStorageById(host, host_threshold_id);
+ DetectThresholdEntry* new_head = ThresholdTimeoutCheck(head, tv);
+ if (new_head != head) {
+ HostSetStorageById(host, host_threshold_id, new_head);
}
+ return new_head == NULL;
+}
- return retval;
+
+int ThresholdIPPairTimeoutCheck(IPPair *pair, struct timeval *tv)
+{
+ DetectThresholdEntry* head = IPPairGetStorageById(pair, ippair_threshold_id);
+ DetectThresholdEntry* new_head = ThresholdTimeoutCheck(head, tv);
+ if (new_head != head) {
+ IPPairSetStorageById(pair, ippair_threshold_id, new_head);
+ }
+ return new_head == NULL;
}
static inline DetectThresholdEntry *
{
DetectThresholdEntry *e;
- for (e = HostGetStorageById(h, threshold_id); e != NULL; e = e->next) {
+ for (e = HostGetStorageById(h, host_threshold_id); e != NULL; e = e->next) {
+ if (e->sid == sid && e->gid == gid)
+ break;
+ }
+
+ return e;
+}
+
+static DetectThresholdEntry *ThresholdIPPairLookupEntry(IPPair *pair, uint32_t sid, uint32_t gid)
+{
+ DetectThresholdEntry *e;
+
+ for (e = IPPairGetStorageById(pair, ippair_threshold_id); e != NULL; e = e->next) {
if (e->sid == sid && e->gid == gid)
break;
}
}
}
+/**
+* \brief Check if the entry reached threshold count limit
+*
+* \param lookup_tsh Current threshold entry
+* \param td Threshold settings
+* \param packet_time used to compare against previous detection and to set timeouts
+*
+* \retval int 1 if threshold reached for this entry
+*
+*/
+static int IsThresholdReached(DetectThresholdEntry* lookup_tsh, const DetectThresholdData *td, uint32_t packet_time)
+{
+ int ret = 0;
+
+ /* Check if we have a timeout enabled, if so,
+ * we still matching (and enabling the new_action) */
+ if (lookup_tsh->tv_timeout != 0) {
+ if ((packet_time - lookup_tsh->tv_timeout) > td->timeout) {
+ /* Ok, we are done, timeout reached */
+ lookup_tsh->tv_timeout = 0;
+ }
+ else {
+ /* Already matching */
+ ret = 1;
+ } /* else - if ((packet_time - lookup_tsh->tv_timeout) > td->timeout) */
+
+ }
+ else {
+ /* Update the matching state with the timeout interval */
+ if ((packet_time - lookup_tsh->tv_sec1) < td->seconds) {
+ lookup_tsh->current_count++;
+ if (lookup_tsh->current_count > td->count) {
+ /* Then we must enable the new action by setting a
+ * timeout */
+ lookup_tsh->tv_timeout = packet_time;
+ ret = 1;
+ }
+ }
+ else {
+ lookup_tsh->tv_sec1 = packet_time;
+ lookup_tsh->current_count = 1;
+ }
+ } /* else - if (lookup_tsh->tv_timeout != 0) */
+
+ return ret;
+}
+
+static void AddEntryToHostStorage(Host *h, DetectThresholdEntry *e, uint32_t packet_time)
+{
+ if (h && e) {
+ e->current_count = 1;
+ e->tv_sec1 = packet_time;
+ e->tv_timeout = 0;
+ e->next = HostGetStorageById(h, host_threshold_id);
+ HostSetStorageById(h, host_threshold_id, e);
+ }
+}
+
+static void AddEntryToIPPairStorage(IPPair *pair, DetectThresholdEntry *e, uint32_t packet_time)
+{
+ if (pair && e) {
+ e->current_count = 1;
+ e->tv_sec1 = packet_time;
+ e->tv_timeout = 0;
+ e->next = IPPairGetStorageById(pair, ippair_threshold_id);
+ IPPairSetStorageById(pair, ippair_threshold_id, e);
+ }
+}
+
+static int ThresholdHandlePacketIPPair(IPPair *pair, Packet *p, const DetectThresholdData *td,
+ uint32_t sid, uint32_t gid, PacketAlert *pa)
+{
+ int ret = 0;
+
+ DetectThresholdEntry *lookup_tsh = ThresholdIPPairLookupEntry(pair, sid, gid);
+ SCLogDebug("ippair lookup_tsh %p sid %u gid %u", lookup_tsh, sid, gid);
+
+ switch (td->type) {
+ case TYPE_RATE:
+ {
+ SCLogDebug("rate_filter");
+ ret = 1;
+ if (lookup_tsh && IsThresholdReached(lookup_tsh, td, p->ts.tv_sec)) {
+ RateFilterSetAction(p, pa, td->new_action);
+ } else if (!lookup_tsh) {
+ DetectThresholdEntry *e = DetectThresholdEntryAlloc(td, p, sid, gid);
+ AddEntryToIPPairStorage(pair, e, p->ts.tv_sec);
+ }
+ break;
+ }
+ default:
+ {
+ SCLogError(SC_ERR_INVALID_VALUE, "type %d is not supported", td->type);
+ break;
+ }
+ }
+
+ return ret;
+}
+
/**
* \retval 2 silent match (no alert but apply actions)
* \retval 1 normal match
ret = 1;
- e->next = HostGetStorageById(h, threshold_id);
- HostSetStorageById(h, threshold_id, e);
+ e->next = HostGetStorageById(h, host_threshold_id);
+ HostSetStorageById(h, host_threshold_id, e);
}
break;
}
e->current_count = 1;
e->tv_sec1 = p->ts.tv_sec;
- e->next = HostGetStorageById(h, threshold_id);
- HostSetStorageById(h, threshold_id, e);
+ e->next = HostGetStorageById(h, host_threshold_id);
+ HostSetStorageById(h, host_threshold_id, e);
}
}
break;
e->current_count = 1;
e->tv_sec1 = p->ts.tv_sec;
- e->next = HostGetStorageById(h, threshold_id);
- HostSetStorageById(h, threshold_id, e);
+ e->next = HostGetStorageById(h, host_threshold_id);
+ HostSetStorageById(h, host_threshold_id, e);
/* for the first match we return 1 to
* indicate we should alert */
e->tv_sec1 = p->ts.tv_sec;
e->tv_usec1 = p->ts.tv_usec;
- e->next = HostGetStorageById(h, threshold_id);
- HostSetStorageById(h, threshold_id, e);
+ e->next = HostGetStorageById(h, host_threshold_id);
+ HostSetStorageById(h, host_threshold_id, e);
}
break;
}
case TYPE_RATE:
{
SCLogDebug("rate_filter");
-
ret = 1;
-
- if (lookup_tsh != NULL) {
- /* Check if we have a timeout enabled, if so,
- * we still matching (and enabling the new_action) */
- if (lookup_tsh->tv_timeout != 0) {
- if ((p->ts.tv_sec - lookup_tsh->tv_timeout) > td->timeout) {
- /* Ok, we are done, timeout reached */
- lookup_tsh->tv_timeout = 0;
- } else {
- /* Already matching */
- /* Take the action to perform */
- RateFilterSetAction(p, pa, td->new_action);
- ret = 1;
- } /* else - if ((p->ts.tv_sec - lookup_tsh->tv_timeout) > td->timeout) */
-
- } else {
- /* Update the matching state with the timeout interval */
- if ( (p->ts.tv_sec - lookup_tsh->tv_sec1) < td->seconds) {
- lookup_tsh->current_count++;
- if (lookup_tsh->current_count > td->count) {
- /* Then we must enable the new action by setting a
- * timeout */
- lookup_tsh->tv_timeout = p->ts.tv_sec;
- /* Take the action to perform */
- RateFilterSetAction(p, pa, td->new_action);
- ret = 1;
- }
- } else {
- lookup_tsh->tv_sec1 = p->ts.tv_sec;
- lookup_tsh->current_count = 1;
- }
- } /* else - if (lookup_tsh->tv_timeout != 0) */
- } else {
- if (td->count == 1) {
- ret = 1;
- }
-
+ if (lookup_tsh && IsThresholdReached(lookup_tsh, td, p->ts.tv_sec)) {
+ RateFilterSetAction(p, pa, td->new_action);
+ } else if (!lookup_tsh) {
DetectThresholdEntry *e = DetectThresholdEntryAlloc(td, p, sid, gid);
- if (e == NULL) {
- break;
- }
-
- e->current_count = 1;
- e->tv_sec1 = p->ts.tv_sec;
- e->tv_timeout = 0;
-
- e->next = HostGetStorageById(h, threshold_id);
- HostSetStorageById(h, threshold_id, e);
+ AddEntryToHostStorage(h, e, p->ts.tv_sec);
}
break;
}
ret = ThresholdHandlePacketHost(dst,p,td,s->id,s->gid,pa);
HostRelease(dst);
}
+ } else if (td->track == TRACK_BOTH) {
+ IPPair *pair = IPPairGetIPPairFromHash(&p->src, &p->dst);
+ if (pair) {
+ ret = ThresholdHandlePacketIPPair(pair, p, td, s->id, s->gid, pa);
+ IPPairRelease(pair);
+ }
} else if (td->track == TRACK_RULE) {
SCMutexLock(&de_ctx->ths_ctx.threshold_table_lock);
ret = ThresholdHandlePacketRule(de_ctx,p,td,s,pa);
#include "detect.h"
#include "host.h"
+#include "ippair.h"
void ThresholdInit(void);
int ThresholdHostStorageId(void);
int ThresholdHostHasThreshold(Host *);
+int ThresholdIPPairHasThreshold(IPPair *pair);
+
const DetectThresholdData *SigGetThresholdTypeIter(const Signature *,
Packet *, const SigMatchData **, int list);
int PacketAlertThreshold(DetectEngineCtx *, DetectEngineThreadCtx *,
void ThresholdHashInit(DetectEngineCtx *);
void ThresholdContextDestroy(DetectEngineCtx *);
-int ThresholdTimeoutCheck(Host *, struct timeval *);
+int ThresholdHostTimeoutCheck(Host *, struct timeval *);
+int ThresholdIPPairTimeoutCheck(IPPair *, struct timeval *);
void ThresholdListFree(void *ptr);
#endif /* __DETECT_ENGINE_THRESHOLD_H__ */
#define TRACK_SRC 2
#define TRACK_RULE 3
#define TRACK_EITHER 4 /**< either src or dst: only used by suppress */
+#define TRACK_BOTH 5 /* used by rate_filter to match detections by both src and dst addresses */
/* Get the new action to take */
#define TH_ACTION_ALERT 0x01
if (TagHostHasTag(h) && TagTimeoutCheck(h, ts) == 0) {
tags = 1;
}
- if (ThresholdHostHasThreshold(h) && ThresholdTimeoutCheck(h, ts) == 0) {
+ if (ThresholdHostHasThreshold(h) && ThresholdHostTimeoutCheck(h, ts) == 0) {
thresholds = 1;
}
if (HostHasHostBits(h) && HostBitsTimedoutCheck(h, ts) == 0) {
#include "ippair.h"
#include "ippair-bit.h"
#include "ippair-timeout.h"
+#include "detect-engine-threshold.h"
uint32_t IPPairGetSpareCount(void)
{
static int IPPairTimedOut(IPPair *h, struct timeval *ts)
{
int vars = 0;
+ int thresholds = 0;
/** never prune a ippair that is used by a packet
* we are currently processing in one of the threads */
vars = 1;
}
- if (vars) {
+ if (ThresholdIPPairHasThreshold(h) && ThresholdIPPairTimeoutCheck(h, ts) == 0) {
+ thresholds = 1;
+ }
+
+ if (vars || thresholds) {
return 0;
}
#include "suricata-common.h"
#include "host.h"
+#include "ippair.h"
#include "detect.h"
#include "detect-engine.h"
#define DETECT_THRESHOLD_REGEX "^,\\s*type\\s*(limit|both|threshold)\\s*,\\s*track\\s*(by_dst|by_src)\\s*,\\s*count\\s*(\\d+)\\s*,\\s*seconds\\s*(\\d+)\\s*$"
/* TODO: "apply_to" */
-#define DETECT_RATE_REGEX "^,\\s*track\\s*(by_dst|by_src|by_rule)\\s*,\\s*count\\s*(\\d+)\\s*,\\s*seconds\\s*(\\d+)\\s*,\\s*new_action\\s*(alert|drop|pass|log|sdrop|reject)\\s*,\\s*timeout\\s*(\\d+)\\s*$"
+#define DETECT_RATE_REGEX "^,\\s*track\\s*(by_dst|by_src|by_both|by_rule)\\s*,\\s*count\\s*(\\d+)\\s*,\\s*seconds\\s*(\\d+)\\s*,\\s*new_action\\s*(alert|drop|pass|log|sdrop|reject)\\s*,\\s*timeout\\s*(\\d+)\\s*$"
/*
* suppress has two form:
parsed_track = TRACK_DST;
else if (strcasecmp(th_track,"by_src") == 0)
parsed_track = TRACK_SRC;
+ else if (strcasecmp(th_track, "by_both") == 0) {
+ parsed_track = TRACK_BOTH;
+ }
else if (strcasecmp(th_track,"by_rule") == 0)
parsed_track = TRACK_RULE;
else {
const char *buffer =
"rate_filter gen_id 1, sig_id 10, track by_src, count 1, seconds 60, new_action drop, timeout 10\n"
"rate_filter gen_id 1, sig_id 100, track by_dst, count 10, seconds 60, new_action pass, timeout 5\n"
- "rate_filter gen_id 1, sig_id 1000, track by_rule, count 100, seconds 60, new_action alert, timeout 30\n";
+ "rate_filter gen_id 1, sig_id 1000, track by_rule, count 100, seconds 60, new_action alert, timeout 30\n"
+ "rate_filter gen_id 1, sig_id 10000, track by_both, count 1000, seconds 60, new_action reject, timeout 21\n";
fd = SCFmemopen((void *)buffer, strlen(buffer), "r");
if (fd == NULL)
const char *buffer =
"rate_filter \\\ngen_id 1, sig_id 10, track by_src, count 1, seconds 60\\\n, new_action drop, timeout 10\n"
"rate_filter gen_id 1, \\\nsig_id 100, track by_dst, \\\ncount 10, seconds 60, new_action pass, timeout 5\n"
- "rate_filter gen_id 1, sig_id 1000, \\\ntrack by_rule, count 100, seconds 60, new_action alert, timeout 30\n";
+ "rate_filter gen_id 1, sig_id 1000, \\\ntrack by_rule, count 100, seconds 60, new_action alert, timeout 30\n"
+ "rate_filter gen_id 1, sig_id 10000, track by_both, count 1000, \\\nseconds 60, new_action reject, timeout 21\n";
fd = SCFmemopen((void *)buffer, strlen(buffer), "r");
if (fd == NULL)
PASS;
}
+/**
+* \brief Creates a dummy rate_filter file, for testing rate filtering by_both source and destination
+*
+* \retval fd Pointer to file descriptor.
+*/
+static FILE *SCThresholdConfGenerateValidDummyFD22(void)
+{
+ FILE *fd = NULL;
+ const char *buffer =
+ "rate_filter gen_id 1, sig_id 10, track by_both, count 2, seconds 5, new_action drop, timeout 6\n";
+
+ fd = SCFmemopen((void *)buffer, strlen(buffer), "r");
+ if (fd == NULL)
+ SCLogDebug("Error with SCFmemopen() called by Threshold Config test code");
+
+ return fd;
+}
+
+/**
+* \test Check if the rate_filter rules work with track by_both
+*
+* \retval 1 on succces
+* \retval 0 on failure
+*/
+static int SCThresholdConfTest22(void)
+{
+ ThreadVars th_v;
+ memset(&th_v, 0, sizeof(th_v));
+
+ IPPairInitConfig(IPPAIR_QUIET);
+
+ struct timeval ts;
+ memset(&ts, 0, sizeof(struct timeval));
+ TimeGet(&ts);
+
+ /* This packet will cause rate_filter */
+ Packet *p1 = UTHBuildPacketSrcDst((uint8_t*)"lalala", 6, IPPROTO_TCP, "172.26.0.1", "172.26.0.10");
+ FAIL_IF_NULL(p1);
+
+ /* Should not be filtered for different destination */
+ Packet *p2 = UTHBuildPacketSrcDst((uint8_t*)"lalala", 6, IPPROTO_TCP, "172.26.0.1", "172.26.0.2");
+ FAIL_IF_NULL(p2);
+
+ /* Should not be filtered when both src and dst the same */
+ Packet *p3 = UTHBuildPacketSrcDst((uint8_t*)"lalala", 6, IPPROTO_TCP, "172.26.0.1", "172.26.0.1");
+ FAIL_IF_NULL(p3);
+
+ DetectEngineThreadCtx *det_ctx = NULL;
+
+ DetectEngineCtx *de_ctx = DetectEngineCtxInit();
+ FAIL_IF_NULL(de_ctx);
+ de_ctx->flags |= DE_QUIET;
+
+ Signature *sig = DetectEngineAppendSig(de_ctx,
+ "alert tcp any any -> any any (msg:\"ratefilter by_both test\"; gid:1; sid:10;)");
+ FAIL_IF_NULL(sig);
+
+ FAIL_IF_NOT_NULL(g_ut_threshold_fp);
+ g_ut_threshold_fp = SCThresholdConfGenerateValidDummyFD22();
+ FAIL_IF_NULL(g_ut_threshold_fp);
+ SCThresholdConfInitContext(de_ctx);
+
+ SigGroupBuild(de_ctx);
+ DetectEngineThreadCtxInit(&th_v, (void *)de_ctx, (void *)&det_ctx);
+
+ TimeGet(&p1->ts);
+ p2->ts = p3->ts = p1->ts;
+
+ /* All should be alerted, none dropped */
+ SigMatchSignatures(&th_v, de_ctx, det_ctx, p1);
+ FAIL_IF(PACKET_TEST_ACTION(p1, ACTION_DROP));
+ FAIL_IF(PacketAlertCheck(p1, 10) != 1);
+
+ SigMatchSignatures(&th_v, de_ctx, det_ctx, p2);
+ FAIL_IF(PACKET_TEST_ACTION(p2, ACTION_DROP));
+ FAIL_IF(PacketAlertCheck(p2, 10) != 1);
+
+ SigMatchSignatures(&th_v, de_ctx, det_ctx, p3);
+ FAIL_IF(PACKET_TEST_ACTION(p3, ACTION_DROP));
+ FAIL_IF(PacketAlertCheck(p3, 10) != 1);
+
+ p1->action = p2->action = p3->action = 0;
+
+ TimeSetIncrementTime(2);
+ TimeGet(&p1->ts);
+ p2->ts = p3->ts = p1->ts;
+
+ /* p1 still shouldn't be dropped after 2nd alert */
+ SigMatchSignatures(&th_v, de_ctx, det_ctx, p1);
+ FAIL_IF(PACKET_TEST_ACTION(p1, ACTION_DROP));
+ FAIL_IF(PacketAlertCheck(p1, 10) != 1);
+
+ p1->action = 0;
+
+ TimeSetIncrementTime(2);
+ TimeGet(&p1->ts);
+ p2->ts = p3->ts = p1->ts;
+
+ /* All should be alerted, only p1 must be dropped due to rate_filter*/
+ SigMatchSignatures(&th_v, de_ctx, det_ctx, p1);
+ FAIL_IF_NOT(PACKET_TEST_ACTION(p1, ACTION_DROP));
+ FAIL_IF(PacketAlertCheck(p1, 10) != 1);
+
+ SigMatchSignatures(&th_v, de_ctx, det_ctx, p2);
+ FAIL_IF(PACKET_TEST_ACTION(p2, ACTION_DROP));
+ FAIL_IF(PacketAlertCheck(p2, 10) != 1);
+
+ SigMatchSignatures(&th_v, de_ctx, det_ctx, p3);
+ FAIL_IF(PACKET_TEST_ACTION(p3, ACTION_DROP));
+ FAIL_IF(PacketAlertCheck(p3, 10) != 1);
+
+ p1->action = p2->action = p3->action = 0;
+
+ TimeSetIncrementTime(7);
+ TimeGet(&p1->ts);
+ p2->ts = p3->ts = p1->ts;
+
+ /* All should be alerted, none dropped (because timeout expired) */
+ SigMatchSignatures(&th_v, de_ctx, det_ctx, p1);
+ FAIL_IF(PACKET_TEST_ACTION(p1, ACTION_DROP));
+ FAIL_IF(PacketAlertCheck(p1, 10) != 1);
+
+ SigMatchSignatures(&th_v, de_ctx, det_ctx, p2);
+ FAIL_IF(PACKET_TEST_ACTION(p2, ACTION_DROP));
+ FAIL_IF(PacketAlertCheck(p2, 10) != 1);
+
+ SigMatchSignatures(&th_v, de_ctx, det_ctx, p3);
+ FAIL_IF(PACKET_TEST_ACTION(p3, ACTION_DROP));
+ FAIL_IF(PacketAlertCheck(p3, 10) != 1);
+
+ UTHFreePacket(p3);
+ UTHFreePacket(p2);
+ UTHFreePacket(p1);
+
+ DetectEngineThreadCtxDeinit(&th_v, (void *)det_ctx);
+ DetectEngineCtxFree(de_ctx);
+ IPPairShutdown();
+ PASS;
+}
+
+/**
+* \brief Creates a dummy rate_filter file, for testing rate filtering by_both source and destination
+*
+* \retval fd Pointer to file descriptor.
+*/
+static FILE *SCThresholdConfGenerateValidDummyFD23(void)
+{
+ FILE *fd = NULL;
+ const char *buffer =
+ "rate_filter gen_id 1, sig_id 10, track by_both, count 1, seconds 5, new_action drop, timeout 6\n";
+
+ fd = SCFmemopen((void *)buffer, strlen(buffer), "r");
+ if (fd == NULL)
+ SCLogDebug("Error with SCFmemopen() called by Threshold Config test code");
+
+ return fd;
+}
+
+/**
+* \test Check if the rate_filter by_both work when similar packets
+* going in opposite direction
+*
+* \retval 1 on succces
+* \retval 0 on failure
+*/
+static int SCThresholdConfTest23(void)
+{
+ ThreadVars th_v;
+ memset(&th_v, 0, sizeof(th_v));
+
+ IPPairInitConfig(IPPAIR_QUIET);
+
+ struct timeval ts;
+ memset(&ts, 0, sizeof(struct timeval));
+ TimeGet(&ts);
+
+ /* Create two packets between same addresses in opposite direction */
+ Packet *p1 = UTHBuildPacketSrcDst((uint8_t*)"lalala", 6, IPPROTO_TCP, "172.26.0.1", "172.26.0.10");
+ FAIL_IF_NULL(p1);
+
+ Packet *p2 = UTHBuildPacketSrcDst((uint8_t*)"lalala", 6, IPPROTO_TCP, "172.26.0.10", "172.26.0.1");
+ FAIL_IF_NULL(p2);
+
+ DetectEngineThreadCtx *det_ctx = NULL;
+
+ DetectEngineCtx *de_ctx = DetectEngineCtxInit();
+ FAIL_IF_NULL(de_ctx);
+ de_ctx->flags |= DE_QUIET;
+
+ Signature *sig = DetectEngineAppendSig(de_ctx,
+ "alert tcp any any -> any any (msg:\"ratefilter by_both test\"; gid:1; sid:10;)");
+ FAIL_IF_NULL(sig);
+
+ FAIL_IF_NOT_NULL(g_ut_threshold_fp);
+ g_ut_threshold_fp = SCThresholdConfGenerateValidDummyFD23();
+ FAIL_IF_NULL(g_ut_threshold_fp);
+ SCThresholdConfInitContext(de_ctx);
+
+ SigGroupBuild(de_ctx);
+ DetectEngineThreadCtxInit(&th_v, (void *)de_ctx, (void *)&det_ctx);
+
+ TimeGet(&p1->ts);
+ SigMatchSignatures(&th_v, de_ctx, det_ctx, p1);
+ /* First packet should be alerted, not dropped */
+ FAIL_IF(PACKET_TEST_ACTION(p1, ACTION_DROP));
+ FAIL_IF(PacketAlertCheck(p1, 10) != 1);
+
+ TimeSetIncrementTime(2);
+ TimeGet(&p2->ts);
+ SigMatchSignatures(&th_v, de_ctx, det_ctx, p2);
+
+ /* Second packet should be dropped because it considered as "the same pair"
+ and rate_filter count reached*/
+ FAIL_IF_NOT(PACKET_TEST_ACTION(p2, ACTION_DROP));
+ FAIL_IF(PacketAlertCheck(p2, 10) != 1);
+
+ UTHFreePacket(p2);
+ UTHFreePacket(p1);
+
+ DetectEngineThreadCtxDeinit(&th_v, (void *)det_ctx);
+ DetectEngineCtxFree(de_ctx);
+ IPPairShutdown();
+ PASS;
+}
#endif /* UNITTESTS */
/**
SCThresholdConfTest20);
UtRegisterTest("SCThresholdConfTest21 - suppress parsing",
SCThresholdConfTest21);
+ UtRegisterTest("SCThresholdConfTest22 - rate_filter by_both",
+ SCThresholdConfTest22);
+ UtRegisterTest("SCThresholdConfTest23 - rate_filter by_both opposite",
+ SCThresholdConfTest23);
+
#endif /* UNITTESTS */
}