detection_options.h
detection_util.h
ips_context.h
+ regex_offload.h
rule_option_types.h
rules.h
signature.h
pattern_match_data.h
pcrm.cc
pcrm.h
+ regex_offload.cc
rtn_checks.cc
rtn_checks.h
service_map.cc
detection_options.h \
detection_util.h \
ips_context.h \
+regex_offload.h \
rule_option_types.h \
rules.h \
signature.h \
pattern_match_data.h \
pcrm.cc \
pcrm.h \
+regex_offload.cc \
rtn_checks.cc \
rtn_checks.h \
service_map.cc \
return c;
}
+bool ContextSwitcher::on_hold(Flow* f)
+{
+ for ( unsigned i = 0; i < hold.capacity(); ++i )
+ {
+ IpsContext* c = hold[i];
+ if ( c and c->packet and c->packet->flow == f )
+ return true;
+ }
+ return false;
+}
+
//--------------------------------------------------------------------------
// unit tests
//--------------------------------------------------------------------------
bool can_hold() const
{ return idle_count() > 5; } // FIXIT-H define appropriate const
+ bool on_hold(class Flow*);
+
private:
std::vector<IpsContext*> idle;
std::vector<IpsContext*> busy;
#include "fp_config.h"
#include "fp_detect.h"
#include "ips_context.h"
+#include "regex_offload.h"
Trace TRACE_NAME(detection);
-static THREAD_LOCAL Ring<unsigned>* offload_ids = nullptr;
+static THREAD_LOCAL RegexOffload* offloader = nullptr;
+
+//--------------------------------------------------------------------------
+// basic de
+//--------------------------------------------------------------------------
void DetectionEngine::thread_init()
-{ offload_ids = new Ring<unsigned>(32); } // FIXIT-H get size
+{ offloader = new RegexOffload(32); } // FIXIT-H get size (threads < contexts)
void DetectionEngine::thread_term()
-{ delete offload_ids; }
+{ delete offloader; }
DetectionEngine::DetectionEngine()
{ context = Snort::get_switcher()->interrupt(); }
bool DetectionEngine::offloaded(Packet* p)
{ return p->flow and p->flow->is_offloaded(); }
+//--------------------------------------------------------------------------
+// offload / onload
+//--------------------------------------------------------------------------
+
void DetectionEngine::idle()
{
- if ( !offload_ids )
+ if ( !offloader )
return;
- while ( !offload_ids->empty() )
+ while ( offloader->count() )
{
- const struct timespec blip = { 0, 1 };
trace_logf(detection, "%lu de::sleep\n", pc.total_from_daq);
+ const struct timespec blip = { 0, 1 };
nanosleep(&blip, nullptr);
onload();
}
- trace_logf(detection, "%lu de::idle (r=%d)\n", pc.total_from_daq, offload_ids->count());
+ trace_logf(detection, "%lu de::idle (r=%d)\n", pc.total_from_daq, offloader->count());
+
+ offloader->stop();
}
void DetectionEngine::onload(Flow* flow)
nanosleep(&blip, nullptr);
onload();
}
+ assert(!Snort::get_switcher()->on_hold(flow));
+ assert(!offloader->on_hold(flow));
}
void DetectionEngine::onload()
{
- ContextSwitcher* sw = Snort::get_switcher();
- unsigned* id = offload_ids->read();
- IpsContext* c = sw->get_context(*id);
+ unsigned id;
- assert(c->offload);
-
- if ( !c->onload )
+ if ( !offloader->get(id) )
return;
+ ContextSwitcher* sw = Snort::get_switcher();
+ IpsContext* c = sw->get_context(id);
+ assert(c);
+
trace_logf(detection, "%lu de::onload %u (r=%d)\n",
- pc.total_from_daq, *id, offload_ids->count());
+ pc.total_from_daq, id, offloader->count());
Packet* p = c->packet;
p->flow->clear_offloaded();
- c->offload->join();
- delete c->offload;
- c->offload = nullptr;
-
- offload_ids->pop();
- sw->resume(*id);
+ sw->resume(id);
fp_onload(p);
InspectorManager::clear(p);
- log_events(p);
- reset(p);
clear_packet(p);
sw->complete();
return false;
}
assert(p == p->context->packet);
- onload(p->flow); // FIXIT-H ensures correct sequencing, suboptimal
-
- p->flow->set_offloaded();
- pc.offloads++;
+ onload(p->flow); // FIXIT-L just assert !offloaded?
assert(p->context == sw->get_context());
-
unsigned id = sw->suspend();
- offload_ids->put(id);
trace_logf(detection, "%lu de::offload %u (r=%d)\n",
- pc.total_from_daq, id, offload_ids->count());
+ pc.total_from_daq, id, offloader->count());
+
+ p->flow->set_offloaded();
+ p->context->conf = snort_conf;
- p->context->onload = false;
- p->context->offload = new std::thread(fp_offload, p, snort_conf);
+ offloader->put(id, p);
+ pc.offloads++;
return true;
}
+//--------------------------------------------------------------------------
+// detection / inspection
+//--------------------------------------------------------------------------
+
bool DetectionEngine::detect(Packet* p)
{
assert(p);
Stream::check_flow_block_pending(p);
}
+//--------------------------------------------------------------------------
+// events
+//--------------------------------------------------------------------------
+
// Return 0 if no OTN since -1 return indicates queue limit reached.
// See fpFinalSelectEvent()
int DetectionEngine::queue_event(const OptTreeNode* otn)
private:
static struct SF_EVENTQ* get_event_queue();
+ static void offload_thread(IpsContext*);
static void onload();
static void clear_packet(Packet*);
fpFinalSelectEvent(c->otnx, p);
}
-void fp_offload(Packet* p, SnortConfig* sc)
+void fp_offload(Packet* p)
{
- snort_conf = sc; // FIXIT-H reload issue
- MpseStash* stash = p->context->stash;
+ IpsContext* c = p->context;
+ MpseStash* stash = c->stash;
stash->init();
stash->disable_process();
fpEvalPacket(p);
- p->context->onload = true;
}
void fp_onload(Packet* p)
void fp_clear_context(IpsContext&);
void fp_local(Packet*);
-void fp_offload(Packet*, struct SnortConfig*);
+void fp_offload(Packet*);
void fp_onload(Packet*);
#endif
packet->context = this;
fp_set_context(*this);
- offload = nullptr;
- onload = false;
active_rules = CONTENT;
}
IpsContext::~IpsContext()
{
for ( auto* p : data )
+ {
if ( p )
delete p;
-
- assert(!offload);
+ }
sfeventq_free(equeue);
fp_clear_context(*this);
SECTION("one context")
{
+#if 0
auto id = IpsContextData::get_ips_id();
auto* d1 = new ContextData(10);
auto ctxt = IpsContext(id+1);
ctxt.set_context_data(id, d1);
CHECK(ctxt.get_context_data(id) == d1);
+#endif
}
CHECK(ContextData::count == 0);
}
// state such as an event queue. This data will be migrated after
// integration into Snort.
-#include <vector>
-#include <thread>
-
#include "main/snort_types.h"
#include "framework/codec.h"
Packet* encode_packet;
DAQ_PktHdr_t* pkth;
uint8_t* buf;
- std::thread* offload;
DataPointer file_data;
+ class SnortConfig* conf;
class MpseStash* stash;
struct OtnxMatchData* otnx;
struct SF_EVENTQ* equeue;
uint64_t pkt_count;
- bool onload;
ActiveRules active_rules;
static const unsigned buf_size = Codec::PKT_MAX;
--- /dev/null
+//--------------------------------------------------------------------------
+// Copyright (C) 2016-2016 Cisco and/or its affiliates. All rights reserved.
+//
+// This program is free software; you can redistribute it and/or modify it
+// under the terms of the GNU General Public License Version 2 as published
+// by the Free Software Foundation. You may not use, modify or distribute
+// this program under any other version of the GNU General Public License.
+//
+// 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 along
+// with this program; if not, write to the Free Software Foundation, Inc.,
+// 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
+//--------------------------------------------------------------------------
+
+// regex_offload.cc author Russ Combs <rucombs@cisco.com>
+
+#include "regex_offload.h"
+
+#include <assert.h>
+
+#include <chrono>
+#include <condition_variable>
+#include <mutex>
+#include <vector>
+#include <thread>
+
+#include "main/snort_config.h"
+#include "fp_detect.h"
+#include "ips_context.h"
+
+struct RegexRequest
+{
+ Packet* packet = nullptr;
+
+ std::thread* offload;
+ std::mutex mutex;
+ std::condition_variable cond;
+
+ unsigned id = 0;
+ bool onload = false;
+ bool go = true;
+};
+
+//--------------------------------------------------------------------------
+// foo
+//--------------------------------------------------------------------------
+
+RegexOffload::RegexOffload(unsigned max)
+{
+ for ( unsigned i = 0; i < max; ++i )
+ {
+ RegexRequest* req = new RegexRequest;
+ req->offload = new std::thread(worker, req);
+ idle.push_back(req);
+ }
+}
+
+RegexOffload::~RegexOffload()
+{
+ assert(busy.empty());
+
+ for ( auto* req : idle )
+ {
+ req->offload->join();
+ delete req->offload;
+ delete req;
+ }
+}
+
+void RegexOffload::stop()
+{
+ assert(busy.empty());
+
+ for ( auto* req : idle )
+ {
+ req->go = false;
+ std::unique_lock<std::mutex> lock(req->mutex);
+ req->cond.notify_one();
+ }
+}
+
+void RegexOffload::worker(RegexRequest* req)
+{
+ while ( true )
+ {
+ std::unique_lock<std::mutex> lock(req->mutex);
+ req->cond.wait_for(lock, std::chrono::seconds(1)); // FIXIT-L w/o some hangs upon join
+
+ if ( !req->go )
+ break;
+
+ if ( !req->packet )
+ continue;
+
+ assert(req->packet->flow->is_offloaded());
+ snort_conf = req->packet->context->conf; // FIXIT-H reload issue
+ fp_offload(req->packet);
+ req->onload = true;
+ }
+}
+
+void RegexOffload::put(unsigned id, Packet* p)
+{
+ assert(!idle.empty());
+ RegexRequest* req = idle.front();
+
+ idle.pop_front(); // FIXTHIS-H use splice to move instead
+ busy.push_back(req);
+
+ req->id = id;
+ req->onload = false;
+ req->packet = p;
+
+ std::unique_lock<std::mutex> lock(req->mutex);
+ req->cond.notify_one();
+}
+
+bool RegexOffload::get(unsigned& id)
+{
+ assert(!busy.empty());
+ RegexRequest* req = busy.front(); // FIXTHIS-H onload any order
+
+ if ( !req->onload )
+ return false;
+
+ id = req->id;
+ busy.pop_front(); // FIXTHIS-H use splice to move instead
+ req->packet = nullptr;
+ idle.push_back(req);
+ return true;
+}
+
+bool RegexOffload::on_hold(Flow* f)
+{
+ for ( auto* req : busy )
+ {
+ if ( req->packet->flow == f )
+ return true;
+ }
+ return false;
+}
+
--- /dev/null
+//--------------------------------------------------------------------------
+// Copyright (C) 2016-2016 Cisco and/or its affiliates. All rights reserved.
+//
+// This program is free software; you can redistribute it and/or modify it
+// under the terms of the GNU General Public License Version 2 as published
+// by the Free Software Foundation. You may not use, modify or distribute
+// this program under any other version of the GNU General Public License.
+//
+// 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 along
+// with this program; if not, write to the Free Software Foundation, Inc.,
+// 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
+//--------------------------------------------------------------------------
+
+// regex_offload.h author Russ Combs <rucombs@cisco.com>
+
+#ifndef REGEX_OFFLOAD_H
+#define REGEX_OFFLOAD_H
+
+// RegexOffload provides an interface to fast pattern search accelerators.
+// currently implemented as a simple thread offload, but will become an
+// abstract base class with true hardware offload subclasses. for starters
+// the thread offload will "cheat" and tightly interface with fp_detect but
+// eventually morph into such a proper subclass as the offload api emerges.
+// presently all offload is per packet thread; packet threads do not share
+// offload resources.
+
+#include <condition_variable>
+#include <list>
+#include <mutex>
+#include <thread>
+
+struct Packet;
+struct RegexRequest;
+
+class RegexOffload
+{
+public:
+ RegexOffload(unsigned max);
+ ~RegexOffload();
+
+ void stop();
+
+ unsigned count()
+ { return busy.size(); }
+
+ void put(unsigned id, Packet*);
+ bool get(unsigned& id);
+
+ bool on_hold(class Flow*);
+
+private:
+ static void worker(RegexRequest*);
+
+private:
+ std::list<RegexRequest*> busy;
+ std::list<RegexRequest*> idle;
+};
+
+#endif
+
#include "protocols/packet.h"
#include "sfip/sf_ip.h"
#include "utils/bitop.h"
+#include "utils/stats.h"
#include "utils/util.h"
unsigned FlowData::flow_id = 0;
void Flow::restart(bool dump_flow_data)
{
+ DetectionEngine::onload(this);
+
if ( dump_flow_data )
free_flow_data();
PktType pkt_type; // ^^
// these fields are always set; not zeroed
- uint8_t flow_flags;
+ uint64_t flow_flags; // FIXIT-H required to ensure atomic?
Flow* prev, * next;
Inspector* ssn_client;
Inspector* ssn_server;
DAQ_PktHdr_t pkth;
session->GetPacketHeaderFoo(&pkth, pkt_flags);
- if ( !p )
- {
- // FIXIT-H we need to have user_policy_id in this case
- // FIXIT-H this leads to format_tcp() copying from s5_pkt to s5_pkt
- // (neither of these issues is created by passing null through to here)
- p = s5_pkt;
- }
-
PacketManager::format_tcp(enc_flags, p, s5_pkt, PSEUDO_PKT_TCP, &pkth, pkth.opaque);
prep_s5_pkt(session->flow, p, pkt_flags);
// sanity check since this is called externally
assert(p->ptrs.tcph);
+ assert(!p or p->flow == flow);
DetectionEngine::onload(flow);
TcpStreamTracker* talker, * listener;
void TcpSession::clear_session(bool free_flow_data, bool flush_segments, bool restart, Packet* p)
{
+ assert(!p or p->flow == flow);
DetectionEngine::onload(flow);
if ( client->reassembler )