mirror of
https://github.com/tailscale/tailscale.git
synced 2024-11-26 11:35:35 +00:00
24bdcbe5c7
* net/dns, net/dns/resolver, wgengine: refactor DNS request path Previously, method calls into the DNS manager/resolver types handled DNS requests rather than DNS packets. This is fine for UDP as one packet corresponds to one request or response, however will not suit an implementation that supports DNS over TCP. To support PRs implementing this in the future, wgengine delegates all handling/construction of packets to the magic DNS endpoint, to the DNS types themselves. Handling IP packets at this level enables future support for both UDP and TCP. Signed-off-by: Tom DNetto <tom@tailscale.com>
1315 lines
38 KiB
Go
1315 lines
38 KiB
Go
// Copyright (c) 2020 Tailscale Inc & AUTHORS All rights reserved.
|
|
// Use of this source code is governed by a BSD-style
|
|
// license that can be found in the LICENSE file.
|
|
|
|
// Package resolver implements a stub DNS resolver that can also serve
|
|
// records out of an internal local zone.
|
|
package resolver
|
|
|
|
import (
|
|
"bufio"
|
|
"context"
|
|
"encoding/hex"
|
|
"errors"
|
|
"fmt"
|
|
"io"
|
|
"net"
|
|
"os"
|
|
"runtime"
|
|
"sort"
|
|
"strings"
|
|
"sync"
|
|
"sync/atomic"
|
|
"time"
|
|
|
|
dns "golang.org/x/net/dns/dnsmessage"
|
|
"inet.af/netaddr"
|
|
"tailscale.com/net/dns/resolvconffile"
|
|
tspacket "tailscale.com/net/packet"
|
|
"tailscale.com/net/tsaddr"
|
|
"tailscale.com/net/tsdial"
|
|
"tailscale.com/net/tstun"
|
|
"tailscale.com/types/dnstype"
|
|
"tailscale.com/types/ipproto"
|
|
"tailscale.com/types/logger"
|
|
"tailscale.com/util/clientmetric"
|
|
"tailscale.com/util/dnsname"
|
|
"tailscale.com/wgengine/monitor"
|
|
)
|
|
|
|
const dnsSymbolicFQDN = "magicdns.localhost-tailscale-daemon."
|
|
|
|
var (
|
|
magicDNSIP = tsaddr.TailscaleServiceIP()
|
|
magicDNSIPv6 = tsaddr.TailscaleServiceIPv6()
|
|
)
|
|
|
|
const magicDNSPort = 53
|
|
|
|
// maxResponseBytes is the maximum size of a response from a Resolver. The
|
|
// actual buffer size will be one larger than this so that we can detect
|
|
// truncation in a platform-agnostic way.
|
|
const maxResponseBytes = 4095
|
|
|
|
// maxActiveQueries returns the maximal number of DNS requests that be
|
|
// can running.
|
|
// If EnqueueRequest is called when this many requests are already pending,
|
|
// the request will be dropped to avoid blocking the caller.
|
|
func maxActiveQueries() int32 {
|
|
if runtime.GOOS == "ios" {
|
|
// For memory paranoia reasons on iOS, match the
|
|
// historical Tailscale 1.x..1.8 behavior for now
|
|
// (just before the 1.10 release).
|
|
return 64
|
|
}
|
|
// But for other platforms, allow more burstiness:
|
|
return 256
|
|
}
|
|
|
|
// defaultTTL is the TTL of all responses from Resolver.
|
|
const defaultTTL = 600 * time.Second
|
|
|
|
// ErrClosed indicates that the resolver has been closed and readers should exit.
|
|
var ErrClosed = errors.New("closed")
|
|
|
|
var (
|
|
errFullQueue = errors.New("request queue full")
|
|
errNotQuery = errors.New("not a DNS query")
|
|
errNotOurName = errors.New("not a Tailscale DNS name")
|
|
)
|
|
|
|
type packet struct {
|
|
bs []byte
|
|
addr netaddr.IPPort // src for a request, dst for a response
|
|
}
|
|
|
|
// Config is a resolver configuration.
|
|
// Given a Config, queries are resolved in the following order:
|
|
// If the query is an exact match for an entry in LocalHosts, return that.
|
|
// Else if the query suffix matches an entry in LocalDomains, return NXDOMAIN.
|
|
// Else forward the query to the most specific matching entry in Routes.
|
|
// Else return SERVFAIL.
|
|
type Config struct {
|
|
// Routes is a map of DNS name suffix to the resolvers to use for
|
|
// queries within that suffix.
|
|
// Queries only match the most specific suffix.
|
|
// To register a "default route", add an entry for ".".
|
|
Routes map[dnsname.FQDN][]dnstype.Resolver
|
|
// LocalHosts is a map of FQDNs to corresponding IPs.
|
|
Hosts map[dnsname.FQDN][]netaddr.IP
|
|
// LocalDomains is a list of DNS name suffixes that should not be
|
|
// routed to upstream resolvers.
|
|
LocalDomains []dnsname.FQDN
|
|
}
|
|
|
|
// WriteToBufioWriter write a debug version of c for logs to w, omitting
|
|
// spammy stuff like *.arpa entries and replacing it with a total count.
|
|
func (c *Config) WriteToBufioWriter(w *bufio.Writer) {
|
|
w.WriteString("{Routes:")
|
|
WriteRoutes(w, c.Routes)
|
|
fmt.Fprintf(w, " Hosts:%v LocalDomains:[", len(c.Hosts))
|
|
space := false
|
|
arpa := 0
|
|
for _, d := range c.LocalDomains {
|
|
if strings.HasSuffix(string(d), ".arpa.") {
|
|
arpa++
|
|
continue
|
|
}
|
|
if space {
|
|
w.WriteByte(' ')
|
|
}
|
|
w.WriteString(string(d))
|
|
space = true
|
|
}
|
|
w.WriteString("]")
|
|
if arpa > 0 {
|
|
fmt.Fprintf(w, "+%darpa", arpa)
|
|
}
|
|
w.WriteString("}")
|
|
}
|
|
|
|
// WriteIPPorts writes vv to w.
|
|
func WriteIPPorts(w *bufio.Writer, vv []netaddr.IPPort) {
|
|
w.WriteByte('[')
|
|
var b []byte
|
|
for i, v := range vv {
|
|
if i > 0 {
|
|
w.WriteByte(' ')
|
|
}
|
|
b = v.AppendTo(b[:0])
|
|
w.Write(b)
|
|
}
|
|
w.WriteByte(']')
|
|
}
|
|
|
|
// WriteDNSResolver writes r to w.
|
|
func WriteDNSResolver(w *bufio.Writer, r dnstype.Resolver) {
|
|
io.WriteString(w, r.Addr)
|
|
if len(r.BootstrapResolution) > 0 {
|
|
w.WriteByte('(')
|
|
var b []byte
|
|
for _, ip := range r.BootstrapResolution {
|
|
ip.AppendTo(b[:0])
|
|
w.Write(b)
|
|
}
|
|
w.WriteByte(')')
|
|
}
|
|
}
|
|
|
|
// WriteDNSResolvers writes resolvers to w.
|
|
func WriteDNSResolvers(w *bufio.Writer, resolvers []dnstype.Resolver) {
|
|
w.WriteByte('[')
|
|
for i, r := range resolvers {
|
|
if i > 0 {
|
|
w.WriteByte(' ')
|
|
}
|
|
WriteDNSResolver(w, r)
|
|
}
|
|
w.WriteByte(']')
|
|
}
|
|
|
|
// WriteRoutes writes routes to w, omitting *.arpa routes and instead
|
|
// summarizing how many of them there were.
|
|
func WriteRoutes(w *bufio.Writer, routes map[dnsname.FQDN][]dnstype.Resolver) {
|
|
var kk []dnsname.FQDN
|
|
arpa := 0
|
|
for k := range routes {
|
|
if strings.HasSuffix(string(k), ".arpa.") {
|
|
arpa++
|
|
continue
|
|
}
|
|
kk = append(kk, k)
|
|
}
|
|
sort.Slice(kk, func(i, j int) bool { return kk[i] < kk[j] })
|
|
w.WriteByte('{')
|
|
for i, k := range kk {
|
|
if i > 0 {
|
|
w.WriteByte(' ')
|
|
}
|
|
w.WriteString(string(k))
|
|
w.WriteByte(':')
|
|
WriteDNSResolvers(w, routes[k])
|
|
}
|
|
w.WriteByte('}')
|
|
if arpa > 0 {
|
|
fmt.Fprintf(w, "+%darpa", arpa)
|
|
}
|
|
}
|
|
|
|
// Resolver is a DNS resolver for nodes on the Tailscale network,
|
|
// associating them with domain names of the form <mynode>.<mydomain>.<root>.
|
|
// If it is asked to resolve a domain that is not of that form,
|
|
// it delegates to upstream nameservers if any are set.
|
|
type Resolver struct {
|
|
logf logger.Logf
|
|
linkMon *monitor.Mon // or nil
|
|
dialer *tsdial.Dialer // non-nil
|
|
saveConfigForTests func(cfg Config) // used in tests to capture resolver config
|
|
// forwarder forwards requests to upstream nameservers.
|
|
forwarder *forwarder
|
|
|
|
activeQueriesAtomic int32 // number of DNS queries in flight
|
|
|
|
// responses is an unbuffered channel to which responses are returned.
|
|
responses chan packet
|
|
// errors is an unbuffered channel to which errors are returned.
|
|
errors chan error
|
|
// closed signals all goroutines to stop.
|
|
closed chan struct{}
|
|
// wg signals when all goroutines have stopped.
|
|
wg sync.WaitGroup
|
|
|
|
// mu guards the following fields from being updated while used.
|
|
mu sync.Mutex
|
|
localDomains []dnsname.FQDN
|
|
hostToIP map[dnsname.FQDN][]netaddr.IP
|
|
ipToHost map[netaddr.IP]dnsname.FQDN
|
|
}
|
|
|
|
type ForwardLinkSelector interface {
|
|
// PickLink returns which network device should be used to query
|
|
// the DNS server at the given IP.
|
|
// The empty string means to use an unspecified default.
|
|
PickLink(netaddr.IP) (linkName string)
|
|
}
|
|
|
|
// New returns a new resolver.
|
|
// linkMon optionally specifies a link monitor to use for socket rebinding.
|
|
func New(logf logger.Logf, linkMon *monitor.Mon, linkSel ForwardLinkSelector, dialer *tsdial.Dialer) *Resolver {
|
|
if dialer == nil {
|
|
panic("nil Dialer")
|
|
}
|
|
r := &Resolver{
|
|
logf: logger.WithPrefix(logf, "resolver: "),
|
|
linkMon: linkMon,
|
|
responses: make(chan packet),
|
|
errors: make(chan error),
|
|
closed: make(chan struct{}),
|
|
hostToIP: map[dnsname.FQDN][]netaddr.IP{},
|
|
ipToHost: map[netaddr.IP]dnsname.FQDN{},
|
|
dialer: dialer,
|
|
}
|
|
r.forwarder = newForwarder(r.logf, r.responses, linkMon, linkSel, dialer)
|
|
return r
|
|
}
|
|
|
|
func (r *Resolver) TestOnlySetHook(hook func(Config)) { r.saveConfigForTests = hook }
|
|
|
|
func (r *Resolver) SetConfig(cfg Config) error {
|
|
if r.saveConfigForTests != nil {
|
|
r.saveConfigForTests(cfg)
|
|
}
|
|
|
|
reverse := make(map[netaddr.IP]dnsname.FQDN, len(cfg.Hosts))
|
|
|
|
for host, ips := range cfg.Hosts {
|
|
for _, ip := range ips {
|
|
reverse[ip] = host
|
|
}
|
|
}
|
|
|
|
r.forwarder.setRoutes(cfg.Routes)
|
|
|
|
r.mu.Lock()
|
|
defer r.mu.Unlock()
|
|
r.localDomains = cfg.LocalDomains
|
|
r.hostToIP = cfg.Hosts
|
|
r.ipToHost = reverse
|
|
return nil
|
|
}
|
|
|
|
// Close shuts down the resolver and ensures poll goroutines have exited.
|
|
// The Resolver cannot be used again after Close is called.
|
|
func (r *Resolver) Close() {
|
|
select {
|
|
case <-r.closed:
|
|
return
|
|
default:
|
|
// continue
|
|
}
|
|
close(r.closed)
|
|
|
|
r.forwarder.Close()
|
|
}
|
|
|
|
// EnqueuePacket handles a packet to the magicDNS endpoint.
|
|
// It takes ownership of the payload and does not block.
|
|
// If the queue is full, the request will be dropped and an error will be returned.
|
|
func (r *Resolver) EnqueuePacket(bs []byte, proto ipproto.Proto, from, to netaddr.IPPort) error {
|
|
if to.Port() != magicDNSPort || proto != ipproto.UDP {
|
|
return nil
|
|
}
|
|
|
|
return r.enqueueRequest(bs, proto, from, to)
|
|
}
|
|
|
|
// enqueueRequest places the given DNS request in the resolver's queue.
|
|
// If the queue is full, the request will be dropped and an error will be returned.
|
|
func (r *Resolver) enqueueRequest(bs []byte, proto ipproto.Proto, from, to netaddr.IPPort) error {
|
|
metricDNSQueryLocal.Add(1)
|
|
select {
|
|
case <-r.closed:
|
|
metricDNSQueryErrorClosed.Add(1)
|
|
return ErrClosed
|
|
default:
|
|
}
|
|
if n := atomic.AddInt32(&r.activeQueriesAtomic, 1); n > maxActiveQueries() {
|
|
atomic.AddInt32(&r.activeQueriesAtomic, -1)
|
|
metricDNSQueryErrorQueue.Add(1)
|
|
return errFullQueue
|
|
}
|
|
go r.handleQuery(packet{bs, from})
|
|
return nil
|
|
}
|
|
|
|
// NextPacket returns the next packet to service traffic for magicDNS. The returned
|
|
// packet is prefixed with unused space consistent with the semantics of injection
|
|
// into tstun.Wrapper.
|
|
// It blocks until a response is available and gives up ownership of the response payload.
|
|
func (r *Resolver) NextPacket() (ipPacket []byte, err error) {
|
|
bs, to, err := r.nextResponse()
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
// Unused space is needed further down the stack. To avoid extra
|
|
// allocations/copying later on, we allocate such space here.
|
|
const offset = tstun.PacketStartOffset
|
|
|
|
var buf []byte
|
|
switch {
|
|
case to.IP().Is4():
|
|
h := tspacket.UDP4Header{
|
|
IP4Header: tspacket.IP4Header{
|
|
Src: magicDNSIP,
|
|
Dst: to.IP(),
|
|
},
|
|
SrcPort: magicDNSPort,
|
|
DstPort: to.Port(),
|
|
}
|
|
hlen := h.Len()
|
|
buf = make([]byte, offset+hlen+len(bs))
|
|
copy(buf[offset+hlen:], bs)
|
|
h.Marshal(buf[offset:])
|
|
case to.IP().Is6():
|
|
h := tspacket.UDP6Header{
|
|
IP6Header: tspacket.IP6Header{
|
|
Src: magicDNSIPv6,
|
|
Dst: to.IP(),
|
|
},
|
|
SrcPort: magicDNSPort,
|
|
DstPort: to.Port(),
|
|
}
|
|
hlen := h.Len()
|
|
buf = make([]byte, offset+hlen+len(bs))
|
|
copy(buf[offset+hlen:], bs)
|
|
h.Marshal(buf[offset:])
|
|
}
|
|
|
|
return buf, nil
|
|
}
|
|
|
|
// nextResponse returns a DNS response to a previously enqueued request.
|
|
// It blocks until a response is available and gives up ownership of the response payload.
|
|
func (r *Resolver) nextResponse() (packet []byte, to netaddr.IPPort, err error) {
|
|
select {
|
|
case <-r.closed:
|
|
return nil, netaddr.IPPort{}, ErrClosed
|
|
case resp := <-r.responses:
|
|
return resp.bs, resp.addr, nil
|
|
case err := <-r.errors:
|
|
return nil, netaddr.IPPort{}, err
|
|
}
|
|
}
|
|
|
|
// parseExitNodeQuery parses a DNS request packet.
|
|
// It returns nil if it's malformed or lacking a question.
|
|
func parseExitNodeQuery(q []byte) *response {
|
|
p := dnsParserPool.Get().(*dnsParser)
|
|
defer dnsParserPool.Put(p)
|
|
p.zeroParser()
|
|
defer p.zeroParser()
|
|
if err := p.parseQuery(q); err != nil {
|
|
return nil
|
|
}
|
|
return p.response()
|
|
}
|
|
|
|
// HandleExitNodeDNSQuery handles a DNS query that arrived from a peer
|
|
// via the peerapi's DoH server. This is only used when the local
|
|
// node is being an exit node.
|
|
//
|
|
// The provided allowName callback is whether a DNS query for a name
|
|
// (as found by parsing q) is allowed.
|
|
//
|
|
// In most (all?) cases, err will be nil. A bogus DNS query q will
|
|
// still result in a response DNS packet (saying there's a failure)
|
|
// and a nil error.
|
|
// TODO: figure out if we even need an error result.
|
|
func (r *Resolver) HandleExitNodeDNSQuery(ctx context.Context, q []byte, from netaddr.IPPort, allowName func(name string) bool) (res []byte, err error) {
|
|
metricDNSExitProxyQuery.Add(1)
|
|
ch := make(chan packet, 1)
|
|
|
|
resp := parseExitNodeQuery(q)
|
|
if resp == nil {
|
|
return nil, errors.New("bad query")
|
|
}
|
|
name := resp.Question.Name.String()
|
|
if !allowName(name) {
|
|
metricDNSExitProxyErrorName.Add(1)
|
|
resp.Header.RCode = dns.RCodeRefused
|
|
return marshalResponse(resp)
|
|
}
|
|
|
|
switch runtime.GOOS {
|
|
default:
|
|
return nil, errors.New("unsupported exit node OS")
|
|
case "windows", "android":
|
|
return handleExitNodeDNSQueryWithNetPkg(ctx, nil, resp)
|
|
case "darwin":
|
|
// /etc/resolv.conf is a lie and only says one upstream DNS
|
|
// but for now that's probably good enough. Later we'll
|
|
// want to blend in everything from scutil --dns.
|
|
fallthrough
|
|
case "linux", "freebsd", "openbsd", "illumos":
|
|
nameserver, err := stubResolverForOS()
|
|
if err != nil {
|
|
r.logf("stubResolverForOS: %v", err)
|
|
metricDNSExitProxyErrorResolvConf.Add(1)
|
|
return nil, err
|
|
}
|
|
// TODO: more than 1 resolver from /etc/resolv.conf?
|
|
|
|
var resolvers []resolverAndDelay
|
|
if nameserver == tsaddr.TailscaleServiceIP() || nameserver == tsaddr.TailscaleServiceIPv6() {
|
|
// If resolv.conf says 100.100.100.100, it's coming right back to us anyway
|
|
// so avoid the loop through the kernel and just do what we
|
|
// would've done anyway. By not passing any resolvers, the forwarder
|
|
// will use its default ones from our DNS config.
|
|
} else {
|
|
resolvers = []resolverAndDelay{{
|
|
name: dnstype.Resolver{Addr: net.JoinHostPort(nameserver.String(), "53")},
|
|
}}
|
|
}
|
|
|
|
err = r.forwarder.forwardWithDestChan(ctx, packet{q, from}, ch, resolvers...)
|
|
if err != nil {
|
|
metricDNSExitProxyErrorForward.Add(1)
|
|
return nil, err
|
|
}
|
|
}
|
|
select {
|
|
case p, ok := <-ch:
|
|
if ok {
|
|
return p.bs, nil
|
|
}
|
|
panic("unexpected close chan")
|
|
default:
|
|
panic("unexpected unreadable chan")
|
|
}
|
|
}
|
|
|
|
// handleExitNodeDNSQueryWithNetPkg takes a DNS query message in q and
|
|
// return a reply (for the ExitDNS DoH service) using the net package's
|
|
// native APIs. This is only used on Windows for now.
|
|
//
|
|
// If resolver is nil, the net.Resolver zero value is used.
|
|
//
|
|
// response contains the pre-serialized response, which notably
|
|
// includes the original question and its header.
|
|
func handleExitNodeDNSQueryWithNetPkg(ctx context.Context, resolver *net.Resolver, resp *response) (res []byte, err error) {
|
|
if resp.Question.Class != dns.ClassINET {
|
|
return nil, errors.New("unsupported class")
|
|
}
|
|
|
|
r := resolver
|
|
if r == nil {
|
|
r = new(net.Resolver)
|
|
}
|
|
name := resp.Question.Name.String()
|
|
|
|
handleError := func(err error) (res []byte, _ error) {
|
|
if isGoNoSuchHostError(err) {
|
|
resp.Header.RCode = dns.RCodeNameError
|
|
return marshalResponse(resp)
|
|
}
|
|
// TODO: map other errors to RCodeServerFailure?
|
|
// Or I guess our caller should do that?
|
|
return nil, err
|
|
}
|
|
|
|
resp.Header.RCode = dns.RCodeSuccess // unless changed below
|
|
|
|
switch resp.Question.Type {
|
|
case dns.TypeA, dns.TypeAAAA:
|
|
network := "ip4"
|
|
if resp.Question.Type == dns.TypeAAAA {
|
|
network = "ip6"
|
|
}
|
|
ips, err := r.LookupIP(ctx, network, name)
|
|
if err != nil {
|
|
return handleError(err)
|
|
}
|
|
for _, stdIP := range ips {
|
|
if ip, ok := netaddr.FromStdIP(stdIP); ok {
|
|
resp.IPs = append(resp.IPs, ip)
|
|
}
|
|
}
|
|
case dns.TypeTXT:
|
|
strs, err := r.LookupTXT(ctx, name)
|
|
if err != nil {
|
|
return handleError(err)
|
|
}
|
|
resp.TXT = strs
|
|
case dns.TypePTR:
|
|
ipStr, ok := unARPA(name)
|
|
if !ok {
|
|
// TODO: is this RCodeFormatError?
|
|
return nil, errors.New("bogus PTR name")
|
|
}
|
|
addrs, err := r.LookupAddr(ctx, ipStr)
|
|
if err != nil {
|
|
return handleError(err)
|
|
}
|
|
if len(addrs) > 0 {
|
|
resp.Name, _ = dnsname.ToFQDN(addrs[0])
|
|
}
|
|
case dns.TypeCNAME:
|
|
cname, err := r.LookupCNAME(ctx, name)
|
|
if err != nil {
|
|
return handleError(err)
|
|
}
|
|
resp.CNAME = cname
|
|
case dns.TypeSRV:
|
|
// Thanks, Go: "To accommodate services publishing SRV
|
|
// records under non-standard names, if both service
|
|
// and proto are empty strings, LookupSRV looks up
|
|
// name directly."
|
|
_, srvs, err := r.LookupSRV(ctx, "", "", name)
|
|
if err != nil {
|
|
return handleError(err)
|
|
}
|
|
resp.SRVs = srvs
|
|
case dns.TypeNS:
|
|
nss, err := r.LookupNS(ctx, name)
|
|
if err != nil {
|
|
return handleError(err)
|
|
}
|
|
resp.NSs = nss
|
|
default:
|
|
return nil, fmt.Errorf("unsupported record type %v", resp.Question.Type)
|
|
}
|
|
return marshalResponse(resp)
|
|
}
|
|
|
|
func isGoNoSuchHostError(err error) bool {
|
|
if de, ok := err.(*net.DNSError); ok {
|
|
return de.IsNotFound
|
|
}
|
|
return false
|
|
}
|
|
|
|
type resolvConfCache struct {
|
|
mod time.Time
|
|
size int64
|
|
ip netaddr.IP
|
|
// TODO: inode/dev?
|
|
}
|
|
|
|
// resolvConfCacheValue contains the most recent stat metadata and parsed
|
|
// version of /etc/resolv.conf.
|
|
var resolvConfCacheValue atomic.Value // of resolvConfCache
|
|
|
|
var errEmptyResolvConf = errors.New("resolv.conf has no nameservers")
|
|
|
|
// stubResolverForOS returns the IP address of the first nameserver in
|
|
// /etc/resolv.conf.
|
|
func stubResolverForOS() (ip netaddr.IP, err error) {
|
|
fi, err := os.Stat(resolvconffile.Path)
|
|
if err != nil {
|
|
return netaddr.IP{}, err
|
|
}
|
|
cur := resolvConfCache{
|
|
mod: fi.ModTime(),
|
|
size: fi.Size(),
|
|
}
|
|
if c, ok := resolvConfCacheValue.Load().(resolvConfCache); ok && c.mod == cur.mod && c.size == cur.size {
|
|
return c.ip, nil
|
|
}
|
|
conf, err := resolvconffile.ParseFile(resolvconffile.Path)
|
|
if err != nil {
|
|
return netaddr.IP{}, err
|
|
}
|
|
if len(conf.Nameservers) == 0 {
|
|
return netaddr.IP{}, errEmptyResolvConf
|
|
}
|
|
ip = conf.Nameservers[0]
|
|
cur.ip = ip
|
|
resolvConfCacheValue.Store(cur)
|
|
return ip, nil
|
|
}
|
|
|
|
// resolveLocal returns an IP for the given domain, if domain is in
|
|
// the local hosts map and has an IP corresponding to the requested
|
|
// typ (A, AAAA, ALL).
|
|
// Returns dns.RCodeRefused to indicate that the local map is not
|
|
// authoritative for domain.
|
|
func (r *Resolver) resolveLocal(domain dnsname.FQDN, typ dns.Type) (netaddr.IP, dns.RCode) {
|
|
metricDNSResolveLocal.Add(1)
|
|
// Reject .onion domains per RFC 7686.
|
|
if dnsname.HasSuffix(domain.WithoutTrailingDot(), ".onion") {
|
|
metricDNSResolveLocalErrorOnion.Add(1)
|
|
return netaddr.IP{}, dns.RCodeNameError
|
|
}
|
|
|
|
// We return a symbolic domain if someone does a reverse lookup on the
|
|
// DNS endpoint. To round out this special case, we also do the inverse
|
|
// (returning the endpoint IP if someone looks up the symbolic domain).
|
|
if domain == dnsSymbolicFQDN {
|
|
switch typ {
|
|
case dns.TypeA:
|
|
return tsaddr.TailscaleServiceIP(), dns.RCodeSuccess
|
|
case dns.TypeAAAA:
|
|
return tsaddr.TailscaleServiceIPv6(), dns.RCodeSuccess
|
|
}
|
|
}
|
|
|
|
r.mu.Lock()
|
|
hosts := r.hostToIP
|
|
localDomains := r.localDomains
|
|
r.mu.Unlock()
|
|
|
|
addrs, found := hosts[domain]
|
|
if !found {
|
|
for _, suffix := range localDomains {
|
|
if suffix.Contains(domain) {
|
|
// We are authoritative for the queried domain.
|
|
metricDNSResolveLocalErrorMissing.Add(1)
|
|
return netaddr.IP{}, dns.RCodeNameError
|
|
}
|
|
}
|
|
// Not authoritative, signal that forwarding is advisable.
|
|
return netaddr.IP{}, dns.RCodeRefused
|
|
}
|
|
|
|
// Refactoring note: this must happen after we check suffixes,
|
|
// otherwise we will respond with NOTIMP to requests that should be forwarded.
|
|
//
|
|
// DNS semantics subtlety: when a DNS name exists, but no records
|
|
// are available for the requested record type, we must return
|
|
// RCodeSuccess with no data, not NXDOMAIN.
|
|
switch typ {
|
|
case dns.TypeA:
|
|
for _, ip := range addrs {
|
|
if ip.Is4() {
|
|
metricDNSResolveLocalOKA.Add(1)
|
|
return ip, dns.RCodeSuccess
|
|
}
|
|
}
|
|
metricDNSResolveLocalNoA.Add(1)
|
|
return netaddr.IP{}, dns.RCodeSuccess
|
|
case dns.TypeAAAA:
|
|
for _, ip := range addrs {
|
|
if ip.Is6() {
|
|
metricDNSResolveLocalOKAAAA.Add(1)
|
|
return ip, dns.RCodeSuccess
|
|
}
|
|
}
|
|
metricDNSResolveLocalNoAAAA.Add(1)
|
|
return netaddr.IP{}, dns.RCodeSuccess
|
|
case dns.TypeALL:
|
|
// Answer with whatever we've got.
|
|
// It could be IPv4, IPv6, or a zero addr.
|
|
// TODO: Return all available resolutions (A and AAAA, if we have them).
|
|
if len(addrs) == 0 {
|
|
metricDNSResolveLocalNoAll.Add(1)
|
|
return netaddr.IP{}, dns.RCodeSuccess
|
|
}
|
|
metricDNSResolveLocalOKAll.Add(1)
|
|
return addrs[0], dns.RCodeSuccess
|
|
|
|
// Leave some some record types explicitly unimplemented.
|
|
// These types relate to recursive resolution or special
|
|
// DNS semantics and might be implemented in the future.
|
|
case dns.TypeNS, dns.TypeSOA, dns.TypeAXFR, dns.TypeHINFO:
|
|
metricDNSResolveNotImplType.Add(1)
|
|
return netaddr.IP{}, dns.RCodeNotImplemented
|
|
|
|
// For everything except for the few types above that are explicitly not implemented, return no records.
|
|
// This is what other DNS systems do: always return NOERROR
|
|
// without any records whenever the requested record type is unknown.
|
|
// You can try this with:
|
|
// dig -t TYPE9824 example.com
|
|
// and note that NOERROR is returned, despite that record type being made up.
|
|
default:
|
|
metricDNSResolveNoRecordType.Add(1)
|
|
// The name exists, but no records exist of the requested type.
|
|
return netaddr.IP{}, dns.RCodeSuccess
|
|
}
|
|
}
|
|
|
|
// resolveReverse returns the unique domain name that maps to the given address.
|
|
func (r *Resolver) resolveLocalReverse(name dnsname.FQDN) (dnsname.FQDN, dns.RCode) {
|
|
var ip netaddr.IP
|
|
var ok bool
|
|
switch {
|
|
case strings.HasSuffix(name.WithTrailingDot(), rdnsv4Suffix):
|
|
ip, ok = rdnsNameToIPv4(name)
|
|
case strings.HasSuffix(name.WithTrailingDot(), rdnsv6Suffix):
|
|
ip, ok = rdnsNameToIPv6(name)
|
|
}
|
|
if !ok {
|
|
// This isn't a well-formed in-addr.arpa or ip6.arpa name, but
|
|
// who knows what upstreams might do, try kicking it up to
|
|
// them. We definitely won't handle it.
|
|
return "", dns.RCodeRefused
|
|
}
|
|
|
|
// If someone curiously does a reverse lookup on the DNS IP, we
|
|
// return a domain that helps indicate that Tailscale is using
|
|
// this IP for a special purpose and it is not a node on their
|
|
// tailnet.
|
|
if ip == tsaddr.TailscaleServiceIP() || ip == tsaddr.TailscaleServiceIPv6() {
|
|
return dnsSymbolicFQDN, dns.RCodeSuccess
|
|
}
|
|
|
|
r.mu.Lock()
|
|
defer r.mu.Unlock()
|
|
ret, ok := r.ipToHost[ip]
|
|
if !ok {
|
|
for _, suffix := range r.localDomains {
|
|
if suffix.Contains(name) {
|
|
// We are authoritative for this chunk of IP space.
|
|
return "", dns.RCodeNameError
|
|
}
|
|
}
|
|
// Not authoritative, signal that forwarding is advisable.
|
|
return "", dns.RCodeRefused
|
|
}
|
|
return ret, dns.RCodeSuccess
|
|
}
|
|
|
|
func (r *Resolver) handleQuery(pkt packet) {
|
|
defer atomic.AddInt32(&r.activeQueriesAtomic, -1)
|
|
|
|
out, err := r.respond(pkt.bs)
|
|
if err == errNotOurName {
|
|
err = r.forwarder.forward(pkt)
|
|
if err == nil {
|
|
// forward will send response into r.responses, nothing to do.
|
|
return
|
|
}
|
|
}
|
|
if err != nil {
|
|
select {
|
|
case <-r.closed:
|
|
case r.errors <- err:
|
|
}
|
|
} else {
|
|
select {
|
|
case <-r.closed:
|
|
case r.responses <- packet{out, pkt.addr}:
|
|
}
|
|
}
|
|
}
|
|
|
|
type response struct {
|
|
Header dns.Header
|
|
Question dns.Question
|
|
|
|
// Name is the response to a PTR query.
|
|
Name dnsname.FQDN
|
|
|
|
// IP and IPs are the responses to an A, AAAA, or ALL query.
|
|
// Either/both/neither can be populated.
|
|
IP netaddr.IP
|
|
IPs []netaddr.IP
|
|
|
|
// TXT is the response to a TXT query.
|
|
// Each one is its own RR with one string.
|
|
TXT []string
|
|
|
|
// CNAME is the response to a CNAME query.
|
|
CNAME string
|
|
|
|
// SRVs are the responses to a SRV query.
|
|
SRVs []*net.SRV
|
|
|
|
// NSs are the responses to an NS query.
|
|
NSs []*net.NS
|
|
}
|
|
|
|
var dnsParserPool = &sync.Pool{
|
|
New: func() any {
|
|
return new(dnsParser)
|
|
},
|
|
}
|
|
|
|
// dnsParser parses DNS queries using x/net/dns/dnsmessage.
|
|
// These structs are pooled with dnsParserPool.
|
|
type dnsParser struct {
|
|
Header dns.Header
|
|
Question dns.Question
|
|
|
|
parser dns.Parser
|
|
}
|
|
|
|
func (p *dnsParser) response() *response {
|
|
return &response{Header: p.Header, Question: p.Question}
|
|
}
|
|
|
|
// zeroParser clears parser so it doesn't retain its most recently
|
|
// parsed DNS query's []byte while it's sitting in a sync.Pool.
|
|
// It's not useful to keep anyway: the next Start will do the same.
|
|
func (p *dnsParser) zeroParser() { p.parser = dns.Parser{} }
|
|
|
|
// parseQuery parses the query in given packet into p.Header and
|
|
// p.Question.
|
|
func (p *dnsParser) parseQuery(query []byte) error {
|
|
defer p.zeroParser()
|
|
p.zeroParser()
|
|
var err error
|
|
p.Header, err = p.parser.Start(query)
|
|
if err != nil {
|
|
return err
|
|
}
|
|
if p.Header.Response {
|
|
return errNotQuery
|
|
}
|
|
p.Question, err = p.parser.Question()
|
|
return err
|
|
}
|
|
|
|
// marshalARecord serializes an A record into an active builder.
|
|
// The caller may continue using the builder following the call.
|
|
func marshalARecord(name dns.Name, ip netaddr.IP, builder *dns.Builder) error {
|
|
var answer dns.AResource
|
|
|
|
answerHeader := dns.ResourceHeader{
|
|
Name: name,
|
|
Type: dns.TypeA,
|
|
Class: dns.ClassINET,
|
|
TTL: uint32(defaultTTL / time.Second),
|
|
}
|
|
ipbytes := ip.As4()
|
|
copy(answer.A[:], ipbytes[:])
|
|
return builder.AResource(answerHeader, answer)
|
|
}
|
|
|
|
// marshalAAAARecord serializes an AAAA record into an active builder.
|
|
// The caller may continue using the builder following the call.
|
|
func marshalAAAARecord(name dns.Name, ip netaddr.IP, builder *dns.Builder) error {
|
|
var answer dns.AAAAResource
|
|
|
|
answerHeader := dns.ResourceHeader{
|
|
Name: name,
|
|
Type: dns.TypeAAAA,
|
|
Class: dns.ClassINET,
|
|
TTL: uint32(defaultTTL / time.Second),
|
|
}
|
|
ipbytes := ip.As16()
|
|
copy(answer.AAAA[:], ipbytes[:])
|
|
return builder.AAAAResource(answerHeader, answer)
|
|
}
|
|
|
|
func marshalIP(name dns.Name, ip netaddr.IP, builder *dns.Builder) error {
|
|
if ip.Is4() {
|
|
return marshalARecord(name, ip, builder)
|
|
}
|
|
if ip.Is6() {
|
|
return marshalAAAARecord(name, ip, builder)
|
|
}
|
|
return nil
|
|
}
|
|
|
|
// marshalPTRRecord serializes a PTR record into an active builder.
|
|
// The caller may continue using the builder following the call.
|
|
func marshalPTRRecord(queryName dns.Name, name dnsname.FQDN, builder *dns.Builder) error {
|
|
var answer dns.PTRResource
|
|
var err error
|
|
|
|
answerHeader := dns.ResourceHeader{
|
|
Name: queryName,
|
|
Type: dns.TypePTR,
|
|
Class: dns.ClassINET,
|
|
TTL: uint32(defaultTTL / time.Second),
|
|
}
|
|
answer.PTR, err = dns.NewName(name.WithTrailingDot())
|
|
if err != nil {
|
|
return err
|
|
}
|
|
return builder.PTRResource(answerHeader, answer)
|
|
}
|
|
|
|
func marshalTXT(queryName dns.Name, txts []string, builder *dns.Builder) error {
|
|
for _, txt := range txts {
|
|
if err := builder.TXTResource(dns.ResourceHeader{
|
|
Name: queryName,
|
|
Type: dns.TypeTXT,
|
|
Class: dns.ClassINET,
|
|
TTL: uint32(defaultTTL / time.Second),
|
|
}, dns.TXTResource{
|
|
TXT: []string{txt},
|
|
}); err != nil {
|
|
return err
|
|
}
|
|
}
|
|
return nil
|
|
}
|
|
|
|
func marshalCNAME(queryName dns.Name, cname string, builder *dns.Builder) error {
|
|
if cname == "" {
|
|
return nil
|
|
}
|
|
name, err := dns.NewName(cname)
|
|
if err != nil {
|
|
return err
|
|
}
|
|
return builder.CNAMEResource(dns.ResourceHeader{
|
|
Name: queryName,
|
|
Type: dns.TypeCNAME,
|
|
Class: dns.ClassINET,
|
|
TTL: uint32(defaultTTL / time.Second),
|
|
}, dns.CNAMEResource{
|
|
CNAME: name,
|
|
})
|
|
}
|
|
|
|
func marshalNS(queryName dns.Name, nss []*net.NS, builder *dns.Builder) error {
|
|
for _, ns := range nss {
|
|
name, err := dns.NewName(ns.Host)
|
|
if err != nil {
|
|
return err
|
|
}
|
|
err = builder.NSResource(dns.ResourceHeader{
|
|
Name: queryName,
|
|
Type: dns.TypeNS,
|
|
Class: dns.ClassINET,
|
|
TTL: uint32(defaultTTL / time.Second),
|
|
}, dns.NSResource{NS: name})
|
|
if err != nil {
|
|
return err
|
|
}
|
|
}
|
|
return nil
|
|
}
|
|
|
|
func marshalSRV(queryName dns.Name, srvs []*net.SRV, builder *dns.Builder) error {
|
|
for _, s := range srvs {
|
|
srvName, err := dns.NewName(s.Target)
|
|
if err != nil {
|
|
return err
|
|
}
|
|
err = builder.SRVResource(dns.ResourceHeader{
|
|
Name: queryName,
|
|
Type: dns.TypeSRV,
|
|
Class: dns.ClassINET,
|
|
TTL: uint32(defaultTTL / time.Second),
|
|
}, dns.SRVResource{
|
|
Target: srvName,
|
|
Priority: s.Priority,
|
|
Port: s.Port,
|
|
Weight: s.Weight,
|
|
})
|
|
if err != nil {
|
|
return err
|
|
}
|
|
}
|
|
return nil
|
|
}
|
|
|
|
// marshalResponse serializes the DNS response into a new buffer.
|
|
func marshalResponse(resp *response) ([]byte, error) {
|
|
resp.Header.Response = true
|
|
resp.Header.Authoritative = true
|
|
if resp.Header.RecursionDesired {
|
|
resp.Header.RecursionAvailable = true
|
|
}
|
|
|
|
builder := dns.NewBuilder(nil, resp.Header)
|
|
|
|
// TODO(bradfitz): I'm not sure why this wasn't enabled
|
|
// before, but for now (2021-12-09) enable it at least when
|
|
// there's more than 1 record (which was never the case
|
|
// before), where it really helps.
|
|
if len(resp.IPs) > 1 {
|
|
builder.EnableCompression()
|
|
}
|
|
|
|
isSuccess := resp.Header.RCode == dns.RCodeSuccess
|
|
|
|
if resp.Question.Type != 0 || isSuccess {
|
|
err := builder.StartQuestions()
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
err = builder.Question(resp.Question)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
}
|
|
|
|
// Only successful responses contain answers.
|
|
if !isSuccess {
|
|
return builder.Finish()
|
|
}
|
|
|
|
err := builder.StartAnswers()
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
switch resp.Question.Type {
|
|
case dns.TypeA, dns.TypeAAAA, dns.TypeALL:
|
|
if err := marshalIP(resp.Question.Name, resp.IP, &builder); err != nil {
|
|
return nil, err
|
|
}
|
|
for _, ip := range resp.IPs {
|
|
if err := marshalIP(resp.Question.Name, ip, &builder); err != nil {
|
|
return nil, err
|
|
}
|
|
}
|
|
case dns.TypePTR:
|
|
err = marshalPTRRecord(resp.Question.Name, resp.Name, &builder)
|
|
case dns.TypeTXT:
|
|
err = marshalTXT(resp.Question.Name, resp.TXT, &builder)
|
|
case dns.TypeCNAME:
|
|
err = marshalCNAME(resp.Question.Name, resp.CNAME, &builder)
|
|
case dns.TypeSRV:
|
|
err = marshalSRV(resp.Question.Name, resp.SRVs, &builder)
|
|
case dns.TypeNS:
|
|
err = marshalNS(resp.Question.Name, resp.NSs, &builder)
|
|
}
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
return builder.Finish()
|
|
}
|
|
|
|
const (
|
|
rdnsv4Suffix = ".in-addr.arpa."
|
|
rdnsv6Suffix = ".ip6.arpa."
|
|
)
|
|
|
|
// hasRDNSBonjourPrefix reports whether name has a Bonjour Service Prefix..
|
|
//
|
|
// https://tools.ietf.org/html/rfc6763 lists
|
|
// "five special RR names" for Bonjour service discovery:
|
|
//
|
|
// b._dns-sd._udp.<domain>.
|
|
// db._dns-sd._udp.<domain>.
|
|
// r._dns-sd._udp.<domain>.
|
|
// dr._dns-sd._udp.<domain>.
|
|
// lb._dns-sd._udp.<domain>.
|
|
func hasRDNSBonjourPrefix(name dnsname.FQDN) bool {
|
|
s := name.WithTrailingDot()
|
|
base, rest, ok := strings.Cut(s, ".")
|
|
if !ok {
|
|
return false // shouldn't happen
|
|
}
|
|
switch base {
|
|
case "b", "db", "r", "dr", "lb":
|
|
default:
|
|
return false
|
|
}
|
|
|
|
return strings.HasPrefix(rest, "_dns-sd._udp.")
|
|
}
|
|
|
|
// rawNameToLower converts a raw DNS name to a string, lowercasing it.
|
|
func rawNameToLower(name []byte) string {
|
|
var sb strings.Builder
|
|
sb.Grow(len(name))
|
|
|
|
for _, b := range name {
|
|
if 'A' <= b && b <= 'Z' {
|
|
b = b - 'A' + 'a'
|
|
}
|
|
sb.WriteByte(b)
|
|
}
|
|
|
|
return sb.String()
|
|
}
|
|
|
|
// ptrNameToIPv4 transforms a PTR name representing an IPv4 address to said address.
|
|
// Such names are IPv4 labels in reverse order followed by .in-addr.arpa.
|
|
// For example,
|
|
// 4.3.2.1.in-addr.arpa
|
|
// is transformed to
|
|
// 1.2.3.4
|
|
func rdnsNameToIPv4(name dnsname.FQDN) (ip netaddr.IP, ok bool) {
|
|
s := strings.TrimSuffix(name.WithTrailingDot(), rdnsv4Suffix)
|
|
ip, err := netaddr.ParseIP(s)
|
|
if err != nil {
|
|
return netaddr.IP{}, false
|
|
}
|
|
if !ip.Is4() {
|
|
return netaddr.IP{}, false
|
|
}
|
|
b := ip.As4()
|
|
return netaddr.IPv4(b[3], b[2], b[1], b[0]), true
|
|
}
|
|
|
|
// ptrNameToIPv6 transforms a PTR name representing an IPv6 address to said address.
|
|
// Such names are dot-separated nibbles in reverse order followed by .ip6.arpa.
|
|
// For example,
|
|
// b.a.9.8.7.6.5.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.0.8.b.d.0.1.0.0.2.ip6.arpa.
|
|
// is transformed to
|
|
// 2001:db8::567:89ab
|
|
func rdnsNameToIPv6(name dnsname.FQDN) (ip netaddr.IP, ok bool) {
|
|
var b [32]byte
|
|
var ipb [16]byte
|
|
|
|
s := strings.TrimSuffix(name.WithTrailingDot(), rdnsv6Suffix)
|
|
// 32 nibbles and 31 dots between them.
|
|
if len(s) != 63 {
|
|
return netaddr.IP{}, false
|
|
}
|
|
|
|
// Dots and hex digits alternate.
|
|
prevDot := true
|
|
// i ranges over name backward; j ranges over b forward.
|
|
for i, j := len(s)-1, 0; i >= 0; i-- {
|
|
thisDot := (s[i] == '.')
|
|
if prevDot == thisDot {
|
|
return netaddr.IP{}, false
|
|
}
|
|
prevDot = thisDot
|
|
|
|
if !thisDot {
|
|
// This is safe assuming alternation.
|
|
// We do not check that non-dots are hex digits: hex.Decode below will do that.
|
|
b[j] = s[i]
|
|
j++
|
|
}
|
|
}
|
|
|
|
_, err := hex.Decode(ipb[:], b[:])
|
|
if err != nil {
|
|
return netaddr.IP{}, false
|
|
}
|
|
|
|
return netaddr.IPFrom16(ipb), true
|
|
}
|
|
|
|
// respondReverse returns a DNS response to a PTR query.
|
|
// It is assumed that resp.Question is populated by respond before this is called.
|
|
func (r *Resolver) respondReverse(query []byte, name dnsname.FQDN, resp *response) ([]byte, error) {
|
|
if hasRDNSBonjourPrefix(name) {
|
|
metricDNSReverseMissBonjour.Add(1)
|
|
return nil, errNotOurName
|
|
}
|
|
|
|
resp.Name, resp.Header.RCode = r.resolveLocalReverse(name)
|
|
if resp.Header.RCode == dns.RCodeRefused {
|
|
metricDNSReverseMissOther.Add(1)
|
|
return nil, errNotOurName
|
|
}
|
|
|
|
metricDNSMagicDNSSuccessReverse.Add(1)
|
|
return marshalResponse(resp)
|
|
}
|
|
|
|
// respond returns a DNS response to query if it can be resolved locally.
|
|
// Otherwise, it returns errNotOurName.
|
|
func (r *Resolver) respond(query []byte) ([]byte, error) {
|
|
parser := dnsParserPool.Get().(*dnsParser)
|
|
defer dnsParserPool.Put(parser)
|
|
|
|
// ParseQuery is sufficiently fast to run on every DNS packet.
|
|
// This is considerably simpler than extracting the name by hand
|
|
// to shave off microseconds in case of delegation.
|
|
err := parser.parseQuery(query)
|
|
// We will not return this error: it is the sender's fault.
|
|
if err != nil {
|
|
if errors.Is(err, dns.ErrSectionDone) {
|
|
metricDNSErrorParseNoQ.Add(1)
|
|
r.logf("parseQuery(%02x): no DNS questions", query)
|
|
} else {
|
|
metricDNSErrorParseQuery.Add(1)
|
|
r.logf("parseQuery(%02x): %v", query, err)
|
|
}
|
|
resp := parser.response()
|
|
resp.Header.RCode = dns.RCodeFormatError
|
|
return marshalResponse(resp)
|
|
}
|
|
rawName := parser.Question.Name.Data[:parser.Question.Name.Length]
|
|
name, err := dnsname.ToFQDN(rawNameToLower(rawName))
|
|
if err != nil {
|
|
metricDNSErrorNotFQDN.Add(1)
|
|
// DNS packet unexpectedly contains an invalid FQDN.
|
|
resp := parser.response()
|
|
resp.Header.RCode = dns.RCodeFormatError
|
|
return marshalResponse(resp)
|
|
}
|
|
|
|
// Always try to handle reverse lookups; delegate inside when not found.
|
|
// This way, queries for existent nodes do not leak,
|
|
// but we behave gracefully if non-Tailscale nodes exist in CGNATRange.
|
|
if parser.Question.Type == dns.TypePTR {
|
|
return r.respondReverse(query, name, parser.response())
|
|
}
|
|
|
|
ip, rcode := r.resolveLocal(name, parser.Question.Type)
|
|
if rcode == dns.RCodeRefused {
|
|
return nil, errNotOurName // sentinel error return value: it requests forwarding
|
|
}
|
|
|
|
resp := parser.response()
|
|
resp.Header.RCode = rcode
|
|
resp.IP = ip
|
|
return marshalResponse(resp)
|
|
}
|
|
|
|
// unARPA maps from "4.4.8.8.in-addr.arpa." to "8.8.4.4", etc.
|
|
func unARPA(a string) (ipStr string, ok bool) {
|
|
const suf4 = ".in-addr.arpa."
|
|
if strings.HasSuffix(a, suf4) {
|
|
s := strings.TrimSuffix(a, suf4)
|
|
// Parse and reverse octets.
|
|
ip, err := netaddr.ParseIP(s)
|
|
if err != nil || !ip.Is4() {
|
|
return "", false
|
|
}
|
|
a4 := ip.As4()
|
|
return netaddr.IPv4(a4[3], a4[2], a4[1], a4[0]).String(), true
|
|
}
|
|
const suf6 = ".ip6.arpa."
|
|
if len(a) == len("e.0.0.2.0.0.0.0.0.0.0.0.0.0.0.0.b.0.8.0.a.0.0.4.0.b.8.f.7.0.6.2.ip6.arpa.") &&
|
|
strings.HasSuffix(a, suf6) {
|
|
var hx [32]byte
|
|
var a16 [16]byte
|
|
for i := range hx {
|
|
hx[31-i] = a[i*2]
|
|
if a[i*2+1] != '.' {
|
|
return "", false
|
|
}
|
|
}
|
|
hex.Decode(a16[:], hx[:])
|
|
return netaddr.IPFrom16(a16).String(), true
|
|
}
|
|
return "", false
|
|
|
|
}
|
|
|
|
var (
|
|
metricDNSQueryLocal = clientmetric.NewCounter("dns_query_local")
|
|
metricDNSQueryErrorClosed = clientmetric.NewCounter("dns_query_local_error_closed")
|
|
metricDNSQueryErrorQueue = clientmetric.NewCounter("dns_query_local_error_queue")
|
|
|
|
metricDNSErrorParseNoQ = clientmetric.NewCounter("dns_query_respond_error_no_question")
|
|
metricDNSErrorParseQuery = clientmetric.NewCounter("dns_query_respond_error_parse")
|
|
metricDNSErrorNotFQDN = clientmetric.NewCounter("dns_query_respond_error_not_fqdn")
|
|
|
|
metricDNSMagicDNSSuccessName = clientmetric.NewCounter("dns_query_magic_success_name")
|
|
metricDNSMagicDNSSuccessReverse = clientmetric.NewCounter("dns_query_magic_success_reverse")
|
|
|
|
metricDNSExitProxyQuery = clientmetric.NewCounter("dns_exit_node_query")
|
|
metricDNSExitProxyErrorName = clientmetric.NewCounter("dns_exit_node_error_name")
|
|
metricDNSExitProxyErrorForward = clientmetric.NewCounter("dns_exit_node_error_forward")
|
|
metricDNSExitProxyErrorResolvConf = clientmetric.NewCounter("dns_exit_node_error_resolvconf")
|
|
|
|
metricDNSFwd = clientmetric.NewCounter("dns_query_fwd")
|
|
metricDNSFwdDropBonjour = clientmetric.NewCounter("dns_query_fwd_drop_bonjour")
|
|
metricDNSFwdErrorName = clientmetric.NewCounter("dns_query_fwd_error_name")
|
|
metricDNSFwdErrorNoUpstream = clientmetric.NewCounter("dns_query_fwd_error_no_upstream")
|
|
metricDNSFwdSuccess = clientmetric.NewCounter("dns_query_fwd_success")
|
|
metricDNSFwdErrorContext = clientmetric.NewCounter("dns_query_fwd_error_context")
|
|
metricDNSFwdErrorContextGotError = clientmetric.NewCounter("dns_query_fwd_error_context_got_error")
|
|
|
|
metricDNSFwdErrorType = clientmetric.NewCounter("dns_query_fwd_error_type")
|
|
metricDNSFwdErrorParseAddr = clientmetric.NewCounter("dns_query_fwd_error_parse_addr")
|
|
|
|
metricDNSFwdUDP = clientmetric.NewCounter("dns_query_fwd_udp") // on entry
|
|
metricDNSFwdUDPWrote = clientmetric.NewCounter("dns_query_fwd_udp_wrote") // sent UDP packet
|
|
metricDNSFwdUDPErrorWrite = clientmetric.NewCounter("dns_query_fwd_udp_error_write")
|
|
metricDNSFwdUDPErrorServer = clientmetric.NewCounter("dns_query_fwd_udp_error_server")
|
|
metricDNSFwdUDPErrorTxID = clientmetric.NewCounter("dns_query_fwd_udp_error_txid")
|
|
metricDNSFwdUDPErrorRead = clientmetric.NewCounter("dns_query_fwd_udp_error_read")
|
|
metricDNSFwdUDPSuccess = clientmetric.NewCounter("dns_query_fwd_udp_success")
|
|
|
|
metricDNSFwdDoH = clientmetric.NewCounter("dns_query_fwd_doh")
|
|
metricDNSFwdDoHErrorStatus = clientmetric.NewCounter("dns_query_fwd_doh_error_status")
|
|
metricDNSFwdDoHErrorCT = clientmetric.NewCounter("dns_query_fwd_doh_error_content_type")
|
|
metricDNSFwdDoHErrorTransport = clientmetric.NewCounter("dns_query_fwd_doh_error_transport")
|
|
metricDNSFwdDoHErrorBody = clientmetric.NewCounter("dns_query_fwd_doh_error_body")
|
|
|
|
metricDNSResolveLocal = clientmetric.NewCounter("dns_resolve_local")
|
|
metricDNSResolveLocalErrorOnion = clientmetric.NewCounter("dns_resolve_local_error_onion")
|
|
metricDNSResolveLocalErrorMissing = clientmetric.NewCounter("dns_resolve_local_error_missing")
|
|
metricDNSResolveLocalErrorRefused = clientmetric.NewCounter("dns_resolve_local_error_refused")
|
|
metricDNSResolveLocalOKA = clientmetric.NewCounter("dns_resolve_local_ok_a")
|
|
metricDNSResolveLocalOKAAAA = clientmetric.NewCounter("dns_resolve_local_ok_aaaa")
|
|
metricDNSResolveLocalOKAll = clientmetric.NewCounter("dns_resolve_local_ok_all")
|
|
metricDNSResolveLocalNoA = clientmetric.NewCounter("dns_resolve_local_no_a")
|
|
metricDNSResolveLocalNoAAAA = clientmetric.NewCounter("dns_resolve_local_no_aaaa")
|
|
metricDNSResolveLocalNoAll = clientmetric.NewCounter("dns_resolve_local_no_all")
|
|
metricDNSResolveNotImplType = clientmetric.NewCounter("dns_resolve_local_not_impl_type")
|
|
metricDNSResolveNoRecordType = clientmetric.NewCounter("dns_resolve_local_no_record_type")
|
|
|
|
metricDNSReverseMissBonjour = clientmetric.NewCounter("dns_reverse_miss_bonjour")
|
|
metricDNSReverseMissOther = clientmetric.NewCounter("dns_reverse_miss_other")
|
|
)
|