Crazy sekai overturns the small pond

This commit is contained in:
世界 2024-10-21 23:38:34 +08:00
parent 253b41936e
commit 8304295c48
No known key found for this signature in database
GPG key ID: CD109927C34A63C4
139 changed files with 2866 additions and 1559 deletions

View file

@ -13,6 +13,7 @@ import (
"github.com/sagernet/sing-box/common/geosite"
C "github.com/sagernet/sing-box/constant"
"github.com/sagernet/sing-box/experimental/deprecated"
R "github.com/sagernet/sing-box/route/rule"
E "github.com/sagernet/sing/common/exceptions"
M "github.com/sagernet/sing/common/metadata"
"github.com/sagernet/sing/common/rw"
@ -32,7 +33,7 @@ func (r *Router) LoadGeosite(code string) (adapter.Rule, error) {
if err != nil {
return nil, err
}
rule, err = NewDefaultRule(r.ctx, r, nil, geosite.Compile(items))
rule, err = R.NewDefaultRule(r.ctx, r, nil, geosite.Compile(items))
if err != nil {
return nil, err
}

583
route/route.go Normal file
View file

@ -0,0 +1,583 @@
package route
import (
"context"
"errors"
"net"
"net/netip"
"os"
"os/user"
"strings"
"syscall"
"time"
"github.com/sagernet/sing-box/adapter"
"github.com/sagernet/sing-box/common/conntrack"
"github.com/sagernet/sing-box/common/process"
"github.com/sagernet/sing-box/common/sniff"
C "github.com/sagernet/sing-box/constant"
"github.com/sagernet/sing-box/option"
"github.com/sagernet/sing-box/outbound"
"github.com/sagernet/sing-box/route/rule"
"github.com/sagernet/sing-dns"
"github.com/sagernet/sing-mux"
"github.com/sagernet/sing-tun"
"github.com/sagernet/sing-vmess"
"github.com/sagernet/sing/common"
"github.com/sagernet/sing/common/buf"
"github.com/sagernet/sing/common/bufio"
"github.com/sagernet/sing/common/bufio/deadline"
E "github.com/sagernet/sing/common/exceptions"
F "github.com/sagernet/sing/common/format"
M "github.com/sagernet/sing/common/metadata"
N "github.com/sagernet/sing/common/network"
"github.com/sagernet/sing/common/uot"
)
// Deprecated: use RouteConnectionEx instead.
func (r *Router) RouteConnection(ctx context.Context, conn net.Conn, metadata adapter.InboundContext) error {
return r.routeConnection(ctx, conn, metadata, nil)
}
func (r *Router) RouteConnectionEx(ctx context.Context, conn net.Conn, metadata adapter.InboundContext, onClose N.CloseHandlerFunc) {
err := r.routeConnection(ctx, conn, metadata, onClose)
if err != nil {
N.CloseOnHandshakeFailure(conn, onClose, err)
if E.IsClosedOrCanceled(err) {
r.logger.DebugContext(ctx, "connection closed: ", err)
} else {
r.logger.ErrorContext(ctx, err)
}
}
}
func (r *Router) routeConnection(ctx context.Context, conn net.Conn, metadata adapter.InboundContext, onClose N.CloseHandlerFunc) error {
if r.pauseManager.IsDevicePaused() {
return E.New("reject connection to ", metadata.Destination, " while device paused")
}
if metadata.InboundDetour != "" {
if metadata.LastInbound == metadata.InboundDetour {
return E.New("routing loop on detour: ", metadata.InboundDetour)
}
detour := r.inboundByTag[metadata.InboundDetour]
if detour == nil {
return E.New("inbound detour not found: ", metadata.InboundDetour)
}
injectable, isInjectable := detour.(adapter.TCPInjectableInbound)
if !isInjectable {
return E.New("inbound detour is not TCP injectable: ", metadata.InboundDetour)
}
metadata.LastInbound = metadata.Inbound
metadata.Inbound = metadata.InboundDetour
metadata.InboundDetour = ""
injectable.NewConnectionEx(ctx, conn, metadata, onClose)
return nil
}
conntrack.KillerCheck()
metadata.Network = N.NetworkTCP
switch metadata.Destination.Fqdn {
case mux.Destination.Fqdn:
return E.New("global multiplex is deprecated since sing-box v1.7.0, enable multiplex in inbound options instead.")
case vmess.MuxDestination.Fqdn:
return E.New("global multiplex (v2ray legacy) not supported since sing-box v1.7.0.")
case uot.MagicAddress:
return E.New("global UoT not supported since sing-box v1.7.0.")
case uot.LegacyMagicAddress:
return E.New("global UoT (legacy) not supported since sing-box v1.7.0.")
}
if deadline.NeedAdditionalReadDeadline(conn) {
conn = deadline.NewConn(conn)
}
selectedRule, _, buffers, err := r.matchRule(ctx, &metadata, conn, nil, -1)
if err != nil {
return err
}
var selectedOutbound adapter.Outbound
var selectReturn bool
if selectedRule != nil {
switch action := selectedRule.Action().(type) {
case *rule.RuleActionRoute:
var loaded bool
selectedOutbound, loaded = r.Outbound(action.Outbound)
if !loaded {
buf.ReleaseMulti(buffers)
return E.New("outbound not found: ", action.Outbound)
}
case *rule.RuleActionReturn:
selectReturn = true
case *rule.RuleActionReject:
buf.ReleaseMulti(buffers)
var rejectErr error
switch action.Method {
case C.RuleActionRejectMethodDefault:
rejectErr = os.ErrClosed
case C.RuleActionRejectMethodPortUnreachable:
rejectErr = syscall.ECONNREFUSED
case C.RuleActionRejectMethodDrop:
rejectErr = tun.ErrDrop
}
N.CloseOnHandshakeFailure(conn, onClose, rejectErr)
return nil
}
}
if selectedRule == nil || selectReturn {
if r.defaultOutboundForConnection == nil {
buf.ReleaseMulti(buffers)
return E.New("missing default outbound with TCP support")
}
selectedOutbound = r.defaultOutboundForConnection
}
if !common.Contains(selectedOutbound.Network(), N.NetworkTCP) {
buf.ReleaseMulti(buffers)
return E.New("TCP is not supported by outbound: ", selectedOutbound.Tag())
}
for _, buffer := range buffers {
conn = bufio.NewCachedConn(conn, buffer)
}
if r.clashServer != nil {
trackerConn, tracker := r.clashServer.RoutedConnection(ctx, conn, metadata, selectedRule)
defer tracker.Leave()
conn = trackerConn
}
if r.v2rayServer != nil {
if statsService := r.v2rayServer.StatsService(); statsService != nil {
conn = statsService.RoutedConnection(metadata.Inbound, selectedOutbound.Tag(), metadata.User, conn)
}
}
legacyOutbound, isLegacy := selectedOutbound.(adapter.ConnectionHandler)
if isLegacy {
err = legacyOutbound.NewConnection(ctx, conn, metadata)
if err != nil {
conn.Close()
if onClose != nil {
onClose(err)
}
return E.Cause(err, "outbound/", selectedOutbound.Type(), "[", selectedOutbound.Tag(), "]")
} else {
if onClose != nil {
onClose(nil)
}
}
return nil
}
// TODO
err = outbound.NewConnection(ctx, selectedOutbound, conn, metadata)
if err != nil {
conn.Close()
if onClose != nil {
onClose(err)
}
return E.Cause(err, "outbound/", selectedOutbound.Type(), "[", selectedOutbound.Tag(), "]")
} else {
if onClose != nil {
onClose(nil)
}
}
return nil
}
func (r *Router) RoutePacketConnection(ctx context.Context, conn N.PacketConn, metadata adapter.InboundContext) error {
err := r.routePacketConnection(ctx, conn, metadata, nil)
if err != nil {
conn.Close()
if E.IsClosedOrCanceled(err) {
r.logger.DebugContext(ctx, "connection closed: ", err)
} else {
r.logger.ErrorContext(ctx, err)
}
}
return nil
}
func (r *Router) RoutePacketConnectionEx(ctx context.Context, conn N.PacketConn, metadata adapter.InboundContext, onClose N.CloseHandlerFunc) {
err := r.routePacketConnection(ctx, conn, metadata, onClose)
if err != nil {
N.CloseOnHandshakeFailure(conn, onClose, err)
if E.IsClosedOrCanceled(err) {
r.logger.DebugContext(ctx, "connection closed: ", err)
} else {
r.logger.ErrorContext(ctx, err)
}
} else if onClose != nil {
onClose(nil)
}
}
func (r *Router) routePacketConnection(ctx context.Context, conn N.PacketConn, metadata adapter.InboundContext, onClose N.CloseHandlerFunc) error {
if r.pauseManager.IsDevicePaused() {
return E.New("reject packet connection to ", metadata.Destination, " while device paused")
}
if metadata.InboundDetour != "" {
if metadata.LastInbound == metadata.InboundDetour {
return E.New("routing loop on detour: ", metadata.InboundDetour)
}
detour := r.inboundByTag[metadata.InboundDetour]
if detour == nil {
return E.New("inbound detour not found: ", metadata.InboundDetour)
}
injectable, isInjectable := detour.(adapter.UDPInjectableInbound)
if !isInjectable {
return E.New("inbound detour is not UDP injectable: ", metadata.InboundDetour)
}
metadata.LastInbound = metadata.Inbound
metadata.Inbound = metadata.InboundDetour
metadata.InboundDetour = ""
injectable.NewPacketConnectionEx(ctx, conn, metadata, onClose)
return nil
}
conntrack.KillerCheck()
// TODO: move to UoT
metadata.Network = N.NetworkUDP
// Currently we don't have deadline usages for UDP connections
/*if deadline.NeedAdditionalReadDeadline(conn) {
conn = deadline.NewPacketConn(bufio.NewNetPacketConn(conn))
}*/
selectedRule, _, buffers, err := r.matchRule(ctx, &metadata, nil, conn, -1)
if err != nil {
return err
}
var selectedOutbound adapter.Outbound
var selectReturn bool
if selectedRule != nil {
switch action := selectedRule.Action().(type) {
case *rule.RuleActionRoute:
var loaded bool
selectedOutbound, loaded = r.Outbound(action.Outbound)
if !loaded {
buf.ReleaseMulti(buffers)
return E.New("outbound not found: ", action.Outbound)
}
metadata.UDPDisableDomainUnmapping = action.UDPDisableDomainUnmapping
case *rule.RuleActionReturn:
selectReturn = true
case *rule.RuleActionReject:
buf.ReleaseMulti(buffers)
N.CloseOnHandshakeFailure(conn, onClose, syscall.ECONNREFUSED)
return nil
}
}
if selectedRule == nil || selectReturn {
if r.defaultOutboundForPacketConnection == nil {
buf.ReleaseMulti(buffers)
return E.New("missing default outbound with UDP support")
}
selectedOutbound = r.defaultOutboundForPacketConnection
}
if !common.Contains(selectedOutbound.Network(), N.NetworkUDP) {
buf.ReleaseMulti(buffers)
return E.New("UDP is not supported by outbound: ", selectedOutbound.Tag())
}
for _, buffer := range buffers {
// TODO: check if metadata.Destination == packet destination
conn = bufio.NewCachedPacketConn(conn, buffer, metadata.Destination)
}
if r.clashServer != nil {
trackerConn, tracker := r.clashServer.RoutedPacketConnection(ctx, conn, metadata, selectedRule)
defer tracker.Leave()
conn = trackerConn
}
if r.v2rayServer != nil {
if statsService := r.v2rayServer.StatsService(); statsService != nil {
conn = statsService.RoutedPacketConnection(metadata.Inbound, selectedOutbound.Tag(), metadata.User, conn)
}
}
if metadata.FakeIP {
conn = bufio.NewNATPacketConn(bufio.NewNetPacketConn(conn), metadata.OriginDestination, metadata.Destination)
}
legacyOutbound, isLegacy := selectedOutbound.(adapter.PacketConnectionHandler)
if isLegacy {
err = legacyOutbound.NewPacketConnection(ctx, conn, metadata)
N.CloseOnHandshakeFailure(conn, onClose, err)
if err != nil {
return E.Cause(err, "outbound/", selectedOutbound.Type(), "[", selectedOutbound.Tag(), "]")
}
return nil
}
// TODO
err = outbound.NewPacketConnection(ctx, selectedOutbound, conn, metadata)
N.CloseOnHandshakeFailure(conn, onClose, err)
if err != nil {
return E.Cause(err, "outbound/", selectedOutbound.Type(), "[", selectedOutbound.Tag(), "]")
}
return nil
}
func (r *Router) matchRule(
ctx context.Context, metadata *adapter.InboundContext,
inputConn net.Conn, inputPacketConn N.PacketConn, ruleIndex int,
) (selectedRule adapter.Rule, selectedRuleIndex int, buffers []*buf.Buffer, fatalErr error) {
if r.processSearcher != nil && metadata.ProcessInfo == nil {
var originDestination netip.AddrPort
if metadata.OriginDestination.IsValid() {
originDestination = metadata.OriginDestination.AddrPort()
} else if metadata.Destination.IsIP() {
originDestination = metadata.Destination.AddrPort()
}
processInfo, fErr := process.FindProcessInfo(r.processSearcher, ctx, metadata.Network, metadata.Source.AddrPort(), originDestination)
if fErr != nil {
r.logger.InfoContext(ctx, "failed to search process: ", fErr)
} else {
if processInfo.ProcessPath != "" {
r.logger.InfoContext(ctx, "found process path: ", processInfo.ProcessPath)
} else if processInfo.PackageName != "" {
r.logger.InfoContext(ctx, "found package name: ", processInfo.PackageName)
} else if processInfo.UserId != -1 {
if /*needUserName &&*/ true {
osUser, _ := user.LookupId(F.ToString(processInfo.UserId))
if osUser != nil {
processInfo.User = osUser.Username
}
}
if processInfo.User != "" {
r.logger.InfoContext(ctx, "found user: ", processInfo.User)
} else {
r.logger.InfoContext(ctx, "found user id: ", processInfo.UserId)
}
}
metadata.ProcessInfo = processInfo
}
}
if r.fakeIPStore != nil && r.fakeIPStore.Contains(metadata.Destination.Addr) {
domain, loaded := r.fakeIPStore.Lookup(metadata.Destination.Addr)
if !loaded {
fatalErr = E.New("missing fakeip record, try to configure experimental.cache_file")
return
}
metadata.OriginDestination = metadata.Destination
metadata.Destination = M.Socksaddr{
Fqdn: domain,
Port: metadata.Destination.Port,
}
metadata.FakeIP = true
r.logger.DebugContext(ctx, "found fakeip domain: ", domain)
}
if r.dnsReverseMapping != nil && metadata.Domain == "" {
domain, loaded := r.dnsReverseMapping.Query(metadata.Destination.Addr)
if loaded {
metadata.Domain = domain
r.logger.DebugContext(ctx, "found reserve mapped domain: ", metadata.Domain)
}
}
if metadata.Destination.IsIPv4() {
metadata.IPVersion = 4
} else if metadata.Destination.IsIPv6() {
metadata.IPVersion = 6
}
//nolint:staticcheck
if metadata.InboundOptions != common.DefaultValue[option.InboundOptions]() {
if metadata.InboundOptions.SniffEnabled {
newBuffers, newErr := r.actionSniff(ctx, metadata, &rule.RuleActionSniff{
OverrideDestination: metadata.InboundOptions.SniffOverrideDestination,
Timeout: time.Duration(metadata.InboundOptions.SniffTimeout),
}, inputConn, inputPacketConn)
if newErr != nil {
fatalErr = newErr
return
}
buffers = append(buffers, newBuffers...)
}
if dns.DomainStrategy(metadata.InboundOptions.DomainStrategy) != dns.DomainStrategyAsIS {
fatalErr = r.actionResolve(ctx, metadata, &rule.RuleActionResolve{
Strategy: dns.DomainStrategy(metadata.InboundOptions.DomainStrategy),
})
if fatalErr != nil {
return
}
}
if metadata.InboundOptions.UDPDisableDomainUnmapping {
metadata.UDPDisableDomainUnmapping = true
}
metadata.InboundOptions = option.InboundOptions{}
}
match:
for ruleIndex < len(r.rules) {
rules := r.rules
if ruleIndex != -1 {
rules = rules[ruleIndex+1:]
}
var (
currentRule adapter.Rule
currentRuleIndex int
matched bool
)
for currentRuleIndex, currentRule = range rules {
if currentRule.Match(metadata) {
matched = true
break
}
}
if !matched {
break
}
r.logger.DebugContext(ctx, "match[", currentRuleIndex, "] ", currentRule, " => ", currentRule.Action())
switch action := currentRule.Action().(type) {
case *rule.RuleActionSniff:
newBuffers, newErr := r.actionSniff(ctx, metadata, action, inputConn, inputPacketConn)
if newErr != nil {
fatalErr = newErr
return
}
buffers = append(buffers, newBuffers...)
case *rule.RuleActionResolve:
fatalErr = r.actionResolve(ctx, metadata, action)
if fatalErr != nil {
return
}
default:
selectedRule = currentRule
selectedRuleIndex = currentRuleIndex
break match
}
ruleIndex = currentRuleIndex
}
if metadata.Destination.Addr.IsUnspecified() {
newBuffers, newErr := r.actionSniff(ctx, metadata, &rule.RuleActionSniff{}, inputConn, inputPacketConn)
if newErr != nil {
fatalErr = newErr
return
}
buffers = append(buffers, newBuffers...)
}
return
}
func (r *Router) actionSniff(
ctx context.Context, metadata *adapter.InboundContext, action *rule.RuleActionSniff,
inputConn net.Conn, inputPacketConn N.PacketConn,
) (buffers []*buf.Buffer, fatalErr error) {
if sniff.Skip(metadata) {
return
} else if inputConn != nil && len(action.StreamSniffers) > 0 {
buffer := buf.NewPacket()
err := sniff.PeekStream(
ctx,
metadata,
inputConn,
buffer,
action.Timeout,
action.StreamSniffers...,
)
if err == nil {
//goland:noinspection GoDeprecation
if action.OverrideDestination && M.IsDomainName(metadata.Domain) {
metadata.Destination = M.Socksaddr{
Fqdn: metadata.Domain,
Port: metadata.Destination.Port,
}
}
if metadata.Domain != "" && metadata.Client != "" {
r.logger.DebugContext(ctx, "sniffed protocol: ", metadata.Protocol, ", domain: ", metadata.Domain, ", client: ", metadata.Client)
} else if metadata.Domain != "" {
r.logger.DebugContext(ctx, "sniffed protocol: ", metadata.Protocol, ", domain: ", metadata.Domain)
} else {
r.logger.DebugContext(ctx, "sniffed protocol: ", metadata.Protocol)
}
}
if !buffer.IsEmpty() {
buffers = append(buffers, buffer)
} else {
buffer.Release()
}
} else if inputPacketConn != nil && len(action.PacketSniffers) > 0 {
for {
var (
buffer = buf.NewPacket()
destination M.Socksaddr
done = make(chan struct{})
err error
)
go func() {
sniffTimeout := C.ReadPayloadTimeout
if action.Timeout > 0 {
sniffTimeout = action.Timeout
}
inputPacketConn.SetReadDeadline(time.Now().Add(sniffTimeout))
destination, err = inputPacketConn.ReadPacket(buffer)
inputPacketConn.SetReadDeadline(time.Time{})
close(done)
}()
select {
case <-done:
case <-ctx.Done():
inputPacketConn.Close()
fatalErr = ctx.Err()
return
}
if err != nil {
buffer.Release()
if !errors.Is(err, os.ErrDeadlineExceeded) {
fatalErr = err
return
}
} else {
// TODO: maybe always override destination
if metadata.Destination.Addr.IsUnspecified() {
metadata.Destination = destination
}
if len(buffers) > 0 {
err = sniff.PeekPacket(
ctx,
metadata,
buffer.Bytes(),
sniff.QUICClientHello,
)
} else {
err = sniff.PeekPacket(
ctx, metadata,
buffer.Bytes(),
action.PacketSniffers...,
)
}
buffers = append(buffers, buffer)
if E.IsMulti(err, sniff.ErrClientHelloFragmented) && len(buffers) == 0 {
r.logger.DebugContext(ctx, "attempt to sniff fragmented QUIC client hello")
continue
}
if metadata.Protocol != "" {
//goland:noinspection GoDeprecation
if action.OverrideDestination && M.IsDomainName(metadata.Domain) {
metadata.Destination = M.Socksaddr{
Fqdn: metadata.Domain,
Port: metadata.Destination.Port,
}
}
if metadata.Domain != "" && metadata.Client != "" {
r.logger.DebugContext(ctx, "sniffed packet protocol: ", metadata.Protocol, ", domain: ", metadata.Domain, ", client: ", metadata.Client)
} else if metadata.Domain != "" {
r.logger.DebugContext(ctx, "sniffed packet protocol: ", metadata.Protocol, ", domain: ", metadata.Domain)
} else if metadata.Client != "" {
r.logger.DebugContext(ctx, "sniffed packet protocol: ", metadata.Protocol, ", client: ", metadata.Client)
} else {
r.logger.DebugContext(ctx, "sniffed packet protocol: ", metadata.Protocol)
}
}
}
break
}
}
return
}
func (r *Router) actionResolve(ctx context.Context, metadata *adapter.InboundContext, action *rule.RuleActionResolve) error {
if metadata.Destination.IsFqdn() {
// TODO: check if WithContext is necessary
addresses, err := r.Lookup(adapter.WithContext(ctx, metadata), metadata.Destination.Fqdn, action.Strategy)
if err != nil {
return err
}
metadata.DestinationAddresses = addresses
r.dnsLogger.DebugContext(ctx, "resolved [", strings.Join(F.MapToString(metadata.DestinationAddresses), " "), "]")
if metadata.Destination.IsIPv4() {
metadata.IPVersion = 4
} else if metadata.Destination.IsIPv6() {
metadata.IPVersion = 6
}
}
return nil
}

View file

@ -8,6 +8,7 @@ import (
"time"
"github.com/sagernet/sing-box/adapter"
R "github.com/sagernet/sing-box/route/rule"
"github.com/sagernet/sing-dns"
"github.com/sagernet/sing/common/cache"
E "github.com/sagernet/sing/common/exceptions"
@ -36,15 +37,16 @@ func (m *DNSReverseMapping) Query(address netip.Addr) (string, bool) {
return domain, loaded
}
func (r *Router) matchDNS(ctx context.Context, allowFakeIP bool, index int, isAddressQuery bool) (context.Context, dns.Transport, dns.DomainStrategy, adapter.DNSRule, int) {
func (r *Router) matchDNS(ctx context.Context, allowFakeIP bool, ruleIndex int, isAddressQuery bool) (dns.Transport, dns.QueryOptions, adapter.DNSRule, int) {
metadata := adapter.ContextFrom(ctx)
if metadata == nil {
panic("no context")
}
if index < len(r.dnsRules) {
var options dns.QueryOptions
if ruleIndex < len(r.dnsRules) {
dnsRules := r.dnsRules
if index != -1 {
dnsRules = dnsRules[index+1:]
if ruleIndex != -1 {
dnsRules = dnsRules[ruleIndex+1:]
}
for currentRuleIndex, rule := range dnsRules {
if rule.WithAddressLimit() && !isAddressQuery {
@ -52,43 +54,42 @@ func (r *Router) matchDNS(ctx context.Context, allowFakeIP bool, index int, isAd
}
metadata.ResetRuleCache()
if rule.Match(metadata) {
detour := rule.Outbound()
transport, loaded := r.transportMap[detour]
if !loaded {
r.dnsLogger.ErrorContext(ctx, "transport not found: ", detour)
continue
displayRuleIndex := currentRuleIndex
if displayRuleIndex != -1 {
displayRuleIndex += displayRuleIndex + 1
}
_, isFakeIP := transport.(adapter.FakeIPTransport)
if isFakeIP && !allowFakeIP {
continue
}
ruleIndex := currentRuleIndex
if index != -1 {
ruleIndex += index + 1
}
r.dnsLogger.DebugContext(ctx, "match[", ruleIndex, "] ", rule.String(), " => ", detour)
if isFakeIP || rule.DisableCache() {
ctx = dns.ContextWithDisableCache(ctx, true)
}
if rewriteTTL := rule.RewriteTTL(); rewriteTTL != nil {
ctx = dns.ContextWithRewriteTTL(ctx, *rewriteTTL)
}
if clientSubnet := rule.ClientSubnet(); clientSubnet != nil {
ctx = dns.ContextWithClientSubnet(ctx, *clientSubnet)
}
if domainStrategy, dsLoaded := r.transportDomainStrategy[transport]; dsLoaded {
return ctx, transport, domainStrategy, rule, ruleIndex
if routeAction, isRoute := rule.Action().(*R.RuleActionDNSRoute); isRoute {
transport, loaded := r.transportMap[routeAction.Server]
if !loaded {
r.dnsLogger.ErrorContext(ctx, "transport not found: ", routeAction.Server)
continue
}
_, isFakeIP := transport.(adapter.FakeIPTransport)
if isFakeIP && !allowFakeIP {
continue
}
options.DisableCache = isFakeIP || routeAction.DisableCache
options.RewriteTTL = routeAction.RewriteTTL
options.ClientSubnet = routeAction.ClientSubnet
if domainStrategy, dsLoaded := r.transportDomainStrategy[transport]; dsLoaded {
options.Strategy = domainStrategy
} else {
options.Strategy = r.defaultDomainStrategy
}
r.dnsLogger.DebugContext(ctx, "match[", displayRuleIndex, "] ", rule.String(), " => ", rule.Action())
return transport, options, rule, currentRuleIndex
} else {
return ctx, transport, r.defaultDomainStrategy, rule, ruleIndex
return nil, options, rule, currentRuleIndex
}
}
}
}
if domainStrategy, dsLoaded := r.transportDomainStrategy[r.defaultTransport]; dsLoaded {
return ctx, r.defaultTransport, domainStrategy, nil, -1
options.Strategy = domainStrategy
} else {
return ctx, r.defaultTransport, r.defaultDomainStrategy, nil, -1
options.Strategy = r.defaultDomainStrategy
}
return r.defaultTransport, options, nil, -1
}
func (r *Router) Exchange(ctx context.Context, message *mDNS.Msg) (*mDNS.Msg, error) {
@ -117,21 +118,18 @@ func (r *Router) Exchange(ctx context.Context, message *mDNS.Msg) (*mDNS.Msg, er
metadata.Domain = fqdnToDomain(message.Question[0].Name)
}
var (
strategy dns.DomainStrategy
options dns.QueryOptions
rule adapter.DNSRule
ruleIndex int
)
ruleIndex = -1
for {
var (
dnsCtx context.Context
addressLimit bool
)
dnsCtx, transport, strategy, rule, ruleIndex = r.matchDNS(ctx, true, ruleIndex, isAddressQuery(message))
dnsCtx = adapter.OverrideContext(dnsCtx)
dnsCtx := adapter.OverrideContext(ctx)
var addressLimit bool
transport, options, rule, ruleIndex = r.matchDNS(ctx, true, ruleIndex, isAddressQuery(message))
if rule != nil && rule.WithAddressLimit() {
addressLimit = true
response, err = r.dnsClient.ExchangeWithResponseCheck(dnsCtx, transport, message, strategy, func(response *mDNS.Msg) bool {
response, err = r.dnsClient.ExchangeWithResponseCheck(dnsCtx, transport, message, options, func(response *mDNS.Msg) bool {
addresses, addrErr := dns.MessageToAddresses(response)
if addrErr != nil {
return false
@ -141,7 +139,7 @@ func (r *Router) Exchange(ctx context.Context, message *mDNS.Msg) (*mDNS.Msg, er
})
} else {
addressLimit = false
response, err = r.dnsClient.Exchange(dnsCtx, transport, message, strategy)
response, err = r.dnsClient.Exchange(dnsCtx, transport, message, options)
}
var rejected bool
if err != nil {
@ -199,31 +197,28 @@ func (r *Router) Lookup(ctx context.Context, domain string, strategy dns.DomainS
metadata.Destination = M.Socksaddr{}
metadata.Domain = domain
var (
transport dns.Transport
transportStrategy dns.DomainStrategy
rule adapter.DNSRule
ruleIndex int
transport dns.Transport
options dns.QueryOptions
rule adapter.DNSRule
ruleIndex int
)
ruleIndex = -1
for {
var (
dnsCtx context.Context
addressLimit bool
)
dnsCtx, transport, transportStrategy, rule, ruleIndex = r.matchDNS(ctx, false, ruleIndex, true)
dnsCtx = adapter.OverrideContext(dnsCtx)
if strategy == dns.DomainStrategyAsIS {
strategy = transportStrategy
dnsCtx := adapter.OverrideContext(ctx)
var addressLimit bool
transport, options, rule, ruleIndex = r.matchDNS(ctx, false, ruleIndex, true)
if strategy != dns.DomainStrategyAsIS {
options.Strategy = strategy
}
if rule != nil && rule.WithAddressLimit() {
addressLimit = true
responseAddrs, err = r.dnsClient.LookupWithResponseCheck(dnsCtx, transport, domain, strategy, func(responseAddrs []netip.Addr) bool {
responseAddrs, err = r.dnsClient.LookupWithResponseCheck(dnsCtx, transport, domain, options, func(responseAddrs []netip.Addr) bool {
metadata.DestinationAddresses = responseAddrs
return rule.MatchAddressLimit(metadata)
})
} else {
addressLimit = false
responseAddrs, err = r.dnsClient.Lookup(dnsCtx, transport, domain, strategy)
responseAddrs, err = r.dnsClient.Lookup(dnsCtx, transport, domain, options)
}
if err != nil {
if errors.Is(err, dns.ErrResponseRejectedCached) {

View file

@ -3,11 +3,9 @@ package route
import (
"context"
"errors"
"net"
"net/netip"
"net/url"
"os"
"os/user"
"runtime"
"strings"
"syscall"
@ -19,22 +17,16 @@ import (
"github.com/sagernet/sing-box/common/geoip"
"github.com/sagernet/sing-box/common/geosite"
"github.com/sagernet/sing-box/common/process"
"github.com/sagernet/sing-box/common/sniff"
"github.com/sagernet/sing-box/common/taskmonitor"
C "github.com/sagernet/sing-box/constant"
"github.com/sagernet/sing-box/experimental/libbox/platform"
"github.com/sagernet/sing-box/log"
"github.com/sagernet/sing-box/option"
"github.com/sagernet/sing-box/outbound"
R "github.com/sagernet/sing-box/route/rule"
"github.com/sagernet/sing-box/transport/fakeip"
"github.com/sagernet/sing-dns"
"github.com/sagernet/sing-mux"
"github.com/sagernet/sing-tun"
"github.com/sagernet/sing-vmess"
"github.com/sagernet/sing/common"
"github.com/sagernet/sing/common/buf"
"github.com/sagernet/sing/common/bufio"
"github.com/sagernet/sing/common/bufio/deadline"
"github.com/sagernet/sing/common/control"
E "github.com/sagernet/sing/common/exceptions"
F "github.com/sagernet/sing/common/format"
@ -42,7 +34,6 @@ import (
N "github.com/sagernet/sing/common/network"
"github.com/sagernet/sing/common/ntp"
"github.com/sagernet/sing/common/task"
"github.com/sagernet/sing/common/uot"
"github.com/sagernet/sing/common/winpowrprof"
"github.com/sagernet/sing/service"
"github.com/sagernet/sing/service/pause"
@ -154,14 +145,14 @@ func NewRouter(
Logger: router.dnsLogger,
})
for i, ruleOptions := range options.Rules {
routeRule, err := NewRule(ctx, router, router.logger, ruleOptions, true)
routeRule, err := R.NewRule(ctx, router, router.logger, ruleOptions, true)
if err != nil {
return nil, E.Cause(err, "parse rule[", i, "]")
}
router.rules = append(router.rules, routeRule)
}
for i, dnsRuleOptions := range dnsOptions.Rules {
dnsRule, err := NewDNSRule(ctx, router, router.logger, dnsRuleOptions, true)
dnsRule, err := R.NewDNSRule(ctx, router, router.logger, dnsRuleOptions, true)
if err != nil {
return nil, E.Cause(err, "parse dns rule[", i, "]")
}
@ -171,7 +162,7 @@ func NewRouter(
if _, exists := router.ruleSetMap[ruleSetOptions.Tag]; exists {
return nil, E.New("duplicate rule-set tag: ", ruleSetOptions.Tag)
}
ruleSet, err := NewRuleSet(ctx, router, router.logger, ruleSetOptions)
ruleSet, err := R.NewRuleSet(ctx, router, router.logger, ruleSetOptions)
if err != nil {
return nil, E.Cause(err, "parse rule-set[", i, "]")
}
@ -437,8 +428,12 @@ func (r *Router) Initialize(inbounds []adapter.Inbound, outbounds []adapter.Outb
r.defaultOutboundForPacketConnection = defaultOutboundForPacketConnection
r.outboundByTag = outboundByTag
for i, rule := range r.rules {
if _, loaded := outboundByTag[rule.Outbound()]; !loaded {
return E.New("outbound not found for rule[", i, "]: ", rule.Outbound())
routeAction, isRoute := rule.Action().(*R.RuleActionRoute)
if !isRoute {
continue
}
if _, loaded := outboundByTag[routeAction.Outbound]; !loaded {
return E.New("outbound not found for rule[", i, "]: ", routeAction.Outbound)
}
}
return nil
@ -804,375 +799,6 @@ func (r *Router) NeedWIFIState() bool {
return r.needWIFIState
}
func (r *Router) RouteConnection(ctx context.Context, conn net.Conn, metadata adapter.InboundContext) error {
if r.pauseManager.IsDevicePaused() {
return E.New("reject connection to ", metadata.Destination, " while device paused")
}
if metadata.InboundDetour != "" {
if metadata.LastInbound == metadata.InboundDetour {
return E.New("routing loop on detour: ", metadata.InboundDetour)
}
detour := r.inboundByTag[metadata.InboundDetour]
if detour == nil {
return E.New("inbound detour not found: ", metadata.InboundDetour)
}
injectable, isInjectable := detour.(adapter.InjectableInbound)
if !isInjectable {
return E.New("inbound detour is not injectable: ", metadata.InboundDetour)
}
if !common.Contains(injectable.Network(), N.NetworkTCP) {
return E.New("inject: TCP unsupported")
}
metadata.LastInbound = metadata.Inbound
metadata.Inbound = metadata.InboundDetour
metadata.InboundDetour = ""
err := injectable.NewConnection(ctx, conn, metadata)
if err != nil {
return E.Cause(err, "inject ", detour.Tag())
}
return nil
}
conntrack.KillerCheck()
metadata.Network = N.NetworkTCP
switch metadata.Destination.Fqdn {
case mux.Destination.Fqdn:
return E.New("global multiplex is deprecated since sing-box v1.7.0, enable multiplex in inbound options instead.")
case vmess.MuxDestination.Fqdn:
return E.New("global multiplex (v2ray legacy) not supported since sing-box v1.7.0.")
case uot.MagicAddress:
return E.New("global UoT not supported since sing-box v1.7.0.")
case uot.LegacyMagicAddress:
return E.New("global UoT (legacy) not supported since sing-box v1.7.0.")
}
if r.fakeIPStore != nil && r.fakeIPStore.Contains(metadata.Destination.Addr) {
domain, loaded := r.fakeIPStore.Lookup(metadata.Destination.Addr)
if !loaded {
return E.New("missing fakeip context")
}
metadata.OriginDestination = metadata.Destination
metadata.Destination = M.Socksaddr{
Fqdn: domain,
Port: metadata.Destination.Port,
}
metadata.FakeIP = true
r.logger.DebugContext(ctx, "found fakeip domain: ", domain)
}
if deadline.NeedAdditionalReadDeadline(conn) {
conn = deadline.NewConn(conn)
}
if metadata.InboundOptions.SniffEnabled && !sniff.Skip(metadata) {
buffer := buf.NewPacket()
err := sniff.PeekStream(
ctx,
&metadata,
conn,
buffer,
time.Duration(metadata.InboundOptions.SniffTimeout),
sniff.TLSClientHello,
sniff.HTTPHost,
sniff.StreamDomainNameQuery,
sniff.SSH,
sniff.BitTorrent,
)
if err == nil {
if metadata.InboundOptions.SniffOverrideDestination && M.IsDomainName(metadata.Domain) {
metadata.Destination = M.Socksaddr{
Fqdn: metadata.Domain,
Port: metadata.Destination.Port,
}
}
if metadata.Domain != "" {
r.logger.DebugContext(ctx, "sniffed protocol: ", metadata.Protocol, ", domain: ", metadata.Domain)
} else {
r.logger.DebugContext(ctx, "sniffed protocol: ", metadata.Protocol)
}
}
if !buffer.IsEmpty() {
conn = bufio.NewCachedConn(conn, buffer)
} else {
buffer.Release()
}
}
if r.dnsReverseMapping != nil && metadata.Domain == "" {
domain, loaded := r.dnsReverseMapping.Query(metadata.Destination.Addr)
if loaded {
metadata.Domain = domain
r.logger.DebugContext(ctx, "found reserve mapped domain: ", metadata.Domain)
}
}
if metadata.Destination.IsFqdn() && dns.DomainStrategy(metadata.InboundOptions.DomainStrategy) != dns.DomainStrategyAsIS {
addresses, err := r.Lookup(adapter.WithContext(ctx, &metadata), metadata.Destination.Fqdn, dns.DomainStrategy(metadata.InboundOptions.DomainStrategy))
if err != nil {
return err
}
metadata.DestinationAddresses = addresses
r.dnsLogger.DebugContext(ctx, "resolved [", strings.Join(F.MapToString(metadata.DestinationAddresses), " "), "]")
}
if metadata.Destination.IsIPv4() {
metadata.IPVersion = 4
} else if metadata.Destination.IsIPv6() {
metadata.IPVersion = 6
}
ctx, matchedRule, detour, err := r.match(ctx, &metadata, r.defaultOutboundForConnection)
if err != nil {
return err
}
if !common.Contains(detour.Network(), N.NetworkTCP) {
return E.New("missing supported outbound, closing connection")
}
if r.clashServer != nil {
trackerConn, tracker := r.clashServer.RoutedConnection(ctx, conn, metadata, matchedRule)
defer tracker.Leave()
conn = trackerConn
}
if r.v2rayServer != nil {
if statsService := r.v2rayServer.StatsService(); statsService != nil {
conn = statsService.RoutedConnection(metadata.Inbound, detour.Tag(), metadata.User, conn)
}
}
return detour.NewConnection(ctx, conn, metadata)
}
func (r *Router) RoutePacketConnection(ctx context.Context, conn N.PacketConn, metadata adapter.InboundContext) error {
if r.pauseManager.IsDevicePaused() {
return E.New("reject packet connection to ", metadata.Destination, " while device paused")
}
if metadata.InboundDetour != "" {
if metadata.LastInbound == metadata.InboundDetour {
return E.New("routing loop on detour: ", metadata.InboundDetour)
}
detour := r.inboundByTag[metadata.InboundDetour]
if detour == nil {
return E.New("inbound detour not found: ", metadata.InboundDetour)
}
injectable, isInjectable := detour.(adapter.InjectableInbound)
if !isInjectable {
return E.New("inbound detour is not injectable: ", metadata.InboundDetour)
}
if !common.Contains(injectable.Network(), N.NetworkUDP) {
return E.New("inject: UDP unsupported")
}
metadata.LastInbound = metadata.Inbound
metadata.Inbound = metadata.InboundDetour
metadata.InboundDetour = ""
err := injectable.NewPacketConnection(ctx, conn, metadata)
if err != nil {
return E.Cause(err, "inject ", detour.Tag())
}
return nil
}
conntrack.KillerCheck()
metadata.Network = N.NetworkUDP
if r.fakeIPStore != nil && r.fakeIPStore.Contains(metadata.Destination.Addr) {
domain, loaded := r.fakeIPStore.Lookup(metadata.Destination.Addr)
if !loaded {
return E.New("missing fakeip context")
}
metadata.OriginDestination = metadata.Destination
metadata.Destination = M.Socksaddr{
Fqdn: domain,
Port: metadata.Destination.Port,
}
metadata.FakeIP = true
r.logger.DebugContext(ctx, "found fakeip domain: ", domain)
}
// Currently we don't have deadline usages for UDP connections
/*if deadline.NeedAdditionalReadDeadline(conn) {
conn = deadline.NewPacketConn(bufio.NewNetPacketConn(conn))
}*/
if metadata.InboundOptions.SniffEnabled || metadata.Destination.Addr.IsUnspecified() {
var bufferList []*buf.Buffer
for {
var (
buffer = buf.NewPacket()
destination M.Socksaddr
done = make(chan struct{})
err error
)
go func() {
sniffTimeout := C.ReadPayloadTimeout
if metadata.InboundOptions.SniffTimeout > 0 {
sniffTimeout = time.Duration(metadata.InboundOptions.SniffTimeout)
}
conn.SetReadDeadline(time.Now().Add(sniffTimeout))
destination, err = conn.ReadPacket(buffer)
conn.SetReadDeadline(time.Time{})
close(done)
}()
select {
case <-done:
case <-ctx.Done():
conn.Close()
return ctx.Err()
}
if err != nil {
buffer.Release()
if !errors.Is(err, os.ErrDeadlineExceeded) {
return err
}
} else {
if metadata.Destination.Addr.IsUnspecified() {
metadata.Destination = destination
}
if metadata.InboundOptions.SniffEnabled {
if len(bufferList) > 0 {
err = sniff.PeekPacket(
ctx,
&metadata,
buffer.Bytes(),
sniff.QUICClientHello,
)
} else {
err = sniff.PeekPacket(
ctx, &metadata,
buffer.Bytes(),
sniff.DomainNameQuery,
sniff.QUICClientHello,
sniff.STUNMessage,
sniff.UTP,
sniff.UDPTracker,
sniff.DTLSRecord)
}
if E.IsMulti(err, sniff.ErrClientHelloFragmented) && len(bufferList) == 0 {
bufferList = append(bufferList, buffer)
r.logger.DebugContext(ctx, "attempt to sniff fragmented QUIC client hello")
continue
}
if metadata.Protocol != "" {
if metadata.InboundOptions.SniffOverrideDestination && M.IsDomainName(metadata.Domain) {
metadata.Destination = M.Socksaddr{
Fqdn: metadata.Domain,
Port: metadata.Destination.Port,
}
}
if metadata.Domain != "" && metadata.Client != "" {
r.logger.DebugContext(ctx, "sniffed packet protocol: ", metadata.Protocol, ", domain: ", metadata.Domain, ", client: ", metadata.Client)
} else if metadata.Domain != "" {
r.logger.DebugContext(ctx, "sniffed packet protocol: ", metadata.Protocol, ", domain: ", metadata.Domain)
} else if metadata.Client != "" {
r.logger.DebugContext(ctx, "sniffed packet protocol: ", metadata.Protocol, ", client: ", metadata.Client)
} else {
r.logger.DebugContext(ctx, "sniffed packet protocol: ", metadata.Protocol)
}
}
}
conn = bufio.NewCachedPacketConn(conn, buffer, destination)
}
for _, cachedBuffer := range common.Reverse(bufferList) {
conn = bufio.NewCachedPacketConn(conn, cachedBuffer, destination)
}
break
}
}
if r.dnsReverseMapping != nil && metadata.Domain == "" {
domain, loaded := r.dnsReverseMapping.Query(metadata.Destination.Addr)
if loaded {
metadata.Domain = domain
r.logger.DebugContext(ctx, "found reserve mapped domain: ", metadata.Domain)
}
}
if metadata.Destination.IsFqdn() && dns.DomainStrategy(metadata.InboundOptions.DomainStrategy) != dns.DomainStrategyAsIS {
addresses, err := r.Lookup(adapter.WithContext(ctx, &metadata), metadata.Destination.Fqdn, dns.DomainStrategy(metadata.InboundOptions.DomainStrategy))
if err != nil {
return err
}
metadata.DestinationAddresses = addresses
r.dnsLogger.DebugContext(ctx, "resolved [", strings.Join(F.MapToString(metadata.DestinationAddresses), " "), "]")
}
if metadata.Destination.IsIPv4() {
metadata.IPVersion = 4
} else if metadata.Destination.IsIPv6() {
metadata.IPVersion = 6
}
ctx, matchedRule, detour, err := r.match(ctx, &metadata, r.defaultOutboundForPacketConnection)
if err != nil {
return err
}
if !common.Contains(detour.Network(), N.NetworkUDP) {
return E.New("missing supported outbound, closing packet connection")
}
if r.clashServer != nil {
trackerConn, tracker := r.clashServer.RoutedPacketConnection(ctx, conn, metadata, matchedRule)
defer tracker.Leave()
conn = trackerConn
}
if r.v2rayServer != nil {
if statsService := r.v2rayServer.StatsService(); statsService != nil {
conn = statsService.RoutedPacketConnection(metadata.Inbound, detour.Tag(), metadata.User, conn)
}
}
if metadata.FakeIP {
conn = bufio.NewNATPacketConn(bufio.NewNetPacketConn(conn), metadata.OriginDestination, metadata.Destination)
}
return detour.NewPacketConnection(ctx, conn, metadata)
}
func (r *Router) match(ctx context.Context, metadata *adapter.InboundContext, defaultOutbound adapter.Outbound) (context.Context, adapter.Rule, adapter.Outbound, error) {
matchRule, matchOutbound := r.match0(ctx, metadata, defaultOutbound)
if contextOutbound, loaded := outbound.TagFromContext(ctx); loaded {
if contextOutbound == matchOutbound.Tag() {
return nil, nil, nil, E.New("connection loopback in outbound/", matchOutbound.Type(), "[", matchOutbound.Tag(), "]")
}
}
ctx = outbound.ContextWithTag(ctx, matchOutbound.Tag())
return ctx, matchRule, matchOutbound, nil
}
func (r *Router) match0(ctx context.Context, metadata *adapter.InboundContext, defaultOutbound adapter.Outbound) (adapter.Rule, adapter.Outbound) {
if r.processSearcher != nil {
var originDestination netip.AddrPort
if metadata.OriginDestination.IsValid() {
originDestination = metadata.OriginDestination.AddrPort()
} else if metadata.Destination.IsIP() {
originDestination = metadata.Destination.AddrPort()
}
processInfo, err := process.FindProcessInfo(r.processSearcher, ctx, metadata.Network, metadata.Source.AddrPort(), originDestination)
if err != nil {
r.logger.InfoContext(ctx, "failed to search process: ", err)
} else {
if processInfo.ProcessPath != "" {
r.logger.InfoContext(ctx, "found process path: ", processInfo.ProcessPath)
} else if processInfo.PackageName != "" {
r.logger.InfoContext(ctx, "found package name: ", processInfo.PackageName)
} else if processInfo.UserId != -1 {
if /*needUserName &&*/ true {
osUser, _ := user.LookupId(F.ToString(processInfo.UserId))
if osUser != nil {
processInfo.User = osUser.Username
}
}
if processInfo.User != "" {
r.logger.InfoContext(ctx, "found user: ", processInfo.User)
} else {
r.logger.InfoContext(ctx, "found user id: ", processInfo.UserId)
}
}
metadata.ProcessInfo = processInfo
}
}
for i, rule := range r.rules {
metadata.ResetRuleCache()
if rule.Match(metadata) {
detour := rule.Outbound()
r.logger.DebugContext(ctx, "match[", i, "] ", rule.String(), " => ", detour)
if outbound, loaded := r.Outbound(detour); loaded {
return rule, outbound
}
r.logger.ErrorContext(ctx, "outbound not found: ", detour)
}
}
return nil, defaultOutbound
}
func (r *Router) InterfaceFinder() control.InterfaceFinder {
return r.interfaceFinder
}

View file

@ -1,4 +1,4 @@
package route
package rule
import (
"io"
@ -20,7 +20,7 @@ type abstractDefaultRule struct {
allItems []RuleItem
ruleSetItem RuleItem
invert bool
outbound string
action adapter.RuleAction
}
func (r *abstractDefaultRule) Type() string {
@ -150,8 +150,8 @@ func (r *abstractDefaultRule) Match(metadata *adapter.InboundContext) bool {
return !r.invert
}
func (r *abstractDefaultRule) Outbound() string {
return r.outbound
func (r *abstractDefaultRule) Action() adapter.RuleAction {
return r.action
}
func (r *abstractDefaultRule) String() string {
@ -163,10 +163,10 @@ func (r *abstractDefaultRule) String() string {
}
type abstractLogicalRule struct {
rules []adapter.HeadlessRule
mode string
invert bool
outbound string
rules []adapter.HeadlessRule
mode string
invert bool
action adapter.RuleAction
}
func (r *abstractLogicalRule) Type() string {
@ -231,8 +231,8 @@ func (r *abstractLogicalRule) Match(metadata *adapter.InboundContext) bool {
}
}
func (r *abstractLogicalRule) Outbound() string {
return r.outbound
func (r *abstractLogicalRule) Action() adapter.RuleAction {
return r.action
}
func (r *abstractLogicalRule) String() string {

228
route/rule/rule_action.go Normal file
View file

@ -0,0 +1,228 @@
package rule
import (
"net/netip"
"strings"
"time"
"github.com/sagernet/sing-box/adapter"
"github.com/sagernet/sing-box/common/sniff"
C "github.com/sagernet/sing-box/constant"
"github.com/sagernet/sing-box/option"
"github.com/sagernet/sing-dns"
E "github.com/sagernet/sing/common/exceptions"
F "github.com/sagernet/sing/common/format"
)
func NewRuleAction(action option.RuleAction) (adapter.RuleAction, error) {
switch action.Action {
case C.RuleActionTypeRoute:
return &RuleActionRoute{
Outbound: action.RouteOptions.Outbound,
UDPDisableDomainUnmapping: action.RouteOptions.UDPDisableDomainUnmapping,
}, nil
case C.RuleActionTypeReturn:
return &RuleActionReject{}, nil
case C.RuleActionTypeReject:
return &RuleActionReject{
Method: string(action.RejectOptions.Method),
}, nil
case C.RuleActionTypeHijackDNS:
return &RuleActionHijackDNS{}, nil
case C.RuleActionTypeSniff:
sniffAction := &RuleActionSniff{
snifferNames: action.SniffOptions.Sniffer,
Timeout: time.Duration(action.SniffOptions.Timeout),
}
return sniffAction, sniffAction.build()
case C.RuleActionTypeResolve:
return &RuleActionResolve{
Strategy: dns.DomainStrategy(action.ResolveOptions.Strategy),
Server: action.ResolveOptions.Server,
}, nil
default:
panic(F.ToString("unknown rule action: ", action.Action))
}
}
func NewDNSRuleAction(action option.DNSRuleAction) adapter.RuleAction {
switch action.Action {
case C.RuleActionTypeRoute:
return &RuleActionDNSRoute{
Server: action.RouteOptions.Server,
DisableCache: action.RouteOptions.DisableCache,
RewriteTTL: action.RouteOptions.RewriteTTL,
ClientSubnet: action.RouteOptions.ClientSubnet.Build(),
}
case C.RuleActionTypeReturn:
return &RuleActionReturn{}
case C.RuleActionTypeReject:
return &RuleActionReject{
Method: string(action.RejectOptions.Method),
}
default:
panic(F.ToString("unknown rule action: ", action.Action))
}
}
type RuleActionRoute struct {
Outbound string
UDPDisableDomainUnmapping bool
}
func (r *RuleActionRoute) Type() string {
return C.RuleActionTypeRoute
}
func (r *RuleActionRoute) String() string {
return F.ToString("route(", r.Outbound, ")")
}
type RuleActionDNSRoute struct {
Server string
DisableCache bool
RewriteTTL *uint32
ClientSubnet netip.Prefix
}
func (r *RuleActionDNSRoute) Type() string {
return C.RuleActionTypeRoute
}
func (r *RuleActionDNSRoute) String() string {
return F.ToString("route(", r.Server, ")")
}
type RuleActionReturn struct{}
func (r *RuleActionReturn) Type() string {
return C.RuleActionTypeReturn
}
func (r *RuleActionReturn) String() string {
return "return"
}
type RuleActionReject struct {
Method string
}
func (r *RuleActionReject) Type() string {
return C.RuleActionTypeReject
}
func (r *RuleActionReject) String() string {
if r.Method == C.RuleActionRejectMethodDefault {
return "reject"
}
return F.ToString("reject(", r.Method, ")")
}
type RuleActionHijackDNS struct{}
func (r *RuleActionHijackDNS) Type() string {
return C.RuleActionTypeHijackDNS
}
func (r *RuleActionHijackDNS) String() string {
return "hijack-dns"
}
type RuleActionSniff struct {
snifferNames []string
StreamSniffers []sniff.StreamSniffer
PacketSniffers []sniff.PacketSniffer
Timeout time.Duration
// Deprecated
OverrideDestination bool
}
func (r *RuleActionSniff) Type() string {
return C.RuleActionTypeSniff
}
func (r *RuleActionSniff) build() error {
if len(r.StreamSniffers) > 0 || len(r.PacketSniffers) > 0 {
return nil
}
if len(r.snifferNames) > 0 {
for _, name := range r.snifferNames {
switch name {
case C.ProtocolTLS:
r.StreamSniffers = append(r.StreamSniffers, sniff.TLSClientHello)
case C.ProtocolHTTP:
r.StreamSniffers = append(r.StreamSniffers, sniff.HTTPHost)
case C.ProtocolQUIC:
r.PacketSniffers = append(r.PacketSniffers, sniff.QUICClientHello)
case C.ProtocolDNS:
r.StreamSniffers = append(r.StreamSniffers, sniff.StreamDomainNameQuery)
r.PacketSniffers = append(r.PacketSniffers, sniff.DomainNameQuery)
case C.ProtocolSTUN:
r.PacketSniffers = append(r.PacketSniffers, sniff.STUNMessage)
case C.ProtocolBitTorrent:
r.StreamSniffers = append(r.StreamSniffers, sniff.BitTorrent)
r.PacketSniffers = append(r.PacketSniffers, sniff.UTP)
r.PacketSniffers = append(r.PacketSniffers, sniff.UDPTracker)
case C.ProtocolDTLS:
r.PacketSniffers = append(r.PacketSniffers, sniff.DTLSRecord)
case C.ProtocolSSH:
r.StreamSniffers = append(r.StreamSniffers, sniff.SSH)
case C.ProtocolRDP:
r.StreamSniffers = append(r.StreamSniffers, sniff.RDP)
default:
return E.New("unknown sniffer: ", name)
}
}
} else {
r.StreamSniffers = []sniff.StreamSniffer{
sniff.TLSClientHello,
sniff.HTTPHost,
sniff.StreamDomainNameQuery,
sniff.BitTorrent,
sniff.SSH,
sniff.RDP,
}
r.PacketSniffers = []sniff.PacketSniffer{
sniff.DomainNameQuery,
sniff.QUICClientHello,
sniff.STUNMessage,
sniff.UTP,
sniff.UDPTracker,
sniff.DTLSRecord,
}
}
return nil
}
func (r *RuleActionSniff) String() string {
if len(r.snifferNames) == 0 && r.Timeout == 0 {
return "sniff"
} else if len(r.snifferNames) > 0 && r.Timeout == 0 {
return F.ToString("sniff(", strings.Join(r.snifferNames, ","), ")")
} else if len(r.snifferNames) == 0 && r.Timeout > 0 {
return F.ToString("sniff(", r.Timeout.String(), ")")
} else {
return F.ToString("sniff(", strings.Join(r.snifferNames, ","), ",", r.Timeout.String(), ")")
}
}
type RuleActionResolve struct {
Strategy dns.DomainStrategy
Server string
}
func (r *RuleActionResolve) Type() string {
return C.RuleActionTypeResolve
}
func (r *RuleActionResolve) String() string {
if r.Strategy == dns.DomainStrategyAsIS && r.Server == "" {
return F.ToString("resolve")
} else if r.Strategy != dns.DomainStrategyAsIS && r.Server == "" {
return F.ToString("resolve(", option.DomainStrategy(r.Strategy).String(), ")")
} else if r.Strategy == dns.DomainStrategyAsIS && r.Server != "" {
return F.ToString("resolve(", r.Server, ")")
} else {
return F.ToString("resolve(", option.DomainStrategy(r.Strategy).String(), ",", r.Server, ")")
}
}

View file

@ -1,4 +1,4 @@
package route
package rule
import (
"context"
@ -17,16 +17,22 @@ func NewRule(ctx context.Context, router adapter.Router, logger log.ContextLogge
if !options.DefaultOptions.IsValid() {
return nil, E.New("missing conditions")
}
if options.DefaultOptions.Outbound == "" && checkOutbound {
return nil, E.New("missing outbound field")
switch options.DefaultOptions.Action {
case "", C.RuleActionTypeRoute:
if options.DefaultOptions.RouteOptions.Outbound == "" && checkOutbound {
return nil, E.New("missing outbound field")
}
}
return NewDefaultRule(ctx, router, logger, options.DefaultOptions)
case C.RuleTypeLogical:
if !options.LogicalOptions.IsValid() {
return nil, E.New("missing conditions")
}
if options.LogicalOptions.Outbound == "" && checkOutbound {
return nil, E.New("missing outbound field")
switch options.LogicalOptions.Action {
case "", C.RuleActionTypeRoute:
if options.LogicalOptions.RouteOptions.Outbound == "" && checkOutbound {
return nil, E.New("missing outbound field")
}
}
return NewLogicalRule(ctx, router, logger, options.LogicalOptions)
default:
@ -46,10 +52,14 @@ type RuleItem interface {
}
func NewDefaultRule(ctx context.Context, router adapter.Router, logger log.ContextLogger, options option.DefaultRule) (*DefaultRule, error) {
action, err := NewRuleAction(options.RuleAction)
if err != nil {
return nil, E.Cause(err, "action")
}
rule := &DefaultRule{
abstractDefaultRule{
invert: options.Invert,
outbound: options.Outbound,
invert: options.Invert,
action: action,
},
}
if len(options.Inbound) > 0 {
@ -244,27 +254,31 @@ type LogicalRule struct {
}
func NewLogicalRule(ctx context.Context, router adapter.Router, logger log.ContextLogger, options option.LogicalRule) (*LogicalRule, error) {
r := &LogicalRule{
action, err := NewRuleAction(options.RuleAction)
if err != nil {
return nil, E.Cause(err, "action")
}
rule := &LogicalRule{
abstractLogicalRule{
rules: make([]adapter.HeadlessRule, len(options.Rules)),
invert: options.Invert,
outbound: options.Outbound,
rules: make([]adapter.HeadlessRule, len(options.Rules)),
invert: options.Invert,
action: action,
},
}
switch options.Mode {
case C.LogicalTypeAnd:
r.mode = C.LogicalTypeAnd
rule.mode = C.LogicalTypeAnd
case C.LogicalTypeOr:
r.mode = C.LogicalTypeOr
rule.mode = C.LogicalTypeOr
default:
return nil, E.New("unknown logical mode: ", options.Mode)
}
for i, subRule := range options.Rules {
rule, err := NewRule(ctx, router, logger, subRule, false)
for i, subOptions := range options.Rules {
subRule, err := NewRule(ctx, router, logger, subOptions, false)
if err != nil {
return nil, E.Cause(err, "sub rule[", i, "]")
}
r.rules[i] = rule
rule.rules[i] = subRule
}
return r, nil
return rule, nil
}

View file

@ -1,8 +1,7 @@
package route
package rule
import (
"context"
"net/netip"
"github.com/sagernet/sing-box/adapter"
C "github.com/sagernet/sing-box/constant"
@ -19,16 +18,22 @@ func NewDNSRule(ctx context.Context, router adapter.Router, logger log.ContextLo
if !options.DefaultOptions.IsValid() {
return nil, E.New("missing conditions")
}
if options.DefaultOptions.Server == "" && checkServer {
return nil, E.New("missing server field")
switch options.DefaultOptions.Action {
case "", C.RuleActionTypeRoute:
if options.DefaultOptions.RouteOptions.Server == "" && checkServer {
return nil, E.New("missing server field")
}
}
return NewDefaultDNSRule(ctx, router, logger, options.DefaultOptions)
case C.RuleTypeLogical:
if !options.LogicalOptions.IsValid() {
return nil, E.New("missing conditions")
}
if options.LogicalOptions.Server == "" && checkServer {
return nil, E.New("missing server field")
switch options.LogicalOptions.Action {
case "", C.RuleActionTypeRoute:
if options.LogicalOptions.RouteOptions.Server == "" && checkServer {
return nil, E.New("missing server field")
}
}
return NewLogicalDNSRule(ctx, router, logger, options.LogicalOptions)
default:
@ -40,20 +45,14 @@ var _ adapter.DNSRule = (*DefaultDNSRule)(nil)
type DefaultDNSRule struct {
abstractDefaultRule
disableCache bool
rewriteTTL *uint32
clientSubnet *netip.Prefix
}
func NewDefaultDNSRule(ctx context.Context, router adapter.Router, logger log.ContextLogger, options option.DefaultDNSRule) (*DefaultDNSRule, error) {
rule := &DefaultDNSRule{
abstractDefaultRule: abstractDefaultRule{
invert: options.Invert,
outbound: options.Server,
invert: options.Invert,
action: NewDNSRuleAction(options.DNSRuleAction),
},
disableCache: options.DisableCache,
rewriteTTL: options.RewriteTTL,
clientSubnet: (*netip.Prefix)(options.ClientSubnet),
}
if len(options.Inbound) > 0 {
item := NewInboundRule(options.Inbound)
@ -245,16 +244,8 @@ func NewDefaultDNSRule(ctx context.Context, router adapter.Router, logger log.Co
return rule, nil
}
func (r *DefaultDNSRule) DisableCache() bool {
return r.disableCache
}
func (r *DefaultDNSRule) RewriteTTL() *uint32 {
return r.rewriteTTL
}
func (r *DefaultDNSRule) ClientSubnet() *netip.Prefix {
return r.clientSubnet
func (r *DefaultDNSRule) Action() adapter.RuleAction {
return r.action
}
func (r *DefaultDNSRule) WithAddressLimit() bool {
@ -289,21 +280,15 @@ var _ adapter.DNSRule = (*LogicalDNSRule)(nil)
type LogicalDNSRule struct {
abstractLogicalRule
disableCache bool
rewriteTTL *uint32
clientSubnet *netip.Prefix
}
func NewLogicalDNSRule(ctx context.Context, router adapter.Router, logger log.ContextLogger, options option.LogicalDNSRule) (*LogicalDNSRule, error) {
r := &LogicalDNSRule{
abstractLogicalRule: abstractLogicalRule{
rules: make([]adapter.HeadlessRule, len(options.Rules)),
invert: options.Invert,
outbound: options.Server,
rules: make([]adapter.HeadlessRule, len(options.Rules)),
invert: options.Invert,
action: NewDNSRuleAction(options.DNSRuleAction),
},
disableCache: options.DisableCache,
rewriteTTL: options.RewriteTTL,
clientSubnet: (*netip.Prefix)(options.ClientSubnet),
}
switch options.Mode {
case C.LogicalTypeAnd:
@ -323,16 +308,8 @@ func NewLogicalDNSRule(ctx context.Context, router adapter.Router, logger log.Co
return r, nil
}
func (r *LogicalDNSRule) DisableCache() bool {
return r.disableCache
}
func (r *LogicalDNSRule) RewriteTTL() *uint32 {
return r.rewriteTTL
}
func (r *LogicalDNSRule) ClientSubnet() *netip.Prefix {
return r.clientSubnet
func (r *LogicalDNSRule) Action() adapter.RuleAction {
return r.action
}
func (r *LogicalDNSRule) WithAddressLimit() bool {

View file

@ -1,4 +1,4 @@
package route
package rule
import (
"github.com/sagernet/sing-box/adapter"

View file

@ -1,4 +1,4 @@
package route
package rule
import (
"strings"

View file

@ -1,4 +1,4 @@
package route
package rule
import (
"strings"

View file

@ -1,4 +1,4 @@
package route
package rule
import (
"net/netip"

View file

@ -1,4 +1,4 @@
package route
package rule
import (
"strings"

View file

@ -1,4 +1,4 @@
package route
package rule
import (
"strings"

View file

@ -1,4 +1,4 @@
package route
package rule
import (
"strings"

View file

@ -1,4 +1,4 @@
package route
package rule
import (
"strings"

View file

@ -1,4 +1,4 @@
package route
package rule
import (
"regexp"

View file

@ -1,4 +1,4 @@
package route
package rule
import (
"net/netip"

View file

@ -1,4 +1,4 @@
package route
package rule
import (
"strings"

View file

@ -1,4 +1,4 @@
package route
package rule
import (
"strings"

View file

@ -1,4 +1,4 @@
package route
package rule
import (
"net/netip"

View file

@ -1,4 +1,4 @@
package route
package rule
import (
"github.com/sagernet/sing-box/adapter"

View file

@ -1,4 +1,4 @@
package route
package rule
import (
"strings"

View file

@ -1,4 +1,4 @@
package route
package rule
import (
"strings"

View file

@ -1,4 +1,4 @@
package route
package rule
import (
"strings"

View file

@ -1,4 +1,4 @@
package route
package rule
import (
"strings"

View file

@ -1,4 +1,4 @@
package route
package rule
import (
"strconv"

View file

@ -1,4 +1,4 @@
package route
package rule
import (
"path/filepath"

View file

@ -1,4 +1,4 @@
package route
package rule
import (
"strings"

View file

@ -1,4 +1,4 @@
package route
package rule
import (
"regexp"

View file

@ -1,4 +1,4 @@
package route
package rule
import (
"strings"

View file

@ -1,4 +1,4 @@
package route
package rule
import (
"strings"

View file

@ -1,4 +1,4 @@
package route
package rule
import (
"strings"

View file

@ -1,4 +1,4 @@
package route
package rule
import (
"strings"

View file

@ -1,4 +1,4 @@
package route
package rule
import (
"strings"

View file

@ -1,4 +1,4 @@
package route
package rule
import (
"strings"

View file

@ -1,4 +1,4 @@
package route
package rule
import (
"strings"

View file

@ -1,4 +1,4 @@
package route
package rule
import (
"context"
@ -41,3 +41,31 @@ func extractIPSetFromRule(rawRule adapter.HeadlessRule) []*netipx.IPSet {
panic("unexpected rule type")
}
}
func hasHeadlessRule(rules []option.HeadlessRule, cond func(rule option.DefaultHeadlessRule) bool) bool {
for _, rule := range rules {
switch rule.Type {
case C.RuleTypeDefault:
if cond(rule.DefaultOptions) {
return true
}
case C.RuleTypeLogical:
if hasHeadlessRule(rule.LogicalOptions.Rules, cond) {
return true
}
}
}
return false
}
func isProcessHeadlessRule(rule option.DefaultHeadlessRule) bool {
return len(rule.ProcessName) > 0 || len(rule.ProcessPath) > 0 || len(rule.ProcessPathRegex) > 0 || len(rule.PackageName) > 0
}
func isWIFIHeadlessRule(rule option.DefaultHeadlessRule) bool {
return len(rule.WIFISSID) > 0 || len(rule.WIFIBSSID) > 0
}
func isIPCIDRHeadlessRule(rule option.DefaultHeadlessRule) bool {
return len(rule.IPCIDR) > 0 || rule.IPSet != nil
}

View file

@ -1,4 +1,4 @@
package route
package rule
import (
"context"

View file

@ -1,4 +1,4 @@
package route
package rule
import (
"bytes"

View file

@ -38,22 +38,6 @@ func hasDNSRule(rules []option.DNSRule, cond func(rule option.DefaultDNSRule) bo
return false
}
func hasHeadlessRule(rules []option.HeadlessRule, cond func(rule option.DefaultHeadlessRule) bool) bool {
for _, rule := range rules {
switch rule.Type {
case C.RuleTypeDefault:
if cond(rule.DefaultOptions) {
return true
}
case C.RuleTypeLogical:
if hasHeadlessRule(rule.LogicalOptions.Rules, cond) {
return true
}
}
}
return false
}
func isGeoIPRule(rule option.DefaultRule) bool {
return len(rule.SourceGeoIP) > 0 && common.Any(rule.SourceGeoIP, notPrivateNode) || len(rule.GeoIP) > 0 && common.Any(rule.GeoIP, notPrivateNode)
}
@ -78,10 +62,6 @@ func isProcessDNSRule(rule option.DefaultDNSRule) bool {
return len(rule.ProcessName) > 0 || len(rule.ProcessPath) > 0 || len(rule.ProcessPathRegex) > 0 || len(rule.PackageName) > 0 || len(rule.User) > 0 || len(rule.UserID) > 0
}
func isProcessHeadlessRule(rule option.DefaultHeadlessRule) bool {
return len(rule.ProcessName) > 0 || len(rule.ProcessPath) > 0 || len(rule.ProcessPathRegex) > 0 || len(rule.PackageName) > 0
}
func notPrivateNode(code string) bool {
return code != "private"
}
@ -93,11 +73,3 @@ func isWIFIRule(rule option.DefaultRule) bool {
func isWIFIDNSRule(rule option.DefaultDNSRule) bool {
return len(rule.WIFISSID) > 0 || len(rule.WIFIBSSID) > 0
}
func isWIFIHeadlessRule(rule option.DefaultHeadlessRule) bool {
return len(rule.WIFISSID) > 0 || len(rule.WIFIBSSID) > 0
}
func isIPCIDRHeadlessRule(rule option.DefaultHeadlessRule) bool {
return len(rule.IPCIDR) > 0 || rule.IPSet != nil
}