diff --git a/.github/actions/spelling/allow.txt b/.github/actions/spelling/allow.txt index 66b6b8e1439be..e7fb701996adb 100644 --- a/.github/actions/spelling/allow.txt +++ b/.github/actions/spelling/allow.txt @@ -554,3 +554,5 @@ zst zstandard ZTE Zync +IPFIX +netflow diff --git a/Cargo.lock b/Cargo.lock index faeb44eef1e6d..6896170b9f301 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -12061,6 +12061,7 @@ dependencies = [ "console-subscriber", "criterion", "csv", + "dashmap", "databend-client", "deadpool", "derivative", diff --git a/Cargo.toml b/Cargo.toml index 3b5d6c81c6640..c18682499e17d 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -227,6 +227,9 @@ vector-config-macros.workspace = true vector-vrl-functions = { path = "lib/vector-vrl/functions" } loki-logproto = { path = "lib/loki-logproto", optional = true } +# Performance optimizations +dashmap.workspace = true + # Tokio / Futures async-stream.workspace = true async-trait.workspace = true @@ -614,6 +617,7 @@ sources-logs = [ "sources-file_descriptor", "sources-redis", "sources-socket", + "sources-netflow", "sources-splunk_hec", "sources-stdin", "sources-syslog", @@ -688,6 +692,7 @@ sources-prometheus-pushgateway = ["sinks-prometheus", "sources-utils-http", "vec sources-pulsar = ["dep:apache-avro", "dep:pulsar"] sources-redis = ["dep:redis"] sources-socket = ["sources-utils-net", "tokio-util/net"] +sources-netflow = ["sources-utils-net-udp", "tokio-util/net", "dep:base64"] sources-splunk_hec = ["dep:roaring"] sources-statsd = ["sources-utils-net", "tokio-util/net"] sources-stdin = ["tokio-util/io"] diff --git a/changelog.d/netflow_source.feature.md b/changelog.d/netflow_source.feature.md new file mode 100644 index 0000000000000..034cb56bcc486 --- /dev/null +++ b/changelog.d/netflow_source.feature.md @@ -0,0 +1,3 @@ +Add NetFlow source implementation supporting NetFlow v5, NetFlow v9, IPFIX, and sFlow protocols. The implementation includes sophisticated template management, enterprise field support, template buffering for missing templates, and comprehensive error handling. Supports all major flow protocols with configurable enterprise field parsing and template caching. + +authors: modev2301 diff --git a/config/examples/netflow.yaml b/config/examples/netflow.yaml new file mode 100644 index 0000000000000..ac96d546841b3 --- /dev/null +++ b/config/examples/netflow.yaml @@ -0,0 +1,67 @@ +# NetFlow Source Example +# ------------------------------------------------------------------------------ +# This example demonstrates how to collect NetFlow, IPFIX, and sFlow data +# from network devices and export them to various destinations. +# +# Docs: https://vector.dev/docs/reference/configuration/sources/netflow + +data_dir: "/var/lib/vector" + +# Collect NetFlow data from network devices +# Example: NetFlow v5, v9, IPFIX, and sFlow packets from routers/switches +# Docs: https://vector.dev/docs/reference/configuration/sources/netflow +sources: + netflow_data: + type: "netflow" + address: "0.0.0.0:2055" + protocols: ["netflow_v5", "netflow_v9", "ipfix", "sflow"] + max_packet_size: 65535 + max_templates: 1000 + template_timeout: 1800 + parse_enterprise_fields: true + parse_options_templates: true + parse_variable_length_fields: true + buffer_missing_templates: true + max_buffered_records: 1000 + +# Parse and enrich the flow data +# Docs: https://vector.dev/docs/reference/configuration/transforms/remap +transforms: + flow_parser: + inputs: ["netflow_data"] + type: "remap" + source: | + # Add timestamp if missing + if !exists(.timestamp) { + .timestamp = now() + } + + # Add source information + .source = "netflow" + + # Parse IP addresses for better visualization + if exists(.src_addr) { + .src_ip = parse_ip!(.src_addr) + } + if exists(.dst_addr) { + .dst_ip = parse_ip!(.dst_addr) + } + +# Send to Elasticsearch for analysis +# Docs: https://vector.dev/docs/reference/configuration/sinks/elasticsearch +sinks: + elasticsearch: + inputs: ["flow_parser"] + type: "elasticsearch" + endpoint: "http://localhost:9200" + index: "netflow-%Y-%m-%d" + encoding: + codec: "json" + + # Also send to console for debugging + # Docs: https://vector.dev/docs/reference/configuration/sinks/console + console: + inputs: ["flow_parser"] + type: "console" + encoding: + codec: "json" diff --git a/src/sources/mod.rs b/src/sources/mod.rs index f7b2b6bb534e8..adba6fd849c65 100644 --- a/src/sources/mod.rs +++ b/src/sources/mod.rs @@ -82,6 +82,8 @@ pub mod pulsar; pub mod redis; #[cfg(feature = "sources-socket")] pub mod socket; +#[cfg(feature = "sources-netflow")] +pub mod netflow; #[cfg(feature = "sources-splunk_hec")] pub mod splunk_hec; #[cfg(feature = "sources-static_metrics")] diff --git a/src/sources/netflow/config.rs b/src/sources/netflow/config.rs new file mode 100644 index 0000000000000..0f3092a7c846b --- /dev/null +++ b/src/sources/netflow/config.rs @@ -0,0 +1,408 @@ +//! Configuration for the NetFlow source. + +use serde::{Deserialize, Serialize}; +use std::net::SocketAddr; +use vector_lib::configurable::configurable_component; +use crate::serde::default_true; + + +/// Configuration for the NetFlow source. +#[derive(Clone, Debug)] +#[configurable_component(source("netflow"))] +#[serde(deny_unknown_fields)] +pub struct NetflowConfig { + /// The address to listen for NetFlow packets on. + #[configurable(metadata(docs::examples = "0.0.0.0:2055"))] + #[configurable(metadata(docs::examples = "0.0.0.0:4739"))] + pub address: SocketAddr, + + /// The maximum size of incoming NetFlow packets and field values. + #[configurable(metadata(docs::type_unit = "bytes"))] + #[configurable(metadata(docs::examples = 1500))] + #[configurable(metadata(docs::examples = 65535))] + #[serde(default = "default_max_packet_size")] + pub max_packet_size: usize, + + /// The maximum number of templates to cache per peer. + #[configurable(metadata(docs::examples = 1000))] + #[configurable(metadata(docs::examples = 5000))] + #[serde(default = "default_max_templates")] + pub max_templates: usize, + + /// The timeout for template cache entries in seconds. + #[configurable(metadata(docs::type_unit = "seconds"))] + #[configurable(metadata(docs::examples = 3600))] + #[configurable(metadata(docs::examples = 7200))] + #[serde(default = "default_template_timeout")] + pub template_timeout: u64, + + + /// Protocols to accept (netflow_v5, netflow_v9, ipfix, sflow). + #[configurable(metadata(docs::examples = "protocols"))] + #[configurable(metadata(docs::examples = "netflow_v5"))] + #[configurable(metadata(docs::examples = "netflow_v9"))] + #[configurable(metadata(docs::examples = "ipfix"))] + #[configurable(metadata(docs::examples = "sflow"))] + #[serde(default = "default_protocols")] + pub protocols: Vec, + + /// Whether to parse enterprise fields. + #[configurable(metadata(docs::examples = false))] + #[configurable(metadata(docs::examples = true))] + #[serde(default = "default_true")] + pub parse_enterprise_fields: bool, + + /// Whether to parse options templates. + #[configurable(metadata(docs::examples = false))] + #[configurable(metadata(docs::examples = true))] + #[serde(default = "default_true")] + pub parse_options_templates: bool, + + /// Whether to parse variable length fields. + #[configurable(metadata(docs::examples = true))] + #[configurable(metadata(docs::examples = false))] + #[serde(default = "default_true")] + pub parse_variable_length_fields: bool, + + /// Custom enterprise fields. + #[serde(default)] + pub enterprise_fields: std::collections::HashMap, // Simplified for now + + /// Whether to buffer data records while waiting for templates. + /// When enabled, data records without templates are buffered for up to `template_timeout` seconds. + #[configurable(metadata(docs::examples = true))] + #[configurable(metadata(docs::examples = false))] + #[serde(default = "default_true")] + pub buffer_missing_templates: bool, + + /// Maximum number of data records to buffer per template while waiting for template definition. + #[configurable(metadata(docs::examples = 100))] + #[configurable(metadata(docs::examples = 1000))] + #[serde(default = "default_max_buffered_records")] + pub max_buffered_records: usize, + + /// How to handle Options Template data records (exporter metadata). + /// Options: "emit_metadata" (emit as separate events), "discard" (ignore), "enrich" (use for enrichment only) + #[configurable(metadata(docs::examples = "emit_metadata"))] + #[configurable(metadata(docs::examples = "discard"))] + #[configurable(metadata(docs::examples = "enrich"))] + #[serde(default = "default_options_template_mode")] + pub options_template_mode: String, + + /// Whether to use strict validation for NetFlow v5 records. + /// When disabled, timing anomalies (last < first) are logged as warnings instead of errors. + #[configurable(metadata(docs::examples = true))] + #[configurable(metadata(docs::examples = false))] + #[serde(default = "default_strict_validation")] + pub strict_validation: bool, +} + +/// Supported flow protocols. +#[derive(Clone, Debug, Deserialize, Serialize, PartialEq, Eq)] +#[serde(rename_all = "snake_case")] +pub enum FlowProtocol { + /// NetFlow version 5 (legacy, fixed format) + NetflowV5, + /// NetFlow version 9 (template-based) + NetflowV9, + /// IPFIX (Internet Protocol Flow Information Export) + Ipfix, + /// sFlow (sampled flow) + Sflow, +} + +/// Configuration for enterprise-specific fields. +#[derive(Clone, Debug, Deserialize, Serialize)] +pub struct EnterpriseFieldConfig { + /// Human-readable name for the field. + pub name: String, + /// Data type for parsing the field value. + pub field_type: FieldType, + /// Optional description of the field. + pub description: Option, +} + +/// Supported field data types for enterprise fields. +#[derive(Clone, Debug, Deserialize, Serialize, PartialEq, Eq)] +#[serde(rename_all = "snake_case")] +pub enum FieldType { + /// 8-bit unsigned integer + Uint8, + /// 16-bit unsigned integer + Uint16, + /// 32-bit unsigned integer + Uint32, + /// 64-bit unsigned integer + Uint64, + /// IPv4 address (4 bytes) + Ipv4Address, + /// IPv6 address (16 bytes) + Ipv6Address, + /// MAC address (6 bytes) + MacAddress, + /// UTF-8 string + String, + /// Raw binary data (base64 encoded) + Binary, + /// Boolean value + Boolean, + /// 32-bit floating point + Float32, + /// 64-bit floating point + Float64, +} + +// Default value functions +const fn default_max_packet_size() -> usize { + 65535 // UDP max +} + +const fn default_max_templates() -> usize { + 1000 +} + +const fn default_template_timeout() -> u64 { + 1800 // 30 minutes - matches typical resend intervals +} + +fn default_protocols() -> Vec { + vec!["netflow_v5".to_string(), "netflow_v9".to_string(), "ipfix".to_string(), "sflow".to_string()] +} + +const fn default_max_buffered_records() -> usize { + 1000 +} + +fn default_options_template_mode() -> String { + "emit_metadata".to_string() +} + +const fn default_strict_validation() -> bool { + true +} + + + + +impl Default for NetflowConfig { + fn default() -> Self { + Self { + address: "0.0.0.0:2055".parse().unwrap(), + max_packet_size: default_max_packet_size(), + max_templates: default_max_templates(), + template_timeout: default_template_timeout(), + protocols: default_protocols(), + parse_enterprise_fields: default_true(), + parse_options_templates: default_true(), + parse_variable_length_fields: default_true(), + enterprise_fields: std::collections::HashMap::new(), + buffer_missing_templates: true, + max_buffered_records: default_max_buffered_records(), + options_template_mode: default_options_template_mode(), + strict_validation: default_strict_validation(), + } + } +} + + + +impl NetflowConfig { + /// Validate the configuration. + pub fn validate(&self) -> Result<(), Vec> { + let mut errors = Vec::new(); + + // Validate numeric ranges + if self.max_packet_size == 0 { + errors.push("max_packet_size must be greater than 0".to_string()); + } + if self.max_packet_size > 65535 { + errors.push("max_packet_size cannot exceed 65535 bytes (UDP max)".to_string()); + } + + if self.max_templates == 0 { + errors.push("max_templates must be greater than 0".to_string()); + } + if self.max_templates > 100_000 { + errors.push("max_templates cannot exceed 100,000 (memory usage)".to_string()); + } + + if self.template_timeout == 0 { + errors.push("template_timeout must be greater than 0".to_string()); + } + + if self.max_buffered_records == 0 { + errors.push("max_buffered_records must be greater than 0".to_string()); + } + if self.max_buffered_records > 10000 { + errors.push("max_buffered_records cannot exceed 10,000 (memory usage)".to_string()); + } + + // Validate protocols list + if self.protocols.is_empty() { + errors.push("at least one protocol must be enabled".to_string()); + } + + + + // Validate enterprise field mappings + for (key, field_name) in &self.enterprise_fields { + if !key.contains(':') { + errors.push(format!( + "enterprise field key '{}' must be in format 'enterprise_id:field_id'", + key + )); + continue; + } + + let parts: Vec<&str> = key.split(':').collect(); + if parts.len() != 2 { + errors.push(format!( + "enterprise field key '{}' must have exactly one colon", + key + )); + continue; + } + + if parts[0].parse::().is_err() { + errors.push(format!( + "enterprise field key '{}' has invalid enterprise_id (must be a number)", + key + )); + } + + if parts[1].parse::().is_err() { + errors.push(format!( + "enterprise field key '{}' has invalid field_id (must be a number)", + key + )); + } + + if field_name.is_empty() { + errors.push(format!( + "enterprise field '{}' must have a non-empty name", + key + )); + } + + // Validate field name doesn't conflict with standard fields + if field_name.starts_with("vector_") { + errors.push(format!( + "enterprise field name '{}' cannot start with 'vector_' (reserved)", + field_name + )); + } + } + + if errors.is_empty() { + Ok(()) + } else { + Err(errors) + } + } + + /// Get the enterprise field name for a specific enterprise and field ID. + pub fn get_enterprise_field(&self, enterprise_id: u32, field_id: u16) -> Option<&String> { + let key = format!("{}:{}", enterprise_id, field_id); + self.enterprise_fields.get(&key) + } + + /// Check if a specific protocol is enabled. + pub fn is_protocol_enabled(&self, protocol: &str) -> bool { + self.protocols.contains(&protocol.to_string()) + } +} + +impl crate::config::GenerateConfig for NetflowConfig { + fn generate_config() -> toml::Value { + toml::Value::try_from(Self::default()).unwrap() + } +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn test_default_config_validation() { + let config = NetflowConfig::default(); + assert!(config.validate().is_ok()); + } + + #[test] + fn test_invalid_max_packet_size() { + let mut config = NetflowConfig::default(); + config.max_packet_size = 0; + + let errors = config.validate().unwrap_err(); + assert!(errors.iter().any(|e| e.contains("max_packet_size must be greater than 0"))); + } + + #[test] + fn test_multicast_address_validation() { + // This test is no longer relevant since we removed multicast_groups + // Keeping the test structure for future use + let config = NetflowConfig::default(); + assert!(config.validate().is_ok()); + } + + #[test] + fn test_enterprise_field_validation() { + let mut config = NetflowConfig::default(); + + // Valid enterprise field + config.enterprise_fields.insert( + "9:1001".to_string(), + "cisco_app_id".to_string(), + ); + assert!(config.validate().is_ok()); + + // Invalid key format + config.enterprise_fields.insert( + "invalid_key".to_string(), + "test".to_string(), + ); + + let errors = config.validate().unwrap_err(); + assert!(errors.iter().any(|e| e.contains("must be in format 'enterprise_id:field_id'"))); + } + + #[test] + fn test_protocol_enablement() { + let config = NetflowConfig::default(); + assert!(config.is_protocol_enabled("netflow_v5")); + assert!(config.is_protocol_enabled("ipfix")); + + let mut limited_config = NetflowConfig::default(); + limited_config.protocols = vec!["netflow_v5".to_string()]; + assert!(limited_config.is_protocol_enabled("netflow_v5")); + assert!(!limited_config.is_protocol_enabled("ipfix")); + } + + #[test] + fn test_enterprise_field_lookup() { + let mut config = NetflowConfig::default(); + config.enterprise_fields.insert( + "9:1001".to_string(), + "cisco_app_id".to_string(), + ); + + let field = config.get_enterprise_field(9, 1001); + assert!(field.is_some()); + assert_eq!(field.unwrap(), "cisco_app_id"); + + let missing = config.get_enterprise_field(9, 1002); + assert!(missing.is_none()); + } + + #[test] + fn test_config_serialization() { + let config = NetflowConfig::default(); + let toml_value = toml::Value::try_from(&config).unwrap(); + assert!(toml_value.is_table()); + + let serialized = toml::to_string(&config).unwrap(); + let deserialized: NetflowConfig = toml::from_str(&serialized).unwrap(); + assert_eq!(config.address, deserialized.address); + assert_eq!(config.protocols, deserialized.protocols); + } +} \ No newline at end of file diff --git a/src/sources/netflow/events.rs b/src/sources/netflow/events.rs new file mode 100644 index 0000000000000..3234e5d9b0191 --- /dev/null +++ b/src/sources/netflow/events.rs @@ -0,0 +1,978 @@ +//! Internal events for NetFlow source monitoring and debugging. +//! +//! This module defines all internal events used by the NetFlow source for +//! logging, metrics, and error reporting. These events provide visibility +//! into the source's operation and help with troubleshooting. + +use std::net::SocketAddr; +use tracing::{debug, error, info, warn}; +use vector_lib::internal_event::{error_stage, error_type, ComponentEventsDropped, InternalEvent, UNINTENTIONAL}; +use metrics::counter; + +/// NetFlow packet received successfully +#[derive(Debug)] +pub struct NetflowEventsReceived { + pub count: usize, + pub byte_size: usize, + pub peer_addr: SocketAddr, +} + +impl InternalEvent for NetflowEventsReceived { + fn emit(self) { + debug!( + message = "NetFlow events received", + count = self.count, + byte_size = self.byte_size, + peer_addr = %self.peer_addr, + ); + + counter!( + "component_received_events_total", + "peer_addr" => self.peer_addr.to_string(), + ) + .increment(self.count as u64); + + counter!( + "component_received_event_bytes_total", + "peer_addr" => self.peer_addr.to_string(), + ) + .increment(self.byte_size as u64); + } +} + +/// NetFlow packet parsing failed +#[derive(Debug)] +pub struct NetflowParseError<'a> { + pub error: &'a str, + pub protocol: &'a str, + pub peer_addr: SocketAddr, +} + +impl<'a> InternalEvent for NetflowParseError<'a> { + fn emit(self) { + // Only log as error for critical parsing failures, not missing templates + if self.error.contains("No template") { + debug!( + message = "Template not yet available, data may be buffered", + error = %self.error, + protocol = %self.protocol, + peer_addr = %self.peer_addr, + error_code = "template_missing", + error_type = error_type::PARSER_FAILED, + stage = error_stage::PROCESSING, + internal_log_rate_limit = true, + ); + } else { + error!( + message = "Failed to parse NetFlow packet", + error = %self.error, + protocol = %self.protocol, + peer_addr = %self.peer_addr, + error_code = "parse_failed", + error_type = error_type::PARSER_FAILED, + stage = error_stage::PROCESSING, + internal_log_rate_limit = true, + ); + } + + counter!( + "component_errors_total", + "error_code" => "parse_failed", + "error_type" => error_type::PARSER_FAILED, + "stage" => error_stage::PROCESSING, + "protocol" => self.protocol.to_string(), + "peer_addr" => self.peer_addr.to_string(), + ) + .increment(1); + } +} + +/// Template processing error +#[derive(Debug)] +pub struct NetflowTemplateError<'a> { + pub error: &'a str, + pub template_id: u16, + pub peer_addr: SocketAddr, +} + +impl<'a> InternalEvent for NetflowTemplateError<'a> { + fn emit(self) { + error!( + message = "Failed to process NetFlow template", + error = %self.error, + template_id = self.template_id, + peer_addr = %self.peer_addr, + error_code = "template_error", + error_type = error_type::PARSER_FAILED, + stage = error_stage::PROCESSING, + internal_log_rate_limit = true, + ); + + counter!( + "component_errors_total", + "error_code" => "template_error", + "error_type" => error_type::PARSER_FAILED, + "stage" => error_stage::PROCESSING, + "peer_addr" => self.peer_addr.to_string(), + ) + .increment(1); + } +} + +/// Field parsing error +#[derive(Debug)] +pub struct NetflowFieldParseError<'a> { + pub error: &'a str, + pub field_type: u16, + pub template_id: u16, + pub peer_addr: SocketAddr, +} + +impl<'a> InternalEvent for NetflowFieldParseError<'a> { + fn emit(self) { + error!( + message = "Failed to parse NetFlow field", + error = %self.error, + field_type = self.field_type, + template_id = self.template_id, + peer_addr = %self.peer_addr, + error_code = "field_parse_error", + error_type = error_type::PARSER_FAILED, + stage = error_stage::PROCESSING, + internal_log_rate_limit = true, + ); + + counter!( + "component_errors_total", + "error_code" => "field_parse_error", + "error_type" => error_type::PARSER_FAILED, + "stage" => error_stage::PROCESSING, + "field_type" => self.field_type.to_string(), + "peer_addr" => self.peer_addr.to_string(), + ) + .increment(1); + } +} + +/// Events dropped due to parsing issues +#[derive(Debug)] +pub struct NetflowEventsDropped { + pub count: usize, + pub reason: &'static str, +} + +impl InternalEvent for NetflowEventsDropped { + fn emit(self) { + // Reduce noise for template-related drops when buffering is enabled + if self.reason.contains("No template") { + debug!( + message = "NetFlow events dropped - template not available", + count = self.count, + reason = %self.reason, + error_code = "template_missing", + error_type = error_type::PARSER_FAILED, + stage = error_stage::PROCESSING, + internal_log_rate_limit = true, + ); + } else { + warn!( + message = "NetFlow events dropped", + count = self.count, + reason = %self.reason, + error_code = "events_dropped", + error_type = error_type::PARSER_FAILED, + stage = error_stage::PROCESSING, + ); + } + + counter!( + "component_errors_total", + "error_code" => "events_dropped", + "error_type" => error_type::PARSER_FAILED, + "stage" => error_stage::PROCESSING, + "reason" => self.reason.to_string(), + ) + .increment(1); + + emit!(ComponentEventsDropped:: { + count: self.count, + reason: self.reason, + }); + } +} + +/// Socket binding error +#[derive(Debug)] +pub struct NetflowBindError { + pub address: SocketAddr, + pub error: std::io::Error, +} + +impl InternalEvent for NetflowBindError { + fn emit(self) { + error!( + message = "Failed to bind NetFlow socket", + address = %self.address, + error = %self.error, + error_code = "socket_bind_failed", + error_type = error_type::CONNECTION_FAILED, + stage = error_stage::RECEIVING, + ); + + counter!( + "component_errors_total", + "error_code" => "socket_bind_failed", + "error_type" => error_type::CONNECTION_FAILED, + "stage" => error_stage::RECEIVING, + "address" => self.address.to_string(), + ) + .increment(1); + } +} + +/// Socket receive error +#[derive(Debug)] +pub struct NetflowReceiveError { + pub error: std::io::Error, +} + +impl InternalEvent for NetflowReceiveError { + fn emit(self) { + error!( + message = "Failed to receive NetFlow packet", + error = %self.error, + error_code = "socket_receive_failed", + error_type = error_type::CONNECTION_FAILED, + stage = error_stage::RECEIVING, + internal_log_rate_limit = true, + ); + + counter!( + "component_errors_total", + "error_code" => "socket_receive_failed", + "error_type" => error_type::CONNECTION_FAILED, + "stage" => error_stage::RECEIVING, + ) + .increment(1); + } +} + +/// Multicast group join error +#[derive(Debug)] +pub struct NetflowMulticastJoinError { + pub group: std::net::Ipv4Addr, + pub interface: std::net::Ipv4Addr, + pub error: std::io::Error, +} + +impl InternalEvent for NetflowMulticastJoinError { + fn emit(self) { + error!( + message = "Failed to join multicast group", + group = %self.group, + interface = %self.interface, + error = %self.error, + error_code = "multicast_join_failed", + error_type = error_type::CONNECTION_FAILED, + stage = error_stage::RECEIVING, + ); + + counter!( + "component_errors_total", + "error_code" => "multicast_join_failed", + "error_type" => error_type::CONNECTION_FAILED, + "stage" => error_stage::RECEIVING, + "group" => self.group.to_string(), + ) + .increment(1); + } +} + +/// Template cache cleanup completed +#[derive(Debug)] +pub struct TemplateCleanupCompleted { + pub removed_count: usize, + pub timeout_seconds: u64, +} + +impl InternalEvent for TemplateCleanupCompleted { + fn emit(self) { + if self.removed_count > 0 { + info!( + message = "Template cache cleanup completed", + removed_count = self.removed_count, + timeout_seconds = self.timeout_seconds, + ); + } else { + debug!( + message = "Template cache cleanup completed, no expired templates", + timeout_seconds = self.timeout_seconds, + ); + } + + counter!( + "netflow_template_cache_cleanups_total", + ) + .increment(1); + + counter!( + "netflow_template_cache_expired_total", + ) + .increment(self.removed_count as u64); + } +} + +/// Template received and cached +#[derive(Debug)] +pub struct TemplateReceived { + pub template_id: u16, + pub field_count: u16, + pub peer_addr: SocketAddr, + pub observation_domain_id: u32, + pub protocol: &'static str, +} + +/// Buffered records processed when template becomes available +#[derive(Debug)] +pub struct BufferedRecordsProcessed { + pub template_id: u16, + pub record_count: usize, + pub peer_addr: SocketAddr, + pub observation_domain_id: u32, +} + +impl InternalEvent for TemplateReceived { + fn emit(self) { + debug!( + message = "Template received and cached", + template_id = self.template_id, + field_count = self.field_count, + peer_addr = %self.peer_addr, + observation_domain_id = self.observation_domain_id, + protocol = self.protocol, + ); + + counter!( + "netflow_templates_received_total", + "protocol" => self.protocol.to_string(), + "peer_addr" => self.peer_addr.to_string(), + ) + .increment(1); + } +} + +impl InternalEvent for BufferedRecordsProcessed { + fn emit(self) { + debug!( + message = "Buffered records processed with new template", + template_id = self.template_id, + record_count = self.record_count, + peer_addr = %self.peer_addr, + observation_domain_id = self.observation_domain_id, + ); + + counter!( + "netflow_buffered_records_processed_total", + "template_id" => self.template_id.to_string(), + "peer_addr" => self.peer_addr.to_string(), + ) + .increment(self.record_count as u64); + } +} + +/// Template cache statistics +#[derive(Debug)] +pub struct TemplateCacheStats { + pub cache_size: usize, + pub max_size: usize, + pub hit_ratio: f64, + pub total_hits: u64, + pub total_misses: u64, +} + +impl InternalEvent for TemplateCacheStats { + fn emit(self) { + debug!( + message = "Template cache statistics", + cache_size = self.cache_size, + max_size = self.max_size, + hit_ratio = self.hit_ratio, + total_hits = self.total_hits, + total_misses = self.total_misses, + ); + + metrics::gauge!( + "netflow_template_cache_size", + ) + .set(self.cache_size as f64); + + metrics::gauge!( + "netflow_template_cache_hit_ratio", + ) + .set(self.hit_ratio); + } +} + +/// Data record parsed successfully +#[derive(Debug)] +pub struct DataRecordParsed { + pub template_id: u16, + pub fields_parsed: usize, + pub record_size: usize, + pub peer_addr: SocketAddr, + pub protocol: &'static str, +} + +impl InternalEvent for DataRecordParsed { + fn emit(self) { + debug!( + message = "Data record parsed successfully", + template_id = self.template_id, + fields_parsed = self.fields_parsed, + record_size = self.record_size, + peer_addr = %self.peer_addr, + protocol = self.protocol, + ); + + counter!( + "netflow_records_parsed_total", + "protocol" => self.protocol.to_string(), + "peer_addr" => self.peer_addr.to_string(), + ) + .increment(1); + + counter!( + "netflow_fields_parsed_total", + "protocol" => self.protocol.to_string(), + ) + .increment(self.fields_parsed as u64); + } +} + +/// Enterprise field encountered +#[derive(Debug)] +pub struct EnterpriseFieldEncountered { + pub enterprise_id: u32, + pub field_type: u16, + pub field_name: String, + pub peer_addr: SocketAddr, +} + +impl InternalEvent for EnterpriseFieldEncountered { + fn emit(self) { + debug!( + message = "Enterprise field encountered", + enterprise_id = self.enterprise_id, + field_type = self.field_type, + field_name = %self.field_name, + peer_addr = %self.peer_addr, + ); + + counter!( + "netflow_enterprise_fields_total", + "enterprise_id" => self.enterprise_id.to_string(), + "field_name" => self.field_name, + ) + .increment(1); + } +} + +/// Unknown enterprise field encountered +#[derive(Debug)] +pub struct UnknownEnterpriseField { + pub enterprise_id: u32, + pub field_type: u16, + pub peer_addr: SocketAddr, +} + +impl InternalEvent for UnknownEnterpriseField { + fn emit(self) { + warn!( + message = "Unknown enterprise field encountered", + enterprise_id = self.enterprise_id, + field_type = self.field_type, + peer_addr = %self.peer_addr, + internal_log_rate_limit = true, + ); + + counter!( + "netflow_unknown_enterprise_fields_total", + "enterprise_id" => self.enterprise_id.to_string(), + ) + .increment(1); + } +} + +/// Protocol version mismatch +#[derive(Debug)] +pub struct ProtocolVersionMismatch { + pub expected: u16, + pub received: u16, + pub peer_addr: SocketAddr, +} + +impl InternalEvent for ProtocolVersionMismatch { + fn emit(self) { + warn!( + message = "Protocol version mismatch", + expected = self.expected, + received = self.received, + peer_addr = %self.peer_addr, + error_code = "version_mismatch", + error_type = error_type::PARSER_FAILED, + stage = error_stage::PROCESSING, + ); + + counter!( + "component_errors_total", + "error_code" => "version_mismatch", + "error_type" => error_type::PARSER_FAILED, + "stage" => error_stage::PROCESSING, + "expected_version" => self.expected.to_string(), + "received_version" => self.received.to_string(), + ) + .increment(1); + } +} + +/// Packet too large error +#[derive(Debug)] +pub struct PacketTooLarge { + pub packet_size: usize, + pub max_size: usize, + pub peer_addr: SocketAddr, +} + +impl InternalEvent for PacketTooLarge { + fn emit(self) { + warn!( + message = "Packet too large, truncating", + packet_size = self.packet_size, + max_size = self.max_size, + peer_addr = %self.peer_addr, + error_code = "packet_too_large", + error_type = error_type::PARSER_FAILED, + stage = error_stage::PROCESSING, + ); + + counter!( + "component_errors_total", + "error_code" => "packet_too_large", + "error_type" => error_type::PARSER_FAILED, + "stage" => error_stage::PROCESSING, + ) + .increment(1); + } +} + +/// Source startup information +#[derive(Debug)] +pub struct NetflowSourceStarted { + pub address: SocketAddr, + pub protocols: Vec, + pub max_templates: usize, + pub template_timeout: u64, +} + +impl InternalEvent for NetflowSourceStarted { + fn emit(self) { + info!( + message = "NetFlow source started", + address = %self.address, + protocols = ?self.protocols, + max_templates = self.max_templates, + template_timeout = self.template_timeout, + ); + + counter!( + "netflow_source_starts_total", + ) + .increment(1); + } +} + +/// Source shutdown information +#[derive(Debug)] +pub struct NetflowSourceStopped { + pub address: SocketAddr, + pub runtime_seconds: u64, +} + +impl InternalEvent for NetflowSourceStopped { + fn emit(self) { + info!( + message = "NetFlow source stopped", + address = %self.address, + runtime_seconds = self.runtime_seconds, + ); + + counter!( + "netflow_source_stops_total", + ) + .increment(1); + } +} + +/// Configuration validation error +#[derive(Debug)] +pub struct ConfigValidationError { + pub errors: Vec, +} + +impl InternalEvent for ConfigValidationError { + fn emit(self) { + error!( + message = "NetFlow configuration validation failed", + errors = ?self.errors, + error_code = "config_validation_failed", + error_type = error_type::CONFIGURATION_FAILED, + stage = error_stage::RECEIVING, + ); + + counter!( + "component_errors_total", + "error_code" => "config_validation_failed", + "error_type" => error_type::CONFIGURATION_FAILED, + "stage" => error_stage::RECEIVING, + ) + .increment(1); + } +} + +/// Variable-length field parsing +#[derive(Debug)] +pub struct VariableLengthFieldParsed { + pub field_type: u16, + pub actual_length: usize, + pub template_id: u16, + pub peer_addr: SocketAddr, +} + +impl InternalEvent for VariableLengthFieldParsed { + fn emit(self) { + debug!( + message = "Variable-length field parsed", + field_type = self.field_type, + actual_length = self.actual_length, + template_id = self.template_id, + peer_addr = %self.peer_addr, + ); + + counter!( + "netflow_variable_length_fields_total", + ) + .increment(1); + } +} + +/// Flow statistics summary +#[derive(Debug)] +pub struct FlowStatsSummary { + pub total_flows: u64, + pub total_packets: u64, + pub total_bytes: u64, + pub unique_peers: usize, + pub active_templates: usize, +} + +impl InternalEvent for FlowStatsSummary { + fn emit(self) { + info!( + message = "Flow statistics summary", + total_flows = self.total_flows, + total_packets = self.total_packets, + total_bytes = self.total_bytes, + unique_peers = self.unique_peers, + active_templates = self.active_templates, + ); + + metrics::gauge!( + "netflow_total_flows", + ) + .set(self.total_flows as f64); + + metrics::gauge!( + "netflow_unique_peers", + ) + .set(self.unique_peers as f64); + + metrics::gauge!( + "netflow_active_templates", + ) + .set(self.active_templates as f64); + } +} + +/// Protocol-specific events +#[derive(Debug)] +pub struct ProtocolSpecificEvent { + pub protocol: &'static str, + pub event_type: &'static str, + pub details: String, + pub peer_addr: SocketAddr, +} + +impl InternalEvent for ProtocolSpecificEvent { + fn emit(self) { + debug!( + message = "Protocol-specific event", + protocol = self.protocol, + event_type = self.event_type, + details = %self.details, + peer_addr = %self.peer_addr, + ); + + counter!( + "netflow_protocol_events_total", + "protocol" => self.protocol.to_string(), + "event_type" => self.event_type.to_string(), + ) + .increment(1); + } +} + +/// Memory usage warning +#[derive(Debug)] +pub struct MemoryUsageWarning { + pub component: &'static str, + pub current_usage: usize, + pub threshold: usize, +} + +impl InternalEvent for MemoryUsageWarning { + fn emit(self) { + warn!( + message = "Memory usage warning", + component = self.component, + current_usage = self.current_usage, + threshold = self.threshold, + error_code = "memory_usage_high", + error_type = error_type::CONFIGURATION_FAILED, + stage = error_stage::PROCESSING, + ); + + counter!( + "component_errors_total", + "error_code" => "memory_usage_high", + "error_type" => error_type::CONFIGURATION_FAILED, + "stage" => error_stage::PROCESSING, + "component" => self.component.to_string(), + ) + .increment(1); + } +} + +#[cfg(test)] +mod tests { + use super::*; + use std::net::{IpAddr, Ipv4Addr}; + + fn test_peer_addr() -> SocketAddr { + SocketAddr::new(IpAddr::V4(Ipv4Addr::new(192, 168, 1, 100)), 2055) + } + + #[test] + fn test_events_can_be_created() { + // Test that all events can be instantiated without panicking + let peer_addr = test_peer_addr(); + + let _events = vec![ + Box::new(NetflowEventsReceived { + count: 10, + byte_size: 1500, + peer_addr, + }) as Box, + + Box::new(NetflowParseError { + error: "test error", + protocol: "netflow_v5", + peer_addr, + }), + + Box::new(NetflowTemplateError { + error: "template error", + template_id: 256, + peer_addr, + }), + + Box::new(NetflowFieldParseError { + error: "field error", + field_type: 1, + template_id: 256, + peer_addr, + }), + + Box::new(NetflowEventsDropped { + count: 5, + reason: "no template", + }), + + Box::new(TemplateReceived { + template_id: 256, + field_count: 10, + peer_addr, + observation_domain_id: 1, + protocol: "ipfix", + }), + + Box::new(EnterpriseFieldEncountered { + enterprise_id: 23867, + field_type: 1, + field_name: "clientIPv4Address".to_string(), + peer_addr, + }), + ]; + + // If we get here without panicking, the test passes + assert!(!_events.is_empty()); + } + + #[test] + fn test_event_emission() { + // Test that events can be emitted without panicking + let peer_addr = test_peer_addr(); + + // These should not panic + NetflowEventsReceived { + count: 1, + byte_size: 100, + peer_addr, + }.emit(); + + TemplateCleanupCompleted { + removed_count: 5, + timeout_seconds: 3600, + }.emit(); + + FlowStatsSummary { + total_flows: 1000, + total_packets: 50000, + total_bytes: 1000000, + unique_peers: 10, + active_templates: 50, + }.emit(); + } + + #[test] + fn test_error_events() { + let peer_addr = test_peer_addr(); + + // Test error events + NetflowParseError { + error: "Invalid packet format", + protocol: "netflow_v9", + peer_addr, + }.emit(); + + NetflowTemplateError { + error: "Template field count mismatch", + template_id: 512, + peer_addr, + }.emit(); + + ProtocolVersionMismatch { + expected: 9, + received: 5, + peer_addr, + }.emit(); + } + + #[test] + fn test_statistics_events() { + let peer_addr = test_peer_addr(); + + TemplateCacheStats { + cache_size: 100, + max_size: 1000, + hit_ratio: 0.95, + total_hits: 950, + total_misses: 50, + }.emit(); + + DataRecordParsed { + template_id: 256, + fields_parsed: 15, + record_size: 60, + peer_addr, + protocol: "ipfix", + }.emit(); + } + + #[test] + fn test_lifecycle_events() { + let address = test_peer_addr(); + + NetflowSourceStarted { + address, + protocols: vec!["netflow_v5".to_string(), "ipfix".to_string()], + max_templates: 1000, + template_timeout: 3600, + }.emit(); + + NetflowSourceStopped { + address, + runtime_seconds: 86400, + }.emit(); + } + + #[test] + fn test_configuration_events() { + ConfigValidationError { + errors: vec![ + "max_packet_size must be > 0".to_string(), + "invalid protocol specified".to_string(), + ], + }.emit(); + } + + #[test] + fn test_enterprise_field_events() { + let peer_addr = test_peer_addr(); + + EnterpriseFieldEncountered { + enterprise_id: 9, + field_type: 1001, + field_name: "cisco_application_id".to_string(), + peer_addr, + }.emit(); + + UnknownEnterpriseField { + enterprise_id: 99999, + field_type: 5000, + peer_addr, + }.emit(); + } + + #[test] + fn test_variable_length_field_event() { + let peer_addr = test_peer_addr(); + + VariableLengthFieldParsed { + field_type: 82, + actual_length: 256, + template_id: 512, + peer_addr, + }.emit(); + } + + #[test] + fn test_memory_warning_event() { + MemoryUsageWarning { + component: "template_cache", + current_usage: 104857600, // 100MB + threshold: 83886080, // 80MB + }.emit(); + } + + #[test] + fn test_protocol_specific_event() { + let peer_addr = test_peer_addr(); + + ProtocolSpecificEvent { + protocol: "sflow", + event_type: "sample_rate_change", + details: "Sample rate changed from 1000 to 2000".to_string(), + peer_addr, + }.emit(); + } +} \ No newline at end of file diff --git a/src/sources/netflow/fields.rs b/src/sources/netflow/fields.rs new file mode 100644 index 0000000000000..7f58141021c10 --- /dev/null +++ b/src/sources/netflow/fields.rs @@ -0,0 +1,1444 @@ +//! Field parsing and enterprise field definitions for NetFlow/IPFIX. +//! +//! This module provides unified field parsing logic with support for: +//! - Standard IPFIX fields +//! - NetFlow v9 fields +//! - Enterprise-specific fields (HPE Aruba, Cisco, etc.) +//! - Custom user-defined enterprise fields + +use crate::sources::netflow::config::{FieldType, NetflowConfig}; +use crate::sources::netflow::events::*; +use crate::sources::netflow::templates::TemplateField; + +use std::collections::HashMap; +use std::sync::OnceLock; +use base64::Engine; +use vector_lib::event::LogEvent; + +use vector_lib::event::Value; + +/// Field information including name and data type. +#[derive(Debug, Clone)] +pub struct FieldInfo { + pub name: &'static str, + pub data_type: DataType, + pub description: &'static str, +} + +/// Supported data types for field parsing. +#[derive(Debug, Clone, PartialEq, Eq)] +pub enum DataType { + UInt8, + UInt16, + UInt32, + UInt64, + Int8, + Int16, + Int32, + Int64, + Float32, + Float64, + Boolean, + MacAddress, + Ipv4Address, + Ipv6Address, + DateTimeSeconds, + DateTimeMilliseconds, + DateTimeMicroseconds, + DateTimeNanoseconds, + String, + Binary, +} + +impl DataType { + /// Parse field data into a Value for insertion into LogEvent. + pub fn parse(&self, data: &[u8], max_length: usize) -> Result { + match self { + DataType::UInt8 => { + if data.len() >= 1 { + Ok(Value::Integer(data[0] as i64)) + } else { + Err("Insufficient data for UInt8".to_string()) + } + } + DataType::UInt16 => { + if data.len() >= 2 { + let value = u16::from_be_bytes([data[0], data[1]]); + Ok(Value::Integer(value as i64)) + } else { + Err("Insufficient data for UInt16".to_string()) + } + } + DataType::UInt32 => { + if data.len() >= 4 { + let value = u32::from_be_bytes([data[0], data[1], data[2], data[3]]); + Ok(Value::Integer(value as i64)) + } else { + Err("Insufficient data for UInt32".to_string()) + } + } + DataType::UInt64 => { + if data.len() >= 8 { + let value = u64::from_be_bytes([ + data[0], data[1], data[2], data[3], + data[4], data[5], data[6], data[7] + ]); + // Convert to i64, clamping if necessary + if value <= i64::MAX as u64 { + Ok(Value::Integer(value as i64)) + } else { + Ok(Value::Integer(i64::MAX)) + } + } else { + Err("Insufficient data for UInt64".to_string()) + } + } + DataType::Int32 => { + if data.len() >= 4 { + let value = i32::from_be_bytes([data[0], data[1], data[2], data[3]]); + Ok(Value::Integer(value as i64)) + } else { + Err("Insufficient data for Int32".to_string()) + } + } + DataType::Float32 => { + if data.len() >= 4 { + let bits = u32::from_be_bytes([data[0], data[1], data[2], data[3]]); + let value = f32::from_bits(bits); + Ok(Value::Float(ordered_float::NotNan::new(value as f64).unwrap())) + } else { + Err("Insufficient data for Float32".to_string()) + } + } + DataType::Float64 => { + if data.len() >= 8 { + let bits = u64::from_be_bytes([ + data[0], data[1], data[2], data[3], + data[4], data[5], data[6], data[7] + ]); + let value = f64::from_bits(bits); + Ok(Value::Float(ordered_float::NotNan::new(value).unwrap())) + } else { + Err("Insufficient data for Float64".to_string()) + } + } + DataType::Boolean => { + if data.len() >= 1 { + Ok(Value::Boolean(data[0] != 0)) + } else { + Err("Insufficient data for Boolean".to_string()) + } + } + DataType::MacAddress => { + if data.len() >= 6 { + let mac = format!( + "{:02x}:{:02x}:{:02x}:{:02x}:{:02x}:{:02x}", + data[0], data[1], data[2], data[3], data[4], data[5] + ); + Ok(Value::Bytes(mac.into())) + } else { + Err("Insufficient data for MAC address".to_string()) + } + } + DataType::Ipv4Address => { + if data.len() >= 4 { + let addr = format!( + "{}.{}.{}.{}", + data[0], data[1], data[2], data[3] + ); + Ok(Value::Bytes(addr.into())) + } else { + Err("Insufficient data for IPv4 address".to_string()) + } + } + DataType::Ipv6Address => { + if data.len() >= 16 { + let addr = format!( + "{:02x}{:02x}:{:02x}{:02x}:{:02x}{:02x}:{:02x}{:02x}:{:02x}{:02x}:{:02x}{:02x}:{:02x}{:02x}:{:02x}{:02x}", + data[0], data[1], data[2], data[3], data[4], data[5], data[6], data[7], + data[8], data[9], data[10], data[11], data[12], data[13], data[14], data[15] + ); + Ok(Value::Bytes(addr.into())) + } else { + Err("Insufficient data for IPv6 address".to_string()) + } + } + DataType::DateTimeSeconds => { + if data.len() >= 4 { + let timestamp = u32::from_be_bytes([data[0], data[1], data[2], data[3]]); + Ok(Value::Integer(timestamp as i64)) + } else { + Err("Insufficient data for DateTime seconds".to_string()) + } + } + DataType::DateTimeMilliseconds => { + if data.len() >= 8 { + let timestamp = u64::from_be_bytes([ + data[0], data[1], data[2], data[3], + data[4], data[5], data[6], data[7] + ]); + Ok(Value::Integer(timestamp as i64)) + } else { + Err("Insufficient data for DateTime milliseconds".to_string()) + } + } + DataType::String => { + if data.is_empty() { + Ok(Value::Bytes("".into())) + } else { + match std::str::from_utf8(data) { + Ok(s) => { + let clean_str = s.trim_matches('\0').trim(); + let truncated = if clean_str.len() > max_length { + format!("{}...", &clean_str[..max_length.saturating_sub(3)]) + } else { + clean_str.to_string() + }; + Ok(Value::Bytes(truncated.into())) + } + Err(_) => { + // Fallback to binary if not valid UTF-8 + let encoded = base64::engine::general_purpose::STANDARD.encode(data); + let truncated = if encoded.len() > max_length { + format!("{}...", &encoded[..max_length.saturating_sub(3)]) + } else { + encoded + }; + Ok(Value::Bytes(truncated.into())) + } + } + } + } + DataType::Binary => { + if data.is_empty() { + Ok(Value::Bytes("".into())) + } else { + let encoded = base64::engine::general_purpose::STANDARD.encode(data); + let truncated = if encoded.len() > max_length { + format!("{}...", &encoded[..max_length.saturating_sub(3)]) + } else { + encoded + }; + Ok(Value::Bytes(truncated.into())) + } + } + // Add other integer types + DataType::Int8 => { + if data.len() >= 1 { + let value = data[0] as i8; + Ok(Value::Integer(value as i64)) + } else { + Err("Insufficient data for Int8".to_string()) + } + } + DataType::Int16 => { + if data.len() >= 2 { + let value = i16::from_be_bytes([data[0], data[1]]); + Ok(Value::Integer(value as i64)) + } else { + Err("Insufficient data for Int16".to_string()) + } + } + DataType::Int64 => { + if data.len() >= 8 { + let value = i64::from_be_bytes([ + data[0], data[1], data[2], data[3], + data[4], data[5], data[6], data[7] + ]); + Ok(Value::Integer(value)) + } else { + Err("Insufficient data for Int64".to_string()) + } + } + DataType::DateTimeMicroseconds | DataType::DateTimeNanoseconds => { + // Handle as UInt64 timestamp + if data.len() >= 8 { + let timestamp = u64::from_be_bytes([ + data[0], data[1], data[2], data[3], + data[4], data[5], data[6], data[7] + ]); + Ok(Value::Integer(timestamp as i64)) + } else { + Err("Insufficient data for DateTime".to_string()) + } + } + } + } +} + +impl From for DataType { + fn from(field_type: FieldType) -> Self { + match field_type { + FieldType::Uint8 => DataType::UInt8, + FieldType::Uint16 => DataType::UInt16, + FieldType::Uint32 => DataType::UInt32, + FieldType::Uint64 => DataType::UInt64, + FieldType::Ipv4Address => DataType::Ipv4Address, + FieldType::Ipv6Address => DataType::Ipv6Address, + FieldType::MacAddress => DataType::MacAddress, + FieldType::String => DataType::String, + FieldType::Binary => DataType::Binary, + FieldType::Boolean => DataType::Boolean, + FieldType::Float32 => DataType::Float32, + FieldType::Float64 => DataType::Float64, + } + } +} + +/// Field parser that handles all field types and enterprise extensions. +#[derive(Clone)] +pub struct FieldParser { + max_field_length: usize, + resolve_protocols: bool, + custom_enterprise_fields: HashMap<(u32, u16), (String, DataType)>, // (enterprise_id, field_id) -> (name, type) +} + +impl FieldParser { + /// Create a new field parser with the given configuration. + pub fn new(config: &NetflowConfig) -> Self { + let mut custom_enterprise_fields = HashMap::new(); + + // Load custom enterprise field mappings from config + for (key, field_name) in &config.enterprise_fields { + if let Some((enterprise_str, field_str)) = key.split_once(':') { + if let (Ok(enterprise_id), Ok(field_id)) = (enterprise_str.parse::(), field_str.parse::()) { + // Try to infer data type from field name or default to UInt32 for numeric fields + let data_type = if field_name.contains("address") || field_name.contains("Address") { + DataType::Ipv4Address + } else if field_name.contains("port") || field_name.contains("Port") { + DataType::UInt16 + } else if field_name.contains("count") || field_name.contains("Count") || + field_name.contains("number") || field_name.contains("Number") || + field_name.contains("id") || field_name.contains("Id") { + DataType::UInt32 + } else { + DataType::UInt32 // Default to UInt32 for most custom fields + }; + + custom_enterprise_fields.insert( + (enterprise_id, field_id), + (field_name.clone(), data_type) + ); + } + } + } + + Self { + max_field_length: config.max_packet_size, + resolve_protocols: true, // Always resolve protocols for better readability + custom_enterprise_fields, + } + } + + /// Parse a field and insert it into the log event. + pub fn parse_field(&self, field: &TemplateField, data: &[u8], log_event: &mut LogEvent) { + let field_info = self.get_field_info(field); + + + match field_info.data_type.parse(data, self.max_field_length) { + Ok(value) => { + log_event.insert(field_info.name, value.clone()); + + // Add protocol name resolution for protocol fields + if self.resolve_protocols && field_info.name == "protocolIdentifier" { + if let Value::Integer(protocol_num) = &value { + let protocol_name = get_protocol_name(*protocol_num as u8); + log_event.insert("protocolName", Value::Bytes(protocol_name.into())); + } + } + } + Err(error) => { + emit!(NetflowFieldParseError { + error: &error, + field_type: field.field_type, + template_id: 0, // Would need to pass this through + peer_addr: std::net::SocketAddr::new( + std::net::IpAddr::V4(std::net::Ipv4Addr::UNSPECIFIED), + 0 + ), + }); + + // Insert raw data as fallback + let encoded = base64::engine::general_purpose::STANDARD.encode(data); + let truncated = if encoded.len() > self.max_field_length { + format!("{}...", &encoded[..self.max_field_length.saturating_sub(3)]) + } else { + encoded + }; + log_event.insert(field_info.name, Value::Bytes(truncated.into())); + } + } + } + + /// Get field information for a template field. + fn get_field_info(&self, field: &TemplateField) -> FieldInfo { + match field.enterprise_number { + Some(enterprise_id) => { + // Check custom enterprise fields first + if let Some((name, data_type)) = self.custom_enterprise_fields.get(&(enterprise_id, field.field_type)) { + return FieldInfo { + name: Box::leak(name.clone().into_boxed_str()), + data_type: data_type.clone(), + description: "Custom enterprise field", + }; + } + + // Check known enterprise fields + match enterprise_id { + 23867 => self.get_hpe_aruba_field(field.field_type), + 9 => self.get_cisco_field(field.field_type), + 2636 => self.get_juniper_field(field.field_type), + _ => self.get_unknown_enterprise_field(enterprise_id, field.field_type), + } + } + None => self.get_standard_field(field.field_type), + } + } + + /// Get standard IPFIX/NetFlow field information. + fn get_standard_field(&self, field_type: u16) -> FieldInfo { + STANDARD_FIELDS.get_or_init(init_standard_fields) + .get(&field_type) + .cloned() + .unwrap_or_else(|| FieldInfo { + name: Box::leak(format!("unknown_field_{}", field_type).into_boxed_str()), + data_type: DataType::Binary, + description: "Unknown standard field", + }) + } + + /// Get HPE Aruba enterprise field information. + fn get_hpe_aruba_field(&self, field_type: u16) -> FieldInfo { + HPE_ARUBA_FIELDS.get_or_init(init_hpe_aruba_fields) + .get(&field_type) + .cloned() + .unwrap_or_else(|| FieldInfo { + name: Box::leak(format!("hpe_aruba_field_{}", field_type).into_boxed_str()), + data_type: DataType::Binary, + description: "Unknown HPE Aruba field", + }) + } + + /// Get Cisco enterprise field information. + fn get_cisco_field(&self, field_type: u16) -> FieldInfo { + CISCO_FIELDS.get_or_init(init_cisco_fields) + .get(&field_type) + .cloned() + .unwrap_or_else(|| FieldInfo { + name: Box::leak(format!("cisco_field_{}", field_type).into_boxed_str()), + data_type: DataType::Binary, + description: "Unknown Cisco field", + }) + } + + /// Get Juniper enterprise field information. + fn get_juniper_field(&self, field_type: u16) -> FieldInfo { + JUNIPER_FIELDS.get_or_init(init_juniper_fields) + .get(&field_type) + .cloned() + .unwrap_or_else(|| FieldInfo { + name: Box::leak(format!("juniper_field_{}", field_type).into_boxed_str()), + data_type: DataType::Binary, + description: "Unknown Juniper field", + }) + } + + /// Get unknown enterprise field information. + fn get_unknown_enterprise_field(&self, enterprise_id: u32, field_type: u16) -> FieldInfo { + FieldInfo { + name: Box::leak(format!("enterprise_{}_{}", enterprise_id, field_type).into_boxed_str()), + data_type: DataType::Binary, + description: "Unknown enterprise field", + } + } +} + +// Static field registries using OnceLock for thread-safe lazy initialization +static STANDARD_FIELDS: OnceLock> = OnceLock::new(); +static HPE_ARUBA_FIELDS: OnceLock> = OnceLock::new(); +static CISCO_FIELDS: OnceLock> = OnceLock::new(); +static JUNIPER_FIELDS: OnceLock> = OnceLock::new(); + +/// Initialize standard IPFIX field definitions. +fn init_standard_fields() -> HashMap { + let mut map = HashMap::new(); + + map.insert(1, FieldInfo { name: "octetDeltaCount", data_type: DataType::UInt64, description: "The number of octets since the" }); + map.insert(2, FieldInfo { name: "packetDeltaCount", data_type: DataType::UInt64, description: "The number of incoming packets since" }); + map.insert(3, FieldInfo { name: "deltaFlowCount", data_type: DataType::UInt64, description: "The conservative count of Original Flows" }); + map.insert(4, FieldInfo { name: "protocolIdentifier", data_type: DataType::UInt8, description: "The value of the protocol number" }); + map.insert(5, FieldInfo { name: "ipClassOfService", data_type: DataType::UInt8, description: "For IPv4 packets, this is the" }); + map.insert(6, FieldInfo { name: "tcpControlBits", data_type: DataType::UInt16, description: "TCP control bits observed for the" }); + map.insert(7, FieldInfo { name: "sourceTransportPort", data_type: DataType::UInt16, description: "The source port identifier in the" }); + map.insert(8, FieldInfo { name: "sourceIPv4Address", data_type: DataType::Ipv4Address, description: "The IPv4 source address in the" }); + map.insert(9, FieldInfo { name: "sourceIPv4PrefixLength", data_type: DataType::UInt8, description: "The number of contiguous bits that" }); + map.insert(10, FieldInfo { name: "ingressInterface", data_type: DataType::UInt32, description: "The index of the IP interface" }); + map.insert(11, FieldInfo { name: "destinationTransportPort", data_type: DataType::UInt16, description: "The destination port identifier in the" }); + map.insert(12, FieldInfo { name: "destinationIPv4Address", data_type: DataType::Ipv4Address, description: "The IPv4 destination address in the" }); + map.insert(13, FieldInfo { name: "destinationIPv4PrefixLength", data_type: DataType::UInt8, description: "The number of contiguous bits that" }); + map.insert(14, FieldInfo { name: "egressInterface", data_type: DataType::UInt32, description: "The index of the IP interface" }); + map.insert(15, FieldInfo { name: "ipNextHopIPv4Address", data_type: DataType::Ipv4Address, description: "The IPv4 address of the next" }); + map.insert(16, FieldInfo { name: "bgpSourceAsNumber", data_type: DataType::UInt32, description: "The autonomous system (AS) number of" }); + map.insert(17, FieldInfo { name: "bgpDestinationAsNumber", data_type: DataType::UInt32, description: "The autonomous system (AS) number of" }); + map.insert(18, FieldInfo { name: "bgpNextHopIPv4Address", data_type: DataType::Ipv4Address, description: "The IPv4 address of the next" }); + map.insert(19, FieldInfo { name: "postMCastPacketDeltaCount", data_type: DataType::UInt64, description: "The number of outgoing multicast packets" }); + map.insert(20, FieldInfo { name: "postMCastOctetDeltaCount", data_type: DataType::UInt64, description: "The number of octets since the" }); + map.insert(21, FieldInfo { name: "flowEndSysUpTime", data_type: DataType::UInt32, description: "The relative timestamp of the last" }); + map.insert(22, FieldInfo { name: "flowStartSysUpTime", data_type: DataType::UInt32, description: "The relative timestamp of the first" }); + map.insert(23, FieldInfo { name: "postOctetDeltaCount", data_type: DataType::UInt64, description: "The definition of this Information Element" }); + map.insert(24, FieldInfo { name: "postPacketDeltaCount", data_type: DataType::UInt64, description: "The definition of this Information Element" }); + map.insert(25, FieldInfo { name: "minimumIpTotalLength", data_type: DataType::UInt64, description: "Length of the smallest packet observed" }); + map.insert(26, FieldInfo { name: "maximumIpTotalLength", data_type: DataType::UInt64, description: "Length of the largest packet observed" }); + map.insert(27, FieldInfo { name: "sourceIPv6Address", data_type: DataType::Ipv6Address, description: "The IPv6 source address in the" }); + map.insert(28, FieldInfo { name: "destinationIPv6Address", data_type: DataType::Ipv6Address, description: "The IPv6 destination address in the" }); + map.insert(29, FieldInfo { name: "sourceIPv6PrefixLength", data_type: DataType::UInt8, description: "The number of contiguous bits that" }); + map.insert(30, FieldInfo { name: "destinationIPv6PrefixLength", data_type: DataType::UInt8, description: "The number of contiguous bits that" }); + map.insert(31, FieldInfo { name: "flowLabelIPv6", data_type: DataType::UInt32, description: "The value of the IPv6 Flow" }); + map.insert(32, FieldInfo { name: "icmpTypeCodeIPv4", data_type: DataType::UInt16, description: "Type and Code of the IPv4" }); + map.insert(33, FieldInfo { name: "igmpType", data_type: DataType::UInt8, description: "The type field of the IGMP" }); + map.insert(34, FieldInfo { name: "samplingInterval", data_type: DataType::UInt32, description: "Deprecated in favor of 305 samplingPacketInterval." }); + map.insert(35, FieldInfo { name: "samplingAlgorithm", data_type: DataType::UInt8, description: "Deprecated in favor of 304 selectorAlgorithm." }); + map.insert(36, FieldInfo { name: "flowActiveTimeout", data_type: DataType::UInt16, description: "The number of seconds after which" }); + map.insert(37, FieldInfo { name: "flowIdleTimeout", data_type: DataType::UInt16, description: "A Flow is considered to be" }); + map.insert(38, FieldInfo { name: "engineType", data_type: DataType::UInt8, description: "Type of flow switching engine in" }); + map.insert(39, FieldInfo { name: "engineId", data_type: DataType::UInt8, description: "Versatile Interface Processor (VIP) or line" }); + map.insert(40, FieldInfo { name: "exportedOctetTotalCount", data_type: DataType::UInt64, description: "The total number of octets that" }); + map.insert(41, FieldInfo { name: "exportedMessageTotalCount", data_type: DataType::UInt64, description: "The total number of IPFIX Messages" }); + map.insert(42, FieldInfo { name: "exportedFlowRecordTotalCount", data_type: DataType::UInt64, description: "The total number of Flow Records" }); + map.insert(43, FieldInfo { name: "ipv4RouterSc", data_type: DataType::Ipv4Address, description: "This is a platform-specific field for" }); + map.insert(44, FieldInfo { name: "sourceIPv4Prefix", data_type: DataType::Ipv4Address, description: "IPv4 source address prefix." }); + map.insert(45, FieldInfo { name: "destinationIPv4Prefix", data_type: DataType::Ipv4Address, description: "IPv4 destination address prefix." }); + map.insert(46, FieldInfo { name: "mplsTopLabelType", data_type: DataType::UInt8, description: "This field identifies the control protocol" }); + map.insert(47, FieldInfo { name: "mplsTopLabelIPv4Address", data_type: DataType::Ipv4Address, description: "The IPv4 address of the system" }); + map.insert(48, FieldInfo { name: "samplerId", data_type: DataType::UInt8, description: "Deprecated in favor of 302 selectorId." }); + map.insert(49, FieldInfo { name: "samplerMode", data_type: DataType::UInt8, description: "Deprecated in favor of 304 selectorAlgorithm." }); + map.insert(50, FieldInfo { name: "samplerRandomInterval", data_type: DataType::UInt32, description: "Deprecated in favor of 305 samplingPacketInterval." }); + map.insert(51, FieldInfo { name: "classId", data_type: DataType::UInt8, description: "Deprecated in favor of 302 selectorId." }); + map.insert(52, FieldInfo { name: "minimumTTL", data_type: DataType::UInt8, description: "Minimum TTL value observed for any" }); + map.insert(53, FieldInfo { name: "maximumTTL", data_type: DataType::UInt8, description: "Maximum TTL value observed for any" }); + map.insert(54, FieldInfo { name: "fragmentIdentification", data_type: DataType::UInt32, description: "The value of the Identification field" }); + map.insert(55, FieldInfo { name: "postIpClassOfService", data_type: DataType::UInt8, description: "The definition of this Information Element" }); + map.insert(56, FieldInfo { name: "sourceMacAddress", data_type: DataType::MacAddress, description: "The IEEE 802 source MAC address" }); + map.insert(57, FieldInfo { name: "postDestinationMacAddress", data_type: DataType::MacAddress, description: "The definition of this Information Element" }); + map.insert(58, FieldInfo { name: "vlanId", data_type: DataType::UInt16, description: "Virtual LAN identifier associated with ingress" }); + map.insert(59, FieldInfo { name: "postVlanId", data_type: DataType::UInt16, description: "Virtual LAN identifier associated with egress" }); + map.insert(60, FieldInfo { name: "ipVersion", data_type: DataType::UInt8, description: "The IP version field in the" }); + map.insert(61, FieldInfo { name: "flowDirection", data_type: DataType::UInt8, description: "The direction of the Flow observed" }); + map.insert(62, FieldInfo { name: "ipNextHopIPv6Address", data_type: DataType::Ipv6Address, description: "The IPv6 address of the next" }); + map.insert(63, FieldInfo { name: "bgpNextHopIPv6Address", data_type: DataType::Ipv6Address, description: "The IPv6 address of the next" }); + map.insert(64, FieldInfo { name: "ipv6ExtensionHeaders", data_type: DataType::UInt32, description: "Deprecated in favor of the ipv6ExtensionHeadersFull" }); + map.insert(70, FieldInfo { name: "mplsTopLabelStackSection", data_type: DataType::Binary, description: "The Label, Exp, and S fields" }); + map.insert(71, FieldInfo { name: "mplsLabelStackSection2", data_type: DataType::Binary, description: "The Label, Exp, and S fields" }); + map.insert(72, FieldInfo { name: "mplsLabelStackSection3", data_type: DataType::Binary, description: "The Label, Exp, and S fields" }); + map.insert(73, FieldInfo { name: "mplsLabelStackSection4", data_type: DataType::Binary, description: "The Label, Exp, and S fields" }); + map.insert(74, FieldInfo { name: "mplsLabelStackSection5", data_type: DataType::Binary, description: "The Label, Exp, and S fields" }); + map.insert(75, FieldInfo { name: "mplsLabelStackSection6", data_type: DataType::Binary, description: "The Label, Exp, and S fields" }); + map.insert(76, FieldInfo { name: "mplsLabelStackSection7", data_type: DataType::Binary, description: "The Label, Exp, and S fields" }); + map.insert(77, FieldInfo { name: "mplsLabelStackSection8", data_type: DataType::Binary, description: "The Label, Exp, and S fields" }); + map.insert(78, FieldInfo { name: "mplsLabelStackSection9", data_type: DataType::Binary, description: "The Label, Exp, and S fields" }); + map.insert(79, FieldInfo { name: "mplsLabelStackSection10", data_type: DataType::Binary, description: "The Label, Exp, and S fields" }); + map.insert(80, FieldInfo { name: "destinationMacAddress", data_type: DataType::MacAddress, description: "The IEEE 802 destination MAC address" }); + map.insert(81, FieldInfo { name: "postSourceMacAddress", data_type: DataType::MacAddress, description: "The definition of this Information Element" }); + map.insert(82, FieldInfo { name: "interfaceName", data_type: DataType::String, description: "A short name uniquely describing an" }); + map.insert(83, FieldInfo { name: "interfaceDescription", data_type: DataType::String, description: "The description of an interface, eg" }); + map.insert(84, FieldInfo { name: "samplerName", data_type: DataType::String, description: "Deprecated in favor of 335 selectorName." }); + map.insert(85, FieldInfo { name: "octetTotalCount", data_type: DataType::UInt64, description: "The total number of octets in" }); + map.insert(86, FieldInfo { name: "packetTotalCount", data_type: DataType::UInt64, description: "The total number of incoming packets" }); + map.insert(87, FieldInfo { name: "flagsAndSamplerId", data_type: DataType::UInt32, description: "Flow flags and the value of" }); + map.insert(88, FieldInfo { name: "fragmentOffset", data_type: DataType::UInt16, description: "The value of the IP fragment" }); + map.insert(89, FieldInfo { name: "forwardingStatus", data_type: DataType::UInt8, description: "This Information Element describes the forwarding" }); + map.insert(90, FieldInfo { name: "mplsVpnRouteDistinguisher", data_type: DataType::Binary, description: "The value of the VPN route" }); + map.insert(91, FieldInfo { name: "mplsTopLabelPrefixLength", data_type: DataType::UInt8, description: "The prefix length of the subnet" }); + map.insert(92, FieldInfo { name: "srcTrafficIndex", data_type: DataType::UInt32, description: "BGP Policy Accounting Source Traffic Index." }); + map.insert(93, FieldInfo { name: "dstTrafficIndex", data_type: DataType::UInt32, description: "BGP Policy Accounting Destination Traffic Index." }); + map.insert(94, FieldInfo { name: "applicationDescription", data_type: DataType::String, description: "Specifies the description of an application." }); + map.insert(95, FieldInfo { name: "applicationId", data_type: DataType::Binary, description: "Specifies an Application ID per [RFC6759]." }); + map.insert(96, FieldInfo { name: "applicationName", data_type: DataType::String, description: "Specifies the name of an application." }); + map.insert(97, FieldInfo { name: "Assigned for NetFlow v9 compatibility", data_type: DataType::String, description: "" }); + map.insert(98, FieldInfo { name: "postIpDiffServCodePoint", data_type: DataType::UInt8, description: "The definition of this Information Element" }); + map.insert(99, FieldInfo { name: "multicastReplicationFactor", data_type: DataType::UInt32, description: "The amount of multicast replication that's" }); + map.insert(100, FieldInfo { name: "className", data_type: DataType::String, description: "Deprecated in favor of 335 selectorName." }); + map.insert(101, FieldInfo { name: "classificationEngineId", data_type: DataType::UInt8, description: "A unique identifier for the engine" }); + map.insert(102, FieldInfo { name: "layer2packetSectionOffset", data_type: DataType::UInt16, description: "Deprecated in favor of 409 sectionOffset." }); + map.insert(103, FieldInfo { name: "layer2packetSectionSize", data_type: DataType::UInt16, description: "Deprecated in favor of 312 dataLinkFrameSize." }); + map.insert(104, FieldInfo { name: "layer2packetSectionData", data_type: DataType::Binary, description: "Deprecated in favor of 315 dataLinkFrameSection." }); + map.insert(128, FieldInfo { name: "bgpNextAdjacentAsNumber", data_type: DataType::UInt32, description: "The autonomous system (AS) number of" }); + map.insert(129, FieldInfo { name: "bgpPrevAdjacentAsNumber", data_type: DataType::UInt32, description: "The autonomous system (AS) number of" }); + map.insert(130, FieldInfo { name: "exporterIPv4Address", data_type: DataType::Ipv4Address, description: "The IPv4 address used by the" }); + map.insert(131, FieldInfo { name: "exporterIPv6Address", data_type: DataType::Ipv6Address, description: "The IPv6 address used by the" }); + map.insert(132, FieldInfo { name: "droppedOctetDeltaCount", data_type: DataType::UInt64, description: "The number of octets since the" }); + map.insert(133, FieldInfo { name: "droppedPacketDeltaCount", data_type: DataType::UInt64, description: "The number of packets since the" }); + map.insert(134, FieldInfo { name: "droppedOctetTotalCount", data_type: DataType::UInt64, description: "The total number of octets in" }); + map.insert(135, FieldInfo { name: "droppedPacketTotalCount", data_type: DataType::UInt64, description: "The number of packets of this" }); + map.insert(136, FieldInfo { name: "flowEndReason", data_type: DataType::UInt8, description: "The reason for Flow termination. Values" }); + map.insert(137, FieldInfo { name: "commonPropertiesId", data_type: DataType::UInt64, description: "An identifier of a set of" }); + map.insert(138, FieldInfo { name: "observationPointId", data_type: DataType::UInt64, description: "An identifier of an Observation Point" }); + map.insert(139, FieldInfo { name: "icmpTypeCodeIPv6", data_type: DataType::UInt16, description: "Type and Code of the IPv6" }); + map.insert(140, FieldInfo { name: "mplsTopLabelIPv6Address", data_type: DataType::Ipv6Address, description: "The IPv6 address of the system" }); + map.insert(141, FieldInfo { name: "lineCardId", data_type: DataType::UInt32, description: "An identifier of a line card" }); + map.insert(142, FieldInfo { name: "portId", data_type: DataType::UInt32, description: "An identifier of a line port" }); + map.insert(143, FieldInfo { name: "meteringProcessId", data_type: DataType::UInt32, description: "An identifier of a Metering Process" }); + map.insert(144, FieldInfo { name: "exportingProcessId", data_type: DataType::UInt32, description: "An identifier of an Exporting Process" }); + map.insert(145, FieldInfo { name: "templateId", data_type: DataType::UInt16, description: "An identifier of a Template that" }); + map.insert(146, FieldInfo { name: "wlanChannelId", data_type: DataType::UInt8, description: "The identifier of the 802.11 (Wi-Fi)" }); + map.insert(147, FieldInfo { name: "wlanSSID", data_type: DataType::String, description: "The Service Set IDentifier (SSID) identifying" }); + map.insert(148, FieldInfo { name: "flowId", data_type: DataType::UInt64, description: "An identifier of a Flow that" }); + map.insert(149, FieldInfo { name: "observationDomainId", data_type: DataType::UInt32, description: "An identifier of an Observation Domain" }); + map.insert(150, FieldInfo { name: "flowStartSeconds", data_type: DataType::DateTimeSeconds, description: "The absolute timestamp of the first" }); + map.insert(151, FieldInfo { name: "flowEndSeconds", data_type: DataType::DateTimeSeconds, description: "The absolute timestamp of the last" }); + map.insert(152, FieldInfo { name: "flowStartMilliseconds", data_type: DataType::DateTimeMilliseconds, description: "The absolute timestamp of the first" }); + map.insert(153, FieldInfo { name: "flowEndMilliseconds", data_type: DataType::DateTimeMilliseconds, description: "The absolute timestamp of the last" }); + map.insert(154, FieldInfo { name: "flowStartMicroseconds", data_type: DataType::DateTimeMicroseconds, description: "The absolute timestamp of the first" }); + map.insert(155, FieldInfo { name: "flowEndMicroseconds", data_type: DataType::DateTimeMicroseconds, description: "The absolute timestamp of the last" }); + map.insert(156, FieldInfo { name: "flowStartNanoseconds", data_type: DataType::DateTimeNanoseconds, description: "The absolute timestamp of the first" }); + map.insert(157, FieldInfo { name: "flowEndNanoseconds", data_type: DataType::DateTimeNanoseconds, description: "The absolute timestamp of the last" }); + map.insert(158, FieldInfo { name: "flowStartDeltaMicroseconds", data_type: DataType::UInt32, description: "This is a relative timestamp only" }); + map.insert(159, FieldInfo { name: "flowEndDeltaMicroseconds", data_type: DataType::UInt32, description: "This is a relative timestamp only" }); + map.insert(160, FieldInfo { name: "systemInitTimeMilliseconds", data_type: DataType::DateTimeMilliseconds, description: "The absolute timestamp of the last" }); + map.insert(161, FieldInfo { name: "flowDurationMilliseconds", data_type: DataType::UInt32, description: "The difference in time between the" }); + map.insert(162, FieldInfo { name: "flowDurationMicroseconds", data_type: DataType::UInt32, description: "The difference in time between the" }); + map.insert(163, FieldInfo { name: "observedFlowTotalCount", data_type: DataType::UInt64, description: "The total number of Flows observed" }); + map.insert(164, FieldInfo { name: "ignoredPacketTotalCount", data_type: DataType::UInt64, description: "The total number of observed IP" }); + map.insert(165, FieldInfo { name: "ignoredOctetTotalCount", data_type: DataType::UInt64, description: "The total number of octets in" }); + map.insert(166, FieldInfo { name: "notSentFlowTotalCount", data_type: DataType::UInt64, description: "The total number of Flow Records" }); + map.insert(167, FieldInfo { name: "notSentPacketTotalCount", data_type: DataType::UInt64, description: "The total number of packets in" }); + map.insert(168, FieldInfo { name: "notSentOctetTotalCount", data_type: DataType::UInt64, description: "The total number of octets in" }); + map.insert(169, FieldInfo { name: "destinationIPv6Prefix", data_type: DataType::Ipv6Address, description: "IPv6 destination address prefix." }); + map.insert(170, FieldInfo { name: "sourceIPv6Prefix", data_type: DataType::Ipv6Address, description: "IPv6 source address prefix." }); + map.insert(171, FieldInfo { name: "postOctetTotalCount", data_type: DataType::UInt64, description: "The definition of this Information Element" }); + map.insert(172, FieldInfo { name: "postPacketTotalCount", data_type: DataType::UInt64, description: "The definition of this Information Element" }); + map.insert(173, FieldInfo { name: "flowKeyIndicator", data_type: DataType::UInt64, description: "This set of bit fields is" }); + map.insert(174, FieldInfo { name: "postMCastPacketTotalCount", data_type: DataType::UInt64, description: "The total number of outgoing multicast" }); + map.insert(175, FieldInfo { name: "postMCastOctetTotalCount", data_type: DataType::UInt64, description: "The total number of octets in" }); + map.insert(176, FieldInfo { name: "icmpTypeIPv4", data_type: DataType::UInt8, description: "Type of the IPv4 ICMP message." }); + map.insert(177, FieldInfo { name: "icmpCodeIPv4", data_type: DataType::UInt8, description: "Code of the IPv4 ICMP message." }); + map.insert(178, FieldInfo { name: "icmpTypeIPv6", data_type: DataType::UInt8, description: "Type of the IPv6 ICMP message." }); + map.insert(179, FieldInfo { name: "icmpCodeIPv6", data_type: DataType::UInt8, description: "Code of the IPv6 ICMP message." }); + map.insert(180, FieldInfo { name: "udpSourcePort", data_type: DataType::UInt16, description: "The source port identifier in the" }); + map.insert(181, FieldInfo { name: "udpDestinationPort", data_type: DataType::UInt16, description: "The destination port identifier in the" }); + map.insert(182, FieldInfo { name: "tcpSourcePort", data_type: DataType::UInt16, description: "The source port identifier in the" }); + map.insert(183, FieldInfo { name: "tcpDestinationPort", data_type: DataType::UInt16, description: "The destination port identifier in the" }); + map.insert(184, FieldInfo { name: "tcpSequenceNumber", data_type: DataType::UInt32, description: "The sequence number in the TCP" }); + map.insert(185, FieldInfo { name: "tcpAcknowledgementNumber", data_type: DataType::UInt32, description: "The acknowledgement number in the TCP" }); + map.insert(186, FieldInfo { name: "tcpWindowSize", data_type: DataType::UInt16, description: "The window field in the TCP" }); + map.insert(187, FieldInfo { name: "tcpUrgentPointer", data_type: DataType::UInt16, description: "The urgent pointer in the TCP" }); + map.insert(188, FieldInfo { name: "tcpHeaderLength", data_type: DataType::UInt8, description: "The length of the TCP header." }); + map.insert(189, FieldInfo { name: "ipHeaderLength", data_type: DataType::UInt8, description: "The length of the IP header." }); + map.insert(190, FieldInfo { name: "totalLengthIPv4", data_type: DataType::UInt16, description: "The total length of the IPv4" }); + map.insert(191, FieldInfo { name: "payloadLengthIPv6", data_type: DataType::UInt16, description: "This Information Element reports the value" }); + map.insert(192, FieldInfo { name: "ipTTL", data_type: DataType::UInt8, description: "For IPv4, the value of the" }); + map.insert(193, FieldInfo { name: "nextHeaderIPv6", data_type: DataType::UInt8, description: "The value of the Next Header" }); + map.insert(194, FieldInfo { name: "mplsPayloadLength", data_type: DataType::UInt32, description: "The size of the MPLS packet" }); + map.insert(195, FieldInfo { name: "ipDiffServCodePoint", data_type: DataType::UInt8, description: "The value of a Differentiated Services" }); + map.insert(196, FieldInfo { name: "ipPrecedence", data_type: DataType::UInt8, description: "The value of the IP Precedence." }); + map.insert(197, FieldInfo { name: "fragmentFlags", data_type: DataType::UInt8, description: "Fragmentation properties indicated by flags in" }); + map.insert(198, FieldInfo { name: "octetDeltaSumOfSquares", data_type: DataType::UInt64, description: "The sum of the squared numbers" }); + map.insert(199, FieldInfo { name: "octetTotalSumOfSquares", data_type: DataType::UInt64, description: "The total sum of the squared" }); + map.insert(200, FieldInfo { name: "mplsTopLabelTTL", data_type: DataType::UInt8, description: "The TTL field from the top" }); + map.insert(201, FieldInfo { name: "mplsLabelStackLength", data_type: DataType::UInt32, description: "The length of the MPLS label" }); + map.insert(202, FieldInfo { name: "mplsLabelStackDepth", data_type: DataType::UInt32, description: "The number of labels in the" }); + map.insert(203, FieldInfo { name: "mplsTopLabelExp", data_type: DataType::UInt8, description: "The Exp field from the top" }); + map.insert(204, FieldInfo { name: "ipPayloadLength", data_type: DataType::UInt32, description: "The effective length of the IP" }); + map.insert(205, FieldInfo { name: "udpMessageLength", data_type: DataType::UInt16, description: "The value of the Length field" }); + map.insert(206, FieldInfo { name: "isMulticast", data_type: DataType::UInt8, description: "If the IP destination address is" }); + map.insert(207, FieldInfo { name: "ipv4IHL", data_type: DataType::UInt8, description: "The value of the Internet Header" }); + map.insert(208, FieldInfo { name: "ipv4Options", data_type: DataType::UInt32, description: "IPv4 options in packets of this" }); + map.insert(209, FieldInfo { name: "tcpOptions", data_type: DataType::UInt64, description: "Deprecated in favor of the tcpOptionsFull" }); + map.insert(210, FieldInfo { name: "paddingOctets", data_type: DataType::Binary, description: "The value of this Information Element" }); + map.insert(211, FieldInfo { name: "collectorIPv4Address", data_type: DataType::Ipv4Address, description: "An IPv4 address to which the" }); + map.insert(212, FieldInfo { name: "collectorIPv6Address", data_type: DataType::Ipv6Address, description: "An IPv6 address to which the" }); + map.insert(213, FieldInfo { name: "exportInterface", data_type: DataType::UInt32, description: "The index of the interface from" }); + map.insert(214, FieldInfo { name: "exportProtocolVersion", data_type: DataType::UInt8, description: "The protocol version used by the" }); + map.insert(215, FieldInfo { name: "exportTransportProtocol", data_type: DataType::UInt8, description: "The value of the protocol number" }); + map.insert(216, FieldInfo { name: "collectorTransportPort", data_type: DataType::UInt16, description: "The destination port identifier to which" }); + map.insert(217, FieldInfo { name: "exporterTransportPort", data_type: DataType::UInt16, description: "The source port identifier from which" }); + map.insert(218, FieldInfo { name: "tcpSynTotalCount", data_type: DataType::UInt64, description: "The total number of packets of" }); + map.insert(219, FieldInfo { name: "tcpFinTotalCount", data_type: DataType::UInt64, description: "The total number of packets of" }); + map.insert(220, FieldInfo { name: "tcpRstTotalCount", data_type: DataType::UInt64, description: "The total number of packets of" }); + map.insert(221, FieldInfo { name: "tcpPshTotalCount", data_type: DataType::UInt64, description: "The total number of packets of" }); + map.insert(222, FieldInfo { name: "tcpAckTotalCount", data_type: DataType::UInt64, description: "The total number of packets of" }); + map.insert(223, FieldInfo { name: "tcpUrgTotalCount", data_type: DataType::UInt64, description: "The total number of packets of" }); + map.insert(224, FieldInfo { name: "ipTotalLength", data_type: DataType::UInt64, description: "The total length of the IP" }); + map.insert(225, FieldInfo { name: "postNATSourceIPv4Address", data_type: DataType::Ipv4Address, description: "The definition of this Information Element" }); + map.insert(226, FieldInfo { name: "postNATDestinationIPv4Address", data_type: DataType::Ipv4Address, description: "The definition of this Information Element" }); + map.insert(227, FieldInfo { name: "postNAPTSourceTransportPort", data_type: DataType::UInt16, description: "The definition of this Information Element" }); + map.insert(228, FieldInfo { name: "postNAPTDestinationTransportPort", data_type: DataType::UInt16, description: "The definition of this Information Element" }); + map.insert(229, FieldInfo { name: "natOriginatingAddressRealm", data_type: DataType::UInt8, description: "Indicates whether the session was created" }); + map.insert(230, FieldInfo { name: "natEvent", data_type: DataType::UInt8, description: "This Information Element identifies a NAT" }); + map.insert(231, FieldInfo { name: "initiatorOctets", data_type: DataType::UInt64, description: "The total number of layer 4" }); + map.insert(232, FieldInfo { name: "responderOctets", data_type: DataType::UInt64, description: "The total number of layer 4" }); + map.insert(233, FieldInfo { name: "firewallEvent", data_type: DataType::UInt8, description: "Indicates a firewall event. Allowed values" }); + map.insert(234, FieldInfo { name: "ingressVRFID", data_type: DataType::UInt32, description: "An unique identifier of the VRFname" }); + map.insert(235, FieldInfo { name: "egressVRFID", data_type: DataType::UInt32, description: "An unique identifier of the VRFname" }); + map.insert(236, FieldInfo { name: "VRFname", data_type: DataType::String, description: "The name of a VPN Routing" }); + map.insert(237, FieldInfo { name: "postMplsTopLabelExp", data_type: DataType::UInt8, description: "The definition of this Information Element" }); + map.insert(238, FieldInfo { name: "tcpWindowScale", data_type: DataType::UInt16, description: "The scale of the window field" }); + map.insert(239, FieldInfo { name: "biflowDirection", data_type: DataType::UInt8, description: "A description of the direction assignment" }); + map.insert(240, FieldInfo { name: "ethernetHeaderLength", data_type: DataType::UInt8, description: "The difference between the length of" }); + map.insert(241, FieldInfo { name: "ethernetPayloadLength", data_type: DataType::UInt16, description: "The length of the MAC Client" }); + map.insert(242, FieldInfo { name: "ethernetTotalLength", data_type: DataType::UInt16, description: "The total length of the Ethernet" }); + map.insert(243, FieldInfo { name: "dot1qVlanId", data_type: DataType::UInt16, description: "The value of the 12-bit VLAN" }); + map.insert(244, FieldInfo { name: "dot1qPriority", data_type: DataType::UInt8, description: "The value of the 3-bit User" }); + map.insert(245, FieldInfo { name: "dot1qCustomerVlanId", data_type: DataType::UInt16, description: "The value represents the Customer VLAN" }); + map.insert(246, FieldInfo { name: "dot1qCustomerPriority", data_type: DataType::UInt8, description: "The value represents the 3-bit Priority" }); + map.insert(247, FieldInfo { name: "metroEvcId", data_type: DataType::String, description: "The EVC Service Attribute which uniquely" }); + map.insert(248, FieldInfo { name: "metroEvcType", data_type: DataType::UInt8, description: "The 3-bit EVC Service Attribute which" }); + map.insert(249, FieldInfo { name: "pseudoWireId", data_type: DataType::UInt32, description: "A 32-bit non-zero connection identifier, which" }); + map.insert(250, FieldInfo { name: "pseudoWireType", data_type: DataType::UInt16, description: "The value of this information element" }); + map.insert(251, FieldInfo { name: "pseudoWireControlWord", data_type: DataType::UInt32, description: "The 32-bit Preferred Pseudo Wire (PW)" }); + map.insert(252, FieldInfo { name: "ingressPhysicalInterface", data_type: DataType::UInt32, description: "The index of a networking device's" }); + map.insert(253, FieldInfo { name: "egressPhysicalInterface", data_type: DataType::UInt32, description: "The index of a networking device's" }); + map.insert(254, FieldInfo { name: "postDot1qVlanId", data_type: DataType::UInt16, description: "The definition of this Information Element" }); + map.insert(255, FieldInfo { name: "postDot1qCustomerVlanId", data_type: DataType::UInt16, description: "The definition of this Information Element" }); + map.insert(256, FieldInfo { name: "ethernetType", data_type: DataType::UInt16, description: "The Ethernet type field of an" }); + map.insert(257, FieldInfo { name: "postIpPrecedence", data_type: DataType::UInt8, description: "The definition of this Information Element" }); + map.insert(258, FieldInfo { name: "collectionTimeMilliseconds", data_type: DataType::DateTimeMilliseconds, description: "The absolute timestamp at which the" }); + map.insert(259, FieldInfo { name: "exportSctpStreamId", data_type: DataType::UInt16, description: "The value of the SCTP Stream" }); + map.insert(260, FieldInfo { name: "maxExportSeconds", data_type: DataType::DateTimeSeconds, description: "The absolute Export Time of the" }); + map.insert(261, FieldInfo { name: "maxFlowEndSeconds", data_type: DataType::DateTimeSeconds, description: "The latest absolute timestamp of the" }); + map.insert(262, FieldInfo { name: "messageMD5Checksum", data_type: DataType::Binary, description: "The MD5 checksum of the IPFIX" }); + map.insert(263, FieldInfo { name: "messageScope", data_type: DataType::UInt8, description: "The presence of this Information Element" }); + map.insert(264, FieldInfo { name: "minExportSeconds", data_type: DataType::DateTimeSeconds, description: "The absolute Export Time of the" }); + map.insert(265, FieldInfo { name: "minFlowStartSeconds", data_type: DataType::DateTimeSeconds, description: "The earliest absolute timestamp of the" }); + map.insert(266, FieldInfo { name: "opaqueOctets", data_type: DataType::Binary, description: "This Information Element is used to" }); + map.insert(267, FieldInfo { name: "sessionScope", data_type: DataType::UInt8, description: "The presence of this Information Element" }); + map.insert(268, FieldInfo { name: "maxFlowEndMicroseconds", data_type: DataType::DateTimeMicroseconds, description: "The latest absolute timestamp of the" }); + map.insert(269, FieldInfo { name: "maxFlowEndMilliseconds", data_type: DataType::DateTimeMilliseconds, description: "The latest absolute timestamp of the" }); + map.insert(270, FieldInfo { name: "maxFlowEndNanoseconds", data_type: DataType::DateTimeNanoseconds, description: "The latest absolute timestamp of the" }); + map.insert(271, FieldInfo { name: "minFlowStartMicroseconds", data_type: DataType::DateTimeMicroseconds, description: "The earliest absolute timestamp of the" }); + map.insert(272, FieldInfo { name: "minFlowStartMilliseconds", data_type: DataType::DateTimeMilliseconds, description: "The earliest absolute timestamp of the" }); + map.insert(273, FieldInfo { name: "minFlowStartNanoseconds", data_type: DataType::DateTimeNanoseconds, description: "The earliest absolute timestamp of the" }); + map.insert(274, FieldInfo { name: "collectorCertificate", data_type: DataType::Binary, description: "The full X.509 certificate, encoded in" }); + map.insert(275, FieldInfo { name: "exporterCertificate", data_type: DataType::Binary, description: "The full X.509 certificate, encoded in" }); + map.insert(276, FieldInfo { name: "dataRecordsReliability", data_type: DataType::Boolean, description: "The export reliability of Data Records," }); + map.insert(277, FieldInfo { name: "observationPointType", data_type: DataType::UInt8, description: "Type of observation point. Values are" }); + map.insert(278, FieldInfo { name: "newConnectionDeltaCount", data_type: DataType::UInt32, description: "This information element counts the number" }); + map.insert(279, FieldInfo { name: "connectionSumDurationSeconds", data_type: DataType::UInt64, description: "This information element aggregates the total" }); + map.insert(280, FieldInfo { name: "connectionTransactionId", data_type: DataType::UInt64, description: "This information element identifies a transaction" }); + map.insert(281, FieldInfo { name: "postNATSourceIPv6Address", data_type: DataType::Ipv6Address, description: "The definition of this Information Element" }); + map.insert(282, FieldInfo { name: "postNATDestinationIPv6Address", data_type: DataType::Ipv6Address, description: "The definition of this Information Element" }); + map.insert(283, FieldInfo { name: "natPoolId", data_type: DataType::UInt32, description: "Locally unique identifier of a NAT" }); + map.insert(284, FieldInfo { name: "natPoolName", data_type: DataType::String, description: "The name of a NAT pool" }); + map.insert(285, FieldInfo { name: "anonymizationFlags", data_type: DataType::UInt16, description: "A flag word describing specialized modifications" }); + map.insert(286, FieldInfo { name: "anonymizationTechnique", data_type: DataType::UInt16, description: "A description of the anonymization technique" }); + map.insert(287, FieldInfo { name: "informationElementIndex", data_type: DataType::UInt16, description: "A zero-based index of an Information" }); + map.insert(288, FieldInfo { name: "p2pTechnology", data_type: DataType::String, description: "Specifies if the Application ID is" }); + map.insert(289, FieldInfo { name: "tunnelTechnology", data_type: DataType::String, description: "Specifies if the Application ID is" }); + map.insert(290, FieldInfo { name: "encryptedTechnology", data_type: DataType::String, description: "Specifies if the Application ID is" }); + map.insert(291, FieldInfo { name: "basicList", data_type: DataType::String, description: "Specifies a generic Information Element with" }); + map.insert(292, FieldInfo { name: "subTemplateList", data_type: DataType::String, description: "Specifies a generic Information Element with" }); + map.insert(293, FieldInfo { name: "subTemplateMultiList", data_type: DataType::String, description: "Specifies a generic Information Element with" }); + map.insert(294, FieldInfo { name: "bgpValidityState", data_type: DataType::UInt8, description: "This element describes the \"validity state\"" }); + map.insert(295, FieldInfo { name: "IPSecSPI", data_type: DataType::UInt32, description: "IPSec Security Parameters Index (SPI)." }); + map.insert(296, FieldInfo { name: "greKey", data_type: DataType::UInt32, description: "GRE key, which is used for" }); + map.insert(297, FieldInfo { name: "natType", data_type: DataType::UInt8, description: "This Information Element identifies the NAT" }); + map.insert(298, FieldInfo { name: "initiatorPackets", data_type: DataType::UInt64, description: "The total number of layer 4" }); + map.insert(299, FieldInfo { name: "responderPackets", data_type: DataType::UInt64, description: "The total number of layer 4" }); + map.insert(300, FieldInfo { name: "observationDomainName", data_type: DataType::String, description: "The name of an observation domain" }); + map.insert(301, FieldInfo { name: "selectionSequenceId", data_type: DataType::UInt64, description: "From all the packets observed at" }); + map.insert(302, FieldInfo { name: "selectorId", data_type: DataType::UInt64, description: "The Selector ID is the unique" }); + map.insert(303, FieldInfo { name: "informationElementId", data_type: DataType::UInt16, description: "This Information Element contains the ID" }); + map.insert(304, FieldInfo { name: "selectorAlgorithm", data_type: DataType::UInt16, description: "This Information Element identifies the packet" }); + map.insert(305, FieldInfo { name: "samplingPacketInterval", data_type: DataType::UInt32, description: "This Information Element specifies the number" }); + map.insert(306, FieldInfo { name: "samplingPacketSpace", data_type: DataType::UInt32, description: "This Information Element specifies the number" }); + map.insert(307, FieldInfo { name: "samplingTimeInterval", data_type: DataType::UInt32, description: "This Information Element specifies the time" }); + map.insert(308, FieldInfo { name: "samplingTimeSpace", data_type: DataType::UInt32, description: "This Information Element specifies the time" }); + map.insert(309, FieldInfo { name: "samplingSize", data_type: DataType::UInt32, description: "This Information Element specifies the number" }); + map.insert(310, FieldInfo { name: "samplingPopulation", data_type: DataType::UInt32, description: "This Information Element specifies the number" }); + map.insert(311, FieldInfo { name: "samplingProbability", data_type: DataType::Float64, description: "This Information Element specifies the probability" }); + map.insert(312, FieldInfo { name: "dataLinkFrameSize", data_type: DataType::UInt16, description: "This Information Element specifies the length" }); + map.insert(313, FieldInfo { name: "ipHeaderPacketSection", data_type: DataType::Binary, description: "This Information Element carries a series" }); + map.insert(314, FieldInfo { name: "ipPayloadPacketSection", data_type: DataType::Binary, description: "This Information Element carries a series" }); + map.insert(315, FieldInfo { name: "dataLinkFrameSection", data_type: DataType::Binary, description: "This Information Element carries n octets" }); + map.insert(316, FieldInfo { name: "mplsLabelStackSection", data_type: DataType::Binary, description: "This Information Element carries a series" }); + map.insert(317, FieldInfo { name: "mplsPayloadPacketSection", data_type: DataType::Binary, description: "The mplsPayloadPacketSection carries a series of" }); + map.insert(318, FieldInfo { name: "selectorIdTotalPktsObserved", data_type: DataType::UInt64, description: "This Information Element specifies the total" }); + map.insert(319, FieldInfo { name: "selectorIdTotalPktsSelected", data_type: DataType::UInt64, description: "This Information Element specifies the total" }); + map.insert(320, FieldInfo { name: "absoluteError", data_type: DataType::Float64, description: "This Information Element specifies the maximum" }); + map.insert(321, FieldInfo { name: "relativeError", data_type: DataType::Float64, description: "This Information Element specifies the maximum" }); + map.insert(322, FieldInfo { name: "observationTimeSeconds", data_type: DataType::DateTimeSeconds, description: "This Information Element specifies the absolute" }); + map.insert(323, FieldInfo { name: "observationTimeMilliseconds", data_type: DataType::DateTimeMilliseconds, description: "This Information Element specifies the absolute" }); + map.insert(324, FieldInfo { name: "observationTimeMicroseconds", data_type: DataType::DateTimeMicroseconds, description: "This Information Element specifies the absolute" }); + map.insert(325, FieldInfo { name: "observationTimeNanoseconds", data_type: DataType::DateTimeNanoseconds, description: "This Information Element specifies the absolute" }); + map.insert(326, FieldInfo { name: "digestHashValue", data_type: DataType::UInt64, description: "This Information Element specifies the value" }); + map.insert(327, FieldInfo { name: "hashIPPayloadOffset", data_type: DataType::UInt64, description: "This Information Element specifies the IP" }); + map.insert(328, FieldInfo { name: "hashIPPayloadSize", data_type: DataType::UInt64, description: "This Information Element specifies the IP" }); + map.insert(329, FieldInfo { name: "hashOutputRangeMin", data_type: DataType::UInt64, description: "This Information Element specifies the value" }); + map.insert(330, FieldInfo { name: "hashOutputRangeMax", data_type: DataType::UInt64, description: "This Information Element specifies the value" }); + map.insert(331, FieldInfo { name: "hashSelectedRangeMin", data_type: DataType::UInt64, description: "This Information Element specifies the value" }); + map.insert(332, FieldInfo { name: "hashSelectedRangeMax", data_type: DataType::UInt64, description: "This Information Element specifies the value" }); + map.insert(333, FieldInfo { name: "hashDigestOutput", data_type: DataType::Boolean, description: "This Information Element contains a boolean" }); + map.insert(334, FieldInfo { name: "hashInitialiserValue", data_type: DataType::UInt64, description: "This Information Element specifies the initialiser" }); + map.insert(335, FieldInfo { name: "selectorName", data_type: DataType::String, description: "The name of a selector identified" }); + map.insert(336, FieldInfo { name: "upperCILimit", data_type: DataType::Float64, description: "This Information Element specifies the upper" }); + map.insert(337, FieldInfo { name: "lowerCILimit", data_type: DataType::Float64, description: "This Information Element specifies the lower" }); + map.insert(338, FieldInfo { name: "confidenceLevel", data_type: DataType::Float64, description: "This Information Element specifies the confidence" }); + map.insert(339, FieldInfo { name: "informationElementDataType", data_type: DataType::UInt8, description: "A description of the abstract data" }); + map.insert(340, FieldInfo { name: "informationElementDescription", data_type: DataType::String, description: "A UTF-8 [RFC3629] encoded Unicode string" }); + map.insert(341, FieldInfo { name: "informationElementName", data_type: DataType::String, description: "A UTF-8 [RFC3629] encoded Unicode string" }); + map.insert(342, FieldInfo { name: "informationElementRangeBegin", data_type: DataType::UInt64, description: "Contains the inclusive low end of" }); + map.insert(343, FieldInfo { name: "informationElementRangeEnd", data_type: DataType::UInt64, description: "Contains the inclusive high end of" }); + map.insert(344, FieldInfo { name: "informationElementSemantics", data_type: DataType::UInt8, description: "A description of the semantics of" }); + map.insert(345, FieldInfo { name: "informationElementUnits", data_type: DataType::UInt16, description: "A description of the units of" }); + map.insert(346, FieldInfo { name: "observationPointId", data_type: DataType::UInt32, description: "Identifier of the observation point" }); + map.insert(347, FieldInfo { name: "virtualStationInterfaceId", data_type: DataType::Binary, description: "Instance Identifier of the interface to" }); + map.insert(348, FieldInfo { name: "virtualStationInterfaceName", data_type: DataType::String, description: "Name of the interface to a" }); + map.insert(349, FieldInfo { name: "virtualStationUUID", data_type: DataType::Binary, description: "Unique Identifier of a Virtual Station." }); + map.insert(350, FieldInfo { name: "virtualStationName", data_type: DataType::String, description: "Name of a Virtual Station. A" }); + map.insert(351, FieldInfo { name: "layer2SegmentId", data_type: DataType::UInt64, description: "Identifier of a layer 2 network" }); + map.insert(352, FieldInfo { name: "layer2OctetDeltaCount", data_type: DataType::UInt64, description: "The number of layer 2 octets" }); + map.insert(353, FieldInfo { name: "layer2OctetTotalCount", data_type: DataType::UInt64, description: "The total number of layer 2" }); + map.insert(354, FieldInfo { name: "ingressUnicastPacketTotalCount", data_type: DataType::UInt64, description: "The total number of incoming unicast" }); + map.insert(355, FieldInfo { name: "ingressMulticastPacketTotalCount", data_type: DataType::UInt64, description: "The total number of incoming multicast" }); + map.insert(356, FieldInfo { name: "ingressBroadcastPacketTotalCount", data_type: DataType::UInt64, description: "The total number of incoming broadcast" }); + map.insert(357, FieldInfo { name: "egressUnicastPacketTotalCount", data_type: DataType::UInt64, description: "The total number of incoming unicast" }); + map.insert(358, FieldInfo { name: "egressBroadcastPacketTotalCount", data_type: DataType::UInt64, description: "The total number of incoming broadcast" }); + map.insert(359, FieldInfo { name: "monitoringIntervalStartMilliSeconds", data_type: DataType::DateTimeMilliseconds, description: "The absolute timestamp at which the" }); + map.insert(360, FieldInfo { name: "monitoringIntervalEndMilliSeconds", data_type: DataType::DateTimeMilliseconds, description: "The absolute timestamp at which the" }); + map.insert(361, FieldInfo { name: "portRangeStart", data_type: DataType::UInt16, description: "The port number identifying the start" }); + map.insert(362, FieldInfo { name: "portRangeEnd", data_type: DataType::UInt16, description: "The port number identifying the end" }); + map.insert(363, FieldInfo { name: "portRangeStepSize", data_type: DataType::UInt16, description: "The step size in a port" }); + map.insert(364, FieldInfo { name: "portRangeNumPorts", data_type: DataType::UInt16, description: "The number of ports in a" }); + map.insert(365, FieldInfo { name: "staMacAddress", data_type: DataType::MacAddress, description: "The IEEE 802 MAC address of" }); + map.insert(366, FieldInfo { name: "staIPv4Address", data_type: DataType::Ipv4Address, description: "The IPv4 address of a wireless" }); + map.insert(367, FieldInfo { name: "wtpMacAddress", data_type: DataType::MacAddress, description: "The IEEE 802 MAC address of" }); + map.insert(368, FieldInfo { name: "ingressInterfaceType", data_type: DataType::UInt32, description: "The type of interface where packets" }); + map.insert(369, FieldInfo { name: "egressInterfaceType", data_type: DataType::UInt32, description: "The type of interface where packets" }); + map.insert(370, FieldInfo { name: "rtpSequenceNumber", data_type: DataType::UInt16, description: "The RTP sequence number per [RFC3550]." }); + map.insert(371, FieldInfo { name: "userName", data_type: DataType::String, description: "User name associated with the flow." }); + map.insert(372, FieldInfo { name: "applicationCategoryName", data_type: DataType::String, description: "An attribute that provides a first" }); + map.insert(373, FieldInfo { name: "applicationSubCategoryName", data_type: DataType::String, description: "An attribute that provides a second" }); + map.insert(374, FieldInfo { name: "applicationGroupName", data_type: DataType::String, description: "An attribute that groups multiple Application" }); + map.insert(375, FieldInfo { name: "originalFlowsPresent", data_type: DataType::UInt64, description: "The non-conservative count of Original Flows" }); + map.insert(376, FieldInfo { name: "originalFlowsInitiated", data_type: DataType::UInt64, description: "The conservative count of Original Flows" }); + map.insert(377, FieldInfo { name: "originalFlowsCompleted", data_type: DataType::UInt64, description: "The conservative count of Original Flows" }); + map.insert(378, FieldInfo { name: "distinctCountOfSourceIPAddress", data_type: DataType::UInt64, description: "The count of distinct source IP" }); + map.insert(379, FieldInfo { name: "distinctCountOfDestinationIPAddress", data_type: DataType::UInt64, description: "The count of distinct destination IP" }); + map.insert(380, FieldInfo { name: "distinctCountOfSourceIPv4Address", data_type: DataType::UInt32, description: "The count of distinct source IPv4" }); + map.insert(381, FieldInfo { name: "distinctCountOfDestinationIPv4Address", data_type: DataType::UInt32, description: "The count of distinct destination IPv4" }); + map.insert(382, FieldInfo { name: "distinctCountOfSourceIPv6Address", data_type: DataType::UInt64, description: "The count of distinct source IPv6" }); + map.insert(383, FieldInfo { name: "distinctCountOfDestinationIPv6Address", data_type: DataType::UInt64, description: "The count of distinct destination IPv6" }); + map.insert(384, FieldInfo { name: "valueDistributionMethod", data_type: DataType::UInt8, description: "A description of the method used" }); + map.insert(385, FieldInfo { name: "rfc3550JitterMilliseconds", data_type: DataType::UInt32, description: "Interarrival jitter as defined in section" }); + map.insert(386, FieldInfo { name: "rfc3550JitterMicroseconds", data_type: DataType::UInt32, description: "Interarrival jitter as defined in section" }); + map.insert(387, FieldInfo { name: "rfc3550JitterNanoseconds", data_type: DataType::UInt32, description: "Interarrival jitter as defined in section" }); + map.insert(388, FieldInfo { name: "dot1qDEI", data_type: DataType::Boolean, description: "The value of the 1-bit Drop" }); + map.insert(389, FieldInfo { name: "dot1qCustomerDEI", data_type: DataType::Boolean, description: "In case of a QinQ frame," }); + map.insert(390, FieldInfo { name: "flowSelectorAlgorithm", data_type: DataType::UInt16, description: "This Information Element identifies the Intermediate" }); + map.insert(391, FieldInfo { name: "flowSelectedOctetDeltaCount", data_type: DataType::UInt64, description: "This Information Element specifies the volume" }); + map.insert(392, FieldInfo { name: "flowSelectedPacketDeltaCount", data_type: DataType::UInt64, description: "This Information Element specifies the volume" }); + map.insert(393, FieldInfo { name: "flowSelectedFlowDeltaCount", data_type: DataType::UInt64, description: "This Information Element specifies the number" }); + map.insert(394, FieldInfo { name: "selectorIDTotalFlowsObserved", data_type: DataType::UInt64, description: "This Information Element specifies the total" }); + map.insert(395, FieldInfo { name: "selectorIDTotalFlowsSelected", data_type: DataType::UInt64, description: "This Information Element specifies the total" }); + map.insert(396, FieldInfo { name: "samplingFlowInterval", data_type: DataType::UInt64, description: "This Information Element specifies the number" }); + map.insert(397, FieldInfo { name: "samplingFlowSpacing", data_type: DataType::UInt64, description: "This Information Element specifies the number" }); + map.insert(398, FieldInfo { name: "flowSamplingTimeInterval", data_type: DataType::UInt64, description: "This Information Element specifies the time" }); + map.insert(399, FieldInfo { name: "flowSamplingTimeSpacing", data_type: DataType::UInt64, description: "This Information Element specifies the time" }); + map.insert(400, FieldInfo { name: "hashFlowDomain", data_type: DataType::UInt16, description: "This Information Element specifies the Information" }); + map.insert(401, FieldInfo { name: "transportOctetDeltaCount", data_type: DataType::UInt64, description: "The number of octets, excluding IP" }); + map.insert(402, FieldInfo { name: "transportPacketDeltaCount", data_type: DataType::UInt64, description: "The number of packets containing at" }); + map.insert(403, FieldInfo { name: "originalExporterIPv4Address", data_type: DataType::Ipv4Address, description: "The IPv4 address used by the" }); + map.insert(404, FieldInfo { name: "originalExporterIPv6Address", data_type: DataType::Ipv6Address, description: "The IPv6 address used by the" }); + map.insert(405, FieldInfo { name: "originalObservationDomainId", data_type: DataType::UInt32, description: "The Observation Domain ID reported by" }); + map.insert(406, FieldInfo { name: "intermediateProcessId", data_type: DataType::UInt32, description: "Description: An identifier of an Intermediate" }); + map.insert(407, FieldInfo { name: "ignoredDataRecordTotalCount", data_type: DataType::UInt64, description: "Description: The total number of received" }); + map.insert(408, FieldInfo { name: "dataLinkFrameType", data_type: DataType::UInt16, description: "This Information Element specifies the type" }); + map.insert(409, FieldInfo { name: "sectionOffset", data_type: DataType::UInt16, description: "This Information Element specifies the offset" }); + map.insert(410, FieldInfo { name: "sectionExportedOctets", data_type: DataType::UInt16, description: "This Information Element specifies the observed" }); + map.insert(411, FieldInfo { name: "dot1qServiceInstanceTag", data_type: DataType::Binary, description: "This Information Element, which is 16" }); + map.insert(412, FieldInfo { name: "dot1qServiceInstanceId", data_type: DataType::UInt32, description: "The value of the 24-bit Backbone" }); + map.insert(413, FieldInfo { name: "dot1qServiceInstancePriority", data_type: DataType::UInt8, description: "The value of the 3-bit Backbone" }); + map.insert(414, FieldInfo { name: "dot1qCustomerSourceMacAddress", data_type: DataType::MacAddress, description: "The value of the Encapsulated Customer" }); + map.insert(415, FieldInfo { name: "dot1qCustomerDestinationMacAddress", data_type: DataType::MacAddress, description: "The value of the Encapsulated Customer" }); + map.insert(417, FieldInfo { name: "postLayer2OctetDeltaCount", data_type: DataType::UInt64, description: "The definition of this Information Element" }); + map.insert(418, FieldInfo { name: "postMCastLayer2OctetDeltaCount", data_type: DataType::UInt64, description: "The number of layer 2 octets" }); + map.insert(420, FieldInfo { name: "postLayer2OctetTotalCount", data_type: DataType::UInt64, description: "The definition of this Information Element" }); + map.insert(421, FieldInfo { name: "postMCastLayer2OctetTotalCount", data_type: DataType::UInt64, description: "The total number of layer 2" }); + map.insert(422, FieldInfo { name: "minimumLayer2TotalLength", data_type: DataType::UInt64, description: "Layer 2 length of the smallest" }); + map.insert(423, FieldInfo { name: "maximumLayer2TotalLength", data_type: DataType::UInt64, description: "Layer 2 length of the largest" }); + map.insert(424, FieldInfo { name: "droppedLayer2OctetDeltaCount", data_type: DataType::UInt64, description: "The number of layer 2 octets" }); + map.insert(425, FieldInfo { name: "droppedLayer2OctetTotalCount", data_type: DataType::UInt64, description: "The total number of octets in" }); + map.insert(426, FieldInfo { name: "ignoredLayer2OctetTotalCount", data_type: DataType::UInt64, description: "The total number of octets in" }); + map.insert(427, FieldInfo { name: "notSentLayer2OctetTotalCount", data_type: DataType::UInt64, description: "The total number of octets in" }); + map.insert(428, FieldInfo { name: "layer2OctetDeltaSumOfSquares", data_type: DataType::UInt64, description: "The sum of the squared numbers" }); + map.insert(429, FieldInfo { name: "layer2OctetTotalSumOfSquares", data_type: DataType::UInt64, description: "The total sum of the squared" }); + map.insert(430, FieldInfo { name: "layer2FrameDeltaCount", data_type: DataType::UInt64, description: "The number of incoming layer 2" }); + map.insert(431, FieldInfo { name: "layer2FrameTotalCount", data_type: DataType::UInt64, description: "The total number of incoming layer" }); + map.insert(432, FieldInfo { name: "pseudoWireDestinationIPv4Address", data_type: DataType::Ipv4Address, description: "The destination IPv4 address of the" }); + map.insert(433, FieldInfo { name: "ignoredLayer2FrameTotalCount", data_type: DataType::UInt64, description: "The total number of observed layer" }); + map.insert(434, FieldInfo { name: "mibObjectValueInteger", data_type: DataType::String, description: "An IPFIX Information Element that denotes" }); + map.insert(435, FieldInfo { name: "mibObjectValueOctetString", data_type: DataType::Binary, description: "An IPFIX Information Element that denotes" }); + map.insert(436, FieldInfo { name: "mibObjectValueOID", data_type: DataType::Binary, description: "An IPFIX Information Element that denotes" }); + map.insert(437, FieldInfo { name: "mibObjectValueBits", data_type: DataType::Binary, description: "An IPFIX Information Element that denotes" }); + map.insert(438, FieldInfo { name: "mibObjectValueIPAddress", data_type: DataType::Ipv4Address, description: "An IPFIX Information Element that denotes" }); + map.insert(439, FieldInfo { name: "mibObjectValueCounter", data_type: DataType::UInt64, description: "An IPFIX Information Element that denotes" }); + map.insert(440, FieldInfo { name: "mibObjectValueGauge", data_type: DataType::UInt32, description: "An IPFIX Information Element that denotes" }); + map.insert(441, FieldInfo { name: "mibObjectValueTimeTicks", data_type: DataType::UInt32, description: "An IPFIX Information Element that denotes" }); + map.insert(442, FieldInfo { name: "mibObjectValueUnsigned", data_type: DataType::UInt32, description: "An IPFIX Information Element that denotes" }); + map.insert(443, FieldInfo { name: "mibObjectValueTable", data_type: DataType::String, description: "An IPFIX Information Element that denotes" }); + map.insert(444, FieldInfo { name: "mibObjectValueRow", data_type: DataType::String, description: "An IPFIX Information Element that denotes" }); + map.insert(445, FieldInfo { name: "mibObjectIdentifier", data_type: DataType::Binary, description: "An IPFIX Information Element that denotes" }); + map.insert(446, FieldInfo { name: "mibSubIdentifier", data_type: DataType::UInt32, description: "A non-negative sub-identifier of an Object" }); + map.insert(447, FieldInfo { name: "mibIndexIndicator", data_type: DataType::UInt64, description: "A set of bit fields that" }); + map.insert(448, FieldInfo { name: "mibCaptureTimeSemantics", data_type: DataType::UInt8, description: "Indicates when in the lifetime of" }); + map.insert(449, FieldInfo { name: "mibContextEngineID", data_type: DataType::Binary, description: "A mibContextEngineID that specifies the SNMP" }); + map.insert(450, FieldInfo { name: "mibContextName", data_type: DataType::String, description: "An Information Element that denotes that" }); + map.insert(451, FieldInfo { name: "mibObjectName", data_type: DataType::String, description: "The name (called a descriptor in" }); + map.insert(452, FieldInfo { name: "mibObjectDescription", data_type: DataType::String, description: "The value of the DESCRIPTION clause" }); + map.insert(453, FieldInfo { name: "mibObjectSyntax", data_type: DataType::String, description: "The value of the SYNTAX clause" }); + map.insert(454, FieldInfo { name: "mibModuleName", data_type: DataType::String, description: "The textual name of the MIB" }); + map.insert(455, FieldInfo { name: "mobileIMSI", data_type: DataType::String, description: "The International Mobile Subscription Identity (IMSI)." }); + map.insert(456, FieldInfo { name: "mobileMSISDN", data_type: DataType::String, description: "The Mobile Station International Subscriber Directory" }); + map.insert(457, FieldInfo { name: "httpStatusCode", data_type: DataType::UInt16, description: "The HTTP Response Status Code, as" }); + map.insert(458, FieldInfo { name: "sourceTransportPortsLimit", data_type: DataType::UInt16, description: "This Information Element contains the maximum" }); + map.insert(459, FieldInfo { name: "httpRequestMethod", data_type: DataType::String, description: "The HTTP request method, as defined" }); + map.insert(460, FieldInfo { name: "httpRequestHost", data_type: DataType::String, description: "The HTTP request host, as defined" }); + map.insert(461, FieldInfo { name: "httpRequestTarget", data_type: DataType::String, description: "The HTTP request target, as defined" }); + map.insert(462, FieldInfo { name: "httpMessageVersion", data_type: DataType::String, description: "The version of an HTTP/1.1 message" }); + map.insert(463, FieldInfo { name: "natInstanceID", data_type: DataType::UInt32, description: "This Information Element uniquely identifies an" }); + map.insert(464, FieldInfo { name: "internalAddressRealm", data_type: DataType::Binary, description: "This Information Element represents the internal" }); + map.insert(465, FieldInfo { name: "externalAddressRealm", data_type: DataType::Binary, description: "This Information Element represents the external" }); + map.insert(466, FieldInfo { name: "natQuotaExceededEvent", data_type: DataType::UInt32, description: "This Information Element identifies the type" }); + map.insert(467, FieldInfo { name: "natThresholdEvent", data_type: DataType::UInt32, description: "This Information Element identifies a type" }); + map.insert(468, FieldInfo { name: "httpUserAgent", data_type: DataType::String, description: "The HTTP User-Agent header field as" }); + map.insert(469, FieldInfo { name: "httpContentType", data_type: DataType::String, description: "The HTTP Content-Type header field as" }); + map.insert(470, FieldInfo { name: "httpReasonPhrase", data_type: DataType::String, description: "The HTTP reason phrase as defined" }); + map.insert(471, FieldInfo { name: "maxSessionEntries", data_type: DataType::UInt32, description: "This element represents the maximum session" }); + map.insert(472, FieldInfo { name: "maxBIBEntries", data_type: DataType::UInt32, description: "This element represents the maximum BIB" }); + map.insert(473, FieldInfo { name: "maxEntriesPerUser", data_type: DataType::UInt32, description: "This element represents the maximum NAT" }); + map.insert(474, FieldInfo { name: "maxSubscribers", data_type: DataType::UInt32, description: "This element represents the maximum subscribers" }); + map.insert(475, FieldInfo { name: "maxFragmentsPendingReassembly", data_type: DataType::UInt32, description: "This element represents the maximum fragments" }); + map.insert(476, FieldInfo { name: "addressPoolHighThreshold", data_type: DataType::UInt32, description: "This element represents the high threshold" }); + map.insert(477, FieldInfo { name: "addressPoolLowThreshold", data_type: DataType::UInt32, description: "This element represents the low threshold" }); + map.insert(478, FieldInfo { name: "addressPortMappingHighThreshold", data_type: DataType::UInt32, description: "This element represents the high threshold" }); + map.insert(479, FieldInfo { name: "addressPortMappingLowThreshold", data_type: DataType::UInt32, description: "This element represents the low threshold" }); + map.insert(480, FieldInfo { name: "addressPortMappingPerUserHighThreshold", data_type: DataType::UInt32, description: "This element represents the high threshold" }); + map.insert(481, FieldInfo { name: "globalAddressMappingHighThreshold", data_type: DataType::UInt32, description: "This element represents the high threshold" }); + map.insert(482, FieldInfo { name: "vpnIdentifier", data_type: DataType::Binary, description: "VPN ID in the format specified" }); + map.insert(483, FieldInfo { name: "bgpCommunity", data_type: DataType::UInt32, description: "BGP community as defined in [RFC1997]" }); + map.insert(484, FieldInfo { name: "bgpSourceCommunityList", data_type: DataType::String, description: "basicList of zero or more bgpCommunity" }); + map.insert(485, FieldInfo { name: "bgpDestinationCommunityList", data_type: DataType::String, description: "basicList of zero or more bgpCommunity" }); + map.insert(486, FieldInfo { name: "bgpExtendedCommunity", data_type: DataType::Binary, description: "BGP Extended Community as defined in" }); + map.insert(487, FieldInfo { name: "bgpSourceExtendedCommunityList", data_type: DataType::String, description: "basicList of zero or more bgpExtendedCommunity" }); + map.insert(488, FieldInfo { name: "bgpDestinationExtendedCommunityList", data_type: DataType::String, description: "basicList of zero or more bgpExtendedCommunity" }); + map.insert(489, FieldInfo { name: "bgpLargeCommunity", data_type: DataType::Binary, description: "BGP Large Community as defined in" }); + map.insert(490, FieldInfo { name: "bgpSourceLargeCommunityList", data_type: DataType::String, description: "basicList of zero or more bgpLargeCommunity" }); + map.insert(491, FieldInfo { name: "bgpDestinationLargeCommunityList", data_type: DataType::String, description: "basicList of zero or more bgpLargeCommunity" }); + map.insert(492, FieldInfo { name: "srhFlagsIPv6", data_type: DataType::UInt8, description: "The 8-bit Flags field defined in" }); + map.insert(493, FieldInfo { name: "srhTagIPv6", data_type: DataType::UInt16, description: "The 16-bit Tag field defined in" }); + map.insert(494, FieldInfo { name: "srhSegmentIPv6", data_type: DataType::Ipv6Address, description: "The 128-bit IPv6 address that represents" }); + map.insert(495, FieldInfo { name: "srhActiveSegmentIPv6", data_type: DataType::Ipv6Address, description: "The 128-bit IPv6 address that represents" }); + map.insert(496, FieldInfo { name: "srhSegmentIPv6BasicList", data_type: DataType::String, description: "The ordered basicList [RFC6313] of zero" }); + map.insert(497, FieldInfo { name: "srhSegmentIPv6ListSection", data_type: DataType::Binary, description: "The SRv6 Segment List as defined" }); + map.insert(498, FieldInfo { name: "srhSegmentsIPv6Left", data_type: DataType::UInt8, description: "The 8-bit unsigned integer defining the" }); + map.insert(499, FieldInfo { name: "srhIPv6Section", data_type: DataType::Binary, description: "The SRH and its TLVs as" }); + map.insert(500, FieldInfo { name: "srhIPv6ActiveSegmentType", data_type: DataType::UInt8, description: "The designator of the routing protocol" }); + map.insert(501, FieldInfo { name: "srhSegmentIPv6LocatorLength", data_type: DataType::UInt8, description: "The length of the SRH segment" }); + map.insert(502, FieldInfo { name: "srhSegmentIPv6EndpointBehavior", data_type: DataType::UInt16, description: "The 16-bit unsigned integer that represents" }); + map.insert(503, FieldInfo { name: "transportChecksum", data_type: DataType::UInt16, description: "The checksum in the transport header." }); + map.insert(504, FieldInfo { name: "icmpHeaderPacketSection", data_type: DataType::Binary, description: "This Information Element carries a series" }); + map.insert(505, FieldInfo { name: "gtpuFlags", data_type: DataType::UInt8, description: "8-bit flags field indicating the version" }); + map.insert(506, FieldInfo { name: "gtpuMsgType", data_type: DataType::UInt8, description: "8-bit Message type field indicating the" }); + map.insert(507, FieldInfo { name: "gtpuTEid", data_type: DataType::UInt32, description: "32-bit tunnel endpoint identifier field defined" }); + map.insert(508, FieldInfo { name: "gtpuSequenceNum", data_type: DataType::UInt16, description: "16-bit sequence number field defined in" }); + map.insert(509, FieldInfo { name: "gtpuQFI", data_type: DataType::UInt8, description: "6-bit QoS flow identifier field defined" }); + map.insert(510, FieldInfo { name: "gtpuPduType", data_type: DataType::UInt8, description: "4-bit PDU type field defined in" }); + map.insert(511, FieldInfo { name: "bgpSourceAsPathList", data_type: DataType::String, description: "Ordered basicList [RFC6313] of zero or" }); + map.insert(512, FieldInfo { name: "bgpDestinationAsPathList", data_type: DataType::String, description: "Ordered basicList [RFC6313] of zero or" }); + map.insert(513, FieldInfo { name: "ipv6ExtensionHeaderType", data_type: DataType::UInt8, description: "Type of an IPv6 extension header" }); + map.insert(514, FieldInfo { name: "ipv6ExtensionHeaderCount", data_type: DataType::UInt8, description: "The number of consecutive occurrences of" }); + map.insert(515, FieldInfo { name: "ipv6ExtensionHeadersFull", data_type: DataType::String, description: "IPv6 extension headers observed in packets" }); + map.insert(516, FieldInfo { name: "ipv6ExtensionHeaderTypeCountList", data_type: DataType::String, description: "As per Section 4.1 of [RFC8200]," }); + map.insert(517, FieldInfo { name: "ipv6ExtensionHeadersLimit", data_type: DataType::Boolean, description: "When set to \"false\", this IE" }); + map.insert(518, FieldInfo { name: "ipv6ExtensionHeadersChainLength", data_type: DataType::UInt32, description: "In theory, there are no limits" }); + map.insert(519, FieldInfo { name: "ipv6ExtensionHeaderChainLengthList", data_type: DataType::String, description: "This IE is used to report" }); + map.insert(520, FieldInfo { name: "tcpOptionsFull", data_type: DataType::String, description: "TCP options in packets of this" }); + map.insert(521, FieldInfo { name: "tcpSharedOptionExID16", data_type: DataType::UInt16, description: "Reports an observed 2-byte ExID in" }); + map.insert(522, FieldInfo { name: "tcpSharedOptionExID32", data_type: DataType::UInt32, description: "Reports an observed 4-byte ExID in" }); + map.insert(523, FieldInfo { name: "tcpSharedOptionExID16List", data_type: DataType::String, description: "Reports observed 2-byte ExIDs in shared" }); + map.insert(524, FieldInfo { name: "tcpSharedOptionExID32List", data_type: DataType::String, description: "Reports observed 4-byte ExIDs in shared" }); + map.insert(525, FieldInfo { name: "udpSafeOptions", data_type: DataType::String, description: "Observed SAFE UDP options in a" }); + map.insert(526, FieldInfo { name: "udpUnsafeOptions", data_type: DataType::UInt64, description: "Observed UNSAFE UDP options in a" }); + map.insert(527, FieldInfo { name: "udpExID", data_type: DataType::UInt16, description: "Observed ExID in an Experimental option" }); + map.insert(528, FieldInfo { name: "udpSafeExIDList", data_type: DataType::String, description: "Observed ExIDs in the Experimental option" }); + map.insert(529, FieldInfo { name: "udpUnsafeExIDList", data_type: DataType::String, description: "Observed ExIDs in the UNSAFE Experimental" }); + + + map +} + +/// Initialize HPE Aruba enterprise field definitions. +fn init_hpe_aruba_fields() -> HashMap { + let mut map = HashMap::new(); + + // HPE Aruba EdgeConnect SD-WAN fields + map.insert(1, FieldInfo { name: "clientIPv4Address", data_type: DataType::Ipv4Address, description: "Client IPv4 address" }); + map.insert(2, FieldInfo { name: "serverIPv4Address", data_type: DataType::Ipv4Address, description: "Server IPv4 address" }); + map.insert(3, FieldInfo { name: "connectionServerOctetDeltaCount", data_type: DataType::UInt64, description: "Server octets" }); + map.insert(4, FieldInfo { name: "connectionServerPacketDeltaCount", data_type: DataType::UInt64, description: "Server packets" }); + map.insert(5, FieldInfo { name: "connectionClientOctetDeltaCount", data_type: DataType::UInt64, description: "Client octets" }); + map.insert(6, FieldInfo { name: "connectionClientPacketDeltaCount", data_type: DataType::UInt64, description: "Client packets" }); + map.insert(7, FieldInfo { name: "connectionInitiator", data_type: DataType::Ipv4Address, description: "Connection initiator" }); + map.insert(8, FieldInfo { name: "applicationHttpHost", data_type: DataType::String, description: "HTTP host header" }); + map.insert(9, FieldInfo { name: "connectionNumberOfConnections", data_type: DataType::UInt8, description: "Number of connections" }); + map.insert(10, FieldInfo { name: "connectionServerResponsesCount", data_type: DataType::UInt8, description: "Server responses count" }); + map.insert(11, FieldInfo { name: "connectionServerResponseDelay", data_type: DataType::UInt32, description: "Server response delay (μs)" }); + map.insert(12, FieldInfo { name: "connectionNetworkToServerDelay", data_type: DataType::UInt32, description: "Network to server delay (μs)" }); + map.insert(13, FieldInfo { name: "connectionNetworkToClientDelay", data_type: DataType::UInt32, description: "Network to client delay (μs)" }); + map.insert(14, FieldInfo { name: "connectionClientPacketRetransmissionCount", data_type: DataType::UInt32, description: "Client retransmissions" }); + map.insert(15, FieldInfo { name: "connectionClientToServerNetworkDelay", data_type: DataType::UInt32, description: "Client to server delay (μs)" }); + map.insert(16, FieldInfo { name: "connectionApplicationDelay", data_type: DataType::UInt32, description: "Application delay (μs)" }); + map.insert(17, FieldInfo { name: "connectionClientToServerResponseDelay", data_type: DataType::UInt32, description: "Client to server response delay (μs)" }); + map.insert(18, FieldInfo { name: "connectionTransactionDuration", data_type: DataType::UInt32, description: "Transaction duration (μs)" }); + map.insert(19, FieldInfo { name: "connectionTransactionDurationMin", data_type: DataType::UInt32, description: "Min transaction duration (μs)" }); + map.insert(20, FieldInfo { name: "connectionTransactionDurationMax", data_type: DataType::UInt32, description: "Max transaction duration (μs)" }); + map.insert(21, FieldInfo { name: "connectionTransactionCompleteCount", data_type: DataType::UInt8, description: "Complete transactions" }); + map.insert(22, FieldInfo { name: "fromZone", data_type: DataType::String, description: "Source zone" }); + map.insert(23, FieldInfo { name: "toZone", data_type: DataType::String, description: "Destination zone" }); + map.insert(24, FieldInfo { name: "tag", data_type: DataType::String, description: "Flow tag" }); + map.insert(25, FieldInfo { name: "overlay", data_type: DataType::String, description: "Overlay identifier" }); + map.insert(26, FieldInfo { name: "direction", data_type: DataType::String, description: "Flow direction" }); + map.insert(27, FieldInfo { name: "applicationCategory", data_type: DataType::String, description: "Application category" }); + + // Options Template fields (for Silver Peak Template 1024) + // Note: Some of these conflict with standard IPFIX field definitions + // but are used differently in Silver Peak Options Templates + map.insert(341, FieldInfo { name: "lowerCILimit", data_type: DataType::Float64, description: "Statistical confidence interval lower limit" }); + map.insert(344, FieldInfo { name: "dataLinkFrameSize", data_type: DataType::UInt16, description: "Frame size at data link layer" }); + map.insert(345, FieldInfo { name: "dataLinkFrameType", data_type: DataType::UInt16, description: "Type of data link frame" }); + + // Legacy fields + map.insert(10001, FieldInfo { name: "overlayTunnelID", data_type: DataType::UInt32, description: "Overlay tunnel ID" }); + map.insert(10002, FieldInfo { name: "policyMatchID", data_type: DataType::UInt32, description: "Policy match ID" }); + map.insert(10003, FieldInfo { name: "applianceName", data_type: DataType::String, description: "Appliance name" }); + map.insert(10004, FieldInfo { name: "WANInterfaceID", data_type: DataType::UInt16, description: "WAN interface ID" }); + map.insert(10005, FieldInfo { name: "QOSQueueID", data_type: DataType::UInt8, description: "QoS queue ID" }); + map.insert(10006, FieldInfo { name: "linkQualityMetrics", data_type: DataType::String, description: "Link quality metrics" }); + + map +} + +/// Initialize Cisco enterprise field definitions. +fn init_cisco_fields() -> HashMap { + let mut map = HashMap::new(); + + // Common Cisco fields + map.insert(1001, FieldInfo { name: "cisco_application_id", data_type: DataType::UInt32,description: "Cisco application identifier" }); + map.insert(1002, FieldInfo { name: "cisco_application_name", data_type: DataType::String, description: "Cisco application name" }); + map.insert(1003, FieldInfo { name: "cisco_application_category", data_type: DataType::String, description: "Cisco application category" }); + map.insert(1004, FieldInfo { name: "cisco_application_subcategory", data_type: DataType::String, description: "Cisco application subcategory" }); + map.insert(1005, FieldInfo { name: "cisco_application_group", data_type: DataType::String, description: "Cisco application group" }); + map.insert(1006, FieldInfo { name: "cisco_connection_id", data_type: DataType::UInt32, description: "Cisco connection identifier" }); + map.insert(1007, FieldInfo { name: "cisco_service_instance_id", data_type: DataType::UInt32, description: "Cisco service instance ID" }); + map.insert(1008, FieldInfo { name: "cisco_threat_type", data_type: DataType::UInt16, description: "Cisco threat type" }); + map.insert(1009, FieldInfo { name: "cisco_threat_subtype", data_type: DataType::UInt16, description: "Cisco threat subtype" }); + map.insert(1010, FieldInfo { name: "cisco_ssl_server_name", data_type: DataType::String, description: "SSL server name indication" }); + map.insert(1011, FieldInfo { name: "cisco_ssl_actual_encryption", data_type: DataType::UInt16, description: "SSL actual encryption" }); + map.insert(1012, FieldInfo { name: "cisco_ssl_server_cert_status", data_type: DataType::UInt8, description: "SSL server certificate status" }); + map.insert(1013, FieldInfo { name: "cisco_url_category", data_type: DataType::String, description: "URL category" }); + map.insert(1014, FieldInfo { name: "cisco_url_reputation", data_type: DataType::UInt8, description: "URL reputation score" }); + map.insert(1015, FieldInfo { name: "cisco_malware_name", data_type: DataType::String, description: "Malware name" }); + + map + } + + /// Initialize Juniper enterprise field definitions. + fn init_juniper_fields() -> HashMap { + let mut map = HashMap::new(); + + // Common Juniper fields + map.insert(1001, FieldInfo { name: "juniper_src_vrf_name", data_type: DataType::String, description: "Source VRF name" }); + map.insert(1002, FieldInfo { name: "juniper_dest_vrf_name", data_type: DataType::String, description: "Destination VRF name" }); + map.insert(1003, FieldInfo { name: "juniper_logical_system_name", data_type: DataType::String, description: "Logical system name" }); + map.insert(1004, FieldInfo { name: "juniper_tenant_id", data_type: DataType::UInt32, description: "Tenant identifier" }); + map.insert(1005, FieldInfo { name: "juniper_virtual_router_name", data_type: DataType::String, description: "Virtual router name" }); + map.insert(1006, FieldInfo { name: "juniper_firewall_rule_name", data_type: DataType::String, description: "Firewall rule name" }); + map.insert(1007, FieldInfo { name: "juniper_nat_rule_name", data_type: DataType::String, description: "NAT rule name" }); + map.insert(1008, FieldInfo { name: "juniper_service_set_name", data_type: DataType::String, description: "Service set name" }); + map.insert(1009, FieldInfo { name: "juniper_interface_description", data_type: DataType::String, description: "Interface description" }); + map.insert(1010, FieldInfo { name: "juniper_routing_instance", data_type: DataType::String, description: "Routing instance name" }); + + map + } + + /// Get protocol name from protocol number. + fn get_protocol_name(protocol: u8) -> &'static str { + match protocol { + 0 => "HOPOPT", + 1 => "ICMP", + 2 => "IGMP", + 3 => "GGP", + 4 => "IPv4", + 5 => "ST", + 6 => "TCP", + 7 => "CBT", + 8 => "EGP", + 9 => "IGP", + 10 => "BBN-RCC-MON", + 11 => "NVP-II", + 12 => "PUP", + 13 => "ARGUS", + 14 => "EMCON", + 15 => "XNET", + 16 => "CHAOS", + 17 => "UDP", + 18 => "MUX", + 19 => "DCN-MEAS", + 20 => "HMP", + 21 => "PRM", + 22 => "XNS-IDP", + 23 => "TRUNK-1", + 24 => "TRUNK-2", + 25 => "LEAF-1", + 26 => "LEAF-2", + 27 => "RDP", + 28 => "IRTP", + 29 => "ISO-TP4", + 30 => "NETBLT", + 31 => "MFE-NSP", + 32 => "MERIT-INP", + 33 => "DCCP", + 34 => "3PC", + 35 => "IDPR", + 36 => "XTP", + 37 => "DDP", + 38 => "IDPR-CMTP", + 39 => "TP++", + 40 => "IL", + 41 => "IPv6", + 42 => "SDRP", + 43 => "IPv6-Route", + 44 => "IPv6-Frag", + 45 => "IDRP", + 46 => "RSVP", + 47 => "GRE", + 48 => "DSR", + 49 => "BNA", + 50 => "ESP", + 51 => "AH", + 52 => "I-NLSP", + 53 => "SWIPE", + 54 => "NARP", + 55 => "MOBILE", + 56 => "TLSP", + 57 => "SKIP", + 58 => "IPv6-ICMP", + 59 => "IPv6-NoNxt", + 60 => "IPv6-Opts", + 61 => "Any-Host-Internal", + 62 => "CFTP", + 63 => "Any-Local-Network", + 64 => "SAT-EXPAK", + 65 => "KRYPTOLAN", + 66 => "RVD", + 67 => "IPPC", + 68 => "Any-Distributed-File-System", + 69 => "SAT-MON", + 70 => "VISA", + 71 => "IPCV", + 72 => "CPNX", + 73 => "CPHB", + 74 => "WSN", + 75 => "PVP", + 76 => "BR-SAT-MON", + 77 => "SUN-ND", + 78 => "WB-MON", + 79 => "WB-EXPAK", + 80 => "ISO-IP", + 81 => "VMTP", + 82 => "SECURE-VMTP", + 83 => "VINES", + 84 => "TTP", + 85 => "NSFNET-IGP", + 86 => "DGP", + 87 => "TCF", + 88 => "EIGRP", + 89 => "OSPFIGP", + 90 => "Sprite-RPC", + 91 => "LARP", + 92 => "MTP", + 93 => "AX.25", + 94 => "IPIP", + 95 => "MICP", + 96 => "SCC-SP", + 97 => "ETHERIP", + 98 => "ENCAP", + 99 => "Any-Private-Encryption", + 100 => "GMTP", + 101 => "IFMP", + 102 => "PNNI", + 103 => "PIM", + 104 => "ARIS", + 105 => "SCPS", + 106 => "QNX", + 107 => "A/N", + 108 => "IPComp", + 109 => "SNP", + 110 => "Compaq-Peer", + 111 => "IPX-in-IP", + 112 => "VRRP", + 113 => "PGM", + 114 => "Any-0-Hop", + 115 => "L2TP", + 116 => "DDX", + 117 => "IATP", + 118 => "STP", + 119 => "SRP", + 120 => "UTI", + 121 => "SMP", + 122 => "SM", + 123 => "PTP", + 124 => "ISIS-over-IPv4", + 125 => "FIRE", + 126 => "CRTP", + 127 => "CRUDP", + 128 => "SSCOPMCE", + 129 => "IPLT", + 130 => "SPS", + 131 => "PIPE", + 132 => "SCTP", + 133 => "FC", + 134 => "RSVP-E2E-IGNORE", + 135 => "Mobility-Header", + 136 => "UDPLite", + 137 => "MPLS-in-IP", + 138 => "MANET", + 139 => "HIP", + 140 => "Shim6", + 141 => "WESP", + 142 => "ROHC", + 143 => "Ethernet", + 253 => "Experimentation", + 254 => "Experimentation", + 255 => "Reserved", + _ => "Unknown", + } + } + + #[cfg(test)] + mod tests { + use super::*; + use crate::sources::netflow::config::NetflowConfig; + use vector_lib::event::LogEvent; + + #[test] + fn test_data_type_parsing() { + // Test UInt32 + let data = vec![0x00, 0x00, 0x01, 0x00]; // 256 + let result = DataType::UInt32.parse(&data, 1024).unwrap(); + if let Value::Integer(val) = result { + assert_eq!(val, 256); + } else { + panic!("Expected integer value"); + } + + // Test IPv4 address + let data = vec![192, 168, 1, 1]; + let result = DataType::Ipv4Address.parse(&data, 1024).unwrap(); + if let Value::Bytes(bytes) = result { + assert_eq!(String::from_utf8(bytes.to_vec()).unwrap(), "192.168.1.1"); + } else { + panic!("Expected bytes value"); + } + + // Test MAC address + let data = vec![0x00, 0x1B, 0x21, 0x3C, 0x4D, 0x5E]; + let result = DataType::MacAddress.parse(&data, 1024).unwrap(); + if let Value::Bytes(bytes) = result { + assert_eq!(String::from_utf8(bytes.to_vec()).unwrap(), "00:1b:21:3c:4d:5e"); + } else { + panic!("Expected bytes value"); + } + + // Test string + let data = b"Hello World\0"; + let result = DataType::String.parse(data, 1024).unwrap(); + if let Value::Bytes(bytes) = result { + assert_eq!(String::from_utf8(bytes.to_vec()).unwrap(), "Hello World"); + } else { + panic!("Expected bytes value"); + } + + // Test binary (invalid UTF-8) + let data = vec![0xFF, 0xFE, 0xFD]; + let result = DataType::Binary.parse(&data, 1024).unwrap(); + if let Value::Bytes(bytes) = result { + let decoded = base64::engine::general_purpose::STANDARD.decode(&bytes).unwrap(); + assert_eq!(decoded, data); + } else { + panic!("Expected bytes value"); + } + } + + #[test] + fn test_field_parser_creation() { + let config = NetflowConfig::default(); + let parser = FieldParser::new(&config); + assert_eq!(parser.max_field_length, 65535); + assert!(parser.resolve_protocols); + } + + #[test] + fn test_standard_field_lookup() { + let config = NetflowConfig::default(); + let parser = FieldParser::new(&config); + + let field = TemplateField { + field_type: 8, // sourceIPv4Address + field_length: 4, + enterprise_number: None, + is_scope: false, + }; + + let field_info = parser.get_field_info(&field); + assert_eq!(field_info.name, "sourceIPv4Address"); + assert!(matches!(field_info.data_type, DataType::Ipv4Address)); + } + + #[test] + fn test_hpe_aruba_field_lookup() { + let config = NetflowConfig::default(); + let parser = FieldParser::new(&config); + + let field = TemplateField { + field_type: 1, // clientIPv4Address + field_length: 4, + enterprise_number: Some(23867), + is_scope: false, + }; + + let field_info = parser.get_field_info(&field); + assert_eq!(field_info.name, "clientIPv4Address"); + assert!(matches!(field_info.data_type, DataType::Ipv4Address)); + } + + #[test] + fn test_custom_enterprise_field() { + let mut config = NetflowConfig::default(); + config.enterprise_fields.insert( + "9:1001".to_string(), + "custom_cisco_field".to_string(), + ); + + let parser = FieldParser::new(&config); + + let field = TemplateField { + field_type: 1001, + field_length: 4, + enterprise_number: Some(9), + is_scope: false, + }; + + let field_info = parser.get_field_info(&field); + assert_eq!(field_info.name, "custom_cisco_field"); + assert!(matches!(field_info.data_type, DataType::UInt32)); + } + + #[test] + fn test_unknown_field_handling() { + let config = NetflowConfig::default(); + let parser = FieldParser::new(&config); + + // Unknown standard field + let field = TemplateField { + field_type: 9999, + field_length: 4, + enterprise_number: None, + is_scope: false, + }; + + let field_info = parser.get_field_info(&field); + assert_eq!(field_info.name, "unknown_field_9999"); + assert!(matches!(field_info.data_type, DataType::Binary)); + + // Unknown enterprise field + let field = TemplateField { + field_type: 1001, + field_length: 4, + enterprise_number: Some(99999), + is_scope: false, + }; + + let field_info = parser.get_field_info(&field); + assert_eq!(field_info.name, "enterprise_99999_1001"); + assert!(matches!(field_info.data_type, DataType::Binary)); + } + + #[test] + fn test_field_parsing_with_protocol_resolution() { + let config = NetflowConfig::default(); + let parser = FieldParser::new(&config); + + let field = TemplateField { + field_type: 4, // protocolIdentifier + field_length: 1, + enterprise_number: None, + is_scope: false, + }; + + let mut log_event = LogEvent::default(); + let data = vec![6]; // TCP + + parser.parse_field(&field, &data, &mut log_event); + + assert_eq!(log_event.get("protocolIdentifier").unwrap().as_integer().unwrap(), 6); + assert_eq!(log_event.get("protocolName").unwrap().as_str().unwrap(), "TCP"); + } + + #[test] + fn test_field_truncation() { + let config = NetflowConfig { + max_packet_size: 10, + ..Default::default() + }; + let parser = FieldParser::new(&config); + + let field = TemplateField { + field_type: 999, // Unknown field, will be parsed as binary + field_length: 20, + enterprise_number: None, + is_scope: false, + }; + + let mut log_event = LogEvent::default(); + let data = vec![0x41; 20]; // 20 'A' characters + + parser.parse_field(&field, &data, &mut log_event); + + let value = log_event.get("unknown_field_999").unwrap().as_str().unwrap(); + assert!(value.len() <= 10); + assert!(value.ends_with("...")); + } + + #[test] + fn test_protocol_name_resolution() { + assert_eq!(get_protocol_name(1), "ICMP"); + assert_eq!(get_protocol_name(6), "TCP"); + assert_eq!(get_protocol_name(17), "UDP"); + assert_eq!(get_protocol_name(47), "GRE"); + assert_eq!(get_protocol_name(50), "ESP"); + assert_eq!(get_protocol_name(255), "Reserved"); + assert_eq!(get_protocol_name(200), "Unknown"); + } + + #[test] + fn test_insufficient_data_handling() { + let data = vec![0x01]; // Only 1 byte + + // Should fail for UInt32 (needs 4 bytes) + let result = DataType::UInt32.parse(&data, 1024); + assert!(result.is_err()); + + // Should succeed for UInt8 (needs 1 byte) + let result = DataType::UInt8.parse(&data, 1024); + assert!(result.is_ok()); + } + + #[test] + fn test_data_type_from_field_type() { + assert!(matches!(DataType::from(FieldType::Uint32), DataType::UInt32)); + assert!(matches!(DataType::from(FieldType::Ipv4Address), DataType::Ipv4Address)); + assert!(matches!(DataType::from(FieldType::String), DataType::String)); + assert!(matches!(DataType::from(FieldType::Binary), DataType::Binary)); + } + } diff --git a/src/sources/netflow/mod.rs b/src/sources/netflow/mod.rs new file mode 100644 index 0000000000000..71365f1e24af7 --- /dev/null +++ b/src/sources/netflow/mod.rs @@ -0,0 +1,446 @@ +//! NetFlow source for Vector. +//! +//! This source listens for NetFlow, IPFIX, and sFlow packets over UDP and parses them +//! into structured log events. + +use crate::config::{DataType, Resource, SourceConfig, SourceContext, SourceOutput}; +use crate::shutdown::ShutdownSignal; +use crate::sources::Source; +use crate::SourceSender; +use crate::sources::netflow::events::*; +use tokio::net::UdpSocket; +use tokio::time::Duration; +use vector_lib::internal_event::InternalEvent; +use std::sync::Arc; +use tracing::{debug, error, info}; +use socket2::{Socket, Domain, Type, Protocol}; + +pub mod config; +pub mod events; +pub mod fields; +pub mod protocols; +pub mod templates; + +pub use config::NetflowConfig; + +/// Create a UDP socket with SO_REUSEPORT enabled for multi-worker support. +/// +/// SO_REUSEPORT allows multiple sockets to bind to the same address, +/// enabling kernel-level load balancing across workers. +async fn create_reuseport_socket(address: std::net::SocketAddr) -> Result { + let domain = if address.is_ipv4() { + Domain::IPV4 + } else { + Domain::IPV6 + }; + + let socket = Socket::new(domain, Type::DGRAM, Some(Protocol::UDP))?; + + // Enable SO_REUSEPORT for multi-worker support + socket.set_reuse_port(true)?; + socket.set_reuse_address(true)?; // Also add this for faster restart + + // Bind to the address + socket.bind(&address.into())?; + + // CRITICAL: Must be nonblocking for tokio + socket.set_nonblocking(true)?; + + // Convert to tokio UdpSocket + let std_socket: std::net::UdpSocket = socket.into(); + UdpSocket::from_std(std_socket) +} + +/// High-performance NetFlow source implementation with multi-socket support. +/// +/// Uses multiple UDP sockets with SO_REUSEPORT to distribute packet processing +/// across all available CPU cores, optimized for high-throughput scenarios +/// (20M+ records/minute). +async fn netflow_source( + config: NetflowConfig, + shutdown: ShutdownSignal, + out: SourceSender, +) -> Result<(), ()> { + // Determine number of worker threads based on available parallelism + let num_workers = std::thread::available_parallelism() + .map(|n| n.get()) + .unwrap_or(1) + .max(1); + + debug!( + address = %config.address, + num_workers = num_workers, + "Starting NetFlow source with multi-socket support" + ); + + // Create shared template cache and protocol parser + let template_cache = Arc::new(templates::TemplateCache::new_with_buffering( + config.max_templates, + config.max_buffered_records + )); + let protocol_parser = Arc::new(protocols::ProtocolParser::new(&config, (*template_cache).clone())); + + // Spawn worker tasks + let mut worker_handles = Vec::with_capacity(num_workers); + + for worker_id in 0..num_workers { + // Create socket with SO_REUSEPORT to allow multiple sockets on same address + let socket = create_reuseport_socket(config.address).await.map_err(|error| { + NetflowBindError { + address: config.address, + error, + }.emit(); + })?; + + // Clone shared resources for this worker + let template_cache = template_cache.clone(); + let protocol_parser = protocol_parser.clone(); + let config = config.clone(); + let shutdown = shutdown.clone(); + let out = out.clone(); + + // Spawn worker task + let handle = tokio::spawn(async move { + netflow_worker( + worker_id, + socket, + config, + template_cache, + protocol_parser, + out, + shutdown, + ).await + }); + + worker_handles.push(handle); + } + + // Wait for all workers to complete + for (worker_id, handle) in worker_handles.into_iter().enumerate() { + if let Err(e) = handle.await { + error!( + worker_id = worker_id, + error = %e, + "NetFlow worker task failed" + ); + return Err(()); + } + } + + info!("All NetFlow workers completed"); + Ok(()) +} + +/// Individual NetFlow worker task that processes packets from a single UDP socket. +/// +/// Each worker runs in its own task and processes packets independently, +/// sharing the template cache and protocol parser with other workers. +async fn netflow_worker( + worker_id: usize, + socket: UdpSocket, + config: NetflowConfig, + template_cache: Arc, + protocol_parser: Arc, + mut out: SourceSender, + mut shutdown: ShutdownSignal, +) -> Result<(), ()> { + debug!( + worker_id = worker_id, + address = %config.address, + "NetFlow worker started" + ); + + // Pre-allocate multiple buffers for better performance + let mut buffers = Vec::with_capacity(8); + for _ in 0..8 { + buffers.push(vec![0u8; config.max_packet_size]); + } + let mut buffer_index = 0; + let mut last_cleanup = std::time::Instant::now(); + + loop { + tokio::select! { + recv_result = socket.recv_from(&mut buffers[buffer_index]) => { + match recv_result { + Ok((len, peer_addr)) => { + if len > config.max_packet_size { + NetflowParseError { + error: "Packet too large", + protocol: "unknown", + peer_addr, + }.emit(); + continue; + } + + let data = &buffers[buffer_index][..len]; + let events = protocol_parser.parse(data, peer_addr, &template_cache); + + if !events.is_empty() { + NetflowEventsReceived { + count: events.len(), + byte_size: len, + peer_addr, + }.emit(); + + if let Err(error) = out.send_batch(events).await { + error!( + worker_id = worker_id, + message = "Error sending events", + %error + ); + return Err(()); + } + } + + // Rotate buffer for next packet + buffer_index = (buffer_index + 1) % buffers.len(); + + // Periodic template cleanup (only one worker should do this) + if worker_id == 0 && last_cleanup.elapsed() > Duration::from_secs(300) { + template_cache.cleanup_expired(config.template_timeout); + last_cleanup = std::time::Instant::now(); + } + } + Err(error) => { + NetflowReceiveError { error }.emit(); + // Don't break on receive errors - keep trying + } + } + } + _ = &mut shutdown => { + debug!( + worker_id = worker_id, + "NetFlow worker shutting down" + ); + break; + } + } + } + + debug!( + worker_id = worker_id, + "NetFlow worker completed" + ); + Ok(()) +} + +#[async_trait::async_trait] +#[typetag::serde(name = "netflow")] +impl SourceConfig for NetflowConfig { + async fn build(&self, cx: SourceContext) -> crate::Result { + let config = self.clone(); + let shutdown = cx.shutdown; + let out = cx.out; + + Ok(Box::pin(netflow_source(config, shutdown, out))) + } + + fn outputs(&self, _global_log_namespace: vector_lib::config::LogNamespace) -> Vec { + vec![SourceOutput::new_maybe_logs( + DataType::Log, + vector_lib::schema::Definition::any() + )] + } + + fn resources(&self) -> Vec { + vec![Resource::udp(self.address)] + } + + fn can_acknowledge(&self) -> bool { + false + } +} + +#[cfg(test)] +mod tests { + use super::*; + use crate::test_util::{collect_ready, next_addr}; + use std::net::UdpSocket as StdUdpSocket; + use vector_lib::event::Event; + #[test] + fn test_netflow_v5_parsing() { + use crate::sources::netflow::protocols::ProtocolParser; + use crate::sources::netflow::templates::TemplateCache; + use std::net::{IpAddr, Ipv4Addr, SocketAddr}; + + let template_cache = TemplateCache::new(1000); + let config = NetflowConfig { + address: "127.0.0.1:2055".parse().unwrap(), + max_packet_size: 1500, + max_templates: 1000, + template_timeout: 3600, + protocols: vec!["netflow_v5".to_string()], + parse_enterprise_fields: true, + parse_options_templates: true, + parse_variable_length_fields: true, + enterprise_fields: std::collections::HashMap::new(), + buffer_missing_templates: true, + max_buffered_records: 100, + options_template_mode: "emit_metadata".to_string(), + strict_validation: true, + }; + + let parser = ProtocolParser::new(&config, template_cache.clone()); + let peer_addr = SocketAddr::new(IpAddr::V4(Ipv4Addr::new(127, 0, 0, 1)), 12345); + + // Create minimal NetFlow v5 packet + let mut packet = vec![0u8; 72]; // 24 header + 48 record + + // NetFlow v5 header + packet[0..2].copy_from_slice(&5u16.to_be_bytes()); // version + packet[2..4].copy_from_slice(&1u16.to_be_bytes()); // count + packet[4..8].copy_from_slice(&12345u32.to_be_bytes()); // sys_uptime + packet[8..12].copy_from_slice(&1609459200u32.to_be_bytes()); // unix_secs + packet[12..16].copy_from_slice(&0u32.to_be_bytes()); // unix_nsecs + packet[16..20].copy_from_slice(&100u32.to_be_bytes()); // flow_sequence + packet[20] = 0; // engine_type + packet[21] = 0; // engine_id + packet[22..24].copy_from_slice(&0u16.to_be_bytes()); // sampling_interval + + // Complete flow record (48 bytes) + packet[24..28].copy_from_slice(&0xC0A80101u32.to_be_bytes()); // src_addr: 192.168.1.1 + packet[28..32].copy_from_slice(&0x0A000001u32.to_be_bytes()); // dst_addr: 10.0.0.1 + packet[32..36].copy_from_slice(&0u32.to_be_bytes()); // next_hop + packet[36..38].copy_from_slice(&0u16.to_be_bytes()); // input + packet[38..40].copy_from_slice(&0u16.to_be_bytes()); // output + packet[40..44].copy_from_slice(&10u32.to_be_bytes()); // d_pkts + packet[44..48].copy_from_slice(&1500u32.to_be_bytes()); // d_octets + packet[48..52].copy_from_slice(&0u32.to_be_bytes()); // first + packet[52..56].copy_from_slice(&0u32.to_be_bytes()); // last + packet[56..58].copy_from_slice(&80u16.to_be_bytes()); // src_port + packet[58..60].copy_from_slice(&443u16.to_be_bytes()); // dst_port + packet[60] = 0; // pad1 + packet[61] = 0; // tcp_flags + packet[62] = 6; // prot (TCP) + packet[63] = 0; // tos + packet[64..66].copy_from_slice(&0u16.to_be_bytes()); // src_as + packet[66..68].copy_from_slice(&0u16.to_be_bytes()); // dst_as + packet[68] = 24; // src_mask + packet[69] = 24; // dst_mask + packet[70..72].copy_from_slice(&0u16.to_be_bytes()); // pad2 + + // Parse packet directly + let events = parser.parse(&packet, peer_addr, &template_cache); + + assert!(!events.is_empty(), "No events received"); + + if let Event::Log(log) = &events[0] { + // Debug: print all available fields + if let Some(map) = log.as_map() { + println!("Available fields: {:?}", map.keys().collect::>()); + } else { + println!("Log event is not a map"); + } + + // Check if flow_type exists first + if let Some(flow_type) = log.get("flow_type") { + if let Some(flow_type_str) = flow_type.as_str() { + assert_eq!(flow_type_str, "netflow_v5_record"); + } else { + panic!("flow_type is not a string: {:?}", flow_type); + } + } else { + panic!("flow_type field not found in event"); + } + + // Check record fields (version is not included in record events) + if let Some(src_addr) = log.get("src_addr") { + if let Some(src_addr_str) = src_addr.as_str() { + assert_eq!(src_addr_str, "192.168.1.1"); + } else { + panic!("src_addr is not a string: {:?}", src_addr); + } + } else { + panic!("src_addr field not found in event"); + } + + if let Some(dst_addr) = log.get("dst_addr") { + if let Some(dst_addr_str) = dst_addr.as_str() { + assert_eq!(dst_addr_str, "10.0.0.1"); + } else { + panic!("dst_addr is not a string: {:?}", dst_addr); + } + } else { + panic!("dst_addr field not found in event"); + } + + // Check other important fields + if let Some(protocol) = log.get("protocol") { + if let Some(protocol_int) = protocol.as_integer() { + assert_eq!(protocol_int, 6); // TCP + } else { + panic!("protocol is not an integer: {:?}", protocol); + } + } else { + panic!("protocol field not found in event"); + } + + if let Some(src_port) = log.get("src_port") { + if let Some(src_port_int) = src_port.as_integer() { + assert_eq!(src_port_int, 80); + } else { + panic!("src_port is not an integer: {:?}", src_port); + } + } else { + panic!("src_port field not found in event"); + } + + if let Some(dst_port) = log.get("dst_port") { + if let Some(dst_port_int) = dst_port.as_integer() { + assert_eq!(dst_port_int, 443); + } else { + panic!("dst_port is not an integer: {:?}", dst_port); + } + } else { + panic!("dst_port field not found in event"); + } + } else { + panic!("Expected Log event, got {:?}", events[0]); + } + } + + #[tokio::test] + async fn test_invalid_packet_handling() { + let addr = next_addr(); + let config = NetflowConfig { + address: addr, + max_packet_size: 1500, + max_templates: 1000, + template_timeout: 3600, + protocols: vec!["netflow_v5".to_string()], + parse_enterprise_fields: true, + parse_options_templates: true, + parse_variable_length_fields: true, + enterprise_fields: std::collections::HashMap::new(), + buffer_missing_templates: true, + max_buffered_records: 100, + options_template_mode: "emit_metadata".to_string(), + strict_validation: true, + }; + + let (tx, rx) = SourceSender::new_test(); + let cx = SourceContext::new_test(tx, None); + let source = config.build(cx).await.unwrap(); + let _source_task = tokio::spawn(source); + + tokio::time::sleep(Duration::from_millis(100)).await; + + // Send invalid packet (too short) + let packet = vec![0u8; 5]; + let socket = StdUdpSocket::bind("127.0.0.1:0").unwrap(); + socket.send_to(&packet, addr).unwrap(); + + // Should either get no events or an unknown protocol event + let events = tokio::time::timeout(Duration::from_millis(500), collect_ready(rx)) + .await + .unwrap_or_default(); + + // Invalid packets might create unknown protocol events or be dropped + // Both behaviors are acceptable + if !events.is_empty() { + if let Event::Log(log) = &events[0] { + assert_eq!(log.get("flow_type").unwrap().as_str().unwrap(), "unknown"); + } + } + } +} diff --git a/src/sources/netflow/protocols/ipfix.rs b/src/sources/netflow/protocols/ipfix.rs new file mode 100644 index 0000000000000..12248c098f173 --- /dev/null +++ b/src/sources/netflow/protocols/ipfix.rs @@ -0,0 +1,1250 @@ +//! IPFIX (Internet Protocol Flow Information Export) protocol parser. +//! +//! IPFIX is the standard protocol for exporting flow information, defined in RFC 7011. +//! It uses templates to define the structure of data records, supporting both standard +//! and enterprise-specific fields. + +use crate::sources::netflow::events::*; +use crate::sources::netflow::fields::FieldParser; +use crate::sources::netflow::templates::{ + TemplateCache, Template, +}; +use crate::sources::netflow::templates::{parse_ipfix_template_fields, parse_ipfix_options_template_fields}; +use std::net::SocketAddr; +use vector_lib::event::{Event, LogEvent, Value}; +use base64::Engine; + + +/// IPFIX protocol constants +const IPFIX_VERSION: u16 = 10; +const IPFIX_HEADER_SIZE: usize = 16; +const MAX_SET_LENGTH: usize = 65535; +const TEMPLATE_SET_ID: u16 = 2; +const OPTIONS_TEMPLATE_SET_ID: u16 = 3; + +/// IPFIX packet header structure +#[derive(Debug, Clone)] +pub struct IpfixHeader { + pub version: u16, + pub length: u16, + pub export_time: u32, + pub sequence_number: u32, + pub observation_domain_id: u32, +} + +impl IpfixHeader { + /// Parse IPFIX header from packet data + pub fn from_bytes(data: &[u8]) -> Result { + if data.len() < IPFIX_HEADER_SIZE { + return Err(format!( + "Packet too short for IPFIX header: {} bytes, need {}", + data.len(), + IPFIX_HEADER_SIZE + )); + } + + let version = u16::from_be_bytes([data[0], data[1]]); + if version != IPFIX_VERSION { + return Err(format!("Invalid IPFIX version: {}, expected {}", version, IPFIX_VERSION)); + } + + let length = u16::from_be_bytes([data[2], data[3]]); + if length as usize > data.len() { + return Err(format!( + "IPFIX length mismatch: header says {}, packet is {}", + length, + data.len() + )); + } + + Ok(Self { + version, + length, + export_time: u32::from_be_bytes([data[4], data[5], data[6], data[7]]), + sequence_number: u32::from_be_bytes([data[8], data[9], data[10], data[11]]), + observation_domain_id: u32::from_be_bytes([data[12], data[13], data[14], data[15]]), + }) + } + + /// Create base log event with header information + pub fn to_log_event(&self) -> LogEvent { + let mut log_event = LogEvent::default(); + log_event.insert("flow_type", "ipfix"); + log_event.insert("version", self.version); + log_event.insert("length", self.length); + log_event.insert("export_time", self.export_time); + log_event.insert("sequence_number", self.sequence_number); + log_event.insert("observation_domain_id", self.observation_domain_id); + log_event + } +} + +/// IPFIX set header structure +#[derive(Debug, Clone)] +pub struct SetHeader { + pub set_id: u16, + pub length: u16, +} + +impl SetHeader { + /// Parse set header from data + pub fn from_bytes(data: &[u8]) -> Result { + if data.len() < 4 { + return Err("Insufficient data for set header".to_string()); + } + + let set_id = u16::from_be_bytes([data[0], data[1]]); + let length = u16::from_be_bytes([data[2], data[3]]); + + if length < 4 { + return Err(format!("Invalid set length: {}, minimum is 4", length)); + } + + if length as usize > MAX_SET_LENGTH { + return Err(format!("Set length too large: {}, maximum is {}", length, MAX_SET_LENGTH)); + } + + Ok(Self { set_id, length }) + } + + /// Check if this is a template set + pub fn is_template_set(&self) -> bool { + self.set_id == TEMPLATE_SET_ID + } + + /// Check if this is an options template set + pub fn is_options_template_set(&self) -> bool { + self.set_id == OPTIONS_TEMPLATE_SET_ID + } + + /// Check if this is a data set + pub fn is_data_set(&self) -> bool { + self.set_id >= 256 + } + + /// Get template ID for data sets + pub fn template_id(&self) -> Option { + if self.is_data_set() { + Some(self.set_id) + } else { + None + } + } +} + +/// IPFIX protocol parser +pub struct IpfixParser { + field_parser: FieldParser, + options_template_mode: String, +} + +impl IpfixParser { + /// Create a new IPFIX parser + pub fn new(field_parser: FieldParser, options_template_mode: String) -> Self { + Self { + field_parser, + options_template_mode, + } + } + + /// Check if packet data looks like IPFIX + pub fn can_parse(data: &[u8]) -> bool { + if data.len() < 2 { + return false; + } + + let version = u16::from_be_bytes([data[0], data[1]]); + version == IPFIX_VERSION + } + + /// Parse IPFIX packet and return events + pub fn parse( + &self, + data: &[u8], + peer_addr: SocketAddr, + template_cache: &TemplateCache, + include_raw_data: bool, + drop_unparseable_records: bool, + buffer_missing_templates: bool, + ) -> Result, String> { + let mut events = Vec::new(); + + // Parse header + let header = IpfixHeader::from_bytes(data)?; + + debug!( + "Parsing IPFIX packet: version={}, length={}, domain={}", + header.version, header.length, header.observation_domain_id + ); + + // Create base event with header info + let mut base_event = header.to_log_event(); + if include_raw_data { + let encoded = base64::Engine::encode(&base64::engine::general_purpose::STANDARD, data); + base_event.insert("raw_data", encoded); + } + + // Parse sets + let mut offset = IPFIX_HEADER_SIZE; + let mut data_events = Vec::new(); + let mut template_count = 0; + let mut data_set_count = 0; + + while offset + 4 <= data.len() && offset < header.length as usize { + let set_header = match SetHeader::from_bytes(&data[offset..]) { + Ok(header) => header, + Err(e) => { + warn!("Invalid set header at offset {}: {}", offset, e); + break; + } + }; + + let set_end = offset + set_header.length as usize; + if set_end > data.len() || set_end > header.length as usize { + warn!( + "Set extends beyond packet boundary: offset={}, set_length={}, packet_length={}", + offset, set_header.length, data.len() + ); + break; + } + + let set_data = &data[offset..set_end]; + + match set_header.set_id { + TEMPLATE_SET_ID => { + let parsed_templates = self.parse_template_set( + set_data, + header.observation_domain_id, + peer_addr, + template_cache, + ); + template_count += parsed_templates; + } + OPTIONS_TEMPLATE_SET_ID => { + let parsed_templates = self.parse_options_template_set( + set_data, + header.observation_domain_id, + peer_addr, + template_cache, + ); + template_count += parsed_templates; + } + template_id if template_id >= 256 => { + let mut set_events = self.parse_data_set( + set_data, + template_id, + header.observation_domain_id, + peer_addr, + template_cache, + drop_unparseable_records, + buffer_missing_templates, + ); + data_set_count += 1; + data_events.append(&mut set_events); + } + _ => { + debug!("Skipping unknown set type: {}", set_header.set_id); + } + } + + offset = set_end; + } + + // Add parsed data events + events.extend(data_events); + + // If no data events were generated, include the header event + if events.is_empty() && !drop_unparseable_records { + base_event.insert("template_count", template_count); + base_event.insert("data_set_count", data_set_count); + events.push(Event::Log(base_event)); + } + + emit!(IpfixPacketProcessed { + peer_addr, + template_count, + data_set_count, + event_count: events.len(), + }); + + Ok(events) + } + + /// Parse template set and cache templates + fn parse_template_set( + &self, + data: &[u8], + observation_domain_id: u32, + peer_addr: SocketAddr, + template_cache: &TemplateCache, + ) -> usize { + let mut template_count = 0; + let mut offset = 4; // Skip set header + + while offset + 4 <= data.len() { + let template_id = u16::from_be_bytes([data[offset], data[offset + 1]]); + let field_count = u16::from_be_bytes([data[offset + 2], data[offset + 3]]); + + debug!( + "Parsing IPFIX template: id={}, fields={}", + template_id, field_count + ); + + // Find end of this template + let mut template_end = offset + 4; + let mut remaining_fields = field_count; + + while remaining_fields > 0 && template_end + 4 <= data.len() { + let field_type = u16::from_be_bytes([data[template_end], data[template_end + 1]]); + template_end += 4; // field_type + field_length + + // Check for enterprise field + if field_type & 0x8000 != 0 && template_end + 4 <= data.len() { + template_end += 4; // enterprise_number + } + + remaining_fields -= 1; + } + + if remaining_fields > 0 { + warn!("Incomplete template data for template {}", template_id); + break; + } + + // Parse template fields + let template_data = &data[offset..template_end]; + + // Debug: Log raw template data for template ID 1024 (only once) + if template_id == 1024 { + debug!( + message = "Template ID 1024 received - raw template data dump", + template_id = template_id, + field_count = field_count, + template_data_length = template_data.len(), + raw_template_hex = format!("{:02x?}", template_data), + raw_template_base64 = base64::engine::general_purpose::STANDARD.encode(template_data), + peer_addr = %peer_addr, + observation_domain_id = observation_domain_id, + ); + } + + match parse_ipfix_template_fields(template_data) { + Ok(fields) => { + let template = Template::new(template_id, fields); + let key = (peer_addr, observation_domain_id, template_id); + template_cache.insert(key, template); + template_count += 1; + + emit!(IpfixTemplateReceived { + template_id, + field_count, + peer_addr, + observation_domain_id, + }); + } + Err(e) => { + emit!(NetflowTemplateError { + error: e.as_str(), + template_id, + peer_addr, + }); + } + } + + offset = template_end; + } + + template_count + } + + /// Parse options template set with proper scope field handling + fn parse_options_template_set( + &self, + data: &[u8], + observation_domain_id: u32, + peer_addr: SocketAddr, + template_cache: &TemplateCache, + ) -> usize { + let mut template_count = 0; + let mut offset = 4; // Skip set header + + while offset + 6 <= data.len() { // Need at least 6 bytes for options template header + let template_id = u16::from_be_bytes([data[offset], data[offset + 1]]); + let field_count = u16::from_be_bytes([data[offset + 2], data[offset + 3]]); + let scope_field_count = u16::from_be_bytes([data[offset + 4], data[offset + 5]]); + + debug!( + "Parsing IPFIX options template: id={}, fields={}, scope_fields={}", + template_id, field_count, scope_field_count + ); + + // Find end of this template + let mut template_end = offset + 6; // Skip template_id, field_count, scope_field_count + let mut remaining_fields = field_count; + + while remaining_fields > 0 && template_end + 4 <= data.len() { + let field_type = u16::from_be_bytes([data[template_end], data[template_end + 1]]); + template_end += 4; // field_type + field_length + + // Check for enterprise field + if field_type & 0x8000 != 0 && template_end + 4 <= data.len() { + template_end += 4; // enterprise_number + } + + remaining_fields -= 1; + } + + if remaining_fields > 0 { + warn!("Incomplete options template data for template {}", template_id); + break; + } + + // Parse options template fields + let template_data = &data[offset..template_end]; + + // Debug: Log raw template data for template ID 1024 (only once) + if template_id == 1024 { + debug!( + message = "Options Template ID 1024 received - raw template data dump", + template_id = template_id, + field_count = field_count, + scope_field_count = scope_field_count, + template_data_length = template_data.len(), + raw_template_hex = format!("{:02x?}", template_data), + raw_template_base64 = base64::engine::general_purpose::STANDARD.encode(template_data), + peer_addr = %peer_addr, + observation_domain_id = observation_domain_id, + ); + } + + match parse_ipfix_options_template_fields(template_data) { + Ok((fields, scope_count)) => { + let template = Template::new_options(template_id, fields, scope_count); + let key = (peer_addr, observation_domain_id, template_id); + template_cache.insert(key, template); + template_count += 1; + + debug!("Cached Options Template: template_id={}, scope_field_count={}", + template_id, scope_count); + + emit!(IpfixTemplateReceived { + template_id, + field_count, + peer_addr, + observation_domain_id, + }); + } + Err(e) => { + emit!(NetflowTemplateError { + error: e.as_str(), + template_id, + peer_addr, + }); + } + } + + offset = template_end; + } + + template_count + } + + /// Parse data set using cached template + fn parse_data_set( + &self, + data: &[u8], + template_id: u16, + observation_domain_id: u32, + peer_addr: SocketAddr, + template_cache: &TemplateCache, + drop_unparseable_records: bool, + buffer_missing_templates: bool, + ) -> Vec { + let mut events = Vec::new(); + let key = (peer_addr, observation_domain_id, template_id); + + let template = match template_cache.get(&key) { + Some(template_arc) => { + debug!("Retrieved template for data set: template_id={}, scope_field_count={}", + template_id, template_arc.scope_field_count); + template_arc + }, + None => { + debug!( + "No template found for data set: template_id={}, domain={}", + template_id, observation_domain_id + ); + + // Try to buffer the data if buffering is enabled + if buffer_missing_templates { + if template_cache.buffer_data_record( + key, + data.to_vec(), + peer_addr, + observation_domain_id, + ) { + debug!( + "Buffered data record for template_id={}, waiting for template", + template_id + ); + return events; // Return empty, data is buffered + } + } + + if drop_unparseable_records { + emit!(NetflowEventsDropped { + count: 1, + reason: "No template available for IPFIX data parsing", + }); + return events; + } + + // Create basic event without template + let mut log_event = LogEvent::default(); + log_event.insert("flow_type", "ipfix_data_unparseable"); + log_event.insert("template_id", template_id); + log_event.insert("observation_domain_id", observation_domain_id); + log_event.insert("data_length", data.len() - 4); // Exclude set header + events.push(Event::Log(log_event)); + return events; + } + }; + + debug!( + "Parsing IPFIX data set: template_id={}, fields={}, template_fields={:?}", + template_id, + template.fields.len(), + template.fields.iter().map(|f| (f.field_type, f.field_length)).collect::>() + ); + + // Calculate record size for fixed-length templates + let record_size = template.record_size(); + let mut offset = 4; // Skip set header + let mut record_count = 0; + const MAX_RECORDS: usize = 10000; // Safety limit + + while offset < data.len() && record_count < MAX_RECORDS { + let mut log_event = LogEvent::default(); + + // Check if this is an Options Template (metadata about exporter) + if template.scope_field_count > 0 { + log_event.insert("flow_type", "ipfix_options_data"); + log_event.insert("data_type", "exporter_metadata"); + } else { + log_event.insert("flow_type", "ipfix_data"); + log_event.insert("data_type", "flow_data"); + } + + log_event.insert("template_id", template_id); + log_event.insert("observation_domain_id", observation_domain_id); + log_event.insert("record_number", record_count); + + let mut field_offset = offset; + let mut fields_parsed = 0; + + for field in &template.fields { + if field_offset >= data.len() { + debug!("Reached end of data while parsing field {}", fields_parsed); + break; + } + + let field_length = if field.field_length == 65535 { + // Variable-length field + let length = self.parse_variable_length(&data[field_offset..]); + match length { + Some((len, consumed)) => { + field_offset += consumed; + len + } + None => { + debug!("Failed to parse variable-length field"); + break; + } + } + } else { + field.field_length as usize + }; + + if field_offset + field_length > data.len() { + debug!( + "Insufficient data for field: offset={}, length={}, remaining={}", + field_offset, + field_length, + data.len() - field_offset + ); + break; + } + + let field_data = &data[field_offset..field_offset + field_length]; + + + self.field_parser.parse_field(field, field_data, &mut log_event); + + field_offset += field_length; + fields_parsed += 1; + } + + // Only emit event if we parsed some fields + if fields_parsed > 0 { + // Check if this is Options Template data and handle accordingly + if template.scope_field_count > 0 { + match self.options_template_mode.as_str() { + "discard" => { + // Don't add to events - effectively discard + }, + "emit_metadata" => { + events.push(Event::Log(log_event)); + }, + "enrich" => { + // Add enrichment metadata but mark as non-flow data + log_event.insert("enrichment_only", true); + events.push(Event::Log(log_event)); + }, + _ => { + warn!("Unknown options_template_mode value: {}, defaulting to discard", self.options_template_mode); + // Default to discard for unknown values + } + } + } else { + // Regular flow data - always emit + debug!( + "Parsed flow record {}: fields={}", + record_count, + fields_parsed + ); + + // Log specific problematic fields for debugging + if let Some(Value::Integer(bytes)) = log_event.get("octetDeltaCount") { + if *bytes > 1_000_000_000_000i64 { + debug!("Large byte count detected: {} bytes", bytes); + } + } + + if let Some(Value::Bytes(protocol)) = log_event.get("protocolName") { + if protocol.as_ref() == b"XNET" { + debug!("XNET protocol detected - possible parsing issue"); + } + } + + events.push(Event::Log(log_event)); + } + } + + // Advance to next record + if let Some(size) = record_size { + // Fixed-length records + offset += size; + if offset > field_offset { + // Skip any remaining padding + offset = field_offset; + } + } else { + // Variable-length records + offset = field_offset; + } + + // Safety check for infinite loops + if offset <= field_offset && record_size.is_none() { + debug!("Record parsing made no progress, stopping"); + break; + } + + record_count += 1; + } + + if record_count >= MAX_RECORDS { + warn!("Hit maximum record limit ({}) for template {}", MAX_RECORDS, template_id); + } + + debug!( + "Parsed {} records from IPFIX data set (template {})", + record_count, template_id + ); + + events + } + + /// Parse variable-length field length encoding + fn parse_variable_length(&self, data: &[u8]) -> Option<(usize, usize)> { + if data.is_empty() { + return None; + } + + let first_byte = data[0]; + if first_byte < 255 { + // Single byte length + Some((first_byte as usize, 1)) + } else if data.len() >= 3 { + // Three byte length (0xFF + 2 bytes) + let length = u16::from_be_bytes([data[1], data[2]]) as usize; + Some((length, 3)) + } else { + None + } + } + +} + +/// Additional event types for IPFIX-specific events +#[derive(Debug)] +pub struct IpfixPacketProcessed { + pub peer_addr: SocketAddr, + pub template_count: usize, + pub data_set_count: usize, + pub event_count: usize, +} + +impl vector_lib::internal_event::InternalEvent for IpfixPacketProcessed { + fn emit(self) { + debug!( + message = "IPFIX packet processed", + peer_addr = %self.peer_addr, + template_count = self.template_count, + data_set_count = self.data_set_count, + event_count = self.event_count, + ); + } +} + +#[derive(Debug)] +pub struct IpfixTemplateReceived { + pub template_id: u16, + pub field_count: u16, + pub peer_addr: SocketAddr, + pub observation_domain_id: u32, +} + +impl vector_lib::internal_event::InternalEvent for IpfixTemplateReceived { + fn emit(self) { + debug!( + message = "IPFIX template received", + template_id = self.template_id, + field_count = self.field_count, + peer_addr = %self.peer_addr, + observation_domain_id = self.observation_domain_id, + ); + } +} + +#[cfg(test)] +mod tests { + use super::*; + use crate::sources::netflow::NetflowConfig; + use crate::sources::netflow::fields::FieldParser; + use crate::sources::netflow::templates::{TemplateCache, TemplateField}; + use base64::Engine; + use std::net::{IpAddr, Ipv4Addr}; + + fn test_peer_addr() -> SocketAddr { + SocketAddr::new(IpAddr::V4(Ipv4Addr::new(192, 168, 1, 100)), 4739) + } + + fn create_ipfix_header() -> Vec { + let mut data = vec![0u8; 16]; + data[0..2].copy_from_slice(&10u16.to_be_bytes()); // version + data[2..4].copy_from_slice(&16u16.to_be_bytes()); // length (16 bytes) + data[4..8].copy_from_slice(&1609459200u32.to_be_bytes()); // export_time + data[8..12].copy_from_slice(&12345u32.to_be_bytes()); // sequence_number + data[12..16].copy_from_slice(&1u32.to_be_bytes()); // observation_domain_id + data + } + + #[test] + fn test_ipfix_header_parsing() { + let data = create_ipfix_header(); + let header = IpfixHeader::from_bytes(&data).unwrap(); + + assert_eq!(header.version, 10); + assert_eq!(header.length, 16); + assert_eq!(header.export_time, 1609459200); + assert_eq!(header.sequence_number, 12345); + assert_eq!(header.observation_domain_id, 1); + } + + #[test] + fn test_invalid_ipfix_header() { + // Too short + let short_data = vec![0u8; 10]; + assert!(IpfixHeader::from_bytes(&short_data).is_err()); + + // Wrong version + let mut wrong_version = create_ipfix_header(); + wrong_version[0..2].copy_from_slice(&9u16.to_be_bytes()); + assert!(IpfixHeader::from_bytes(&wrong_version).is_err()); + + // Length mismatch + let mut wrong_length = create_ipfix_header(); + wrong_length[2..4].copy_from_slice(&100u16.to_be_bytes()); + assert!(IpfixHeader::from_bytes(&wrong_length).is_err()); + } + + #[test] + fn test_set_header_parsing() { + let mut data = vec![0u8; 8]; + data[0..2].copy_from_slice(&2u16.to_be_bytes()); // template set + data[2..4].copy_from_slice(&8u16.to_be_bytes()); // length + + let header = SetHeader::from_bytes(&data).unwrap(); + assert_eq!(header.set_id, 2); + assert_eq!(header.length, 8); + assert!(header.is_template_set()); + assert!(!header.is_data_set()); + + // Data set + data[0..2].copy_from_slice(&256u16.to_be_bytes()); + let header = SetHeader::from_bytes(&data).unwrap(); + assert!(header.is_data_set()); + assert_eq!(header.template_id(), Some(256)); + } + + #[test] + fn test_can_parse() { + // Valid IPFIX + let ipfix_data = create_ipfix_header(); + assert!(IpfixParser::can_parse(&ipfix_data)); + + // Invalid version + let mut invalid_data = ipfix_data.clone(); + invalid_data[0..2].copy_from_slice(&5u16.to_be_bytes()); + assert!(!IpfixParser::can_parse(&invalid_data)); + + // Too short + let short_data = vec![0u8; 1]; + assert!(!IpfixParser::can_parse(&short_data)); + } + + #[test] + fn test_template_parsing() { + let config = NetflowConfig::default(); + let field_parser = FieldParser::new(&config); + let parser = IpfixParser::new(field_parser, "emit_metadata".to_string()); + let template_cache = TemplateCache::new(100); + + // Create IPFIX packet with template + let mut data = create_ipfix_header(); + data[2..4].copy_from_slice(&28u16.to_be_bytes()); // Update length + + // Template set header + data.extend_from_slice(&2u16.to_be_bytes()); // set_id + data.extend_from_slice(&12u16.to_be_bytes()); // set_length + + // Template definition + data.extend_from_slice(&256u16.to_be_bytes()); // template_id + data.extend_from_slice(&1u16.to_be_bytes()); // field_count + data.extend_from_slice(&8u16.to_be_bytes()); // field_type (sourceIPv4Address) + data.extend_from_slice(&4u16.to_be_bytes()); // field_length + + let events = parser.parse(&data, test_peer_addr(), &template_cache, false, false, true).unwrap(); + + // Should have base event with template info + assert!(!events.is_empty()); + + // Template should be cached + let key = (test_peer_addr(), 1, 256); + assert!(template_cache.get(&key).is_some()); + } + + #[test] + fn test_data_parsing_without_template() { + let config = NetflowConfig::default(); + let field_parser = FieldParser::new(&config); + let parser = IpfixParser::new(field_parser, "emit_metadata".to_string()); + let template_cache = TemplateCache::new(100); + + // Create IPFIX packet with data set (no template) + let mut data = create_ipfix_header(); + data[2..4].copy_from_slice(&24u16.to_be_bytes()); // Update length + + // Data set header + data.extend_from_slice(&256u16.to_be_bytes()); // template_id + data.extend_from_slice(&8u16.to_be_bytes()); // set_length + data.extend_from_slice(&[0x01, 0x02, 0x03, 0x04]); // Some data + + let _events = parser.parse(&data, test_peer_addr(), &template_cache, false, false, true).unwrap(); + + // With buffering enabled, data should be buffered and we get a header event + // or create unparseable event if buffering fails + assert!(!_events.is_empty()); + if let Event::Log(log) = &_events[0] { + let flow_type = log.get("flow_type").unwrap().as_str().unwrap(); + // With buffering enabled, we should get either: + // 1. A header event (flow_type = "ipfix") when data is buffered + // 2. An unparseable event if buffering fails + assert!(flow_type == "ipfix" || flow_type.contains("unparseable")); + } + } + + #[test] + fn test_data_parsing_with_template() { + let config = NetflowConfig::default(); + let field_parser = FieldParser::new(&config); + let parser = IpfixParser::new(field_parser, "emit_metadata".to_string()); + let template_cache = TemplateCache::new(100); + + // First, add a template to cache + let template = Template::new( + 256, + vec![TemplateField { + field_type: 8, // sourceIPv4Address + field_length: 4, + enterprise_number: None, + is_scope: false, + }], + ); + let key = (test_peer_addr(), 1, 256); + template_cache.insert(key, template); + + // Create IPFIX packet with data set + let mut data = create_ipfix_header(); + data[2..4].copy_from_slice(&24u16.to_be_bytes()); // Update length + + // Data set header + data.extend_from_slice(&256u16.to_be_bytes()); // template_id + data.extend_from_slice(&8u16.to_be_bytes()); // set_length + data.extend_from_slice(&[192, 168, 1, 1]); // IPv4 address data + + let events = parser.parse(&data, test_peer_addr(), &template_cache, false, false, true).unwrap(); + + // Should parse data using template + assert!(!events.is_empty()); + if let Event::Log(log) = &events[0] { + assert_eq!(log.get("flow_type").unwrap().as_str().unwrap(), "ipfix_data"); + assert_eq!(log.get("template_id").unwrap().as_integer().unwrap(), 256); + assert_eq!(log.get("sourceIPv4Address").unwrap().as_str().unwrap(), "192.168.1.1"); + } + } + + #[test] + fn test_variable_length_parsing() { + let config = NetflowConfig::default(); + let field_parser = FieldParser::new(&config); + let parser = IpfixParser::new(field_parser, "emit_metadata".to_string()); + + // Test single-byte length + let data = vec![10, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10]; + let result = parser.parse_variable_length(&data); + assert_eq!(result, Some((10, 1))); + + // Test three-byte length + let data = vec![255, 0, 20]; + let result = parser.parse_variable_length(&data); + assert_eq!(result, Some((20, 3))); + + // Test insufficient data + let data = vec![255, 0]; // Missing second length byte + let result = parser.parse_variable_length(&data); + assert_eq!(result, None); + } + + #[test] + fn test_enterprise_field_template() { + let config = NetflowConfig::default(); + let field_parser = FieldParser::new(&config); + let parser = IpfixParser::new(field_parser, "emit_metadata".to_string()); + let template_cache = TemplateCache::new(100); + + // Create IPFIX packet with enterprise field template + let mut data = create_ipfix_header(); + data[2..4].copy_from_slice(&32u16.to_be_bytes()); // Update length + + // Template set header + data.extend_from_slice(&2u16.to_be_bytes()); // set_id + data.extend_from_slice(&16u16.to_be_bytes()); // set_length + + // Template definition with enterprise field + data.extend_from_slice(&256u16.to_be_bytes()); // template_id + data.extend_from_slice(&1u16.to_be_bytes()); // field_count + data.extend_from_slice(&0x8001u16.to_be_bytes()); // field_type with enterprise bit + data.extend_from_slice(&4u16.to_be_bytes()); // field_length + data.extend_from_slice(&23867u32.to_be_bytes()); // enterprise_number (HPE Aruba) + + let _events = parser.parse(&data, test_peer_addr(), &template_cache, false, false, true).unwrap(); + + // Template should be cached with enterprise field + let key = (test_peer_addr(), 1, 256); + let template = template_cache.get(&key).unwrap(); + assert_eq!(template.fields.len(), 1); + assert_eq!(template.fields[0].field_type, 1); // Enterprise bit stripped + assert_eq!(template.fields[0].enterprise_number, Some(23867)); + } + + #[test] + fn test_malformed_packet_handling() { + let config = NetflowConfig::default(); + let field_parser = FieldParser::new(&config); + let parser = IpfixParser::new(field_parser, "emit_metadata".to_string()); + let template_cache = TemplateCache::new(100); + + // Test with corrupted set length + let mut data = create_ipfix_header(); + data[2..4].copy_from_slice(&24u16.to_be_bytes()); + + // Corrupted set header + data.extend_from_slice(&2u16.to_be_bytes()); // set_id + data.extend_from_slice(&8u16.to_be_bytes()); // set_length (valid) + data.extend_from_slice(&[0u8; 2]); // Incomplete template data + + let result = parser.parse(&data, test_peer_addr(), &template_cache, false, false, true); + // Should handle gracefully - either return base event or handle error gracefully + if result.is_ok() { + let events = result.unwrap(); + // Should have header event with template count if events are present + if !events.is_empty() { + if let Event::Log(log) = &events[0] { + assert!(log.get("template_count").is_some()); + assert!(log.get("data_set_count").is_some()); + } + } + } else { + // If parsing fails due to malformed data, that's also acceptable + // The important thing is that it doesn't panic + println!("Parse failed as expected for malformed packet: {:?}", result.err()); + } + } + + #[test] + fn test_drop_unparseable_records() { + let config = NetflowConfig::default(); + let field_parser = FieldParser::new(&config); + let parser = IpfixParser::new(field_parser, "emit_metadata".to_string()); + let template_cache = TemplateCache::new(100); + + // Create IPFIX packet with data set (no template) + let mut data = create_ipfix_header(); + data[2..4].copy_from_slice(&24u16.to_be_bytes()); + + // Data set header + data.extend_from_slice(&256u16.to_be_bytes()); // template_id + data.extend_from_slice(&8u16.to_be_bytes()); // set_length + data.extend_from_slice(&[0x01, 0x02, 0x03, 0x04]); // Some data + + // With drop_unparseable_records = true, should get no events + let events = parser.parse(&data, test_peer_addr(), &template_cache, false, true, true).unwrap(); + assert!(events.is_empty()); + + // With drop_unparseable_records = false, should get unparseable event + let events = parser.parse(&data, test_peer_addr(), &template_cache, false, false, true).unwrap(); + assert!(!events.is_empty()); + } + + #[test] + fn test_multiple_records_in_data_set() { + let config = NetflowConfig::default(); + let field_parser = FieldParser::new(&config); + let parser = IpfixParser::new(field_parser, "emit_metadata".to_string()); + let template_cache = TemplateCache::new(100); + + // Add template with fixed-length field + let template = Template::new( + 256, + vec![TemplateField { + field_type: 8, // sourceIPv4Address + field_length: 4, + enterprise_number: None, + is_scope: false, + }], + ); + let key = (test_peer_addr(), 1, 256); + template_cache.insert(key, template); + + // Create IPFIX packet with multiple records + let mut data = create_ipfix_header(); + data[2..4].copy_from_slice(&28u16.to_be_bytes()); // Update length + + // Data set header + data.extend_from_slice(&256u16.to_be_bytes()); // template_id + data.extend_from_slice(&12u16.to_be_bytes()); // set_length (4 header + 8 data) + + // Two IPv4 records + data.extend_from_slice(&[192, 168, 1, 1]); // First record + data.extend_from_slice(&[10, 0, 0, 1]); // Second record + + let events = parser.parse(&data, test_peer_addr(), &template_cache, false, false, true).unwrap(); + + // Should get two data events + assert_eq!(events.len(), 2); + + if let Event::Log(log1) = &events[0] { + assert_eq!(log1.get("sourceIPv4Address").unwrap().as_str().unwrap(), "192.168.1.1"); + assert_eq!(log1.get("record_number").unwrap().as_integer().unwrap(), 0); + } + + if let Event::Log(log2) = &events[1] { + assert_eq!(log2.get("sourceIPv4Address").unwrap().as_str().unwrap(), "10.0.0.1"); + assert_eq!(log2.get("record_number").unwrap().as_integer().unwrap(), 1); + } + } + + #[test] + fn test_mixed_template_and_data_packet() { + let config = NetflowConfig::default(); + let field_parser = FieldParser::new(&config); + let parser = IpfixParser::new(field_parser, "emit_metadata".to_string()); + let template_cache = TemplateCache::new(100); + + // Create IPFIX packet with both template and data + let mut data = create_ipfix_header(); + data[2..4].copy_from_slice(&36u16.to_be_bytes()); // Update length + + // Template set + data.extend_from_slice(&2u16.to_be_bytes()); // set_id + data.extend_from_slice(&12u16.to_be_bytes()); // set_length + data.extend_from_slice(&256u16.to_be_bytes()); // template_id + data.extend_from_slice(&1u16.to_be_bytes()); // field_count + data.extend_from_slice(&8u16.to_be_bytes()); // field_type + data.extend_from_slice(&4u16.to_be_bytes()); // field_length + + // Data set + data.extend_from_slice(&256u16.to_be_bytes()); // template_id + data.extend_from_slice(&8u16.to_be_bytes()); // set_length + data.extend_from_slice(&[192, 168, 1, 1]); // IPv4 data + + let events = parser.parse(&data, test_peer_addr(), &template_cache, false, false, true).unwrap(); + + // Should get data event (template was cached and immediately used) + assert!(!events.is_empty()); + + // Should have cached the template + let key = (test_peer_addr(), 1, 256); + assert!(template_cache.get(&key).is_some()); + } + + #[test] + fn test_options_template_parsing() { + // Test the parse_ipfix_options_template_fields function directly + let template_data = vec![ + 0x01, 0x01, // template_id = 257 + 0x00, 0x02, // field_count = 2 + 0x00, 0x01, // scope_field_count = 1 + 0x00, 0x95, 0x00, 0x04, // observationDomainId (149), length 4 + 0x00, 0x08, 0x00, 0x04, // sourceIPv4Address (8), length 4 + ]; + + let result = crate::sources::netflow::templates::parse_ipfix_options_template_fields(&template_data); + assert!(result.is_ok()); + + let (fields, scope_field_count) = result.unwrap(); + assert_eq!(scope_field_count, 1); + assert_eq!(fields.len(), 2); + assert_eq!(fields[0].is_scope, true); + assert_eq!(fields[1].is_scope, false); + + // Test template creation with scope field count + let template = crate::sources::netflow::templates::Template::new_options(257, fields, scope_field_count); + assert_eq!(template.template_id, 257); + assert_eq!(template.scope_field_count, 1); + assert_eq!(template.fields.len(), 2); + } + + #[test] + fn test_raw_data_inclusion() { + let config = NetflowConfig::default(); + let field_parser = FieldParser::new(&config); + let parser = IpfixParser::new(field_parser, "emit_metadata".to_string()); + let template_cache = TemplateCache::new(100); + + let data = create_ipfix_header(); + + // Test with raw data inclusion + let events = parser.parse(&data, test_peer_addr(), &template_cache, true, false, true).unwrap(); + assert!(!events.is_empty()); + + if let Event::Log(log) = &events[0] { + assert!(log.get("raw_data").is_some()); + let raw_data = log.get("raw_data").unwrap().as_str().unwrap(); + + // Should be valid base64 + assert!(base64::engine::general_purpose::STANDARD.decode(raw_data.as_bytes()).is_ok()); + } + + // Test without raw data inclusion + let events = parser.parse(&data, test_peer_addr(), &template_cache, false, false, true).unwrap(); + assert!(!events.is_empty()); + + if let Event::Log(log) = &events[0] { + assert!(log.get("raw_data").is_none()); + } + } + + #[test] + fn test_record_safety_limits() { + let config = NetflowConfig::default(); + let field_parser = FieldParser::new(&config); + let parser = IpfixParser::new(field_parser, "emit_metadata".to_string()); + let template_cache = TemplateCache::new(100); + + // Add template with very small field to potentially create many records + let template = Template::new( + 256, + vec![TemplateField { + field_type: 4, // protocolIdentifier + field_length: 1, + enterprise_number: None, + is_scope: false, + }], + ); + let key = (test_peer_addr(), 1, 256); + template_cache.insert(key, template); + + // Create IPFIX packet with large data set + let mut data = create_ipfix_header(); + let data_size = 10000; // Large data set + data[2..4].copy_from_slice(&((16 + 4 + data_size) as u16).to_be_bytes()); + + // Data set header + data.extend_from_slice(&256u16.to_be_bytes()); // template_id + data.extend_from_slice(&((4 + data_size) as u16).to_be_bytes()); // set_length + + // Add lots of data (each record is 1 byte) + data.extend(vec![6u8; data_size]); // All TCP protocol + + let events = parser.parse(&data, test_peer_addr(), &template_cache, false, false, true).unwrap(); + + // Should be limited by MAX_RECORDS safety limit + assert!(events.len() <= 10000); // MAX_RECORDS constant + + // All events should be valid + for event in &events { + if let Event::Log(log) = event { + assert_eq!(log.get("flow_type").unwrap().as_str().unwrap(), "ipfix_data"); + assert_eq!(log.get("protocolIdentifier").unwrap().as_integer().unwrap(), 6); + } + } + } + + #[test] + fn test_header_to_log_event() { + let data = create_ipfix_header(); + let header = IpfixHeader::from_bytes(&data).unwrap(); + let log_event = header.to_log_event(); + + assert_eq!(log_event.get("flow_type").unwrap().as_str().unwrap(), "ipfix"); + assert_eq!(log_event.get("version").unwrap().as_integer().unwrap(), 10); + assert_eq!(log_event.get("length").unwrap().as_integer().unwrap(), 16); + assert_eq!(log_event.get("export_time").unwrap().as_integer().unwrap(), 1609459200); + assert_eq!(log_event.get("sequence_number").unwrap().as_integer().unwrap(), 12345); + assert_eq!(log_event.get("observation_domain_id").unwrap().as_integer().unwrap(), 1); + } + + #[test] + fn test_set_header_edge_cases() { + // Test minimum valid length + let data = vec![0, 2, 0, 4]; // set_id=2, length=4 + let header = SetHeader::from_bytes(&data).unwrap(); + assert_eq!(header.length, 4); + + // Test invalid length (too small) + let data = vec![0, 2, 0, 3]; // length=3 (less than minimum 4) + assert!(SetHeader::from_bytes(&data).is_err()); + + // Test length too large + let data = vec![0, 2, 255, 255]; // length=65535 + let header = SetHeader::from_bytes(&data).unwrap(); + assert_eq!(header.length, 65535); + + // Test length beyond maximum + let mut data = vec![0, 2]; + data.extend_from_slice(&((MAX_SET_LENGTH + 1) as u16).to_be_bytes()); + assert!(SetHeader::from_bytes(&data).is_err()); + } +} \ No newline at end of file diff --git a/src/sources/netflow/protocols/mod.rs b/src/sources/netflow/protocols/mod.rs new file mode 100644 index 0000000000000..6a8a80cc2dce1 --- /dev/null +++ b/src/sources/netflow/protocols/mod.rs @@ -0,0 +1,638 @@ +//! Protocol parsing coordination for NetFlow/IPFIX/sFlow. +//! +//! This module provides a unified interface for parsing different flow protocols +//! while maintaining protocol-specific logic in separate modules. + +use crate::sources::netflow::config::NetflowConfig; +use crate::sources::netflow::events::*; +use crate::sources::netflow::fields::FieldParser; +use crate::sources::netflow::templates::TemplateCache; + +use std::net::SocketAddr; +use tracing::{debug, warn}; +use vector_lib::event::Event; + + +pub mod ipfix; +pub mod netflow_v5; +pub mod netflow_v9; +pub mod sflow; + +pub use ipfix::IpfixParser; +pub use netflow_v5::NetflowV5Parser; +pub use netflow_v9::NetflowV9Parser; +pub use sflow::SflowParser; + +/// Detected protocol type from packet analysis +#[derive(Debug, Clone, PartialEq, Eq)] +pub enum DetectedProtocol { + NetflowV5, + NetflowV9, + Ipfix, + Sflow, + Unknown(u16), // Contains the version/type that was detected +} + +impl DetectedProtocol { + /// Get the protocol name as a string + pub fn as_str(&self) -> &'static str { + match self { + DetectedProtocol::NetflowV5 => "netflow_v5", + DetectedProtocol::NetflowV9 => "netflow_v9", + DetectedProtocol::Ipfix => "ipfix", + DetectedProtocol::Sflow => "sflow", + DetectedProtocol::Unknown(_) => "unknown", + } + } + + /// Check if this protocol is enabled in configuration + pub fn is_enabled(&self, config: &NetflowConfig) -> bool { + let flow_protocol = match self { + DetectedProtocol::NetflowV5 => "netflow_v5", + DetectedProtocol::NetflowV9 => "netflow_v9", + DetectedProtocol::Ipfix => "ipfix", + DetectedProtocol::Sflow => "sflow", + DetectedProtocol::Unknown(_) => return false, + }; + + config.is_protocol_enabled(flow_protocol) + } +} + +/// Main protocol parser that coordinates all flow protocol parsers +pub struct ProtocolParser { + netflow_v5: NetflowV5Parser, + netflow_v9: NetflowV9Parser, + ipfix: IpfixParser, + sflow: SflowParser, + enabled_protocols: Vec, +} + +impl ProtocolParser { + /// Create a new protocol parser with the given configuration + pub fn new(config: &NetflowConfig, _template_cache: TemplateCache) -> Self { + let field_parser = FieldParser::new(config); + + Self { + netflow_v5: NetflowV5Parser::new(field_parser.clone(), config.strict_validation), + netflow_v9: NetflowV9Parser::new(field_parser.clone()), + ipfix: IpfixParser::new(field_parser.clone(), config.options_template_mode.clone()), + sflow: SflowParser::new(), + enabled_protocols: config.protocols.iter().map(|s| s.to_string()).collect(), + } + } + + /// Parse a packet and return flow events + pub fn parse(&self, data: &[u8], peer_addr: SocketAddr, template_cache: &TemplateCache) -> Vec { + // Detect protocol type + let protocol = self.detect_protocol(data); + + debug!( + "Detected protocol: {} from peer {}", + protocol.as_str(), + peer_addr + ); + + // Check if protocol is enabled + let config_stub = NetflowConfig { + protocols: self.enabled_protocols.clone(), + ..Default::default() + }; + + if !protocol.is_enabled(&config_stub) { + debug!( + "Protocol {} is disabled, ignoring packet from {}", + protocol.as_str(), + peer_addr + ); + + emit!(ProtocolDisabled { + protocol: protocol.as_str(), + peer_addr, + }); + + // For unknown protocols, still generate an event + if let DetectedProtocol::Unknown(version) = protocol { + return vec![self.create_unknown_protocol_event(data, peer_addr, version)]; + } + + return Vec::new(); + } + + // Parse using appropriate parser + let parse_result = match protocol { + DetectedProtocol::NetflowV5 => { + self.netflow_v5.parse( + data, + peer_addr, + true, // include_raw_data + ) + } + DetectedProtocol::NetflowV9 => { + self.netflow_v9.parse( + data, + peer_addr, + template_cache, + false, // include_raw_data + true, // drop_unparseable_records + ) + } + DetectedProtocol::Ipfix => { + self.ipfix.parse( + data, + peer_addr, + template_cache, + false, // include_raw_data + true, // drop_unparseable_records + true, // buffer_missing_templates + ) + } + DetectedProtocol::Sflow => { + self.sflow.parse( + data, + peer_addr, + false, // include_raw_data + ) + } + DetectedProtocol::Unknown(version) => { + Ok(vec![self.create_unknown_protocol_event(data, peer_addr, version)]) + } + }; + + match parse_result { + Ok(events) => { + if !events.is_empty() { + emit!(ProtocolParseSuccess { + protocol: protocol.as_str(), + peer_addr, + event_count: events.len(), + byte_size: data.len(), + }); + } + events + } + Err(error) => { + emit!(NetflowParseError { + error: &error, + protocol: protocol.as_str(), + peer_addr, + }); + + // Return basic event with error info instead of empty + vec![self.create_parse_error_event(data, peer_addr, &protocol, &error)] + } + } + } + + /// Detect the protocol type from packet data + fn detect_protocol(&self, data: &[u8]) -> DetectedProtocol { + if data.len() < 2 { + return DetectedProtocol::Unknown(0); + } + + // Check NetFlow/IPFIX version (first 2 bytes, big-endian) + let version = u16::from_be_bytes([data[0], data[1]]); + + match version { + 5 => { + // NetFlow v5 - verify packet structure + if NetflowV5Parser::can_parse(data) { + DetectedProtocol::NetflowV5 + } else { + DetectedProtocol::Unknown(version) + } + } + 9 => { + // NetFlow v9 - verify packet structure + if NetflowV9Parser::can_parse(data) { + DetectedProtocol::NetflowV9 + } else { + DetectedProtocol::Unknown(version) + } + } + 10 => { + // IPFIX - verify packet structure + if IpfixParser::can_parse(data) { + DetectedProtocol::Ipfix + } else { + DetectedProtocol::Unknown(version) + } + } + _ => { + // Check for sFlow (version is at different offset) + if data.len() >= 4 { + let sflow_version = u32::from_be_bytes([data[0], data[1], data[2], data[3]]); + if sflow_version == 5 && SflowParser::can_parse(data) { + return DetectedProtocol::Sflow; + } + } + + DetectedProtocol::Unknown(version) + } + } + } + + /// Create an event for unknown protocol packets + fn create_unknown_protocol_event( + &self, + data: &[u8], + peer_addr: SocketAddr, + version: u16, + ) -> Event { + let mut log_event = vector_lib::event::LogEvent::default(); + + log_event.insert("flow_type", "unknown"); + log_event.insert("version", version); + log_event.insert("peer_addr", peer_addr.to_string()); + log_event.insert("packet_length", data.len()); + + // Include some packet analysis + if data.len() >= 4 { + log_event.insert("first_4_bytes", hex::encode(&data[..4])); + } + + if data.len() >= 8 { + log_event.insert("first_8_bytes", hex::encode(&data[..8])); + } + + // Include raw data if configured + // Removed include_raw_data, so this block is removed. + + Event::Log(log_event) + } + + /// Create an event for parse errors + fn create_parse_error_event( + &self, + data: &[u8], + peer_addr: SocketAddr, + protocol: &DetectedProtocol, + error: &str, + ) -> Event { + let mut log_event = vector_lib::event::LogEvent::default(); + + log_event.insert("flow_type", "parse_error"); + log_event.insert("detected_protocol", protocol.as_str()); + log_event.insert("peer_addr", peer_addr.to_string()); + log_event.insert("packet_length", data.len()); + log_event.insert("parse_error", error); + + if let DetectedProtocol::Unknown(version) = protocol { + log_event.insert("detected_version", *version); + } + + // Include packet header for debugging + if data.len() >= 16 { + log_event.insert("packet_header", hex::encode(&data[..16])); + } else { + log_event.insert("packet_header", hex::encode(data)); + } + + // Include raw data if configured + // Removed include_raw_data, so this block is removed. + + Event::Log(log_event) + } + + /// Get statistics about supported protocols + pub fn get_protocol_stats(&self) -> ProtocolStats { + ProtocolStats { + enabled_protocols: self.enabled_protocols.clone(), + total_enabled: self.enabled_protocols.len(), + } + } +} + +/// Statistics about protocol support +#[derive(Debug, Clone)] +pub struct ProtocolStats { + pub enabled_protocols: Vec, + pub total_enabled: usize, +} + +/// Parse flow data using the protocol parser - main entry point +pub fn parse_flow_data( + data: &[u8], + peer_addr: SocketAddr, + template_cache: &TemplateCache, + config: &NetflowConfig, +) -> Result, String> { + if data.is_empty() { + return Err("Empty packet received".to_string()); + } + + let parser = ProtocolParser::new(config, template_cache.clone()); + let events = parser.parse(data, peer_addr, template_cache); + + Ok(events) +} + +/// Additional internal events for protocol coordination +#[derive(Debug)] +pub struct ProtocolDisabled { + pub protocol: &'static str, + pub peer_addr: SocketAddr, +} + +impl vector_lib::internal_event::InternalEvent for ProtocolDisabled { + fn emit(self) { + debug!( + message = "Protocol disabled, ignoring packet", + protocol = self.protocol, + peer_addr = %self.peer_addr, + ); + } +} + +#[derive(Debug)] +pub struct ProtocolParseSuccess { + pub protocol: &'static str, + pub peer_addr: SocketAddr, + pub event_count: usize, + pub byte_size: usize, +} + +impl vector_lib::internal_event::InternalEvent for ProtocolParseSuccess { + fn emit(self) { + debug!( + message = "Protocol parsed successfully", + protocol = self.protocol, + peer_addr = %self.peer_addr, + event_count = self.event_count, + byte_size = self.byte_size, + ); + + // Emit metrics + // Metrics are handled by the ComponentEventsReceived event + } +} + +#[derive(Debug)] +pub struct ProtocolDetectionFailed { + pub peer_addr: SocketAddr, + pub packet_length: usize, + pub first_bytes: String, +} + +impl vector_lib::internal_event::InternalEvent for ProtocolDetectionFailed { + fn emit(self) { + warn!( + message = "Failed to detect protocol", + peer_addr = %self.peer_addr, + packet_length = self.packet_length, + first_bytes = %self.first_bytes, + ); + } +} + +#[cfg(test)] +mod tests { + use super::*; + use crate::sources::netflow::config::NetflowConfig; + use crate::sources::netflow::templates::TemplateCache; + use std::net::{IpAddr, Ipv4Addr}; + + fn test_peer_addr() -> SocketAddr { + SocketAddr::new(IpAddr::V4(Ipv4Addr::new(192, 168, 1, 100)), 2055) + } + + fn test_config() -> NetflowConfig { + NetflowConfig::default() + } + + fn create_netflow_v5_packet() -> Vec { + let mut packet = vec![0u8; 72]; // 24 header + 48 record + packet[0..2].copy_from_slice(&5u16.to_be_bytes()); // version + packet[2..4].copy_from_slice(&1u16.to_be_bytes()); // count + packet[4..8].copy_from_slice(&12345u32.to_be_bytes()); // sys_uptime + packet + } + + fn create_netflow_v9_packet() -> Vec { + let mut packet = vec![0u8; 20]; // Just header + packet[0..2].copy_from_slice(&9u16.to_be_bytes()); // version + packet[2..4].copy_from_slice(&0u16.to_be_bytes()); // count + packet + } + + fn create_ipfix_packet() -> Vec { + let mut packet = vec![0u8; 16]; // IPFIX header + packet[0..2].copy_from_slice(&10u16.to_be_bytes()); // version + packet[2..4].copy_from_slice(&16u16.to_be_bytes()); // length + packet + } + + fn create_sflow_packet() -> Vec { + let mut packet = vec![0u8; 28]; // sFlow header + packet[0..4].copy_from_slice(&5u32.to_be_bytes()); // version + packet[4..8].copy_from_slice(&1u32.to_be_bytes()); // agent_address_type (IPv4) + packet[24..28].copy_from_slice(&0u32.to_be_bytes()); // num_samples + packet + } + + #[test] + fn test_protocol_detection() { + let template_cache = TemplateCache::new(100); + let config = test_config(); + let parser = ProtocolParser::new(&config, template_cache); + + // Test NetFlow v5 + let nf5_packet = create_netflow_v5_packet(); + let protocol = parser.detect_protocol(&nf5_packet); + assert_eq!(protocol, DetectedProtocol::NetflowV5); + + // Test NetFlow v9 + let nf9_packet = create_netflow_v9_packet(); + let protocol = parser.detect_protocol(&nf9_packet); + assert_eq!(protocol, DetectedProtocol::NetflowV9); + + // Test IPFIX + let ipfix_packet = create_ipfix_packet(); + let protocol = parser.detect_protocol(&ipfix_packet); + assert_eq!(protocol, DetectedProtocol::Ipfix); + + // Test sFlow + let sflow_packet = create_sflow_packet(); + let protocol = parser.detect_protocol(&sflow_packet); + assert_eq!(protocol, DetectedProtocol::Sflow); + + // Test unknown protocol + let unknown_packet = vec![0xFF, 0xFF, 0x00, 0x00]; + let protocol = parser.detect_protocol(&unknown_packet); + assert_eq!(protocol, DetectedProtocol::Unknown(0xFFFF)); + } + + #[test] + fn test_protocol_enabled_check() { + let config = NetflowConfig { + protocols: vec!["netflow_v5".to_string(), "ipfix".to_string()], + ..Default::default() + }; + + assert!(DetectedProtocol::NetflowV5.is_enabled(&config)); + assert!(DetectedProtocol::Ipfix.is_enabled(&config)); + assert!(!DetectedProtocol::NetflowV9.is_enabled(&config)); + assert!(!DetectedProtocol::Sflow.is_enabled(&config)); + assert!(!DetectedProtocol::Unknown(123).is_enabled(&config)); + } + + #[test] + fn test_parse_disabled_protocol() { + let template_cache = TemplateCache::new(100); + let config = NetflowConfig { + protocols: vec!["ipfix".to_string()], // Only IPFIX enabled + ..Default::default() + }; + let parser = ProtocolParser::new(&config, template_cache.clone()); + + // Try to parse NetFlow v5 (disabled) + let nf5_packet = create_netflow_v5_packet(); + let events = parser.parse(&nf5_packet, test_peer_addr(), &template_cache); + + // Should return empty (protocol disabled) + assert!(events.is_empty()); + } + + #[test] + fn test_parse_enabled_protocol() { + let template_cache = TemplateCache::new(100); + let config = test_config(); // All protocols enabled + let parser = ProtocolParser::new(&config, template_cache.clone()); + + // Parse NetFlow v5 + let nf5_packet = create_netflow_v5_packet(); + let events = parser.parse(&nf5_packet, test_peer_addr(), &template_cache); + + // Should return events + assert!(!events.is_empty()); + if let Event::Log(log) = &events[0] { + assert_eq!(log.get("flow_type").unwrap().as_str().unwrap(), "netflow_v5_record"); + } + } + + #[test] + fn test_unknown_protocol_event() { + let template_cache = TemplateCache::new(100); + let config = test_config(); + let parser = ProtocolParser::new(&config, template_cache.clone()); + + // Create unknown protocol packet + let unknown_packet = vec![0x99, 0x99, 0x00, 0x00, 0x11, 0x22, 0x33, 0x44]; + let events = parser.parse(&unknown_packet, test_peer_addr(), &template_cache); + + assert!(!events.is_empty()); + if let Event::Log(log) = &events[0] { + assert_eq!(log.get("flow_type").unwrap().as_str().unwrap(), "unknown"); + assert_eq!(log.get("version").unwrap().as_integer().unwrap(), 0x9999); + assert!(log.get("first_4_bytes").is_some()); + assert!(log.get("first_8_bytes").is_some()); + } + } + + #[test] + fn test_parse_error_event() { + let template_cache = TemplateCache::new(100); + let config = test_config(); + let parser = ProtocolParser::new(&config, template_cache.clone()); + + // Create malformed IPFIX packet (detected as IPFIX but fails during parsing) + // We'll create a packet that claims to be longer than it actually is + let mut malformed_packet = vec![0u8; 32]; // IPFIX header size + malformed_packet[0..2].copy_from_slice(&10u16.to_be_bytes()); // version (IPFIX) + malformed_packet[2..4].copy_from_slice(&64u16.to_be_bytes()); // length = 64 (but packet is only 32 bytes) + malformed_packet[4..8].copy_from_slice(&12345u32.to_be_bytes()); // export time + malformed_packet[8..12].copy_from_slice(&1u32.to_be_bytes()); // sequence number + malformed_packet[12..16].copy_from_slice(&1u32.to_be_bytes()); // observation domain + // The packet claims to be 64 bytes but is only 32 bytes, which should cause parsing to fail + let events = parser.parse(&malformed_packet, test_peer_addr(), &template_cache); + + assert!(!events.is_empty()); + if let Event::Log(log) = &events[0] { + assert_eq!(log.get("flow_type").unwrap().as_str().unwrap(), "parse_error"); + assert_eq!(log.get("detected_protocol").unwrap().as_str().unwrap(), "ipfix"); + assert!(log.get("parse_error").is_some()); + } + } + + #[test] + fn test_raw_data_inclusion() { + let template_cache = TemplateCache::new(100); + let config = NetflowConfig { + protocols: vec!["netflow_v5".to_string()], + ..Default::default() + }; + let parser = ProtocolParser::new(&config, template_cache.clone()); + + let nf5_packet = create_netflow_v5_packet(); + let events = parser.parse(&nf5_packet, test_peer_addr(), &template_cache); + + assert!(!events.is_empty()); + if let Event::Log(log) = &events[0] { + assert!(log.get("raw_data").is_some()); + } + } + + #[test] + fn test_empty_packet() { + let template_cache = TemplateCache::new(100); + let config = test_config(); + + let result = parse_flow_data(&[], test_peer_addr(), &template_cache, &config); + assert!(result.is_err()); + assert_eq!(result.unwrap_err(), "Empty packet received"); + } + + #[test] + fn test_protocol_stats() { + let template_cache = TemplateCache::new(100); + let config = NetflowConfig { + protocols: vec!["netflow_v5".to_string(), "ipfix".to_string()], + ..Default::default() + }; + let parser = ProtocolParser::new(&config, template_cache); + + let stats = parser.get_protocol_stats(); + assert_eq!(stats.total_enabled, 2); + assert!(stats.enabled_protocols.contains(&"netflow_v5".to_string())); + assert!(stats.enabled_protocols.contains(&"ipfix".to_string())); + } + + #[test] + fn test_protocol_as_str() { + assert_eq!(DetectedProtocol::NetflowV5.as_str(), "netflow_v5"); + assert_eq!(DetectedProtocol::NetflowV9.as_str(), "netflow_v9"); + assert_eq!(DetectedProtocol::Ipfix.as_str(), "ipfix"); + assert_eq!(DetectedProtocol::Sflow.as_str(), "sflow"); + assert_eq!(DetectedProtocol::Unknown(123).as_str(), "unknown"); + } + + #[test] + fn test_short_packet_detection() { + let template_cache = TemplateCache::new(100); + let config = test_config(); + let parser = ProtocolParser::new(&config, template_cache); + + // Single byte packet + let short_packet = vec![0x05]; + let protocol = parser.detect_protocol(&short_packet); + assert_eq!(protocol, DetectedProtocol::Unknown(0)); + + // Empty packet + let empty_packet = vec![]; + let protocol = parser.detect_protocol(&empty_packet); + assert_eq!(protocol, DetectedProtocol::Unknown(0)); + } + + #[test] + fn test_main_parse_function() { + let template_cache = TemplateCache::new(100); + let config = test_config(); + + let nf5_packet = create_netflow_v5_packet(); + let result = parse_flow_data(&nf5_packet, test_peer_addr(), &template_cache, &config); + + assert!(result.is_ok()); + let events = result.unwrap(); + assert!(!events.is_empty()); + } +} \ No newline at end of file diff --git a/src/sources/netflow/protocols/netflow_v5.rs b/src/sources/netflow/protocols/netflow_v5.rs new file mode 100644 index 0000000000000..0f3c8f6a08276 --- /dev/null +++ b/src/sources/netflow/protocols/netflow_v5.rs @@ -0,0 +1,1024 @@ +//! NetFlow v5 protocol parser. +//! +//! NetFlow v5 is a fixed-format flow export protocol that defines a standard +//! structure for flow records. Unlike template-based protocols (v9/IPFIX), +//! NetFlow v5 has a rigid 48-byte record format that contains predefined fields. + +use crate::sources::netflow::events::*; +use crate::sources::netflow::fields::FieldParser; + +use std::net::SocketAddr; +use tracing::{debug, warn}; +use vector_lib::event::{Event, LogEvent}; + + +/// NetFlow v5 protocol constants +const NETFLOW_V5_VERSION: u16 = 5; +const NETFLOW_V5_HEADER_SIZE: usize = 24; +const NETFLOW_V5_RECORD_SIZE: usize = 48; +const MAX_FLOW_COUNT: u16 = 1000; // Sanity check for flow count + +/// NetFlow v5 packet header structure +#[derive(Debug, Clone)] +pub struct NetflowV5Header { + pub version: u16, + pub count: u16, + pub sys_uptime: u32, + pub unix_secs: u32, + pub unix_nsecs: u32, + pub flow_sequence: u32, + pub engine_type: u8, + pub engine_id: u8, + pub sampling_interval: u16, +} + +impl NetflowV5Header { + /// Parse NetFlow v5 header from packet data + pub fn from_bytes(data: &[u8]) -> Result { + if data.len() < NETFLOW_V5_HEADER_SIZE { + return Err(format!( + "Packet too short for NetFlow v5 header: {} bytes, need {}", + data.len(), + NETFLOW_V5_HEADER_SIZE + )); + } + + let version = u16::from_be_bytes([data[0], data[1]]); + if version != NETFLOW_V5_VERSION { + return Err(format!( + "Invalid NetFlow v5 version: {}, expected {}", + version, NETFLOW_V5_VERSION + )); + } + + let count = u16::from_be_bytes([data[2], data[3]]); + if count > MAX_FLOW_COUNT { + return Err(format!( + "Unreasonable flow count: {}, maximum expected {}", + count, MAX_FLOW_COUNT + )); + } + + // Validate packet length matches expected size + let expected_length = NETFLOW_V5_HEADER_SIZE + (count as usize * NETFLOW_V5_RECORD_SIZE); + if data.len() < expected_length { + return Err(format!( + "Packet too short for {} flow records: {} bytes, need {}", + count, data.len(), expected_length + )); + } + + Ok(Self { + version, + count, + sys_uptime: u32::from_be_bytes([data[4], data[5], data[6], data[7]]), + unix_secs: u32::from_be_bytes([data[8], data[9], data[10], data[11]]), + unix_nsecs: u32::from_be_bytes([data[12], data[13], data[14], data[15]]), + flow_sequence: u32::from_be_bytes([data[16], data[17], data[18], data[19]]), + engine_type: data[20], + engine_id: data[21], + sampling_interval: u16::from_be_bytes([data[22], data[23]]), + }) + } + + /// Create base log event with header information + pub fn to_log_event(&self) -> LogEvent { + let mut log_event = LogEvent::default(); + log_event.insert("flow_type", "netflow_v5"); + log_event.insert("version", self.version); + log_event.insert("count", self.count); + log_event.insert("sys_uptime", self.sys_uptime); + log_event.insert("unix_secs", self.unix_secs); + log_event.insert("unix_nsecs", self.unix_nsecs); + log_event.insert("flow_sequence", self.flow_sequence); + log_event.insert("engine_type", self.engine_type); + log_event.insert("engine_id", self.engine_id); + log_event.insert("sampling_interval", self.sampling_interval); + log_event + } + + /// Get sampling rate from sampling interval + pub fn sampling_rate(&self) -> u32 { + if self.sampling_interval == 0 { + 1 // No sampling + } else { + // Upper 14 bits are the sampling mode, lower 14 bits are the interval + let interval = self.sampling_interval & 0x3FFF; + if interval == 0 { 1 } else { interval as u32 } + } + } + + /// Check if sampling is enabled + pub fn is_sampled(&self) -> bool { + self.sampling_interval != 0 + } +} + +/// NetFlow v5 flow record structure +#[derive(Debug, Clone)] +pub struct NetflowV5Record { + pub src_addr: u32, + pub dst_addr: u32, + pub next_hop: u32, + pub input: u16, + pub output: u16, + pub d_pkts: u32, + pub d_octets: u32, + pub first: u32, + pub last: u32, + pub src_port: u16, + pub dst_port: u16, + pub pad1: u8, + pub tcp_flags: u8, + pub prot: u8, + pub tos: u8, + pub src_as: u16, + pub dst_as: u16, + pub src_mask: u8, + pub dst_mask: u8, + pub pad2: u16, +} + +impl NetflowV5Record { + /// Parse NetFlow v5 record from packet data + pub fn from_bytes(data: &[u8]) -> Result { + if data.len() < NETFLOW_V5_RECORD_SIZE { + return Err(format!( + "Insufficient data for NetFlow v5 record: {} bytes, need {}", + data.len(), + NETFLOW_V5_RECORD_SIZE + )); + } + + Ok(Self { + src_addr: u32::from_be_bytes([data[0], data[1], data[2], data[3]]), + dst_addr: u32::from_be_bytes([data[4], data[5], data[6], data[7]]), + next_hop: u32::from_be_bytes([data[8], data[9], data[10], data[11]]), + input: u16::from_be_bytes([data[12], data[13]]), + output: u16::from_be_bytes([data[14], data[15]]), + d_pkts: u32::from_be_bytes([data[16], data[17], data[18], data[19]]), + d_octets: u32::from_be_bytes([data[20], data[21], data[22], data[23]]), + first: u32::from_be_bytes([data[24], data[25], data[26], data[27]]), + last: u32::from_be_bytes([data[28], data[29], data[30], data[31]]), + src_port: u16::from_be_bytes([data[32], data[33]]), + dst_port: u16::from_be_bytes([data[34], data[35]]), + pad1: data[36], + tcp_flags: data[37], + prot: data[38], + tos: data[39], + src_as: u16::from_be_bytes([data[40], data[41]]), + dst_as: u16::from_be_bytes([data[42], data[43]]), + src_mask: data[44], + dst_mask: data[45], + pad2: u16::from_be_bytes([data[46], data[47]]), + }) + } + + /// Convert IPv4 address to string + fn ipv4_to_string(addr: u32) -> String { + format!( + "{}.{}.{}.{}", + (addr >> 24) & 0xFF, + (addr >> 16) & 0xFF, + (addr >> 8) & 0xFF, + addr & 0xFF + ) + } + + /// Get protocol name from protocol number + fn get_protocol_name(protocol: u8) -> &'static str { + match protocol { + 1 => "ICMP", + 6 => "TCP", + 17 => "UDP", + 47 => "GRE", + 50 => "ESP", + 51 => "AH", + 89 => "OSPF", + 132 => "SCTP", + _ => "Other", + } + } + + /// Convert record to log event + pub fn to_log_event(&self, record_number: usize, resolve_protocols: bool) -> LogEvent { + let mut log_event = LogEvent::default(); + + // Flow identification + log_event.insert("flow_type", "netflow_v5_record"); + log_event.insert("record_number", record_number); + + // Network addresses + log_event.insert("src_addr", Self::ipv4_to_string(self.src_addr)); + log_event.insert("dst_addr", Self::ipv4_to_string(self.dst_addr)); + log_event.insert("next_hop", Self::ipv4_to_string(self.next_hop)); + + // Interface information + log_event.insert("input_interface", self.input); + log_event.insert("output_interface", self.output); + + // Traffic counters + log_event.insert("packets", self.d_pkts); + log_event.insert("octets", self.d_octets); + + // Timing information + log_event.insert("first_switched", self.first); + log_event.insert("last_switched", self.last); + + // Port information + log_event.insert("src_port", self.src_port); + log_event.insert("dst_port", self.dst_port); + + // Protocol information + log_event.insert("protocol", self.prot); + if resolve_protocols { + log_event.insert("protocol_name", Self::get_protocol_name(self.prot)); + } + + // TCP flags + log_event.insert("tcp_flags", self.tcp_flags); + if self.prot == 6 { // TCP + log_event.insert("tcp_flags_urg", (self.tcp_flags & 0x20) != 0); + log_event.insert("tcp_flags_ack", (self.tcp_flags & 0x10) != 0); + log_event.insert("tcp_flags_psh", (self.tcp_flags & 0x08) != 0); + log_event.insert("tcp_flags_rst", (self.tcp_flags & 0x04) != 0); + log_event.insert("tcp_flags_syn", (self.tcp_flags & 0x02) != 0); + log_event.insert("tcp_flags_fin", (self.tcp_flags & 0x01) != 0); + } + + // Type of Service + log_event.insert("tos", self.tos); + log_event.insert("dscp", (self.tos >> 2) & 0x3F); // DSCP is upper 6 bits + log_event.insert("ecn", self.tos & 0x03); // ECN is lower 2 bits + + // AS information + log_event.insert("src_as", self.src_as); + log_event.insert("dst_as", self.dst_as); + + // Subnet mask information + log_event.insert("src_mask", self.src_mask); + log_event.insert("dst_mask", self.dst_mask); + + // Calculate flow duration if possible + if self.last > self.first { + log_event.insert("flow_duration_ms", self.last - self.first); + } + + // Calculate bytes per packet + if self.d_pkts > 0 { + log_event.insert("bytes_per_packet", self.d_octets / self.d_pkts); + } + + // Flow direction heuristics + let flow_direction = self.determine_flow_direction(); + log_event.insert("flow_direction", flow_direction); + + log_event + } + + /// Determine flow direction based on port analysis + fn determine_flow_direction(&self) -> &'static str { + // Common server ports + let server_ports = [ + 21, 22, 23, 25, 53, 80, 110, 143, 443, 993, 995, + 1433, 3306, 5432, 6379, 27017 + ]; + + let src_is_server = server_ports.contains(&self.src_port); + let dst_is_server = server_ports.contains(&self.dst_port); + + match (src_is_server, dst_is_server) { + (true, false) => "outbound", // Server to client + (false, true) => "inbound", // Client to server + _ => { + // Use port number heuristic + if self.src_port < self.dst_port { + "outbound" + } else if self.dst_port < self.src_port { + "inbound" + } else { + "unknown" + } + } + } + } + + /// Validate record data for reasonableness + pub fn validate(&self) -> Result<(), String> { + // Check for obviously invalid data + if self.d_pkts == 0 && self.d_octets > 0 { + return Err("Invalid record: zero packets but non-zero octets".to_string()); + } + + if self.d_octets > 0 && self.d_pkts > 0 { + let bytes_per_packet = self.d_octets / self.d_pkts; + if bytes_per_packet < 20 || bytes_per_packet > 65535 { + return Err(format!("Invalid bytes per packet: {}", bytes_per_packet)); + } + } + + // Check timing - relaxed validation for timing anomalies + if self.last < self.first { + let time_diff = self.first - self.last; + // Only reject if the difference is more than 1 hour (3600 seconds) + if time_diff > 3600 { + return Err(format!( + "Invalid timing: last_switched ({}) < first_switched ({}) by {} seconds", + self.last, self.first, time_diff + )); + } + } + + // Check for private/reserved addresses in next_hop if it's not zero + if self.next_hop != 0 { + let is_private = self.is_private_ipv4(self.next_hop); + let is_zero = self.next_hop == 0; + let is_broadcast = self.next_hop == 0xFFFFFFFF; + + if !is_private && !is_zero && !is_broadcast { + // Public next hop - this is normal for routed traffic + } + } + + Ok(()) + } + + /// Check if IPv4 address is in private range + fn is_private_ipv4(&self, addr: u32) -> bool { + let a = (addr >> 24) & 0xFF; + let b = (addr >> 16) & 0xFF; + + // 10.0.0.0/8, 172.16.0.0/12, 192.168.0.0/16 + a == 10 || (a == 172 && b >= 16 && b <= 31) || (a == 192 && b == 168) + } +} + +/// NetFlow v5 protocol parser +pub struct NetflowV5Parser { + strict_validation: bool, +} + +impl NetflowV5Parser { + /// Create a new NetFlow v5 parser + pub fn new(_field_parser: FieldParser, strict_validation: bool) -> Self { + Self { strict_validation } + } + + /// Check if packet data looks like NetFlow v5 + pub fn can_parse(data: &[u8]) -> bool { + if data.len() < NETFLOW_V5_HEADER_SIZE { + return false; + } + + let version = u16::from_be_bytes([data[0], data[1]]); + if version != NETFLOW_V5_VERSION { + return false; + } + + let count = u16::from_be_bytes([data[2], data[3]]); + if count == 0 || count > MAX_FLOW_COUNT { + return false; + } + + // Check if packet length is reasonable + let expected_length = NETFLOW_V5_HEADER_SIZE + (count as usize * NETFLOW_V5_RECORD_SIZE); + if data.len() < expected_length { + return false; + } + + true + } + + /// Parse NetFlow v5 packet and return events + pub fn parse( + &self, + data: &[u8], + peer_addr: SocketAddr, + include_raw_data: bool, + ) -> Result, String> { + let mut events = Vec::new(); + + // Parse header + let header = NetflowV5Header::from_bytes(data)?; + + debug!( + "Parsing NetFlow v5 packet: version={}, count={}, sequence={}", + header.version, header.count, header.flow_sequence + ); + + // Create base event with header info + let mut base_event = header.to_log_event(); + base_event.insert("sampling_rate", header.sampling_rate()); + base_event.insert("is_sampled", header.is_sampled()); + + if include_raw_data { + let encoded = base64::Engine::encode(&base64::engine::general_purpose::STANDARD, data); + base_event.insert("raw_data", encoded); + } + + // Parse flow records + let mut record_offset = NETFLOW_V5_HEADER_SIZE; + let mut valid_records = 0; + let mut invalid_records = 0; + + for i in 0..header.count { + let record_end = record_offset + NETFLOW_V5_RECORD_SIZE; + if record_end > data.len() { + warn!( + "Insufficient data for record {}: offset={}, need={}", + i, record_offset, NETFLOW_V5_RECORD_SIZE + ); + break; + } + + let record_data = &data[record_offset..record_end]; + match NetflowV5Record::from_bytes(record_data) { + Ok(record) => { + if self.strict_validation { + if let Err(validation_error) = record.validate() { + warn!( + "Invalid NetFlow v5 record {}: {}", + i, validation_error + ); + invalid_records += 1; + record_offset = record_end; + continue; + } + } else { + if let Err(validation_error) = record.validate() { + debug!( + "NetFlow v5 record {} validation warning: {}", + i, validation_error + ); + } + } + + // Convert to log event + let mut record_event = record.to_log_event(i as usize, true); // resolve_protocols = true + + // Add header context to each record + record_event.insert("packet_sequence", header.flow_sequence); + record_event.insert("engine_type", header.engine_type); + record_event.insert("engine_id", header.engine_id); + record_event.insert("sys_uptime", header.sys_uptime); + record_event.insert("unix_secs", header.unix_secs); + record_event.insert("sampling_rate", header.sampling_rate()); + + // Add raw data if requested + if include_raw_data { + let encoded = base64::Engine::encode(&base64::engine::general_purpose::STANDARD, data); + record_event.insert("raw_data", encoded); + } + + events.push(Event::Log(record_event)); + valid_records += 1; + + emit!(DataRecordParsed { + template_id: 0, // NetFlow v5 doesn't use templates + fields_parsed: 20, // Approximate field count + record_size: NETFLOW_V5_RECORD_SIZE, + peer_addr, + protocol: "netflow_v5", + }); + } + Err(e) => { + warn!("Failed to parse NetFlow v5 record {}: {}", i, e); + invalid_records += 1; + } + } + + record_offset = record_end; + } + + // If no valid records were parsed, include header event + if events.is_empty() { + base_event.insert("valid_records", valid_records); + base_event.insert("invalid_records", invalid_records); + events.push(Event::Log(base_event)); + } + + emit!(NetflowV5PacketProcessed { + peer_addr, + total_records: header.count as usize, + valid_records, + invalid_records, + event_count: events.len(), + }); + + Ok(events) + } +} + +/// NetFlow v5 specific events +#[derive(Debug)] +pub struct NetflowV5PacketProcessed { + pub peer_addr: SocketAddr, + pub total_records: usize, + pub valid_records: usize, + pub invalid_records: usize, + pub event_count: usize, +} + +impl vector_lib::internal_event::InternalEvent for NetflowV5PacketProcessed { + fn emit(self) { + debug!( + message = "NetFlow v5 packet processed", + peer_addr = %self.peer_addr, + total_records = self.total_records, + valid_records = self.valid_records, + invalid_records = self.invalid_records, + event_count = self.event_count, + ); + + if self.invalid_records > 0 { + warn!( + message = "NetFlow v5 packet contained invalid records", + peer_addr = %self.peer_addr, + invalid_records = self.invalid_records, + total_records = self.total_records, + ); + } + } +} + +#[cfg(test)] +mod tests { + use super::*; + use crate::sources::netflow::config::NetflowConfig; + use crate::sources::netflow::fields::FieldParser; + use std::net::{IpAddr, Ipv4Addr}; + + fn test_peer_addr() -> SocketAddr { + SocketAddr::new(IpAddr::V4(Ipv4Addr::new(192, 168, 1, 100)), 2055) + } + + fn create_netflow_v5_packet(record_count: u16) -> Vec { + let mut packet = vec![0u8; NETFLOW_V5_HEADER_SIZE + (record_count as usize * NETFLOW_V5_RECORD_SIZE)]; + + // Header + packet[0..2].copy_from_slice(&5u16.to_be_bytes()); // version + packet[2..4].copy_from_slice(&record_count.to_be_bytes()); // count + packet[4..8].copy_from_slice(&12345u32.to_be_bytes()); // sys_uptime + packet[8..12].copy_from_slice(&1609459200u32.to_be_bytes()); // unix_secs + packet[12..16].copy_from_slice(&0u32.to_be_bytes()); // unix_nsecs + packet[16..20].copy_from_slice(&100u32.to_be_bytes()); // flow_sequence + packet[20] = 0; // engine_type + packet[21] = 0; // engine_id + packet[22..24].copy_from_slice(&0u16.to_be_bytes()); // sampling_interval + + // Add sample records + for i in 0..record_count { + let record_offset = NETFLOW_V5_HEADER_SIZE + (i as usize * NETFLOW_V5_RECORD_SIZE); + + // Sample flow record + packet[record_offset..record_offset + 4].copy_from_slice(&0xC0A80101u32.to_be_bytes()); // src: 192.168.1.1 + packet[record_offset + 4..record_offset + 8].copy_from_slice(&0x0A000001u32.to_be_bytes()); // dst: 10.0.0.1 + packet[record_offset + 16..record_offset + 20].copy_from_slice(&10u32.to_be_bytes()); // packets + packet[record_offset + 20..record_offset + 24].copy_from_slice(&1500u32.to_be_bytes()); // octets + packet[record_offset + 32..record_offset + 34].copy_from_slice(&80u16.to_be_bytes()); // src_port + packet[record_offset + 34..record_offset + 36].copy_from_slice(&443u16.to_be_bytes()); // dst_port + packet[record_offset + 38] = 6; // protocol (TCP) + } + + packet + } + + #[test] + fn test_netflow_v5_header_parsing() { + let packet = create_netflow_v5_packet(1); + let header = NetflowV5Header::from_bytes(&packet).unwrap(); + + assert_eq!(header.version, 5); + assert_eq!(header.count, 1); + assert_eq!(header.sys_uptime, 12345); + assert_eq!(header.unix_secs, 1609459200); + assert_eq!(header.flow_sequence, 100); + assert_eq!(header.engine_type, 0); + assert_eq!(header.engine_id, 0); + assert_eq!(header.sampling_interval, 0); + } + + #[test] + fn test_invalid_netflow_v5_header() { + // Too short + let short_packet = vec![0u8; 10]; + assert!(NetflowV5Header::from_bytes(&short_packet).is_err()); + + // Wrong version + let mut wrong_version = create_netflow_v5_packet(1); + wrong_version[0..2].copy_from_slice(&9u16.to_be_bytes()); + assert!(NetflowV5Header::from_bytes(&wrong_version).is_err()); + + // Too many flows + let mut too_many = create_netflow_v5_packet(1); + too_many[2..4].copy_from_slice(&2000u16.to_be_bytes()); + assert!(NetflowV5Header::from_bytes(&too_many).is_err()); + + // Packet too short for claimed record count + let mut short_for_records = vec![0u8; NETFLOW_V5_HEADER_SIZE + 10]; + short_for_records[0..2].copy_from_slice(&5u16.to_be_bytes()); // version + short_for_records[2..4].copy_from_slice(&2u16.to_be_bytes()); // count = 2 + assert!(NetflowV5Header::from_bytes(&short_for_records).is_err()); + } + + #[test] + fn test_can_parse() { + // Valid NetFlow v5 + let packet = create_netflow_v5_packet(1); + assert!(NetflowV5Parser::can_parse(&packet)); + + // Invalid version + let mut invalid_version = packet.clone(); + invalid_version[0..2].copy_from_slice(&10u16.to_be_bytes()); + assert!(!NetflowV5Parser::can_parse(&invalid_version)); + + // Zero count + let mut zero_count = packet.clone(); + zero_count[2..4].copy_from_slice(&0u16.to_be_bytes()); + assert!(!NetflowV5Parser::can_parse(&zero_count)); + + // Too short + let short_packet = vec![0u8; 10]; + assert!(!NetflowV5Parser::can_parse(&short_packet)); + } + + #[test] + fn test_record_parsing() { + let record_data = vec![ + 192, 168, 1, 1, // src_addr + 10, 0, 0, 1, // dst_addr + 0, 0, 0, 0, // next_hop + 0, 1, // input + 0, 2, // output + 0, 0, 0, 10, // d_pkts + 0, 0, 5, 220, // d_octets (1500) + 0, 0, 0, 100, // first + 0, 0, 0, 200, // last + 0, 80, // src_port + 1, 187, // dst_port (443) + 0, // pad1 + 24, // tcp_flags (ACK+PSH) + 6, // protocol (TCP) + 0, // tos + 0, 100, // src_as + 0, 200, // dst_as + 24, // src_mask + 8, // dst_mask + 0, 0, // pad2 + ]; + + let record = NetflowV5Record::from_bytes(&record_data).unwrap(); + + assert_eq!(record.src_addr, 0xC0A80101); // 192.168.1.1 + assert_eq!(record.dst_addr, 0x0A000001); // 10.0.0.1 + assert_eq!(record.src_port, 80); + assert_eq!(record.dst_port, 443); + assert_eq!(record.prot, 6); + assert_eq!(record.d_pkts, 10); + assert_eq!(record.d_octets, 1500); + assert_eq!(record.tcp_flags, 24); + } + + #[test] + fn test_record_to_log_event() { + let record_data = vec![ + 192, 168, 1, 1, // src_addr + 10, 0, 0, 1, // dst_addr + 0, 0, 0, 0, // next_hop + 0, 1, // input + 0, 2, // output + 0, 0, 0, 10, // d_pkts + 0, 0, 5, 220, // d_octets (1500) + 0, 0, 0, 100, // first + 0, 0, 0, 200, // last + 0, 80, // src_port + 1, 187, // dst_port (443) + 0, // pad1 + 24, // tcp_flags (ACK+PSH) + 6, // protocol (TCP) + 0, // tos + 0, 100, // src_as + 0, 200, // dst_as + 24, // src_mask + 8, // dst_mask + 0, 0, // pad2 + ]; + + let record = NetflowV5Record::from_bytes(&record_data).unwrap(); + let log_event = record.to_log_event(0, true); + + assert_eq!(log_event.get("flow_type").unwrap().as_str().unwrap(), "netflow_v5_record"); + assert_eq!(log_event.get("src_addr").unwrap().as_str().unwrap(), "192.168.1.1"); + assert_eq!(log_event.get("dst_addr").unwrap().as_str().unwrap(), "10.0.0.1"); + assert_eq!(log_event.get("src_port").unwrap().as_integer().unwrap(), 80); + assert_eq!(log_event.get("dst_port").unwrap().as_integer().unwrap(), 443); + assert_eq!(log_event.get("protocol").unwrap().as_integer().unwrap(), 6); + assert_eq!(log_event.get("protocol_name").unwrap().as_str().unwrap(), "TCP"); + assert_eq!(log_event.get("packets").unwrap().as_integer().unwrap(), 10); + assert_eq!(log_event.get("octets").unwrap().as_integer().unwrap(), 1500); + assert_eq!(log_event.get("flow_duration_ms").unwrap().as_integer().unwrap(), 100); + assert_eq!(log_event.get("bytes_per_packet").unwrap().as_integer().unwrap(), 150); + + // Check TCP flags + assert_eq!(log_event.get("tcp_flags_ack").unwrap().as_boolean().unwrap(), true); + assert_eq!(log_event.get("tcp_flags_psh").unwrap().as_boolean().unwrap(), true); + assert_eq!(log_event.get("tcp_flags_syn").unwrap().as_boolean().unwrap(), false); + + // Check DSCP/ECN + assert_eq!(log_event.get("dscp").unwrap().as_integer().unwrap(), 0); + assert_eq!(log_event.get("ecn").unwrap().as_integer().unwrap(), 0); + } + + #[test] + fn test_sampling_rate_calculation() { + let mut packet = create_netflow_v5_packet(1); + + // Test no sampling + packet[22..24].copy_from_slice(&0u16.to_be_bytes()); + let header = NetflowV5Header::from_bytes(&packet).unwrap(); + assert_eq!(header.sampling_rate(), 1); + assert!(!header.is_sampled()); + + // Test 1:100 sampling + packet[22..24].copy_from_slice(&100u16.to_be_bytes()); + let header = NetflowV5Header::from_bytes(&packet).unwrap(); + assert_eq!(header.sampling_rate(), 100); + assert!(header.is_sampled()); + } + + #[test] + fn test_flow_direction_detection() { + let mut record_data = vec![0u8; NETFLOW_V5_RECORD_SIZE]; + + // Client to server (high port to low port) + record_data[32..34].copy_from_slice(&50000u16.to_be_bytes()); // src_port + record_data[34..36].copy_from_slice(&80u16.to_be_bytes()); // dst_port (HTTP) + let record = NetflowV5Record::from_bytes(&record_data).unwrap(); + assert_eq!(record.determine_flow_direction(), "inbound"); + + // Server to client (low port to high port) + record_data[32..34].copy_from_slice(&80u16.to_be_bytes()); // src_port (HTTP) + record_data[34..36].copy_from_slice(&50000u16.to_be_bytes()); // dst_port + let record = NetflowV5Record::from_bytes(&record_data).unwrap(); + assert_eq!(record.determine_flow_direction(), "outbound"); + } + + #[test] + fn test_record_validation() { + let mut record_data = vec![0u8; NETFLOW_V5_RECORD_SIZE]; + + // Valid record + record_data[16..20].copy_from_slice(&10u32.to_be_bytes()); // packets + record_data[20..24].copy_from_slice(&1500u32.to_be_bytes()); // octets + record_data[24..28].copy_from_slice(&100u32.to_be_bytes()); // first + record_data[28..32].copy_from_slice(&200u32.to_be_bytes()); // last + let record = NetflowV5Record::from_bytes(&record_data).unwrap(); + assert!(record.validate().is_ok()); + + // Invalid: zero packets but non-zero octets + record_data[16..20].copy_from_slice(&0u32.to_be_bytes()); // packets = 0 + record_data[20..24].copy_from_slice(&1500u32.to_be_bytes()); // octets = 1500 + let record = NetflowV5Record::from_bytes(&record_data).unwrap(); + assert!(record.validate().is_err()); + + // Invalid: last < first by more than 1 hour (3600 seconds) + record_data[16..20].copy_from_slice(&10u32.to_be_bytes()); // packets + record_data[20..24].copy_from_slice(&1500u32.to_be_bytes()); // octets + record_data[24..28].copy_from_slice(&4000u32.to_be_bytes()); // first + record_data[28..32].copy_from_slice(&100u32.to_be_bytes()); // last (3900 seconds difference) + let record = NetflowV5Record::from_bytes(&record_data).unwrap(); + assert!(record.validate().is_err()); + } + + #[test] + fn test_private_ip_detection() { + let record = NetflowV5Record { + src_addr: 0, dst_addr: 0, next_hop: 0, input: 0, output: 0, + d_pkts: 0, d_octets: 0, first: 0, last: 0, src_port: 0, + dst_port: 0, pad1: 0, tcp_flags: 0, prot: 0, tos: 0, + src_as: 0, dst_as: 0, src_mask: 0, dst_mask: 0, pad2: 0, + }; + + // 10.0.0.1 + assert!(record.is_private_ipv4(0x0A000001)); + // 172.16.0.1 + assert!(record.is_private_ipv4(0xAC100001)); + // 192.168.1.1 + assert!(record.is_private_ipv4(0xC0A80101)); + // 8.8.8.8 (public) + assert!(!record.is_private_ipv4(0x08080808)); + } + + #[test] + fn test_full_packet_parsing() { + let config = NetflowConfig::default(); + let field_parser = FieldParser::new(&config); + let parser = NetflowV5Parser::new(field_parser, true); + + let packet = create_netflow_v5_packet(2); + let events = parser.parse(&packet, test_peer_addr(), false).unwrap(); + + // Should get 2 record events + assert_eq!(events.len(), 2); + + for (i, event) in events.iter().enumerate() { + if let Event::Log(log) = event { + assert_eq!(log.get("flow_type").unwrap().as_str().unwrap(), "netflow_v5_record"); + assert_eq!(log.get("record_number").unwrap().as_integer().unwrap(), i as i64); + assert_eq!(log.get("src_addr").unwrap().as_str().unwrap(), "192.168.1.1"); + assert_eq!(log.get("dst_addr").unwrap().as_str().unwrap(), "10.0.0.1"); + assert_eq!(log.get("packet_sequence").unwrap().as_integer().unwrap(), 100); + } + } + } + + #[test] + fn test_parsing_with_raw_data() { + let config = NetflowConfig::default(); + let field_parser = FieldParser::new(&config); + let parser = NetflowV5Parser::new(field_parser, true); + + let packet = create_netflow_v5_packet(1); + + // Test with raw data inclusion + let events = parser.parse(&packet, test_peer_addr(), true).unwrap(); + assert!(!events.is_empty()); + + // Raw data should be included in individual records when requested + for event in &events { + if let Event::Log(log) = event { + let flow_type = log.get("flow_type").unwrap().as_str().unwrap(); + if flow_type == "netflow_v5_record" { + // Records should include raw data when requested + assert!(log.get("raw_data").is_some()); + } + } + } + } + + #[test] + fn test_zero_records_packet() { + let config = NetflowConfig::default(); + let field_parser = FieldParser::new(&config); + let parser = NetflowV5Parser::new(field_parser, true); + + let packet = create_netflow_v5_packet(0); + let events = parser.parse(&packet, test_peer_addr(), false).unwrap(); + + // Should get header event only + assert_eq!(events.len(), 1); + + if let Event::Log(log) = &events[0] { + assert_eq!(log.get("flow_type").unwrap().as_str().unwrap(), "netflow_v5"); + assert_eq!(log.get("count").unwrap().as_integer().unwrap(), 0); + } + } + + #[test] + fn test_malformed_record_handling() { + let config = NetflowConfig::default(); + let field_parser = FieldParser::new(&config); + let parser = NetflowV5Parser::new(field_parser, true); + + // Create packet with malformed record (invalid packet count) + let mut packet = create_netflow_v5_packet(1); + // Make the record invalid by setting packets=0, octets=1000 + let record_offset = NETFLOW_V5_HEADER_SIZE; + packet[record_offset + 16..record_offset + 20].copy_from_slice(&0u32.to_be_bytes()); // packets = 0 + packet[record_offset + 20..record_offset + 24].copy_from_slice(&1000u32.to_be_bytes()); // octets = 1000 + + let events = parser.parse(&packet, test_peer_addr(), false).unwrap(); + + // Should handle gracefully, likely with header event + assert!(!events.is_empty()); + } + + #[test] + fn test_invalid_bytes_per_packet() { + let mut record_data = vec![0u8; NETFLOW_V5_RECORD_SIZE]; + + // Invalid: 1 packet, 10 bytes (too small) + record_data[16..20].copy_from_slice(&1u32.to_be_bytes()); // packets = 1 + record_data[20..24].copy_from_slice(&10u32.to_be_bytes()); // octets = 10 + let record = NetflowV5Record::from_bytes(&record_data).unwrap(); + assert!(record.validate().is_err()); + + // Invalid: 1 packet, 100000 bytes (too large) + record_data[20..24].copy_from_slice(&100000u32.to_be_bytes()); // octets = 100000 + let record = NetflowV5Record::from_bytes(&record_data).unwrap(); + assert!(record.validate().is_err()); + } + + #[test] + fn test_tcp_flags_parsing() { + let mut record_data = vec![0u8; NETFLOW_V5_RECORD_SIZE]; + + // Set TCP protocol and flags + record_data[38] = 6; // protocol = TCP + record_data[37] = 0b00010010; // SYN + ACK flags + + let record = NetflowV5Record::from_bytes(&record_data).unwrap(); + let log_event = record.to_log_event(0, true); + + assert_eq!(log_event.get("tcp_flags_syn").unwrap().as_boolean().unwrap(), true); + assert_eq!(log_event.get("tcp_flags_ack").unwrap().as_boolean().unwrap(), true); + assert_eq!(log_event.get("tcp_flags_fin").unwrap().as_boolean().unwrap(), false); + assert_eq!(log_event.get("tcp_flags_rst").unwrap().as_boolean().unwrap(), false); + assert_eq!(log_event.get("tcp_flags_psh").unwrap().as_boolean().unwrap(), false); + assert_eq!(log_event.get("tcp_flags_urg").unwrap().as_boolean().unwrap(), false); + } + + #[test] + fn test_dscp_ecn_parsing() { + let mut record_data = vec![0u8; NETFLOW_V5_RECORD_SIZE]; + + // Set TOS field: DSCP=26 (AF31), ECN=1 + record_data[39] = (26 << 2) | 1; // DSCP in upper 6 bits, ECN in lower 2 + + let record = NetflowV5Record::from_bytes(&record_data).unwrap(); + let log_event = record.to_log_event(0, true); + + assert_eq!(log_event.get("tos").unwrap().as_integer().unwrap(), (26 << 2) | 1); + assert_eq!(log_event.get("dscp").unwrap().as_integer().unwrap(), 26); + assert_eq!(log_event.get("ecn").unwrap().as_integer().unwrap(), 1); + } + + #[test] + fn test_multiple_records_with_validation() { + let config = NetflowConfig::default(); + let field_parser = FieldParser::new(&config); + let parser = NetflowV5Parser::new(field_parser, true); + + // Create packet with mix of valid and invalid records + let mut packet = create_netflow_v5_packet(3); + + // Make second record invalid (zero packets, non-zero octets) + let second_record_offset = NETFLOW_V5_HEADER_SIZE + NETFLOW_V5_RECORD_SIZE; + packet[second_record_offset + 16..second_record_offset + 20].copy_from_slice(&0u32.to_be_bytes()); // packets = 0 + packet[second_record_offset + 20..second_record_offset + 24].copy_from_slice(&1500u32.to_be_bytes()); // octets = 1500 + + let events = parser.parse(&packet, test_peer_addr(), false).unwrap(); + + // Should get 2 valid records (first and third) + assert_eq!(events.len(), 2); + + // Check record numbers + if let Event::Log(log) = &events[0] { + assert_eq!(log.get("record_number").unwrap().as_integer().unwrap(), 0); + } + if let Event::Log(log) = &events[1] { + assert_eq!(log.get("record_number").unwrap().as_integer().unwrap(), 2); + } + } + + #[test] + fn test_protocol_name_resolution() { + let mut record_data = vec![0u8; NETFLOW_V5_RECORD_SIZE]; + + // Test various protocols + let protocols = vec![ + (1, "ICMP"), + (6, "TCP"), + (17, "UDP"), + (47, "GRE"), + (50, "ESP"), + (89, "OSPF"), + (132, "SCTP"), + (255, "Other"), + ]; + + for (proto_num, proto_name) in protocols { + record_data[38] = proto_num; + let record = NetflowV5Record::from_bytes(&record_data).unwrap(); + let log_event = record.to_log_event(0, true); + + assert_eq!(log_event.get("protocol").unwrap().as_integer().unwrap(), proto_num as i64); + assert_eq!(log_event.get("protocol_name").unwrap().as_str().unwrap(), proto_name); + } + } + + #[test] + fn test_header_to_log_event() { + let packet = create_netflow_v5_packet(1); + let header = NetflowV5Header::from_bytes(&packet).unwrap(); + let log_event = header.to_log_event(); + + assert_eq!(log_event.get("flow_type").unwrap().as_str().unwrap(), "netflow_v5"); + assert_eq!(log_event.get("version").unwrap().as_integer().unwrap(), 5); + assert_eq!(log_event.get("count").unwrap().as_integer().unwrap(), 1); + assert_eq!(log_event.get("sys_uptime").unwrap().as_integer().unwrap(), 12345); + assert_eq!(log_event.get("unix_secs").unwrap().as_integer().unwrap(), 1609459200); + assert_eq!(log_event.get("flow_sequence").unwrap().as_integer().unwrap(), 100); + assert_eq!(log_event.get("engine_type").unwrap().as_integer().unwrap(), 0); + assert_eq!(log_event.get("engine_id").unwrap().as_integer().unwrap(), 0); + assert_eq!(log_event.get("sampling_interval").unwrap().as_integer().unwrap(), 0); + } + + #[test] + fn test_ipv4_address_conversion() { + assert_eq!(NetflowV5Record::ipv4_to_string(0xC0A80101), "192.168.1.1"); + assert_eq!(NetflowV5Record::ipv4_to_string(0x08080808), "8.8.8.8"); + assert_eq!(NetflowV5Record::ipv4_to_string(0x00000000), "0.0.0.0"); + assert_eq!(NetflowV5Record::ipv4_to_string(0xFFFFFFFF), "255.255.255.255"); + } +} \ No newline at end of file diff --git a/src/sources/netflow/protocols/netflow_v9.rs b/src/sources/netflow/protocols/netflow_v9.rs new file mode 100644 index 0000000000000..dc503b1a3b6a1 --- /dev/null +++ b/src/sources/netflow/protocols/netflow_v9.rs @@ -0,0 +1,1205 @@ +//! NetFlow v9 protocol parser. +//! +//! NetFlow v9 is a template-based flow export protocol that allows flexible +//! field definitions. Unlike NetFlow v5's fixed format, v9 uses templates +//! to define the structure of flow records, enabling custom fields and +//! variable record formats. + +use crate::sources::netflow::events::*; +use crate::sources::netflow::fields::FieldParser; +use crate::sources::netflow::templates::{ + TemplateCache, Template, TemplateField, + parse_netflow_v9_template_fields, +}; +use std::net::SocketAddr; +use vector_lib::event::{Event, LogEvent}; + + +/// NetFlow v9 protocol constants +const NETFLOW_V9_VERSION: u16 = 9; +const NETFLOW_V9_HEADER_SIZE: usize = 20; +const MAX_SET_LENGTH: usize = 65535; +const TEMPLATE_SET_ID: u16 = 0; +const OPTIONS_TEMPLATE_SET_ID: u16 = 1; + +/// NetFlow v9 packet header structure +#[derive(Debug, Clone)] +pub struct NetflowV9Header { + pub version: u16, + pub count: u16, + pub sys_uptime: u32, + pub unix_secs: u32, + pub flow_sequence: u32, + pub source_id: u32, +} + +impl NetflowV9Header { + /// Parse NetFlow v9 header from packet data + pub fn from_bytes(data: &[u8]) -> Result { + if data.len() < NETFLOW_V9_HEADER_SIZE { + return Err(format!( + "Packet too short for NetFlow v9 header: {} bytes, need {}", + data.len(), + NETFLOW_V9_HEADER_SIZE + )); + } + + let version = u16::from_be_bytes([data[0], data[1]]); + if version != NETFLOW_V9_VERSION { + return Err(format!( + "Invalid NetFlow v9 version: {}, expected {}", + version, NETFLOW_V9_VERSION + )); + } + + Ok(Self { + version, + count: u16::from_be_bytes([data[2], data[3]]), + sys_uptime: u32::from_be_bytes([data[4], data[5], data[6], data[7]]), + unix_secs: u32::from_be_bytes([data[8], data[9], data[10], data[11]]), + flow_sequence: u32::from_be_bytes([data[12], data[13], data[14], data[15]]), + source_id: u32::from_be_bytes([data[16], data[17], data[18], data[19]]), + }) + } + + /// Create base log event with header information + pub fn to_log_event(&self) -> LogEvent { + let mut log_event = LogEvent::default(); + log_event.insert("flow_type", "netflow_v9"); + log_event.insert("version", self.version); + log_event.insert("count", self.count); + log_event.insert("sys_uptime", self.sys_uptime); + log_event.insert("unix_secs", self.unix_secs); + log_event.insert("flow_sequence", self.flow_sequence); + log_event.insert("source_id", self.source_id); + log_event + } +} + +/// NetFlow v9 set header structure +#[derive(Debug, Clone)] +pub struct SetHeader { + pub set_id: u16, + pub length: u16, +} + +impl SetHeader { + /// Parse set header from data + pub fn from_bytes(data: &[u8]) -> Result { + if data.len() < 4 { + return Err("Insufficient data for set header".to_string()); + } + + let set_id = u16::from_be_bytes([data[0], data[1]]); + let length = u16::from_be_bytes([data[2], data[3]]); + + if length < 4 { + return Err(format!("Invalid set length: {}, minimum is 4", length)); + } + + if length as usize > MAX_SET_LENGTH { + return Err(format!("Set length too large: {}, maximum is {}", length, MAX_SET_LENGTH)); + } + + Ok(Self { set_id, length }) + } + + /// Check if this is a template set + pub fn is_template_set(&self) -> bool { + self.set_id == TEMPLATE_SET_ID + } + + /// Check if this is an options template set + pub fn is_options_template_set(&self) -> bool { + self.set_id == OPTIONS_TEMPLATE_SET_ID + } + + /// Check if this is a data set + pub fn is_data_set(&self) -> bool { + self.set_id >= 256 + } + + /// Get template ID for data sets + pub fn template_id(&self) -> Option { + if self.is_data_set() { + Some(self.set_id) + } else { + None + } + } +} + +/// NetFlow v9 protocol parser +pub struct NetflowV9Parser { + field_parser: FieldParser, +} + +impl NetflowV9Parser { + /// Create a new NetFlow v9 parser + pub fn new(field_parser: FieldParser) -> Self { + Self { + field_parser, + } + } + + /// Check if packet data looks like NetFlow v9 + pub fn can_parse(data: &[u8]) -> bool { + if data.len() < NETFLOW_V9_HEADER_SIZE { + return false; + } + + let version = u16::from_be_bytes([data[0], data[1]]); + if version != NETFLOW_V9_VERSION { + return false; + } + + // Additional validation - check if count field is reasonable + let count = u16::from_be_bytes([data[2], data[3]]); + if count > 1000 { + // Sanity check - more than 1000 sets seems unreasonable + return false; + } + + true + } + + /// Parse NetFlow v9 packet and return events + pub fn parse( + &self, + data: &[u8], + peer_addr: SocketAddr, + template_cache: &TemplateCache, + include_raw_data: bool, + drop_unparseable_records: bool, + ) -> Result, String> { + let mut events = Vec::new(); + + // Parse header + let header = NetflowV9Header::from_bytes(data)?; + + debug!( + "Parsing NetFlow v9 packet: version={}, count={}, source_id={}", + header.version, header.count, header.source_id + ); + + // Create base event with header info + let mut base_event = header.to_log_event(); + if include_raw_data { + let encoded = base64::Engine::encode(&base64::engine::general_purpose::STANDARD, data); + base_event.insert("raw_data", encoded); + } + + // Parse sets + let mut offset = NETFLOW_V9_HEADER_SIZE; + let mut data_events = Vec::new(); + let mut template_count = 0; + let mut data_set_count = 0; + let mut sets_processed = 0; + + while offset + 4 <= data.len() && sets_processed < header.count { + let set_header = match SetHeader::from_bytes(&data[offset..]) { + Ok(header) => header, + Err(e) => { + warn!("Invalid set header at offset {}: {}", offset, e); + break; + } + }; + + let set_end = offset + set_header.length as usize; + if set_end > data.len() { + warn!( + "Set extends beyond packet boundary: offset={}, set_length={}, packet_length={}", + offset, set_header.length, data.len() + ); + break; + } + + let set_data = &data[offset..set_end]; + + println!("Processing set with ID: {}", set_header.set_id); + println!("TEMPLATE_SET_ID: {}, OPTIONS_TEMPLATE_SET_ID: {}", TEMPLATE_SET_ID, OPTIONS_TEMPLATE_SET_ID); + println!("set_header.set_id == TEMPLATE_SET_ID: {}", set_header.set_id == TEMPLATE_SET_ID); + println!("set_header.set_id == OPTIONS_TEMPLATE_SET_ID: {}", set_header.set_id == OPTIONS_TEMPLATE_SET_ID); + println!("About to match on set_id: {}", set_header.set_id); + match set_header.set_id { + TEMPLATE_SET_ID => { + println!("MATCH ARM 0: Processing template set"); + let parsed_templates = self.parse_template_set( + set_data, + header.source_id, + peer_addr, + template_cache, + ); + template_count += parsed_templates; + } + OPTIONS_TEMPLATE_SET_ID => { + println!("MATCH ARM 1: Processing options template set"); + let parsed_templates = self.parse_options_template_set( + set_data, + header.source_id, + peer_addr, + template_cache, + ); + template_count += parsed_templates; + } + template_id if template_id >= 256 => { + let mut set_events = self.parse_data_set( + set_data, + template_id, + header.source_id, + peer_addr, + template_cache, + drop_unparseable_records, + ); + data_set_count += 1; + data_events.append(&mut set_events); + } + _ => { + debug!("Skipping unknown set type: {}", set_header.set_id); + } + } + + offset = set_end; + sets_processed += 1; + } + + // Add parsed data events + events.extend(data_events); + + // If no data events were generated, include the header event + if events.is_empty() && !drop_unparseable_records { + base_event.insert("template_count", template_count); + base_event.insert("data_set_count", data_set_count); + base_event.insert("sets_processed", sets_processed); + events.push(Event::Log(base_event)); + } + + emit!(NetflowV9PacketProcessed { + peer_addr, + template_count, + data_set_count, + event_count: events.len(), + sets_processed: sets_processed as usize, + }); + + Ok(events) + } + + /// Parse template set and cache templates + fn parse_template_set( + &self, + data: &[u8], + source_id: u32, + peer_addr: SocketAddr, + template_cache: &TemplateCache, + ) -> usize { + let mut template_count = 0; + let mut offset = 4; // Skip set header + + while offset + 4 <= data.len() { + let template_id = u16::from_be_bytes([data[offset], data[offset + 1]]); + let field_count = u16::from_be_bytes([data[offset + 2], data[offset + 3]]); + + debug!( + "Parsing NetFlow v9 template: id={}, fields={}", + template_id, field_count + ); + + // Validate template ID range (256-65535 for data templates) + // Note: Options templates can have IDs < 256, so we validate here for regular templates + if template_id < 256 { + warn!("Invalid template ID {}, must be >= 256", template_id); + offset += 4; + continue; + } + + // Calculate template end + let template_end = offset + 4 + (field_count as usize * 4); + if template_end > data.len() { + warn!("Template {} extends beyond set boundary", template_id); + break; + } + + // Parse template fields + let template_data = &data[offset..template_end]; + match parse_netflow_v9_template_fields(template_data) { + Ok(fields) => { + // Validate fields + if fields.len() != field_count as usize { + warn!( + "Template {} field count mismatch: expected {}, got {}", + template_id, field_count, fields.len() + ); + offset = template_end; + continue; + } + + // Check for variable-length fields (not supported in NetFlow v9) + let has_variable_fields = fields.iter().any(|f| f.field_length == 0 || f.field_length == 65535); + if has_variable_fields { + warn!("Template {} contains variable-length fields, which are not supported in NetFlow v9", template_id); + offset = template_end; + continue; + } + + let template = Template::new(template_id, fields); + let key = (peer_addr, source_id, template_id); + template_cache.insert(key, template); + template_count += 1; + + emit!(TemplateReceived { + template_id, + field_count, + peer_addr, + observation_domain_id: source_id, + protocol: "netflow_v9", + }); + } + Err(e) => { + emit!(NetflowTemplateError { + error: e.as_str(), + template_id, + peer_addr, + }); + } + } + + offset = template_end; + } + + template_count + } + + /// Parse options template set + fn parse_options_template_set( + &self, + data: &[u8], + source_id: u32, + peer_addr: SocketAddr, + template_cache: &TemplateCache, + ) -> usize { + let mut template_count = 0; + let mut offset = 4; // Skip set header + + while offset + 6 <= data.len() { + let template_id = u16::from_be_bytes([data[offset], data[offset + 1]]); + let scope_field_count = u16::from_be_bytes([data[offset + 2], data[offset + 3]]); + let option_field_count = u16::from_be_bytes([data[offset + 4], data[offset + 5]]); + + println!("Parsing NetFlow v9 options template: id={}, scope_fields={}, option_fields={}", + template_id, scope_field_count, option_field_count); + + let total_fields = scope_field_count + option_field_count; + let template_end = offset + 6 + (total_fields as usize * 4); + + if template_end > data.len() { + warn!("Options template {} extends beyond set boundary", template_id); + break; + } + + // For simplicity, parse options templates like regular templates + // In a full implementation, scope fields would be handled differently + let mut fields = Vec::new(); + let mut field_offset = offset + 6; + + for _ in 0..total_fields { + if field_offset + 4 > data.len() { + break; + } + + let field_type = u16::from_be_bytes([data[field_offset], data[field_offset + 1]]); + let field_length = u16::from_be_bytes([data[field_offset + 2], data[field_offset + 3]]); + + fields.push(TemplateField { + field_type, + field_length, + enterprise_number: None, // NetFlow v9 doesn't use enterprise numbers + is_scope: false, + }); + + field_offset += 4; + } + + if fields.len() == total_fields as usize { + let template = Template::new(template_id, fields); + let key = (peer_addr, source_id, template_id); + println!("About to insert options template with key: {:?}", key); + template_cache.insert(key, template); + template_count += 1; + println!("Inserted options template, count: {}", template_count); + + emit!(TemplateReceived { + template_id, + field_count: total_fields, + peer_addr, + observation_domain_id: source_id, + protocol: "netflow_v9_options", + }); + } else { + println!("Fields mismatch: expected {}, got {}", total_fields, fields.len()); + } + + offset = template_end; + } + + template_count + } + + /// Parse data set using cached template + fn parse_data_set( + &self, + data: &[u8], + template_id: u16, + source_id: u32, + peer_addr: SocketAddr, + template_cache: &TemplateCache, + drop_unparseable_records: bool, + ) -> Vec { + let mut events = Vec::new(); + let key = (peer_addr, source_id, template_id); + + let template = match template_cache.get(&key) { + Some(template) => template, + None => { + debug!( + "No template found for NetFlow v9 data set: template_id={}, source_id={}", + template_id, source_id + ); + + if drop_unparseable_records { + emit!(NetflowEventsDropped { + count: 1, + reason: "No template available for NetFlow v9 data parsing", + }); + return events; + } + + // Create basic event without template + let mut log_event = LogEvent::default(); + log_event.insert("flow_type", "netflow_v9_data_unparseable"); + log_event.insert("template_id", template_id); + log_event.insert("source_id", source_id); + log_event.insert("data_length", data.len() - 4); // Exclude set header + events.push(Event::Log(log_event)); + return events; + } + }; + + debug!( + "Parsing NetFlow v9 data set: template_id={}, fields={}", + template_id, + template.fields.len() + ); + + // NetFlow v9 only supports fixed-length records + let record_size = match template.record_size() { + Some(size) => size, + None => { + warn!( + "Template {} has variable-length fields, not supported in NetFlow v9", + template_id + ); + return events; + } + }; + + let mut offset = 4; // Skip set header + let mut record_count = 0; + const MAX_RECORDS: usize = 10000; // Safety limit + + while offset + record_size <= data.len() && record_count < MAX_RECORDS { + let mut log_event = LogEvent::default(); + log_event.insert("flow_type", "netflow_v9_data"); + log_event.insert("template_id", template_id); + log_event.insert("source_id", source_id); + log_event.insert("record_number", record_count); + + let mut field_offset = offset; + let mut fields_parsed = 0; + + for field in &template.fields { + if field_offset + field.field_length as usize > data.len() { + debug!( + "Insufficient data for field: offset={}, length={}, remaining={}", + field_offset, + field.field_length, + data.len() - field_offset + ); + break; + } + + let field_data = &data[field_offset..field_offset + field.field_length as usize]; + self.field_parser.parse_field(field, field_data, &mut log_event); + + field_offset += field.field_length as usize; + fields_parsed += 1; + } + + // Only emit event if we parsed all fields successfully + if fields_parsed == template.fields.len() { + events.push(Event::Log(log_event)); + + emit!(DataRecordParsed { + template_id, + fields_parsed, + record_size, + peer_addr, + protocol: "netflow_v9", + }); + } else { + debug!( + "Incomplete record parsing: {}/{} fields parsed", + fields_parsed, + template.fields.len() + ); + } + + offset += record_size; + record_count += 1; + } + + if record_count >= MAX_RECORDS { + warn!("Hit maximum record limit ({}) for template {}", MAX_RECORDS, template_id); + } + + debug!( + "Parsed {} records from NetFlow v9 data set (template {})", + record_count, template_id + ); + + events + } +} + +/// NetFlow v9 specific events +#[derive(Debug)] +pub struct NetflowV9PacketProcessed { + pub peer_addr: SocketAddr, + pub template_count: usize, + pub data_set_count: usize, + pub event_count: usize, + pub sets_processed: usize, +} + +impl vector_lib::internal_event::InternalEvent for NetflowV9PacketProcessed { + fn emit(self) { + debug!( + message = "NetFlow v9 packet processed", + peer_addr = %self.peer_addr, + template_count = self.template_count, + data_set_count = self.data_set_count, + event_count = self.event_count, + sets_processed = self.sets_processed, + ); + } +} + +#[cfg(test)] +mod tests { + use super::*; + use crate::sources::netflow::config::NetflowConfig; + use crate::sources::netflow::fields::FieldParser; + use crate::sources::netflow::templates::TemplateCache; + use base64::Engine; + use std::net::{IpAddr, Ipv4Addr}; + + fn test_peer_addr() -> SocketAddr { + SocketAddr::new(IpAddr::V4(Ipv4Addr::new(192, 168, 1, 100)), 2055) + } + + fn create_netflow_v9_header() -> Vec { + let mut data = vec![0u8; 20]; + data[0..2].copy_from_slice(&9u16.to_be_bytes()); // version + data[2..4].copy_from_slice(&1u16.to_be_bytes()); // count + data[4..8].copy_from_slice(&12345u32.to_be_bytes()); // sys_uptime + data[8..12].copy_from_slice(&1609459200u32.to_be_bytes()); // unix_secs + data[12..16].copy_from_slice(&100u32.to_be_bytes()); // flow_sequence + data[16..20].copy_from_slice(&1u32.to_be_bytes()); // source_id + data + } + + #[test] + fn test_netflow_v9_header_parsing() { + let data = create_netflow_v9_header(); + let header = NetflowV9Header::from_bytes(&data).unwrap(); + + assert_eq!(header.version, 9); + assert_eq!(header.count, 1); + assert_eq!(header.sys_uptime, 12345); + assert_eq!(header.unix_secs, 1609459200); + assert_eq!(header.flow_sequence, 100); + assert_eq!(header.source_id, 1); + } + + #[test] + fn test_invalid_netflow_v9_header() { + // Too short + let short_data = vec![0u8; 10]; + assert!(NetflowV9Header::from_bytes(&short_data).is_err()); + + // Wrong version + let mut wrong_version = create_netflow_v9_header(); + wrong_version[0..2].copy_from_slice(&5u16.to_be_bytes()); + assert!(NetflowV9Header::from_bytes(&wrong_version).is_err()); + } + + #[test] + fn test_can_parse() { + // Valid NetFlow v9 + let nf9_data = create_netflow_v9_header(); + assert!(NetflowV9Parser::can_parse(&nf9_data)); + + // Invalid version + let mut invalid_data = nf9_data.clone(); + invalid_data[0..2].copy_from_slice(&10u16.to_be_bytes()); + assert!(!NetflowV9Parser::can_parse(&invalid_data)); + + // Too short + let short_data = vec![0u8; 10]; + assert!(!NetflowV9Parser::can_parse(&short_data)); + + // Unreasonable count + let mut bad_count = nf9_data.clone(); + bad_count[2..4].copy_from_slice(&2000u16.to_be_bytes()); + assert!(!NetflowV9Parser::can_parse(&bad_count)); + } + + #[test] + fn test_set_header_parsing() { + let mut data = vec![0u8; 8]; + data[0..2].copy_from_slice(&0u16.to_be_bytes()); // template set + data[2..4].copy_from_slice(&8u16.to_be_bytes()); // length + + let header = SetHeader::from_bytes(&data).unwrap(); + assert_eq!(header.set_id, 0); + assert_eq!(header.length, 8); + assert!(header.is_template_set()); + assert!(!header.is_data_set()); + + // Data set + data[0..2].copy_from_slice(&256u16.to_be_bytes()); + let header = SetHeader::from_bytes(&data).unwrap(); + assert!(header.is_data_set()); + assert_eq!(header.template_id(), Some(256)); + } + + #[test] + fn test_template_parsing() { + let config = NetflowConfig::default(); + let field_parser = FieldParser::new(&config); + let template_cache = TemplateCache::new(100); + let parser = NetflowV9Parser::new(field_parser); + + // Create NetFlow v9 packet with template + let mut data = create_netflow_v9_header(); + data[2..4].copy_from_slice(&1u16.to_be_bytes()); // count = 1 + + // Template set header + data.extend_from_slice(&0u16.to_be_bytes()); // set_id (template) + data.extend_from_slice(&12u16.to_be_bytes()); // set_length + + // Template definition + data.extend_from_slice(&256u16.to_be_bytes()); // template_id + data.extend_from_slice(&1u16.to_be_bytes()); // field_count + data.extend_from_slice(&8u16.to_be_bytes()); // field_type (sourceIPv4Address) + data.extend_from_slice(&4u16.to_be_bytes()); // field_length + + let events = parser.parse(&data, test_peer_addr(), &template_cache, false, false).unwrap(); + + // Should have base event with template info + assert!(!events.is_empty()); + + // Template should be cached + let key = (test_peer_addr(), 1, 256); + assert!(template_cache.get(&key).is_some()); + } + + #[test] + fn test_data_parsing_with_template() { + let config = NetflowConfig::default(); + let field_parser = FieldParser::new(&config); + let template_cache = TemplateCache::new(100); + let parser = NetflowV9Parser::new(field_parser); + + // First, add a template to cache + let template = Template::new( + 256, + vec![TemplateField { + field_type: 8, // sourceIPv4Address + field_length: 4, + enterprise_number: None, + is_scope: false, + }], + ); + let key = (test_peer_addr(), 1, 256); + template_cache.insert(key, template); + + // Create NetFlow v9 packet with data set + let mut data = create_netflow_v9_header(); + + // Data set header + data.extend_from_slice(&256u16.to_be_bytes()); // template_id + data.extend_from_slice(&8u16.to_be_bytes()); // set_length + data.extend_from_slice(&[192, 168, 1, 1]); // IPv4 address data + + let events = parser.parse(&data, test_peer_addr(), &template_cache, false, false).unwrap(); + + // Should parse data using template + assert!(!events.is_empty()); + if let Event::Log(log) = &events[0] { + assert_eq!(log.get("flow_type").unwrap().as_str().unwrap(), "netflow_v9_data"); + assert_eq!(log.get("template_id").unwrap().as_integer().unwrap(), 256); + assert_eq!(log.get("sourceIPv4Address").unwrap().as_str().unwrap(), "192.168.1.1"); + } + } + + #[test] + fn test_data_parsing_without_template() { + let config = NetflowConfig::default(); + let field_parser = FieldParser::new(&config); + let template_cache = TemplateCache::new(100); + let parser = NetflowV9Parser::new(field_parser); + + // Create NetFlow v9 packet with data set (no template) + let mut data = create_netflow_v9_header(); + + // Data set header + data.extend_from_slice(&256u16.to_be_bytes()); // template_id + data.extend_from_slice(&8u16.to_be_bytes()); // set_length + data.extend_from_slice(&[0x01, 0x02, 0x03, 0x04]); // Some data + + let events = parser.parse(&data, test_peer_addr(), &template_cache, false, false).unwrap(); + + // Should create unparseable event + assert!(!events.is_empty()); + if let Event::Log(log) = &events[0] { + assert!(log.get("flow_type").unwrap().as_str().unwrap().contains("unparseable")); + } + } + + #[test] + fn test_options_template_parsing() { + let config = NetflowConfig::default(); + let field_parser = FieldParser::new(&config); + let template_cache = TemplateCache::new(100); + let parser = NetflowV9Parser::new(field_parser); + + // Create NetFlow v9 packet with options template + let mut data = create_netflow_v9_header(); + data[2..4].copy_from_slice(&1u16.to_be_bytes()); // count = 1 set + + // Options template set header + data.extend_from_slice(&1u16.to_be_bytes()); // set_id (options template) + data.extend_from_slice(&18u16.to_be_bytes()); // set_length (4 header + 14 template data) + + // Options template definition + data.extend_from_slice(&257u16.to_be_bytes()); // template_id + data.extend_from_slice(&1u16.to_be_bytes()); // scope_field_count + data.extend_from_slice(&1u16.to_be_bytes()); // option_field_count + data.extend_from_slice(&1u16.to_be_bytes()); // scope field type + data.extend_from_slice(&4u16.to_be_bytes()); // scope field length + data.extend_from_slice(&2u16.to_be_bytes()); // option field type + data.extend_from_slice(&4u16.to_be_bytes()); // option field length + println!("Data length: {}", data.len()); + println!("Expected length: {}", 20 + 18); // NetFlow header (20) + set (18) + println!("Data: {:?}", data); + + let events = parser.parse(&data, test_peer_addr(), &template_cache, false, false).unwrap(); + + // Should parse options template + assert!(!events.is_empty()); + + // Debug: print all cached templates + let debug_templates = template_cache.debug_templates(10); + println!("Cached templates: {:?}", debug_templates); + println!("Template cache stats: {:?}", template_cache.stats()); + + // Template should be cached - check with correct key + let key = (test_peer_addr(), 1, 257); + let template = template_cache.get(&key); + assert!(template.is_some(), "Template should be cached for key {:?}", key); + } + + #[test] + fn test_multiple_records_in_data_set() { + let config = NetflowConfig::default(); + let field_parser = FieldParser::new(&config); + let template_cache = TemplateCache::new(100); + let parser = NetflowV9Parser::new(field_parser); + + // Add template with fixed-length field + let template = Template::new( + 256, + vec![TemplateField { + field_type: 8, // sourceIPv4Address + field_length: 4, + enterprise_number: None, + is_scope: false, + }], + ); + let key = (test_peer_addr(), 1, 256); + template_cache.insert(key, template); + + // Create NetFlow v9 packet with multiple records + let mut data = create_netflow_v9_header(); + + // Data set header + data.extend_from_slice(&256u16.to_be_bytes()); // template_id + data.extend_from_slice(&12u16.to_be_bytes()); // set_length (4 header + 8 data) + + // Two IPv4 records + data.extend_from_slice(&[192, 168, 1, 1]); // First record + data.extend_from_slice(&[10, 0, 0, 1]); // Second record + + let events = parser.parse(&data, test_peer_addr(), &template_cache, false, false).unwrap(); + + // Should get two data events + assert_eq!(events.len(), 2); + + if let Event::Log(log1) = &events[0] { + assert_eq!(log1.get("sourceIPv4Address").unwrap().as_str().unwrap(), "192.168.1.1"); + assert_eq!(log1.get("record_number").unwrap().as_integer().unwrap(), 0); + } + + if let Event::Log(log2) = &events[1] { + assert_eq!(log2.get("sourceIPv4Address").unwrap().as_str().unwrap(), "10.0.0.1"); + assert_eq!(log2.get("record_number").unwrap().as_integer().unwrap(), 1); + } + } + + #[test] + fn test_mixed_template_and_data_packet() { + let config = NetflowConfig::default(); + let field_parser = FieldParser::new(&config); + let template_cache = TemplateCache::new(100); + let parser = NetflowV9Parser::new(field_parser); + + // Create NetFlow v9 packet with both template and data + let mut data = create_netflow_v9_header(); + data[2..4].copy_from_slice(&2u16.to_be_bytes()); // count = 2 sets + + // Template set + data.extend_from_slice(&0u16.to_be_bytes()); // set_id (template) + data.extend_from_slice(&12u16.to_be_bytes()); // set_length + data.extend_from_slice(&256u16.to_be_bytes()); // template_id + data.extend_from_slice(&1u16.to_be_bytes()); // field_count + data.extend_from_slice(&8u16.to_be_bytes()); // field_type + data.extend_from_slice(&4u16.to_be_bytes()); // field_length + + // Data set + data.extend_from_slice(&256u16.to_be_bytes()); // template_id + data.extend_from_slice(&8u16.to_be_bytes()); // set_length + data.extend_from_slice(&[192, 168, 1, 1]); // IPv4 data + + let events = parser.parse(&data, test_peer_addr(), &template_cache, false, false).unwrap(); + + // Should get data event (template was cached and immediately used) + assert!(!events.is_empty()); + + // Should have cached the template + let key = (test_peer_addr(), 1, 256); + assert!(template_cache.get(&key).is_some()); + + // Should have at least one data event + let data_events: Vec<_> = events.iter() + .filter(|e| { + if let Event::Log(log) = e { + log.get("flow_type").unwrap().as_str().unwrap() == "netflow_v9_data" + } else { + false + } + }) + .collect(); + assert!(!data_events.is_empty()); + } + + #[test] + fn test_invalid_template_id() { + let config = NetflowConfig::default(); + let field_parser = FieldParser::new(&config); + let template_cache = TemplateCache::new(100); + let parser = NetflowV9Parser::new(field_parser); + + // Create NetFlow v9 packet with invalid template ID + let mut data = create_netflow_v9_header(); + + // Template set with invalid template ID + data.extend_from_slice(&0u16.to_be_bytes()); // set_id (template) + data.extend_from_slice(&12u16.to_be_bytes()); // set_length + data.extend_from_slice(&100u16.to_be_bytes()); // invalid template_id (< 256) + data.extend_from_slice(&1u16.to_be_bytes()); // field_count + data.extend_from_slice(&8u16.to_be_bytes()); // field_type + data.extend_from_slice(&4u16.to_be_bytes()); // field_length + + let events = parser.parse(&data, test_peer_addr(), &template_cache, false, false).unwrap(); + + // Should handle gracefully + assert!(!events.is_empty()); + + // Template should not be cached due to invalid ID + let key = (test_peer_addr(), 1, 100); + assert!(template_cache.get(&key).is_none()); + } + + #[test] + fn test_template_field_count_mismatch() { + let config = NetflowConfig::default(); + let field_parser = FieldParser::new(&config); + let template_cache = TemplateCache::new(100); + let parser = NetflowV9Parser::new(field_parser); + + // Create NetFlow v9 packet with mismatched field count + let mut data = create_netflow_v9_header(); + + // Template set with wrong field count + data.extend_from_slice(&0u16.to_be_bytes()); // set_id (template) + data.extend_from_slice(&12u16.to_be_bytes()); // set_length + data.extend_from_slice(&256u16.to_be_bytes()); // template_id + data.extend_from_slice(&2u16.to_be_bytes()); // field_count (says 2) + data.extend_from_slice(&8u16.to_be_bytes()); // field_type + data.extend_from_slice(&4u16.to_be_bytes()); // field_length + // Missing second field + + let events = parser.parse(&data, test_peer_addr(), &template_cache, false, false).unwrap(); + + // Should handle gracefully + assert!(!events.is_empty()); + + // Template should not be cached due to mismatch + let key = (test_peer_addr(), 1, 256); + assert!(template_cache.get(&key).is_none()); + } + + #[test] + fn test_variable_length_field_rejection() { + let config = NetflowConfig::default(); + let field_parser = FieldParser::new(&config); + let template_cache = TemplateCache::new(100); + let parser = NetflowV9Parser::new(field_parser); + + // Create template with variable-length field + let template = Template::new( + 256, + vec![TemplateField { + field_type: 8, + field_length: 65535, // Variable length + enterprise_number: None, + is_scope: false, + }], + ); + let key = (test_peer_addr(), 1, 256); + template_cache.insert(key.clone(), template); + + // Create NetFlow v9 packet with data set + let mut data = create_netflow_v9_header(); + + // Data set header + data.extend_from_slice(&256u16.to_be_bytes()); // template_id + data.extend_from_slice(&8u16.to_be_bytes()); // set_length + data.extend_from_slice(&[0x01, 0x02, 0x03, 0x04]); + + let events = parser.parse(&data, test_peer_addr(), &template_cache, false, false).unwrap(); + + // Should reject variable-length fields in NetFlow v9 + // Events will be empty or contain error info + if !events.is_empty() { + // If events are generated, they should not be successful data events + for event in &events { + if let Event::Log(log) = event { + let flow_type = log.get("flow_type").unwrap().as_str().unwrap(); + assert_ne!(flow_type, "netflow_v9_data"); + } + } + } + } + + #[test] + fn test_drop_unparseable_records() { + let config = NetflowConfig::default(); + let field_parser = FieldParser::new(&config); + let template_cache = TemplateCache::new(100); + let parser = NetflowV9Parser::new(field_parser); + + // Create NetFlow v9 packet with data set (no template) + let mut data = create_netflow_v9_header(); + + // Data set header + data.extend_from_slice(&256u16.to_be_bytes()); // template_id + data.extend_from_slice(&8u16.to_be_bytes()); // set_length + data.extend_from_slice(&[0x01, 0x02, 0x03, 0x04]); + + // With drop_unparseable_records = true, should get no events + let events = parser.parse(&data, test_peer_addr(), &template_cache, false, true).unwrap(); + assert!(events.is_empty()); + + // With drop_unparseable_records = false, should get unparseable event + let events = parser.parse(&data, test_peer_addr(), &template_cache, false, false).unwrap(); + assert!(!events.is_empty()); + } + + #[test] + fn test_malformed_packet_handling() { + let config = NetflowConfig::default(); + let field_parser = FieldParser::new(&config); + let template_cache = TemplateCache::new(100); + let parser = NetflowV9Parser::new(field_parser); + + // Test with corrupted set length + let mut data = create_netflow_v9_header(); + + // Corrupted set header + data.extend_from_slice(&0u16.to_be_bytes()); // set_id + data.extend_from_slice(&100u16.to_be_bytes()); // Invalid large length + + let result = parser.parse(&data, test_peer_addr(), &template_cache, false, false); + + // Should handle gracefully and return base event + assert!(result.is_ok()); + let events = result.unwrap(); + assert!(!events.is_empty()); + } + + #[test] + fn test_raw_data_inclusion() { + let config = NetflowConfig::default(); + let field_parser = FieldParser::new(&config); + let template_cache = TemplateCache::new(100); + let parser = NetflowV9Parser::new(field_parser); + + let data = create_netflow_v9_header(); + + // Test with raw data inclusion + let events = parser.parse(&data, test_peer_addr(), &template_cache, true, false).unwrap(); + assert!(!events.is_empty()); + + if let Event::Log(log) = &events[0] { + assert!(log.get("raw_data").is_some()); + let raw_data = log.get("raw_data").unwrap().as_str().unwrap(); + + // Should be valid base64 + assert!(base64::engine::general_purpose::STANDARD.decode(raw_data.as_bytes()).is_ok()); + } + + // Test without raw data inclusion + let events = parser.parse(&data, test_peer_addr(), &template_cache, false, false).unwrap(); + assert!(!events.is_empty()); + + if let Event::Log(log) = &events[0] { + assert!(log.get("raw_data").is_none()); + } + } + + #[test] + fn test_record_safety_limits() { + let config = NetflowConfig::default(); + let field_parser = FieldParser::new(&config); + let template_cache = TemplateCache::new(100); + let parser = NetflowV9Parser::new(field_parser); + + // Add template with small field to potentially create many records + let template = Template::new( + 256, + vec![TemplateField { + field_type: 4, // protocolIdentifier + field_length: 1, + enterprise_number: None, + is_scope: false, + }], + ); + let key = (test_peer_addr(), 1, 256); + template_cache.insert(key, template); + + // Create NetFlow v9 packet with large data set + let mut data = create_netflow_v9_header(); + let data_size = 10000; // Large data set + + // Data set header + data.extend_from_slice(&256u16.to_be_bytes()); // template_id + data.extend_from_slice(&((4 + data_size) as u16).to_be_bytes()); // set_length + + // Add lots of data (each record is 1 byte) + data.extend(vec![6u8; data_size]); // All TCP protocol + + let events = parser.parse(&data, test_peer_addr(), &template_cache, false, false).unwrap(); + + // Should be limited by MAX_RECORDS safety limit + assert!(events.len() <= 10000); // MAX_RECORDS constant + + // All events should be valid + for event in &events { + if let Event::Log(log) = event { + assert_eq!(log.get("flow_type").unwrap().as_str().unwrap(), "netflow_v9_data"); + assert_eq!(log.get("protocolIdentifier").unwrap().as_integer().unwrap(), 6); + } + } + } + + #[test] + fn test_header_to_log_event() { + let data = create_netflow_v9_header(); + let header = NetflowV9Header::from_bytes(&data).unwrap(); + let log_event = header.to_log_event(); + + assert_eq!(log_event.get("flow_type").unwrap().as_str().unwrap(), "netflow_v9"); + assert_eq!(log_event.get("version").unwrap().as_integer().unwrap(), 9); + assert_eq!(log_event.get("count").unwrap().as_integer().unwrap(), 1); + assert_eq!(log_event.get("sys_uptime").unwrap().as_integer().unwrap(), 12345); + assert_eq!(log_event.get("unix_secs").unwrap().as_integer().unwrap(), 1609459200); + assert_eq!(log_event.get("flow_sequence").unwrap().as_integer().unwrap(), 100); + assert_eq!(log_event.get("source_id").unwrap().as_integer().unwrap(), 1); + } + + #[test] + fn test_incomplete_record_handling() { + let config = NetflowConfig::default(); + let field_parser = FieldParser::new(&config); + let template_cache = TemplateCache::new(100); + let parser = NetflowV9Parser::new(field_parser); + + // Add template with 8-byte record + let template = Template::new( + 256, + vec![ + TemplateField { + field_type: 8, // sourceIPv4Address + field_length: 4, + enterprise_number: None, + is_scope: false, + }, + TemplateField { + field_type: 12, // destinationIPv4Address + field_length: 4, + enterprise_number: None, + is_scope: false, + }, + ], + ); + let key = (test_peer_addr(), 1, 256); + template_cache.insert(key, template); + + // Create NetFlow v9 packet with incomplete record + let mut data = create_netflow_v9_header(); + + // Data set header + data.extend_from_slice(&256u16.to_be_bytes()); // template_id + data.extend_from_slice(&10u16.to_be_bytes()); // set_length + + // Incomplete record (only 6 bytes instead of 8) + data.extend_from_slice(&[192, 168, 1, 1, 10, 0]); // Missing 2 bytes + + let events = parser.parse(&data, test_peer_addr(), &template_cache, false, false).unwrap(); + + // Should handle incomplete records gracefully + // Either no events or events that don't include the incomplete record + for event in &events { + if let Event::Log(log) = event { + let flow_type = log.get("flow_type").unwrap().as_str().unwrap(); + if flow_type == "netflow_v9_data" { + // If a data event was created, it should have both fields + assert!(log.get("sourceIPv4Address").is_some()); + assert!(log.get("destinationIPv4Address").is_some()); + } + } + } + } +} diff --git a/src/sources/netflow/protocols/sflow.rs b/src/sources/netflow/protocols/sflow.rs new file mode 100644 index 0000000000000..a983c50699120 --- /dev/null +++ b/src/sources/netflow/protocols/sflow.rs @@ -0,0 +1,1646 @@ +//! sFlow protocol parser. +//! +//! sFlow (Sampled Flow) is a packet sampling technology for monitoring +//! network traffic. Unlike NetFlow which exports flow records, sFlow +//! exports raw packet samples and interface counters at regular intervals. + +use crate::sources::netflow::events::*; +use std::net::SocketAddr; +use vector_lib::event::{Event, LogEvent}; + + +/// sFlow protocol constants +const SFLOW_VERSION: u32 = 5; +const SFLOW_HEADER_SIZE: usize = 28; +const MAX_SAMPLES: u32 = 1000; // Sanity check for sample count + +/// sFlow datagram header structure +#[derive(Debug, Clone)] +pub struct SflowHeader { + pub version: u32, + pub agent_address_type: u32, + pub agent_address: [u8; 16], // Support both IPv4 and IPv6 + pub sub_agent_id: u32, + pub sequence_number: u32, + pub sys_uptime: u32, + pub num_samples: u32, +} + +impl SflowHeader { + /// Parse sFlow header from packet data + pub fn from_bytes(data: &[u8]) -> Result { + if data.len() < SFLOW_HEADER_SIZE { + return Err(format!( + "Packet too short for sFlow header: {} bytes, need at least {}", + data.len(), + SFLOW_HEADER_SIZE + )); + } + + let version = u32::from_be_bytes([data[0], data[1], data[2], data[3]]); + if version != SFLOW_VERSION { + return Err(format!( + "Unsupported sFlow version: {}, expected {}", + version, SFLOW_VERSION + )); + } + + let agent_address_type = u32::from_be_bytes([data[4], data[5], data[6], data[7]]); + + // Parse agent address based on type + let mut agent_address = [0u8; 16]; + let address_size = match agent_address_type { + 1 => { // IPv4 + if data.len() < 12 + 4 { + return Err("Insufficient data for IPv4 agent address".to_string()); + } + agent_address[12..16].copy_from_slice(&data[8..12]); + 4 + } + 2 => { // IPv6 + if data.len() < 8 + 16 { + return Err("Insufficient data for IPv6 agent address".to_string()); + } + agent_address.copy_from_slice(&data[8..24]); + 16 + } + _ => { + return Err(format!("Unsupported agent address type: {}", agent_address_type)); + } + }; + + let base_offset = 8 + address_size; + if data.len() < base_offset + 16 { + return Err("Insufficient data for sFlow header".to_string()); + } + + let sub_agent_id = u32::from_be_bytes([ + data[base_offset], data[base_offset + 1], + data[base_offset + 2], data[base_offset + 3] + ]); + let sequence_number = u32::from_be_bytes([ + data[base_offset + 4], data[base_offset + 5], + data[base_offset + 6], data[base_offset + 7] + ]); + let sys_uptime = u32::from_be_bytes([ + data[base_offset + 8], data[base_offset + 9], + data[base_offset + 10], data[base_offset + 11] + ]); + let num_samples = u32::from_be_bytes([ + data[base_offset + 12], data[base_offset + 13], + data[base_offset + 14], data[base_offset + 15] + ]); + + if num_samples > MAX_SAMPLES { + return Err(format!( + "Unreasonable sample count: {}, maximum expected {}", + num_samples, MAX_SAMPLES + )); + } + + Ok(Self { + version, + agent_address_type, + agent_address, + sub_agent_id, + sequence_number, + sys_uptime, + num_samples, + }) + } + + /// Get agent address as string + pub fn agent_address_string(&self) -> String { + match self.agent_address_type { + 1 => { // IPv4 + format!( + "{}.{}.{}.{}", + self.agent_address[12], self.agent_address[13], + self.agent_address[14], self.agent_address[15] + ) + } + 2 => { // IPv6 + format!( + "{:02x}{:02x}:{:02x}{:02x}:{:02x}{:02x}:{:02x}{:02x}:{:02x}{:02x}:{:02x}{:02x}:{:02x}{:02x}:{:02x}{:02x}", + self.agent_address[0], self.agent_address[1], self.agent_address[2], self.agent_address[3], + self.agent_address[4], self.agent_address[5], self.agent_address[6], self.agent_address[7], + self.agent_address[8], self.agent_address[9], self.agent_address[10], self.agent_address[11], + self.agent_address[12], self.agent_address[13], self.agent_address[14], self.agent_address[15] + ) + } + _ => "unknown".to_string(), + } + } + + /// Get header size in bytes + pub fn header_size(&self) -> usize { + match self.agent_address_type { + 1 => 28, // IPv4: 8 + 4 + 16 = 28 + 2 => 40, // IPv6: 8 + 16 + 16 = 40 + _ => 28, + } + } + + /// Create base log event with header information + pub fn to_log_event(&self) -> LogEvent { + let mut log_event = LogEvent::default(); + log_event.insert("flow_type", "sflow"); + log_event.insert("version", self.version); + log_event.insert("agent_address_type", self.agent_address_type); + log_event.insert("agent_address", self.agent_address_string()); + log_event.insert("sub_agent_id", self.sub_agent_id); + log_event.insert("sequence_number", self.sequence_number); + log_event.insert("sys_uptime", self.sys_uptime); + log_event.insert("num_samples", self.num_samples); + log_event + } +} + +/// sFlow sample types +#[derive(Debug, Clone, Copy, PartialEq, Eq)] +pub enum SampleType { + FlowSample = 1, + CounterSample = 2, + ExpandedFlowSample = 3, + ExpandedCounterSample = 4, +} + +impl SampleType { + pub fn from_u32(value: u32) -> Option { + match value { + 1 => Some(SampleType::FlowSample), + 2 => Some(SampleType::CounterSample), + 3 => Some(SampleType::ExpandedFlowSample), + 4 => Some(SampleType::ExpandedCounterSample), + _ => None, + } + } + + pub fn as_str(&self) -> &'static str { + match self { + SampleType::FlowSample => "flow_sample", + SampleType::CounterSample => "counter_sample", + SampleType::ExpandedFlowSample => "expanded_flow_sample", + SampleType::ExpandedCounterSample => "expanded_counter_sample", + } + } +} + +/// sFlow sample header +#[derive(Debug, Clone)] +pub struct SampleHeader { + pub sample_type: SampleType, + pub sample_length: u32, +} + +impl SampleHeader { + pub fn from_bytes(data: &[u8]) -> Result { + if data.len() < 8 { + return Err("Insufficient data for sample header".to_string()); + } + + let sample_type_raw = u32::from_be_bytes([data[0], data[1], data[2], data[3]]); + let sample_type = SampleType::from_u32(sample_type_raw) + .ok_or_else(|| format!("Unknown sample type: {}", sample_type_raw))?; + + let sample_length = u32::from_be_bytes([data[4], data[5], data[6], data[7]]); + + if sample_length < 8 { + return Err(format!("Invalid sample length: {}", sample_length)); + } + + Ok(Self { + sample_type, + sample_length, + }) + } +} + +/// sFlow flow sample structure +#[derive(Debug, Clone)] +pub struct FlowSample { + pub sequence_number: u32, + pub source_id_type: u8, + pub source_id_index: u32, + pub sampling_rate: u32, + pub sample_pool: u32, + pub drops: u32, + pub input_interface: u32, + pub output_interface: u32, + pub flow_records: Vec, +} + +/// sFlow counter sample structure +#[derive(Debug, Clone)] +pub struct CounterSample { + pub sequence_number: u32, + pub source_id_type: u8, + pub source_id_index: u32, + pub counter_records: Vec, +} + +/// sFlow flow record +#[derive(Debug, Clone)] +pub struct FlowRecord { + pub record_type: u32, + pub record_length: u32, + pub data: Vec, +} + +/// sFlow counter record +#[derive(Debug, Clone)] +pub struct CounterRecord { + pub record_type: u32, + pub record_length: u32, + pub data: Vec, +} + +/// sFlow protocol parser +pub struct SflowParser; + +impl SflowParser { + /// Create a new sFlow parser + pub fn new() -> Self { + Self + } + + /// Check if packet data looks like sFlow + pub fn can_parse(data: &[u8]) -> bool { + if data.len() < 4 { + return false; + } + + let version = u32::from_be_bytes([data[0], data[1], data[2], data[3]]); + if version != SFLOW_VERSION { + return false; + } + + // Additional validation + if data.len() < 8 { + return false; + } + + let agent_address_type = u32::from_be_bytes([data[4], data[5], data[6], data[7]]); + if agent_address_type != 1 && agent_address_type != 2 { + return false; + } + + true + } + + /// Parse sFlow packet and return events + pub fn parse( + &self, + data: &[u8], + peer_addr: SocketAddr, + include_raw_data: bool, + ) -> Result, String> { + let mut events = Vec::new(); + + // Parse header + let header = SflowHeader::from_bytes(data)?; + + debug!( + "Parsing sFlow packet: version={}, samples={}, agent={}", + header.version, header.num_samples, header.agent_address_string() + ); + + // Create base event with header info + let mut base_event = header.to_log_event(); + if include_raw_data { + let encoded = base64::Engine::encode(&base64::engine::general_purpose::STANDARD, data); + base_event.insert("raw_data", encoded); + } + + // Parse samples + let mut offset = header.header_size(); + let mut flow_samples = 0; + let mut counter_samples = 0; + let mut unknown_samples = 0; + + for sample_index in 0..header.num_samples { + if offset + 8 > data.len() { + warn!("Insufficient data for sample {} header", sample_index); + break; + } + + let sample_header = match SampleHeader::from_bytes(&data[offset..]) { + Ok(header) => header, + Err(e) => { + warn!("Invalid sample header at index {}: {}", sample_index, e); + unknown_samples += 1; + break; + } + }; + + let sample_end = offset + sample_header.sample_length as usize; + if sample_end > data.len() { + warn!( + "Sample {} extends beyond packet boundary: offset={}, length={}, packet_size={}", + sample_index, offset, sample_header.sample_length, data.len() + ); + break; + } + + let sample_data = &data[offset..sample_end]; + + match sample_header.sample_type { + SampleType::FlowSample | SampleType::ExpandedFlowSample => { + match self.parse_flow_sample(sample_data, sample_index, &header, peer_addr) { + Ok(mut sample_events) => { + flow_samples += 1; + events.append(&mut sample_events); + } + Err(e) => { + warn!("Failed to parse flow sample {}: {}", sample_index, e); + unknown_samples += 1; + } + } + } + SampleType::CounterSample | SampleType::ExpandedCounterSample => { + match self.parse_counter_sample(sample_data, sample_index, &header, peer_addr) { + Ok(mut sample_events) => { + counter_samples += 1; + events.append(&mut sample_events); + } + Err(e) => { + warn!("Failed to parse counter sample {}: {}", sample_index, e); + unknown_samples += 1; + } + } + } + } + + offset = sample_end; + } + + // If no sample events were generated, include the header event + if events.is_empty() { + base_event.insert("flow_samples", flow_samples); + base_event.insert("counter_samples", counter_samples); + base_event.insert("unknown_samples", unknown_samples); + events.push(Event::Log(base_event)); + } + + emit!(SflowPacketProcessed { + peer_addr, + flow_samples, + counter_samples, + unknown_samples, + event_count: events.len(), + }); + + Ok(events) + } + + /// Parse flow sample + fn parse_flow_sample( + &self, + data: &[u8], + sample_index: u32, + header: &SflowHeader, + peer_addr: SocketAddr, + ) -> Result, String> { + if data.len() < 32 { + return Err("Insufficient data for flow sample".to_string()); + } + + let sequence_number = u32::from_be_bytes([data[8], data[9], data[10], data[11]]); + let source_id = u32::from_be_bytes([data[12], data[13], data[14], data[15]]); + let source_id_type = ((source_id >> 24) & 0xFF) as u8; + let source_id_index = source_id & 0x00FFFFFF; + let sampling_rate = u32::from_be_bytes([data[16], data[17], data[18], data[19]]); + let sample_pool = u32::from_be_bytes([data[20], data[21], data[22], data[23]]); + let drops = u32::from_be_bytes([data[24], data[25], data[26], data[27]]); + let input_interface = u32::from_be_bytes([data[28], data[29], data[30], data[31]]); + + let mut log_event = LogEvent::default(); + log_event.insert("flow_type", "sflow_flow_sample"); + log_event.insert("sample_index", sample_index); + log_event.insert("agent_address", header.agent_address_string()); + log_event.insert("sequence_number", sequence_number); + log_event.insert("source_id_type", source_id_type); + log_event.insert("source_id_index", source_id_index); + log_event.insert("sampling_rate", sampling_rate); + log_event.insert("sample_pool", sample_pool); + log_event.insert("drops", drops); + log_event.insert("input_interface", input_interface); + + // Parse output interface (if present) + if data.len() >= 36 { + let output_interface = u32::from_be_bytes([data[32], data[33], data[34], data[35]]); + log_event.insert("output_interface", output_interface); + } + + // Parse flow records (simplified for now) + if data.len() >= 40 { + let num_flow_records = u32::from_be_bytes([data[36], data[37], data[38], data[39]]); + log_event.insert("num_flow_records", num_flow_records); + + // Parse individual flow records + let mut record_offset = 40; + let mut records_parsed = 0; + + for _record_index in 0..num_flow_records.min(100) { // Limit to prevent DoS + if record_offset + 8 > data.len() { + break; + } + + let record_type = u32::from_be_bytes([ + data[record_offset], data[record_offset + 1], + data[record_offset + 2], data[record_offset + 3] + ]); + let record_length = u32::from_be_bytes([ + data[record_offset + 4], data[record_offset + 5], + data[record_offset + 6], data[record_offset + 7] + ]); + + let record_end = record_offset + 8 + record_length as usize; + if record_end > data.len() { + break; + } + + // Parse specific record types + match record_type { + 1 => { // Raw packet header + self.parse_raw_packet_header( + &data[record_offset + 8..record_end], + &mut log_event, + ); + } + 2 => { // Ethernet frame data + self.parse_ethernet_frame_data( + &data[record_offset + 8..record_end], + &mut log_event, + ); + } + 1001 => { // Extended switch data + self.parse_extended_switch_data( + &data[record_offset + 8..record_end], + &mut log_event, + ); + } + 1002 => { // Extended router data + self.parse_extended_router_data( + &data[record_offset + 8..record_end], + &mut log_event, + ); + } + _ => { + // Unknown record type + debug!("Unknown flow record type: {}", record_type); + } + } + + record_offset = record_end; + records_parsed += 1; + } + + log_event.insert("records_parsed", records_parsed); + } + + emit!(DataRecordParsed { + template_id: 0, // sFlow doesn't use templates + fields_parsed: log_event.as_map().unwrap().len(), + record_size: data.len(), + peer_addr, + protocol: "sflow", + }); + + Ok(vec![Event::Log(log_event)]) + } + + /// Parse counter sample + fn parse_counter_sample( + &self, + data: &[u8], + sample_index: u32, + header: &SflowHeader, + _peer_addr: SocketAddr, + ) -> Result, String> { + if data.len() < 20 { + return Err("Insufficient data for counter sample".to_string()); + } + + let sequence_number = u32::from_be_bytes([data[8], data[9], data[10], data[11]]); + let source_id = u32::from_be_bytes([data[12], data[13], data[14], data[15]]); + let source_id_type = ((source_id >> 24) & 0xFF) as u8; + let source_id_index = source_id & 0x00FFFFFF; + let num_counter_records = u32::from_be_bytes([data[16], data[17], data[18], data[19]]); + + let mut log_event = LogEvent::default(); + log_event.insert("flow_type", "sflow_counter_sample"); + log_event.insert("sample_index", sample_index); + log_event.insert("agent_address", header.agent_address_string()); + log_event.insert("sequence_number", sequence_number); + log_event.insert("source_id_type", source_id_type); + log_event.insert("source_id_index", source_id_index); + log_event.insert("num_counter_records", num_counter_records); + + // Parse counter records + let mut record_offset = 20; + let mut records_parsed = 0; + + for _record_index in 0..num_counter_records.min(50) { // Limit to prevent DoS + if record_offset + 8 > data.len() { + break; + } + + let record_type = u32::from_be_bytes([ + data[record_offset], data[record_offset + 1], + data[record_offset + 2], data[record_offset + 3] + ]); + let record_length = u32::from_be_bytes([ + data[record_offset + 4], data[record_offset + 5], + data[record_offset + 6], data[record_offset + 7] + ]); + + let record_end = record_offset + 8 + record_length as usize; + if record_end > data.len() { + break; + } + + // Parse specific counter record types + match record_type { + 1 => { // Generic interface counters + self.parse_interface_counters( + &data[record_offset + 8..record_end], + &mut log_event, + ); + } + 2 => { // Ethernet interface counters + self.parse_ethernet_counters( + &data[record_offset + 8..record_end], + &mut log_event, + ); + } + _ => { + debug!("Unknown counter record type: {}", record_type); + } + } + + record_offset = record_end; + records_parsed += 1; + } + + log_event.insert("records_parsed", records_parsed); + + Ok(vec![Event::Log(log_event)]) + } + + /// Parse raw packet header record + fn parse_raw_packet_header(&self, data: &[u8], log_event: &mut LogEvent) { + if data.len() < 16 { + return; + } + + let header_protocol = u32::from_be_bytes([data[0], data[1], data[2], data[3]]); + let frame_length = u32::from_be_bytes([data[4], data[5], data[6], data[7]]); + let payload_removed = u32::from_be_bytes([data[8], data[9], data[10], data[11]]); + let header_length = u32::from_be_bytes([data[12], data[13], data[14], data[15]]); + + log_event.insert("header_protocol", header_protocol); + log_event.insert("frame_length", frame_length); + log_event.insert("payload_removed", payload_removed); + log_event.insert("header_length", header_length); + + // Parse packet header if present + if data.len() > 16 && header_length > 0 { + let header_end = 16 + header_length as usize; + if header_end <= data.len() { + let packet_header = &data[16..header_end]; + self.parse_packet_header(packet_header, log_event, header_protocol); + } + } + } + + /// Parse Ethernet frame data record + fn parse_ethernet_frame_data(&self, data: &[u8], log_event: &mut LogEvent) { + if data.len() < 4 { + return; + } + + let ethernet_length = u32::from_be_bytes([data[0], data[1], data[2], data[3]]); + log_event.insert("ethernet_length", ethernet_length); + + // Parse Ethernet header if present + if data.len() >= 18 && ethernet_length >= 14 { + let eth_header = &data[4..18]; // 14 bytes Ethernet header + + // Destination MAC + let dst_mac = format!( + "{:02x}:{:02x}:{:02x}:{:02x}:{:02x}:{:02x}", + eth_header[0], eth_header[1], eth_header[2], + eth_header[3], eth_header[4], eth_header[5] + ); + log_event.insert("dst_mac", dst_mac); + + // Source MAC + let src_mac = format!( + "{:02x}:{:02x}:{:02x}:{:02x}:{:02x}:{:02x}", + eth_header[6], eth_header[7], eth_header[8], + eth_header[9], eth_header[10], eth_header[11] + ); + log_event.insert("src_mac", src_mac); + + // EtherType + let ethertype = u16::from_be_bytes([eth_header[12], eth_header[13]]); + log_event.insert("ethertype", ethertype); + + // Common EtherType names + let ethertype_name = match ethertype { + 0x0800 => "IPv4", + 0x0806 => "ARP", + 0x86DD => "IPv6", + 0x8100 => "VLAN", + 0x8847 => "MPLS", + _ => "Other", + }; + log_event.insert("ethertype_name", ethertype_name); + } + } + + /// Parse extended switch data record + fn parse_extended_switch_data(&self, data: &[u8], log_event: &mut LogEvent) { + if data.len() < 16 { + return; + } + + let src_vlan = u32::from_be_bytes([data[0], data[1], data[2], data[3]]); + let src_priority = u32::from_be_bytes([data[4], data[5], data[6], data[7]]); + let dst_vlan = u32::from_be_bytes([data[8], data[9], data[10], data[11]]); + let dst_priority = u32::from_be_bytes([data[12], data[13], data[14], data[15]]); + + log_event.insert("src_vlan", src_vlan); + log_event.insert("src_priority", src_priority); + log_event.insert("dst_vlan", dst_vlan); + log_event.insert("dst_priority", dst_priority); + } + + /// Parse extended router data record + fn parse_extended_router_data(&self, data: &[u8], log_event: &mut LogEvent) { + if data.len() < 20 { + return; + } + + let next_hop_type = u32::from_be_bytes([data[0], data[1], data[2], data[3]]); + log_event.insert("next_hop_type", next_hop_type); + + match next_hop_type { + 1 => { // IPv4 + if data.len() >= 8 { + let next_hop = u32::from_be_bytes([data[4], data[5], data[6], data[7]]); + let next_hop_str = format!( + "{}.{}.{}.{}", + (next_hop >> 24) & 0xFF, + (next_hop >> 16) & 0xFF, + (next_hop >> 8) & 0xFF, + next_hop & 0xFF + ); + log_event.insert("next_hop", next_hop_str); + } + } + 2 => { // IPv6 + if data.len() >= 20 { + let next_hop = &data[4..20]; + let next_hop_str = format!( + "{:02x}{:02x}:{:02x}{:02x}:{:02x}{:02x}:{:02x}{:02x}:{:02x}{:02x}:{:02x}{:02x}:{:02x}{:02x}:{:02x}{:02x}", + next_hop[0], next_hop[1], next_hop[2], next_hop[3], + next_hop[4], next_hop[5], next_hop[6], next_hop[7], + next_hop[8], next_hop[9], next_hop[10], next_hop[11], + next_hop[12], next_hop[13], next_hop[14], next_hop[15] + ); + log_event.insert("next_hop", next_hop_str); + } + } + _ => {} + } + + // Parse additional router data if present + let base_offset = match next_hop_type { + 1 => 8, // IPv4 + 2 => 20, // IPv6 + _ => 4, + }; + + if data.len() >= base_offset + 16 { + let src_mask = u32::from_be_bytes([ + data[base_offset], data[base_offset + 1], + data[base_offset + 2], data[base_offset + 3] + ]); + let dst_mask = u32::from_be_bytes([ + data[base_offset + 4], data[base_offset + 5], + data[base_offset + 6], data[base_offset + 7] + ]); + + log_event.insert("src_mask", src_mask); + log_event.insert("dst_mask", dst_mask); + } + } + + /// Parse interface counters record + fn parse_interface_counters(&self, data: &[u8], log_event: &mut LogEvent) { + if data.len() < 88 { + return; + } + + let if_index = u32::from_be_bytes([data[0], data[1], data[2], data[3]]); + let if_type = u32::from_be_bytes([data[4], data[5], data[6], data[7]]); + let if_speed = u64::from_be_bytes([ + data[8], data[9], data[10], data[11], + data[12], data[13], data[14], data[15] + ]); + let if_direction = u32::from_be_bytes([data[16], data[17], data[18], data[19]]); + let if_status = u32::from_be_bytes([data[20], data[21], data[22], data[23]]); + + // Input counters + let if_in_octets = u64::from_be_bytes([ + data[24], data[25], data[26], data[27], + data[28], data[29], data[30], data[31] + ]); + let if_in_ucast_pkts = u32::from_be_bytes([data[32], data[33], data[34], data[35]]); + let if_in_mcast_pkts = u32::from_be_bytes([data[36], data[37], data[38], data[39]]); + let if_in_bcast_pkts = u32::from_be_bytes([data[40], data[41], data[42], data[43]]); + let if_in_discards = u32::from_be_bytes([data[44], data[45], data[46], data[47]]); + let if_in_errors = u32::from_be_bytes([data[48], data[49], data[50], data[51]]); + let if_in_unknown_protos = u32::from_be_bytes([data[52], data[53], data[54], data[55]]); + + // Output counters + let if_out_octets = u64::from_be_bytes([ + data[56], data[57], data[58], data[59], + data[60], data[61], data[62], data[63] + ]); + let if_out_ucast_pkts = u32::from_be_bytes([data[64], data[65], data[66], data[67]]); + let if_out_mcast_pkts = u32::from_be_bytes([data[68], data[69], data[70], data[71]]); + let if_out_bcast_pkts = u32::from_be_bytes([data[72], data[73], data[74], data[75]]); + let if_out_discards = u32::from_be_bytes([data[76], data[77], data[78], data[79]]); + let if_out_errors = u32::from_be_bytes([data[80], data[81], data[82], data[83]]); + let if_promiscuous_mode = u32::from_be_bytes([data[84], data[85], data[86], data[87]]); + + log_event.insert("if_index", if_index); + log_event.insert("if_type", if_type); + log_event.insert("if_speed", if_speed as i64); + log_event.insert("if_direction", if_direction); + log_event.insert("if_status", if_status); + log_event.insert("if_in_octets", if_in_octets as i64); + log_event.insert("if_in_ucast_pkts", if_in_ucast_pkts); + log_event.insert("if_in_mcast_pkts", if_in_mcast_pkts); + log_event.insert("if_in_bcast_pkts", if_in_bcast_pkts); + log_event.insert("if_in_discards", if_in_discards); + log_event.insert("if_in_errors", if_in_errors); + log_event.insert("if_in_unknown_protos", if_in_unknown_protos); + log_event.insert("if_out_octets", if_out_octets as i64); + log_event.insert("if_out_ucast_pkts", if_out_ucast_pkts); + log_event.insert("if_out_mcast_pkts", if_out_mcast_pkts); + log_event.insert("if_out_bcast_pkts", if_out_bcast_pkts); + log_event.insert("if_out_discards", if_out_discards); + log_event.insert("if_out_errors", if_out_errors); + log_event.insert("if_promiscuous_mode", if_promiscuous_mode); + + // Calculate utilization if speed is known + if if_speed > 0 { + let total_octets = if_in_octets + if_out_octets; + let utilization = (total_octets as f64 * 8.0) / if_speed as f64 * 100.0; + log_event.insert("if_utilization_percent", utilization); + } + } + + /// Parse Ethernet interface counters record + fn parse_ethernet_counters(&self, data: &[u8], log_event: &mut LogEvent) { + if data.len() < 52 { + return; + } + + let dot3_stats_alignment_errors = u32::from_be_bytes([data[0], data[1], data[2], data[3]]); + let dot3_stats_fcs_errors = u32::from_be_bytes([data[4], data[5], data[6], data[7]]); + let dot3_stats_single_collision_frames = u32::from_be_bytes([data[8], data[9], data[10], data[11]]); + let dot3_stats_multiple_collision_frames = u32::from_be_bytes([data[12], data[13], data[14], data[15]]); + let dot3_stats_sqe_test_errors = u32::from_be_bytes([data[16], data[17], data[18], data[19]]); + let dot3_stats_deferred_transmissions = u32::from_be_bytes([data[20], data[21], data[22], data[23]]); + let dot3_stats_late_collisions = u32::from_be_bytes([data[24], data[25], data[26], data[27]]); + let dot3_stats_excessive_collisions = u32::from_be_bytes([data[28], data[29], data[30], data[31]]); + let dot3_stats_internal_mac_transmit_errors = u32::from_be_bytes([data[32], data[33], data[34], data[35]]); + let dot3_stats_carrier_sense_errors = u32::from_be_bytes([data[36], data[37], data[38], data[39]]); + let dot3_stats_frame_too_longs = u32::from_be_bytes([data[40], data[41], data[42], data[43]]); + let dot3_stats_internal_mac_receive_errors = u32::from_be_bytes([data[44], data[45], data[46], data[47]]); + let dot3_stats_symbol_errors = u32::from_be_bytes([data[48], data[49], data[50], data[51]]); + + log_event.insert("dot3_stats_alignment_errors", dot3_stats_alignment_errors); + log_event.insert("dot3_stats_fcs_errors", dot3_stats_fcs_errors); + log_event.insert("dot3_stats_single_collision_frames", dot3_stats_single_collision_frames); + log_event.insert("dot3_stats_multiple_collision_frames", dot3_stats_multiple_collision_frames); + log_event.insert("dot3_stats_sqe_test_errors", dot3_stats_sqe_test_errors); + log_event.insert("dot3_stats_deferred_transmissions", dot3_stats_deferred_transmissions); + log_event.insert("dot3_stats_late_collisions", dot3_stats_late_collisions); + log_event.insert("dot3_stats_excessive_collisions", dot3_stats_excessive_collisions); + log_event.insert("dot3_stats_internal_mac_transmit_errors", dot3_stats_internal_mac_transmit_errors); + log_event.insert("dot3_stats_carrier_sense_errors", dot3_stats_carrier_sense_errors); + log_event.insert("dot3_stats_frame_too_longs", dot3_stats_frame_too_longs); + log_event.insert("dot3_stats_internal_mac_receive_errors", dot3_stats_internal_mac_receive_errors); + log_event.insert("dot3_stats_symbol_errors", dot3_stats_symbol_errors); + } + + /// Parse packet header based on protocol + fn parse_packet_header(&self, header_data: &[u8], log_event: &mut LogEvent, protocol: u32) { + match protocol { + 1 => { // Ethernet + if header_data.len() >= 14 { + self.parse_ethernet_header(header_data, log_event); + + // Check for IP payload + if header_data.len() > 14 { + let ethertype = u16::from_be_bytes([header_data[12], header_data[13]]); + match ethertype { + 0x0800 => { // IPv4 + if header_data.len() >= 34 { // 14 eth + 20 ip minimum + self.parse_ipv4_header(&header_data[14..], log_event); + } + } + 0x86DD => { // IPv6 + if header_data.len() >= 54 { // 14 eth + 40 ip minimum + self.parse_ipv6_header(&header_data[14..], log_event); + } + } + _ => {} + } + } + } + } + _ => { + debug!("Unknown header protocol: {}", protocol); + } + } + } + + /// Parse Ethernet header + fn parse_ethernet_header(&self, data: &[u8], log_event: &mut LogEvent) { + if data.len() < 14 { + return; + } + + // Already parsed in parse_ethernet_frame_data, but add here for completeness + let dst_mac = format!( + "{:02x}:{:02x}:{:02x}:{:02x}:{:02x}:{:02x}", + data[0], data[1], data[2], data[3], data[4], data[5] + ); + let src_mac = format!( + "{:02x}:{:02x}:{:02x}:{:02x}:{:02x}:{:02x}", + data[6], data[7], data[8], data[9], data[10], data[11] + ); + let ethertype = u16::from_be_bytes([data[12], data[13]]); + + log_event.insert("eth_dst", dst_mac); + log_event.insert("eth_src", src_mac); + log_event.insert("eth_type", ethertype); + } + + /// Parse IPv4 header + fn parse_ipv4_header(&self, data: &[u8], log_event: &mut LogEvent) { + if data.len() < 20 { + return; + } + + let version = (data[0] >> 4) & 0x0F; + let ihl = data[0] & 0x0F; + let tos = data[1]; + let total_length = u16::from_be_bytes([data[2], data[3]]); + let identification = u16::from_be_bytes([data[4], data[5]]); + let flags = (data[6] >> 5) & 0x07; + let fragment_offset = u16::from_be_bytes([data[6], data[7]]) & 0x1FFF; + let ttl = data[8]; + let protocol = data[9]; + let checksum = u16::from_be_bytes([data[10], data[11]]); + let src_addr = u32::from_be_bytes([data[12], data[13], data[14], data[15]]); + let dst_addr = u32::from_be_bytes([data[16], data[17], data[18], data[19]]); + + log_event.insert("ip_version", version); + log_event.insert("ip_ihl", ihl); + log_event.insert("ip_tos", tos); + log_event.insert("ip_total_length", total_length); + log_event.insert("ip_identification", identification); + log_event.insert("ip_flags", flags); + log_event.insert("ip_fragment_offset", fragment_offset); + log_event.insert("ip_ttl", ttl); + log_event.insert("ip_protocol", protocol); + log_event.insert("ip_checksum", checksum); + + let src_addr_str = format!( + "{}.{}.{}.{}", + (src_addr >> 24) & 0xFF, + (src_addr >> 16) & 0xFF, + (src_addr >> 8) & 0xFF, + src_addr & 0xFF + ); + let dst_addr_str = format!( + "{}.{}.{}.{}", + (dst_addr >> 24) & 0xFF, + (dst_addr >> 16) & 0xFF, + (dst_addr >> 8) & 0xFF, + dst_addr & 0xFF + ); + + log_event.insert("ip_src", src_addr_str); + log_event.insert("ip_dst", dst_addr_str); + + // Parse protocol-specific headers + let header_length = (ihl * 4) as usize; + if data.len() > header_length { + let payload = &data[header_length..]; + match protocol { + 6 => { // TCP + self.parse_tcp_header(payload, log_event); + } + 17 => { // UDP + self.parse_udp_header(payload, log_event); + } + 1 => { // ICMP + self.parse_icmp_header(payload, log_event); + } + _ => {} + } + } + } + + /// Parse IPv6 header + fn parse_ipv6_header(&self, data: &[u8], log_event: &mut LogEvent) { + if data.len() < 40 { + return; + } + + let version = (data[0] >> 4) & 0x0F; + let traffic_class = ((data[0] & 0x0F) << 4) | ((data[1] >> 4) & 0x0F); + let flow_label = u32::from_be_bytes([0, data[1] & 0x0F, data[2], data[3]]); + let payload_length = u16::from_be_bytes([data[4], data[5]]); + let next_header = data[6]; + let hop_limit = data[7]; + + log_event.insert("ipv6_version", version); + log_event.insert("ipv6_traffic_class", traffic_class); + log_event.insert("ipv6_flow_label", flow_label); + log_event.insert("ipv6_payload_length", payload_length); + log_event.insert("ipv6_next_header", next_header); + log_event.insert("ipv6_hop_limit", hop_limit); + + // Format IPv6 addresses + let src_addr = format!( + "{:02x}{:02x}:{:02x}{:02x}:{:02x}{:02x}:{:02x}{:02x}:{:02x}{:02x}:{:02x}{:02x}:{:02x}{:02x}:{:02x}{:02x}", + data[8], data[9], data[10], data[11], data[12], data[13], data[14], data[15], + data[16], data[17], data[18], data[19], data[20], data[21], data[22], data[23] + ); + let dst_addr = format!( + "{:02x}{:02x}:{:02x}{:02x}:{:02x}{:02x}:{:02x}{:02x}:{:02x}{:02x}:{:02x}{:02x}:{:02x}{:02x}:{:02x}{:02x}", + data[24], data[25], data[26], data[27], data[28], data[29], data[30], data[31], + data[32], data[33], data[34], data[35], data[36], data[37], data[38], data[39] + ); + + log_event.insert("ipv6_src", src_addr); + log_event.insert("ipv6_dst", dst_addr); + } + + /// Parse TCP header + fn parse_tcp_header(&self, data: &[u8], log_event: &mut LogEvent) { + if data.len() < 20 { + return; + } + + let src_port = u16::from_be_bytes([data[0], data[1]]); + let dst_port = u16::from_be_bytes([data[2], data[3]]); + let sequence = u32::from_be_bytes([data[4], data[5], data[6], data[7]]); + let acknowledgment = u32::from_be_bytes([data[8], data[9], data[10], data[11]]); + let header_length = ((data[12] >> 4) & 0x0F) * 4; + let flags = data[13]; + let window_size = u16::from_be_bytes([data[14], data[15]]); + let checksum = u16::from_be_bytes([data[16], data[17]]); + let urgent_pointer = u16::from_be_bytes([data[18], data[19]]); + + log_event.insert("tcp_src_port", src_port); + log_event.insert("tcp_dst_port", dst_port); + log_event.insert("tcp_sequence", sequence); + log_event.insert("tcp_acknowledgment", acknowledgment); + log_event.insert("tcp_header_length", header_length); + log_event.insert("tcp_flags", flags); + log_event.insert("tcp_window_size", window_size); + log_event.insert("tcp_checksum", checksum); + log_event.insert("tcp_urgent_pointer", urgent_pointer); + + // Parse individual TCP flags + log_event.insert("tcp_flag_urg", (flags & 0x20) != 0); + log_event.insert("tcp_flag_ack", (flags & 0x10) != 0); + log_event.insert("tcp_flag_psh", (flags & 0x08) != 0); + log_event.insert("tcp_flag_rst", (flags & 0x04) != 0); + log_event.insert("tcp_flag_syn", (flags & 0x02) != 0); + log_event.insert("tcp_flag_fin", (flags & 0x01) != 0); + } + + /// Parse UDP header + fn parse_udp_header(&self, data: &[u8], log_event: &mut LogEvent) { + if data.len() < 8 { + return; + } + + let src_port = u16::from_be_bytes([data[0], data[1]]); + let dst_port = u16::from_be_bytes([data[2], data[3]]); + let length = u16::from_be_bytes([data[4], data[5]]); + let checksum = u16::from_be_bytes([data[6], data[7]]); + + log_event.insert("udp_src_port", src_port); + log_event.insert("udp_dst_port", dst_port); + log_event.insert("udp_length", length); + log_event.insert("udp_checksum", checksum); + } + + /// Parse ICMP header + fn parse_icmp_header(&self, data: &[u8], log_event: &mut LogEvent) { + if data.len() < 8 { + return; + } + + let icmp_type = data[0]; + let icmp_code = data[1]; + let checksum = u16::from_be_bytes([data[2], data[3]]); + let rest_of_header = u32::from_be_bytes([data[4], data[5], data[6], data[7]]); + + log_event.insert("icmp_type", icmp_type); + log_event.insert("icmp_code", icmp_code); + log_event.insert("icmp_checksum", checksum); + log_event.insert("icmp_rest_of_header", rest_of_header); + + // Common ICMP types + let icmp_type_name = match icmp_type { + 0 => "Echo Reply", + 3 => "Destination Unreachable", + 4 => "Source Quench", + 5 => "Redirect", + 8 => "Echo Request", + 11 => "Time Exceeded", + 12 => "Parameter Problem", + 13 => "Timestamp Request", + 14 => "Timestamp Reply", + _ => "Other", + }; + log_event.insert("icmp_type_name", icmp_type_name); + } +} + +/// sFlow specific events +#[derive(Debug)] +pub struct SflowPacketProcessed { + pub peer_addr: SocketAddr, + pub flow_samples: usize, + pub counter_samples: usize, + pub unknown_samples: usize, + pub event_count: usize, +} + +impl vector_lib::internal_event::InternalEvent for SflowPacketProcessed { + fn emit(self) { + debug!( + message = "sFlow packet processed", + peer_addr = %self.peer_addr, + flow_samples = self.flow_samples, + counter_samples = self.counter_samples, + unknown_samples = self.unknown_samples, + event_count = self.event_count, + ); + + if self.unknown_samples > 0 { + warn!( + message = "sFlow packet contained unknown samples", + peer_addr = %self.peer_addr, + unknown_samples = self.unknown_samples, + ); + } + } +} + +#[cfg(test)] +mod tests { + use super::*; + use crate::sources::netflow::NetflowConfig; + use crate::sources::netflow::fields::FieldParser; + use base64::Engine; + use std::net::{IpAddr, Ipv4Addr}; + + fn test_peer_addr() -> SocketAddr { + SocketAddr::new(IpAddr::V4(Ipv4Addr::new(192, 168, 1, 100)), 6343) + } + + fn create_sflow_header(num_samples: u32) -> Vec { + let mut data = vec![0u8; 28]; + data[0..4].copy_from_slice(&5u32.to_be_bytes()); // version + data[4..8].copy_from_slice(&1u32.to_be_bytes()); // agent_address_type (IPv4) + data[8..12].copy_from_slice(&0xC0A80101u32.to_be_bytes()); // agent_address (192.168.1.1) + data[12..16].copy_from_slice(&0u32.to_be_bytes()); // sub_agent_id + data[16..20].copy_from_slice(&12345u32.to_be_bytes()); // sequence_number + data[20..24].copy_from_slice(&54321u32.to_be_bytes()); // sys_uptime + data[24..28].copy_from_slice(&num_samples.to_be_bytes()); // num_samples + data + } + + #[test] + fn test_sflow_header_parsing() { + let data = create_sflow_header(1); + let header = SflowHeader::from_bytes(&data).unwrap(); + + assert_eq!(header.version, 5); + assert_eq!(header.agent_address_type, 1); + assert_eq!(header.agent_address_string(), "192.168.1.1"); + assert_eq!(header.sub_agent_id, 0); + assert_eq!(header.sequence_number, 12345); + assert_eq!(header.sys_uptime, 54321); + assert_eq!(header.num_samples, 1); + } + + #[test] + fn test_invalid_sflow_header() { + // Too short + let short_data = vec![0u8; 10]; + assert!(SflowHeader::from_bytes(&short_data).is_err()); + + // Wrong version + let mut wrong_version = create_sflow_header(1); + wrong_version[0..4].copy_from_slice(&4u32.to_be_bytes()); + assert!(SflowHeader::from_bytes(&wrong_version).is_err()); + + // Too many samples + let mut too_many = create_sflow_header(1); + too_many[24..28].copy_from_slice(&2000u32.to_be_bytes()); + assert!(SflowHeader::from_bytes(&too_many).is_err()); + + // Unknown address type + let mut unknown_addr = create_sflow_header(1); + unknown_addr[4..8].copy_from_slice(&99u32.to_be_bytes()); + assert!(SflowHeader::from_bytes(&unknown_addr).is_err()); + } + + #[test] + fn test_can_parse() { + // Valid sFlow + let packet = create_sflow_header(1); + assert!(SflowParser::can_parse(&packet)); + + // Invalid version + let mut invalid_version = packet.clone(); + invalid_version[0..4].copy_from_slice(&4u32.to_be_bytes()); + assert!(!SflowParser::can_parse(&invalid_version)); + + // Too short + let short_packet = vec![0u8; 3]; + assert!(!SflowParser::can_parse(&short_packet)); + + // Invalid address type + let mut invalid_addr = packet.clone(); + invalid_addr[4..8].copy_from_slice(&99u32.to_be_bytes()); + assert!(!SflowParser::can_parse(&invalid_addr)); + } + + #[test] + fn test_sample_header_parsing() { + let mut data = vec![0u8; 8]; + data[0..4].copy_from_slice(&1u32.to_be_bytes()); // flow sample + data[4..8].copy_from_slice(&32u32.to_be_bytes()); // length + + let header = SampleHeader::from_bytes(&data).unwrap(); + assert_eq!(header.sample_type, SampleType::FlowSample); + assert_eq!(header.sample_length, 32); + + // Test other sample types + data[0..4].copy_from_slice(&2u32.to_be_bytes()); // counter sample + let header = SampleHeader::from_bytes(&data).unwrap(); + assert_eq!(header.sample_type, SampleType::CounterSample); + + // Invalid sample type + data[0..4].copy_from_slice(&99u32.to_be_bytes()); + assert!(SampleHeader::from_bytes(&data).is_err()); + + // Invalid length + data[0..4].copy_from_slice(&1u32.to_be_bytes()); + data[4..8].copy_from_slice(&3u32.to_be_bytes()); // length < 8 + assert!(SampleHeader::from_bytes(&data).is_err()); + } + + #[test] + fn test_sample_type_conversion() { + assert_eq!(SampleType::from_u32(1), Some(SampleType::FlowSample)); + assert_eq!(SampleType::from_u32(2), Some(SampleType::CounterSample)); + assert_eq!(SampleType::from_u32(3), Some(SampleType::ExpandedFlowSample)); + assert_eq!(SampleType::from_u32(4), Some(SampleType::ExpandedCounterSample)); + assert_eq!(SampleType::from_u32(99), None); + + assert_eq!(SampleType::FlowSample.as_str(), "flow_sample"); + assert_eq!(SampleType::CounterSample.as_str(), "counter_sample"); + } + + #[test] + fn test_ipv6_agent_address() { + let mut data = vec![0u8; 40]; + data[0..4].copy_from_slice(&5u32.to_be_bytes()); // version + data[4..8].copy_from_slice(&2u32.to_be_bytes()); // agent_address_type (IPv6) + + // IPv6 address: 2001:db8::1 + data[8..24].copy_from_slice(&[ + 0x20, 0x01, 0x0d, 0xb8, 0x00, 0x00, 0x00, 0x00, + 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x01 + ]); + + data[24..28].copy_from_slice(&0u32.to_be_bytes()); // sub_agent_id + data[28..32].copy_from_slice(&12345u32.to_be_bytes()); // sequence_number + data[32..36].copy_from_slice(&54321u32.to_be_bytes()); // sys_uptime + data[36..40].copy_from_slice(&1u32.to_be_bytes()); // num_samples + + let header = SflowHeader::from_bytes(&data).unwrap(); + assert_eq!(header.agent_address_type, 2); + assert_eq!(header.agent_address_string(), "2001:0db8:0000:0000:0000:0000:0000:0001"); + assert_eq!(header.header_size(), 40); + } + + #[test] + fn test_header_to_log_event() { + let data = create_sflow_header(2); + let header = SflowHeader::from_bytes(&data).unwrap(); + let log_event = header.to_log_event(); + + assert_eq!(log_event.get("flow_type").unwrap().as_str().unwrap(), "sflow"); + assert_eq!(log_event.get("version").unwrap().as_integer().unwrap(), 5); + assert_eq!(log_event.get("agent_address_type").unwrap().as_integer().unwrap(), 1); + assert_eq!(log_event.get("agent_address").unwrap().as_str().unwrap(), "192.168.1.1"); + assert_eq!(log_event.get("sequence_number").unwrap().as_integer().unwrap(), 12345); + assert_eq!(log_event.get("num_samples").unwrap().as_integer().unwrap(), 2); + } + + #[test] + fn test_basic_packet_parsing() { + let config = NetflowConfig::default(); + let _field_parser = FieldParser::new(&config); + let parser = SflowParser::new(); + + let packet = create_sflow_header(0); + let events = parser.parse(&packet, test_peer_addr(), false).unwrap(); + + // Should get header event for zero samples + assert_eq!(events.len(), 1); + + if let Event::Log(log) = &events[0] { + assert_eq!(log.get("flow_type").unwrap().as_str().unwrap(), "sflow"); + assert_eq!(log.get("num_samples").unwrap().as_integer().unwrap(), 0); + } + } + + #[test] + fn test_flow_sample_parsing() { + let config = NetflowConfig::default(); + let _field_parser = FieldParser::new(&config); + let parser = SflowParser::new(); + + // Create sFlow packet with flow sample + let mut packet = create_sflow_header(1); + + // Add flow sample + packet.extend_from_slice(&1u32.to_be_bytes()); // sample_type (flow) + packet.extend_from_slice(&36u32.to_be_bytes()); // sample_length (8 header + 28 data) + packet.extend_from_slice(&100u32.to_be_bytes()); // sequence_number + packet.extend_from_slice(&0x01000001u32.to_be_bytes()); // source_id (type=1, index=1) + packet.extend_from_slice(&1000u32.to_be_bytes()); // sampling_rate + packet.extend_from_slice(&5000u32.to_be_bytes()); // sample_pool + packet.extend_from_slice(&0u32.to_be_bytes()); // drops + packet.extend_from_slice(&1u32.to_be_bytes()); // input_interface + packet.extend_from_slice(&2u32.to_be_bytes()); // output_interface + packet.extend_from_slice(&0u32.to_be_bytes()); // num_flow_records + + let events = parser.parse(&packet, test_peer_addr(), false).unwrap(); + + assert_eq!(events.len(), 1); + + if let Event::Log(log) = &events[0] { + assert_eq!(log.get("flow_type").unwrap().as_str().unwrap(), "sflow_flow_sample"); + assert_eq!(log.get("sample_index").unwrap().as_integer().unwrap(), 0); + assert_eq!(log.get("sequence_number").unwrap().as_integer().unwrap(), 100); + assert_eq!(log.get("source_id_type").unwrap().as_integer().unwrap(), 1); + assert_eq!(log.get("source_id_index").unwrap().as_integer().unwrap(), 1); + assert_eq!(log.get("sampling_rate").unwrap().as_integer().unwrap(), 1000); + assert_eq!(log.get("sample_pool").unwrap().as_integer().unwrap(), 5000); + assert_eq!(log.get("input_interface").unwrap().as_integer().unwrap(), 1); + assert_eq!(log.get("output_interface").unwrap().as_integer().unwrap(), 2); + } + } + + #[test] + fn test_counter_sample_parsing() { + let config = NetflowConfig::default(); + let _field_parser = FieldParser::new(&config); + let parser = SflowParser::new(); + + // Create sFlow packet with counter sample + let mut packet = create_sflow_header(1); + + // Add counter sample + packet.extend_from_slice(&2u32.to_be_bytes()); // sample_type (counter) + packet.extend_from_slice(&20u32.to_be_bytes()); // sample_length (8 header + 12 data) + packet.extend_from_slice(&200u32.to_be_bytes()); // sequence_number + packet.extend_from_slice(&0x01000002u32.to_be_bytes()); // source_id (type=1, index=2) + packet.extend_from_slice(&0u32.to_be_bytes()); // num_counter_records + + let events = parser.parse(&packet, test_peer_addr(), false).unwrap(); + + assert_eq!(events.len(), 1); + + if let Event::Log(log) = &events[0] { + assert_eq!(log.get("flow_type").unwrap().as_str().unwrap(), "sflow_counter_sample"); + assert_eq!(log.get("sample_index").unwrap().as_integer().unwrap(), 0); + assert_eq!(log.get("sequence_number").unwrap().as_integer().unwrap(), 200); + assert_eq!(log.get("source_id_type").unwrap().as_integer().unwrap(), 1); + assert_eq!(log.get("source_id_index").unwrap().as_integer().unwrap(), 2); + assert_eq!(log.get("num_counter_records").unwrap().as_integer().unwrap(), 0); + } + } + + #[test] + fn test_raw_data_inclusion() { + let config = NetflowConfig::default(); + let _field_parser = FieldParser::new(&config); + let parser = SflowParser::new(); + + let packet = create_sflow_header(0); + + // Test with raw data inclusion + let events = parser.parse(&packet, test_peer_addr(), true).unwrap(); + assert!(!events.is_empty()); + + if let Event::Log(log) = &events[0] { + assert!(log.get("raw_data").is_some()); + let raw_data = log.get("raw_data").unwrap().as_str().unwrap(); + + // Should be valid base64 + assert!(base64::engine::general_purpose::STANDARD.decode(raw_data.as_bytes()).is_ok()); + } + + // Test without raw data inclusion + let events = parser.parse(&packet, test_peer_addr(), false).unwrap(); + assert!(!events.is_empty()); + + if let Event::Log(log) = &events[0] { + assert!(log.get("raw_data").is_none()); + } + } + + #[test] + fn test_malformed_sample_handling() { + let config = NetflowConfig::default(); + let _field_parser = FieldParser::new(&config); + let parser = SflowParser::new(); + + // Create packet with malformed sample + let mut packet = create_sflow_header(1); + + // Add truncated sample + packet.extend_from_slice(&1u32.to_be_bytes()); // sample_type + packet.extend_from_slice(&100u32.to_be_bytes()); // claimed length (too large) + packet.extend_from_slice(&[0u8; 10]); // only 10 bytes of data + + let events = parser.parse(&packet, test_peer_addr(), false).unwrap(); + + // Should handle gracefully and return header event + assert!(!events.is_empty()); + } + + #[test] + fn test_ethernet_header_parsing() { + let config = NetflowConfig::default(); + let _field_parser = FieldParser::new(&config); + let parser = SflowParser::new(); + + let eth_data = vec![ + 0x00, 0x1B, 0x21, 0x3C, 0x4D, 0x5E, // dst MAC + 0x00, 0x1A, 0x2B, 0x3C, 0x4D, 0x5F, // src MAC + 0x08, 0x00, // ethertype (IPv4) + ]; + + let mut log_event = LogEvent::default(); + parser.parse_ethernet_header(ð_data, &mut log_event); + + assert_eq!(log_event.get("eth_dst").unwrap().as_str().unwrap(), "00:1b:21:3c:4d:5e"); + assert_eq!(log_event.get("eth_src").unwrap().as_str().unwrap(), "00:1a:2b:3c:4d:5f"); + assert_eq!(log_event.get("eth_type").unwrap().as_integer().unwrap(), 0x0800); + } + + #[test] + fn test_ipv4_header_parsing() { + let config = NetflowConfig::default(); + let _field_parser = FieldParser::new(&config); + let parser = SflowParser::new(); + + let ipv4_data = vec![ + 0x45, // version=4, ihl=5 + 0x00, // tos + 0x00, 0x3C, // total_length + 0x1C, 0x46, // identification + 0x40, 0x00, // flags=2, fragment_offset=0 + 0x40, // ttl=64 + 0x06, // protocol=TCP + 0xB1, 0xE6, // checksum + 0xC0, 0xA8, 0x01, 0x01, // src: 192.168.1.1 + 0x08, 0x08, 0x08, 0x08, // dst: 8.8.8.8 + ]; + + let mut log_event = LogEvent::default(); + parser.parse_ipv4_header(&ipv4_data, &mut log_event); + + assert_eq!(log_event.get("ip_version").unwrap().as_integer().unwrap(), 4); + assert_eq!(log_event.get("ip_ihl").unwrap().as_integer().unwrap(), 5); + assert_eq!(log_event.get("ip_protocol").unwrap().as_integer().unwrap(), 6); + assert_eq!(log_event.get("ip_ttl").unwrap().as_integer().unwrap(), 64); + assert_eq!(log_event.get("ip_src").unwrap().as_str().unwrap(), "192.168.1.1"); + assert_eq!(log_event.get("ip_dst").unwrap().as_str().unwrap(), "8.8.8.8"); + } + + #[test] + fn test_tcp_header_parsing() { + let config = NetflowConfig::default(); + let _field_parser = FieldParser::new(&config); + let parser = SflowParser::new(); + + let tcp_data = vec![ + 0x00, 0x50, // src_port: 80 + 0x01, 0xBB, // dst_port: 443 + 0x00, 0x00, 0x00, 0x01, // sequence + 0x00, 0x00, 0x00, 0x02, // acknowledgment + 0x50, // header_length=20 + 0x18, // flags: ACK+PSH + 0x20, 0x00, // window_size + 0x00, 0x00, // checksum + 0x00, 0x00, // urgent_pointer + ]; + + let mut log_event = LogEvent::default(); + parser.parse_tcp_header(&tcp_data, &mut log_event); + + assert_eq!(log_event.get("tcp_src_port").unwrap().as_integer().unwrap(), 80); + assert_eq!(log_event.get("tcp_dst_port").unwrap().as_integer().unwrap(), 443); + assert_eq!(log_event.get("tcp_flags").unwrap().as_integer().unwrap(), 0x18); + assert_eq!(log_event.get("tcp_flag_ack").unwrap().as_boolean().unwrap(), true); + assert_eq!(log_event.get("tcp_flag_psh").unwrap().as_boolean().unwrap(), true); + assert_eq!(log_event.get("tcp_flag_syn").unwrap().as_boolean().unwrap(), false); + } + + #[test] + fn test_udp_header_parsing() { + let config = NetflowConfig::default(); + let _field_parser = FieldParser::new(&config); + let parser = SflowParser::new(); + + let udp_data = vec![ + 0x00, 0x35, // src_port: 53 (DNS) + 0xC0, 0x00, // dst_port: 49152 + 0x00, 0x20, // length: 32 + 0x00, 0x00, // checksum + ]; + + let mut log_event = LogEvent::default(); + parser.parse_udp_header(&udp_data, &mut log_event); + + assert_eq!(log_event.get("udp_src_port").unwrap().as_integer().unwrap(), 53); + assert_eq!(log_event.get("udp_dst_port").unwrap().as_integer().unwrap(), 49152); + assert_eq!(log_event.get("udp_length").unwrap().as_integer().unwrap(), 32); + } + + #[test] + fn test_icmp_header_parsing() { + let config = NetflowConfig::default(); + let _field_parser = FieldParser::new(&config); + let parser = SflowParser::new(); + + let icmp_data = vec![ + 0x08, // type: Echo Request + 0x00, // code + 0x00, 0x00, // checksum + 0x00, 0x01, 0x00, 0x02, // rest of header + ]; + + let mut log_event = LogEvent::default(); + parser.parse_icmp_header(&icmp_data, &mut log_event); + + assert_eq!(log_event.get("icmp_type").unwrap().as_integer().unwrap(), 8); + assert_eq!(log_event.get("icmp_code").unwrap().as_integer().unwrap(), 0); + assert_eq!(log_event.get("icmp_type_name").unwrap().as_str().unwrap(), "Echo Request"); + } + + #[test] + fn test_interface_counters_parsing() { + let config = NetflowConfig::default(); + let _field_parser = FieldParser::new(&config); + let parser = SflowParser::new(); + + let mut counter_data = vec![0u8; 88]; + + // Set some test values + counter_data[0..4].copy_from_slice(&1u32.to_be_bytes()); // if_index + counter_data[4..8].copy_from_slice(&6u32.to_be_bytes()); // if_type (ethernet) + counter_data[8..16].copy_from_slice(&1000000000u64.to_be_bytes()); // if_speed (1 Gbps) + counter_data[24..32].copy_from_slice(&1000000u64.to_be_bytes()); // if_in_octets + counter_data[32..36].copy_from_slice(&10000u32.to_be_bytes()); // if_in_ucast_pkts + counter_data[56..64].copy_from_slice(&2000000u64.to_be_bytes()); // if_out_octets + counter_data[64..68].copy_from_slice(&20000u32.to_be_bytes()); // if_out_ucast_pkts + + let mut log_event = LogEvent::default(); + parser.parse_interface_counters(&counter_data, &mut log_event); + + assert_eq!(log_event.get("if_index").unwrap().as_integer().unwrap(), 1); + assert_eq!(log_event.get("if_type").unwrap().as_integer().unwrap(), 6); + assert_eq!(log_event.get("if_speed").unwrap().as_integer().unwrap(), 1000000000); + assert_eq!(log_event.get("if_in_octets").unwrap().as_integer().unwrap(), 1000000); + assert_eq!(log_event.get("if_out_octets").unwrap().as_integer().unwrap(), 2000000); + + // Check calculated utilization + assert!(log_event.get("if_utilization_percent").is_some()); + let utilization = log_event.get("if_utilization_percent").unwrap().as_float().unwrap(); + assert!(utilization > ordered_float::NotNan::new(0.0).unwrap()); + } + + #[test] + fn test_ethernet_counters_parsing() { + let config = NetflowConfig::default(); + let _field_parser = FieldParser::new(&config); + let parser = SflowParser::new(); + + let mut counter_data = vec![0u8; 52]; + + // Set some test values + counter_data[0..4].copy_from_slice(&10u32.to_be_bytes()); // alignment_errors + counter_data[4..8].copy_from_slice(&5u32.to_be_bytes()); // fcs_errors + counter_data[24..28].copy_from_slice(&2u32.to_be_bytes()); // late_collisions + + let mut log_event = LogEvent::default(); + parser.parse_ethernet_counters(&counter_data, &mut log_event); + + assert_eq!(log_event.get("dot3_stats_alignment_errors").unwrap().as_integer().unwrap(), 10); + assert_eq!(log_event.get("dot3_stats_fcs_errors").unwrap().as_integer().unwrap(), 5); + assert_eq!(log_event.get("dot3_stats_late_collisions").unwrap().as_integer().unwrap(), 2); + } + + #[test] + fn test_multiple_samples() { + let config = NetflowConfig::default(); + let _field_parser = FieldParser::new(&config); + let parser = SflowParser::new(); + + // Create packet with multiple samples + let mut packet = create_sflow_header(2); + + // First sample (flow) + packet.extend_from_slice(&1u32.to_be_bytes()); // sample_type + packet.extend_from_slice(&44u32.to_be_bytes()); // sample_length + packet.extend([0u8; 36]); // sample data + + // Second sample (counter) + packet.extend_from_slice(&2u32.to_be_bytes()); // sample_type + packet.extend_from_slice(&28u32.to_be_bytes()); // sample_length + packet.extend([0u8; 20]); // sample data + + let events = parser.parse(&packet, test_peer_addr(), false).unwrap(); + + // Should get 2 sample events + assert_eq!(events.len(), 2); + + let flow_event = events.iter().find(|e| { + if let Event::Log(log) = e { + log.get("flow_type").unwrap().as_str().unwrap().contains("flow") + } else { + false + } + }); + assert!(flow_event.is_some()); + + let counter_event = events.iter().find(|e| { + if let Event::Log(log) = e { + log.get("flow_type").unwrap().as_str().unwrap().contains("counter") + } else { + false + } + }); + assert!(counter_event.is_some()); + } + + #[test] + fn test_unknown_sample_type() { + let config = NetflowConfig::default(); + let _field_parser = FieldParser::new(&config); + let parser = SflowParser::new(); + + let mut packet = create_sflow_header(1); + + // Add unknown sample type + packet.extend_from_slice(&99u32.to_be_bytes()); // unknown sample_type + packet.extend_from_slice(&16u32.to_be_bytes()); // sample_length + packet.extend([0u8; 8]); // minimal sample data + + let events = parser.parse(&packet, test_peer_addr(), false).unwrap(); + + // Should return header event due to unknown sample + assert_eq!(events.len(), 1); + + if let Event::Log(log) = &events[0] { + assert_eq!(log.get("flow_type").unwrap().as_str().unwrap(), "sflow"); + assert_eq!(log.get("unknown_samples").unwrap().as_integer().unwrap(), 1); + } + } +} \ No newline at end of file diff --git a/src/sources/netflow/templates.rs b/src/sources/netflow/templates.rs new file mode 100644 index 0000000000000..8c7f971ddaa5b --- /dev/null +++ b/src/sources/netflow/templates.rs @@ -0,0 +1,1040 @@ +//! Template management for NetFlow v9 and IPFIX. +//! +//! NetFlow v9 and IPFIX use templates to define the structure of data records. +//! This module provides thread-safe template caching with automatic cleanup. + + +use std::net::SocketAddr; +use std::sync::{Arc, RwLock}; +#[cfg(not(test))] +use std::sync::atomic::{AtomicU64, Ordering}; +use std::time::{Duration, Instant}; +#[cfg(not(test))] +use std::time::SystemTime; +use std::collections::VecDeque; + +#[cfg(not(test))] +use dashmap::DashMap; +#[cfg(test)] +use std::collections::HashMap; + +use crate::sources::netflow::events::*; +use tracing::debug; + + +/// Unique identifier for a template. +/// +/// Templates are identified by the combination of: +/// - Source address (which exporter sent it) +/// - Observation domain ID (IPFIX) / Source ID (NetFlow v9) +/// - Template ID +pub type TemplateKey = (SocketAddr, u32, u16); + +/// Template field definition. +#[derive(Debug, Clone, PartialEq, Eq)] +pub struct TemplateField { + /// IPFIX/NetFlow field type identifier + pub field_type: u16, + /// Length of the field in bytes + pub field_length: u16, + /// Enterprise number for vendor-specific fields (IPFIX only) + pub enterprise_number: Option, + /// Whether this field is a scope field (Options Templates only) + pub is_scope: bool, +} + +/// Template definition containing field layout. +#[derive(Debug, Clone)] +pub struct Template { + /// Template identifier + pub template_id: u16, + /// List of fields in this template + pub fields: Vec, + /// Number of scope fields (Options Templates only, 0 for regular templates) + pub scope_field_count: u16, + /// When this template was created/last used + pub created: Instant, + /// Last time this template was accessed + pub last_used: Instant, + /// Number of times this template has been used + pub usage_count: u64, +} + +/// Buffered data record waiting for template. +#[derive(Debug, Clone)] +pub struct BufferedDataRecord { + /// Raw data bytes + pub data: Vec, + /// When this record was buffered + pub buffered_at: Instant, + /// Peer address that sent this data + pub peer_addr: SocketAddr, + /// Observation domain ID + pub observation_domain_id: u32, +} + + +impl Template { + /// Create a new template. + pub fn new(template_id: u16, fields: Vec) -> Self { + let now = Instant::now(); + Self { + template_id, + fields, + scope_field_count: 0, // Regular templates have no scope fields + created: now, + last_used: now, + usage_count: 0, + } + } + + /// Create a new options template with scope fields. + pub fn new_options(template_id: u16, fields: Vec, scope_field_count: u16) -> Self { + let now = Instant::now(); + Self { + template_id, + fields, + scope_field_count, + created: now, + last_used: now, + usage_count: 0, + } + } + + /// Mark template as used and update statistics. + pub fn mark_used(&mut self) { + self.last_used = Instant::now(); + self.usage_count = self.usage_count.saturating_add(1); + } + + /// Check if template has expired based on last usage. + pub fn is_expired(&self, timeout: Duration) -> bool { + self.last_used.elapsed() > timeout + } + + /// Calculate total record size in bytes (for fixed-length templates). + pub fn record_size(&self) -> Option { + let mut total_size = 0; + + for field in &self.fields { + // Variable-length fields (length 65535) make the total size indeterminate + if field.field_length == 65535 { + return None; + } + total_size += field.field_length as usize; + } + + Some(total_size) + } + + /// Check if template has any variable-length fields. + pub fn has_variable_fields(&self) -> bool { + self.fields.iter().any(|f| f.field_length == 65535) + } +} + +/// High-performance thread-safe template cache with lock-free reads and automatic cleanup. +/// +/// Uses DashMap for lock-free concurrent access, providing significant performance improvements +/// for high-throughput NetFlow/IPFIX processing scenarios (20M+ records/minute). +#[derive(Clone)] +pub struct TemplateCache { + #[cfg(not(test))] + cache: Arc, AtomicU64)>>, + #[cfg(test)] + cache: Arc>>>, + max_size: usize, + stats: Arc>, + /// Buffered data records waiting for templates + buffered_records: Arc>>>, + /// Maximum number of records to buffer per template + max_buffered_records: usize, +} + +/// Cache statistics for monitoring and debugging. +#[derive(Debug, Default, Clone)] +pub struct CacheStats { + pub hits: u64, + pub misses: u64, + pub insertions: u64, + pub evictions: u64, + pub expired_removals: u64, + pub current_size: usize, +} + +impl CacheStats { + /// Calculate cache hit ratio. + pub fn hit_ratio(&self) -> f64 { + let total = self.hits + self.misses; + if total == 0 { + 0.0 + } else { + self.hits as f64 / total as f64 + } + } +} + +impl TemplateCache { + /// Create a new template cache with the specified maximum size. + pub fn new(max_size: usize) -> Self { + Self::new_with_buffering(max_size, 100) + } + + /// Create a new template cache with buffering support. + /// + /// Uses DashMap for high-performance concurrent access, optimized for high-throughput + /// NetFlow/IPFIX processing scenarios. + pub fn new_with_buffering(max_size: usize, max_buffered_records: usize) -> Self { + #[cfg(not(test))] + let cache = Arc::new(DashMap::with_capacity(max_size)); + + #[cfg(test)] + let cache = Arc::new(RwLock::new(HashMap::new())); + + Self { + cache, + max_size, + stats: Arc::new(RwLock::new(CacheStats::default())), + buffered_records: Arc::new(RwLock::new(std::collections::HashMap::new())), + max_buffered_records, + } + } + + /// Get a template from the cache. + /// + /// Uses lock-free read access for high performance in concurrent scenarios. + /// Returns an Arc