osbase/net/dns/kubernetes: add Kubernetes DNS handler

This adds a DNS server handler for Kubernetes DNS service discovery. It
is partially based on the CoreDNS Kubernetes plugin. The query handler
however is written completely from scratch. The handler in the CoreDNS
plugin is very weird; it first handles each query type separately, and
generates msg.Service objects which then need to be converted to dns
records. The new implementation is much simpler, and also more correct:
It handles ANY queries, and follows the rules for NXDOMAIN (If a name is
NXDOMAIN for one qtype, it is NXDOMAIN for all qtypes, and subdomains of
the name are also NXDOMAIN.)

Change-Id: Id1d498ca5384a3b047587ed73e95e4871d82d499
Reviewed-on: https://review.monogon.dev/c/monogon/+/3259
Reviewed-by: Lorenz Brun <lorenz@monogon.tech>
Tested-by: Jenkins CI
diff --git a/osbase/net/dns/kubernetes/BUILD.bazel b/osbase/net/dns/kubernetes/BUILD.bazel
new file mode 100644
index 0000000..a23fc46
--- /dev/null
+++ b/osbase/net/dns/kubernetes/BUILD.bazel
@@ -0,0 +1,43 @@
+load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test")
+
+go_library(
+    name = "kubernetes",
+    srcs = [
+        "controller.go",
+        "handler.go",
+        "kubernetes.go",
+        "metrics.go",
+    ],
+    importpath = "source.monogon.dev/osbase/net/dns/kubernetes",
+    visibility = ["//visibility:public"],
+    deps = [
+        "//osbase/net/dns",
+        "//osbase/net/dns/kubernetes/object",
+        "//osbase/supervisor",
+        "@com_github_miekg_dns//:dns",
+        "@com_github_prometheus_client_golang//prometheus",
+        "@io_k8s_api//core/v1:core",
+        "@io_k8s_api//discovery/v1:discovery",
+        "@io_k8s_apimachinery//pkg/apis/meta/v1:meta",
+        "@io_k8s_apimachinery//pkg/runtime",
+        "@io_k8s_apimachinery//pkg/watch",
+        "@io_k8s_client_go//kubernetes",
+        "@io_k8s_client_go//tools/cache",
+    ],
+)
+
+go_test(
+    name = "kubernetes_test",
+    srcs = ["handler_test.go"],
+    embed = [":kubernetes"],
+    deps = [
+        "//osbase/net/dns",
+        "@com_github_miekg_dns//:dns",
+        "@io_k8s_api//core/v1:core",
+        "@io_k8s_api//discovery/v1:discovery",
+        "@io_k8s_apimachinery//pkg/apis/meta/v1:meta",
+        "@io_k8s_apimachinery//pkg/util/intstr",
+        "@io_k8s_client_go//kubernetes/fake",
+        "@io_k8s_utils//ptr",
+    ],
+)
diff --git a/osbase/net/dns/kubernetes/LICENSE-3RD-PARTY.txt b/osbase/net/dns/kubernetes/LICENSE-3RD-PARTY.txt
new file mode 100644
index 0000000..98f9935
--- /dev/null
+++ b/osbase/net/dns/kubernetes/LICENSE-3RD-PARTY.txt
@@ -0,0 +1,13 @@
+Copyright 2016-2024 The CoreDNS authors and contributors
+
+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.
diff --git a/osbase/net/dns/kubernetes/controller.go b/osbase/net/dns/kubernetes/controller.go
new file mode 100644
index 0000000..61a33fc
--- /dev/null
+++ b/osbase/net/dns/kubernetes/controller.go
@@ -0,0 +1,307 @@
+package kubernetes
+
+// Taken and modified from the Kubernetes plugin of CoreDNS, under Apache 2.0.
+
+import (
+	"context"
+	"errors"
+	"sync/atomic"
+	"time"
+
+	api "k8s.io/api/core/v1"
+	discovery "k8s.io/api/discovery/v1"
+	meta "k8s.io/apimachinery/pkg/apis/meta/v1"
+	"k8s.io/apimachinery/pkg/runtime"
+	"k8s.io/apimachinery/pkg/watch"
+	"k8s.io/client-go/kubernetes"
+	"k8s.io/client-go/tools/cache"
+
+	"source.monogon.dev/osbase/net/dns/kubernetes/object"
+)
+
+const (
+	svcIPIndex           = "ServiceIP"
+	epNameNamespaceIndex = "EndpointNameNamespace"
+	epIPIndex            = "EndpointsIP"
+)
+
+// epLabelSelector selects EndpointSlices that belong to headless services.
+// Endpoint DNS records are only served for headless services,
+// and we can save resources by not fetching other EndpointSlices.
+const epLabelSelector = api.IsHeadlessService
+
+type dnsController interface {
+	GetSvc(string) *object.Service
+	SvcIndexReverse(string) []*object.Service
+	EpIndex(string) []*object.Endpoints
+	EpIndexReverse(string) []*object.Endpoints
+	NamespaceExists(string) bool
+
+	Start(<-chan struct{})
+	HasSynced() bool
+
+	// Modified returns the timestamp of the most recent changes to services.
+	Modified() int64
+}
+
+type dnsControl struct {
+	// modified tracks timestamp of the most recent changes
+	modified  atomic.Int64
+	hasSynced atomic.Bool
+
+	svcController cache.Controller
+	epController  cache.Controller
+	nsController  cache.Controller
+
+	svcLister cache.Indexer
+	epLister  cache.Indexer
+	nsLister  cache.Store
+}
+
+// newdnsController creates a controller.
+func newdnsController(ctx context.Context, kubeClient kubernetes.Interface) *dnsControl {
+	dns := dnsControl{}
+
+	dns.svcLister, dns.svcController = object.NewIndexerInformer(
+		&cache.ListWatch{
+			ListFunc:  serviceListFunc(ctx, kubeClient, api.NamespaceAll),
+			WatchFunc: serviceWatchFunc(ctx, kubeClient, api.NamespaceAll),
+		},
+		&api.Service{},
+		&dns,
+		cache.Indexers{svcIPIndex: svcIPIndexFunc},
+		object.DefaultProcessor(object.ToService),
+	)
+
+	dns.epLister, dns.epController = object.NewIndexerInformer(
+		&cache.ListWatch{
+			ListFunc:  endpointSliceListFunc(ctx, kubeClient, api.NamespaceAll),
+			WatchFunc: endpointSliceWatchFunc(ctx, kubeClient, api.NamespaceAll),
+		},
+		&discovery.EndpointSlice{},
+		&dns,
+		cache.Indexers{epNameNamespaceIndex: epNameNamespaceIndexFunc, epIPIndex: epIPIndexFunc},
+		object.DefaultProcessor(object.EndpointSliceToEndpoints),
+	)
+
+	dns.nsLister, dns.nsController = object.NewIndexerInformer(
+		&cache.ListWatch{
+			ListFunc:  namespaceListFunc(ctx, kubeClient),
+			WatchFunc: namespaceWatchFunc(ctx, kubeClient),
+		},
+		&api.Namespace{},
+		&dns,
+		cache.Indexers{},
+		object.DefaultProcessor(object.ToNamespace),
+	)
+
+	return &dns
+}
+
+func svcIPIndexFunc(obj interface{}) ([]string, error) {
+	svc, ok := obj.(*object.Service)
+	if !ok {
+		return nil, errObj
+	}
+	idx := make([]string, len(svc.ClusterIPs))
+	copy(idx, svc.ClusterIPs)
+	return idx, nil
+}
+
+func epNameNamespaceIndexFunc(obj interface{}) ([]string, error) {
+	s, ok := obj.(*object.Endpoints)
+	if !ok {
+		return nil, errObj
+	}
+	return []string{s.Index}, nil
+}
+
+func epIPIndexFunc(obj interface{}) ([]string, error) {
+	ep, ok := obj.(*object.Endpoints)
+	if !ok {
+		return nil, errObj
+	}
+	idx := make([]string, len(ep.Addresses))
+	for i, addr := range ep.Addresses {
+		idx[i] = addr.IP
+	}
+	return idx, nil
+}
+
+func serviceListFunc(ctx context.Context, c kubernetes.Interface, ns string) func(meta.ListOptions) (runtime.Object, error) {
+	return func(opts meta.ListOptions) (runtime.Object, error) {
+		return c.CoreV1().Services(ns).List(ctx, opts)
+	}
+}
+
+func endpointSliceListFunc(ctx context.Context, c kubernetes.Interface, ns string) func(meta.ListOptions) (runtime.Object, error) {
+	return func(opts meta.ListOptions) (runtime.Object, error) {
+		opts.LabelSelector = epLabelSelector
+		return c.DiscoveryV1().EndpointSlices(ns).List(ctx, opts)
+	}
+}
+
+func namespaceListFunc(ctx context.Context, c kubernetes.Interface) func(meta.ListOptions) (runtime.Object, error) {
+	return func(opts meta.ListOptions) (runtime.Object, error) {
+		return c.CoreV1().Namespaces().List(ctx, opts)
+	}
+}
+
+func serviceWatchFunc(ctx context.Context, c kubernetes.Interface, ns string) func(options meta.ListOptions) (watch.Interface, error) {
+	return func(options meta.ListOptions) (watch.Interface, error) {
+		return c.CoreV1().Services(ns).Watch(ctx, options)
+	}
+}
+
+func endpointSliceWatchFunc(ctx context.Context, c kubernetes.Interface, ns string) func(options meta.ListOptions) (watch.Interface, error) {
+	return func(options meta.ListOptions) (watch.Interface, error) {
+		options.LabelSelector = epLabelSelector
+		return c.DiscoveryV1().EndpointSlices(ns).Watch(ctx, options)
+	}
+}
+
+func namespaceWatchFunc(ctx context.Context, c kubernetes.Interface) func(options meta.ListOptions) (watch.Interface, error) {
+	return func(options meta.ListOptions) (watch.Interface, error) {
+		return c.CoreV1().Namespaces().Watch(ctx, options)
+	}
+}
+
+// Start starts the controller.
+func (dns *dnsControl) Start(stopCh <-chan struct{}) {
+	go dns.svcController.Run(stopCh)
+	go dns.epController.Run(stopCh)
+	go dns.nsController.Run(stopCh)
+}
+
+// HasSynced returns true if the initial data has been
+// completely loaded into memory.
+func (dns *dnsControl) HasSynced() bool {
+	if dns.hasSynced.Load() {
+		return true
+	}
+	a := dns.svcController.HasSynced()
+	b := dns.epController.HasSynced()
+	c := dns.nsController.HasSynced()
+	if a && b && c {
+		dns.hasSynced.Store(true)
+		return true
+	}
+	return false
+}
+
+func (dns *dnsControl) GetSvc(key string) *object.Service {
+	o, exists, err := dns.svcLister.GetByKey(key)
+	if err != nil || !exists {
+		return nil
+	}
+	s, ok := o.(*object.Service)
+	if ok {
+		return s
+	}
+	return nil
+}
+
+func (dns *dnsControl) SvcIndexReverse(ip string) (svcs []*object.Service) {
+	os, err := dns.svcLister.ByIndex(svcIPIndex, ip)
+	if err != nil {
+		return nil
+	}
+	svcs = make([]*object.Service, 0, len(os))
+	for _, o := range os {
+		s, ok := o.(*object.Service)
+		if ok {
+			svcs = append(svcs, s)
+		}
+	}
+	return svcs
+}
+
+func (dns *dnsControl) EpIndex(idx string) (ep []*object.Endpoints) {
+	os, err := dns.epLister.ByIndex(epNameNamespaceIndex, idx)
+	if err != nil {
+		return nil
+	}
+	ep = make([]*object.Endpoints, 0, len(os))
+	for _, o := range os {
+		e, ok := o.(*object.Endpoints)
+		if ok {
+			ep = append(ep, e)
+		}
+	}
+	return ep
+}
+
+func (dns *dnsControl) EpIndexReverse(ip string) (ep []*object.Endpoints) {
+	os, err := dns.epLister.ByIndex(epIPIndex, ip)
+	if err != nil {
+		return nil
+	}
+	ep = make([]*object.Endpoints, 0, len(os))
+	for _, o := range os {
+		e, ok := o.(*object.Endpoints)
+		if ok {
+			ep = append(ep, e)
+		}
+	}
+	return ep
+}
+
+// NamespaceExists returns true if a namespace with this name exists.
+func (dns *dnsControl) NamespaceExists(name string) bool {
+	_, exists, _ := dns.nsLister.GetByKey(name)
+	return exists
+}
+
+func (dns *dnsControl) OnAdd(obj interface{}, isInInitialList bool) {
+	dns.updateModified()
+	switch obj := obj.(type) {
+	case *object.Endpoints:
+		// Don't record latency during initial sync, because measuring latency only
+		// makes sense for changes that happen while the service is running.
+		if !isInInitialList {
+			recordDNSProgrammingLatency(obj.LastChangeTriggerTime)
+		}
+	}
+}
+
+func (dns *dnsControl) OnDelete(obj interface{}) {
+	dns.updateModified()
+	// Note: We cannot record programming latency on deletes, because the trigger
+	// time annotation is not updated when the object is deleted.
+}
+
+func (dns *dnsControl) OnUpdate(oldObj, newObj interface{}) {
+	// If both objects have the same resource version, they are identical.
+	if oldObj.(meta.Object).GetResourceVersion() == newObj.(meta.Object).GetResourceVersion() {
+		return
+	}
+
+	switch newObj := newObj.(type) {
+	case *object.Service:
+		if object.ServiceModified(oldObj.(*object.Service), newObj) {
+			dns.updateModified()
+		}
+	case *object.Endpoints:
+		oldObj := oldObj.(*object.Endpoints)
+		if object.EndpointsModified(oldObj, newObj) {
+			dns.updateModified()
+			// If the trigger time has not changed, the process that last updated the
+			// object did not update the trigger time, so we can't know the latency.
+			if oldObj.LastChangeTriggerTime != newObj.LastChangeTriggerTime {
+				recordDNSProgrammingLatency(newObj.LastChangeTriggerTime)
+			}
+		}
+	}
+}
+
+func (dns *dnsControl) Modified() int64 {
+	return dns.modified.Load()
+}
+
+// updateModified set dns.modified to the current time.
+func (dns *dnsControl) updateModified() {
+	unix := time.Now().Unix()
+	dns.modified.Store(unix)
+}
+
+var errObj = errors.New("obj was not of the correct type")
diff --git a/osbase/net/dns/kubernetes/handler.go b/osbase/net/dns/kubernetes/handler.go
new file mode 100644
index 0000000..e6adf84
--- /dev/null
+++ b/osbase/net/dns/kubernetes/handler.go
@@ -0,0 +1,421 @@
+package kubernetes
+
+import (
+	"math/rand/v2"
+	"net"
+	"net/netip"
+
+	"github.com/miekg/dns"
+
+	netDNS "source.monogon.dev/osbase/net/dns"
+	"source.monogon.dev/osbase/net/dns/kubernetes/object"
+)
+
+const (
+	// DNSSchemaVersion is the schema version: https://github.com/kubernetes/dns/blob/master/docs/specification.md
+	DNSSchemaVersion = "1.1.0"
+	// defaultTTL to apply to all answers.
+	defaultTTL = 5
+)
+
+func (k *Kubernetes) HandleDNS(r *netDNS.Request) {
+	if netDNS.IsSubDomain(k.clusterDomain, r.QnameCanonical) {
+		r.SetAuthoritative()
+
+		subdomain := r.QnameCanonical[:len(r.QnameCanonical)-len(k.clusterDomain)]
+		subdomain, last := netDNS.SplitLastLabel(subdomain)
+		if last == "svc" {
+			k.handleService(r, subdomain)
+		} else if last == "" {
+			if r.Qtype == dns.TypeSOA || r.Qtype == dns.TypeANY {
+				r.Reply.Answer = append(r.Reply.Answer, k.makeSOA(r.Qname))
+			}
+			if r.Qtype == dns.TypeNS || r.Qtype == dns.TypeANY {
+				r.Reply.Answer = append(r.Reply.Answer, k.makeNS(r.Qname))
+			}
+		} else if last == "dns-version" && subdomain == "" {
+			if r.Qtype == dns.TypeTXT || r.Qtype == dns.TypeANY {
+				rr := new(dns.TXT)
+				rr.Hdr = dns.RR_Header{Name: r.Qname, Rrtype: dns.TypeTXT, Class: dns.ClassINET, Ttl: defaultTTL}
+				rr.Txt = []string{DNSSchemaVersion}
+				r.Reply.Answer = append(r.Reply.Answer, rr)
+			}
+		} else if last == "dns" && (subdomain == "" || subdomain == "ns.") {
+			// Name exists but has no records.
+		} else {
+			r.Reply.Rcode = dns.RcodeNameError
+		}
+
+		if r.Handled {
+			return
+		}
+		if len(r.Reply.Answer) == 0 {
+			zone := r.Qname[len(r.Qname)-len(k.clusterDomain):]
+			r.Reply.Ns = []dns.RR{k.makeSOA(zone)}
+		}
+		r.SendReply()
+		return
+	}
+
+	reverseIP, reverseBits, extra := netDNS.ParseReverse(r.QnameCanonical)
+	if reverseIP.IsValid() {
+		for _, ipRange := range k.ipRanges {
+			if !ipRange.Contains(reverseIP) || reverseBits < ipRange.Bits() {
+				continue
+			}
+
+			r.SetAuthoritative()
+
+			zoneBits := 0
+			if reverseIP.BitLen() == 32 {
+				zoneBits = (ipRange.Bits() + 7) & ^7
+			} else {
+				zoneBits = (ipRange.Bits() + 3) & ^3
+			}
+
+			if extra {
+				// Name with extra labels does not exist.
+				r.Reply.Rcode = dns.RcodeNameError
+			} else {
+				if reverseBits == reverseIP.BitLen() {
+					k.handleReverse(r, reverseIP)
+				}
+				if reverseBits == zoneBits {
+					if r.Qtype == dns.TypeSOA || r.Qtype == dns.TypeANY {
+						r.Reply.Answer = append(r.Reply.Answer, k.makeSOA(r.Qname))
+					}
+					if r.Qtype == dns.TypeNS || r.Qtype == dns.TypeANY {
+						r.Reply.Answer = append(r.Reply.Answer, k.makeNS(r.Qname))
+					}
+				}
+			}
+
+			if len(r.Reply.Answer) == 0 {
+				zoneDots := 0
+				if reverseIP.BitLen() == 32 {
+					zoneDots = 3 + zoneBits/8
+				} else {
+					zoneDots = 3 + zoneBits/4
+				}
+				zoneStart := len(r.Qname)
+				for zoneStart > 0 {
+					if r.Qname[zoneStart-1] == '.' {
+						zoneDots--
+						if zoneDots == 0 {
+							break
+						}
+					}
+					zoneStart--
+				}
+				zone := r.Qname[zoneStart:]
+				r.Reply.Ns = []dns.RR{k.makeSOA(zone)}
+			}
+			r.SendReply()
+			return
+		}
+	}
+}
+
+func (k *Kubernetes) handleService(r *netDNS.Request, subdomain string) {
+	if subdomain == "" {
+		// Name exists but has no records.
+		return
+	}
+
+	rest, namespace := netDNS.SplitLastLabel(subdomain)
+	if rest == "" {
+		// Name exists if the namespace exists, and has no records.
+		if !k.apiConn.NamespaceExists(namespace) {
+			k.notFound(r)
+		}
+		return
+	}
+
+	serviceSub, _ := netDNS.SplitLastLabel(rest)
+	rest, hostnameOrProto := netDNS.SplitLastLabel(serviceSub)
+
+	var proto string
+	var portName string
+	var hostname string
+	switch hostnameOrProto {
+	case "_tcp", "_udp", "_sctp":
+		proto = hostnameOrProto[1:]
+		rest, portName = netDNS.SplitLastLabel(rest)
+		if len(portName) >= 2 && portName[0] == '_' {
+			portName = portName[1:]
+		} else if portName != "" {
+			r.Reply.Rcode = dns.RcodeNameError
+			return
+		}
+		// If portName is empty, the name exists if the parent exists,
+		// but has no records.
+	default:
+		hostname = hostnameOrProto
+	}
+
+	if rest != "" {
+		// The query name has too many labels.
+		r.Reply.Rcode = dns.RcodeNameError
+		return
+	}
+
+	// serviceKey is "<service>.<ns>"
+	serviceKey := subdomain[len(serviceSub) : len(subdomain)-1]
+	service := k.apiConn.GetSvc(serviceKey)
+	if service == nil {
+		k.notFound(r)
+		return
+	}
+
+	// External service
+	if service.ExternalName != "" {
+		if serviceSub != "" {
+			// External services don't have subdomains.
+			r.Reply.Rcode = dns.RcodeNameError
+			return
+		}
+		if service.ExternalName == object.ExternalNameInvalid {
+			// The service has an invalid ExternalName, return an error.
+			r.AddExtendedError(dns.ExtendedErrorCodeInvalidData, "Kubernetes service has invalid externalName")
+			r.Reply.Rcode = dns.RcodeServerFailure
+			return
+		}
+		// We already ensure that ExternalName is valid and fully qualified
+		// when constructing the object.Service.
+		r.AddCNAME(service.ExternalName, defaultTTL)
+		return
+	}
+
+	// Headless service.
+	if service.Headless {
+		found := false
+		haveIP := make(map[string]struct{})
+		haveSRV := make(map[srvItem]struct{})
+		existingAnswer := len(r.Reply.Answer)
+		existingExtra := len(r.Reply.Extra)
+		for _, ep := range k.apiConn.EpIndex(serviceKey) {
+			if portName != "" {
+				// _<port>._<proto>.<service>.<ns>.svc.
+				var portNumber uint16
+				for _, p := range ep.Ports {
+					if p.Name == portName && p.Protocol == proto {
+						portNumber = p.Port
+						break
+					}
+				}
+				if portNumber == 0 {
+					continue
+				}
+				for _, addr := range ep.Addresses {
+					found = true
+					if r.Qtype == dns.TypeSRV || r.Qtype == dns.TypeANY {
+						targetName := addr.Hostname + r.Qname[len(serviceSub)-1:]
+						if !isDuplicateSRV(haveSRV, addr.Hostname, "", portNumber) {
+							addSRV(r, portNumber, targetName)
+						}
+						if !isDuplicateSRV(haveSRV, addr.Hostname, addr.IP, 0) {
+							addAddrExtra(r, targetName, net.IP(addr.IP))
+						}
+					}
+				}
+			} else {
+				// <service>.<ns>.svc. or <hostname>.<service>.<ns>.svc.
+				for _, addr := range ep.Addresses {
+					if hostname != "" && hostname != addr.Hostname {
+						continue
+					}
+					found = true
+					if proto != "" {
+						// _<proto>.<service>.<ns>.svc. has no records
+						// and exists if its parent exists.
+						break
+					}
+					if _, ok := haveIP[addr.IP]; !ok {
+						haveIP[addr.IP] = struct{}{}
+						addAddr(r, net.IP(addr.IP))
+					}
+				}
+			}
+		}
+		shuffleRRs(r.Reply.Answer[existingAnswer:])
+		shuffleRRs(r.Reply.Extra[existingExtra:])
+		if !found {
+			k.notFound(r)
+		}
+		return
+	}
+
+	if hostname != "" {
+		// Non-headless services don't have hostname records.
+		r.Reply.Rcode = dns.RcodeNameError
+		return
+	}
+
+	// ClusterIP service
+	if proto == "" {
+		// <service>.<ns>.svc. for ClusterIP service.
+		for _, ip := range service.ClusterIPs {
+			addAddr(r, net.IP(ip))
+		}
+		// The specification does not define what to return if the service has
+		// no (valid) clusterIP. We return an empty response with no error.
+		return
+	}
+
+	if portName == "" {
+		// _<proto>.<service>.<ns>.svc. exists but has no records.
+		return
+	}
+
+	// _<port>._<proto>.<service>.<ns>.svc. for ClusterIP service.
+	var portNumber uint16
+	for _, p := range service.Ports {
+		if p.Name == portName && p.Protocol == proto {
+			portNumber = p.Port
+			break
+		}
+	}
+	if portNumber == 0 {
+		r.Reply.Rcode = dns.RcodeNameError
+		return
+	}
+	if r.Qtype == dns.TypeSRV || r.Qtype == dns.TypeANY {
+		targetName := r.Qname[len(serviceSub):]
+		addSRV(r, portNumber, targetName)
+		for _, ip := range service.ClusterIPs {
+			addAddrExtra(r, targetName, net.IP(ip))
+		}
+	}
+}
+
+func (k *Kubernetes) handleReverse(r *netDNS.Request, ip netip.Addr) {
+	stringIP := string(ip.AsSlice())
+	found := false
+	for _, service := range k.apiConn.SvcIndexReverse(stringIP) {
+		found = true
+		if r.Qtype == dns.TypePTR || r.Qtype == dns.TypeANY {
+			rr := new(dns.PTR)
+			rr.Hdr = dns.RR_Header{Name: r.Qname, Rrtype: dns.TypePTR, Class: dns.ClassINET, Ttl: defaultTTL}
+			rr.Ptr = service.Name + "." + service.Namespace + ".svc." + k.clusterDomain
+			r.Reply.Answer = append(r.Reply.Answer, rr)
+		}
+	}
+	haveName := make(map[string]struct{})
+	for _, ep := range k.apiConn.EpIndexReverse(stringIP) {
+		for _, addr := range ep.Addresses {
+			if addr.IP == stringIP {
+				found = true
+				if r.Qtype == dns.TypePTR || r.Qtype == dns.TypeANY {
+					ptr := addr.Hostname + "." + ep.Index + ".svc." + k.clusterDomain
+					if _, ok := haveName[ptr]; ok {
+						continue
+					}
+					haveName[ptr] = struct{}{}
+					rr := new(dns.PTR)
+					rr.Hdr = dns.RR_Header{Name: r.Qname, Rrtype: dns.TypePTR, Class: dns.ClassINET, Ttl: defaultTTL}
+					rr.Ptr = ptr
+					r.Reply.Answer = append(r.Reply.Answer, rr)
+				}
+			}
+		}
+	}
+	if !found {
+		k.notFound(r)
+	}
+}
+
+func (k *Kubernetes) makeSOA(zone string) *dns.SOA {
+	header := dns.RR_Header{Name: zone, Rrtype: dns.TypeSOA, Class: dns.ClassINET, Ttl: defaultTTL}
+	return &dns.SOA{
+		Hdr:     header,
+		Mbox:    "nobody.invalid.",
+		Ns:      k.nsDomain,
+		Serial:  uint32(k.apiConn.Modified()),
+		Refresh: 7200,
+		Retry:   1800,
+		Expire:  86400,
+		Minttl:  defaultTTL,
+	}
+}
+
+func (k *Kubernetes) makeNS(zone string) *dns.NS {
+	rr := new(dns.NS)
+	rr.Hdr = dns.RR_Header{Name: zone, Rrtype: dns.TypeNS, Class: dns.ClassINET, Ttl: defaultTTL}
+	rr.Ns = k.nsDomain
+	return rr
+}
+
+func addAddr(r *netDNS.Request, ip net.IP) {
+	if len(ip) == net.IPv4len && (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: defaultTTL}
+		rr.A = ip
+		r.Reply.Answer = append(r.Reply.Answer, rr)
+	}
+	if len(ip) == net.IPv6len && (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: defaultTTL}
+		rr.AAAA = ip
+		r.Reply.Answer = append(r.Reply.Answer, rr)
+	}
+}
+
+func addAddrExtra(r *netDNS.Request, name string, ip net.IP) {
+	if len(ip) == net.IPv4len {
+		rr := new(dns.A)
+		rr.Hdr = dns.RR_Header{Name: name, Rrtype: dns.TypeA, Class: dns.ClassINET, Ttl: defaultTTL}
+		rr.A = ip
+		r.Reply.Extra = append(r.Reply.Extra, rr)
+	}
+	if len(ip) == net.IPv6len {
+		rr := new(dns.AAAA)
+		rr.Hdr = dns.RR_Header{Name: name, Rrtype: dns.TypeAAAA, Class: dns.ClassINET, Ttl: defaultTTL}
+		rr.AAAA = ip
+		r.Reply.Extra = append(r.Reply.Extra, rr)
+	}
+}
+
+func addSRV(r *netDNS.Request, portNumber uint16, targetName string) {
+	rr := new(dns.SRV)
+	rr.Hdr = dns.RR_Header{Name: r.Qname, Rrtype: dns.TypeSRV, Class: dns.ClassINET, Ttl: defaultTTL}
+	rr.Priority = 0
+	rr.Weight = 0
+	rr.Port = portNumber
+	rr.Target = targetName
+	r.Reply.Answer = append(r.Reply.Answer, rr)
+}
+
+// notFound should be called if a name was not found, but could exist
+// if there are Kubernetes object that are not yet available locally.
+func (k *Kubernetes) notFound(r *netDNS.Request) {
+	if !k.apiConn.HasSynced() {
+		// We don't know if the name exists or not, so return an error.
+		r.AddExtendedError(dns.ExtendedErrorCodeNotReady, "Kubernetes objects not yet synced")
+		r.Reply.Rcode = dns.RcodeServerFailure
+	} else {
+		r.Reply.Rcode = dns.RcodeNameError
+	}
+}
+
+type srvItem struct {
+	name string
+	addr string
+	port uint16
+}
+
+// isDuplicateSRV returns true if the (name, addr, port) combination already
+// exists in m, and adds it to m if not.
+func isDuplicateSRV(m map[srvItem]struct{}, name, addr string, port uint16) bool {
+	_, ok := m[srvItem{name, addr, port}]
+	if !ok {
+		m[srvItem{name, addr, port}] = struct{}{}
+	}
+	return ok
+}
+
+// shuffleRRs shuffles a slice of RRs for some load balancing.
+func shuffleRRs(rrs []dns.RR) {
+	rand.Shuffle(len(rrs), func(i, j int) {
+		rrs[i], rrs[j] = rrs[j], rrs[i]
+	})
+}
diff --git a/osbase/net/dns/kubernetes/handler_test.go b/osbase/net/dns/kubernetes/handler_test.go
new file mode 100644
index 0000000..76bd32d
--- /dev/null
+++ b/osbase/net/dns/kubernetes/handler_test.go
@@ -0,0 +1,783 @@
+package kubernetes
+
+import (
+	"context"
+	"fmt"
+	"net/netip"
+	"slices"
+	"strings"
+	"testing"
+	"time"
+
+	"github.com/miekg/dns"
+	api "k8s.io/api/core/v1"
+	discovery "k8s.io/api/discovery/v1"
+	meta "k8s.io/apimachinery/pkg/apis/meta/v1"
+	"k8s.io/apimachinery/pkg/util/intstr"
+	"k8s.io/client-go/kubernetes/fake"
+	"k8s.io/utils/ptr"
+
+	netDNS "source.monogon.dev/osbase/net/dns"
+)
+
+const testdataClusterDomain = "cluster.local"
+
+var testdataIPRanges = []string{
+	// service IP
+	"10.0.0.1/16",
+	"1234:abcd::/64",
+	// pod IP
+	"172.32.0.0/11",
+	"170::/14",
+}
+
+var testdataNamespaces = []string{"testns"}
+
+var testdataServices = []*api.Service{
+	{
+		ObjectMeta: meta.ObjectMeta{
+			Name:      "svc-clusterip",
+			Namespace: "testns",
+		},
+		Spec: api.ServiceSpec{
+			Type:       api.ServiceTypeClusterIP,
+			ClusterIPs: []string{"10.0.0.10"},
+			Ports: []api.ServicePort{
+				{Name: "http", Protocol: api.ProtocolTCP, Port: 80, TargetPort: intstr.FromInt32(82)},
+			},
+		},
+	},
+	{
+		ObjectMeta: meta.ObjectMeta{
+			Name:      "svc-dualstack",
+			Namespace: "testns",
+		},
+		Spec: api.ServiceSpec{
+			Type:       api.ServiceTypeClusterIP,
+			ClusterIPs: []string{"10.0.0.11", "1234:abcd::11"},
+			Ports: []api.ServicePort{
+				{Name: "http", Protocol: api.ProtocolTCP, Port: 80},
+				{Name: "dns", Protocol: api.ProtocolUDP, Port: 53},
+			},
+		},
+	},
+	{
+		ObjectMeta: meta.ObjectMeta{
+			Name:      "svc-headless",
+			Namespace: "testns",
+		},
+		Spec: api.ServiceSpec{
+			Type:       api.ServiceTypeClusterIP,
+			ClusterIP:  api.ClusterIPNone,
+			ClusterIPs: []string{api.ClusterIPNone},
+			Ports: []api.ServicePort{
+				{Name: "http", Protocol: api.ProtocolTCP, Port: 80, TargetPort: intstr.FromString("http")},
+			},
+		},
+	},
+	{
+		ObjectMeta: meta.ObjectMeta{
+			Name:      "svc-headless-notready",
+			Namespace: "testns",
+		},
+		Spec: api.ServiceSpec{
+			Type:       api.ServiceTypeClusterIP,
+			ClusterIP:  api.ClusterIPNone,
+			ClusterIPs: []string{api.ClusterIPNone},
+			Ports: []api.ServicePort{
+				{Name: "http", Protocol: api.ProtocolTCP, Port: 80, TargetPort: intstr.FromString("http")},
+			},
+		},
+	},
+	{
+		ObjectMeta: meta.ObjectMeta{
+			Name:      "svc-external",
+			Namespace: "testns",
+		},
+		Spec: api.ServiceSpec{
+			Type:         api.ServiceTypeExternalName,
+			ExternalName: "external.example.com",
+		},
+	},
+}
+
+var testdataEndpointSlices = []*discovery.EndpointSlice{
+	{
+		ObjectMeta: meta.ObjectMeta{
+			Name:      "svc-clusterip-slice",
+			Namespace: "testns",
+			Labels:    map[string]string{discovery.LabelServiceName: "svc-clusterip"},
+		},
+		Endpoints: []discovery.Endpoint{
+			{Addresses: []string{"172.45.0.1"}},
+		},
+	},
+	{
+		ObjectMeta: meta.ObjectMeta{
+			Name:      "svc-headless-slice1",
+			Namespace: "testns",
+			Labels: map[string]string{
+				discovery.LabelServiceName: "svc-headless",
+				api.IsHeadlessService:      "",
+			},
+		},
+		Endpoints: []discovery.Endpoint{
+			{
+				Addresses: []string{"172.45.0.2"},
+			},
+			{
+				Addresses: []string{"172.45.0.2"},
+			},
+			{
+				Hostname:   ptr.To("pod3"),
+				Addresses:  []string{"172.45.0.3"},
+				Conditions: discovery.EndpointConditions{Ready: ptr.To(true)},
+			},
+			{
+				Addresses:  []string{"172.45.0.4"},
+				Conditions: discovery.EndpointConditions{Ready: ptr.To(false)},
+			},
+			{
+				Hostname:  ptr.To("pod5"),
+				Addresses: []string{"172.45.0.5", "172.45.0.2"},
+			},
+		},
+		Ports: []discovery.EndpointPort{
+			{Name: ptr.To("http"), Port: ptr.To(int32(8000)), Protocol: ptr.To(api.ProtocolTCP)},
+		},
+	},
+	{
+		ObjectMeta: meta.ObjectMeta{
+			Name:      "svc-headless-slice2",
+			Namespace: "testns",
+			Labels: map[string]string{
+				discovery.LabelServiceName: "svc-headless",
+				api.IsHeadlessService:      "",
+			},
+		},
+		Endpoints: []discovery.Endpoint{
+			{
+				Hostname:   ptr.To("pod3"),
+				Addresses:  []string{"172::3"},
+				Conditions: discovery.EndpointConditions{Ready: ptr.To(false)},
+			},
+			{
+				Hostname:  ptr.To("pod5"),
+				Addresses: []string{"172::5"},
+			},
+			{
+				Addresses: []string{"172::7"},
+			},
+		},
+		Ports: []discovery.EndpointPort{
+			{Name: ptr.To("http"), Port: ptr.To(int32(8001)), Protocol: ptr.To(api.ProtocolTCP)},
+		},
+	},
+	{
+		ObjectMeta: meta.ObjectMeta{
+			Name:      "svc-headless-notready-slice1",
+			Namespace: "testns",
+			Labels: map[string]string{
+				discovery.LabelServiceName: "svc-headless-notready",
+				api.IsHeadlessService:      "",
+			},
+		},
+		Endpoints: []discovery.Endpoint{
+			{
+				Addresses:  []string{"172.45.0.20"},
+				Conditions: discovery.EndpointConditions{Ready: ptr.To(false)},
+			},
+			{
+				Hostname:   ptr.To("pod21"),
+				Addresses:  []string{"172.45.0.21"},
+				Conditions: discovery.EndpointConditions{Ready: ptr.To(false)},
+			},
+		},
+		Ports: []discovery.EndpointPort{
+			{Name: ptr.To("http"), Port: ptr.To(int32(8000)), Protocol: ptr.To(api.ProtocolTCP)},
+		},
+	},
+}
+
+// handlerTestcase contains a query name, and the expected records
+// under that name given the above test data.
+type handlerTestcase struct {
+	// Query name
+	qname string
+
+	// Expected reply
+
+	rcode         int
+	answer, extra []string
+	notHandled    bool
+	// zone is the zone that is expected in the NS SOA if the answer is empty.
+	// If zone is empty, defaults to "cluster.local."
+	zone string
+}
+
+// nameErrorIfSynced means name error if synced, else server failure.
+const nameErrorIfSynced = -1
+
+var handlerTestcases = []handlerTestcase{
+	// cluster domain root
+	{
+		qname: "cluster.local.",
+		answer: []string{
+			"cluster.local.	5	IN	SOA	ns.dns.cluster.local. nobody.invalid. 12345 7200 1800 86400 5",
+			"cluster.local.	5	IN	NS	ns.dns.cluster.local.",
+		},
+	},
+	{
+		qname: "example.cluster.local.",
+		rcode: dns.RcodeNameError,
+	},
+	// dns-version
+	{
+		qname: "dns-version.cluster.local.",
+		answer: []string{
+			`dns-version.cluster.local.	5	IN	TXT	"1.1.0"`,
+		},
+	},
+	{
+		qname: "example.dns-version.cluster.local.",
+		rcode: dns.RcodeNameError,
+	},
+	// ns.dns
+	{
+		qname: "dns.cluster.local.",
+	},
+	{
+		qname: "ns.dns.cluster.local.",
+	},
+	{
+		qname: "example.dns.cluster.local.",
+		rcode: dns.RcodeNameError,
+	},
+	// svc
+	{
+		qname: "svc.cluster.local.",
+	},
+	// namespace
+	{
+		qname: "testns.svc.cluster.local.",
+	},
+	{
+		qname: "inexistent-ns.svc.cluster.local.",
+		rcode: nameErrorIfSynced,
+	},
+	// cluster IP service
+	{
+		qname: "svc-clusterip.testns.svc.cluster.local.",
+		answer: []string{
+			"svc-clusterip.testns.svc.cluster.local.	5	IN	A	10.0.0.10",
+		},
+	},
+	{
+		qname: "_http._tcp.svc-clusterip.testns.svc.cluster.local.",
+		answer: []string{
+			"_http._tcp.svc-clusterip.testns.svc.cluster.local.	5	IN	SRV	0 0 80 svc-clusterip.testns.svc.cluster.local.",
+		},
+		extra: []string{
+			"svc-clusterip.testns.svc.cluster.local.	5	IN	A	10.0.0.10",
+		},
+	},
+	{
+		qname: "_udp.svc-clusterip.testns.svc.cluster.local.",
+	},
+	{
+		qname: "_http._udp.svc-clusterip.testns.svc.cluster.local.",
+		rcode: dns.RcodeNameError,
+	},
+	{
+		qname: "http._tcp.svc-clusterip.testns.svc.cluster.local.",
+		rcode: dns.RcodeNameError,
+	},
+	{
+		qname: "example._http._tcp.svc-clusterip.testns.svc.cluster.local.",
+		rcode: dns.RcodeNameError,
+	},
+	{
+		qname: "10.0.0.10.in-addr.arpa.",
+		answer: []string{
+			"10.0.0.10.in-addr.arpa.	5	IN	PTR	svc-clusterip.testns.svc.cluster.local.",
+		},
+		zone: "0.10.in-addr.arpa.",
+	},
+	{
+		qname: "172-45-0-1.svc-clusterip.testns.svc.cluster.local.",
+		rcode: dns.RcodeNameError,
+	},
+	{
+		qname: "1.0.45.172.in-addr.arpa.",
+		rcode: nameErrorIfSynced,
+		zone:  "45.172.in-addr.arpa.",
+	},
+	// dual stack cluster IP service
+	{
+		qname: "svc-dualstack.testns.svc.cluster.local.",
+		answer: []string{
+			"svc-dualstack.testns.svc.cluster.local.	5	IN	A	10.0.0.11",
+			"svc-dualstack.testns.svc.cluster.local.	5	IN	AAAA	1234:abcd::11",
+		},
+	},
+	{
+		qname: "_http._tcp.svc-dualstack.testns.svc.cluster.local.",
+		answer: []string{
+			"_http._tcp.svc-dualstack.testns.svc.cluster.local.	5	IN	SRV	0 0 80 svc-dualstack.testns.svc.cluster.local.",
+		},
+		extra: []string{
+			"svc-dualstack.testns.svc.cluster.local.	5	IN	A	10.0.0.11",
+			"svc-dualstack.testns.svc.cluster.local.	5	IN	AAAA	1234:abcd::11",
+		},
+	},
+	{
+		qname: "11.0.0.10.in-addr.arpa.",
+		answer: []string{
+			"11.0.0.10.in-addr.arpa.	5	IN	PTR	svc-dualstack.testns.svc.cluster.local.",
+		},
+		zone: "0.10.in-addr.arpa.",
+	},
+	{
+		qname: "1.1.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.d.c.b.a.4.3.2.1.ip6.arpa.",
+		answer: []string{
+			"1.1.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.d.c.b.a.4.3.2.1.ip6.arpa.	5	IN	PTR	svc-dualstack.testns.svc.cluster.local.",
+		},
+		zone: "0.0.0.0.0.0.0.0.d.c.b.a.4.3.2.1.ip6.arpa.",
+	},
+	// headless service
+	{
+		qname: "svc-headless.testns.svc.cluster.local.",
+		answer: []string{
+			"svc-headless.testns.svc.cluster.local.	5	IN	A	172.45.0.2",
+			"svc-headless.testns.svc.cluster.local.	5	IN	A	172.45.0.3",
+			"svc-headless.testns.svc.cluster.local.	5	IN	A	172.45.0.5",
+			"svc-headless.testns.svc.cluster.local.	5	IN	AAAA	172::5",
+			"svc-headless.testns.svc.cluster.local.	5	IN	AAAA	172::7",
+		},
+	},
+	{
+		qname: "_http._tcp.svc-headless.testns.svc.cluster.local.",
+		answer: []string{
+			"_http._tcp.svc-headless.testns.svc.cluster.local.	5	IN	SRV	0 0 8000 172-45-0-2.svc-headless.testns.svc.cluster.local.",
+			"_http._tcp.svc-headless.testns.svc.cluster.local.	5	IN	SRV	0 0 8000 pod3.svc-headless.testns.svc.cluster.local.",
+			"_http._tcp.svc-headless.testns.svc.cluster.local.	5	IN	SRV	0 0 8000 pod5.svc-headless.testns.svc.cluster.local.",
+			"_http._tcp.svc-headless.testns.svc.cluster.local.	5	IN	SRV	0 0 8001 pod5.svc-headless.testns.svc.cluster.local.",
+			"_http._tcp.svc-headless.testns.svc.cluster.local.	5	IN	SRV	0 0 8001 172--7.svc-headless.testns.svc.cluster.local.",
+		},
+		extra: []string{
+			"172-45-0-2.svc-headless.testns.svc.cluster.local.	5	IN	A	172.45.0.2",
+			"pod3.svc-headless.testns.svc.cluster.local.	5	IN	A	172.45.0.3",
+			"pod5.svc-headless.testns.svc.cluster.local.	5	IN	A	172.45.0.5",
+			"pod5.svc-headless.testns.svc.cluster.local.	5	IN	A	172.45.0.2",
+			"pod5.svc-headless.testns.svc.cluster.local.	5	IN	AAAA	172::5",
+			"172--7.svc-headless.testns.svc.cluster.local.	5	IN	AAAA	172::7",
+		},
+	},
+	{
+		qname: "_udp.svc-headless.testns.svc.cluster.local.",
+	},
+	{
+		qname: "_http._udp.svc-headless.testns.svc.cluster.local.",
+		rcode: nameErrorIfSynced,
+	},
+	{
+		qname: "http._tcp.svc-headless.testns.svc.cluster.local.",
+		rcode: dns.RcodeNameError,
+	},
+	{
+		qname: "_._udp.svc-headless.testns.svc.cluster.local.",
+		rcode: dns.RcodeNameError,
+	},
+	{
+		qname: "example._http._tcp.svc-headless.testns.svc.cluster.local.",
+		rcode: dns.RcodeNameError,
+	},
+	{
+		qname: "172-45-0-2.svc-headless.testns.svc.cluster.local.",
+		answer: []string{
+			"172-45-0-2.svc-headless.testns.svc.cluster.local.	5	IN	A	172.45.0.2",
+		},
+	},
+	{
+		qname: "pod5.svc-headless.testns.svc.cluster.local.",
+		answer: []string{
+			"pod5.svc-headless.testns.svc.cluster.local.	5	IN	A	172.45.0.5",
+			"pod5.svc-headless.testns.svc.cluster.local.	5	IN	A	172.45.0.2",
+			"pod5.svc-headless.testns.svc.cluster.local.	5	IN	AAAA	172::5",
+		},
+	},
+	{
+		qname: "example.pod5.svc-headless.testns.svc.cluster.local.",
+		rcode: dns.RcodeNameError,
+	},
+	{
+		qname: "172-45-0-5.svc-headless.testns.svc.cluster.local.",
+		rcode: nameErrorIfSynced,
+	},
+	{
+		qname: "2.0.45.172.in-addr.arpa.",
+		answer: []string{
+			"2.0.45.172.in-addr.arpa.	5	IN	PTR	172-45-0-2.svc-headless.testns.svc.cluster.local.",
+			"2.0.45.172.in-addr.arpa.	5	IN	PTR	pod5.svc-headless.testns.svc.cluster.local.",
+		},
+		zone: "45.172.in-addr.arpa.",
+	},
+	{
+		qname: "5.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.2.7.1.0.ip6.arpa.",
+		answer: []string{
+			"5.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.2.7.1.0.ip6.arpa.	5	IN	PTR	pod5.svc-headless.testns.svc.cluster.local.",
+		},
+		zone: "2.7.1.0.ip6.arpa.",
+	},
+	// not ready headless service
+	{
+		qname: "svc-headless-notready.testns.svc.cluster.local.",
+		rcode: nameErrorIfSynced,
+	},
+	{
+		qname: "_tcp.svc-headless-notready.testns.svc.cluster.local.",
+		rcode: nameErrorIfSynced,
+	},
+	{
+		qname: "_http._tcp.svc-headless-notready.testns.svc.cluster.local.",
+		rcode: nameErrorIfSynced,
+	},
+	{
+		qname: "pod21.svc-headless-notready.testns.svc.cluster.local.",
+		rcode: nameErrorIfSynced,
+	},
+	{
+		qname: "21.0.45.172.in-addr.arpa.",
+		rcode: nameErrorIfSynced,
+		zone:  "45.172.in-addr.arpa.",
+	},
+	// external service
+	{
+		qname: "svc-external.testns.svc.cluster.local.",
+		answer: []string{
+			"svc-external.testns.svc.cluster.local.	5	IN	CNAME	external.example.com.",
+		},
+	},
+	{
+		qname: "_tcp.svc-external.testns.svc.cluster.local.",
+		rcode: dns.RcodeNameError,
+	},
+	{
+		qname: "_http._tcp.svc-external.testns.svc.cluster.local.",
+		rcode: dns.RcodeNameError,
+	},
+	{
+		qname: "pod.svc-external.testns.svc.cluster.local.",
+		rcode: dns.RcodeNameError,
+	},
+	// service does not exist
+	{
+		qname: "inexistent-svc.testns.svc.cluster.local.",
+		rcode: nameErrorIfSynced,
+	},
+	{
+		qname: "_tcp.inexistent-svc.testns.svc.cluster.local.",
+		rcode: nameErrorIfSynced,
+	},
+	{
+		qname: "_http._tcp.inexistent-svc.testns.svc.cluster.local.",
+		rcode: nameErrorIfSynced,
+	},
+	{
+		qname: "example._tcp.inexistent-svc.testns.svc.cluster.local.",
+		rcode: dns.RcodeNameError,
+	},
+	{
+		qname: "example._http._tcp.inexistent-svc.testns.svc.cluster.local.",
+		rcode: dns.RcodeNameError,
+	},
+	{
+		qname: "pod.inexistent-svc.testns.svc.cluster.local.",
+		rcode: nameErrorIfSynced,
+	},
+	{
+		qname: "example.pod.inexistent-svc.testns.svc.cluster.local.",
+		rcode: dns.RcodeNameError,
+	},
+	// names which do not exist but will get queried because of ndots=5
+	{
+		qname: "www.example.com.cluster.local.",
+		rcode: dns.RcodeNameError,
+	},
+	{
+		qname: "www.example.com.svc.cluster.local.",
+		rcode: nameErrorIfSynced,
+	},
+	{
+		qname: "www.example.com.testns.svc.cluster.local.",
+		rcode: dns.RcodeNameError,
+	},
+	// names which are not handled
+	{
+		qname:      "www.example.com.",
+		notHandled: true,
+	},
+	{
+		qname:      "12.0.31.172.in-addr.arpa.",
+		notHandled: true,
+	},
+	{
+		qname:      "5.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.4.7.1.0.ip6.arpa.",
+		notHandled: true,
+	},
+	{
+		qname:      "10.in-addr.arpa.",
+		notHandled: true,
+	},
+	{
+		qname:      "7.1.0.ip6.arpa.",
+		notHandled: true,
+	},
+	// reverse lookup zone
+	{
+		qname: "45.172.in-addr.arpa.",
+		answer: []string{
+			"45.172.in-addr.arpa.	5	IN	SOA	ns.dns.cluster.local. nobody.invalid. 12345 7200 1800 86400 5",
+			"45.172.in-addr.arpa.	5	IN	NS	ns.dns.cluster.local.",
+		},
+		zone: "45.172.in-addr.arpa.",
+	},
+	{
+		qname: "255.45.172.in-addr.arpa.",
+		zone:  "45.172.in-addr.arpa.",
+	},
+	{
+		qname: "02.0.45.172.in-addr.arpa.",
+		rcode: dns.RcodeNameError,
+		zone:  "45.172.in-addr.arpa.",
+	},
+	{
+		qname: "1.2.0.45.172.in-addr.arpa.",
+		rcode: dns.RcodeNameError,
+		zone:  "45.172.in-addr.arpa.",
+	},
+	{
+		qname: "2.7.1.0.ip6.arpa.",
+		answer: []string{
+			"2.7.1.0.ip6.arpa.	5	IN	SOA	ns.dns.cluster.local. nobody.invalid. 12345 7200 1800 86400 5",
+			"2.7.1.0.ip6.arpa.	5	IN	NS	ns.dns.cluster.local.",
+		},
+		zone: "2.7.1.0.ip6.arpa.",
+	},
+	{
+		qname: "a.2.7.1.0.ip6.arpa.",
+		zone:  "2.7.1.0.ip6.arpa.",
+	},
+	{
+		qname: "x.a.2.7.1.0.ip6.arpa.",
+		rcode: dns.RcodeNameError,
+		zone:  "2.7.1.0.ip6.arpa.",
+	},
+	// mixed case
+	{
+		qname: "SvC-cLUSteRIp.TesTNS.sVC.ClUSTer.locAL.",
+		answer: []string{
+			"SvC-cLUSteRIp.TesTNS.sVC.ClUSTer.locAL.	5	IN	A	10.0.0.10",
+		},
+		zone: "ClUSTer.locAL.",
+	},
+	{
+		qname: "_hTTp._tCp.SVc-hEADlEsS.teSTNs.SVC.ClUSTer.locAL.",
+		answer: []string{
+			"_hTTp._tCp.SVc-hEADlEsS.teSTNs.SVC.ClUSTer.locAL.	5	IN	SRV	0 0 8000 172-45-0-2.SVc-hEADlEsS.teSTNs.SVC.ClUSTer.locAL.",
+			"_hTTp._tCp.SVc-hEADlEsS.teSTNs.SVC.ClUSTer.locAL.	5	IN	SRV	0 0 8000 pod3.SVc-hEADlEsS.teSTNs.SVC.ClUSTer.locAL.",
+			"_hTTp._tCp.SVc-hEADlEsS.teSTNs.SVC.ClUSTer.locAL.	5	IN	SRV	0 0 8000 pod5.SVc-hEADlEsS.teSTNs.SVC.ClUSTer.locAL.",
+			"_hTTp._tCp.SVc-hEADlEsS.teSTNs.SVC.ClUSTer.locAL.	5	IN	SRV	0 0 8001 pod5.SVc-hEADlEsS.teSTNs.SVC.ClUSTer.locAL.",
+			"_hTTp._tCp.SVc-hEADlEsS.teSTNs.SVC.ClUSTer.locAL.	5	IN	SRV	0 0 8001 172--7.SVc-hEADlEsS.teSTNs.SVC.ClUSTer.locAL.",
+		},
+		extra: []string{
+			"172-45-0-2.SVc-hEADlEsS.teSTNs.SVC.ClUSTer.locAL.	5	IN	A	172.45.0.2",
+			"pod3.SVc-hEADlEsS.teSTNs.SVC.ClUSTer.locAL.	5	IN	A	172.45.0.3",
+			"pod5.SVc-hEADlEsS.teSTNs.SVC.ClUSTer.locAL.	5	IN	A	172.45.0.5",
+			"pod5.SVc-hEADlEsS.teSTNs.SVC.ClUSTer.locAL.	5	IN	A	172.45.0.2",
+			"pod5.SVc-hEADlEsS.teSTNs.SVC.ClUSTer.locAL.	5	IN	AAAA	172::5",
+			"172--7.SVc-hEADlEsS.teSTNs.SVC.ClUSTer.locAL.	5	IN	AAAA	172::7",
+		},
+		zone: "ClUSTer.locAL.",
+	},
+	{
+		qname: "1.1.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.d.C.b.a.4.3.2.1.iP6.ARpa.",
+		answer: []string{
+			"1.1.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.d.C.b.a.4.3.2.1.iP6.ARpa.	5	IN	PTR	svc-dualstack.testns.svc.cluster.local.",
+		},
+		zone: "0.0.0.0.0.0.0.0.d.C.b.a.4.3.2.1.iP6.ARpa.",
+	},
+}
+
+// TestHandler constructs a fake Kubernetes clientset containing the above
+// testdata, and then evaluates each test case in handlerTestcases.
+func TestHandler(t *testing.T) {
+	ctx := context.Background()
+	client := fake.NewSimpleClientset()
+
+	// Add resources
+	for _, name := range testdataNamespaces {
+		namespace := &api.Namespace{
+			ObjectMeta: meta.ObjectMeta{Name: name},
+		}
+		_, err := client.CoreV1().Namespaces().Create(ctx, namespace, meta.CreateOptions{})
+		if err != nil {
+			t.Fatal(err)
+		}
+	}
+	for _, service := range testdataServices {
+		_, err := client.CoreV1().Services(service.Namespace).Create(ctx, service, meta.CreateOptions{})
+		if err != nil {
+			t.Fatal(err)
+		}
+	}
+	for _, slice := range testdataEndpointSlices {
+		_, err := client.DiscoveryV1().EndpointSlices(slice.Namespace).Create(ctx, slice, meta.CreateOptions{})
+		if err != nil {
+			t.Fatal(err)
+		}
+	}
+
+	// Create handler
+	var ipRanges []netip.Prefix
+	for _, ipRange := range testdataIPRanges {
+		ipRanges = append(ipRanges, netip.MustParsePrefix(ipRange))
+	}
+	handler := New(testdataClusterDomain, ipRanges)
+	handler.ClientSet = client
+
+	wrapper := &dnsControllerWrapper{dnsController: newdnsController(ctx, handler.ClientSet)}
+	handler.apiConn = wrapper
+
+	stopCh := make(chan struct{})
+	defer close(stopCh)
+	handler.apiConn.Start(stopCh)
+	for !wrapper.dnsController.HasSynced() {
+		time.Sleep(time.Millisecond)
+	}
+
+	for _, hasSynced := range []bool{true, false} {
+		wrapper.hasSynced = hasSynced
+		for _, testcase := range handlerTestcases {
+			if testcase.zone == "" {
+				testcase.zone = "cluster.local."
+			}
+			if testcase.rcode == nameErrorIfSynced {
+				if hasSynced {
+					testcase.rcode = dns.RcodeNameError
+				} else {
+					testcase.rcode = dns.RcodeServerFailure
+				}
+			}
+
+			qtypes := []uint16{
+				dns.TypeANY, dns.TypeA, dns.TypeAAAA, dns.TypeSRV, dns.TypeTXT,
+				dns.TypeNS, dns.TypeSOA, dns.TypePTR, dns.TypeMX, dns.TypeCNAME,
+			}
+			for _, qtype := range qtypes {
+				doHandlerTestcase(t, handler, testcase, qtype)
+			}
+		}
+	}
+
+	wrapper.hasSynced = false
+	testNotSyncedOpt(t, handler)
+}
+
+func doHandlerTestcase(t *testing.T, handler *Kubernetes, testcase handlerTestcase, qtype uint16) {
+	// Create request
+	req := netDNS.CreateTestRequest(testcase.qname, qtype, "udp")
+	req.Reply.RecursionDesired = false
+	req.Qopt = nil
+	req.Ropt = nil
+
+	handler.HandleDNS(req)
+
+	caseName := fmt.Sprintf("Query %s %s", testcase.qname, dns.TypeToString[qtype])
+	if !handler.apiConn.HasSynced() {
+		caseName += " not_synced"
+	}
+
+	if req.Handled != !testcase.notHandled {
+		t.Errorf("%s: Expected handled %v, got %v", caseName,
+			!testcase.notHandled, req.Handled,
+		)
+		return
+	}
+	if !req.Handled {
+		return
+	}
+
+	if req.Reply.Rcode != testcase.rcode {
+		t.Errorf("%s: Expected rcode %s, got %s", caseName,
+			dns.RcodeToString[testcase.rcode], dns.RcodeToString[req.Reply.Rcode],
+		)
+		return
+	}
+
+	// Create expected answer
+	var answer []string
+	for _, rr := range testcase.answer {
+		rrParsed, err := dns.NewRR(rr)
+		if err != nil {
+			t.Fatalf("Failed to parse DNS RR %q: %v", rr, err)
+		}
+		if qtype == dns.TypeANY || qtype == rrParsed.Header().Rrtype || rrParsed.Header().Rrtype == dns.TypeCNAME {
+			answer = append(answer, rr)
+		}
+	}
+	var extra []string
+	var ns []string
+	if len(answer) != 0 {
+		extra = testcase.extra
+	} else {
+		ns = []string{
+			testcase.zone + "	5	IN	SOA	ns.dns.cluster.local. nobody.invalid. 12345 7200 1800 86400 5",
+		}
+	}
+
+	checkReplySection(t, caseName, "answer", answer, req.Reply.Answer)
+	checkReplySection(t, caseName, "ns", ns, req.Reply.Ns)
+	checkReplySection(t, caseName, "extra", extra, req.Reply.Extra)
+}
+
+func checkReplySection(t *testing.T, caseName string, sectionName string, expected []string, got []dns.RR) {
+	slices.Sort(expected)
+	var gotStr []string
+	for _, rr := range got {
+		gotStr = append(gotStr, rr.String())
+	}
+	slices.Sort(gotStr)
+	if !slices.Equal(expected, gotStr) {
+		t.Errorf("%s: Expected %s:\n%s\nGot:\n%v", caseName, sectionName,
+			strings.Join(expected, "\n"), strings.Join(gotStr, "\n"))
+	}
+}
+
+// testNotSyncedOpt tests that we get the Not Ready extended error
+// when not synced and an OPT is present and no result was found.
+func testNotSyncedOpt(t *testing.T, handler *Kubernetes) {
+	req := netDNS.CreateTestRequest("inexistent-ns.svc.cluster.local.", dns.TypeA, "udp")
+
+	handler.HandleDNS(req)
+	extra := []string{
+		"\n" +
+			";; OPT PSEUDOSECTION:\n" +
+			"; EDNS: version 0; flags:; udp: 1232\n" +
+			"; EDE: 14 (Not Ready): (Kubernetes objects not yet synced)",
+	}
+	checkReplySection(t, "testNotSyncedOpt", "extra", extra, req.Reply.Extra)
+}
+
+type dnsControllerWrapper struct {
+	dnsController
+	hasSynced bool
+}
+
+func (dns *dnsControllerWrapper) HasSynced() bool {
+	return dns.hasSynced
+}
+
+func (dns *dnsControllerWrapper) Modified() int64 {
+	return 12345
+}
diff --git a/osbase/net/dns/kubernetes/kubernetes.go b/osbase/net/dns/kubernetes/kubernetes.go
new file mode 100644
index 0000000..efcb60e
--- /dev/null
+++ b/osbase/net/dns/kubernetes/kubernetes.go
@@ -0,0 +1,47 @@
+// Package kubernetes provides the kubernetes backend.
+package kubernetes
+
+// Taken and modified from the Kubernetes plugin of CoreDNS, under Apache 2.0.
+
+import (
+	"context"
+	"net/netip"
+
+	"github.com/miekg/dns"
+	"k8s.io/client-go/kubernetes"
+
+	"source.monogon.dev/osbase/supervisor"
+)
+
+// Kubernetes is a DNS handler that implements the Kubernetes
+// DNS-Based Service Discovery specification.
+// https://github.com/kubernetes/dns/blob/master/docs/specification.md
+type Kubernetes struct {
+	clusterDomain string
+	nsDomain      string
+	ipRanges      []netip.Prefix
+	// A Kubernetes ClientSet with read access to endpoints and services
+	ClientSet kubernetes.Interface
+	apiConn   dnsController
+}
+
+// New returns an initialized Kubernetes. Kubernetes DNS records will be served
+// under the clusterDomain. Additionally, reverse queries for services and pods
+// are served under the given ipRanges.
+func New(clusterDomain string, ipRanges []netip.Prefix) *Kubernetes {
+	k := new(Kubernetes)
+	k.clusterDomain = dns.CanonicalName(clusterDomain)
+	k.nsDomain = "ns.dns." + k.clusterDomain
+	k.ipRanges = ipRanges
+	return k
+}
+
+// Run maintains the in-memory cache of Kubernetes services and endpoints.
+func (k *Kubernetes) Run(ctx context.Context) error {
+	k.apiConn = newdnsController(ctx, k.ClientSet)
+	k.apiConn.Start(ctx.Done())
+
+	supervisor.Signal(ctx, supervisor.SignalHealthy)
+	<-ctx.Done()
+	return ctx.Err()
+}
diff --git a/osbase/net/dns/kubernetes/metrics.go b/osbase/net/dns/kubernetes/metrics.go
new file mode 100644
index 0000000..83deeee
--- /dev/null
+++ b/osbase/net/dns/kubernetes/metrics.go
@@ -0,0 +1,44 @@
+package kubernetes
+
+// Taken and modified from the Kubernetes plugin of CoreDNS, under Apache 2.0.
+
+import (
+	"time"
+
+	"github.com/prometheus/client_golang/prometheus"
+
+	"source.monogon.dev/osbase/net/dns"
+)
+
+var (
+	// dnsProgrammingLatency is defined as the time it took to program a DNS
+	// instance - from the time a service or pod has changed to the time the
+	// change was propagated and was available to be served by a DNS server.
+	// The definition of this SLI can be found at https://github.com/kubernetes/community/blob/master/sig-scalability/slos/dns_programming_latency.md
+	// Note that the metrics is partially based on the time exported by the
+	// endpoints controller on the master machine. The measurement may be
+	// inaccurate if there is a clock drift between the node and master machine.
+	// The service_kind label can be one of:
+	//   * cluster_ip
+	//   * headless_with_selector
+	//   * headless_without_selector
+	dnsProgrammingLatency = dns.MetricsFactory.NewHistogramVec(prometheus.HistogramOpts{
+		Namespace: "dnsserver",
+		Subsystem: "kubernetes",
+		Name:      "dns_programming_duration_seconds",
+		// From 1 millisecond to ~17 minutes.
+		Buckets: prometheus.ExponentialBuckets(0.001, 2, 20),
+		Help:    "In Cluster DNS Programming Latency in seconds",
+	}, []string{"service_kind"})
+)
+
+func recordDNSProgrammingLatency(lastChangeTriggerTime time.Time) {
+	if !lastChangeTriggerTime.IsZero() {
+		// If we're here it means that the Endpoints object is for a headless service
+		// and that the Endpoints object was created by the endpoints-controller
+		// (because the LastChangeTriggerTime annotation is set). It means that the
+		// corresponding service is a "headless service with selector".
+		dnsProgrammingLatency.WithLabelValues("headless_with_selector").
+			Observe(time.Since(lastChangeTriggerTime).Seconds())
+	}
+}
diff --git a/osbase/net/dns/kubernetes/object/BUILD.bazel b/osbase/net/dns/kubernetes/object/BUILD.bazel
new file mode 100644
index 0000000..d851b57
--- /dev/null
+++ b/osbase/net/dns/kubernetes/object/BUILD.bazel
@@ -0,0 +1,36 @@
+load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test")
+
+go_library(
+    name = "object",
+    srcs = [
+        "endpoint.go",
+        "informer.go",
+        "namespace.go",
+        "object.go",
+        "service.go",
+    ],
+    importpath = "source.monogon.dev/osbase/net/dns/kubernetes/object",
+    visibility = ["//osbase/net/dns/kubernetes:__subpackages__"],
+    deps = [
+        "@com_github_miekg_dns//:dns",
+        "@io_k8s_api//core/v1:core",
+        "@io_k8s_api//discovery/v1:discovery",
+        "@io_k8s_apimachinery//pkg/api/meta",
+        "@io_k8s_apimachinery//pkg/apis/meta/v1:meta",
+        "@io_k8s_apimachinery//pkg/runtime",
+        "@io_k8s_apimachinery//pkg/runtime/schema",
+        "@io_k8s_apimachinery//pkg/types",
+        "@io_k8s_client_go//tools/cache",
+    ],
+)
+
+go_test(
+    name = "object_test",
+    srcs = ["informer_test.go"],
+    embed = [":object"],
+    deps = [
+        "@io_k8s_api//core/v1:core",
+        "@io_k8s_apimachinery//pkg/apis/meta/v1:meta",
+        "@io_k8s_client_go//tools/cache",
+    ],
+)
diff --git a/osbase/net/dns/kubernetes/object/endpoint.go b/osbase/net/dns/kubernetes/object/endpoint.go
new file mode 100644
index 0000000..a675ad2
--- /dev/null
+++ b/osbase/net/dns/kubernetes/object/endpoint.go
@@ -0,0 +1,174 @@
+package object
+
+// Taken and modified from the Kubernetes plugin of CoreDNS, under Apache 2.0.
+
+import (
+	"fmt"
+	"net/netip"
+	"regexp"
+	"slices"
+	"strings"
+	"time"
+
+	api "k8s.io/api/core/v1"
+	discovery "k8s.io/api/discovery/v1"
+	meta "k8s.io/apimachinery/pkg/apis/meta/v1"
+	"k8s.io/apimachinery/pkg/runtime"
+)
+
+// Endpoints is a stripped down discovery.EndpointSlice
+// with only the items we need.
+type Endpoints struct {
+	Version               string
+	Name                  string
+	Namespace             string
+	LastChangeTriggerTime time.Time
+	Index                 string
+	Addresses             []EndpointAddress
+	Ports                 []Port
+
+	*Empty
+}
+
+// EndpointAddress is a tuple that describes single IP address.
+type EndpointAddress struct {
+	// IP contains the IP address in binary format.
+	IP       string
+	Hostname string
+}
+
+// Port is a tuple that describes a single port.
+type Port struct {
+	Port     uint16
+	Name     string
+	Protocol string
+}
+
+// EndpointsKey returns a string using for the index.
+func EndpointsKey(name, namespace string) string { return name + "." + namespace }
+
+var hostnameRegexp = regexp.MustCompile(`^[-a-z0-9]{1,63}$`)
+
+// EndpointSliceToEndpoints converts a *discovery.EndpointSlice to a *Endpoints.
+func EndpointSliceToEndpoints(obj meta.Object) (meta.Object, error) {
+	ends, ok := obj.(*discovery.EndpointSlice)
+	if !ok {
+		return nil, fmt.Errorf("unexpected object %v", obj)
+	}
+	e := &Endpoints{
+		Version:   ends.GetResourceVersion(),
+		Name:      ends.GetName(),
+		Namespace: ends.GetNamespace(),
+		Index:     EndpointsKey(ends.Labels[discovery.LabelServiceName], ends.GetNamespace()),
+	}
+
+	// In case of parse error, the value is time.Zero.
+	e.LastChangeTriggerTime, _ = time.Parse(time.RFC3339Nano, ends.Annotations[api.EndpointsLastChangeTriggerTime])
+
+	e.Ports = make([]Port, 0, len(ends.Ports))
+	for _, p := range ends.Ports {
+		if p.Port != nil && *p.Port >= 1 && *p.Port <= 0xffff &&
+			p.Name != nil && *p.Name != "" && p.Protocol != nil {
+			ep := Port{
+				Port:     uint16(*p.Port),
+				Name:     strings.ToLower(*p.Name),
+				Protocol: strings.ToLower(string(*p.Protocol)),
+			}
+			e.Ports = append(e.Ports, ep)
+		}
+	}
+
+	for _, end := range ends.Endpoints {
+		if !endpointsliceReady(end.Conditions.Ready) {
+			continue
+		}
+
+		var endHostname string
+		if end.Hostname != nil {
+			endHostname = *end.Hostname
+		}
+		if endHostname != "" && !hostnameRegexp.MatchString(endHostname) {
+			endHostname = ""
+		}
+
+		for _, rawIP := range end.Addresses {
+			parsedIP, err := netip.ParseAddr(rawIP)
+			if err != nil || parsedIP.Zone() != "" {
+				continue
+			}
+			parsedIP = parsedIP.Unmap()
+			// The IP address is converted to a binary string, not human readable.
+			// That way we don't need to parse it again later.
+			ea := EndpointAddress{IP: string(parsedIP.AsSlice())}
+			if endHostname != "" {
+				ea.Hostname = endHostname
+			} else {
+				ea.Hostname = strings.ReplaceAll(strings.ReplaceAll(parsedIP.String(), ".", "-"), ":", "-")
+			}
+			e.Addresses = append(e.Addresses, ea)
+		}
+	}
+
+	*ends = discovery.EndpointSlice{}
+
+	return e, nil
+}
+
+func endpointsliceReady(ready *bool) bool {
+	// Per API docs: a nil value indicates an unknown state. In most cases
+	// consumers should interpret this unknown state as ready.
+	if ready == nil {
+		return true
+	}
+	return *ready
+}
+
+var _ runtime.Object = &Endpoints{}
+
+// DeepCopyObject implements the ObjectKind interface.
+func (e *Endpoints) DeepCopyObject() runtime.Object {
+	e1 := &Endpoints{
+		Version:   e.Version,
+		Name:      e.Name,
+		Namespace: e.Namespace,
+		Index:     e.Index,
+		Addresses: make([]EndpointAddress, len(e.Addresses)),
+		Ports:     make([]Port, len(e.Ports)),
+	}
+	copy(e1.Addresses, e.Addresses)
+	copy(e1.Ports, e.Ports)
+	return e1
+}
+
+// GetNamespace implements the metav1.Object interface.
+func (e *Endpoints) GetNamespace() string { return e.Namespace }
+
+// SetNamespace implements the metav1.Object interface.
+func (e *Endpoints) SetNamespace(namespace string) {}
+
+// GetName implements the metav1.Object interface.
+func (e *Endpoints) GetName() string { return e.Name }
+
+// SetName implements the metav1.Object interface.
+func (e *Endpoints) SetName(name string) {}
+
+// GetResourceVersion implements the metav1.Object interface.
+func (e *Endpoints) GetResourceVersion() string { return e.Version }
+
+// SetResourceVersion implements the metav1.Object interface.
+func (e *Endpoints) SetResourceVersion(version string) {}
+
+// EndpointsModified checks if the update to an endpoint is something
+// that matters to us or if they are effectively equivalent.
+func EndpointsModified(a, b *Endpoints) bool {
+	if a.Index != b.Index {
+		return true
+	}
+	if !slices.Equal(a.Addresses, b.Addresses) {
+		return true
+	}
+	if !slices.Equal(a.Ports, b.Ports) {
+		return true
+	}
+	return false
+}
diff --git a/osbase/net/dns/kubernetes/object/informer.go b/osbase/net/dns/kubernetes/object/informer.go
new file mode 100644
index 0000000..8ec1f21
--- /dev/null
+++ b/osbase/net/dns/kubernetes/object/informer.go
@@ -0,0 +1,96 @@
+package object
+
+// Taken and modified from the Kubernetes plugin of CoreDNS, under Apache 2.0.
+
+import (
+	"fmt"
+
+	"k8s.io/apimachinery/pkg/api/meta"
+	metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
+	"k8s.io/apimachinery/pkg/runtime"
+	"k8s.io/client-go/tools/cache"
+)
+
+// KeyFunc works like cache.DeletionHandlingMetaNamespaceKeyFunc
+// but uses format "<name>.<namespace>" instead of "<namespace>/<name>".
+// This makes lookup for a service slightly more efficient, because we can
+// just use a slice of the query name instead of constructing a new string.
+func KeyFunc(obj interface{}) (string, error) {
+	if d, ok := obj.(cache.DeletedFinalStateUnknown); ok {
+		return d.Key, nil
+	}
+	objMeta, err := meta.Accessor(obj)
+	if err != nil {
+		return "", fmt.Errorf("object has no meta: %v", err)
+	}
+	if len(objMeta.GetNamespace()) == 0 {
+		return objMeta.GetName(), nil
+	}
+	return objMeta.GetName() + "." + objMeta.GetNamespace(), nil
+}
+
+// NewIndexerInformer is a copy of the cache.NewIndexerInformer function,
+// but allows custom process function.
+func NewIndexerInformer(lw cache.ListerWatcher, objType runtime.Object, h cache.ResourceEventHandler, indexers cache.Indexers, builder ProcessorBuilder) (cache.Indexer, cache.Controller) {
+	clientState := cache.NewIndexer(KeyFunc, indexers)
+
+	cfg := &cache.Config{
+		Queue:            cache.NewDeltaFIFOWithOptions(cache.DeltaFIFOOptions{KeyFunction: KeyFunc, KnownObjects: clientState}),
+		ListerWatcher:    lw,
+		ObjectType:       objType,
+		FullResyncPeriod: 0,
+		RetryOnError:     false,
+		Process:          builder(clientState, h),
+	}
+	return clientState, cache.New(cfg)
+}
+
+// DefaultProcessor is based on the Process function from
+// cache.NewIndexerInformer except it does a conversion.
+func DefaultProcessor(convert ToFunc) ProcessorBuilder {
+	return func(clientState cache.Indexer, h cache.ResourceEventHandler) cache.ProcessFunc {
+		return func(obj interface{}, isInitialList bool) error {
+			for _, d := range obj.(cache.Deltas) {
+				switch d.Type {
+				case cache.Sync, cache.Added, cache.Updated:
+					metaObj := d.Object.(metav1.Object)
+					obj, err := convert(metaObj)
+					if err != nil {
+						return err
+					}
+					if old, exists, err := clientState.Get(obj); err == nil && exists {
+						if err := clientState.Update(obj); err != nil {
+							return err
+						}
+						h.OnUpdate(old, obj)
+					} else {
+						if err := clientState.Add(obj); err != nil {
+							return err
+						}
+						h.OnAdd(obj, isInitialList)
+					}
+				case cache.Deleted:
+					var obj interface{}
+					obj, ok := d.Object.(cache.DeletedFinalStateUnknown)
+					if !ok {
+						var err error
+						metaObj, ok := d.Object.(metav1.Object)
+						if !ok {
+							return fmt.Errorf("unexpected object %v", d.Object)
+						}
+						obj, err = convert(metaObj)
+						if err != nil {
+							return err
+						}
+					}
+
+					if err := clientState.Delete(obj); err != nil {
+						return err
+					}
+					h.OnDelete(obj)
+				}
+			}
+			return nil
+		}
+	}
+}
diff --git a/osbase/net/dns/kubernetes/object/informer_test.go b/osbase/net/dns/kubernetes/object/informer_test.go
new file mode 100644
index 0000000..68face5
--- /dev/null
+++ b/osbase/net/dns/kubernetes/object/informer_test.go
@@ -0,0 +1,119 @@
+package object
+
+// Taken and modified from the Kubernetes plugin of CoreDNS, under Apache 2.0.
+
+import (
+	"testing"
+
+	api "k8s.io/api/core/v1"
+	metav1 "k8s.io/apimachinery/pkg/apis/meta/v1"
+	"k8s.io/client-go/tools/cache"
+)
+
+func TestDefaultProcessor(t *testing.T) {
+	pbuild := DefaultProcessor(ToService)
+	reh := cache.ResourceEventHandlerFuncs{}
+	idx := cache.NewIndexer(cache.DeletionHandlingMetaNamespaceKeyFunc, cache.Indexers{})
+	processor := pbuild(idx, reh)
+	testProcessor(t, processor, idx)
+}
+
+func testProcessor(t *testing.T, processor cache.ProcessFunc, idx cache.Indexer) {
+	obj := &api.Service{
+		ObjectMeta: metav1.ObjectMeta{Name: "service1", Namespace: "test1"},
+		Spec: api.ServiceSpec{
+			Type:         api.ServiceTypeExternalName,
+			ExternalName: "example.com.",
+			Ports:        []api.ServicePort{{Port: 80}},
+		},
+	}
+	obj2 := &api.Service{
+		ObjectMeta: metav1.ObjectMeta{Name: "service2", Namespace: "test1"},
+		Spec: api.ServiceSpec{
+			ClusterIP:  "5.6.7.8",
+			ClusterIPs: []string{"5.6.7.8"},
+			Ports:      []api.ServicePort{{Port: 80}},
+		},
+	}
+
+	// Add the objects
+	err := processor(cache.Deltas{
+		{Type: cache.Added, Object: obj.DeepCopy()},
+		{Type: cache.Added, Object: obj2.DeepCopy()},
+	}, false)
+	if err != nil {
+		t.Fatalf("add failed: %v", err)
+	}
+	got, exists, err := idx.Get(obj)
+	if err != nil {
+		t.Fatalf("get added object failed: %v", err)
+	}
+	if !exists {
+		t.Fatal("added object not found in index")
+	}
+	svc, ok := got.(*Service)
+	if !ok {
+		t.Fatal("object in index was incorrect type")
+	}
+	if svc.ExternalName != obj.Spec.ExternalName {
+		t.Fatalf("expected '%v', got '%v'", obj.Spec.ExternalName, svc.ExternalName)
+	}
+
+	// Update an object
+	obj.Spec.ExternalName = "2.example.com."
+	err = processor(cache.Deltas{{
+		Type:   cache.Updated,
+		Object: obj.DeepCopy(),
+	}}, false)
+	if err != nil {
+		t.Fatalf("update failed: %v", err)
+	}
+	got, exists, err = idx.Get(obj)
+	if err != nil {
+		t.Fatalf("get updated object failed: %v", err)
+	}
+	if !exists {
+		t.Fatal("updated object not found in index")
+	}
+	svc, ok = got.(*Service)
+	if !ok {
+		t.Fatal("object in index was incorrect type")
+	}
+	if svc.ExternalName != obj.Spec.ExternalName {
+		t.Fatalf("expected '%v', got '%v'", obj.Spec.ExternalName, svc.ExternalName)
+	}
+
+	// Delete an object
+	err = processor(cache.Deltas{{
+		Type:   cache.Deleted,
+		Object: obj2.DeepCopy(),
+	}}, false)
+	if err != nil {
+		t.Fatalf("delete test failed: %v", err)
+	}
+	_, exists, err = idx.Get(obj2)
+	if err != nil {
+		t.Fatalf("get deleted object failed: %v", err)
+	}
+	if exists {
+		t.Fatal("deleted object found in index")
+	}
+
+	// Delete an object via tombstone
+	key, _ := cache.MetaNamespaceKeyFunc(obj)
+	tombstone := cache.DeletedFinalStateUnknown{Key: key, Obj: svc}
+	err = processor(cache.Deltas{{
+		Type:   cache.Deleted,
+		Object: tombstone,
+	}}, false)
+	if err != nil {
+		t.Fatalf("tombstone delete test failed: %v", err)
+	}
+	_, exists, err = idx.Get(svc)
+	if err != nil {
+		t.Fatalf("get tombstone deleted object failed: %v", err)
+	}
+	if exists {
+		t.Fatal("tombstone deleted object found in index")
+	}
+}
diff --git a/osbase/net/dns/kubernetes/object/namespace.go b/osbase/net/dns/kubernetes/object/namespace.go
new file mode 100644
index 0000000..f7e7e56
--- /dev/null
+++ b/osbase/net/dns/kubernetes/object/namespace.go
@@ -0,0 +1,62 @@
+package object
+
+// Taken and modified from the Kubernetes plugin of CoreDNS, under Apache 2.0.
+
+import (
+	"fmt"
+
+	api "k8s.io/api/core/v1"
+	meta "k8s.io/apimachinery/pkg/apis/meta/v1"
+	"k8s.io/apimachinery/pkg/runtime"
+)
+
+// Namespace is a stripped down api.Namespace with only the items we need.
+type Namespace struct {
+	Version string
+	Name    string
+
+	*Empty
+}
+
+// ToNamespace returns a function that converts an api.Namespace to a *Namespace.
+func ToNamespace(obj meta.Object) (meta.Object, error) {
+	ns, ok := obj.(*api.Namespace)
+	if !ok {
+		return nil, fmt.Errorf("unexpected object %v", obj)
+	}
+	n := &Namespace{
+		Version: ns.GetResourceVersion(),
+		Name:    ns.GetName(),
+	}
+	*ns = api.Namespace{}
+	return n, nil
+}
+
+var _ runtime.Object = &Namespace{}
+
+// DeepCopyObject implements the ObjectKind interface.
+func (n *Namespace) DeepCopyObject() runtime.Object {
+	n1 := &Namespace{
+		Version: n.Version,
+		Name:    n.Name,
+	}
+	return n1
+}
+
+// GetNamespace implements the metav1.Object interface.
+func (n *Namespace) GetNamespace() string { return "" }
+
+// SetNamespace implements the metav1.Object interface.
+func (n *Namespace) SetNamespace(namespace string) {}
+
+// GetName implements the metav1.Object interface.
+func (n *Namespace) GetName() string { return n.Name }
+
+// SetName implements the metav1.Object interface.
+func (n *Namespace) SetName(name string) {}
+
+// GetResourceVersion implements the metav1.Object interface.
+func (n *Namespace) GetResourceVersion() string { return n.Version }
+
+// SetResourceVersion implements the metav1.Object interface.
+func (n *Namespace) SetResourceVersion(version string) {}
diff --git a/osbase/net/dns/kubernetes/object/object.go b/osbase/net/dns/kubernetes/object/object.go
new file mode 100644
index 0000000..05724fc
--- /dev/null
+++ b/osbase/net/dns/kubernetes/object/object.go
@@ -0,0 +1,106 @@
+// Package object holds functions that convert the objects from the k8s API in
+// to a more memory efficient structures.
+//
+// Adding new fields to any of the structures defined in pod.go, endpoint.go
+// and service.go should not be done lightly as this increases the memory use
+// and will leads to OOMs in the k8s scale test.
+package object
+
+// Taken and modified from the Kubernetes plugin of CoreDNS, under Apache 2.0.
+
+import (
+	v1 "k8s.io/apimachinery/pkg/apis/meta/v1"
+	"k8s.io/apimachinery/pkg/runtime/schema"
+	"k8s.io/apimachinery/pkg/types"
+	"k8s.io/client-go/tools/cache"
+)
+
+// ToFunc converts one v1.Object to another v1.Object.
+type ToFunc func(v1.Object) (v1.Object, error)
+
+// ProcessorBuilder returns function to process cache events.
+type ProcessorBuilder func(cache.Indexer, cache.ResourceEventHandler) cache.ProcessFunc
+
+// Empty is an empty struct.
+type Empty struct{}
+
+// GetObjectKind implements the ObjectKind interface as a noop.
+func (e *Empty) GetObjectKind() schema.ObjectKind { return schema.EmptyObjectKind }
+
+// GetGenerateName implements the metav1.Object interface.
+func (e *Empty) GetGenerateName() string { return "" }
+
+// SetGenerateName implements the metav1.Object interface.
+func (e *Empty) SetGenerateName(name string) {}
+
+// GetUID implements the metav1.Object interface.
+func (e *Empty) GetUID() types.UID { return "" }
+
+// SetUID implements the metav1.Object interface.
+func (e *Empty) SetUID(uid types.UID) {}
+
+// GetGeneration implements the metav1.Object interface.
+func (e *Empty) GetGeneration() int64 { return 0 }
+
+// SetGeneration implements the metav1.Object interface.
+func (e *Empty) SetGeneration(generation int64) {}
+
+// GetSelfLink implements the metav1.Object interface.
+func (e *Empty) GetSelfLink() string { return "" }
+
+// SetSelfLink implements the metav1.Object interface.
+func (e *Empty) SetSelfLink(selfLink string) {}
+
+// GetCreationTimestamp implements the metav1.Object interface.
+func (e *Empty) GetCreationTimestamp() v1.Time { return v1.Time{} }
+
+// SetCreationTimestamp implements the metav1.Object interface.
+func (e *Empty) SetCreationTimestamp(timestamp v1.Time) {}
+
+// GetDeletionTimestamp implements the metav1.Object interface.
+func (e *Empty) GetDeletionTimestamp() *v1.Time { return &v1.Time{} }
+
+// SetDeletionTimestamp implements the metav1.Object interface.
+func (e *Empty) SetDeletionTimestamp(timestamp *v1.Time) {}
+
+// GetDeletionGracePeriodSeconds implements the metav1.Object interface.
+func (e *Empty) GetDeletionGracePeriodSeconds() *int64 { return nil }
+
+// SetDeletionGracePeriodSeconds implements the metav1.Object interface.
+func (e *Empty) SetDeletionGracePeriodSeconds(*int64) {}
+
+// GetLabels implements the metav1.Object interface.
+func (e *Empty) GetLabels() map[string]string { return nil }
+
+// SetLabels implements the metav1.Object interface.
+func (e *Empty) SetLabels(labels map[string]string) {}
+
+// GetAnnotations implements the metav1.Object interface.
+func (e *Empty) GetAnnotations() map[string]string { return nil }
+
+// SetAnnotations implements the metav1.Object interface.
+func (e *Empty) SetAnnotations(annotations map[string]string) {}
+
+// GetFinalizers implements the metav1.Object interface.
+func (e *Empty) GetFinalizers() []string { return nil }
+
+// SetFinalizers implements the metav1.Object interface.
+func (e *Empty) SetFinalizers(finalizers []string) {}
+
+// GetOwnerReferences implements the metav1.Object interface.
+func (e *Empty) GetOwnerReferences() []v1.OwnerReference { return nil }
+
+// SetOwnerReferences implements the metav1.Object interface.
+func (e *Empty) SetOwnerReferences([]v1.OwnerReference) {}
+
+// GetZZZ_DeprecatedClusterName implements the metav1.Object interface.
+func (e *Empty) GetZZZ_DeprecatedClusterName() string { return "" }
+
+// SetZZZ_DeprecatedClusterName implements the metav1.Object interface.
+func (e *Empty) SetZZZ_DeprecatedClusterName(clusterName string) {}
+
+// GetManagedFields implements the metav1.Object interface.
+func (e *Empty) GetManagedFields() []v1.ManagedFieldsEntry { return nil }
+
+// SetManagedFields implements the metav1.Object interface.
+func (e *Empty) SetManagedFields(managedFields []v1.ManagedFieldsEntry) {}
diff --git a/osbase/net/dns/kubernetes/object/service.go b/osbase/net/dns/kubernetes/object/service.go
new file mode 100644
index 0000000..5a6ca77
--- /dev/null
+++ b/osbase/net/dns/kubernetes/object/service.go
@@ -0,0 +1,144 @@
+package object
+
+// Taken and modified from the Kubernetes plugin of CoreDNS, under Apache 2.0.
+
+import (
+	"fmt"
+	"net/netip"
+	"regexp"
+	"slices"
+	"strings"
+
+	"github.com/miekg/dns"
+	api "k8s.io/api/core/v1"
+	meta "k8s.io/apimachinery/pkg/apis/meta/v1"
+	"k8s.io/apimachinery/pkg/runtime"
+)
+
+// Service is a stripped down api.Service with only the items we need.
+type Service struct {
+	Version   string
+	Name      string
+	Namespace string
+	// ClusterIPs contains IP addresses in binary format.
+	ClusterIPs   []string
+	ExternalName string
+	Ports        []Port
+	Headless     bool
+
+	*Empty
+}
+
+var domainNameRegexp = regexp.MustCompile(`^([-a-z0-9]{1,63}\.)+$`)
+
+const ExternalNameInvalid = "."
+
+// ToService converts an api.Service to a *Service.
+func ToService(obj meta.Object) (meta.Object, error) {
+	svc, ok := obj.(*api.Service)
+	if !ok {
+		return nil, fmt.Errorf("unexpected object %v", obj)
+	}
+
+	s := &Service{
+		Version:   svc.GetResourceVersion(),
+		Name:      svc.GetName(),
+		Namespace: svc.GetNamespace(),
+	}
+
+	if svc.Spec.Type == api.ServiceTypeExternalName {
+		// Make the name fully qualified.
+		externalName := dns.Fqdn(svc.Spec.ExternalName)
+		// Check if the name is valid. Even names that pass Kubernetes validation
+		// can fail this check, because Kubernetes does not validate that labels
+		// must be at most 63 characters.
+		if !domainNameRegexp.MatchString(externalName) || len(externalName) > 254 {
+			externalName = ExternalNameInvalid
+		}
+		s.ExternalName = externalName
+	} else {
+		if svc.Spec.ClusterIP == api.ClusterIPNone {
+			s.Headless = true
+		} else {
+			s.ClusterIPs = make([]string, 0, len(svc.Spec.ClusterIPs))
+			for _, rawIP := range svc.Spec.ClusterIPs {
+				parsedIP, err := netip.ParseAddr(rawIP)
+				if err != nil || parsedIP.Zone() != "" {
+					continue
+				}
+				parsedIP = parsedIP.Unmap()
+				s.ClusterIPs = append(s.ClusterIPs, string(parsedIP.AsSlice()))
+			}
+
+			s.Ports = make([]Port, 0, len(svc.Spec.Ports))
+			for _, p := range svc.Spec.Ports {
+				if p.Port >= 1 && p.Port <= 0xffff && p.Name != "" {
+					ep := Port{
+						Port:     uint16(p.Port),
+						Name:     strings.ToLower(p.Name),
+						Protocol: strings.ToLower(string(p.Protocol)),
+					}
+					s.Ports = append(s.Ports, ep)
+				}
+			}
+		}
+	}
+
+	*svc = api.Service{}
+
+	return s, nil
+}
+
+var _ runtime.Object = &Service{}
+
+// DeepCopyObject implements the ObjectKind interface.
+func (s *Service) DeepCopyObject() runtime.Object {
+	s1 := &Service{
+		Version:      s.Version,
+		Name:         s.Name,
+		Namespace:    s.Namespace,
+		ClusterIPs:   make([]string, len(s.ClusterIPs)),
+		ExternalName: s.ExternalName,
+		Ports:        make([]Port, len(s.Ports)),
+		Headless:     s.Headless,
+	}
+	copy(s1.ClusterIPs, s.ClusterIPs)
+	copy(s1.Ports, s.Ports)
+	return s1
+}
+
+// GetNamespace implements the metav1.Object interface.
+func (s *Service) GetNamespace() string { return s.Namespace }
+
+// SetNamespace implements the metav1.Object interface.
+func (s *Service) SetNamespace(namespace string) {}
+
+// GetName implements the metav1.Object interface.
+func (s *Service) GetName() string { return s.Name }
+
+// SetName implements the metav1.Object interface.
+func (s *Service) SetName(name string) {}
+
+// GetResourceVersion implements the metav1.Object interface.
+func (s *Service) GetResourceVersion() string { return s.Version }
+
+// SetResourceVersion implements the metav1.Object interface.
+func (s *Service) SetResourceVersion(version string) {}
+
+// ServiceModified checks if the update to a service is something
+// that matters to us or if they are effectively equivalent.
+func ServiceModified(oldSvc, newSvc *Service) bool {
+	if oldSvc.ExternalName != newSvc.ExternalName {
+		return true
+	}
+	if oldSvc.Headless != newSvc.Headless {
+		return true
+	}
+	if !slices.Equal(oldSvc.ClusterIPs, newSvc.ClusterIPs) {
+		return true
+	}
+	if !slices.Equal(oldSvc.Ports, newSvc.Ports) {
+		return true
+	}
+	return false
+}