ipn/ipnlocal: add a C2N endpoint for fetching a netmap

For debugging purposes, add a new C2N endpoint returning the current
netmap. Optionally, coordination server can send a new "candidate" map
response, which the client will generate a separate netmap for.
Coordination server can later compare two netmaps, detecting unexpected
changes to the client state.

Updates tailscale/corp#32095

Signed-off-by: Anton Tolchanov <anton@tailscale.com>
This commit is contained in:
Anton Tolchanov 2025-08-13 15:00:35 +01:00 committed by Anton Tolchanov
parent 394718a4ca
commit 4a04161828
9 changed files with 506 additions and 9 deletions

View File

@ -1160,6 +1160,27 @@ func (c *Direct) sendMapRequest(ctx context.Context, isStreaming bool, nu Netmap
return nil
}
// NetmapFromMapResponseForDebug returns a NetworkMap from the given MapResponse.
// It is intended for debugging only.
func NetmapFromMapResponseForDebug(ctx context.Context, pr persist.PersistView, resp *tailcfg.MapResponse) (*netmap.NetworkMap, error) {
if resp == nil {
return nil, errors.New("nil MapResponse")
}
if resp.Node == nil {
return nil, errors.New("MapResponse lacks Node")
}
nu := &rememberLastNetmapUpdater{}
sess := newMapSession(pr.PrivateNodeKey(), nu, nil)
defer sess.Close()
if err := sess.HandleNonKeepAliveMapResponse(ctx, resp); err != nil {
return nil, fmt.Errorf("HandleNonKeepAliveMapResponse: %w", err)
}
return sess.netmap(), nil
}
func (c *Direct) handleDebugMessage(ctx context.Context, debug *tailcfg.Debug) error {
if code := debug.Exit; code != nil {
c.logf("exiting process with status %v per controlplane", *code)

View File

@ -20,6 +20,7 @@ import (
"go4.org/mem"
"tailscale.com/control/controlknobs"
"tailscale.com/health"
"tailscale.com/ipn"
"tailscale.com/tailcfg"
"tailscale.com/tstest"
"tailscale.com/tstime"
@ -27,6 +28,7 @@ import (
"tailscale.com/types/key"
"tailscale.com/types/logger"
"tailscale.com/types/netmap"
"tailscale.com/types/persist"
"tailscale.com/types/ptr"
"tailscale.com/util/eventbus/eventbustest"
"tailscale.com/util/mak"
@ -1419,3 +1421,27 @@ func TestNetmapDisplayMessageIntegration(t *testing.T) {
t.Errorf("unexpected message contents (-want +got):\n%s", diff)
}
}
func TestNetmapForMapResponseForDebug(t *testing.T) {
mr := &tailcfg.MapResponse{
Node: &tailcfg.Node{
ID: 1,
Name: "foo.bar.ts.net.",
},
Peers: []*tailcfg.Node{
{ID: 2, Name: "peer1.bar.ts.net.", HomeDERP: 1},
{ID: 3, Name: "peer2.bar.ts.net.", HomeDERP: 1},
},
}
ms := newTestMapSession(t, nil)
nm1 := ms.netmapForResponse(mr)
prefs := &ipn.Prefs{Persist: &persist.Persist{PrivateNodeKey: ms.privateNodeKey}}
nm2, err := NetmapFromMapResponseForDebug(t.Context(), prefs.View().Persist(), mr)
if err != nil {
t.Fatal(err)
}
if !reflect.DeepEqual(nm1, nm2) {
t.Errorf("mismatch\nnm1: %s\nnm2: %s\n", logger.AsJSON(nm1), logger.AsJSON(nm2))
}
}

View File

@ -13,19 +13,23 @@ import (
"os/exec"
"path"
"path/filepath"
"reflect"
"runtime"
"strconv"
"strings"
"time"
"tailscale.com/clientupdate"
"tailscale.com/control/controlclient"
"tailscale.com/envknob"
"tailscale.com/ipn"
"tailscale.com/net/sockstats"
"tailscale.com/posture"
"tailscale.com/tailcfg"
"tailscale.com/types/netmap"
"tailscale.com/util/clientmetric"
"tailscale.com/util/goroutines"
"tailscale.com/util/httpm"
"tailscale.com/util/set"
"tailscale.com/util/syspolicy/pkey"
"tailscale.com/util/syspolicy/ptype"
@ -44,6 +48,7 @@ var c2nHandlers = map[methodAndPath]c2nHandler{
req("/debug/metrics"): handleC2NDebugMetrics,
req("/debug/component-logging"): handleC2NDebugComponentLogging,
req("/debug/logheap"): handleC2NDebugLogHeap,
req("/debug/netmap"): handleC2NDebugNetMap,
// PPROF - We only expose a subset of typical pprof endpoints for security.
req("/debug/pprof/heap"): handleC2NPprof,
@ -142,6 +147,66 @@ func handleC2NLogtailFlush(b *LocalBackend, w http.ResponseWriter, r *http.Reque
}
}
func handleC2NDebugNetMap(b *LocalBackend, w http.ResponseWriter, r *http.Request) {
ctx := r.Context()
if r.Method != httpm.POST && r.Method != httpm.GET {
http.Error(w, "method not allowed", http.StatusMethodNotAllowed)
return
}
b.logf("c2n: %s /debug/netmap received", r.Method)
// redactAndMarshal redacts private keys from the given netmap, clears fields
// that should be omitted, and marshals it to JSON.
redactAndMarshal := func(nm *netmap.NetworkMap, omitFields []string) (json.RawMessage, error) {
for _, f := range omitFields {
field := reflect.ValueOf(nm).Elem().FieldByName(f)
if !field.IsValid() {
b.logf("c2n: /debug/netmap: unknown field %q in omitFields", f)
continue
}
field.SetZero()
}
nm, _ = redactNetmapPrivateKeys(nm)
return json.Marshal(nm)
}
var omitFields []string
resp := &tailcfg.C2NDebugNetmapResponse{}
if r.Method == httpm.POST {
var req tailcfg.C2NDebugNetmapRequest
if err := json.NewDecoder(r.Body).Decode(&req); err != nil {
http.Error(w, fmt.Sprintf("failed to decode request body: %v", err), http.StatusBadRequest)
return
}
omitFields = req.OmitFields
if req.Candidate != nil {
cand, err := controlclient.NetmapFromMapResponseForDebug(ctx, b.unsanitizedPersist(), req.Candidate)
if err != nil {
http.Error(w, fmt.Sprintf("failed to convert candidate MapResponse: %v", err), http.StatusBadRequest)
return
}
candJSON, err := redactAndMarshal(cand, omitFields)
if err != nil {
http.Error(w, fmt.Sprintf("failed to marshal candidate netmap: %v", err), http.StatusInternalServerError)
return
}
resp.Candidate = candJSON
}
}
var err error
resp.Current, err = redactAndMarshal(b.currentNode().netMapWithPeers(), omitFields)
if err != nil {
http.Error(w, fmt.Sprintf("failed to marshal current netmap: %v", err), http.StatusInternalServerError)
return
}
writeJSON(w, resp)
}
func handleC2NDebugGoroutines(_ *LocalBackend, w http.ResponseWriter, r *http.Request) {
w.Header().Set("Content-Type", "text/plain")
w.Write(goroutines.ScrubbedGoroutineDump(true))

View File

@ -4,9 +4,11 @@
package ipnlocal
import (
"bytes"
"cmp"
"crypto/x509"
"encoding/json"
"fmt"
"net/http/httptest"
"net/url"
"os"
@ -18,8 +20,15 @@ import (
"tailscale.com/ipn/store/mem"
"tailscale.com/tailcfg"
"tailscale.com/tstest"
"tailscale.com/types/key"
"tailscale.com/types/logger"
"tailscale.com/types/netmap"
"tailscale.com/types/opt"
"tailscale.com/types/views"
"tailscale.com/util/must"
gcmp "github.com/google/go-cmp/cmp"
"github.com/google/go-cmp/cmp/cmpopts"
)
func TestHandleC2NTLSCertStatus(t *testing.T) {
@ -132,3 +141,177 @@ func TestHandleC2NTLSCertStatus(t *testing.T) {
}
}
// reflectNonzero returns a non-zero value for a given reflect.Value.
func reflectNonzero(t reflect.Type) reflect.Value {
switch t.Kind() {
case reflect.Bool:
return reflect.ValueOf(true)
case reflect.String:
if reflect.TypeFor[opt.Bool]() == t {
return reflect.ValueOf("true").Convert(t)
}
return reflect.ValueOf("foo").Convert(t)
case reflect.Int64:
return reflect.ValueOf(int64(1)).Convert(t)
case reflect.Slice:
return reflect.MakeSlice(t, 1, 1)
case reflect.Ptr:
return reflect.New(t.Elem())
case reflect.Map:
return reflect.MakeMap(t)
case reflect.Struct:
switch t {
case reflect.TypeFor[key.NodePrivate]():
return reflect.ValueOf(key.NewNode())
}
}
panic(fmt.Sprintf("unhandled %v", t))
}
// setFieldsToRedact sets fields in the given netmap to non-zero values
// according to the fieldMap, which maps field names to whether they
// should be reset (true) or not (false).
func setFieldsToRedact(t *testing.T, nm *netmap.NetworkMap, fieldMap map[string]bool) {
t.Helper()
v := reflect.ValueOf(nm).Elem()
for i := range v.NumField() {
name := v.Type().Field(i).Name
f := v.Field(i)
if !f.CanSet() {
continue
}
shouldReset, ok := fieldMap[name]
if !ok {
t.Errorf("fieldMap missing field %q", name)
}
if shouldReset {
f.Set(reflectNonzero(f.Type()))
}
}
}
func TestRedactNetmapPrivateKeys(t *testing.T) {
fieldMap := map[string]bool{
// Private fields (should be redacted):
"PrivateKey": true,
// Public fields (should not be redacted):
"AllCaps": false,
"CollectServices": false,
"DERPMap": false,
"DNS": false,
"DisplayMessages": false,
"Domain": false,
"DomainAuditLogID": false,
"Expiry": false,
"MachineKey": false,
"Name": false,
"NodeKey": false,
"PacketFilter": false,
"PacketFilterRules": false,
"Peers": false,
"SSHPolicy": false,
"SelfNode": false,
"TKAEnabled": false,
"TKAHead": false,
"UserProfiles": false,
}
nm := &netmap.NetworkMap{}
setFieldsToRedact(t, nm, fieldMap)
got, _ := redactNetmapPrivateKeys(nm)
if !reflect.DeepEqual(got, &netmap.NetworkMap{}) {
t.Errorf("redacted netmap is not empty: %+v", got)
}
}
func TestHandleC2NDebugNetmap(t *testing.T) {
nm := &netmap.NetworkMap{
Name: "myhost",
SelfNode: (&tailcfg.Node{
ID: 100,
Name: "myhost",
StableID: "deadbeef",
Key: key.NewNode().Public(),
Hostinfo: (&tailcfg.Hostinfo{Hostname: "myhost"}).View(),
}).View(),
Peers: []tailcfg.NodeView{
(&tailcfg.Node{
ID: 101,
Name: "peer1",
StableID: "deadbeef",
Key: key.NewNode().Public(),
Hostinfo: (&tailcfg.Hostinfo{Hostname: "peer1"}).View(),
}).View(),
},
PrivateKey: key.NewNode(),
}
withoutPrivateKey := *nm
withoutPrivateKey.PrivateKey = key.NodePrivate{}
for _, tt := range []struct {
name string
req *tailcfg.C2NDebugNetmapRequest
want *netmap.NetworkMap
}{
{
name: "simple_get",
want: &withoutPrivateKey,
},
{
name: "post_no_omit",
req: &tailcfg.C2NDebugNetmapRequest{},
want: &withoutPrivateKey,
},
{
name: "post_omit_peers_and_name",
req: &tailcfg.C2NDebugNetmapRequest{OmitFields: []string{"Peers", "Name"}},
want: &netmap.NetworkMap{
SelfNode: nm.SelfNode,
},
},
{
name: "post_omit_nonexistent_field",
req: &tailcfg.C2NDebugNetmapRequest{OmitFields: []string{"ThisFieldDoesNotExist"}},
want: &withoutPrivateKey,
},
} {
t.Run(tt.name, func(t *testing.T) {
b := newTestLocalBackend(t)
b.currentNode().SetNetMap(nm)
rec := httptest.NewRecorder()
req := httptest.NewRequest("GET", "/debug/netmap", nil)
if tt.req != nil {
b, err := json.Marshal(tt.req)
if err != nil {
t.Fatalf("json.Marshal: %v", err)
}
req = httptest.NewRequest("POST", "/debug/netmap", bytes.NewReader(b))
}
handleC2NDebugNetMap(b, rec, req)
res := rec.Result()
wantStatus := 200
if res.StatusCode != wantStatus {
t.Fatalf("status code = %v; want %v. Body: %s", res.Status, wantStatus, rec.Body.Bytes())
}
var resp tailcfg.C2NDebugNetmapResponse
if err := json.Unmarshal(rec.Body.Bytes(), &resp); err != nil {
t.Fatalf("bad JSON: %v", err)
}
got := &netmap.NetworkMap{}
if err := json.Unmarshal(resp.Current, got); err != nil {
t.Fatalf("bad JSON: %v", err)
}
if diff := gcmp.Diff(tt.want, got,
gcmp.AllowUnexported(netmap.NetworkMap{}, key.NodePublic{}, views.Slice[tailcfg.FilterRule]{}),
cmpopts.EquateComparable(key.MachinePublic{}),
); diff != "" {
t.Errorf("netmap mismatch (-want +got):\n%s", diff)
}
})
}
}

View File

@ -1223,6 +1223,13 @@ func (b *LocalBackend) sanitizedPrefsLocked() ipn.PrefsView {
return stripKeysFromPrefs(b.pm.CurrentPrefs())
}
// unsanitizedPersist returns the current PersistView, including any private keys.
func (b *LocalBackend) unsanitizedPersist() persist.PersistView {
b.mu.Lock()
defer b.mu.Unlock()
return b.pm.CurrentPrefs().Persist()
}
// Status returns the latest status of the backend and its
// sub-components.
func (b *LocalBackend) Status() *ipnstate.Status {
@ -3257,21 +3264,34 @@ func (b *LocalBackend) WatchNotificationsAs(ctx context.Context, actor ipnauth.A
// listener.
func filterPrivateKeys(fn func(roNotify *ipn.Notify) (keepGoing bool)) func(*ipn.Notify) bool {
return func(n *ipn.Notify) bool {
if n.NetMap == nil || n.NetMap.PrivateKey.IsZero() {
redacted, changed := redactNetmapPrivateKeys(n.NetMap)
if !changed {
return fn(n)
}
// The netmap in n is shared across all watchers, so to mutate it for a
// single watcher we have to clone the notify and the netmap. We can
// make shallow clones, at least.
nm2 := *n.NetMap
n2 := *n
n2.NetMap = &nm2
n2.NetMap.PrivateKey = key.NodePrivate{}
n2.NetMap = redacted
return fn(&n2)
}
}
// redactNetmapPrivateKeys returns a copy of nm with private keys zeroed out.
// If no change was needed, it returns nm unmodified.
func redactNetmapPrivateKeys(nm *netmap.NetworkMap) (redacted *netmap.NetworkMap, changed bool) {
if nm == nil || nm.PrivateKey.IsZero() {
return nm, false
}
// The netmap might be shared across watchers, so make at least a shallow
// clone before mutating it.
nm2 := *nm
nm2.PrivateKey = key.NodePrivate{}
return &nm2, true
}
// appendHealthActions returns an IPN listener func that wraps the supplied IPN
// listener func and transforms health messages passed to the wrapped listener.
// If health messages with PrimaryActions are present, it appends the label &

View File

@ -5,7 +5,10 @@
package tailcfg
import "net/netip"
import (
"encoding/json"
"net/netip"
)
// C2NSSHUsernamesRequest is the request for the /ssh/usernames.
// A GET request without a request body is equivalent to the zero value of this type.
@ -117,3 +120,29 @@ type C2NVIPServicesResponse struct {
// changes. This value matches what is reported in latest [Hostinfo.ServicesHash].
ServicesHash string
}
// C2NDebugNetmapRequest is the request (from control to node) for the
// /debug/netmap handler.
type C2NDebugNetmapRequest struct {
// Candidate is an optional full MapResponse to be used for generating a candidate
// network map. If unset, only the current network map is returned.
Candidate *MapResponse `json:"candidate,omitzero"`
// OmitFields is an optional list of netmap fields to omit from the response.
// If unset, no fields are omitted.
OmitFields []string `json:"omitFields,omitzero"`
}
// C2NDebugNetmapResponse is the response (from node to control) from the
// /debug/netmap handler. It contains the current network map and, if a
// candidate full MapResponse was provided in the request, a candidate network
// map generated from it.
// To avoid import cycles, and reflect the non-stable nature of
// netmap.NetworkMap values, they are returned as json.RawMessage.
type C2NDebugNetmapResponse struct {
// Current is the current network map (netmap.NetworkMap).
Current json.RawMessage `json:"current"`
// Candidate is a network map produced based on the candidate MapResponse.
Candidate json.RawMessage `json:"candidate,omitzero"`
}

View File

@ -171,7 +171,8 @@ type CapabilityVersion int
// - 124: 2025-08-08: removed NodeAttrDisableMagicSockCryptoRouting support, crypto routing is now mandatory
// - 125: 2025-08-11: dnstype.Resolver adds UseWithExitNode field.
// - 126: 2025-09-17: Client uses seamless key renewal unless disabled by control (tailscale/corp#31479)
const CurrentCapabilityVersion CapabilityVersion = 126
// - 127: 2025-09-19: can handle C2N /debug/netmap.
const CurrentCapabilityVersion CapabilityVersion = 127
// ID is an integer ID for a user, node, or login allocated by the
// control plane.

View File

@ -27,6 +27,7 @@ import (
"testing"
"time"
"github.com/google/go-cmp/cmp"
"github.com/miekg/dns"
"go4.org/mem"
"tailscale.com/client/local"
@ -41,6 +42,7 @@ import (
"tailscale.com/tstest"
"tailscale.com/tstest/integration/testcontrol"
"tailscale.com/types/key"
"tailscale.com/types/netmap"
"tailscale.com/types/opt"
"tailscale.com/types/ptr"
"tailscale.com/util/must"
@ -1623,3 +1625,146 @@ func TestPeerRelayPing(t *testing.T) {
}
}
}
func TestC2NDebugNetmap(t *testing.T) {
tstest.Shard(t)
tstest.Parallel(t)
env := NewTestEnv(t)
var testNodes []*TestNode
var nodes []*tailcfg.Node
for i := range 2 {
n := NewTestNode(t, env)
d := n.StartDaemon()
defer d.MustCleanShutdown(t)
n.AwaitResponding()
n.MustUp()
n.AwaitRunning()
testNodes = append(testNodes, n)
controlNodes := env.Control.AllNodes()
if len(controlNodes) != i+1 {
t.Fatalf("expected %d nodes, got %d nodes", i+1, len(controlNodes))
}
for _, cn := range controlNodes {
if n.MustStatus().Self.PublicKey == cn.Key {
nodes = append(nodes, cn)
break
}
}
}
// getC2NNetmap fetches the current netmap. If a candidate map response is provided,
// a candidate netmap is also fetched and compared to the current netmap.
getC2NNetmap := func(node key.NodePublic, cand *tailcfg.MapResponse) *netmap.NetworkMap {
t.Helper()
ctx, cancel := context.WithTimeout(t.Context(), 5*time.Second)
defer cancel()
var req *http.Request
if cand != nil {
body := must.Get(json.Marshal(&tailcfg.C2NDebugNetmapRequest{Candidate: cand}))
req = must.Get(http.NewRequestWithContext(ctx, "POST", "/debug/netmap", bytes.NewReader(body)))
} else {
req = must.Get(http.NewRequestWithContext(ctx, "GET", "/debug/netmap", nil))
}
httpResp := must.Get(env.Control.NodeRoundTripper(node).RoundTrip(req))
defer httpResp.Body.Close()
if httpResp.StatusCode != 200 {
t.Errorf("unexpected status code: %d", httpResp.StatusCode)
return nil
}
respBody := must.Get(io.ReadAll(httpResp.Body))
var resp tailcfg.C2NDebugNetmapResponse
must.Do(json.Unmarshal(respBody, &resp))
var current netmap.NetworkMap
must.Do(json.Unmarshal(resp.Current, &current))
if !current.PrivateKey.IsZero() {
t.Errorf("current netmap has non-zero private key: %v", current.PrivateKey)
}
// Check candidate netmap if we sent a map response.
if cand != nil {
var candidate netmap.NetworkMap
must.Do(json.Unmarshal(resp.Candidate, &candidate))
if !candidate.PrivateKey.IsZero() {
t.Errorf("candidate netmap has non-zero private key: %v", candidate.PrivateKey)
}
if diff := cmp.Diff(current.SelfNode, candidate.SelfNode); diff != "" {
t.Errorf("SelfNode differs (-current +candidate):\n%s", diff)
}
if diff := cmp.Diff(current.Peers, candidate.Peers); diff != "" {
t.Errorf("Peers differ (-current +candidate):\n%s", diff)
}
}
return &current
}
for _, n := range nodes {
mr := must.Get(env.Control.MapResponse(&tailcfg.MapRequest{NodeKey: n.Key}))
nm := getC2NNetmap(n.Key, mr)
// Make sure peers do not have "testcap" initially (we'll change this later).
if len(nm.Peers) != 1 || nm.Peers[0].CapMap().Contains("testcap") {
t.Fatalf("expected 1 peer without testcap, got: %v", nm.Peers)
}
// Make sure nodes think each other are offline initially.
if nm.Peers[0].Online().Get() {
t.Fatalf("expected 1 peer to be offline, got: %v", nm.Peers)
}
}
// Send a delta update to n0, setting "testcap" on node 1.
env.Control.AddRawMapResponse(nodes[0].Key, &tailcfg.MapResponse{
PeersChangedPatch: []*tailcfg.PeerChange{{
NodeID: nodes[1].ID, CapMap: tailcfg.NodeCapMap{"testcap": []tailcfg.RawMessage{}},
}},
})
// node 0 should see node 1 with "testcap".
must.Do(tstest.WaitFor(5*time.Second, func() error {
st := testNodes[0].MustStatus()
p, ok := st.Peer[nodes[1].Key]
if !ok {
return fmt.Errorf("node 0 (%s) doesn't see node 1 (%s) as peer\n%v", nodes[0].Key, nodes[1].Key, st)
}
if _, ok := p.CapMap["testcap"]; !ok {
return fmt.Errorf("node 0 (%s) sees node 1 (%s) as peer but without testcap\n%v", nodes[0].Key, nodes[1].Key, p)
}
return nil
}))
// Check that node 0's current netmap has "testcap" for node 1.
nm := getC2NNetmap(nodes[0].Key, nil)
if len(nm.Peers) != 1 || !nm.Peers[0].CapMap().Contains("testcap") {
t.Errorf("current netmap missing testcap: %v", nm.Peers[0].CapMap())
}
// Send a delta update to n1, marking node 0 as online.
env.Control.AddRawMapResponse(nodes[1].Key, &tailcfg.MapResponse{
PeersChangedPatch: []*tailcfg.PeerChange{{
NodeID: nodes[0].ID, Online: ptr.To(true),
}},
})
// node 1 should see node 0 as online.
must.Do(tstest.WaitFor(5*time.Second, func() error {
st := testNodes[1].MustStatus()
p, ok := st.Peer[nodes[0].Key]
if !ok || !p.Online {
return fmt.Errorf("node 0 (%s) doesn't see node 1 (%s) as an online peer\n%v", nodes[0].Key, nodes[1].Key, st)
}
return nil
}))
// The netmap from node 1 should show node 0 as online.
nm = getC2NNetmap(nodes[1].Key, nil)
if len(nm.Peers) != 1 || !nm.Peers[0].Online().Get() {
t.Errorf("expected peer to be online; got %+v", nm.Peers[0].AsStruct())
}
}

View File

@ -336,7 +336,7 @@ func (s *Server) serveUnhandled(w http.ResponseWriter, r *http.Request) {
func (s *Server) serveC2N(w http.ResponseWriter, r *http.Request) {
if err := func() error {
if r.Method != httpm.POST {
return fmt.Errorf("POST required")
return errors.New("POST required")
}
token, ok := strings.CutPrefix(r.URL.Path, "/c2n/")
if !ok {
@ -1148,18 +1148,25 @@ func (s *Server) canGenerateAutomaticMapResponseFor(nk key.NodePublic) bool {
func (s *Server) hasPendingRawMapMessage(nk key.NodePublic) bool {
s.mu.Lock()
defer s.mu.Unlock()
_, ok := s.msgToSend[nk].(*tailcfg.MapResponse)
_, ok := s.msgToSend[nk]
return ok
}
func (s *Server) takeRawMapMessage(nk key.NodePublic) (mapResJSON []byte, ok bool) {
s.mu.Lock()
defer s.mu.Unlock()
mr, ok := s.msgToSend[nk].(*tailcfg.MapResponse)
mr, ok := s.msgToSend[nk]
if !ok {
return nil, false
}
delete(s.msgToSend, nk)
// If it's a bare PingRequest, wrap it in a MapResponse.
switch pr := mr.(type) {
case *tailcfg.PingRequest:
mr = &tailcfg.MapResponse{PingRequest: pr}
}
var err error
mapResJSON, err = json.Marshal(mr)
if err != nil {