mirror of
https://github.com/XTLS/Xray-core.git
synced 2024-11-04 14:13:03 +00:00
cd4631ce99
* DNS: add clientip for specific nameserver * Refactoring: DNS App * DNS: add DNS over QUIC support * Feat: add disableCache option for DNS * Feat: add queryStrategy option for DNS * Feat: add disableFallback & skipFallback option for DNS * Feat: DNS hosts support multiple addresses * Feat: DNS transport over TCP * DNS: fix typo & refine code * DNS: refine code * Add disableFallbackIfMatch dns option * Feat: routing and freedom outbound ignore Fake DNS Turn off fake DNS for request sent from Routing and Freedom outbound. Fake DNS now only apply to DNS outbound. This is important for Android, where VPN service take over all system DNS traffic and pass it to core. "UseIp" option can be used in Freedom outbound to avoid getting fake IP and fail connection. * Fix test * Fix dns return * Fix local dns return empty * Apply timeout to dns outbound * Update app/dns/config.go Co-authored-by: Loyalsoldier <10487845+loyalsoldier@users.noreply.github.com> Co-authored-by: Ye Zhihao <vigilans@foxmail.com> Co-authored-by: maskedeken <52683904+maskedeken@users.noreply.github.com> Co-authored-by: V2Fly Team <51714622+vcptr@users.noreply.github.com> Co-authored-by: CalmLong <37164399+calmlong@users.noreply.github.com> Co-authored-by: Shelikhoo <xiaokangwang@outlook.com> Co-authored-by: 秋のかえで <autmaple@protonmail.com> Co-authored-by: 朱聖黎 <digglife@gmail.com> Co-authored-by: rurirei <72071920+rurirei@users.noreply.github.com> Co-authored-by: yuhan6665 <1588741+yuhan6665@users.noreply.github.com> Co-authored-by: Arthur Morgan <4637240+badO1a5A90@users.noreply.github.com>
363 lines
8.2 KiB
Go
363 lines
8.2 KiB
Go
package dns
|
|
|
|
import (
|
|
"bytes"
|
|
"context"
|
|
"encoding/binary"
|
|
"net/url"
|
|
"sync"
|
|
"sync/atomic"
|
|
"time"
|
|
|
|
"golang.org/x/net/dns/dnsmessage"
|
|
|
|
"github.com/xtls/xray-core/common"
|
|
"github.com/xtls/xray-core/common/buf"
|
|
"github.com/xtls/xray-core/common/net"
|
|
"github.com/xtls/xray-core/common/net/cnc"
|
|
"github.com/xtls/xray-core/common/protocol/dns"
|
|
"github.com/xtls/xray-core/common/session"
|
|
"github.com/xtls/xray-core/common/signal/pubsub"
|
|
"github.com/xtls/xray-core/common/task"
|
|
dns_feature "github.com/xtls/xray-core/features/dns"
|
|
"github.com/xtls/xray-core/features/routing"
|
|
"github.com/xtls/xray-core/transport/internet"
|
|
)
|
|
|
|
// TCPNameServer implemented DNS over TCP (RFC7766).
|
|
type TCPNameServer struct {
|
|
sync.RWMutex
|
|
name string
|
|
destination *net.Destination
|
|
ips map[string]*record
|
|
pub *pubsub.Service
|
|
cleanup *task.Periodic
|
|
reqID uint32
|
|
dial func(context.Context) (net.Conn, error)
|
|
}
|
|
|
|
// NewTCPNameServer creates DNS over TCP server object for remote resolving.
|
|
func NewTCPNameServer(url *url.URL, dispatcher routing.Dispatcher) (*TCPNameServer, error) {
|
|
s, err := baseTCPNameServer(url, "TCP")
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
s.dial = func(ctx context.Context) (net.Conn, error) {
|
|
link, err := dispatcher.Dispatch(ctx, *s.destination)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
return cnc.NewConnection(
|
|
cnc.ConnectionInputMulti(link.Writer),
|
|
cnc.ConnectionOutputMulti(link.Reader),
|
|
), nil
|
|
}
|
|
|
|
return s, nil
|
|
}
|
|
|
|
// NewTCPLocalNameServer creates DNS over TCP client object for local resolving
|
|
func NewTCPLocalNameServer(url *url.URL) (*TCPNameServer, error) {
|
|
s, err := baseTCPNameServer(url, "TCPL")
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
s.dial = func(ctx context.Context) (net.Conn, error) {
|
|
return internet.DialSystem(ctx, *s.destination, nil)
|
|
}
|
|
|
|
return s, nil
|
|
}
|
|
|
|
func baseTCPNameServer(url *url.URL, prefix string) (*TCPNameServer, error) {
|
|
var err error
|
|
port := net.Port(53)
|
|
if url.Port() != "" {
|
|
port, err = net.PortFromString(url.Port())
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
}
|
|
dest := net.TCPDestination(net.ParseAddress(url.Hostname()), port)
|
|
|
|
s := &TCPNameServer{
|
|
destination: &dest,
|
|
ips: make(map[string]*record),
|
|
pub: pubsub.NewService(),
|
|
name: prefix + "//" + dest.NetAddr(),
|
|
}
|
|
s.cleanup = &task.Periodic{
|
|
Interval: time.Minute,
|
|
Execute: s.Cleanup,
|
|
}
|
|
|
|
return s, nil
|
|
}
|
|
|
|
// Name implements Server.
|
|
func (s *TCPNameServer) Name() string {
|
|
return s.name
|
|
}
|
|
|
|
// Cleanup clears expired items from cache
|
|
func (s *TCPNameServer) Cleanup() error {
|
|
now := time.Now()
|
|
s.Lock()
|
|
defer s.Unlock()
|
|
|
|
if len(s.ips) == 0 {
|
|
return newError("nothing to do. stopping...")
|
|
}
|
|
|
|
for domain, record := range s.ips {
|
|
if record.A != nil && record.A.Expire.Before(now) {
|
|
record.A = nil
|
|
}
|
|
if record.AAAA != nil && record.AAAA.Expire.Before(now) {
|
|
record.AAAA = nil
|
|
}
|
|
|
|
if record.A == nil && record.AAAA == nil {
|
|
newError(s.name, " cleanup ", domain).AtDebug().WriteToLog()
|
|
delete(s.ips, domain)
|
|
} else {
|
|
s.ips[domain] = record
|
|
}
|
|
}
|
|
|
|
if len(s.ips) == 0 {
|
|
s.ips = make(map[string]*record)
|
|
}
|
|
|
|
return nil
|
|
}
|
|
|
|
func (s *TCPNameServer) updateIP(req *dnsRequest, ipRec *IPRecord) {
|
|
elapsed := time.Since(req.start)
|
|
|
|
s.Lock()
|
|
rec, found := s.ips[req.domain]
|
|
if !found {
|
|
rec = &record{}
|
|
}
|
|
updated := false
|
|
|
|
switch req.reqType {
|
|
case dnsmessage.TypeA:
|
|
if isNewer(rec.A, ipRec) {
|
|
rec.A = ipRec
|
|
updated = true
|
|
}
|
|
case dnsmessage.TypeAAAA:
|
|
addr := make([]net.Address, 0)
|
|
for _, ip := range ipRec.IP {
|
|
if len(ip.IP()) == net.IPv6len {
|
|
addr = append(addr, ip)
|
|
}
|
|
}
|
|
ipRec.IP = addr
|
|
if isNewer(rec.AAAA, ipRec) {
|
|
rec.AAAA = ipRec
|
|
updated = true
|
|
}
|
|
}
|
|
newError(s.name, " got answer: ", req.domain, " ", req.reqType, " -> ", ipRec.IP, " ", elapsed).AtInfo().WriteToLog()
|
|
|
|
if updated {
|
|
s.ips[req.domain] = rec
|
|
}
|
|
switch req.reqType {
|
|
case dnsmessage.TypeA:
|
|
s.pub.Publish(req.domain+"4", nil)
|
|
case dnsmessage.TypeAAAA:
|
|
s.pub.Publish(req.domain+"6", nil)
|
|
}
|
|
s.Unlock()
|
|
common.Must(s.cleanup.Start())
|
|
}
|
|
|
|
func (s *TCPNameServer) newReqID() uint16 {
|
|
return uint16(atomic.AddUint32(&s.reqID, 1))
|
|
}
|
|
|
|
func (s *TCPNameServer) sendQuery(ctx context.Context, domain string, clientIP net.IP, option dns_feature.IPOption) {
|
|
newError(s.name, " querying DNS for: ", domain).AtDebug().WriteToLog(session.ExportIDToError(ctx))
|
|
|
|
reqs := buildReqMsgs(domain, option, s.newReqID, genEDNS0Options(clientIP))
|
|
|
|
var deadline time.Time
|
|
if d, ok := ctx.Deadline(); ok {
|
|
deadline = d
|
|
} else {
|
|
deadline = time.Now().Add(time.Second * 5)
|
|
}
|
|
|
|
for _, req := range reqs {
|
|
go func(r *dnsRequest) {
|
|
dnsCtx := ctx
|
|
|
|
if inbound := session.InboundFromContext(ctx); inbound != nil {
|
|
dnsCtx = session.ContextWithInbound(dnsCtx, inbound)
|
|
}
|
|
|
|
dnsCtx = session.ContextWithContent(dnsCtx, &session.Content{
|
|
Protocol: "dns",
|
|
SkipDNSResolve: true,
|
|
})
|
|
|
|
var cancel context.CancelFunc
|
|
dnsCtx, cancel = context.WithDeadline(dnsCtx, deadline)
|
|
defer cancel()
|
|
|
|
b, err := dns.PackMessage(r.msg)
|
|
if err != nil {
|
|
newError("failed to pack dns query").Base(err).AtError().WriteToLog()
|
|
return
|
|
}
|
|
|
|
conn, err := s.dial(dnsCtx)
|
|
if err != nil {
|
|
newError("failed to dial namesever").Base(err).AtError().WriteToLog()
|
|
return
|
|
}
|
|
defer conn.Close()
|
|
dnsReqBuf := buf.New()
|
|
binary.Write(dnsReqBuf, binary.BigEndian, uint16(b.Len()))
|
|
dnsReqBuf.Write(b.Bytes())
|
|
b.Release()
|
|
|
|
_, err = conn.Write(dnsReqBuf.Bytes())
|
|
if err != nil {
|
|
newError("failed to send query").Base(err).AtError().WriteToLog()
|
|
return
|
|
}
|
|
dnsReqBuf.Release()
|
|
|
|
respBuf := buf.New()
|
|
defer respBuf.Release()
|
|
n, err := respBuf.ReadFullFrom(conn, 2)
|
|
if err != nil && n == 0 {
|
|
newError("failed to read response length").Base(err).AtError().WriteToLog()
|
|
return
|
|
}
|
|
var length int16
|
|
err = binary.Read(bytes.NewReader(respBuf.Bytes()), binary.BigEndian, &length)
|
|
if err != nil {
|
|
newError("failed to parse response length").Base(err).AtError().WriteToLog()
|
|
return
|
|
}
|
|
respBuf.Clear()
|
|
n, err = respBuf.ReadFullFrom(conn, int32(length))
|
|
if err != nil && n == 0 {
|
|
newError("failed to read response length").Base(err).AtError().WriteToLog()
|
|
return
|
|
}
|
|
|
|
rec, err := parseResponse(respBuf.Bytes())
|
|
if err != nil {
|
|
newError("failed to parse DNS over TCP response").Base(err).AtError().WriteToLog()
|
|
return
|
|
}
|
|
|
|
s.updateIP(r, rec)
|
|
}(req)
|
|
}
|
|
}
|
|
|
|
func (s *TCPNameServer) findIPsForDomain(domain string, option dns_feature.IPOption) ([]net.IP, error) {
|
|
s.RLock()
|
|
record, found := s.ips[domain]
|
|
s.RUnlock()
|
|
|
|
if !found {
|
|
return nil, errRecordNotFound
|
|
}
|
|
|
|
var err4 error
|
|
var err6 error
|
|
var ips []net.Address
|
|
var ip6 []net.Address
|
|
|
|
if option.IPv4Enable {
|
|
ips, err4 = record.A.getIPs()
|
|
}
|
|
|
|
if option.IPv6Enable {
|
|
ip6, err6 = record.AAAA.getIPs()
|
|
ips = append(ips, ip6...)
|
|
}
|
|
|
|
if len(ips) > 0 {
|
|
return toNetIP(ips)
|
|
}
|
|
|
|
if err4 != nil {
|
|
return nil, err4
|
|
}
|
|
|
|
if err6 != nil {
|
|
return nil, err6
|
|
}
|
|
|
|
return nil, dns_feature.ErrEmptyResponse
|
|
}
|
|
|
|
// QueryIP implements Server.
|
|
func (s *TCPNameServer) QueryIP(ctx context.Context, domain string, clientIP net.IP, option dns_feature.IPOption, disableCache bool) ([]net.IP, error) {
|
|
fqdn := Fqdn(domain)
|
|
|
|
if disableCache {
|
|
newError("DNS cache is disabled. Querying IP for ", domain, " at ", s.name).AtDebug().WriteToLog()
|
|
} else {
|
|
ips, err := s.findIPsForDomain(fqdn, option)
|
|
if err != errRecordNotFound {
|
|
newError(s.name, " cache HIT ", domain, " -> ", ips).Base(err).AtDebug().WriteToLog()
|
|
return ips, err
|
|
}
|
|
}
|
|
|
|
// ipv4 and ipv6 belong to different subscription groups
|
|
var sub4, sub6 *pubsub.Subscriber
|
|
if option.IPv4Enable {
|
|
sub4 = s.pub.Subscribe(fqdn + "4")
|
|
defer sub4.Close()
|
|
}
|
|
if option.IPv6Enable {
|
|
sub6 = s.pub.Subscribe(fqdn + "6")
|
|
defer sub6.Close()
|
|
}
|
|
done := make(chan interface{})
|
|
go func() {
|
|
if sub4 != nil {
|
|
select {
|
|
case <-sub4.Wait():
|
|
case <-ctx.Done():
|
|
}
|
|
}
|
|
if sub6 != nil {
|
|
select {
|
|
case <-sub6.Wait():
|
|
case <-ctx.Done():
|
|
}
|
|
}
|
|
close(done)
|
|
}()
|
|
s.sendQuery(ctx, fqdn, clientIP, option)
|
|
|
|
for {
|
|
ips, err := s.findIPsForDomain(fqdn, option)
|
|
if err != errRecordNotFound {
|
|
return ips, err
|
|
}
|
|
|
|
select {
|
|
case <-ctx.Done():
|
|
return nil, ctx.Err()
|
|
case <-done:
|
|
}
|
|
}
|
|
}
|