Skip to content

Commit

Permalink
bpf, ipcache: unconditionally assume LPM trie delete/dump support
Browse files Browse the repository at this point in the history
Delete [1] and dump [2] operations on LPM trie maps are supported since
Linux kernel version 4.16. For Cilium v1.14, the minimum required kernel
version is 4.19.57 [3].  Thus we can now unconditionally assume support
for delete and dump operations on the ipcache map.

[1] torvalds/linux@e454cf5
[2] torvalds/linux@b471f2f
[3] https://docs.cilium.io/en/latest/operations/system_requirements/#base-requirements

For #22116

Signed-off-by: Tobias Klauser <tobias@cilium.io>
  • Loading branch information
tklauser committed Mar 15, 2023
1 parent a6b99de commit 9108e0d
Show file tree
Hide file tree
Showing 3 changed files with 18 additions and 254 deletions.
134 changes: 18 additions & 116 deletions pkg/datapath/ipcache/listener.go
Original file line number Diff line number Diff line change
Expand Up @@ -7,7 +7,6 @@ import (
"context"
"fmt"
"net"
"os"
"sync"
"time"

Expand Down Expand Up @@ -171,7 +170,7 @@ func (l *BPFListener) OnIPIdentityCacheChange(modType ipcache.CacheModification,
}).Warning("unable to update bpf map")
}
case ipcache.Delete:
err := l.bpfMap.DeleteWithOverwrite(&key)
err := l.bpfMap.Delete(&key)
if err != nil {
scopedLog.WithError(err).WithFields(logrus.Fields{
"key": key.String(),
Expand Down Expand Up @@ -207,132 +206,35 @@ func (l *BPFListener) updateStaleEntriesFunction(keysToRemove map[string]*ipcach
}
}

// handleMapShuffleFailure attempts to move the map with name 'backup' back to
// 'realized', and logs a warning message if this can't be achieved.
func handleMapShuffleFailure(src, dst string) {
backupPath := bpf.MapPath(src)
realizedPath := bpf.MapPath(dst)

if err := os.Rename(backupPath, realizedPath); err != nil {
log.WithError(err).WithFields(logrus.Fields{
logfields.BPFMapPath: realizedPath,
}).Warningf("Unable to recover during error renaming map paths")
}
}

// shuffleMaps attempts to move the map with name 'realized' to 'backup' and
// 'pending' to 'realized'. If an error occurs, attempts to return the maps
// back to their original paths.
func shuffleMaps(realized, backup, pending string) error {
realizedPath := bpf.MapPath(realized)
backupPath := bpf.MapPath(backup)
pendingPath := bpf.MapPath(pending)

if err := os.Rename(realizedPath, backupPath); err != nil && !os.IsNotExist(err) {
return fmt.Errorf("Unable to back up existing ipcache: %s", err)
}

if err := os.Rename(pendingPath, realizedPath); err != nil {
handleMapShuffleFailure(backup, realized)
return fmt.Errorf("Unable to shift ipcache into new location: %s", err)
}

return nil
}

// garbageCollect implements GC of the ipcache map in one of two ways:
//
// On Linux 4.9, 4.10 or 4.16 and later:
// garbageCollect implements GC of the ipcache map in the following way:
//
// Periodically sweep through every element in the BPF map and check it
// against the in-memory copy of the map. If it doesn't exist in memory,
// delete the entry.
//
// On Linux 4.11 to 4.15:
//
// Create a brand new map, populate it with all of the IPCache entries from
// the in-memory cache, delete the old map, and trigger regeneration of all
// BPF programs so that they pick up the new map.
//
// Returns an error if garbage collection failed to occur.
func (l *BPFListener) garbageCollect(ctx context.Context) (*sync.WaitGroup, error) {
log.Debug("Running garbage collection for BPF IPCache")

if ipcacheMap.SupportsDelete() {
// Since controllers run asynchronously, need to make sure
// IPIdentityCache is not being updated concurrently while we
// do GC;
l.ipcache.RLock()
defer l.ipcache.RUnlock()
// Since controllers run asynchronously, need to make sure
// IPIdentityCache is not being updated concurrently while we
// do GC;
l.ipcache.RLock()
defer l.ipcache.RUnlock()

keysToRemove := map[string]*ipcacheMap.Key{}
if err := l.bpfMap.DumpWithCallback(l.updateStaleEntriesFunction(keysToRemove)); err != nil {
return nil, fmt.Errorf("error dumping ipcache BPF map: %s", err)
}

// Remove all keys which are not in in-memory cache from BPF map
// for consistency.
for _, k := range keysToRemove {
log.WithFields(logrus.Fields{logfields.BPFMapKey: k}).
Debug("deleting from ipcache BPF map")
if err := l.bpfMap.DeleteWithOverwrite(k); err != nil {
return nil, fmt.Errorf("error deleting key %s from ipcache BPF map: %s", k, err)
}
}
} else {
// Since controllers run asynchronously, need to make sure
// IPIdentityCache is not being updated concurrently while we
// do GC;
l.ipcache.RLock()

// Populate the map at the new path
pendingMapName := fmt.Sprintf("%s_pending", ipcacheMap.Name)
pendingMap := ipcacheMap.NewMap(pendingMapName)
if _, err := pendingMap.OpenOrCreate(); err != nil {
l.ipcache.RUnlock()
return nil, fmt.Errorf("Unable to create %s map: %s", pendingMapName, err)
}
pendingListener := newListener(pendingMap, l.datapath, nil, l.ipcache)
l.ipcache.DumpToListenerLocked(pendingListener)
err := pendingMap.Close()
if err != nil {
log.WithError(err).WithField("map-name", pendingMapName).Warning("unable to close map")
}

// Move the maps around on the filesystem so that BPF reload
// will pick up the new paths without requiring recompilation.
backupMapName := fmt.Sprintf("%s_old", ipcacheMap.Name)
if err := shuffleMaps(ipcacheMap.Name, backupMapName, pendingMapName); err != nil {
l.ipcache.RUnlock()
return nil, err
}

// Reopen the ipcache map so that new writes and reads will use
// the new map
if err := ipcacheMap.Reopen(); err != nil {
handleMapShuffleFailure(backupMapName, ipcacheMap.Name)
l.ipcache.RUnlock()
return nil, err
}

// Unlock the ipcache as in order for
// TriggerReloadWithoutCompile() to succeed, other endpoint
// regenerations which are blocking on the ipcache lock may
// need to succeed first (#11946)
l.ipcache.RUnlock()
keysToRemove := map[string]*ipcacheMap.Key{}
if err := l.bpfMap.DumpWithCallback(l.updateStaleEntriesFunction(keysToRemove)); err != nil {
return nil, fmt.Errorf("error dumping ipcache BPF map: %s", err)
}

wg, err := l.datapath.TriggerReloadWithoutCompile("datapath ipcache")
if err != nil {
// We can't really undo the map rename again as ipcache
// operations had already been permitted so the backup
// map is potentially outdated. Fail hard to restart
// the agent so we reconstruct the ipcache from
// scratch.
log.WithError(err).Fatal("Endpoint datapath reload triggered by ipcache GC failed. Inconsistent state.")
// Remove all keys which are not in in-memory cache from BPF map
// for consistency.
for _, k := range keysToRemove {
log.WithFields(logrus.Fields{logfields.BPFMapKey: k}).
Debug("deleting from ipcache BPF map")
if err := l.bpfMap.Delete(k); err != nil {
return nil, fmt.Errorf("error deleting key %s from ipcache BPF map: %s", k, err)
}

_ = os.RemoveAll(bpf.MapPath(backupMapName))
return wg, nil
}
return nil, nil
}
Expand Down
91 changes: 0 additions & 91 deletions pkg/maps/ipcache/ipcache.go
Original file line number Diff line number Diff line change
Expand Up @@ -4,25 +4,18 @@
package ipcache

import (
"errors"
"fmt"
"net"
"net/netip"
"sync"
"unsafe"

"golang.org/x/sys/unix"

"github.com/cilium/cilium/pkg/bpf"
cmtypes "github.com/cilium/cilium/pkg/clustermesh/types"
"github.com/cilium/cilium/pkg/logging"
"github.com/cilium/cilium/pkg/logging/logfields"
"github.com/cilium/cilium/pkg/option"
"github.com/cilium/cilium/pkg/types"
)

var log = logging.DefaultLogger.WithField(logfields.LogSubsys, "map-ipcache")

const (
// MaxEntries is the maximum number of keys that can be present in the
// RemoteEndpointMap.
Expand Down Expand Up @@ -167,15 +160,6 @@ func (v *RemoteEndpointInfo) GetValuePtr() unsafe.Pointer { return unsafe.Pointe
// Map represents an IPCache BPF map.
type Map struct {
bpf.Map

// detectDeleteSupport is used to initialize 'supportsDelete' the first
// time that a delete is issued from the datapath.
detectDeleteSupport sync.Once

// deleteSupport is set to 'true' initially, then is updated to set
// whether the underlying kernel supports delete operations on the map
// the first time that supportsDelete() is called.
deleteSupport bool
}

func newIPCacheMap(name string) *bpf.Map {
Expand All @@ -196,36 +180,7 @@ func NewMap(name string) *Map {
return &Map{
Map: *newIPCacheMap(name).WithCache().WithPressureMetric().
WithEvents(option.Config.GetEventBufferConfig(name)),
deleteSupport: true,
}
}

// delete removes a key from the ipcache BPF map, and returns whether the
// kernel supports the delete operation (true) or not (false), and any error
// that may have occurred while attempting to delete the entry.
//
// If "overwrite" is true, then if delete is not supported the entry's value
// will be overwritten with zeroes to signify that it's an invalid entry.
func (m *Map) delete(k bpf.MapKey, overwrite bool) (bool, error) {
err := m.Delete(k)
var errno unix.Errno
if ok := errors.As(err, &errno); ok && errno == unix.ENOSYS {
if overwrite {
// Older kernels do not support deletion of LPM map entries so zero out
// the entry instead of attempting a deletion
return false, m.Update(k, &RemoteEndpointInfo{})
}
return false, nil
}
return true, err
}

// DeleteWithOverwrite removes a key from the ipcache BPF map.
// If delete is not supported, the entry's value will be overwritten with
// zeroes to signify that it's an invalid entry.
func (m *Map) DeleteWithOverwrite(k bpf.MapKey) error {
_, err := m.delete(k, true)
return err
}

// GetMaxPrefixLengths determines how many unique prefix lengths are supported
Expand All @@ -234,52 +189,6 @@ func (m *Map) GetMaxPrefixLengths() (ipv6, ipv4 int) {
return net.IPv6len*8 + 1, net.IPv4len*8 + 1
}

func (m *Map) supportsDelete() bool {
m.detectDeleteSupport.Do(func() {
// Create a separate map for the probing since this map may not have been created yet.
probeMap := newIPCacheMap(m.Name() + "_probe")
err := probeMap.CreateUnpinned()
if err != nil {
log.WithError(err).Warn("Failed to open IPCache map for feature probing, assuming delete and dump unsupported")
m.deleteSupport = false
return
}
defer probeMap.Close()

// Entry is invalid because IPCache needs a family specified.
invalidEntry := &Key{}
err = probeMap.Delete(invalidEntry)
var errno unix.Errno
if ok := errors.As(err, &errno); ok && errno == unix.ENOSYS {
m.deleteSupport = false
} else {
m.deleteSupport = true
}
log.Debugf("Detected IPCache delete operation support: %t", m.deleteSupport)

// Detect dump support
err = probeMap.Dump(map[string][]string{})
dumpSupport := err == nil
log.Debugf("Detected IPCache dump operation support: %t", dumpSupport)

// In addition to delete support, ability to dump the map is
// also required in order to run the garbage collector which
// will iterate over the map and delete entries.
m.deleteSupport = m.deleteSupport && dumpSupport

if !m.deleteSupport {
log.Infof("Periodic IPCache map swap will occur due to lack of kernel support for LPM delete operation. Upgrade to Linux 4.15 or higher to avoid this.")
}
})
return m.deleteSupport
}

// SupportsDelete determines whether the underlying kernel map type supports
// the delete operation.
func SupportsDelete() bool {
return IPCacheMap().supportsDelete()
}

var (
// IPCache is a mapping of all endpoint IPs in the cluster which this
// Cilium agent is a part of to their corresponding security identities.
Expand Down
47 changes: 0 additions & 47 deletions pkg/maps/ipcache/ipcache_privileged_test.go

This file was deleted.

0 comments on commit 9108e0d

Please sign in to comment.