More stats improvements

Broke out the stats collection into a bunch of new Bifs
in stats.bif.  Scripts that use stats collection functions
have also been updated.  More work to do.
This commit is contained in:
Seth Hall 2016-01-07 16:20:24 -05:00
parent 6aeeb94d76
commit 6d836b7956
27 changed files with 479 additions and 353 deletions

View file

@ -18,7 +18,7 @@ export {
event net_stats_update(last_stat: NetStats) event net_stats_update(last_stat: NetStats)
{ {
local ns = net_stats(); local ns = get_net_stats();
local new_dropped = ns$pkts_dropped - last_stat$pkts_dropped; local new_dropped = ns$pkts_dropped - last_stat$pkts_dropped;
if ( new_dropped > 0 ) if ( new_dropped > 0 )
{ {
@ -38,5 +38,5 @@ event bro_init()
# Since this currently only calculates packet drops, let's skip the stats # Since this currently only calculates packet drops, let's skip the stats
# collection if reading traces. # collection if reading traces.
if ( ! reading_traces() ) if ( ! reading_traces() )
schedule stats_collection_interval { net_stats_update(net_stats()) }; schedule stats_collection_interval { net_stats_update(get_net_stats()) };
} }

View file

@ -455,30 +455,15 @@ type NetStats: record {
bytes_recvd: count &default=0; ##< Bytes received by Bro. bytes_recvd: count &default=0; ##< Bytes received by Bro.
}; };
## Statistics about Bro's resource consumption. type ConnStats: record {
## total_conns: count; ##<
## .. bro:see:: resource_usage current_conns: count; ##<
## current_conns_extern: count; ##<
## .. note:: All process-level values refer to Bro's main process only, not to sess_current_conns: count; ##<
## the child process it spawns for doing communication.
type bro_resources: record {
version: string; ##< Bro version string.
debug: bool; ##< True if compiled with --enable-debug.
start_time: time; ##< Start time of process.
real_time: interval; ##< Elapsed real time since Bro started running.
user_time: interval; ##< User CPU seconds.
system_time: interval; ##< System CPU seconds.
mem: count; ##< Maximum memory consumed, in KB.
minor_faults: count; ##< Page faults not requiring actual I/O.
major_faults: count; ##< Page faults requiring actual I/O.
num_swap: count; ##< Times swapped out.
blocking_input: count; ##< Blocking input operations.
blocking_output: count; ##< Blocking output operations.
num_context: count; ##< Number of involuntary context switches.
num_packets: count; ##< Total number of packets processed to date. num_packets: count;
num_fragments: count; ##< Current number of fragments pending reassembly. num_fragments: count;
max_fragments: count; ##< Maximum number of concurrently buffered fragments so far. max_fragments: count;
num_tcp_conns: count; ##< Current number of TCP connections in memory. num_tcp_conns: count; ##< Current number of TCP connections in memory.
max_tcp_conns: count; ##< Maximum number of concurrent TCP connections so far. max_tcp_conns: count; ##< Maximum number of concurrent TCP connections so far.
@ -492,27 +477,49 @@ type bro_resources: record {
max_icmp_conns: count; ##< Maximum number of concurrent ICMP connections so far. max_icmp_conns: count; ##< Maximum number of concurrent ICMP connections so far.
cumulative_icmp_conns: count; ##< cumulative_icmp_conns: count; ##<
num_timers: count; ##< Current number of pending timers. killed_by_inactivity: count;
max_timers: count; ##< Maximum number of concurrent timers pending so far. };
## Statistics about Bro's process.
##
## .. bro:see:: get_proc_stats
##
## .. note:: All process-level values refer to Bro's main process only, not to
## the child process it spawns for doing communication.
type ProcStats: record {
debug: bool; ##< True if compiled with --enable-debug.
start_time: time; ##< Start time of process.
real_time: interval; ##< Elapsed real time since Bro started running.
user_time: interval; ##< User CPU seconds.
system_time: interval; ##< System CPU seconds.
mem: count; ##< Maximum memory consumed, in KB.
minor_faults: count; ##< Page faults not requiring actual I/O.
major_faults: count; ##< Page faults requiring actual I/O.
num_swap: count; ##< Times swapped out.
blocking_input: count; ##< Blocking input operations.
blocking_output: count; ##< Blocking output operations.
num_context: count; ##< Number of involuntary context switches.
};
type EventStats: record {
num_events_queued: count; ##< Total number of events queued so far. num_events_queued: count; ##< Total number of events queued so far.
num_events_dispatched: count; ##< Total number of events dispatched so far. num_events_dispatched: count; ##< Total number of events dispatched so far.
};
total_conns: count; ##< ## Summary statistics of all regular expression matchers.
current_conns: count; ##< ##
current_conns_extern: count; ##< ## .. bro:see:: get_reassembler_stats
sess_current_conns: count; ##< type ReassemblerStats: record {
file_size: count; ##< Byte size of File reassembly tracking.
reassem_file_size: count; ##< Size of File reassembly tracking. frag_size: count; ##< Byte size of Fragment reassembly tracking.
reassem_frag_size: count; ##< Size of Fragment reassembly tracking. tcp_size: count; ##< Byte size of TCP reassembly tracking.
reassem_tcp_size: count; ##< Size of TCP reassembly tracking. unknown_size: count; ##< Byte size of reassembly tracking for unknown purposes.
reassem_unknown_size: count; ##< Size of reassembly tracking for unknown purposes.
}; };
## Summary statistics of all regular expression matchers. ## Summary statistics of all regular expression matchers.
## ##
## .. bro:see:: get_matcher_stats ## .. bro:see:: get_matcher_stats
type matcher_stats: record { type MatcherStats: record {
matchers: count; ##< Number of distinct RE matchers. matchers: count; ##< Number of distinct RE matchers.
dfa_states: count; ##< Number of DFA states across all matchers. dfa_states: count; ##< Number of DFA states across all matchers.
computed: count; ##< Number of computed DFA state transitions. computed: count; ##< Number of computed DFA state transitions.
@ -522,16 +529,44 @@ type matcher_stats: record {
avg_nfa_states: count; ##< Average number of NFA states across all matchers. avg_nfa_states: count; ##< Average number of NFA states across all matchers.
}; };
type TimerStats: record {
num_timers: count; ##< Current number of pending timers.
max_timers: count; ##< Maximum number of concurrent timers pending so far.
};
type FileAnalysisStats: record {
current: count;
max: count;
cumulative: count;
};
type DNSStats: record {
requests: count;
successful: count;
failed: count;
pending: count;
cached_hosts: count;
cached_addresses: count;
};
## Statistics about number of gaps in TCP connections. ## Statistics about number of gaps in TCP connections.
## ##
## .. bro:see:: get_gap_summary ## .. bro:see:: get_gap_stats
type gap_info: record { type GapStats: record {
ack_events: count; ##< How many ack events *could* have had gaps. ack_events: count; ##< How many ack events *could* have had gaps.
ack_bytes: count; ##< How many bytes those covered. ack_bytes: count; ##< How many bytes those covered.
gap_events: count; ##< How many *did* have gaps. gap_events: count; ##< How many *did* have gaps.
gap_bytes: count; ##< How many bytes were missing in the gaps. gap_bytes: count; ##< How many bytes were missing in the gaps.
}; };
type PatternStats: record {
};
type ThreadStats: record {
num_threads: count;
};
## Deprecated. ## Deprecated.
## ##
## .. todo:: Remove. It's still declared internally but doesn't seem used anywhere ## .. todo:: Remove. It's still declared internally but doesn't seem used anywhere

View file

@ -26,7 +26,7 @@ event ChecksumOffloading::check()
if ( done ) if ( done )
return; return;
local pkts_recvd = net_stats()$pkts_recvd; local pkts_recvd = get_net_stats()$pkts_recvd;
local bad_ip_checksum_pct = (pkts_recvd != 0) ? (bad_ip_checksums*1.0 / pkts_recvd*1.0) : 0; local bad_ip_checksum_pct = (pkts_recvd != 0) ? (bad_ip_checksums*1.0 / pkts_recvd*1.0) : 0;
local bad_tcp_checksum_pct = (pkts_recvd != 0) ? (bad_tcp_checksums*1.0 / pkts_recvd*1.0) : 0; local bad_tcp_checksum_pct = (pkts_recvd != 0) ? (bad_tcp_checksums*1.0 / pkts_recvd*1.0) : 0;
local bad_udp_checksum_pct = (pkts_recvd != 0) ? (bad_udp_checksums*1.0 / pkts_recvd*1.0) : 0; local bad_udp_checksum_pct = (pkts_recvd != 0) ? (bad_udp_checksums*1.0 / pkts_recvd*1.0) : 0;

View file

@ -56,7 +56,7 @@ event CaptureLoss::take_measurement(last_ts: time, last_acks: count, last_gaps:
} }
local now = network_time(); local now = network_time();
local g = get_gap_summary(); local g = get_gap_stats();
local acks = g$ack_events - last_acks; local acks = g$ack_events - last_acks;
local gaps = g$gap_events - last_gaps; local gaps = g$gap_events - last_gaps;
local pct_lost = (acks == 0) ? 0.0 : (100 * (1.0 * gaps) / (1.0 * acks)); local pct_lost = (acks == 0) ? 0.0 : (100 * (1.0 * gaps) / (1.0 * acks));

View file

@ -1,6 +1,4 @@
##! Log memory/packet/lag statistics. Differs from ##! Log memory/packet/lag statistics.
##! :doc:`/scripts/policy/misc/profiling.bro` in that this
##! is lighter-weight (much less info, and less load to generate).
@load base/frameworks/notice @load base/frameworks/notice
@ -10,7 +8,7 @@ export {
redef enum Log::ID += { LOG }; redef enum Log::ID += { LOG };
## How often stats are reported. ## How often stats are reported.
const stats_report_interval = 5min &redef; const stats_report_interval = 1sec &redef;
type Info: record { type Info: record {
## Timestamp for the measurement. ## Timestamp for the measurement.
@ -27,6 +25,13 @@ export {
## interval. ## interval.
events_queued: count &log; events_queued: count &log;
## TCP connections currently in memory.
active_tcp_conns: count &log;
## UDP connections currently in memory.
active_udp_conns: count &log;
## ICMP connections currently in memory.
active_icmp_conns: count &log;
## TCP connections seen since last stats interval. ## TCP connections seen since last stats interval.
tcp_conns: count &log; tcp_conns: count &log;
## UDP connections seen since last stats interval. ## UDP connections seen since last stats interval.
@ -69,11 +74,14 @@ event bro_init() &priority=5
Log::create_stream(Stats::LOG, [$columns=Info, $ev=log_stats, $path="stats"]); Log::create_stream(Stats::LOG, [$columns=Info, $ev=log_stats, $path="stats"]);
} }
event check_stats(last_ts: time, last_ns: NetStats, last_res: bro_resources) event check_stats(last_ts: time, last_ns: NetStats, last_cs: ConnStats, last_ps: ProcStats, last_es: EventStats, last_rs: ReassemblerStats)
{ {
local now = current_time(); local now = current_time();
local ns = net_stats(); local ns = get_net_stats();
local res = resource_usage(); local cs = get_conn_stats();
local ps = get_proc_stats();
local es = get_event_stats();
local rs = get_reassembler_stats();
if ( bro_is_terminating() ) if ( bro_is_terminating() )
# No more stats will be written or scheduled when Bro is # No more stats will be written or scheduled when Bro is
@ -82,21 +90,27 @@ event check_stats(last_ts: time, last_ns: NetStats, last_res: bro_resources)
local info: Info = [$ts=now, local info: Info = [$ts=now,
$peer=peer_description, $peer=peer_description,
$mem=res$mem/1000000, $mem=ps$mem/1000000,
$pkts_proc=res$num_packets - last_res$num_packets, $pkts_proc=ns$pkts_recvd - last_ns$pkts_recvd,
$events_proc=res$num_events_dispatched - last_res$num_events_dispatched,
$events_queued=res$num_events_queued - last_res$num_events_queued, $active_tcp_conns=cs$num_tcp_conns,
$tcp_conns=res$cumulative_tcp_conns - last_res$cumulative_tcp_conns, $tcp_conns=cs$cumulative_tcp_conns - last_cs$cumulative_tcp_conns,
$udp_conns=res$cumulative_udp_conns - last_res$cumulative_udp_conns, $active_udp_conns=cs$num_udp_conns,
$icmp_conns=res$cumulative_icmp_conns - last_res$cumulative_icmp_conns, $udp_conns=cs$cumulative_udp_conns - last_cs$cumulative_udp_conns,
$reassem_tcp_size=res$reassem_tcp_size, $active_icmp_conns=cs$num_icmp_conns,
$reassem_file_size=res$reassem_file_size, $icmp_conns=cs$cumulative_icmp_conns - last_cs$cumulative_icmp_conns,
$reassem_frag_size=res$reassem_frag_size,
$reassem_unknown_size=res$reassem_unknown_size $reassem_tcp_size=rs$tcp_size,
$reassem_file_size=rs$file_size,
$reassem_frag_size=rs$frag_size,
$reassem_unknown_size=rs$unknown_size,
$events_proc=es$num_events_dispatched - last_es$num_events_dispatched,
$events_queued=es$num_events_queued - last_es$num_events_queued
]; ];
# Someone's going to have to explain what this is and add a field to the Info record. # Someone's going to have to explain what this is and add a field to the Info record.
# info$util = 100.0*((res$user_time + res$system_time) - (last_res$user_time + last_res$system_time))/(now-last_ts); # info$util = 100.0*((ps$user_time + ps$system_time) - (last_ps$user_time + last_ps$system_time))/(now-last_ts);
if ( reading_live_traffic() ) if ( reading_live_traffic() )
{ {
@ -108,10 +122,10 @@ event check_stats(last_ts: time, last_ns: NetStats, last_res: bro_resources)
} }
Log::write(Stats::LOG, info); Log::write(Stats::LOG, info);
schedule stats_report_interval { check_stats(now, ns, res) }; schedule stats_report_interval { check_stats(now, ns, cs, ps, es, rs) };
} }
event bro_init() event bro_init()
{ {
schedule stats_report_interval { check_stats(current_time(), net_stats(), resource_usage()) }; schedule stats_report_interval { check_stats(current_time(), get_net_stats(), get_conn_stats(), get_proc_stats(), get_event_stats(), get_reassembler_stats()) };
} }

View file

@ -118,6 +118,7 @@ include(BifCl)
set(BIF_SRCS set(BIF_SRCS
bro.bif bro.bif
stats.bif
event.bif event.bif
const.bif const.bif
types.bif types.bif

View file

@ -108,9 +108,9 @@ bool ConnectionTimer::DoUnserialize(UnserialInfo* info)
return true; return true;
} }
unsigned int Connection::total_connections = 0; uint64 Connection::total_connections = 0;
unsigned int Connection::current_connections = 0; uint64 Connection::current_connections = 0;
unsigned int Connection::external_connections = 0; uint64 Connection::external_connections = 0;
IMPLEMENT_SERIAL(Connection, SER_CONNECTION); IMPLEMENT_SERIAL(Connection, SER_CONNECTION);

View file

@ -220,11 +220,11 @@ public:
unsigned int MemoryAllocation() const; unsigned int MemoryAllocation() const;
unsigned int MemoryAllocationConnVal() const; unsigned int MemoryAllocationConnVal() const;
static unsigned int TotalConnections() static uint64 TotalConnections()
{ return total_connections; } { return total_connections; }
static unsigned int CurrentConnections() static uint64 CurrentConnections()
{ return current_connections; } { return current_connections; }
static unsigned int CurrentExternalConnections() static uint64 CurrentExternalConnections()
{ return external_connections; } { return external_connections; }
// Returns true if the history was already seen, false otherwise. // Returns true if the history was already seen, false otherwise.
@ -315,9 +315,9 @@ protected:
unsigned int saw_first_orig_packet:1, saw_first_resp_packet:1; unsigned int saw_first_orig_packet:1, saw_first_resp_packet:1;
// Count number of connections. // Count number of connections.
static unsigned int total_connections; static uint64 total_connections;
static unsigned int current_connections; static uint64 current_connections;
static unsigned int external_connections; static uint64 external_connections;
string history; string history;
uint32 hist_seen; uint32 hist_seen;

View file

@ -9,6 +9,8 @@
unsigned int DFA_State::transition_counter = 0; unsigned int DFA_State::transition_counter = 0;
uint64 total_dfa_states = 0;
DFA_State::DFA_State(int arg_state_num, const EquivClass* ec, DFA_State::DFA_State(int arg_state_num, const EquivClass* ec,
NFA_state_list* arg_nfa_states, NFA_state_list* arg_nfa_states,
AcceptingSet* arg_accept) AcceptingSet* arg_accept)
@ -20,6 +22,8 @@ DFA_State::DFA_State(int arg_state_num, const EquivClass* ec,
mark = 0; mark = 0;
centry = 0; centry = 0;
++total_dfa_states;
SymPartition(ec); SymPartition(ec);
xtions = new DFA_State*[num_sym]; xtions = new DFA_State*[num_sym];
@ -433,19 +437,6 @@ void DFA_Machine::Dump(FILE* f)
start_state->ClearMarks(); start_state->ClearMarks();
} }
void DFA_Machine::DumpStats(FILE* f)
{
DFA_State_Cache::Stats stats;
dfa_state_cache->GetStats(&stats);
fprintf(f, "Computed dfa_states = %d; Classes = %d; Computed trans. = %d; Uncomputed trans. = %d\n",
stats.dfa_states, EC()->NumClasses(),
stats.computed, stats.uncomputed);
fprintf(f, "DFA cache hits = %d; misses = %d\n",
stats.hits, stats.misses);
}
unsigned int DFA_Machine::MemoryAllocation() const unsigned int DFA_Machine::MemoryAllocation() const
{ {
DFA_State_Cache::Stats s; DFA_State_Cache::Stats s;

View file

@ -19,6 +19,8 @@ class DFA_Machine;
class DFA_State; class DFA_State;
struct CacheEntry; struct CacheEntry;
extern uint64 total_dfa_states;
class DFA_State : public BroObj { class DFA_State : public BroObj {
public: public:
DFA_State(int state_num, const EquivClass* ec, DFA_State(int state_num, const EquivClass* ec,
@ -132,7 +134,6 @@ public:
void Describe(ODesc* d) const; void Describe(ODesc* d) const;
void Dump(FILE* f); void Dump(FILE* f);
void DumpStats(FILE* f);
unsigned int MemoryAllocation() const; unsigned int MemoryAllocation() const;

View file

@ -628,10 +628,12 @@ void builtin_error(const char* msg, BroObj* arg)
} }
#include "bro.bif.func_h" #include "bro.bif.func_h"
#include "stats.bif.func_h"
#include "reporter.bif.func_h" #include "reporter.bif.func_h"
#include "strings.bif.func_h" #include "strings.bif.func_h"
#include "bro.bif.func_def" #include "bro.bif.func_def"
#include "stats.bif.func_def"
#include "reporter.bif.func_def" #include "reporter.bif.func_def"
#include "strings.bif.func_def" #include "strings.bif.func_def"
@ -640,13 +642,23 @@ void builtin_error(const char* msg, BroObj* arg)
void init_builtin_funcs() void init_builtin_funcs()
{ {
bro_resources = internal_type("bro_resources")->AsRecordType(); ProcStats = internal_type("ProcStats")->AsRecordType();
net_stats = internal_type("NetStats")->AsRecordType(); NetStats = internal_type("NetStats")->AsRecordType();
matcher_stats = internal_type("matcher_stats")->AsRecordType(); MatcherStats = internal_type("MatcherStats")->AsRecordType();
ConnStats = internal_type("ConnStats")->AsRecordType();
ReassemblerStats = internal_type("ReassemblerStats")->AsRecordType();
DNSStats = internal_type("DNSStats")->AsRecordType();
GapStats = internal_type("GapStats")->AsRecordType();
EventStats = internal_type("EventStats")->AsRecordType();
TimerStats = internal_type("TimerStats")->AsRecordType();
FileAnalysisStats = internal_type("FileAnalysisStats")->AsRecordType();
ThreadStats = internal_type("ThreadStats")->AsRecordType();
PatternStats = internal_type("PatternStats")->AsRecordType();
var_sizes = internal_type("var_sizes")->AsTableType(); var_sizes = internal_type("var_sizes")->AsTableType();
gap_info = internal_type("gap_info")->AsRecordType();
#include "bro.bif.func_init" #include "bro.bif.func_init"
#include "stats.bif.func_init"
#include "reporter.bif.func_init" #include "reporter.bif.func_init"
#include "strings.bif.func_init" #include "strings.bif.func_init"

View file

@ -285,11 +285,6 @@ void NFA_Machine::Dump(FILE* f)
first_state->ClearMarks(); first_state->ClearMarks();
} }
void NFA_Machine::DumpStats(FILE* f)
{
fprintf(f, "highest NFA state ID is %d\n", nfa_state_id);
}
NFA_Machine* make_alternate(NFA_Machine* m1, NFA_Machine* m2) NFA_Machine* make_alternate(NFA_Machine* m1, NFA_Machine* m2)
{ {
if ( ! m1 ) if ( ! m1 )

View file

@ -105,7 +105,6 @@ public:
void Describe(ODesc* d) const; void Describe(ODesc* d) const;
void Dump(FILE* f); void Dump(FILE* f);
void DumpStats(FILE* f);
unsigned int MemoryAllocation() const unsigned int MemoryAllocation() const
{ return padded_sizeof(*this) + first_state->TotalMemoryAllocation(); } { return padded_sizeof(*this) + first_state->TotalMemoryAllocation(); }

View file

@ -197,7 +197,6 @@ Val* pkt_profile_file;
int load_sample_freq; int load_sample_freq;
double gap_report_freq; double gap_report_freq;
RecordType* gap_info;
int packet_filter_default; int packet_filter_default;

View file

@ -200,9 +200,6 @@ extern Val* pkt_profile_file;
extern int load_sample_freq; extern int load_sample_freq;
extern double gap_report_freq;
extern RecordType* gap_info;
extern int packet_filter_default; extern int packet_filter_default;
extern int sig_max_group_size; extern int sig_max_group_size;

View file

@ -1163,15 +1163,11 @@ void NetSessions::GetStats(SessionStats& s) const
s.cumulative_ICMP_conns = icmp_conns.NumCumulativeInserts(); s.cumulative_ICMP_conns = icmp_conns.NumCumulativeInserts();
s.num_fragments = fragments.Length(); s.num_fragments = fragments.Length();
s.num_packets = num_packets_processed; s.num_packets = num_packets_processed;
s.num_timers = timer_mgr->Size();
s.num_events_queued = num_events_queued;
s.num_events_dispatched = num_events_dispatched;
s.max_TCP_conns = tcp_conns.MaxLength(); s.max_TCP_conns = tcp_conns.MaxLength();
s.max_UDP_conns = udp_conns.MaxLength(); s.max_UDP_conns = udp_conns.MaxLength();
s.max_ICMP_conns = icmp_conns.MaxLength(); s.max_ICMP_conns = icmp_conns.MaxLength();
s.max_fragments = fragments.MaxLength(); s.max_fragments = fragments.MaxLength();
s.max_timers = timer_mgr->PeakSize();
} }
Connection* NetSessions::NewConn(HashKey* k, double t, const ConnID* id, Connection* NetSessions::NewConn(HashKey* k, double t, const ConnID* id,

View file

@ -46,10 +46,6 @@ struct SessionStats {
int num_fragments; int num_fragments;
int max_fragments; int max_fragments;
uint64 num_packets; uint64 num_packets;
int num_timers;
int max_timers;
uint64 num_events_queued;
uint64 num_events_dispatched;
}; };
// Drains and deletes a timer manager if it hasn't seen any advances // Drains and deletes a timer manager if it hasn't seen any advances

View file

@ -14,7 +14,7 @@
#include "broker/Manager.h" #include "broker/Manager.h"
#endif #endif
int killed_by_inactivity = 0; uint64 killed_by_inactivity = 0;
uint64 tot_ack_events = 0; uint64 tot_ack_events = 0;
uint64 tot_ack_bytes = 0; uint64 tot_ack_bytes = 0;
@ -82,7 +82,7 @@ void ProfileLogger::Log()
struct timeval tv_utime = r.ru_utime; struct timeval tv_utime = r.ru_utime;
struct timeval tv_stime = r.ru_stime; struct timeval tv_stime = r.ru_stime;
unsigned int total, malloced; uint64 total, malloced;
get_memory_usage(&total, &malloced); get_memory_usage(&total, &malloced);
static unsigned int first_total = 0; static unsigned int first_total = 0;
@ -110,7 +110,7 @@ void ProfileLogger::Log()
file->Write(fmt("\n%.06f ------------------------\n", network_time)); file->Write(fmt("\n%.06f ------------------------\n", network_time));
} }
file->Write(fmt("%.06f Memory: total=%dK total_adj=%dK malloced: %dK\n", file->Write(fmt("%.06f Memory: total=%" PRId64 "K total_adj=%" PRId64 "K malloced: %" PRId64 "K\n",
network_time, total / 1024, (total - first_total) / 1024, network_time, total / 1024, (total - first_total) / 1024,
malloced / 1024)); malloced / 1024));
@ -120,7 +120,7 @@ void ProfileLogger::Log()
int conn_mem_use = expensive ? sessions->ConnectionMemoryUsage() : 0; int conn_mem_use = expensive ? sessions->ConnectionMemoryUsage() : 0;
file->Write(fmt("%.06f Conns: total=%d current=%d/%d ext=%d mem=%dK avg=%.1f table=%dK connvals=%dK\n", file->Write(fmt("%.06f Conns: total=%" PRIu64 " current=%" PRIu64 "/%" PRIi32 " ext=%" PRIu64 " mem=%" PRIi32 "K avg=%.1f table=%" PRIu32 "K connvals=%" PRIu32 "K\n",
network_time, network_time,
Connection::TotalConnections(), Connection::TotalConnections(),
Connection::CurrentConnections(), Connection::CurrentConnections(),
@ -161,7 +161,7 @@ void ProfileLogger::Log()
)); ));
*/ */
file->Write(fmt("%.06f Connections expired due to inactivity: %d\n", file->Write(fmt("%.06f Connections expired due to inactivity: %" PRIu64 "\n",
network_time, killed_by_inactivity)); network_time, killed_by_inactivity));
file->Write(fmt("%.06f Total reassembler data: %" PRIu64 "K\n", network_time, file->Write(fmt("%.06f Total reassembler data: %" PRIu64 "K\n", network_time,
@ -465,10 +465,10 @@ void PacketProfiler::ProfilePkt(double t, unsigned int bytes)
double curr_Rtime = double curr_Rtime =
ptimestamp.tv_sec + ptimestamp.tv_usec / 1e6; ptimestamp.tv_sec + ptimestamp.tv_usec / 1e6;
unsigned int curr_mem; uint64 curr_mem;
get_memory_usage(&curr_mem, 0); get_memory_usage(&curr_mem, 0);
file->Write(fmt("%.06f %.03f %d %d %.03f %.03f %.03f %d\n", file->Write(fmt("%.06f %.03f %" PRIu64 " %" PRIu64 " %.03f %.03f %.03f %" PRIu64 "\n",
t, time-last_timestamp, pkt_cnt, byte_cnt, t, time-last_timestamp, pkt_cnt, byte_cnt,
curr_Rtime - last_Rtime, curr_Rtime - last_Rtime,
curr_Utime - last_Utime, curr_Utime - last_Utime,

View file

@ -102,7 +102,7 @@ extern ProfileLogger* segment_logger;
extern SampleLogger* sample_logger; extern SampleLogger* sample_logger;
// Connection statistics. // Connection statistics.
extern int killed_by_inactivity; extern uint64 killed_by_inactivity;
// Content gap statistics. // Content gap statistics.
extern uint64 tot_ack_events; extern uint64 tot_ack_events;
@ -127,9 +127,9 @@ protected:
double update_freq; double update_freq;
double last_Utime, last_Stime, last_Rtime; double last_Utime, last_Stime, last_Rtime;
double last_timestamp, time; double last_timestamp, time;
unsigned int last_mem; uint64 last_mem;
unsigned int pkt_cnt; uint64 pkt_cnt;
unsigned int byte_cnt; uint64 byte_cnt;
}; };
#endif #endif

View file

@ -63,26 +63,6 @@ function get_resp_seq%(cid: conn_id%): count
} }
%} %}
## Returns statistics about TCP gaps.
##
## Returns: A record with TCP gap statistics.
##
## .. bro:see:: do_profiling
## net_stats
## resource_usage
## dump_rule_stats
## get_matcher_stats
function get_gap_summary%(%): gap_info
%{
RecordVal* r = new RecordVal(gap_info);
r->Assign(0, new Val(tot_ack_events, TYPE_COUNT));
r->Assign(1, new Val(tot_ack_bytes, TYPE_COUNT));
r->Assign(2, new Val(tot_gap_events, TYPE_COUNT));
r->Assign(3, new Val(tot_gap_bytes, TYPE_COUNT));
return r;
%}
## Associates a file handle with a connection for writing TCP byte stream ## Associates a file handle with a connection for writing TCP byte stream
## contents. ## contents.
## ##

View file

@ -26,15 +26,8 @@
using namespace std; using namespace std;
RecordType* net_stats;
RecordType* bro_resources;
RecordType* matcher_stats;
TableType* var_sizes; TableType* var_sizes;
// This one is extern, since it's used beyond just built-ins,
// and hence it's declared in NetVar.{h,cc}.
extern RecordType* gap_info;
static iosource::PktDumper* addl_pkt_dumper = 0; static iosource::PktDumper* addl_pkt_dumper = 0;
bro_int_t parse_int(const char*& fmt) bro_int_t parse_int(const char*& fmt)
@ -1661,169 +1654,6 @@ function reading_traces%(%): bool
return new Val(reading_traces, TYPE_BOOL); return new Val(reading_traces, TYPE_BOOL);
%} %}
## Returns packet capture statistics. Statistics include the number of
## packets *(i)* received by Bro, *(ii)* dropped, and *(iii)* seen on the
## link (not always available).
##
## Returns: A record of packet statistics.
##
## .. bro:see:: do_profiling
## resource_usage
## get_matcher_stats
## dump_rule_stats
## get_gap_summary
function net_stats%(%): NetStats
%{
unsigned int recv = 0;
unsigned int drop = 0;
unsigned int link = 0;
unsigned int bytes_recv = 0;
const iosource::Manager::PktSrcList& pkt_srcs(iosource_mgr->GetPktSrcs());
for ( iosource::Manager::PktSrcList::const_iterator i = pkt_srcs.begin();
i != pkt_srcs.end(); i++ )
{
iosource::PktSrc* ps = *i;
struct iosource::PktSrc::Stats stat;
ps->Statistics(&stat);
recv += stat.received;
drop += stat.dropped;
link += stat.link;
bytes_recv += stat.bytes_received;
}
RecordVal* ns = new RecordVal(net_stats);
ns->Assign(0, new Val(recv, TYPE_COUNT));
ns->Assign(1, new Val(drop, TYPE_COUNT));
ns->Assign(2, new Val(link, TYPE_COUNT));
ns->Assign(3, new Val(bytes_recv, TYPE_COUNT));
return ns;
%}
## Returns Bro process statistics. Statistics include real/user/sys CPU time,
## memory usage, page faults, number of TCP/UDP/ICMP connections, timers,
## and events queued/dispatched.
##
## Returns: A record with resource usage statistics.
##
## .. bro:see:: do_profiling
## net_stats
## get_matcher_stats
## dump_rule_stats
## get_gap_summary
function resource_usage%(%): bro_resources
%{
struct rusage r;
if ( getrusage(RUSAGE_SELF, &r) < 0 )
reporter->InternalError("getrusage() failed in bro_resource_usage()");
double elapsed_time = current_time() - bro_start_time;
double user_time =
double(r.ru_utime.tv_sec) + double(r.ru_utime.tv_usec) / 1e6;
double system_time =
double(r.ru_stime.tv_sec) + double(r.ru_stime.tv_usec) / 1e6;
RecordVal* res = new RecordVal(bro_resources);
int n = 0;
res->Assign(n++, new StringVal(bro_version()));
#ifdef DEBUG
res->Assign(n++, new Val(1, TYPE_COUNT));
#else
res->Assign(n++, new Val(0, TYPE_COUNT));
#endif
res->Assign(n++, new Val(bro_start_time, TYPE_TIME));
res->Assign(n++, new IntervalVal(elapsed_time, Seconds));
res->Assign(n++, new IntervalVal(user_time, Seconds));
res->Assign(n++, new IntervalVal(system_time, Seconds));
unsigned int total_mem;
get_memory_usage(&total_mem, 0);
res->Assign(n++, new Val(unsigned(total_mem), TYPE_COUNT));
res->Assign(n++, new Val(unsigned(r.ru_minflt), TYPE_COUNT));
res->Assign(n++, new Val(unsigned(r.ru_majflt), TYPE_COUNT));
res->Assign(n++, new Val(unsigned(r.ru_nswap), TYPE_COUNT));
res->Assign(n++, new Val(unsigned(r.ru_inblock), TYPE_COUNT));
res->Assign(n++, new Val(unsigned(r.ru_oublock), TYPE_COUNT));
res->Assign(n++, new Val(unsigned(r.ru_nivcsw), TYPE_COUNT));
SessionStats s;
if ( sessions )
sessions->GetStats(s);
#define ADD_STAT(x) \
res->Assign(n++, new Val(unsigned(sessions ? x : 0), TYPE_COUNT));
ADD_STAT(s.num_packets);
ADD_STAT(s.num_fragments);
ADD_STAT(s.max_fragments);
ADD_STAT(s.num_TCP_conns);
ADD_STAT(s.max_TCP_conns);
ADD_STAT(s.cumulative_TCP_conns);
ADD_STAT(s.num_UDP_conns);
ADD_STAT(s.max_UDP_conns);
ADD_STAT(s.cumulative_UDP_conns);
ADD_STAT(s.num_ICMP_conns);
ADD_STAT(s.max_ICMP_conns);
ADD_STAT(s.cumulative_ICMP_conns);
ADD_STAT(s.num_timers);
ADD_STAT(s.max_timers);
ADD_STAT(s.num_events_queued);
ADD_STAT(s.num_events_dispatched);
res->Assign(n++, new Val(unsigned(Connection::TotalConnections()), TYPE_COUNT));
res->Assign(n++, new Val(unsigned(Connection::CurrentConnections()), TYPE_COUNT));
res->Assign(n++, new Val(unsigned(Connection::CurrentExternalConnections()), TYPE_COUNT));
res->Assign(n++, new Val(unsigned(sessions->CurrentConnections()), TYPE_COUNT));
res->Assign(n++, new Val(unsigned(Reassembler::MemoryAllocation(REASSEM_FILE)), TYPE_COUNT));
res->Assign(n++, new Val(unsigned(Reassembler::MemoryAllocation(REASSEM_FRAG)), TYPE_COUNT));
res->Assign(n++, new Val(unsigned(Reassembler::MemoryAllocation(REASSEM_TCP)), TYPE_COUNT));
res->Assign(n++, new Val(unsigned(Reassembler::MemoryAllocation(REASSEM_UNKNOWN)), TYPE_COUNT));
return res;
%}
## Returns statistics about the regular expression engine. Statistics include
## the number of distinct matchers, DFA states, DFA state transitions, memory
## usage of DFA states, cache hits/misses, and average number of NFA states
## across all matchers.
##
## Returns: A record with matcher statistics.
##
## .. bro:see:: do_profiling
## net_stats
## resource_usage
## dump_rule_stats
## get_gap_summary
function get_matcher_stats%(%): matcher_stats
%{
RuleMatcher::Stats s;
memset(&s, 0, sizeof(s));
if ( rule_matcher )
rule_matcher->GetStats(&s);
RecordVal* r = new RecordVal(matcher_stats);
r->Assign(0, new Val(s.matchers, TYPE_COUNT));
r->Assign(1, new Val(s.dfa_states, TYPE_COUNT));
r->Assign(2, new Val(s.computed, TYPE_COUNT));
r->Assign(3, new Val(s.mem, TYPE_COUNT));
r->Assign(4, new Val(s.hits, TYPE_COUNT));
r->Assign(5, new Val(s.misses, TYPE_COUNT));
r->Assign(6, new Val(s.avg_nfa_states, TYPE_COUNT));
return r;
%}
## Generates a table of the size of all global variables. The table index is ## Generates a table of the size of all global variables. The table index is
## the variable name and the value is the variable size in bytes. ## the variable name and the value is the variable size in bytes.
@ -1964,8 +1794,7 @@ function record_fields%(rec: any%): record_field_table
## .. bro:see:: net_stats ## .. bro:see:: net_stats
## resource_usage ## resource_usage
## get_matcher_stats ## get_matcher_stats
## dump_rule_stats ## get_gap_stats
## get_gap_summary
function do_profiling%(%) : any function do_profiling%(%) : any
%{ %{
if ( profiling_logger ) if ( profiling_logger )
@ -2030,8 +1859,8 @@ function is_local_interface%(ip: addr%) : bool
## .. bro:see:: do_profiling ## .. bro:see:: do_profiling
## resource_usage ## resource_usage
## get_matcher_stats ## get_matcher_stats
## net_stats ## get_net_stats
## get_gap_summary ## get_gap_stats
## ##
## .. todo:: The return value should be changed to any or check appropriately. ## .. todo:: The return value should be changed to any or check appropriately.
function dump_rule_stats%(f: file%): bool function dump_rule_stats%(f: file%): bool

View file

@ -366,26 +366,6 @@ event ack_above_hole%(c: connection%);
## the two. ## the two.
event content_gap%(c: connection, is_orig: bool, seq: count, length: count%); event content_gap%(c: connection, is_orig: bool, seq: count, length: count%);
## Summarizes the amount of missing TCP payload at regular intervals.
## Internally, Bro tracks (1) the number of :bro:id:`ack_above_hole` events,
## including the number of bytes missing; and (2) the total number of TCP
## acks seen, with the total volume of bytes that have been acked. This event
## reports these statistics in :bro:id:`gap_report_freq` intervals for the
## purpose of determining packet loss.
##
## dt: The time that has passed since the last ``gap_report`` interval.
##
## info: The gap statistics.
##
## .. bro:see:: content_gap ack_above_hole
##
## .. note::
##
## Bro comes with a script :doc:`/scripts/policy/misc/capture-loss.bro` that
## uses this event to estimate packet loss and report when a predefined
## threshold is exceeded.
event gap_report%(dt: interval, info: gap_info%);
## Generated when a protocol analyzer confirms that a connection is indeed ## Generated when a protocol analyzer confirms that a connection is indeed
## using that protocol. Bro's dynamic protocol detection heuristically activates ## using that protocol. Bro's dynamic protocol detection heuristically activates
## analyzers as soon as it believes a connection *could* be using a particular ## analyzers as soon as it believes a connection *could* be using a particular

View file

@ -302,6 +302,15 @@ public:
*/ */
std::string DetectMIME(const u_char* data, uint64 len) const; std::string DetectMIME(const u_char* data, uint64 len) const;
uint64 CurrentFiles()
{ return id_map.Length(); }
uint64 MaxFiles()
{ return id_map.MaxLength(); }
uint64 CumulativeFiles()
{ return id_map.NumCumulativeInserts(); }
protected: protected:
friend class FileTimer; friend class FileTimer;

View file

@ -1172,8 +1172,8 @@ int main(int argc, char** argv)
double time_net_start = current_time(true);; double time_net_start = current_time(true);;
unsigned int mem_net_start_total; uint64 mem_net_start_total;
unsigned int mem_net_start_malloced; uint64 mem_net_start_malloced;
if ( time_bro ) if ( time_bro )
{ {
@ -1181,7 +1181,7 @@ int main(int argc, char** argv)
fprintf(stderr, "# initialization %.6f\n", time_net_start - time_start); fprintf(stderr, "# initialization %.6f\n", time_net_start - time_start);
fprintf(stderr, "# initialization %uM/%uM\n", fprintf(stderr, "# initialization %" PRIu64 "M/%" PRIu64 "M\n",
mem_net_start_total / 1024 / 1024, mem_net_start_total / 1024 / 1024,
mem_net_start_malloced / 1024 / 1024); mem_net_start_malloced / 1024 / 1024);
} }
@ -1190,8 +1190,8 @@ int main(int argc, char** argv)
double time_net_done = current_time(true);; double time_net_done = current_time(true);;
unsigned int mem_net_done_total; uint64 mem_net_done_total;
unsigned int mem_net_done_malloced; uint64 mem_net_done_malloced;
if ( time_bro ) if ( time_bro )
{ {
@ -1200,7 +1200,7 @@ int main(int argc, char** argv)
fprintf(stderr, "# total time %.6f, processing %.6f\n", fprintf(stderr, "# total time %.6f, processing %.6f\n",
time_net_done - time_start, time_net_done - time_net_start); time_net_done - time_start, time_net_done - time_net_start);
fprintf(stderr, "# total mem %uM/%uM, processing %uM/%uM\n", fprintf(stderr, "# total mem %" PRId64 "M/%" PRId64 "M, processing %" PRId64 "M/%" PRId64 "M\n",
mem_net_done_total / 1024 / 1024, mem_net_done_total / 1024 / 1024,
mem_net_done_malloced / 1024 / 1024, mem_net_done_malloced / 1024 / 1024,
(mem_net_done_total - mem_net_start_total) / 1024 / 1024, (mem_net_done_total - mem_net_start_total) / 1024 / 1024,

293
src/stats.bif Normal file
View file

@ -0,0 +1,293 @@
%%{ // C segment
#include "util.h"
#include "threading/Manager.h"
RecordType* ProcStats;
RecordType* NetStats;
RecordType* MatcherStats;
RecordType* ReassemblerStats;
RecordType* DNSStats;
RecordType* ConnStats;
RecordType* GapStats;
RecordType* EventStats;
RecordType* ThreadStats;
RecordType* PatternStats;
RecordType* TimerStats;
RecordType* FileAnalysisStats;
%%}
## Returns packet capture statistics. Statistics include the number of
## packets *(i)* received by Bro, *(ii)* dropped, and *(iii)* seen on the
## link (not always available).
##
## Returns: A record of packet statistics.
##
## .. bro:see:: do_profiling
## get_proc_stats
## get_matcher_stats
## get_gap_stats
function get_net_stats%(%): NetStats
%{
uint64 recv = 0;
uint64 drop = 0;
uint64 link = 0;
uint64 bytes_recv = 0;
const iosource::Manager::PktSrcList& pkt_srcs(iosource_mgr->GetPktSrcs());
for ( iosource::Manager::PktSrcList::const_iterator i = pkt_srcs.begin();
i != pkt_srcs.end(); i++ )
{
iosource::PktSrc* ps = *i;
struct iosource::PktSrc::Stats stat;
ps->Statistics(&stat);
recv += stat.received;
drop += stat.dropped;
link += stat.link;
bytes_recv += stat.bytes_received;
}
RecordVal* r = new RecordVal(NetStats);
int n = 0;
r->Assign(n++, new Val(recv, TYPE_COUNT));
r->Assign(n++, new Val(drop, TYPE_COUNT));
r->Assign(n++, new Val(link, TYPE_COUNT));
r->Assign(n++, new Val(bytes_recv, TYPE_COUNT));
return r;
%}
function get_conn_stats%(%): ConnStats
%{
RecordVal* r = new RecordVal(ConnStats);
int n = 0;
r->Assign(n++, new Val(Connection::TotalConnections(), TYPE_COUNT));
r->Assign(n++, new Val(Connection::CurrentConnections(), TYPE_COUNT));
r->Assign(n++, new Val(Connection::CurrentExternalConnections(), TYPE_COUNT));
r->Assign(n++, new Val(sessions->CurrentConnections(), TYPE_COUNT));
SessionStats s;
if ( sessions )
sessions->GetStats(s);
#define ADD_STAT(x) \
r->Assign(n++, new Val(unsigned(sessions ? x : 0), TYPE_COUNT));
ADD_STAT(s.num_packets);
ADD_STAT(s.num_fragments);
ADD_STAT(s.max_fragments);
ADD_STAT(s.num_TCP_conns);
ADD_STAT(s.max_TCP_conns);
ADD_STAT(s.cumulative_TCP_conns);
ADD_STAT(s.num_UDP_conns);
ADD_STAT(s.max_UDP_conns);
ADD_STAT(s.cumulative_UDP_conns);
ADD_STAT(s.num_ICMP_conns);
ADD_STAT(s.max_ICMP_conns);
ADD_STAT(s.cumulative_ICMP_conns);
r->Assign(n++, new Val(killed_by_inactivity, TYPE_COUNT));
return r;
%}
## Returns Bro process statistics. Statistics include real/user/sys CPU time,
## memory usage, page faults, number of TCP/UDP/ICMP connections, timers,
## and events queued/dispatched.
##
## Returns: A record with resource usage statistics.
##
## .. bro:see:: do_profiling
## get_net_stats
## get_matcher_stats
## get_gap_stats
function get_proc_stats%(%): ProcStats
%{
struct rusage ru;
if ( getrusage(RUSAGE_SELF, &ru) < 0 )
reporter->InternalError("getrusage() failed in get_proc_stats()");
RecordVal* r = new RecordVal(ProcStats);
int n = 0;
double elapsed_time = current_time() - bro_start_time;
double user_time =
double(ru.ru_utime.tv_sec) + double(ru.ru_utime.tv_usec) / 1e6;
double system_time =
double(ru.ru_stime.tv_sec) + double(ru.ru_stime.tv_usec) / 1e6;
#ifdef DEBUG
r->Assign(n++, new Val(1, TYPE_COUNT));
#else
r->Assign(n++, new Val(0, TYPE_COUNT));
#endif
r->Assign(n++, new Val(bro_start_time, TYPE_TIME));
r->Assign(n++, new IntervalVal(elapsed_time, Seconds));
r->Assign(n++, new IntervalVal(user_time, Seconds));
r->Assign(n++, new IntervalVal(system_time, Seconds));
uint64 total_mem;
get_memory_usage(&total_mem, NULL);
r->Assign(n++, new Val(unsigned(total_mem), TYPE_COUNT));
r->Assign(n++, new Val(unsigned(ru.ru_minflt), TYPE_COUNT));
r->Assign(n++, new Val(unsigned(ru.ru_majflt), TYPE_COUNT));
r->Assign(n++, new Val(unsigned(ru.ru_nswap), TYPE_COUNT));
r->Assign(n++, new Val(unsigned(ru.ru_inblock), TYPE_COUNT));
r->Assign(n++, new Val(unsigned(ru.ru_oublock), TYPE_COUNT));
r->Assign(n++, new Val(unsigned(ru.ru_nivcsw), TYPE_COUNT));
return r;
%}
function get_event_stats%(%): EventStats
%{
RecordVal* r = new RecordVal(EventStats);
int n = 0;
r->Assign(n++, new Val(num_events_queued, TYPE_COUNT));
r->Assign(n++, new Val(num_events_dispatched, TYPE_COUNT));
return r;
%}
function get_reassembler_stats%(%): ReassemblerStats
%{
RecordVal* r = new RecordVal(ReassemblerStats);
int n = 0;
r->Assign(n++, new Val(Reassembler::MemoryAllocation(REASSEM_FILE), TYPE_COUNT));
r->Assign(n++, new Val(Reassembler::MemoryAllocation(REASSEM_FRAG), TYPE_COUNT));
r->Assign(n++, new Val(Reassembler::MemoryAllocation(REASSEM_TCP), TYPE_COUNT));
r->Assign(n++, new Val(Reassembler::MemoryAllocation(REASSEM_UNKNOWN), TYPE_COUNT));
return r;
%}
function get_dns_stats%(%): DNSStats
%{
RecordVal* r = new RecordVal(DNSStats);
int n = 0;
DNS_Mgr::Stats dstats;
dns_mgr->GetStats(&dstats);
r->Assign(n++, new Val(unsigned(dstats.requests), TYPE_COUNT));
r->Assign(n++, new Val(unsigned(dstats.successful), TYPE_COUNT));
r->Assign(n++, new Val(unsigned(dstats.failed), TYPE_COUNT));
r->Assign(n++, new Val(unsigned(dstats.pending), TYPE_COUNT));
r->Assign(n++, new Val(unsigned(dstats.cached_hosts), TYPE_COUNT));
r->Assign(n++, new Val(unsigned(dstats.cached_addresses), TYPE_COUNT));
return r;
%}
function get_pattern_stats%(%): PatternStats
%{
RecordVal* r = new RecordVal(PatternStats);
int n = 0;
//DFA_State_Cache::Stats stats;
//dfa_state_cache->GetStats(&stats);
//fprintf(f, "Computed dfa_states = %d; Classes = %d; Computed trans. = %d; Uncomputed trans. = %d\n",
// stats.dfa_states, EC()->NumClasses(),
// stats.computed, stats.uncomputed);
//
//fprintf(f, "DFA cache hits = %d; misses = %d\n",
// stats.hits, stats.misses);
return r;
%}
function get_timer_stats%(%): TimerStats
%{
RecordVal* r = new RecordVal(TimerStats);
int n = 0;
r->Assign(n++, new Val(unsigned(timer_mgr->Size()), TYPE_COUNT));
r->Assign(n++, new Val(unsigned(timer_mgr->PeakSize()), TYPE_COUNT));
return r;
%}
function get_file_analysis_stats%(%): FileAnalysisStats
%{
RecordVal* r = new RecordVal(FileAnalysisStats);
int n = 0;
r->Assign(n++, new Val(file_mgr->CurrentFiles(), TYPE_COUNT));
r->Assign(n++, new Val(file_mgr->MaxFiles(), TYPE_COUNT));
r->Assign(n++, new Val(file_mgr->CumulativeFiles(), TYPE_COUNT));
return r;
%}
function get_thread_stats%(%): ThreadStats
%{
RecordVal* r = new RecordVal(ThreadStats);
int n = 0;
r->Assign(n++, new Val(thread_mgr->NumThreads(), TYPE_COUNT));
return r;
%}
## Returns statistics about TCP gaps.
##
## Returns: A record with TCP gap statistics.
##
## .. bro:see:: do_profiling
## get_net_stats
## get_proc_stats
## get_matcher_stats
function get_gap_stats%(%): GapStats
%{
RecordVal* r = new RecordVal(GapStats);
int n = 0;
r->Assign(n++, new Val(tot_ack_events, TYPE_COUNT));
r->Assign(n++, new Val(tot_ack_bytes, TYPE_COUNT));
r->Assign(n++, new Val(tot_gap_events, TYPE_COUNT));
r->Assign(n++, new Val(tot_gap_bytes, TYPE_COUNT));
return r;
%}
## Returns statistics about the regular expression engine. Statistics include
## the number of distinct matchers, DFA states, DFA state transitions, memory
## usage of DFA states, cache hits/misses, and average number of NFA states
## across all matchers.
##
## Returns: A record with matcher statistics.
##
## .. bro:see:: get_net_stats
## get_proc_stats
## get_gap_summary
function get_matcher_stats%(%): MatcherStats
%{
RecordVal* r = new RecordVal(MatcherStats);
int n = 0;
RuleMatcher::Stats s;
memset(&s, 0, sizeof(s));
if ( rule_matcher )
rule_matcher->GetStats(&s);
r->Assign(n++, new Val(s.matchers, TYPE_COUNT));
r->Assign(n++, new Val(s.dfa_states, TYPE_COUNT));
r->Assign(n++, new Val(s.computed, TYPE_COUNT));
r->Assign(n++, new Val(s.mem, TYPE_COUNT));
r->Assign(n++, new Val(s.hits, TYPE_COUNT));
r->Assign(n++, new Val(s.misses, TYPE_COUNT));
r->Assign(n++, new Val(s.avg_nfa_states, TYPE_COUNT));
return r;
%}

View file

@ -1655,9 +1655,9 @@ extern "C" void out_of_memory(const char* where)
abort(); abort();
} }
void get_memory_usage(unsigned int* total, unsigned int* malloced) void get_memory_usage(uint64* total, uint64* malloced)
{ {
unsigned int ret_total; uint64 ret_total;
#ifdef HAVE_MALLINFO #ifdef HAVE_MALLINFO
struct mallinfo mi = mallinfo(); struct mallinfo mi = mallinfo();

View file

@ -502,8 +502,7 @@ inline int safe_vsnprintf(char* str, size_t size, const char* format, va_list al
// Returns total memory allocations and (if available) amount actually // Returns total memory allocations and (if available) amount actually
// handed out by malloc. // handed out by malloc.
extern void get_memory_usage(unsigned int* total, extern void get_memory_usage(uint64* total, uint64* malloced);
unsigned int* malloced);
// Class to be used as a third argument for STL maps to be able to use // Class to be used as a third argument for STL maps to be able to use
// char*'s as keys. Otherwise the pointer values will be compared instead of // char*'s as keys. Otherwise the pointer values will be compared instead of