refactor: v2 (#150)

This commit is contained in:
Louis
2023-08-09 19:47:20 -07:00
committed by GitHub
parent 1298b9408d
commit ae56e41786
75 changed files with 4905 additions and 4828 deletions

View File

@@ -30,17 +30,20 @@ Minimal changes in the decoding libraries.
## Modularity
In order to enable load-balancing and optimizations, the GoFlow library has a `decoder` which converts
the payload of a flow packet into a Go structure.
In order to enable load-balancing and optimizations, the GoFlow2 library has a `decoder` which converts
the payload of a flow packet into a structure.
The `producer` functions (one per protocol) then converts those structures into a protobuf (`pb/flow.pb`)
which contains the fields a network engineer is interested in.
The flow packets usually contains multiples samples
This acts as an abstraction of a sample.
The `producer` converts the samples into another format.
Out of the box, this repository provides a protobuf producer (`pb/flow.pb`)
and a raw producer.
In the case of the protobuf producer, the records in a single flow packet
are extracted and made in their own protobuf. Custom mapping allows
to add new fields without rebuilding the proto.
The `format` directory offers various utilities to process the protobuf. It can convert
The `format` directory offers various utilities to format a message. It calls specific
functions to marshal as JSON or text for instance.
The `transport` provides different way of processing the protobuf. Either sending it via Kafka or
The `transport` provides different way of processing the message. Either sending it via Kafka or
send it to a file (or stdout).
GoFlow2 is a wrapper of all the functions and chains thems.
@@ -103,55 +106,47 @@ By default, the samples received will be printed in JSON format on the stdout.
```json
{
"Type": "SFLOW_5",
"TimeFlowEnd": 1621820000,
"TimeFlowStart": 1621820000,
"TimeReceived": 1621820000,
"Bytes": 70,
"Packets": 1,
"SamplingRate": 100,
"SamplerAddress": "192.168.1.254",
"DstAddr": "10.0.0.1",
"DstMac": "ff:ff:ff:ff:ff:ff",
"SrcAddr": "192.168.1.1",
"SrcMac": "ff:ff:ff:ff:ff:ff",
"InIf": 1,
"OutIf": 2,
"Etype": 2048,
"EtypeName": "IPv4",
"Proto": 6,
"ProtoName": "TCP",
"SrcPort": 443,
"DstPort": 46344,
"FragmentId": 54044,
"FragmentOffset": 16384,
...
"IPTTL": 64,
"IPTos": 0,
"TCPFlags": 16,
"type": "SFLOW_5",
"time_received_ns": 1681583295157626000,
"sequence_num": 2999,
"sampling_rate": 100,
"sampler_address": "192.168.0.1",
"time_flow_start_ns": 1681583295157626000,
"time_flow_end_ns": 1681583295157626000,
"bytes": 1500,
"packets": 1,
"src_addr": "fd01::1",
"dst_addr": "fd01::2",
"etype": "IPv6",
"proto": "TCP",
"src_port": 443,
"dst_port": 50001
}
```
If you are using a log integration (e.g: Loki with Promtail, Splunk, Fluentd, Google Cloud Logs, etc.),
just send the output into a file.
```bash
$ ./goflow2 -transport.file /var/logs/goflow2.log
```
To enable Kafka and send protobuf, use the following arguments:
```bash
$ ./goflow2 -transport=kafka -transport.kafka.brokers=localhost:9092 -transport.kafka.topic=flows -format=pb
$ ./goflow2 -transport=kafka \
-transport.kafka.brokers=localhost:9092 \
-transport.kafka.topic=flows \
-format=bin
```
By default, the distribution will be randomized.
To partition the feed (any field of the protobuf is available), the following options can be used:
```
-transport.kafka.hashing=true \
-format.hash=SamplerAddress,DstAS
```
In order to partition the field, you need to configure the `key`
in the formatter.
By default, compression is disabled when sending data to Kafka.
To change the kafka compression type of the producer side configure the following option:
```
-transport.kafka.compression.type=gzip
```
@@ -189,9 +184,9 @@ in the InIf protobuf field without changing the code.
ipfix:
mapping:
- field: 252
destination: InIf
destination: in_if
- field: 253
destination: OutIf
destination: out_if
```
### Output format considerations
@@ -218,7 +213,8 @@ with a database for Autonomous System Number and Country.
Similar output options as GoFlow are provided.
```bash
$ ./goflow2 -transport.file.sep= -format=pb -format.protobuf.fixedlen=true | ./enricher -db.asn path-to/GeoLite2-ASN.mmdb -db.country path-to/GeoLite2-Country.mmdb
$ ./goflow2 -transport.file.sep= -format=bin | \
./enricher -db.asn path-to/GeoLite2-ASN.mmdb -db.country path-to/GeoLite2-Country.mmdb
```
For a more scalable production setting, Kafka and protobuf are recommended.
@@ -226,14 +222,19 @@ Stream operations (aggregation and filtering) can be done with stream-processor
For instance Flink, or the more recent Kafka Streams and kSQLdb.
Direct storage can be done with data-warehouses like Clickhouse.
In some cases, the consumer will require protobuf messages to be prefixed by
length. To do this, use the flag `-format.protobuf.fixedlen=true`.
Each protobuf message is prefixed by its varint length.
This repository contains [examples of pipelines](./compose) with docker-compose.
The available pipelines are:
* [Kafka+Clickhouse+Grafana](./compose/kcg)
* [Logstash+Elastic+Kibana](./compose/elk)
## Security notes and assumptions
By default, the buffer for UDP is 9000 bytes.
Protections were added to avoid DOS on sFlow since the various length fields are 32 bits.
There are assumptions on how many records and list items a sample can have (eg: AS-Path).
## User stories
Are you using GoFlow2 in production at scale? Add yourself here!

View File

@@ -2,35 +2,30 @@ package main
import (
"bufio"
"bytes"
"context"
"encoding/binary"
"errors"
"flag"
"fmt"
"io"
"net"
"net/http"
"os"
"strings"
"github.com/oschwald/geoip2-golang"
"github.com/golang/protobuf/proto"
flowmessage "github.com/netsampler/goflow2/cmd/enricher/pb"
flowmessage "github.com/netsampler/goflow2/v2/cmd/enricher/pb"
// import various formatters
"github.com/netsampler/goflow2/format"
_ "github.com/netsampler/goflow2/format/json"
_ "github.com/netsampler/goflow2/format/protobuf"
_ "github.com/netsampler/goflow2/format/text"
"github.com/netsampler/goflow2/v2/format"
_ "github.com/netsampler/goflow2/v2/format/binary"
_ "github.com/netsampler/goflow2/v2/format/json"
_ "github.com/netsampler/goflow2/v2/format/text"
// import various transports
"github.com/netsampler/goflow2/transport"
_ "github.com/netsampler/goflow2/transport/file"
_ "github.com/netsampler/goflow2/transport/kafka"
"github.com/netsampler/goflow2/v2/transport"
_ "github.com/netsampler/goflow2/v2/transport/file"
_ "github.com/netsampler/goflow2/v2/transport/kafka"
"github.com/prometheus/client_golang/prometheus/promhttp"
"github.com/oschwald/geoip2-golang"
log "github.com/sirupsen/logrus"
"google.golang.org/protobuf/encoding/protodelim"
)
var (
@@ -49,19 +44,9 @@ var (
Format = flag.String("format", "json", fmt.Sprintf("Choose the format (available: %s)", strings.Join(format.GetFormats(), ", ")))
Transport = flag.String("transport", "file", fmt.Sprintf("Choose the transport (available: %s)", strings.Join(transport.GetTransports(), ", ")))
MetricsAddr = flag.String("metrics.addr", ":8081", "Metrics address")
MetricsPath = flag.String("metrics.path", "/metrics", "Metrics path")
TemplatePath = flag.String("templates.path", "/templates", "NetFlow/IPFIX templates list")
Version = flag.Bool("v", false, "Print version")
)
func httpServer() {
http.Handle(*MetricsPath, promhttp.Handler())
log.Fatal(http.ListenAndServe(*MetricsAddr, nil))
}
func MapAsn(db *geoip2.Reader, addr []byte, dest *uint32) {
entry, err := db.ASN(net.IP(addr))
if err != nil {
@@ -117,61 +102,31 @@ func main() {
defer dbCountry.Close()
}
ctx := context.Background()
formatter, err := format.FindFormat(ctx, *Format)
formatter, err := format.FindFormat(*Format)
if err != nil {
log.Fatal(err)
}
transporter, err := transport.FindTransport(ctx, *Transport)
transporter, err := transport.FindTransport(*Transport)
if err != nil {
log.Fatal(err)
}
defer transporter.Close(ctx)
defer transporter.Close()
switch *LogFmt {
case "json":
log.SetFormatter(&log.JSONFormatter{})
}
log.Info("Starting enricher")
go httpServer()
log.Info("starting enricher")
rdr := bufio.NewReader(os.Stdin)
msg := &flowmessage.FlowMessageExt{}
lenBufSize := binary.MaxVarintLen64
for {
msgLen, err := rdr.Peek(lenBufSize)
if err != nil && err != io.EOF {
log.Error(err)
continue
}
l, vn := proto.DecodeVarint(msgLen)
if l == 0 {
continue
}
_, err = rdr.Discard(vn)
if err != nil {
log.Error(err)
continue
}
line := make([]byte, l)
_, err = io.ReadFull(rdr, line)
if err != nil && err != io.EOF {
log.Error(err)
continue
}
line = bytes.TrimSuffix(line, []byte("\n"))
err = proto.Unmarshal(line, msg)
if err != nil {
if err := protodelim.UnmarshalFrom(rdr, msg); err != nil && errors.Is(err, io.EOF) {
return
} else if err != nil {
log.Error(err)
continue
}

View File

@@ -1,7 +1,7 @@
// Code generated by protoc-gen-go. DO NOT EDIT.
// versions:
// protoc-gen-go v1.26.0
// protoc v3.21.4
// protoc v3.21.12
// source: cmd/enricher/pb/flowext.proto
package flowpb

View File

@@ -2,34 +2,47 @@ package main
import (
"context"
"errors"
"flag"
"fmt"
"io"
"net"
"net/http"
"net/url"
"os"
"os/signal"
"strconv"
"strings"
"sync"
"syscall"
"time"
// import various formatters
"github.com/netsampler/goflow2/format"
_ "github.com/netsampler/goflow2/format/json"
_ "github.com/netsampler/goflow2/format/protobuf"
_ "github.com/netsampler/goflow2/format/text"
// decoders
"github.com/netsampler/goflow2/v2/decoders/netflow"
// import various transports
"github.com/netsampler/goflow2/transport"
_ "github.com/netsampler/goflow2/transport/file"
_ "github.com/netsampler/goflow2/transport/kafka"
// various formatters
"github.com/netsampler/goflow2/v2/format"
_ "github.com/netsampler/goflow2/v2/format/binary"
_ "github.com/netsampler/goflow2/v2/format/json"
_ "github.com/netsampler/goflow2/v2/format/text"
// import various NetFlow/IPFIX templates
"github.com/netsampler/goflow2/decoders/netflow/templates"
_ "github.com/netsampler/goflow2/decoders/netflow/templates/file"
_ "github.com/netsampler/goflow2/decoders/netflow/templates/memory"
// various transports
"github.com/netsampler/goflow2/v2/transport"
_ "github.com/netsampler/goflow2/v2/transport/file"
_ "github.com/netsampler/goflow2/v2/transport/kafka"
// various producers
"github.com/netsampler/goflow2/v2/producer"
protoproducer "github.com/netsampler/goflow2/v2/producer/proto"
rawproducer "github.com/netsampler/goflow2/v2/producer/raw"
// core libraries
"github.com/netsampler/goflow2/v2/metrics"
"github.com/netsampler/goflow2/v2/utils"
"github.com/netsampler/goflow2/utils"
"github.com/prometheus/client_golang/prometheus/promhttp"
log "github.com/sirupsen/logrus"
"gopkg.in/yaml.v2"
)
var (
@@ -37,20 +50,16 @@ var (
buildinfos = ""
AppVersion = "GoFlow2 " + version + " " + buildinfos
ReusePort = flag.Bool("reuseport", false, "Enable so_reuseport")
ListenAddresses = flag.String("listen", "sflow://:6343,netflow://:2055", "listen addresses")
Workers = flag.Int("workers", 1, "Number of workers per collector")
LogLevel = flag.String("loglevel", "info", "Log level")
LogFmt = flag.String("logfmt", "normal", "Log formatter")
NetFlowTemplates = flag.String("netflow.templates", "memory", fmt.Sprintf("Choose the format (available: %s)", strings.Join(templates.GetTemplates(), ", ")))
Produce = flag.String("produce", "sample", "Producer method (sample or raw)")
Format = flag.String("format", "json", fmt.Sprintf("Choose the format (available: %s)", strings.Join(format.GetFormats(), ", ")))
Transport = flag.String("transport", "file", fmt.Sprintf("Choose the transport (available: %s)", strings.Join(transport.GetTransports(), ", ")))
MetricsAddr = flag.String("metrics.addr", ":8080", "Metrics address")
MetricsPath = flag.String("metrics.path", "/metrics", "Metrics path")
Addr = flag.String("addr", ":8080", "HTTP server address")
TemplatePath = flag.String("templates.path", "/templates", "NetFlow/IPFIX templates list")
@@ -59,10 +68,11 @@ var (
Version = flag.Bool("v", false, "Print version")
)
func httpServer( /*state *utils.StateNetFlow*/ ) {
http.Handle(*MetricsPath, promhttp.Handler())
//http.HandleFunc(*TemplatePath, state.ServeHTTPTemplates)
log.Fatal(http.ListenAndServe(*MetricsAddr, nil))
func LoadMapping(f io.Reader) (*protoproducer.ProducerConfig, error) {
config := &protoproducer.ProducerConfig{}
dec := yaml.NewDecoder(f)
err := dec.Decode(config)
return config, err
}
func main() {
@@ -76,124 +86,242 @@ func main() {
lvl, _ := log.ParseLevel(*LogLevel)
log.SetLevel(lvl)
var config utils.ProducerConfig
if *MappingFile != "" {
f, err := os.Open(*MappingFile)
if err != nil {
log.Fatal(err)
}
config, err = utils.LoadMapping(f)
f.Close()
if err != nil {
log.Fatal(err)
}
}
ctx := context.Background()
formatter, err := format.FindFormat(ctx, *Format)
if err != nil {
log.Fatal(err)
}
transporter, err := transport.FindTransport(ctx, *Transport)
if err != nil {
log.Fatal(err)
}
defer transporter.Close(ctx)
// the following is only useful when parsing NetFlowV9/IPFIX (template-based flow)
templateSystem, err := templates.FindTemplateSystem(ctx, *NetFlowTemplates)
if err != nil {
log.Fatal(err)
}
defer templateSystem.Close(ctx)
switch *LogFmt {
case "json":
log.SetFormatter(&log.JSONFormatter{})
}
log.Info("Starting GoFlow2")
formatter, err := format.FindFormat(*Format)
if err != nil {
log.Fatal(err)
}
go httpServer()
//go httpServer(sNF)
transporter, err := transport.FindTransport(*Transport)
if err != nil {
log.Fatal(err)
}
wg := &sync.WaitGroup{}
for _, listenAddress := range strings.Split(*ListenAddresses, ",") {
wg.Add(1)
go func(listenAddress string) {
defer wg.Done()
listenAddrUrl, err := url.Parse(listenAddress)
var flowProducer producer.ProducerInterface
// instanciate a producer
// unlike transport and format, the producer requires extensive configurations and can be chained
if *Produce == "sample" {
var cfgProducer *protoproducer.ProducerConfig
if *MappingFile != "" {
f, err := os.Open(*MappingFile)
if err != nil {
log.Fatal(err)
}
numSockets := 1
if listenAddrUrl.Query().Has("count") {
if numSocketsTmp, err := strconv.ParseUint(listenAddrUrl.Query().Get("count"), 10, 64); err != nil {
log.Fatal(err)
} else {
numSockets = int(numSocketsTmp)
}
}
if numSockets == 0 {
numSockets = 1
}
hostname := listenAddrUrl.Hostname()
port, err := strconv.ParseUint(listenAddrUrl.Port(), 10, 64)
cfgProducer, err = LoadMapping(f)
f.Close()
if err != nil {
log.Errorf("Port %s could not be converted to integer", listenAddrUrl.Port())
return
log.Fatal(err)
}
}
logFields := log.Fields{
"scheme": listenAddrUrl.Scheme,
"hostname": hostname,
"port": port,
"count": numSockets,
}
log.WithFields(logFields).Info("Starting collection")
for i := 0; i < numSockets; i++ {
if listenAddrUrl.Scheme == "sflow" {
sSFlow := &utils.StateSFlow{
Format: formatter,
Transport: transporter,
Logger: log.StandardLogger(),
Config: config,
}
err = sSFlow.FlowRoutine(*Workers, hostname, int(port), *ReusePort)
} else if listenAddrUrl.Scheme == "netflow" {
sNF := utils.NewStateNetFlow()
sNF.Format = formatter
sNF.Transport = transporter
sNF.Logger = log.StandardLogger()
sNF.Config = config
sNF.TemplateSystem = templateSystem
err = sNF.FlowRoutine(*Workers, hostname, int(port), *ReusePort)
} else if listenAddrUrl.Scheme == "nfl" {
sNFL := &utils.StateNFLegacy{
Format: formatter,
Transport: transporter,
Logger: log.StandardLogger(),
}
err = sNFL.FlowRoutine(*Workers, hostname, int(port), *ReusePort)
} else {
log.Errorf("scheme %s does not exist", listenAddrUrl.Scheme)
return
}
if err != nil {
log.WithFields(logFields).Fatal(err)
}
}
}(listenAddress)
flowProducer, err = protoproducer.CreateProtoProducer(cfgProducer, protoproducer.CreateSamplingSystem)
if err != nil {
log.Fatal(err)
}
} else if *Produce == "raw" {
flowProducer = &rawproducer.RawProducer{}
} else {
log.Fatalf("producer %s does not exist", *Produce)
}
// wrap producer with Prometheus metrics
flowProducer = metrics.WrapPromProducer(flowProducer)
wg := &sync.WaitGroup{}
var collecting bool
http.Handle("/metrics", promhttp.Handler())
http.HandleFunc("/__health", func(wr http.ResponseWriter, r *http.Request) {
if !collecting {
wr.WriteHeader(http.StatusServiceUnavailable)
if _, err := wr.Write([]byte("Not OK\n")); err != nil {
log.WithError(err).Error("error writing HTTP")
}
} else {
wr.WriteHeader(http.StatusOK)
if _, err := wr.Write([]byte("OK\n")); err != nil {
log.WithError(err).Error("error writing HTTP")
}
}
})
srv := http.Server{
Addr: *Addr,
ReadHeaderTimeout: time.Second * 5,
}
if *Addr != "" {
wg.Add(1)
go func() {
defer wg.Done()
l := log.WithFields(log.Fields{
"http": *Addr,
})
err := srv.ListenAndServe()
if err != nil && !errors.Is(err, http.ErrServerClosed) {
l.WithError(err).Fatal("HTTP server error")
}
l.Info("closed HTTP server")
}()
}
log.Info("starting GoFlow2")
c := make(chan os.Signal, 1)
signal.Notify(c, os.Interrupt, syscall.SIGTERM)
var receivers []*utils.UDPReceiver
var pipes []utils.FlowPipe
q := make(chan bool)
for _, listenAddress := range strings.Split(*ListenAddresses, ",") {
listenAddrUrl, err := url.Parse(listenAddress)
if err != nil {
log.Fatal(err)
}
numSockets := 1
if listenAddrUrl.Query().Has("count") {
if numSocketsTmp, err := strconv.ParseUint(listenAddrUrl.Query().Get("count"), 10, 64); err != nil {
log.Fatal(err)
} else {
numSockets = int(numSocketsTmp)
}
}
if numSockets == 0 {
numSockets = 1
}
hostname := listenAddrUrl.Hostname()
port, err := strconv.ParseUint(listenAddrUrl.Port(), 10, 64)
if err != nil {
log.Errorf("Port %s could not be converted to integer", listenAddrUrl.Port())
return
}
logFields := log.Fields{
"scheme": listenAddrUrl.Scheme,
"hostname": hostname,
"port": port,
"count": numSockets,
}
l := log.WithFields(logFields)
l.Info("starting collection")
cfg := &utils.UDPReceiverConfig{
Sockets: numSockets,
}
recv, err := utils.NewUDPReceiver(cfg)
if err != nil {
log.WithError(err).Fatal("error creating UDP receiver")
}
cfgPipe := &utils.PipeConfig{
Format: formatter,
Transport: transporter,
Producer: flowProducer,
NetFlowTemplater: metrics.NewDefaultPromTemplateSystem, // wrap template system to get Prometheus info
}
var decodeFunc utils.DecoderFunc
var p utils.FlowPipe
if listenAddrUrl.Scheme == "sflow" {
p = utils.NewSFlowPipe(cfgPipe)
} else if listenAddrUrl.Scheme == "netflow" {
p = utils.NewNetFlowPipe(cfgPipe)
} else {
l.Errorf("scheme %s does not exist", listenAddrUrl.Scheme)
return
}
decodeFunc = metrics.PromDecoderWrapper(p.DecodeFlow, listenAddrUrl.Scheme)
pipes = append(pipes, p)
// starts receivers
// the function either returns an error
if err := recv.Start(hostname, int(port), decodeFunc); err != nil {
l.Fatal(err)
} else {
wg.Add(1)
go func() {
defer wg.Done()
for {
select {
case <-q:
return
case err := <-recv.Errors():
l := l.WithError(err)
if errors.Is(err, netflow.ErrorTemplateNotFound) {
l.Warn("template error")
} else if errors.Is(err, net.ErrClosed) {
l.Info("closed receiver")
} else {
l.Error("error")
}
}
}
}()
receivers = append(receivers, recv)
}
}
// special routine to handle kafka errors transmitted as a stream
wg.Add(1)
go func() {
defer wg.Done()
var transportErr <-chan error
if transportErrorFct, ok := transporter.TransportDriver.(interface {
Errors() <-chan error
}); ok {
transportErr = transportErrorFct.Errors()
}
for {
select {
case <-q:
return
case err := <-transportErr:
if err == nil {
return
}
l := log.WithError(err)
l.Error("transport error")
}
}
}()
collecting = true
<-c
collecting = false
// stops receivers first, udp sockets will be down
for _, recv := range receivers {
if err := recv.Stop(); err != nil {
log.WithError(err).Error("error stopping receiver")
}
}
// then stop pipe
for _, pipe := range pipes {
pipe.Close()
}
// close producer
flowProducer.Close()
// close transporter (eg: flushes message to Kafka)
transporter.Close()
log.Info("closed transporter")
// close http server (prometheus + health check)
ctx, cancel := context.WithTimeout(context.Background(), time.Second*5)
if err := srv.Shutdown(ctx); err != nil {
log.WithError(err).Error("error shutting-down HTTP server")
}
cancel()
close(q) // close errors
wg.Wait()
}

View File

@@ -1,31 +1,61 @@
formatter:
fields: # list of fields to format in JSON
- type
- time_received_ns
- sequence_num
- sampling_rate
- flow_direction
- sampler_address
- time_flow_start_ns
- time_flow_end_ns
- bytes
- packets
- src_addr
- src_net
- dst_addr
- dst_net
- etype
- proto
- src_port
- dst_port
- in_if
- out_if
- src_mac
- dst_mac
# additional fields
- icmp_name # virtual column
key:
- sampler_address
protobuf: # manual protobuf fields addition
- name: flow_direction
index: 42
type: varint
- name: bi_flow_direction
index: 41
type: varint
- name: ingress_vrf_id
index: 39
type: varint
- name: egress_vrf_id
index: 40
type: varint
# Decoder mappings
ipfix:
mapping:
- field: 7 # IPFIX_FIELD_sourceTransportPort
destination: CustomInteger1
- field: 11 # IPFIX_FIELD_destinationTransportPort
destination: CustomInteger2
# penprovided: false
# pen: 0
- field: 137
destination: CustomList_1
penprovided: true
pen: 2636
- field: 61
destination: flow_direction
- field: 239
destination: bi_flow_direction
- field: 234
destination: ingress_vrf_id
- field: 235
destination: egress_vrf_id
netflowv9:
mapping:
- field: 7
destination: CustomInteger1
- field: 11
destination: CustomInteger2
- field: 34 # samplingInterval
destination: SamplingRate
- field: 34 # samplingInterval provided within the template
destination: sampling_rate
endian: little
- field: 61
destination: flow_direction
sflow:
mapping:
- layer: 4 # Layer 4: TCP or UDP
offset: 0 # Source port
length: 16 # 2 bytes
destination: CustomInteger1
- layer: 4
offset: 16 # Destination port
length: 16 # 2 bytes
destination: CustomInteger2
mapping: []

View File

@@ -17,8 +17,8 @@ clickhouse client -n <<-EOSQL
CREATE TABLE IF NOT EXISTS flows
(
time_received UInt64,
time_flow_start UInt64,
time_received_ns UInt64,
time_flow_start_ns UInt64,
sequence_num UInt32,
sampling_rate UInt64,
@@ -49,8 +49,9 @@ clickhouse client -n <<-EOSQL
CREATE TABLE IF NOT EXISTS flows_raw
(
date Date,
time_received DateTime,
time_flow_start DateTime,
time_inserted_ns DateTime64(9),
time_received_ns DateTime64(9),
time_flow_start_ns DateTime64(9),
sequence_num UInt32,
sampling_rate UInt64,
@@ -72,12 +73,32 @@ clickhouse client -n <<-EOSQL
packets UInt64
) ENGINE = MergeTree()
PARTITION BY date
ORDER BY time_received;
ORDER BY time_received_ns;
CREATE MATERIALIZED VIEW IF NOT EXISTS flows_raw_view TO flows_raw
AS SELECT
toDate(time_received) AS date,
*
toDate(time_received_ns) AS date,
now() AS time_inserted_ns,
toDateTime64(time_received_ns/1000000000, 9) AS time_received_ns,
toDateTime64(time_flow_start_ns/1000000000, 9) AS time_flow_start_ns,
sequence_num,
sampling_rate,
sampler_address,
src_addr,
dst_addr,
src_as,
dst_as,
etype,
proto,
src_port,
dst_port,
bytes,
packets
FROM flows;
CREATE TABLE IF NOT EXISTS flows_5m
@@ -106,7 +127,7 @@ clickhouse client -n <<-EOSQL
AS
SELECT
date,
toStartOfFiveMinute(time_received) AS timeslot,
toStartOfFiveMinute(time_received_ns) AS timeslot,
src_as,
dst_as,

View File

@@ -57,8 +57,7 @@ services:
- -transport.kafka.brokers=kafka:9092
- -transport=kafka
- -transport.kafka.topic=flows
- -format=pb
- -format.protobuf.fixedlen=true
- -format=bin
db:
image: clickhouse/clickhouse-server:22.8.14.53-alpine
ports:

View File

@@ -70,8 +70,8 @@
"database": "default",
"dateColDataType": "date",
"dateLoading": false,
"dateTimeColDataType": "time_flow_start",
"dateTimeType": "DATETIME",
"dateTimeColDataType": "time_flow_start_ns",
"dateTimeType": "DATETIME64",
"datetimeLoading": false,
"format": "time_series",
"group": [],
@@ -190,8 +190,8 @@
"database": "default",
"dateColDataType": "date",
"dateLoading": false,
"dateTimeColDataType": "time_flow_start",
"dateTimeType": "DATETIME",
"dateTimeColDataType": "time_flow_start_ns",
"dateTimeType": "DATETIME64",
"datetimeLoading": false,
"format": "table",
"group": [],
@@ -315,8 +315,8 @@
"database": "default",
"dateColDataType": "date",
"dateLoading": false,
"dateTimeColDataType": "time_flow_start",
"dateTimeType": "DATETIME",
"dateTimeColDataType": "time_flow_start_ns",
"dateTimeType": "DATETIME64",
"datetimeLoading": false,
"extrapolate": true,
"format": "table",
@@ -437,8 +437,8 @@
"database": "default",
"dateColDataType": "date",
"dateLoading": false,
"dateTimeColDataType": "time_flow_start",
"dateTimeType": "DATETIME",
"dateTimeColDataType": "time_flow_start_ns",
"dateTimeType": "DATETIME64",
"datetimeLoading": false,
"format": "table",
"group": [],
@@ -562,8 +562,8 @@
"database": "default",
"dateColDataType": "date",
"dateLoading": false,
"dateTimeColDataType": "time_flow_start",
"dateTimeType": "DATETIME",
"dateTimeColDataType": "time_flow_start_ns",
"dateTimeType": "DATETIME64",
"datetimeLoading": false,
"extrapolate": true,
"format": "table",

View File

@@ -1,115 +0,0 @@
package decoder
import (
"time"
)
type Message interface{}
type MessageDecoded interface{}
type DecoderFunc func(Message interface{}) error
type DoneCallback func(string, int, time.Time, time.Time)
type ErrorCallback func(string, int, time.Time, time.Time, error)
// Worker structure
type Worker struct {
Id int
DecoderParams DecoderParams
WorkerPool chan chan Message
Name string
InMsg chan Message
Quit chan bool
}
// Create a worker and add it to the pool.
func CreateWorker(workerPool chan chan Message, decoderParams DecoderParams, id int, name string) Worker {
return Worker{
Id: id,
DecoderParams: decoderParams,
WorkerPool: workerPool,
Name: name,
InMsg: make(chan Message),
Quit: make(chan bool),
}
}
// Start the worker. Launches a goroutine to process NFv9 messages.
// The worker will add its input channel of NFv9 messages to decode to the pool.
func (w Worker) Start() {
go func() {
//log.Debugf("Worker %v started", w.Id)
for {
select {
case <-w.Quit:
break
case w.WorkerPool <- w.InMsg:
msg := <-w.InMsg
timeTrackStart := time.Now()
err := w.DecoderParams.DecoderFunc(msg)
timeTrackStop := time.Now()
if err != nil && w.DecoderParams.ErrorCallback != nil {
w.DecoderParams.ErrorCallback(w.Name, w.Id, timeTrackStart, timeTrackStop, err)
} else if err == nil && w.DecoderParams.DoneCallback != nil {
w.DecoderParams.DoneCallback(w.Name, w.Id, timeTrackStart, timeTrackStop)
}
}
}
//log.Debugf("Worker %v done", w.Id)
}()
}
// Stop the worker.
func (w Worker) Stop() {
//log.Debugf("Stopping worker %v", w.Id)
w.Quit <- true
}
// Processor structure
type Processor struct {
workerpool chan chan Message
workerlist []Worker
DecoderParams DecoderParams
Name string
}
// Decoder structure. Define the function to call and the config specific to the type of packets.
type DecoderParams struct {
DecoderFunc DecoderFunc
DoneCallback DoneCallback
ErrorCallback ErrorCallback
}
// Create a message processor which is going to create all the workers and set-up the pool.
func CreateProcessor(numWorkers int, decoderParams DecoderParams, name string) Processor {
processor := Processor{
workerpool: make(chan chan Message),
workerlist: make([]Worker, numWorkers),
DecoderParams: decoderParams,
Name: name,
}
for i := 0; i < numWorkers; i++ {
worker := CreateWorker(processor.workerpool, decoderParams, i, name)
processor.workerlist[i] = worker
}
return processor
}
// Start message processor
func (p Processor) Start() {
for _, worker := range p.workerlist {
worker.Start()
}
}
func (p Processor) Stop() {
for _, worker := range p.workerlist {
worker.Stop()
}
}
// Send a message to be decoded to the pool.
func (p Processor) ProcessMessage(msg Message) {
sendChannel := <-p.workerpool
sendChannel <- msg
}

View File

@@ -0,0 +1,22 @@
package netflow
import (
"encoding/json"
"fmt"
)
func (p *IPFIXPacket) MarshalJSON() ([]byte, error) {
return json.Marshal(*p) // this is a trick to avoid having the JSON marshaller defaults to MarshalText
}
func (p *NFv9Packet) MarshalJSON() ([]byte, error) {
return json.Marshal(*p) // this is a trick to avoid having the JSON marshaller defaults to MarshalText
}
func (p *IPFIXPacket) MarshalText() ([]byte, error) {
return []byte(fmt.Sprintf("IPFIX count:%d seq:%d", len(p.FlowSets), p.SequenceNumber)), nil
}
func (p *NFv9Packet) MarshalText() ([]byte, error) {
return []byte(fmt.Sprintf("NetFlowV%d count:%d seq:%d", p.Version, p.Count, p.SequenceNumber)), nil
}

View File

@@ -446,25 +446,25 @@ const (
)
type IPFIXPacket struct {
Version uint16
Length uint16
ExportTime uint32
SequenceNumber uint32
ObservationDomainId uint32
FlowSets []interface{}
Version uint16 `json:"version"`
Length uint16 `json:"length"`
ExportTime uint32 `json:"export-time"`
SequenceNumber uint32 `json:"sequence-number"`
ObservationDomainId uint32 `json:"observation-domain-id"`
FlowSets []interface{} `json:"flow-sets"`
}
type IPFIXOptionsTemplateFlowSet struct {
FlowSetHeader
Records []IPFIXOptionsTemplateRecord
Records []IPFIXOptionsTemplateRecord `json:"records"`
}
type IPFIXOptionsTemplateRecord struct {
TemplateId uint16
FieldCount uint16
ScopeFieldCount uint16
Options []Field
Scopes []Field
TemplateId uint16 `json:"template-id"`
FieldCount uint16 `json:"field-count"`
ScopeFieldCount uint16 `json:"scope-field-count"`
Options []Field `json:"options"`
Scopes []Field `json:"scopes"`
}
func IPFIXTypeToString(typeId uint16) string {

View File

@@ -2,47 +2,39 @@ package netflow
import (
"bytes"
"context"
"encoding/binary"
"fmt"
"sync"
"github.com/netsampler/goflow2/decoders/netflow/templates"
"github.com/netsampler/goflow2/decoders/utils"
"github.com/netsampler/goflow2/v2/decoders/utils"
)
type FlowBaseTemplateSet map[uint16]map[uint32]map[uint16]interface{}
type NetFlowTemplateSystem interface {
GetTemplate(version uint16, obsDomainId uint32, templateId uint16) (interface{}, error)
AddTemplate(version uint16, obsDomainId uint32, template interface{})
type DecoderError struct {
Decoder string
Err error
}
// Transition structure to ease the conversion with the new template systems
type TemplateWrapper struct {
Ctx context.Context
Key string
Inner templates.TemplateInterface
func (e *DecoderError) Error() string {
return fmt.Sprintf("%s %s", e.Decoder, e.Err.Error())
}
func (w *TemplateWrapper) getTemplateId(template interface{}) (templateId uint16) {
switch templateIdConv := template.(type) {
case IPFIXOptionsTemplateRecord:
templateId = templateIdConv.TemplateId
case NFv9OptionsTemplateRecord:
templateId = templateIdConv.TemplateId
case TemplateRecord:
templateId = templateIdConv.TemplateId
}
return templateId
func (e *DecoderError) Unwrap() error {
return e.Err
}
func (w TemplateWrapper) GetTemplate(version uint16, obsDomainId uint32, templateId uint16) (interface{}, error) {
return w.Inner.GetTemplate(w.Ctx, &templates.TemplateKey{w.Key, version, obsDomainId, templateId})
type FlowError struct {
Version uint16
Type string
ObsDomainId uint32
TemplateId uint16
Err error
}
func (w TemplateWrapper) AddTemplate(version uint16, obsDomainId uint32, template interface{}) {
w.Inner.AddTemplate(w.Ctx, &templates.TemplateKey{w.Key, version, obsDomainId, w.getTemplateId(template)}, template)
func (e *FlowError) Error() string {
return fmt.Sprintf("[version:%d type:%s obsDomainId:%v: templateId:%d] %s", e.Version, e.Type, e.ObsDomainId, e.TemplateId, e.Err.Error())
}
func (e *FlowError) Unwrap() error {
return e.Err
}
func DecodeNFv9OptionsTemplateSet(payload *bytes.Buffer) ([]NFv9OptionsTemplateRecord, error) {
@@ -58,14 +50,14 @@ func DecodeNFv9OptionsTemplateSet(payload *bytes.Buffer) ([]NFv9OptionsTemplateR
sizeScope := int(optsTemplateRecord.ScopeLength) / 4
sizeOptions := int(optsTemplateRecord.OptionLength) / 4
if sizeScope < 0 || sizeOptions < 0 {
return records, fmt.Errorf("Error decoding OptionsTemplateSet: negative length.")
return records, fmt.Errorf("NFv9OptionsTemplateSet: negative length")
}
fields := make([]Field, sizeScope)
fields := make([]Field, sizeScope) // max 16383 entries, 65KB
for i := 0; i < sizeScope; i++ {
field := Field{}
if err := DecodeField(payload, &field, false); err != nil {
return records, err
return records, fmt.Errorf("NFv9OptionsTemplateSet: scope:%d [%w]", i, err)
}
fields[i] = field
}
@@ -75,7 +67,7 @@ func DecodeNFv9OptionsTemplateSet(payload *bytes.Buffer) ([]NFv9OptionsTemplateR
for i := 0; i < sizeOptions; i++ {
field := Field{}
if err := DecodeField(payload, &field, false); err != nil {
return records, err
return records, fmt.Errorf("NFv9OptionsTemplateSet: option:%d [%w]", i, err)
}
fields[i] = field
}
@@ -88,12 +80,14 @@ func DecodeNFv9OptionsTemplateSet(payload *bytes.Buffer) ([]NFv9OptionsTemplateR
}
func DecodeField(payload *bytes.Buffer, field *Field, pen bool) error {
err := utils.BinaryDecoder(payload, &field.Type, &field.Length)
if pen && err == nil && field.Type&0x8000 != 0 {
field.PenProvided = true
err = utils.BinaryDecoder(payload, &field.Pen)
if err := utils.BinaryDecoder(payload, &field.Type, &field.Length); err != nil {
return err
}
return err
if pen && field.Type&0x8000 != 0 {
field.PenProvided = true
return utils.BinaryDecoder(payload, &field.Pen)
}
return nil
}
func DecodeIPFIXOptionsTemplateSet(payload *bytes.Buffer) ([]IPFIXOptionsTemplateRecord, error) {
@@ -101,16 +95,19 @@ func DecodeIPFIXOptionsTemplateSet(payload *bytes.Buffer) ([]IPFIXOptionsTemplat
var err error
for payload.Len() >= 4 {
optsTemplateRecord := IPFIXOptionsTemplateRecord{}
err = utils.BinaryDecoder(payload, &optsTemplateRecord.TemplateId, &optsTemplateRecord.FieldCount, &optsTemplateRecord.ScopeFieldCount)
err = utils.BinaryDecoder(payload,
&optsTemplateRecord.TemplateId,
&optsTemplateRecord.FieldCount,
&optsTemplateRecord.ScopeFieldCount)
if err != nil {
return records, err
return records, fmt.Errorf("IPFIXOptionsTemplateSet: header [%w]", err)
}
fields := make([]Field, int(optsTemplateRecord.ScopeFieldCount))
fields := make([]Field, int(optsTemplateRecord.ScopeFieldCount)) // max 65532 which would be 589KB
for i := 0; i < int(optsTemplateRecord.ScopeFieldCount); i++ {
field := Field{}
if err := DecodeField(payload, &field, true); err != nil {
return records, err
return records, fmt.Errorf("IPFIXOptionsTemplateSet: scope:%d [%w]", i, err)
}
fields[i] = field
}
@@ -118,13 +115,13 @@ func DecodeIPFIXOptionsTemplateSet(payload *bytes.Buffer) ([]IPFIXOptionsTemplat
optionsSize := int(optsTemplateRecord.FieldCount) - int(optsTemplateRecord.ScopeFieldCount)
if optionsSize < 0 {
return records, fmt.Errorf("Error decoding OptionsTemplateSet: negative length.")
return records, fmt.Errorf("IPFIXOptionsTemplateSet: negative length")
}
fields = make([]Field, optionsSize)
for i := 0; i < optionsSize; i++ {
field := Field{}
if err := DecodeField(payload, &field, true); err != nil {
return records, err
return records, fmt.Errorf("IPFIXOptionsTemplateSet: option:%d [%w]", i, err)
}
fields[i] = field
}
@@ -143,24 +140,25 @@ func DecodeTemplateSet(version uint16, payload *bytes.Buffer) ([]TemplateRecord,
templateRecord := TemplateRecord{}
err = utils.BinaryDecoder(payload, &templateRecord.TemplateId, &templateRecord.FieldCount)
if err != nil {
return records, err
return records, fmt.Errorf("TemplateSet: reading header [%w]", err)
}
if int(templateRecord.FieldCount) < 0 {
return records, fmt.Errorf("Error decoding TemplateSet: zero count.")
return records, fmt.Errorf("TemplateSet: zero count")
}
fields := make([]Field, int(templateRecord.FieldCount))
fields := make([]Field, int(templateRecord.FieldCount)) // max 65532 which would be 589KB
for i := 0; i < int(templateRecord.FieldCount); i++ {
field := Field{}
err := utils.BinaryDecoder(payload, &field.Type, &field.Length)
if err == nil && version == 10 && field.Type&0x8000 != 0 {
if err := utils.BinaryDecoder(payload, &field.Type, &field.Length); err != nil {
return records, fmt.Errorf("TemplateSet: reading field [%w]", err)
}
if version == 10 && field.Type&0x8000 != 0 {
field.PenProvided = true
field.Type = field.Type ^ 0x8000
err = utils.BinaryDecoder(payload, &field.Pen)
}
if err != nil {
return records, err
if err := utils.BinaryDecoder(payload, &field.Pen); err != nil {
return records, fmt.Errorf("TemplateSet: reading enterprise field [%w]", err)
}
}
fields[i] = field
}
@@ -177,30 +175,27 @@ func GetTemplateSize(version uint16, template []Field) int {
if templateField.Length == 0xffff {
continue
}
sum += int(templateField.Length)
}
return sum
}
func DecodeDataSetUsingFields(version uint16, payload *bytes.Buffer, listFields []Field) []DataField {
func DecodeDataSetUsingFields(version uint16, payload *bytes.Buffer, listFields []Field) ([]DataField, error) {
dataFields := make([]DataField, len(listFields))
for payload.Len() >= GetTemplateSize(version, listFields) {
dataFields := make([]DataField, len(listFields))
for i, templateField := range listFields {
finalLength := int(templateField.Length)
if templateField.Length == 0xffff {
var variableLen8 byte
var variableLen16 uint16
err := utils.BinaryDecoder(payload, &variableLen8)
if err != nil {
return []DataField{}
if err := utils.BinaryDecoder(payload, &variableLen8); err != nil {
return nil, err
}
if variableLen8 == 0xff {
err := utils.BinaryDecoder(payload, &variableLen16)
if err != nil {
return []DataField{}
if err := utils.BinaryDecoder(payload, &variableLen16); err != nil {
return nil, err
}
finalLength = int(variableLen16)
} else {
@@ -217,29 +212,8 @@ func DecodeDataSetUsingFields(version uint16, payload *bytes.Buffer, listFields
}
dataFields[i] = nfvalue
}
return dataFields
}
return []DataField{}
}
type ErrorTemplateNotFound struct {
version uint16
obsDomainId uint32
templateId uint16
typeTemplate string
}
func NewErrorTemplateNotFound(version uint16, obsDomainId uint32, templateId uint16, typeTemplate string) *ErrorTemplateNotFound {
return &ErrorTemplateNotFound{
version: version,
obsDomainId: obsDomainId,
templateId: templateId,
typeTemplate: typeTemplate,
}
}
func (e *ErrorTemplateNotFound) Error() string {
return fmt.Sprintf("No %v template %v found for and domain id %v", e.typeTemplate, e.templateId, e.obsDomainId)
return dataFields, nil
}
func DecodeOptionsDataSet(version uint16, payload *bytes.Buffer, listFieldsScopes, listFieldsOption []Field) ([]OptionsDataRecord, error) {
@@ -249,8 +223,14 @@ func DecodeOptionsDataSet(version uint16, payload *bytes.Buffer, listFieldsScope
listFieldsOptionSize := GetTemplateSize(version, listFieldsOption)
for payload.Len() >= listFieldsScopesSize+listFieldsOptionSize {
scopeValues := DecodeDataSetUsingFields(version, payload, listFieldsScopes)
optionValues := DecodeDataSetUsingFields(version, payload, listFieldsOption)
scopeValues, err := DecodeDataSetUsingFields(version, payload, listFieldsScopes)
if err != nil {
return records, fmt.Errorf("OptionsDataSet: scope [%w]", err)
}
optionValues, err := DecodeDataSetUsingFields(version, payload, listFieldsOption)
if err != nil {
return records, fmt.Errorf("OptionsDataSet: options [%w]", err)
}
record := OptionsDataRecord{
ScopesValues: scopeValues,
@@ -267,7 +247,10 @@ func DecodeDataSet(version uint16, payload *bytes.Buffer, listFields []Field) ([
listFieldsSize := GetTemplateSize(version, listFields)
for payload.Len() >= listFieldsSize {
values := DecodeDataSetUsingFields(version, payload, listFields)
values, err := DecodeDataSetUsingFields(version, payload, listFields)
if err != nil {
return records, fmt.Errorf("DataSet: fields [%w]", err)
}
record := DataRecord{
Values: values,
@@ -278,121 +261,25 @@ func DecodeDataSet(version uint16, payload *bytes.Buffer, listFields []Field) ([
return records, nil
}
func (ts *BasicTemplateSystem) GetTemplates() map[uint16]map[uint32]map[uint16]interface{} {
ts.templateslock.RLock()
tmp := ts.templates
ts.templateslock.RUnlock()
return tmp
}
func (ts *BasicTemplateSystem) AddTemplate(version uint16, obsDomainId uint32, template interface{}) {
ts.templateslock.Lock()
defer ts.templateslock.Unlock()
_, exists := ts.templates[version]
if exists != true {
ts.templates[version] = make(map[uint32]map[uint16]interface{})
}
_, exists = ts.templates[version][obsDomainId]
if exists != true {
ts.templates[version][obsDomainId] = make(map[uint16]interface{})
}
var templateId uint16
switch templateIdConv := template.(type) {
case IPFIXOptionsTemplateRecord:
templateId = templateIdConv.TemplateId
case NFv9OptionsTemplateRecord:
templateId = templateIdConv.TemplateId
case TemplateRecord:
templateId = templateIdConv.TemplateId
}
ts.templates[version][obsDomainId][templateId] = template
}
func (ts *BasicTemplateSystem) GetTemplate(version uint16, obsDomainId uint32, templateId uint16) (interface{}, error) {
ts.templateslock.RLock()
defer ts.templateslock.RUnlock()
templatesVersion, okver := ts.templates[version]
if okver {
templatesObsDom, okobs := templatesVersion[obsDomainId]
if okobs {
template, okid := templatesObsDom[templateId]
if okid {
return template, nil
}
}
}
return nil, NewErrorTemplateNotFound(version, obsDomainId, templateId, "info")
}
type BasicTemplateSystem struct {
templates FlowBaseTemplateSet
templateslock *sync.RWMutex
}
func CreateTemplateSystem() *BasicTemplateSystem {
ts := &BasicTemplateSystem{
templates: make(FlowBaseTemplateSet),
templateslock: &sync.RWMutex{},
}
return ts
}
func DecodeMessage(payload *bytes.Buffer, templates NetFlowTemplateSystem) (interface{}, error) {
return DecodeMessageContext(context.Background(), payload, "", templates)
}
func DecodeMessageContext(ctx context.Context, payload *bytes.Buffer, templateKey string, tpli NetFlowTemplateSystem) (interface{}, error) {
var size uint16
packetNFv9 := NFv9Packet{}
packetIPFIX := IPFIXPacket{}
var returnItem interface{}
var version uint16
var obsDomainId uint32
if err := binary.Read(payload, binary.BigEndian, &version); err != nil {
return nil, fmt.Errorf("Error decoding version: %v", err)
}
if version == 9 {
err := utils.BinaryDecoder(payload, &packetNFv9.Count, &packetNFv9.SystemUptime, &packetNFv9.UnixSeconds, &packetNFv9.SequenceNumber, &packetNFv9.SourceId)
if err != nil {
return nil, fmt.Errorf("Error decoding NetFlow v9 header: %v", err)
}
size = packetNFv9.Count
packetNFv9.Version = version
returnItem = *(&packetNFv9)
obsDomainId = packetNFv9.SourceId
} else if version == 10 {
err := utils.BinaryDecoder(payload, &packetIPFIX.Length, &packetIPFIX.ExportTime, &packetIPFIX.SequenceNumber, &packetIPFIX.ObservationDomainId)
if err != nil {
return nil, fmt.Errorf("Error decoding IPFIX header: %v", err)
}
size = packetIPFIX.Length
packetIPFIX.Version = version
returnItem = *(&packetIPFIX)
obsDomainId = packetIPFIX.ObservationDomainId
} else {
return nil, fmt.Errorf("NetFlow/IPFIX version error: %d", version)
}
func DecodeMessageCommon(payload *bytes.Buffer, templates NetFlowTemplateSystem, obsDomainId uint32, size, version uint16) (interface{}, error) {
var flowSet interface{}
for i := 0; ((i < int(size) && version == 9) || version == 10) && payload.Len() > 0; i++ {
fsheader := FlowSetHeader{}
if err := utils.BinaryDecoder(payload, &fsheader); err != nil {
return returnItem, fmt.Errorf("Error decoding FlowSet header: %v", err)
return flowSet, fmt.Errorf("header [%w]", err)
}
nextrelpos := int(fsheader.Length) - binary.Size(fsheader)
if nextrelpos < 0 {
return returnItem, fmt.Errorf("Error decoding packet: non-terminated stream")
return flowSet, fmt.Errorf("negative length")
}
var flowSet interface{}
if fsheader.Id == 0 && version == 9 {
templateReader := bytes.NewBuffer(payload.Next(nextrelpos))
records, err := DecodeTemplateSet(version, templateReader)
if err != nil {
return returnItem, fmt.Errorf("Error decoding FlowSet header: %v", err)
return flowSet, &FlowError{version, "FlowSet", obsDomainId, fsheader.Id, err}
}
templatefs := TemplateFlowSet{
FlowSetHeader: fsheader,
@@ -401,10 +288,11 @@ func DecodeMessageContext(ctx context.Context, payload *bytes.Buffer, templateKe
flowSet = templatefs
if tpli != nil {
if templates != nil {
for _, record := range records {
tpli.AddTemplate(version, obsDomainId, record)
//tpli.AddTemplate(ctx, templates.NewTemplateKey(templateKey, version, obsDomainId, record.TemplateId), record)
if err := templates.AddTemplate(version, obsDomainId, record.TemplateId, record); err != nil {
return flowSet, &FlowError{version, "FlowSet", obsDomainId, fsheader.Id, err}
}
}
}
@@ -412,7 +300,7 @@ func DecodeMessageContext(ctx context.Context, payload *bytes.Buffer, templateKe
templateReader := bytes.NewBuffer(payload.Next(nextrelpos))
records, err := DecodeNFv9OptionsTemplateSet(templateReader)
if err != nil {
return returnItem, fmt.Errorf("Error decoding NetFlow OptionsTemplateSet: %v", err)
return flowSet, &FlowError{version, "NetFlow OptionsTemplateSet", obsDomainId, fsheader.Id, err}
}
optsTemplatefs := NFv9OptionsTemplateFlowSet{
FlowSetHeader: fsheader,
@@ -420,10 +308,11 @@ func DecodeMessageContext(ctx context.Context, payload *bytes.Buffer, templateKe
}
flowSet = optsTemplatefs
if tpli != nil {
if templates != nil {
for _, record := range records {
tpli.AddTemplate(version, obsDomainId, record)
//tpli.AddTemplate(ctx, templates.NewTemplateKey(templateKey, version, obsDomainId, record.TemplateId), record)
if err := templates.AddTemplate(version, obsDomainId, record.TemplateId, record); err != nil {
return flowSet, &FlowError{version, "OptionsTemplateSet", obsDomainId, fsheader.Id, err}
}
}
}
@@ -431,7 +320,7 @@ func DecodeMessageContext(ctx context.Context, payload *bytes.Buffer, templateKe
templateReader := bytes.NewBuffer(payload.Next(nextrelpos))
records, err := DecodeTemplateSet(version, templateReader)
if err != nil {
return returnItem, fmt.Errorf("Error decoding IPFIX TemplateSet: %v", err)
return flowSet, &FlowError{version, "IPFIX TemplateSet", obsDomainId, fsheader.Id, err}
}
templatefs := TemplateFlowSet{
FlowSetHeader: fsheader,
@@ -439,10 +328,11 @@ func DecodeMessageContext(ctx context.Context, payload *bytes.Buffer, templateKe
}
flowSet = templatefs
if tpli != nil {
if templates != nil {
for _, record := range records {
tpli.AddTemplate(version, obsDomainId, record)
//tpli.AddTemplate(ctx, templates.NewTemplateKey(templateKey, version, obsDomainId, record.TemplateId), record)
if err := templates.AddTemplate(version, obsDomainId, record.TemplateId, record); err != nil {
return flowSet, &FlowError{version, "IPFIX TemplateSet", obsDomainId, fsheader.Id, err}
}
}
}
@@ -450,7 +340,7 @@ func DecodeMessageContext(ctx context.Context, payload *bytes.Buffer, templateKe
templateReader := bytes.NewBuffer(payload.Next(nextrelpos))
records, err := DecodeIPFIXOptionsTemplateSet(templateReader)
if err != nil {
return returnItem, fmt.Errorf("Error decoding IPFIX OptionsTemplateSet: %v", err)
return flowSet, &FlowError{version, "IPFIX OptionsTemplateSet", obsDomainId, fsheader.Id, err}
}
optsTemplatefs := IPFIXOptionsTemplateFlowSet{
FlowSetHeader: fsheader,
@@ -458,77 +348,128 @@ func DecodeMessageContext(ctx context.Context, payload *bytes.Buffer, templateKe
}
flowSet = optsTemplatefs
if tpli != nil {
if templates != nil {
for _, record := range records {
tpli.AddTemplate(version, obsDomainId, record)
//tpli.AddTemplate(ctx, templates.NewTemplateKey(templateKey, version, obsDomainId, record.TemplateId), record)
if err := templates.AddTemplate(version, obsDomainId, record.TemplateId, record); err != nil {
return flowSet, &FlowError{version, "IPFIX OptionsTemplateSet", obsDomainId, fsheader.Id, err}
}
}
}
} else if fsheader.Id >= 256 {
dataReader := bytes.NewBuffer(payload.Next(nextrelpos))
if tpli == nil {
if templates == nil {
continue
}
template, err := tpli.GetTemplate(version, obsDomainId, fsheader.Id)
//template, err := tpli.GetTemplate(ctx, templates.NewTemplateKey(templateKey, version, obsDomainId, fsheader.Id))
if err == nil {
switch templatec := template.(type) {
case TemplateRecord:
records, err := DecodeDataSet(version, dataReader, templatec.Fields)
if err != nil {
return returnItem, fmt.Errorf("Error decoding DataSet: %v", err)
}
datafs := DataFlowSet{
FlowSetHeader: fsheader,
Records: records,
}
flowSet = datafs
case IPFIXOptionsTemplateRecord:
records, err := DecodeOptionsDataSet(version, dataReader, templatec.Scopes, templatec.Options)
if err != nil {
return returnItem, fmt.Errorf("Error decoding DataSet: %v", err)
}
datafs := OptionsDataFlowSet{
FlowSetHeader: fsheader,
Records: records,
}
flowSet = datafs
case NFv9OptionsTemplateRecord:
records, err := DecodeOptionsDataSet(version, dataReader, templatec.Scopes, templatec.Options)
if err != nil {
return returnItem, fmt.Errorf("Error decoding OptionDataSet: %v", err)
}
datafs := OptionsDataFlowSet{
FlowSetHeader: fsheader,
Records: records,
}
flowSet = datafs
}
} else {
return returnItem, err
template, err := templates.GetTemplate(version, obsDomainId, fsheader.Id)
if err != nil {
return flowSet, &FlowError{version, "Decode", obsDomainId, fsheader.Id, err}
}
switch templatec := template.(type) {
case TemplateRecord:
records, err := DecodeDataSet(version, dataReader, templatec.Fields)
if err != nil {
return flowSet, &FlowError{version, "DataSet", obsDomainId, fsheader.Id, err}
}
datafs := DataFlowSet{
FlowSetHeader: fsheader,
Records: records,
}
flowSet = datafs
case IPFIXOptionsTemplateRecord:
records, err := DecodeOptionsDataSet(version, dataReader, templatec.Scopes, templatec.Options)
if err != nil {
return flowSet, &FlowError{version, "DataSet", obsDomainId, fsheader.Id, err}
}
datafs := OptionsDataFlowSet{
FlowSetHeader: fsheader,
Records: records,
}
flowSet = datafs
case NFv9OptionsTemplateRecord:
records, err := DecodeOptionsDataSet(version, dataReader, templatec.Scopes, templatec.Options)
if err != nil {
return flowSet, &FlowError{version, "OptionDataSet", obsDomainId, fsheader.Id, err}
}
datafs := OptionsDataFlowSet{
FlowSetHeader: fsheader,
Records: records,
}
flowSet = datafs
}
} else {
return returnItem, fmt.Errorf("Error with ID %d", fsheader.Id)
return flowSet, &FlowError{version, "Decode", obsDomainId, fsheader.Id, fmt.Errorf("ID error")}
}
if version == 9 && flowSet != nil {
packetNFv9.FlowSets = append(packetNFv9.FlowSets, flowSet)
} else if version == 10 && flowSet != nil {
packetIPFIX.FlowSets = append(packetIPFIX.FlowSets, flowSet)
}
}
return flowSet, nil
}
func DecodeMessageNetFlow(payload *bytes.Buffer, templates NetFlowTemplateSystem, packetNFv9 *NFv9Packet) error {
packetNFv9.Version = 9
if err := utils.BinaryDecoder(payload,
&packetNFv9.Count,
&packetNFv9.SystemUptime,
&packetNFv9.UnixSeconds,
&packetNFv9.SequenceNumber,
&packetNFv9.SourceId); err != nil {
return &DecoderError{"NetFlowV9 header", err}
}
/*size = packetNFv9.Count
packetNFv9.Version = version
obsDomainId = packetNFv9.SourceId*/
flowSet, err := DecodeMessageCommon(payload, templates, packetNFv9.SourceId, packetNFv9.Count, 9)
packetNFv9.FlowSets = append(packetNFv9.FlowSets, flowSet)
if err != nil {
return &DecoderError{"NetFlowV9", err}
}
return nil
}
func DecodeMessageIPFIX(payload *bytes.Buffer, templates NetFlowTemplateSystem, packetIPFIX *IPFIXPacket) error {
packetIPFIX.Version = 10
if err := utils.BinaryDecoder(payload,
&packetIPFIX.Length,
&packetIPFIX.ExportTime,
&packetIPFIX.SequenceNumber,
&packetIPFIX.ObservationDomainId); err != nil {
return &DecoderError{"IPFIX header", err}
}
/*size = packetIPFIX.Length
packetIPFIX.Version = version
obsDomainId = packetIPFIX.ObservationDomainId*/
flowSet, err := DecodeMessageCommon(payload, templates, packetIPFIX.ObservationDomainId, packetIPFIX.Length, 10)
packetIPFIX.FlowSets = append(packetIPFIX.FlowSets, flowSet)
if err != nil {
return &DecoderError{"IPFIX", err}
}
return nil
}
func DecodeMessageVersion(payload *bytes.Buffer, templates NetFlowTemplateSystem, packetNFv9 *NFv9Packet, packetIPFIX *IPFIXPacket) error {
var version uint16
if err := utils.BinaryDecoder(payload, &version); err != nil {
return &DecoderError{"IPFIX/NetFlowV9 version", err}
}
if version == 9 {
return packetNFv9, nil
if err := DecodeMessageNetFlow(payload, templates, packetNFv9); err != nil {
return &DecoderError{"NetFlowV9", err}
}
return nil
} else if version == 10 {
return packetIPFIX, nil
} else {
return returnItem, fmt.Errorf("Unknown version: %d", version)
if err := DecodeMessageIPFIX(payload, templates, packetIPFIX); err != nil {
return &DecoderError{"IPFIX", err}
}
return nil
}
return &DecoderError{"IPFIX/NetFlowV9", fmt.Errorf("unknown version %d", version)}
}

View File

@@ -22,10 +22,9 @@ func TestDecodeNetFlowV9(t *testing.T) {
0x00, 0xea, 0x00, 0x04, 0x00, 0xeb, 0x00, 0x04,
}
buf := bytes.NewBuffer(template)
dec, err := DecodeMessage(buf, templates)
var decNfv9 NFv9Packet
err := DecodeMessageVersion(buf, templates, &decNfv9, nil)
assert.Nil(t, err)
assert.NotNil(t, dec)
decNfv9 := dec.(NFv9Packet)
assert.Equal(t,
NFv9Packet{
Version: 9,
@@ -206,10 +205,10 @@ func TestDecodeNetFlowV9(t *testing.T) {
0x28, 0x00, 0x40, 0x00, 0x01, 0x60, 0x00, 0x00, 0x02, 0x60, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00,
}
buf = bytes.NewBuffer(data[:89]) // truncate: we don't want to test for everything
dec, err = DecodeMessage(buf, templates)
decNfv9 = NFv9Packet{} // reset
err = DecodeMessageVersion(buf, templates, &decNfv9, nil)
assert.Nil(t, err)
assert.NotNil(t, dec)
decNfv9 = dec.(NFv9Packet)
assert.Equal(t,
NFv9Packet{
Version: 9,

View File

@@ -102,26 +102,26 @@ const (
)
type NFv9Packet struct {
Version uint16
Count uint16
SystemUptime uint32
UnixSeconds uint32
SequenceNumber uint32
SourceId uint32
FlowSets []interface{}
Version uint16 `json:"version"`
Count uint16 `json:"count"`
SystemUptime uint32 `json:"system-uptime"`
UnixSeconds uint32 `json:"unix-seconds"`
SequenceNumber uint32 `json:"sequence-number"`
SourceId uint32 `json:"source-id"`
FlowSets []interface{} `json:"flow-sets"`
}
type NFv9OptionsTemplateFlowSet struct {
FlowSetHeader
Records []NFv9OptionsTemplateRecord
Records []NFv9OptionsTemplateRecord `json:"records"`
}
type NFv9OptionsTemplateRecord struct {
TemplateId uint16
ScopeLength uint16
OptionLength uint16
Scopes []Field
Options []Field
TemplateId uint16 `json:"template-id"`
ScopeLength uint16 `json:"scope-length"`
OptionLength uint16 `json:"option-length"`
Scopes []Field `json:"scopes"`
Options []Field `json:"options"`
}
func NFv9TypeToString(typeId uint16) string {

View File

@@ -11,10 +11,10 @@ type FlowSetHeader struct {
// 0 for TemplateFlowSet
// 1 for OptionsTemplateFlowSet
// 256-65535 for DataFlowSet (used as TemplateId)
Id uint16
Id uint16 `json:"id"`
// The total length of this FlowSet in bytes (including padding).
Length uint16
Length uint16 `json:"length"`
}
// TemplateFlowSet is a collection of templates that describe structure of Data
@@ -23,7 +23,7 @@ type TemplateFlowSet struct {
FlowSetHeader
// List of Template Records
Records []TemplateRecord
Records []TemplateRecord `json:"records"`
}
// DataFlowSet is a collection of Data Records (actual NetFlow data) and Options
@@ -31,13 +31,13 @@ type TemplateFlowSet struct {
type DataFlowSet struct {
FlowSetHeader
Records []DataRecord
Records []DataRecord `json:"records"`
}
type OptionsDataFlowSet struct {
FlowSetHeader
Records []OptionsDataRecord
Records []OptionsDataRecord `json:"records"`
}
// TemplateRecord is a single template that describes structure of a Flow Record
@@ -47,30 +47,30 @@ type TemplateRecord struct {
// Template ID. This uniqueness is local to the Observation Domain that
// generated the Template ID. Template IDs of Data FlowSets are numbered
// from 256 to 65535.
TemplateId uint16
TemplateId uint16 `json:"template-id"`
// Number of fields in this Template Record. Because a Template FlowSet
// usually contains multiple Template Records, this field allows the
// Collector to determine the end of the current Template Record and
// the start of the next.
FieldCount uint16
FieldCount uint16 `json:"field-count"`
// List of fields in this Template Record.
Fields []Field
Fields []Field `json:"fields"`
}
type DataRecord struct {
Values []DataField
Values []DataField `json:"values"`
}
// OptionsDataRecord is meta data sent alongide actual NetFlow data. Combined
// with OptionsTemplateRecord it can be decoded to a single data row.
type OptionsDataRecord struct {
// List of Scope values stored in raw format as []byte
ScopesValues []DataField
ScopesValues []DataField `json:"scope-values"`
// List of Optons values stored in raw format as []byte
OptionsValues []DataField
OptionsValues []DataField `json:"option-values"`
}
// Field describes type and length of a single value in a Flow Data Record.
@@ -78,23 +78,23 @@ type OptionsDataRecord struct {
// what record value will look like.
type Field struct {
// A numeric value that represents the type of field.
PenProvided bool
Type uint16
PenProvided bool `json:"pen-provided"`
Type uint16 `json:"type"`
// The length (in bytes) of the field.
Length uint16
Length uint16 `json:"length"`
Pen uint32
Pen uint32 `json:"pen"`
}
type DataField struct {
// A numeric value that represents the type of field.
PenProvided bool
Type uint16
Pen uint32
PenProvided bool `json:"pen-provided"`
Type uint16 `json:"type"`
Pen uint32 `json:"pen"`
// The value (in bytes) of the field.
Value interface{}
Value interface{} `json:"value"`
//Value []byte
}

View File

@@ -0,0 +1,100 @@
package netflow
import (
"fmt"
"sync"
)
var (
ErrorTemplateNotFound = fmt.Errorf("Error template not found")
)
type FlowBaseTemplateSet map[uint16]map[uint32]map[uint16]interface{}
// Store interface that allows storing, removing and retrieving template data
type NetFlowTemplateSystem interface {
RemoveTemplate(version uint16, obsDomainId uint32, templateId uint16) (interface{}, error)
GetTemplate(version uint16, obsDomainId uint32, templateId uint16) (interface{}, error)
AddTemplate(version uint16, obsDomainId uint32, templateId uint16, template interface{}) error
}
func (ts *BasicTemplateSystem) GetTemplates() map[uint16]map[uint32]map[uint16]interface{} {
ts.templateslock.RLock()
tmp := ts.templates
ts.templateslock.RUnlock()
return tmp
}
func (ts *BasicTemplateSystem) AddTemplate(version uint16, obsDomainId uint32, templateId uint16, template interface{}) error {
ts.templateslock.Lock()
defer ts.templateslock.Unlock()
_, exists := ts.templates[version]
if !exists {
ts.templates[version] = make(map[uint32]map[uint16]interface{})
}
_, exists = ts.templates[version][obsDomainId]
if !exists {
ts.templates[version][obsDomainId] = make(map[uint16]interface{})
}
/*var templateId uint16
switch templateIdConv := template.(type) {
case IPFIXOptionsTemplateRecord:
templateId = templateIdConv.TemplateId
case NFv9OptionsTemplateRecord:
templateId = templateIdConv.TemplateId
case TemplateRecord:
templateId = templateIdConv.TemplateId
}*/
ts.templates[version][obsDomainId][templateId] = template
return nil
}
func (ts *BasicTemplateSystem) GetTemplate(version uint16, obsDomainId uint32, templateId uint16) (interface{}, error) {
ts.templateslock.RLock()
defer ts.templateslock.RUnlock()
if templatesVersion, ok := ts.templates[version]; ok {
if templatesObsDom, ok := templatesVersion[obsDomainId]; ok {
if template, ok := templatesObsDom[templateId]; ok {
return template, nil
}
}
}
return nil, ErrorTemplateNotFound
}
func (ts *BasicTemplateSystem) RemoveTemplate(version uint16, obsDomainId uint32, templateId uint16) (interface{}, error) {
ts.templateslock.RLock()
defer ts.templateslock.RUnlock()
if templatesVersion, ok := ts.templates[version]; ok {
if templatesObsDom, ok := templatesVersion[obsDomainId]; ok {
if template, ok := templatesObsDom[templateId]; ok {
delete(templatesObsDom, templateId)
if len(templatesObsDom) == 0 {
delete(templatesVersion, obsDomainId)
if len(templatesVersion) == 0 {
delete(ts.templates, version)
}
}
return template, nil
}
}
}
return nil, ErrorTemplateNotFound
}
type BasicTemplateSystem struct {
templates FlowBaseTemplateSet
templateslock *sync.RWMutex
}
// Creates a basic store for NetFlow and IPFIX templates.
// Everyting is stored in memory.
func CreateTemplateSystem() NetFlowTemplateSystem {
ts := &BasicTemplateSystem{
templates: make(FlowBaseTemplateSet),
templateslock: &sync.RWMutex{},
}
return ts
}

View File

@@ -1,204 +0,0 @@
package file
import (
"context"
"encoding/json"
"flag"
"fmt"
"github.com/netsampler/goflow2/decoders/netflow"
"github.com/netsampler/goflow2/decoders/netflow/templates"
"github.com/netsampler/goflow2/decoders/netflow/templates/memory"
"os"
"sync"
)
type TemplateFileObject struct {
Key *templates.TemplateKey
Data *TemplateFileData
}
type TemplateFileData struct {
Type string
Data interface{}
}
func (d *TemplateFileData) UnmarshalJSON(b []byte) error {
var s struct {
Type string
Data interface{} `json:"-"`
}
if err := json.Unmarshal(b, &s); err != nil {
return err
}
switch s.Type {
case "NFv9OptionsTemplateRecord":
newS := new(struct {
Type string
Data netflow.NFv9OptionsTemplateRecord
})
if err := json.Unmarshal(b, newS); err != nil {
return err
}
d.Type = newS.Type
d.Data = newS.Data
case "TemplateRecord":
newS := new(struct {
Type string
Data netflow.TemplateRecord
})
if err := json.Unmarshal(b, newS); err != nil {
return err
}
d.Type = newS.Type
d.Data = newS.Data
case "IPFIXOptionsTemplateRecord":
newS := new(struct {
Type string
Data netflow.IPFIXOptionsTemplateRecord
})
if err := json.Unmarshal(b, newS); err != nil {
return err
}
d.Type = newS.Type
d.Data = newS.Data
}
return nil
}
type TemplateFile struct {
Templates []*TemplateFileObject `json:"templates"`
}
func (f *TemplateFile) Add(key *templates.TemplateKey, data interface{}) {
var typeName string
switch data.(type) {
case netflow.NFv9OptionsTemplateRecord:
typeName = "NFv9OptionsTemplateRecord"
case netflow.TemplateRecord:
typeName = "TemplateRecord"
case netflow.IPFIXOptionsTemplateRecord:
typeName = "IPFIXOptionsTemplateRecord"
default:
return
}
f.Templates = append(f.Templates, &TemplateFileObject{
Key: key,
Data: &TemplateFileData{
Type: typeName,
Data: data,
},
})
}
func NewTemplateFile() *TemplateFile {
return &TemplateFile{
Templates: make([]*TemplateFileObject, 0),
}
}
type FileDriver struct {
memDriver *memory.MemoryDriver
path string
lock *sync.Mutex
}
func (d *FileDriver) Prepare() error {
d.memDriver = memory.Driver
d.lock = &sync.Mutex{}
flag.StringVar(&d.path, "netflow.templates.file.path", "./templates.json", "Path of file to store templates")
return nil
}
func (d *FileDriver) Init(ctx context.Context) error {
var err error
if err = d.memDriver.Init(ctx); err != nil {
return err
}
f, err := os.OpenFile(d.path, os.O_RDWR|os.O_CREATE, 0755)
if err != nil {
return err
}
defer f.Close()
dec := json.NewDecoder(f)
tf := NewTemplateFile()
if err = dec.Decode(tf); err != nil {
// log error
}
for _, template := range tf.Templates {
if err := d.memDriver.AddTemplate(ctx, template.Key, template.Data.Data); err != nil {
// log error
continue
}
}
return nil
}
func (d *FileDriver) Close(ctx context.Context) error {
if err := d.memDriver.Close(ctx); err != nil {
return err
}
return nil
}
func (d *FileDriver) ListTemplates(ctx context.Context, ch chan *templates.TemplateKey) error {
return d.memDriver.ListTemplates(ctx, ch)
}
func (d *FileDriver) AddTemplate(ctx context.Context, key *templates.TemplateKey, template interface{}) error {
d.lock.Lock()
defer d.lock.Unlock()
if err := d.memDriver.AddTemplate(ctx, key, template); err != nil {
return err
}
tf := NewTemplateFile()
ch := make(chan *templates.TemplateKey, 5)
go func() {
if err := d.memDriver.ListTemplates(ctx, ch); err != nil {
// log error
close(ch)
}
}()
for key := range ch {
if key == nil {
break
}
if template, err := d.memDriver.GetTemplate(ctx, key); err != nil {
// log error
continue
} else {
tf.Add(key, template)
}
}
tmpPath := fmt.Sprintf("%s-tmp", d.path)
f, err := os.OpenFile(tmpPath, os.O_RDWR|os.O_CREATE, 0755)
if err != nil {
return err
}
enc := json.NewEncoder(f)
if err := enc.Encode(tf); err != nil {
f.Close()
return err
}
return os.Rename(tmpPath, d.path)
}
func (d *FileDriver) GetTemplate(ctx context.Context, key *templates.TemplateKey) (interface{}, error) {
return d.memDriver.GetTemplate(ctx, key)
}
func init() {
d := &FileDriver{}
templates.RegisterTemplateDriver("file", d)
}

View File

@@ -1,73 +0,0 @@
package memory
import (
"context"
"github.com/netsampler/goflow2/decoders/netflow/templates"
"sync"
)
var (
Driver = &MemoryDriver{}
)
type templateData struct {
key *templates.TemplateKey
data interface{}
}
type MemoryDriver struct {
lock *sync.RWMutex
templates map[string]templateData
}
func (d *MemoryDriver) Prepare() error {
// could have an expiry
return nil
}
func (d *MemoryDriver) Init(context.Context) error {
d.lock = &sync.RWMutex{}
d.templates = make(map[string]templateData)
return nil
}
func (d *MemoryDriver) Close(context.Context) error {
return nil
}
func (d *MemoryDriver) ListTemplates(ctx context.Context, ch chan *templates.TemplateKey) error {
d.lock.RLock()
defer d.lock.RUnlock()
for _, v := range d.templates {
select {
case ch <- v.key:
case <-ctx.Done():
return ctx.Err()
}
}
select {
case ch <- nil:
}
return nil
}
func (d *MemoryDriver) AddTemplate(ctx context.Context, key *templates.TemplateKey, template interface{}) error {
d.lock.Lock()
defer d.lock.Unlock()
d.templates[key.String()] = templateData{
key: key,
data: template,
}
return nil
}
func (d *MemoryDriver) GetTemplate(ctx context.Context, key *templates.TemplateKey) (interface{}, error) {
d.lock.RLock()
defer d.lock.RUnlock()
return d.templates[key.String()].data, nil
}
func init() {
templates.RegisterTemplateDriver("memory", Driver)
}

View File

@@ -1,139 +0,0 @@
package templates
import (
"context"
"fmt"
"strconv"
"strings"
"sync"
)
var (
templateDrivers = make(map[string]TemplateDriver) // might be better to change into "factory"
lock = &sync.RWMutex{}
)
type TemplateDriver interface {
TemplateInterface
Prepare() error // Prepare driver (eg: flag registration)
Init(context.Context) error // Initialize driver (eg: parse keying)
Close(context.Context) error // Close drive (eg: close file)
}
type TemplateKey struct {
TemplateKey string
Version uint16
ObsDomainId uint32
TemplateId uint16
}
func NewTemplateKey(templateKey string, version uint16, obsDomainId uint32, templateId uint16) *TemplateKey {
return &TemplateKey{
TemplateKey: templateKey,
Version: version,
ObsDomainId: obsDomainId,
TemplateId: templateId,
}
}
func (k *TemplateKey) String() string {
return fmt.Sprintf("%s-%d-%d-%d", k.TemplateKey, k.Version, k.ObsDomainId, k.TemplateId)
}
func ParseTemplateKey(key string, k *TemplateKey) error {
if k != nil {
return nil
}
var version uint16
var obsDomainId uint32
var templateId uint16
keySplit := strings.Split(key, "-")
if len(keySplit) != 4 {
return fmt.Errorf("template key format is invalid")
}
templateKey := keySplit[0]
if val, err := strconv.ParseUint(keySplit[1], 10, 64); err != nil {
return fmt.Errorf("template key version is invalid")
} else {
version = uint16(val)
}
if val, err := strconv.ParseUint(keySplit[2], 10, 64); err != nil {
fmt.Errorf("template key observation domain I Dis invalid")
} else {
obsDomainId = uint32(val)
}
if val, err := strconv.ParseUint(keySplit[3], 10, 64); err != nil {
fmt.Errorf("template key template ID is invalid")
} else {
templateId = uint16(val)
}
k.TemplateKey = templateKey
k.Version = version
k.ObsDomainId = obsDomainId
k.TemplateId = templateId
return nil
}
type TemplateInterface interface {
ListTemplates(ctx context.Context, ch chan *TemplateKey) error
GetTemplate(ctx context.Context, key *TemplateKey) (interface{}, error)
AddTemplate(ctx context.Context, key *TemplateKey, template interface{}) error // add expiration
}
type TemplateSystem struct {
driver TemplateDriver
}
func (t *TemplateSystem) ListTemplates(ctx context.Context, ch chan *TemplateKey) error {
return t.driver.ListTemplates(ctx, ch)
}
func (t *TemplateSystem) AddTemplate(ctx context.Context, key *TemplateKey, template interface{}) error {
return t.driver.AddTemplate(ctx, key, template)
}
func (t *TemplateSystem) GetTemplate(ctx context.Context, key *TemplateKey) (interface{}, error) {
return t.driver.GetTemplate(ctx, key)
}
func (t *TemplateSystem) Close(ctx context.Context) error {
return t.driver.Close(ctx)
}
func RegisterTemplateDriver(name string, t TemplateDriver) {
lock.Lock()
templateDrivers[name] = t
lock.Unlock()
if err := t.Prepare(); err != nil {
panic(err)
}
}
func FindTemplateSystem(ctx context.Context, name string) (*TemplateSystem, error) {
lock.RLock()
t, ok := templateDrivers[name]
lock.RUnlock()
if !ok {
return nil, fmt.Errorf("Template %s not found", name)
}
err := t.Init(ctx)
return &TemplateSystem{t}, err
}
func GetTemplates() []string {
lock.RLock()
defer lock.RUnlock()
t := make([]string, len(templateDrivers))
var i int
for k, _ := range templateDrivers {
t[i] = k
i++
}
return t
}

View File

@@ -0,0 +1,60 @@
package netflowlegacy
import (
"encoding/json"
"fmt"
"time"
)
func (p *PacketNetFlowV5) MarshalJSON() ([]byte, error) {
return json.Marshal(*p) // this is a trick to avoid having the JSON marshaller defaults to MarshalText
}
func (p *PacketNetFlowV5) MarshalText() ([]byte, error) {
return []byte(fmt.Sprintf("NetFlowV%d seq:%d count:%d", p.Version, p.FlowSequence, p.Count)), nil
}
func (p PacketNetFlowV5) String() string {
str := "NetFlow v5 Packet\n"
str += "-----------------\n"
str += fmt.Sprintf(" Version: %v\n", p.Version)
str += fmt.Sprintf(" Count: %v\n", p.Count)
unixSeconds := time.Unix(int64(p.UnixSecs), int64(p.UnixNSecs))
str += fmt.Sprintf(" SystemUptime: %v\n", time.Duration(p.SysUptime)*time.Millisecond)
str += fmt.Sprintf(" UnixSeconds: %v\n", unixSeconds.String())
str += fmt.Sprintf(" FlowSequence: %v\n", p.FlowSequence)
str += fmt.Sprintf(" EngineType: %v\n", p.EngineType)
str += fmt.Sprintf(" EngineId: %v\n", p.EngineId)
str += fmt.Sprintf(" SamplingInterval: %v\n", p.SamplingInterval)
str += fmt.Sprintf(" Records (%v):\n", len(p.Records))
for i, record := range p.Records {
str += fmt.Sprintf(" Record %v:\n", i)
str += record.String()
}
return str
}
func (r RecordsNetFlowV5) String() string {
str := fmt.Sprintf(" SrcAddr: %v\n", r.SrcAddr)
str += fmt.Sprintf(" DstAddr: %v\n", r.DstAddr)
str += fmt.Sprintf(" NextHop: %v\n", r.NextHop)
str += fmt.Sprintf(" Input: %v\n", r.Input)
str += fmt.Sprintf(" Output: %v\n", r.Output)
str += fmt.Sprintf(" DPkts: %v\n", r.DPkts)
str += fmt.Sprintf(" DOctets: %v\n", r.DOctets)
str += fmt.Sprintf(" First: %v\n", time.Duration(r.First)*time.Millisecond)
str += fmt.Sprintf(" Last: %v\n", time.Duration(r.Last)*time.Millisecond)
str += fmt.Sprintf(" SrcPort: %v\n", r.SrcPort)
str += fmt.Sprintf(" DstPort: %v\n", r.DstPort)
str += fmt.Sprintf(" TCPFlags: %v\n", r.TCPFlags)
str += fmt.Sprintf(" Proto: %v\n", r.Proto)
str += fmt.Sprintf(" Tos: %v\n", r.Tos)
str += fmt.Sprintf(" SrcAS: %v\n", r.SrcAS)
str += fmt.Sprintf(" DstAS: %v\n", r.DstAS)
str += fmt.Sprintf(" SrcMask: %v\n", r.SrcMask)
str += fmt.Sprintf(" DstMask: %v\n", r.DstMask)
return str
}

View File

@@ -4,58 +4,55 @@ import (
"bytes"
"fmt"
"github.com/netsampler/goflow2/decoders/utils"
"github.com/netsampler/goflow2/v2/decoders/utils"
)
type ErrorVersion struct {
version uint16
type DecoderError struct {
Err error
}
func NewErrorVersion(version uint16) *ErrorVersion {
return &ErrorVersion{
version: version,
}
func (e *DecoderError) Error() string {
return fmt.Sprintf("NetFlowLegacy %s", e.Err.Error())
}
func (e *ErrorVersion) Error() string {
return fmt.Sprintf("Unknown NetFlow version %v (only decodes v5)", e.version)
func (e *DecoderError) Unwrap() error {
return e.Err
}
func DecodeMessage(payload *bytes.Buffer) (interface{}, error) {
func DecodeMessageVersion(payload *bytes.Buffer, packet *PacketNetFlowV5) error {
var version uint16
err := utils.BinaryDecoder(payload, &version)
if err != nil {
return nil, err
if err := utils.BinaryDecoder(payload, &version); err != nil {
return err
}
packet := PacketNetFlowV5{}
if version == 5 {
packet.Version = version
utils.BinaryDecoder(payload,
&(packet.Count),
&(packet.SysUptime),
&(packet.UnixSecs),
&(packet.UnixNSecs),
&(packet.FlowSequence),
&(packet.EngineType),
&(packet.EngineId),
&(packet.SamplingInterval),
)
packet.SamplingInterval = packet.SamplingInterval & 0x3FFF
packet.Records = make([]RecordsNetFlowV5, int(packet.Count))
for i := 0; i < int(packet.Count) && payload.Len() >= 48; i++ {
record := RecordsNetFlowV5{}
err := utils.BinaryDecoder(payload, &record)
if err != nil {
return packet, err
}
packet.Records[i] = record
}
return packet, nil
} else {
return nil, NewErrorVersion(version)
packet.Version = version
if packet.Version != 5 {
return &DecoderError{fmt.Errorf("unknown version %d", version)}
}
return DecodeMessage(payload, packet)
}
func DecodeMessage(payload *bytes.Buffer, packet *PacketNetFlowV5) error {
if err := utils.BinaryDecoder(payload,
&packet.Count,
&packet.SysUptime,
&packet.UnixSecs,
&packet.UnixNSecs,
&packet.FlowSequence,
&packet.EngineType,
&packet.EngineId,
&packet.SamplingInterval,
); err != nil {
return &DecoderError{err}
}
packet.Records = make([]RecordsNetFlowV5, int(packet.Count)) // maximum is 65535 which would be 3MB
for i := 0; i < int(packet.Count) && payload.Len() >= 48; i++ {
record := RecordsNetFlowV5{}
if err := utils.BinaryDecoder(payload, &record); err != nil {
return &DecoderError{err}
}
packet.Records[i] = record
}
return nil
}

View File

@@ -32,10 +32,8 @@ func TestDecodeNetFlowV5(t *testing.T) {
}
buf := bytes.NewBuffer(data)
dec, err := DecodeMessage(buf)
assert.Nil(t, err)
assert.NotNil(t, dec)
decNfv5 := dec.(PacketNetFlowV5)
var decNfv5 PacketNetFlowV5
assert.Nil(t, DecodeMessageVersion(buf, &decNfv5))
assert.Equal(t, uint16(5), decNfv5.Version)
assert.Equal(t, uint16(9), decNfv5.Records[0].Input)
}

View File

@@ -1,96 +1,47 @@
package netflowlegacy
import (
"encoding/binary"
"fmt"
"net"
"time"
)
type PacketNetFlowV5 struct {
Version uint16
Count uint16
SysUptime uint32
UnixSecs uint32
UnixNSecs uint32
FlowSequence uint32
EngineType uint8
EngineId uint8
SamplingInterval uint16
Records []RecordsNetFlowV5
Version uint16 `json:"version"`
Count uint16 `json:"count"`
SysUptime uint32 `json:"sys-uptime"`
UnixSecs uint32 `json:"unix-secs"`
UnixNSecs uint32 `json:"unix-nsecs"`
FlowSequence uint32 `json:"flow-sequence"`
EngineType uint8 `json:"engine-type"`
EngineId uint8 `json:"engine-id"`
SamplingInterval uint16 `json:"sampling-interval"`
Records []RecordsNetFlowV5 `json:"records"`
}
type RecordsNetFlowV5 struct {
SrcAddr uint32
DstAddr uint32
NextHop uint32
Input uint16
Output uint16
DPkts uint32
DOctets uint32
First uint32
Last uint32
SrcPort uint16
DstPort uint16
Pad1 byte
TCPFlags uint8
Proto uint8
Tos uint8
SrcAS uint16
DstAS uint16
SrcMask uint8
DstMask uint8
Pad2 uint16
SrcAddr IPAddress `json:"src-addr"`
DstAddr IPAddress `json:"dst-addr"`
NextHop IPAddress `json:"next-hop"`
Input uint16 `json:"input"`
Output uint16 `json:"output"`
DPkts uint32 `json:"dpkts"`
DOctets uint32 `json:"doctets"`
First uint32 `json:"first"`
Last uint32 `json:"last"`
SrcPort uint16 `json:"src-port"`
DstPort uint16 `json:"dst-port"`
Pad1 byte `json:"pad1"`
TCPFlags uint8 `json:"tcp-flgas"`
Proto uint8 `json:"proto"`
Tos uint8 `json:"tos"`
SrcAS uint16 `json:"src-as"`
DstAS uint16 `json:"dst-as"`
SrcMask uint8 `json:"src-mask"`
DstMask uint8 `json:"dst-mask"`
Pad2 uint16 `json:"pad2"`
}
func (p PacketNetFlowV5) String() string {
str := "NetFlow v5 Packet\n"
str += "-----------------\n"
str += fmt.Sprintf(" Version: %v\n", p.Version)
str += fmt.Sprintf(" Count: %v\n", p.Count)
type IPAddress uint32 // purely for the formatting purpose
unixSeconds := time.Unix(int64(p.UnixSecs), int64(p.UnixNSecs))
str += fmt.Sprintf(" SystemUptime: %v\n", time.Duration(p.SysUptime)*time.Millisecond)
str += fmt.Sprintf(" UnixSeconds: %v\n", unixSeconds.String())
str += fmt.Sprintf(" FlowSequence: %v\n", p.FlowSequence)
str += fmt.Sprintf(" EngineType: %v\n", p.EngineType)
str += fmt.Sprintf(" EngineId: %v\n", p.EngineId)
str += fmt.Sprintf(" SamplingInterval: %v\n", p.SamplingInterval)
str += fmt.Sprintf(" Records (%v):\n", len(p.Records))
for i, record := range p.Records {
str += fmt.Sprintf(" Record %v:\n", i)
str += record.String()
}
return str
}
func (r RecordsNetFlowV5) String() string {
srcaddr := make(net.IP, 4)
binary.BigEndian.PutUint32(srcaddr, r.SrcAddr)
dstaddr := make(net.IP, 4)
binary.BigEndian.PutUint32(dstaddr, r.DstAddr)
nexthop := make(net.IP, 4)
binary.BigEndian.PutUint32(nexthop, r.NextHop)
str := fmt.Sprintf(" SrcAddr: %v\n", srcaddr.String())
str += fmt.Sprintf(" DstAddr: %v\n", dstaddr.String())
str += fmt.Sprintf(" NextHop: %v\n", nexthop.String())
str += fmt.Sprintf(" Input: %v\n", r.Input)
str += fmt.Sprintf(" Output: %v\n", r.Output)
str += fmt.Sprintf(" DPkts: %v\n", r.DPkts)
str += fmt.Sprintf(" DOctets: %v\n", r.DOctets)
str += fmt.Sprintf(" First: %v\n", time.Duration(r.First)*time.Millisecond)
str += fmt.Sprintf(" Last: %v\n", time.Duration(r.Last)*time.Millisecond)
str += fmt.Sprintf(" SrcPort: %v\n", r.SrcPort)
str += fmt.Sprintf(" DstPort: %v\n", r.DstPort)
str += fmt.Sprintf(" TCPFlags: %v\n", r.TCPFlags)
str += fmt.Sprintf(" Proto: %v\n", r.Proto)
str += fmt.Sprintf(" Tos: %v\n", r.Tos)
str += fmt.Sprintf(" SrcAS: %v\n", r.SrcAS)
str += fmt.Sprintf(" DstAS: %v\n", r.DstAS)
str += fmt.Sprintf(" SrcMask: %v\n", r.SrcMask)
str += fmt.Sprintf(" DstMask: %v\n", r.DstMask)
return str
func (s *IPAddress) MarshalJSON() ([]byte, error) {
return []byte(fmt.Sprintf("\"%d.%d.%d.%d\"", *s>>24, (*s>>16)&0xFF, (*s>>8)&0xFF, *s&0xFF)), nil
}

View File

@@ -1,103 +1,118 @@
package sflow
import (
"fmt"
"net"
)
type SampledHeader struct {
Protocol uint32
FrameLength uint32
Stripped uint32
OriginalLength uint32
HeaderData []byte
Protocol uint32 `json:"protocol"`
FrameLength uint32 `json:"frame-length"`
Stripped uint32 `json:"stripped"`
OriginalLength uint32 `json:"original-length"`
HeaderData []byte `json:"header-data"`
}
type SampledEthernet struct {
Length uint32
SrcMac []byte
DstMac []byte
EthType uint32
Length uint32 `json:"length"`
SrcMac MacAddress `json:"src-mac"`
DstMac MacAddress `json:"dst-mac"`
EthType uint32 `json:"eth-type"`
}
type SampledIP_Base struct {
Length uint32
Protocol uint32
SrcIP []byte
DstIP []byte
SrcPort uint32
DstPort uint32
TcpFlags uint32
type SampledIPBase struct {
Length uint32 `json:"length"`
Protocol uint32 `json:"protocol"`
SrcIP IPAddress `json:"src-ip"`
DstIP IPAddress `json:"dst-ip"`
SrcPort uint32 `json:"src-port"`
DstPort uint32 `json:"dst-port"`
TcpFlags uint32 `json:"tcp-flags"`
}
type MacAddress []byte // purely for the formatting purpose
func (s *MacAddress) MarshalJSON() ([]byte, error) {
return []byte(fmt.Sprintf("\"%s\"", net.HardwareAddr([]byte(*s)).String())), nil
}
type SampledIPv4 struct {
Base SampledIP_Base
Tos uint32
SampledIPBase
Tos uint32 `json:"tos"`
}
type SampledIPv6 struct {
Base SampledIP_Base
Priority uint32
SampledIPBase
Priority uint32 `json:"priority"`
}
type ExtendedSwitch struct {
SrcVlan uint32
SrcPriority uint32
DstVlan uint32
DstPriority uint32
SrcVlan uint32 `json:"src-vlan"`
SrcPriority uint32 `json:"src-priority"`
DstVlan uint32 `json:"dst-vlan"`
DstPriority uint32 `json:"dst-priority"`
}
type ExtendedRouter struct {
NextHopIPVersion uint32
NextHop []byte
SrcMaskLen uint32
DstMaskLen uint32
NextHopIPVersion uint32 `json:"next-hop-ip-version"`
NextHop IPAddress `json:"next-hop"`
SrcMaskLen uint32 `json:"src-mask-len"`
DstMaskLen uint32 `json:"dst-mask-len"`
}
type ExtendedGateway struct {
NextHopIPVersion uint32
NextHop []byte
AS uint32
SrcAS uint32
SrcPeerAS uint32
ASDestinations uint32
ASPathType uint32
ASPathLength uint32
ASPath []uint32
CommunitiesLength uint32
Communities []uint32
LocalPref uint32
NextHopIPVersion uint32 `json:"next-hop-ip-version"`
NextHop IPAddress `json:"next-hop"`
AS uint32 `json:"as"`
SrcAS uint32 `json:"src-as"`
SrcPeerAS uint32 `json:"src-peer-as"`
ASDestinations uint32 `json:"as-destinations"`
ASPathType uint32 `json:"as-path-type"`
ASPathLength uint32 `json:"as-path-length"`
ASPath []uint32 `json:"as-path"`
CommunitiesLength uint32 `json:"communities-length"`
Communities []uint32 `json:"communities"`
LocalPref uint32 `json:"local-pref"`
}
type IfCounters struct {
IfIndex uint32
IfType uint32
IfSpeed uint64
IfDirection uint32
IfStatus uint32
IfInOctets uint64
IfInUcastPkts uint32
IfInMulticastPkts uint32
IfInBroadcastPkts uint32
IfInDiscards uint32
IfInErrors uint32
IfInUnknownProtos uint32
IfOutOctets uint64
IfOutUcastPkts uint32
IfOutMulticastPkts uint32
IfOutBroadcastPkts uint32
IfOutDiscards uint32
IfOutErrors uint32
IfPromiscuousMode uint32
IfIndex uint32 `json:"if-index"`
IfType uint32 `json:"if-type"`
IfSpeed uint64 `json:"if-speed"`
IfDirection uint32 `json:"if-direction"`
IfStatus uint32 `json:"if-status"`
IfInOctets uint64 `json:"if-in-octets"`
IfInUcastPkts uint32 `json:"if-in-ucast-pkts"`
IfInMulticastPkts uint32 `json:"if-in-multicast-pkts"`
IfInBroadcastPkts uint32 `json:"if-in-broadcast-pkts"`
IfInDiscards uint32 `json:"if-in-discards"`
IfInErrors uint32 `json:"if-in-errors"`
IfInUnknownProtos uint32 `json:"if-in-unknown-protos"`
IfOutOctets uint64 `json:"if-out-octets"`
IfOutUcastPkts uint32 `json:"if-out-ucast-pkts"`
IfOutMulticastPkts uint32 `json:"if-out-multicast-pkts"`
IfOutBroadcastPkts uint32 `json:"if-out-broadcast-pkts"`
IfOutDiscards uint32 `json:"if-out-discards"`
IfOutErrors uint32 `json:"if-out-errors"`
IfPromiscuousMode uint32 `json:"if-promiscuous-mode"`
}
type EthernetCounters struct {
Dot3StatsAlignmentErrors uint32
Dot3StatsFCSErrors uint32
Dot3StatsSingleCollisionFrames uint32
Dot3StatsMultipleCollisionFrames uint32
Dot3StatsSQETestErrors uint32
Dot3StatsDeferredTransmissions uint32
Dot3StatsLateCollisions uint32
Dot3StatsExcessiveCollisions uint32
Dot3StatsInternalMacTransmitErrors uint32
Dot3StatsCarrierSenseErrors uint32
Dot3StatsFrameTooLongs uint32
Dot3StatsInternalMacReceiveErrors uint32
Dot3StatsSymbolErrors uint32
Dot3StatsAlignmentErrors uint32 `json:"dot3-stats-aligment-errors"`
Dot3StatsFCSErrors uint32 `json:"dot3-stats-fcse-errors"`
Dot3StatsSingleCollisionFrames uint32 `json:"dot3-stats-single-collision-frames"`
Dot3StatsMultipleCollisionFrames uint32 `json:"dot3-stats-multiple-collision-frames"`
Dot3StatsSQETestErrors uint32 `json:"dot3-stats-seq-test-errors"`
Dot3StatsDeferredTransmissions uint32 `json:"dot3-stats-deferred-transmissions"`
Dot3StatsLateCollisions uint32 `json:"dot3-stats-late-collisions"`
Dot3StatsExcessiveCollisions uint32 `json:"dot3-stats-excessive-collisions"`
Dot3StatsInternalMacTransmitErrors uint32 `json:"dot3-stats-internal-mac-transmit-errors"`
Dot3StatsCarrierSenseErrors uint32 `json:"dot3-stats-carrier-sense-errors"`
Dot3StatsFrameTooLongs uint32 `json:"dot3-stats-frame-too-longs"`
Dot3StatsInternalMacReceiveErrors uint32 `json:"dot3-stats-internal-mac-receive-errors"`
Dot3StatsSymbolErrors uint32 `json:"dot3-stats-symbol-errors"`
}
type RawRecord struct {
Data []byte `json:"data"`
}

16
decoders/sflow/format.go Normal file
View File

@@ -0,0 +1,16 @@
package sflow
import (
"encoding/json"
"fmt"
"net/netip"
)
func (p *Packet) MarshalJSON() ([]byte, error) {
return json.Marshal(*p) // this is a trick to avoid having the JSON marshaller defaults to MarshalText
}
func (p *Packet) MarshalText() ([]byte, error) {
agentIP, _ := netip.AddrFromSlice(p.AgentIP)
return []byte(fmt.Sprintf("sFlow%d agent:%s seq:%d count:%d", p.Version, agentIP.String(), p.SequenceNumber, p.SamplesCount)), nil
}

View File

@@ -1,73 +1,81 @@
package sflow
import (
"fmt"
"net/netip"
)
type Packet struct {
Version uint32
IPVersion uint32
AgentIP []byte
SubAgentId uint32
SequenceNumber uint32
Uptime uint32
SamplesCount uint32
Samples []interface{}
Version uint32 `json:"version"`
IPVersion uint32 `json:"ip-version"`
AgentIP IPAddress `json:"agent-ip"`
SubAgentId uint32 `json:"sub-agent-id"`
SequenceNumber uint32 `json:"sequence-number"`
Uptime uint32 `json:"uptime"`
SamplesCount uint32 `json:"samples-count"`
Samples []interface{} `json:"samples"`
}
type IPAddress []byte // purely for the formatting purpose
func (s IPAddress) MarshalJSON() ([]byte, error) {
ip, _ := netip.AddrFromSlice([]byte(s))
return []byte(fmt.Sprintf("\"%s\"", ip.String())), nil
}
type SampleHeader struct {
Format uint32
Length uint32
Format uint32 `json:"format"`
Length uint32 `json:"length"`
SampleSequenceNumber uint32
SourceIdType uint32
SourceIdValue uint32
SampleSequenceNumber uint32 `json:"sample-sequence-number"`
SourceIdType uint32 `json:"source-id-type"`
SourceIdValue uint32 `json:"source-id-value"`
}
type FlowSample struct {
Header SampleHeader
Header SampleHeader `json:"header"`
SamplingRate uint32
SamplePool uint32
Drops uint32
Input uint32
Output uint32
FlowRecordsCount uint32
Records []FlowRecord
SamplingRate uint32 `json:"sampling-rate"`
SamplePool uint32 `json:"sample-pool"`
Drops uint32 `json:"drops"`
Input uint32 `json:"input"`
Output uint32 `json:"output"`
FlowRecordsCount uint32 `json:"flow-records-count"`
Records []FlowRecord `json:"records"`
}
type CounterSample struct {
Header SampleHeader
Header SampleHeader `json:"header"`
CounterRecordsCount uint32
Records []CounterRecord
CounterRecordsCount uint32 `json:"counter-records-count"`
Records []CounterRecord `json:"records"`
}
type ExpandedFlowSample struct {
Header SampleHeader
Header SampleHeader `json:"header"`
SamplingRate uint32
SamplePool uint32
Drops uint32
InputIfFormat uint32
InputIfValue uint32
OutputIfFormat uint32
OutputIfValue uint32
FlowRecordsCount uint32
Records []FlowRecord
SamplingRate uint32 `json:"sampling-rate"`
SamplePool uint32 `json:"sample-pool"`
Drops uint32 `json:"drops"`
InputIfFormat uint32 `json:"input-if-format"`
InputIfValue uint32 `json:"input-if-value"`
OutputIfFormat uint32 `json:"output-if-format"`
OutputIfValue uint32 `json:"output-if-value"`
FlowRecordsCount uint32 `json:"flow-records-count"`
Records []FlowRecord `json:"records"`
}
type RecordHeader struct {
DataFormat uint32
Length uint32
DataFormat uint32 `json:"data-format"`
Length uint32 `json:"length"`
}
type FlowRecord struct {
Header RecordHeader
Data interface{}
}
type FlowRecordRaw struct {
Data []byte
Header RecordHeader `json:"header"`
Data interface{} `json:"data"`
}
type CounterRecord struct {
Header RecordHeader
Data interface{}
Header RecordHeader `json:"header"`
Data interface{} `json:"data"`
}

View File

@@ -2,10 +2,9 @@ package sflow
import (
"bytes"
"errors"
"fmt"
"github.com/netsampler/goflow2/decoders/utils"
"github.com/netsampler/goflow2/v2/decoders/utils"
)
const (
@@ -18,95 +17,49 @@ const (
FORMAT_IPV6 = 4
)
type ErrorDecodingSFlow struct {
msg string
type DecoderError struct {
Err error
}
func NewErrorDecodingSFlow(msg string) *ErrorDecodingSFlow {
return &ErrorDecodingSFlow{
msg: msg,
}
func (e *DecoderError) Error() string {
return fmt.Sprintf("sFlow %s", e.Err.Error())
}
func (e *ErrorDecodingSFlow) Error() string {
return fmt.Sprintf("Error decoding sFlow: %v", e.msg)
func (e *DecoderError) Unwrap() error {
return e.Err
}
type ErrorDataFormat struct {
dataformat uint32
type FlowError struct {
Format uint32
Seq uint32
Err error
}
func NewErrorDataFormat(dataformat uint32) *ErrorDataFormat {
return &ErrorDataFormat{
dataformat: dataformat,
}
func (e *FlowError) Error() string {
return fmt.Sprintf("[format:%d seq:%d] %s", e.Format, e.Seq, e.Err.Error())
}
func (e *ErrorDataFormat) Error() string {
return fmt.Sprintf("Unknown data format %v", e.dataformat)
func (e *FlowError) Unwrap() error {
return e.Err
}
type ErrorIPVersion struct {
version uint32
type RecordError struct {
DataFormat uint32
Err error
}
func NewErrorIPVersion(version uint32) *ErrorIPVersion {
return &ErrorIPVersion{
version: version,
}
func (e *RecordError) Error() string {
return fmt.Sprintf("[data-format:%d] %s", e.DataFormat, e.Err.Error())
}
func (e *ErrorIPVersion) Error() string {
return fmt.Sprintf("Unknown IP version: %v", e.version)
}
type ErrorVersion struct {
version uint32
}
func NewErrorVersion(version uint32) *ErrorVersion {
return &ErrorVersion{
version: version,
}
}
func (e *ErrorVersion) Error() string {
return fmt.Sprintf("Unknown sFlow version %v (supported v5)", e.version)
}
func DecodeCounterRecord(header *RecordHeader, payload *bytes.Buffer) (CounterRecord, error) {
counterRecord := CounterRecord{
Header: *header,
}
switch (*header).DataFormat {
case 1:
ifCounters := IfCounters{}
err := utils.BinaryDecoder(payload, &ifCounters)
if err != nil {
return counterRecord, err
}
counterRecord.Data = ifCounters
case 2:
ethernetCounters := EthernetCounters{}
err := utils.BinaryDecoder(payload, &ethernetCounters)
if err != nil {
return counterRecord, err
}
counterRecord.Data = ethernetCounters
default:
counterRecord.Data = &FlowRecordRaw{
Data: payload.Next(int(header.Length)),
}
}
return counterRecord, nil
func (e *RecordError) Unwrap() error {
return e.Err
}
func DecodeIP(payload *bytes.Buffer) (uint32, []byte, error) {
var ipVersion uint32
err := utils.BinaryDecoder(payload, &ipVersion)
if err != nil {
return 0, nil, err
if err := utils.BinaryDecoder(payload, &ipVersion); err != nil {
return 0, nil, fmt.Errorf("DecodeIP: [%w]", err)
}
var ip []byte
if ipVersion == 1 {
@@ -114,173 +67,181 @@ func DecodeIP(payload *bytes.Buffer) (uint32, []byte, error) {
} else if ipVersion == 2 {
ip = make([]byte, 16)
} else {
return ipVersion, ip, NewErrorIPVersion(ipVersion)
return ipVersion, ip, fmt.Errorf("DecodeIP: unknown IP version %d", ipVersion)
}
if payload.Len() >= len(ip) {
err := utils.BinaryDecoder(payload, &ip)
if err != nil {
return 0, nil, err
if err := utils.BinaryDecoder(payload, &ip); err != nil {
return 0, nil, fmt.Errorf("DecodeIP: [%w]", err)
}
} else {
return ipVersion, ip, NewErrorDecodingSFlow(fmt.Sprintf("Not enough data: %v, needs %v.", payload.Len(), len(ip)))
return ipVersion, ip, fmt.Errorf("DecodeIP: truncated data (need %d, got %d)", len(ip), payload.Len())
}
return ipVersion, ip, nil
}
func DecodeCounterRecord(header *RecordHeader, payload *bytes.Buffer) (CounterRecord, error) {
counterRecord := CounterRecord{
Header: *header,
}
switch header.DataFormat {
case 1:
var ifCounters IfCounters
if err := utils.BinaryDecoder(payload, &ifCounters); err != nil {
return counterRecord, &RecordError{header.DataFormat, err}
}
counterRecord.Data = ifCounters
case 2:
var ethernetCounters EthernetCounters
if err := utils.BinaryDecoder(payload, &ethernetCounters); err != nil {
return counterRecord, &RecordError{header.DataFormat, err}
}
counterRecord.Data = ethernetCounters
default:
var rawRecord RawRecord
rawRecord.Data = payload.Bytes()
counterRecord.Data = rawRecord
}
return counterRecord, nil
}
func DecodeFlowRecord(header *RecordHeader, payload *bytes.Buffer) (FlowRecord, error) {
flowRecord := FlowRecord{
Header: *header,
}
switch (*header).DataFormat {
var err error
switch header.DataFormat {
case FORMAT_EXT_SWITCH:
extendedSwitch := ExtendedSwitch{}
err := utils.BinaryDecoder(payload, &extendedSwitch)
if err != nil {
return flowRecord, err
return flowRecord, &RecordError{header.DataFormat, err}
}
flowRecord.Data = extendedSwitch
case FORMAT_RAW_PKT:
sampledHeader := SampledHeader{}
err := utils.BinaryDecoder(payload, &(sampledHeader.Protocol), &(sampledHeader.FrameLength), &(sampledHeader.Stripped), &(sampledHeader.OriginalLength))
if err != nil {
return flowRecord, err
if err := utils.BinaryDecoder(payload,
&sampledHeader.Protocol,
&sampledHeader.FrameLength,
&sampledHeader.Stripped,
&sampledHeader.OriginalLength); err != nil {
return flowRecord, &RecordError{header.DataFormat, err}
}
sampledHeader.HeaderData = payload.Bytes()
flowRecord.Data = sampledHeader
case FORMAT_IPV4:
sampledIPBase := SampledIP_Base{
SrcIP: make([]byte, 4),
DstIP: make([]byte, 4),
sampledIP := SampledIPv4{
SampledIPBase: SampledIPBase{
SrcIP: make([]byte, 4),
DstIP: make([]byte, 4),
},
}
err := utils.BinaryDecoder(payload, &sampledIPBase)
if err != nil {
return flowRecord, err
if err := utils.BinaryDecoder(payload, &sampledIP.SampledIPBase, &sampledIP.Tos); err != nil {
return flowRecord, &RecordError{header.DataFormat, err}
}
sampledIPv4 := SampledIPv4{
Base: sampledIPBase,
}
err = utils.BinaryDecoder(payload, &(sampledIPv4.Tos))
if err != nil {
return flowRecord, err
}
flowRecord.Data = sampledIPv4
flowRecord.Data = sampledIP
case FORMAT_IPV6:
sampledIPBase := SampledIP_Base{
SrcIP: make([]byte, 16),
DstIP: make([]byte, 16),
sampledIP := SampledIPv6{
SampledIPBase: SampledIPBase{
SrcIP: make([]byte, 16),
DstIP: make([]byte, 16),
},
}
err := utils.BinaryDecoder(payload, &sampledIPBase)
if err != nil {
return flowRecord, err
if err := utils.BinaryDecoder(payload, &sampledIP.SampledIPBase, &sampledIP.Priority); err != nil {
return flowRecord, &RecordError{header.DataFormat, err}
}
sampledIPv6 := SampledIPv6{
Base: sampledIPBase,
}
err = utils.BinaryDecoder(payload, &(sampledIPv6.Priority))
if err != nil {
return flowRecord, err
}
flowRecord.Data = sampledIPv6
flowRecord.Data = sampledIP
case FORMAT_EXT_ROUTER:
extendedRouter := ExtendedRouter{}
ipVersion, ip, err := DecodeIP(payload)
if err != nil {
return flowRecord, err
if extendedRouter.NextHopIPVersion, extendedRouter.NextHop, err = DecodeIP(payload); err != nil {
return flowRecord, &RecordError{header.DataFormat, err}
}
extendedRouter.NextHopIPVersion = ipVersion
extendedRouter.NextHop = ip
err = utils.BinaryDecoder(payload, &(extendedRouter.SrcMaskLen), &(extendedRouter.DstMaskLen))
if err != nil {
return flowRecord, err
if err := utils.BinaryDecoder(payload, &extendedRouter.SrcMaskLen, &extendedRouter.DstMaskLen); err != nil {
return flowRecord, &RecordError{header.DataFormat, err}
}
flowRecord.Data = extendedRouter
case FORMAT_EXT_GATEWAY:
extendedGateway := ExtendedGateway{}
ipVersion, ip, err := DecodeIP(payload)
if err != nil {
return flowRecord, err
if extendedGateway.NextHopIPVersion, extendedGateway.NextHop, err = DecodeIP(payload); err != nil {
return flowRecord, &RecordError{header.DataFormat, err}
}
extendedGateway.NextHopIPVersion = ipVersion
extendedGateway.NextHop = ip
err = utils.BinaryDecoder(payload, &(extendedGateway.AS), &(extendedGateway.SrcAS), &(extendedGateway.SrcPeerAS),
&(extendedGateway.ASDestinations))
if err != nil {
return flowRecord, err
if err := utils.BinaryDecoder(payload, &extendedGateway.AS, &extendedGateway.SrcAS, &extendedGateway.SrcPeerAS,
&extendedGateway.ASDestinations); err != nil {
return flowRecord, &RecordError{header.DataFormat, err}
}
var asPath []uint32
if extendedGateway.ASDestinations != 0 {
err := utils.BinaryDecoder(payload, &(extendedGateway.ASPathType), &(extendedGateway.ASPathLength))
if err != nil {
return flowRecord, err
if err := utils.BinaryDecoder(payload, &extendedGateway.ASPathType, &extendedGateway.ASPathLength); err != nil {
return flowRecord, &RecordError{header.DataFormat, err}
}
// protection for as-path length
if extendedGateway.ASPathLength > 1000 {
return flowRecord, &RecordError{header.DataFormat, fmt.Errorf("as-path length of %d seems quite large", extendedGateway.ASPathLength)}
}
if int(extendedGateway.ASPathLength) > payload.Len()-4 {
return flowRecord, errors.New(fmt.Sprintf("Invalid AS path length: %v.", extendedGateway.ASPathLength))
return flowRecord, &RecordError{header.DataFormat, fmt.Errorf("invalid AS path length: %d", extendedGateway.ASPathLength)}
}
asPath = make([]uint32, extendedGateway.ASPathLength)
asPath = make([]uint32, extendedGateway.ASPathLength) // max size of 1000 for protection
if len(asPath) > 0 {
err = utils.BinaryDecoder(payload, asPath)
if err != nil {
return flowRecord, err
if err := utils.BinaryDecoder(payload, asPath); err != nil {
return flowRecord, &RecordError{header.DataFormat, err}
}
}
}
extendedGateway.ASPath = asPath
err = utils.BinaryDecoder(payload, &(extendedGateway.CommunitiesLength))
if err != nil {
return flowRecord, err
if err := utils.BinaryDecoder(payload, &extendedGateway.CommunitiesLength); err != nil {
return flowRecord, &RecordError{header.DataFormat, err}
}
// protection for communities length
if extendedGateway.CommunitiesLength > 1000 {
return flowRecord, &RecordError{header.DataFormat, fmt.Errorf("communities length of %d seems quite large", extendedGateway.ASPathLength)}
}
if int(extendedGateway.CommunitiesLength) > payload.Len()-4 {
return flowRecord, errors.New(fmt.Sprintf("Invalid Communities length: %v.", extendedGateway.ASPathLength))
return flowRecord, &RecordError{header.DataFormat, fmt.Errorf("invalid communities length: %d", extendedGateway.ASPathLength)}
}
communities := make([]uint32, extendedGateway.CommunitiesLength)
communities := make([]uint32, extendedGateway.CommunitiesLength) // max size of 1000 for protection
if len(communities) > 0 {
err = utils.BinaryDecoder(payload, communities)
if err != nil {
return flowRecord, err
if err := utils.BinaryDecoder(payload, communities); err != nil {
return flowRecord, &RecordError{header.DataFormat, err}
}
}
err = utils.BinaryDecoder(payload, &(extendedGateway.LocalPref))
if err != nil {
return flowRecord, err
if err := utils.BinaryDecoder(payload, &extendedGateway.LocalPref); err != nil {
return flowRecord, &RecordError{header.DataFormat, err}
}
extendedGateway.Communities = communities
flowRecord.Data = extendedGateway
default:
//return flowRecord, errors.New(fmt.Sprintf("Unknown data format %v.", (*header).DataFormat))
flowRecord.Data = &FlowRecordRaw{
Data: payload.Next(int(header.Length)),
}
var rawRecord RawRecord
rawRecord.Data = payload.Bytes()
flowRecord.Data = rawRecord
}
return flowRecord, nil
}
func DecodeSample(header *SampleHeader, payload *bytes.Buffer) (interface{}, error) {
format := (*header).Format
format := header.Format
var sample interface{}
err := utils.BinaryDecoder(payload, &((*header).SampleSequenceNumber))
if err != nil {
return sample, err
if err := utils.BinaryDecoder(payload, &header.SampleSequenceNumber); err != nil {
return sample, fmt.Errorf("header seq [%w]", err)
}
seq := header.SampleSequenceNumber
if format == FORMAT_RAW_PKT || format == FORMAT_ETH {
var sourceId uint32
err = utils.BinaryDecoder(payload, &sourceId)
if err != nil {
return sample, err
if err := utils.BinaryDecoder(payload, &sourceId); err != nil {
return sample, &FlowError{format, seq, fmt.Errorf("header source [%w]", err)}
}
(*header).SourceIdType = sourceId >> 24
(*header).SourceIdValue = sourceId & 0x00ffffff
header.SourceIdType = sourceId >> 24
header.SourceIdValue = sourceId & 0x00ffffff
} else if format == FORMAT_IPV4 || format == FORMAT_IPV6 {
err = utils.BinaryDecoder(payload, &((*header).SourceIdType), &((*header).SourceIdValue))
if err != nil {
return sample, err
if err := utils.BinaryDecoder(payload, &header.SourceIdType, &header.SourceIdValue); err != nil {
return sample, &FlowError{format, seq, fmt.Errorf("header source [%w]", err)}
}
} else {
return nil, NewErrorDataFormat(format)
return sample, &FlowError{format, seq, fmt.Errorf("unknown format %d", format)}
}
var recordsCount uint32
@@ -291,34 +252,43 @@ func DecodeSample(header *SampleHeader, payload *bytes.Buffer) (interface{}, err
flowSample = FlowSample{
Header: *header,
}
err = utils.BinaryDecoder(payload, &(flowSample.SamplingRate), &(flowSample.SamplePool),
&(flowSample.Drops), &(flowSample.Input), &(flowSample.Output), &(flowSample.FlowRecordsCount))
if err != nil {
return sample, err
if err := utils.BinaryDecoder(payload, &flowSample.SamplingRate, &flowSample.SamplePool,
&flowSample.Drops, &flowSample.Input, &flowSample.Output, &flowSample.FlowRecordsCount); err != nil {
return sample, &FlowError{format, seq, fmt.Errorf("raw [%w]", err)}
}
recordsCount = flowSample.FlowRecordsCount
flowSample.Records = make([]FlowRecord, recordsCount)
if recordsCount > 1000 { // protection against ddos
return sample, &FlowError{format, seq, fmt.Errorf("too many flow records: %d", recordsCount)}
}
flowSample.Records = make([]FlowRecord, recordsCount) // max size of 1000 for protection
sample = flowSample
} else if format == FORMAT_ETH || format == FORMAT_IPV6 {
err = utils.BinaryDecoder(payload, &recordsCount)
if err != nil {
return sample, err
if err := utils.BinaryDecoder(payload, &recordsCount); err != nil {
return sample, &FlowError{format, seq, fmt.Errorf("eth [%w]", err)}
}
if recordsCount > 1000 { // protection against ddos
return sample, &FlowError{format, seq, fmt.Errorf("too many flow records: %d", recordsCount)}
}
counterSample = CounterSample{
Header: *header,
CounterRecordsCount: recordsCount,
}
counterSample.Records = make([]CounterRecord, recordsCount)
counterSample.Records = make([]CounterRecord, recordsCount) // max size of 1000 for protection
sample = counterSample
} else if format == FORMAT_IPV4 {
expandedFlowSample = ExpandedFlowSample{
Header: *header,
}
err = utils.BinaryDecoder(payload, &(expandedFlowSample.SamplingRate), &(expandedFlowSample.SamplePool),
&(expandedFlowSample.Drops), &(expandedFlowSample.InputIfFormat), &(expandedFlowSample.InputIfValue),
&(expandedFlowSample.OutputIfFormat), &(expandedFlowSample.OutputIfValue), &(expandedFlowSample.FlowRecordsCount))
if err != nil {
return sample, err
if err := utils.BinaryDecoder(payload,
&expandedFlowSample.SamplingRate,
&expandedFlowSample.SamplePool,
&expandedFlowSample.Drops,
&expandedFlowSample.InputIfFormat,
&expandedFlowSample.InputIfValue,
&expandedFlowSample.OutputIfFormat,
&expandedFlowSample.OutputIfValue,
&expandedFlowSample.FlowRecordsCount); err != nil {
return sample, &FlowError{format, seq, fmt.Errorf("IPv4 [%w]", err)}
}
recordsCount = expandedFlowSample.FlowRecordsCount
expandedFlowSample.Records = make([]FlowRecord, recordsCount)
@@ -326,9 +296,8 @@ func DecodeSample(header *SampleHeader, payload *bytes.Buffer) (interface{}, err
}
for i := 0; i < int(recordsCount) && payload.Len() >= 8; i++ {
recordHeader := RecordHeader{}
err = utils.BinaryDecoder(payload, &(recordHeader.DataFormat), &(recordHeader.Length))
if err != nil {
return sample, err
if err := utils.BinaryDecoder(payload, &recordHeader.DataFormat, &recordHeader.Length); err != nil {
return sample, &FlowError{format, seq, fmt.Errorf("record header [%w]", err)}
}
if int(recordHeader.Length) > payload.Len() {
break
@@ -337,7 +306,7 @@ func DecodeSample(header *SampleHeader, payload *bytes.Buffer) (interface{}, err
if format == FORMAT_RAW_PKT || format == FORMAT_IPV4 {
record, err := DecodeFlowRecord(&recordHeader, recordReader)
if err != nil {
continue
return sample, &FlowError{format, seq, fmt.Errorf("record [%w]", err)}
}
if format == FORMAT_RAW_PKT {
flowSample.Records[i] = record
@@ -347,7 +316,7 @@ func DecodeSample(header *SampleHeader, payload *bytes.Buffer) (interface{}, err
} else if format == FORMAT_ETH || format == FORMAT_IPV6 {
record, err := DecodeCounterRecord(&recordHeader, recordReader)
if err != nil {
continue
return sample, &FlowError{format, seq, fmt.Errorf("counter [%w]", err)}
}
counterSample.Records[i] = record
}
@@ -355,63 +324,68 @@ func DecodeSample(header *SampleHeader, payload *bytes.Buffer) (interface{}, err
return sample, nil
}
func DecodeMessage(payload *bytes.Buffer) (interface{}, error) {
func DecodeMessageVersion(payload *bytes.Buffer, packetV5 *Packet) error {
var version uint32
err := utils.BinaryDecoder(payload, &version)
if err != nil {
return nil, err
if err := utils.BinaryDecoder(payload, &version); err != nil {
return &DecoderError{err}
}
packetV5 := Packet{}
if version == 5 {
packetV5.Version = version
err = utils.BinaryDecoder(payload, &(packetV5.IPVersion))
if err != nil {
return packetV5, err
}
var ip []byte
if packetV5.IPVersion == 1 {
ip = make([]byte, 4)
err = utils.BinaryDecoder(payload, ip)
if err != nil {
return packetV5, err
}
} else if packetV5.IPVersion == 2 {
ip = make([]byte, 16)
err = utils.BinaryDecoder(payload, ip)
if err != nil {
return packetV5, err
}
} else {
return nil, NewErrorIPVersion(packetV5.IPVersion)
}
packetV5.Version = version
packetV5.AgentIP = ip
err = utils.BinaryDecoder(payload, &(packetV5.SubAgentId), &(packetV5.SequenceNumber), &(packetV5.Uptime), &(packetV5.SamplesCount))
if err != nil {
return packetV5, err
}
packetV5.Samples = make([]interface{}, int(packetV5.SamplesCount))
for i := 0; i < int(packetV5.SamplesCount) && payload.Len() >= 8; i++ {
header := SampleHeader{}
err = utils.BinaryDecoder(payload, &(header.Format), &(header.Length))
if err != nil {
return packetV5, err
}
if int(header.Length) > payload.Len() {
break
}
sampleReader := bytes.NewBuffer(payload.Next(int(header.Length)))
sample, err := DecodeSample(&header, sampleReader)
if err != nil {
continue
} else {
packetV5.Samples[i] = sample
}
}
return packetV5, nil
} else {
return nil, NewErrorVersion(version)
if version != 5 {
return &DecoderError{fmt.Errorf("unknown version %d", version)}
}
return DecodeMessage(payload, packetV5)
}
func DecodeMessage(payload *bytes.Buffer, packetV5 *Packet) error {
if err := utils.BinaryDecoder(payload, &packetV5.IPVersion); err != nil {
return &DecoderError{err}
}
var ip []byte
if packetV5.IPVersion == 1 {
ip = make([]byte, 4)
if err := utils.BinaryDecoder(payload, ip); err != nil {
return &DecoderError{fmt.Errorf("IPv4 [%w]", err)}
}
} else if packetV5.IPVersion == 2 {
ip = make([]byte, 16)
if err := utils.BinaryDecoder(payload, ip); err != nil {
return &DecoderError{fmt.Errorf("IPv6 [%w]", err)}
}
} else {
return &DecoderError{fmt.Errorf("unknown IP version %d", packetV5.IPVersion)}
}
packetV5.AgentIP = ip
if err := utils.BinaryDecoder(payload,
&packetV5.SubAgentId,
&packetV5.SequenceNumber,
&packetV5.Uptime,
&packetV5.SamplesCount); err != nil {
return &DecoderError{err}
}
if packetV5.SamplesCount > 1000 {
return &DecoderError{fmt.Errorf("too many samples: %d", packetV5.SamplesCount)}
}
packetV5.Samples = make([]interface{}, int(packetV5.SamplesCount)) // max size of 1000 for protection
for i := 0; i < int(packetV5.SamplesCount) && payload.Len() >= 8; i++ {
header := SampleHeader{}
if err := utils.BinaryDecoder(payload, &header.Format, &header.Length); err != nil {
return &DecoderError{fmt.Errorf("header [%w]", err)}
}
if int(header.Length) > payload.Len() {
break
}
sampleReader := bytes.NewBuffer(payload.Next(int(header.Length)))
sample, err := DecodeSample(&header, sampleReader)
if err != nil {
return &DecoderError{fmt.Errorf("sample [%w]", err)}
} else {
packetV5.Samples[i] = sample
}
}
return nil
}

View File

@@ -22,16 +22,16 @@ func TestSFlowDecode(t *testing.T) {
0x77, 0x61, 0x62, 0x63, 0x64, 0x65, 0x66, 0x67, 0x68, 0x69, 0x00, 0x00,
}
buf := bytes.NewBuffer(data)
_, err := DecodeMessage(buf)
assert.Nil(t, err)
var packet Packet
assert.Nil(t, DecodeMessageVersion(buf, &packet))
}
func TestExpandedSFlowDecode(t *testing.T) {
data := getExpandedSFlowDecode()
buf := bytes.NewBuffer(data)
_, err := DecodeMessage(buf)
assert.Nil(t, err)
var packet Packet
assert.Nil(t, DecodeMessageVersion(buf, &packet))
}
func getExpandedSFlowDecode() []byte {

View File

@@ -1,6 +1,6 @@
# Protocols
You can find information on the protocols in the links below:
You can find information on the protocols:
* [sFlow](https://sflow.org/developers/specifications.php)
* [NetFlow v5](https://www.cisco.com/c/en/us/td/docs/net_mgmt/netflow_collection_engine/3-6/user/guide/format.html)
* [NetFlow v9](https://www.cisco.com/en/US/technologies/tk648/tk362/technologies_white_paper09186a00800a3db9.html)
@@ -11,101 +11,240 @@ The mapping to the protobuf format is listed in the table below.
| Field | Description | NetFlow v5 | sFlow | NetFlow v9 | IPFIX |
| - | - | - | - | - | - |
|Type|Type of flow message|NETFLOW_V5|SFLOW_5|NETFLOW_V9|IPFIX|
|TimeReceived|Timestamp of when the message was received|Included|Included|Included|Included|
|SequenceNum|Sequence number of the flow packet|Included|Included|Included|Included|
|SamplingRate|Sampling rate of the flow|Included|Included|Included|Included|
|FlowDirection|Direction of the flow| | |DIRECTION (61)|flowDirection (61)|
|SamplerAddress|Address of the device that generated the packet|IP source of packet|Agent IP|IP source of packet|IP source of packet|
|TimeFlowStart|Time the flow started|System uptime and first|=TimeReceived|System uptime and FIRST_SWITCHED (22)|flowStartXXX (150, 152, 154, 156)|
|TimeFlowEnd|Time the flow ended|System uptime and last|=TimeReceived|System uptime and LAST_SWITCHED (23)|flowEndXXX (151, 153, 155, 157)|
|Bytes|Number of bytes in flow|dOctets|Length of sample|IN_BYTES (1) OUT_BYTES (23)|octetDeltaCount (1) postOctetDeltaCount (23)|
|Packets|Number of packets in flow|dPkts|=1|IN_PKTS (2) OUT_PKTS (24)|packetDeltaCount (1) postPacketDeltaCount (24)|
|SrcAddr|Source address (IP)|srcaddr (IPv4 only)|Included|Included|IPV4_SRC_ADDR (8) IPV6_SRC_ADDR (27)|sourceIPv4Address/sourceIPv6Address (8/27)|
|DstAddr|Destination address (IP)|dstaddr (IPv4 only)|Included|Included|IPV4_DST_ADDR (12) IPV6_DST_ADDR (28)|destinationIPv4Address (12)destinationIPv6Address (28)|
|Etype|Ethernet type (0x86dd for IPv6...)|IPv4|Included|Included|Included|
|Proto|Protocol (UDP, TCP, ICMP...)|prot|Included|PROTOCOL (4)|protocolIdentifier (4)|
|SrcPort|Source port (when UDP/TCP/SCTP)|srcport|Included|L4_SRC_PORT (7)|sourceTransportPort (7)|
|DstPort|Destination port (when UDP/TCP/SCTP)|dstport|Included|L4_DST_PORT (11)|destinationTransportPort (11)|
|InIf|Input interface|input|Included|INPUT_SNMP (10)|ingressInterface (10)|
|OutIf|Output interface|output|Included|OUTPUT_SNMP (14)|egressInterface (14)|
|SrcMac|Source mac address| |Included|IN_SRC_MAC (56)|sourceMacAddress (56)|
|DstMac|Destination mac address| |Included|OUT_DST_MAC (57)|postDestinationMacAddress (57)|
|SrcVlan|Source VLAN ID| |From ExtendedSwitch|SRC_VLAN (58)|vlanId (58)|
|DstVlan|Destination VLAN ID| |From ExtendedSwitch|DST_VLAN (59)|postVlanId (59)|
|VlanId|802.11q VLAN ID| |Included|SRC_VLAN (58)|vlanId (58)|
|IngressVrfID|VRF ID| | | |ingressVRFID (234)|
|EgressVrfID|VRF ID| | | |egressVRFID (235)|
|IPTos|IP Type of Service|tos|Included|SRC_TOS (5)|ipClassOfService (5)|
|ForwardingStatus|Forwarding status| | |FORWARDING_STATUS (89)|forwardingStatus (89)|
|IPTTL|IP Time to Live| |Included|IPTTL (52)|minimumTTL (52|
|TCPFlags|TCP flags|tcp_flags|Included|TCP_FLAGS (6)|tcpControlBits (6)|
|IcmpType|ICMP Type| |Included|ICMP_TYPE (32)|icmpTypeXXX (176, 178) icmpTypeCodeXXX (32, 139)|
|IcmpCode|ICMP Code| |Included|ICMP_TYPE (32)|icmpCodeXXX (177, 179) icmpTypeCodeXXX (32, 139)|
|IPv6FlowLabel|IPv6 Flow Label| |Included|IPV6_FLOW_LABEL (31)|flowLabelIPv6 (31)|
|FragmentId|IP Fragment ID| |Included|IPV4_IDENT (54)|fragmentIdentification (54)|
|FragmentOffset|IP Fragment Offset| |Included|FRAGMENT_OFFSET (88)|fragmentOffset (88) and fragmentFlags (197)|
|BiFlowDirection|BiFlow Identification| | | |biflowDirection (239)|
|SrcAS|Source AS number|src_as|From ExtendedGateway|SRC_AS (16)|bgpSourceAsNumber (16)|
|DstAS|Destination AS number|dst_as|From ExtendedGateway|DST_AS (17)|bgpDestinationAsNumber (17)|
|NextHop|Nexthop address|nexthop|From ExtendedRouter|IPV4_NEXT_HOP (15) IPV6_NEXT_HOP (62)|ipNextHopIPv4Address (15) ipNextHopIPv6Address (62)|
|NextHopAS|Nexthop AS number| |From ExtendedGateway| | |
|SrcNet|Source address mask|src_mask|From ExtendedRouter|SRC_MASK (9) IPV6_SRC_MASK (29)|sourceIPv4PrefixLength (9) sourceIPv6PrefixLength (29)|
|DstNet|Destination address mask|dst_mask|From ExtendedRouter|DST_MASK (13) IPV6_DST_MASK (30)|destinationIPv4PrefixLength (13) destinationIPv6PrefixLength (30)|
|BgpNextHop|BGP Nexthop address| |From ExtendedGateway|BGP_IPV4_NEXT_HOP (18) BGP_IPV6_NEXT_HOP (63)|bgpNextHopIPv4Address (18) bgpNextHopIPv6Address (63)|
|BgpCommunities|BGP Communities| |From ExtendedGateway| | |
|ASPath|AS Path| |From ExtendedGateway| | |
|SrcNet|Source address mask|src_mask|From ExtendedRouter|SRC_MASK (9) IPV6_SRC_MASK (29)|sourceIPv4PrefixLength (9) sourceIPv6PrefixLength (29)|
|DstNet|Destination address mask|dst_mask|From ExtendedRouter|DST_MASK (13) IPV6_DST_MASK (30)|destinationIPv4PrefixLength (13) destinationIPv6PrefixLength (30)|
|HasMPLS|Indicates the presence of MPLS header||Included|||
|MPLSCount|Count of MPLS layers||Included|||
|MPLSxTTL|TTL of the MPLS label||Included|||
|MPLSxLabel|MPLS label||Included|||
|time_received_ns|Timestamp in nanoseconds of when the message was received|Included|Included|Included|Included|
|sequence_num|Sequence number of the flow packet|Included|Included|Included|Included|
|sampling_rate|Sampling rate of the flow|Included|Included|Included|Included|
|sampler_address|Address of the device that generated the packet|IP source of packet|Agent IP|IP source of packet|IP source of packet|
|time_flow_start_ns|Time the flow started in nanoseconds|System uptime and first|=TimeReceived|System uptime and FIRST_SWITCHED (22)|flowStartXXX (150, 152, 154, 156)|
|time_flow_end_ns|Time the flow ended in nanoseconds|System uptime and last|=TimeReceived|System uptime and LAST_SWITCHED (23)|flowEndXXX (151, 153, 155, 157)|
|bytes|Number of bytes in flow|dOctets|Length of sample|IN_BYTES (1) OUT_BYTES (23)|octetDeltaCount (1) postOctetDeltaCount (23)|
|packets|Number of packets in flow|dPkts|=1|IN_PKTS (2) OUT_PKTS (24)|packetDeltaCount (1) postPacketDeltaCount (24)|
|src_addr|Source address (IP)|srcaddr (IPv4 only)|Included|Included|IPV4_SRC_ADDR (8) IPV6_SRC_ADDR (27)|sourceIPv4Address/sourceIPv6Address (8/27)|
|dst_addr|Destination address (IP)|dstaddr (IPv4 only)|Included|Included|IPV4_DST_ADDR (12) IPV6_DST_ADDR (28)|destinationIPv4Address (12)destinationIPv6Address (28)|
|etype|Ethernet type (0x86dd for IPv6...)|IPv4|Included|Included|Included|
|proto|Protocol (UDP, TCP, ICMP...)|prot|Included|PROTOCOL (4)|protocolIdentifier (4)|
|src_port|Source port (when UDP/TCP/SCTP)|srcport|Included|L4_SRC_PORT (7)|sourceTransportPort (7)|
|dst_port|Destination port (when UDP/TCP/SCTP)|dstport|Included|L4_DST_PORT (11)|destinationTransportPort (11)|
|in_if|Input interface|input|Included|INPUT_SNMP (10)|ingressInterface (10)|
|out_if|Output interface|output|Included|OUTPUT_SNMP (14)|egressInterface (14)|
|src_mac|Source mac address| |Included|IN_SRC_MAC (56)|sourceMacAddress (56)|
|dst_mac|Destination mac address| |Included|OUT_DST_MAC (57)|postDestinationMacAddress (57)|
|src_vlan|Source VLAN ID| |From ExtendedSwitch|SRC_VLAN (58)|vlanId (58)|
|dst_vlan|Destination VLAN ID| |From ExtendedSwitch|DST_VLAN (59)|postVlanId (59)|
|vlan_id|802.11q VLAN ID| |Included|SRC_VLAN (58)|vlanId (58)|
|ip_tos|IP Type of Service|tos|Included|SRC_TOS (5)|ipClassOfService (5)|
|forwarding_status|Forwarding status| | |FORWARDING_STATUS (89)|forwardingStatus (89)|
|ip_ttl|IP Time to Live| |Included|IPTTL (52)|minimumTTL (52|
|tcp_flags|TCP flags|tcp_flags|Included|TCP_FLAGS (6)|tcpControlBits (6)|
|icmp_type|ICMP Type| |Included|ICMP_TYPE (32)|icmpTypeXXX (176, 178) icmpTypeCodeXXX (32, 139)|
|icmp_code|ICMP Code| |Included|ICMP_TYPE (32)|icmpCodeXXX (177, 179) icmpTypeCodeXXX (32, 139)|
|ipv6_flow_label|IPv6 Flow Label| |Included|IPV6_FLOW_LABEL (31)|flowLabelIPv6 (31)|
|fragment_id|IP Fragment ID| |Included|IPV4_IDENT (54)|fragmentIdentification (54)|
|fragment_offset|IP Fragment Offset| |Included|FRAGMENT_OFFSET (88)|fragmentOffset (88) and fragmentFlags (197)|
|src_as|Source AS number|src_as|From ExtendedGateway|SRC_AS (16)|bgpSourceAsNumber (16)|
|dst_as|Destination AS number|dst_as|From ExtendedGateway|DST_AS (17)|bgpDestinationAsNumber (17)|
|next_hop|Nexthop address|nexthop|From ExtendedRouter|IPV4_NEXT_HOP (15) IPV6_NEXT_HOP (62)|ipNextHopIPv4Address (15) ipNextHopIPv6Address (62)|
|next_hop_as|Nexthop AS number| |From ExtendedGateway| | |
|src_net|Source address mask|src_mask|From ExtendedRouter|SRC_MASK (9) IPV6_SRC_MASK (29)|sourceIPv4PrefixLength (9) sourceIPv6PrefixLength (29)|
|dst_net|Destination address mask|dst_mask|From ExtendedRouter|DST_MASK (13) IPV6_DST_MASK (30)|destinationIPv4PrefixLength (13) destinationIPv6PrefixLength (30)|
|bgp_next_hop|BGP Nexthop address| |From ExtendedGateway|BGP_IPV4_NEXT_HOP (18) BGP_IPV6_NEXT_HOP (63)|bgpNextHopIPv4Address (18) bgpNextHopIPv6Address (63)|
|bgp_communities|BGP Communities| |From ExtendedGateway| | |
|as_path|AS Path| |From ExtendedGateway| | |destinationIPv6PrefixLength (30)|
|mpls_ttl|TTL of the MPLS label||Included|||
|mpls_label|MPLS label list||Included|||
## Add new custom fields
## Producers
If you are using enterprise fields that you need decoded
or if you are looking for specific bytes inside the packet sample.
When using the **raw** producer, you can access a sample:
This feature is only available when sending Protobufs (no text output).
```bash
$ go run main.go -produce raw -format json
```
The [`mapping.yaml`](../cmd/goflow2/mapping.yaml) example file
will collect source and destination port again, use it with `-mapping=mapping.yaml` in the CLI.
This can be useful if you need to debug received packets
or looking to dive into a specific protocol (eg: the sFlow counters).
```json
{
"type": "sflow",
"message":
{
"version": 5,
"ip-version": 1,
"agent-ip": "127.0.0.1",
"sub-agent-id": 100000,
"sequence-number": 1234,
"uptime": 19070720,
"samples-count": 1,
"samples":
[
{
"header":
{
"format": 2,
"length": 124,
"sample-sequence-number": 340,
"source-id-type": 0,
"source-id-value": 6
},
"counter-records-count": 1,
"records":
[
{
"header":
{
"data-format": 1,
"length": 88
},
"data":
{
"if-index": 6,
"if-type": 6,
"if-speed": 0,
"if-direction": 0,
"if-status": 3,
"if-in-octets": 0,
"if-in-ucast-pkts": 1000,
"if-in-multicast-pkts": 0,
"if-in-broadcast-pkts": 0,
"if-in-discards": 0,
"if-in-errors": 0,
"if-in-unknown-protos": 0,
"if-out-octets": 0,
"if-out-ucast-pkts": 2000,
"if-out-multicast-pkts": 0,
"if-out-broadcast-pkts": 0,
"if-out-discards": 0,
"if-out-errors": 0,
"if-promiscuous-mode": 0
}
}
]
}
]
},
"src": "[::ffff:127.0.0.1]:50001",
"time_received": "2023-04-15T20:44:42.723694Z"
}
```
When using the **Protobuf** producer, you have access to various configuration options.
The [`mapping.yaml`](../cmd/goflow2/mapping.yaml) file can be used with `-mapping=mapping.yaml` in the CLI.
It enables features like:
* Add protobuf fields
* Renaming fields (JSON/text)
* Hashing key (for Kafka)
* Mapping new values from samples
For example, you can rename:
```yaml
formatter:
rename: # only for JSON/text
src_mac: src_macaddr
dst_mac: dst_macaddr
```
### Columns and renderers
By default, all the columns above will be printed when using JSON or text.
To restrict to a subset of columns, in the mapping file, list the ones you want:
```yaml
formatter:
fields:
- src_addr
```
There is a support for virtual columns (eg: `icmp_name`).
Renderers are a special handling of fields:
```yaml
formatter
render:
src_mac: mac
dst_mac: mac
dst_net: none # overrides: render the network as integer instead of prefix based on src/dst addr
```
You can assign a specific formatter.
### Map custom fields
If you are using enterprise fields that you need decoded or if you are looking for specific bytes inside the packet sample.
Data coming from the flows can be added to the protobuf either as an unsigned/signed integer a slice of bytes.
The `sflow` section allow to extract data from packet samples inside sFlow and inside IPFIX (dataframe).
The following layers are available:
* 0: no offset
* 3: network layer, offsets to IP/IPv6 header
* 4: transport layer, offsets to TCP/UDP header
* 3, ipv4, ipv6, arp: network layer, offsets to IP/IPv6 header
* 4, icmp, icmp6, udp, tcp: transport layer, offsets to TCP/UDP/ICMP header
* 7: application layer, offsets to the TCP/UDP payload
The data extracted will then be added to either an existing field (see samping rate below),
or to a newly defined field.
In order to display them with JSON or text, you need to specify them in `fields`.
```yaml
formatter:
fields:
- sampling_rate
- custom_src_port
- juniper_properties
protobuf:
- name: juniper_properties
index: 1001
type: varint
array: true
ipfix:
mapping:
- field: 7 # NetFlow or IPFIX field ID
destination: CustomInteger1 # Name of the field inside the Protobuf
penprovided: false # Has an enterprise number (optional)
pen: 0 # Enterprise number (optional)
- field: 34 # samplingInterval provided within the template
destination: sampling_rate
endian: little # special endianness
- field: 137 # Juniper Properties
destination: juniper_properties
penprovided: true # has an enterprise number
pen: 2636 # Juniper enterprise
netflowv9:
mapping: []
# ... similar to above, Enterprise number will not be supported
# ... similar to above but the enterprise number will not be supported
sflow:
mapping:
- layer: 4 # Layer
mapping: # also inside an IPFIX dataFrame
- layer: "4" # Layer
offset: 0 # Source port
length: 16 # 2 bytes
destination: CustomInteger1
destination: custom_src_port
```
Without editing and recompiling the [protobuf](../pb/flow.proto), you can use up to 5 integers and 5 slices of bytes:
Another example if you wish to decode the TTL from the IP:
```protobuf
// Custom allocations
uint64 CustomInteger1 = 1001;
[...]
bytes CustomBytes1 = 1011;
[...]
```yaml
formatter:
protobuf: # manual protobuf fields addition
- name: egress_vrf_id
index: 40
type: varint
ipfix:
mapping:
- field: 51
destination: ip_ttl_test
netflowv9:
mapping:
- field: 51
destination: ip_ttl_test
sflow:
mapping:
- layer: "ipv4"
offset: 64
length: 8
destination: ip_ttl_test
- layer: "ipv6"
offset: 56
length: 8
destination: ip_ttl_test
```

35
format/binary/binary.go Normal file
View File

@@ -0,0 +1,35 @@
package binary
import (
"encoding"
"github.com/netsampler/goflow2/v2/format"
)
type BinaryDriver struct {
}
func (d *BinaryDriver) Prepare() error {
return nil
}
func (d *BinaryDriver) Init() error {
return nil
}
func (d *BinaryDriver) Format(data interface{}) ([]byte, []byte, error) {
var key []byte
if dataIf, ok := data.(interface{ Key() []byte }); ok {
key = dataIf.Key()
}
if dataIf, ok := data.(encoding.BinaryMarshaler); ok {
text, err := dataIf.MarshalBinary()
return key, text, err
}
return key, nil, format.ErrorNoSerializer
}
func init() {
d := &BinaryDriver{}
format.RegisterFormatDriver("bin", d)
}

View File

@@ -1,56 +0,0 @@
package common
import (
"flag"
"fmt"
"reflect"
"strings"
"sync"
)
var (
fieldsVar string
fields []string // Hashing fields
hashDeclared bool
hashDeclaredLock = &sync.Mutex{}
)
func HashFlag() {
hashDeclaredLock.Lock()
defer hashDeclaredLock.Unlock()
if hashDeclared {
return
}
hashDeclared = true
flag.StringVar(&fieldsVar, "format.hash", "SamplerAddress", "List of fields to do hashing, separated by commas")
}
func ManualHashInit() error {
fields = strings.Split(fieldsVar, ",")
return nil
}
func HashProtoLocal(msg interface{}) string {
return HashProto(fields, msg)
}
func HashProto(fields []string, msg interface{}) string {
var keyStr string
if msg != nil {
vfm := reflect.ValueOf(msg)
vfm = reflect.Indirect(vfm)
for _, kf := range fields {
fieldValue := vfm.FieldByName(kf)
if fieldValue.IsValid() {
keyStr += fmt.Sprintf("%v-", fieldValue)
}
}
}
return keyStr
}

View File

@@ -1,36 +0,0 @@
package common
import (
"flag"
"strings"
"sync"
)
var (
selectorVar string
selector []string // Hashing fields
selectorTag string // Hashing fields
selectorDeclared bool
selectorDeclaredLock = &sync.Mutex{}
)
func SelectorFlag() {
selectorDeclaredLock.Lock()
defer selectorDeclaredLock.Unlock()
if selectorDeclared {
return
}
selectorDeclared = true
flag.StringVar(&selectorVar, "format.selector", "", "List of fields to do keep in output")
flag.StringVar(&selectorTag, "format.tag", "", "Use format tag")
}
func ManualSelectorInit() error {
if selectorVar == "" {
return nil
}
selector = strings.Split(selectorVar, ",")
return nil
}

View File

@@ -1,246 +0,0 @@
package common
import (
"encoding/binary"
"fmt"
"net"
"reflect"
"strings"
)
const (
FORMAT_TYPE_UNKNOWN = iota
FORMAT_TYPE_STRING_FUNC
FORMAT_TYPE_STRING
FORMAT_TYPE_INTEGER
FORMAT_TYPE_IP
FORMAT_TYPE_MAC
FORMAT_TYPE_BYTES
)
var (
EtypeName = map[uint32]string{
0x806: "ARP",
0x800: "IPv4",
0x86dd: "IPv6",
}
ProtoName = map[uint32]string{
1: "ICMP",
6: "TCP",
17: "UDP",
58: "ICMPv6",
132: "SCTP",
}
IcmpTypeName = map[uint32]string{
0: "EchoReply",
3: "DestinationUnreachable",
8: "Echo",
9: "RouterAdvertisement",
10: "RouterSolicitation",
11: "TimeExceeded",
}
Icmp6TypeName = map[uint32]string{
1: "DestinationUnreachable",
2: "PacketTooBig",
3: "TimeExceeded",
128: "EchoRequest",
129: "EchoReply",
133: "RouterSolicitation",
134: "RouterAdvertisement",
}
TextFields = map[string]int{
"Type": FORMAT_TYPE_STRING_FUNC,
"SamplerAddress": FORMAT_TYPE_IP,
"SrcAddr": FORMAT_TYPE_IP,
"DstAddr": FORMAT_TYPE_IP,
"SrcMac": FORMAT_TYPE_MAC,
"DstMac": FORMAT_TYPE_MAC,
"NextHop": FORMAT_TYPE_IP,
"MPLSLabelIP": FORMAT_TYPE_IP,
}
RenderExtras = map[string]RenderExtraFunction{
"EtypeName": RenderExtraFunctionEtypeName,
"ProtoName": RenderExtraFunctionProtoName,
"IcmpName": RenderExtraFunctionIcmpName,
}
)
/*
func AddTextField(name string, jtype int) {
TextFields = append(TextFields, name)
TextFieldsTypes = append(TextFieldsTypes, jtype)
}*/
type RenderExtraFunction func(interface{}) string
func RenderExtraFetchNumbers(msg interface{}, fields []string) []uint64 {
vfm := reflect.ValueOf(msg)
vfm = reflect.Indirect(vfm)
values := make([]uint64, len(fields))
for i, kf := range fields {
fieldValue := vfm.FieldByName(kf)
if fieldValue.IsValid() {
values[i] = fieldValue.Uint()
}
}
return values
}
func RenderExtraFunctionEtypeName(msg interface{}) string {
num := RenderExtraFetchNumbers(msg, []string{"Etype"})
return EtypeName[uint32(num[0])]
}
func RenderExtraFunctionProtoName(msg interface{}) string {
num := RenderExtraFetchNumbers(msg, []string{"Proto"})
return ProtoName[uint32(num[0])]
}
func RenderExtraFunctionIcmpName(msg interface{}) string {
num := RenderExtraFetchNumbers(msg, []string{"Proto", "IcmpCode", "IcmpType"})
return IcmpCodeType(uint32(num[0]), uint32(num[1]), uint32(num[2]))
}
func IcmpCodeType(proto, icmpCode, icmpType uint32) string {
if proto == 1 {
return IcmpTypeName[icmpType]
} else if proto == 58 {
return Icmp6TypeName[icmpType]
}
return ""
}
func RenderIP(addr []byte) string {
if addr == nil || (len(addr) != 4 && len(addr) != 16) {
return ""
}
return net.IP(addr).String()
}
func FormatMessageReflectText(msg interface{}, ext string) string {
return FormatMessageReflectCustom(msg, ext, "", " ", "=", false)
}
func FormatMessageReflectJSON(msg interface{}, ext string) string {
return fmt.Sprintf("{%s}", FormatMessageReflectCustom(msg, ext, "\"", ",", ":", true))
}
func ExtractTag(name, original string, tag reflect.StructTag) string {
lookup, ok := tag.Lookup(name)
if !ok {
return original
}
before, _, _ := strings.Cut(lookup, ",")
return before
}
func FormatMessageReflectCustom(msg interface{}, ext, quotes, sep, sign string, null bool) string {
customSelector := selector
reMap := make(map[string]string)
vfm := reflect.ValueOf(msg)
vfm = reflect.Indirect(vfm)
vft := vfm.Type()
if len(customSelector) == 0 || selectorTag != "" {
/*
// we would need proto v2
msgR := msg.ProtoReflect()
customSelector = make([]string, msgR.Fields().Len())
for i := 0; i<len(customSelector);i++ {
customSelector[i] = msgR.Fields().Get(i).TextName()
}*/
customSelectorTmp := make([]string, vft.NumField())
for i := 0; i < len(customSelectorTmp); i++ {
field := vft.Field(i)
if !field.IsExported() {
continue
}
fieldName := field.Name
if selectorTag != "" {
fieldName = ExtractTag(selectorTag, field.Name, field.Tag)
reMap[fieldName] = field.Name
}
customSelectorTmp[i] = fieldName
}
if len(customSelector) == 0 {
customSelector = customSelectorTmp
}
}
fstr := make([]string, len(customSelector))
var i int
for _, s := range customSelector {
fieldName := s
if fieldNameMap, ok := reMap[fieldName]; ok {
fieldName = fieldNameMap
}
fieldValue := vfm.FieldByName(fieldName)
// todo: replace s by json mapping of protobuf
if fieldValue.IsValid() {
if fieldType, ok := TextFields[fieldName]; ok {
switch fieldType {
case FORMAT_TYPE_STRING_FUNC:
strMethod := fieldValue.MethodByName("String").Call([]reflect.Value{})
fstr[i] = fmt.Sprintf("%s%s%s%s%q", quotes, s, quotes, sign, strMethod[0].String())
case FORMAT_TYPE_STRING:
fstr[i] = fmt.Sprintf("%s%s%s%s%q", quotes, s, quotes, sign, fieldValue.String())
case FORMAT_TYPE_INTEGER:
fstr[i] = fmt.Sprintf("%s%s%s%s%d", quotes, s, quotes, sign, fieldValue.Uint())
case FORMAT_TYPE_IP:
ip := fieldValue.Bytes()
fstr[i] = fmt.Sprintf("%s%s%s%s%q", quotes, s, quotes, sign, RenderIP(ip))
case FORMAT_TYPE_MAC:
mac := make([]byte, 8)
binary.BigEndian.PutUint64(mac, fieldValue.Uint())
fstr[i] = fmt.Sprintf("%s%s%s%s%q", quotes, s, quotes, sign, net.HardwareAddr(mac[2:]).String())
case FORMAT_TYPE_BYTES:
fstr[i] = fmt.Sprintf("%s%s%s%s%.2x", quotes, s, quotes, sign, fieldValue.Bytes())
default:
if null {
fstr[i] = fmt.Sprintf("%s%s%s%snull", quotes, s, quotes, sign)
} else {
}
}
} else if renderer, ok := RenderExtras[fieldName]; ok {
fstr[i] = fmt.Sprintf("%s%s%s%s%q", quotes, s, quotes, sign, renderer(msg))
} else {
// handle specific types here
switch fieldValue.Kind() {
case reflect.String:
fstr[i] = fmt.Sprintf("%s%s%s%s%q", quotes, s, quotes, sign, fieldValue.Interface())
case reflect.Slice:
c := fieldValue.Len()
v := "["
for i := 0; i < c; i++ {
v += fmt.Sprintf("%v", fieldValue.Index(i).Interface())
if i < c-1 {
v += ","
}
}
v += "]"
fstr[i] = fmt.Sprintf("%s%s%s%s%s", quotes, s, quotes, sign, v)
default:
fstr[i] = fmt.Sprintf("%s%s%s%s%v", quotes, s, quotes, sign, fieldValue.Interface())
}
}
i++
}
}
fstr = fstr[0:i]
return strings.Join(fstr, sep)
}

View File

@@ -1,7 +1,6 @@
package format
import (
"context"
"fmt"
"sync"
)
@@ -9,14 +8,28 @@ import (
var (
formatDrivers = make(map[string]FormatDriver)
lock = &sync.RWMutex{}
ErrorFormat = fmt.Errorf("format error")
ErrorNoSerializer = fmt.Errorf("message is not serializable")
)
type DriverFormatError struct {
Driver string
Err error
}
func (e *DriverFormatError) Error() string {
return fmt.Sprintf("%s for %s format", e.Err.Error(), e.Driver)
}
func (e *DriverFormatError) Unwrap() []error {
return []error{ErrorFormat, e.Err}
}
type FormatDriver interface {
Prepare() error // Prepare driver (eg: flag registration)
Init(context.Context) error // Initialize driver (eg: parse keying)
Init() error // Initialize driver (eg: parse keying)
Format(data interface{}) ([]byte, []byte, error) // Send a message
//FormatInterface // set this and remove Format
}
type FormatInterface interface {
@@ -24,11 +37,19 @@ type FormatInterface interface {
}
type Format struct {
driver FormatDriver
FormatDriver
name string
}
func (t *Format) Format(data interface{}) ([]byte, []byte, error) {
return t.driver.Format(data)
key, text, err := t.FormatDriver.Format(data)
if err != nil {
err = &DriverFormatError{
t.name,
err,
}
}
return key, text, err
}
func RegisterFormatDriver(name string, t FormatDriver) {
@@ -41,16 +62,19 @@ func RegisterFormatDriver(name string, t FormatDriver) {
}
}
func FindFormat(ctx context.Context, name string) (*Format, error) {
func FindFormat(name string) (*Format, error) {
lock.RLock()
t, ok := formatDrivers[name]
lock.RUnlock()
if !ok {
return nil, fmt.Errorf("Format %s not found", name)
return nil, fmt.Errorf("%w %s not found", ErrorFormat, name)
}
err := t.Init(ctx)
return &Format{t}, err
err := t.Init()
if err != nil {
err = &DriverFormatError{name, err}
}
return &Format{t, name}, err
}
func GetFormats() []string {
@@ -58,7 +82,7 @@ func GetFormats() []string {
defer lock.RUnlock()
t := make([]string, len(formatDrivers))
var i int
for k, _ := range formatDrivers {
for k := range formatDrivers {
t[i] = k
i++
}

View File

@@ -1,38 +1,29 @@
package json
import (
"context"
"fmt"
"github.com/golang/protobuf/proto"
"github.com/netsampler/goflow2/format"
"github.com/netsampler/goflow2/format/common"
"encoding/json"
"github.com/netsampler/goflow2/v2/format"
)
type JsonDriver struct {
}
func (d *JsonDriver) Prepare() error {
common.HashFlag()
common.SelectorFlag()
return nil
}
func (d *JsonDriver) Init(context.Context) error {
err := common.ManualHashInit()
if err != nil {
return err
}
return common.ManualSelectorInit()
func (d *JsonDriver) Init() error {
return nil
}
func (d *JsonDriver) Format(data interface{}) ([]byte, []byte, error) {
msg, ok := data.(proto.Message)
if !ok {
return nil, nil, fmt.Errorf("message is not protobuf")
var key []byte
if dataIf, ok := data.(interface{ Key() []byte }); ok {
key = dataIf.Key()
}
key := common.HashProtoLocal(msg)
return []byte(key), []byte(common.FormatMessageReflectJSON(msg, "")), nil
output, err := json.Marshal(data)
return key, output, err
}
func init() {

View File

@@ -1,46 +0,0 @@
package protobuf
import (
"context"
"flag"
"fmt"
"github.com/golang/protobuf/proto"
"github.com/netsampler/goflow2/format"
"github.com/netsampler/goflow2/format/common"
)
type ProtobufDriver struct {
fixedLen bool
}
func (d *ProtobufDriver) Prepare() error {
common.HashFlag()
flag.BoolVar(&d.fixedLen, "format.protobuf.fixedlen", false, "Prefix the protobuf with message length")
return nil
}
func (d *ProtobufDriver) Init(context.Context) error {
return common.ManualHashInit()
}
func (d *ProtobufDriver) Format(data interface{}) ([]byte, []byte, error) {
msg, ok := data.(proto.Message)
if !ok {
return nil, nil, fmt.Errorf("message is not protobuf")
}
key := common.HashProtoLocal(msg)
if !d.fixedLen {
b, err := proto.Marshal(msg)
return []byte(key), b, err
} else {
buf := proto.NewBuffer([]byte{})
err := buf.EncodeMessage(msg)
return []byte(key), buf.Bytes(), err
}
}
func init() {
d := &ProtobufDriver{}
format.RegisterFormatDriver("pb", d)
}

View File

@@ -1,38 +1,35 @@
package text
import (
"context"
"fmt"
"github.com/golang/protobuf/proto"
"github.com/netsampler/goflow2/format"
"github.com/netsampler/goflow2/format/common"
"encoding"
"github.com/netsampler/goflow2/v2/format"
)
type TextDriver struct {
}
func (d *TextDriver) Prepare() error {
common.HashFlag()
common.SelectorFlag()
return nil
}
func (d *TextDriver) Init(context.Context) error {
err := common.ManualHashInit()
if err != nil {
return err
}
return common.ManualSelectorInit()
func (d *TextDriver) Init() error {
return nil
}
func (d *TextDriver) Format(data interface{}) ([]byte, []byte, error) {
msg, ok := data.(proto.Message)
if !ok {
return nil, nil, fmt.Errorf("message is not protobuf")
var key []byte
if dataIf, ok := data.(interface{ Key() []byte }); ok {
key = dataIf.Key()
}
key := common.HashProtoLocal(msg)
return []byte(key), []byte(common.FormatMessageReflectText(msg, "")), nil
if dataIf, ok := data.(encoding.TextMarshaler); ok {
text, err := dataIf.MarshalText()
return key, text, err
}
if dataIf, ok := data.(interface{ String() string }); ok {
return key, []byte(dataIf.String()), nil
}
return key, nil, format.ErrorNoSerializer
}
func init() {

26
go.mod
View File

@@ -1,17 +1,16 @@
module github.com/netsampler/goflow2
module github.com/netsampler/goflow2/v2
go 1.18
go 1.20
require (
github.com/Shopify/sarama v1.38.1
github.com/golang/protobuf v1.5.3
github.com/libp2p/go-reuseport v0.3.0
github.com/oschwald/geoip2-golang v1.9.0
github.com/prometheus/client_golang v1.16.0
github.com/sirupsen/logrus v1.9.3
github.com/stretchr/testify v1.8.4
github.com/libp2p/go-reuseport v0.2.0
github.com/oschwald/geoip2-golang v1.8.0
github.com/prometheus/client_golang v1.15.0
github.com/sirupsen/logrus v1.9.0
github.com/stretchr/testify v1.8.2
github.com/xdg-go/scram v1.1.2
google.golang.org/protobuf v1.31.0
google.golang.org/protobuf v1.30.0
gopkg.in/yaml.v2 v2.4.0
)
@@ -22,6 +21,7 @@ require (
github.com/eapache/go-resiliency v1.3.0 // indirect
github.com/eapache/go-xerial-snappy v0.0.0-20230111030713-bf00bc1b83b6 // indirect
github.com/eapache/queue v1.1.0 // indirect
github.com/golang/protobuf v1.5.3 // indirect
github.com/golang/snappy v0.0.4 // indirect
github.com/hashicorp/errwrap v1.0.0 // indirect
github.com/hashicorp/go-multierror v1.1.1 // indirect
@@ -34,19 +34,19 @@ require (
github.com/klauspost/compress v1.15.14 // indirect
github.com/kr/text v0.2.0 // indirect
github.com/matttproud/golang_protobuf_extensions v1.0.4 // indirect
github.com/oschwald/maxminddb-golang v1.11.0 // indirect
github.com/oschwald/maxminddb-golang v1.10.0 // indirect
github.com/pierrec/lz4/v4 v4.1.17 // indirect
github.com/pmezard/go-difflib v1.0.0 // indirect
github.com/prometheus/client_model v0.3.0 // indirect
github.com/prometheus/common v0.42.0 // indirect
github.com/prometheus/procfs v0.10.1 // indirect
github.com/prometheus/procfs v0.9.0 // indirect
github.com/rcrowley/go-metrics v0.0.0-20201227073835-cf1acfcdf475 // indirect
github.com/rogpeppe/go-internal v1.9.0 // indirect
github.com/rogpeppe/go-internal v1.10.0 // indirect
github.com/xdg-go/pbkdf2 v1.0.0 // indirect
github.com/xdg-go/stringprep v1.0.4 // indirect
golang.org/x/crypto v0.0.0-20220722155217-630584e8d5aa // indirect
golang.org/x/net v0.7.0 // indirect
golang.org/x/sys v0.9.0 // indirect
golang.org/x/sys v0.6.0 // indirect
golang.org/x/text v0.7.0 // indirect
gopkg.in/yaml.v3 v3.0.1 // indirect
)

43
go.sum
View File

@@ -51,40 +51,41 @@ github.com/klauspost/compress v1.15.14/go.mod h1:QPwzmACJjUTFsnSHH934V6woptycfrD
github.com/kr/pretty v0.3.1 h1:flRD4NNwYAUpkphVc1HcthR4KEIFJ65n8Mw5qdRn3LE=
github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY=
github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE=
github.com/libp2p/go-reuseport v0.3.0 h1:iiZslO5byUYZEg9iCwJGf5h+sf1Agmqx2V2FDjPyvUw=
github.com/libp2p/go-reuseport v0.3.0/go.mod h1:laea40AimhtfEqysZ71UpYj4S+R9VpH8PgqLo7L+SwI=
github.com/libp2p/go-reuseport v0.2.0 h1:18PRvIMlpY6ZK85nIAicSBuXXvrYoSw3dsBAR7zc560=
github.com/libp2p/go-reuseport v0.2.0/go.mod h1:bvVho6eLMm6Bz5hmU0LYN3ixd3nPPvtIlaURZZgOY4k=
github.com/matttproud/golang_protobuf_extensions v1.0.4 h1:mmDVorXM7PCGKw94cs5zkfA9PSy5pEvNWRP0ET0TIVo=
github.com/matttproud/golang_protobuf_extensions v1.0.4/go.mod h1:BSXmuO+STAnVfrANrmjBb36TMTDstsz7MSK+HVaYKv4=
github.com/oschwald/geoip2-golang v1.9.0 h1:uvD3O6fXAXs+usU+UGExshpdP13GAqp4GBrzN7IgKZc=
github.com/oschwald/geoip2-golang v1.9.0/go.mod h1:BHK6TvDyATVQhKNbQBdrj9eAvuwOMi2zSFXizL3K81Y=
github.com/oschwald/maxminddb-golang v1.11.0 h1:aSXMqYR/EPNjGE8epgqwDay+P30hCBZIveY0WZbAWh0=
github.com/oschwald/maxminddb-golang v1.11.0/go.mod h1:YmVI+H0zh3ySFR3w+oz8PCfglAFj3PuCmui13+P9zDg=
github.com/oschwald/geoip2-golang v1.8.0 h1:KfjYB8ojCEn/QLqsDU0AzrJ3R5Qa9vFlx3z6SLNcKTs=
github.com/oschwald/geoip2-golang v1.8.0/go.mod h1:R7bRvYjOeaoenAp9sKRS8GX5bJWcZ0laWO5+DauEktw=
github.com/oschwald/maxminddb-golang v1.10.0 h1:Xp1u0ZhqkSuopaKmk1WwHtjF0H9Hd9181uj2MQ5Vndg=
github.com/oschwald/maxminddb-golang v1.10.0/go.mod h1:Y2ELenReaLAZ0b400URyGwvYxHV1dLIxBuyOsyYjHK0=
github.com/pierrec/lz4/v4 v4.1.17 h1:kV4Ip+/hUBC+8T6+2EgburRtkE9ef4nbY3f4dFhGjMc=
github.com/pierrec/lz4/v4 v4.1.17/go.mod h1:gZWDp/Ze/IJXGXf23ltt2EXimqmTUXEy0GFuRQyBid4=
github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM=
github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4=
github.com/prometheus/client_golang v1.16.0 h1:yk/hx9hDbrGHovbci4BY+pRMfSuuat626eFsHb7tmT8=
github.com/prometheus/client_golang v1.16.0/go.mod h1:Zsulrv/L9oM40tJ7T815tM89lFEugiJ9HzIqaAx4LKc=
github.com/prometheus/client_golang v1.15.0 h1:5fCgGYogn0hFdhyhLbw7hEsWxufKtY9klyvdNfFlFhM=
github.com/prometheus/client_golang v1.15.0/go.mod h1:e9yaBhRPU2pPNsZwE+JdQl0KEt1N9XgF6zxWmaC0xOk=
github.com/prometheus/client_model v0.3.0 h1:UBgGFHqYdG/TPFD1B1ogZywDqEkwp3fBMvqdiQ7Xew4=
github.com/prometheus/client_model v0.3.0/go.mod h1:LDGWKZIo7rky3hgvBe+caln+Dr3dPggB5dvjtD7w9+w=
github.com/prometheus/common v0.42.0 h1:EKsfXEYo4JpWMHH5cg+KOUWeuJSov1Id8zGR8eeI1YM=
github.com/prometheus/common v0.42.0/go.mod h1:xBwqVerjNdUDjgODMpudtOMwlOwf2SaTr1yjz4b7Zbc=
github.com/prometheus/procfs v0.10.1 h1:kYK1Va/YMlutzCGazswoHKo//tZVlFpKYh+PymziUAg=
github.com/prometheus/procfs v0.10.1/go.mod h1:nwNm2aOCAYw8uTR/9bWRREkZFxAUcWzPHWJq+XBB/FM=
github.com/prometheus/procfs v0.9.0 h1:wzCHvIvM5SxWqYvwgVL7yJY8Lz3PKn49KQtpgMYJfhI=
github.com/prometheus/procfs v0.9.0/go.mod h1:+pB4zwohETzFnmlpe6yd2lSc+0/46IYZRB/chUwxUZY=
github.com/rcrowley/go-metrics v0.0.0-20201227073835-cf1acfcdf475 h1:N/ElC8H3+5XpJzTSTfLsJV/mx9Q9g7kxmchpfZyxgzM=
github.com/rcrowley/go-metrics v0.0.0-20201227073835-cf1acfcdf475/go.mod h1:bCqnVzQkZxMG4s8nGwiZ5l3QUCyqpo9Y+/ZMZ9VjZe4=
github.com/rogpeppe/go-internal v1.9.0 h1:73kH8U+JUqXU8lRuOHeVHaa/SZPifC7BkcraZVejAe8=
github.com/rogpeppe/go-internal v1.9.0/go.mod h1:WtVeX8xhTBvf0smdhujwtBcq4Qrzq/fJaraNFVN+nFs=
github.com/sirupsen/logrus v1.9.3 h1:dueUQJ1C2q9oE3F7wvmSGAaVtTmUizReu6fjN8uqzbQ=
github.com/sirupsen/logrus v1.9.3/go.mod h1:naHLuLoDiP4jHNo9R0sCBMtWGeIprob74mVsIT4qYEQ=
github.com/rogpeppe/go-internal v1.10.0 h1:TMyTOH3F/DB16zRVcYyreMH6GnZZrwQVAoYjRBZyWFQ=
github.com/rogpeppe/go-internal v1.10.0/go.mod h1:UQnix2H7Ngw/k4C5ijL5+65zddjncjaFoBhdsK/akog=
github.com/sirupsen/logrus v1.9.0 h1:trlNQbNUG3OdDrDil03MCb1H2o9nJ1x4/5LYw7byDE0=
github.com/sirupsen/logrus v1.9.0/go.mod h1:naHLuLoDiP4jHNo9R0sCBMtWGeIprob74mVsIT4qYEQ=
github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME=
github.com/stretchr/objx v0.4.0/go.mod h1:YvHI0jy2hoMjB+UWwv71VJQ9isScKT/TqJzVSSt89Yw=
github.com/stretchr/objx v0.5.0/go.mod h1:Yh+to48EsGEfYuaHDzXPcE3xhTkx73EhmCGUpEOglKo=
github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4=
github.com/stretchr/testify v1.7.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg=
github.com/stretchr/testify v1.7.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg=
github.com/stretchr/testify v1.8.0/go.mod h1:yNjHg4UonilssWZ8iaSj1OCr/vHnekPRkoO+kdMU+MU=
github.com/stretchr/testify v1.8.4 h1:CcVxjf3Q8PM0mHUKJCdn+eZZtm5yQwehR5yeSVQQcUk=
github.com/stretchr/testify v1.8.4/go.mod h1:sz/lmYIOXD/1dqDmKjjqLyZ2RngseejIcXlSw2iwfAo=
github.com/stretchr/testify v1.8.2 h1:+h33VjcLVPDHtOdpUCuF+7gSuG3yGIftsP1YvFihtJ8=
github.com/stretchr/testify v1.8.2/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o6fzry7u4=
github.com/xdg-go/pbkdf2 v1.0.0 h1:Su7DPu48wXMwC3bs7MCNG+z4FhcyEuz5dlvchbq0B0c=
github.com/xdg-go/pbkdf2 v1.0.0/go.mod h1:jrpuAogTd400dnrH08LKmI/xc1MbPOebTwRqcT5RDeI=
github.com/xdg-go/scram v1.1.2 h1:FHX5I5B4i4hKRVRBCFRxq1iQRej7WO3hhBuJf+UUySY=
@@ -108,7 +109,7 @@ golang.org/x/net v0.7.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs=
golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
golang.org/x/sync v0.0.0-20220722155255-886fb9371eb4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM=
golang.org/x/sync v0.2.0 h1:PUR+T4wwASmuSTYdKjYHI5TD22Wy5ogLU5qZCOLxBrI=
golang.org/x/sync v0.1.0 h1:wsuoTGHzEhffawBOhz5CYhcrV4IdKZbEyZjBMuTp12o=
golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY=
golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
golang.org/x/sys v0.0.0-20210423082822-04245dca01da/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
@@ -116,8 +117,8 @@ golang.org/x/sys v0.0.0-20210615035016-665e8c7367d1/go.mod h1:oPkhp1MJrh7nUepCBc
golang.org/x/sys v0.0.0-20220520151302-bc2c85ada10a/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
golang.org/x/sys v0.0.0-20220715151400-c0bba94af5f8/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
golang.org/x/sys v0.0.0-20220722155257-8c9f86f7a55f/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
golang.org/x/sys v0.9.0 h1:KS/R3tvhPqvJvwcKfnBHJwwthS11LRhmM5D59eEXa0s=
golang.org/x/sys v0.9.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
golang.org/x/sys v0.6.0 h1:MVltZSvRTcU2ljQOhs94SXPftV6DCNnZViHeQps87pQ=
golang.org/x/sys v0.6.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo=
golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8=
golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ=
@@ -134,8 +135,8 @@ golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8T
golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0=
google.golang.org/protobuf v1.26.0-rc.1/go.mod h1:jlhhOSvTdKEhbULTjvd4ARK9grFBp09yW+WbY/TyQbw=
google.golang.org/protobuf v1.26.0/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc=
google.golang.org/protobuf v1.31.0 h1:g0LDEJHgrBl9N9r17Ru3sqWhkIx2NB67okBHPwC7hs8=
google.golang.org/protobuf v1.31.0/go.mod h1:HV8QOd/L58Z+nl8r43ehVNZIU/HEI6OcFqwMG9pJV4I=
google.golang.org/protobuf v1.30.0 h1:kPPoIgf3TsEvrm0PFe15JQ+570QVxYzEvvHqChK+cng=
google.golang.org/protobuf v1.30.0/go.mod h1:HV8QOd/L58Z+nl8r43ehVNZIU/HEI6OcFqwMG9pJV4I=
gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0=
gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c h1:Hei/4ADfdWqJk1ZMxUNpqntNwaWcugrBjAiHlqqRiVk=
gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI=

180
metrics/decoder.go Normal file
View File

@@ -0,0 +1,180 @@
package metrics
import (
"errors"
"fmt"
"time"
"github.com/netsampler/goflow2/v2/decoders/netflow"
"github.com/netsampler/goflow2/v2/utils"
"github.com/prometheus/client_golang/prometheus"
)
func PromDecoderWrapper(wrapped utils.DecoderFunc, name string) utils.DecoderFunc {
return func(msg interface{}) error {
pkt, ok := msg.(*utils.Message)
if !ok {
return fmt.Errorf("flow is not *Message")
}
remote := pkt.Src.Addr().Unmap().String()
localIP := pkt.Dst.Addr().Unmap().String()
port := fmt.Sprintf("%d", pkt.Dst.Port())
size := len(pkt.Payload)
MetricTrafficBytes.With(
prometheus.Labels{
"remote_ip": remote,
"local_ip": localIP,
"local_port": port,
"type": name,
}).
Add(float64(size))
MetricTrafficPackets.With(
prometheus.Labels{
"remote_ip": remote,
"local_ip": localIP,
"local_port": port,
"type": name,
}).
Inc()
MetricPacketSizeSum.With(
prometheus.Labels{
"remote_ip": remote,
"local_ip": localIP,
"local_port": port,
"type": name,
}).
Observe(float64(size))
timeTrackStart := time.Now().UTC()
err := wrapped(msg)
timeTrackStop := time.Now().UTC()
DecoderTime.With(
prometheus.Labels{
"name": name,
}).
Observe(float64((timeTrackStop.Sub(timeTrackStart)).Nanoseconds()) / 1000000000)
if err != nil {
if errors.Is(err, netflow.ErrorTemplateNotFound) {
NetFlowErrors.With(
prometheus.Labels{
"router": remote,
"error": "template_not_found",
}).
Inc()
}
switch err.(type) {
default:
DecoderErrors.With(
prometheus.Labels{
"router": remote,
"name": name,
}).
Inc()
}
}
return err
}
}
func recordCommonNetFlowMetrics(version uint16, key string, flowSets []interface{}) {
versionStr := fmt.Sprintf("%d", version)
NetFlowStats.With(
prometheus.Labels{
"router": key,
"version": versionStr,
}).
Inc()
for _, fs := range flowSets {
switch fsConv := fs.(type) {
case netflow.TemplateFlowSet:
NetFlowSetStatsSum.With(
prometheus.Labels{
"router": key,
"version": versionStr,
"type": "TemplateFlowSet",
}).
Inc()
NetFlowSetRecordsStatsSum.With(
prometheus.Labels{
"router": key,
"version": versionStr,
"type": "TemplateFlowSet",
}).
Add(float64(len(fsConv.Records)))
case netflow.NFv9OptionsTemplateFlowSet:
NetFlowSetStatsSum.With(
prometheus.Labels{
"router": key,
"version": versionStr,
"type": "OptionsTemplateFlowSet",
}).
Inc()
NetFlowSetRecordsStatsSum.With(
prometheus.Labels{
"router": key,
"version": versionStr,
"type": "OptionsTemplateFlowSet",
}).
Add(float64(len(fsConv.Records)))
case netflow.IPFIXOptionsTemplateFlowSet:
NetFlowSetStatsSum.With(
prometheus.Labels{
"router": key,
"version": versionStr,
"type": "OptionsTemplateFlowSet",
}).
Inc()
NetFlowSetRecordsStatsSum.With(
prometheus.Labels{
"router": key,
"version": versionStr,
"type": "OptionsTemplateFlowSet",
}).
Add(float64(len(fsConv.Records)))
case netflow.OptionsDataFlowSet:
NetFlowSetStatsSum.With(
prometheus.Labels{
"router": key,
"version": versionStr,
"type": "OptionsDataFlowSet",
}).
Inc()
NetFlowSetRecordsStatsSum.With(
prometheus.Labels{
"router": key,
"version": versionStr,
"type": "OptionsDataFlowSet",
}).
Add(float64(len(fsConv.Records)))
case netflow.DataFlowSet:
NetFlowSetStatsSum.With(
prometheus.Labels{
"router": key,
"version": versionStr,
"type": "DataFlowSet",
}).
Inc()
NetFlowSetRecordsStatsSum.With(
prometheus.Labels{
"router": key,
"version": versionStr,
"type": "DataFlowSet",
}).
Add(float64(len(fsConv.Records)))
}
}
}

134
metrics/metrics.go Normal file
View File

@@ -0,0 +1,134 @@
package metrics
import (
"github.com/prometheus/client_golang/prometheus"
)
const (
NAMESPACE = "goflow2"
)
var (
MetricTrafficBytes = prometheus.NewCounterVec(
prometheus.CounterOpts{
Name: "flow_traffic_bytes_total",
Help: "Bytes received by the application.",
Namespace: NAMESPACE,
},
[]string{"remote_ip", "local_ip", "local_port", "type"},
)
MetricTrafficPackets = prometheus.NewCounterVec(
prometheus.CounterOpts{
Name: "flow_traffic_packets_total",
Help: "Packets received by the application.",
Namespace: NAMESPACE},
[]string{"remote_ip", "local_ip", "local_port", "type"},
)
MetricPacketSizeSum = prometheus.NewSummaryVec(
prometheus.SummaryOpts{
Name: "flow_traffic_size_bytes",
Help: "Summary of packet size.",
Namespace: NAMESPACE, Objectives: map[float64]float64{0.5: 0.05, 0.9: 0.01, 0.99: 0.001},
},
[]string{"remote_ip", "local_ip", "local_port", "type"},
)
DecoderErrors = prometheus.NewCounterVec(
prometheus.CounterOpts{
Name: "flow_decoder_error_total",
Help: "NetFlow/sFlow processed errors.",
Namespace: NAMESPACE},
[]string{"router", "name"},
)
DecoderTime = prometheus.NewSummaryVec(
prometheus.SummaryOpts{
Name: "flow_decoding_time_seconds",
Help: "Decoding time summary.",
Namespace: NAMESPACE, Objectives: map[float64]float64{0.5: 0.05, 0.9: 0.01, 0.99: 0.001},
},
[]string{"name"},
)
NetFlowStats = prometheus.NewCounterVec(
prometheus.CounterOpts{
Name: "flow_process_nf_total",
Help: "NetFlows processed.",
Namespace: NAMESPACE},
[]string{"router", "version"},
)
NetFlowErrors = prometheus.NewCounterVec(
prometheus.CounterOpts{
Name: "flow_process_nf_errors_total",
Help: "NetFlows processed errors.",
Namespace: NAMESPACE},
[]string{"router", "error"},
)
NetFlowSetRecordsStatsSum = prometheus.NewCounterVec(
prometheus.CounterOpts{
Name: "flow_process_nf_flowset_records_total",
Help: "NetFlows FlowSets sum of records.",
Namespace: NAMESPACE},
[]string{"router", "version", "type"}, // data-template, data, opts...
)
NetFlowSetStatsSum = prometheus.NewCounterVec(
prometheus.CounterOpts{
Name: "flow_process_nf_flowset_total",
Help: "NetFlows FlowSets sum.",
Namespace: NAMESPACE},
[]string{"router", "version", "type"}, // data-template, data, opts...
)
NetFlowTimeStatsSum = prometheus.NewSummaryVec(
prometheus.SummaryOpts{
Name: "flow_process_nf_delay_seconds",
Help: "NetFlows time difference between time of flow and processing.",
Namespace: NAMESPACE, Objectives: map[float64]float64{0.5: 0.05, 0.9: 0.01, 0.99: 0.001},
},
[]string{"router", "version"},
)
NetFlowTemplatesStats = prometheus.NewCounterVec(
prometheus.CounterOpts{
Name: "flow_process_nf_templates_total",
Help: "NetFlows Template count.",
Namespace: NAMESPACE},
[]string{"router", "version", "obs_domain_id", "template_id", "type"}, // options/template
)
SFlowStats = prometheus.NewCounterVec(
prometheus.CounterOpts{
Name: "flow_process_sf_total",
Help: "sFlows processed.",
Namespace: NAMESPACE},
[]string{"router", "agent", "version"},
)
SFlowSampleStatsSum = prometheus.NewCounterVec(
prometheus.CounterOpts{
Name: "flow_process_sf_samples_total",
Help: "SFlows samples sum.",
Namespace: NAMESPACE},
[]string{"router", "agent", "version", "type"}, // counter, flow, expanded...
)
SFlowSampleRecordsStatsSum = prometheus.NewCounterVec(
prometheus.CounterOpts{
Name: "flow_process_sf_samples_records_total",
Help: "SFlows samples sum of records.",
Namespace: NAMESPACE},
[]string{"router", "agent", "version", "type"}, // data-template, data, opts...
)
)
func init() {
prometheus.MustRegister(MetricTrafficBytes)
prometheus.MustRegister(MetricTrafficPackets)
prometheus.MustRegister(MetricPacketSizeSum)
prometheus.MustRegister(DecoderErrors)
prometheus.MustRegister(DecoderTime)
prometheus.MustRegister(NetFlowStats)
prometheus.MustRegister(NetFlowErrors)
prometheus.MustRegister(NetFlowSetRecordsStatsSum)
prometheus.MustRegister(NetFlowSetStatsSum)
prometheus.MustRegister(NetFlowTimeStatsSum)
prometheus.MustRegister(NetFlowTemplatesStats)
prometheus.MustRegister(SFlowStats)
prometheus.MustRegister(SFlowSampleStatsSum)
prometheus.MustRegister(SFlowSampleRecordsStatsSum)
}

151
metrics/producer.go Normal file
View File

@@ -0,0 +1,151 @@
package metrics
import (
"net/netip"
"github.com/netsampler/goflow2/v2/decoders/netflow"
"github.com/netsampler/goflow2/v2/decoders/netflowlegacy"
"github.com/netsampler/goflow2/v2/decoders/sflow"
flowmessage "github.com/netsampler/goflow2/v2/pb"
"github.com/netsampler/goflow2/v2/producer"
"github.com/prometheus/client_golang/prometheus"
)
type PromProducerWrapper struct {
wrapped producer.ProducerInterface
}
func (p *PromProducerWrapper) Produce(msg interface{}, args *producer.ProduceArgs) ([]producer.ProducerMessage, error) {
flowMessageSet, err := p.wrapped.Produce(msg, args)
if err != nil {
return flowMessageSet, err
}
key := args.Src.Addr().Unmap().String()
var nfvariant bool
var versionStr string
switch packet := msg.(type) {
case *sflow.Packet:
agentStr := "unk"
agentIp, ok := netip.AddrFromSlice(packet.AgentIP)
if ok {
agentStr = agentIp.String()
}
SFlowStats.With(
prometheus.Labels{
"router": key,
"agent": agentStr,
"version": "5",
}).
Inc()
for _, samples := range packet.Samples {
typeStr := "unknown"
countRec := 0
switch samplesConv := samples.(type) {
case sflow.FlowSample:
typeStr = "FlowSample"
countRec = len(samplesConv.Records)
case sflow.CounterSample:
typeStr = "CounterSample"
if samplesConv.Header.Format == 4 {
typeStr = "Expanded" + typeStr
}
countRec = len(samplesConv.Records)
case sflow.ExpandedFlowSample:
typeStr = "ExpandedFlowSample"
countRec = len(samplesConv.Records)
}
SFlowSampleStatsSum.With(
prometheus.Labels{
"router": key,
"agent": agentStr,
"version": "5",
"type": typeStr,
}).
Inc()
SFlowSampleRecordsStatsSum.With(
prometheus.Labels{
"router": key,
"agent": agentStr,
"version": "5",
"type": typeStr,
}).
Add(float64(countRec))
}
case *netflowlegacy.PacketNetFlowV5:
NetFlowStats.With(
prometheus.Labels{
"router": key,
"version": "5",
}).
Inc()
NetFlowSetStatsSum.With(
prometheus.Labels{
"router": key,
"version": "5",
"type": "DataFlowSet",
}).
Add(float64(packet.Count))
case *netflow.NFv9Packet:
NetFlowStats.With(
prometheus.Labels{
"router": key,
"version": "9",
}).
Inc()
recordCommonNetFlowMetrics(9, key, packet.FlowSets)
nfvariant = true
versionStr = "9"
case *netflow.IPFIXPacket:
NetFlowStats.With(
prometheus.Labels{
"router": key,
"version": "10",
}).
Inc()
recordCommonNetFlowMetrics(10, key, packet.FlowSets)
nfvariant = true
versionStr = "10"
}
if nfvariant {
for _, msg := range flowMessageSet {
fmsg, ok := msg.(*flowmessage.FlowMessage)
if !ok {
continue
}
timeDiff := fmsg.TimeReceivedNs - fmsg.TimeFlowEndNs
NetFlowTimeStatsSum.With(
prometheus.Labels{
"router": key,
"version": versionStr,
}).
Observe(float64(timeDiff))
}
}
return flowMessageSet, err
}
func (p *PromProducerWrapper) Close() {
p.wrapped.Close()
}
func (p *PromProducerWrapper) Commit(flowMessageSet []producer.ProducerMessage) {
p.wrapped.Commit(flowMessageSet)
}
// Wraps a producer with metrics
func WrapPromProducer(wrapped producer.ProducerInterface) producer.ProducerInterface {
return &PromProducerWrapper{
wrapped: wrapped,
}
}
// metrics template system

72
metrics/templates.go Normal file
View File

@@ -0,0 +1,72 @@
package metrics
import (
"strconv"
"github.com/netsampler/goflow2/v2/decoders/netflow"
"github.com/prometheus/client_golang/prometheus"
)
type PromTemplateSystem struct {
key string
wrapped netflow.NetFlowTemplateSystem
}
// A default Prometheus template generator function to be used by a pipe
func NewDefaultPromTemplateSystem(key string) netflow.NetFlowTemplateSystem {
return NewPromTemplateSystem(key, netflow.CreateTemplateSystem())
}
// Creates a Prometheus template system that wraps another template system.
// The key argument is providing the router information for metrics.
func NewPromTemplateSystem(key string, wrapped netflow.NetFlowTemplateSystem) netflow.NetFlowTemplateSystem {
return &PromTemplateSystem{
key: key,
wrapped: wrapped,
}
}
func (s *PromTemplateSystem) getLabels(version uint16, obsDomainId uint32, templateId uint16, template interface{}) prometheus.Labels {
typeStr := "options_template"
switch template.(type) {
case netflow.TemplateRecord:
typeStr = "template"
}
return prometheus.Labels{
"router": s.key,
"version": strconv.Itoa(int(version)),
"obs_domain_id": strconv.Itoa(int(obsDomainId)),
"template_id": strconv.Itoa(int(templateId)),
"type": typeStr,
}
}
func (s *PromTemplateSystem) AddTemplate(version uint16, obsDomainId uint32, templateId uint16, template interface{}) error {
err := s.wrapped.AddTemplate(version, obsDomainId, templateId, template)
labels := s.getLabels(version, obsDomainId, templateId, template)
NetFlowTemplatesStats.With(
labels).
Inc()
return err
}
func (s *PromTemplateSystem) GetTemplate(version uint16, obsDomainId uint32, templateId uint16) (interface{}, error) {
return s.wrapped.GetTemplate(version, obsDomainId, templateId)
}
func (s *PromTemplateSystem) RemoveTemplate(version uint16, obsDomainId uint32, templateId uint16) (interface{}, error) {
template, err := s.wrapped.RemoveTemplate(version, obsDomainId, templateId)
if err == nil {
labels := s.getLabels(version, obsDomainId, templateId, template)
NetFlowTemplatesStats.Delete(labels)
}
return template, err
}

View File

@@ -1,7 +1,7 @@
// Code generated by protoc-gen-go. DO NOT EDIT.
// versions:
// protoc-gen-go v1.26.0
// protoc v3.21.4
// protoc v3.21.12
// source: pb/flow.proto
package flowpb
@@ -80,18 +80,15 @@ type FlowMessage struct {
sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields
Type FlowMessage_FlowType `protobuf:"varint,1,opt,name=type,proto3,enum=flowpb.FlowMessage_FlowType" json:"type,omitempty"`
TimeReceived uint64 `protobuf:"varint,2,opt,name=time_received,json=timeReceived,proto3" json:"time_received,omitempty"`
SequenceNum uint32 `protobuf:"varint,4,opt,name=sequence_num,json=sequenceNum,proto3" json:"sequence_num,omitempty"`
SamplingRate uint64 `protobuf:"varint,3,opt,name=sampling_rate,json=samplingRate,proto3" json:"sampling_rate,omitempty"`
FlowDirection uint32 `protobuf:"varint,42,opt,name=flow_direction,json=flowDirection,proto3" json:"flow_direction,omitempty"`
Type FlowMessage_FlowType `protobuf:"varint,1,opt,name=type,proto3,enum=flowpb.FlowMessage_FlowType" json:"type,omitempty"`
TimeReceivedNs uint64 `protobuf:"varint,110,opt,name=time_received_ns,json=timeReceivedNs,proto3" json:"time_received_ns,omitempty"`
SequenceNum uint32 `protobuf:"varint,4,opt,name=sequence_num,json=sequenceNum,proto3" json:"sequence_num,omitempty"`
SamplingRate uint64 `protobuf:"varint,3,opt,name=sampling_rate,json=samplingRate,proto3" json:"sampling_rate,omitempty"`
// Sampler information
SamplerAddress []byte `protobuf:"bytes,11,opt,name=sampler_address,json=samplerAddress,proto3" json:"sampler_address,omitempty"`
// Found inside packet
TimeFlowStart uint64 `protobuf:"varint,38,opt,name=time_flow_start,json=timeFlowStart,proto3" json:"time_flow_start,omitempty"`
TimeFlowEnd uint64 `protobuf:"varint,5,opt,name=time_flow_end,json=timeFlowEnd,proto3" json:"time_flow_end,omitempty"`
TimeFlowStartMs uint64 `protobuf:"varint,63,opt,name=time_flow_start_ms,json=timeFlowStartMs,proto3" json:"time_flow_start_ms,omitempty"`
TimeFlowEndMs uint64 `protobuf:"varint,64,opt,name=time_flow_end_ms,json=timeFlowEndMs,proto3" json:"time_flow_end_ms,omitempty"`
TimeFlowStartNs uint64 `protobuf:"varint,111,opt,name=time_flow_start_ns,json=timeFlowStartNs,proto3" json:"time_flow_start_ns,omitempty"`
TimeFlowEndNs uint64 `protobuf:"varint,112,opt,name=time_flow_end_ns,json=timeFlowEndNs,proto3" json:"time_flow_end_ns,omitempty"`
// Size of the sampled packet
Bytes uint64 `protobuf:"varint,9,opt,name=bytes,proto3" json:"bytes,omitempty"`
Packets uint64 `protobuf:"varint,10,opt,name=packets,proto3" json:"packets,omitempty"`
@@ -116,9 +113,6 @@ type FlowMessage struct {
DstVlan uint32 `protobuf:"varint,34,opt,name=dst_vlan,json=dstVlan,proto3" json:"dst_vlan,omitempty"`
// 802.1q VLAN in sampled packet
VlanId uint32 `protobuf:"varint,29,opt,name=vlan_id,json=vlanId,proto3" json:"vlan_id,omitempty"`
// VRF
IngressVrfId uint32 `protobuf:"varint,39,opt,name=ingress_vrf_id,json=ingressVrfId,proto3" json:"ingress_vrf_id,omitempty"`
EgressVrfId uint32 `protobuf:"varint,40,opt,name=egress_vrf_id,json=egressVrfId,proto3" json:"egress_vrf_id,omitempty"`
// IP and TCP special flags
IpTos uint32 `protobuf:"varint,23,opt,name=ip_tos,json=ipTos,proto3" json:"ip_tos,omitempty"`
ForwardingStatus uint32 `protobuf:"varint,24,opt,name=forwarding_status,json=forwardingStatus,proto3" json:"forwarding_status,omitempty"`
@@ -128,9 +122,8 @@ type FlowMessage struct {
IcmpCode uint32 `protobuf:"varint,32,opt,name=icmp_code,json=icmpCode,proto3" json:"icmp_code,omitempty"`
Ipv6FlowLabel uint32 `protobuf:"varint,37,opt,name=ipv6_flow_label,json=ipv6FlowLabel,proto3" json:"ipv6_flow_label,omitempty"`
// Fragments (IPv4/IPv6)
FragmentId uint32 `protobuf:"varint,35,opt,name=fragment_id,json=fragmentId,proto3" json:"fragment_id,omitempty"`
FragmentOffset uint32 `protobuf:"varint,36,opt,name=fragment_offset,json=fragmentOffset,proto3" json:"fragment_offset,omitempty"`
BiFlowDirection uint32 `protobuf:"varint,41,opt,name=bi_flow_direction,json=biFlowDirection,proto3" json:"bi_flow_direction,omitempty"`
FragmentId uint32 `protobuf:"varint,35,opt,name=fragment_id,json=fragmentId,proto3" json:"fragment_id,omitempty"`
FragmentOffset uint32 `protobuf:"varint,36,opt,name=fragment_offset,json=fragmentOffset,proto3" json:"fragment_offset,omitempty"`
// Autonomous system information
SrcAs uint32 `protobuf:"varint,14,opt,name=src_as,json=srcAs,proto3" json:"src_as,omitempty"`
DstAs uint32 `protobuf:"varint,15,opt,name=dst_as,json=dstAs,proto3" json:"dst_as,omitempty"`
@@ -144,31 +137,11 @@ type FlowMessage struct {
BgpCommunities []uint32 `protobuf:"varint,101,rep,packed,name=bgp_communities,json=bgpCommunities,proto3" json:"bgp_communities,omitempty"`
AsPath []uint32 `protobuf:"varint,102,rep,packed,name=as_path,json=asPath,proto3" json:"as_path,omitempty"`
// MPLS information
HasMpls bool `protobuf:"varint,53,opt,name=has_mpls,json=hasMpls,proto3" json:"has_mpls,omitempty"`
MplsCount uint32 `protobuf:"varint,54,opt,name=mpls_count,json=mplsCount,proto3" json:"mpls_count,omitempty"`
Mpls_1Ttl uint32 `protobuf:"varint,55,opt,name=mpls_1_ttl,json=mpls1Ttl,proto3" json:"mpls_1_ttl,omitempty"` // First TTL
Mpls_1Label uint32 `protobuf:"varint,56,opt,name=mpls_1_label,json=mpls1Label,proto3" json:"mpls_1_label,omitempty"` // First Label
Mpls_2Ttl uint32 `protobuf:"varint,57,opt,name=mpls_2_ttl,json=mpls2Ttl,proto3" json:"mpls_2_ttl,omitempty"` // Second TTL
Mpls_2Label uint32 `protobuf:"varint,58,opt,name=mpls_2_label,json=mpls2Label,proto3" json:"mpls_2_label,omitempty"` // Second Label
Mpls_3Ttl uint32 `protobuf:"varint,59,opt,name=mpls_3_ttl,json=mpls3Ttl,proto3" json:"mpls_3_ttl,omitempty"` // Third TTL
Mpls_3Label uint32 `protobuf:"varint,60,opt,name=mpls_3_label,json=mpls3Label,proto3" json:"mpls_3_label,omitempty"` // Third Label
MplsLastTtl uint32 `protobuf:"varint,61,opt,name=mpls_last_ttl,json=mplsLastTtl,proto3" json:"mpls_last_ttl,omitempty"` // Last TTL
MplsLastLabel uint32 `protobuf:"varint,62,opt,name=mpls_last_label,json=mplsLastLabel,proto3" json:"mpls_last_label,omitempty"` // Last Label
MplsLabelIp []byte `protobuf:"bytes,65,opt,name=mpls_label_ip,json=mplsLabelIp,proto3" json:"mpls_label_ip,omitempty"` // MPLS TOP Label IP
ObservationDomainId uint32 `protobuf:"varint,70,opt,name=observation_domain_id,json=observationDomainId,proto3" json:"observation_domain_id,omitempty"`
ObservationPointId uint32 `protobuf:"varint,71,opt,name=observation_point_id,json=observationPointId,proto3" json:"observation_point_id,omitempty"`
// Custom allocations
CustomInteger_1 uint64 `protobuf:"varint,1001,opt,name=custom_integer_1,json=customInteger1,proto3" json:"custom_integer_1,omitempty"`
CustomInteger_2 uint64 `protobuf:"varint,1002,opt,name=custom_integer_2,json=customInteger2,proto3" json:"custom_integer_2,omitempty"`
CustomInteger_3 uint64 `protobuf:"varint,1003,opt,name=custom_integer_3,json=customInteger3,proto3" json:"custom_integer_3,omitempty"`
CustomInteger_4 uint64 `protobuf:"varint,1004,opt,name=custom_integer_4,json=customInteger4,proto3" json:"custom_integer_4,omitempty"`
CustomInteger_5 uint64 `protobuf:"varint,1005,opt,name=custom_integer_5,json=customInteger5,proto3" json:"custom_integer_5,omitempty"`
CustomBytes_1 []byte `protobuf:"bytes,1011,opt,name=custom_bytes_1,json=customBytes1,proto3" json:"custom_bytes_1,omitempty"`
CustomBytes_2 []byte `protobuf:"bytes,1012,opt,name=custom_bytes_2,json=customBytes2,proto3" json:"custom_bytes_2,omitempty"`
CustomBytes_3 []byte `protobuf:"bytes,1013,opt,name=custom_bytes_3,json=customBytes3,proto3" json:"custom_bytes_3,omitempty"`
CustomBytes_4 []byte `protobuf:"bytes,1014,opt,name=custom_bytes_4,json=customBytes4,proto3" json:"custom_bytes_4,omitempty"`
CustomBytes_5 []byte `protobuf:"bytes,1015,opt,name=custom_bytes_5,json=customBytes5,proto3" json:"custom_bytes_5,omitempty"`
CustomList_1 []uint32 `protobuf:"varint,1021,rep,packed,name=custom_list_1,json=customList1,proto3" json:"custom_list_1,omitempty"`
MplsTtl []uint32 `protobuf:"varint,80,rep,packed,name=mpls_ttl,json=mplsTtl,proto3" json:"mpls_ttl,omitempty"`
MplsLabel []uint32 `protobuf:"varint,81,rep,packed,name=mpls_label,json=mplsLabel,proto3" json:"mpls_label,omitempty"`
MplsIp [][]byte `protobuf:"bytes,82,rep,name=mpls_ip,json=mplsIp,proto3" json:"mpls_ip,omitempty"`
ObservationDomainId uint32 `protobuf:"varint,70,opt,name=observation_domain_id,json=observationDomainId,proto3" json:"observation_domain_id,omitempty"`
ObservationPointId uint32 `protobuf:"varint,71,opt,name=observation_point_id,json=observationPointId,proto3" json:"observation_point_id,omitempty"`
}
func (x *FlowMessage) Reset() {
@@ -210,9 +183,9 @@ func (x *FlowMessage) GetType() FlowMessage_FlowType {
return FlowMessage_FLOWUNKNOWN
}
func (x *FlowMessage) GetTimeReceived() uint64 {
func (x *FlowMessage) GetTimeReceivedNs() uint64 {
if x != nil {
return x.TimeReceived
return x.TimeReceivedNs
}
return 0
}
@@ -231,13 +204,6 @@ func (x *FlowMessage) GetSamplingRate() uint64 {
return 0
}
func (x *FlowMessage) GetFlowDirection() uint32 {
if x != nil {
return x.FlowDirection
}
return 0
}
func (x *FlowMessage) GetSamplerAddress() []byte {
if x != nil {
return x.SamplerAddress
@@ -245,30 +211,16 @@ func (x *FlowMessage) GetSamplerAddress() []byte {
return nil
}
func (x *FlowMessage) GetTimeFlowStart() uint64 {
func (x *FlowMessage) GetTimeFlowStartNs() uint64 {
if x != nil {
return x.TimeFlowStart
return x.TimeFlowStartNs
}
return 0
}
func (x *FlowMessage) GetTimeFlowEnd() uint64 {
func (x *FlowMessage) GetTimeFlowEndNs() uint64 {
if x != nil {
return x.TimeFlowEnd
}
return 0
}
func (x *FlowMessage) GetTimeFlowStartMs() uint64 {
if x != nil {
return x.TimeFlowStartMs
}
return 0
}
func (x *FlowMessage) GetTimeFlowEndMs() uint64 {
if x != nil {
return x.TimeFlowEndMs
return x.TimeFlowEndNs
}
return 0
}
@@ -378,20 +330,6 @@ func (x *FlowMessage) GetVlanId() uint32 {
return 0
}
func (x *FlowMessage) GetIngressVrfId() uint32 {
if x != nil {
return x.IngressVrfId
}
return 0
}
func (x *FlowMessage) GetEgressVrfId() uint32 {
if x != nil {
return x.EgressVrfId
}
return 0
}
func (x *FlowMessage) GetIpTos() uint32 {
if x != nil {
return x.IpTos
@@ -455,13 +393,6 @@ func (x *FlowMessage) GetFragmentOffset() uint32 {
return 0
}
func (x *FlowMessage) GetBiFlowDirection() uint32 {
if x != nil {
return x.BiFlowDirection
}
return 0
}
func (x *FlowMessage) GetSrcAs() uint32 {
if x != nil {
return x.SrcAs
@@ -525,79 +456,23 @@ func (x *FlowMessage) GetAsPath() []uint32 {
return nil
}
func (x *FlowMessage) GetHasMpls() bool {
func (x *FlowMessage) GetMplsTtl() []uint32 {
if x != nil {
return x.HasMpls
return x.MplsTtl
}
return false
return nil
}
func (x *FlowMessage) GetMplsCount() uint32 {
func (x *FlowMessage) GetMplsLabel() []uint32 {
if x != nil {
return x.MplsCount
return x.MplsLabel
}
return 0
return nil
}
func (x *FlowMessage) GetMpls_1Ttl() uint32 {
func (x *FlowMessage) GetMplsIp() [][]byte {
if x != nil {
return x.Mpls_1Ttl
}
return 0
}
func (x *FlowMessage) GetMpls_1Label() uint32 {
if x != nil {
return x.Mpls_1Label
}
return 0
}
func (x *FlowMessage) GetMpls_2Ttl() uint32 {
if x != nil {
return x.Mpls_2Ttl
}
return 0
}
func (x *FlowMessage) GetMpls_2Label() uint32 {
if x != nil {
return x.Mpls_2Label
}
return 0
}
func (x *FlowMessage) GetMpls_3Ttl() uint32 {
if x != nil {
return x.Mpls_3Ttl
}
return 0
}
func (x *FlowMessage) GetMpls_3Label() uint32 {
if x != nil {
return x.Mpls_3Label
}
return 0
}
func (x *FlowMessage) GetMplsLastTtl() uint32 {
if x != nil {
return x.MplsLastTtl
}
return 0
}
func (x *FlowMessage) GetMplsLastLabel() uint32 {
if x != nil {
return x.MplsLastLabel
}
return 0
}
func (x *FlowMessage) GetMplsLabelIp() []byte {
if x != nil {
return x.MplsLabelIp
return x.MplsIp
}
return nil
}
@@ -616,244 +491,108 @@ func (x *FlowMessage) GetObservationPointId() uint32 {
return 0
}
func (x *FlowMessage) GetCustomInteger_1() uint64 {
if x != nil {
return x.CustomInteger_1
}
return 0
}
func (x *FlowMessage) GetCustomInteger_2() uint64 {
if x != nil {
return x.CustomInteger_2
}
return 0
}
func (x *FlowMessage) GetCustomInteger_3() uint64 {
if x != nil {
return x.CustomInteger_3
}
return 0
}
func (x *FlowMessage) GetCustomInteger_4() uint64 {
if x != nil {
return x.CustomInteger_4
}
return 0
}
func (x *FlowMessage) GetCustomInteger_5() uint64 {
if x != nil {
return x.CustomInteger_5
}
return 0
}
func (x *FlowMessage) GetCustomBytes_1() []byte {
if x != nil {
return x.CustomBytes_1
}
return nil
}
func (x *FlowMessage) GetCustomBytes_2() []byte {
if x != nil {
return x.CustomBytes_2
}
return nil
}
func (x *FlowMessage) GetCustomBytes_3() []byte {
if x != nil {
return x.CustomBytes_3
}
return nil
}
func (x *FlowMessage) GetCustomBytes_4() []byte {
if x != nil {
return x.CustomBytes_4
}
return nil
}
func (x *FlowMessage) GetCustomBytes_5() []byte {
if x != nil {
return x.CustomBytes_5
}
return nil
}
func (x *FlowMessage) GetCustomList_1() []uint32 {
if x != nil {
return x.CustomList_1
}
return nil
}
var File_pb_flow_proto protoreflect.FileDescriptor
var file_pb_flow_proto_rawDesc = []byte{
0x0a, 0x0d, 0x70, 0x62, 0x2f, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12,
0x06, 0x66, 0x6c, 0x6f, 0x77, 0x70, 0x62, 0x22, 0x90, 0x13, 0x0a, 0x0b, 0x46, 0x6c, 0x6f, 0x77,
0x06, 0x66, 0x6c, 0x6f, 0x77, 0x70, 0x62, 0x22, 0xd6, 0x0b, 0x0a, 0x0b, 0x46, 0x6c, 0x6f, 0x77,
0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x30, 0x0a, 0x04, 0x74, 0x79, 0x70, 0x65, 0x18,
0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1c, 0x2e, 0x66, 0x6c, 0x6f, 0x77, 0x70, 0x62, 0x2e, 0x46,
0x6c, 0x6f, 0x77, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x2e, 0x46, 0x6c, 0x6f, 0x77, 0x54,
0x79, 0x70, 0x65, 0x52, 0x04, 0x74, 0x79, 0x70, 0x65, 0x12, 0x23, 0x0a, 0x0d, 0x74, 0x69, 0x6d,
0x65, 0x5f, 0x72, 0x65, 0x63, 0x65, 0x69, 0x76, 0x65, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04,
0x52, 0x0c, 0x74, 0x69, 0x6d, 0x65, 0x52, 0x65, 0x63, 0x65, 0x69, 0x76, 0x65, 0x64, 0x12, 0x21,
0x0a, 0x0c, 0x73, 0x65, 0x71, 0x75, 0x65, 0x6e, 0x63, 0x65, 0x5f, 0x6e, 0x75, 0x6d, 0x18, 0x04,
0x20, 0x01, 0x28, 0x0d, 0x52, 0x0b, 0x73, 0x65, 0x71, 0x75, 0x65, 0x6e, 0x63, 0x65, 0x4e, 0x75,
0x6d, 0x12, 0x23, 0x0a, 0x0d, 0x73, 0x61, 0x6d, 0x70, 0x6c, 0x69, 0x6e, 0x67, 0x5f, 0x72, 0x61,
0x74, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0c, 0x73, 0x61, 0x6d, 0x70, 0x6c, 0x69,
0x6e, 0x67, 0x52, 0x61, 0x74, 0x65, 0x12, 0x25, 0x0a, 0x0e, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x64,
0x69, 0x72, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x2a, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0d,
0x66, 0x6c, 0x6f, 0x77, 0x44, 0x69, 0x72, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x27, 0x0a,
0x0f, 0x73, 0x61, 0x6d, 0x70, 0x6c, 0x65, 0x72, 0x5f, 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73,
0x18, 0x0b, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0e, 0x73, 0x61, 0x6d, 0x70, 0x6c, 0x65, 0x72, 0x41,
0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x12, 0x26, 0x0a, 0x0f, 0x74, 0x69, 0x6d, 0x65, 0x5f, 0x66,
0x6c, 0x6f, 0x77, 0x5f, 0x73, 0x74, 0x61, 0x72, 0x74, 0x18, 0x26, 0x20, 0x01, 0x28, 0x04, 0x52,
0x0d, 0x74, 0x69, 0x6d, 0x65, 0x46, 0x6c, 0x6f, 0x77, 0x53, 0x74, 0x61, 0x72, 0x74, 0x12, 0x22,
0x0a, 0x0d, 0x74, 0x69, 0x6d, 0x65, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x65, 0x6e, 0x64, 0x18,
0x05, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0b, 0x74, 0x69, 0x6d, 0x65, 0x46, 0x6c, 0x6f, 0x77, 0x45,
0x6e, 0x64, 0x12, 0x2b, 0x0a, 0x12, 0x74, 0x69, 0x6d, 0x65, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x5f,
0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x6d, 0x73, 0x18, 0x3f, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0f,
0x74, 0x69, 0x6d, 0x65, 0x46, 0x6c, 0x6f, 0x77, 0x53, 0x74, 0x61, 0x72, 0x74, 0x4d, 0x73, 0x12,
0x27, 0x0a, 0x10, 0x74, 0x69, 0x6d, 0x65, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x65, 0x6e, 0x64,
0x5f, 0x6d, 0x73, 0x18, 0x40, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0d, 0x74, 0x69, 0x6d, 0x65, 0x46,
0x6c, 0x6f, 0x77, 0x45, 0x6e, 0x64, 0x4d, 0x73, 0x12, 0x14, 0x0a, 0x05, 0x62, 0x79, 0x74, 0x65,
0x73, 0x18, 0x09, 0x20, 0x01, 0x28, 0x04, 0x52, 0x05, 0x62, 0x79, 0x74, 0x65, 0x73, 0x12, 0x18,
0x0a, 0x07, 0x70, 0x61, 0x63, 0x6b, 0x65, 0x74, 0x73, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x04, 0x52,
0x07, 0x70, 0x61, 0x63, 0x6b, 0x65, 0x74, 0x73, 0x12, 0x19, 0x0a, 0x08, 0x73, 0x72, 0x63, 0x5f,
0x61, 0x64, 0x64, 0x72, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x07, 0x73, 0x72, 0x63, 0x41,
0x64, 0x64, 0x72, 0x12, 0x19, 0x0a, 0x08, 0x64, 0x73, 0x74, 0x5f, 0x61, 0x64, 0x64, 0x72, 0x18,
0x07, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x07, 0x64, 0x73, 0x74, 0x41, 0x64, 0x64, 0x72, 0x12, 0x14,
0x0a, 0x05, 0x65, 0x74, 0x79, 0x70, 0x65, 0x18, 0x1e, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x05, 0x65,
0x74, 0x79, 0x70, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x18, 0x14, 0x20,
0x01, 0x28, 0x0d, 0x52, 0x05, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x19, 0x0a, 0x08, 0x73, 0x72,
0x63, 0x5f, 0x70, 0x6f, 0x72, 0x74, 0x18, 0x15, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x07, 0x73, 0x72,
0x63, 0x50, 0x6f, 0x72, 0x74, 0x12, 0x19, 0x0a, 0x08, 0x64, 0x73, 0x74, 0x5f, 0x70, 0x6f, 0x72,
0x74, 0x18, 0x16, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x07, 0x64, 0x73, 0x74, 0x50, 0x6f, 0x72, 0x74,
0x12, 0x13, 0x0a, 0x05, 0x69, 0x6e, 0x5f, 0x69, 0x66, 0x18, 0x12, 0x20, 0x01, 0x28, 0x0d, 0x52,
0x04, 0x69, 0x6e, 0x49, 0x66, 0x12, 0x15, 0x0a, 0x06, 0x6f, 0x75, 0x74, 0x5f, 0x69, 0x66, 0x18,
0x13, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x05, 0x6f, 0x75, 0x74, 0x49, 0x66, 0x12, 0x17, 0x0a, 0x07,
0x73, 0x72, 0x63, 0x5f, 0x6d, 0x61, 0x63, 0x18, 0x1b, 0x20, 0x01, 0x28, 0x04, 0x52, 0x06, 0x73,
0x72, 0x63, 0x4d, 0x61, 0x63, 0x12, 0x17, 0x0a, 0x07, 0x64, 0x73, 0x74, 0x5f, 0x6d, 0x61, 0x63,
0x18, 0x1c, 0x20, 0x01, 0x28, 0x04, 0x52, 0x06, 0x64, 0x73, 0x74, 0x4d, 0x61, 0x63, 0x12, 0x19,
0x0a, 0x08, 0x73, 0x72, 0x63, 0x5f, 0x76, 0x6c, 0x61, 0x6e, 0x18, 0x21, 0x20, 0x01, 0x28, 0x0d,
0x52, 0x07, 0x73, 0x72, 0x63, 0x56, 0x6c, 0x61, 0x6e, 0x12, 0x19, 0x0a, 0x08, 0x64, 0x73, 0x74,
0x5f, 0x76, 0x6c, 0x61, 0x6e, 0x18, 0x22, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x07, 0x64, 0x73, 0x74,
0x56, 0x6c, 0x61, 0x6e, 0x12, 0x17, 0x0a, 0x07, 0x76, 0x6c, 0x61, 0x6e, 0x5f, 0x69, 0x64, 0x18,
0x1d, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x06, 0x76, 0x6c, 0x61, 0x6e, 0x49, 0x64, 0x12, 0x24, 0x0a,
0x0e, 0x69, 0x6e, 0x67, 0x72, 0x65, 0x73, 0x73, 0x5f, 0x76, 0x72, 0x66, 0x5f, 0x69, 0x64, 0x18,
0x27, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0c, 0x69, 0x6e, 0x67, 0x72, 0x65, 0x73, 0x73, 0x56, 0x72,
0x66, 0x49, 0x64, 0x12, 0x22, 0x0a, 0x0d, 0x65, 0x67, 0x72, 0x65, 0x73, 0x73, 0x5f, 0x76, 0x72,
0x66, 0x5f, 0x69, 0x64, 0x18, 0x28, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0b, 0x65, 0x67, 0x72, 0x65,
0x73, 0x73, 0x56, 0x72, 0x66, 0x49, 0x64, 0x12, 0x15, 0x0a, 0x06, 0x69, 0x70, 0x5f, 0x74, 0x6f,
0x73, 0x18, 0x17, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x05, 0x69, 0x70, 0x54, 0x6f, 0x73, 0x12, 0x2b,
0x0a, 0x11, 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72, 0x64, 0x69, 0x6e, 0x67, 0x5f, 0x73, 0x74, 0x61,
0x74, 0x75, 0x73, 0x18, 0x18, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x10, 0x66, 0x6f, 0x72, 0x77, 0x61,
0x72, 0x64, 0x69, 0x6e, 0x67, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x15, 0x0a, 0x06, 0x69,
0x70, 0x5f, 0x74, 0x74, 0x6c, 0x18, 0x19, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x05, 0x69, 0x70, 0x54,
0x74, 0x6c, 0x12, 0x1b, 0x0a, 0x09, 0x74, 0x63, 0x70, 0x5f, 0x66, 0x6c, 0x61, 0x67, 0x73, 0x18,
0x1a, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x08, 0x74, 0x63, 0x70, 0x46, 0x6c, 0x61, 0x67, 0x73, 0x12,
0x1b, 0x0a, 0x09, 0x69, 0x63, 0x6d, 0x70, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x1f, 0x20, 0x01,
0x28, 0x0d, 0x52, 0x08, 0x69, 0x63, 0x6d, 0x70, 0x54, 0x79, 0x70, 0x65, 0x12, 0x1b, 0x0a, 0x09,
0x69, 0x63, 0x6d, 0x70, 0x5f, 0x63, 0x6f, 0x64, 0x65, 0x18, 0x20, 0x20, 0x01, 0x28, 0x0d, 0x52,
0x08, 0x69, 0x63, 0x6d, 0x70, 0x43, 0x6f, 0x64, 0x65, 0x12, 0x26, 0x0a, 0x0f, 0x69, 0x70, 0x76,
0x36, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x6c, 0x61, 0x62, 0x65, 0x6c, 0x18, 0x25, 0x20, 0x01,
0x28, 0x0d, 0x52, 0x0d, 0x69, 0x70, 0x76, 0x36, 0x46, 0x6c, 0x6f, 0x77, 0x4c, 0x61, 0x62, 0x65,
0x6c, 0x12, 0x1f, 0x0a, 0x0b, 0x66, 0x72, 0x61, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64,
0x18, 0x23, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0a, 0x66, 0x72, 0x61, 0x67, 0x6d, 0x65, 0x6e, 0x74,
0x49, 0x64, 0x12, 0x27, 0x0a, 0x0f, 0x66, 0x72, 0x61, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x6f,
0x66, 0x66, 0x73, 0x65, 0x74, 0x18, 0x24, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0e, 0x66, 0x72, 0x61,
0x67, 0x6d, 0x65, 0x6e, 0x74, 0x4f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x12, 0x2a, 0x0a, 0x11, 0x62,
0x69, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x64, 0x69, 0x72, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e,
0x18, 0x29, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0f, 0x62, 0x69, 0x46, 0x6c, 0x6f, 0x77, 0x44, 0x69,
0x72, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x15, 0x0a, 0x06, 0x73, 0x72, 0x63, 0x5f, 0x61,
0x73, 0x18, 0x0e, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x05, 0x73, 0x72, 0x63, 0x41, 0x73, 0x12, 0x15,
0x0a, 0x06, 0x64, 0x73, 0x74, 0x5f, 0x61, 0x73, 0x18, 0x0f, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x05,
0x64, 0x73, 0x74, 0x41, 0x73, 0x12, 0x19, 0x0a, 0x08, 0x6e, 0x65, 0x78, 0x74, 0x5f, 0x68, 0x6f,
0x70, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x07, 0x6e, 0x65, 0x78, 0x74, 0x48, 0x6f, 0x70,
0x12, 0x1e, 0x0a, 0x0b, 0x6e, 0x65, 0x78, 0x74, 0x5f, 0x68, 0x6f, 0x70, 0x5f, 0x61, 0x73, 0x18,
0x0d, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x09, 0x6e, 0x65, 0x78, 0x74, 0x48, 0x6f, 0x70, 0x41, 0x73,
0x12, 0x17, 0x0a, 0x07, 0x73, 0x72, 0x63, 0x5f, 0x6e, 0x65, 0x74, 0x18, 0x10, 0x20, 0x01, 0x28,
0x0d, 0x52, 0x06, 0x73, 0x72, 0x63, 0x4e, 0x65, 0x74, 0x12, 0x17, 0x0a, 0x07, 0x64, 0x73, 0x74,
0x5f, 0x6e, 0x65, 0x74, 0x18, 0x11, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x06, 0x64, 0x73, 0x74, 0x4e,
0x65, 0x74, 0x12, 0x20, 0x0a, 0x0c, 0x62, 0x67, 0x70, 0x5f, 0x6e, 0x65, 0x78, 0x74, 0x5f, 0x68,
0x6f, 0x70, 0x18, 0x64, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0a, 0x62, 0x67, 0x70, 0x4e, 0x65, 0x78,
0x74, 0x48, 0x6f, 0x70, 0x12, 0x27, 0x0a, 0x0f, 0x62, 0x67, 0x70, 0x5f, 0x63, 0x6f, 0x6d, 0x6d,
0x75, 0x6e, 0x69, 0x74, 0x69, 0x65, 0x73, 0x18, 0x65, 0x20, 0x03, 0x28, 0x0d, 0x52, 0x0e, 0x62,
0x67, 0x70, 0x43, 0x6f, 0x6d, 0x6d, 0x75, 0x6e, 0x69, 0x74, 0x69, 0x65, 0x73, 0x12, 0x17, 0x0a,
0x07, 0x61, 0x73, 0x5f, 0x70, 0x61, 0x74, 0x68, 0x18, 0x66, 0x20, 0x03, 0x28, 0x0d, 0x52, 0x06,
0x61, 0x73, 0x50, 0x61, 0x74, 0x68, 0x12, 0x19, 0x0a, 0x08, 0x68, 0x61, 0x73, 0x5f, 0x6d, 0x70,
0x6c, 0x73, 0x18, 0x35, 0x20, 0x01, 0x28, 0x08, 0x52, 0x07, 0x68, 0x61, 0x73, 0x4d, 0x70, 0x6c,
0x73, 0x12, 0x1d, 0x0a, 0x0a, 0x6d, 0x70, 0x6c, 0x73, 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18,
0x36, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x09, 0x6d, 0x70, 0x6c, 0x73, 0x43, 0x6f, 0x75, 0x6e, 0x74,
0x12, 0x1c, 0x0a, 0x0a, 0x6d, 0x70, 0x6c, 0x73, 0x5f, 0x31, 0x5f, 0x74, 0x74, 0x6c, 0x18, 0x37,
0x20, 0x01, 0x28, 0x0d, 0x52, 0x08, 0x6d, 0x70, 0x6c, 0x73, 0x31, 0x54, 0x74, 0x6c, 0x12, 0x20,
0x0a, 0x0c, 0x6d, 0x70, 0x6c, 0x73, 0x5f, 0x31, 0x5f, 0x6c, 0x61, 0x62, 0x65, 0x6c, 0x18, 0x38,
0x20, 0x01, 0x28, 0x0d, 0x52, 0x0a, 0x6d, 0x70, 0x6c, 0x73, 0x31, 0x4c, 0x61, 0x62, 0x65, 0x6c,
0x12, 0x1c, 0x0a, 0x0a, 0x6d, 0x70, 0x6c, 0x73, 0x5f, 0x32, 0x5f, 0x74, 0x74, 0x6c, 0x18, 0x39,
0x20, 0x01, 0x28, 0x0d, 0x52, 0x08, 0x6d, 0x70, 0x6c, 0x73, 0x32, 0x54, 0x74, 0x6c, 0x12, 0x20,
0x0a, 0x0c, 0x6d, 0x70, 0x6c, 0x73, 0x5f, 0x32, 0x5f, 0x6c, 0x61, 0x62, 0x65, 0x6c, 0x18, 0x3a,
0x20, 0x01, 0x28, 0x0d, 0x52, 0x0a, 0x6d, 0x70, 0x6c, 0x73, 0x32, 0x4c, 0x61, 0x62, 0x65, 0x6c,
0x12, 0x1c, 0x0a, 0x0a, 0x6d, 0x70, 0x6c, 0x73, 0x5f, 0x33, 0x5f, 0x74, 0x74, 0x6c, 0x18, 0x3b,
0x20, 0x01, 0x28, 0x0d, 0x52, 0x08, 0x6d, 0x70, 0x6c, 0x73, 0x33, 0x54, 0x74, 0x6c, 0x12, 0x20,
0x0a, 0x0c, 0x6d, 0x70, 0x6c, 0x73, 0x5f, 0x33, 0x5f, 0x6c, 0x61, 0x62, 0x65, 0x6c, 0x18, 0x3c,
0x20, 0x01, 0x28, 0x0d, 0x52, 0x0a, 0x6d, 0x70, 0x6c, 0x73, 0x33, 0x4c, 0x61, 0x62, 0x65, 0x6c,
0x12, 0x22, 0x0a, 0x0d, 0x6d, 0x70, 0x6c, 0x73, 0x5f, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x74, 0x74,
0x6c, 0x18, 0x3d, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0b, 0x6d, 0x70, 0x6c, 0x73, 0x4c, 0x61, 0x73,
0x74, 0x54, 0x74, 0x6c, 0x12, 0x26, 0x0a, 0x0f, 0x6d, 0x70, 0x6c, 0x73, 0x5f, 0x6c, 0x61, 0x73,
0x74, 0x5f, 0x6c, 0x61, 0x62, 0x65, 0x6c, 0x18, 0x3e, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0d, 0x6d,
0x70, 0x6c, 0x73, 0x4c, 0x61, 0x73, 0x74, 0x4c, 0x61, 0x62, 0x65, 0x6c, 0x12, 0x22, 0x0a, 0x0d,
0x6d, 0x70, 0x6c, 0x73, 0x5f, 0x6c, 0x61, 0x62, 0x65, 0x6c, 0x5f, 0x69, 0x70, 0x18, 0x41, 0x20,
0x01, 0x28, 0x0c, 0x52, 0x0b, 0x6d, 0x70, 0x6c, 0x73, 0x4c, 0x61, 0x62, 0x65, 0x6c, 0x49, 0x70,
0x12, 0x32, 0x0a, 0x15, 0x6f, 0x62, 0x73, 0x65, 0x72, 0x76, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f,
0x64, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x46, 0x20, 0x01, 0x28, 0x0d, 0x52,
0x13, 0x6f, 0x62, 0x73, 0x65, 0x72, 0x76, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x44, 0x6f, 0x6d, 0x61,
0x69, 0x6e, 0x49, 0x64, 0x12, 0x30, 0x0a, 0x14, 0x6f, 0x62, 0x73, 0x65, 0x72, 0x76, 0x61, 0x74,
0x69, 0x6f, 0x6e, 0x5f, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x47, 0x20, 0x01,
0x28, 0x0d, 0x52, 0x12, 0x6f, 0x62, 0x73, 0x65, 0x72, 0x76, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x50,
0x6f, 0x69, 0x6e, 0x74, 0x49, 0x64, 0x12, 0x29, 0x0a, 0x10, 0x63, 0x75, 0x73, 0x74, 0x6f, 0x6d,
0x5f, 0x69, 0x6e, 0x74, 0x65, 0x67, 0x65, 0x72, 0x5f, 0x31, 0x18, 0xe9, 0x07, 0x20, 0x01, 0x28,
0x04, 0x52, 0x0e, 0x63, 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x49, 0x6e, 0x74, 0x65, 0x67, 0x65, 0x72,
0x31, 0x12, 0x29, 0x0a, 0x10, 0x63, 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x5f, 0x69, 0x6e, 0x74, 0x65,
0x67, 0x65, 0x72, 0x5f, 0x32, 0x18, 0xea, 0x07, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0e, 0x63, 0x75,
0x73, 0x74, 0x6f, 0x6d, 0x49, 0x6e, 0x74, 0x65, 0x67, 0x65, 0x72, 0x32, 0x12, 0x29, 0x0a, 0x10,
0x63, 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x5f, 0x69, 0x6e, 0x74, 0x65, 0x67, 0x65, 0x72, 0x5f, 0x33,
0x18, 0xeb, 0x07, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0e, 0x63, 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x49,
0x6e, 0x74, 0x65, 0x67, 0x65, 0x72, 0x33, 0x12, 0x29, 0x0a, 0x10, 0x63, 0x75, 0x73, 0x74, 0x6f,
0x6d, 0x5f, 0x69, 0x6e, 0x74, 0x65, 0x67, 0x65, 0x72, 0x5f, 0x34, 0x18, 0xec, 0x07, 0x20, 0x01,
0x28, 0x04, 0x52, 0x0e, 0x63, 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x49, 0x6e, 0x74, 0x65, 0x67, 0x65,
0x72, 0x34, 0x12, 0x29, 0x0a, 0x10, 0x63, 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x5f, 0x69, 0x6e, 0x74,
0x65, 0x67, 0x65, 0x72, 0x5f, 0x35, 0x18, 0xed, 0x07, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0e, 0x63,
0x75, 0x73, 0x74, 0x6f, 0x6d, 0x49, 0x6e, 0x74, 0x65, 0x67, 0x65, 0x72, 0x35, 0x12, 0x25, 0x0a,
0x0e, 0x63, 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x5f, 0x62, 0x79, 0x74, 0x65, 0x73, 0x5f, 0x31, 0x18,
0xf3, 0x07, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0c, 0x63, 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x42, 0x79,
0x74, 0x65, 0x73, 0x31, 0x12, 0x25, 0x0a, 0x0e, 0x63, 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x5f, 0x62,
0x79, 0x74, 0x65, 0x73, 0x5f, 0x32, 0x18, 0xf4, 0x07, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0c, 0x63,
0x75, 0x73, 0x74, 0x6f, 0x6d, 0x42, 0x79, 0x74, 0x65, 0x73, 0x32, 0x12, 0x25, 0x0a, 0x0e, 0x63,
0x75, 0x73, 0x74, 0x6f, 0x6d, 0x5f, 0x62, 0x79, 0x74, 0x65, 0x73, 0x5f, 0x33, 0x18, 0xf5, 0x07,
0x20, 0x01, 0x28, 0x0c, 0x52, 0x0c, 0x63, 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x42, 0x79, 0x74, 0x65,
0x73, 0x33, 0x12, 0x25, 0x0a, 0x0e, 0x63, 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x5f, 0x62, 0x79, 0x74,
0x65, 0x73, 0x5f, 0x34, 0x18, 0xf6, 0x07, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0c, 0x63, 0x75, 0x73,
0x74, 0x6f, 0x6d, 0x42, 0x79, 0x74, 0x65, 0x73, 0x34, 0x12, 0x25, 0x0a, 0x0e, 0x63, 0x75, 0x73,
0x74, 0x6f, 0x6d, 0x5f, 0x62, 0x79, 0x74, 0x65, 0x73, 0x5f, 0x35, 0x18, 0xf7, 0x07, 0x20, 0x01,
0x28, 0x0c, 0x52, 0x0c, 0x63, 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x42, 0x79, 0x74, 0x65, 0x73, 0x35,
0x12, 0x23, 0x0a, 0x0d, 0x63, 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x5f, 0x6c, 0x69, 0x73, 0x74, 0x5f,
0x31, 0x18, 0xfd, 0x07, 0x20, 0x03, 0x28, 0x0d, 0x52, 0x0b, 0x63, 0x75, 0x73, 0x74, 0x6f, 0x6d,
0x4c, 0x69, 0x73, 0x74, 0x31, 0x22, 0x53, 0x0a, 0x08, 0x46, 0x6c, 0x6f, 0x77, 0x54, 0x79, 0x70,
0x65, 0x12, 0x0f, 0x0a, 0x0b, 0x46, 0x4c, 0x4f, 0x57, 0x55, 0x4e, 0x4b, 0x4e, 0x4f, 0x57, 0x4e,
0x10, 0x00, 0x12, 0x0b, 0x0a, 0x07, 0x53, 0x46, 0x4c, 0x4f, 0x57, 0x5f, 0x35, 0x10, 0x01, 0x12,
0x0e, 0x0a, 0x0a, 0x4e, 0x45, 0x54, 0x46, 0x4c, 0x4f, 0x57, 0x5f, 0x56, 0x35, 0x10, 0x02, 0x12,
0x0e, 0x0a, 0x0a, 0x4e, 0x45, 0x54, 0x46, 0x4c, 0x4f, 0x57, 0x5f, 0x56, 0x39, 0x10, 0x03, 0x12,
0x09, 0x0a, 0x05, 0x49, 0x50, 0x46, 0x49, 0x58, 0x10, 0x04, 0x42, 0x29, 0x5a, 0x27, 0x67, 0x69,
0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x6e, 0x65, 0x74, 0x73, 0x61, 0x6d, 0x70,
0x6c, 0x65, 0x72, 0x2f, 0x67, 0x6f, 0x66, 0x6c, 0x6f, 0x77, 0x32, 0x2f, 0x70, 0x62, 0x3b, 0x66,
0x6c, 0x6f, 0x77, 0x70, 0x62, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
0x79, 0x70, 0x65, 0x52, 0x04, 0x74, 0x79, 0x70, 0x65, 0x12, 0x28, 0x0a, 0x10, 0x74, 0x69, 0x6d,
0x65, 0x5f, 0x72, 0x65, 0x63, 0x65, 0x69, 0x76, 0x65, 0x64, 0x5f, 0x6e, 0x73, 0x18, 0x6e, 0x20,
0x01, 0x28, 0x04, 0x52, 0x0e, 0x74, 0x69, 0x6d, 0x65, 0x52, 0x65, 0x63, 0x65, 0x69, 0x76, 0x65,
0x64, 0x4e, 0x73, 0x12, 0x21, 0x0a, 0x0c, 0x73, 0x65, 0x71, 0x75, 0x65, 0x6e, 0x63, 0x65, 0x5f,
0x6e, 0x75, 0x6d, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0b, 0x73, 0x65, 0x71, 0x75, 0x65,
0x6e, 0x63, 0x65, 0x4e, 0x75, 0x6d, 0x12, 0x23, 0x0a, 0x0d, 0x73, 0x61, 0x6d, 0x70, 0x6c, 0x69,
0x6e, 0x67, 0x5f, 0x72, 0x61, 0x74, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0c, 0x73,
0x61, 0x6d, 0x70, 0x6c, 0x69, 0x6e, 0x67, 0x52, 0x61, 0x74, 0x65, 0x12, 0x27, 0x0a, 0x0f, 0x73,
0x61, 0x6d, 0x70, 0x6c, 0x65, 0x72, 0x5f, 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x18, 0x0b,
0x20, 0x01, 0x28, 0x0c, 0x52, 0x0e, 0x73, 0x61, 0x6d, 0x70, 0x6c, 0x65, 0x72, 0x41, 0x64, 0x64,
0x72, 0x65, 0x73, 0x73, 0x12, 0x2b, 0x0a, 0x12, 0x74, 0x69, 0x6d, 0x65, 0x5f, 0x66, 0x6c, 0x6f,
0x77, 0x5f, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x6e, 0x73, 0x18, 0x6f, 0x20, 0x01, 0x28, 0x04,
0x52, 0x0f, 0x74, 0x69, 0x6d, 0x65, 0x46, 0x6c, 0x6f, 0x77, 0x53, 0x74, 0x61, 0x72, 0x74, 0x4e,
0x73, 0x12, 0x27, 0x0a, 0x10, 0x74, 0x69, 0x6d, 0x65, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x65,
0x6e, 0x64, 0x5f, 0x6e, 0x73, 0x18, 0x70, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0d, 0x74, 0x69, 0x6d,
0x65, 0x46, 0x6c, 0x6f, 0x77, 0x45, 0x6e, 0x64, 0x4e, 0x73, 0x12, 0x14, 0x0a, 0x05, 0x62, 0x79,
0x74, 0x65, 0x73, 0x18, 0x09, 0x20, 0x01, 0x28, 0x04, 0x52, 0x05, 0x62, 0x79, 0x74, 0x65, 0x73,
0x12, 0x18, 0x0a, 0x07, 0x70, 0x61, 0x63, 0x6b, 0x65, 0x74, 0x73, 0x18, 0x0a, 0x20, 0x01, 0x28,
0x04, 0x52, 0x07, 0x70, 0x61, 0x63, 0x6b, 0x65, 0x74, 0x73, 0x12, 0x19, 0x0a, 0x08, 0x73, 0x72,
0x63, 0x5f, 0x61, 0x64, 0x64, 0x72, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x07, 0x73, 0x72,
0x63, 0x41, 0x64, 0x64, 0x72, 0x12, 0x19, 0x0a, 0x08, 0x64, 0x73, 0x74, 0x5f, 0x61, 0x64, 0x64,
0x72, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x07, 0x64, 0x73, 0x74, 0x41, 0x64, 0x64, 0x72,
0x12, 0x14, 0x0a, 0x05, 0x65, 0x74, 0x79, 0x70, 0x65, 0x18, 0x1e, 0x20, 0x01, 0x28, 0x0d, 0x52,
0x05, 0x65, 0x74, 0x79, 0x70, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x18,
0x14, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x05, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x19, 0x0a, 0x08,
0x73, 0x72, 0x63, 0x5f, 0x70, 0x6f, 0x72, 0x74, 0x18, 0x15, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x07,
0x73, 0x72, 0x63, 0x50, 0x6f, 0x72, 0x74, 0x12, 0x19, 0x0a, 0x08, 0x64, 0x73, 0x74, 0x5f, 0x70,
0x6f, 0x72, 0x74, 0x18, 0x16, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x07, 0x64, 0x73, 0x74, 0x50, 0x6f,
0x72, 0x74, 0x12, 0x13, 0x0a, 0x05, 0x69, 0x6e, 0x5f, 0x69, 0x66, 0x18, 0x12, 0x20, 0x01, 0x28,
0x0d, 0x52, 0x04, 0x69, 0x6e, 0x49, 0x66, 0x12, 0x15, 0x0a, 0x06, 0x6f, 0x75, 0x74, 0x5f, 0x69,
0x66, 0x18, 0x13, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x05, 0x6f, 0x75, 0x74, 0x49, 0x66, 0x12, 0x17,
0x0a, 0x07, 0x73, 0x72, 0x63, 0x5f, 0x6d, 0x61, 0x63, 0x18, 0x1b, 0x20, 0x01, 0x28, 0x04, 0x52,
0x06, 0x73, 0x72, 0x63, 0x4d, 0x61, 0x63, 0x12, 0x17, 0x0a, 0x07, 0x64, 0x73, 0x74, 0x5f, 0x6d,
0x61, 0x63, 0x18, 0x1c, 0x20, 0x01, 0x28, 0x04, 0x52, 0x06, 0x64, 0x73, 0x74, 0x4d, 0x61, 0x63,
0x12, 0x19, 0x0a, 0x08, 0x73, 0x72, 0x63, 0x5f, 0x76, 0x6c, 0x61, 0x6e, 0x18, 0x21, 0x20, 0x01,
0x28, 0x0d, 0x52, 0x07, 0x73, 0x72, 0x63, 0x56, 0x6c, 0x61, 0x6e, 0x12, 0x19, 0x0a, 0x08, 0x64,
0x73, 0x74, 0x5f, 0x76, 0x6c, 0x61, 0x6e, 0x18, 0x22, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x07, 0x64,
0x73, 0x74, 0x56, 0x6c, 0x61, 0x6e, 0x12, 0x17, 0x0a, 0x07, 0x76, 0x6c, 0x61, 0x6e, 0x5f, 0x69,
0x64, 0x18, 0x1d, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x06, 0x76, 0x6c, 0x61, 0x6e, 0x49, 0x64, 0x12,
0x15, 0x0a, 0x06, 0x69, 0x70, 0x5f, 0x74, 0x6f, 0x73, 0x18, 0x17, 0x20, 0x01, 0x28, 0x0d, 0x52,
0x05, 0x69, 0x70, 0x54, 0x6f, 0x73, 0x12, 0x2b, 0x0a, 0x11, 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72,
0x64, 0x69, 0x6e, 0x67, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x18, 0x20, 0x01, 0x28,
0x0d, 0x52, 0x10, 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72, 0x64, 0x69, 0x6e, 0x67, 0x53, 0x74, 0x61,
0x74, 0x75, 0x73, 0x12, 0x15, 0x0a, 0x06, 0x69, 0x70, 0x5f, 0x74, 0x74, 0x6c, 0x18, 0x19, 0x20,
0x01, 0x28, 0x0d, 0x52, 0x05, 0x69, 0x70, 0x54, 0x74, 0x6c, 0x12, 0x1b, 0x0a, 0x09, 0x74, 0x63,
0x70, 0x5f, 0x66, 0x6c, 0x61, 0x67, 0x73, 0x18, 0x1a, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x08, 0x74,
0x63, 0x70, 0x46, 0x6c, 0x61, 0x67, 0x73, 0x12, 0x1b, 0x0a, 0x09, 0x69, 0x63, 0x6d, 0x70, 0x5f,
0x74, 0x79, 0x70, 0x65, 0x18, 0x1f, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x08, 0x69, 0x63, 0x6d, 0x70,
0x54, 0x79, 0x70, 0x65, 0x12, 0x1b, 0x0a, 0x09, 0x69, 0x63, 0x6d, 0x70, 0x5f, 0x63, 0x6f, 0x64,
0x65, 0x18, 0x20, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x08, 0x69, 0x63, 0x6d, 0x70, 0x43, 0x6f, 0x64,
0x65, 0x12, 0x26, 0x0a, 0x0f, 0x69, 0x70, 0x76, 0x36, 0x5f, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x6c,
0x61, 0x62, 0x65, 0x6c, 0x18, 0x25, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0d, 0x69, 0x70, 0x76, 0x36,
0x46, 0x6c, 0x6f, 0x77, 0x4c, 0x61, 0x62, 0x65, 0x6c, 0x12, 0x1f, 0x0a, 0x0b, 0x66, 0x72, 0x61,
0x67, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x23, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0a,
0x66, 0x72, 0x61, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x12, 0x27, 0x0a, 0x0f, 0x66, 0x72,
0x61, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x6f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x18, 0x24, 0x20,
0x01, 0x28, 0x0d, 0x52, 0x0e, 0x66, 0x72, 0x61, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x4f, 0x66, 0x66,
0x73, 0x65, 0x74, 0x12, 0x15, 0x0a, 0x06, 0x73, 0x72, 0x63, 0x5f, 0x61, 0x73, 0x18, 0x0e, 0x20,
0x01, 0x28, 0x0d, 0x52, 0x05, 0x73, 0x72, 0x63, 0x41, 0x73, 0x12, 0x15, 0x0a, 0x06, 0x64, 0x73,
0x74, 0x5f, 0x61, 0x73, 0x18, 0x0f, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x05, 0x64, 0x73, 0x74, 0x41,
0x73, 0x12, 0x19, 0x0a, 0x08, 0x6e, 0x65, 0x78, 0x74, 0x5f, 0x68, 0x6f, 0x70, 0x18, 0x0c, 0x20,
0x01, 0x28, 0x0c, 0x52, 0x07, 0x6e, 0x65, 0x78, 0x74, 0x48, 0x6f, 0x70, 0x12, 0x1e, 0x0a, 0x0b,
0x6e, 0x65, 0x78, 0x74, 0x5f, 0x68, 0x6f, 0x70, 0x5f, 0x61, 0x73, 0x18, 0x0d, 0x20, 0x01, 0x28,
0x0d, 0x52, 0x09, 0x6e, 0x65, 0x78, 0x74, 0x48, 0x6f, 0x70, 0x41, 0x73, 0x12, 0x17, 0x0a, 0x07,
0x73, 0x72, 0x63, 0x5f, 0x6e, 0x65, 0x74, 0x18, 0x10, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x06, 0x73,
0x72, 0x63, 0x4e, 0x65, 0x74, 0x12, 0x17, 0x0a, 0x07, 0x64, 0x73, 0x74, 0x5f, 0x6e, 0x65, 0x74,
0x18, 0x11, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x06, 0x64, 0x73, 0x74, 0x4e, 0x65, 0x74, 0x12, 0x20,
0x0a, 0x0c, 0x62, 0x67, 0x70, 0x5f, 0x6e, 0x65, 0x78, 0x74, 0x5f, 0x68, 0x6f, 0x70, 0x18, 0x64,
0x20, 0x01, 0x28, 0x0c, 0x52, 0x0a, 0x62, 0x67, 0x70, 0x4e, 0x65, 0x78, 0x74, 0x48, 0x6f, 0x70,
0x12, 0x27, 0x0a, 0x0f, 0x62, 0x67, 0x70, 0x5f, 0x63, 0x6f, 0x6d, 0x6d, 0x75, 0x6e, 0x69, 0x74,
0x69, 0x65, 0x73, 0x18, 0x65, 0x20, 0x03, 0x28, 0x0d, 0x52, 0x0e, 0x62, 0x67, 0x70, 0x43, 0x6f,
0x6d, 0x6d, 0x75, 0x6e, 0x69, 0x74, 0x69, 0x65, 0x73, 0x12, 0x17, 0x0a, 0x07, 0x61, 0x73, 0x5f,
0x70, 0x61, 0x74, 0x68, 0x18, 0x66, 0x20, 0x03, 0x28, 0x0d, 0x52, 0x06, 0x61, 0x73, 0x50, 0x61,
0x74, 0x68, 0x12, 0x19, 0x0a, 0x08, 0x6d, 0x70, 0x6c, 0x73, 0x5f, 0x74, 0x74, 0x6c, 0x18, 0x50,
0x20, 0x03, 0x28, 0x0d, 0x52, 0x07, 0x6d, 0x70, 0x6c, 0x73, 0x54, 0x74, 0x6c, 0x12, 0x1d, 0x0a,
0x0a, 0x6d, 0x70, 0x6c, 0x73, 0x5f, 0x6c, 0x61, 0x62, 0x65, 0x6c, 0x18, 0x51, 0x20, 0x03, 0x28,
0x0d, 0x52, 0x09, 0x6d, 0x70, 0x6c, 0x73, 0x4c, 0x61, 0x62, 0x65, 0x6c, 0x12, 0x17, 0x0a, 0x07,
0x6d, 0x70, 0x6c, 0x73, 0x5f, 0x69, 0x70, 0x18, 0x52, 0x20, 0x03, 0x28, 0x0c, 0x52, 0x06, 0x6d,
0x70, 0x6c, 0x73, 0x49, 0x70, 0x12, 0x32, 0x0a, 0x15, 0x6f, 0x62, 0x73, 0x65, 0x72, 0x76, 0x61,
0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x64, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x46,
0x20, 0x01, 0x28, 0x0d, 0x52, 0x13, 0x6f, 0x62, 0x73, 0x65, 0x72, 0x76, 0x61, 0x74, 0x69, 0x6f,
0x6e, 0x44, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x49, 0x64, 0x12, 0x30, 0x0a, 0x14, 0x6f, 0x62, 0x73,
0x65, 0x72, 0x76, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x5f, 0x69,
0x64, 0x18, 0x47, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x12, 0x6f, 0x62, 0x73, 0x65, 0x72, 0x76, 0x61,
0x74, 0x69, 0x6f, 0x6e, 0x50, 0x6f, 0x69, 0x6e, 0x74, 0x49, 0x64, 0x22, 0x53, 0x0a, 0x08, 0x46,
0x6c, 0x6f, 0x77, 0x54, 0x79, 0x70, 0x65, 0x12, 0x0f, 0x0a, 0x0b, 0x46, 0x4c, 0x4f, 0x57, 0x55,
0x4e, 0x4b, 0x4e, 0x4f, 0x57, 0x4e, 0x10, 0x00, 0x12, 0x0b, 0x0a, 0x07, 0x53, 0x46, 0x4c, 0x4f,
0x57, 0x5f, 0x35, 0x10, 0x01, 0x12, 0x0e, 0x0a, 0x0a, 0x4e, 0x45, 0x54, 0x46, 0x4c, 0x4f, 0x57,
0x5f, 0x56, 0x35, 0x10, 0x02, 0x12, 0x0e, 0x0a, 0x0a, 0x4e, 0x45, 0x54, 0x46, 0x4c, 0x4f, 0x57,
0x5f, 0x56, 0x39, 0x10, 0x03, 0x12, 0x09, 0x0a, 0x05, 0x49, 0x50, 0x46, 0x49, 0x58, 0x10, 0x04,
0x42, 0x29, 0x5a, 0x27, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x6e,
0x65, 0x74, 0x73, 0x61, 0x6d, 0x70, 0x6c, 0x65, 0x72, 0x2f, 0x67, 0x6f, 0x66, 0x6c, 0x6f, 0x77,
0x32, 0x2f, 0x70, 0x62, 0x3b, 0x66, 0x6c, 0x6f, 0x77, 0x70, 0x62, 0x62, 0x06, 0x70, 0x72, 0x6f,
0x74, 0x6f, 0x33,
}
var (

View File

@@ -13,20 +13,18 @@ message FlowMessage {
}
FlowType type = 1;
uint64 time_received = 2;
uint64 time_received_ns = 110;
uint32 sequence_num = 4;
uint64 sampling_rate = 3;
uint32 flow_direction = 42;
//uint32 flow_direction = 42;
// Sampler information
bytes sampler_address = 11;
// Found inside packet
uint64 time_flow_start = 38;
uint64 time_flow_end = 5;
uint64 time_flow_start_ms = 63;
uint64 time_flow_end_ms = 64;
uint64 time_flow_start_ns = 111;
uint64 time_flow_end_ns = 112;
// Size of the sampled packet
uint64 bytes = 9;
@@ -60,10 +58,6 @@ message FlowMessage {
// 802.1q VLAN in sampled packet
uint32 vlan_id = 29;
// VRF
uint32 ingress_vrf_id = 39;
uint32 egress_vrf_id = 40;
// IP and TCP special flags
uint32 ip_tos = 23;
uint32 forwarding_status = 24;
@@ -75,7 +69,6 @@ message FlowMessage {
// Fragments (IPv4/IPv6)
uint32 fragment_id = 35;
uint32 fragment_offset = 36;
uint32 bi_flow_direction = 41;
// Autonomous system information
uint32 src_as = 14;
@@ -94,38 +87,10 @@ message FlowMessage {
repeated uint32 as_path = 102;
// MPLS information
bool has_mpls = 53;
uint32 mpls_count = 54;
uint32 mpls_1_ttl = 55; // First TTL
uint32 mpls_1_label = 56; // First Label
uint32 mpls_2_ttl = 57; // Second TTL
uint32 mpls_2_label = 58; // Second Label
uint32 mpls_3_ttl = 59; // Third TTL
uint32 mpls_3_label = 60; // Third Label
uint32 mpls_last_ttl = 61; // Last TTL
uint32 mpls_last_label = 62; // Last Label
bytes mpls_label_ip = 65; // MPLS TOP Label IP
repeated uint32 mpls_ttl = 80;
repeated uint32 mpls_label = 81;
repeated bytes mpls_ip = 82;
uint32 observation_domain_id = 70;
uint32 observation_point_id = 71;
// Custom fields: start after ID 1000:
// uint32 my_custom_field = 1000;
// Custom allocations
uint64 custom_integer_1 = 1001;
uint64 custom_integer_2 = 1002;
uint64 custom_integer_3 = 1003;
uint64 custom_integer_4 = 1004;
uint64 custom_integer_5 = 1005;
bytes custom_bytes_1 = 1011;
bytes custom_bytes_2 = 1012;
bytes custom_bytes_3 = 1013;
bytes custom_bytes_4 = 1014;
bytes custom_bytes_5 = 1015;
repeated uint32 custom_list_1 = 1021;
}

24
producer/producer.go Normal file
View File

@@ -0,0 +1,24 @@
package producer
import (
"net/netip"
"time"
)
// Interface of the messages
type ProducerMessage interface{}
type ProducerInterface interface {
// Converts a message into a list of flow samples
Produce(msg interface{}, args *ProduceArgs) ([]ProducerMessage, error)
// Indicates to the producer the messages returned were processed
Commit([]ProducerMessage)
Close()
}
type ProduceArgs struct {
Src netip.AddrPort
Dst netip.AddrPort
SamplerAddress netip.Addr
TimeReceived time.Time
}

View File

@@ -1,81 +0,0 @@
package producer
import (
"encoding/binary"
"errors"
"net"
"github.com/netsampler/goflow2/decoders/netflowlegacy"
flowmessage "github.com/netsampler/goflow2/pb"
)
func ConvertNetFlowLegacyRecord(baseTime uint32, uptime uint32, record netflowlegacy.RecordsNetFlowV5) *flowmessage.FlowMessage {
flowMessage := &flowmessage.FlowMessage{}
flowMessage.Type = flowmessage.FlowMessage_NETFLOW_V5
timeDiffFirst := (uptime - record.First)
timeDiffLast := (uptime - record.Last)
flowMessage.TimeFlowStart = uint64(baseTime - timeDiffFirst/1000)
flowMessage.TimeFlowStartMs = uint64(baseTime)*1000 - uint64(timeDiffFirst)
flowMessage.TimeFlowEnd = uint64(baseTime - timeDiffLast/1000)
flowMessage.TimeFlowEndMs = uint64(baseTime)*1000 - uint64(timeDiffLast)
v := make(net.IP, 4)
binary.BigEndian.PutUint32(v, record.NextHop)
flowMessage.NextHop = v
v = make(net.IP, 4)
binary.BigEndian.PutUint32(v, record.SrcAddr)
flowMessage.SrcAddr = v
v = make(net.IP, 4)
binary.BigEndian.PutUint32(v, record.DstAddr)
flowMessage.DstAddr = v
flowMessage.Etype = 0x800
flowMessage.SrcAs = uint32(record.SrcAS)
flowMessage.DstAs = uint32(record.DstAS)
flowMessage.SrcNet = uint32(record.SrcMask)
flowMessage.DstNet = uint32(record.DstMask)
flowMessage.Proto = uint32(record.Proto)
flowMessage.TcpFlags = uint32(record.TCPFlags)
flowMessage.IpTos = uint32(record.Tos)
flowMessage.InIf = uint32(record.Input)
flowMessage.OutIf = uint32(record.Output)
flowMessage.SrcPort = uint32(record.SrcPort)
flowMessage.DstPort = uint32(record.DstPort)
flowMessage.Packets = uint64(record.DPkts)
flowMessage.Bytes = uint64(record.DOctets)
return flowMessage
}
func SearchNetFlowLegacyRecords(baseTime uint32, uptime uint32, dataRecords []netflowlegacy.RecordsNetFlowV5) []*flowmessage.FlowMessage {
var flowMessageSet []*flowmessage.FlowMessage
for _, record := range dataRecords {
fmsg := ConvertNetFlowLegacyRecord(baseTime, uptime, record)
if fmsg != nil {
flowMessageSet = append(flowMessageSet, fmsg)
}
}
return flowMessageSet
}
func ProcessMessageNetFlowLegacy(msgDec interface{}) ([]*flowmessage.FlowMessage, error) {
switch packet := msgDec.(type) {
case netflowlegacy.PacketNetFlowV5:
seqnum := packet.FlowSequence
samplingRate := packet.SamplingInterval
baseTime := packet.UnixSecs
uptime := packet.SysUptime
flowMessageSet := SearchNetFlowLegacyRecords(baseTime, uptime, packet.Records)
for _, fmsg := range flowMessageSet {
fmsg.SequenceNum = seqnum
fmsg.SamplingRate = uint64(samplingRate)
}
return flowMessageSet, nil
default:
return []*flowmessage.FlowMessage{}, errors.New("Bad NetFlow v5 version")
}
}

View File

@@ -1,349 +0,0 @@
package producer
import (
"encoding/binary"
"errors"
"net"
"github.com/netsampler/goflow2/decoders/sflow"
flowmessage "github.com/netsampler/goflow2/pb"
)
func GetSFlowFlowSamples(packet *sflow.Packet) []interface{} {
var flowSamples []interface{}
for _, sample := range packet.Samples {
switch sample.(type) {
case sflow.FlowSample:
flowSamples = append(flowSamples, sample)
case sflow.ExpandedFlowSample:
flowSamples = append(flowSamples, sample)
}
}
return flowSamples
}
func ParseSampledHeader(flowMessage *flowmessage.FlowMessage, sampledHeader *sflow.SampledHeader) error {
return ParseSampledHeaderConfig(flowMessage, sampledHeader, nil)
}
func ParseEthernetHeader(flowMessage *flowmessage.FlowMessage, data []byte, config *SFlowMapper) {
var hasMpls bool
var countMpls uint32
var firstLabelMpls uint32
var firstTtlMpls uint8
var secondLabelMpls uint32
var secondTtlMpls uint8
var thirdLabelMpls uint32
var thirdTtlMpls uint8
var lastLabelMpls uint32
var lastTtlMpls uint8
var nextHeader byte
var tcpflags byte
srcIP := net.IP{}
dstIP := net.IP{}
offset := 14
var srcMac uint64
var dstMac uint64
var tos byte
var ttl byte
var identification uint16
var fragOffset uint16
var flowLabel uint32
var srcPort uint16
var dstPort uint16
for _, configLayer := range GetSFlowConfigLayer(config, 0) {
extracted := GetBytes(data, configLayer.Offset, configLayer.Length)
MapCustom(flowMessage, extracted, configLayer.Destination, configLayer.Endian)
}
etherType := data[12:14]
dstMac = binary.BigEndian.Uint64(append([]byte{0, 0}, data[0:6]...))
srcMac = binary.BigEndian.Uint64(append([]byte{0, 0}, data[6:12]...))
(*flowMessage).SrcMac = srcMac
(*flowMessage).DstMac = dstMac
encap := true
iterations := 0
for encap && iterations <= 1 {
encap = false
if etherType[0] == 0x81 && etherType[1] == 0x0 { // VLAN 802.1Q
(*flowMessage).VlanId = uint32(binary.BigEndian.Uint16(data[14:16]))
offset += 4
etherType = data[16:18]
}
if etherType[0] == 0x88 && etherType[1] == 0x47 { // MPLS
iterateMpls := true
hasMpls = true
for iterateMpls {
if len(data) < offset+5 {
iterateMpls = false
break
}
label := binary.BigEndian.Uint32(append([]byte{0}, data[offset:offset+3]...)) >> 4
//exp := data[offset+2] > 1
bottom := data[offset+2] & 1
mplsTtl := data[offset+3]
offset += 4
if bottom == 1 || label <= 15 || offset > len(data) {
if data[offset]&0xf0>>4 == 4 {
etherType = []byte{0x8, 0x0}
} else if data[offset]&0xf0>>4 == 6 {
etherType = []byte{0x86, 0xdd}
}
iterateMpls = false
}
if countMpls == 0 {
firstLabelMpls = label
firstTtlMpls = mplsTtl
} else if countMpls == 1 {
secondLabelMpls = label
secondTtlMpls = mplsTtl
} else if countMpls == 2 {
thirdLabelMpls = label
thirdTtlMpls = mplsTtl
} else {
lastLabelMpls = label
lastTtlMpls = mplsTtl
}
countMpls++
}
}
for _, configLayer := range GetSFlowConfigLayer(config, 3) {
extracted := GetBytes(data, offset*8+configLayer.Offset, configLayer.Length)
MapCustom(flowMessage, extracted, configLayer.Destination, configLayer.Endian)
}
if etherType[0] == 0x8 && etherType[1] == 0x0 { // IPv4
if len(data) >= offset+20 {
nextHeader = data[offset+9]
srcIP = data[offset+12 : offset+16]
dstIP = data[offset+16 : offset+20]
tos = data[offset+1]
ttl = data[offset+8]
identification = binary.BigEndian.Uint16(data[offset+4 : offset+6])
fragOffset = binary.BigEndian.Uint16(data[offset+6 : offset+8])
offset += 20
}
} else if etherType[0] == 0x86 && etherType[1] == 0xdd { // IPv6
if len(data) >= offset+40 {
nextHeader = data[offset+6]
srcIP = data[offset+8 : offset+24]
dstIP = data[offset+24 : offset+40]
tostmp := uint32(binary.BigEndian.Uint16(data[offset : offset+2]))
tos = uint8(tostmp & 0x0ff0 >> 4)
ttl = data[offset+7]
flowLabel = binary.BigEndian.Uint32(data[offset : offset+4])
offset += 40
}
} else if etherType[0] == 0x8 && etherType[1] == 0x6 { // ARP
} /*else {
return errors.New(fmt.Sprintf("Unknown EtherType: %v\n", etherType))
} */
for _, configLayer := range GetSFlowConfigLayer(config, 4) {
extracted := GetBytes(data, offset*8+configLayer.Offset, configLayer.Length)
MapCustom(flowMessage, extracted, configLayer.Destination, configLayer.Endian)
}
appOffset := 0
if len(data) >= offset+4 && (nextHeader == 17 || nextHeader == 6) {
srcPort = binary.BigEndian.Uint16(data[offset+0 : offset+2])
dstPort = binary.BigEndian.Uint16(data[offset+2 : offset+4])
}
if nextHeader == 17 {
appOffset = 8
}
if len(data) > offset+13 && nextHeader == 6 {
tcpflags = data[offset+13]
appOffset = int(data[13]>>4) * 4
}
// ICMP and ICMPv6
if len(data) >= offset+2 && (nextHeader == 1 || nextHeader == 58) {
(*flowMessage).IcmpType = uint32(data[offset+0])
(*flowMessage).IcmpCode = uint32(data[offset+1])
}
if appOffset > 0 {
for _, configLayer := range GetSFlowConfigLayer(config, 7) {
extracted := GetBytes(data, (offset+appOffset)*8+configLayer.Offset, configLayer.Length)
MapCustom(flowMessage, extracted, configLayer.Destination, configLayer.Endian)
}
}
iterations++
}
(*flowMessage).HasMpls = hasMpls
(*flowMessage).MplsCount = countMpls
(*flowMessage).Mpls_1Label = firstLabelMpls
(*flowMessage).Mpls_1Ttl = uint32(firstTtlMpls)
(*flowMessage).Mpls_2Label = secondLabelMpls
(*flowMessage).Mpls_2Ttl = uint32(secondTtlMpls)
(*flowMessage).Mpls_3Label = thirdLabelMpls
(*flowMessage).Mpls_3Ttl = uint32(thirdTtlMpls)
(*flowMessage).MplsLastLabel = lastLabelMpls
(*flowMessage).MplsLastTtl = uint32(lastTtlMpls)
(*flowMessage).Etype = uint32(binary.BigEndian.Uint16(etherType[0:2]))
(*flowMessage).Ipv6FlowLabel = flowLabel & 0xFFFFF
(*flowMessage).SrcPort = uint32(srcPort)
(*flowMessage).DstPort = uint32(dstPort)
(*flowMessage).SrcAddr = srcIP
(*flowMessage).DstAddr = dstIP
(*flowMessage).Proto = uint32(nextHeader)
(*flowMessage).IpTos = uint32(tos)
(*flowMessage).IpTtl = uint32(ttl)
(*flowMessage).TcpFlags = uint32(tcpflags)
(*flowMessage).FragmentId = uint32(identification)
(*flowMessage).FragmentOffset = uint32(fragOffset)
}
func ParseSampledHeaderConfig(flowMessage *flowmessage.FlowMessage, sampledHeader *sflow.SampledHeader, config *SFlowMapper) error {
data := (*sampledHeader).HeaderData
switch (*sampledHeader).Protocol {
case 1: // Ethernet
ParseEthernetHeader(flowMessage, data, config)
}
return nil
}
func SearchSFlowSamples(samples []interface{}) []*flowmessage.FlowMessage {
return SearchSFlowSamples(samples)
}
func SearchSFlowSamplesConfig(samples []interface{}, config *SFlowMapper) []*flowmessage.FlowMessage {
var flowMessageSet []*flowmessage.FlowMessage
for _, flowSample := range samples {
var records []sflow.FlowRecord
flowMessage := &flowmessage.FlowMessage{}
flowMessage.Type = flowmessage.FlowMessage_SFLOW_5
switch flowSample := flowSample.(type) {
case sflow.FlowSample:
records = flowSample.Records
flowMessage.SamplingRate = uint64(flowSample.SamplingRate)
flowMessage.InIf = flowSample.Input
flowMessage.OutIf = flowSample.Output
case sflow.ExpandedFlowSample:
records = flowSample.Records
flowMessage.SamplingRate = uint64(flowSample.SamplingRate)
flowMessage.InIf = flowSample.InputIfValue
flowMessage.OutIf = flowSample.OutputIfValue
}
ipNh := net.IP{}
ipSrc := net.IP{}
ipDst := net.IP{}
flowMessage.Packets = 1
for _, record := range records {
switch recordData := record.Data.(type) {
case sflow.SampledHeader:
flowMessage.Bytes = uint64(recordData.FrameLength)
ParseSampledHeaderConfig(flowMessage, &recordData, config)
case sflow.SampledIPv4:
ipSrc = recordData.Base.SrcIP
ipDst = recordData.Base.DstIP
flowMessage.SrcAddr = ipSrc
flowMessage.DstAddr = ipDst
flowMessage.Bytes = uint64(recordData.Base.Length)
flowMessage.Proto = recordData.Base.Protocol
flowMessage.SrcPort = recordData.Base.SrcPort
flowMessage.DstPort = recordData.Base.DstPort
flowMessage.IpTos = recordData.Tos
flowMessage.Etype = 0x800
case sflow.SampledIPv6:
ipSrc = recordData.Base.SrcIP
ipDst = recordData.Base.DstIP
flowMessage.SrcAddr = ipSrc
flowMessage.DstAddr = ipDst
flowMessage.Bytes = uint64(recordData.Base.Length)
flowMessage.Proto = recordData.Base.Protocol
flowMessage.SrcPort = recordData.Base.SrcPort
flowMessage.DstPort = recordData.Base.DstPort
flowMessage.IpTos = recordData.Priority
flowMessage.Etype = 0x86dd
case sflow.ExtendedRouter:
ipNh = recordData.NextHop
flowMessage.NextHop = ipNh
flowMessage.SrcNet = recordData.SrcMaskLen
flowMessage.DstNet = recordData.DstMaskLen
case sflow.ExtendedGateway:
ipNh = recordData.NextHop
flowMessage.BgpNextHop = ipNh
flowMessage.BgpCommunities = recordData.Communities
flowMessage.AsPath = recordData.ASPath
if len(recordData.ASPath) > 0 {
flowMessage.DstAs = recordData.ASPath[len(recordData.ASPath)-1]
flowMessage.NextHopAs = recordData.ASPath[0]
} else {
flowMessage.DstAs = recordData.AS
}
if recordData.SrcAS > 0 {
flowMessage.SrcAs = recordData.SrcAS
} else {
flowMessage.SrcAs = recordData.AS
}
case sflow.ExtendedSwitch:
flowMessage.SrcVlan = recordData.SrcVlan
flowMessage.DstVlan = recordData.DstVlan
}
}
flowMessageSet = append(flowMessageSet, flowMessage)
}
return flowMessageSet
}
func ProcessMessageSFlow(msgDec interface{}) ([]*flowmessage.FlowMessage, error) {
return ProcessMessageSFlowConfig(msgDec, nil)
}
func ProcessMessageSFlowConfig(msgDec interface{}, config *ProducerConfigMapped) ([]*flowmessage.FlowMessage, error) {
switch packet := msgDec.(type) {
case sflow.Packet:
seqnum := packet.SequenceNumber
var agent net.IP
agent = packet.AgentIP
var cfg *SFlowMapper
if config != nil {
cfg = config.SFlow
}
flowSamples := GetSFlowFlowSamples(&packet)
flowMessageSet := SearchSFlowSamplesConfig(flowSamples, cfg)
for _, fmsg := range flowMessageSet {
fmsg.SamplerAddress = agent
fmsg.SequenceNum = seqnum
}
return flowMessageSet, nil
default:
return []*flowmessage.FlowMessage{}, errors.New("Bad sFlow version")
}
}

320
producer/proto/custom.go Normal file
View File

@@ -0,0 +1,320 @@
package protoproducer
import (
"fmt"
"reflect"
"github.com/netsampler/goflow2/v2/decoders/netflow"
)
type NetFlowMapField struct {
PenProvided bool `yaml:"penprovided"`
Type uint16 `yaml:"field"`
Pen uint32 `yaml:"pen"`
Destination string `yaml:"destination"`
Endian EndianType `yaml:"endianness"`
//DestinationLength uint8 `json:"dlen"` // could be used if populating a slice of uint16 that aren't in protobuf
}
type IPFIXProducerConfig struct {
Mapping []NetFlowMapField `yaml:"mapping"`
//PacketMapping []SFlowMapField `json:"packet-mapping"` // for embedded frames: use sFlow configuration
}
type NetFlowV9ProducerConfig struct {
Mapping []NetFlowMapField `json:"mapping"`
}
type SFlowMapField struct {
Layer string `yaml:"layer"`
Offset int `yaml:"offset"` // offset in bits
Length int `yaml:"length"` // length in bits
Destination string `yaml:"destination"`
Endian EndianType `yaml:"endianness"`
//DestinationLength uint8 `json:"dlen"`
}
type SFlowProducerConfig struct {
Mapping []SFlowMapField `yaml:"mapping"`
}
type ProtobufFormatterConfig struct {
Name string
Index int32
Type string
Array bool
}
type FormatterConfig struct {
Fields []string `yaml:"fields"`
Key []string `yaml:"key"`
Render map[string]RendererID `yaml:"render"`
Rename map[string]string `yaml:"rename"`
Protobuf []ProtobufFormatterConfig `yaml:"protobuf"`
}
type ProducerConfig struct {
Formatter FormatterConfig `yaml:"formatter"`
IPFIX IPFIXProducerConfig `yaml:"ipfix"`
NetFlowV9 NetFlowV9ProducerConfig `yaml:"netflowv9"`
SFlow SFlowProducerConfig `yaml:"sflow"` // also used for IPFIX data frames
// should do a rename map list for when printing
}
type DataMap struct {
MapConfigBase
}
type FormatterConfigMapper struct {
fields []string
key []string
reMap map[string]string // map from a potential json name into the protobuf structure
rename map[string]string // manually renaming fields
render map[string]RenderFunc
pbMap map[string]ProtobufFormatterConfig
numToPb map[int32]ProtobufFormatterConfig
isSlice map[string]bool
}
type NetFlowMapper struct {
data map[string]DataMap // maps field to destination
}
func (m *NetFlowMapper) Map(field netflow.DataField) (DataMap, bool) {
mapped, found := m.data[fmt.Sprintf("%v-%d-%d", field.PenProvided, field.Pen, field.Type)]
return mapped, found
}
type DataMapLayer struct {
MapConfigBase
Offset int
Length int
}
type SFlowMapper struct {
data map[string][]DataMapLayer // map layer to list of offsets
}
func GetSFlowConfigLayer(m *SFlowMapper, layer string) []DataMapLayer {
if m == nil {
return nil
}
return m.data[layer]
}
func mapFieldsSFlow(fields []SFlowMapField) *SFlowMapper {
ret := make(map[string][]DataMapLayer)
for _, field := range fields {
retLayerEntry := DataMapLayer{
Offset: field.Offset,
Length: field.Length,
}
retLayerEntry.Destination = field.Destination
retLayerEntry.Endianness = field.Endian
retLayer := ret[field.Layer]
retLayer = append(retLayer, retLayerEntry)
ret[field.Layer] = retLayer
}
return &SFlowMapper{ret}
}
func mapFieldsNetFlow(fields []NetFlowMapField) *NetFlowMapper {
ret := make(map[string]DataMap)
for _, field := range fields {
dm := DataMap{}
dm.Destination = field.Destination
dm.Endianness = field.Endian
ret[fmt.Sprintf("%v-%d-%d", field.PenProvided, field.Pen, field.Type)] = dm
}
return &NetFlowMapper{ret}
}
type producerConfigMapped struct {
Formatter *FormatterConfigMapper
IPFIX *NetFlowMapper
NetFlowV9 *NetFlowMapper
SFlow *SFlowMapper
}
func (c *producerConfigMapped) finalizemapDest(v *MapConfigBase) error {
if vv, ok := c.Formatter.pbMap[v.Destination]; ok {
v.ProtoIndex = vv.Index
if pt, ok := ProtoTypeMap[vv.Type]; ok {
v.ProtoType = pt
} else {
return fmt.Errorf("could not map %s to a ProtoType", vv.Type)
}
v.ProtoArray = vv.Array
}
return nil
}
func (c *producerConfigMapped) finalizeSFlowMapper(m *SFlowMapper) error {
if m == nil {
return nil
}
for k, vlist := range m.data {
for i, v := range vlist {
if err := c.finalizemapDest(&(v.MapConfigBase)); err != nil {
return err
}
m.data[k][i] = v
}
}
return nil
}
func (c *producerConfigMapped) finalizeNetFlowMapper(m *NetFlowMapper) error {
if m == nil {
return nil
}
for k, v := range m.data {
if err := c.finalizemapDest(&(v.MapConfigBase)); err != nil {
return err
}
m.data[k] = v
}
return nil
}
func (c *producerConfigMapped) finalize() error {
if c.Formatter == nil {
return nil
}
if err := c.finalizeNetFlowMapper(c.IPFIX); err != nil {
return err
}
if err := c.finalizeNetFlowMapper(c.NetFlowV9); err != nil {
return err
}
if err := c.finalizeSFlowMapper(c.SFlow); err != nil {
return err
}
return nil
}
func mapFormat(cfg *ProducerConfig) (*FormatterConfigMapper, error) {
formatterMapped := &FormatterConfigMapper{}
selectorTag := "json"
var msg ProtoProducerMessage
msgT := reflect.TypeOf(&msg.FlowMessage).Elem() // required indirect otherwise go vet indicates TypeOf copies lock
reMap := make(map[string]string)
numToPb := make(map[int32]ProtobufFormatterConfig)
var fields []string
for i := 0; i < msgT.NumField(); i++ {
field := msgT.Field(i)
if !field.IsExported() {
continue
}
fieldName := field.Name
if selectorTag != "" {
fieldName = ExtractTag(selectorTag, fieldName, field.Tag)
reMap[fieldName] = field.Name
fields = append(fields, fieldName)
}
//customSelectorTmp[i] = fieldName
}
formatterMapped.reMap = reMap
pbMap := make(map[string]ProtobufFormatterConfig)
formatterMapped.render = make(map[string]RenderFunc)
formatterMapped.rename = make(map[string]string)
formatterMapped.isSlice = map[string]bool{
"BgpCommunities": true,
"AsPath": true,
"MplsIp": true,
"MplsLabel": true,
"MplsTtl": true,
} // todo: improve this with defaults
for k, v := range defaultRenderers {
formatterMapped.render[k] = v
}
if cfg != nil {
cfgFormatter := cfg.Formatter
// manual protobuf fields to add
for _, pbField := range cfgFormatter.Protobuf {
reMap[pbField.Name] = "" // special dynamic protobuf
pbMap[pbField.Name] = pbField // todo: check if type is valid
numToPb[pbField.Index] = pbField
formatterMapped.isSlice[pbField.Name] = pbField.Array
}
// populate manual renames
for k, v := range cfgFormatter.Rename {
formatterMapped.rename[k] = v
}
// populate key
for _, v := range cfgFormatter.Key {
if _, ok := reMap[v]; !ok {
return formatterMapped, fmt.Errorf("key field %s does not exist", v)
}
formatterMapped.key = append(formatterMapped.key, v)
}
// process renderers
for k, v := range cfgFormatter.Render {
if kk, ok := reMap[k]; ok && kk != "" {
k = kk
}
if renderer, ok := renderers[v]; ok {
formatterMapped.render[k] = renderer
} else {
return formatterMapped, fmt.Errorf("field %s is not a renderer", v) // todo: make proper error
}
}
// if the config does not contain any fields initially, we set with the protobuf ones
if len(cfgFormatter.Fields) == 0 {
formatterMapped.fields = fields
} else {
for _, field := range cfgFormatter.Fields {
if _, ok := reMap[field]; !ok {
// check if it's a virtual field
if _, ok := formatterMapped.render[field]; !ok {
return formatterMapped, fmt.Errorf("field %s in config not found in protobuf", field) // todo: make proper error
}
}
}
formatterMapped.fields = cfgFormatter.Fields
}
formatterMapped.pbMap = pbMap
formatterMapped.numToPb = numToPb
} else {
formatterMapped.fields = fields
}
return formatterMapped, nil
}
func mapConfig(cfg *ProducerConfig) (*producerConfigMapped, error) {
newCfg := &producerConfigMapped{}
if cfg != nil {
newCfg.IPFIX = mapFieldsNetFlow(cfg.IPFIX.Mapping)
newCfg.NetFlowV9 = mapFieldsNetFlow(cfg.NetFlowV9.Mapping)
newCfg.SFlow = mapFieldsSFlow(cfg.SFlow.Mapping)
}
var err error
if newCfg.Formatter, err = mapFormat(cfg); err != nil {
return newCfg, err
}
return newCfg, newCfg.finalize()
}

245
producer/proto/messages.go Normal file
View File

@@ -0,0 +1,245 @@
package protoproducer
import (
"bytes"
"encoding/hex"
"fmt"
"hash"
"hash/fnv"
"reflect"
"strings"
"sync"
"google.golang.org/protobuf/encoding/protodelim"
"google.golang.org/protobuf/encoding/protowire"
flowmessage "github.com/netsampler/goflow2/v2/pb"
)
type ProtoProducerMessage struct {
flowmessage.FlowMessage
formatter *FormatterConfigMapper
}
var protoMessagePool = sync.Pool{
New: func() any {
return &ProtoProducerMessage{}
},
}
func (m *ProtoProducerMessage) MarshalBinary() ([]byte, error) {
buf := bytes.NewBuffer([]byte{})
_, err := protodelim.MarshalTo(buf, m)
return buf.Bytes(), err
}
func (m *ProtoProducerMessage) MarshalText() ([]byte, error) {
return []byte(m.FormatMessageReflectText("")), nil
}
func (m *ProtoProducerMessage) baseKey(h hash.Hash) {
if m.formatter == nil || len(m.formatter.key) == 0 {
return
}
vfm := reflect.ValueOf(m)
vfm = reflect.Indirect(vfm)
unkMap := m.mapUnknown() // todo: should be able to reuse if set in structure
for _, s := range m.formatter.key {
fieldName := s
// get original name from structure
if fieldNameMap, ok := m.formatter.reMap[fieldName]; ok && fieldNameMap != "" {
fieldName = fieldNameMap
}
fieldValue := vfm.FieldByName(fieldName)
// if does not exist from structure,
// fetch from unknown (only numbered) fields
// that were parsed above
if !fieldValue.IsValid() {
if unkField, ok := unkMap[s]; ok {
fieldValue = reflect.ValueOf(unkField)
} else {
continue
}
}
h.Write([]byte(fmt.Sprintf("%v", fieldValue.Interface())))
}
}
func (m *ProtoProducerMessage) Key() []byte {
h := fnv.New32()
m.baseKey(h)
return h.Sum(nil)
}
func (m *ProtoProducerMessage) MarshalJSON() ([]byte, error) {
return []byte(m.FormatMessageReflectJSON("")), nil
}
func (m *ProtoProducerMessage) FormatMessageReflectText(ext string) string {
return m.FormatMessageReflectCustom(ext, "", " ", "=", false)
}
func (m *ProtoProducerMessage) FormatMessageReflectJSON(ext string) string {
return fmt.Sprintf("{%s}", m.FormatMessageReflectCustom(ext, "\"", ",", ":", true))
}
func ExtractTag(name, original string, tag reflect.StructTag) string {
lookup, ok := tag.Lookup(name)
if !ok {
return original
}
before, _, _ := strings.Cut(lookup, ",")
return before
}
func (m *ProtoProducerMessage) mapUnknown() map[string]interface{} {
unkMap := make(map[string]interface{})
fmr := m.ProtoReflect()
unk := fmr.GetUnknown()
var offset int
for offset < len(unk) {
num, dataType, length := protowire.ConsumeTag(unk[offset:])
offset += length
length = protowire.ConsumeFieldValue(num, dataType, unk[offset:])
data := unk[offset : offset+length]
offset += length
// we check if the index is listed in the config
if pbField, ok := m.formatter.numToPb[int32(num)]; ok {
var dest interface{}
var value interface{}
if dataType == protowire.VarintType {
v, _ := protowire.ConsumeVarint(data)
value = v
} else if dataType == protowire.BytesType {
v, _ := protowire.ConsumeString(data)
value = hex.EncodeToString([]byte(v))
} else {
continue
}
if pbField.Array {
var destSlice []interface{}
if dest, ok := unkMap[pbField.Name]; !ok {
destSlice = make([]interface{}, 0)
} else {
destSlice = dest.([]interface{})
}
destSlice = append(destSlice, value)
dest = destSlice
} else {
dest = value
}
unkMap[pbField.Name] = dest
}
}
return unkMap
}
func (m *ProtoProducerMessage) FormatMessageReflectCustom(ext, quotes, sep, sign string, null bool) string {
vfm := reflect.ValueOf(m)
vfm = reflect.Indirect(vfm)
var i int
fstr := make([]string, len(m.formatter.fields)) // todo: reuse with pool
unkMap := m.mapUnknown()
// iterate through the fields requested by the user
for _, s := range m.formatter.fields {
fieldName := s
fieldFinalName := s
if fieldRename, ok := m.formatter.rename[s]; ok && fieldRename != "" {
fieldFinalName = fieldRename
}
// get original name from structure
if fieldNameMap, ok := m.formatter.reMap[fieldName]; ok && fieldNameMap != "" {
fieldName = fieldNameMap
}
// get renderer
renderer, okRenderer := m.formatter.render[fieldName]
if !okRenderer {
renderer = NilRenderer
}
fieldValue := vfm.FieldByName(fieldName)
// if does not exist from structure,
// fetch from unknown (only numbered) fields
// that were parsed above
if !fieldValue.IsValid() {
if unkField, ok := unkMap[s]; ok {
fieldValue = reflect.ValueOf(unkField)
} else if !okRenderer { // not a virtual field
continue
}
}
isSlice := m.formatter.isSlice[fieldName]
// render each item of the array independently
// note: isSlice is necessary to consider certain byte arrays in their entirety
// eg: IP addresses
if isSlice {
c := fieldValue.Len()
v := "["
for i := 0; i < c; i++ {
fieldValueI := fieldValue.Index(i)
var val interface{}
if fieldValueI.IsValid() {
val = fieldValueI.Interface()
}
rendered := renderer(m, fieldName, val)
if rendered == nil {
continue
}
renderedType := reflect.TypeOf(rendered)
if renderedType.Kind() == reflect.String {
v += fmt.Sprintf("%s%v%s", quotes, rendered, quotes)
} else {
v += fmt.Sprintf("%v", rendered)
}
if i < c-1 {
v += ","
}
}
v += "]"
fstr[i] = fmt.Sprintf("%s%s%s%s%s", quotes, fieldFinalName, quotes, sign, v)
} else {
var val interface{}
if fieldValue.IsValid() {
val = fieldValue.Interface()
}
rendered := renderer(m, fieldName, val)
if rendered == nil {
continue
}
renderedType := reflect.TypeOf(rendered)
if renderedType.Kind() == reflect.String {
fstr[i] = fmt.Sprintf("%s%s%s%s%s%v%s", quotes, fieldFinalName, quotes, sign, quotes, rendered, quotes)
} else {
fstr[i] = fmt.Sprintf("%s%s%s%s%v", quotes, fieldFinalName, quotes, sign, rendered)
}
}
i++
}
fstr = fstr[0:i]
return strings.Join(fstr, sep)
}

View File

@@ -1,16 +1,16 @@
package producer
package protoproducer
import (
"bytes"
"encoding/binary"
"errors"
"fmt"
"net"
"sync"
"time"
"github.com/netsampler/goflow2/decoders/netflow"
flowmessage "github.com/netsampler/goflow2/pb"
"github.com/netsampler/goflow2/v2/decoders/netflow"
flowmessage "github.com/netsampler/goflow2/v2/pb"
"github.com/netsampler/goflow2/v2/producer"
)
type SamplingRateSystem interface {
@@ -19,13 +19,15 @@ type SamplingRateSystem interface {
}
type basicSamplingRateSystem struct {
sampling map[uint16]map[uint32]uint32
//sampling map[uint16]map[uint32]uint32
sampling map[string]uint32
samplinglock *sync.RWMutex
}
func CreateSamplingSystem() SamplingRateSystem {
ts := &basicSamplingRateSystem{
sampling: make(map[uint16]map[uint32]uint32),
//sampling: make(map[uint16]map[uint32]uint32),
sampling: make(map[string]uint32),
samplinglock: &sync.RWMutex{},
}
return ts
@@ -34,25 +36,30 @@ func CreateSamplingSystem() SamplingRateSystem {
func (s *basicSamplingRateSystem) AddSamplingRate(version uint16, obsDomainId uint32, samplingRate uint32) {
s.samplinglock.Lock()
defer s.samplinglock.Unlock()
_, exists := s.sampling[version]
/*_, exists := s.sampling[version]
if exists != true {
s.sampling[version] = make(map[uint32]uint32)
}
s.sampling[version][obsDomainId] = samplingRate
s.sampling[version][obsDomainId] = samplingRate*/
s.sampling[fmt.Sprintf("%d-%d", version, obsDomainId)] = samplingRate
}
func (s *basicSamplingRateSystem) GetSamplingRate(version uint16, obsDomainId uint32) (uint32, error) {
s.samplinglock.RLock()
defer s.samplinglock.RUnlock()
samplingVersion, okver := s.sampling[version]
/*samplingVersion, okver := s.sampling[version]
if okver {
samplingRate, okid := samplingVersion[obsDomainId]
if okid {
return samplingRate, nil
}
return 0, errors.New("") // TBC
}*/
if samplingRate, ok := s.sampling[fmt.Sprintf("%d-%d", version, obsDomainId)]; ok {
return samplingRate, nil
}
return 0, errors.New("") // TBC
return 0, errors.New("") // TBC // todo: now
}
type SingleSamplingRateSystem struct {
@@ -75,26 +82,30 @@ func NetFlowLookFor(dataFields []netflow.DataField, typeId uint16) (bool, interf
return false, nil
}
func NetFlowPopulate(dataFields []netflow.DataField, typeId uint16, addr interface{}) bool {
func NetFlowPopulate(dataFields []netflow.DataField, typeId uint16, addr interface{}) (bool, error) {
exists, value := NetFlowLookFor(dataFields, typeId)
if exists && value != nil {
valueBytes, ok := value.([]byte)
valueReader := bytes.NewReader(valueBytes)
if ok {
switch addrt := addr.(type) {
case *(net.IP):
*addrt = valueBytes
//case *(net.IP):
// *addrt = valueBytes
case *(time.Time):
t := uint64(0)
binary.Read(valueReader, binary.BigEndian, &t)
if err := binary.Read(valueReader, binary.BigEndian, &t); err != nil {
return false, err
}
t64 := int64(t / 1000)
*addrt = time.Unix(t64, 0)
default:
binary.Read(valueReader, binary.BigEndian, addr)
if err := binary.Read(valueReader, binary.BigEndian, addr); err != nil {
return false, err
}
}
}
}
return exists
return exists, nil
}
func WriteUDecoded(o uint64, out interface{}) error {
@@ -108,7 +119,7 @@ func WriteUDecoded(o uint64, out interface{}) error {
case *uint64:
*t = o
default:
return errors.New("The parameter is not a pointer to a byte/uint16/uint32/uint64 structure")
return fmt.Errorf("the parameter is not a pointer to a byte/uint16/uint32/uint64 structure")
}
return nil
}
@@ -124,7 +135,7 @@ func WriteDecoded(o int64, out interface{}) error {
case *int64:
*t = o
default:
return errors.New("The parameter is not a pointer to a int8/int16/int32/int64 structure")
return fmt.Errorf("the parameter is not a pointer to a int8/int16/int32/int64 structure")
}
return nil
}
@@ -149,7 +160,7 @@ func DecodeUNumber(b []byte, out interface{}) error {
iter++
}
} else {
return errors.New(fmt.Sprintf("Non-regular number of bytes for a number: %v", l))
return fmt.Errorf("non-regular number of bytes for a number: %v", l)
}
}
return WriteUDecoded(o, out)
@@ -175,7 +186,7 @@ func DecodeUNumberLE(b []byte, out interface{}) error {
iter++
}
} else {
return errors.New(fmt.Sprintf("Non-regular number of bytes for a number: %v", l))
return fmt.Errorf("non-regular number of bytes for a number: %v", l)
}
}
return WriteUDecoded(o, out)
@@ -201,7 +212,7 @@ func DecodeNumber(b []byte, out interface{}) error {
iter++
}
} else {
return errors.New(fmt.Sprintf("Non-regular number of bytes for a number: %v", l))
return fmt.Errorf("non-regular number of bytes for a number: %v", l)
}
}
return WriteDecoded(o, out)
@@ -227,7 +238,7 @@ func DecodeNumberLE(b []byte, out interface{}) error {
iter++
}
} else {
return errors.New(fmt.Sprintf("Non-regular number of bytes for a number: %v", l))
return fmt.Errorf("non-regular number of bytes for a number: %v", l)
}
}
return WriteDecoded(o, out)
@@ -256,9 +267,13 @@ func addrReplaceCheck(dstAddr *[]byte, v []byte, eType *uint32, ipv6 bool) {
}
}
func ConvertNetFlowDataSet(version uint16, baseTime uint32, uptime uint32, record []netflow.DataField, mapperNetFlow *NetFlowMapper, mapperSFlow *SFlowMapper) *flowmessage.FlowMessage {
flowMessage := &flowmessage.FlowMessage{}
func ConvertNetFlowDataSet(flowMessage *ProtoProducerMessage, version uint16, baseTime uint32, uptime uint32, record []netflow.DataField, mapperNetFlow *NetFlowMapper, mapperSFlow *SFlowMapper) error {
var time uint64
baseTimeNs := uint64(baseTime) * 1000000000
// the following should be overriden if the template contains timing information
// otherwise, defaults to the export time
flowMessage.TimeFlowStartNs = baseTimeNs
flowMessage.TimeFlowEndNs = baseTimeNs
if version == 9 {
flowMessage.Type = flowmessage.FlowMessage_NETFLOW_V9
@@ -274,7 +289,9 @@ func ConvertNetFlowDataSet(version uint16, baseTime uint32, uptime uint32, recor
continue
}
MapCustomNetFlow(flowMessage, df, mapperNetFlow)
if err := MapCustomNetFlow(flowMessage, df, mapperNetFlow); err != nil {
return err
}
if df.PenProvided {
continue
@@ -283,46 +300,78 @@ func ConvertNetFlowDataSet(version uint16, baseTime uint32, uptime uint32, recor
switch df.Type {
case netflow.IPFIX_FIELD_observationPointId:
DecodeUNumber(v, &(flowMessage.ObservationPointId))
if err := DecodeUNumber(v, &(flowMessage.ObservationPointId)); err != nil {
return err
}
// Statistics
case netflow.NFV9_FIELD_IN_BYTES:
DecodeUNumber(v, &(flowMessage.Bytes))
if err := DecodeUNumber(v, &(flowMessage.Bytes)); err != nil {
return err
}
case netflow.NFV9_FIELD_IN_PKTS:
DecodeUNumber(v, &(flowMessage.Packets))
if err := DecodeUNumber(v, &(flowMessage.Packets)); err != nil {
return err
}
case netflow.NFV9_FIELD_OUT_BYTES:
DecodeUNumber(v, &(flowMessage.Bytes))
if err := DecodeUNumber(v, &(flowMessage.Bytes)); err != nil {
return err
}
case netflow.NFV9_FIELD_OUT_PKTS:
DecodeUNumber(v, &(flowMessage.Packets))
if err := DecodeUNumber(v, &(flowMessage.Packets)); err != nil {
return err
}
// L4
case netflow.NFV9_FIELD_L4_SRC_PORT:
DecodeUNumber(v, &(flowMessage.SrcPort))
if err := DecodeUNumber(v, &(flowMessage.SrcPort)); err != nil {
return err
}
case netflow.NFV9_FIELD_L4_DST_PORT:
DecodeUNumber(v, &(flowMessage.DstPort))
if err := DecodeUNumber(v, &(flowMessage.DstPort)); err != nil {
return err
}
case netflow.NFV9_FIELD_PROTOCOL:
DecodeUNumber(v, &(flowMessage.Proto))
if err := DecodeUNumber(v, &(flowMessage.Proto)); err != nil {
return err
}
// Network
case netflow.NFV9_FIELD_SRC_AS:
DecodeUNumber(v, &(flowMessage.SrcAs))
if err := DecodeUNumber(v, &(flowMessage.SrcAs)); err != nil {
return err
}
case netflow.NFV9_FIELD_DST_AS:
DecodeUNumber(v, &(flowMessage.DstAs))
if err := DecodeUNumber(v, &(flowMessage.DstAs)); err != nil {
return err
}
// Interfaces
case netflow.NFV9_FIELD_INPUT_SNMP:
DecodeUNumber(v, &(flowMessage.InIf))
if err := DecodeUNumber(v, &(flowMessage.InIf)); err != nil {
return err
}
case netflow.NFV9_FIELD_OUTPUT_SNMP:
DecodeUNumber(v, &(flowMessage.OutIf))
if err := DecodeUNumber(v, &(flowMessage.OutIf)); err != nil {
return err
}
case netflow.NFV9_FIELD_FORWARDING_STATUS:
DecodeUNumber(v, &(flowMessage.ForwardingStatus))
if err := DecodeUNumber(v, &(flowMessage.ForwardingStatus)); err != nil {
return err
}
case netflow.NFV9_FIELD_SRC_TOS:
DecodeUNumber(v, &(flowMessage.IpTos))
if err := DecodeUNumber(v, &(flowMessage.IpTos)); err != nil {
return err
}
case netflow.NFV9_FIELD_TCP_FLAGS:
DecodeUNumber(v, &(flowMessage.TcpFlags))
if err := DecodeUNumber(v, &(flowMessage.TcpFlags)); err != nil {
return err
}
case netflow.NFV9_FIELD_MIN_TTL:
DecodeUNumber(v, &(flowMessage.IpTtl))
if err := DecodeUNumber(v, &(flowMessage.IpTtl)); err != nil {
return err
}
// IP
case netflow.NFV9_FIELD_IP_PROTOCOL_VERSION:
@@ -341,9 +390,13 @@ func ConvertNetFlowDataSet(version uint16, baseTime uint32, uptime uint32, recor
addrReplaceCheck(&(flowMessage.DstAddr), v, &(flowMessage.Etype), false)
case netflow.NFV9_FIELD_SRC_MASK:
DecodeUNumber(v, &(flowMessage.SrcNet))
if err := DecodeUNumber(v, &(flowMessage.SrcNet)); err != nil {
return err
}
case netflow.NFV9_FIELD_DST_MASK:
DecodeUNumber(v, &(flowMessage.DstNet))
if err := DecodeUNumber(v, &(flowMessage.DstNet)); err != nil {
return err
}
case netflow.NFV9_FIELD_IPV6_SRC_ADDR:
addrReplaceCheck(&(flowMessage.SrcAddr), v, &(flowMessage.Etype), true)
@@ -352,9 +405,13 @@ func ConvertNetFlowDataSet(version uint16, baseTime uint32, uptime uint32, recor
addrReplaceCheck(&(flowMessage.DstAddr), v, &(flowMessage.Etype), true)
case netflow.NFV9_FIELD_IPV6_SRC_MASK:
DecodeUNumber(v, &(flowMessage.SrcNet))
if err := DecodeUNumber(v, &(flowMessage.SrcNet)); err != nil {
return err
}
case netflow.NFV9_FIELD_IPV6_DST_MASK:
DecodeUNumber(v, &(flowMessage.DstNet))
if err := DecodeUNumber(v, &(flowMessage.DstNet)); err != nil {
return err
}
case netflow.NFV9_FIELD_IPV4_NEXT_HOP:
flowMessage.NextHop = v
@@ -369,81 +426,118 @@ func ConvertNetFlowDataSet(version uint16, baseTime uint32, uptime uint32, recor
// ICMP
case netflow.NFV9_FIELD_ICMP_TYPE:
var icmpTypeCode uint16
DecodeUNumber(v, &icmpTypeCode)
if err := DecodeUNumber(v, &icmpTypeCode); err != nil {
return err
}
flowMessage.IcmpType = uint32(icmpTypeCode >> 8)
flowMessage.IcmpCode = uint32(icmpTypeCode & 0xff)
case netflow.IPFIX_FIELD_icmpTypeCodeIPv6:
var icmpTypeCode uint16
DecodeUNumber(v, &icmpTypeCode)
if err := DecodeUNumber(v, &icmpTypeCode); err != nil {
return err
}
flowMessage.IcmpType = uint32(icmpTypeCode >> 8)
flowMessage.IcmpCode = uint32(icmpTypeCode & 0xff)
case netflow.IPFIX_FIELD_icmpTypeIPv4:
DecodeUNumber(v, &(flowMessage.IcmpType))
if err := DecodeUNumber(v, &(flowMessage.IcmpType)); err != nil {
return err
}
case netflow.IPFIX_FIELD_icmpTypeIPv6:
DecodeUNumber(v, &(flowMessage.IcmpType))
if err := DecodeUNumber(v, &(flowMessage.IcmpType)); err != nil {
return err
}
case netflow.IPFIX_FIELD_icmpCodeIPv4:
DecodeUNumber(v, &(flowMessage.IcmpCode))
if err := DecodeUNumber(v, &(flowMessage.IcmpCode)); err != nil {
return err
}
case netflow.IPFIX_FIELD_icmpCodeIPv6:
DecodeUNumber(v, &(flowMessage.IcmpCode))
if err := DecodeUNumber(v, &(flowMessage.IcmpCode)); err != nil {
return err
}
// Mac
case netflow.NFV9_FIELD_IN_SRC_MAC:
DecodeUNumber(v, &(flowMessage.SrcMac))
if err := DecodeUNumber(v, &(flowMessage.SrcMac)); err != nil {
return err
}
case netflow.NFV9_FIELD_IN_DST_MAC:
DecodeUNumber(v, &(flowMessage.DstMac))
if err := DecodeUNumber(v, &(flowMessage.DstMac)); err != nil {
return err
}
case netflow.NFV9_FIELD_OUT_SRC_MAC:
DecodeUNumber(v, &(flowMessage.SrcMac))
if err := DecodeUNumber(v, &(flowMessage.SrcMac)); err != nil {
return err
}
case netflow.NFV9_FIELD_OUT_DST_MAC:
DecodeUNumber(v, &(flowMessage.DstMac))
if err := DecodeUNumber(v, &(flowMessage.DstMac)); err != nil {
return err
}
case netflow.NFV9_FIELD_SRC_VLAN:
DecodeUNumber(v, &(flowMessage.VlanId))
DecodeUNumber(v, &(flowMessage.SrcVlan))
if err := DecodeUNumber(v, &(flowMessage.VlanId)); err != nil {
return err
}
if err := DecodeUNumber(v, &(flowMessage.SrcVlan)); err != nil {
return err
}
case netflow.NFV9_FIELD_DST_VLAN:
DecodeUNumber(v, &(flowMessage.DstVlan))
case netflow.IPFIX_FIELD_ingressVRFID:
DecodeUNumber(v, &(flowMessage.IngressVrfId))
case netflow.IPFIX_FIELD_egressVRFID:
DecodeUNumber(v, &(flowMessage.EgressVrfId))
if err := DecodeUNumber(v, &(flowMessage.DstVlan)); err != nil {
return err
}
case netflow.NFV9_FIELD_IPV4_IDENT:
DecodeUNumber(v, &(flowMessage.FragmentId))
if err := DecodeUNumber(v, &(flowMessage.FragmentId)); err != nil {
return err
}
case netflow.NFV9_FIELD_FRAGMENT_OFFSET:
var fragOffset uint32
DecodeUNumber(v, &fragOffset)
if err := DecodeUNumber(v, &fragOffset); err != nil {
return err
}
flowMessage.FragmentOffset |= fragOffset
case netflow.IPFIX_FIELD_fragmentFlags:
var ipFlags uint32
DecodeUNumber(v, &ipFlags)
if err := DecodeUNumber(v, &ipFlags); err != nil {
return err
}
flowMessage.FragmentOffset |= ipFlags
case netflow.NFV9_FIELD_IPV6_FLOW_LABEL:
DecodeUNumber(v, &(flowMessage.Ipv6FlowLabel))
case netflow.IPFIX_FIELD_biflowDirection:
DecodeUNumber(v, &(flowMessage.BiFlowDirection))
case netflow.NFV9_FIELD_DIRECTION:
DecodeUNumber(v, &(flowMessage.FlowDirection))
if err := DecodeUNumber(v, &(flowMessage.Ipv6FlowLabel)); err != nil {
return err
}
// MPLS
case netflow.IPFIX_FIELD_mplsTopLabelStackSection:
var mplsLabel uint32
DecodeUNumber(v, &mplsLabel)
flowMessage.Mpls_1Label = uint32(mplsLabel >> 4)
flowMessage.HasMpls = true
if err := DecodeUNumber(v, &mplsLabel); err != nil {
return err
}
if len(flowMessage.MplsLabel) < 1 {
flowMessage.MplsLabel = make([]uint32, 1)
}
flowMessage.MplsLabel[0] = uint32(mplsLabel >> 4)
case netflow.IPFIX_FIELD_mplsLabelStackSection2:
var mplsLabel uint32
DecodeUNumber(v, &mplsLabel)
flowMessage.Mpls_2Label = uint32(mplsLabel >> 4)
if err := DecodeUNumber(v, &mplsLabel); err != nil {
return err
}
if len(flowMessage.MplsLabel) < 2 {
flowMessage.MplsLabel = make([]uint32, 2)
}
flowMessage.MplsLabel[1] = uint32(mplsLabel >> 4)
case netflow.IPFIX_FIELD_mplsLabelStackSection3:
var mplsLabel uint32
DecodeUNumber(v, &mplsLabel)
flowMessage.Mpls_3Label = uint32(mplsLabel >> 4)
if err := DecodeUNumber(v, &mplsLabel); err != nil {
return err
}
if len(flowMessage.MplsLabel) < 3 {
flowMessage.MplsLabel = make([]uint32, 3)
}
flowMessage.MplsLabel[2] = uint32(mplsLabel >> 4)
case netflow.IPFIX_FIELD_mplsTopLabelIPv4Address:
flowMessage.MplsLabelIp = v
flowMessage.MplsIp = append(flowMessage.MplsIp, v)
case netflow.IPFIX_FIELD_mplsTopLabelIPv6Address:
flowMessage.MplsLabelIp = v
flowMessage.MplsIp = append(flowMessage.MplsIp, v)
default:
if version == 9 {
@@ -451,65 +545,81 @@ func ConvertNetFlowDataSet(version uint16, baseTime uint32, uptime uint32, recor
switch df.Type {
case netflow.NFV9_FIELD_FIRST_SWITCHED:
var timeFirstSwitched uint32
DecodeUNumber(v, &timeFirstSwitched)
if err := DecodeUNumber(v, &timeFirstSwitched); err != nil {
return err
}
timeDiff := (uptime - timeFirstSwitched)
flowMessage.TimeFlowStart = uint64(baseTime - timeDiff/1000)
flowMessage.TimeFlowStartMs = uint64(baseTime)*1000 - uint64(timeDiff)
flowMessage.TimeFlowStartNs = baseTimeNs - uint64(timeDiff)*1000000000
case netflow.NFV9_FIELD_LAST_SWITCHED:
var timeLastSwitched uint32
DecodeUNumber(v, &timeLastSwitched)
if err := DecodeUNumber(v, &timeLastSwitched); err != nil {
return err
}
timeDiff := (uptime - timeLastSwitched)
flowMessage.TimeFlowEnd = uint64(baseTime - timeDiff/1000)
flowMessage.TimeFlowEndMs = uint64(baseTime)*1000 - uint64(timeDiff)
flowMessage.TimeFlowEndNs = baseTimeNs - uint64(timeDiff)*1000000000
}
} else if version == 10 {
switch df.Type {
case netflow.IPFIX_FIELD_flowStartSeconds:
DecodeUNumber(v, &time)
flowMessage.TimeFlowStart = time
flowMessage.TimeFlowStartMs = time * 1000
if err := DecodeUNumber(v, &time); err != nil {
return err
}
flowMessage.TimeFlowStartNs = time * 1000000000
case netflow.IPFIX_FIELD_flowStartMilliseconds:
DecodeUNumber(v, &time)
flowMessage.TimeFlowStart = time / 1000
flowMessage.TimeFlowStartMs = time
if err := DecodeUNumber(v, &time); err != nil {
return err
}
flowMessage.TimeFlowStartNs = time * 1000000
case netflow.IPFIX_FIELD_flowStartMicroseconds:
DecodeUNumber(v, &time)
flowMessage.TimeFlowStart = time / 1000000
flowMessage.TimeFlowStartMs = time / 1000
if err := DecodeUNumber(v, &time); err != nil {
return err
}
flowMessage.TimeFlowStartNs = time * 1000
case netflow.IPFIX_FIELD_flowStartNanoseconds:
DecodeUNumber(v, &time)
flowMessage.TimeFlowStart = time / 1000000000
flowMessage.TimeFlowStartMs = time / 1000000
if err := DecodeUNumber(v, &time); err != nil {
return err
}
flowMessage.TimeFlowStartNs = time
case netflow.IPFIX_FIELD_flowEndSeconds:
DecodeUNumber(v, &time)
flowMessage.TimeFlowEnd = time
flowMessage.TimeFlowEndMs = time * 1000
if err := DecodeUNumber(v, &time); err != nil {
return err
}
flowMessage.TimeFlowEndNs = time * 1000000000
case netflow.IPFIX_FIELD_flowEndMilliseconds:
DecodeUNumber(v, &time)
flowMessage.TimeFlowEnd = time / 1000
flowMessage.TimeFlowEndMs = time
if err := DecodeUNumber(v, &time); err != nil {
return err
}
flowMessage.TimeFlowEndNs = time * 1000000
case netflow.IPFIX_FIELD_flowEndMicroseconds:
DecodeUNumber(v, &time)
flowMessage.TimeFlowEnd = time / 1000000
flowMessage.TimeFlowEndMs = time / 1000
if err := DecodeUNumber(v, &time); err != nil {
return err
}
flowMessage.TimeFlowEndNs = time * 1000
case netflow.IPFIX_FIELD_flowEndNanoseconds:
DecodeUNumber(v, &time)
flowMessage.TimeFlowEnd = time / 1000000000
flowMessage.TimeFlowEndMs = time / 1000000
if err := DecodeUNumber(v, &time); err != nil {
return err
}
flowMessage.TimeFlowEndNs = time
case netflow.IPFIX_FIELD_flowStartDeltaMicroseconds:
DecodeUNumber(v, &time)
flowMessage.TimeFlowStart = uint64(baseTime) - time/1000000
flowMessage.TimeFlowStartMs = uint64(baseTime)*1000 - time/1000
if err := DecodeUNumber(v, &time); err != nil {
return err
}
flowMessage.TimeFlowStartNs = baseTimeNs - time*1000
case netflow.IPFIX_FIELD_flowEndDeltaMicroseconds:
DecodeUNumber(v, &time)
flowMessage.TimeFlowEnd = uint64(baseTime) - time/1000000
flowMessage.TimeFlowEndMs = uint64(baseTime)*1000 - time/1000
if err := DecodeUNumber(v, &time); err != nil {
return err
}
flowMessage.TimeFlowEndNs = baseTimeNs - time*1000
// RFC7133
case netflow.IPFIX_FIELD_dataLinkFrameSize:
DecodeUNumber(v, &(flowMessage.Bytes))
if err := DecodeUNumber(v, &(flowMessage.Bytes)); err != nil {
return err
}
flowMessage.Packets = 1
case netflow.IPFIX_FIELD_dataLinkFrameSection:
ParseEthernetHeader(flowMessage, v, mapperSFlow)
if err := ParseEthernetHeader(flowMessage, v, mapperSFlow); err != nil {
return err
}
flowMessage.Packets = 1
if flowMessage.Bytes == 0 {
flowMessage.Bytes = uint64(len(v))
@@ -519,52 +629,51 @@ func ConvertNetFlowDataSet(version uint16, baseTime uint32, uptime uint32, recor
}
}
return flowMessage
return nil
}
func SearchNetFlowDataSetsRecords(version uint16, baseTime uint32, uptime uint32, dataRecords []netflow.DataRecord, mapperNetFlow *NetFlowMapper, mapperSFlow *SFlowMapper) []*flowmessage.FlowMessage {
var flowMessageSet []*flowmessage.FlowMessage
func SearchNetFlowDataSetsRecords(version uint16, baseTime uint32, uptime uint32, dataRecords []netflow.DataRecord, mapperNetFlow *NetFlowMapper, mapperSFlow *SFlowMapper) (flowMessageSet []producer.ProducerMessage, err error) {
for _, record := range dataRecords {
fmsg := ConvertNetFlowDataSet(version, baseTime, uptime, record.Values, mapperNetFlow, mapperSFlow)
fmsg := protoMessagePool.Get().(*ProtoProducerMessage)
fmsg.Reset()
if err := ConvertNetFlowDataSet(fmsg, version, baseTime, uptime, record.Values, mapperNetFlow, mapperSFlow); err != nil {
return flowMessageSet, err
}
if fmsg != nil {
flowMessageSet = append(flowMessageSet, fmsg)
}
}
return flowMessageSet
return flowMessageSet, nil
}
func SearchNetFlowDataSets(version uint16, baseTime uint32, uptime uint32, dataFlowSet []netflow.DataFlowSet, mapperNetFlow *NetFlowMapper, mapperSFlow *SFlowMapper) []*flowmessage.FlowMessage {
var flowMessageSet []*flowmessage.FlowMessage
func SearchNetFlowDataSets(version uint16, baseTime uint32, uptime uint32, dataFlowSet []netflow.DataFlowSet, mapperNetFlow *NetFlowMapper, mapperSFlow *SFlowMapper) (flowMessageSet []producer.ProducerMessage, err error) {
for _, dataFlowSetItem := range dataFlowSet {
fmsg := SearchNetFlowDataSetsRecords(version, baseTime, uptime, dataFlowSetItem.Records, mapperNetFlow, mapperSFlow)
fmsg, err := SearchNetFlowDataSetsRecords(version, baseTime, uptime, dataFlowSetItem.Records, mapperNetFlow, mapperSFlow)
if err != nil {
return flowMessageSet, err
}
if fmsg != nil {
flowMessageSet = append(flowMessageSet, fmsg...)
}
}
return flowMessageSet
return flowMessageSet, nil
}
func SearchNetFlowOptionDataSets(dataFlowSet []netflow.OptionsDataFlowSet) (uint32, bool) {
var samplingRate uint32
var found bool
func SearchNetFlowOptionDataSets(dataFlowSet []netflow.OptionsDataFlowSet) (samplingRate uint32, found bool, err error) {
for _, dataFlowSetItem := range dataFlowSet {
for _, record := range dataFlowSetItem.Records {
b := NetFlowPopulate(record.OptionsValues, 305, &samplingRate)
if b {
return samplingRate, b
if found, err := NetFlowPopulate(record.OptionsValues, 305, &samplingRate); err != nil || found {
return samplingRate, found, err
}
b = NetFlowPopulate(record.OptionsValues, 50, &samplingRate)
if b {
return samplingRate, b
if found, err := NetFlowPopulate(record.OptionsValues, 50, &samplingRate); err != nil || found {
return samplingRate, found, err
}
b = NetFlowPopulate(record.OptionsValues, 34, &samplingRate)
if b {
return samplingRate, b
if found, err := NetFlowPopulate(record.OptionsValues, 34, &samplingRate); err != nil || found {
return samplingRate, found, err
}
}
}
return samplingRate, found
return samplingRate, found, err
}
func SplitNetFlowSets(packetNFv9 netflow.NFv9Packet) ([]netflow.DataFlowSet, []netflow.TemplateFlowSet, []netflow.NFv9OptionsTemplateFlowSet, []netflow.OptionsDataFlowSet) {
@@ -607,76 +716,85 @@ func SplitIPFIXSets(packetIPFIX netflow.IPFIXPacket) ([]netflow.DataFlowSet, []n
return dataFlowSet, templatesFlowSet, optionsTemplatesFlowSet, optionsDataFlowSet
}
func ProcessMessageNetFlow(msgDec interface{}, samplingRateSys SamplingRateSystem) ([]*flowmessage.FlowMessage, error) {
return ProcessMessageNetFlowConfig(msgDec, samplingRateSys, nil)
// Convert a NetFlow datastructure to a FlowMessage protobuf
// Does not put sampling rate
func ProcessMessageIPFIXConfig(packet *netflow.IPFIXPacket, samplingRateSys SamplingRateSystem, config *producerConfigMapped) (flowMessageSet []producer.ProducerMessage, err error) {
dataFlowSet, _, _, optionDataFlowSet := SplitIPFIXSets(*packet)
seqnum := packet.SequenceNumber
baseTime := packet.ExportTime
obsDomainId := packet.ObservationDomainId
var cfgIpfix *NetFlowMapper
var cfgSflow *SFlowMapper
if config != nil {
cfgIpfix = config.IPFIX
cfgSflow = config.SFlow
}
flowMessageSet, err = SearchNetFlowDataSets(10, baseTime, 0, dataFlowSet, cfgIpfix, cfgSflow)
if err != nil {
return flowMessageSet, err
}
samplingRate, found, err := SearchNetFlowOptionDataSets(optionDataFlowSet)
if err != nil {
return flowMessageSet, err
}
if samplingRateSys != nil {
if found {
samplingRateSys.AddSamplingRate(10, obsDomainId, samplingRate)
} else {
samplingRate, _ = samplingRateSys.GetSamplingRate(10, obsDomainId)
}
}
for _, msg := range flowMessageSet {
fmsg, ok := msg.(*ProtoProducerMessage)
if !ok {
continue
}
fmsg.SequenceNum = seqnum
fmsg.SamplingRate = uint64(samplingRate)
fmsg.ObservationDomainId = obsDomainId
}
return flowMessageSet, nil
}
// Convert a NetFlow datastructure to a FlowMessage protobuf
// Does not put sampling rate
func ProcessMessageNetFlowConfig(msgDec interface{}, samplingRateSys SamplingRateSystem, config *ProducerConfigMapped) ([]*flowmessage.FlowMessage, error) {
seqnum := uint32(0)
var baseTime uint32
var uptime uint32
func ProcessMessageNetFlowV9Config(packet *netflow.NFv9Packet, samplingRateSys SamplingRateSystem, config *producerConfigMapped) (flowMessageSet []producer.ProducerMessage, err error) {
dataFlowSet, _, _, optionDataFlowSet := SplitNetFlowSets(*packet)
var flowMessageSet []*flowmessage.FlowMessage
seqnum := packet.SequenceNumber
baseTime := packet.UnixSeconds
uptime := packet.SystemUptime
obsDomainId := packet.SourceId
switch msgDecConv := msgDec.(type) {
case netflow.NFv9Packet:
dataFlowSet, _, _, optionDataFlowSet := SplitNetFlowSets(msgDecConv)
seqnum = msgDecConv.SequenceNumber
baseTime = msgDecConv.UnixSeconds
uptime = msgDecConv.SystemUptime
obsDomainId := msgDecConv.SourceId
var cfg *NetFlowMapper
if config != nil {
cfg = config.NetFlowV9
}
flowMessageSet = SearchNetFlowDataSets(9, baseTime, uptime, dataFlowSet, cfg, nil)
samplingRate, found := SearchNetFlowOptionDataSets(optionDataFlowSet)
if samplingRateSys != nil {
if found {
samplingRateSys.AddSamplingRate(9, obsDomainId, samplingRate)
} else {
samplingRate, _ = samplingRateSys.GetSamplingRate(9, obsDomainId)
}
}
for _, fmsg := range flowMessageSet {
fmsg.SequenceNum = seqnum
fmsg.SamplingRate = uint64(samplingRate)
}
case netflow.IPFIXPacket:
dataFlowSet, _, _, optionDataFlowSet := SplitIPFIXSets(msgDecConv)
seqnum = msgDecConv.SequenceNumber
baseTime = msgDecConv.ExportTime
obsDomainId := msgDecConv.ObservationDomainId
var cfgIpfix *NetFlowMapper
var cfgSflow *SFlowMapper
if config != nil {
cfgIpfix = config.IPFIX
cfgSflow = config.SFlow
}
flowMessageSet = SearchNetFlowDataSets(10, baseTime, uptime, dataFlowSet, cfgIpfix, cfgSflow)
samplingRate, found := SearchNetFlowOptionDataSets(optionDataFlowSet)
if samplingRateSys != nil {
if found {
samplingRateSys.AddSamplingRate(10, obsDomainId, samplingRate)
} else {
samplingRate, _ = samplingRateSys.GetSamplingRate(10, obsDomainId)
}
}
for _, fmsg := range flowMessageSet {
fmsg.SequenceNum = seqnum
fmsg.SamplingRate = uint64(samplingRate)
fmsg.ObservationDomainId = obsDomainId
}
default:
return flowMessageSet, errors.New("Bad NetFlow/IPFIX version")
var cfg *NetFlowMapper
if config != nil {
cfg = config.NetFlowV9
}
flowMessageSet, err = SearchNetFlowDataSets(9, baseTime, uptime, dataFlowSet, cfg, nil)
if err != nil {
return flowMessageSet, err
}
samplingRate, found, err := SearchNetFlowOptionDataSets(optionDataFlowSet)
if err != nil {
return flowMessageSet, err
}
if samplingRateSys != nil {
if found {
samplingRateSys.AddSamplingRate(9, obsDomainId, samplingRate)
} else {
samplingRate, _ = samplingRateSys.GetSamplingRate(9, obsDomainId)
}
}
for _, msg := range flowMessageSet {
fmsg, ok := msg.(*ProtoProducerMessage)
if !ok {
continue
}
fmsg.SequenceNum = seqnum
fmsg.SamplingRate = uint64(samplingRate)
}
return flowMessageSet, nil
}

View File

@@ -0,0 +1,72 @@
package protoproducer
import (
"encoding/binary"
"github.com/netsampler/goflow2/v2/decoders/netflowlegacy"
flowmessage "github.com/netsampler/goflow2/v2/pb"
"github.com/netsampler/goflow2/v2/producer"
)
func ConvertNetFlowLegacyRecord(flowMessage *ProtoProducerMessage, baseTime uint64, uptime uint32, record netflowlegacy.RecordsNetFlowV5) {
flowMessage.Type = flowmessage.FlowMessage_NETFLOW_V5
timeDiffFirst := (uptime - record.First)
timeDiffLast := (uptime - record.Last)
flowMessage.TimeFlowStartNs = baseTime - uint64(timeDiffFirst)*1000000000
flowMessage.TimeFlowEndNs = baseTime - uint64(timeDiffLast)*1000000000
v := make([]byte, 4)
binary.BigEndian.PutUint32(v, uint32(record.NextHop))
flowMessage.NextHop = v
v = make([]byte, 4)
binary.BigEndian.PutUint32(v, uint32(record.SrcAddr))
flowMessage.SrcAddr = v
v = make([]byte, 4)
binary.BigEndian.PutUint32(v, uint32(record.DstAddr))
flowMessage.DstAddr = v
flowMessage.Etype = 0x800
flowMessage.SrcAs = uint32(record.SrcAS)
flowMessage.DstAs = uint32(record.DstAS)
flowMessage.SrcNet = uint32(record.SrcMask)
flowMessage.DstNet = uint32(record.DstMask)
flowMessage.Proto = uint32(record.Proto)
flowMessage.TcpFlags = uint32(record.TCPFlags)
flowMessage.IpTos = uint32(record.Tos)
flowMessage.InIf = uint32(record.Input)
flowMessage.OutIf = uint32(record.Output)
flowMessage.SrcPort = uint32(record.SrcPort)
flowMessage.DstPort = uint32(record.DstPort)
flowMessage.Packets = uint64(record.DPkts)
flowMessage.Bytes = uint64(record.DOctets)
}
func SearchNetFlowLegacyRecords(baseTime uint64, uptime uint32, dataRecords []netflowlegacy.RecordsNetFlowV5) (flowMessageSet []producer.ProducerMessage) {
for _, record := range dataRecords {
fmsg := protoMessagePool.Get().(*ProtoProducerMessage)
fmsg.Reset()
ConvertNetFlowLegacyRecord(fmsg, baseTime, uptime, record)
flowMessageSet = append(flowMessageSet, fmsg)
}
return flowMessageSet
}
func ProcessMessageNetFlowLegacy(packet *netflowlegacy.PacketNetFlowV5) ([]producer.ProducerMessage, error) {
seqnum := packet.FlowSequence
samplingRate := packet.SamplingInterval & 0x3FFF
baseTime := uint64(packet.UnixSecs)*1000000000 + uint64(packet.UnixNSecs)
uptime := packet.SysUptime
flowMessageSet := SearchNetFlowLegacyRecords(baseTime, uptime, packet.Records)
for _, msg := range flowMessageSet {
fmsg, ok := msg.(*ProtoProducerMessage)
if !ok {
continue
}
fmsg.SequenceNum = seqnum
fmsg.SamplingRate = uint64(samplingRate)
}
return flowMessageSet, nil
}

View File

@@ -0,0 +1,378 @@
package protoproducer
import (
"encoding/binary"
"github.com/netsampler/goflow2/v2/decoders/sflow"
flowmessage "github.com/netsampler/goflow2/v2/pb"
"github.com/netsampler/goflow2/v2/producer"
)
func GetSFlowFlowSamples(packet *sflow.Packet) []interface{} {
var flowSamples []interface{}
for _, sample := range packet.Samples {
switch sample.(type) {
case sflow.FlowSample:
flowSamples = append(flowSamples, sample)
case sflow.ExpandedFlowSample:
flowSamples = append(flowSamples, sample)
}
}
return flowSamples
}
func ParseSampledHeader(flowMessage *ProtoProducerMessage, sampledHeader *sflow.SampledHeader) error {
return ParseSampledHeaderConfig(flowMessage, sampledHeader, nil)
}
func ParseEthernetHeader(flowMessage *ProtoProducerMessage, data []byte, config *SFlowMapper) error {
var mplsLabel []uint32
var mplsTtl []uint32
var nextHeader byte
var tcpflags byte
var srcIP, dstIP []byte
offset := 14
var srcMac uint64
var dstMac uint64
var tos byte
var ttl byte
var identification uint16
var fragOffset uint16
var flowLabel uint32
var srcPort uint16
var dstPort uint16
for _, configLayer := range GetSFlowConfigLayer(config, "0") {
extracted := GetBytes(data, configLayer.Offset, configLayer.Length)
if err := MapCustom(flowMessage, extracted, configLayer.MapConfigBase); err != nil {
return err
}
}
etherType := data[12:14]
dstMac = binary.BigEndian.Uint64(append([]byte{0, 0}, data[0:6]...))
srcMac = binary.BigEndian.Uint64(append([]byte{0, 0}, data[6:12]...))
flowMessage.SrcMac = srcMac
flowMessage.DstMac = dstMac
encap := true
iterations := 0
for encap && iterations <= 1 {
encap = false
if etherType[0] == 0x81 && etherType[1] == 0x0 { // VLAN 802.1Q
flowMessage.VlanId = uint32(binary.BigEndian.Uint16(data[14:16]))
offset += 4
etherType = data[16:18]
}
if etherType[0] == 0x88 && etherType[1] == 0x47 { // MPLS
iterateMpls := true
for iterateMpls {
if len(data) < offset+5 {
// stop iterating mpls, not enough payload left
break
}
label := binary.BigEndian.Uint32(append([]byte{0}, data[offset:offset+3]...)) >> 4
//exp := data[offset+2] > 1
bottom := data[offset+2] & 1
ttl := data[offset+3]
offset += 4
if bottom == 1 || label <= 15 || offset > len(data) {
if data[offset]&0xf0>>4 == 4 {
etherType = []byte{0x8, 0x0}
} else if data[offset]&0xf0>>4 == 6 {
etherType = []byte{0x86, 0xdd}
}
iterateMpls = false // stop iterating mpls, bottom of stack
}
mplsLabel = append(mplsLabel, label)
mplsTtl = append(mplsTtl, uint32(ttl))
}
}
for _, configLayer := range GetSFlowConfigLayer(config, "3") {
extracted := GetBytes(data, offset*8+configLayer.Offset, configLayer.Length)
if err := MapCustom(flowMessage, extracted, configLayer.MapConfigBase); err != nil {
return err
}
}
if etherType[0] == 0x8 && etherType[1] == 0x0 { // IPv4
if len(data) >= offset+20 {
nextHeader = data[offset+9]
srcIP = data[offset+12 : offset+16]
dstIP = data[offset+16 : offset+20]
tos = data[offset+1]
ttl = data[offset+8]
identification = binary.BigEndian.Uint16(data[offset+4 : offset+6])
fragOffset = binary.BigEndian.Uint16(data[offset+6 : offset+8])
for _, configLayer := range GetSFlowConfigLayer(config, "ipv4") {
extracted := GetBytes(data, offset*8+configLayer.Offset, configLayer.Length)
if err := MapCustom(flowMessage, extracted, configLayer.MapConfigBase); err != nil {
return err
}
}
offset += 20
}
} else if etherType[0] == 0x86 && etherType[1] == 0xdd { // IPv6
if len(data) >= offset+40 {
nextHeader = data[offset+6]
srcIP = data[offset+8 : offset+24]
dstIP = data[offset+24 : offset+40]
tostmp := uint32(binary.BigEndian.Uint16(data[offset : offset+2]))
tos = uint8(tostmp & 0x0ff0 >> 4)
ttl = data[offset+7]
flowLabel = binary.BigEndian.Uint32(data[offset : offset+4])
for _, configLayer := range GetSFlowConfigLayer(config, "ipv6") {
extracted := GetBytes(data, offset*8+configLayer.Offset, configLayer.Length)
if err := MapCustom(flowMessage, extracted, configLayer.MapConfigBase); err != nil {
return err
}
}
offset += 40
}
} else if etherType[0] == 0x8 && etherType[1] == 0x6 { // ARP
for _, configLayer := range GetSFlowConfigLayer(config, "arp") {
extracted := GetBytes(data, offset*8+configLayer.Offset, configLayer.Length)
if err := MapCustom(flowMessage, extracted, configLayer.MapConfigBase); err != nil {
return err
}
}
}
for _, configLayer := range GetSFlowConfigLayer(config, "4") {
extracted := GetBytes(data, offset*8+configLayer.Offset, configLayer.Length)
if err := MapCustom(flowMessage, extracted, configLayer.MapConfigBase); err != nil {
return err
}
}
appOffset := 0
if len(data) >= offset+4 && (nextHeader == 17 || nextHeader == 6) {
srcPort = binary.BigEndian.Uint16(data[offset+0 : offset+2])
dstPort = binary.BigEndian.Uint16(data[offset+2 : offset+4])
}
if nextHeader == 17 {
appOffset = 8
for _, configLayer := range GetSFlowConfigLayer(config, "udp") {
extracted := GetBytes(data, offset*8+configLayer.Offset, configLayer.Length)
if err := MapCustom(flowMessage, extracted, configLayer.MapConfigBase); err != nil {
return err
}
}
}
if len(data) > offset+13 && nextHeader == 6 {
tcpflags = data[offset+13]
appOffset = int(data[13]>>4) * 4
for _, configLayer := range GetSFlowConfigLayer(config, "tcp") {
extracted := GetBytes(data, offset*8+configLayer.Offset, configLayer.Length)
if err := MapCustom(flowMessage, extracted, configLayer.MapConfigBase); err != nil {
return err
}
}
}
// ICMP and ICMPv6
if len(data) >= offset+2 && (nextHeader == 1 || nextHeader == 58) {
flowMessage.IcmpType = uint32(data[offset+0])
flowMessage.IcmpCode = uint32(data[offset+1])
if nextHeader == 1 {
for _, configLayer := range GetSFlowConfigLayer(config, "icmp") {
extracted := GetBytes(data, offset*8+configLayer.Offset, configLayer.Length)
if err := MapCustom(flowMessage, extracted, configLayer.MapConfigBase); err != nil {
return err
}
}
} else if nextHeader == 58 {
for _, configLayer := range GetSFlowConfigLayer(config, "icmp6") {
extracted := GetBytes(data, offset*8+configLayer.Offset, configLayer.Length)
if err := MapCustom(flowMessage, extracted, configLayer.MapConfigBase); err != nil {
return err
}
}
}
}
if appOffset > 0 {
for _, configLayer := range GetSFlowConfigLayer(config, "7") {
extracted := GetBytes(data, (offset+appOffset)*8+configLayer.Offset, configLayer.Length)
if err := MapCustom(flowMessage, extracted, configLayer.MapConfigBase); err != nil {
return err
}
}
}
iterations++
}
flowMessage.MplsLabel = mplsLabel
flowMessage.MplsTtl = mplsTtl
flowMessage.Etype = uint32(binary.BigEndian.Uint16(etherType[0:2]))
flowMessage.Ipv6FlowLabel = flowLabel & 0xFFFFF
flowMessage.SrcPort = uint32(srcPort)
flowMessage.DstPort = uint32(dstPort)
flowMessage.SrcAddr = srcIP
flowMessage.DstAddr = dstIP
flowMessage.Proto = uint32(nextHeader)
flowMessage.IpTos = uint32(tos)
flowMessage.IpTtl = uint32(ttl)
flowMessage.TcpFlags = uint32(tcpflags)
flowMessage.FragmentId = uint32(identification)
flowMessage.FragmentOffset = uint32(fragOffset)
return nil
}
func ParseSampledHeaderConfig(flowMessage *ProtoProducerMessage, sampledHeader *sflow.SampledHeader, config *SFlowMapper) error {
data := (*sampledHeader).HeaderData
switch (*sampledHeader).Protocol {
case 1: // Ethernet
if err := ParseEthernetHeader(flowMessage, data, config); err != nil {
return err
}
}
return nil
}
func SearchSFlowSampleConfig(flowMessage *ProtoProducerMessage, flowSample interface{}, config *SFlowMapper) error {
var records []sflow.FlowRecord
flowMessage.Type = flowmessage.FlowMessage_SFLOW_5
switch flowSample := flowSample.(type) {
case sflow.FlowSample:
records = flowSample.Records
flowMessage.SamplingRate = uint64(flowSample.SamplingRate)
flowMessage.InIf = flowSample.Input
flowMessage.OutIf = flowSample.Output
case sflow.ExpandedFlowSample:
records = flowSample.Records
flowMessage.SamplingRate = uint64(flowSample.SamplingRate)
flowMessage.InIf = flowSample.InputIfValue
flowMessage.OutIf = flowSample.OutputIfValue
}
var ipNh, ipSrc, ipDst []byte
flowMessage.Packets = 1
for _, record := range records {
switch recordData := record.Data.(type) {
case sflow.SampledHeader:
flowMessage.Bytes = uint64(recordData.FrameLength)
if err := ParseSampledHeaderConfig(flowMessage, &recordData, config); err != nil {
return err
}
case sflow.SampledIPv4:
ipSrc = recordData.SrcIP
ipDst = recordData.DstIP
flowMessage.SrcAddr = ipSrc
flowMessage.DstAddr = ipDst
flowMessage.Bytes = uint64(recordData.Length)
flowMessage.Proto = recordData.Protocol
flowMessage.SrcPort = recordData.SrcPort
flowMessage.DstPort = recordData.DstPort
flowMessage.IpTos = recordData.Tos
flowMessage.Etype = 0x800
case sflow.SampledIPv6:
ipSrc = recordData.SrcIP
ipDst = recordData.DstIP
flowMessage.SrcAddr = ipSrc
flowMessage.DstAddr = ipDst
flowMessage.Bytes = uint64(recordData.Length)
flowMessage.Proto = recordData.Protocol
flowMessage.SrcPort = recordData.SrcPort
flowMessage.DstPort = recordData.DstPort
flowMessage.IpTos = recordData.Priority
flowMessage.Etype = 0x86dd
case sflow.ExtendedRouter:
ipNh = recordData.NextHop
flowMessage.NextHop = ipNh
flowMessage.SrcNet = recordData.SrcMaskLen
flowMessage.DstNet = recordData.DstMaskLen
case sflow.ExtendedGateway:
ipNh = recordData.NextHop
flowMessage.BgpNextHop = ipNh
flowMessage.BgpCommunities = recordData.Communities
flowMessage.AsPath = recordData.ASPath
if len(recordData.ASPath) > 0 {
flowMessage.DstAs = recordData.ASPath[len(recordData.ASPath)-1]
flowMessage.NextHopAs = recordData.ASPath[0]
} else {
flowMessage.DstAs = recordData.AS
}
if recordData.SrcAS > 0 {
flowMessage.SrcAs = recordData.SrcAS
} else {
flowMessage.SrcAs = recordData.AS
}
case sflow.ExtendedSwitch:
flowMessage.SrcVlan = recordData.SrcVlan
flowMessage.DstVlan = recordData.DstVlan
}
}
return nil
}
func SearchSFlowSamplesConfig(samples []interface{}, config *SFlowMapper) (flowMessageSet []producer.ProducerMessage, err error) {
for _, flowSample := range samples {
fmsg := protoMessagePool.Get().(*ProtoProducerMessage)
fmsg.Reset()
if err := SearchSFlowSampleConfig(fmsg, flowSample, config); err != nil {
return nil, err
}
flowMessageSet = append(flowMessageSet, fmsg)
}
return flowMessageSet, nil
}
// Converts an sFlow message
func ProcessMessageSFlowConfig(packet *sflow.Packet, config *producerConfigMapped) (flowMessageSet []producer.ProducerMessage, err error) {
seqnum := packet.SequenceNumber
agent := packet.AgentIP
var cfg *SFlowMapper
if config != nil {
cfg = config.SFlow
}
flowSamples := GetSFlowFlowSamples(packet)
flowMessageSet, err = SearchSFlowSamplesConfig(flowSamples, cfg)
if err != nil {
return flowMessageSet, err
}
for _, msg := range flowMessageSet {
fmsg, ok := msg.(*ProtoProducerMessage)
if !ok {
continue
}
fmsg.SamplerAddress = agent
fmsg.SequenceNum = seqnum
}
return flowMessageSet, nil
}

View File

@@ -1,10 +1,10 @@
package producer
package protoproducer
import (
"testing"
"github.com/netsampler/goflow2/decoders/netflow"
"github.com/netsampler/goflow2/decoders/sflow"
"github.com/netsampler/goflow2/v2/decoders/netflow"
"github.com/netsampler/goflow2/v2/decoders/sflow"
"github.com/stretchr/testify/assert"
)
@@ -29,13 +29,13 @@ func TestProcessMessageNetFlow(t *testing.T) {
FlowSets: dfs,
}
testsr := &SingleSamplingRateSystem{1}
_, err := ProcessMessageNetFlow(pktnf9, testsr)
_, err := ProcessMessageNetFlowV9Config(&pktnf9, testsr, nil)
assert.Nil(t, err)
pktipfix := netflow.IPFIXPacket{
FlowSets: dfs,
}
_, err = ProcessMessageNetFlow(pktipfix, testsr)
_, err = ProcessMessageIPFIXConfig(&pktipfix, testsr, nil)
assert.Nil(t, err)
}
@@ -73,13 +73,14 @@ func TestProcessMessageSFlow(t *testing.T) {
},
},
}
_, err := ProcessMessageSFlow(pkt)
_, err := ProcessMessageSFlowConfig(&pkt, nil)
assert.Nil(t, err)
}
func TestExpandedSFlowDecode(t *testing.T) {
flowMessages, err := ProcessMessageSFlow(getSflowPacket())
flowMessage := flowMessages[0]
flowMessages, err := ProcessMessageSFlowConfig(getSflowPacket(), nil)
flowMessageIf := flowMessages[0]
flowMessage := flowMessageIf.(*ProtoProducerMessage)
assert.Nil(t, err)
@@ -89,8 +90,8 @@ func TestExpandedSFlowDecode(t *testing.T) {
assert.Equal(t, []byte{0x09, 0x09, 0x09, 0x09}, flowMessage.NextHop)
}
func getSflowPacket() sflow.Packet {
return sflow.Packet{
func getSflowPacket() *sflow.Packet {
pkt := sflow.Packet{
Version: 5,
IPVersion: 1,
AgentIP: []uint8{1, 2, 3, 4},
@@ -188,4 +189,5 @@ func getSflowPacket() sflow.Packet {
},
},
}
return &pkt
}

105
producer/proto/proto.go Normal file
View File

@@ -0,0 +1,105 @@
package protoproducer
import (
"fmt"
"sync"
"github.com/netsampler/goflow2/v2/decoders/netflow"
"github.com/netsampler/goflow2/v2/decoders/netflowlegacy"
"github.com/netsampler/goflow2/v2/decoders/sflow"
"github.com/netsampler/goflow2/v2/producer"
)
type ProtoProducer struct {
cfgMapped *producerConfigMapped
samplinglock *sync.RWMutex
sampling map[string]SamplingRateSystem
samplingRateSystem func() SamplingRateSystem
}
func (p *ProtoProducer) enrich(flowMessageSet []producer.ProducerMessage, cb func(msg *ProtoProducerMessage)) {
for _, msg := range flowMessageSet {
fmsg, ok := msg.(*ProtoProducerMessage)
if !ok {
continue
}
cb(fmsg)
}
}
func (p *ProtoProducer) getSamplingRateSystem(args *producer.ProduceArgs) SamplingRateSystem {
key := args.Src.String()
p.samplinglock.RLock()
sampling, ok := p.sampling[key]
p.samplinglock.RUnlock()
if !ok {
sampling = p.samplingRateSystem()
p.samplinglock.Lock()
p.sampling[key] = sampling
p.samplinglock.Unlock()
}
return sampling
}
func (p *ProtoProducer) Produce(msg interface{}, args *producer.ProduceArgs) (flowMessageSet []producer.ProducerMessage, err error) {
tr := uint64(args.TimeReceived.UnixNano())
sa, _ := args.SamplerAddress.Unmap().MarshalBinary()
switch msgConv := msg.(type) {
case *netflowlegacy.PacketNetFlowV5:
flowMessageSet, err = ProcessMessageNetFlowLegacy(msgConv)
p.enrich(flowMessageSet, func(fmsg *ProtoProducerMessage) {
fmsg.SamplerAddress = sa
})
case *netflow.NFv9Packet:
samplingRateSystem := p.getSamplingRateSystem(args)
flowMessageSet, err = ProcessMessageNetFlowV9Config(msgConv, samplingRateSystem, p.cfgMapped)
p.enrich(flowMessageSet, func(fmsg *ProtoProducerMessage) {
fmsg.TimeReceivedNs = tr
fmsg.SamplerAddress = sa
})
case *netflow.IPFIXPacket:
samplingRateSystem := p.getSamplingRateSystem(args)
flowMessageSet, err = ProcessMessageIPFIXConfig(msgConv, samplingRateSystem, p.cfgMapped)
p.enrich(flowMessageSet, func(fmsg *ProtoProducerMessage) {
fmsg.TimeReceivedNs = tr
fmsg.SamplerAddress = sa
})
case *sflow.Packet:
flowMessageSet, err = ProcessMessageSFlowConfig(msgConv, p.cfgMapped)
p.enrich(flowMessageSet, func(fmsg *ProtoProducerMessage) {
fmsg.TimeReceivedNs = tr
fmsg.TimeFlowStartNs = tr
fmsg.TimeFlowEndNs = tr
})
default:
return flowMessageSet, fmt.Errorf("flow not recognized")
}
p.enrich(flowMessageSet, func(fmsg *ProtoProducerMessage) {
fmsg.formatter = p.cfgMapped.Formatter
})
return flowMessageSet, err
}
func (p *ProtoProducer) Commit(flowMessageSet []producer.ProducerMessage) {
for _, fmsg := range flowMessageSet {
protoMessagePool.Put(fmsg)
}
}
func (p *ProtoProducer) Close() {}
func CreateProtoProducer(cfg *ProducerConfig, samplingRateSystem func() SamplingRateSystem) (producer.ProducerInterface, error) {
cfgMapped, err := mapConfig(cfg)
return &ProtoProducer{
cfgMapped: cfgMapped,
samplinglock: &sync.RWMutex{},
sampling: make(map[string]SamplingRateSystem),
samplingRateSystem: samplingRateSystem,
}, err
}

208
producer/proto/reflect.go Normal file
View File

@@ -0,0 +1,208 @@
package protoproducer
import (
"fmt"
"reflect"
"google.golang.org/protobuf/encoding/protowire"
"github.com/netsampler/goflow2/v2/decoders/netflow"
)
type EndianType string
type ProtoType string
var (
BigEndian EndianType = "big"
LittleEndian EndianType = "little"
ProtoString ProtoType = "string"
ProtoVarint ProtoType = "varint"
ProtoTypeMap = map[string]ProtoType{
string(ProtoString): ProtoString,
string(ProtoVarint): ProtoVarint,
"bytes": ProtoString,
}
)
func GetBytes(d []byte, offset int, length int) []byte {
if length == 0 {
return nil
}
leftBytes := offset / 8
rightBytes := (offset + length) / 8
if (offset+length)%8 != 0 {
rightBytes += 1
}
if leftBytes >= len(d) {
return nil
}
if rightBytes > len(d) {
rightBytes = len(d)
}
chunk := make([]byte, rightBytes-leftBytes)
offsetMod8 := (offset % 8)
shiftAnd := byte(0xff >> (8 - offsetMod8))
var shifted byte
for i := range chunk {
j := len(chunk) - 1 - i
cur := d[j+leftBytes]
chunk[j] = (cur << offsetMod8) | shifted
shifted = shiftAnd & cur
}
last := len(chunk) - 1
shiftAndLast := byte(0xff << ((8 - ((offset + length) % 8)) % 8))
chunk[last] = chunk[last] & shiftAndLast
return chunk
}
func IsUInt(k reflect.Kind) bool {
return k == reflect.Uint8 || k == reflect.Uint16 || k == reflect.Uint32 || k == reflect.Uint64
}
func IsInt(k reflect.Kind) bool {
return k == reflect.Int8 || k == reflect.Int16 || k == reflect.Int32 || k == reflect.Int64
}
// Structure to help the MapCustom functions
// populate the protobuf data
type MapConfigBase struct {
// Used if the field inside the protobuf exists
// also serves as the field when rendering with text
Destination string
Endianness EndianType
// The following fields are used for mapping
// when the destination field does not exist
// inside the protobuf
ProtoIndex int32
ProtoType ProtoType
ProtoArray bool
}
func MapCustomNetFlow(flowMessage *ProtoProducerMessage, df netflow.DataField, mapper *NetFlowMapper) error {
if mapper == nil {
return nil
}
mapped, ok := mapper.Map(df)
if ok {
v := df.Value.([]byte)
if err := MapCustom(flowMessage, v, mapped.MapConfigBase); err != nil {
return err
}
}
return nil
}
func MapCustom(flowMessage *ProtoProducerMessage, v []byte, cfg MapConfigBase) error {
vfm := reflect.ValueOf(flowMessage)
vfm = reflect.Indirect(vfm)
fieldValue := vfm.FieldByName(cfg.Destination)
if fieldValue.IsValid() {
typeDest := fieldValue.Type()
fieldValueAddr := fieldValue.Addr()
if typeDest.Kind() == reflect.Slice {
if typeDest.Elem().Kind() == reflect.Uint8 {
fieldValue.SetBytes(v)
} else {
item := reflect.New(typeDest.Elem())
if IsUInt(typeDest.Elem().Kind()) {
if cfg.Endianness == LittleEndian {
if err := DecodeUNumberLE(v, item.Interface()); err != nil {
return err
}
} else {
if err := DecodeUNumber(v, item.Interface()); err != nil {
return err
}
}
} else if IsInt(typeDest.Elem().Kind()) {
if cfg.Endianness == LittleEndian {
if err := DecodeNumberLE(v, item.Interface()); err != nil {
return err
}
} else {
if err := DecodeNumber(v, item.Interface()); err != nil {
return err
}
}
}
itemi := reflect.Indirect(item)
tmpFieldValue := reflect.Append(fieldValue, itemi)
fieldValue.Set(tmpFieldValue)
}
} else if fieldValueAddr.IsValid() {
if IsUInt(typeDest.Kind()) {
if cfg.Endianness == LittleEndian {
if err := DecodeUNumberLE(v, fieldValueAddr.Interface()); err != nil {
return err
}
} else {
if err := DecodeUNumber(v, fieldValueAddr.Interface()); err != nil {
return err
}
}
} else if IsInt(typeDest.Kind()) {
if cfg.Endianness == LittleEndian {
if err := DecodeNumberLE(v, fieldValueAddr.Interface()); err != nil {
return err
}
} else {
if err := DecodeNumber(v, fieldValueAddr.Interface()); err != nil {
return err
}
}
}
}
} else if cfg.ProtoIndex > 0 {
fmr := flowMessage.ProtoReflect()
unk := fmr.GetUnknown()
if !cfg.ProtoArray {
var offset int
for offset < len(unk) {
num, _, length := protowire.ConsumeField(unk[offset:])
offset += length
if int32(num) == cfg.ProtoIndex {
// only one allowed
break
}
}
}
var dstVar uint64
if cfg.ProtoType == ProtoVarint {
if cfg.Endianness == LittleEndian {
if err := DecodeUNumberLE(v, &dstVar); err != nil {
return err
}
} else {
if err := DecodeUNumber(v, &dstVar); err != nil {
return err
}
}
// support signed int?
unk = protowire.AppendTag(unk, protowire.Number(cfg.ProtoIndex), protowire.VarintType)
unk = protowire.AppendVarint(unk, dstVar)
} else if cfg.ProtoType == ProtoString {
unk = protowire.AppendTag(unk, protowire.Number(cfg.ProtoIndex), protowire.BytesType)
unk = protowire.AppendString(unk, string(v))
} else {
return fmt.Errorf("could not insert into protobuf unknown")
}
fmr.SetUnknown(unk)
}
return nil
}

156
producer/proto/render.go Normal file
View File

@@ -0,0 +1,156 @@
package protoproducer
import (
"encoding/binary"
"encoding/hex"
"net"
"net/netip"
)
type RenderFunc func(msg *ProtoProducerMessage, fieldName string, data interface{}) interface{}
type RendererID string
const (
RendererNone RendererID = "none"
RendererIP RendererID = "ip"
RendererMac RendererID = "mac"
RendererEtype RendererID = "etype"
RendererProto RendererID = "proto"
RendererType RendererID = "type"
RendererNetwork RendererID = "network"
)
var (
renderers = map[RendererID]RenderFunc{
RendererNone: NilRenderer,
RendererIP: IPRenderer,
RendererMac: MacRenderer,
RendererEtype: EtypeRenderer,
RendererProto: ProtoRenderer,
}
defaultRenderers = map[string]RenderFunc{
"SrcMac": MacRenderer,
"DstMac": MacRenderer,
"SrcAddr": IPRenderer,
"DstAddr": IPRenderer,
"SamplerAddress": IPRenderer,
"NextHop": IPRenderer,
"MplsLabelIp": IPRenderer,
"Etype": EtypeRenderer,
"Proto": ProtoRenderer,
"SrcNet": NetworkRenderer,
"DstNet": NetworkRenderer,
"icmp_name": ICMPRenderer,
}
etypeName = map[uint32]string{
0x806: "ARP",
0x800: "IPv4",
0x86dd: "IPv6",
}
protoName = map[uint32]string{
1: "ICMP",
6: "TCP",
17: "UDP",
58: "ICMPv6",
132: "SCTP",
}
icmpTypeName = map[uint32]string{
0: "EchoReply",
3: "DestinationUnreachable",
8: "Echo",
9: "RouterAdvertisement",
10: "RouterSolicitation",
11: "TimeExceeded",
}
icmp6TypeName = map[uint32]string{
1: "DestinationUnreachable",
2: "PacketTooBig",
3: "TimeExceeded",
128: "EchoRequest",
129: "EchoReply",
133: "RouterSolicitation",
134: "RouterAdvertisement",
}
)
func NilRenderer(msg *ProtoProducerMessage, fieldName string, data interface{}) interface{} {
if dataIf, ok := data.(interface {
String() string
}); ok {
return dataIf.String()
}
if dataC, ok := data.([]byte); ok {
return hex.EncodeToString(dataC)
}
return data
}
func MacRenderer(msg *ProtoProducerMessage, fieldName string, data interface{}) interface{} {
if dataC, ok := data.(uint64); ok {
var mac [8]byte
binary.BigEndian.PutUint64(mac[:], dataC)
return net.HardwareAddr(mac[2:]).String()
}
return NilRenderer(msg, fieldName, data)
}
func RenderIP(addr []byte) string {
if addr == nil || (len(addr) != 4 && len(addr) != 16) {
return ""
}
ip, _ := netip.AddrFromSlice(addr)
return ip.String()
}
func IPRenderer(msg *ProtoProducerMessage, fieldName string, data interface{}) interface{} {
if dataC, ok := data.([]byte); ok {
return RenderIP(dataC)
}
return NilRenderer(msg, fieldName, data)
}
func EtypeRenderer(msg *ProtoProducerMessage, fieldName string, data interface{}) interface{} {
if dataC, ok := data.(uint32); ok {
return etypeName[dataC]
}
return "unknown"
}
func ProtoRenderer(msg *ProtoProducerMessage, fieldName string, data interface{}) interface{} {
if dataC, ok := data.(uint32); ok {
return protoName[dataC]
}
return "unknown"
}
func NetworkRenderer(msg *ProtoProducerMessage, fieldName string, data interface{}) interface{} {
var addr netip.Addr
if fieldName == "SrcNet" {
addr, _ = netip.AddrFromSlice(msg.SrcAddr)
} else if fieldName == "DstNet" {
addr, _ = netip.AddrFromSlice(msg.DstAddr)
}
if dataC, ok := data.(uint32); ok {
prefix, _ := addr.Prefix(int(dataC))
return prefix.String()
}
return "unknown"
}
func IcmpCodeType(proto, icmpCode, icmpType uint32) string {
if proto == 1 {
return icmpTypeName[icmpType]
} else if proto == 58 {
return icmp6TypeName[icmpType]
}
return "unknown"
}
func ICMPRenderer(msg *ProtoProducerMessage, fieldName string, data interface{}) interface{} {
return IcmpCodeType(uint32(msg.Proto), uint32(msg.IcmpCode), uint32(msg.IcmpType))
}

74
producer/raw/raw.go Normal file
View File

@@ -0,0 +1,74 @@
package rawproducer
import (
"encoding/json"
"fmt"
"net/netip"
"time"
"github.com/netsampler/goflow2/v2/decoders/netflow"
"github.com/netsampler/goflow2/v2/decoders/netflowlegacy"
"github.com/netsampler/goflow2/v2/decoders/sflow"
"github.com/netsampler/goflow2/v2/producer"
)
// Producer that keeps the same format
// as the original flow samples.
// This can be used for debugging (eg: getting NetFlow Option Templates)
type RawProducer struct {
}
// Raw message
type RawMessage struct {
Message interface{} `json:"message"`
Src netip.AddrPort `json:"src"`
TimeReceived time.Time `json:"time_received"`
}
func (m RawMessage) MarshalJSON() ([]byte, error) {
typeStr := "unknown"
switch m.Message.(type) {
case *netflowlegacy.PacketNetFlowV5:
typeStr = "netflowv5"
case *netflow.NFv9Packet:
typeStr = "netflowv9"
case *netflow.IPFIXPacket:
typeStr = "ipfix"
case *sflow.Packet:
typeStr = "sflow"
}
tmpStruct := struct {
Type string `json:"type"`
Message interface{} `json:"message"`
Src *netip.AddrPort `json:"src"`
TimeReceived *time.Time `json:"time_received"`
}{
Type: typeStr,
Message: m.Message,
Src: &m.Src,
TimeReceived: &m.TimeReceived,
}
return json.Marshal(tmpStruct)
}
func (m RawMessage) MarshalText() ([]byte, error) {
var msgContents []byte
var err error
if msg, ok := m.Message.(interface {
MarshalText() ([]byte, error)
}); ok {
msgContents, err = msg.MarshalText()
}
return []byte(fmt.Sprintf("%s %s: %s", m.TimeReceived.String(), m.Src.String(), string(msgContents))), err
}
func (p *RawProducer) Produce(msg interface{}, args *producer.ProduceArgs) ([]producer.ProducerMessage, error) {
// should return msg wrapped
// []*interface{msg,}
return []producer.ProducerMessage{RawMessage{msg, args.Src, args.TimeReceived}}, nil
}
func (p *RawProducer) Commit(flowMessageSet []producer.ProducerMessage) {}
func (p *RawProducer) Close() {}

View File

@@ -1,233 +0,0 @@
package producer
import (
"fmt"
"reflect"
"github.com/netsampler/goflow2/decoders/netflow"
flowmessage "github.com/netsampler/goflow2/pb"
)
type EndianType string
var (
BigEndian EndianType = "big"
LittleEndian EndianType = "little"
)
func GetBytes(d []byte, offset int, length int) []byte {
if length == 0 {
return nil
}
leftBytes := offset / 8
rightBytes := (offset + length) / 8
if (offset+length)%8 != 0 {
rightBytes += 1
}
if leftBytes >= len(d) {
return nil
}
if rightBytes > len(d) {
rightBytes = len(d)
}
chunk := make([]byte, rightBytes-leftBytes)
offsetMod8 := (offset % 8)
shiftAnd := byte(0xff >> (8 - offsetMod8))
var shifted byte
for i := range chunk {
j := len(chunk) - 1 - i
cur := d[j+leftBytes]
chunk[j] = (cur << offsetMod8) | shifted
shifted = shiftAnd & cur
}
last := len(chunk) - 1
shiftAndLast := byte(0xff << ((8 - ((offset + length) % 8)) % 8))
chunk[last] = chunk[last] & shiftAndLast
return chunk
}
func IsUInt(k reflect.Kind) bool {
return k == reflect.Uint8 || k == reflect.Uint16 || k == reflect.Uint32 || k == reflect.Uint64
}
func IsInt(k reflect.Kind) bool {
return k == reflect.Int8 || k == reflect.Int16 || k == reflect.Int32 || k == reflect.Int64
}
func MapCustomNetFlow(flowMessage *flowmessage.FlowMessage, df netflow.DataField, mapper *NetFlowMapper) {
if mapper == nil {
return
}
mapped, ok := mapper.Map(df)
if ok {
v := df.Value.([]byte)
MapCustom(flowMessage, v, mapped.Destination, mapped.Endian)
}
}
func MapCustom(flowMessage *flowmessage.FlowMessage, v []byte, destination string, endianness EndianType) {
vfm := reflect.ValueOf(flowMessage)
vfm = reflect.Indirect(vfm)
fieldValue := vfm.FieldByName(destination)
if fieldValue.IsValid() {
typeDest := fieldValue.Type()
fieldValueAddr := fieldValue.Addr()
if typeDest.Kind() == reflect.Slice {
if typeDest.Elem().Kind() == reflect.Uint8 {
fieldValue.SetBytes(v)
} else {
item := reflect.New(typeDest.Elem())
if IsUInt(typeDest.Elem().Kind()) {
if endianness == LittleEndian {
DecodeUNumberLE(v, item.Interface())
} else {
DecodeUNumber(v, item.Interface())
}
} else if IsUInt(typeDest.Elem().Kind()) {
if endianness == LittleEndian {
DecodeUNumberLE(v, item.Interface())
} else {
DecodeUNumber(v, item.Interface())
}
}
itemi := reflect.Indirect(item)
tmpFieldValue := reflect.Append(fieldValue, itemi)
fieldValue.Set(tmpFieldValue)
}
} else if fieldValueAddr.IsValid() && IsUInt(typeDest.Kind()) {
if endianness == LittleEndian {
DecodeUNumberLE(v, fieldValueAddr.Interface())
} else {
DecodeUNumber(v, fieldValueAddr.Interface())
}
} else if fieldValueAddr.IsValid() && IsInt(typeDest.Kind()) {
if endianness == LittleEndian {
DecodeUNumberLE(v, fieldValueAddr.Interface())
} else {
DecodeUNumber(v, fieldValueAddr.Interface())
}
}
}
}
type NetFlowMapField struct {
PenProvided bool `json:"penprovided" yaml:"penprovided"`
Type uint16 `json:"field" yaml:"field"`
Pen uint32 `json:"pen" yaml:"pen"`
Destination string `json:"destination" yaml:"destination"`
Endian EndianType `json:"endianness" yaml:"endianness"`
//DestinationLength uint8 `json:"dlen"` // could be used if populating a slice of uint16 that aren't in protobuf
}
type IPFIXProducerConfig struct {
Mapping []NetFlowMapField `json:"mapping"`
//PacketMapping []SFlowMapField `json:"packet-mapping"` // for embedded frames: use sFlow configuration
}
type NetFlowV9ProducerConfig struct {
Mapping []NetFlowMapField `json:"mapping"`
}
type SFlowMapField struct {
Layer int `json:"layer"`
Offset int `json:"offset"` // offset in bits
Length int `json:"length"` // length in bits
Destination string `json:"destination" yaml:"destination"`
Endian EndianType `json:"endianness" yaml:"endianness"`
//DestinationLength uint8 `json:"dlen"`
}
type SFlowProducerConfig struct {
Mapping []SFlowMapField `json:"mapping"`
}
type ProducerConfig struct {
IPFIX IPFIXProducerConfig `json:"ipfix"`
NetFlowV9 NetFlowV9ProducerConfig `json:"netflowv9"`
SFlow SFlowProducerConfig `json:"sflow"` // also used for IPFIX data frames
// should do a rename map list for when printing
}
type DataMap struct {
Destination string
Endian EndianType
}
type NetFlowMapper struct {
data map[string]DataMap // maps field to destination
}
func (m *NetFlowMapper) Map(field netflow.DataField) (DataMap, bool) {
mapped, found := m.data[fmt.Sprintf("%v-%d-%d", field.PenProvided, field.Pen, field.Type)]
return mapped, found
}
func MapFieldsNetFlow(fields []NetFlowMapField) *NetFlowMapper {
ret := make(map[string]DataMap)
for _, field := range fields {
ret[fmt.Sprintf("%v-%d-%d", field.PenProvided, field.Pen, field.Type)] = DataMap{Destination: field.Destination, Endian: field.Endian}
}
return &NetFlowMapper{ret}
}
type DataMapLayer struct {
Offset int
Length int
Destination string
Endian EndianType
}
type SFlowMapper struct {
data map[int][]DataMapLayer // map layer to list of offsets
}
func GetSFlowConfigLayer(m *SFlowMapper, layer int) []DataMapLayer {
if m == nil {
return nil
}
return m.data[layer]
}
func MapFieldsSFlow(fields []SFlowMapField) *SFlowMapper {
ret := make(map[int][]DataMapLayer)
for _, field := range fields {
retLayerEntry := DataMapLayer{
Offset: field.Offset,
Length: field.Length,
Destination: field.Destination,
Endian: field.Endian,
}
retLayer := ret[field.Layer]
retLayer = append(retLayer, retLayerEntry)
ret[field.Layer] = retLayer
}
return &SFlowMapper{ret}
}
type ProducerConfigMapped struct {
IPFIX *NetFlowMapper `json:"ipfix"`
NetFlowV9 *NetFlowMapper `json:"netflowv9"`
SFlow *SFlowMapper `json:"sflow"`
}
func NewProducerConfigMapped(config *ProducerConfig) *ProducerConfigMapped {
newCfg := &ProducerConfigMapped{}
if config != nil {
newCfg.IPFIX = MapFieldsNetFlow(config.IPFIX.Mapping)
newCfg.NetFlowV9 = MapFieldsNetFlow(config.NetFlowV9.Mapping)
newCfg.SFlow = MapFieldsSFlow(config.SFlow.Mapping)
}
return newCfg
}

View File

@@ -1,10 +1,9 @@
package file
import (
"context"
"flag"
"fmt"
"github.com/netsampler/goflow2/transport"
"github.com/netsampler/goflow2/v2/transport"
"io"
"os"
"os/signal"
@@ -38,7 +37,7 @@ func (d *FileDriver) openFile() error {
return err
}
func (d *FileDriver) Init(context.Context) error {
func (d *FileDriver) Init() error {
d.q = make(chan bool, 1)
if d.fileDestination == "" {
@@ -61,8 +60,11 @@ func (d *FileDriver) Init(context.Context) error {
case <-c:
d.lock.Lock()
d.file.Close()
d.openFile()
err := d.openFile()
d.lock.Unlock()
if err != nil {
return
}
// if there is an error, keeps using the old file
case <-d.q:
return
@@ -82,7 +84,7 @@ func (d *FileDriver) Send(key, data []byte) error {
return err
}
func (d *FileDriver) Close(context.Context) error {
func (d *FileDriver) Close() error {
if d.fileDestination != "" {
d.lock.Lock()
d.file.Close()

View File

@@ -1,27 +1,25 @@
package kafka
import (
"context"
"crypto/tls"
"crypto/x509"
"errors"
"flag"
"fmt"
"net"
"os"
"strconv"
"strings"
"time"
sarama "github.com/Shopify/sarama"
"github.com/netsampler/goflow2/transport"
"github.com/netsampler/goflow2/utils"
"github.com/netsampler/goflow2/v2/transport"
log "github.com/sirupsen/logrus"
sarama "github.com/Shopify/sarama"
)
type KafkaDriver struct {
kafkaTLS bool
kafkaSASL string
kafkaSCRAM string
kafkaTopic string
kafkaSrv string
kafkaBrk string
@@ -29,8 +27,6 @@ type KafkaDriver struct {
kafkaFlushBytes int
kafkaFlushFrequency time.Duration
kafkaLogErrors bool
kafkaHashing bool
kafkaVersion string
kafkaCompressionCodec string
@@ -38,6 +34,20 @@ type KafkaDriver struct {
producer sarama.AsyncProducer
q chan bool
errors chan error
}
// Error specifically for inner Kafka errors
type KafkaTransportError struct {
Err error
}
func (e *KafkaTransportError) Error() string {
return fmt.Sprintf("kafka transport %s", e.Err.Error())
}
func (e *KafkaTransportError) Unwrap() []error {
return []error{transport.ErrorTransport, e.Err}
}
type KafkaSASLAlgorithm string
@@ -85,17 +95,19 @@ func (d *KafkaDriver) Prepare() error {
flag.IntVar(&d.kafkaFlushBytes, "transport.kafka.flushbytes", int(sarama.MaxRequestSize), "Kafka flush bytes")
flag.DurationVar(&d.kafkaFlushFrequency, "transport.kafka.flushfreq", time.Second*5, "Kafka flush frequency")
flag.BoolVar(&d.kafkaLogErrors, "transport.kafka.log.err", false, "Log Kafka errors")
flag.BoolVar(&d.kafkaHashing, "transport.kafka.hashing", false, "Enable partition hashing")
//flag.StringVar(&d.kafkaKeying, "transport.kafka.key", "SamplerAddress,DstAS", "Kafka list of fields to do hashing on (partition) separated by commas")
flag.StringVar(&d.kafkaVersion, "transport.kafka.version", "2.8.0", "Kafka version")
flag.StringVar(&d.kafkaCompressionCodec, "transport.kafka.compression", "", "Kafka default compression")
return nil
}
func (d *KafkaDriver) Init(context.Context) error {
func (d *KafkaDriver) Errors() <-chan error {
return d.errors
}
func (d *KafkaDriver) Init() error {
kafkaConfigVersion, err := sarama.ParseKafkaVersion(d.kafkaVersion)
if err != nil {
return err
@@ -104,7 +116,7 @@ func (d *KafkaDriver) Init(context.Context) error {
kafkaConfig := sarama.NewConfig()
kafkaConfig.Version = kafkaConfigVersion
kafkaConfig.Producer.Return.Successes = false
kafkaConfig.Producer.Return.Errors = d.kafkaLogErrors
kafkaConfig.Producer.Return.Errors = true
kafkaConfig.Producer.MaxMessageBytes = d.kafkaMaxMsgBytes
kafkaConfig.Producer.Flush.Bytes = d.kafkaFlushBytes
kafkaConfig.Producer.Flush.Frequency = d.kafkaFlushFrequency
@@ -122,7 +134,7 @@ func (d *KafkaDriver) Init(context.Context) error {
*/
if cc, ok := compressionCodecs[strings.ToLower(d.kafkaCompressionCodec)]; !ok {
return errors.New("compression codec does not exist")
return fmt.Errorf("compression codec does not exist")
} else {
kafkaConfig.Producer.Compression = cc
}
@@ -131,10 +143,13 @@ func (d *KafkaDriver) Init(context.Context) error {
if d.kafkaTLS {
rootCAs, err := x509.SystemCertPool()
if err != nil {
return errors.New(fmt.Sprintf("Error initializing TLS: %v", err))
return fmt.Errorf("error initializing TLS: %v", err)
}
kafkaConfig.Net.TLS.Enable = true
kafkaConfig.Net.TLS.Config = &tls.Config{RootCAs: rootCAs}
kafkaConfig.Net.TLS.Config = &tls.Config{
RootCAs: rootCAs,
MinVersion: tls.VersionTLS12,
}
}
if d.kafkaHashing {
@@ -152,7 +167,7 @@ func (d *KafkaDriver) Init(context.Context) error {
kafkaConfig.Net.SASL.User = os.Getenv("KAFKA_SASL_USER")
kafkaConfig.Net.SASL.Password = os.Getenv("KAFKA_SASL_PASS")
if kafkaConfig.Net.SASL.User == "" && kafkaConfig.Net.SASL.Password == "" {
return errors.New("Kafka SASL config from environment was unsuccessful. KAFKA_SASL_USER and KAFKA_SASL_PASS need to be set.")
return fmt.Errorf("Kafka SASL config from environment was unsuccessful. KAFKA_SASL_USER and KAFKA_SASL_PASS need to be set.")
}
if kafkaSASL == KAFKA_SASL_SCRAM_SHA256 || kafkaSASL == KAFKA_SASL_SCRAM_SHA512 {
@@ -174,7 +189,7 @@ func (d *KafkaDriver) Init(context.Context) error {
var addrs []string
if d.kafkaSrv != "" {
addrs, _ = utils.GetServiceAddresses(d.kafkaSrv)
addrs, _ = GetServiceAddresses(d.kafkaSrv)
} else {
addrs = strings.Split(d.kafkaBrk, ",")
}
@@ -187,20 +202,27 @@ func (d *KafkaDriver) Init(context.Context) error {
d.q = make(chan bool)
if d.kafkaLogErrors {
go func() {
for {
go func() {
for {
select {
case msg := <-kafkaProducer.Errors():
var err error
if msg != nil {
err = &KafkaTransportError{msg}
}
select {
case msg := <-kafkaProducer.Errors():
//if log != nil {
log.Error(msg)
//}
case <-d.q:
case d.errors <- err:
default:
}
if msg == nil {
return
}
case <-d.q:
return
}
}()
}
}
}()
return err
}
@@ -214,13 +236,27 @@ func (d *KafkaDriver) Send(key, data []byte) error {
return nil
}
func (d *KafkaDriver) Close(context.Context) error {
func (d *KafkaDriver) Close() error {
d.producer.Close()
close(d.q)
return nil
}
// todo: deprecate?
func GetServiceAddresses(srv string) (addrs []string, err error) {
_, srvs, err := net.LookupSRV("", "", srv)
if err != nil {
return nil, fmt.Errorf("service discovery: %v\n", err)
}
for _, srv := range srvs {
addrs = append(addrs, net.JoinHostPort(srv.Target, strconv.Itoa(int(srv.Port))))
}
return addrs, nil
}
func init() {
d := &KafkaDriver{}
d := &KafkaDriver{
errors: make(chan error),
}
transport.RegisterTransportDriver("kafka", d)
}

View File

@@ -1,7 +1,6 @@
package transport
import (
"context"
"fmt"
"sync"
)
@@ -9,12 +8,27 @@ import (
var (
transportDrivers = make(map[string]TransportDriver)
lock = &sync.RWMutex{}
ErrorTransport = fmt.Errorf("transport error")
)
type DriverTransportError struct {
Driver string
Err error
}
func (e *DriverTransportError) Error() string {
return fmt.Sprintf("%s for %s transport", e.Err.Error(), e.Driver)
}
func (e *DriverTransportError) Unwrap() []error {
return []error{ErrorTransport, e.Err}
}
type TransportDriver interface {
Prepare() error // Prepare driver (eg: flag registration)
Init(context.Context) error // Initialize driver (eg: start connections, open files...)
Close(context.Context) error // Close driver (eg: close connections and files...)
Init() error // Initialize driver (eg: start connections, open files...)
Close() error // Close driver (eg: close connections and files...)
Send(key, data []byte) error // Send a formatted message
}
@@ -23,14 +37,22 @@ type TransportInterface interface {
}
type Transport struct {
driver TransportDriver
TransportDriver
name string
}
func (t *Transport) Close(ctx context.Context) {
t.driver.Close(ctx)
func (t *Transport) Close() error {
if err := t.TransportDriver.Close(); err != nil {
return &DriverTransportError{t.name, err}
}
return nil
}
func (t *Transport) Send(key, data []byte) error {
return t.driver.Send(key, data)
if err := t.TransportDriver.Send(key, data); err != nil {
return &DriverTransportError{t.name, err}
}
return nil
}
func RegisterTransportDriver(name string, t TransportDriver) {
@@ -43,16 +65,19 @@ func RegisterTransportDriver(name string, t TransportDriver) {
}
}
func FindTransport(ctx context.Context, name string) (*Transport, error) {
func FindTransport(name string) (*Transport, error) {
lock.RLock()
t, ok := transportDrivers[name]
lock.RUnlock()
if !ok {
return nil, fmt.Errorf("Transport %s not found", name)
return nil, fmt.Errorf("%w %s not found", ErrorTransport, name)
}
err := t.Init(ctx)
return &Transport{t}, err
err := t.Init()
if err != nil {
err = &DriverTransportError{name, err}
}
return &Transport{t, name}, err
}
func GetTransports() []string {
@@ -60,7 +85,7 @@ func GetTransports() []string {
defer lock.RUnlock()
t := make([]string, len(transportDrivers))
var i int
for k, _ := range transportDrivers {
for k := range transportDrivers {
t[i] = k
i++
}

View File

@@ -1,171 +0,0 @@
package utils
import (
"strconv"
"time"
"github.com/prometheus/client_golang/prometheus"
)
var (
MetricTrafficBytes = prometheus.NewCounterVec(
prometheus.CounterOpts{
Name: "flow_traffic_bytes",
Help: "Bytes received by the application.",
},
[]string{"remote_ip", "local_ip", "local_port", "type"},
)
MetricTrafficPackets = prometheus.NewCounterVec(
prometheus.CounterOpts{
Name: "flow_traffic_packets",
Help: "Packets received by the application.",
},
[]string{"remote_ip", "local_ip", "local_port", "type"},
)
MetricPacketSizeSum = prometheus.NewSummaryVec(
prometheus.SummaryOpts{
Name: "flow_traffic_summary_size_bytes",
Help: "Summary of packet size.",
Objectives: map[float64]float64{0.5: 0.05, 0.9: 0.01, 0.99: 0.001},
},
[]string{"remote_ip", "local_ip", "local_port", "type"},
)
DecoderStats = prometheus.NewCounterVec(
prometheus.CounterOpts{
Name: "flow_decoder_count",
Help: "Decoder processed count.",
},
[]string{"worker", "name"},
)
DecoderErrors = prometheus.NewCounterVec(
prometheus.CounterOpts{
Name: "flow_decoder_error_count",
Help: "Decoder processed error count.",
},
[]string{"worker", "name"},
)
DecoderTime = prometheus.NewSummaryVec(
prometheus.SummaryOpts{
Name: "flow_summary_decoding_time_us",
Help: "Decoding time summary.",
Objectives: map[float64]float64{0.5: 0.05, 0.9: 0.01, 0.99: 0.001},
},
[]string{"name"},
)
DecoderProcessTime = prometheus.NewSummaryVec(
prometheus.SummaryOpts{
Name: "flow_summary_processing_time_us",
Help: "Processing time summary.",
Objectives: map[float64]float64{0.5: 0.05, 0.9: 0.01, 0.99: 0.001},
},
[]string{"name"},
)
NetFlowStats = prometheus.NewCounterVec(
prometheus.CounterOpts{
Name: "flow_process_nf_count",
Help: "NetFlows processed.",
},
[]string{"router", "version"},
)
NetFlowErrors = prometheus.NewCounterVec(
prometheus.CounterOpts{
Name: "flow_process_nf_errors_count",
Help: "NetFlows processed errors.",
},
[]string{"router", "error"},
)
NetFlowSetRecordsStatsSum = prometheus.NewCounterVec(
prometheus.CounterOpts{
Name: "flow_process_nf_flowset_records_sum",
Help: "NetFlows FlowSets sum of records.",
},
[]string{"router", "version", "type"}, // data-template, data, opts...
)
NetFlowSetStatsSum = prometheus.NewCounterVec(
prometheus.CounterOpts{
Name: "flow_process_nf_flowset_sum",
Help: "NetFlows FlowSets sum.",
},
[]string{"router", "version", "type"}, // data-template, data, opts...
)
NetFlowTimeStatsSum = prometheus.NewSummaryVec(
prometheus.SummaryOpts{
Name: "flow_process_nf_delay_summary_seconds",
Help: "NetFlows time difference between time of flow and processing.",
Objectives: map[float64]float64{0.5: 0.05, 0.9: 0.01, 0.99: 0.001},
},
[]string{"router", "version"},
)
NetFlowTemplatesStats = prometheus.NewCounterVec(
prometheus.CounterOpts{
Name: "flow_process_nf_templates_count",
Help: "NetFlows Template count.",
},
[]string{"router", "version", "obs_domain_id", "template_id", "type"}, // options/template
)
SFlowStats = prometheus.NewCounterVec(
prometheus.CounterOpts{
Name: "flow_process_sf_count",
Help: "sFlows processed.",
},
[]string{"router", "agent", "version"},
)
SFlowErrors = prometheus.NewCounterVec(
prometheus.CounterOpts{
Name: "flow_process_sf_errors_count",
Help: "sFlows processed errors.",
},
[]string{"router", "error"},
)
SFlowSampleStatsSum = prometheus.NewCounterVec(
prometheus.CounterOpts{
Name: "flow_process_sf_samples_sum",
Help: "SFlows samples sum.",
},
[]string{"router", "agent", "version", "type"}, // counter, flow, expanded...
)
SFlowSampleRecordsStatsSum = prometheus.NewCounterVec(
prometheus.CounterOpts{
Name: "flow_process_sf_samples_records_sum",
Help: "SFlows samples sum of records.",
},
[]string{"router", "agent", "version", "type"}, // data-template, data, opts...
)
)
func init() {
prometheus.MustRegister(MetricTrafficBytes)
prometheus.MustRegister(MetricTrafficPackets)
prometheus.MustRegister(MetricPacketSizeSum)
prometheus.MustRegister(DecoderStats)
prometheus.MustRegister(DecoderErrors)
prometheus.MustRegister(DecoderTime)
prometheus.MustRegister(DecoderProcessTime)
prometheus.MustRegister(NetFlowStats)
prometheus.MustRegister(NetFlowErrors)
prometheus.MustRegister(NetFlowSetRecordsStatsSum)
prometheus.MustRegister(NetFlowSetStatsSum)
prometheus.MustRegister(NetFlowTimeStatsSum)
prometheus.MustRegister(NetFlowTemplatesStats)
prometheus.MustRegister(SFlowStats)
prometheus.MustRegister(SFlowErrors)
prometheus.MustRegister(SFlowSampleStatsSum)
prometheus.MustRegister(SFlowSampleRecordsStatsSum)
}
func DefaultAccountCallback(name string, id int, start, end time.Time) {
DecoderProcessTime.With(
prometheus.Labels{
"name": name,
}).
Observe(float64((end.Sub(start)).Nanoseconds()) / 1000)
DecoderStats.With(
prometheus.Labels{
"worker": strconv.Itoa(id),
"name": name,
}).
Inc()
}

View File

@@ -1,377 +0,0 @@
package utils
import (
"bytes"
"context"
"sync"
"time"
"github.com/netsampler/goflow2/decoders/netflow"
"github.com/netsampler/goflow2/decoders/netflow/templates"
"github.com/netsampler/goflow2/format"
flowmessage "github.com/netsampler/goflow2/pb"
"github.com/netsampler/goflow2/producer"
"github.com/netsampler/goflow2/transport"
"github.com/prometheus/client_golang/prometheus"
)
/*
type TemplateSystem struct {
key string
templates *netflow.BasicTemplateSystem
}
func (s *TemplateSystem) AddTemplate(version uint16, obsDomainId uint32, template interface{}) {
s.templates.AddTemplate(version, obsDomainId, template)
typeStr := "options_template"
var templateId uint16
switch templateIdConv := template.(type) {
case netflow.IPFIXOptionsTemplateRecord:
templateId = templateIdConv.TemplateId
case netflow.NFv9OptionsTemplateRecord:
templateId = templateIdConv.TemplateId
case netflow.TemplateRecord:
templateId = templateIdConv.TemplateId
typeStr = "template"
}
NetFlowTemplatesStats.With(
prometheus.Labels{
"router": s.key,
"version": strconv.Itoa(int(version)),
"obs_domain_id": strconv.Itoa(int(obsDomainId)),
"template_id": strconv.Itoa(int(templateId)),
"type": typeStr,
}).
Inc()
}
func (s *TemplateSystem) GetTemplate(version uint16, obsDomainId uint32, templateId uint16) (interface{}, error) {
return s.templates.GetTemplate(version, obsDomainId, templateId)
}
*/
type StateNetFlow struct {
stopper
Format format.FormatInterface
Transport transport.TransportInterface
Logger Logger
/*templateslock *sync.RWMutex
templates map[string]*TemplateSystem*/
samplinglock *sync.RWMutex
sampling map[string]producer.SamplingRateSystem
Config *producer.ProducerConfig
configMapped *producer.ProducerConfigMapped
TemplateSystem templates.TemplateInterface
ctx context.Context
}
func NewStateNetFlow() *StateNetFlow {
return &StateNetFlow{
ctx: context.Background(),
samplinglock: &sync.RWMutex{},
sampling: make(map[string]producer.SamplingRateSystem),
}
}
func (s *StateNetFlow) DecodeFlow(msg interface{}) error {
pkt := msg.(BaseMessage)
buf := bytes.NewBuffer(pkt.Payload)
key := pkt.Src.String()
samplerAddress := pkt.Src
if samplerAddress.To4() != nil {
samplerAddress = samplerAddress.To4()
}
s.samplinglock.RLock()
sampling, ok := s.sampling[key]
s.samplinglock.RUnlock()
if !ok {
sampling = producer.CreateSamplingSystem()
s.samplinglock.Lock()
s.sampling[key] = sampling
s.samplinglock.Unlock()
}
ts := uint64(time.Now().UTC().Unix())
if pkt.SetTime {
ts = uint64(pkt.RecvTime.UTC().Unix())
}
timeTrackStart := time.Now()
msgDec, err := netflow.DecodeMessageContext(s.ctx, buf, key, netflow.TemplateWrapper{s.ctx, key, s.TemplateSystem})
if err != nil {
switch err.(type) {
case *netflow.ErrorTemplateNotFound:
NetFlowErrors.With(
prometheus.Labels{
"router": key,
"error": "template_not_found",
}).
Inc()
default:
NetFlowErrors.With(
prometheus.Labels{
"router": key,
"error": "error_decoding",
}).
Inc()
}
return err
}
var flowMessageSet []*flowmessage.FlowMessage
switch msgDecConv := msgDec.(type) {
case netflow.NFv9Packet:
NetFlowStats.With(
prometheus.Labels{
"router": key,
"version": "9",
}).
Inc()
for _, fs := range msgDecConv.FlowSets {
switch fsConv := fs.(type) {
case netflow.TemplateFlowSet:
NetFlowSetStatsSum.With(
prometheus.Labels{
"router": key,
"version": "9",
"type": "TemplateFlowSet",
}).
Inc()
NetFlowSetRecordsStatsSum.With(
prometheus.Labels{
"router": key,
"version": "9",
"type": "TemplateFlowSet",
}).
Add(float64(len(fsConv.Records)))
case netflow.NFv9OptionsTemplateFlowSet:
NetFlowSetStatsSum.With(
prometheus.Labels{
"router": key,
"version": "9",
"type": "OptionsTemplateFlowSet",
}).
Inc()
NetFlowSetRecordsStatsSum.With(
prometheus.Labels{
"router": key,
"version": "9",
"type": "OptionsTemplateFlowSet",
}).
Add(float64(len(fsConv.Records)))
case netflow.OptionsDataFlowSet:
NetFlowSetStatsSum.With(
prometheus.Labels{
"router": key,
"version": "9",
"type": "OptionsDataFlowSet",
}).
Inc()
NetFlowSetRecordsStatsSum.With(
prometheus.Labels{
"router": key,
"version": "9",
"type": "OptionsDataFlowSet",
}).
Add(float64(len(fsConv.Records)))
case netflow.DataFlowSet:
NetFlowSetStatsSum.With(
prometheus.Labels{
"router": key,
"version": "9",
"type": "DataFlowSet",
}).
Inc()
NetFlowSetRecordsStatsSum.With(
prometheus.Labels{
"router": key,
"version": "9",
"type": "DataFlowSet",
}).
Add(float64(len(fsConv.Records)))
}
}
flowMessageSet, err = producer.ProcessMessageNetFlowConfig(msgDecConv, sampling, s.configMapped)
for _, fmsg := range flowMessageSet {
fmsg.TimeReceived = ts
fmsg.SamplerAddress = samplerAddress
timeDiff := fmsg.TimeReceived - fmsg.TimeFlowEnd
NetFlowTimeStatsSum.With(
prometheus.Labels{
"router": key,
"version": "9",
}).
Observe(float64(timeDiff))
}
case netflow.IPFIXPacket:
NetFlowStats.With(
prometheus.Labels{
"router": key,
"version": "10",
}).
Inc()
for _, fs := range msgDecConv.FlowSets {
switch fsConv := fs.(type) {
case netflow.TemplateFlowSet:
NetFlowSetStatsSum.With(
prometheus.Labels{
"router": key,
"version": "10",
"type": "TemplateFlowSet",
}).
Inc()
NetFlowSetRecordsStatsSum.With(
prometheus.Labels{
"router": key,
"version": "10",
"type": "TemplateFlowSet",
}).
Add(float64(len(fsConv.Records)))
case netflow.IPFIXOptionsTemplateFlowSet:
NetFlowSetStatsSum.With(
prometheus.Labels{
"router": key,
"version": "10",
"type": "OptionsTemplateFlowSet",
}).
Inc()
NetFlowSetRecordsStatsSum.With(
prometheus.Labels{
"router": key,
"version": "10",
"type": "OptionsTemplateFlowSet",
}).
Add(float64(len(fsConv.Records)))
case netflow.OptionsDataFlowSet:
NetFlowSetStatsSum.With(
prometheus.Labels{
"router": key,
"version": "10",
"type": "OptionsDataFlowSet",
}).
Inc()
NetFlowSetRecordsStatsSum.With(
prometheus.Labels{
"router": key,
"version": "10",
"type": "OptionsDataFlowSet",
}).
Add(float64(len(fsConv.Records)))
case netflow.DataFlowSet:
NetFlowSetStatsSum.With(
prometheus.Labels{
"router": key,
"version": "10",
"type": "DataFlowSet",
}).
Inc()
NetFlowSetRecordsStatsSum.With(
prometheus.Labels{
"router": key,
"version": "10",
"type": "DataFlowSet",
}).
Add(float64(len(fsConv.Records)))
}
}
flowMessageSet, err = producer.ProcessMessageNetFlowConfig(msgDecConv, sampling, s.configMapped)
for _, fmsg := range flowMessageSet {
fmsg.TimeReceived = ts
fmsg.SamplerAddress = samplerAddress
timeDiff := fmsg.TimeReceived - fmsg.TimeFlowEnd
NetFlowTimeStatsSum.With(
prometheus.Labels{
"router": key,
"version": "10",
}).
Observe(float64(timeDiff))
}
}
timeTrackStop := time.Now()
DecoderTime.With(
prometheus.Labels{
"name": "NetFlow",
}).
Observe(float64((timeTrackStop.Sub(timeTrackStart)).Nanoseconds()) / 1000)
for _, fmsg := range flowMessageSet {
if s.Format != nil {
key, data, err := s.Format.Format(fmsg)
if err != nil && s.Logger != nil {
s.Logger.Error(err)
}
if err == nil && s.Transport != nil {
err = s.Transport.Send(key, data)
if err != nil {
s.Logger.Error(err)
}
}
}
}
return nil
}
/*
func (s *StateNetFlow) ServeHTTPTemplates(w http.ResponseWriter, r *http.Request) {
tmp := make(map[string]map[uint16]map[uint32]map[uint16]interface{})
s.templateslock.RLock()
for key, templatesrouterstr := range s.templates {
templatesrouter := templatesrouterstr.templates.GetTemplates()
tmp[key] = templatesrouter
}
s.templateslock.RUnlock()
enc := json.NewEncoder(w)
enc.Encode(tmp)
}
func (s *StateNetFlow) InitTemplates() {
s.templates = make(map[string]*TemplateSystem)
s.templateslock = &sync.RWMutex{}
s.sampling = make(map[string]producer.SamplingRateSystem)
s.samplinglock = &sync.RWMutex{}
}*/
func (s *StateNetFlow) initConfig() {
s.configMapped = producer.NewProducerConfigMapped(s.Config)
}
func (s *StateNetFlow) FlowRoutine(workers int, addr string, port int, reuseport bool) error {
if err := s.start(); err != nil {
return err
}
//s.InitTemplates()
s.initConfig()
return UDPStoppableRoutine(s.stopCh, "NetFlow", s.DecodeFlow, workers, addr, port, reuseport, s.Logger)
}
// FlowRoutineCtx?

View File

@@ -1,111 +0,0 @@
package utils
import (
"bytes"
"time"
"github.com/netsampler/goflow2/decoders/netflowlegacy"
"github.com/netsampler/goflow2/format"
flowmessage "github.com/netsampler/goflow2/pb"
"github.com/netsampler/goflow2/producer"
"github.com/netsampler/goflow2/transport"
"github.com/prometheus/client_golang/prometheus"
)
type StateNFLegacy struct {
stopper
Format format.FormatInterface
Transport transport.TransportInterface
Logger Logger
}
func NewStateNFLegacy() *StateNFLegacy {
return &StateNFLegacy{}
}
func (s *StateNFLegacy) DecodeFlow(msg interface{}) error {
pkt := msg.(BaseMessage)
buf := bytes.NewBuffer(pkt.Payload)
key := pkt.Src.String()
samplerAddress := pkt.Src
if samplerAddress.To4() != nil {
samplerAddress = samplerAddress.To4()
}
ts := uint64(time.Now().UTC().Unix())
if pkt.SetTime {
ts = uint64(pkt.RecvTime.UTC().Unix())
}
timeTrackStart := time.Now()
msgDec, err := netflowlegacy.DecodeMessage(buf)
if err != nil {
switch err.(type) {
case *netflowlegacy.ErrorVersion:
NetFlowErrors.With(
prometheus.Labels{
"router": key,
"error": "error_version",
}).
Inc()
}
return err
}
switch msgDecConv := msgDec.(type) {
case netflowlegacy.PacketNetFlowV5:
NetFlowStats.With(
prometheus.Labels{
"router": key,
"version": "5",
}).
Inc()
NetFlowSetStatsSum.With(
prometheus.Labels{
"router": key,
"version": "5",
"type": "DataFlowSet",
}).
Add(float64(msgDecConv.Count))
}
var flowMessageSet []*flowmessage.FlowMessage
flowMessageSet, err = producer.ProcessMessageNetFlowLegacy(msgDec)
timeTrackStop := time.Now()
DecoderTime.With(
prometheus.Labels{
"name": "NetFlowV5",
}).
Observe(float64((timeTrackStop.Sub(timeTrackStart)).Nanoseconds()) / 1000)
for _, fmsg := range flowMessageSet {
fmsg.TimeReceived = ts
fmsg.SamplerAddress = samplerAddress
if s.Format != nil {
key, data, err := s.Format.Format(fmsg)
if err != nil && s.Logger != nil {
s.Logger.Error(err)
}
if err == nil && s.Transport != nil {
err = s.Transport.Send(key, data)
if err != nil {
s.Logger.Error(err)
}
}
}
}
return nil
}
func (s *StateNFLegacy) FlowRoutine(workers int, addr string, port int, reuseport bool) error {
if err := s.start(); err != nil {
return err
}
return UDPStoppableRoutine(s.stopCh, "NetFlowV5", s.DecodeFlow, workers, addr, port, reuseport, s.Logger)
}

224
utils/pipe.go Normal file
View File

@@ -0,0 +1,224 @@
package utils
import (
"bytes"
"fmt"
"sync"
"github.com/netsampler/goflow2/v2/decoders/netflow"
"github.com/netsampler/goflow2/v2/decoders/netflowlegacy"
"github.com/netsampler/goflow2/v2/decoders/sflow"
"github.com/netsampler/goflow2/v2/decoders/utils"
"github.com/netsampler/goflow2/v2/format"
"github.com/netsampler/goflow2/v2/producer"
"github.com/netsampler/goflow2/v2/transport"
"github.com/netsampler/goflow2/v2/utils/templates"
)
type FlowPipe interface {
DecodeFlow(msg interface{}) error
Close()
}
type flowpipe struct {
format format.FormatInterface
transport transport.TransportInterface
producer producer.ProducerInterface
netFlowTemplater templates.TemplateSystemGenerator
}
type PipeConfig struct {
Format format.FormatInterface
Transport transport.TransportInterface
Producer producer.ProducerInterface
NetFlowTemplater templates.TemplateSystemGenerator
}
func (p *flowpipe) formatSend(flowMessageSet []producer.ProducerMessage) error {
for _, msg := range flowMessageSet {
// todo: pass normal
if p.format != nil {
key, data, err := p.format.Format(msg)
if err != nil {
return err
}
if p.transport != nil {
if err = p.transport.Send(key, data); err != nil {
return err
}
}
// send to pool for reuse
}
}
return nil
}
func (p *flowpipe) parseConfig(cfg *PipeConfig) {
p.format = cfg.Format
p.transport = cfg.Transport
p.producer = cfg.Producer
if cfg.NetFlowTemplater != nil {
p.netFlowTemplater = cfg.NetFlowTemplater
} else {
p.netFlowTemplater = templates.DefaultTemplateGenerator
}
}
type SFlowPipe struct {
flowpipe
}
type NetFlowPipe struct {
flowpipe
templateslock *sync.RWMutex
templates map[string]netflow.NetFlowTemplateSystem
}
type PipeMessageError struct {
Message *Message
Err error
}
func (e *PipeMessageError) Error() string {
return fmt.Sprintf("message from %s %s", e.Message.Src.String(), e.Err.Error())
}
func (e *PipeMessageError) Unwrap() error {
return e.Err
}
func NewSFlowPipe(cfg *PipeConfig) *SFlowPipe {
p := &SFlowPipe{}
p.parseConfig(cfg)
return p
}
func (p *SFlowPipe) Close() {
}
func (p *SFlowPipe) DecodeFlow(msg interface{}) error {
pkt, ok := msg.(*Message)
if !ok {
return fmt.Errorf("flow is not *Message")
}
buf := bytes.NewBuffer(pkt.Payload)
//key := pkt.Src.String()
var packet sflow.Packet
if err := sflow.DecodeMessageVersion(buf, &packet); err != nil {
return &PipeMessageError{pkt, err}
}
args := producer.ProduceArgs{
Src: pkt.Src,
Dst: pkt.Dst,
TimeReceived: pkt.Received,
SamplerAddress: pkt.Src.Addr(),
}
if p.producer == nil {
return nil
}
flowMessageSet, err := p.producer.Produce(&packet, &args)
defer p.producer.Commit(flowMessageSet)
if err != nil {
return &PipeMessageError{pkt, err}
}
return p.formatSend(flowMessageSet)
}
func NewNetFlowPipe(cfg *PipeConfig) *NetFlowPipe {
p := &NetFlowPipe{
templateslock: &sync.RWMutex{},
templates: make(map[string]netflow.NetFlowTemplateSystem),
}
p.parseConfig(cfg)
return p
}
func (p *NetFlowPipe) DecodeFlow(msg interface{}) error {
pkt, ok := msg.(*Message)
if !ok {
return fmt.Errorf("flow is not *Message")
}
buf := bytes.NewBuffer(pkt.Payload)
key := pkt.Src.String()
p.templateslock.RLock()
templates, ok := p.templates[key]
p.templateslock.RUnlock()
if !ok {
templates = p.netFlowTemplater(key)
p.templateslock.Lock()
p.templates[key] = templates
p.templateslock.Unlock()
}
var packetV5 netflowlegacy.PacketNetFlowV5
var packetNFv9 netflow.NFv9Packet
var packetIPFIX netflow.IPFIXPacket
// decode the version
var version uint16
if err := utils.BinaryDecoder(buf, &version); err != nil {
return &PipeMessageError{pkt, err}
}
switch version {
case 5:
packetV5.Version = 5
if err := netflowlegacy.DecodeMessage(buf, &packetV5); err != nil {
return &PipeMessageError{pkt, err}
}
case 9:
packetNFv9.Version = 9
if err := netflow.DecodeMessageNetFlow(buf, templates, &packetNFv9); err != nil {
return &PipeMessageError{pkt, err}
}
case 10:
packetIPFIX.Version = 10
if err := netflow.DecodeMessageIPFIX(buf, templates, &packetIPFIX); err != nil {
return &PipeMessageError{pkt, err}
}
default:
return &PipeMessageError{pkt, fmt.Errorf("Not a NetFlow packet")}
}
var flowMessageSet []producer.ProducerMessage
var err error
args := producer.ProduceArgs{
Src: pkt.Src,
Dst: pkt.Dst,
TimeReceived: pkt.Received,
SamplerAddress: pkt.Src.Addr(),
}
if p.producer == nil {
return nil
}
switch version {
case 5:
flowMessageSet, err = p.producer.Produce(&packetV5, &args)
case 9:
flowMessageSet, err = p.producer.Produce(&packetNFv9, &args)
case 10:
flowMessageSet, err = p.producer.Produce(&packetIPFIX, &args)
}
defer p.producer.Commit(flowMessageSet)
if err != nil {
return &PipeMessageError{pkt, err}
}
return p.formatSend(flowMessageSet)
}
func (p *NetFlowPipe) Close() {
}

View File

@@ -1,170 +0,0 @@
package utils
import (
"bytes"
"net"
"time"
"github.com/netsampler/goflow2/decoders/sflow"
"github.com/netsampler/goflow2/format"
flowmessage "github.com/netsampler/goflow2/pb"
"github.com/netsampler/goflow2/producer"
"github.com/netsampler/goflow2/transport"
"github.com/prometheus/client_golang/prometheus"
)
type StateSFlow struct {
stopper
Format format.FormatInterface
Transport transport.TransportInterface
Logger Logger
Config *producer.ProducerConfig
configMapped *producer.ProducerConfigMapped
}
func NewStateSFlow() *StateSFlow {
return &StateSFlow{}
}
func (s *StateSFlow) DecodeFlow(msg interface{}) error {
pkt := msg.(BaseMessage)
buf := bytes.NewBuffer(pkt.Payload)
key := pkt.Src.String()
ts := uint64(time.Now().UTC().Unix())
if pkt.SetTime {
ts = uint64(pkt.RecvTime.UTC().Unix())
}
timeTrackStart := time.Now()
msgDec, err := sflow.DecodeMessage(buf)
if err != nil {
switch err.(type) {
case *sflow.ErrorVersion:
SFlowErrors.With(
prometheus.Labels{
"router": key,
"error": "error_version",
}).
Inc()
case *sflow.ErrorIPVersion:
SFlowErrors.With(
prometheus.Labels{
"router": key,
"error": "error_ip_version",
}).
Inc()
case *sflow.ErrorDataFormat:
SFlowErrors.With(
prometheus.Labels{
"router": key,
"error": "error_data_format",
}).
Inc()
default:
SFlowErrors.With(
prometheus.Labels{
"router": key,
"error": "error_decoding",
}).
Inc()
}
return err
}
switch msgDecConv := msgDec.(type) {
case sflow.Packet:
agentStr := net.IP(msgDecConv.AgentIP).String()
SFlowStats.With(
prometheus.Labels{
"router": key,
"agent": agentStr,
"version": "5",
}).
Inc()
for _, samples := range msgDecConv.Samples {
typeStr := "unknown"
countRec := 0
switch samplesConv := samples.(type) {
case sflow.FlowSample:
typeStr = "FlowSample"
countRec = len(samplesConv.Records)
case sflow.CounterSample:
typeStr = "CounterSample"
if samplesConv.Header.Format == 4 {
typeStr = "Expanded" + typeStr
}
countRec = len(samplesConv.Records)
case sflow.ExpandedFlowSample:
typeStr = "ExpandedFlowSample"
countRec = len(samplesConv.Records)
}
SFlowSampleStatsSum.With(
prometheus.Labels{
"router": key,
"agent": agentStr,
"version": "5",
"type": typeStr,
}).
Inc()
SFlowSampleRecordsStatsSum.With(
prometheus.Labels{
"router": key,
"agent": agentStr,
"version": "5",
"type": typeStr,
}).
Add(float64(countRec))
}
}
var flowMessageSet []*flowmessage.FlowMessage
flowMessageSet, err = producer.ProcessMessageSFlowConfig(msgDec, s.configMapped)
timeTrackStop := time.Now()
DecoderTime.With(
prometheus.Labels{
"name": "sFlow",
}).
Observe(float64((timeTrackStop.Sub(timeTrackStart)).Nanoseconds()) / 1000)
for _, fmsg := range flowMessageSet {
fmsg.TimeReceived = ts
fmsg.TimeFlowStart = ts
fmsg.TimeFlowEnd = ts
if s.Format != nil {
key, data, err := s.Format.Format(fmsg)
if err != nil && s.Logger != nil {
s.Logger.Error(err)
}
if err == nil && s.Transport != nil {
err = s.Transport.Send(key, data)
if err != nil {
s.Logger.Error(err)
}
}
}
}
return nil
}
func (s *StateSFlow) initConfig() {
s.configMapped = producer.NewProducerConfigMapped(s.Config)
}
func (s *StateSFlow) FlowRoutine(workers int, addr string, port int, reuseport bool) error {
if err := s.start(); err != nil {
return err
}
s.initConfig()
return UDPStoppableRoutine(s.stopCh, "sFlow", s.DecodeFlow, workers, addr, port, reuseport, s.Logger)
}

View File

@@ -1,92 +0,0 @@
package utils
import (
"github.com/stretchr/testify/assert"
"testing"
)
func TestDecodeFlowExpandedSFlow(t *testing.T) {
msg := BaseMessage{
Src: []byte{},
Port: 1,
Payload: getExpandedSFlowDecode(),
}
s := &StateSFlow{}
assert.Nil(t, s.DecodeFlow(msg))
}
func getExpandedSFlowDecode() []byte {
return []byte{
0, 0, 0, 5, 0, 0, 0, 1, 1, 2, 3, 4, 0, 0, 0, 0, 5, 167, 139, 219, 5, 118,
138, 184, 0, 0, 0, 6, 0, 0, 0, 3, 0, 0, 0, 220, 2, 144, 194, 214, 0, 0, 0, 0,
0, 5, 6, 164, 0, 0, 3, 255, 6, 6, 189, 2, 0, 0, 0, 0, 0, 0, 0, 0, 0, 5,
6, 164, 0, 0, 0, 0, 0, 5, 6, 171, 0, 0, 0, 2, 0, 0, 3, 233, 0, 0, 0, 6,
0, 0, 5, 7, 0, 0, 0, 0, 0, 0, 5, 7, 0, 0, 0, 0, 0, 0, 0, 1, 0, 0,
0, 144, 0, 0, 0, 1, 0, 0, 5, 234, 0, 0, 0, 4, 0, 0, 0, 128, 8, 6, 168, 250,
146, 253, 116, 131, 239, 8, 101, 183, 129, 0, 5, 7, 8, 0, 9, 0, 5, 212, 0, 2, 4, 0,
3, 6, 252, 8, 9, 187, 169, 1, 4, 7, 186, 201, 1, 187, 249, 6, 160, 7, 5, 240, 6, 4,
4, 0, 0, 6, 0, 123, 119, 210, 0, 0, 165, 105, 7, 171, 145, 234, 102, 0, 252, 187, 162, 227,
104, 188, 126, 232, 156, 164, 2, 115, 6, 100, 0, 185, 6, 4, 119, 5, 213, 1, 215, 208, 8, 4,
118, 183, 241, 225, 130, 186, 2, 250, 220, 153, 189, 3, 4, 4, 1, 8, 210, 119, 172, 9, 164, 233,
1, 8, 171, 226, 196, 195, 3, 152, 9, 5, 6, 181, 4, 7, 0, 0, 0, 3, 0, 0, 0, 220,
9, 107, 215, 156, 0, 0, 0, 0, 0, 5, 6, 165, 0, 0, 3, 255, 226, 123, 0, 100, 0, 0,
0, 0, 0, 0, 0, 0, 0, 5, 6, 165, 0, 0, 0, 0, 0, 5, 6, 164, 0, 0, 0, 2,
0, 0, 3, 233, 0, 0, 0, 6, 0, 0, 3, 184, 0, 0, 0, 0, 0, 0, 3, 184, 0, 0,
0, 0, 0, 0, 0, 1, 0, 0, 0, 144, 0, 0, 0, 1, 0, 0, 5, 190, 0, 0, 0, 4,
0, 0, 0, 128, 116, 131, 239, 8, 101, 183, 144, 226, 186, 134, 8, 1, 129, 0, 3, 184, 8, 0,
9, 0, 5, 168, 7, 127, 4, 0, 4, 6, 163, 211, 185, 9, 220, 7, 0, 254, 3, 8, 0, 9,
130, 136, 179, 1, 2, 2, 7, 5, 250, 4, 128, 6, 0, 1, 7, 1, 0, 0, 1, 1, 8, 0,
6, 9, 250, 9, 4, 113, 121, 4, 160, 125, 0, 4, 9, 209, 241, 194, 190, 148, 161, 186, 6, 192,
246, 190, 170, 2, 238, 190, 128, 221, 223, 1, 218, 225, 3, 9, 7, 226, 220, 231, 127, 3, 3, 252,
7, 9, 161, 247, 218, 8, 8, 174, 133, 4, 213, 245, 149, 218, 5, 4, 200, 128, 139, 5, 0, 115,
0, 0, 0, 3, 0, 0, 0, 220, 2, 144, 194, 215, 0, 0, 0, 0, 0, 5, 6, 164, 0, 0,
3, 255, 6, 6, 253, 1, 0, 0, 0, 0, 0, 0, 0, 0, 0, 5, 6, 164, 0, 0, 0, 0,
0, 5, 6, 171, 0, 0, 0, 2, 0, 0, 3, 233, 0, 0, 0, 6, 0, 0, 0, 104, 0, 0,
0, 0, 0, 0, 0, 104, 0, 0, 0, 0, 0, 0, 0, 1, 0, 0, 0, 144, 0, 0, 0, 1,
0, 0, 5, 242, 0, 0, 0, 4, 0, 0, 0, 128, 116, 131, 239, 7, 9, 1, 116, 131, 239, 8,
101, 183, 129, 0, 0, 104, 8, 0, 9, 0, 5, 220, 152, 143, 4, 0, 1, 6, 5, 179, 9, 187,
191, 101, 190, 2, 144, 182, 0, 0, 130, 4, 252, 4, 160, 192, 138, 8, 219, 124, 128, 6, 0, 235,
180, 213, 0, 0, 1, 1, 8, 0, 9, 124, 6, 1, 9, 1, 252, 3, 194, 8, 195, 209, 115, 1,
5, 152, 204, 2, 6, 4, 1, 119, 254, 9, 1, 170, 0, 192, 2, 7, 190, 9, 149, 5, 101, 2,
128, 122, 0, 190, 1, 109, 188, 175, 4, 8, 152, 1, 142, 108, 2, 100, 2, 124, 125, 195, 5, 8,
233, 126, 7, 4, 243, 4, 3, 153, 0, 0, 0, 3, 0, 0, 0, 220, 5, 1, 150, 6, 0, 0,
0, 0, 0, 5, 6, 167, 0, 0, 3, 255, 6, 5, 105, 220, 0, 0, 0, 0, 0, 0, 0, 0,
0, 5, 6, 167, 0, 0, 0, 0, 0, 5, 6, 164, 0, 0, 0, 2, 0, 0, 3, 233, 0, 0,
0, 6, 0, 0, 5, 7, 0, 0, 0, 0, 0, 0, 5, 7, 0, 0, 0, 0, 0, 0, 0, 1,
0, 0, 0, 144, 0, 0, 0, 1, 0, 0, 2, 2, 0, 0, 0, 4, 0, 0, 0, 128, 116, 131,
239, 8, 101, 183, 152, 3, 130, 1, 196, 153, 129, 0, 5, 7, 8, 0, 9, 0, 2, 0, 0, 0,
4, 0, 126, 7, 119, 188, 185, 9, 221, 8, 2, 116, 144, 0, 9, 139, 3, 112, 2, 0, 8, 124,
255, 251, 0, 0, 131, 2, 0, 0, 0, 246, 3, 3, 107, 5, 0, 0, 0, 0, 9, 173, 2, 217,
6, 248, 0, 0, 9, 173, 2, 217, 8, 248, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255,
255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 6, 9, 153,
215, 157, 0, 255, 0, 8, 1, 0, 9, 8, 9, 6, 164, 103, 9, 5, 0, 0, 0, 3, 0, 0,
0, 152, 5, 201, 2, 175, 0, 0, 0, 0, 0, 5, 6, 5, 0, 0, 3, 255, 1, 8, 9, 1,
0, 0, 0, 0, 0, 0, 0, 0, 0, 5, 6, 5, 0, 0, 0, 0, 0, 5, 6, 164, 0, 0,
0, 2, 0, 0, 3, 233, 0, 0, 0, 6, 0, 0, 0, 3, 0, 0, 0, 0, 0, 0, 0, 3,
0, 0, 0, 0, 0, 0, 0, 1, 0, 0, 0, 6, 0, 0, 0, 1, 0, 0, 0, 4, 0, 0,
0, 4, 0, 0, 0, 0, 116, 131, 239, 8, 101, 183, 218, 177, 4, 251, 217, 207, 8, 0, 9, 0,
0, 8, 0, 0, 0, 0, 9, 7, 8, 161, 106, 3, 109, 6, 185, 9, 220, 215, 0, 123, 9, 184,
0, 8, 116, 122, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 8, 3, 130, 6,
0, 0, 0, 3, 0, 0, 0, 220, 2, 144, 194, 216, 0, 0, 0, 0, 0, 5, 6, 164, 0, 0,
3, 255, 6, 7, 1, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 5, 6, 164, 0, 0, 0, 0,
0, 5, 6, 165, 0, 0, 0, 2, 0, 0, 3, 233, 0, 0, 0, 6, 0, 0, 3, 202, 0, 0,
0, 0, 0, 0, 3, 202, 0, 0, 0, 0, 0, 0, 0, 1, 0, 0, 0, 144, 0, 0, 0, 1,
0, 0, 5, 242, 0, 0, 0, 4, 0, 0, 0, 128, 144, 226, 186, 135, 4, 241, 116, 131, 239, 8,
101, 183, 129, 0, 3, 202, 8, 0, 9, 0, 5, 220, 147, 0, 4, 0, 7, 6, 225, 131, 1, 159,
7, 185, 195, 181, 170, 8, 9, 117, 7, 175, 8, 3, 191, 135, 190, 150, 196, 102, 0, 6, 0, 119,
116, 113, 0, 0, 201, 244, 240, 206, 2, 117, 4, 139, 8, 4, 240, 223, 247, 123, 6, 0, 239, 0,
9, 116, 152, 153, 191, 0, 124, 2, 7, 8, 3, 178, 166, 150, 3, 218, 163, 175, 121, 8, 4, 210,
4, 5, 166, 5, 178, 1, 6, 222, 172, 186, 6, 241, 232, 8, 188, 192, 2, 220, 128, 1, 8, 7,
194, 130, 220, 5, 2, 0, 158, 195, 0, 4, 3, 2, 160, 158, 157, 2, 102, 3, 7, 3, 0, 0,
1, 3, 3, 4, 1, 1, 4, 2, 187, 255, 188, 3, 4, 138, 9, 180, 104, 233, 212, 239, 123, 237,
112, 8, 133, 129, 152, 138, 7, 195, 8, 171, 237, 3, 4, 223, 116, 214, 151, 9, 151, 102, 0, 0,
0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0,
0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0,
0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0,
0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0,
0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0,
0, 0, 0, 0,
}
}

View File

@@ -1,33 +0,0 @@
package utils
import (
"errors"
)
// ErrAlreadyStarted error happens when you try to start twice a flow routine
var ErrAlreadyStarted = errors.New("the routine is already started")
// stopper mechanism, common for all the flow routines
type stopper struct {
stopCh chan struct{}
}
func (s *stopper) start() error {
if s.stopCh != nil {
return ErrAlreadyStarted
}
s.stopCh = make(chan struct{})
return nil
}
func (s *stopper) Shutdown() {
if s.stopCh != nil {
select {
case <-s.stopCh:
default:
close(s.stopCh)
}
s.stopCh = nil
}
}

View File

@@ -1,51 +0,0 @@
package utils
import (
"testing"
"time"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
)
func TestStopper(t *testing.T) {
r := routine{}
require.False(t, r.Running)
require.NoError(t, r.StartRoutine())
assert.True(t, r.Running)
r.Shutdown()
assert.Eventually(t, func() bool {
return r.Running == false
}, time.Second, time.Millisecond)
// after shutdown, we can start it again
require.NoError(t, r.StartRoutine())
assert.True(t, r.Running)
}
func TestStopper_CannotStartTwice(t *testing.T) {
r := routine{}
require.False(t, r.Running)
require.NoError(t, r.StartRoutine())
assert.ErrorIs(t, r.StartRoutine(), ErrAlreadyStarted)
}
type routine struct {
stopper
Running bool
}
func (p *routine) StartRoutine() error {
if err := p.start(); err != nil {
return err
}
p.Running = true
waitForGoRoutine := make(chan struct{})
go func() {
close(waitForGoRoutine)
<-p.stopCh
p.Running = false
}()
<-waitForGoRoutine
return nil
}

View File

@@ -0,0 +1,14 @@
package templates
import (
"github.com/netsampler/goflow2/v2/decoders/netflow"
)
// Function that Create Template Systems.
// This is meant to be used by a pipe
type TemplateSystemGenerator func(key string) netflow.NetFlowTemplateSystem
// Default template generator
func DefaultTemplateGenerator(key string) netflow.NetFlowTemplateSystem {
return netflow.CreateTemplateSystem()
}

278
utils/udp.go Normal file
View File

@@ -0,0 +1,278 @@
package utils
import (
"fmt"
"net"
"net/netip"
"sync"
"time"
reuseport "github.com/libp2p/go-reuseport"
)
// Callback used to decode a UDP message
type DecoderFunc func(msg interface{}) error
type udpPacket struct {
src *net.UDPAddr
dst *net.UDPAddr
size int
payload []byte
received time.Time
}
type Message struct {
Src netip.AddrPort
Dst netip.AddrPort
Payload []byte
Received time.Time
}
var packetPool = sync.Pool{
New: func() any {
return &udpPacket{
payload: make([]byte, 9000),
}
},
}
type UDPReceiver struct {
ready chan bool
q chan bool
wg *sync.WaitGroup
dispatch chan *udpPacket
errCh chan error // linked to receiver, never closed
decodersCnt int
blocking bool
workers int
sockets int
}
type UDPReceiverConfig struct {
Workers int
Sockets int
Blocking bool
QueueSize int
}
func NewUDPReceiver(cfg *UDPReceiverConfig) (*UDPReceiver, error) {
r := &UDPReceiver{
wg: &sync.WaitGroup{},
sockets: 2,
workers: 2,
ready: make(chan bool),
errCh: make(chan error),
}
dispatchSize := 1000000
if cfg != nil {
if cfg.Sockets <= 0 {
cfg.Sockets = 1
}
if cfg.Workers <= 0 {
cfg.Workers = cfg.Sockets
}
r.sockets = cfg.Sockets
r.workers = cfg.Workers
dispatchSize = cfg.QueueSize
r.blocking = cfg.Blocking
}
if dispatchSize == 0 {
r.dispatch = make(chan *udpPacket) // synchronous mode
} else {
r.dispatch = make(chan *udpPacket, dispatchSize)
}
err := r.init()
return r, err
}
// Initialize channels that are related to a session
// Once the user calls Stop, they can restart the capture
func (r *UDPReceiver) init() error {
r.q = make(chan bool)
r.decodersCnt = 0
select {
case <-r.ready:
return fmt.Errorf("receiver is already stopped")
default:
close(r.ready)
}
return nil
}
func (r *UDPReceiver) logError(err error) {
select {
case r.errCh <- err:
default:
}
}
func (r *UDPReceiver) Errors() <-chan error {
return r.errCh
}
func (r *UDPReceiver) receive(addr string, port int, started chan bool) error {
pconn, err := reuseport.ListenPacket("udp", fmt.Sprintf("%s:%d", addr, port))
close(started)
if err != nil {
return err
}
q := make(chan bool)
// function to quit
go func() {
select {
case <-q: // if routine has exited before
case <-r.q: // upon general close
}
pconn.Close()
}()
defer close(q)
udpconn, ok := pconn.(*net.UDPConn)
if !ok {
return err
}
localAddr, _ := udpconn.LocalAddr().(*net.UDPAddr)
for {
pkt := packetPool.Get().(*udpPacket)
pkt.size, pkt.src, err = udpconn.ReadFromUDP(pkt.payload)
if err != nil {
packetPool.Put(pkt)
return err
}
pkt.dst = localAddr
pkt.received = time.Now().UTC()
if pkt.size == 0 {
// error
continue
}
if r.blocking {
// does not drop
// if combined with synchronous mode
select {
case r.dispatch <- pkt:
case <-r.q:
return nil
}
} else {
select {
case r.dispatch <- pkt:
case <-r.q:
return nil
default:
packetPool.Put(pkt)
// increase counter
}
}
}
}
type ReceiverError struct {
Err error
}
func (e *ReceiverError) Error() string {
return "receiver: " + e.Err.Error()
}
func (e *ReceiverError) Unwrap() error {
return e.Err
}
// Start the processing routines
func (r *UDPReceiver) decoders(workers int, decodeFunc DecoderFunc) error {
for i := 0; i < workers; i++ {
r.wg.Add(1)
r.decodersCnt += 1
go func() {
defer r.wg.Done()
for pkt := range r.dispatch {
if pkt == nil {
return
}
if decodeFunc != nil {
msg := Message{
Src: pkt.src.AddrPort(),
Dst: pkt.dst.AddrPort(),
Payload: pkt.payload[0:pkt.size],
Received: pkt.received,
}
if err := decodeFunc(&msg); err != nil {
r.logError(&ReceiverError{err})
}
}
packetPool.Put(pkt)
}
}()
}
return nil
}
// Starts the UDP receiving workers
func (r *UDPReceiver) receivers(sockets int, addr string, port int) error {
for i := 0; i < sockets; i++ {
r.wg.Add(1)
started := make(chan bool)
go func() {
defer r.wg.Done()
if err := r.receive(addr, port, started); err != nil {
r.logError(&ReceiverError{err})
}
}()
<-started
}
return nil
}
// Start UDP receivers and the processing routines
func (r *UDPReceiver) Start(addr string, port int, decodeFunc DecoderFunc) error {
select {
case <-r.ready:
r.ready = make(chan bool)
default:
return fmt.Errorf("receiver is already started")
}
if err := r.decoders(r.workers, decodeFunc); err != nil {
return err
}
if err := r.receivers(r.workers, addr, port); err != nil {
return err
}
return nil
}
// Stops the routines
func (r *UDPReceiver) Stop() error {
select {
case <-r.q:
default:
close(r.q)
}
for i := 0; i < r.decodersCnt; i++ {
r.dispatch <- nil
}
r.wg.Wait()
return r.init() // recreates the closed channels
}

63
utils/udp_test.go Normal file
View File

@@ -0,0 +1,63 @@
package utils
import (
"fmt"
"net"
"testing"
//"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
)
func TestUDPReceiver(t *testing.T) {
addr := "[::1]"
port, err := getFreeUDPPort()
require.NoError(t, err)
t.Logf("starting UDP receiver on %s:%d\n", addr, port)
r, err := NewUDPReceiver(nil)
require.NoError(t, err)
require.NoError(t, r.Start(addr, port, nil))
sendMessage := func(msg string) error {
conn, err := net.Dial("udp", fmt.Sprintf("%s:%d", addr, port))
if err != nil {
return err
}
defer conn.Close()
_, err = conn.Write([]byte(msg))
return err
}
require.NoError(t, sendMessage("message"))
t.Log("sending message\n")
require.NoError(t, r.Stop())
}
func TestUDPClose(t *testing.T) {
addr := "[::1]"
port, err := getFreeUDPPort()
require.NoError(t, err)
t.Logf("starting UDP receiver on %s:%d\n", addr, port)
r, err := NewUDPReceiver(nil)
require.NoError(t, err)
require.NoError(t, r.Start(addr, port, nil))
require.NoError(t, r.Stop())
require.NoError(t, r.Start(addr, port, nil))
require.Error(t, r.Start(addr, port, nil))
require.NoError(t, r.Stop())
require.Error(t, r.Stop())
}
func getFreeUDPPort() (int, error) {
a, err := net.ResolveUDPAddr("udp", "127.0.0.1:0")
if err != nil {
return 0, err
}
l, err := net.ListenUDP("udp", a)
if err != nil {
return 0, err
}
defer l.Close()
return l.LocalAddr().(*net.UDPAddr).Port, nil
}

View File

@@ -1,234 +0,0 @@
package utils
import (
"errors"
"fmt"
"io"
"net"
"strconv"
"sync"
"time"
reuseport "github.com/libp2p/go-reuseport"
decoder "github.com/netsampler/goflow2/decoders"
"github.com/netsampler/goflow2/decoders/netflow"
flowmessage "github.com/netsampler/goflow2/pb"
"github.com/netsampler/goflow2/producer"
"github.com/prometheus/client_golang/prometheus"
"gopkg.in/yaml.v2"
)
type ProducerConfig *producer.ProducerConfig
func LoadMapping(f io.Reader) (ProducerConfig, error) {
config := &producer.ProducerConfig{}
dec := yaml.NewDecoder(f)
err := dec.Decode(config)
return config, err
}
func GetServiceAddresses(srv string) (addrs []string, err error) {
_, srvs, err := net.LookupSRV("", "", srv)
if err != nil {
return nil, errors.New(fmt.Sprintf("Service discovery: %v\n", err))
}
for _, srv := range srvs {
addrs = append(addrs, net.JoinHostPort(srv.Target, strconv.Itoa(int(srv.Port))))
}
return addrs, nil
}
type Logger interface {
Printf(string, ...interface{})
Errorf(string, ...interface{})
Warnf(string, ...interface{})
Warn(...interface{})
Error(...interface{})
Debug(...interface{})
Debugf(string, ...interface{})
Infof(string, ...interface{})
Fatalf(string, ...interface{})
}
type BaseMessage struct {
Src net.IP
Port int
Payload []byte
SetTime bool
RecvTime time.Time
}
type Transport interface {
Send([]*flowmessage.FlowMessage)
}
type Formatter interface {
Format([]*flowmessage.FlowMessage)
}
/*
type DefaultLogTransport struct {
}
func (s *DefaultLogTransport) Publish(msgs []*flowmessage.FlowMessage) {
for _, msg := range msgs {
fmt.Printf("%v\n", FlowMessageToString(msg))
}
}
type DefaultJSONTransport struct {
}
func (s *DefaultJSONTransport) Publish(msgs []*flowmessage.FlowMessage) {
for _, msg := range msgs {
fmt.Printf("%v\n", FlowMessageToJSON(msg))
}
}
*/
type DefaultErrorCallback struct {
Logger Logger
}
func (cb *DefaultErrorCallback) Callback(name string, id int, start, end time.Time, err error) {
if _, ok := err.(*netflow.ErrorTemplateNotFound); ok {
return
}
if cb.Logger != nil {
cb.Logger.Errorf("Error from: %v (%v) duration: %v. %v", name, id, end.Sub(start), err)
}
}
func UDPRoutine(name string, decodeFunc decoder.DecoderFunc, workers int, addr string, port int, sockReuse bool, logger Logger) error {
return UDPStoppableRoutine(make(chan struct{}), name, decodeFunc, workers, addr, port, sockReuse, logger)
}
// UDPStoppableRoutine runs a UDPRoutine that can be stopped by closing the stopCh passed as argument
func UDPStoppableRoutine(stopCh <-chan struct{}, name string, decodeFunc decoder.DecoderFunc, workers int, addr string, port int, sockReuse bool, logger Logger) error {
ecb := DefaultErrorCallback{
Logger: logger,
}
decoderParams := decoder.DecoderParams{
DecoderFunc: decodeFunc,
DoneCallback: DefaultAccountCallback,
ErrorCallback: ecb.Callback,
}
processor := decoder.CreateProcessor(workers, decoderParams, name)
processor.Start()
addrUDP := net.UDPAddr{
IP: net.ParseIP(addr),
Port: port,
}
var udpconn *net.UDPConn
var err error
if sockReuse {
pconn, err := reuseport.ListenPacket("udp", addrUDP.String())
if err != nil {
return err
}
defer pconn.Close()
var ok bool
udpconn, ok = pconn.(*net.UDPConn)
if !ok {
return err
}
} else {
udpconn, err = net.ListenUDP("udp", &addrUDP)
if err != nil {
return err
}
defer udpconn.Close()
}
payload := make([]byte, 9000)
localIP := addrUDP.IP.String()
if addrUDP.IP == nil {
localIP = ""
}
type udpData struct {
size int
pktAddr *net.UDPAddr
payload []byte
}
udpDataCh := make(chan udpData)
defer close(udpDataCh)
wg := &sync.WaitGroup{}
wg.Add(1)
go func() {
defer wg.Done()
for {
u := udpData{}
u.size, u.pktAddr, _ = udpconn.ReadFromUDP(payload)
if u.size == 0 { // Ignore 0 byte packets.
continue
}
u.payload = make([]byte, u.size)
copy(u.payload, payload[0:u.size])
select {
case <-stopCh:
return
default:
udpDataCh <- u
}
}
}()
func() {
for {
select {
case u := <-udpDataCh:
process(u.size, u.payload, u.pktAddr, processor, localIP, addrUDP, name)
case <-stopCh:
return
}
}
}()
for _ = range udpDataCh {
// drain
}
wg.Wait()
return nil
}
func process(size int, payload []byte, pktAddr *net.UDPAddr, processor decoder.Processor, localIP string, addrUDP net.UDPAddr, name string) {
baseMessage := BaseMessage{
Src: pktAddr.IP,
Port: pktAddr.Port,
Payload: payload,
}
processor.ProcessMessage(baseMessage)
MetricTrafficBytes.With(
prometheus.Labels{
"remote_ip": pktAddr.IP.String(),
"local_ip": localIP,
"local_port": strconv.Itoa(addrUDP.Port),
"type": name,
}).
Add(float64(size))
MetricTrafficPackets.With(
prometheus.Labels{
"remote_ip": pktAddr.IP.String(),
"local_ip": localIP,
"local_port": strconv.Itoa(addrUDP.Port),
"type": name,
}).
Inc()
MetricPacketSizeSum.With(
prometheus.Labels{
"remote_ip": pktAddr.IP.String(),
"local_ip": localIP,
"local_port": strconv.Itoa(addrUDP.Port),
"type": name,
}).
Observe(float64(size))
}

View File

@@ -1,93 +0,0 @@
package utils
import (
"fmt"
"net"
"testing"
"time"
"github.com/sirupsen/logrus"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
)
func TestCancelUDPRoutine(t *testing.T) {
testTimeout := time.After(10 * time.Second)
port, err := getFreeUDPPort()
require.NoError(t, err)
dp := dummyFlowProcessor{}
go func() {
require.NoError(t, dp.FlowRoutine("127.0.0.1", port))
}()
// wait slightly so we give time to the server to accept requests
time.Sleep(100 * time.Millisecond)
sendMessage := func(msg string) error {
conn, err := net.Dial("udp", fmt.Sprintf("127.0.0.1:%d", port))
if err != nil {
return err
}
defer conn.Close()
_, err = conn.Write([]byte(msg))
return err
}
require.NoError(t, sendMessage("message 1"))
require.NoError(t, sendMessage("message 2"))
require.NoError(t, sendMessage("message 3"))
readMessage := func() string {
select {
case msg := <-dp.receivedMessages:
return string(msg.(BaseMessage).Payload)
case <-testTimeout:
require.Fail(t, "test timed out while waiting for message")
return ""
}
}
// in UDP, messages might arrive out of order or duplicate, so whe just verify they arrive
// to avoid flaky tests
require.Contains(t, []string{"message 1", "message 2", "message 3"}, readMessage())
require.Contains(t, []string{"message 1", "message 2", "message 3"}, readMessage())
require.Contains(t, []string{"message 1", "message 2", "message 3"}, readMessage())
dp.Shutdown()
time.Sleep(100 * time.Millisecond)
_ = sendMessage("no more messages should be processed")
select {
case msg := <-dp.receivedMessages:
assert.Fail(t, fmt.Sprint(msg))
default:
// everything is correct
}
}
type dummyFlowProcessor struct {
stopper
receivedMessages chan interface{}
}
func (d *dummyFlowProcessor) FlowRoutine(host string, port int) error {
_ = d.start()
d.receivedMessages = make(chan interface{})
return UDPStoppableRoutine(d.stopCh, "test_udp", func(msg interface{}) error {
d.receivedMessages <- msg
return nil
}, 3, host, port, false, logrus.StandardLogger())
}
func getFreeUDPPort() (int, error) {
a, err := net.ResolveUDPAddr("udp", "127.0.0.1:0")
if err != nil {
return 0, err
}
l, err := net.ListenUDP("udp", a)
if err != nil {
return 0, err
}
defer l.Close()
return l.LocalAddr().(*net.UDPAddr).Port, nil
}