treewide: integrate new DNS server

This integrates the new DNS server into the network service, replacing 
CoreDNS.

Change-Id: I1d2e0fd3315dc2c602a8f805ed701633799e9986
Reviewed-on: https://review.monogon.dev/c/monogon/+/3260
Tested-by: Jenkins CI
Reviewed-by: Lorenz Brun <lorenz@monogon.tech>
diff --git a/metropolis/node/core/BUILD.bazel b/metropolis/node/core/BUILD.bazel
index f881f04..c6fdb18 100644
--- a/metropolis/node/core/BUILD.bazel
+++ b/metropolis/node/core/BUILD.bazel
@@ -38,6 +38,7 @@
         "//metropolis/proto/api",
         "//metropolis/version",
         "//osbase/logtree",
+        "//osbase/net/dns",
         "//osbase/pstore",
         "//osbase/supervisor",
         "//osbase/sysctl",
diff --git a/metropolis/node/core/localstorage/storage.go b/metropolis/node/core/localstorage/storage.go
index 8b7159d..40cec71 100644
--- a/metropolis/node/core/localstorage/storage.go
+++ b/metropolis/node/core/localstorage/storage.go
@@ -47,7 +47,7 @@
 	ESP ESPDirectory `dir:"esp"`
 	// Persistent Data partition, mounted from encrypted and authenticated storage.
 	Data DataDirectory `dir:"data"`
-	// FHS-standard /etc directory, containes /etc/hosts, /etc/machine-id, and
+	// FHS-standard /etc directory, contains /etc/hosts, /etc/machine-id, and
 	// other compatibility files.
 	Etc EtcDirectory `dir:"etc"`
 	// Ephemeral data, used by runtime, stored in tmpfs. Things like sockets,
@@ -157,8 +157,6 @@
 
 type EtcDirectory struct {
 	declarative.Directory
-	// Symlinked to /ephemeral/hosts, baked into the erofs system image
-	Hosts declarative.File `file:"hosts"`
 	// Symlinked to /ephemeral/machine-id, baked into the erofs system image
 	MachineID declarative.File `file:"machine-id"`
 }
@@ -168,7 +166,6 @@
 	Consensus         EphemeralConsensusDirectory  `dir:"consensus"`
 	Containerd        EphemeralContainerdDirectory `dir:"containerd"`
 	FlexvolumePlugins declarative.Directory        `dir:"flexvolume_plugins"`
-	Hosts             declarative.File             `file:"hosts"`
 	MachineID         declarative.File             `file:"machine-id"`
 }
 
diff --git a/metropolis/node/core/main.go b/metropolis/node/core/main.go
index ede3478..3d2969f 100644
--- a/metropolis/node/core/main.go
+++ b/metropolis/node/core/main.go
@@ -19,14 +19,12 @@
 import (
 	"context"
 	"fmt"
-	"net"
 	"os"
 	"strings"
 	"time"
 
 	"golang.org/x/sys/unix"
 
-	"source.monogon.dev/metropolis/node"
 	"source.monogon.dev/metropolis/node/core/cluster"
 	"source.monogon.dev/metropolis/node/core/devmgr"
 	"source.monogon.dev/metropolis/node/core/localstorage"
@@ -39,6 +37,7 @@
 	"source.monogon.dev/metropolis/node/core/update"
 	mversion "source.monogon.dev/metropolis/version"
 	"source.monogon.dev/osbase/logtree"
+	"source.monogon.dev/osbase/net/dns"
 	"source.monogon.dev/osbase/supervisor"
 	"source.monogon.dev/osbase/tpm"
 	"source.monogon.dev/version"
@@ -120,9 +119,9 @@
 		haveTPM = false
 	}
 
-	networkSvc := network.New(nil)
+	metrics.CoreRegistry.MustRegister(dns.MetricsRegistry)
+	networkSvc := network.New(nil, []string{"hosts", "kubernetes"})
 	networkSvc.DHCPVendorClassID = "dev.monogon.metropolis.node.v1"
-	networkSvc.ExtraDNSListenerIPs = []net.IP{node.ContainerDNSIP}
 	timeSvc := timesvc.New()
 	devmgrSvc := devmgr.New()
 
diff --git a/metropolis/node/core/network/BUILD.bazel b/metropolis/node/core/network/BUILD.bazel
index 8584a3b..614e0dc 100644
--- a/metropolis/node/core/network/BUILD.bazel
+++ b/metropolis/node/core/network/BUILD.bazel
@@ -14,10 +14,11 @@
         "//go/algorithm/toposort",
         "//metropolis/node/core/network/dhcp4c",
         "//metropolis/node/core/network/dhcp4c/callback",
-        "//metropolis/node/core/network/dns",
         "//net/proto",
         "//osbase/event/memory",
         "//osbase/logtree",
+        "//osbase/net/dns",
+        "//osbase/net/dns/forward",
         "//osbase/supervisor",
         "//osbase/sysctl",
         "@com_github_google_nftables//:nftables",
diff --git a/metropolis/node/core/network/dns/BUILD.bazel b/metropolis/node/core/network/dns/BUILD.bazel
deleted file mode 100644
index 6d51db1..0000000
--- a/metropolis/node/core/network/dns/BUILD.bazel
+++ /dev/null
@@ -1,18 +0,0 @@
-load("@io_bazel_rules_go//go:def.bzl", "go_library")
-
-go_library(
-    name = "dns",
-    srcs = [
-        "coredns.go",
-        "directives.go",
-    ],
-    importpath = "source.monogon.dev/metropolis/node/core/network/dns",
-    visibility = ["//metropolis/node:__subpackages__"],
-    deps = [
-        "//osbase/fileargs",
-        "//osbase/supervisor",
-        "@org_golang_x_sys//unix",
-    ],
-)
-
-exports_files(["resolv.conf"])
diff --git a/metropolis/node/core/network/dns/coredns.go b/metropolis/node/core/network/dns/coredns.go
deleted file mode 100644
index 472ab5c..0000000
--- a/metropolis/node/core/network/dns/coredns.go
+++ /dev/null
@@ -1,173 +0,0 @@
-// Copyright 2020 The Monogon Project Authors.
-//
-// SPDX-License-Identifier: Apache-2.0
-//
-// Licensed under the Apache License, Version 2.0 (the "License");
-// you may not use this file except in compliance with the License.
-// You may obtain a copy of the License at
-//
-//     http://www.apache.org/licenses/LICENSE-2.0
-//
-// Unless required by applicable law or agreed to in writing, software
-// distributed under the License is distributed on an "AS IS" BASIS,
-// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-// See the License for the specific language governing permissions and
-// limitations under the License.
-
-// Package DNS provides a DNS server using CoreDNS.
-package dns
-
-import (
-	"bytes"
-	"context"
-	"fmt"
-	"net"
-	"os"
-	"os/exec"
-	"strings"
-	"sync"
-
-	"golang.org/x/sys/unix"
-
-	"source.monogon.dev/osbase/fileargs"
-	"source.monogon.dev/osbase/supervisor"
-)
-
-const corefileBase = `
-.:53 {
-    errors
-	hosts {
-		fallthrough
-	}
-	
-    cache 30
-    loadbalance
-    reload 10s
-`
-
-type Service struct {
-	// Extra IPs the DNS service listens on and serves requests from.
-	ExtraListenerIPs      []net.IP
-	directiveRegistration chan *ExtraDirective
-	directives            map[string]ExtraDirective
-	cmd                   *exec.Cmd
-	args                  *fileargs.FileArgs
-	signalChan            chan os.Signal
-	// stateMu guards access to the directives, cmd and args fields
-	stateMu sync.Mutex
-}
-
-// New creates a new CoreDNS service. By default it only listens on the loopback
-// IPs ::1/127.0.0.1, but extra listener IPs can be added in extraListenerIPs.
-// The given channel can then be used to dynamically register and unregister
-// directives in the configuaration.
-// To register a new directive, send an ExtraDirective on the channel. To
-// remove it again, use CancelDirective() to create a removal message.
-func New(directiveRegistration chan *ExtraDirective) *Service {
-	return &Service{
-		directives:            map[string]ExtraDirective{},
-		directiveRegistration: directiveRegistration,
-		signalChan:            make(chan os.Signal),
-	}
-}
-
-func (s *Service) makeCorefile(fargs *fileargs.FileArgs) []byte {
-	var corefile bytes.Buffer
-	corefile.WriteString(corefileBase)
-	bindIPs := []string{"127.0.0.1", "::1"}
-	for _, ip := range s.ExtraListenerIPs {
-		bindIPs = append(bindIPs, ip.String())
-	}
-	fmt.Fprintf(&corefile, "\tbind %s\n", strings.Join(bindIPs, " "))
-	for _, dir := range s.directives {
-		resolvedDir := dir.directive
-		for fname, fcontent := range dir.files {
-			resolvedDir = strings.ReplaceAll(resolvedDir, fmt.Sprintf("$FILE(%v)", fname), fargs.ArgPath(fname, fcontent))
-		}
-		corefile.WriteString(resolvedDir)
-		corefile.WriteString("\n")
-	}
-	corefile.WriteString("\n}")
-	return corefile.Bytes()
-}
-
-// CancelDirective creates a message to cancel the given directive.
-func CancelDirective(d *ExtraDirective) *ExtraDirective {
-	return &ExtraDirective{
-		ID: d.ID,
-	}
-}
-
-// Run runs the DNS service consisting of the CoreDNS process and the directive
-// registration process
-func (s *Service) Run(ctx context.Context) error {
-	s.stateMu.Lock()
-	if s.args == nil {
-		args, err := fileargs.New()
-		if err != nil {
-			s.stateMu.Unlock()
-			return fmt.Errorf("failed to create fileargs: %w", err)
-		}
-		s.args = args
-	}
-	s.stateMu.Unlock()
-	supervisor.Run(ctx, "coredns", s.runCoreDNS)
-	supervisor.Run(ctx, "registration", s.runRegistration)
-	supervisor.Signal(ctx, supervisor.SignalHealthy)
-	supervisor.Signal(ctx, supervisor.SignalDone)
-	return nil
-}
-
-// runCoreDNS runs the CoreDNS proceess
-func (s *Service) runCoreDNS(ctx context.Context) error {
-	s.stateMu.Lock()
-	s.cmd = exec.CommandContext(ctx, "/kubernetes/bin/coredns",
-		s.args.FileOpt("-conf", "Corefile", s.makeCorefile(s.args)),
-	)
-
-	if s.args.Error() != nil {
-		s.stateMu.Unlock()
-		return fmt.Errorf("failed to use fileargs: %w", s.args.Error())
-	}
-
-	s.stateMu.Unlock()
-	return supervisor.RunCommand(ctx, s.cmd, supervisor.SignalChan(s.signalChan))
-}
-
-// runRegistration runs the background registration runnable which has a
-// different lifecycle from the CoreDNS runnable. It is responsible for
-// managing dynamic directives.
-func (s *Service) runRegistration(ctx context.Context) error {
-	supervisor.Signal(ctx, supervisor.SignalHealthy)
-	for {
-		select {
-		case <-ctx.Done():
-			return nil
-		case d := <-s.directiveRegistration:
-			s.processRegistration(ctx, d)
-		}
-	}
-}
-
-func (s *Service) processRegistration(ctx context.Context, d *ExtraDirective) {
-	s.stateMu.Lock()
-	defer s.stateMu.Unlock()
-	if d.directive == "" {
-		delete(s.directives, d.ID)
-	} else {
-		s.directives[d.ID] = *d
-	}
-	s.args.ArgPath("Corefile", s.makeCorefile(s.args))
-	if s.args.Error() != nil {
-		supervisor.Logger(ctx).Errorf("error creating new Corefile: %v", s.args.Error())
-	}
-	// If the signal sending thread is not ready, do nothing. Sending signals is
-	// unreliable anyways as the handler might not be installed yet or another
-	// reload might be in progress. Doing it this way saves a significant amount
-	// of complexity.
-	select {
-	case s.signalChan <- unix.SIGUSR1:
-	default:
-		supervisor.Logger(ctx).Infof("Reload signal could not be sent, relying on restart/reload to pick up changes")
-	}
-}
diff --git a/metropolis/node/core/network/dns/directives.go b/metropolis/node/core/network/dns/directives.go
deleted file mode 100644
index 10199fe..0000000
--- a/metropolis/node/core/network/dns/directives.go
+++ /dev/null
@@ -1,80 +0,0 @@
-// Copyright 2020 The Monogon Project Authors.
-//
-// SPDX-License-Identifier: Apache-2.0
-//
-// Licensed under the Apache License, Version 2.0 (the "License");
-// you may not use this file except in compliance with the License.
-// You may obtain a copy of the License at
-//
-//     http://www.apache.org/licenses/LICENSE-2.0
-//
-// Unless required by applicable law or agreed to in writing, software
-// distributed under the License is distributed on an "AS IS" BASIS,
-// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-// See the License for the specific language governing permissions and
-// limitations under the License.
-
-package dns
-
-import (
-	"fmt"
-	"net"
-	"strings"
-)
-
-// ExtraDirective contains additional config directives for CoreDNS.
-type ExtraDirective struct {
-	// ID is the identifier of this directive. There can only be one directive
-	// with a given ID active at once. The ID is also used to identify which
-	// directive to purge.
-	ID string
-	// directive contains a full CoreDNS directive as a string. It can also use
-	// the $FILE(<filename>) macro, which will be expanded to the path of a
-	// file from the files field.
-	directive string
-	// files contains additional files used in the configuration. The map key
-	// is used as the filename.
-	files map[string][]byte
-}
-
-// NewUpstreamDirective creates a forward with no fallthrough that forwards all
-// requests not yet matched to the given upstream DNS servers.
-func NewUpstreamDirective(dnsServers []net.IP) *ExtraDirective {
-	var strb strings.Builder
-	if len(dnsServers) > 0 {
-		strb.WriteString("forward .")
-		for _, ip := range dnsServers {
-			strb.WriteString(" ")
-			strb.WriteString(ip.String())
-		}
-	}
-	return &ExtraDirective{
-		directive: strb.String(),
-	}
-}
-
-var kubernetesDirective = `
-kubernetes %v in-addr.arpa ip6.arpa {
-	kubeconfig $FILE(kubeconfig) default
-	pods insecure
-	fallthrough in-addr.arpa ip6.arpa
-	ttl 30
-}
-`
-
-// NewKubernetesDirective creates a directive running a "Kubernetes DNS-Based
-// Service Discovery" compliant service under clusterDomain. The given
-// kubeconfig needs at least read access to services, endpoints and
-// endpointslices.
-//
-//	[1] https://github.com/kubernetes/dns/blob/master/docs/specification.md
-func NewKubernetesDirective(clusterDomain string, kubeconfig []byte) *ExtraDirective {
-	var prefix string
-	return &ExtraDirective{
-		ID:        "k8s-clusterdns",
-		directive: prefix + fmt.Sprintf(kubernetesDirective, clusterDomain),
-		files: map[string][]byte{
-			"kubeconfig": kubeconfig,
-		},
-	}
-}
diff --git a/metropolis/node/core/network/dns/resolv.conf b/metropolis/node/core/network/dns/resolv.conf
deleted file mode 100644
index eed7161..0000000
--- a/metropolis/node/core/network/dns/resolv.conf
+++ /dev/null
@@ -1,2 +0,0 @@
-# NOTE: This is baked into the rootfs. All DNS-related settings are in CoreDNS at //metropolis/node/core/network/dns.
-nameserver 127.0.0.1
\ No newline at end of file
diff --git a/metropolis/node/core/network/hostsfile/BUILD.bazel b/metropolis/node/core/network/hostsfile/BUILD.bazel
index c315c56..d7b5fd9 100644
--- a/metropolis/node/core/network/hostsfile/BUILD.bazel
+++ b/metropolis/node/core/network/hostsfile/BUILD.bazel
@@ -12,7 +12,9 @@
         "//metropolis/node/core/network",
         "//metropolis/proto/common",
         "//osbase/event",
+        "//osbase/net/dns",
         "//osbase/supervisor",
+        "@com_github_miekg_dns//:dns",
         "@org_golang_google_grpc//:grpc",
         "@org_golang_google_protobuf//proto",
         "@org_golang_x_sys//unix",
diff --git a/metropolis/node/core/network/hostsfile/hostsfile.go b/metropolis/node/core/network/hostsfile/hostsfile.go
index 883d8cf..2023eb3 100644
--- a/metropolis/node/core/network/hostsfile/hostsfile.go
+++ b/metropolis/node/core/network/hostsfile/hostsfile.go
@@ -17,12 +17,14 @@
 package hostsfile
 
 import (
-	"bytes"
 	"context"
 	"fmt"
 	"net"
-	"sort"
+	"regexp"
+	"strings"
+	"sync"
 
+	"github.com/miekg/dns"
 	"golang.org/x/sys/unix"
 	"google.golang.org/grpc"
 	"google.golang.org/protobuf/proto"
@@ -31,6 +33,7 @@
 	"source.monogon.dev/metropolis/node/core/localstorage"
 	"source.monogon.dev/metropolis/node/core/network"
 	"source.monogon.dev/osbase/event"
+	netDNS "source.monogon.dev/osbase/net/dns"
 	"source.monogon.dev/osbase/supervisor"
 
 	ipb "source.monogon.dev/metropolis/node/core/curator/proto/api"
@@ -39,7 +42,7 @@
 
 type Config struct {
 	// Network will be read to retrieve the current status of the Network service.
-	Network event.Value[*network.Status]
+	Network *network.Service
 	// Ephemeral is the root of the ephemeral storage of the node, into which the
 	// service will write its managed files.
 	Ephemeral *localstorage.EphemeralDirectory
@@ -63,6 +66,12 @@
 	// available information about the cluster nodes. It is automatically created and
 	// closed by Run.
 	clusterC chan nodeMap
+
+	// nodes contains the current information about nodes maintained by Run.
+	nodes nodeMap
+
+	// mu guards the nodes field.
+	mu sync.RWMutex
 }
 
 type ClusterDialer func(ctx context.Context) (*grpc.ClientConn, error)
@@ -71,7 +80,7 @@
 // either hostsfile or ClusterDirectory.
 type nodeInfo struct {
 	// address is the node's IP address.
-	address string
+	address net.IP
 	// local is true if address belongs to the local node.
 	local bool
 	// controlPlane is true if this node can be expected to run the control plane
@@ -81,7 +90,7 @@
 }
 
 func (n *nodeInfo) equals(o *nodeInfo) bool {
-	if n.address != o.address {
+	if !n.address.Equal(o.address) {
 		return false
 	}
 	if n.controlPlane != o.controlPlane {
@@ -93,32 +102,6 @@
 // nodeMap is a map from node ID (effectively DNS name) to node IP address.
 type nodeMap map[string]nodeInfo
 
-// hosts generates a complete /etc/hosts file based on the contents of the
-// nodeMap. Apart from the addresses in the nodeMap, entries for localhost
-// pointing to 127.0.0.1 and ::1 will also be generated.
-func (m nodeMap) hosts(ctx context.Context) []byte {
-	var nodeIdsSorted []string
-	for k := range m {
-		nodeIdsSorted = append(nodeIdsSorted, k)
-	}
-	sort.Slice(nodeIdsSorted, func(i, j int) bool {
-		return nodeIdsSorted[i] < nodeIdsSorted[j]
-	})
-
-	lines := [][]byte{
-		[]byte("127.0.0.1 localhost"),
-		[]byte("::1 localhost"),
-	}
-	for _, nid := range nodeIdsSorted {
-		addr := m[nid].address
-		line := fmt.Sprintf("%s %s", addr, nid)
-		lines = append(lines, []byte(line))
-	}
-	lines = append(lines, []byte(""))
-
-	return bytes.Join(lines, []byte("\n"))
-}
-
 // clusterDirectory builds a ClusterDirectory based on nodeMap contents. If m
 // is empty, an empty ClusterDirectory is returned.
 func (m nodeMap) clusterDirectory(ctx context.Context) *cpb.ClusterDirectory {
@@ -129,7 +112,7 @@
 		}
 		supervisor.Logger(ctx).Infof("ClusterDirectory entry: %s", ni.address)
 		addresses := []*cpb.ClusterDirectory_Node_Address{
-			{Host: ni.address},
+			{Host: ni.address.String()},
 		}
 		node := &cpb.ClusterDirectory_Node{
 			Id:        nid,
@@ -163,7 +146,7 @@
 					continue
 				}
 				nodes[node.Id] = nodeInfo{
-					address:      node.Addresses[0].Host,
+					address:      net.ParseIP(node.Addresses[0].Host),
 					local:        false,
 					controlPlane: true,
 				}
@@ -176,7 +159,7 @@
 	localC := make(chan *network.Status)
 	s.clusterC = make(chan nodeMap)
 
-	if err := supervisor.Run(ctx, "local", event.Pipe(s.Network, localC)); err != nil {
+	if err := supervisor.Run(ctx, "local", event.Pipe(&s.Network.Status, localC)); err != nil {
 		return err
 	}
 	if err := supervisor.Run(ctx, "cluster", s.runCluster); err != nil {
@@ -192,11 +175,13 @@
 		return fmt.Errorf("failed to set runtime hostname: %w", err)
 	}
 
-	// Immediately write an /etc/hosts just containing localhost and persisted
-	// cluster directory nodes, even if we don't yet have a network address.
-	if err := s.Ephemeral.Hosts.Write(nodes.hosts(ctx), 0644); err != nil {
-		return fmt.Errorf("failed to write %s: %w", s.Ephemeral.Hosts.FullPath(), err)
-	}
+	// Set the nodes field to start serving DNS queries.
+	s.mu.Lock()
+	s.nodes = nodes
+	s.mu.Unlock()
+
+	// Inject the hostsfile service into the DNS service as handler for hosts queries.
+	s.Network.DNS.SetHandler("hosts", s)
 
 	supervisor.Signal(ctx, supervisor.SignalHealthy)
 
@@ -216,15 +201,17 @@
 			if st.ExternalAddress == nil {
 				continue
 			}
-			u := st.ExternalAddress.String()
-			if nodes[s.NodeID].address == u {
+			u := st.ExternalAddress
+			if nodes[s.NodeID].address.Equal(u) {
 				continue
 			}
 			supervisor.Logger(ctx).Infof("Got new local address: %s", u)
+			s.mu.Lock()
 			nodes[s.NodeID] = nodeInfo{
 				address: u,
 				local:   true,
 			}
+			s.mu.Unlock()
 			changed = true
 		case u := <-s.clusterC:
 			// Loop through the nodeMap from the cluster subrunnable, making note of what
@@ -248,7 +235,9 @@
 					if existing.controlPlane != info.controlPlane {
 						changed = true
 						existing.controlPlane = info.controlPlane
+						s.mu.Lock()
 						nodes[id] = existing
+						s.mu.Unlock()
 					}
 					continue
 				}
@@ -256,7 +245,9 @@
 					continue
 				}
 				supervisor.Logger(ctx).Infof("Update for node %s: address %s, control plane %v", id, info.address, info.controlPlane)
+				s.mu.Lock()
 				nodes[id] = info
+				s.mu.Unlock()
 				changed = true
 			}
 			haveRemoteData = true
@@ -266,11 +257,6 @@
 			continue
 		}
 
-		supervisor.Logger(ctx).Infof("Updating hosts file: %d nodes", len(nodes))
-		if err := s.Ephemeral.Hosts.Write(nodes.hosts(ctx), 0644); err != nil {
-			return fmt.Errorf("failed to write %s: %w", s.Ephemeral.Hosts.FullPath(), err)
-		}
-
 		// Check that we are self-resolvable.
 		if _, err := net.ResolveIPAddr("ip", s.NodeID); err != nil {
 			supervisor.Logger(ctx).Errorf("Failed to self-resolve %q: %v", s.NodeID, err)
@@ -301,7 +287,7 @@
 	nodes := make(nodeMap)
 	return watcher.WatchNodes(ctx, s.Curator, watcher.SimpleFollower{
 		FilterFn: func(a *ipb.Node) bool {
-			if a.Status == nil || a.Status.ExternalAddress == "" {
+			if a.Status == nil || net.ParseIP(a.Status.ExternalAddress) == nil {
 				return false
 			}
 			return true
@@ -317,7 +303,7 @@
 		},
 		OnNewUpdated: func(new *ipb.Node) error {
 			nodes[new.Id] = nodeInfo{
-				address:      new.Status.ExternalAddress,
+				address:      net.ParseIP(new.Status.ExternalAddress),
 				local:        false,
 				controlPlane: new.Roles.ConsensusMember != nil,
 			}
@@ -339,3 +325,50 @@
 		},
 	})
 }
+
+const hostsTtl = 10
+
+var reHostname = regexp.MustCompile(`^metropolis-[0-9a-f]+\.$`)
+
+// HandleDNS serves DNS queries for node IDs.
+func (s *Service) HandleDNS(r *netDNS.Request) {
+	// Regexp matching is somewhat expensive, so we first do a cheaper
+	// string comparison to filter out most names.
+	if !strings.HasPrefix(r.QnameCanonical, "metropolis-") ||
+		!reHostname.MatchString(r.QnameCanonical) {
+		return
+	}
+
+	r.SetAuthoritative()
+
+	// Extract node ID by removing the trailing dot.
+	nodeID := r.QnameCanonical[:len(r.QnameCanonical)-1]
+
+	s.mu.RLock()
+	node, ok := s.nodes[nodeID]
+	s.mu.RUnlock()
+
+	if ok {
+		if v4 := node.address.To4(); v4 != nil {
+			if r.Qtype == dns.TypeA || r.Qtype == dns.TypeANY {
+				rr := new(dns.A)
+				rr.Hdr = dns.RR_Header{Name: r.Qname, Rrtype: dns.TypeA, Class: dns.ClassINET, Ttl: hostsTtl}
+				rr.A = v4
+				r.Reply.Answer = append(r.Reply.Answer, rr)
+			}
+		} else if len(node.address) == net.IPv6len {
+			if r.Qtype == dns.TypeAAAA || r.Qtype == dns.TypeANY {
+				rr := new(dns.AAAA)
+				rr.Hdr = dns.RR_Header{Name: r.Qname, Rrtype: dns.TypeAAAA, Class: dns.ClassINET, Ttl: hostsTtl}
+				rr.AAAA = node.address
+				r.Reply.Answer = append(r.Reply.Answer, rr)
+			}
+		} else {
+			r.AddExtendedError(dns.ExtendedErrorCodeInvalidData, "host IP address is missing or invalid")
+			r.Reply.Rcode = dns.RcodeServerFailure
+		}
+	} else {
+		r.Reply.Rcode = dns.RcodeNameError
+	}
+	r.SendReply()
+}
diff --git a/metropolis/node/core/network/main.go b/metropolis/node/core/network/main.go
index 078eab8..06b443a 100644
--- a/metropolis/node/core/network/main.go
+++ b/metropolis/node/core/network/main.go
@@ -20,6 +20,7 @@
 	"context"
 	"fmt"
 	"net"
+	"slices"
 	"strconv"
 
 	"github.com/google/nftables"
@@ -29,8 +30,9 @@
 
 	"source.monogon.dev/metropolis/node/core/network/dhcp4c"
 	dhcpcb "source.monogon.dev/metropolis/node/core/network/dhcp4c/callback"
-	"source.monogon.dev/metropolis/node/core/network/dns"
 	"source.monogon.dev/osbase/event/memory"
+	"source.monogon.dev/osbase/net/dns"
+	"source.monogon.dev/osbase/net/dns/forward"
 	"source.monogon.dev/osbase/supervisor"
 	"source.monogon.dev/osbase/sysctl"
 
@@ -47,18 +49,11 @@
 	// autoconfiguration.
 	StaticConfig *netpb.Net
 
-	// List of IPs which get configured onto the loopback interface and the
-	// integrated DNS server is serving on. Cannot be changed at runtime.
-	// This is a hack to work around CoreDNS not being able to change listeners
-	// on-the-fly without breaking everything. This will go away once its
-	// frontend got replaced by something which can do that.
-	ExtraDNSListenerIPs []net.IP
-
 	// Vendor Class identifier of the system
 	DHCPVendorClassID string
 
-	dnsReg chan *dns.ExtraDirective
-	dnsSvc *dns.Service
+	DNS        *dns.Service
+	dnsForward *forward.Forward
 
 	// dhcp client for the 'main' interface of the node.
 	dhcp *dhcp4c.Client
@@ -80,12 +75,17 @@
 // New instantiates a new network service. If autoconfiguration is desired,
 // staticConfig must be set to nil. If staticConfig is set to a non-nil value,
 // it will be used instead of autoconfiguration.
-func New(staticConfig *netpb.Net) *Service {
-	dnsReg := make(chan *dns.ExtraDirective)
-	dnsSvc := dns.New(dnsReg)
+// If dnsHandlerNames is non-nil, DNS handlers with these names must be set
+// on the DNS service with s.DNS.SetHandler. When serving DNS queries, they
+// will be tried in the order they appear here before forwarding.
+func New(staticConfig *netpb.Net, dnsHandlerNames []string) *Service {
+	dnsSvc := dns.New(slices.Concat(dnsHandlerNames, []string{"forward"}))
+	dnsForward := forward.New()
+	dnsSvc.SetHandler("forward", dnsForward)
+
 	return &Service{
-		dnsReg:       dnsReg,
-		dnsSvc:       dnsSvc,
+		DNS:          dnsSvc,
+		dnsForward:   dnsForward,
 		StaticConfig: staticConfig,
 	}
 }
@@ -98,13 +98,6 @@
 	ExternalAddress net.IP
 }
 
-// ConfigureDNS sets a DNS ExtraDirective on the built-in DNS server of the
-// network Service. See //metropolis/node/core/network/dns for more
-// information.
-func (s *Service) ConfigureDNS(d *dns.ExtraDirective) {
-	s.dnsReg <- d
-}
-
 func singleIPtoNetlinkAddr(ip net.IP, label string) *netlink.Addr {
 	var mask net.IPMask
 	if ip.To4() == nil {
@@ -129,6 +122,34 @@
 	}
 }
 
+// AddLoopbackIP adds the given IP to a loopback interface which can then be
+// used to bind listeners to. Once this function returns, the IP is assigned and
+// is ready to use. It's recommended to use defer to call ReleaseLoopbackIP to
+// make sure IPs are released when the goroutine using it exits.
+func (s *Service) AddLoopbackIP(ip net.IP) error {
+	loopbackIf, err := netlink.LinkByName("lo")
+	if err != nil {
+		return fmt.Errorf("no loopback interface: %w", err)
+	}
+	if err := netlink.AddrAdd(loopbackIf, singleIPtoNetlinkAddr(ip, "localsvc")); err != nil {
+		return fmt.Errorf("failed to add IP: %w", err)
+	}
+	return nil
+}
+
+// ReleaseLoopbackIP releases an IP allocated by AddLoopbackIP.
+// Calling it multiple times for the same IP is an error.
+func (s *Service) ReleaseLoopbackIP(ip net.IP) error {
+	loopbackIf, err := netlink.LinkByName("lo")
+	if err != nil {
+		return fmt.Errorf("no loopback interface: %w", err)
+	}
+	if err := netlink.AddrDel(loopbackIf, singleIPtoNetlinkAddr(ip, "localsvc")); err != nil {
+		return fmt.Errorf("failed to delete IP: %w", err)
+	}
+	return nil
+}
+
 // nfifname converts an interface name into 16 bytes padded with zeroes (for
 // nftables)
 func nfifname(n string) []byte {
@@ -140,14 +161,18 @@
 // statusCallback is the main DHCP client callback connecting updates to the
 // 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.
+// Metropolis via an event value that the network configuration has changed.
 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))
+			newAddrs := make([]string, len(newServers))
+			for i, ip := range newServers {
+				newAddrs[i] = net.JoinHostPort(ip.String(), "53")
+			}
+			s.dnsForward.DNSServers.Set(newAddrs)
 		}
 
 		var newAddress net.IP
@@ -193,8 +218,6 @@
 
 func (s *Service) Run(ctx context.Context) error {
 	logger := supervisor.Logger(ctx)
-	s.dnsSvc.ExtraListenerIPs = s.ExtraDNSListenerIPs
-	supervisor.Run(ctx, "dns", s.dnsSvc.Run)
 
 	earlySysctlOpts := sysctl.Options{
 		// Enable strict reverse path filtering on all interfaces (important
@@ -219,6 +242,14 @@
 		logger.Errorf("Applying quirks failed, continuing without: %v", err)
 	}
 
+	loopbackIf, err := netlink.LinkByName("lo")
+	if err != nil {
+		logger.Fatalf("No loopback interface: %v", err)
+	}
+	if err := netlink.LinkSetUp(loopbackIf); err != nil {
+		logger.Errorf("Failed to bring up loopback interface: %v", err)
+	}
+
 	supervisor.Run(ctx, "announce", s.runNeighborAnnounce)
 
 	// Choose between autoconfig and static config runnables
@@ -228,6 +259,9 @@
 		supervisor.Run(ctx, "static", s.runStaticConfig)
 	}
 
+	supervisor.Run(ctx, "dns", s.DNS.Run)
+	supervisor.Run(ctx, "dns-forward", s.dnsForward.Run)
+
 	s.natTable = s.nftConn.AddTable(&nftables.Table{
 		Family: nftables.TableFamilyIPv4,
 		Name:   "nat",
@@ -328,15 +362,6 @@
 			} else {
 				logger.Infof("Ignoring non-Ethernet interface %s", attrs.Name)
 			}
-		} else if link.Attrs().Name == "lo" {
-			if err := netlink.LinkSetUp(link); err != nil {
-				logger.Errorf("Failed to bring up loopback interface: %v", err)
-			}
-			for _, addr := range s.ExtraDNSListenerIPs {
-				if err := netlink.AddrAdd(link, singleIPtoNetlinkAddr(addr, "")); err != nil {
-					logger.Errorf("Failed to assign extra loopback IP: %v", err)
-				}
-			}
 		}
 	}
 	if len(ethernetLinks) != 1 {
diff --git a/metropolis/node/core/network/static.go b/metropolis/node/core/network/static.go
index 93b9306..5afefdf 100644
--- a/metropolis/node/core/network/static.go
+++ b/metropolis/node/core/network/static.go
@@ -18,7 +18,6 @@
 	"source.monogon.dev/go/algorithm/toposort"
 	"source.monogon.dev/metropolis/node/core/network/dhcp4c"
 	dhcpcb "source.monogon.dev/metropolis/node/core/network/dhcp4c/callback"
-	"source.monogon.dev/metropolis/node/core/network/dns"
 	"source.monogon.dev/osbase/logtree"
 	"source.monogon.dev/osbase/supervisor"
 	"source.monogon.dev/osbase/sysctl"
@@ -39,21 +38,10 @@
 		return err
 	}
 
-	hostDevices, loopbackLink, err := listHostDeviceIfaces()
+	hostDevices, err := listHostDeviceIfaces()
 	if err != nil {
 		return err
 	}
-	if loopbackLink == nil {
-		return errors.New("no loopback interface present, weird/broken kernel?")
-	}
-	if err := netlink.LinkSetUp(loopbackLink); err != nil {
-		l.Error("Failed to enable loopback interface: %w", err)
-	}
-	for _, addr := range s.ExtraDNSListenerIPs {
-		if err := netlink.AddrAdd(loopbackLink, singleIPtoNetlinkAddr(addr, "")); err != nil {
-			l.Errorf("Failed to assign extra loopback IP: %v", err)
-		}
-	}
 
 	var hasIPv4Autoconfig bool
 
@@ -151,17 +139,18 @@
 		}
 		l.Infof("Configured interface %q", i.Name)
 	}
-	var nsIPList []net.IP
+	var nsAddrList []string
 	for _, ns := range s.StaticConfig.Nameserver {
 		nsIP := net.ParseIP(ns.Ip)
 		if nsIP == nil {
 			l.Warningf("failed to parse %q as nameserver IP", ns.Ip)
 			continue
 		}
-		nsIPList = append(nsIPList, nsIP)
+		nsAddr := net.JoinHostPort(nsIP.String(), "53")
+		nsAddrList = append(nsAddrList, nsAddr)
 	}
-	if len(nsIPList) > 0 {
-		s.ConfigureDNS(dns.NewUpstreamDirective(nsIPList))
+	if len(nsAddrList) > 0 {
+		s.dnsForward.DNSServers.Set(nsAddrList)
 	}
 
 	if !hasIPv4Autoconfig {
@@ -257,24 +246,15 @@
 	driver string
 }
 
-func listHostDeviceIfaces() ([]deviceIfData, netlink.Link, error) {
+func listHostDeviceIfaces() ([]deviceIfData, error) {
 	links, err := netlink.LinkList()
 	if err != nil {
-		return nil, nil, fmt.Errorf("failed to list network links: %w", err)
+		return nil, fmt.Errorf("failed to list network links: %w", err)
 	}
 
 	var hostDevices []deviceIfData
 
-	var loopbackLink netlink.Link
-
 	for _, link := range links {
-		// Modern Linux versions always create a loopback device named "lo" with
-		// constant interface index 1 in every network namespace. Since Linux
-		// 3.6 there is a BUG_ON in the loopback driver, asserting that this is
-		// true for every loopback interface created.
-		if link.Attrs().Index == 1 {
-			loopbackLink = link
-		}
 		d, ok := link.(*netlink.Device)
 		if !ok {
 			continue
@@ -289,7 +269,7 @@
 			driver: driver,
 		})
 	}
-	return hostDevices, loopbackLink, nil
+	return hostDevices, nil
 }
 
 func deviceIfaceFromSpec(it *netpb.Interface_Device, hostDevices []deviceIfData, l logtree.LeveledLogger) (*netlink.Device, error) {
diff --git a/metropolis/node/core/roleserve/BUILD.bazel b/metropolis/node/core/roleserve/BUILD.bazel
index 9d9bdcd..7f3f0d3 100644
--- a/metropolis/node/core/roleserve/BUILD.bazel
+++ b/metropolis/node/core/roleserve/BUILD.bazel
@@ -41,6 +41,7 @@
         "//osbase/event",
         "//osbase/event/memory",
         "//osbase/logtree",
+        "//osbase/net/dns",
         "//osbase/pki",
         "//osbase/supervisor",
         "@org_golang_google_grpc//:grpc",
diff --git a/metropolis/node/core/roleserve/worker_hostsfile.go b/metropolis/node/core/roleserve/worker_hostsfile.go
index 6e1a1ca..7beff2f 100644
--- a/metropolis/node/core/roleserve/worker_hostsfile.go
+++ b/metropolis/node/core/roleserve/worker_hostsfile.go
@@ -13,8 +13,9 @@
 )
 
 // workerHostsfile run the //metropolis/node/core/network/hostsfile service,
-// which in turn populates /etc/hosts, /etc/machine-id and updates the ESP-stored
-// ClusterDirectory (used to Join the cluster after a machine reboots).
+// which in turn serves DNS for hosts, populates /etc/machine-id
+// and updates the ESP-stored ClusterDirectory
+// (used to Join the cluster after a machine reboots).
 type workerHostsfile struct {
 	storageRoot *localstorage.Root
 
@@ -41,7 +42,7 @@
 
 	svc := hostsfile.Service{
 		Config: hostsfile.Config{
-			Network:               &s.network.Status,
+			Network:               s.network,
 			Ephemeral:             &s.storageRoot.Ephemeral,
 			ESP:                   &s.storageRoot.ESP,
 			NodeID:                cc.nodeID(),
diff --git a/metropolis/node/core/roleserve/worker_kubernetes.go b/metropolis/node/core/roleserve/worker_kubernetes.go
index eaa092a..b50fe20 100644
--- a/metropolis/node/core/roleserve/worker_kubernetes.go
+++ b/metropolis/node/core/roleserve/worker_kubernetes.go
@@ -16,6 +16,7 @@
 	cpb "source.monogon.dev/metropolis/proto/common"
 	"source.monogon.dev/osbase/event"
 	"source.monogon.dev/osbase/event/memory"
+	"source.monogon.dev/osbase/net/dns"
 	"source.monogon.dev/osbase/supervisor"
 )
 
@@ -157,7 +158,6 @@
 			Node:           d.node,
 			ServiceIPRange: serviceIPRange,
 			ClusterNet:     clusterIPRange,
-			ClusterDomain:  clusterDomain,
 			KPKI:           pki,
 			Root:           s.storageRoot,
 			Consensus:      d.lcp.consensus,
@@ -206,6 +206,7 @@
 			supervisor.Logger(ctx).Infof("Got new startup data.")
 			if d.roles.KubernetesWorker == nil {
 				supervisor.Logger(ctx).Infof("No Kubernetes worker role, not starting.")
+				s.network.DNS.SetHandler("kubernetes", dns.EmptyDNSHandler{})
 				continue
 			}
 			break
diff --git a/metropolis/node/core/update/e2e/testos/main.go b/metropolis/node/core/update/e2e/testos/main.go
index 69897eb..7d91d9e 100644
--- a/metropolis/node/core/update/e2e/testos/main.go
+++ b/metropolis/node/core/update/e2e/testos/main.go
@@ -86,7 +86,7 @@
 
 func testosRunnable(ctx context.Context) error {
 	supervisor.Logger(ctx).Info("TESTOS_VARIANT=" + Variant)
-	networkSvc := network.New(nil)
+	networkSvc := network.New(nil, nil)
 	networkSvc.DHCPVendorClassID = "dev.monogon.testos.v1"
 	supervisor.Run(ctx, "networking", networkSvc.Run)
 
diff --git a/metropolis/node/core/update/e2e/testos/testos.bzl b/metropolis/node/core/update/e2e/testos/testos.bzl
index 2ce3e31..9db31c2 100644
--- a/metropolis/node/core/update/e2e/testos/testos.bzl
+++ b/metropolis/node/core/update/e2e/testos/testos.bzl
@@ -10,8 +10,7 @@
         name = "rootfs_" + variant,
         files = {
             ":testos_" + variant: "/init",
-            "//metropolis/node/core/network/dns:resolv.conf": "/etc/resolv.conf",
-            "@com_github_coredns_coredns//:coredns": "/kubernetes/bin/coredns",
+            "//osbase/net/dns:resolv.conf": "/etc/resolv.conf",
         },
         fsspecs = [
             "//osbase/build:earlydev.fsspec",