From 5c749734b14c2d3be8689b0c5b72ae8d1ddec099 Mon Sep 17 00:00:00 2001 From: Ole Troan Date: Mon, 13 Mar 2017 13:39:52 +0100 Subject: Flowprobe: Stateful flows and IPv6, L4 recording Change-Id: I67839281623721bf42f0a918a53356143d9dc78a Signed-off-by: Ole Troan Signed-off-by: Pavel Kotucek Signed-off-by: Ole Troan --- src/plugins/flowprobe/flowprobe.h | 167 ++++++++++++++++++++++++++++++++++++++ 1 file changed, 167 insertions(+) create mode 100644 src/plugins/flowprobe/flowprobe.h (limited to 'src/plugins/flowprobe/flowprobe.h') diff --git a/src/plugins/flowprobe/flowprobe.h b/src/plugins/flowprobe/flowprobe.h new file mode 100644 index 00000000..196c92a7 --- /dev/null +++ b/src/plugins/flowprobe/flowprobe.h @@ -0,0 +1,167 @@ +/* + * flowprobe.h - ipfix probe plug-in header file + * + * Copyright (c) 2016 Cisco and/or its affiliates. + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at: + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +#ifndef __included_flowprobe_h__ +#define __included_flowprobe_h__ + +#include +#include +#include + +#include +#include +#include +#include +#include + +/* Default timers in seconds */ +#define FLOWPROBE_TIMER_ACTIVE (15) +#define FLOWPROBE_TIMER_PASSIVE 120 // XXXX: FOR TESTING (30*60) +#define FLOWPROBE_LOG2_HASHSIZE (18) + +typedef enum +{ + FLOW_RECORD_L2 = 1 << 0, + FLOW_RECORD_L3 = 1 << 1, + FLOW_RECORD_L4 = 1 << 2, + FLOW_RECORD_L2_IP4 = 1 << 3, + FLOW_RECORD_L2_IP6 = 1 << 4, + FLOW_N_RECORDS = 1 << 5, +} flowprobe_record_t; + +/* *INDENT-OFF* */ +typedef enum __attribute__ ((__packed__)) +{ + FLOW_VARIANT_IP4, + FLOW_VARIANT_IP6, + FLOW_VARIANT_L2, + FLOW_VARIANT_L2_IP4, + FLOW_VARIANT_L2_IP6, + FLOW_N_VARIANTS, +} flowprobe_variant_t; +/* *INDENT-ON* */ + +STATIC_ASSERT (sizeof (flowprobe_variant_t) == 1, + "flowprobe_variant_t is expected to be 1 byte, " + "revisit padding in flowprobe_key_t"); + +#define FLOW_MAXIMUM_EXPORT_ENTRIES (1024) + +typedef struct +{ + /* what to collect per variant */ + flowprobe_record_t flags; + /** ipfix buffers under construction, per-worker thread */ + vlib_buffer_t **buffers_per_worker; + /** frames containing ipfix buffers, per-worker thread */ + vlib_frame_t **frames_per_worker; + /** next record offset, per worker thread */ + u16 *next_record_offset_per_worker; +} flowprobe_protocol_context_t; + +#define FLOWPROBE_KEY_IN_U32 22 +/* *INDENT-OFF* */ +typedef CLIB_PACKED (union +{ + struct { + u32 rx_sw_if_index; + u32 tx_sw_if_index; + u8 src_mac[6]; + u8 dst_mac[6]; + u16 ethertype; + ip46_address_t src_address; + ip46_address_t dst_address; + u8 protocol; + u16 src_port; + u16 dst_port; + flowprobe_variant_t which; + }; + u32 as_u32[FLOWPROBE_KEY_IN_U32]; +}) flowprobe_key_t; +/* *INDENT-ON* */ + +STATIC_ASSERT (sizeof (flowprobe_key_t) == FLOWPROBE_KEY_IN_U32 * + sizeof (u32), "flowprobe_key_t padding is wrong"); + +typedef struct +{ + flowprobe_key_t key; + u64 packetcount; + u64 octetcount; + f64 last_updated; + f64 last_exported; + u32 passive_timer_handle; +} flowprobe_entry_t; + +/** + * @file + * @brief flow-per-packet plugin header file + */ +typedef struct +{ + /** API message ID base */ + u16 msg_id_base; + + flowprobe_protocol_context_t context[FLOW_N_VARIANTS]; + u16 template_reports[FLOW_N_RECORDS]; + u16 template_size[FLOW_N_RECORDS]; + + /** Time reference pair */ + u64 nanosecond_time_0; + f64 vlib_time_0; + + /** Per CPU flow-state */ + u8 ht_log2len; /* Hash table size is 2^log2len */ + u32 **hash_per_worker; + flowprobe_entry_t **pool_per_worker; + /* *INDENT-OFF* */ + TWT (tw_timer_wheel) ** timers_per_worker; + /* *INDENT-ON* */ + u32 **expired_passive_per_worker; + + flowprobe_record_t record; + u32 active_timer; + u32 passive_timer; + flowprobe_entry_t *stateless_entry; + + bool initialized; + bool disabled; + + u16 template_per_flow[FLOW_N_VARIANTS]; + u8 *flow_per_interface; + + /** convenience vlib_main_t pointer */ + vlib_main_t *vlib_main; + /** convenience vnet_main_t pointer */ + vnet_main_t *vnet_main; +} flowprobe_main_t; + +extern flowprobe_main_t flowprobe_main; + +void flowprobe_flush_callback_ip4 (void); +void flowprobe_flush_callback_ip6 (void); +void flowprobe_flush_callback_l2 (void); +u8 *format_flowprobe_entry (u8 * s, va_list * args); + +#endif + +/* + * fd.io coding-style-patch-verification: ON + * + * Local Variables: + * eval: (c-set-style "gnu") + * End: + */ -- cgit 1.2.3-korg From ed92925f4d5535d7dd3e6de058ae90af209d5a8f Mon Sep 17 00:00:00 2001 From: Ole Troan Date: Tue, 13 Jun 2017 21:15:40 +0200 Subject: FLOWPROBE: Add flowstartns, flowendns and tcpcontrolbits - fixed problem with tcp_flag - changed flowtimestamp into NTP format Change-Id: I4ef05d6c69c5c078a0c80d59c5ccb0c85b924ba6 Signed-off-by: Ole Troan --- src/plugins/flowprobe/flowprobe.c | 19 ++++++- src/plugins/flowprobe/flowprobe.h | 15 ++++++ src/plugins/flowprobe/node.c | 68 +++++++++++++++++++----- src/vppinfra/time.h | 14 +++++ test/test_flowprobe.py | 106 ++++++++++++++++++++++++++++++++++---- 5 files changed, 198 insertions(+), 24 deletions(-) (limited to 'src/plugins/flowprobe/flowprobe.h') diff --git a/src/plugins/flowprobe/flowprobe.c b/src/plugins/flowprobe/flowprobe.c index 8975f89c..884b5a2e 100644 --- a/src/plugins/flowprobe/flowprobe.c +++ b/src/plugins/flowprobe/flowprobe.c @@ -164,7 +164,7 @@ flowprobe_template_l2_fields (ipfix_field_specifier_t * f) static inline ipfix_field_specifier_t * flowprobe_template_common_fields (ipfix_field_specifier_t * f) { -#define flowprobe_template_common_field_count() 3 +#define flowprobe_template_common_field_count() 5 /* ingressInterface, TLV type 10, u32 */ f->e_id_length = ipfix_e_id_length (0 /* enterprise */ , ingressInterface, 4); @@ -180,13 +180,23 @@ flowprobe_template_common_fields (ipfix_field_specifier_t * f) packetDeltaCount, 8); f++; + /* flowStartNanoseconds, TLV type 156, u64 */ + f->e_id_length = ipfix_e_id_length (0 /* enterprise */ , + flowStartNanoseconds, 8); + f++; + + /* flowEndNanoseconds, TLV type 157, u64 */ + f->e_id_length = ipfix_e_id_length (0 /* enterprise */ , + flowEndNanoseconds, 8); + f++; + return f; } static inline ipfix_field_specifier_t * flowprobe_template_l4_fields (ipfix_field_specifier_t * f) { -#define flowprobe_template_l4_field_count() 2 +#define flowprobe_template_l4_field_count() 3 /* sourceTransportPort, TLV type 7, u16 */ f->e_id_length = ipfix_e_id_length (0 /* enterprise */ , sourceTransportPort, 2); @@ -195,6 +205,11 @@ flowprobe_template_l4_fields (ipfix_field_specifier_t * f) f->e_id_length = ipfix_e_id_length (0 /* enterprise */ , destinationTransportPort, 2); f++; + /* tcpControlBits, TLV type 6, u16 */ + f->e_id_length = ipfix_e_id_length (0 /* enterprise */ , + tcpControlBits, 2); + f++; + return f; } diff --git a/src/plugins/flowprobe/flowprobe.h b/src/plugins/flowprobe/flowprobe.h index 196c92a7..760e9241 100644 --- a/src/plugins/flowprobe/flowprobe.h +++ b/src/plugins/flowprobe/flowprobe.h @@ -96,14 +96,29 @@ typedef CLIB_PACKED (union STATIC_ASSERT (sizeof (flowprobe_key_t) == FLOWPROBE_KEY_IN_U32 * sizeof (u32), "flowprobe_key_t padding is wrong"); +typedef struct +{ + u32 sec; + u32 nsec; +} timestamp_nsec_t; + typedef struct { flowprobe_key_t key; u64 packetcount; u64 octetcount; + timestamp_nsec_t flow_start; + timestamp_nsec_t flow_end; f64 last_updated; f64 last_exported; u32 passive_timer_handle; + union + { + struct + { + u16 flags; + } tcp; + } prot; } flowprobe_entry_t; /** diff --git a/src/plugins/flowprobe/node.c b/src/plugins/flowprobe/node.c index 80bfa9b5..604a0a0e 100644 --- a/src/plugins/flowprobe/node.c +++ b/src/plugins/flowprobe/node.c @@ -21,6 +21,7 @@ #include #include #include +#include static void flowprobe_export_entry (vlib_main_t * vm, flowprobe_entry_t * e); @@ -144,6 +145,11 @@ flowprobe_get_variant (flowprobe_variant_t which, return which; } +/* + * NTP rfc868 : 2 208 988 800 corresponds to 00:00 1 Jan 1970 GMT + */ +#define NTP_TIMESTAMP 2208988800L + static inline u32 flowprobe_common_add (vlib_buffer_t * to_b, flowprobe_entry_t * e, u16 offset) { @@ -164,6 +170,22 @@ flowprobe_common_add (vlib_buffer_t * to_b, flowprobe_entry_t * e, u16 offset) clib_memcpy (to_b->data + offset, &packetdelta, sizeof (u64)); offset += sizeof (u64); + /* flowStartNanoseconds */ + u32 t = clib_host_to_net_u32 (e->flow_start.sec + NTP_TIMESTAMP); + clib_memcpy (to_b->data + offset, &t, sizeof (u32)); + offset += sizeof (u32); + t = clib_host_to_net_u32 (e->flow_start.nsec); + clib_memcpy (to_b->data + offset, &t, sizeof (u32)); + offset += sizeof (u32); + + /* flowEndNanoseconds */ + t = clib_host_to_net_u32 (e->flow_end.sec + NTP_TIMESTAMP); + clib_memcpy (to_b->data + offset, &t, sizeof (u32)); + offset += sizeof (u32); + t = clib_host_to_net_u32 (e->flow_end.nsec); + clib_memcpy (to_b->data + offset, &t, sizeof (u32)); + offset += sizeof (u32); + return offset - start; } @@ -252,6 +274,11 @@ flowprobe_l4_add (vlib_buffer_t * to_b, flowprobe_entry_t * e, u16 offset) clib_memcpy (to_b->data + offset, &e->key.dst_port, 2); offset += 2; + /* tcp control bits */ + u16 control_bits = htons (e->prot.tcp.flags); + clib_memcpy (to_b->data + offset, &control_bits, 2); + offset += 2; + return offset - start; } @@ -328,7 +355,7 @@ flowprobe_create (u32 my_cpu_number, flowprobe_key_t * k, u32 * poolindex) static inline void add_to_flow_record_state (vlib_main_t * vm, vlib_node_runtime_t * node, flowprobe_main_t * fm, vlib_buffer_t * b, - u64 timestamp, u16 length, + timestamp_nsec_t timestamp, u16 length, flowprobe_variant_t which, flowprobe_trace_t * t) { if (fm->disabled) @@ -348,6 +375,8 @@ add_to_flow_record_state (vlib_main_t * vm, vlib_node_runtime_t * node, ip4_header_t *ip4 = 0; ip6_header_t *ip6 = 0; udp_header_t *udp = 0; + tcp_header_t *tcp = 0; + u8 tcp_flags = 0; if (flags & FLOW_RECORD_L3 || flags & FLOW_RECORD_L4) { @@ -369,7 +398,6 @@ add_to_flow_record_state (vlib_main_t * vm, vlib_node_runtime_t * node, if (collect_ip6 && ethertype == ETHERNET_TYPE_IP6) { ip6 = (ip6_header_t *) (eth + 1); - udp = (udp_header_t *) (ip6 + 1); if (flags & FLOW_RECORD_L3) { k.src_address.as_u64[0] = ip6->src_address.as_u64[0]; @@ -378,27 +406,42 @@ add_to_flow_record_state (vlib_main_t * vm, vlib_node_runtime_t * node, k.dst_address.as_u64[1] = ip6->dst_address.as_u64[1]; } k.protocol = ip6->protocol; + if (k.protocol == IP_PROTOCOL_UDP) + udp = (udp_header_t *) (ip6 + 1); + else if (k.protocol == IP_PROTOCOL_TCP) + tcp = (tcp_header_t *) (ip6 + 1); + octets = clib_net_to_host_u16 (ip6->payload_length) + sizeof (ip6_header_t); } if (collect_ip4 && ethertype == ETHERNET_TYPE_IP4) { ip4 = (ip4_header_t *) (eth + 1); - udp = (udp_header_t *) (ip4 + 1); if (flags & FLOW_RECORD_L3) { k.src_address.ip4.as_u32 = ip4->src_address.as_u32; k.dst_address.ip4.as_u32 = ip4->dst_address.as_u32; } k.protocol = ip4->protocol; + if ((flags & FLOW_RECORD_L4) && k.protocol == IP_PROTOCOL_UDP) + udp = (udp_header_t *) (ip4 + 1); + else if ((flags & FLOW_RECORD_L4) && k.protocol == IP_PROTOCOL_TCP) + tcp = (tcp_header_t *) (ip4 + 1); + octets = clib_net_to_host_u16 (ip4->length); } - if ((flags & FLOW_RECORD_L4) && udp && - (k.protocol == IP_PROTOCOL_TCP || k.protocol == IP_PROTOCOL_UDP)) + + if (udp) { k.src_port = udp->src_port; k.dst_port = udp->dst_port; } + else if (tcp) + { + k.src_port = tcp->src_port; + k.dst_port = tcp->dst_port; + tcp_flags = tcp->flags; + } if (t) { @@ -436,6 +479,7 @@ add_to_flow_record_state (vlib_main_t * vm, vlib_node_runtime_t * node, { e = flowprobe_create (my_cpu_number, &k, &poolindex); e->last_exported = now; + e->flow_start = timestamp; } } else @@ -450,7 +494,8 @@ add_to_flow_record_state (vlib_main_t * vm, vlib_node_runtime_t * node, e->packetcount++; e->octetcount += octets; e->last_updated = now; - + e->flow_end = timestamp; + e->prot.tcp.flags |= tcp_flags; if (fm->active_timer == 0 || (now > e->last_exported + fm->active_timer)) flowprobe_export_entry (vm, e); @@ -677,10 +722,9 @@ flowprobe_node_fn (vlib_main_t * vm, u32 n_left_from, *from, *to_next; flowprobe_next_t next_index; flowprobe_main_t *fm = &flowprobe_main; - u64 now; + timestamp_nsec_t timestamp; - now = (u64) ((vlib_time_now (vm) - fm->vlib_time_0) * 1e9); - now += fm->nanosecond_time_0; + unix_time_now_nsec_fraction (×tamp.sec, ×tamp.nsec); from = vlib_frame_vector_args (frame); n_left_from = frame->n_vectors; @@ -735,7 +779,7 @@ flowprobe_node_fn (vlib_main_t * vm, u16 ethertype0 = clib_net_to_host_u16 (eh0->type); if (PREDICT_TRUE ((b0->flags & VLIB_BUFFER_FLOW_REPORT) == 0)) - add_to_flow_record_state (vm, node, fm, b0, now, len0, + add_to_flow_record_state (vm, node, fm, b0, timestamp, len0, flowprobe_get_variant (which, fm->context[which].flags, ethertype0), 0); @@ -745,7 +789,7 @@ flowprobe_node_fn (vlib_main_t * vm, u16 ethertype1 = clib_net_to_host_u16 (eh1->type); if (PREDICT_TRUE ((b1->flags & VLIB_BUFFER_FLOW_REPORT) == 0)) - add_to_flow_record_state (vm, node, fm, b1, now, len1, + add_to_flow_record_state (vm, node, fm, b1, timestamp, len1, flowprobe_get_variant (which, fm->context[which].flags, ethertype1), 0); @@ -787,7 +831,7 @@ flowprobe_node_fn (vlib_main_t * vm, && (b0->flags & VLIB_BUFFER_IS_TRACED))) t = vlib_add_trace (vm, node, b0, sizeof (*t)); - add_to_flow_record_state (vm, node, fm, b0, now, len0, + add_to_flow_record_state (vm, node, fm, b0, timestamp, len0, flowprobe_get_variant (which, fm->context[which].flags, ethertype0), t); diff --git a/src/vppinfra/time.h b/src/vppinfra/time.h index 3b89cf78..3fdc7d43 100644 --- a/src/vppinfra/time.h +++ b/src/vppinfra/time.h @@ -242,6 +242,15 @@ unix_time_now_nsec (void) return 1e9 * ts.tv_sec + ts.tv_nsec; } +always_inline void +unix_time_now_nsec_fraction (u32 * sec, u32 * nsec) +{ + struct timespec ts; + syscall (SYS_clock_gettime, CLOCK_REALTIME, &ts); + *sec = ts.tv_sec; + *nsec = ts.tv_nsec; +} + always_inline f64 unix_usage_now (void) { @@ -274,6 +283,11 @@ unix_time_now_nsec (void) return 0; } +always_inline void +unix_time_now_nsec_fraction (u32 * sec, u32 * nsec) +{ +} + always_inline f64 unix_usage_now (void) { diff --git a/test/test_flowprobe.py b/test/test_flowprobe.py index f7dde175..df6b4230 100644 --- a/test/test_flowprobe.py +++ b/test/test_flowprobe.py @@ -3,10 +3,11 @@ import random import socket import unittest import time +import re from scapy.packet import Raw from scapy.layers.l2 import Ether -from scapy.layers.inet import IP, UDP +from scapy.layers.inet import IP, TCP, UDP from scapy.layers.inet6 import IPv6 from framework import VppTestCase, VppTestRunner, running_extended_tests @@ -14,6 +15,7 @@ from vpp_object import VppObject from vpp_pg_interface import CaptureTimeoutError from util import ppp from ipfix import IPFIX, Set, Template, Data, IPFIXDecoder +from vpp_ip_route import VppIpRoute, VppRoutePath class VppCFLOW(VppObject): @@ -113,7 +115,7 @@ class MethodHolder(VppTestCase): super(MethodHolder, cls).setUpClass() try: # Create pg interfaces - cls.create_pg_interfaces(range(7)) + cls.create_pg_interfaces(range(9)) # Packet sizes cls.pg_if_packet_sizes = [64, 512, 1518, 9018] @@ -140,6 +142,9 @@ class MethodHolder(VppTestCase): cls.pg3.resolve_arp() cls.pg4.config_ip4() cls.pg4.resolve_arp() + cls.pg7.config_ip4() + cls.pg8.config_ip4() + cls.pg8.configure_ipv4_neighbors() cls.pg5.config_ip6() cls.pg5.resolve_ndp() @@ -176,7 +181,8 @@ class MethodHolder(VppTestCase): p /= IP(src=src_if.remote_ip4, dst=dst_if.remote_ip4) else: p /= IPv6(src=src_if.remote_ip6, dst=dst_if.remote_ip6) - p /= (UDP(sport=1234, dport=4321) / Raw(payload)) + p /= UDP(sport=1234, dport=4321) + p /= Raw(payload) info.data = p.copy() self.extend_packet(p, pkt_size) self.pkts.append(p) @@ -311,12 +317,12 @@ class MethodHolder(VppTestCase): return p -class Timers(MethodHolder): +class Flowprobe(MethodHolder): """Template verification, timer tests""" def test_0001(self): """ timer less than template timeout""" - self.logger.info("FFP_TEST_START_0002") + self.logger.info("FFP_TEST_START_0001") self.pg_enable_capture(self.pg_interfaces) self.pkts = [] @@ -327,20 +333,20 @@ class Timers(MethodHolder): # template packet should arrive immediately templates = ipfix.verify_templates(ipfix_decoder) - self.create_stream(packets=2) + self.create_stream(packets=1) self.send_packets() - capture = self.pg2.get_capture(2) + capture = self.pg2.get_capture(1) # make sure the one packet we expect actually showed up cflow = self.wait_for_cflow_packet(self.collector, templates[1], 15) self.verify_cflow_data(ipfix_decoder, capture, cflow) ipfix.remove_vpp_config() - self.logger.info("FFP_TEST_FINISH_0002") + self.logger.info("FFP_TEST_FINISH_0001") def test_0002(self): """ timer greater than template timeout""" - self.logger.info("FFP_TEST_START_0003") + self.logger.info("FFP_TEST_START_0002") self.pg_enable_capture(self.pg_interfaces) self.pkts = [] @@ -364,7 +370,87 @@ class Timers(MethodHolder): self.verify_cflow_data(ipfix_decoder, capture, cflow) ipfix.remove_vpp_config() - self.logger.info("FFP_TEST_FINISH_0003") + self.logger.info("FFP_TEST_FINISH_0002") + + def test_cflow_packet(self): + """verify cflow packet fields""" + self.logger.info("FFP_TEST_START_0000") + self.pg_enable_capture(self.pg_interfaces) + self.pkts = [] + + ipfix = VppCFLOW(test=self, intf='pg8', datapath="ip4", + layer='l2 l3 l4', active=2) + ipfix.add_vpp_config() + + route_9001 = VppIpRoute(self, "9.0.0.0", 24, + [VppRoutePath(self.pg8._remote_hosts[0].ip4, + self.pg8.sw_if_index)]) + route_9001.add_vpp_config() + + ipfix_decoder = IPFIXDecoder() + templates = ipfix.verify_templates(ipfix_decoder, count=1) + + self.pkts = [(Ether(dst=self.pg7.local_mac, + src=self.pg7.remote_mac) / + IP(src=self.pg7.remote_ip4, dst="9.0.0.100") / + TCP(sport=1234, dport=4321, flags=80) / + Raw('\xa5' * 100))] + + nowUTC = int(time.time()) + nowUNIX = nowUTC+2208988800 + self.send_packets(src_if=self.pg7, dst_if=self.pg8) + + cflow = self.wait_for_cflow_packet(self.collector, templates[0], 10) + self.collector.get_capture(2) + + if cflow[0].haslayer(IPFIX): + self.assertEqual(cflow[IPFIX].version, 10) + self.assertEqual(cflow[IPFIX].observationDomainID, 1) + self.assertEqual(cflow[IPFIX].sequenceNumber, 0) + self.assertAlmostEqual(cflow[IPFIX].exportTime, nowUTC, delta=5) + if cflow.haslayer(Data): + record = ipfix_decoder.decode_data_set(cflow[0].getlayer(Set))[0] + # ingress interface + self.assertEqual(int(record[10].encode('hex'), 16), 8) + # egress interface + self.assertEqual(int(record[14].encode('hex'), 16), 9) + # packets + self.assertEqual(int(record[2].encode('hex'), 16), 1) + # src mac + self.assertEqual(':'.join(re.findall('..', record[56].encode( + 'hex'))), self.pg8.local_mac) + # dst mac + self.assertEqual(':'.join(re.findall('..', record[80].encode( + 'hex'))), self.pg8.remote_mac) + flowTimestamp = int(record[156].encode('hex'), 16) >> 32 + # flow start timestamp + self.assertAlmostEqual(flowTimestamp, nowUNIX, delta=1) + flowTimestamp = int(record[157].encode('hex'), 16) >> 32 + # flow end timestamp + self.assertAlmostEqual(flowTimestamp, nowUNIX, delta=1) + # ethernet type + self.assertEqual(int(record[256].encode('hex'), 16), 8) + # src ip + self.assertEqual('.'.join(re.findall('..', record[8].encode( + 'hex'))), + '.'.join('{:02x}'.format(int(n)) for n in + self.pg7.remote_ip4.split('.'))) + # dst ip + self.assertEqual('.'.join(re.findall('..', record[12].encode( + 'hex'))), + '.'.join('{:02x}'.format(int(n)) for n in + "9.0.0.100".split('.'))) + # protocol (TCP) + self.assertEqual(int(record[4].encode('hex'), 16), 6) + # src port + self.assertEqual(int(record[7].encode('hex'), 16), 1234) + # dst port + self.assertEqual(int(record[11].encode('hex'), 16), 4321) + # tcp flags + self.assertEqual(int(record[6].encode('hex'), 16), 80) + + ipfix.remove_vpp_config() + self.logger.info("FFP_TEST_FINISH_0000") class Datapath(MethodHolder): -- cgit 1.2.3-korg From 50b60ebcc306b162a5d7b0916830638a2e99d378 Mon Sep 17 00:00:00 2001 From: Pierre Pfister Date: Wed, 9 Aug 2017 10:42:06 +0200 Subject: Flowprobe: Fix flow start time and hash computation Upon hash collision, the flow start time was not reset. The hash computation techniques (crc32 or xxhash) also both had bugs which are now fixed. Change-Id: I94d72997f34018d1699324264f7dded2a5cbd776 Signed-off-by: Pierre Pfister --- src/plugins/flowprobe/flowprobe.h | 34 +++++++++++++--------------------- src/plugins/flowprobe/node.c | 12 ++++++++---- 2 files changed, 21 insertions(+), 25 deletions(-) (limited to 'src/plugins/flowprobe/flowprobe.h') diff --git a/src/plugins/flowprobe/flowprobe.h b/src/plugins/flowprobe/flowprobe.h index 760e9241..02ee053c 100644 --- a/src/plugins/flowprobe/flowprobe.h +++ b/src/plugins/flowprobe/flowprobe.h @@ -72,30 +72,22 @@ typedef struct u16 *next_record_offset_per_worker; } flowprobe_protocol_context_t; -#define FLOWPROBE_KEY_IN_U32 22 /* *INDENT-OFF* */ -typedef CLIB_PACKED (union -{ - struct { - u32 rx_sw_if_index; - u32 tx_sw_if_index; - u8 src_mac[6]; - u8 dst_mac[6]; - u16 ethertype; - ip46_address_t src_address; - ip46_address_t dst_address; - u8 protocol; - u16 src_port; - u16 dst_port; - flowprobe_variant_t which; - }; - u32 as_u32[FLOWPROBE_KEY_IN_U32]; -}) flowprobe_key_t; +typedef struct __attribute__ ((aligned (8))) { + u32 rx_sw_if_index; + u32 tx_sw_if_index; + u8 src_mac[6]; + u8 dst_mac[6]; + u16 ethertype; + ip46_address_t src_address; + ip46_address_t dst_address; + u8 protocol; + u16 src_port; + u16 dst_port; + flowprobe_variant_t which; +} flowprobe_key_t; /* *INDENT-ON* */ -STATIC_ASSERT (sizeof (flowprobe_key_t) == FLOWPROBE_KEY_IN_U32 * - sizeof (u32), "flowprobe_key_t padding is wrong"); - typedef struct { u32 sec; diff --git a/src/plugins/flowprobe/node.c b/src/plugins/flowprobe/node.c index c4610a77..2f7d0025 100644 --- a/src/plugins/flowprobe/node.c +++ b/src/plugins/flowprobe/node.c @@ -289,10 +289,13 @@ flowprobe_hash (flowprobe_key_t * k) u32 h = 0; #ifdef clib_crc32c_uses_intrinsics - h = clib_crc32c ((u8 *) k->as_u32, FLOWPROBE_KEY_IN_U32); + h = clib_crc32c ((u8 *) k, sizeof (*k)); #else - u64 tmp = - k->as_u32[0] ^ k->as_u32[1] ^ k->as_u32[2] ^ k->as_u32[3] ^ k->as_u32[4]; + int i; + u64 tmp = 0; + for (i = 0; i < sizeof (*k) / 8; i++) + tmp ^= ((u64 *) k)[i]; + h = clib_xxhash (tmp); #endif @@ -370,7 +373,7 @@ add_to_flow_record_state (vlib_main_t * vm, vlib_node_runtime_t * node, ethernet_header_t *eth = vlib_buffer_get_current (b); u16 ethertype = clib_net_to_host_u16 (eth->type); /* *INDENT-OFF* */ - flowprobe_key_t k = { {0} }; + flowprobe_key_t k = {}; /* *INDENT-ON* */ ip4_header_t *ip4 = 0; ip6_header_t *ip6 = 0; @@ -472,6 +475,7 @@ add_to_flow_record_state (vlib_main_t * vm, vlib_node_runtime_t * node, if (e->packetcount) flowprobe_export_entry (vm, e); e->key = k; + e->flow_start = timestamp; vlib_node_increment_counter (vm, node->node_index, FLOWPROBE_ERROR_COLLISION, 1); } -- cgit 1.2.3-korg