Add flow ACL IDs (#3421)

This commit is contained in:
Viktor Liu 2025-03-04 16:43:07 +01:00 committed by GitHub
parent 619c549547
commit 8c81a823fa
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
26 changed files with 333 additions and 273 deletions

View File

@ -75,6 +75,7 @@ func (m *aclManager) init(stateManager *statemanager.Manager) error {
}
func (m *aclManager) AddPeerFiltering(
id []byte,
ip net.IP,
protocol firewall.Protocol,
sPort *firewall.Port,

View File

@ -96,21 +96,22 @@ func (m *Manager) Init(stateManager *statemanager.Manager) error {
//
// Comment will be ignored because some system this feature is not supported
func (m *Manager) AddPeerFiltering(
id []byte,
ip net.IP,
protocol firewall.Protocol,
proto firewall.Protocol,
sPort *firewall.Port,
dPort *firewall.Port,
action firewall.Action,
ipsetName string,
_ string,
) ([]firewall.Rule, error) {
m.mutex.Lock()
defer m.mutex.Unlock()
return m.aclMgr.AddPeerFiltering(ip, protocol, sPort, dPort, action, ipsetName)
return m.aclMgr.AddPeerFiltering(id, ip, proto, sPort, dPort, action, ipsetName)
}
func (m *Manager) AddRouteFiltering(
id []byte,
sources []netip.Prefix,
destination netip.Prefix,
proto firewall.Protocol,
@ -125,7 +126,7 @@ func (m *Manager) AddRouteFiltering(
return nil, fmt.Errorf("unsupported IP version: %s", destination.Addr().String())
}
return m.router.AddRouteFiltering(sources, destination, proto, sPort, dPort, action)
return m.router.AddRouteFiltering(id, sources, destination, proto, sPort, dPort, action)
}
// DeletePeerRule from the firewall by rule definition
@ -196,13 +197,13 @@ func (m *Manager) AllowNetbird() error {
}
_, err := m.AddPeerFiltering(
nil,
net.IP{0, 0, 0, 0},
"all",
nil,
nil,
firewall.ActionAccept,
"",
"",
)
if err != nil {
return fmt.Errorf("allow netbird interface traffic: %w", err)

View File

@ -75,7 +75,7 @@ func TestIptablesManager(t *testing.T) {
IsRange: true,
Values: []uint16{8043, 8046},
}
rule2, err = manager.AddPeerFiltering(ip, "tcp", port, nil, fw.ActionAccept, "", "accept HTTPS traffic from ports range")
rule2, err = manager.AddPeerFiltering(nil, ip, "tcp", port, nil, fw.ActionAccept, "")
require.NoError(t, err, "failed to add rule")
for _, r := range rule2 {
@ -97,7 +97,7 @@ func TestIptablesManager(t *testing.T) {
// add second rule
ip := net.ParseIP("10.20.0.3")
port := &fw.Port{Values: []uint16{5353}}
_, err = manager.AddPeerFiltering(ip, "udp", nil, port, fw.ActionAccept, "", "accept Fake DNS traffic")
_, err = manager.AddPeerFiltering(nil, ip, "udp", nil, port, fw.ActionAccept, "")
require.NoError(t, err, "failed to add rule")
err = manager.Reset(nil)
@ -148,7 +148,7 @@ func TestIptablesManagerIPSet(t *testing.T) {
port := &fw.Port{
Values: []uint16{443},
}
rule2, err = manager.AddPeerFiltering(ip, "tcp", port, nil, fw.ActionAccept, "default", "accept HTTPS traffic from ports range")
rule2, err = manager.AddPeerFiltering(nil, ip, "tcp", port, nil, fw.ActionAccept, "default")
for _, r := range rule2 {
require.NoError(t, err, "failed to add rule")
require.Equal(t, r.(*Rule).ipsetName, "default-sport", "ipset name must be set")
@ -216,7 +216,7 @@ func TestIptablesCreatePerformance(t *testing.T) {
start := time.Now()
for i := 0; i < testMax; i++ {
port := &fw.Port{Values: []uint16{uint16(1000 + i)}}
_, err = manager.AddPeerFiltering(ip, "tcp", nil, port, fw.ActionAccept, "", "accept HTTP traffic")
_, err = manager.AddPeerFiltering(nil, ip, "tcp", nil, port, fw.ActionAccept, "")
require.NoError(t, err, "failed to add rule")
}

View File

@ -15,7 +15,7 @@ import (
nberrors "github.com/netbirdio/netbird/client/errors"
firewall "github.com/netbirdio/netbird/client/firewall/manager"
"github.com/netbirdio/netbird/client/internal/acl/id"
nbid "github.com/netbirdio/netbird/client/internal/acl/id"
"github.com/netbirdio/netbird/client/internal/routemanager/ipfwdstate"
"github.com/netbirdio/netbird/client/internal/routemanager/refcounter"
"github.com/netbirdio/netbird/client/internal/statemanager"
@ -121,6 +121,7 @@ func (r *router) init(stateManager *statemanager.Manager) error {
}
func (r *router) AddRouteFiltering(
id []byte,
sources []netip.Prefix,
destination netip.Prefix,
proto firewall.Protocol,
@ -128,7 +129,7 @@ func (r *router) AddRouteFiltering(
dPort *firewall.Port,
action firewall.Action,
) (firewall.Rule, error) {
ruleKey := id.GenerateRouteRuleKey(sources, destination, proto, sPort, dPort, action)
ruleKey := nbid.GenerateRouteRuleKey(sources, destination, proto, sPort, dPort, action)
if _, ok := r.rules[string(ruleKey)]; ok {
return ruleKey, nil
}

View File

@ -330,7 +330,7 @@ func TestRouter_AddRouteFiltering(t *testing.T) {
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
ruleKey, err := r.AddRouteFiltering(tt.sources, tt.destination, tt.proto, tt.sPort, tt.dPort, tt.action)
ruleKey, err := r.AddRouteFiltering(nil, tt.sources, tt.destination, tt.proto, tt.sPort, tt.dPort, tt.action)
require.NoError(t, err, "AddRouteFiltering failed")
// Check if the rule is in the internal map

View File

@ -65,13 +65,13 @@ type Manager interface {
// If comment argument is empty firewall manager should set
// rule ID as comment for the rule
AddPeerFiltering(
id []byte,
ip net.IP,
proto Protocol,
sPort *Port,
dPort *Port,
action Action,
ipsetName string,
comment string,
) ([]Rule, error)
// DeletePeerRule from the firewall by rule definition
@ -80,7 +80,15 @@ type Manager interface {
// IsServerRouteSupported returns true if the firewall supports server side routing operations
IsServerRouteSupported() bool
AddRouteFiltering(source []netip.Prefix, destination netip.Prefix, proto Protocol, sPort *Port, dPort *Port, action Action) (Rule, error)
AddRouteFiltering(
id []byte,
sources []netip.Prefix,
destination netip.Prefix,
proto Protocol,
sPort *Port,
dPort *Port,
action Action,
) (Rule, error)
// DeleteRouteRule deletes a routing rule
DeleteRouteRule(rule Rule) error

View File

@ -84,13 +84,13 @@ func (m *AclManager) init(workTable *nftables.Table) error {
// If comment argument is empty firewall manager should set
// rule ID as comment for the rule
func (m *AclManager) AddPeerFiltering(
id []byte,
ip net.IP,
proto firewall.Protocol,
sPort *firewall.Port,
dPort *firewall.Port,
action firewall.Action,
ipsetName string,
comment string,
) ([]firewall.Rule, error) {
var ipset *nftables.Set
if ipsetName != "" {
@ -102,7 +102,7 @@ func (m *AclManager) AddPeerFiltering(
}
newRules := make([]firewall.Rule, 0, 2)
ioRule, err := m.addIOFiltering(ip, proto, sPort, dPort, action, ipset, comment)
ioRule, err := m.addIOFiltering(ip, proto, sPort, dPort, action, ipset)
if err != nil {
return nil, err
}
@ -256,7 +256,6 @@ func (m *AclManager) addIOFiltering(
dPort *firewall.Port,
action firewall.Action,
ipset *nftables.Set,
comment string,
) (*Rule, error) {
ruleId := generatePeerRuleId(ip, sPort, dPort, action, ipset)
if r, ok := m.rules[ruleId]; ok {
@ -338,7 +337,7 @@ func (m *AclManager) addIOFiltering(
mainExpressions = append(mainExpressions, &expr.Verdict{Kind: expr.VerdictDrop})
}
userData := []byte(strings.Join([]string{ruleId, comment}, " "))
userData := []byte(ruleId)
chain := m.chainInputRules
nftRule := m.rConn.AddRule(&nftables.Rule{

View File

@ -113,13 +113,13 @@ func (m *Manager) Init(stateManager *statemanager.Manager) error {
// If comment argument is empty firewall manager should set
// rule ID as comment for the rule
func (m *Manager) AddPeerFiltering(
id []byte,
ip net.IP,
proto firewall.Protocol,
sPort *firewall.Port,
dPort *firewall.Port,
action firewall.Action,
ipsetName string,
comment string,
) ([]firewall.Rule, error) {
m.mutex.Lock()
defer m.mutex.Unlock()
@ -129,10 +129,11 @@ func (m *Manager) AddPeerFiltering(
return nil, fmt.Errorf("unsupported IP version: %s", ip.String())
}
return m.aclManager.AddPeerFiltering(ip, proto, sPort, dPort, action, ipsetName, comment)
return m.aclManager.AddPeerFiltering(id, ip, proto, sPort, dPort, action, ipsetName)
}
func (m *Manager) AddRouteFiltering(
id []byte,
sources []netip.Prefix,
destination netip.Prefix,
proto firewall.Protocol,
@ -147,7 +148,7 @@ func (m *Manager) AddRouteFiltering(
return nil, fmt.Errorf("unsupported IP version: %s", destination.Addr().String())
}
return m.router.AddRouteFiltering(sources, destination, proto, sPort, dPort, action)
return m.router.AddRouteFiltering(id, sources, destination, proto, sPort, dPort, action)
}
// DeletePeerRule from the firewall by rule definition

View File

@ -74,7 +74,7 @@ func TestNftablesManager(t *testing.T) {
testClient := &nftables.Conn{}
rule, err := manager.AddPeerFiltering(ip, fw.ProtocolTCP, nil, &fw.Port{Values: []uint16{53}}, fw.ActionDrop, "", "")
rule, err := manager.AddPeerFiltering(nil, ip, fw.ProtocolTCP, nil, &fw.Port{Values: []uint16{53}}, fw.ActionDrop, "")
require.NoError(t, err, "failed to add rule")
err = manager.Flush()
@ -201,7 +201,7 @@ func TestNFtablesCreatePerformance(t *testing.T) {
start := time.Now()
for i := 0; i < testMax; i++ {
port := &fw.Port{Values: []uint16{uint16(1000 + i)}}
_, err = manager.AddPeerFiltering(ip, "tcp", nil, port, fw.ActionAccept, "", "accept HTTP traffic")
_, err = manager.AddPeerFiltering(nil, ip, "tcp", nil, port, fw.ActionAccept, "")
require.NoError(t, err, "failed to add rule")
if i%100 == 0 {
@ -283,10 +283,11 @@ func TestNftablesManagerCompatibilityWithIptables(t *testing.T) {
})
ip := net.ParseIP("100.96.0.1")
_, err = manager.AddPeerFiltering(ip, fw.ProtocolTCP, nil, &fw.Port{Values: []uint16{80}}, fw.ActionAccept, "", "test rule")
_, err = manager.AddPeerFiltering(nil, ip, fw.ProtocolTCP, nil, &fw.Port{Values: []uint16{80}}, fw.ActionAccept, "")
require.NoError(t, err, "failed to add peer filtering rule")
_, err = manager.AddRouteFiltering(
nil,
[]netip.Prefix{netip.MustParsePrefix("192.168.2.0/24")},
netip.MustParsePrefix("10.1.0.0/24"),
fw.ProtocolTCP,

View File

@ -20,7 +20,7 @@ import (
nberrors "github.com/netbirdio/netbird/client/errors"
firewall "github.com/netbirdio/netbird/client/firewall/manager"
"github.com/netbirdio/netbird/client/internal/acl/id"
nbid "github.com/netbirdio/netbird/client/internal/acl/id"
"github.com/netbirdio/netbird/client/internal/routemanager/ipfwdstate"
"github.com/netbirdio/netbird/client/internal/routemanager/refcounter"
nbnet "github.com/netbirdio/netbird/util/net"
@ -228,6 +228,7 @@ func (r *router) createContainers() error {
// AddRouteFiltering appends a nftables rule to the routing chain
func (r *router) AddRouteFiltering(
id []byte,
sources []netip.Prefix,
destination netip.Prefix,
proto firewall.Protocol,
@ -236,7 +237,7 @@ func (r *router) AddRouteFiltering(
action firewall.Action,
) (firewall.Rule, error) {
ruleKey := id.GenerateRouteRuleKey(sources, destination, proto, sPort, dPort, action)
ruleKey := nbid.GenerateRouteRuleKey(sources, destination, proto, sPort, dPort, action)
if _, ok := r.rules[string(ruleKey)]; ok {
return ruleKey, nil
}

View File

@ -311,7 +311,7 @@ func TestRouter_AddRouteFiltering(t *testing.T) {
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
ruleKey, err := r.AddRouteFiltering(tt.sources, tt.destination, tt.proto, tt.sPort, tt.dPort, tt.action)
ruleKey, err := r.AddRouteFiltering(nil, tt.sources, tt.destination, tt.proto, tt.sPort, tt.dPort, tt.action)
require.NoError(t, err, "AddRouteFiltering failed")
t.Cleanup(func() {

View File

@ -12,6 +12,7 @@ import (
// PeerRule to handle management of rules
type PeerRule struct {
id string
mgmtId []byte
ip net.IP
ipLayer gopacket.LayerType
matchByIP bool
@ -19,7 +20,6 @@ type PeerRule struct {
sPort *firewall.Port
dPort *firewall.Port
drop bool
comment string
udpHook func([]byte) bool
}
@ -31,6 +31,7 @@ func (r *PeerRule) ID() string {
type RouteRule struct {
id string
mgmtId []byte
sources []netip.Prefix
destination netip.Prefix
proto firewall.Protocol

View File

@ -317,12 +317,19 @@ func (m *Manager) handleLocalDelivery(trace *PacketTrace, packetData []byte, d *
}
trace.AddResult(StageRouting, "Packet destined for local delivery", true)
blocked := m.peerACLsBlock(srcIP, packetData, m.incomingRules, d)
msg := "Allowed by peer ACL rules"
if blocked {
msg = "Blocked by peer ACL rules"
ruleId, blocked := m.peerACLsBlock(srcIP, packetData, m.incomingRules, d)
strRuleId := "implicit"
if ruleId != nil {
strRuleId = string(ruleId)
}
msg := fmt.Sprintf("Allowed by peer ACL rules (%s)", strRuleId)
if blocked {
msg = fmt.Sprintf("Blocked by peer ACL rules (%s)", strRuleId)
}
trace.AddResult(StagePeerACL, msg, !blocked)
if m.netstack {
@ -353,11 +360,16 @@ func (m *Manager) handleNativeRouter(trace *PacketTrace) *PacketTrace {
func (m *Manager) handleRouteACLs(trace *PacketTrace, d *decoder, srcIP, dstIP net.IP) *PacketTrace {
proto, _ := getProtocolFromPacket(d)
srcPort, dstPort := getPortsFromPacket(d)
allowed := m.routeACLsPass(srcIP, dstIP, proto, srcPort, dstPort)
id, allowed := m.routeACLsPass(srcIP, dstIP, proto, srcPort, dstPort)
msg := "Allowed by route ACLs"
strId := string(id)
if id == nil {
strId = "implicit"
}
msg := fmt.Sprintf("Allowed by route ACLs (%s)", strId)
if !allowed {
msg = "Blocked by route ACLs"
msg = fmt.Sprintf("Blocked by route ACLs (%s)", strId)
}
trace.AddResult(StageRouteACL, msg, allowed)

View File

@ -221,6 +221,7 @@ func (m *Manager) blockInvalidRouted(iface common.IFaceMapper) error {
log.Debugf("blocking invalid routed traffic for %s", wgPrefix)
if _, err := m.AddRouteFiltering(
nil,
[]netip.Prefix{netip.PrefixFrom(netip.IPv4Unspecified(), 0)},
wgPrefix,
firewall.ProtocolALL,
@ -351,21 +352,21 @@ func (m *Manager) RemoveNatRule(pair firewall.RouterPair) error {
// If comment argument is empty firewall manager should set
// rule ID as comment for the rule
func (m *Manager) AddPeerFiltering(
id []byte,
ip net.IP,
proto firewall.Protocol,
sPort *firewall.Port,
dPort *firewall.Port,
action firewall.Action,
_ string,
comment string,
) ([]firewall.Rule, error) {
r := PeerRule{
id: uuid.New().String(),
mgmtId: id,
ip: ip,
ipLayer: layers.LayerTypeIPv6,
matchByIP: true,
drop: action == firewall.ActionDrop,
comment: comment,
}
if ipNormalized := ip.To4(); ipNormalized != nil {
r.ipLayer = layers.LayerTypeIPv4
@ -403,6 +404,7 @@ func (m *Manager) AddPeerFiltering(
}
func (m *Manager) AddRouteFiltering(
id []byte,
sources []netip.Prefix,
destination netip.Prefix,
proto firewall.Protocol,
@ -411,7 +413,7 @@ func (m *Manager) AddRouteFiltering(
action firewall.Action,
) (firewall.Rule, error) {
if m.nativeRouter && m.nativeFirewall != nil {
return m.nativeFirewall.AddRouteFiltering(sources, destination, proto, sPort, dPort, action)
return m.nativeFirewall.AddRouteFiltering(id, sources, destination, proto, sPort, dPort, action)
}
m.mutex.Lock()
@ -419,7 +421,9 @@ func (m *Manager) AddRouteFiltering(
ruleID := uuid.New().String()
rule := RouteRule{
// TODO: consolidate these IDs
id: ruleID,
mgmtId: id,
sources: sources,
destination: destination,
proto: proto,
@ -655,22 +659,23 @@ func (m *Manager) dropFilter(packetData []byte) bool {
// handleLocalTraffic handles local traffic.
// If it returns true, the packet should be dropped.
func (m *Manager) handleLocalTraffic(d *decoder, srcIP, dstIP net.IP, packetData []byte) bool {
if m.peerACLsBlock(srcIP, packetData, m.incomingRules, d) {
srcIP, _ := netip.AddrFromSlice(srcIP)
dstIP, _ := netip.AddrFromSlice(dstIP)
if ruleId, blocked := m.peerACLsBlock(srcIP, packetData, m.incomingRules, d); blocked {
srcAddr, _ := netip.AddrFromSlice(srcIP)
dstAddr, _ := netip.AddrFromSlice(dstIP)
_, pnum := getProtocolFromPacket(d)
srcPort, dstPort := getPortsFromPacket(d)
m.logger.Trace("Dropping local packet (ACL denied): proto=%v src=%s:%d dst=%s:%d",
pnum, srcIP, srcPort, dstIP, dstPort)
m.logger.Trace("Dropping local packet (ACL denied): rule_id=%s proto=%v src=%s:%d dst=%s:%d",
ruleId, pnum, srcAddr, srcPort, dstAddr, dstPort)
m.flowLogger.StoreEvent(nftypes.EventFields{
FlowID: uuid.New(),
Type: nftypes.TypeDrop,
RuleID: ruleId,
Direction: nftypes.Ingress,
Protocol: pnum,
SourceIP: srcIP,
DestIP: dstIP,
SourceIP: srcAddr,
DestIP: dstAddr,
SourcePort: srcPort,
DestPort: dstPort,
// TODO: icmp type/code
@ -726,20 +731,21 @@ func (m *Manager) handleRoutedTraffic(d *decoder, srcIP, dstIP net.IP, packetDat
proto, pnum := getProtocolFromPacket(d)
srcPort, dstPort := getPortsFromPacket(d)
if !m.routeACLsPass(srcIP, dstIP, proto, srcPort, dstPort) {
srcIP, _ := netip.AddrFromSlice(srcIP)
dstIP, _ := netip.AddrFromSlice(dstIP)
if id, pass := m.routeACLsPass(srcIP, dstIP, proto, srcPort, dstPort); !pass {
srcAddr, _ := netip.AddrFromSlice(srcIP)
dstAddr, _ := netip.AddrFromSlice(dstIP)
m.logger.Trace("Dropping routed packet (ACL denied): proto=%v src=%s:%d dst=%s:%d",
pnum, srcIP, srcPort, dstIP, dstPort)
m.logger.Trace("Dropping routed packet (ACL denied): rule_id=%s proto=%v src=%s:%d dst=%s:%d",
id, pnum, srcIP, srcPort, dstIP, dstPort)
m.flowLogger.StoreEvent(nftypes.EventFields{
FlowID: uuid.New(),
Type: nftypes.TypeDrop,
RuleID: id,
Direction: nftypes.Ingress,
Protocol: pnum,
SourceIP: srcIP,
DestIP: dstIP,
SourceIP: srcAddr,
DestIP: dstAddr,
SourcePort: srcPort,
DestPort: dstPort,
// TODO: icmp type/code
@ -838,25 +844,25 @@ func (m *Manager) isSpecialICMP(d *decoder) bool {
icmpType == layers.ICMPv4TypeTimeExceeded
}
func (m *Manager) peerACLsBlock(srcIP net.IP, packetData []byte, rules map[string]RuleSet, d *decoder) bool {
func (m *Manager) peerACLsBlock(srcIP net.IP, packetData []byte, rules map[string]RuleSet, d *decoder) ([]byte, bool) {
if m.isSpecialICMP(d) {
return false
return nil, false
}
if filter, ok := validateRule(srcIP, packetData, rules[srcIP.String()], d); ok {
return filter
if mgmtId, filter, ok := validateRule(srcIP, packetData, rules[srcIP.String()], d); ok {
return mgmtId, filter
}
if filter, ok := validateRule(srcIP, packetData, rules["0.0.0.0"], d); ok {
return filter
if mgmtId, filter, ok := validateRule(srcIP, packetData, rules["0.0.0.0"], d); ok {
return mgmtId, filter
}
if filter, ok := validateRule(srcIP, packetData, rules["::"], d); ok {
return filter
if mgmtId, filter, ok := validateRule(srcIP, packetData, rules["::"], d); ok {
return mgmtId, filter
}
// Default policy: DROP ALL
return true
return nil, true
}
func portsMatch(rulePort *firewall.Port, packetPort uint16) bool {
@ -876,7 +882,7 @@ func portsMatch(rulePort *firewall.Port, packetPort uint16) bool {
return false
}
func validateRule(ip net.IP, packetData []byte, rules map[string]PeerRule, d *decoder) (bool, bool) {
func validateRule(ip net.IP, packetData []byte, rules map[string]PeerRule, d *decoder) ([]byte, bool, bool) {
payloadLayer := d.decoded[1]
for _, rule := range rules {
if rule.matchByIP && !ip.Equal(rule.ip) {
@ -884,7 +890,7 @@ func validateRule(ip net.IP, packetData []byte, rules map[string]PeerRule, d *de
}
if rule.protoLayer == layerTypeAll {
return rule.drop, true
return rule.mgmtId, rule.drop, true
}
if payloadLayer != rule.protoLayer {
@ -894,27 +900,27 @@ func validateRule(ip net.IP, packetData []byte, rules map[string]PeerRule, d *de
switch payloadLayer {
case layers.LayerTypeTCP:
if portsMatch(rule.sPort, uint16(d.tcp.SrcPort)) && portsMatch(rule.dPort, uint16(d.tcp.DstPort)) {
return rule.drop, true
return rule.mgmtId, rule.drop, true
}
case layers.LayerTypeUDP:
// if rule has UDP hook (and if we are here we match this rule)
// we ignore rule.drop and call this hook
if rule.udpHook != nil {
return rule.udpHook(packetData), true
return rule.mgmtId, rule.udpHook(packetData), true
}
if portsMatch(rule.sPort, uint16(d.udp.SrcPort)) && portsMatch(rule.dPort, uint16(d.udp.DstPort)) {
return rule.drop, true
return rule.mgmtId, rule.drop, true
}
case layers.LayerTypeICMPv4, layers.LayerTypeICMPv6:
return rule.drop, true
return rule.mgmtId, rule.drop, true
}
}
return false, false
return nil, false, false
}
// routeACLsPass returns treu if the packet is allowed by the route ACLs
func (m *Manager) routeACLsPass(srcIP, dstIP net.IP, proto firewall.Protocol, srcPort, dstPort uint16) bool {
func (m *Manager) routeACLsPass(srcIP, dstIP net.IP, proto firewall.Protocol, srcPort, dstPort uint16) ([]byte, bool) {
m.mutex.RLock()
defer m.mutex.RUnlock()
@ -923,10 +929,10 @@ func (m *Manager) routeACLsPass(srcIP, dstIP net.IP, proto firewall.Protocol, sr
for _, rule := range m.routeRules {
if m.ruleMatches(rule, srcAddr, dstAddr, proto, srcPort, dstPort) {
return rule.action == firewall.ActionAccept
return rule.mgmtId, rule.action == firewall.ActionAccept
}
}
return false
return nil, false
}
func (m *Manager) ruleMatches(rule RouteRule, srcAddr, dstAddr netip.Addr, proto firewall.Protocol, srcPort, dstPort uint16) bool {
@ -975,7 +981,6 @@ func (m *Manager) AddUDPPacketHook(
protoLayer: layers.LayerTypeUDP,
dPort: &firewall.Port{Values: []uint16{dPort}},
ipLayer: layers.LayerTypeIPv6,
comment: fmt.Sprintf("UDP Hook direction: %v, ip:%v, dport:%d", in, ip, dPort),
udpHook: hook,
}

View File

@ -93,8 +93,7 @@ func BenchmarkCoreFiltering(b *testing.B) {
stateful: false,
setupFunc: func(m *Manager) {
// Single rule allowing all traffic
_, err := m.AddPeerFiltering(net.ParseIP("0.0.0.0"), fw.ProtocolALL, nil, nil,
fw.ActionAccept, "", "allow all")
_, err := m.AddPeerFiltering(nil, net.ParseIP("0.0.0.0"), fw.ProtocolALL, nil, nil, fw.ActionAccept, "")
require.NoError(b, err)
},
desc: "Baseline: Single 'allow all' rule without connection tracking",
@ -114,10 +113,15 @@ func BenchmarkCoreFiltering(b *testing.B) {
// Add explicit rules matching return traffic pattern
for i := 0; i < 1000; i++ { // Simulate realistic ruleset size
ip := generateRandomIPs(1)[0]
_, err := m.AddPeerFiltering(ip, fw.ProtocolTCP,
_, err := m.AddPeerFiltering(
nil,
ip,
fw.ProtocolTCP,
&fw.Port{Values: []uint16{uint16(1024 + i)}},
&fw.Port{Values: []uint16{80}},
fw.ActionAccept, "", "explicit return")
fw.ActionAccept,
"",
)
require.NoError(b, err)
}
},
@ -128,8 +132,15 @@ func BenchmarkCoreFiltering(b *testing.B) {
stateful: true,
setupFunc: func(m *Manager) {
// Add some basic rules but rely on state for established connections
_, err := m.AddPeerFiltering(net.ParseIP("0.0.0.0"), fw.ProtocolTCP, nil, nil,
fw.ActionDrop, "", "default drop")
_, err := m.AddPeerFiltering(
nil,
net.ParseIP("0.0.0.0"),
fw.ProtocolTCP,
nil,
nil,
fw.ActionDrop,
"",
)
require.NoError(b, err)
},
desc: "Connection tracking with established connections",
@ -590,10 +601,7 @@ func BenchmarkLongLivedConnections(b *testing.B) {
// Setup initial state based on scenario
if sc.rules {
// Single rule to allow all return traffic from port 80
_, err := manager.AddPeerFiltering(net.ParseIP("0.0.0.0"), fw.ProtocolTCP,
&fw.Port{Values: []uint16{80}},
nil,
fw.ActionAccept, "", "return traffic")
_, err := manager.AddPeerFiltering(nil, net.ParseIP("0.0.0.0"), fw.ProtocolTCP, &fw.Port{Values: []uint16{80}}, nil, fw.ActionAccept, "")
require.NoError(b, err)
}
@ -681,10 +689,7 @@ func BenchmarkShortLivedConnections(b *testing.B) {
// Setup initial state based on scenario
if sc.rules {
// Single rule to allow all return traffic from port 80
_, err := manager.AddPeerFiltering(net.ParseIP("0.0.0.0"), fw.ProtocolTCP,
&fw.Port{Values: []uint16{80}},
nil,
fw.ActionAccept, "", "return traffic")
_, err := manager.AddPeerFiltering(nil, net.ParseIP("0.0.0.0"), fw.ProtocolTCP, &fw.Port{Values: []uint16{80}}, nil, fw.ActionAccept, "")
require.NoError(b, err)
}
@ -799,10 +804,7 @@ func BenchmarkParallelLongLivedConnections(b *testing.B) {
// Setup initial state based on scenario
if sc.rules {
_, err := manager.AddPeerFiltering(net.ParseIP("0.0.0.0"), fw.ProtocolTCP,
&fw.Port{Values: []uint16{80}},
nil,
fw.ActionAccept, "", "return traffic")
_, err := manager.AddPeerFiltering(nil, net.ParseIP("0.0.0.0"), fw.ProtocolTCP, &fw.Port{Values: []uint16{80}}, nil, fw.ActionAccept, "")
require.NoError(b, err)
}
@ -886,10 +888,7 @@ func BenchmarkParallelShortLivedConnections(b *testing.B) {
})
if sc.rules {
_, err := manager.AddPeerFiltering(net.ParseIP("0.0.0.0"), fw.ProtocolTCP,
&fw.Port{Values: []uint16{80}},
nil,
fw.ActionAccept, "", "return traffic")
_, err := manager.AddPeerFiltering(nil, net.ParseIP("0.0.0.0"), fw.ProtocolTCP, &fw.Port{Values: []uint16{80}}, nil, fw.ActionAccept, "")
require.NoError(b, err)
}
@ -1033,14 +1032,7 @@ func BenchmarkRouteACLs(b *testing.B) {
}
for _, r := range rules {
_, err := manager.AddRouteFiltering(
r.sources,
r.dest,
r.proto,
nil,
r.port,
fw.ActionAccept,
)
_, err := manager.AddRouteFiltering(nil, r.sources, r.dest, r.proto, nil, r.port, fw.ActionAccept)
if err != nil {
b.Fatal(err)
}

View File

@ -199,13 +199,13 @@ func TestPeerACLFiltering(t *testing.T) {
for _, tc := range testCases {
t.Run(tc.name, func(t *testing.T) {
rules, err := manager.AddPeerFiltering(
nil,
net.ParseIP(tc.ruleIP),
tc.ruleProto,
tc.ruleSrcPort,
tc.ruleDstPort,
tc.ruleAction,
"",
tc.name,
)
require.NoError(t, err)
require.NotEmpty(t, rules)
@ -803,6 +803,7 @@ func TestRouteACLFiltering(t *testing.T) {
for _, tc := range testCases {
t.Run(tc.name, func(t *testing.T) {
rule, err := manager.AddRouteFiltering(
nil,
tc.rule.sources,
tc.rule.dest,
tc.rule.proto,
@ -822,7 +823,7 @@ func TestRouteACLFiltering(t *testing.T) {
// testing routeACLsPass only and not DropIncoming, as routed packets are dropped after being passed
// to the forwarder
isAllowed := manager.routeACLsPass(srcIP, dstIP, tc.proto, tc.srcPort, tc.dstPort)
_, isAllowed := manager.routeACLsPass(srcIP, dstIP, tc.proto, tc.srcPort, tc.dstPort)
require.Equal(t, tc.shouldPass, isAllowed)
})
}
@ -985,6 +986,7 @@ func TestRouteACLOrder(t *testing.T) {
var rules []fw.Rule
for _, r := range tc.rules {
rule, err := manager.AddRouteFiltering(
nil,
r.sources,
r.dest,
r.proto,
@ -1007,7 +1009,7 @@ func TestRouteACLOrder(t *testing.T) {
srcIP := net.ParseIP(p.srcIP)
dstIP := net.ParseIP(p.dstIP)
isAllowed := manager.routeACLsPass(srcIP, dstIP, p.proto, p.srcPort, p.dstPort)
_, isAllowed := manager.routeACLsPass(srcIP, dstIP, p.proto, p.srcPort, p.dstPort)
require.Equal(t, p.shouldPass, isAllowed, "packet %d failed", i)
}
})

View File

@ -96,9 +96,8 @@ func TestManagerAddPeerFiltering(t *testing.T) {
proto := fw.ProtocolTCP
port := &fw.Port{Values: []uint16{80}}
action := fw.ActionDrop
comment := "Test rule"
rule, err := m.AddPeerFiltering(ip, proto, nil, port, action, "", comment)
rule, err := m.AddPeerFiltering(nil, ip, proto, nil, port, action, "")
if err != nil {
t.Errorf("failed to add filtering: %v", err)
return
@ -130,9 +129,8 @@ func TestManagerDeleteRule(t *testing.T) {
proto := fw.ProtocolTCP
port := &fw.Port{Values: []uint16{80}}
action := fw.ActionDrop
comment := "Test rule 2"
rule2, err := m.AddPeerFiltering(ip, proto, nil, port, action, "", comment)
rule2, err := m.AddPeerFiltering(nil, ip, proto, nil, port, action, "")
if err != nil {
t.Errorf("failed to add filtering: %v", err)
return
@ -250,9 +248,8 @@ func TestManagerReset(t *testing.T) {
proto := fw.ProtocolTCP
port := &fw.Port{Values: []uint16{80}}
action := fw.ActionDrop
comment := "Test rule"
_, err = m.AddPeerFiltering(ip, proto, nil, port, action, "", comment)
_, err = m.AddPeerFiltering(nil, ip, proto, nil, port, action, "")
if err != nil {
t.Errorf("failed to add filtering: %v", err)
return
@ -296,9 +293,8 @@ func TestNotMatchByIP(t *testing.T) {
ip := net.ParseIP("0.0.0.0")
proto := fw.ProtocolUDP
action := fw.ActionAccept
comment := "Test rule"
_, err = m.AddPeerFiltering(ip, proto, nil, nil, action, "", comment)
_, err = m.AddPeerFiltering(nil, ip, proto, nil, nil, action, "")
if err != nil {
t.Errorf("failed to add filtering: %v", err)
return
@ -498,7 +494,7 @@ func TestUSPFilterCreatePerformance(t *testing.T) {
start := time.Now()
for i := 0; i < testMax; i++ {
port := &fw.Port{Values: []uint16{uint16(1000 + i)}}
_, err = manager.AddPeerFiltering(ip, "tcp", nil, port, fw.ActionAccept, "", "accept HTTP traffic")
_, err = manager.AddPeerFiltering(nil, ip, "tcp", nil, port, fw.ActionAccept, "")
require.NoError(t, err, "failed to add rule")
}

View File

@ -240,7 +240,7 @@ func (d *DefaultManager) applyRouteACL(rule *mgmProto.RouteFirewallRule) (id.Rul
dPorts := convertPortInfo(rule.PortInfo)
addedRule, err := d.firewall.AddRouteFiltering(sources, destination, protocol, nil, dPorts, action)
addedRule, err := d.firewall.AddRouteFiltering(rule.Id, sources, destination, protocol, nil, dPorts, action)
if err != nil {
return "", fmt.Errorf("add route rule: %w", err)
}
@ -281,7 +281,7 @@ func (d *DefaultManager) protoRuleToFirewallRule(
}
}
ruleID := d.getPeerRuleID(ip, protocol, int(r.Direction), port, action, "")
ruleID := d.getPeerRuleID(ip, protocol, int(r.Direction), port, action)
if rulesPair, ok := d.peerRulesPairs[ruleID]; ok {
return ruleID, rulesPair, nil
}
@ -289,11 +289,11 @@ func (d *DefaultManager) protoRuleToFirewallRule(
var rules []firewall.Rule
switch r.Direction {
case mgmProto.RuleDirection_IN:
rules, err = d.addInRules(ip, protocol, port, action, ipsetName, "")
rules, err = d.addInRules(r.Id, ip, protocol, port, action, ipsetName)
case mgmProto.RuleDirection_OUT:
// TODO: Remove this soon. Outbound rules are obsolete.
// We only maintain this for return traffic (inbound dir) which is now handled by the stateful firewall already
rules, err = d.addOutRules(ip, protocol, port, action, ipsetName, "")
rules, err = d.addOutRules(r.Id, ip, protocol, port, action, ipsetName)
default:
return "", nil, fmt.Errorf("invalid direction, skipping firewall rule")
}
@ -322,14 +322,14 @@ func portInfoEmpty(portInfo *mgmProto.PortInfo) bool {
}
func (d *DefaultManager) addInRules(
id []byte,
ip net.IP,
protocol firewall.Protocol,
port *firewall.Port,
action firewall.Action,
ipsetName string,
comment string,
) ([]firewall.Rule, error) {
rule, err := d.firewall.AddPeerFiltering(ip, protocol, nil, port, action, ipsetName, comment)
rule, err := d.firewall.AddPeerFiltering(id, ip, protocol, nil, port, action, ipsetName)
if err != nil {
return nil, fmt.Errorf("add firewall rule: %w", err)
}
@ -338,18 +338,18 @@ func (d *DefaultManager) addInRules(
}
func (d *DefaultManager) addOutRules(
id []byte,
ip net.IP,
protocol firewall.Protocol,
port *firewall.Port,
action firewall.Action,
ipsetName string,
comment string,
) ([]firewall.Rule, error) {
if shouldSkipInvertedRule(protocol, port) {
return nil, nil
}
rule, err := d.firewall.AddPeerFiltering(ip, protocol, port, nil, action, ipsetName, comment)
rule, err := d.firewall.AddPeerFiltering(id, ip, protocol, port, nil, action, ipsetName)
if err != nil {
return nil, fmt.Errorf("add firewall rule: %w", err)
}
@ -364,9 +364,8 @@ func (d *DefaultManager) getPeerRuleID(
direction int,
port *firewall.Port,
action firewall.Action,
comment string,
) id.RuleID {
idStr := ip.String() + string(proto) + strconv.Itoa(direction) + strconv.Itoa(int(action)) + comment
idStr := ip.String() + string(proto) + strconv.Itoa(direction) + strconv.Itoa(int(action))
if port != nil {
idStr += port.String()
}

View File

@ -88,7 +88,7 @@ func (h *Manager) allowDNSFirewall() error {
return nil
}
dnsRules, err := h.firewall.AddPeerFiltering(net.IP{0, 0, 0, 0}, firewall.ProtocolUDP, nil, dport, firewall.ActionAccept, "", "")
dnsRules, err := h.firewall.AddPeerFiltering(nil, net.IP{0, 0, 0, 0}, firewall.ProtocolUDP, nil, dport, firewall.ActionAccept, "")
if err != nil {
log.Errorf("failed to add allow DNS router rules, err: %v", err)
return err

View File

@ -495,13 +495,13 @@ func (e *Engine) initFirewall() error {
// this rule is static and will be torn down on engine down by the firewall manager
if _, err := e.firewall.AddPeerFiltering(
nil,
net.IP{0, 0, 0, 0},
firewallManager.ProtocolUDP,
nil,
&port,
firewallManager.ActionAccept,
"",
"",
); err != nil {
log.Errorf("failed to allow rosenpass interface traffic: %v", err)
return nil
@ -525,6 +525,7 @@ func (e *Engine) blockLanAccess() {
v4 := netip.PrefixFrom(netip.IPv4Unspecified(), 0)
for _, network := range toBlock {
if _, err := e.firewall.AddRouteFiltering(
nil,
[]netip.Prefix{v4},
network,
firewallManager.ProtocolALL,

View File

@ -12,7 +12,7 @@ import (
"github.com/netbirdio/netbird/client/internal/netflow/conntrack"
"github.com/netbirdio/netbird/client/internal/netflow/logger"
"github.com/netbirdio/netbird/client/internal/netflow/types"
nftypes "github.com/netbirdio/netbird/client/internal/netflow/types"
"github.com/netbirdio/netbird/flow/client"
"github.com/netbirdio/netbird/flow/proto"
)
@ -20,19 +20,19 @@ import (
// Manager handles netflow tracking and logging
type Manager struct {
mux sync.Mutex
logger types.FlowLogger
flowConfig *types.FlowConfig
conntrack types.ConnTracker
logger nftypes.FlowLogger
flowConfig *nftypes.FlowConfig
conntrack nftypes.ConnTracker
ctx context.Context
receiverClient *client.GRPCClient
publicKey []byte
}
// NewManager creates a new netflow manager
func NewManager(ctx context.Context, iface types.IFaceMapper, publicKey []byte) *Manager {
func NewManager(ctx context.Context, iface nftypes.IFaceMapper, publicKey []byte) *Manager {
flowLogger := logger.New(ctx)
var ct types.ConnTracker
var ct nftypes.ConnTracker
if runtime.GOOS == "linux" && iface != nil && !iface.IsUserspaceBind() {
ct = conntrack.New(flowLogger, iface)
}
@ -46,7 +46,7 @@ func NewManager(ctx context.Context, iface types.IFaceMapper, publicKey []byte)
}
// Update applies new flow configuration settings
func (m *Manager) Update(update *types.FlowConfig) error {
func (m *Manager) Update(update *nftypes.FlowConfig) error {
if update == nil {
return nil
}
@ -99,7 +99,7 @@ func (m *Manager) Close() {
}
// GetLogger returns the flow logger
func (m *Manager) GetLogger() types.FlowLogger {
func (m *Manager) GetLogger() nftypes.FlowLogger {
return m.logger
}
@ -137,20 +137,21 @@ func (m *Manager) receiveACKs() {
}
}
func (m *Manager) send(event *types.Event) error {
func (m *Manager) send(event *nftypes.Event) error {
if m.receiverClient == nil {
return nil
}
return m.receiverClient.Send(m.ctx, toProtoEvent(m.publicKey, event))
}
func toProtoEvent(publicKey []byte, event *types.Event) *proto.FlowEvent {
func toProtoEvent(publicKey []byte, event *nftypes.Event) *proto.FlowEvent {
protoEvent := &proto.FlowEvent{
EventId: event.ID,
FlowId: event.FlowID.String(),
Timestamp: timestamppb.New(event.Timestamp),
PublicKey: publicKey,
EventFields: &proto.EventFields{
FlowFields: &proto.FlowFields{
FlowId: event.FlowID[:],
RuleId: event.RuleID,
Type: proto.Type(event.Type),
Direction: proto.Direction(event.Direction),
Protocol: uint32(event.Protocol),
@ -158,8 +159,8 @@ func toProtoEvent(publicKey []byte, event *types.Event) *proto.FlowEvent {
DestIp: event.DestIP.AsSlice(),
},
}
if event.Protocol == 1 {
protoEvent.EventFields.ConnectionInfo = &proto.EventFields_IcmpInfo{
if event.Protocol == nftypes.ICMP {
protoEvent.FlowFields.ConnectionInfo = &proto.FlowFields_IcmpInfo{
IcmpInfo: &proto.ICMPInfo{
IcmpType: uint32(event.ICMPType),
IcmpCode: uint32(event.ICMPCode),
@ -168,7 +169,7 @@ func toProtoEvent(publicKey []byte, event *types.Event) *proto.FlowEvent {
return protoEvent
}
protoEvent.EventFields.ConnectionInfo = &proto.EventFields_PortInfo{
protoEvent.FlowFields.ConnectionInfo = &proto.FlowFields_PortInfo{
PortInfo: &proto.PortInfo{
SourcePort: uint32(event.SourcePort),
DestPort: uint32(event.DestPort),

View File

@ -69,6 +69,7 @@ type Event struct {
type EventFields struct {
FlowID uuid.UUID
Type Type
RuleID []byte
Direction Direction
Protocol Protocol
SourceIP netip.Addr

View File

@ -131,13 +131,11 @@ type FlowEvent struct {
// Unique client event identifier
EventId string `protobuf:"bytes,1,opt,name=event_id,json=eventId,proto3" json:"event_id,omitempty"`
// Unique client flow session identifier
FlowId string `protobuf:"bytes,2,opt,name=flow_id,json=flowId,proto3" json:"flow_id,omitempty"`
// When the event occurred
Timestamp *timestamppb.Timestamp `protobuf:"bytes,3,opt,name=timestamp,proto3" json:"timestamp,omitempty"`
// Public key of the sending peer
PublicKey []byte `protobuf:"bytes,4,opt,name=public_key,json=publicKey,proto3" json:"public_key,omitempty"`
EventFields *EventFields `protobuf:"bytes,5,opt,name=event_fields,json=eventFields,proto3" json:"event_fields,omitempty"`
PublicKey []byte `protobuf:"bytes,4,opt,name=public_key,json=publicKey,proto3" json:"public_key,omitempty"`
FlowFields *FlowFields `protobuf:"bytes,5,opt,name=flow_fields,json=flowFields,proto3" json:"flow_fields,omitempty"`
}
func (x *FlowEvent) Reset() {
@ -179,13 +177,6 @@ func (x *FlowEvent) GetEventId() string {
return ""
}
func (x *FlowEvent) GetFlowId() string {
if x != nil {
return x.FlowId
}
return ""
}
func (x *FlowEvent) GetTimestamp() *timestamppb.Timestamp {
if x != nil {
return x.Timestamp
@ -200,9 +191,9 @@ func (x *FlowEvent) GetPublicKey() []byte {
return nil
}
func (x *FlowEvent) GetEventFields() *EventFields {
func (x *FlowEvent) GetFlowFields() *FlowFields {
if x != nil {
return x.EventFields
return x.FlowFields
}
return nil
}
@ -255,32 +246,36 @@ func (x *FlowEventAck) GetEventId() string {
return ""
}
type EventFields struct {
type FlowFields struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields
// Event type
Type Type `protobuf:"varint,1,opt,name=type,proto3,enum=flow.Type" json:"type,omitempty"`
// Unique client flow session identifier
FlowId []byte `protobuf:"bytes,1,opt,name=flow_id,json=flowId,proto3" json:"flow_id,omitempty"`
// Flow type
Type Type `protobuf:"varint,2,opt,name=type,proto3,enum=flow.Type" json:"type,omitempty"`
// RuleId identifies the rule that allowed or denied the connection
RuleId []byte `protobuf:"bytes,3,opt,name=rule_id,json=ruleId,proto3" json:"rule_id,omitempty"`
// Initiating traffic direction
Direction Direction `protobuf:"varint,2,opt,name=direction,proto3,enum=flow.Direction" json:"direction,omitempty"`
Direction Direction `protobuf:"varint,4,opt,name=direction,proto3,enum=flow.Direction" json:"direction,omitempty"`
// IP protocol number
Protocol uint32 `protobuf:"varint,3,opt,name=protocol,proto3" json:"protocol,omitempty"`
Protocol uint32 `protobuf:"varint,5,opt,name=protocol,proto3" json:"protocol,omitempty"`
// Source IP address
SourceIp []byte `protobuf:"bytes,4,opt,name=source_ip,json=sourceIp,proto3" json:"source_ip,omitempty"`
SourceIp []byte `protobuf:"bytes,6,opt,name=source_ip,json=sourceIp,proto3" json:"source_ip,omitempty"`
// Destination IP address
DestIp []byte `protobuf:"bytes,5,opt,name=dest_ip,json=destIp,proto3" json:"dest_ip,omitempty"`
DestIp []byte `protobuf:"bytes,7,opt,name=dest_ip,json=destIp,proto3" json:"dest_ip,omitempty"`
// Layer 4 -specific information
//
// Types that are assignable to ConnectionInfo:
//
// *EventFields_PortInfo
// *EventFields_IcmpInfo
ConnectionInfo isEventFields_ConnectionInfo `protobuf_oneof:"connection_info"`
// *FlowFields_PortInfo
// *FlowFields_IcmpInfo
ConnectionInfo isFlowFields_ConnectionInfo `protobuf_oneof:"connection_info"`
}
func (x *EventFields) Reset() {
*x = EventFields{}
func (x *FlowFields) Reset() {
*x = FlowFields{}
if protoimpl.UnsafeEnabled {
mi := &file_flow_proto_msgTypes[2]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
@ -288,13 +283,13 @@ func (x *EventFields) Reset() {
}
}
func (x *EventFields) String() string {
func (x *FlowFields) String() string {
return protoimpl.X.MessageStringOf(x)
}
func (*EventFields) ProtoMessage() {}
func (*FlowFields) ProtoMessage() {}
func (x *EventFields) ProtoReflect() protoreflect.Message {
func (x *FlowFields) ProtoReflect() protoreflect.Message {
mi := &file_flow_proto_msgTypes[2]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
@ -306,84 +301,98 @@ func (x *EventFields) ProtoReflect() protoreflect.Message {
return mi.MessageOf(x)
}
// Deprecated: Use EventFields.ProtoReflect.Descriptor instead.
func (*EventFields) Descriptor() ([]byte, []int) {
// Deprecated: Use FlowFields.ProtoReflect.Descriptor instead.
func (*FlowFields) Descriptor() ([]byte, []int) {
return file_flow_proto_rawDescGZIP(), []int{2}
}
func (x *EventFields) GetType() Type {
func (x *FlowFields) GetFlowId() []byte {
if x != nil {
return x.FlowId
}
return nil
}
func (x *FlowFields) GetType() Type {
if x != nil {
return x.Type
}
return Type_TYPE_UNKNOWN
}
func (x *EventFields) GetDirection() Direction {
func (x *FlowFields) GetRuleId() []byte {
if x != nil {
return x.RuleId
}
return nil
}
func (x *FlowFields) GetDirection() Direction {
if x != nil {
return x.Direction
}
return Direction_DIRECTION_UNKNOWN
}
func (x *EventFields) GetProtocol() uint32 {
func (x *FlowFields) GetProtocol() uint32 {
if x != nil {
return x.Protocol
}
return 0
}
func (x *EventFields) GetSourceIp() []byte {
func (x *FlowFields) GetSourceIp() []byte {
if x != nil {
return x.SourceIp
}
return nil
}
func (x *EventFields) GetDestIp() []byte {
func (x *FlowFields) GetDestIp() []byte {
if x != nil {
return x.DestIp
}
return nil
}
func (m *EventFields) GetConnectionInfo() isEventFields_ConnectionInfo {
func (m *FlowFields) GetConnectionInfo() isFlowFields_ConnectionInfo {
if m != nil {
return m.ConnectionInfo
}
return nil
}
func (x *EventFields) GetPortInfo() *PortInfo {
if x, ok := x.GetConnectionInfo().(*EventFields_PortInfo); ok {
func (x *FlowFields) GetPortInfo() *PortInfo {
if x, ok := x.GetConnectionInfo().(*FlowFields_PortInfo); ok {
return x.PortInfo
}
return nil
}
func (x *EventFields) GetIcmpInfo() *ICMPInfo {
if x, ok := x.GetConnectionInfo().(*EventFields_IcmpInfo); ok {
func (x *FlowFields) GetIcmpInfo() *ICMPInfo {
if x, ok := x.GetConnectionInfo().(*FlowFields_IcmpInfo); ok {
return x.IcmpInfo
}
return nil
}
type isEventFields_ConnectionInfo interface {
isEventFields_ConnectionInfo()
type isFlowFields_ConnectionInfo interface {
isFlowFields_ConnectionInfo()
}
type EventFields_PortInfo struct {
type FlowFields_PortInfo struct {
// TCP/UDP port information
PortInfo *PortInfo `protobuf:"bytes,6,opt,name=port_info,json=portInfo,proto3,oneof"`
PortInfo *PortInfo `protobuf:"bytes,8,opt,name=port_info,json=portInfo,proto3,oneof"`
}
type EventFields_IcmpInfo struct {
type FlowFields_IcmpInfo struct {
// ICMP type and code
IcmpInfo *ICMPInfo `protobuf:"bytes,7,opt,name=icmp_info,json=icmpInfo,proto3,oneof"`
IcmpInfo *ICMPInfo `protobuf:"bytes,9,opt,name=icmp_info,json=icmpInfo,proto3,oneof"`
}
func (*EventFields_PortInfo) isEventFields_ConnectionInfo() {}
func (*FlowFields_PortInfo) isFlowFields_ConnectionInfo() {}
func (*EventFields_IcmpInfo) isEventFields_ConnectionInfo() {}
func (*FlowFields_IcmpInfo) isFlowFields_ConnectionInfo() {}
// TCP/UDP port information
type PortInfo struct {
@ -503,63 +512,64 @@ var file_flow_proto_rawDesc = []byte{
0x0a, 0x0a, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x04, 0x66, 0x6c,
0x6f, 0x77, 0x1a, 0x1f, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f,
0x62, 0x75, 0x66, 0x2f, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x2e, 0x70, 0x72,
0x6f, 0x74, 0x6f, 0x22, 0xce, 0x01, 0x0a, 0x09, 0x46, 0x6c, 0x6f, 0x77, 0x45, 0x76, 0x65, 0x6e,
0x6f, 0x74, 0x6f, 0x22, 0xb2, 0x01, 0x0a, 0x09, 0x46, 0x6c, 0x6f, 0x77, 0x45, 0x76, 0x65, 0x6e,
0x74, 0x12, 0x19, 0x0a, 0x08, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20,
0x01, 0x28, 0x09, 0x52, 0x07, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x12, 0x17, 0x0a, 0x07,
0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x66,
0x6c, 0x6f, 0x77, 0x49, 0x64, 0x12, 0x38, 0x0a, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61,
0x6d, 0x70, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c,
0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73,
0x74, 0x61, 0x6d, 0x70, 0x52, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x12,
0x1d, 0x0a, 0x0a, 0x70, 0x75, 0x62, 0x6c, 0x69, 0x63, 0x5f, 0x6b, 0x65, 0x79, 0x18, 0x04, 0x20,
0x01, 0x28, 0x0c, 0x52, 0x09, 0x70, 0x75, 0x62, 0x6c, 0x69, 0x63, 0x4b, 0x65, 0x79, 0x12, 0x34,
0x0a, 0x0c, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x18, 0x05,
0x20, 0x01, 0x28, 0x0b, 0x32, 0x11, 0x2e, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x45, 0x76, 0x65, 0x6e,
0x74, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x52, 0x0b, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x46, 0x69,
0x65, 0x6c, 0x64, 0x73, 0x22, 0x29, 0x0a, 0x0c, 0x46, 0x6c, 0x6f, 0x77, 0x45, 0x76, 0x65, 0x6e,
0x74, 0x41, 0x63, 0x6b, 0x12, 0x19, 0x0a, 0x08, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64,
0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x22,
0x9f, 0x02, 0x0a, 0x0b, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x12,
0x1e, 0x0a, 0x04, 0x74, 0x79, 0x70, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x0a, 0x2e,
0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x54, 0x79, 0x70, 0x65, 0x52, 0x04, 0x74, 0x79, 0x70, 0x65, 0x12,
0x2d, 0x0a, 0x09, 0x64, 0x69, 0x72, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01,
0x28, 0x0e, 0x32, 0x0f, 0x2e, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x44, 0x69, 0x72, 0x65, 0x63, 0x74,
0x69, 0x6f, 0x6e, 0x52, 0x09, 0x64, 0x69, 0x72, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x1a,
0x0a, 0x08, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0d,
0x52, 0x08, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x12, 0x1b, 0x0a, 0x09, 0x73, 0x6f,
0x75, 0x72, 0x63, 0x65, 0x5f, 0x69, 0x70, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x08, 0x73,
0x6f, 0x75, 0x72, 0x63, 0x65, 0x49, 0x70, 0x12, 0x17, 0x0a, 0x07, 0x64, 0x65, 0x73, 0x74, 0x5f,
0x69, 0x70, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x06, 0x64, 0x65, 0x73, 0x74, 0x49, 0x70,
0x12, 0x2d, 0x0a, 0x09, 0x70, 0x6f, 0x72, 0x74, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x18, 0x06, 0x20,
0x01, 0x28, 0x0b, 0x32, 0x0e, 0x2e, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x50, 0x6f, 0x72, 0x74, 0x49,
0x6e, 0x66, 0x6f, 0x48, 0x00, 0x52, 0x08, 0x70, 0x6f, 0x72, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x12,
0x2d, 0x0a, 0x09, 0x69, 0x63, 0x6d, 0x70, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x18, 0x07, 0x20, 0x01,
0x28, 0x0b, 0x32, 0x0e, 0x2e, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x49, 0x43, 0x4d, 0x50, 0x49, 0x6e,
0x66, 0x6f, 0x48, 0x00, 0x52, 0x08, 0x69, 0x63, 0x6d, 0x70, 0x49, 0x6e, 0x66, 0x6f, 0x42, 0x11,
0x0a, 0x0f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x6e, 0x66,
0x6f, 0x22, 0x48, 0x0a, 0x08, 0x50, 0x6f, 0x72, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x1f, 0x0a,
0x0b, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x70, 0x6f, 0x72, 0x74, 0x18, 0x01, 0x20, 0x01,
0x28, 0x0d, 0x52, 0x0a, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x50, 0x6f, 0x72, 0x74, 0x12, 0x1b,
0x0a, 0x09, 0x64, 0x65, 0x73, 0x74, 0x5f, 0x70, 0x6f, 0x72, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28,
0x0d, 0x52, 0x08, 0x64, 0x65, 0x73, 0x74, 0x50, 0x6f, 0x72, 0x74, 0x22, 0x44, 0x0a, 0x08, 0x49,
0x43, 0x4d, 0x50, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x1b, 0x0a, 0x09, 0x69, 0x63, 0x6d, 0x70, 0x5f,
0x74, 0x79, 0x70, 0x65, 0x18, 0x01, 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, 0x02, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x08, 0x69, 0x63, 0x6d, 0x70, 0x43, 0x6f, 0x64,
0x65, 0x2a, 0x45, 0x0a, 0x04, 0x54, 0x79, 0x70, 0x65, 0x12, 0x10, 0x0a, 0x0c, 0x54, 0x59, 0x50,
0x45, 0x5f, 0x55, 0x4e, 0x4b, 0x4e, 0x4f, 0x57, 0x4e, 0x10, 0x00, 0x12, 0x0e, 0x0a, 0x0a, 0x54,
0x59, 0x50, 0x45, 0x5f, 0x53, 0x54, 0x41, 0x52, 0x54, 0x10, 0x01, 0x12, 0x0c, 0x0a, 0x08, 0x54,
0x59, 0x50, 0x45, 0x5f, 0x45, 0x4e, 0x44, 0x10, 0x02, 0x12, 0x0d, 0x0a, 0x09, 0x54, 0x59, 0x50,
0x45, 0x5f, 0x44, 0x52, 0x4f, 0x50, 0x10, 0x03, 0x2a, 0x3b, 0x0a, 0x09, 0x44, 0x69, 0x72, 0x65,
0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x15, 0x0a, 0x11, 0x44, 0x49, 0x52, 0x45, 0x43, 0x54, 0x49,
0x4f, 0x4e, 0x5f, 0x55, 0x4e, 0x4b, 0x4e, 0x4f, 0x57, 0x4e, 0x10, 0x00, 0x12, 0x0b, 0x0a, 0x07,
0x49, 0x4e, 0x47, 0x52, 0x45, 0x53, 0x53, 0x10, 0x01, 0x12, 0x0a, 0x0a, 0x06, 0x45, 0x47, 0x52,
0x45, 0x53, 0x53, 0x10, 0x02, 0x32, 0x42, 0x0a, 0x0b, 0x46, 0x6c, 0x6f, 0x77, 0x53, 0x65, 0x72,
0x76, 0x69, 0x63, 0x65, 0x12, 0x33, 0x0a, 0x06, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x0f,
0x2e, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x46, 0x6c, 0x6f, 0x77, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x1a,
0x12, 0x2e, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x46, 0x6c, 0x6f, 0x77, 0x45, 0x76, 0x65, 0x6e, 0x74,
0x41, 0x63, 0x6b, 0x22, 0x00, 0x28, 0x01, 0x30, 0x01, 0x42, 0x08, 0x5a, 0x06, 0x2f, 0x70, 0x72,
0x6f, 0x74, 0x6f, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
0x01, 0x28, 0x09, 0x52, 0x07, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x12, 0x38, 0x0a, 0x09,
0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32,
0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75,
0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x09, 0x74, 0x69, 0x6d,
0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x12, 0x1d, 0x0a, 0x0a, 0x70, 0x75, 0x62, 0x6c, 0x69, 0x63,
0x5f, 0x6b, 0x65, 0x79, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x09, 0x70, 0x75, 0x62, 0x6c,
0x69, 0x63, 0x4b, 0x65, 0x79, 0x12, 0x31, 0x0a, 0x0b, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x66, 0x69,
0x65, 0x6c, 0x64, 0x73, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x66, 0x6c, 0x6f,
0x77, 0x2e, 0x46, 0x6c, 0x6f, 0x77, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x52, 0x0a, 0x66, 0x6c,
0x6f, 0x77, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x22, 0x29, 0x0a, 0x0c, 0x46, 0x6c, 0x6f, 0x77,
0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x63, 0x6b, 0x12, 0x19, 0x0a, 0x08, 0x65, 0x76, 0x65, 0x6e,
0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x65, 0x76, 0x65, 0x6e,
0x74, 0x49, 0x64, 0x22, 0xd0, 0x02, 0x0a, 0x0a, 0x46, 0x6c, 0x6f, 0x77, 0x46, 0x69, 0x65, 0x6c,
0x64, 0x73, 0x12, 0x17, 0x0a, 0x07, 0x66, 0x6c, 0x6f, 0x77, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20,
0x01, 0x28, 0x0c, 0x52, 0x06, 0x66, 0x6c, 0x6f, 0x77, 0x49, 0x64, 0x12, 0x1e, 0x0a, 0x04, 0x74,
0x79, 0x70, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x0a, 0x2e, 0x66, 0x6c, 0x6f, 0x77,
0x2e, 0x54, 0x79, 0x70, 0x65, 0x52, 0x04, 0x74, 0x79, 0x70, 0x65, 0x12, 0x17, 0x0a, 0x07, 0x72,
0x75, 0x6c, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x06, 0x72, 0x75,
0x6c, 0x65, 0x49, 0x64, 0x12, 0x2d, 0x0a, 0x09, 0x64, 0x69, 0x72, 0x65, 0x63, 0x74, 0x69, 0x6f,
0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x0f, 0x2e, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x44,
0x69, 0x72, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x09, 0x64, 0x69, 0x72, 0x65, 0x63, 0x74,
0x69, 0x6f, 0x6e, 0x12, 0x1a, 0x0a, 0x08, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x18,
0x05, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x08, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x12,
0x1b, 0x0a, 0x09, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x69, 0x70, 0x18, 0x06, 0x20, 0x01,
0x28, 0x0c, 0x52, 0x08, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x49, 0x70, 0x12, 0x17, 0x0a, 0x07,
0x64, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x70, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x06, 0x64,
0x65, 0x73, 0x74, 0x49, 0x70, 0x12, 0x2d, 0x0a, 0x09, 0x70, 0x6f, 0x72, 0x74, 0x5f, 0x69, 0x6e,
0x66, 0x6f, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0e, 0x2e, 0x66, 0x6c, 0x6f, 0x77, 0x2e,
0x50, 0x6f, 0x72, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x48, 0x00, 0x52, 0x08, 0x70, 0x6f, 0x72, 0x74,
0x49, 0x6e, 0x66, 0x6f, 0x12, 0x2d, 0x0a, 0x09, 0x69, 0x63, 0x6d, 0x70, 0x5f, 0x69, 0x6e, 0x66,
0x6f, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0e, 0x2e, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x49,
0x43, 0x4d, 0x50, 0x49, 0x6e, 0x66, 0x6f, 0x48, 0x00, 0x52, 0x08, 0x69, 0x63, 0x6d, 0x70, 0x49,
0x6e, 0x66, 0x6f, 0x42, 0x11, 0x0a, 0x0f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f,
0x6e, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x22, 0x48, 0x0a, 0x08, 0x50, 0x6f, 0x72, 0x74, 0x49, 0x6e,
0x66, 0x6f, 0x12, 0x1f, 0x0a, 0x0b, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x70, 0x6f, 0x72,
0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0a, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x50,
0x6f, 0x72, 0x74, 0x12, 0x1b, 0x0a, 0x09, 0x64, 0x65, 0x73, 0x74, 0x5f, 0x70, 0x6f, 0x72, 0x74,
0x18, 0x02, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x08, 0x64, 0x65, 0x73, 0x74, 0x50, 0x6f, 0x72, 0x74,
0x22, 0x44, 0x0a, 0x08, 0x49, 0x43, 0x4d, 0x50, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x1b, 0x0a, 0x09,
0x69, 0x63, 0x6d, 0x70, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x01, 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, 0x02, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x08, 0x69, 0x63,
0x6d, 0x70, 0x43, 0x6f, 0x64, 0x65, 0x2a, 0x45, 0x0a, 0x04, 0x54, 0x79, 0x70, 0x65, 0x12, 0x10,
0x0a, 0x0c, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x55, 0x4e, 0x4b, 0x4e, 0x4f, 0x57, 0x4e, 0x10, 0x00,
0x12, 0x0e, 0x0a, 0x0a, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x53, 0x54, 0x41, 0x52, 0x54, 0x10, 0x01,
0x12, 0x0c, 0x0a, 0x08, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x45, 0x4e, 0x44, 0x10, 0x02, 0x12, 0x0d,
0x0a, 0x09, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x44, 0x52, 0x4f, 0x50, 0x10, 0x03, 0x2a, 0x3b, 0x0a,
0x09, 0x44, 0x69, 0x72, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x15, 0x0a, 0x11, 0x44, 0x49,
0x52, 0x45, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x55, 0x4e, 0x4b, 0x4e, 0x4f, 0x57, 0x4e, 0x10,
0x00, 0x12, 0x0b, 0x0a, 0x07, 0x49, 0x4e, 0x47, 0x52, 0x45, 0x53, 0x53, 0x10, 0x01, 0x12, 0x0a,
0x0a, 0x06, 0x45, 0x47, 0x52, 0x45, 0x53, 0x53, 0x10, 0x02, 0x32, 0x42, 0x0a, 0x0b, 0x46, 0x6c,
0x6f, 0x77, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x33, 0x0a, 0x06, 0x45, 0x76, 0x65,
0x6e, 0x74, 0x73, 0x12, 0x0f, 0x2e, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x46, 0x6c, 0x6f, 0x77, 0x45,
0x76, 0x65, 0x6e, 0x74, 0x1a, 0x12, 0x2e, 0x66, 0x6c, 0x6f, 0x77, 0x2e, 0x46, 0x6c, 0x6f, 0x77,
0x45, 0x76, 0x65, 0x6e, 0x74, 0x41, 0x63, 0x6b, 0x22, 0x00, 0x28, 0x01, 0x30, 0x01, 0x42, 0x08,
0x5a, 0x06, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
}
var (
@ -581,18 +591,18 @@ var file_flow_proto_goTypes = []interface{}{
(Direction)(0), // 1: flow.Direction
(*FlowEvent)(nil), // 2: flow.FlowEvent
(*FlowEventAck)(nil), // 3: flow.FlowEventAck
(*EventFields)(nil), // 4: flow.EventFields
(*FlowFields)(nil), // 4: flow.FlowFields
(*PortInfo)(nil), // 5: flow.PortInfo
(*ICMPInfo)(nil), // 6: flow.ICMPInfo
(*timestamppb.Timestamp)(nil), // 7: google.protobuf.Timestamp
}
var file_flow_proto_depIdxs = []int32{
7, // 0: flow.FlowEvent.timestamp:type_name -> google.protobuf.Timestamp
4, // 1: flow.FlowEvent.event_fields:type_name -> flow.EventFields
0, // 2: flow.EventFields.type:type_name -> flow.Type
1, // 3: flow.EventFields.direction:type_name -> flow.Direction
5, // 4: flow.EventFields.port_info:type_name -> flow.PortInfo
6, // 5: flow.EventFields.icmp_info:type_name -> flow.ICMPInfo
4, // 1: flow.FlowEvent.flow_fields:type_name -> flow.FlowFields
0, // 2: flow.FlowFields.type:type_name -> flow.Type
1, // 3: flow.FlowFields.direction:type_name -> flow.Direction
5, // 4: flow.FlowFields.port_info:type_name -> flow.PortInfo
6, // 5: flow.FlowFields.icmp_info:type_name -> flow.ICMPInfo
2, // 6: flow.FlowService.Events:input_type -> flow.FlowEvent
3, // 7: flow.FlowService.Events:output_type -> flow.FlowEventAck
7, // [7:8] is the sub-list for method output_type
@ -633,7 +643,7 @@ func file_flow_proto_init() {
}
}
file_flow_proto_msgTypes[2].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*EventFields); i {
switch v := v.(*FlowFields); i {
case 0:
return &v.state
case 1:
@ -670,8 +680,8 @@ func file_flow_proto_init() {
}
}
file_flow_proto_msgTypes[2].OneofWrappers = []interface{}{
(*EventFields_PortInfo)(nil),
(*EventFields_IcmpInfo)(nil),
(*FlowFields_PortInfo)(nil),
(*FlowFields_IcmpInfo)(nil),
}
type x struct{}
out := protoimpl.TypeBuilder{

View File

@ -15,16 +15,13 @@ message FlowEvent {
// Unique client event identifier
string event_id = 1;
// Unique client flow session identifier
string flow_id = 2;
// When the event occurred
google.protobuf.Timestamp timestamp = 3;
// Public key of the sending peer
bytes public_key = 4;
EventFields event_fields = 5;
FlowFields flow_fields = 5;
}
message FlowEventAck {
@ -32,29 +29,35 @@ message FlowEventAck {
string event_id = 1;
}
message EventFields {
// Event type
Type type = 1;
message FlowFields {
// Unique client flow session identifier
bytes flow_id = 1;
// Flow type
Type type = 2;
// RuleId identifies the rule that allowed or denied the connection
bytes rule_id = 3;
// Initiating traffic direction
Direction direction = 2;
Direction direction = 4;
// IP protocol number
uint32 protocol = 3;
uint32 protocol = 5;
// Source IP address
bytes source_ip = 4;
bytes source_ip = 6;
// Destination IP address
bytes dest_ip = 5;
bytes dest_ip = 7;
// Layer 4 -specific information
oneof connection_info {
// TCP/UDP port information
PortInfo port_info = 6;
PortInfo port_info = 8;
// ICMP type and code
ICMPInfo icmp_info = 7;
ICMPInfo icmp_info = 9;
}
}

View File

@ -2729,6 +2729,8 @@ type FirewallRule struct {
Protocol RuleProtocol `protobuf:"varint,4,opt,name=Protocol,proto3,enum=management.RuleProtocol" json:"Protocol,omitempty"`
Port string `protobuf:"bytes,5,opt,name=Port,proto3" json:"Port,omitempty"`
PortInfo *PortInfo `protobuf:"bytes,6,opt,name=PortInfo,proto3" json:"PortInfo,omitempty"`
// Id is the unique rule Id
Id []byte `protobuf:"bytes,7,opt,name=id,proto3" json:"id,omitempty"`
}
func (x *FirewallRule) Reset() {
@ -2805,6 +2807,13 @@ func (x *FirewallRule) GetPortInfo() *PortInfo {
return nil
}
func (x *FirewallRule) GetId() []byte {
if x != nil {
return x.Id
}
return nil
}
type NetworkAddress struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
@ -3010,6 +3019,8 @@ type RouteFirewallRule struct {
Domains []string `protobuf:"bytes,7,rep,name=domains,proto3" json:"domains,omitempty"`
// CustomProtocol is a custom protocol ID.
CustomProtocol uint32 `protobuf:"varint,8,opt,name=customProtocol,proto3" json:"customProtocol,omitempty"`
// Id is the unique rule Id
Id []byte `protobuf:"bytes,9,opt,name=id,proto3" json:"id,omitempty"`
}
func (x *RouteFirewallRule) Reset() {
@ -3100,6 +3111,13 @@ func (x *RouteFirewallRule) GetCustomProtocol() uint32 {
return 0
}
func (x *RouteFirewallRule) GetId() []byte {
if x != nil {
return x.Id
}
return nil
}
type ForwardingRule struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
@ -3110,7 +3128,6 @@ type ForwardingRule struct {
// portInfo is the ingress destination port information, where the traffic arrives in the gateway node
DestinationPort *PortInfo `protobuf:"bytes,2,opt,name=destinationPort,proto3" json:"destinationPort,omitempty"`
// IP address of the translated address (remote peer) to send traffic to
// todo type pending
TranslatedAddress []byte `protobuf:"bytes,3,opt,name=translatedAddress,proto3" json:"translatedAddress,omitempty"`
// Translated port information, where the traffic should be forwarded to
TranslatedPort *PortInfo `protobuf:"bytes,4,opt,name=translatedPort,proto3" json:"translatedPort,omitempty"`
@ -3605,7 +3622,7 @@ var file_management_proto_rawDesc = []byte{
0x0a, 0x02, 0x49, 0x50, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x49, 0x50, 0x12, 0x16,
0x0a, 0x06, 0x4e, 0x53, 0x54, 0x79, 0x70, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06,
0x4e, 0x53, 0x54, 0x79, 0x70, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x50, 0x6f, 0x72, 0x74, 0x18, 0x03,
0x20, 0x01, 0x28, 0x03, 0x52, 0x04, 0x50, 0x6f, 0x72, 0x74, 0x22, 0x8b, 0x02, 0x0a, 0x0c, 0x46,
0x20, 0x01, 0x28, 0x03, 0x52, 0x04, 0x50, 0x6f, 0x72, 0x74, 0x22, 0x9b, 0x02, 0x0a, 0x0c, 0x46,
0x69, 0x72, 0x65, 0x77, 0x61, 0x6c, 0x6c, 0x52, 0x75, 0x6c, 0x65, 0x12, 0x16, 0x0a, 0x06, 0x50,
0x65, 0x65, 0x72, 0x49, 0x50, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x50, 0x65, 0x65,
0x72, 0x49, 0x50, 0x12, 0x37, 0x0a, 0x09, 0x44, 0x69, 0x72, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e,
@ -3622,7 +3639,8 @@ var file_management_proto_rawDesc = []byte{
0x52, 0x04, 0x50, 0x6f, 0x72, 0x74, 0x12, 0x30, 0x0a, 0x08, 0x50, 0x6f, 0x72, 0x74, 0x49, 0x6e,
0x66, 0x6f, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67,
0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x50, 0x6f, 0x72, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x08,
0x50, 0x6f, 0x72, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x22, 0x38, 0x0a, 0x0e, 0x4e, 0x65, 0x74, 0x77,
0x50, 0x6f, 0x72, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x07,
0x20, 0x01, 0x28, 0x0c, 0x52, 0x02, 0x69, 0x64, 0x22, 0x38, 0x0a, 0x0e, 0x4e, 0x65, 0x74, 0x77,
0x6f, 0x72, 0x6b, 0x41, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x12, 0x14, 0x0a, 0x05, 0x6e, 0x65,
0x74, 0x49, 0x50, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x6e, 0x65, 0x74, 0x49, 0x50,
0x12, 0x10, 0x0a, 0x03, 0x6d, 0x61, 0x63, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6d,
@ -3637,7 +3655,7 @@ var file_management_proto_rawDesc = []byte{
0x67, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x72, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28,
0x0d, 0x52, 0x05, 0x73, 0x74, 0x61, 0x72, 0x74, 0x12, 0x10, 0x0a, 0x03, 0x65, 0x6e, 0x64, 0x18,
0x02, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x03, 0x65, 0x6e, 0x64, 0x42, 0x0f, 0x0a, 0x0d, 0x70, 0x6f,
0x72, 0x74, 0x53, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0xd1, 0x02, 0x0a, 0x11,
0x72, 0x74, 0x53, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0xe1, 0x02, 0x0a, 0x11,
0x52, 0x6f, 0x75, 0x74, 0x65, 0x46, 0x69, 0x72, 0x65, 0x77, 0x61, 0x6c, 0x6c, 0x52, 0x75, 0x6c,
0x65, 0x12, 0x22, 0x0a, 0x0c, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x52, 0x61, 0x6e, 0x67, 0x65,
0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0c, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x52,
@ -3658,7 +3676,8 @@ var file_management_proto_rawDesc = []byte{
0x07, 0x64, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x73, 0x18, 0x07, 0x20, 0x03, 0x28, 0x09, 0x52, 0x07,
0x64, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x73, 0x12, 0x26, 0x0a, 0x0e, 0x63, 0x75, 0x73, 0x74, 0x6f,
0x6d, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0d, 0x52,
0x0e, 0x63, 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x22,
0x0e, 0x63, 0x75, 0x73, 0x74, 0x6f, 0x6d, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x12,
0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x02, 0x69, 0x64, 0x22,
0xf2, 0x01, 0x0a, 0x0e, 0x46, 0x6f, 0x72, 0x77, 0x61, 0x72, 0x64, 0x69, 0x6e, 0x67, 0x52, 0x75,
0x6c, 0x65, 0x12, 0x34, 0x0a, 0x08, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x18, 0x01,
0x20, 0x01, 0x28, 0x0e, 0x32, 0x18, 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e,

View File

@ -445,6 +445,9 @@ message FirewallRule {
RuleProtocol Protocol = 4;
string Port = 5;
PortInfo PortInfo = 6;
// Id is the unique rule Id
bytes id = 7;
}
message NetworkAddress {
@ -494,6 +497,9 @@ message RouteFirewallRule {
// CustomProtocol is a custom protocol ID.
uint32 customProtocol = 8;
// Id is the unique rule Id
bytes id = 9;
}
message ForwardingRule {
@ -504,7 +510,6 @@ message ForwardingRule {
PortInfo destinationPort = 2;
// IP address of the translated address (remote peer) to send traffic to
// todo type pending
bytes translatedAddress = 3;
// Translated port information, where the traffic should be forwarded to