m/n/c/network: fix panic when DHCP lease expires

The statusCallback of the network service previously accessed
new.AssignedIP without checking if new is nil, which caused a panic when
the DHCP lease expired. When subsequently the DHCP service was restarted
and a new lease obtained, CoreDNS was left without upstream servers
configured. The reason for this is that just before the panic, CoreDNS
was configured with an empty list of upstreams, but the lease field of
the DHCP service was not updated. When the lease callback was called
again with the new lease, old and new lease had the same DNS servers, so
CoreDNS was not configured to use the upstreams.

To fix the panic, this adds a check for a nil lease before accessing
AssignedIP. I looked if all consumers of the ExternalAddress Status can
handle nil, and added a nil check in the statuspush worker. The
apiserver stops when the lease is lost, and starts again once it is
reacquired; I'm not sure if this is the intended behavior.

The DNS problem occured because the old lease passed to the callback was
not the last lease that the callback had seen, and it then mistakenly
suppressed the update. In general, a callback cannot rely on the old
lease being the last lease that the callback has been called with. For
example, when a callback earlier in the Compose chain returns an error,
later callbacks are not called, so a callback may not see all lease
changes. Because the old lease parameter cannot be trusted, I removed
it. Callbacks which need the previous lease should keep track of it
themselves.

For manually testing lease expiry, I modified
metropolis/test/nanoswitch/nanoswitch.go like this:

+		start := time.Now()
 		server, err := server4.NewServer(link.Attrs().Name, &laddr, func(conn net.PacketConn, peer net.Addr, m *dhcpv4.DHCPv4) {
 			if m == nil {
 				return
 			}
+			if start.Add(50*time.Second).Before(time.Now()) && start.Add(90*time.Second).After(time.Now()) {
+				supervisor.Logger(ctx).Infof("Dropping DHCP packet")
+				return
+			}
 			reply, err := dhcpv4.NewReplyFromRequest(m)

Change-Id: Ifa0c039769c37ee53033ce013eed4f1af6f02142
Reviewed-on: https://review.monogon.dev/c/monogon/+/3214
Tested-by: Jenkins CI
Reviewed-by: Lorenz Brun <lorenz@monogon.tech>
diff --git a/metropolis/node/core/network/dhcp4c/callback/callback.go b/metropolis/node/core/network/dhcp4c/callback/callback.go
index cc21b4f..2acc539 100644
--- a/metropolis/node/core/network/dhcp4c/callback/callback.go
+++ b/metropolis/node/core/network/dhcp4c/callback/callback.go
@@ -43,9 +43,9 @@
 
 // Compose can be used to chain multiple callbacks
 func Compose(callbacks ...dhcp4c.LeaseCallback) dhcp4c.LeaseCallback {
-	return func(old, new *dhcp4c.Lease) error {
+	return func(lease *dhcp4c.Lease) error {
 		for _, cb := range callbacks {
-			if err := cb(old, new); err != nil {
+			if err := cb(lease); err != nil {
 				return err
 			}
 		}
@@ -70,8 +70,8 @@
 // IFA_F_PERMANENT set, so it's not possible to run multiple dynamic addressing
 // clients on a single interface.
 func ManageIP(iface netlink.Link) dhcp4c.LeaseCallback {
-	return func(old, new *dhcp4c.Lease) error {
-		newNet := new.IPNet()
+	return func(lease *dhcp4c.Lease) error {
+		newNet := lease.IPNet()
 
 		addrs, err := netlink.AddrList(iface, netlink.FAMILY_V4)
 		if err != nil {
@@ -85,7 +85,7 @@
 				// So don't touch addresses which match on these properties as
 				// AddrReplace will atomically reconfigure them anyways without
 				// interrupting things.
-				if isIPNetEqual(addr.IPNet, newNet) && addr.Peer == nil && new != nil {
+				if isIPNetEqual(addr.IPNet, newNet) && addr.Peer == nil && lease != nil {
 					continue
 				}
 
@@ -95,10 +95,9 @@
 			}
 		}
 
-		if new != nil {
-
-			remainingLifetimeSecs := int(math.Ceil(time.Until(new.ExpiresAt).Seconds()))
-			newBroadcastIP := dhcpv4.GetIP(dhcpv4.OptionBroadcastAddress, new.Options)
+		if lease != nil {
+			remainingLifetimeSecs := int(math.Ceil(time.Until(lease.ExpiresAt).Seconds()))
+			newBroadcastIP := dhcpv4.GetIP(dhcpv4.OptionBroadcastAddress, lease.Options)
 			if err := netlink.AddrReplace(iface, &netlink.Addr{
 				IPNet:       newNet,
 				ValidLft:    remainingLifetimeSecs,
@@ -117,8 +116,8 @@
 // It takes ownership of all RTPROTO_DHCP routes on the given interface, so it's
 // not possible to run multiple DHCP clients on the given interface.
 func ManageRoutes(iface netlink.Link) dhcp4c.LeaseCallback {
-	return func(old, new *dhcp4c.Lease) error {
-		newRoutes := new.Routes()
+	return func(lease *dhcp4c.Lease) error {
+		newRoutes := lease.Routes()
 
 		dhcpRoutes, err := netlink.RouteListFiltered(netlink.FAMILY_V4, &netlink.Route{
 			Protocol:  unix.RTPROT_DHCP,
@@ -155,7 +154,7 @@
 				Protocol:  unix.RTPROT_DHCP,
 				Dst:       route.Dest,
 				Gw:        route.Router,
-				Src:       new.AssignedIP,
+				Src:       lease.AssignedIP,
 				LinkIndex: iface.Attrs().Index,
 				Scope:     netlink.SCOPE_UNIVERSE,
 			}
diff --git a/metropolis/node/core/network/dhcp4c/callback/callback_test.go b/metropolis/node/core/network/dhcp4c/callback/callback_test.go
index a1c459a..e83596e 100644
--- a/metropolis/node/core/network/dhcp4c/callback/callback_test.go
+++ b/metropolis/node/core/network/dhcp4c/callback/callback_test.go
@@ -58,24 +58,22 @@
 	}
 
 	var tests = []struct {
-		name               string
-		initialAddrs       []netlink.Addr
-		oldLease, newLease *dhcp4c.Lease
-		expectedAddrs      []netlink.Addr
+		name          string
+		initialAddrs  []netlink.Addr
+		newLease      *dhcp4c.Lease
+		expectedAddrs []netlink.Addr
 	}{
 		// Lifetimes are necessary, otherwise the Kernel sets the
 		// IFA_F_PERMANENT flag behind our back.
 		{
 			name:          "RemoveOldIPs",
 			initialAddrs:  []netlink.Addr{{IPNet: &testNet1, ValidLft: 60}, {IPNet: &testNet2, ValidLft: 60}},
-			oldLease:      nil,
 			newLease:      nil,
 			expectedAddrs: nil,
 		},
 		{
 			name:         "IgnoresPermanentIPs",
 			initialAddrs: []netlink.Addr{{IPNet: &testNet1, Flags: unix.IFA_F_PERMANENT}, {IPNet: &testNet2, ValidLft: 60}},
-			oldLease:     nil,
 			newLease:     trivialLeaseFromNet(testNet2),
 			expectedAddrs: []netlink.Addr{
 				{IPNet: &testNet1, Flags: unix.IFA_F_PERMANENT, ValidLft: math.MaxUint32, PreferedLft: math.MaxUint32, Broadcast: testNet1Broadcast},
@@ -85,7 +83,6 @@
 		{
 			name:         "AssignsNewIP",
 			initialAddrs: []netlink.Addr{},
-			oldLease:     nil,
 			newLease:     trivialLeaseFromNet(testNet2),
 			expectedAddrs: []netlink.Addr{
 				{IPNet: &testNet2, ValidLft: 1, PreferedLft: 1, Broadcast: testNet2Broadcast},
@@ -94,7 +91,6 @@
 		{
 			name:         "UpdatesIP",
 			initialAddrs: []netlink.Addr{},
-			oldLease:     trivialLeaseFromNet(testNet2),
 			newLease:     trivialLeaseFromNet(testNet1),
 			expectedAddrs: []netlink.Addr{
 				{IPNet: &testNet1, ValidLft: 1, PreferedLft: 1, Broadcast: testNet1Broadcast},
@@ -103,7 +99,6 @@
 		{
 			name:          "RemovesIPOnRelease",
 			initialAddrs:  []netlink.Addr{{IPNet: &testNet1, ValidLft: 60, PreferedLft: 60}},
-			oldLease:      trivialLeaseFromNet(testNet1),
 			newLease:      nil,
 			expectedAddrs: nil,
 		},
@@ -131,7 +126,7 @@
 				test.expectedAddrs[i].LinkIndex = testLink.Index
 			}
 			cb := ManageIP(testLink)
-			if err := cb(test.oldLease, test.newLease); err != nil {
+			if err := cb(test.newLease); err != nil {
 				t.Fatalf("callback returned an error: %v", err)
 			}
 			addrs, err := netlink.AddrList(testLink, netlink.FAMILY_V4)
@@ -179,15 +174,14 @@
 	// correct link index for this test interface at runtime for both
 	// initialRoutes and expectedRoutes.
 	var tests = []struct {
-		name               string
-		initialRoutes      []netlink.Route
-		oldLease, newLease *dhcp4c.Lease
-		expectedRoutes     []netlink.Route
+		name           string
+		initialRoutes  []netlink.Route
+		newLease       *dhcp4c.Lease
+		expectedRoutes []netlink.Route
 	}{
 		{
 			name:          "AddsDefaultRoute",
 			initialRoutes: []netlink.Route{},
-			oldLease:      nil,
 			newLease:      leaseAddRouter(trivialLeaseFromNet(testNet1), testNet1Router),
 			expectedRoutes: []netlink.Route{{
 				Protocol: unix.RTPROT_DHCP,
@@ -205,7 +199,6 @@
 		{
 			name:           "IgnoresLeasesWithoutRouter",
 			initialRoutes:  []netlink.Route{},
-			oldLease:       nil,
 			newLease:       trivialLeaseFromNet(testNet1),
 			expectedRoutes: nil,
 		},
@@ -219,7 +212,6 @@
 				Gw:        testNet2Router,
 				Scope:     netlink.SCOPE_UNIVERSE,
 			}},
-			oldLease:       nil,
 			newLease:       nil,
 			expectedRoutes: nil,
 		},
@@ -232,7 +224,6 @@
 				Protocol:  unix.RTPROT_BIRD,
 				Gw:        testNet2Router,
 			}},
-			oldLease: trivialLeaseFromNet(testNet1),
 			newLease: nil,
 			expectedRoutes: []netlink.Route{{
 				Protocol:  unix.RTPROT_BIRD,
@@ -253,7 +244,6 @@
 				Protocol:  unix.RTPROT_DHCP,
 				Gw:        testNet2Router,
 			}},
-			oldLease:       leaseAddRouter(trivialLeaseFromNet(testNet2), testNet2Router),
 			newLease:       nil,
 			expectedRoutes: nil,
 		},
@@ -267,7 +257,6 @@
 				Src:       testNet1.IP,
 				Gw:        testNet1Router,
 			}},
-			oldLease: leaseAddRouter(trivialLeaseFromNet(testNet1), testNet1Router),
 			newLease: leaseAddRouter(trivialLeaseFromNet(testNet2), testNet2Router),
 			expectedRoutes: []netlink.Route{{
 				Protocol:  unix.RTPROT_DHCP,
@@ -283,7 +272,6 @@
 		{
 			name:          "AddsClasslessStaticRoutes",
 			initialRoutes: []netlink.Route{},
-			oldLease:      nil,
 			newLease: leaseAddClasslessRoutes(
 				// Router should be ignored
 				leaseAddRouter(trivialLeaseFromNet(testNet1), testNet1Router),
@@ -359,7 +347,7 @@
 			}
 
 			cb := ManageRoutes(testLink)
-			if err := cb(test.oldLease, test.newLease); err != nil {
+			if err := cb(test.newLease); err != nil {
 				t.Fatalf("callback returned an error: %v", err)
 			}
 			routes, err := netlink.RouteListFiltered(netlink.FAMILY_V4, &netlink.Route{}, 0)
diff --git a/metropolis/node/core/network/dhcp4c/dhcpc.go b/metropolis/node/core/network/dhcp4c/dhcpc.go
index d14dee8..be31700 100644
--- a/metropolis/node/core/network/dhcp4c/dhcpc.go
+++ b/metropolis/node/core/network/dhcp4c/dhcpc.go
@@ -129,7 +129,7 @@
 	Open() error
 }
 
-type LeaseCallback func(old, new *Lease) error
+type LeaseCallback func(*Lease) error
 
 // Client implements a DHCPv4 client.
 //
@@ -652,7 +652,7 @@
 func (c *Client) cleanup() {
 	c.unicastConn.Close()
 	if c.lease != nil {
-		c.LeaseCallback(leaseFromAck(c.lease, c.leaseDeadline), nil)
+		c.LeaseCallback(nil)
 	}
 	c.broadcastConn.Close()
 }
@@ -670,11 +670,11 @@
 		}
 	}
 	c.state = stateDiscovering
+	c.lease = nil
 	c.DiscoverBackoff.Reset()
-	if err := c.LeaseCallback(leaseFromAck(c.lease, c.leaseDeadline), nil); err != nil {
+	if err := c.LeaseCallback(nil); err != nil {
 		return fmt.Errorf("lease callback failed: %w", err)
 	}
-	c.lease = nil
 	return nil
 }
 
@@ -688,12 +688,11 @@
 func (c *Client) transitionToBound(ack *dhcpv4.DHCPv4, sentTime time.Time) error {
 	// Guaranteed to exist, leases without a lease time are filtered
 	leaseTime := ack.IPAddressLeaseTime(0)
-	origLeaseDeadline := c.leaseDeadline
 	c.leaseDeadline = sentTime.Add(leaseTime)
 	c.leaseBoundDeadline = sentTime.Add(ack.IPAddressRenewalTime(time.Duration(float64(leaseTime) * 0.5)))
 	c.leaseRenewDeadline = sentTime.Add(ack.IPAddressRebindingTime(time.Duration(float64(leaseTime) * 0.85)))
 
-	if err := c.LeaseCallback(leaseFromAck(c.lease, origLeaseDeadline), leaseFromAck(ack, c.leaseDeadline)); err != nil {
+	if err := c.LeaseCallback(leaseFromAck(ack, c.leaseDeadline)); err != nil {
 		return fmt.Errorf("lease callback failed: %w", err)
 	}
 
diff --git a/metropolis/node/core/network/dhcp4c/dhcpc_test.go b/metropolis/node/core/network/dhcp4c/dhcpc_test.go
index ca3994a..d9d5dd2 100644
--- a/metropolis/node/core/network/dhcp4c/dhcpc_test.go
+++ b/metropolis/node/core/network/dhcp4c/dhcpc_test.go
@@ -264,9 +264,8 @@
 
 	p.bmt.sendPackets(offer)
 	p.c.offer = offer
-	p.c.LeaseCallback = func(old, new *Lease) error {
-		assert.Nil(t, old, "old lease is not nil for new lease")
-		assert.Equal(t, testIP, new.AssignedIP, "new lease has wrong IP")
+	p.c.LeaseCallback = func(lease *Lease) error {
+		assert.Equal(t, testIP, lease.AssignedIP, "new lease has wrong IP")
 		return nil
 	}
 
@@ -319,10 +318,9 @@
 	offer.YourIPAddr = testIP
 
 	p := newPuppetClient(stateDiscovering)
-	p.c.LeaseCallback = func(old, new *Lease) error {
-		assert.Nil(t, old, "old is not nil")
-		assert.Equal(t, testIP, new.AssignedIP, "callback called with wrong IP")
-		assert.Equal(t, p.ft.Now().Add(leaseTime), new.ExpiresAt, "invalid ExpiresAt")
+	p.c.LeaseCallback = func(lease *Lease) error {
+		assert.Equal(t, testIP, lease.AssignedIP, "callback called with wrong IP")
+		assert.Equal(t, p.ft.Now().Add(leaseTime), lease.ExpiresAt, "invalid ExpiresAt")
 		return nil
 	}
 	p.bmt.sendPackets(offer)
@@ -372,12 +370,10 @@
 	assert.Equal(t, stateRenewing, p.c.state, "DHCP client not renewing")
 	offer.UpdateOption(dhcpv4.OptGeneric(TestOption(1), []byte{0x12}))
 	p.umt.sendPackets(offer)
-	p.c.LeaseCallback = func(old, new *Lease) error {
-		assert.Equal(t, testIP, old.AssignedIP, "callback called with wrong old IP")
-		assert.Equal(t, testIP, new.AssignedIP, "callback called with wrong IP")
-		assert.Equal(t, p.ft.Now().Add(leaseTime), new.ExpiresAt, "invalid ExpiresAt")
-		assert.Empty(t, old.Options.Get(TestOption(1)), "old contains options from new")
-		assert.Equal(t, []byte{0x12}, new.Options.Get(TestOption(1)), "renewal didn't add new option")
+	p.c.LeaseCallback = func(lease *Lease) error {
+		assert.Equal(t, testIP, lease.AssignedIP, "callback called with wrong IP")
+		assert.Equal(t, p.ft.Now().Add(leaseTime), lease.ExpiresAt, "invalid ExpiresAt")
+		assert.Equal(t, []byte{0x12}, lease.Options.Get(TestOption(1)), "renewal didn't add new option")
 		return nil
 	}
 	if err := p.c.runState(context.Background()); err != nil {
@@ -410,7 +406,7 @@
 	startTime := p.ft.Now()
 	p.ft.Advance(5 * time.Second)
 
-	p.c.LeaseCallback = func(old, new *Lease) error {
+	p.c.LeaseCallback = func(*Lease) error {
 		t.Fatal("Lease callback called without valid offer")
 		return nil
 	}
@@ -462,12 +458,10 @@
 	offer.UpdateOption(dhcpv4.OptGeneric(TestOption(1), []byte{0x12})) // Mark answer
 	p.bmt.sendPackets(offer)
 	p.bmt.sentPacket = nil
-	p.c.LeaseCallback = func(old, new *Lease) error {
-		assert.Equal(t, testIP, old.AssignedIP, "callback called with wrong old IP")
-		assert.Equal(t, testIP, new.AssignedIP, "callback called with wrong IP")
-		assert.Equal(t, p.ft.Now().Add(leaseTime), new.ExpiresAt, "invalid ExpiresAt")
-		assert.Empty(t, old.Options.Get(TestOption(1)), "old contains options from new")
-		assert.Equal(t, []byte{0x12}, new.Options.Get(TestOption(1)), "renewal didn't add new option")
+	p.c.LeaseCallback = func(lease *Lease) error {
+		assert.Equal(t, testIP, lease.AssignedIP, "callback called with wrong IP")
+		assert.Equal(t, p.ft.Now().Add(leaseTime), lease.ExpiresAt, "invalid ExpiresAt")
+		assert.Equal(t, []byte{0x12}, lease.Options.Get(TestOption(1)), "renewal didn't add new option")
 		return nil
 	}
 	if err := p.c.runState(context.Background()); err != nil {
@@ -495,9 +489,8 @@
 	p.c.leaseDeadline = p.ft.Now().Add(10000 * time.Millisecond)
 
 	p.ft.Advance(9 * time.Second)
-	p.c.LeaseCallback = func(old, new *Lease) error {
-		assert.Equal(t, testIP, old.AssignedIP, "callback called with wrong old IP")
-		assert.Nil(t, new, "transition to discovering didn't clear new lease on callback")
+	p.c.LeaseCallback = func(lease *Lease) error {
+		assert.Nil(t, lease, "transition to discovering didn't clear new lease on callback")
 		return nil
 	}
 	for i := 0; i < 10; i++ {
diff --git a/metropolis/node/core/network/main.go b/metropolis/node/core/network/main.go
index 487a0eb..5404660 100644
--- a/metropolis/node/core/network/main.go
+++ b/metropolis/node/core/network/main.go
@@ -62,6 +62,10 @@
 
 	// dhcp client for the 'main' interface of the node.
 	dhcp *dhcp4c.Client
+	// dhcpAddress is the current address obtained from DHCP.
+	dhcpAddress net.IP
+	// dnsServers are the current DNS servers obtained from DHCP.
+	dnsServers dhcp4c.DNSServers
 
 	// nftConn is a shared file descriptor handle to nftables, automatically
 	// initialized on first use.
@@ -92,7 +96,6 @@
 // meaningful to them.
 type Status struct {
 	ExternalAddress net.IP
-	DNSServers      dhcp4c.DNSServers
 }
 
 // ConfigureDNS sets a DNS ExtraDirective on the built-in DNS server of the
@@ -138,19 +141,33 @@
 // current lease to the rest of Metropolis. It updates the DNS service's
 // configuration to use the received upstream servers, and notifies the rest of
 // Metropolis via en event value that the network configuration has changed.
-func (s *Service) statusCallback(old, new *dhcp4c.Lease) error {
-	// Reconfigure DNS if needed.
-	oldServers := old.DNSServers()
-	newServers := new.DNSServers()
-	if !newServers.Equal(oldServers) {
-		s.ConfigureDNS(dns.NewUpstreamDirective(newServers))
+func (s *Service) statusCallback(ctx context.Context) dhcp4c.LeaseCallback {
+	return func(lease *dhcp4c.Lease) error {
+		// Reconfigure DNS if needed.
+		newServers := lease.DNSServers()
+		if !newServers.Equal(s.dnsServers) {
+			s.dnsServers = newServers
+			s.ConfigureDNS(dns.NewUpstreamDirective(newServers))
+		}
+
+		var newAddress net.IP
+		if lease != nil {
+			newAddress = lease.AssignedIP
+		}
+		if !newAddress.Equal(s.dhcpAddress) {
+			s.dhcpAddress = newAddress
+			// Notify status waiters.
+			s.Status.Set(&Status{
+				ExternalAddress: newAddress,
+			})
+			if newAddress != nil {
+				supervisor.Logger(ctx).Infof("New DHCP address: %s", newAddress)
+			} else {
+				supervisor.Logger(ctx).Warning("Lost DHCP address")
+			}
+		}
+		return nil
 	}
-	// Notify status waiters.
-	s.Status.Set(&Status{
-		ExternalAddress: new.AssignedIP,
-		DNSServers:      new.DNSServers(),
-	})
-	return nil
 }
 
 func (s *Service) useInterface(ctx context.Context, iface netlink.Link) error {
@@ -161,12 +178,7 @@
 	}
 	s.dhcp.VendorClassIdentifier = s.DHCPVendorClassID
 	s.dhcp.RequestedOptions = []dhcpv4.OptionCode{dhcpv4.OptionRouter, dhcpv4.OptionDomainNameServer, dhcpv4.OptionClasslessStaticRoute}
-	s.dhcp.LeaseCallback = dhcpcb.Compose(dhcpcb.ManageIP(iface), dhcpcb.ManageRoutes(iface), s.statusCallback, func(old, new *dhcp4c.Lease) error {
-		if old == nil || !old.AssignedIP.Equal(new.AssignedIP) {
-			supervisor.Logger(ctx).Infof("New DHCP address: %s", new.AssignedIP.String())
-		}
-		return nil
-	})
+	s.dhcp.LeaseCallback = dhcpcb.Compose(dhcpcb.ManageIP(iface), dhcpcb.ManageRoutes(iface), s.statusCallback(ctx))
 	err = supervisor.Run(ctx, "dhcp", s.dhcp.Run)
 	if err != nil {
 		return err
diff --git a/metropolis/node/core/network/static.go b/metropolis/node/core/network/static.go
index d69815c..f0cc4e2 100644
--- a/metropolis/node/core/network/static.go
+++ b/metropolis/node/core/network/static.go
@@ -185,7 +185,6 @@
 		}
 		s.Status.Set(&Status{
 			ExternalAddress: selectedAddr,
-			DNSServers:      nsIPList,
 		})
 	}
 
@@ -201,7 +200,7 @@
 		return fmt.Errorf("failed creating DHCPv4 client: %w", err)
 	}
 	c.RequestedOptions = []dhcpv4.OptionCode{dhcpv4.OptionRouter, dhcpv4.OptionDomainNameServer, dhcpv4.OptionClasslessStaticRoute}
-	c.LeaseCallback = dhcpcb.Compose(dhcpcb.ManageIP(lnk), dhcpcb.ManageRoutes(lnk), s.statusCallback)
+	c.LeaseCallback = dhcpcb.Compose(dhcpcb.ManageIP(lnk), dhcpcb.ManageRoutes(lnk), s.statusCallback(ctx))
 	return supervisor.Run(ctx, "dhcp-"+lnk.Attrs().Name, c.Run)
 }
 
diff --git a/metropolis/node/core/roleserve/worker_statuspush.go b/metropolis/node/core/roleserve/worker_statuspush.go
index 1b269c6..0cbc442 100644
--- a/metropolis/node/core/roleserve/worker_statuspush.go
+++ b/metropolis/node/core/roleserve/worker_statuspush.go
@@ -126,6 +126,9 @@
 			if err != nil {
 				return fmt.Errorf("getting network status failed: %w", err)
 			}
+			if st.ExternalAddress == nil {
+				continue
+			}
 			select {
 			case chans.address <- st.ExternalAddress.String():
 			case <-ctx.Done():