ipn/ipnlocal/netmapcache: add a package to split and cache network maps (#18497)

This commit is based on part of #17925, reworked as a separate package.

Add a package that can store and load netmap.NetworkMap values in persistent
storage, using a basic columnar representation. This commit includes a default
storage interface based on plain files, but the interface can be implemented
with more structured storage if we want to later.

The tests are set up to require that all the fields of the NetworkMap are
handled, except those explicitly designated as not-cached, and check that a
fully-populated value can round-trip correctly through the cache.  Adding or
removing fields, either in the NetworkMap or in the cached representation, will
trigger either build failures (e.g., for type mismatch) or test failures (e.g.,
for representation changes or missing fields). This isn't quite as nice as
automatically updating the representation, which I also prototyped, but is much
simpler to maintain and less code.

This commit does not yet hook up the cache to the backend, that will be a
subsequent change.

Updates #12639

Change-Id: Icb48639e1d61f2aec59904ecd172c73e05ba7bf9
Signed-off-by: M. J. Fromberger <fromberger@tailscale.com>
pull/14700/merge
M. J. Fromberger 1 day ago committed by GitHub
parent 6e44cb6ab3
commit 9385dfe7f6
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194

@ -151,4 +151,4 @@
});
};
}
# nix-direnv cache busting line: sha256-WeMTOkERj4hvdg4yPaZ1gRgKnhRIBXX55kUVbX/k/xM=
# nix-direnv cache busting line: sha256-+tOYqRV8ZUA95dfVyRpjnJvwuSMobu/EhtXxq4bwvio=

@ -23,6 +23,7 @@ require (
github.com/coder/websocket v1.8.12
github.com/coreos/go-iptables v0.7.1-0.20240112124308-65c67c9f46e6
github.com/coreos/go-systemd v0.0.0-20191104093116-d3cd4ed1dbcf
github.com/creachadair/mds v0.25.9
github.com/creachadair/msync v0.7.1
github.com/creachadair/taskgroup v0.13.2
github.com/creack/pty v1.1.23

@ -1 +1 @@
sha256-WeMTOkERj4hvdg4yPaZ1gRgKnhRIBXX55kUVbX/k/xM=
sha256-+tOYqRV8ZUA95dfVyRpjnJvwuSMobu/EhtXxq4bwvio=

@ -0,0 +1,351 @@
// Copyright (c) Tailscale Inc & contributors
// SPDX-License-Identifier: BSD-3-Clause
// Package netmapcache implements a persistent cache for [netmap.NetworkMap]
// values, allowing a client to start up using stale but previously-valid state
// even if a connection to the control plane is not immediately available.
package netmapcache
import (
"cmp"
"context"
"crypto/sha256"
"encoding/hex"
jsonv1 "encoding/json"
"errors"
"fmt"
"io/fs"
"iter"
"os"
"path/filepath"
"slices"
"strings"
"time"
"tailscale.com/feature/buildfeatures"
"tailscale.com/tailcfg"
"tailscale.com/types/netmap"
"tailscale.com/util/mak"
"tailscale.com/util/set"
)
var (
// ErrKeyNotFound is a sentinel error reported by implementations of the [Store]
// interface when loading a key that is not found in the store.
ErrKeyNotFound = errors.New("storage key not found")
// ErrCacheNotAvailable is a sentinel error reported by cache methods when
// the netmap caching feature is not enabled in the build.
ErrCacheNotAvailable = errors.New("netmap cache is not available")
)
// A Cache manages a columnar cache of a [netmap.NetworkMap]. Each Cache holds
// a single netmap value; use [Cache.Store] to update or replace the cached
// value and [Cache.Load] to read the cached value.
type Cache struct {
store Store
// wantKeys records the storage keys from the last write or load of a cached
// netmap. This is used to prune keys that are no longer referenced after an
// update.
wantKeys set.Set[string]
// lastWrote records the last values written to each stored key.
//
// TODO(creachadair): This is meant to avoid disk writes, but I'm not
// convinced we need it. Or maybe just track hashes of the content rather
// than caching a complete copy.
lastWrote map[string]lastWrote
}
// NewCache constructs a new empty [Cache] from the given [Store].
// It will panic if s == nil.
func NewCache(s Store) *Cache {
if s == nil {
panic("a non-nil Store is required")
}
return &Cache{
store: s,
wantKeys: make(set.Set[string]),
lastWrote: make(map[string]lastWrote),
}
}
type lastWrote struct {
digest string
at time.Time
}
func (c *Cache) writeJSON(ctx context.Context, key string, v any) error {
j, err := jsonv1.Marshal(v)
if err != nil {
return fmt.Errorf("JSON marshalling %q: %w", key, err)
}
// TODO(creachadair): Maybe use a hash instead of the contents? Do we need
// this at all?
last, ok := c.lastWrote[key]
if ok && cacheDigest(j) == last.digest {
return nil
}
if err := c.store.Store(ctx, key, j); err != nil {
return err
}
// Track the storage keys the current map is using, for storage GC.
c.wantKeys.Add(key)
c.lastWrote[key] = lastWrote{
digest: cacheDigest(j),
at: time.Now(),
}
return nil
}
func (c *Cache) removeUnwantedKeys(ctx context.Context) error {
var errs []error
for key, err := range c.store.List(ctx, "") {
if err != nil {
errs = append(errs, err)
break
}
if !c.wantKeys.Contains(key) {
if err := c.store.Remove(ctx, key); err != nil {
errs = append(errs, fmt.Errorf("remove key %q: %w", key, err))
}
delete(c.lastWrote, key) // even if removal failed, we don't want it
}
}
return errors.Join(errs...)
}
// FileStore implements the [Store] interface using a directory of files, in
// which each key is encoded as a filename in the directory.
// The caller is responsible to ensure the directory path exists before
// using the store methods.
type FileStore string
// List implements part of the [Store] interface.
func (s FileStore) List(ctx context.Context, prefix string) iter.Seq2[string, error] {
return func(yield func(string, error) bool) {
des, err := os.ReadDir(string(s))
if os.IsNotExist(err) {
return // nothing to read
} else if err != nil {
yield("", err)
return
}
// os.ReadDir reports entries already sorted, and the encoding preserves that.
for _, de := range des {
key, err := hex.DecodeString(de.Name())
if err != nil {
yield("", err)
return
}
name := string(key)
if !strings.HasPrefix(name, prefix) {
continue
} else if !yield(name, nil) {
return
}
}
}
}
// Load implements part of the [Store] interface.
func (s FileStore) Load(ctx context.Context, key string) ([]byte, error) {
return os.ReadFile(filepath.Join(string(s), hex.EncodeToString([]byte(key))))
}
// Store implements part of the [Store] interface.
func (s FileStore) Store(ctx context.Context, key string, value []byte) error {
return os.WriteFile(filepath.Join(string(s), hex.EncodeToString([]byte(key))), value, 0600)
}
// Remove implements part of the [Store] interface.
func (s FileStore) Remove(ctx context.Context, key string) error {
err := os.Remove(filepath.Join(string(s), hex.EncodeToString([]byte(key))))
if errors.Is(err, fs.ErrNotExist) {
return nil
}
return err
}
// Store records nm in the cache, replacing any previously-cached values.
func (c *Cache) Store(ctx context.Context, nm *netmap.NetworkMap) error {
if !buildfeatures.HasCacheNetMap || nm == nil || nm.Cached {
return nil
}
if selfID := nm.User(); selfID == 0 {
return errors.New("no user in netmap")
}
clear(c.wantKeys)
if err := c.writeJSON(ctx, "misc", netmapMisc{
MachineKey: &nm.MachineKey,
CollectServices: &nm.CollectServices,
DisplayMessages: &nm.DisplayMessages,
TKAEnabled: &nm.TKAEnabled,
TKAHead: &nm.TKAHead,
Domain: &nm.Domain,
DomainAuditLogID: &nm.DomainAuditLogID,
}); err != nil {
return err
}
if err := c.writeJSON(ctx, "dns", netmapDNS{DNS: &nm.DNS}); err != nil {
return err
}
if err := c.writeJSON(ctx, "derpmap", netmapDERPMap{DERPMap: &nm.DERPMap}); err != nil {
return err
}
if err := c.writeJSON(ctx, "self", netmapNode{Node: &nm.SelfNode}); err != nil {
return err
// N.B. The NodeKey and AllCaps fields can be recovered from SelfNode on
// load, and do not need to be stored separately.
}
for _, p := range nm.Peers {
key := fmt.Sprintf("peer-%s", p.StableID())
if err := c.writeJSON(ctx, key, netmapNode{Node: &p}); err != nil {
return err
}
}
for uid, u := range nm.UserProfiles {
key := fmt.Sprintf("user-%d", uid)
if err := c.writeJSON(ctx, key, netmapUserProfile{UserProfile: &u}); err != nil {
return err
}
}
if buildfeatures.HasSSH && nm.SSHPolicy != nil {
if err := c.writeJSON(ctx, "ssh", netmapSSH{SSHPolicy: &nm.SSHPolicy}); err != nil {
return err
}
}
return c.removeUnwantedKeys(ctx)
}
// Load loads the cached [netmap.NetworkMap] value stored in c, if one is available.
// It reports [ErrCacheNotAvailable] if no cached data are available.
// On success, the Cached field of the returned network map is true.
func (c *Cache) Load(ctx context.Context) (*netmap.NetworkMap, error) {
if !buildfeatures.HasCacheNetMap {
return nil, ErrCacheNotAvailable
}
nm := netmap.NetworkMap{Cached: true}
// At minimum, we require that the cache contain a "self" node, or the data
// are not usable.
if self, err := c.store.Load(ctx, "self"); errors.Is(err, ErrKeyNotFound) {
return nil, ErrCacheNotAvailable
} else if err := jsonv1.Unmarshal(self, &netmapNode{Node: &nm.SelfNode}); err != nil {
return nil, err
}
c.wantKeys.Add("self")
// If we successfully recovered a SelfNode, pull out its related fields.
if s := nm.SelfNode; s.Valid() {
nm.NodeKey = s.Key()
nm.AllCaps = make(set.Set[tailcfg.NodeCapability])
for _, c := range s.Capabilities().All() {
nm.AllCaps.Add(c)
}
for c := range s.CapMap().All() {
nm.AllCaps.Add(c)
}
}
// Unmarshal the contents of each specified cache entry directly into the
// fields of the output. See the comment in types.go for more detail.
if err := c.readJSON(ctx, "misc", &netmapMisc{
MachineKey: &nm.MachineKey,
CollectServices: &nm.CollectServices,
DisplayMessages: &nm.DisplayMessages,
TKAEnabled: &nm.TKAEnabled,
TKAHead: &nm.TKAHead,
Domain: &nm.Domain,
DomainAuditLogID: &nm.DomainAuditLogID,
}); err != nil {
return nil, err
}
if err := c.readJSON(ctx, "dns", &netmapDNS{DNS: &nm.DNS}); err != nil {
return nil, err
}
if err := c.readJSON(ctx, "derpmap", &netmapDERPMap{DERPMap: &nm.DERPMap}); err != nil {
return nil, err
}
for key, err := range c.store.List(ctx, "peer-") {
if err != nil {
return nil, err
}
var peer tailcfg.NodeView
if err := c.readJSON(ctx, key, &netmapNode{Node: &peer}); err != nil {
return nil, err
}
nm.Peers = append(nm.Peers, peer)
}
slices.SortFunc(nm.Peers, func(a, b tailcfg.NodeView) int { return cmp.Compare(a.ID(), b.ID()) })
for key, err := range c.store.List(ctx, "user-") {
if err != nil {
return nil, err
}
var up tailcfg.UserProfileView
if err := c.readJSON(ctx, key, &netmapUserProfile{UserProfile: &up}); err != nil {
return nil, err
}
mak.Set(&nm.UserProfiles, up.ID(), up)
}
if err := c.readJSON(ctx, "ssh", &netmapSSH{SSHPolicy: &nm.SSHPolicy}); err != nil {
return nil, err
}
return &nm, nil
}
func (c *Cache) readJSON(ctx context.Context, key string, value any) error {
data, err := c.store.Load(ctx, key)
if errors.Is(err, ErrKeyNotFound) {
return nil
} else if err != nil {
return err
}
if err := jsonv1.Unmarshal(data, value); err != nil {
return err
}
c.wantKeys.Add(key)
c.lastWrote[key] = lastWrote{digest: cacheDigest(data), at: time.Now()}
return nil
}
// Store is the interface to persistent key-value storage used by a [Cache].
type Store interface {
// List lists all the stored keys having the specified prefixes, in
// lexicographic order.
//
// Each pair yielded by the iterator is either a valid storage key and a nil
// error, or an empty key and a non-nil error. After reporting an error, the
// iterator must immediately return.
List(ctx context.Context, prefix string) iter.Seq2[string, error]
// Load fetches the contents of the specified key.
// If the key is not found in the store, Load must report [ErrKeyNotFound].
Load(ctx context.Context, key string) ([]byte, error)
// Store marshals and stores the contents of the specified value under key.
// If the key already exists, its contents are replaced.
Store(ctx context.Context, key string, value []byte) error
// Remove removes the specified key from the store. If the key does not exist,
// Remove reports success (nil).
Remove(ctx context.Context, key string) error
}
// cacheDigest computes a string digest of the specified data, for use in
// detecting cache hits.
func cacheDigest(data []byte) string { h := sha256.Sum256(data); return string(h[:]) }

@ -0,0 +1,298 @@
// Copyright (c) Tailscale Inc & contributors
// SPDX-License-Identifier: BSD-3-Clause
package netmapcache_test
import (
"context"
"errors"
"flag"
"fmt"
"iter"
"os"
"reflect"
"slices"
"strings"
"testing"
"github.com/creachadair/mds/mtest"
"github.com/google/go-cmp/cmp"
"github.com/google/go-cmp/cmp/cmpopts"
"tailscale.com/ipn/ipnlocal/netmapcache"
"tailscale.com/tailcfg"
"tailscale.com/tka"
"tailscale.com/types/key"
"tailscale.com/types/netmap"
"tailscale.com/types/views"
"tailscale.com/util/set"
)
// Input values for valid-looking placeholder values for keys, hashes, etc.
const (
testNodeKeyString = "nodekey:0123456789abcdef0123456789abcdef0123456789abcdef0123456789abcdef"
testMachineKeyString = "mkey:fedcba9876543210fedcba9876543210fedcba9876543210fedcba9876543210"
testAUMHashString = "APPLEPEARPLUMCHERRYAPPLEPEARPLUMCHERRYAPPLEPEARPLUMA" // base32, no padding
)
var keepTestOutput = flag.String("keep-output", "", "directory to keep test output (if empty, use a test temp)")
var (
testNode1 = (&tailcfg.Node{
ID: 99001,
StableID: "n99001FAKE",
Name: "test1.example.com.",
}).View()
testNode2 = (&tailcfg.Node{
ID: 99002,
StableID: "n99002FAKE",
Name: "test2.example.com.",
}).View()
// The following fields are set in init.
testNodeKey key.NodePublic
testMachineKey key.MachinePublic
testAUMHash tka.AUMHash
testMap *netmap.NetworkMap
)
func init() {
if err := testNodeKey.UnmarshalText([]byte(testNodeKeyString)); err != nil {
panic(fmt.Sprintf("invalid test nodekey %q: %v", testNodeKeyString, err))
}
if err := testMachineKey.UnmarshalText([]byte(testMachineKeyString)); err != nil {
panic(fmt.Sprintf("invalid test machine key %q: %v", testMachineKeyString, err))
}
if err := testAUMHash.UnmarshalText([]byte(testAUMHashString)); err != nil {
panic(fmt.Sprintf("invalid test AUM hash %q: %v", testAUMHashString, err))
}
// The following network map must have a non-zero non-empty value for every
// field that is to be stored in the cache. The test checks for this using
// reflection, as a way to ensure that new fields added to the type are
// covered by a test (see checkFieldCoverage).
//
// The exact values are unimportant, except that they should be values that
// give us confidence that a network map round-tripped through the cache and
// compared will accurately reflect the information we care about.
testMap = &netmap.NetworkMap{
Cached: false, // not cached, this is metadata for the cache machinery
PacketFilter: nil, // not cached
PacketFilterRules: views.Slice[tailcfg.FilterRule]{}, // not cached
// Fields stored under the "self" key.
// Note that SelfNode must have a valid user in order to be considered
// cacheable. Moreover, it must mention all the capabilities we expect
// to see advertised in the AllCaps set, and its public key must match the
// one advertised in the NodeKey field.
SelfNode: (&tailcfg.Node{
ID: 12345,
StableID: "n12345FAKE",
User: 30337,
Name: "test.example.com.",
Key: testNodeKey,
Capabilities: []tailcfg.NodeCapability{"cap1"},
CapMap: map[tailcfg.NodeCapability][]tailcfg.RawMessage{
"cap2": nil,
},
}).View(),
AllCaps: set.Of[tailcfg.NodeCapability]("cap1", "cap2"),
NodeKey: testNodeKey,
DNS: tailcfg.DNSConfig{Domains: []string{"example1.com", "example2.ac.uk"}}, // "dns"
SSHPolicy: &tailcfg.SSHPolicy{Rules: []*tailcfg.SSHRule{{ // "ssh"
SSHUsers: map[string]string{"amelie": "ubuntu"},
Action: &tailcfg.SSHAction{Message: "hello", Accept: true},
AcceptEnv: []string{"MAGIC_SSH_*"},
}}},
DERPMap: &tailcfg.DERPMap{ // "derp"
HomeParams: &tailcfg.DERPHomeParams{
RegionScore: map[int]float64{10: 0.31, 20: 0.141, 30: 0.592},
},
OmitDefaultRegions: true,
},
// Peers stored under "peer-<stableID>" keys.
Peers: []tailcfg.NodeView{testNode1, testNode2},
// Profiles stored under "user-<id>" keys.
UserProfiles: map[tailcfg.UserID]tailcfg.UserProfileView{
12345: (&tailcfg.UserProfile{ID: 12345, DisplayName: "me"}).View(),
67890: (&tailcfg.UserProfile{ID: 67890, DisplayName: "you"}).View(),
},
// Fields stored under "misc"
MachineKey: testMachineKey,
CollectServices: true,
DisplayMessages: map[tailcfg.DisplayMessageID]tailcfg.DisplayMessage{
"test-message-1": {Title: "hello", Text: "this is your wakeup call"},
"test-message-2": {Title: "goodbye", Text: "good night", ImpactsConnectivity: true},
},
TKAEnabled: true,
TKAHead: testAUMHash,
Domain: "example.com",
DomainAuditLogID: "0f1e2d3c4b5a67890f1e2d3c4b5a67890f1e2d3c4b5a67890f1e2d3c4b5a6789",
}
}
func TestNewStore(t *testing.T) {
mtest.MustPanicf(t, func() { netmapcache.NewCache(nil) }, "NewCache should panic for a nil store")
}
func TestRoundTrip(t *testing.T) {
checkFieldCoverage(t, testMap)
dir := *keepTestOutput
if dir == "" {
dir = t.TempDir()
} else if err := os.MkdirAll(dir, 0700); err != nil {
t.Fatalf("Create --keep-output directory: %v", err)
}
tests := []struct {
name string
store netmapcache.Store
}{
{"MemStore", make(testStore)},
{"FileStore", netmapcache.FileStore(dir)},
}
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
c := netmapcache.NewCache(tt.store)
if err := c.Store(t.Context(), testMap); err != nil {
t.Fatalf("Store netmap failed; %v", err)
}
cmap, err := c.Load(t.Context())
if err != nil {
t.Fatalf("Load netmap failed: %v", err)
}
if !cmap.Cached {
t.Error("Cached map is not marked as such")
}
opts := []cmp.Option{
cmpopts.IgnoreFields(netmap.NetworkMap{}, skippedMapFields...),
cmpopts.EquateComparable(key.NodePublic{}, key.MachinePublic{}),
}
if diff := cmp.Diff(cmap, testMap, opts...); diff != "" {
t.Fatalf("Cached map differs (-got, +want):\n%s", diff)
}
})
}
}
func TestInvalidCache(t *testing.T) {
t.Run("Empty", func(t *testing.T) {
c := netmapcache.NewCache(make(testStore))
got, err := c.Load(t.Context())
if !errors.Is(err, netmapcache.ErrCacheNotAvailable) {
t.Errorf("Load from empty cache: got %+v, %v; want nil, %v", got, err, netmapcache.ErrCacheNotAvailable)
}
})
t.Run("Incomplete", func(t *testing.T) {
s := make(testStore)
c := netmapcache.NewCache(s)
if err := c.Store(t.Context(), testMap); err != nil {
t.Fatalf("Store initial netmap: %v", err)
}
// Drop the "self" node from the cache, and verify it makes the results
// unloadable.
if err := s.Remove(t.Context(), "self"); err != nil {
t.Fatalf("Remove self: %v", err)
}
got, err := c.Load(t.Context())
if !errors.Is(err, netmapcache.ErrCacheNotAvailable) {
t.Errorf("Load from invalid cache: got %+v, %v; want nil, %v", got, err, netmapcache.ErrCacheNotAvailable)
}
})
}
// skippedMapFields are the names of fields that should not be considered by
// network map caching, and thus skipped when comparing test results.
var skippedMapFields = []string{
"Cached", "PacketFilter", "PacketFilterRules",
}
// checkFieldCoverage logs an error in t if any of the fields of nm are zero
// valued, except those listed in skippedMapFields.
//
// This ensures if any new fields are added to the [netmap.NetworkMap] type in
// the future, the test will fail until non-trivial test data are added to this
// test, or the fields are recorded as skipped. It also helps ensure that
// changing the field types or deleting fields will make compilation fail, so
// the tests get updated.
func checkFieldCoverage(t *testing.T, nm *netmap.NetworkMap) {
t.Helper()
mt := reflect.TypeOf(nm).Elem()
mv := reflect.ValueOf(nm).Elem()
for i := 0; i < mt.NumField(); i++ {
f := mt.Field(i)
if slices.Contains(skippedMapFields, f.Name) {
continue
}
fv := mv.Field(i)
if fv.IsZero() {
t.Errorf("Field %d (%q) of test value is zero (%+v). "+
"A non-zero value is required for each cached field in the test value.",
i, f.Name, fv.Interface())
}
}
// Verify that skip-listed fields exist on the type. FieldByName thwarts the
// linker, but it's OK in a test.
for _, skip := range skippedMapFields {
if _, ok := mt.FieldByName(skip); !ok {
t.Errorf("Skipped field %q not found on type %T. "+
"If a field was deleted from the type, you may need to update skippedMapFields.",
skip, nm)
}
}
if t.Failed() {
t.FailNow()
}
}
// testStore is an in-memory implementation of the [netmapcache.Store] interface.
type testStore map[string][]byte
func (t testStore) List(_ context.Context, prefix string) iter.Seq2[string, error] {
var matching []string
for key := range t {
if strings.HasPrefix(key, prefix) {
matching = append(matching, key)
}
}
slices.Sort(matching)
return func(yield func(string, error) bool) {
for _, key := range matching {
if !yield(key, nil) {
return
}
}
}
}
func (t testStore) Load(_ context.Context, key string) ([]byte, error) {
val, ok := t[key]
if !ok {
return nil, netmapcache.ErrKeyNotFound
}
return val, nil
}
func (t testStore) Store(_ context.Context, key string, value []byte) error {
t[key] = value
return nil
}
func (t testStore) Remove(_ context.Context, key string) error { delete(t, key); return nil }

@ -0,0 +1,52 @@
// Copyright (c) Tailscale Inc & contributors
// SPDX-License-Identifier: BSD-3-Clause
package netmapcache
import (
"tailscale.com/tailcfg"
"tailscale.com/tka"
"tailscale.com/types/key"
)
// The fields in the following wrapper types are all pointers, even when their
// target type is also a pointer, so that they can be used to unmarshal
// directly into the fields of another value. These wrappers intentionally do
// not omit zero or empty values, since we want the cache to reflect the value
// the object had at the time it was written, even if the default changes
// later.
//
// Moreover, these are all struct types so that each cached record will be a
// JSON object even if the underlying value marshals to an array or primitive
// type, and so that we have a seam if we want to replace or version the cached
// representation separately from the default JSON layout.
type netmapMisc struct {
MachineKey *key.MachinePublic
CollectServices *bool
DisplayMessages *map[tailcfg.DisplayMessageID]tailcfg.DisplayMessage
TKAEnabled *bool
TKAHead *tka.AUMHash
Domain *string
DomainAuditLogID *string
}
type netmapSSH struct {
SSHPolicy **tailcfg.SSHPolicy
}
type netmapDNS struct {
DNS *tailcfg.DNSConfig
}
type netmapDERPMap struct {
DERPMap **tailcfg.DERPMap
}
type netmapNode struct {
Node *tailcfg.NodeView
}
type netmapUserProfile struct {
UserProfile *tailcfg.UserProfileView
}

@ -16,4 +16,4 @@
) {
src = ./.;
}).shellNix
# nix-direnv cache busting line: sha256-WeMTOkERj4hvdg4yPaZ1gRgKnhRIBXX55kUVbX/k/xM=
# nix-direnv cache busting line: sha256-+tOYqRV8ZUA95dfVyRpjnJvwuSMobu/EhtXxq4bwvio=

@ -27,6 +27,8 @@ import (
// The fields should all be considered read-only. They might
// alias parts of previous NetworkMap values.
type NetworkMap struct {
Cached bool // whether this NetworkMap was loaded from disk cache (as opposed to live from network)
SelfNode tailcfg.NodeView
AllCaps set.Set[tailcfg.NodeCapability] // set version of SelfNode.Capabilities + SelfNode.CapMap
NodeKey key.NodePublic

Loading…
Cancel
Save