mirror of
https://github.com/SagerNet/sing-box.git
synced 2025-04-04 20:37:37 +03:00
refactor: Modular inbounds/outbounds
This commit is contained in:
parent
9f7683818f
commit
e233fd4fe5
152 changed files with 3116 additions and 2926 deletions
42
protocol/block/outbound.go
Normal file
42
protocol/block/outbound.go
Normal file
|
@ -0,0 +1,42 @@
|
|||
package block
|
||||
|
||||
import (
|
||||
"context"
|
||||
"net"
|
||||
"syscall"
|
||||
|
||||
"github.com/sagernet/sing-box/adapter"
|
||||
"github.com/sagernet/sing-box/adapter/outbound"
|
||||
C "github.com/sagernet/sing-box/constant"
|
||||
"github.com/sagernet/sing-box/log"
|
||||
"github.com/sagernet/sing-box/option"
|
||||
"github.com/sagernet/sing/common/logger"
|
||||
M "github.com/sagernet/sing/common/metadata"
|
||||
N "github.com/sagernet/sing/common/network"
|
||||
)
|
||||
|
||||
func RegisterOutbound(registry *outbound.Registry) {
|
||||
outbound.Register[option.StubOptions](registry, C.TypeBlock, New)
|
||||
}
|
||||
|
||||
type Outbound struct {
|
||||
outbound.Adapter
|
||||
logger logger.ContextLogger
|
||||
}
|
||||
|
||||
func New(ctx context.Context, router adapter.Router, logger log.ContextLogger, tag string, _ option.StubOptions) (adapter.Outbound, error) {
|
||||
return &Outbound{
|
||||
Adapter: outbound.NewAdapter(C.TypeBlock, []string{N.NetworkTCP, N.NetworkUDP}, tag, nil),
|
||||
logger: logger,
|
||||
}, nil
|
||||
}
|
||||
|
||||
func (h *Outbound) DialContext(ctx context.Context, network string, destination M.Socksaddr) (net.Conn, error) {
|
||||
h.logger.InfoContext(ctx, "blocked connection to ", destination)
|
||||
return nil, syscall.EPERM
|
||||
}
|
||||
|
||||
func (h *Outbound) ListenPacket(ctx context.Context, destination M.Socksaddr) (net.PacketConn, error) {
|
||||
h.logger.InfoContext(ctx, "blocked packet connection to ", destination)
|
||||
return nil, syscall.EPERM
|
||||
}
|
139
protocol/direct/inbound.go
Normal file
139
protocol/direct/inbound.go
Normal file
|
@ -0,0 +1,139 @@
|
|||
package direct
|
||||
|
||||
import (
|
||||
"context"
|
||||
"net"
|
||||
"time"
|
||||
|
||||
"github.com/sagernet/sing-box/adapter"
|
||||
"github.com/sagernet/sing-box/adapter/inbound"
|
||||
"github.com/sagernet/sing-box/common/listener"
|
||||
C "github.com/sagernet/sing-box/constant"
|
||||
"github.com/sagernet/sing-box/log"
|
||||
"github.com/sagernet/sing-box/option"
|
||||
"github.com/sagernet/sing/common/buf"
|
||||
M "github.com/sagernet/sing/common/metadata"
|
||||
N "github.com/sagernet/sing/common/network"
|
||||
"github.com/sagernet/sing/common/udpnat2"
|
||||
)
|
||||
|
||||
func RegisterInbound(registry *inbound.Registry) {
|
||||
inbound.Register[option.DirectInboundOptions](registry, C.TypeDirect, NewInbound)
|
||||
}
|
||||
|
||||
type Inbound struct {
|
||||
inbound.Adapter
|
||||
ctx context.Context
|
||||
router adapter.ConnectionRouterEx
|
||||
logger log.ContextLogger
|
||||
listener *listener.Listener
|
||||
udpNat *udpnat.Service
|
||||
overrideOption int
|
||||
overrideDestination M.Socksaddr
|
||||
}
|
||||
|
||||
func NewInbound(ctx context.Context, router adapter.Router, logger log.ContextLogger, tag string, options option.DirectInboundOptions) (adapter.Inbound, error) {
|
||||
options.UDPFragmentDefault = true
|
||||
inbound := &Inbound{
|
||||
Adapter: inbound.NewAdapter(C.TypeDirect, tag),
|
||||
ctx: ctx,
|
||||
router: router,
|
||||
logger: logger,
|
||||
}
|
||||
if options.OverrideAddress != "" && options.OverridePort != 0 {
|
||||
inbound.overrideOption = 1
|
||||
inbound.overrideDestination = M.ParseSocksaddrHostPort(options.OverrideAddress, options.OverridePort)
|
||||
} else if options.OverrideAddress != "" {
|
||||
inbound.overrideOption = 2
|
||||
inbound.overrideDestination = M.ParseSocksaddrHostPort(options.OverrideAddress, options.OverridePort)
|
||||
} else if options.OverridePort != 0 {
|
||||
inbound.overrideOption = 3
|
||||
inbound.overrideDestination = M.Socksaddr{Port: options.OverridePort}
|
||||
}
|
||||
var udpTimeout time.Duration
|
||||
if options.UDPTimeout != 0 {
|
||||
udpTimeout = time.Duration(options.UDPTimeout)
|
||||
} else {
|
||||
udpTimeout = C.UDPTimeout
|
||||
}
|
||||
inbound.udpNat = udpnat.New(inbound, inbound.preparePacketConnection, udpTimeout, false)
|
||||
inbound.listener = listener.New(listener.Options{
|
||||
Context: ctx,
|
||||
Logger: logger,
|
||||
Network: options.Network.Build(),
|
||||
Listen: options.ListenOptions,
|
||||
ConnectionHandler: inbound,
|
||||
PacketHandler: inbound,
|
||||
})
|
||||
return inbound, nil
|
||||
}
|
||||
|
||||
func (i *Inbound) Start() error {
|
||||
return i.listener.Start()
|
||||
}
|
||||
|
||||
func (i *Inbound) Close() error {
|
||||
return i.listener.Close()
|
||||
}
|
||||
|
||||
func (i *Inbound) NewConnection(ctx context.Context, conn net.Conn, metadata adapter.InboundContext) error {
|
||||
switch i.overrideOption {
|
||||
case 1:
|
||||
metadata.Destination = i.overrideDestination
|
||||
case 2:
|
||||
destination := i.overrideDestination
|
||||
destination.Port = metadata.Destination.Port
|
||||
metadata.Destination = destination
|
||||
case 3:
|
||||
metadata.Destination.Port = i.overrideDestination.Port
|
||||
}
|
||||
i.logger.InfoContext(ctx, "inbound connection to ", metadata.Destination)
|
||||
return i.router.RouteConnection(ctx, conn, metadata)
|
||||
}
|
||||
|
||||
func (i *Inbound) NewPacketEx(buffer *buf.Buffer, source M.Socksaddr) {
|
||||
var destination M.Socksaddr
|
||||
switch i.overrideOption {
|
||||
case 1:
|
||||
destination = i.overrideDestination
|
||||
case 2:
|
||||
destination = i.overrideDestination
|
||||
destination.Port = source.Port
|
||||
case 3:
|
||||
destination = source
|
||||
destination.Port = i.overrideDestination.Port
|
||||
}
|
||||
i.udpNat.NewPacket([][]byte{buffer.Bytes()}, source, destination, nil)
|
||||
}
|
||||
|
||||
func (i *Inbound) NewConnectionEx(ctx context.Context, conn net.Conn, metadata adapter.InboundContext, onClose N.CloseHandlerFunc) {
|
||||
i.logger.InfoContext(ctx, "inbound connection to ", metadata.Destination)
|
||||
metadata.Inbound = i.Tag()
|
||||
metadata.InboundType = i.Type()
|
||||
i.router.RouteConnectionEx(ctx, conn, metadata, onClose)
|
||||
}
|
||||
|
||||
func (i *Inbound) NewPacketConnectionEx(ctx context.Context, conn N.PacketConn, source M.Socksaddr, destination M.Socksaddr, onClose N.CloseHandlerFunc) {
|
||||
i.logger.InfoContext(ctx, "inbound packet connection from ", source)
|
||||
i.logger.InfoContext(ctx, "inbound packet connection to ", destination)
|
||||
var metadata adapter.InboundContext
|
||||
metadata.Inbound = i.Tag()
|
||||
metadata.InboundType = i.Type()
|
||||
metadata.Source = source
|
||||
metadata.Destination = destination
|
||||
metadata.OriginDestination = i.listener.UDPAddr()
|
||||
i.router.RoutePacketConnectionEx(ctx, conn, metadata, onClose)
|
||||
}
|
||||
|
||||
func (i *Inbound) preparePacketConnection(source M.Socksaddr, destination M.Socksaddr, userData any) (bool, context.Context, N.PacketWriter, N.CloseHandlerFunc) {
|
||||
return true, log.ContextWithNewID(i.ctx), &directPacketWriter{i.listener.PacketWriter(), source}, nil
|
||||
}
|
||||
|
||||
type directPacketWriter struct {
|
||||
writer N.PacketWriter
|
||||
source M.Socksaddr
|
||||
}
|
||||
|
||||
func (w *directPacketWriter) WritePacket(buffer *buf.Buffer, addr M.Socksaddr) error {
|
||||
return w.writer.WritePacket(buffer, w.source)
|
||||
}
|
186
protocol/direct/loopback_detect.go
Normal file
186
protocol/direct/loopback_detect.go
Normal file
|
@ -0,0 +1,186 @@
|
|||
package direct
|
||||
|
||||
import (
|
||||
"net"
|
||||
"net/netip"
|
||||
"sync"
|
||||
|
||||
"github.com/sagernet/sing-box/adapter"
|
||||
M "github.com/sagernet/sing/common/metadata"
|
||||
N "github.com/sagernet/sing/common/network"
|
||||
)
|
||||
|
||||
type loopBackDetector struct {
|
||||
router adapter.Router
|
||||
connAccess sync.RWMutex
|
||||
packetConnAccess sync.RWMutex
|
||||
connMap map[netip.AddrPort]netip.AddrPort
|
||||
packetConnMap map[uint16]uint16
|
||||
}
|
||||
|
||||
func newLoopBackDetector(router adapter.Router) *loopBackDetector {
|
||||
return &loopBackDetector{
|
||||
router: router,
|
||||
connMap: make(map[netip.AddrPort]netip.AddrPort),
|
||||
packetConnMap: make(map[uint16]uint16),
|
||||
}
|
||||
}
|
||||
|
||||
func (l *loopBackDetector) NewConn(conn net.Conn) net.Conn {
|
||||
source := M.AddrPortFromNet(conn.LocalAddr())
|
||||
if !source.IsValid() {
|
||||
return conn
|
||||
}
|
||||
if udpConn, isUDPConn := conn.(abstractUDPConn); isUDPConn {
|
||||
if !source.Addr().IsLoopback() {
|
||||
_, err := l.router.InterfaceFinder().InterfaceByAddr(source.Addr())
|
||||
if err != nil {
|
||||
return conn
|
||||
}
|
||||
}
|
||||
if !N.IsPublicAddr(source.Addr()) {
|
||||
return conn
|
||||
}
|
||||
l.packetConnAccess.Lock()
|
||||
l.packetConnMap[source.Port()] = M.AddrPortFromNet(conn.RemoteAddr()).Port()
|
||||
l.packetConnAccess.Unlock()
|
||||
return &loopBackDetectUDPWrapper{abstractUDPConn: udpConn, detector: l, connPort: source.Port()}
|
||||
} else {
|
||||
l.connAccess.Lock()
|
||||
l.connMap[source] = M.AddrPortFromNet(conn.RemoteAddr())
|
||||
l.connAccess.Unlock()
|
||||
return &loopBackDetectWrapper{Conn: conn, detector: l, connAddr: source}
|
||||
}
|
||||
}
|
||||
|
||||
func (l *loopBackDetector) NewPacketConn(conn N.NetPacketConn, destination M.Socksaddr) N.NetPacketConn {
|
||||
source := M.AddrPortFromNet(conn.LocalAddr())
|
||||
if !source.IsValid() {
|
||||
return conn
|
||||
}
|
||||
if !source.Addr().IsLoopback() {
|
||||
_, err := l.router.InterfaceFinder().InterfaceByAddr(source.Addr())
|
||||
if err != nil {
|
||||
return conn
|
||||
}
|
||||
}
|
||||
l.packetConnAccess.Lock()
|
||||
l.packetConnMap[source.Port()] = destination.AddrPort().Port()
|
||||
l.packetConnAccess.Unlock()
|
||||
return &loopBackDetectPacketWrapper{NetPacketConn: conn, detector: l, connPort: source.Port()}
|
||||
}
|
||||
|
||||
func (l *loopBackDetector) CheckConn(source netip.AddrPort, local netip.AddrPort) bool {
|
||||
l.connAccess.RLock()
|
||||
defer l.connAccess.RUnlock()
|
||||
destination, loaded := l.connMap[source]
|
||||
return loaded && destination != local
|
||||
}
|
||||
|
||||
func (l *loopBackDetector) CheckPacketConn(source netip.AddrPort, local netip.AddrPort) bool {
|
||||
if !source.IsValid() {
|
||||
return false
|
||||
}
|
||||
if !source.Addr().IsLoopback() {
|
||||
_, err := l.router.InterfaceFinder().InterfaceByAddr(source.Addr())
|
||||
if err != nil {
|
||||
return false
|
||||
}
|
||||
}
|
||||
if N.IsPublicAddr(source.Addr()) {
|
||||
return false
|
||||
}
|
||||
l.packetConnAccess.RLock()
|
||||
defer l.packetConnAccess.RUnlock()
|
||||
destinationPort, loaded := l.packetConnMap[source.Port()]
|
||||
return loaded && destinationPort != local.Port()
|
||||
}
|
||||
|
||||
type loopBackDetectWrapper struct {
|
||||
net.Conn
|
||||
detector *loopBackDetector
|
||||
connAddr netip.AddrPort
|
||||
closeOnce sync.Once
|
||||
}
|
||||
|
||||
func (w *loopBackDetectWrapper) Close() error {
|
||||
w.closeOnce.Do(func() {
|
||||
w.detector.connAccess.Lock()
|
||||
delete(w.detector.connMap, w.connAddr)
|
||||
w.detector.connAccess.Unlock()
|
||||
})
|
||||
return w.Conn.Close()
|
||||
}
|
||||
|
||||
func (w *loopBackDetectWrapper) ReaderReplaceable() bool {
|
||||
return true
|
||||
}
|
||||
|
||||
func (w *loopBackDetectWrapper) WriterReplaceable() bool {
|
||||
return true
|
||||
}
|
||||
|
||||
func (w *loopBackDetectWrapper) Upstream() any {
|
||||
return w.Conn
|
||||
}
|
||||
|
||||
type loopBackDetectPacketWrapper struct {
|
||||
N.NetPacketConn
|
||||
detector *loopBackDetector
|
||||
connPort uint16
|
||||
closeOnce sync.Once
|
||||
}
|
||||
|
||||
func (w *loopBackDetectPacketWrapper) Close() error {
|
||||
w.closeOnce.Do(func() {
|
||||
w.detector.packetConnAccess.Lock()
|
||||
delete(w.detector.packetConnMap, w.connPort)
|
||||
w.detector.packetConnAccess.Unlock()
|
||||
})
|
||||
return w.NetPacketConn.Close()
|
||||
}
|
||||
|
||||
func (w *loopBackDetectPacketWrapper) ReaderReplaceable() bool {
|
||||
return true
|
||||
}
|
||||
|
||||
func (w *loopBackDetectPacketWrapper) WriterReplaceable() bool {
|
||||
return true
|
||||
}
|
||||
|
||||
func (w *loopBackDetectPacketWrapper) Upstream() any {
|
||||
return w.NetPacketConn
|
||||
}
|
||||
|
||||
type abstractUDPConn interface {
|
||||
net.Conn
|
||||
net.PacketConn
|
||||
}
|
||||
|
||||
type loopBackDetectUDPWrapper struct {
|
||||
abstractUDPConn
|
||||
detector *loopBackDetector
|
||||
connPort uint16
|
||||
closeOnce sync.Once
|
||||
}
|
||||
|
||||
func (w *loopBackDetectUDPWrapper) Close() error {
|
||||
w.closeOnce.Do(func() {
|
||||
w.detector.packetConnAccess.Lock()
|
||||
delete(w.detector.packetConnMap, w.connPort)
|
||||
w.detector.packetConnAccess.Unlock()
|
||||
})
|
||||
return w.abstractUDPConn.Close()
|
||||
}
|
||||
|
||||
func (w *loopBackDetectUDPWrapper) ReaderReplaceable() bool {
|
||||
return true
|
||||
}
|
||||
|
||||
func (w *loopBackDetectUDPWrapper) WriterReplaceable() bool {
|
||||
return true
|
||||
}
|
||||
|
||||
func (w *loopBackDetectUDPWrapper) Upstream() any {
|
||||
return w.abstractUDPConn
|
||||
}
|
170
protocol/direct/outbound.go
Normal file
170
protocol/direct/outbound.go
Normal file
|
@ -0,0 +1,170 @@
|
|||
package direct
|
||||
|
||||
import (
|
||||
"context"
|
||||
"net"
|
||||
"net/netip"
|
||||
"time"
|
||||
|
||||
"github.com/sagernet/sing-box/adapter"
|
||||
"github.com/sagernet/sing-box/adapter/outbound"
|
||||
"github.com/sagernet/sing-box/common/dialer"
|
||||
C "github.com/sagernet/sing-box/constant"
|
||||
"github.com/sagernet/sing-box/log"
|
||||
"github.com/sagernet/sing-box/option"
|
||||
"github.com/sagernet/sing-dns"
|
||||
"github.com/sagernet/sing/common/bufio"
|
||||
E "github.com/sagernet/sing/common/exceptions"
|
||||
"github.com/sagernet/sing/common/logger"
|
||||
M "github.com/sagernet/sing/common/metadata"
|
||||
N "github.com/sagernet/sing/common/network"
|
||||
)
|
||||
|
||||
func RegisterOutbound(registry *outbound.Registry) {
|
||||
outbound.Register[option.DirectOutboundOptions](registry, C.TypeDirect, NewOutbound)
|
||||
}
|
||||
|
||||
var _ N.ParallelDialer = (*Outbound)(nil)
|
||||
|
||||
type Outbound struct {
|
||||
outbound.Adapter
|
||||
logger logger.ContextLogger
|
||||
dialer N.Dialer
|
||||
domainStrategy dns.DomainStrategy
|
||||
fallbackDelay time.Duration
|
||||
overrideOption int
|
||||
overrideDestination M.Socksaddr
|
||||
// loopBack *loopBackDetector
|
||||
}
|
||||
|
||||
func NewOutbound(ctx context.Context, router adapter.Router, logger log.ContextLogger, tag string, options option.DirectOutboundOptions) (adapter.Outbound, error) {
|
||||
options.UDPFragmentDefault = true
|
||||
outboundDialer, err := dialer.New(router, options.DialerOptions)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
outbound := &Outbound{
|
||||
Adapter: outbound.NewAdapterWithDialerOptions(C.TypeDirect, []string{N.NetworkTCP, N.NetworkUDP}, tag, options.DialerOptions),
|
||||
logger: logger,
|
||||
domainStrategy: dns.DomainStrategy(options.DomainStrategy),
|
||||
fallbackDelay: time.Duration(options.FallbackDelay),
|
||||
dialer: outboundDialer,
|
||||
// loopBack: newLoopBackDetector(router),
|
||||
}
|
||||
if options.ProxyProtocol != 0 {
|
||||
return nil, E.New("Proxy Protocol is deprecated and removed in sing-box 1.6.0")
|
||||
}
|
||||
if options.OverrideAddress != "" && options.OverridePort != 0 {
|
||||
outbound.overrideOption = 1
|
||||
outbound.overrideDestination = M.ParseSocksaddrHostPort(options.OverrideAddress, options.OverridePort)
|
||||
} else if options.OverrideAddress != "" {
|
||||
outbound.overrideOption = 2
|
||||
outbound.overrideDestination = M.ParseSocksaddrHostPort(options.OverrideAddress, options.OverridePort)
|
||||
} else if options.OverridePort != 0 {
|
||||
outbound.overrideOption = 3
|
||||
outbound.overrideDestination = M.Socksaddr{Port: options.OverridePort}
|
||||
}
|
||||
return outbound, nil
|
||||
}
|
||||
|
||||
func (h *Outbound) DialContext(ctx context.Context, network string, destination M.Socksaddr) (net.Conn, error) {
|
||||
ctx, metadata := adapter.ExtendContext(ctx)
|
||||
metadata.Outbound = h.Tag()
|
||||
metadata.Destination = destination
|
||||
switch h.overrideOption {
|
||||
case 1:
|
||||
destination = h.overrideDestination
|
||||
case 2:
|
||||
newDestination := h.overrideDestination
|
||||
newDestination.Port = destination.Port
|
||||
destination = newDestination
|
||||
case 3:
|
||||
destination.Port = h.overrideDestination.Port
|
||||
}
|
||||
network = N.NetworkName(network)
|
||||
switch network {
|
||||
case N.NetworkTCP:
|
||||
h.logger.InfoContext(ctx, "outbound connection to ", destination)
|
||||
case N.NetworkUDP:
|
||||
h.logger.InfoContext(ctx, "outbound packet connection to ", destination)
|
||||
}
|
||||
/*conn, err := h.dialer.DialContext(ctx, network, destination)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return h.loopBack.NewConn(conn), nil*/
|
||||
return h.dialer.DialContext(ctx, network, destination)
|
||||
}
|
||||
|
||||
func (h *Outbound) DialParallel(ctx context.Context, network string, destination M.Socksaddr, destinationAddresses []netip.Addr) (net.Conn, error) {
|
||||
ctx, metadata := adapter.ExtendContext(ctx)
|
||||
metadata.Outbound = h.Tag()
|
||||
metadata.Destination = destination
|
||||
switch h.overrideOption {
|
||||
case 1, 2:
|
||||
// override address
|
||||
return h.DialContext(ctx, network, destination)
|
||||
case 3:
|
||||
destination.Port = h.overrideDestination.Port
|
||||
}
|
||||
network = N.NetworkName(network)
|
||||
switch network {
|
||||
case N.NetworkTCP:
|
||||
h.logger.InfoContext(ctx, "outbound connection to ", destination)
|
||||
case N.NetworkUDP:
|
||||
h.logger.InfoContext(ctx, "outbound packet connection to ", destination)
|
||||
}
|
||||
var domainStrategy dns.DomainStrategy
|
||||
if h.domainStrategy != dns.DomainStrategyAsIS {
|
||||
domainStrategy = h.domainStrategy
|
||||
} else {
|
||||
domainStrategy = dns.DomainStrategy(metadata.InboundOptions.DomainStrategy)
|
||||
}
|
||||
return N.DialParallel(ctx, h.dialer, network, destination, destinationAddresses, domainStrategy == dns.DomainStrategyPreferIPv6, h.fallbackDelay)
|
||||
}
|
||||
|
||||
func (h *Outbound) ListenPacket(ctx context.Context, destination M.Socksaddr) (net.PacketConn, error) {
|
||||
ctx, metadata := adapter.ExtendContext(ctx)
|
||||
metadata.Outbound = h.Tag()
|
||||
metadata.Destination = destination
|
||||
originDestination := destination
|
||||
switch h.overrideOption {
|
||||
case 1:
|
||||
destination = h.overrideDestination
|
||||
case 2:
|
||||
newDestination := h.overrideDestination
|
||||
newDestination.Port = destination.Port
|
||||
destination = newDestination
|
||||
case 3:
|
||||
destination.Port = h.overrideDestination.Port
|
||||
}
|
||||
if h.overrideOption == 0 {
|
||||
h.logger.InfoContext(ctx, "outbound packet connection")
|
||||
} else {
|
||||
h.logger.InfoContext(ctx, "outbound packet connection to ", destination)
|
||||
}
|
||||
conn, err := h.dialer.ListenPacket(ctx, destination)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
// conn = h.loopBack.NewPacketConn(bufio.NewPacketConn(conn), destination)
|
||||
if originDestination != destination {
|
||||
conn = bufio.NewNATPacketConn(bufio.NewPacketConn(conn), destination, originDestination)
|
||||
}
|
||||
return conn, nil
|
||||
}
|
||||
|
||||
/*func (h *Outbound) NewConnection(ctx context.Context, conn net.Conn, metadata adapter.InboundContext) error {
|
||||
if h.loopBack.CheckConn(metadata.Source.AddrPort(), M.AddrPortFromNet(conn.LocalAddr())) {
|
||||
return E.New("reject loopback connection to ", metadata.Destination)
|
||||
}
|
||||
return NewConnection(ctx, h, conn, metadata)
|
||||
}
|
||||
|
||||
func (h *Outbound) NewPacketConnection(ctx context.Context, conn N.PacketConn, metadata adapter.InboundContext) error {
|
||||
if h.loopBack.CheckPacketConn(metadata.Source.AddrPort(), M.AddrPortFromNet(conn.LocalAddr())) {
|
||||
return E.New("reject loopback packet connection to ", metadata.Destination)
|
||||
}
|
||||
return NewPacketConnection(ctx, h, conn, metadata)
|
||||
}
|
||||
*/
|
218
protocol/dns/handle.go
Normal file
218
protocol/dns/handle.go
Normal file
|
@ -0,0 +1,218 @@
|
|||
package dns
|
||||
|
||||
import (
|
||||
"context"
|
||||
"encoding/binary"
|
||||
"net"
|
||||
|
||||
"github.com/sagernet/sing-box/adapter"
|
||||
C "github.com/sagernet/sing-box/constant"
|
||||
"github.com/sagernet/sing-dns"
|
||||
"github.com/sagernet/sing/common"
|
||||
"github.com/sagernet/sing/common/buf"
|
||||
"github.com/sagernet/sing/common/bufio"
|
||||
"github.com/sagernet/sing/common/canceler"
|
||||
M "github.com/sagernet/sing/common/metadata"
|
||||
N "github.com/sagernet/sing/common/network"
|
||||
"github.com/sagernet/sing/common/task"
|
||||
|
||||
mDNS "github.com/miekg/dns"
|
||||
)
|
||||
|
||||
func HandleStreamDNSRequest(ctx context.Context, router adapter.Router, conn net.Conn, metadata adapter.InboundContext) error {
|
||||
var queryLength uint16
|
||||
err := binary.Read(conn, binary.BigEndian, &queryLength)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if queryLength == 0 {
|
||||
return dns.RCodeFormatError
|
||||
}
|
||||
buffer := buf.NewSize(int(queryLength))
|
||||
defer buffer.Release()
|
||||
_, err = buffer.ReadFullFrom(conn, int(queryLength))
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
var message mDNS.Msg
|
||||
err = message.Unpack(buffer.Bytes())
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
metadataInQuery := metadata
|
||||
go func() error {
|
||||
response, err := router.Exchange(adapter.WithContext(ctx, &metadataInQuery), &message)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
responseBuffer := buf.NewPacket()
|
||||
defer responseBuffer.Release()
|
||||
responseBuffer.Resize(2, 0)
|
||||
n, err := response.PackBuffer(responseBuffer.FreeBytes())
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
responseBuffer.Truncate(len(n))
|
||||
binary.BigEndian.PutUint16(responseBuffer.ExtendHeader(2), uint16(len(n)))
|
||||
_, err = conn.Write(responseBuffer.Bytes())
|
||||
return err
|
||||
}()
|
||||
return nil
|
||||
}
|
||||
|
||||
func NewDNSPacketConnection(ctx context.Context, router adapter.Router, conn N.PacketConn, cachedPackets []*N.PacketBuffer, metadata adapter.InboundContext) error {
|
||||
metadata.Destination = M.Socksaddr{}
|
||||
var reader N.PacketReader = conn
|
||||
var counters []N.CountFunc
|
||||
cachedPackets = common.Reverse(cachedPackets)
|
||||
for {
|
||||
reader, counters = N.UnwrapCountPacketReader(reader, counters)
|
||||
if cachedReader, isCached := reader.(N.CachedPacketReader); isCached {
|
||||
packet := cachedReader.ReadCachedPacket()
|
||||
if packet != nil {
|
||||
cachedPackets = append(cachedPackets, packet)
|
||||
continue
|
||||
}
|
||||
}
|
||||
if readWaiter, created := bufio.CreatePacketReadWaiter(reader); created {
|
||||
readWaiter.InitializeReadWaiter(N.ReadWaitOptions{})
|
||||
return newDNSPacketConnection(ctx, router, conn, readWaiter, counters, cachedPackets, metadata)
|
||||
}
|
||||
break
|
||||
}
|
||||
fastClose, cancel := common.ContextWithCancelCause(ctx)
|
||||
timeout := canceler.New(fastClose, cancel, C.DNSTimeout)
|
||||
var group task.Group
|
||||
group.Append0(func(_ context.Context) error {
|
||||
for {
|
||||
var message mDNS.Msg
|
||||
var destination M.Socksaddr
|
||||
var err error
|
||||
if len(cachedPackets) > 0 {
|
||||
packet := cachedPackets[0]
|
||||
cachedPackets = cachedPackets[1:]
|
||||
for _, counter := range counters {
|
||||
counter(int64(packet.Buffer.Len()))
|
||||
}
|
||||
err = message.Unpack(packet.Buffer.Bytes())
|
||||
packet.Buffer.Release()
|
||||
if err != nil {
|
||||
cancel(err)
|
||||
return err
|
||||
}
|
||||
destination = packet.Destination
|
||||
} else {
|
||||
buffer := buf.NewPacket()
|
||||
destination, err = conn.ReadPacket(buffer)
|
||||
if err != nil {
|
||||
buffer.Release()
|
||||
cancel(err)
|
||||
return err
|
||||
}
|
||||
for _, counter := range counters {
|
||||
counter(int64(buffer.Len()))
|
||||
}
|
||||
err = message.Unpack(buffer.Bytes())
|
||||
buffer.Release()
|
||||
if err != nil {
|
||||
cancel(err)
|
||||
return err
|
||||
}
|
||||
timeout.Update()
|
||||
}
|
||||
metadataInQuery := metadata
|
||||
go func() error {
|
||||
response, err := router.Exchange(adapter.WithContext(ctx, &metadataInQuery), &message)
|
||||
if err != nil {
|
||||
cancel(err)
|
||||
return err
|
||||
}
|
||||
timeout.Update()
|
||||
responseBuffer, err := dns.TruncateDNSMessage(&message, response, 1024)
|
||||
if err != nil {
|
||||
cancel(err)
|
||||
return err
|
||||
}
|
||||
err = conn.WritePacket(responseBuffer, destination)
|
||||
if err != nil {
|
||||
cancel(err)
|
||||
}
|
||||
return err
|
||||
}()
|
||||
}
|
||||
})
|
||||
group.Cleanup(func() {
|
||||
conn.Close()
|
||||
})
|
||||
return group.Run(fastClose)
|
||||
}
|
||||
|
||||
func newDNSPacketConnection(ctx context.Context, router adapter.Router, conn N.PacketConn, readWaiter N.PacketReadWaiter, readCounters []N.CountFunc, cached []*N.PacketBuffer, metadata adapter.InboundContext) error {
|
||||
fastClose, cancel := common.ContextWithCancelCause(ctx)
|
||||
timeout := canceler.New(fastClose, cancel, C.DNSTimeout)
|
||||
var group task.Group
|
||||
group.Append0(func(_ context.Context) error {
|
||||
for {
|
||||
var (
|
||||
message mDNS.Msg
|
||||
destination M.Socksaddr
|
||||
err error
|
||||
buffer *buf.Buffer
|
||||
)
|
||||
if len(cached) > 0 {
|
||||
packet := cached[0]
|
||||
cached = cached[1:]
|
||||
for _, counter := range readCounters {
|
||||
counter(int64(packet.Buffer.Len()))
|
||||
}
|
||||
err = message.Unpack(packet.Buffer.Bytes())
|
||||
packet.Buffer.Release()
|
||||
destination = packet.Destination
|
||||
N.PutPacketBuffer(packet)
|
||||
if err != nil {
|
||||
cancel(err)
|
||||
return err
|
||||
}
|
||||
} else {
|
||||
buffer, destination, err = readWaiter.WaitReadPacket()
|
||||
if err != nil {
|
||||
cancel(err)
|
||||
return err
|
||||
}
|
||||
for _, counter := range readCounters {
|
||||
counter(int64(buffer.Len()))
|
||||
}
|
||||
err = message.Unpack(buffer.Bytes())
|
||||
buffer.Release()
|
||||
if err != nil {
|
||||
cancel(err)
|
||||
return err
|
||||
}
|
||||
timeout.Update()
|
||||
}
|
||||
metadataInQuery := metadata
|
||||
go func() error {
|
||||
response, err := router.Exchange(adapter.WithContext(ctx, &metadataInQuery), &message)
|
||||
if err != nil {
|
||||
cancel(err)
|
||||
return err
|
||||
}
|
||||
timeout.Update()
|
||||
responseBuffer, err := dns.TruncateDNSMessage(&message, response, 1024)
|
||||
if err != nil {
|
||||
cancel(err)
|
||||
return err
|
||||
}
|
||||
err = conn.WritePacket(responseBuffer, destination)
|
||||
if err != nil {
|
||||
cancel(err)
|
||||
}
|
||||
return err
|
||||
}()
|
||||
}
|
||||
})
|
||||
group.Cleanup(func() {
|
||||
conn.Close()
|
||||
})
|
||||
return group.Run(fastClose)
|
||||
}
|
61
protocol/dns/outbound.go
Normal file
61
protocol/dns/outbound.go
Normal file
|
@ -0,0 +1,61 @@
|
|||
package dns
|
||||
|
||||
import (
|
||||
"context"
|
||||
"net"
|
||||
"os"
|
||||
"time"
|
||||
|
||||
"github.com/sagernet/sing-box/adapter"
|
||||
"github.com/sagernet/sing-box/adapter/outbound"
|
||||
C "github.com/sagernet/sing-box/constant"
|
||||
"github.com/sagernet/sing-box/log"
|
||||
"github.com/sagernet/sing-box/option"
|
||||
"github.com/sagernet/sing/common/logger"
|
||||
M "github.com/sagernet/sing/common/metadata"
|
||||
N "github.com/sagernet/sing/common/network"
|
||||
)
|
||||
|
||||
func RegisterOutbound(registry *outbound.Registry) {
|
||||
outbound.Register[option.StubOptions](registry, C.TypeDNS, NewOutbound)
|
||||
}
|
||||
|
||||
type Outbound struct {
|
||||
outbound.Adapter
|
||||
router adapter.Router
|
||||
logger logger.ContextLogger
|
||||
}
|
||||
|
||||
func NewOutbound(ctx context.Context, router adapter.Router, logger log.ContextLogger, tag string, options option.StubOptions) (adapter.Outbound, error) {
|
||||
return &Outbound{
|
||||
Adapter: outbound.NewAdapter(C.TypeDNS, []string{N.NetworkTCP, N.NetworkUDP}, tag, nil),
|
||||
router: router,
|
||||
logger: logger,
|
||||
}, nil
|
||||
}
|
||||
|
||||
func (d *Outbound) DialContext(ctx context.Context, network string, destination M.Socksaddr) (net.Conn, error) {
|
||||
return nil, os.ErrInvalid
|
||||
}
|
||||
|
||||
func (d *Outbound) ListenPacket(ctx context.Context, destination M.Socksaddr) (net.PacketConn, error) {
|
||||
return nil, os.ErrInvalid
|
||||
}
|
||||
|
||||
// Deprecated
|
||||
func (d *Outbound) NewConnection(ctx context.Context, conn net.Conn, metadata adapter.InboundContext) error {
|
||||
metadata.Destination = M.Socksaddr{}
|
||||
defer conn.Close()
|
||||
for {
|
||||
conn.SetReadDeadline(time.Now().Add(C.DNSTimeout))
|
||||
err := HandleStreamDNSRequest(ctx, d.router, conn, metadata)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Deprecated
|
||||
func (d *Outbound) NewPacketConnection(ctx context.Context, conn N.PacketConn, metadata adapter.InboundContext) error {
|
||||
return NewDNSPacketConnection(ctx, d.router, conn, nil, metadata)
|
||||
}
|
177
protocol/group/selector.go
Normal file
177
protocol/group/selector.go
Normal file
|
@ -0,0 +1,177 @@
|
|||
package group
|
||||
|
||||
import (
|
||||
"context"
|
||||
"net"
|
||||
|
||||
"github.com/sagernet/sing-box/adapter"
|
||||
"github.com/sagernet/sing-box/adapter/outbound"
|
||||
"github.com/sagernet/sing-box/common/interrupt"
|
||||
C "github.com/sagernet/sing-box/constant"
|
||||
"github.com/sagernet/sing-box/log"
|
||||
"github.com/sagernet/sing-box/option"
|
||||
E "github.com/sagernet/sing/common/exceptions"
|
||||
"github.com/sagernet/sing/common/logger"
|
||||
M "github.com/sagernet/sing/common/metadata"
|
||||
N "github.com/sagernet/sing/common/network"
|
||||
"github.com/sagernet/sing/service"
|
||||
)
|
||||
|
||||
func RegisterSelector(registry *outbound.Registry) {
|
||||
outbound.Register[option.SelectorOutboundOptions](registry, C.TypeSelector, NewSelector)
|
||||
}
|
||||
|
||||
var _ adapter.OutboundGroup = (*Selector)(nil)
|
||||
|
||||
type Selector struct {
|
||||
outbound.Adapter
|
||||
ctx context.Context
|
||||
router adapter.Router
|
||||
logger logger.ContextLogger
|
||||
tags []string
|
||||
defaultTag string
|
||||
outbounds map[string]adapter.Outbound
|
||||
selected adapter.Outbound
|
||||
interruptGroup *interrupt.Group
|
||||
interruptExternalConnections bool
|
||||
}
|
||||
|
||||
func NewSelector(ctx context.Context, router adapter.Router, logger log.ContextLogger, tag string, options option.SelectorOutboundOptions) (adapter.Outbound, error) {
|
||||
outbound := &Selector{
|
||||
Adapter: outbound.NewAdapter(C.TypeSelector, nil, tag, options.Outbounds),
|
||||
ctx: ctx,
|
||||
router: router,
|
||||
logger: logger,
|
||||
tags: options.Outbounds,
|
||||
defaultTag: options.Default,
|
||||
outbounds: make(map[string]adapter.Outbound),
|
||||
interruptGroup: interrupt.NewGroup(),
|
||||
interruptExternalConnections: options.InterruptExistConnections,
|
||||
}
|
||||
if len(outbound.tags) == 0 {
|
||||
return nil, E.New("missing tags")
|
||||
}
|
||||
return outbound, nil
|
||||
}
|
||||
|
||||
func (s *Selector) Network() []string {
|
||||
if s.selected == nil {
|
||||
return []string{N.NetworkTCP, N.NetworkUDP}
|
||||
}
|
||||
return s.selected.Network()
|
||||
}
|
||||
|
||||
func (s *Selector) Start() error {
|
||||
for i, tag := range s.tags {
|
||||
detour, loaded := s.router.Outbound(tag)
|
||||
if !loaded {
|
||||
return E.New("outbound ", i, " not found: ", tag)
|
||||
}
|
||||
s.outbounds[tag] = detour
|
||||
}
|
||||
|
||||
if s.Tag() != "" {
|
||||
cacheFile := service.FromContext[adapter.CacheFile](s.ctx)
|
||||
if cacheFile != nil {
|
||||
selected := cacheFile.LoadSelected(s.Tag())
|
||||
if selected != "" {
|
||||
detour, loaded := s.outbounds[selected]
|
||||
if loaded {
|
||||
s.selected = detour
|
||||
return nil
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if s.defaultTag != "" {
|
||||
detour, loaded := s.outbounds[s.defaultTag]
|
||||
if !loaded {
|
||||
return E.New("default outbound not found: ", s.defaultTag)
|
||||
}
|
||||
s.selected = detour
|
||||
return nil
|
||||
}
|
||||
|
||||
s.selected = s.outbounds[s.tags[0]]
|
||||
return nil
|
||||
}
|
||||
|
||||
func (s *Selector) Now() string {
|
||||
selected := s.selected
|
||||
if selected == nil {
|
||||
return s.tags[0]
|
||||
}
|
||||
return selected.Tag()
|
||||
}
|
||||
|
||||
func (s *Selector) All() []string {
|
||||
return s.tags
|
||||
}
|
||||
|
||||
func (s *Selector) SelectOutbound(tag string) bool {
|
||||
detour, loaded := s.outbounds[tag]
|
||||
if !loaded {
|
||||
return false
|
||||
}
|
||||
if s.selected == detour {
|
||||
return true
|
||||
}
|
||||
s.selected = detour
|
||||
if s.Tag() != "" {
|
||||
cacheFile := service.FromContext[adapter.CacheFile](s.ctx)
|
||||
if cacheFile != nil {
|
||||
err := cacheFile.StoreSelected(s.Tag(), tag)
|
||||
if err != nil {
|
||||
s.logger.Error("store selected: ", err)
|
||||
}
|
||||
}
|
||||
}
|
||||
s.interruptGroup.Interrupt(s.interruptExternalConnections)
|
||||
return true
|
||||
}
|
||||
|
||||
func (s *Selector) DialContext(ctx context.Context, network string, destination M.Socksaddr) (net.Conn, error) {
|
||||
conn, err := s.selected.DialContext(ctx, network, destination)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return s.interruptGroup.NewConn(conn, interrupt.IsExternalConnectionFromContext(ctx)), nil
|
||||
}
|
||||
|
||||
func (s *Selector) ListenPacket(ctx context.Context, destination M.Socksaddr) (net.PacketConn, error) {
|
||||
conn, err := s.selected.ListenPacket(ctx, destination)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return s.interruptGroup.NewPacketConn(conn, interrupt.IsExternalConnectionFromContext(ctx)), nil
|
||||
}
|
||||
|
||||
// TODO
|
||||
// Deprecated
|
||||
func (s *Selector) NewConnection(ctx context.Context, conn net.Conn, metadata adapter.InboundContext) error {
|
||||
ctx = interrupt.ContextWithIsExternalConnection(ctx)
|
||||
if legacyHandler, ok := s.selected.(adapter.ConnectionHandler); ok {
|
||||
return legacyHandler.NewConnection(ctx, conn, metadata)
|
||||
} else {
|
||||
return outbound.NewConnection(ctx, s.selected, conn, metadata)
|
||||
}
|
||||
}
|
||||
|
||||
// TODO
|
||||
// Deprecated
|
||||
func (s *Selector) NewPacketConnection(ctx context.Context, conn N.PacketConn, metadata adapter.InboundContext) error {
|
||||
ctx = interrupt.ContextWithIsExternalConnection(ctx)
|
||||
if legacyHandler, ok := s.selected.(adapter.PacketConnectionHandler); ok {
|
||||
return legacyHandler.NewPacketConnection(ctx, conn, metadata)
|
||||
} else {
|
||||
return outbound.NewPacketConnection(ctx, s.selected, conn, metadata)
|
||||
}
|
||||
}
|
||||
|
||||
func RealTag(detour adapter.Outbound) string {
|
||||
if group, isGroup := detour.(adapter.OutboundGroup); isGroup {
|
||||
return group.Now()
|
||||
}
|
||||
return detour.Tag()
|
||||
}
|
430
protocol/group/urltest.go
Normal file
430
protocol/group/urltest.go
Normal file
|
@ -0,0 +1,430 @@
|
|||
package group
|
||||
|
||||
import (
|
||||
"context"
|
||||
"net"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/sagernet/sing-box/adapter"
|
||||
"github.com/sagernet/sing-box/adapter/outbound"
|
||||
"github.com/sagernet/sing-box/common/interrupt"
|
||||
"github.com/sagernet/sing-box/common/urltest"
|
||||
C "github.com/sagernet/sing-box/constant"
|
||||
"github.com/sagernet/sing-box/log"
|
||||
"github.com/sagernet/sing-box/option"
|
||||
"github.com/sagernet/sing/common"
|
||||
"github.com/sagernet/sing/common/atomic"
|
||||
"github.com/sagernet/sing/common/batch"
|
||||
E "github.com/sagernet/sing/common/exceptions"
|
||||
M "github.com/sagernet/sing/common/metadata"
|
||||
N "github.com/sagernet/sing/common/network"
|
||||
"github.com/sagernet/sing/service"
|
||||
"github.com/sagernet/sing/service/pause"
|
||||
)
|
||||
|
||||
func RegisterURLTest(registry *outbound.Registry) {
|
||||
outbound.Register[option.URLTestOutboundOptions](registry, C.TypeURLTest, NewURLTest)
|
||||
}
|
||||
|
||||
var (
|
||||
_ adapter.OutboundGroup = (*URLTest)(nil)
|
||||
_ adapter.InterfaceUpdateListener = (*URLTest)(nil)
|
||||
)
|
||||
|
||||
type URLTest struct {
|
||||
outbound.Adapter
|
||||
ctx context.Context
|
||||
router adapter.Router
|
||||
logger log.ContextLogger
|
||||
tags []string
|
||||
link string
|
||||
interval time.Duration
|
||||
tolerance uint16
|
||||
idleTimeout time.Duration
|
||||
group *URLTestGroup
|
||||
interruptExternalConnections bool
|
||||
}
|
||||
|
||||
func NewURLTest(ctx context.Context, router adapter.Router, logger log.ContextLogger, tag string, options option.URLTestOutboundOptions) (adapter.Outbound, error) {
|
||||
outbound := &URLTest{
|
||||
Adapter: outbound.NewAdapter(C.TypeURLTest, []string{N.NetworkTCP, N.NetworkUDP}, tag, options.Outbounds),
|
||||
ctx: ctx,
|
||||
router: router,
|
||||
logger: logger,
|
||||
tags: options.Outbounds,
|
||||
link: options.URL,
|
||||
interval: time.Duration(options.Interval),
|
||||
tolerance: options.Tolerance,
|
||||
idleTimeout: time.Duration(options.IdleTimeout),
|
||||
interruptExternalConnections: options.InterruptExistConnections,
|
||||
}
|
||||
if len(outbound.tags) == 0 {
|
||||
return nil, E.New("missing tags")
|
||||
}
|
||||
return outbound, nil
|
||||
}
|
||||
|
||||
func (s *URLTest) Start() error {
|
||||
outbounds := make([]adapter.Outbound, 0, len(s.tags))
|
||||
for i, tag := range s.tags {
|
||||
detour, loaded := s.router.Outbound(tag)
|
||||
if !loaded {
|
||||
return E.New("outbound ", i, " not found: ", tag)
|
||||
}
|
||||
outbounds = append(outbounds, detour)
|
||||
}
|
||||
group, err := NewURLTestGroup(
|
||||
s.ctx,
|
||||
s.router,
|
||||
s.logger,
|
||||
outbounds,
|
||||
s.link,
|
||||
s.interval,
|
||||
s.tolerance,
|
||||
s.idleTimeout,
|
||||
s.interruptExternalConnections,
|
||||
)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
s.group = group
|
||||
return nil
|
||||
}
|
||||
|
||||
func (s *URLTest) PostStart() error {
|
||||
s.group.PostStart()
|
||||
return nil
|
||||
}
|
||||
|
||||
func (s *URLTest) Close() error {
|
||||
return common.Close(
|
||||
common.PtrOrNil(s.group),
|
||||
)
|
||||
}
|
||||
|
||||
func (s *URLTest) Now() string {
|
||||
if s.group.selectedOutboundTCP != nil {
|
||||
return s.group.selectedOutboundTCP.Tag()
|
||||
} else if s.group.selectedOutboundUDP != nil {
|
||||
return s.group.selectedOutboundUDP.Tag()
|
||||
}
|
||||
return ""
|
||||
}
|
||||
|
||||
func (s *URLTest) All() []string {
|
||||
return s.tags
|
||||
}
|
||||
|
||||
func (s *URLTest) URLTest(ctx context.Context) (map[string]uint16, error) {
|
||||
return s.group.URLTest(ctx)
|
||||
}
|
||||
|
||||
func (s *URLTest) CheckOutbounds() {
|
||||
s.group.CheckOutbounds(true)
|
||||
}
|
||||
|
||||
func (s *URLTest) DialContext(ctx context.Context, network string, destination M.Socksaddr) (net.Conn, error) {
|
||||
s.group.Touch()
|
||||
var outbound adapter.Outbound
|
||||
switch N.NetworkName(network) {
|
||||
case N.NetworkTCP:
|
||||
outbound = s.group.selectedOutboundTCP
|
||||
case N.NetworkUDP:
|
||||
outbound = s.group.selectedOutboundUDP
|
||||
default:
|
||||
return nil, E.Extend(N.ErrUnknownNetwork, network)
|
||||
}
|
||||
if outbound == nil {
|
||||
outbound, _ = s.group.Select(network)
|
||||
}
|
||||
if outbound == nil {
|
||||
return nil, E.New("missing supported outbound")
|
||||
}
|
||||
conn, err := outbound.DialContext(ctx, network, destination)
|
||||
if err == nil {
|
||||
return s.group.interruptGroup.NewConn(conn, interrupt.IsExternalConnectionFromContext(ctx)), nil
|
||||
}
|
||||
s.logger.ErrorContext(ctx, err)
|
||||
s.group.history.DeleteURLTestHistory(outbound.Tag())
|
||||
return nil, err
|
||||
}
|
||||
|
||||
func (s *URLTest) ListenPacket(ctx context.Context, destination M.Socksaddr) (net.PacketConn, error) {
|
||||
s.group.Touch()
|
||||
outbound := s.group.selectedOutboundUDP
|
||||
if outbound == nil {
|
||||
outbound, _ = s.group.Select(N.NetworkUDP)
|
||||
}
|
||||
if outbound == nil {
|
||||
return nil, E.New("missing supported outbound")
|
||||
}
|
||||
conn, err := outbound.ListenPacket(ctx, destination)
|
||||
if err == nil {
|
||||
return s.group.interruptGroup.NewPacketConn(conn, interrupt.IsExternalConnectionFromContext(ctx)), nil
|
||||
}
|
||||
s.logger.ErrorContext(ctx, err)
|
||||
s.group.history.DeleteURLTestHistory(outbound.Tag())
|
||||
return nil, err
|
||||
}
|
||||
|
||||
// TODO
|
||||
// Deprecated
|
||||
func (s *URLTest) NewConnection(ctx context.Context, conn net.Conn, metadata adapter.InboundContext) error {
|
||||
ctx = interrupt.ContextWithIsExternalConnection(ctx)
|
||||
return outbound.NewConnection(ctx, s, conn, metadata)
|
||||
}
|
||||
|
||||
// TODO
|
||||
// Deprecated
|
||||
func (s *URLTest) NewPacketConnection(ctx context.Context, conn N.PacketConn, metadata adapter.InboundContext) error {
|
||||
ctx = interrupt.ContextWithIsExternalConnection(ctx)
|
||||
return outbound.NewPacketConnection(ctx, s, conn, metadata)
|
||||
}
|
||||
|
||||
func (s *URLTest) InterfaceUpdated() {
|
||||
go s.group.CheckOutbounds(true)
|
||||
return
|
||||
}
|
||||
|
||||
type URLTestGroup struct {
|
||||
ctx context.Context
|
||||
router adapter.Router
|
||||
logger log.Logger
|
||||
outbounds []adapter.Outbound
|
||||
link string
|
||||
interval time.Duration
|
||||
tolerance uint16
|
||||
idleTimeout time.Duration
|
||||
history *urltest.HistoryStorage
|
||||
checking atomic.Bool
|
||||
pauseManager pause.Manager
|
||||
selectedOutboundTCP adapter.Outbound
|
||||
selectedOutboundUDP adapter.Outbound
|
||||
interruptGroup *interrupt.Group
|
||||
interruptExternalConnections bool
|
||||
|
||||
access sync.Mutex
|
||||
ticker *time.Ticker
|
||||
close chan struct{}
|
||||
started bool
|
||||
lastActive atomic.TypedValue[time.Time]
|
||||
}
|
||||
|
||||
func NewURLTestGroup(
|
||||
ctx context.Context,
|
||||
router adapter.Router,
|
||||
logger log.Logger,
|
||||
outbounds []adapter.Outbound,
|
||||
link string,
|
||||
interval time.Duration,
|
||||
tolerance uint16,
|
||||
idleTimeout time.Duration,
|
||||
interruptExternalConnections bool,
|
||||
) (*URLTestGroup, error) {
|
||||
if interval == 0 {
|
||||
interval = C.DefaultURLTestInterval
|
||||
}
|
||||
if tolerance == 0 {
|
||||
tolerance = 50
|
||||
}
|
||||
if idleTimeout == 0 {
|
||||
idleTimeout = C.DefaultURLTestIdleTimeout
|
||||
}
|
||||
if interval > idleTimeout {
|
||||
return nil, E.New("interval must be less or equal than idle_timeout")
|
||||
}
|
||||
var history *urltest.HistoryStorage
|
||||
if history = service.PtrFromContext[urltest.HistoryStorage](ctx); history != nil {
|
||||
} else if clashServer := router.ClashServer(); clashServer != nil {
|
||||
history = clashServer.HistoryStorage()
|
||||
} else {
|
||||
history = urltest.NewHistoryStorage()
|
||||
}
|
||||
return &URLTestGroup{
|
||||
ctx: ctx,
|
||||
router: router,
|
||||
logger: logger,
|
||||
outbounds: outbounds,
|
||||
link: link,
|
||||
interval: interval,
|
||||
tolerance: tolerance,
|
||||
idleTimeout: idleTimeout,
|
||||
history: history,
|
||||
close: make(chan struct{}),
|
||||
pauseManager: service.FromContext[pause.Manager](ctx),
|
||||
interruptGroup: interrupt.NewGroup(),
|
||||
interruptExternalConnections: interruptExternalConnections,
|
||||
}, nil
|
||||
}
|
||||
|
||||
func (g *URLTestGroup) PostStart() {
|
||||
g.started = true
|
||||
g.lastActive.Store(time.Now())
|
||||
go g.CheckOutbounds(false)
|
||||
}
|
||||
|
||||
func (g *URLTestGroup) Touch() {
|
||||
if !g.started {
|
||||
return
|
||||
}
|
||||
if g.ticker != nil {
|
||||
g.lastActive.Store(time.Now())
|
||||
return
|
||||
}
|
||||
g.access.Lock()
|
||||
defer g.access.Unlock()
|
||||
if g.ticker != nil {
|
||||
return
|
||||
}
|
||||
g.ticker = time.NewTicker(g.interval)
|
||||
go g.loopCheck()
|
||||
}
|
||||
|
||||
func (g *URLTestGroup) Close() error {
|
||||
if g.ticker == nil {
|
||||
return nil
|
||||
}
|
||||
g.ticker.Stop()
|
||||
close(g.close)
|
||||
return nil
|
||||
}
|
||||
|
||||
func (g *URLTestGroup) Select(network string) (adapter.Outbound, bool) {
|
||||
var minDelay uint16
|
||||
var minOutbound adapter.Outbound
|
||||
switch network {
|
||||
case N.NetworkTCP:
|
||||
if g.selectedOutboundTCP != nil {
|
||||
if history := g.history.LoadURLTestHistory(RealTag(g.selectedOutboundTCP)); history != nil {
|
||||
minOutbound = g.selectedOutboundTCP
|
||||
minDelay = history.Delay
|
||||
}
|
||||
}
|
||||
case N.NetworkUDP:
|
||||
if g.selectedOutboundUDP != nil {
|
||||
if history := g.history.LoadURLTestHistory(RealTag(g.selectedOutboundUDP)); history != nil {
|
||||
minOutbound = g.selectedOutboundUDP
|
||||
minDelay = history.Delay
|
||||
}
|
||||
}
|
||||
}
|
||||
for _, detour := range g.outbounds {
|
||||
if !common.Contains(detour.Network(), network) {
|
||||
continue
|
||||
}
|
||||
history := g.history.LoadURLTestHistory(RealTag(detour))
|
||||
if history == nil {
|
||||
continue
|
||||
}
|
||||
if minDelay == 0 || minDelay > history.Delay+g.tolerance {
|
||||
minDelay = history.Delay
|
||||
minOutbound = detour
|
||||
}
|
||||
}
|
||||
if minOutbound == nil {
|
||||
for _, detour := range g.outbounds {
|
||||
if !common.Contains(detour.Network(), network) {
|
||||
continue
|
||||
}
|
||||
return detour, false
|
||||
}
|
||||
return nil, false
|
||||
}
|
||||
return minOutbound, true
|
||||
}
|
||||
|
||||
func (g *URLTestGroup) loopCheck() {
|
||||
if time.Now().Sub(g.lastActive.Load()) > g.interval {
|
||||
g.lastActive.Store(time.Now())
|
||||
g.CheckOutbounds(false)
|
||||
}
|
||||
for {
|
||||
select {
|
||||
case <-g.close:
|
||||
return
|
||||
case <-g.ticker.C:
|
||||
}
|
||||
if time.Now().Sub(g.lastActive.Load()) > g.idleTimeout {
|
||||
g.access.Lock()
|
||||
g.ticker.Stop()
|
||||
g.ticker = nil
|
||||
g.access.Unlock()
|
||||
return
|
||||
}
|
||||
g.pauseManager.WaitActive()
|
||||
g.CheckOutbounds(false)
|
||||
}
|
||||
}
|
||||
|
||||
func (g *URLTestGroup) CheckOutbounds(force bool) {
|
||||
_, _ = g.urlTest(g.ctx, force)
|
||||
}
|
||||
|
||||
func (g *URLTestGroup) URLTest(ctx context.Context) (map[string]uint16, error) {
|
||||
return g.urlTest(ctx, false)
|
||||
}
|
||||
|
||||
func (g *URLTestGroup) urlTest(ctx context.Context, force bool) (map[string]uint16, error) {
|
||||
result := make(map[string]uint16)
|
||||
if g.checking.Swap(true) {
|
||||
return result, nil
|
||||
}
|
||||
defer g.checking.Store(false)
|
||||
b, _ := batch.New(ctx, batch.WithConcurrencyNum[any](10))
|
||||
checked := make(map[string]bool)
|
||||
var resultAccess sync.Mutex
|
||||
for _, detour := range g.outbounds {
|
||||
tag := detour.Tag()
|
||||
realTag := RealTag(detour)
|
||||
if checked[realTag] {
|
||||
continue
|
||||
}
|
||||
history := g.history.LoadURLTestHistory(realTag)
|
||||
if !force && history != nil && time.Now().Sub(history.Time) < g.interval {
|
||||
continue
|
||||
}
|
||||
checked[realTag] = true
|
||||
p, loaded := g.router.Outbound(realTag)
|
||||
if !loaded {
|
||||
continue
|
||||
}
|
||||
b.Go(realTag, func() (any, error) {
|
||||
testCtx, cancel := context.WithTimeout(g.ctx, C.TCPTimeout)
|
||||
defer cancel()
|
||||
t, err := urltest.URLTest(testCtx, g.link, p)
|
||||
if err != nil {
|
||||
g.logger.Debug("outbound ", tag, " unavailable: ", err)
|
||||
g.history.DeleteURLTestHistory(realTag)
|
||||
} else {
|
||||
g.logger.Debug("outbound ", tag, " available: ", t, "ms")
|
||||
g.history.StoreURLTestHistory(realTag, &urltest.History{
|
||||
Time: time.Now(),
|
||||
Delay: t,
|
||||
})
|
||||
resultAccess.Lock()
|
||||
result[tag] = t
|
||||
resultAccess.Unlock()
|
||||
}
|
||||
return nil, nil
|
||||
})
|
||||
}
|
||||
b.Wait()
|
||||
g.performUpdateCheck()
|
||||
return result, nil
|
||||
}
|
||||
|
||||
func (g *URLTestGroup) performUpdateCheck() {
|
||||
var updated bool
|
||||
if outbound, exists := g.Select(N.NetworkTCP); outbound != nil && (g.selectedOutboundTCP == nil || (exists && outbound != g.selectedOutboundTCP)) {
|
||||
g.selectedOutboundTCP = outbound
|
||||
updated = true
|
||||
}
|
||||
if outbound, exists := g.Select(N.NetworkUDP); outbound != nil && (g.selectedOutboundUDP == nil || (exists && outbound != g.selectedOutboundUDP)) {
|
||||
g.selectedOutboundUDP = outbound
|
||||
updated = true
|
||||
}
|
||||
if updated {
|
||||
g.interruptGroup.Interrupt(g.interruptExternalConnections)
|
||||
}
|
||||
}
|
122
protocol/http/inbound.go
Normal file
122
protocol/http/inbound.go
Normal file
|
@ -0,0 +1,122 @@
|
|||
package http
|
||||
|
||||
import (
|
||||
std_bufio "bufio"
|
||||
"context"
|
||||
"net"
|
||||
|
||||
"github.com/sagernet/sing-box/adapter"
|
||||
"github.com/sagernet/sing-box/adapter/inbound"
|
||||
"github.com/sagernet/sing-box/common/listener"
|
||||
"github.com/sagernet/sing-box/common/tls"
|
||||
"github.com/sagernet/sing-box/common/uot"
|
||||
C "github.com/sagernet/sing-box/constant"
|
||||
"github.com/sagernet/sing-box/log"
|
||||
"github.com/sagernet/sing-box/option"
|
||||
"github.com/sagernet/sing/common"
|
||||
"github.com/sagernet/sing/common/auth"
|
||||
E "github.com/sagernet/sing/common/exceptions"
|
||||
N "github.com/sagernet/sing/common/network"
|
||||
"github.com/sagernet/sing/protocol/http"
|
||||
)
|
||||
|
||||
func RegisterInbound(registry *inbound.Registry) {
|
||||
inbound.Register[option.HTTPMixedInboundOptions](registry, C.TypeHTTP, NewInbound)
|
||||
}
|
||||
|
||||
var _ adapter.TCPInjectableInbound = (*Inbound)(nil)
|
||||
|
||||
type Inbound struct {
|
||||
inbound.Adapter
|
||||
router adapter.ConnectionRouterEx
|
||||
logger log.ContextLogger
|
||||
listener *listener.Listener
|
||||
authenticator *auth.Authenticator
|
||||
tlsConfig tls.ServerConfig
|
||||
}
|
||||
|
||||
func NewInbound(ctx context.Context, router adapter.Router, logger log.ContextLogger, tag string, options option.HTTPMixedInboundOptions) (adapter.Inbound, error) {
|
||||
inbound := &Inbound{
|
||||
Adapter: inbound.NewAdapter(C.TypeHTTP, tag),
|
||||
router: uot.NewRouter(router, logger),
|
||||
logger: logger,
|
||||
authenticator: auth.NewAuthenticator(options.Users),
|
||||
}
|
||||
if options.TLS != nil {
|
||||
tlsConfig, err := tls.NewServer(ctx, logger, common.PtrValueOrDefault(options.TLS))
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
inbound.tlsConfig = tlsConfig
|
||||
}
|
||||
inbound.listener = listener.New(listener.Options{
|
||||
Context: ctx,
|
||||
Logger: logger,
|
||||
Network: []string{N.NetworkTCP},
|
||||
Listen: options.ListenOptions,
|
||||
ConnectionHandler: inbound,
|
||||
SetSystemProxy: options.SetSystemProxy,
|
||||
SystemProxySOCKS: false,
|
||||
})
|
||||
return inbound, nil
|
||||
}
|
||||
|
||||
func (h *Inbound) Start() error {
|
||||
if h.tlsConfig != nil {
|
||||
err := h.tlsConfig.Start()
|
||||
if err != nil {
|
||||
return E.Cause(err, "create TLS config")
|
||||
}
|
||||
}
|
||||
return h.listener.Start()
|
||||
}
|
||||
|
||||
func (h *Inbound) Close() error {
|
||||
return common.Close(
|
||||
&h.listener,
|
||||
h.tlsConfig,
|
||||
)
|
||||
}
|
||||
|
||||
func (h *Inbound) NewConnectionEx(ctx context.Context, conn net.Conn, metadata adapter.InboundContext, onClose N.CloseHandlerFunc) {
|
||||
err := h.newConnection(ctx, conn, metadata, onClose)
|
||||
N.CloseOnHandshakeFailure(conn, onClose, err)
|
||||
if err != nil {
|
||||
h.logger.ErrorContext(ctx, E.Cause(err, "process connection from ", metadata.Source))
|
||||
}
|
||||
}
|
||||
|
||||
func (h *Inbound) newConnection(ctx context.Context, conn net.Conn, metadata adapter.InboundContext, onClose N.CloseHandlerFunc) error {
|
||||
var err error
|
||||
if h.tlsConfig != nil {
|
||||
conn, err = tls.ServerHandshake(ctx, conn, h.tlsConfig)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
return http.HandleConnectionEx(ctx, conn, std_bufio.NewReader(conn), h.authenticator, nil, adapter.NewUpstreamHandlerEx(metadata, h.newUserConnection, h.streamUserPacketConnection), metadata.Source, onClose)
|
||||
}
|
||||
|
||||
func (h *Inbound) newUserConnection(ctx context.Context, conn net.Conn, metadata adapter.InboundContext, onClose N.CloseHandlerFunc) {
|
||||
user, loaded := auth.UserFromContext[string](ctx)
|
||||
if !loaded {
|
||||
h.logger.InfoContext(ctx, "inbound connection to ", metadata.Destination)
|
||||
h.router.RouteConnectionEx(ctx, conn, metadata, onClose)
|
||||
return
|
||||
}
|
||||
metadata.User = user
|
||||
h.logger.InfoContext(ctx, "[", user, "] inbound connection to ", metadata.Destination)
|
||||
h.router.RouteConnectionEx(ctx, conn, metadata, onClose)
|
||||
}
|
||||
|
||||
func (h *Inbound) streamUserPacketConnection(ctx context.Context, conn N.PacketConn, metadata adapter.InboundContext, onClose N.CloseHandlerFunc) {
|
||||
user, loaded := auth.UserFromContext[string](ctx)
|
||||
if !loaded {
|
||||
h.logger.InfoContext(ctx, "inbound packet connection to ", metadata.Destination)
|
||||
h.router.RoutePacketConnectionEx(ctx, conn, metadata, onClose)
|
||||
return
|
||||
}
|
||||
metadata.User = user
|
||||
h.logger.InfoContext(ctx, "[", user, "] inbound packet connection to ", metadata.Destination)
|
||||
h.router.RoutePacketConnectionEx(ctx, conn, metadata, onClose)
|
||||
}
|
65
protocol/http/outbound.go
Normal file
65
protocol/http/outbound.go
Normal file
|
@ -0,0 +1,65 @@
|
|||
package http
|
||||
|
||||
import (
|
||||
"context"
|
||||
"net"
|
||||
"os"
|
||||
|
||||
"github.com/sagernet/sing-box/adapter"
|
||||
"github.com/sagernet/sing-box/adapter/outbound"
|
||||
"github.com/sagernet/sing-box/common/dialer"
|
||||
"github.com/sagernet/sing-box/common/tls"
|
||||
C "github.com/sagernet/sing-box/constant"
|
||||
"github.com/sagernet/sing-box/log"
|
||||
"github.com/sagernet/sing-box/option"
|
||||
"github.com/sagernet/sing/common"
|
||||
"github.com/sagernet/sing/common/logger"
|
||||
M "github.com/sagernet/sing/common/metadata"
|
||||
N "github.com/sagernet/sing/common/network"
|
||||
sHTTP "github.com/sagernet/sing/protocol/http"
|
||||
)
|
||||
|
||||
func RegisterOutbound(registry *outbound.Registry) {
|
||||
outbound.Register[option.HTTPOutboundOptions](registry, C.TypeHTTP, NewOutbound)
|
||||
}
|
||||
|
||||
type Outbound struct {
|
||||
outbound.Adapter
|
||||
logger logger.ContextLogger
|
||||
client *sHTTP.Client
|
||||
}
|
||||
|
||||
func NewOutbound(ctx context.Context, router adapter.Router, logger log.ContextLogger, tag string, options option.HTTPOutboundOptions) (adapter.Outbound, error) {
|
||||
outboundDialer, err := dialer.New(router, options.DialerOptions)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
detour, err := tls.NewDialerFromOptions(ctx, router, outboundDialer, options.Server, common.PtrValueOrDefault(options.TLS))
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return &Outbound{
|
||||
Adapter: outbound.NewAdapterWithDialerOptions(C.TypeHTTP, []string{N.NetworkTCP}, tag, options.DialerOptions),
|
||||
logger: logger,
|
||||
client: sHTTP.NewClient(sHTTP.Options{
|
||||
Dialer: detour,
|
||||
Server: options.ServerOptions.Build(),
|
||||
Username: options.Username,
|
||||
Password: options.Password,
|
||||
Path: options.Path,
|
||||
Headers: options.Headers.Build(),
|
||||
}),
|
||||
}, nil
|
||||
}
|
||||
|
||||
func (h *Outbound) DialContext(ctx context.Context, network string, destination M.Socksaddr) (net.Conn, error) {
|
||||
ctx, metadata := adapter.ExtendContext(ctx)
|
||||
metadata.Outbound = h.Tag()
|
||||
metadata.Destination = destination
|
||||
h.logger.InfoContext(ctx, "outbound connection to ", destination)
|
||||
return h.client.DialContext(ctx, network, destination)
|
||||
}
|
||||
|
||||
func (h *Outbound) ListenPacket(ctx context.Context, destination M.Socksaddr) (net.PacketConn, error) {
|
||||
return nil, os.ErrInvalid
|
||||
}
|
175
protocol/hysteria/inbound.go
Normal file
175
protocol/hysteria/inbound.go
Normal file
|
@ -0,0 +1,175 @@
|
|||
package hysteria
|
||||
|
||||
import (
|
||||
"context"
|
||||
"net"
|
||||
"time"
|
||||
|
||||
"github.com/sagernet/sing-box/adapter"
|
||||
"github.com/sagernet/sing-box/adapter/inbound"
|
||||
"github.com/sagernet/sing-box/common/humanize"
|
||||
"github.com/sagernet/sing-box/common/listener"
|
||||
"github.com/sagernet/sing-box/common/tls"
|
||||
C "github.com/sagernet/sing-box/constant"
|
||||
"github.com/sagernet/sing-box/log"
|
||||
"github.com/sagernet/sing-box/option"
|
||||
"github.com/sagernet/sing-quic/hysteria"
|
||||
"github.com/sagernet/sing/common"
|
||||
"github.com/sagernet/sing/common/auth"
|
||||
E "github.com/sagernet/sing/common/exceptions"
|
||||
N "github.com/sagernet/sing/common/network"
|
||||
)
|
||||
|
||||
func RegisterInbound(registry *inbound.Registry) {
|
||||
inbound.Register[option.HysteriaInboundOptions](registry, C.TypeHysteria, NewInbound)
|
||||
}
|
||||
|
||||
type Inbound struct {
|
||||
inbound.Adapter
|
||||
router adapter.Router
|
||||
logger log.ContextLogger
|
||||
listener *listener.Listener
|
||||
tlsConfig tls.ServerConfig
|
||||
service *hysteria.Service[int]
|
||||
userNameList []string
|
||||
}
|
||||
|
||||
func NewInbound(ctx context.Context, router adapter.Router, logger log.ContextLogger, tag string, options option.HysteriaInboundOptions) (adapter.Inbound, error) {
|
||||
options.UDPFragmentDefault = true
|
||||
if options.TLS == nil || !options.TLS.Enabled {
|
||||
return nil, C.ErrTLSRequired
|
||||
}
|
||||
tlsConfig, err := tls.NewServer(ctx, logger, common.PtrValueOrDefault(options.TLS))
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
inbound := &Inbound{
|
||||
Adapter: inbound.NewAdapter(C.TypeHysteria, tag),
|
||||
router: router,
|
||||
logger: logger,
|
||||
listener: listener.New(listener.Options{
|
||||
Context: ctx,
|
||||
Logger: logger,
|
||||
Listen: options.ListenOptions,
|
||||
}),
|
||||
tlsConfig: tlsConfig,
|
||||
}
|
||||
var sendBps, receiveBps uint64
|
||||
if len(options.Up) > 0 {
|
||||
sendBps, err = humanize.ParseBytes(options.Up)
|
||||
if err != nil {
|
||||
return nil, E.Cause(err, "invalid up speed format: ", options.Up)
|
||||
}
|
||||
} else {
|
||||
sendBps = uint64(options.UpMbps) * hysteria.MbpsToBps
|
||||
}
|
||||
if len(options.Down) > 0 {
|
||||
receiveBps, err = humanize.ParseBytes(options.Down)
|
||||
if err != nil {
|
||||
return nil, E.Cause(err, "invalid down speed format: ", options.Down)
|
||||
}
|
||||
} else {
|
||||
receiveBps = uint64(options.DownMbps) * hysteria.MbpsToBps
|
||||
}
|
||||
var udpTimeout time.Duration
|
||||
if options.UDPTimeout != 0 {
|
||||
udpTimeout = time.Duration(options.UDPTimeout)
|
||||
} else {
|
||||
udpTimeout = C.UDPTimeout
|
||||
}
|
||||
service, err := hysteria.NewService[int](hysteria.ServiceOptions{
|
||||
Context: ctx,
|
||||
Logger: logger,
|
||||
SendBPS: sendBps,
|
||||
ReceiveBPS: receiveBps,
|
||||
XPlusPassword: options.Obfs,
|
||||
TLSConfig: tlsConfig,
|
||||
UDPTimeout: udpTimeout,
|
||||
Handler: adapter.NewUpstreamHandler(adapter.InboundContext{}, inbound.newConnection, inbound.newPacketConnection, nil),
|
||||
|
||||
// Legacy options
|
||||
|
||||
ConnReceiveWindow: options.ReceiveWindowConn,
|
||||
StreamReceiveWindow: options.ReceiveWindowClient,
|
||||
MaxIncomingStreams: int64(options.MaxConnClient),
|
||||
DisableMTUDiscovery: options.DisableMTUDiscovery,
|
||||
})
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
userList := make([]int, 0, len(options.Users))
|
||||
userNameList := make([]string, 0, len(options.Users))
|
||||
userPasswordList := make([]string, 0, len(options.Users))
|
||||
for index, user := range options.Users {
|
||||
userList = append(userList, index)
|
||||
userNameList = append(userNameList, user.Name)
|
||||
var password string
|
||||
if user.AuthString != "" {
|
||||
password = user.AuthString
|
||||
} else {
|
||||
password = string(user.Auth)
|
||||
}
|
||||
userPasswordList = append(userPasswordList, password)
|
||||
}
|
||||
service.UpdateUsers(userList, userPasswordList)
|
||||
inbound.service = service
|
||||
inbound.userNameList = userNameList
|
||||
return inbound, nil
|
||||
}
|
||||
|
||||
func (h *Inbound) newConnection(ctx context.Context, conn net.Conn, metadata adapter.InboundContext) error {
|
||||
ctx = log.ContextWithNewID(ctx)
|
||||
metadata.Inbound = h.Tag()
|
||||
metadata.InboundType = h.Type()
|
||||
metadata.InboundDetour = h.listener.ListenOptions().Detour
|
||||
metadata.InboundOptions = h.listener.ListenOptions().InboundOptions
|
||||
h.logger.InfoContext(ctx, "inbound connection from ", metadata.Source)
|
||||
userID, _ := auth.UserFromContext[int](ctx)
|
||||
if userName := h.userNameList[userID]; userName != "" {
|
||||
metadata.User = userName
|
||||
h.logger.InfoContext(ctx, "[", userName, "] inbound connection to ", metadata.Destination)
|
||||
} else {
|
||||
h.logger.InfoContext(ctx, "inbound connection to ", metadata.Destination)
|
||||
}
|
||||
return h.router.RouteConnection(ctx, conn, metadata)
|
||||
}
|
||||
|
||||
func (h *Inbound) newPacketConnection(ctx context.Context, conn N.PacketConn, metadata adapter.InboundContext) error {
|
||||
ctx = log.ContextWithNewID(ctx)
|
||||
metadata.Inbound = h.Tag()
|
||||
metadata.InboundType = h.Type()
|
||||
metadata.InboundDetour = h.listener.ListenOptions().Detour
|
||||
metadata.InboundOptions = h.listener.ListenOptions().InboundOptions
|
||||
metadata.OriginDestination = h.listener.UDPAddr()
|
||||
h.logger.InfoContext(ctx, "inbound packet connection from ", metadata.Source)
|
||||
userID, _ := auth.UserFromContext[int](ctx)
|
||||
if userName := h.userNameList[userID]; userName != "" {
|
||||
metadata.User = userName
|
||||
h.logger.InfoContext(ctx, "[", userName, "] inbound packet connection to ", metadata.Destination)
|
||||
} else {
|
||||
h.logger.InfoContext(ctx, "inbound packet connection to ", metadata.Destination)
|
||||
}
|
||||
return h.router.RoutePacketConnection(ctx, conn, metadata)
|
||||
}
|
||||
|
||||
func (h *Inbound) Start() error {
|
||||
if h.tlsConfig != nil {
|
||||
err := h.tlsConfig.Start()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
packetConn, err := h.listener.ListenUDP()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
return h.service.Start(packetConn)
|
||||
}
|
||||
|
||||
func (h *Inbound) Close() error {
|
||||
return common.Close(
|
||||
&h.listener,
|
||||
h.tlsConfig,
|
||||
common.PtrOrNil(h.service),
|
||||
)
|
||||
}
|
131
protocol/hysteria/outbound.go
Normal file
131
protocol/hysteria/outbound.go
Normal file
|
@ -0,0 +1,131 @@
|
|||
package hysteria
|
||||
|
||||
import (
|
||||
"context"
|
||||
"net"
|
||||
"os"
|
||||
|
||||
"github.com/sagernet/sing-box/adapter"
|
||||
"github.com/sagernet/sing-box/adapter/outbound"
|
||||
"github.com/sagernet/sing-box/common/dialer"
|
||||
"github.com/sagernet/sing-box/common/humanize"
|
||||
"github.com/sagernet/sing-box/common/tls"
|
||||
C "github.com/sagernet/sing-box/constant"
|
||||
"github.com/sagernet/sing-box/log"
|
||||
"github.com/sagernet/sing-box/option"
|
||||
"github.com/sagernet/sing-box/protocol/tuic"
|
||||
"github.com/sagernet/sing-quic/hysteria"
|
||||
"github.com/sagernet/sing/common"
|
||||
"github.com/sagernet/sing/common/bufio"
|
||||
E "github.com/sagernet/sing/common/exceptions"
|
||||
"github.com/sagernet/sing/common/logger"
|
||||
M "github.com/sagernet/sing/common/metadata"
|
||||
N "github.com/sagernet/sing/common/network"
|
||||
)
|
||||
|
||||
func RegisterOutbound(registry *outbound.Registry) {
|
||||
outbound.Register[option.HysteriaOutboundOptions](registry, C.TypeHysteria, NewOutbound)
|
||||
}
|
||||
|
||||
var (
|
||||
_ adapter.Outbound = (*tuic.Outbound)(nil)
|
||||
_ adapter.InterfaceUpdateListener = (*tuic.Outbound)(nil)
|
||||
)
|
||||
|
||||
type Outbound struct {
|
||||
outbound.Adapter
|
||||
logger logger.ContextLogger
|
||||
client *hysteria.Client
|
||||
}
|
||||
|
||||
func NewOutbound(ctx context.Context, router adapter.Router, logger log.ContextLogger, tag string, options option.HysteriaOutboundOptions) (adapter.Outbound, error) {
|
||||
options.UDPFragmentDefault = true
|
||||
if options.TLS == nil || !options.TLS.Enabled {
|
||||
return nil, C.ErrTLSRequired
|
||||
}
|
||||
tlsConfig, err := tls.NewClient(ctx, options.Server, common.PtrValueOrDefault(options.TLS))
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
outboundDialer, err := dialer.New(router, options.DialerOptions)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
networkList := options.Network.Build()
|
||||
var password string
|
||||
if options.AuthString != "" {
|
||||
password = options.AuthString
|
||||
} else {
|
||||
password = string(options.Auth)
|
||||
}
|
||||
var sendBps, receiveBps uint64
|
||||
if len(options.Up) > 0 {
|
||||
sendBps, err = humanize.ParseBytes(options.Up)
|
||||
if err != nil {
|
||||
return nil, E.Cause(err, "invalid up speed format: ", options.Up)
|
||||
}
|
||||
} else {
|
||||
sendBps = uint64(options.UpMbps) * hysteria.MbpsToBps
|
||||
}
|
||||
if len(options.Down) > 0 {
|
||||
receiveBps, err = humanize.ParseBytes(options.Down)
|
||||
if err != nil {
|
||||
return nil, E.Cause(err, "invalid down speed format: ", options.Down)
|
||||
}
|
||||
} else {
|
||||
receiveBps = uint64(options.DownMbps) * hysteria.MbpsToBps
|
||||
}
|
||||
client, err := hysteria.NewClient(hysteria.ClientOptions{
|
||||
Context: ctx,
|
||||
Dialer: outboundDialer,
|
||||
Logger: logger,
|
||||
ServerAddress: options.ServerOptions.Build(),
|
||||
SendBPS: sendBps,
|
||||
ReceiveBPS: receiveBps,
|
||||
XPlusPassword: options.Obfs,
|
||||
Password: password,
|
||||
TLSConfig: tlsConfig,
|
||||
UDPDisabled: !common.Contains(networkList, N.NetworkUDP),
|
||||
|
||||
ConnReceiveWindow: options.ReceiveWindowConn,
|
||||
StreamReceiveWindow: options.ReceiveWindow,
|
||||
DisableMTUDiscovery: options.DisableMTUDiscovery,
|
||||
})
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return &Outbound{
|
||||
Adapter: outbound.NewAdapterWithDialerOptions(C.TypeHysteria, networkList, tag, options.DialerOptions),
|
||||
logger: logger,
|
||||
client: client,
|
||||
}, nil
|
||||
}
|
||||
|
||||
func (h *Outbound) DialContext(ctx context.Context, network string, destination M.Socksaddr) (net.Conn, error) {
|
||||
switch N.NetworkName(network) {
|
||||
case N.NetworkTCP:
|
||||
h.logger.InfoContext(ctx, "outbound connection to ", destination)
|
||||
return h.client.DialConn(ctx, destination)
|
||||
case N.NetworkUDP:
|
||||
conn, err := h.ListenPacket(ctx, destination)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return bufio.NewBindPacketConn(conn, destination), nil
|
||||
default:
|
||||
return nil, E.New("unsupported network: ", network)
|
||||
}
|
||||
}
|
||||
|
||||
func (h *Outbound) ListenPacket(ctx context.Context, destination M.Socksaddr) (net.PacketConn, error) {
|
||||
h.logger.InfoContext(ctx, "outbound packet connection to ", destination)
|
||||
return h.client.ListenPacket(ctx, destination)
|
||||
}
|
||||
|
||||
func (h *Outbound) InterfaceUpdated() {
|
||||
h.client.CloseWithError(E.New("network changed"))
|
||||
}
|
||||
|
||||
func (h *Outbound) Close() error {
|
||||
return h.client.CloseWithError(os.ErrClosed)
|
||||
}
|
186
protocol/hysteria2/inbound.go
Normal file
186
protocol/hysteria2/inbound.go
Normal file
|
@ -0,0 +1,186 @@
|
|||
package hysteria2
|
||||
|
||||
import (
|
||||
"context"
|
||||
"net"
|
||||
"net/http"
|
||||
"net/http/httputil"
|
||||
"net/url"
|
||||
"time"
|
||||
|
||||
"github.com/sagernet/sing-box/adapter"
|
||||
"github.com/sagernet/sing-box/adapter/inbound"
|
||||
"github.com/sagernet/sing-box/common/listener"
|
||||
"github.com/sagernet/sing-box/common/tls"
|
||||
C "github.com/sagernet/sing-box/constant"
|
||||
"github.com/sagernet/sing-box/log"
|
||||
"github.com/sagernet/sing-box/option"
|
||||
"github.com/sagernet/sing-quic/hysteria"
|
||||
"github.com/sagernet/sing-quic/hysteria2"
|
||||
"github.com/sagernet/sing/common"
|
||||
"github.com/sagernet/sing/common/auth"
|
||||
E "github.com/sagernet/sing/common/exceptions"
|
||||
N "github.com/sagernet/sing/common/network"
|
||||
)
|
||||
|
||||
func RegisterInbound(registry *inbound.Registry) {
|
||||
inbound.Register[option.Hysteria2InboundOptions](registry, C.TypeHysteria2, NewInbound)
|
||||
}
|
||||
|
||||
type Inbound struct {
|
||||
inbound.Adapter
|
||||
router adapter.Router
|
||||
logger log.ContextLogger
|
||||
listener *listener.Listener
|
||||
tlsConfig tls.ServerConfig
|
||||
service *hysteria2.Service[int]
|
||||
userNameList []string
|
||||
}
|
||||
|
||||
func NewInbound(ctx context.Context, router adapter.Router, logger log.ContextLogger, tag string, options option.Hysteria2InboundOptions) (adapter.Inbound, error) {
|
||||
options.UDPFragmentDefault = true
|
||||
if options.TLS == nil || !options.TLS.Enabled {
|
||||
return nil, C.ErrTLSRequired
|
||||
}
|
||||
tlsConfig, err := tls.NewServer(ctx, logger, common.PtrValueOrDefault(options.TLS))
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
var salamanderPassword string
|
||||
if options.Obfs != nil {
|
||||
if options.Obfs.Password == "" {
|
||||
return nil, E.New("missing obfs password")
|
||||
}
|
||||
switch options.Obfs.Type {
|
||||
case hysteria2.ObfsTypeSalamander:
|
||||
salamanderPassword = options.Obfs.Password
|
||||
default:
|
||||
return nil, E.New("unknown obfs type: ", options.Obfs.Type)
|
||||
}
|
||||
}
|
||||
var masqueradeHandler http.Handler
|
||||
if options.Masquerade != "" {
|
||||
masqueradeURL, err := url.Parse(options.Masquerade)
|
||||
if err != nil {
|
||||
return nil, E.Cause(err, "parse masquerade URL")
|
||||
}
|
||||
switch masqueradeURL.Scheme {
|
||||
case "file":
|
||||
masqueradeHandler = http.FileServer(http.Dir(masqueradeURL.Path))
|
||||
case "http", "https":
|
||||
masqueradeHandler = &httputil.ReverseProxy{
|
||||
Rewrite: func(r *httputil.ProxyRequest) {
|
||||
r.SetURL(masqueradeURL)
|
||||
r.Out.Host = r.In.Host
|
||||
},
|
||||
ErrorHandler: func(w http.ResponseWriter, r *http.Request, err error) {
|
||||
w.WriteHeader(http.StatusBadGateway)
|
||||
},
|
||||
}
|
||||
default:
|
||||
return nil, E.New("unknown masquerade URL scheme: ", masqueradeURL.Scheme)
|
||||
}
|
||||
}
|
||||
inbound := &Inbound{
|
||||
Adapter: inbound.NewAdapter(C.TypeHysteria2, tag),
|
||||
router: router,
|
||||
logger: logger,
|
||||
listener: listener.New(listener.Options{
|
||||
Context: ctx,
|
||||
Logger: logger,
|
||||
Listen: options.ListenOptions,
|
||||
}),
|
||||
tlsConfig: tlsConfig,
|
||||
}
|
||||
var udpTimeout time.Duration
|
||||
if options.UDPTimeout != 0 {
|
||||
udpTimeout = time.Duration(options.UDPTimeout)
|
||||
} else {
|
||||
udpTimeout = C.UDPTimeout
|
||||
}
|
||||
service, err := hysteria2.NewService[int](hysteria2.ServiceOptions{
|
||||
Context: ctx,
|
||||
Logger: logger,
|
||||
BrutalDebug: options.BrutalDebug,
|
||||
SendBPS: uint64(options.UpMbps * hysteria.MbpsToBps),
|
||||
ReceiveBPS: uint64(options.DownMbps * hysteria.MbpsToBps),
|
||||
SalamanderPassword: salamanderPassword,
|
||||
TLSConfig: tlsConfig,
|
||||
IgnoreClientBandwidth: options.IgnoreClientBandwidth,
|
||||
UDPTimeout: udpTimeout,
|
||||
Handler: adapter.NewUpstreamHandler(adapter.InboundContext{}, inbound.newConnection, inbound.newPacketConnection, nil),
|
||||
MasqueradeHandler: masqueradeHandler,
|
||||
})
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
userList := make([]int, 0, len(options.Users))
|
||||
userNameList := make([]string, 0, len(options.Users))
|
||||
userPasswordList := make([]string, 0, len(options.Users))
|
||||
for index, user := range options.Users {
|
||||
userList = append(userList, index)
|
||||
userNameList = append(userNameList, user.Name)
|
||||
userPasswordList = append(userPasswordList, user.Password)
|
||||
}
|
||||
service.UpdateUsers(userList, userPasswordList)
|
||||
inbound.service = service
|
||||
inbound.userNameList = userNameList
|
||||
return inbound, nil
|
||||
}
|
||||
|
||||
func (h *Inbound) newConnection(ctx context.Context, conn net.Conn, metadata adapter.InboundContext) error {
|
||||
ctx = log.ContextWithNewID(ctx)
|
||||
metadata.Inbound = h.Tag()
|
||||
metadata.InboundType = h.Type()
|
||||
metadata.InboundDetour = h.listener.ListenOptions().Detour
|
||||
metadata.InboundOptions = h.listener.ListenOptions().InboundOptions
|
||||
h.logger.InfoContext(ctx, "inbound connection from ", metadata.Source)
|
||||
userID, _ := auth.UserFromContext[int](ctx)
|
||||
if userName := h.userNameList[userID]; userName != "" {
|
||||
metadata.User = userName
|
||||
h.logger.InfoContext(ctx, "[", userName, "] inbound connection to ", metadata.Destination)
|
||||
} else {
|
||||
h.logger.InfoContext(ctx, "inbound connection to ", metadata.Destination)
|
||||
}
|
||||
return h.router.RouteConnection(ctx, conn, metadata)
|
||||
}
|
||||
|
||||
func (h *Inbound) newPacketConnection(ctx context.Context, conn N.PacketConn, metadata adapter.InboundContext) error {
|
||||
ctx = log.ContextWithNewID(ctx)
|
||||
metadata.Inbound = h.Tag()
|
||||
metadata.InboundType = h.Type()
|
||||
metadata.InboundDetour = h.listener.ListenOptions().Detour
|
||||
metadata.InboundOptions = h.listener.ListenOptions().InboundOptions
|
||||
metadata.OriginDestination = h.listener.UDPAddr()
|
||||
h.logger.InfoContext(ctx, "inbound packet connection from ", metadata.Source)
|
||||
userID, _ := auth.UserFromContext[int](ctx)
|
||||
if userName := h.userNameList[userID]; userName != "" {
|
||||
metadata.User = userName
|
||||
h.logger.InfoContext(ctx, "[", userName, "] inbound packet connection to ", metadata.Destination)
|
||||
} else {
|
||||
h.logger.InfoContext(ctx, "inbound packet connection to ", metadata.Destination)
|
||||
}
|
||||
return h.router.RoutePacketConnection(ctx, conn, metadata)
|
||||
}
|
||||
|
||||
func (h *Inbound) Start() error {
|
||||
if h.tlsConfig != nil {
|
||||
err := h.tlsConfig.Start()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
packetConn, err := h.listener.ListenUDP()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
return h.service.Start(packetConn)
|
||||
}
|
||||
|
||||
func (h *Inbound) Close() error {
|
||||
return common.Close(
|
||||
&h.listener,
|
||||
h.tlsConfig,
|
||||
common.PtrOrNil(h.service),
|
||||
)
|
||||
}
|
117
protocol/hysteria2/outbound.go
Normal file
117
protocol/hysteria2/outbound.go
Normal file
|
@ -0,0 +1,117 @@
|
|||
package hysteria2
|
||||
|
||||
import (
|
||||
"context"
|
||||
"net"
|
||||
"os"
|
||||
|
||||
"github.com/sagernet/sing-box/adapter"
|
||||
"github.com/sagernet/sing-box/adapter/outbound"
|
||||
"github.com/sagernet/sing-box/common/dialer"
|
||||
"github.com/sagernet/sing-box/common/tls"
|
||||
C "github.com/sagernet/sing-box/constant"
|
||||
"github.com/sagernet/sing-box/log"
|
||||
"github.com/sagernet/sing-box/option"
|
||||
"github.com/sagernet/sing-box/protocol/tuic"
|
||||
"github.com/sagernet/sing-quic/hysteria"
|
||||
"github.com/sagernet/sing-quic/hysteria2"
|
||||
"github.com/sagernet/sing/common"
|
||||
"github.com/sagernet/sing/common/bufio"
|
||||
E "github.com/sagernet/sing/common/exceptions"
|
||||
"github.com/sagernet/sing/common/logger"
|
||||
M "github.com/sagernet/sing/common/metadata"
|
||||
N "github.com/sagernet/sing/common/network"
|
||||
)
|
||||
|
||||
func RegisterOutbound(registry *outbound.Registry) {
|
||||
outbound.Register[option.Hysteria2OutboundOptions](registry, C.TypeHysteria2, NewOutbound)
|
||||
}
|
||||
|
||||
var (
|
||||
_ adapter.Outbound = (*tuic.Outbound)(nil)
|
||||
_ adapter.InterfaceUpdateListener = (*tuic.Outbound)(nil)
|
||||
)
|
||||
|
||||
type Outbound struct {
|
||||
outbound.Adapter
|
||||
logger logger.ContextLogger
|
||||
client *hysteria2.Client
|
||||
}
|
||||
|
||||
func NewOutbound(ctx context.Context, router adapter.Router, logger log.ContextLogger, tag string, options option.Hysteria2OutboundOptions) (adapter.Outbound, error) {
|
||||
options.UDPFragmentDefault = true
|
||||
if options.TLS == nil || !options.TLS.Enabled {
|
||||
return nil, C.ErrTLSRequired
|
||||
}
|
||||
tlsConfig, err := tls.NewClient(ctx, options.Server, common.PtrValueOrDefault(options.TLS))
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
var salamanderPassword string
|
||||
if options.Obfs != nil {
|
||||
if options.Obfs.Password == "" {
|
||||
return nil, E.New("missing obfs password")
|
||||
}
|
||||
switch options.Obfs.Type {
|
||||
case hysteria2.ObfsTypeSalamander:
|
||||
salamanderPassword = options.Obfs.Password
|
||||
default:
|
||||
return nil, E.New("unknown obfs type: ", options.Obfs.Type)
|
||||
}
|
||||
}
|
||||
outboundDialer, err := dialer.New(router, options.DialerOptions)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
networkList := options.Network.Build()
|
||||
client, err := hysteria2.NewClient(hysteria2.ClientOptions{
|
||||
Context: ctx,
|
||||
Dialer: outboundDialer,
|
||||
Logger: logger,
|
||||
BrutalDebug: options.BrutalDebug,
|
||||
ServerAddress: options.ServerOptions.Build(),
|
||||
SendBPS: uint64(options.UpMbps * hysteria.MbpsToBps),
|
||||
ReceiveBPS: uint64(options.DownMbps * hysteria.MbpsToBps),
|
||||
SalamanderPassword: salamanderPassword,
|
||||
Password: options.Password,
|
||||
TLSConfig: tlsConfig,
|
||||
UDPDisabled: !common.Contains(networkList, N.NetworkUDP),
|
||||
})
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return &Outbound{
|
||||
Adapter: outbound.NewAdapterWithDialerOptions(C.TypeHysteria2, networkList, tag, options.DialerOptions),
|
||||
logger: logger,
|
||||
client: client,
|
||||
}, nil
|
||||
}
|
||||
|
||||
func (h *Outbound) DialContext(ctx context.Context, network string, destination M.Socksaddr) (net.Conn, error) {
|
||||
switch N.NetworkName(network) {
|
||||
case N.NetworkTCP:
|
||||
h.logger.InfoContext(ctx, "outbound connection to ", destination)
|
||||
return h.client.DialConn(ctx, destination)
|
||||
case N.NetworkUDP:
|
||||
conn, err := h.ListenPacket(ctx, destination)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return bufio.NewBindPacketConn(conn, destination), nil
|
||||
default:
|
||||
return nil, E.New("unsupported network: ", network)
|
||||
}
|
||||
}
|
||||
|
||||
func (h *Outbound) ListenPacket(ctx context.Context, destination M.Socksaddr) (net.PacketConn, error) {
|
||||
h.logger.InfoContext(ctx, "outbound packet connection to ", destination)
|
||||
return h.client.ListenPacket(ctx)
|
||||
}
|
||||
|
||||
func (h *Outbound) InterfaceUpdated() {
|
||||
h.client.CloseWithError(E.New("network changed"))
|
||||
}
|
||||
|
||||
func (h *Outbound) Close() error {
|
||||
return h.client.CloseWithError(os.ErrClosed)
|
||||
}
|
109
protocol/mixed/inbound.go
Normal file
109
protocol/mixed/inbound.go
Normal file
|
@ -0,0 +1,109 @@
|
|||
package mixed
|
||||
|
||||
import (
|
||||
std_bufio "bufio"
|
||||
"context"
|
||||
"net"
|
||||
|
||||
"github.com/sagernet/sing-box/adapter"
|
||||
"github.com/sagernet/sing-box/adapter/inbound"
|
||||
"github.com/sagernet/sing-box/common/listener"
|
||||
"github.com/sagernet/sing-box/common/uot"
|
||||
C "github.com/sagernet/sing-box/constant"
|
||||
"github.com/sagernet/sing-box/log"
|
||||
"github.com/sagernet/sing-box/option"
|
||||
"github.com/sagernet/sing/common/auth"
|
||||
E "github.com/sagernet/sing/common/exceptions"
|
||||
N "github.com/sagernet/sing/common/network"
|
||||
"github.com/sagernet/sing/protocol/http"
|
||||
"github.com/sagernet/sing/protocol/socks"
|
||||
"github.com/sagernet/sing/protocol/socks/socks4"
|
||||
"github.com/sagernet/sing/protocol/socks/socks5"
|
||||
)
|
||||
|
||||
func RegisterInbound(registry *inbound.Registry) {
|
||||
inbound.Register[option.HTTPMixedInboundOptions](registry, C.TypeMixed, NewInbound)
|
||||
}
|
||||
|
||||
var _ adapter.TCPInjectableInbound = (*Inbound)(nil)
|
||||
|
||||
type Inbound struct {
|
||||
inbound.Adapter
|
||||
router adapter.ConnectionRouterEx
|
||||
logger log.ContextLogger
|
||||
listener *listener.Listener
|
||||
authenticator *auth.Authenticator
|
||||
}
|
||||
|
||||
func NewInbound(ctx context.Context, router adapter.Router, logger log.ContextLogger, tag string, options option.HTTPMixedInboundOptions) (adapter.Inbound, error) {
|
||||
inbound := &Inbound{
|
||||
Adapter: inbound.NewAdapter(C.TypeMixed, tag),
|
||||
router: uot.NewRouter(router, logger),
|
||||
logger: logger,
|
||||
authenticator: auth.NewAuthenticator(options.Users),
|
||||
}
|
||||
inbound.listener = listener.New(listener.Options{
|
||||
Context: ctx,
|
||||
Logger: logger,
|
||||
Network: []string{N.NetworkTCP},
|
||||
Listen: options.ListenOptions,
|
||||
ConnectionHandler: inbound,
|
||||
SetSystemProxy: options.SetSystemProxy,
|
||||
SystemProxySOCKS: true,
|
||||
})
|
||||
return inbound, nil
|
||||
}
|
||||
|
||||
func (h *Inbound) Start() error {
|
||||
return h.listener.Start()
|
||||
}
|
||||
|
||||
func (h *Inbound) Close() error {
|
||||
return h.listener.Close()
|
||||
}
|
||||
|
||||
func (h *Inbound) NewConnectionEx(ctx context.Context, conn net.Conn, metadata adapter.InboundContext, onClose N.CloseHandlerFunc) {
|
||||
err := h.newConnection(ctx, conn, metadata, onClose)
|
||||
N.CloseOnHandshakeFailure(conn, onClose, err)
|
||||
if err != nil {
|
||||
h.logger.ErrorContext(ctx, E.Cause(err, "process connection from ", metadata.Source))
|
||||
}
|
||||
}
|
||||
|
||||
func (h *Inbound) newConnection(ctx context.Context, conn net.Conn, metadata adapter.InboundContext, onClose N.CloseHandlerFunc) error {
|
||||
reader := std_bufio.NewReader(conn)
|
||||
headerBytes, err := reader.Peek(1)
|
||||
if err != nil {
|
||||
return E.Cause(err, "peek first byte")
|
||||
}
|
||||
switch headerBytes[0] {
|
||||
case socks4.Version, socks5.Version:
|
||||
return socks.HandleConnectionEx(ctx, conn, reader, h.authenticator, nil, adapter.NewUpstreamHandlerEx(metadata, h.newUserConnection, h.streamUserPacketConnection), metadata.Source, metadata.Destination, onClose)
|
||||
default:
|
||||
return http.HandleConnectionEx(ctx, conn, reader, h.authenticator, nil, adapter.NewUpstreamHandlerEx(metadata, h.newUserConnection, h.streamUserPacketConnection), metadata.Source, onClose)
|
||||
}
|
||||
}
|
||||
|
||||
func (h *Inbound) newUserConnection(ctx context.Context, conn net.Conn, metadata adapter.InboundContext, onClose N.CloseHandlerFunc) {
|
||||
user, loaded := auth.UserFromContext[string](ctx)
|
||||
if !loaded {
|
||||
h.logger.InfoContext(ctx, "inbound connection to ", metadata.Destination)
|
||||
h.router.RouteConnectionEx(ctx, conn, metadata, onClose)
|
||||
return
|
||||
}
|
||||
metadata.User = user
|
||||
h.logger.InfoContext(ctx, "[", user, "] inbound connection to ", metadata.Destination)
|
||||
h.router.RouteConnectionEx(ctx, conn, metadata, onClose)
|
||||
}
|
||||
|
||||
func (h *Inbound) streamUserPacketConnection(ctx context.Context, conn N.PacketConn, metadata adapter.InboundContext, onClose N.CloseHandlerFunc) {
|
||||
user, loaded := auth.UserFromContext[string](ctx)
|
||||
if !loaded {
|
||||
h.logger.InfoContext(ctx, "inbound packet connection to ", metadata.Destination)
|
||||
h.router.RoutePacketConnectionEx(ctx, conn, metadata, onClose)
|
||||
return
|
||||
}
|
||||
metadata.User = user
|
||||
h.logger.InfoContext(ctx, "[", user, "] inbound packet connection to ", metadata.Destination)
|
||||
h.router.RoutePacketConnectionEx(ctx, conn, metadata, onClose)
|
||||
}
|
248
protocol/naive/inbound.go
Normal file
248
protocol/naive/inbound.go
Normal file
|
@ -0,0 +1,248 @@
|
|||
package naive
|
||||
|
||||
import (
|
||||
"context"
|
||||
"io"
|
||||
"math/rand"
|
||||
"net"
|
||||
"net/http"
|
||||
|
||||
"github.com/sagernet/sing-box/adapter"
|
||||
"github.com/sagernet/sing-box/adapter/inbound"
|
||||
"github.com/sagernet/sing-box/common/listener"
|
||||
"github.com/sagernet/sing-box/common/tls"
|
||||
"github.com/sagernet/sing-box/common/uot"
|
||||
C "github.com/sagernet/sing-box/constant"
|
||||
"github.com/sagernet/sing-box/log"
|
||||
"github.com/sagernet/sing-box/option"
|
||||
"github.com/sagernet/sing-box/transport/v2rayhttp"
|
||||
"github.com/sagernet/sing/common"
|
||||
"github.com/sagernet/sing/common/auth"
|
||||
E "github.com/sagernet/sing/common/exceptions"
|
||||
"github.com/sagernet/sing/common/logger"
|
||||
M "github.com/sagernet/sing/common/metadata"
|
||||
N "github.com/sagernet/sing/common/network"
|
||||
sHttp "github.com/sagernet/sing/protocol/http"
|
||||
)
|
||||
|
||||
var ConfigureHTTP3ListenerFunc func(listener *listener.Listener, handler http.Handler, tlsConfig tls.ServerConfig, logger logger.Logger) (io.Closer, error)
|
||||
|
||||
func RegisterInbound(registry *inbound.Registry) {
|
||||
inbound.Register[option.NaiveInboundOptions](registry, C.TypeNaive, NewInbound)
|
||||
}
|
||||
|
||||
type Inbound struct {
|
||||
inbound.Adapter
|
||||
ctx context.Context
|
||||
router adapter.ConnectionRouterEx
|
||||
logger logger.ContextLogger
|
||||
listener *listener.Listener
|
||||
network []string
|
||||
networkIsDefault bool
|
||||
authenticator *auth.Authenticator
|
||||
tlsConfig tls.ServerConfig
|
||||
httpServer *http.Server
|
||||
h3Server io.Closer
|
||||
}
|
||||
|
||||
func NewInbound(ctx context.Context, router adapter.Router, logger log.ContextLogger, tag string, options option.NaiveInboundOptions) (adapter.Inbound, error) {
|
||||
inbound := &Inbound{
|
||||
Adapter: inbound.NewAdapter(C.TypeNaive, tag),
|
||||
ctx: ctx,
|
||||
router: uot.NewRouter(router, logger),
|
||||
logger: logger,
|
||||
listener: listener.New(listener.Options{
|
||||
Context: ctx,
|
||||
Logger: logger,
|
||||
Listen: options.ListenOptions,
|
||||
}),
|
||||
networkIsDefault: options.Network == "",
|
||||
network: options.Network.Build(),
|
||||
authenticator: auth.NewAuthenticator(options.Users),
|
||||
}
|
||||
if common.Contains(inbound.network, N.NetworkUDP) {
|
||||
if options.TLS == nil || !options.TLS.Enabled {
|
||||
return nil, E.New("TLS is required for QUIC server")
|
||||
}
|
||||
}
|
||||
if len(options.Users) == 0 {
|
||||
return nil, E.New("missing users")
|
||||
}
|
||||
if options.TLS != nil {
|
||||
tlsConfig, err := tls.NewServer(ctx, logger, common.PtrValueOrDefault(options.TLS))
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
inbound.tlsConfig = tlsConfig
|
||||
}
|
||||
return inbound, nil
|
||||
}
|
||||
|
||||
func (n *Inbound) Start() error {
|
||||
var tlsConfig *tls.STDConfig
|
||||
if n.tlsConfig != nil {
|
||||
err := n.tlsConfig.Start()
|
||||
if err != nil {
|
||||
return E.Cause(err, "create TLS config")
|
||||
}
|
||||
tlsConfig, err = n.tlsConfig.Config()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
if common.Contains(n.network, N.NetworkTCP) {
|
||||
tcpListener, err := n.listener.ListenTCP()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
n.httpServer = &http.Server{
|
||||
Handler: n,
|
||||
TLSConfig: tlsConfig,
|
||||
BaseContext: func(listener net.Listener) context.Context {
|
||||
return n.ctx
|
||||
},
|
||||
}
|
||||
go func() {
|
||||
var sErr error
|
||||
if tlsConfig != nil {
|
||||
sErr = n.httpServer.ServeTLS(tcpListener, "", "")
|
||||
} else {
|
||||
sErr = n.httpServer.Serve(tcpListener)
|
||||
}
|
||||
if sErr != nil && !E.IsClosedOrCanceled(sErr) {
|
||||
n.logger.Error("http server serve error: ", sErr)
|
||||
}
|
||||
}()
|
||||
}
|
||||
|
||||
if common.Contains(n.network, N.NetworkUDP) {
|
||||
http3Server, err := ConfigureHTTP3ListenerFunc(n.listener, n, n.tlsConfig, n.logger)
|
||||
if err == nil {
|
||||
n.h3Server = http3Server
|
||||
} else if len(n.network) > 1 {
|
||||
n.logger.Warn(E.Cause(err, "naive http3 disabled"))
|
||||
} else {
|
||||
return err
|
||||
}
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (n *Inbound) Close() error {
|
||||
return common.Close(
|
||||
&n.listener,
|
||||
common.PtrOrNil(n.httpServer),
|
||||
n.h3Server,
|
||||
n.tlsConfig,
|
||||
)
|
||||
}
|
||||
|
||||
func (n *Inbound) ServeHTTP(writer http.ResponseWriter, request *http.Request) {
|
||||
ctx := log.ContextWithNewID(request.Context())
|
||||
if request.Method != "CONNECT" {
|
||||
rejectHTTP(writer, http.StatusBadRequest)
|
||||
n.badRequest(ctx, request, E.New("not CONNECT request"))
|
||||
return
|
||||
} else if request.Header.Get("Padding") == "" {
|
||||
rejectHTTP(writer, http.StatusBadRequest)
|
||||
n.badRequest(ctx, request, E.New("missing naive padding"))
|
||||
return
|
||||
}
|
||||
userName, password, authOk := sHttp.ParseBasicAuth(request.Header.Get("Proxy-Authorization"))
|
||||
if authOk {
|
||||
authOk = n.authenticator.Verify(userName, password)
|
||||
}
|
||||
if !authOk {
|
||||
rejectHTTP(writer, http.StatusProxyAuthRequired)
|
||||
n.badRequest(ctx, request, E.New("authorization failed"))
|
||||
return
|
||||
}
|
||||
writer.Header().Set("Padding", generateNaivePaddingHeader())
|
||||
writer.WriteHeader(http.StatusOK)
|
||||
writer.(http.Flusher).Flush()
|
||||
|
||||
hostPort := request.URL.Host
|
||||
if hostPort == "" {
|
||||
hostPort = request.Host
|
||||
}
|
||||
source := sHttp.SourceAddress(request)
|
||||
destination := M.ParseSocksaddr(hostPort)
|
||||
|
||||
if hijacker, isHijacker := writer.(http.Hijacker); isHijacker {
|
||||
conn, _, err := hijacker.Hijack()
|
||||
if err != nil {
|
||||
n.badRequest(ctx, request, E.New("hijack failed"))
|
||||
return
|
||||
}
|
||||
n.newConnection(ctx, false, &naiveH1Conn{Conn: conn}, userName, source, destination)
|
||||
} else {
|
||||
n.newConnection(ctx, true, &naiveH2Conn{reader: request.Body, writer: writer, flusher: writer.(http.Flusher)}, userName, source, destination)
|
||||
}
|
||||
}
|
||||
|
||||
func (n *Inbound) newConnection(ctx context.Context, waitForClose bool, conn net.Conn, userName string, source M.Socksaddr, destination M.Socksaddr) {
|
||||
if userName != "" {
|
||||
n.logger.InfoContext(ctx, "[", userName, "] inbound connection from ", source)
|
||||
n.logger.InfoContext(ctx, "[", userName, "] inbound connection to ", destination)
|
||||
} else {
|
||||
n.logger.InfoContext(ctx, "inbound connection from ", source)
|
||||
n.logger.InfoContext(ctx, "inbound connection to ", destination)
|
||||
}
|
||||
var metadata adapter.InboundContext
|
||||
metadata.Inbound = n.Tag()
|
||||
metadata.InboundType = n.Type()
|
||||
metadata.InboundDetour = n.listener.ListenOptions().Detour
|
||||
metadata.InboundOptions = n.listener.ListenOptions().InboundOptions
|
||||
metadata.Source = source
|
||||
metadata.Destination = destination
|
||||
metadata.OriginDestination = M.SocksaddrFromNet(conn.LocalAddr()).Unwrap()
|
||||
metadata.User = userName
|
||||
if !waitForClose {
|
||||
n.router.RouteConnectionEx(ctx, conn, metadata, nil)
|
||||
} else {
|
||||
done := make(chan struct{})
|
||||
wrapper := v2rayhttp.NewHTTP2Wrapper(conn)
|
||||
n.router.RouteConnectionEx(ctx, conn, metadata, N.OnceClose(func(it error) {
|
||||
close(done)
|
||||
}))
|
||||
<-done
|
||||
wrapper.CloseWrapper()
|
||||
}
|
||||
}
|
||||
|
||||
func (n *Inbound) badRequest(ctx context.Context, request *http.Request, err error) {
|
||||
n.logger.ErrorContext(ctx, E.Cause(err, "process connection from ", request.RemoteAddr))
|
||||
}
|
||||
|
||||
func rejectHTTP(writer http.ResponseWriter, statusCode int) {
|
||||
hijacker, ok := writer.(http.Hijacker)
|
||||
if !ok {
|
||||
writer.WriteHeader(statusCode)
|
||||
return
|
||||
}
|
||||
conn, _, err := hijacker.Hijack()
|
||||
if err != nil {
|
||||
writer.WriteHeader(statusCode)
|
||||
return
|
||||
}
|
||||
if tcpConn, isTCP := common.Cast[*net.TCPConn](conn); isTCP {
|
||||
tcpConn.SetLinger(0)
|
||||
}
|
||||
conn.Close()
|
||||
}
|
||||
|
||||
func generateNaivePaddingHeader() string {
|
||||
paddingLen := rand.Intn(32) + 30
|
||||
padding := make([]byte, paddingLen)
|
||||
bits := rand.Uint64()
|
||||
for i := 0; i < 16; i++ {
|
||||
// Codes that won't be Huffman coded.
|
||||
padding[i] = "!#$()+<>?@[]^`{}"[bits&15]
|
||||
bits >>= 4
|
||||
}
|
||||
for i := 16; i < paddingLen; i++ {
|
||||
padding[i] = '~'
|
||||
}
|
||||
return string(padding)
|
||||
}
|
423
protocol/naive/inbound_conn.go
Normal file
423
protocol/naive/inbound_conn.go
Normal file
|
@ -0,0 +1,423 @@
|
|||
package naive
|
||||
|
||||
import (
|
||||
"encoding/binary"
|
||||
"io"
|
||||
"math/rand"
|
||||
"net"
|
||||
"net/http"
|
||||
"os"
|
||||
"strings"
|
||||
"time"
|
||||
|
||||
"github.com/sagernet/sing/common"
|
||||
"github.com/sagernet/sing/common/buf"
|
||||
M "github.com/sagernet/sing/common/metadata"
|
||||
"github.com/sagernet/sing/common/rw"
|
||||
)
|
||||
|
||||
const kFirstPaddings = 8
|
||||
|
||||
type naiveH1Conn struct {
|
||||
net.Conn
|
||||
readPadding int
|
||||
writePadding int
|
||||
readRemaining int
|
||||
paddingRemaining int
|
||||
}
|
||||
|
||||
func (c *naiveH1Conn) Read(p []byte) (n int, err error) {
|
||||
n, err = c.read(p)
|
||||
return n, wrapHttpError(err)
|
||||
}
|
||||
|
||||
func (c *naiveH1Conn) read(p []byte) (n int, err error) {
|
||||
if c.readRemaining > 0 {
|
||||
if len(p) > c.readRemaining {
|
||||
p = p[:c.readRemaining]
|
||||
}
|
||||
n, err = c.Conn.Read(p)
|
||||
if err != nil {
|
||||
return
|
||||
}
|
||||
c.readRemaining -= n
|
||||
return
|
||||
}
|
||||
if c.paddingRemaining > 0 {
|
||||
err = rw.SkipN(c.Conn, c.paddingRemaining)
|
||||
if err != nil {
|
||||
return
|
||||
}
|
||||
c.paddingRemaining = 0
|
||||
}
|
||||
if c.readPadding < kFirstPaddings {
|
||||
var paddingHdr []byte
|
||||
if len(p) >= 3 {
|
||||
paddingHdr = p[:3]
|
||||
} else {
|
||||
paddingHdr = make([]byte, 3)
|
||||
}
|
||||
_, err = io.ReadFull(c.Conn, paddingHdr)
|
||||
if err != nil {
|
||||
return
|
||||
}
|
||||
originalDataSize := int(binary.BigEndian.Uint16(paddingHdr[:2]))
|
||||
paddingSize := int(paddingHdr[2])
|
||||
if len(p) > originalDataSize {
|
||||
p = p[:originalDataSize]
|
||||
}
|
||||
n, err = c.Conn.Read(p)
|
||||
if err != nil {
|
||||
return
|
||||
}
|
||||
c.readPadding++
|
||||
c.readRemaining = originalDataSize - n
|
||||
c.paddingRemaining = paddingSize
|
||||
return
|
||||
}
|
||||
return c.Conn.Read(p)
|
||||
}
|
||||
|
||||
func (c *naiveH1Conn) Write(p []byte) (n int, err error) {
|
||||
for pLen := len(p); pLen > 0; {
|
||||
var data []byte
|
||||
if pLen > 65535 {
|
||||
data = p[:65535]
|
||||
p = p[65535:]
|
||||
pLen -= 65535
|
||||
} else {
|
||||
data = p
|
||||
pLen = 0
|
||||
}
|
||||
var writeN int
|
||||
writeN, err = c.write(data)
|
||||
n += writeN
|
||||
if err != nil {
|
||||
break
|
||||
}
|
||||
}
|
||||
return n, wrapHttpError(err)
|
||||
}
|
||||
|
||||
func (c *naiveH1Conn) write(p []byte) (n int, err error) {
|
||||
if c.writePadding < kFirstPaddings {
|
||||
paddingSize := rand.Intn(256)
|
||||
|
||||
buffer := buf.NewSize(3 + len(p) + paddingSize)
|
||||
defer buffer.Release()
|
||||
header := buffer.Extend(3)
|
||||
binary.BigEndian.PutUint16(header, uint16(len(p)))
|
||||
header[2] = byte(paddingSize)
|
||||
|
||||
common.Must1(buffer.Write(p))
|
||||
_, err = c.Conn.Write(buffer.Bytes())
|
||||
if err == nil {
|
||||
n = len(p)
|
||||
}
|
||||
c.writePadding++
|
||||
return
|
||||
}
|
||||
return c.Conn.Write(p)
|
||||
}
|
||||
|
||||
func (c *naiveH1Conn) FrontHeadroom() int {
|
||||
if c.writePadding < kFirstPaddings {
|
||||
return 3
|
||||
}
|
||||
return 0
|
||||
}
|
||||
|
||||
func (c *naiveH1Conn) RearHeadroom() int {
|
||||
if c.writePadding < kFirstPaddings {
|
||||
return 255
|
||||
}
|
||||
return 0
|
||||
}
|
||||
|
||||
func (c *naiveH1Conn) WriterMTU() int {
|
||||
if c.writePadding < kFirstPaddings {
|
||||
return 65535
|
||||
}
|
||||
return 0
|
||||
}
|
||||
|
||||
func (c *naiveH1Conn) WriteBuffer(buffer *buf.Buffer) error {
|
||||
defer buffer.Release()
|
||||
if c.writePadding < kFirstPaddings {
|
||||
bufferLen := buffer.Len()
|
||||
if bufferLen > 65535 {
|
||||
return common.Error(c.Write(buffer.Bytes()))
|
||||
}
|
||||
paddingSize := rand.Intn(256)
|
||||
header := buffer.ExtendHeader(3)
|
||||
binary.BigEndian.PutUint16(header, uint16(bufferLen))
|
||||
header[2] = byte(paddingSize)
|
||||
buffer.Extend(paddingSize)
|
||||
c.writePadding++
|
||||
}
|
||||
return wrapHttpError(common.Error(c.Conn.Write(buffer.Bytes())))
|
||||
}
|
||||
|
||||
// FIXME
|
||||
/*func (c *naiveH1Conn) WriteTo(w io.Writer) (n int64, err error) {
|
||||
if c.readPadding < kFirstPaddings {
|
||||
n, err = bufio.WriteToN(c, w, kFirstPaddings-c.readPadding)
|
||||
} else {
|
||||
n, err = bufio.Copy(w, c.Conn)
|
||||
}
|
||||
return n, wrapHttpError(err)
|
||||
}
|
||||
|
||||
func (c *naiveH1Conn) ReadFrom(r io.Reader) (n int64, err error) {
|
||||
if c.writePadding < kFirstPaddings {
|
||||
n, err = bufio.ReadFromN(c, r, kFirstPaddings-c.writePadding)
|
||||
} else {
|
||||
n, err = bufio.Copy(c.Conn, r)
|
||||
}
|
||||
return n, wrapHttpError(err)
|
||||
}
|
||||
*/
|
||||
|
||||
func (c *naiveH1Conn) Upstream() any {
|
||||
return c.Conn
|
||||
}
|
||||
|
||||
func (c *naiveH1Conn) ReaderReplaceable() bool {
|
||||
return c.readPadding == kFirstPaddings
|
||||
}
|
||||
|
||||
func (c *naiveH1Conn) WriterReplaceable() bool {
|
||||
return c.writePadding == kFirstPaddings
|
||||
}
|
||||
|
||||
type naiveH2Conn struct {
|
||||
reader io.Reader
|
||||
writer io.Writer
|
||||
flusher http.Flusher
|
||||
rAddr net.Addr
|
||||
readPadding int
|
||||
writePadding int
|
||||
readRemaining int
|
||||
paddingRemaining int
|
||||
}
|
||||
|
||||
func (c *naiveH2Conn) Read(p []byte) (n int, err error) {
|
||||
n, err = c.read(p)
|
||||
return n, wrapHttpError(err)
|
||||
}
|
||||
|
||||
func (c *naiveH2Conn) read(p []byte) (n int, err error) {
|
||||
if c.readRemaining > 0 {
|
||||
if len(p) > c.readRemaining {
|
||||
p = p[:c.readRemaining]
|
||||
}
|
||||
n, err = c.reader.Read(p)
|
||||
if err != nil {
|
||||
return
|
||||
}
|
||||
c.readRemaining -= n
|
||||
return
|
||||
}
|
||||
if c.paddingRemaining > 0 {
|
||||
err = rw.SkipN(c.reader, c.paddingRemaining)
|
||||
if err != nil {
|
||||
return
|
||||
}
|
||||
c.paddingRemaining = 0
|
||||
}
|
||||
if c.readPadding < kFirstPaddings {
|
||||
var paddingHdr []byte
|
||||
if len(p) >= 3 {
|
||||
paddingHdr = p[:3]
|
||||
} else {
|
||||
paddingHdr = make([]byte, 3)
|
||||
}
|
||||
_, err = io.ReadFull(c.reader, paddingHdr)
|
||||
if err != nil {
|
||||
return
|
||||
}
|
||||
originalDataSize := int(binary.BigEndian.Uint16(paddingHdr[:2]))
|
||||
paddingSize := int(paddingHdr[2])
|
||||
if len(p) > originalDataSize {
|
||||
p = p[:originalDataSize]
|
||||
}
|
||||
n, err = c.reader.Read(p)
|
||||
if err != nil {
|
||||
return
|
||||
}
|
||||
c.readPadding++
|
||||
c.readRemaining = originalDataSize - n
|
||||
c.paddingRemaining = paddingSize
|
||||
return
|
||||
}
|
||||
return c.reader.Read(p)
|
||||
}
|
||||
|
||||
func (c *naiveH2Conn) Write(p []byte) (n int, err error) {
|
||||
for pLen := len(p); pLen > 0; {
|
||||
var data []byte
|
||||
if pLen > 65535 {
|
||||
data = p[:65535]
|
||||
p = p[65535:]
|
||||
pLen -= 65535
|
||||
} else {
|
||||
data = p
|
||||
pLen = 0
|
||||
}
|
||||
var writeN int
|
||||
writeN, err = c.write(data)
|
||||
n += writeN
|
||||
if err != nil {
|
||||
break
|
||||
}
|
||||
}
|
||||
if err == nil {
|
||||
c.flusher.Flush()
|
||||
}
|
||||
return n, wrapHttpError(err)
|
||||
}
|
||||
|
||||
func (c *naiveH2Conn) write(p []byte) (n int, err error) {
|
||||
if c.writePadding < kFirstPaddings {
|
||||
paddingSize := rand.Intn(256)
|
||||
|
||||
buffer := buf.NewSize(3 + len(p) + paddingSize)
|
||||
defer buffer.Release()
|
||||
header := buffer.Extend(3)
|
||||
binary.BigEndian.PutUint16(header, uint16(len(p)))
|
||||
header[2] = byte(paddingSize)
|
||||
|
||||
common.Must1(buffer.Write(p))
|
||||
_, err = c.writer.Write(buffer.Bytes())
|
||||
if err == nil {
|
||||
n = len(p)
|
||||
}
|
||||
c.writePadding++
|
||||
return
|
||||
}
|
||||
return c.writer.Write(p)
|
||||
}
|
||||
|
||||
func (c *naiveH2Conn) FrontHeadroom() int {
|
||||
if c.writePadding < kFirstPaddings {
|
||||
return 3
|
||||
}
|
||||
return 0
|
||||
}
|
||||
|
||||
func (c *naiveH2Conn) RearHeadroom() int {
|
||||
if c.writePadding < kFirstPaddings {
|
||||
return 255
|
||||
}
|
||||
return 0
|
||||
}
|
||||
|
||||
func (c *naiveH2Conn) WriterMTU() int {
|
||||
if c.writePadding < kFirstPaddings {
|
||||
return 65535
|
||||
}
|
||||
return 0
|
||||
}
|
||||
|
||||
func (c *naiveH2Conn) WriteBuffer(buffer *buf.Buffer) error {
|
||||
defer buffer.Release()
|
||||
if c.writePadding < kFirstPaddings {
|
||||
bufferLen := buffer.Len()
|
||||
if bufferLen > 65535 {
|
||||
return common.Error(c.Write(buffer.Bytes()))
|
||||
}
|
||||
paddingSize := rand.Intn(256)
|
||||
header := buffer.ExtendHeader(3)
|
||||
binary.BigEndian.PutUint16(header, uint16(bufferLen))
|
||||
header[2] = byte(paddingSize)
|
||||
buffer.Extend(paddingSize)
|
||||
c.writePadding++
|
||||
}
|
||||
err := common.Error(c.writer.Write(buffer.Bytes()))
|
||||
if err == nil {
|
||||
c.flusher.Flush()
|
||||
}
|
||||
return wrapHttpError(err)
|
||||
}
|
||||
|
||||
// FIXME
|
||||
/*func (c *naiveH2Conn) WriteTo(w io.Writer) (n int64, err error) {
|
||||
if c.readPadding < kFirstPaddings {
|
||||
n, err = bufio.WriteToN(c, w, kFirstPaddings-c.readPadding)
|
||||
} else {
|
||||
n, err = bufio.Copy(w, c.reader)
|
||||
}
|
||||
return n, wrapHttpError(err)
|
||||
}
|
||||
|
||||
func (c *naiveH2Conn) ReadFrom(r io.Reader) (n int64, err error) {
|
||||
if c.writePadding < kFirstPaddings {
|
||||
n, err = bufio.ReadFromN(c, r, kFirstPaddings-c.writePadding)
|
||||
} else {
|
||||
n, err = bufio.Copy(c.writer, r)
|
||||
}
|
||||
return n, wrapHttpError(err)
|
||||
}*/
|
||||
|
||||
func (c *naiveH2Conn) Close() error {
|
||||
return common.Close(
|
||||
c.reader,
|
||||
c.writer,
|
||||
)
|
||||
}
|
||||
|
||||
func (c *naiveH2Conn) LocalAddr() net.Addr {
|
||||
return M.Socksaddr{}
|
||||
}
|
||||
|
||||
func (c *naiveH2Conn) RemoteAddr() net.Addr {
|
||||
return c.rAddr
|
||||
}
|
||||
|
||||
func (c *naiveH2Conn) SetDeadline(t time.Time) error {
|
||||
return os.ErrInvalid
|
||||
}
|
||||
|
||||
func (c *naiveH2Conn) SetReadDeadline(t time.Time) error {
|
||||
return os.ErrInvalid
|
||||
}
|
||||
|
||||
func (c *naiveH2Conn) SetWriteDeadline(t time.Time) error {
|
||||
return os.ErrInvalid
|
||||
}
|
||||
|
||||
func (c *naiveH2Conn) NeedAdditionalReadDeadline() bool {
|
||||
return true
|
||||
}
|
||||
|
||||
func (c *naiveH2Conn) UpstreamReader() any {
|
||||
return c.reader
|
||||
}
|
||||
|
||||
func (c *naiveH2Conn) UpstreamWriter() any {
|
||||
return c.writer
|
||||
}
|
||||
|
||||
func (c *naiveH2Conn) ReaderReplaceable() bool {
|
||||
return c.readPadding == kFirstPaddings
|
||||
}
|
||||
|
||||
func (c *naiveH2Conn) WriterReplaceable() bool {
|
||||
return c.writePadding == kFirstPaddings
|
||||
}
|
||||
|
||||
func wrapHttpError(err error) error {
|
||||
if err == nil {
|
||||
return err
|
||||
}
|
||||
if strings.Contains(err.Error(), "client disconnected") {
|
||||
return net.ErrClosed
|
||||
}
|
||||
if strings.Contains(err.Error(), "body closed by handler") {
|
||||
return net.ErrClosed
|
||||
}
|
||||
if strings.Contains(err.Error(), "canceled with error code 268") {
|
||||
return io.EOF
|
||||
}
|
||||
return err
|
||||
}
|
52
protocol/naive/quic/inbound_init.go
Normal file
52
protocol/naive/quic/inbound_init.go
Normal file
|
@ -0,0 +1,52 @@
|
|||
package quic
|
||||
|
||||
import (
|
||||
"io"
|
||||
"net/http"
|
||||
|
||||
"github.com/sagernet/quic-go"
|
||||
"github.com/sagernet/quic-go/http3"
|
||||
"github.com/sagernet/sing-box/common/listener"
|
||||
"github.com/sagernet/sing-box/common/tls"
|
||||
"github.com/sagernet/sing-box/protocol/naive"
|
||||
"github.com/sagernet/sing-quic"
|
||||
E "github.com/sagernet/sing/common/exceptions"
|
||||
"github.com/sagernet/sing/common/logger"
|
||||
)
|
||||
|
||||
func init() {
|
||||
naive.ConfigureHTTP3ListenerFunc = func(listener *listener.Listener, handler http.Handler, tlsConfig tls.ServerConfig, logger logger.Logger) (io.Closer, error) {
|
||||
err := qtls.ConfigureHTTP3(tlsConfig)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
udpConn, err := listener.ListenUDP()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
quicListener, err := qtls.ListenEarly(udpConn, tlsConfig, &quic.Config{
|
||||
MaxIncomingStreams: 1 << 60,
|
||||
Allow0RTT: true,
|
||||
})
|
||||
if err != nil {
|
||||
udpConn.Close()
|
||||
return nil, err
|
||||
}
|
||||
|
||||
h3Server := &http3.Server{
|
||||
Handler: handler,
|
||||
}
|
||||
|
||||
go func() {
|
||||
sErr := h3Server.ServeListener(quicListener)
|
||||
udpConn.Close()
|
||||
if sErr != nil && !E.IsClosedOrCanceled(sErr) {
|
||||
logger.Error("http3 server closed: ", sErr)
|
||||
}
|
||||
}()
|
||||
|
||||
return quicListener, nil
|
||||
}
|
||||
}
|
65
protocol/redirect/redirect.go
Normal file
65
protocol/redirect/redirect.go
Normal file
|
@ -0,0 +1,65 @@
|
|||
package redirect
|
||||
|
||||
import (
|
||||
"context"
|
||||
"net"
|
||||
|
||||
"github.com/sagernet/sing-box/adapter"
|
||||
"github.com/sagernet/sing-box/adapter/inbound"
|
||||
"github.com/sagernet/sing-box/common/listener"
|
||||
"github.com/sagernet/sing-box/common/redir"
|
||||
C "github.com/sagernet/sing-box/constant"
|
||||
"github.com/sagernet/sing-box/log"
|
||||
"github.com/sagernet/sing-box/option"
|
||||
M "github.com/sagernet/sing/common/metadata"
|
||||
N "github.com/sagernet/sing/common/network"
|
||||
)
|
||||
|
||||
func RegisterRedirect(registry *inbound.Registry) {
|
||||
inbound.Register[option.RedirectInboundOptions](registry, C.TypeRedirect, NewRedirect)
|
||||
}
|
||||
|
||||
type Redirect struct {
|
||||
inbound.Adapter
|
||||
router adapter.Router
|
||||
logger log.ContextLogger
|
||||
listener *listener.Listener
|
||||
}
|
||||
|
||||
func NewRedirect(ctx context.Context, router adapter.Router, logger log.ContextLogger, tag string, options option.RedirectInboundOptions) (adapter.Inbound, error) {
|
||||
redirect := &Redirect{
|
||||
Adapter: inbound.NewAdapter(C.TypeRedirect, tag),
|
||||
router: router,
|
||||
logger: logger,
|
||||
}
|
||||
redirect.listener = listener.New(listener.Options{
|
||||
Context: ctx,
|
||||
Logger: logger,
|
||||
Network: []string{N.NetworkTCP},
|
||||
Listen: options.ListenOptions,
|
||||
ConnectionHandler: redirect,
|
||||
})
|
||||
return redirect, nil
|
||||
}
|
||||
|
||||
func (h *Redirect) Start() error {
|
||||
return h.listener.Start()
|
||||
}
|
||||
|
||||
func (h *Redirect) Close() error {
|
||||
return h.listener.Close()
|
||||
}
|
||||
|
||||
func (h *Redirect) NewConnectionEx(ctx context.Context, conn net.Conn, metadata adapter.InboundContext, onClose N.CloseHandlerFunc) {
|
||||
destination, err := redir.GetOriginalDestination(conn)
|
||||
if err != nil {
|
||||
conn.Close()
|
||||
h.logger.ErrorContext(ctx, "process connection from ", conn.RemoteAddr(), ": get redirect destination: ", err)
|
||||
return
|
||||
}
|
||||
metadata.Inbound = h.Tag()
|
||||
metadata.InboundType = h.Type()
|
||||
metadata.Destination = M.SocksaddrFromNetIP(destination)
|
||||
h.logger.InfoContext(ctx, "inbound connection to ", metadata.Destination)
|
||||
h.router.RouteConnectionEx(ctx, conn, metadata, onClose)
|
||||
}
|
158
protocol/redirect/tproxy.go
Normal file
158
protocol/redirect/tproxy.go
Normal file
|
@ -0,0 +1,158 @@
|
|||
package redirect
|
||||
|
||||
import (
|
||||
"context"
|
||||
"net"
|
||||
"net/netip"
|
||||
"syscall"
|
||||
"time"
|
||||
|
||||
"github.com/sagernet/sing-box/adapter"
|
||||
"github.com/sagernet/sing-box/adapter/inbound"
|
||||
"github.com/sagernet/sing-box/common/listener"
|
||||
"github.com/sagernet/sing-box/common/redir"
|
||||
C "github.com/sagernet/sing-box/constant"
|
||||
"github.com/sagernet/sing-box/log"
|
||||
"github.com/sagernet/sing-box/option"
|
||||
"github.com/sagernet/sing/common"
|
||||
"github.com/sagernet/sing/common/buf"
|
||||
"github.com/sagernet/sing/common/control"
|
||||
E "github.com/sagernet/sing/common/exceptions"
|
||||
M "github.com/sagernet/sing/common/metadata"
|
||||
N "github.com/sagernet/sing/common/network"
|
||||
"github.com/sagernet/sing/common/udpnat2"
|
||||
)
|
||||
|
||||
func RegisterTProxy(registry *inbound.Registry) {
|
||||
inbound.Register[option.TProxyInboundOptions](registry, C.TypeTProxy, NewTProxy)
|
||||
}
|
||||
|
||||
type TProxy struct {
|
||||
inbound.Adapter
|
||||
ctx context.Context
|
||||
router adapter.Router
|
||||
logger log.ContextLogger
|
||||
listener *listener.Listener
|
||||
udpNat *udpnat.Service
|
||||
}
|
||||
|
||||
func NewTProxy(ctx context.Context, router adapter.Router, logger log.ContextLogger, tag string, options option.TProxyInboundOptions) (adapter.Inbound, error) {
|
||||
tproxy := &TProxy{
|
||||
Adapter: inbound.NewAdapter(C.TypeTProxy, tag),
|
||||
ctx: ctx,
|
||||
router: router,
|
||||
logger: logger,
|
||||
}
|
||||
var udpTimeout time.Duration
|
||||
if options.UDPTimeout != 0 {
|
||||
udpTimeout = time.Duration(options.UDPTimeout)
|
||||
} else {
|
||||
udpTimeout = C.UDPTimeout
|
||||
}
|
||||
tproxy.udpNat = udpnat.New(tproxy, tproxy.preparePacketConnection, udpTimeout, false)
|
||||
tproxy.listener = listener.New(listener.Options{
|
||||
Context: ctx,
|
||||
Logger: logger,
|
||||
Network: options.Network.Build(),
|
||||
Listen: options.ListenOptions,
|
||||
ConnectionHandler: tproxy,
|
||||
OOBPacketHandler: tproxy,
|
||||
})
|
||||
return tproxy, nil
|
||||
}
|
||||
|
||||
func (t *TProxy) Start() error {
|
||||
err := t.listener.Start()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if listener := t.listener.TCPListener(); listener != nil {
|
||||
err = control.Conn(common.MustCast[syscall.Conn](listener), func(fd uintptr) error {
|
||||
return redir.TProxy(fd, M.SocksaddrFromNet(listener.Addr()).Addr.Is6())
|
||||
})
|
||||
if err != nil {
|
||||
return E.Cause(err, "configure tproxy TCP listener")
|
||||
}
|
||||
}
|
||||
if conn := t.listener.UDPConn(); conn != nil {
|
||||
err = control.Conn(conn, func(fd uintptr) error {
|
||||
return redir.TProxy(fd, M.SocksaddrFromNet(conn.LocalAddr()).Addr.Is6())
|
||||
})
|
||||
if err != nil {
|
||||
return E.Cause(err, "configure tproxy UDP listener")
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (t *TProxy) Close() error {
|
||||
return t.listener.Close()
|
||||
}
|
||||
|
||||
func (t *TProxy) NewConnectionEx(ctx context.Context, conn net.Conn, metadata adapter.InboundContext, onClose N.CloseHandlerFunc) {
|
||||
metadata.Destination = M.SocksaddrFromNet(conn.LocalAddr()).Unwrap()
|
||||
t.logger.InfoContext(ctx, "inbound connection to ", metadata.Destination)
|
||||
t.router.RouteConnectionEx(ctx, conn, metadata, onClose)
|
||||
}
|
||||
|
||||
func (t *TProxy) NewPacketConnectionEx(ctx context.Context, conn N.PacketConn, source M.Socksaddr, destination M.Socksaddr, onClose N.CloseHandlerFunc) {
|
||||
t.logger.InfoContext(ctx, "inbound packet connection from ", source)
|
||||
t.logger.InfoContext(ctx, "inbound packet connection to ", destination)
|
||||
var metadata adapter.InboundContext
|
||||
metadata.Inbound = t.Tag()
|
||||
metadata.InboundType = t.Type()
|
||||
metadata.Source = source
|
||||
metadata.Destination = destination
|
||||
metadata.OriginDestination = t.listener.UDPAddr()
|
||||
t.router.RoutePacketConnectionEx(ctx, conn, metadata, onClose)
|
||||
}
|
||||
|
||||
func (t *TProxy) NewPacketEx(buffer *buf.Buffer, oob []byte, source M.Socksaddr) {
|
||||
destination, err := redir.GetOriginalDestinationFromOOB(oob)
|
||||
if err != nil {
|
||||
t.logger.Warn("process packet from ", source, ": get tproxy destination: ", err)
|
||||
return
|
||||
}
|
||||
t.udpNat.NewPacket([][]byte{buffer.Bytes()}, source, M.SocksaddrFromNetIP(destination), nil)
|
||||
}
|
||||
|
||||
type tproxyPacketWriter struct {
|
||||
ctx context.Context
|
||||
source netip.AddrPort
|
||||
destination M.Socksaddr
|
||||
conn *net.UDPConn
|
||||
}
|
||||
|
||||
func (t *TProxy) preparePacketConnection(source M.Socksaddr, destination M.Socksaddr, userData any) (bool, context.Context, N.PacketWriter, N.CloseHandlerFunc) {
|
||||
ctx := log.ContextWithNewID(t.ctx)
|
||||
writer := &tproxyPacketWriter{ctx: ctx, source: source.AddrPort(), destination: destination}
|
||||
return true, ctx, writer, func(it error) {
|
||||
common.Close(common.PtrOrNil(writer.conn))
|
||||
}
|
||||
}
|
||||
|
||||
func (w *tproxyPacketWriter) WritePacket(buffer *buf.Buffer, destination M.Socksaddr) error {
|
||||
defer buffer.Release()
|
||||
conn := w.conn
|
||||
if w.destination == destination && conn != nil {
|
||||
_, err := conn.WriteToUDPAddrPort(buffer.Bytes(), w.source)
|
||||
if err != nil {
|
||||
w.conn = nil
|
||||
}
|
||||
return err
|
||||
}
|
||||
var listener net.ListenConfig
|
||||
listener.Control = control.Append(listener.Control, control.ReuseAddr())
|
||||
listener.Control = control.Append(listener.Control, redir.TProxyWriteBack())
|
||||
packetConn, err := listener.ListenPacket(w.ctx, "udp", destination.String())
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
udpConn := packetConn.(*net.UDPConn)
|
||||
if w.destination == destination {
|
||||
w.conn = udpConn
|
||||
} else {
|
||||
defer udpConn.Close()
|
||||
}
|
||||
return common.Error(udpConn.WriteToUDPAddrPort(buffer.Bytes(), w.source))
|
||||
}
|
179
protocol/shadowsocks/inbound.go
Normal file
179
protocol/shadowsocks/inbound.go
Normal file
|
@ -0,0 +1,179 @@
|
|||
package shadowsocks
|
||||
|
||||
import (
|
||||
"context"
|
||||
"net"
|
||||
"time"
|
||||
|
||||
"github.com/sagernet/sing-box/adapter"
|
||||
"github.com/sagernet/sing-box/adapter/inbound"
|
||||
"github.com/sagernet/sing-box/common/listener"
|
||||
"github.com/sagernet/sing-box/common/mux"
|
||||
"github.com/sagernet/sing-box/common/uot"
|
||||
C "github.com/sagernet/sing-box/constant"
|
||||
"github.com/sagernet/sing-box/log"
|
||||
"github.com/sagernet/sing-box/option"
|
||||
"github.com/sagernet/sing-shadowsocks"
|
||||
"github.com/sagernet/sing-shadowsocks/shadowaead"
|
||||
"github.com/sagernet/sing-shadowsocks/shadowaead_2022"
|
||||
"github.com/sagernet/sing/common"
|
||||
"github.com/sagernet/sing/common/buf"
|
||||
E "github.com/sagernet/sing/common/exceptions"
|
||||
"github.com/sagernet/sing/common/logger"
|
||||
M "github.com/sagernet/sing/common/metadata"
|
||||
N "github.com/sagernet/sing/common/network"
|
||||
"github.com/sagernet/sing/common/ntp"
|
||||
)
|
||||
|
||||
func RegisterInbound(registry *inbound.Registry) {
|
||||
inbound.Register[option.ShadowsocksInboundOptions](registry, C.TypeShadowsocks, NewInbound)
|
||||
}
|
||||
|
||||
func NewInbound(ctx context.Context, router adapter.Router, logger log.ContextLogger, tag string, options option.ShadowsocksInboundOptions) (adapter.Inbound, error) {
|
||||
if len(options.Users) > 0 && len(options.Destinations) > 0 {
|
||||
return nil, E.New("users and destinations options must not be combined")
|
||||
}
|
||||
if len(options.Users) > 0 {
|
||||
return newMultiInbound(ctx, router, logger, tag, options)
|
||||
} else if len(options.Destinations) > 0 {
|
||||
return newRelayInbound(ctx, router, logger, tag, options)
|
||||
} else {
|
||||
return newInbound(ctx, router, logger, tag, options)
|
||||
}
|
||||
}
|
||||
|
||||
var _ adapter.TCPInjectableInbound = (*Inbound)(nil)
|
||||
|
||||
type Inbound struct {
|
||||
inbound.Adapter
|
||||
ctx context.Context
|
||||
router adapter.ConnectionRouterEx
|
||||
logger logger.ContextLogger
|
||||
listener *listener.Listener
|
||||
service shadowsocks.Service
|
||||
}
|
||||
|
||||
func newInbound(ctx context.Context, router adapter.Router, logger log.ContextLogger, tag string, options option.ShadowsocksInboundOptions) (*Inbound, error) {
|
||||
inbound := &Inbound{
|
||||
Adapter: inbound.NewAdapter(C.TypeShadowsocks, tag),
|
||||
ctx: ctx,
|
||||
router: uot.NewRouter(router, logger),
|
||||
logger: logger,
|
||||
}
|
||||
var err error
|
||||
inbound.router, err = mux.NewRouterWithOptions(router, logger, common.PtrValueOrDefault(options.Multiplex))
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
var udpTimeout time.Duration
|
||||
if options.UDPTimeout != 0 {
|
||||
udpTimeout = time.Duration(options.UDPTimeout)
|
||||
} else {
|
||||
udpTimeout = C.UDPTimeout
|
||||
}
|
||||
switch {
|
||||
case options.Method == shadowsocks.MethodNone:
|
||||
inbound.service = shadowsocks.NewNoneService(int64(udpTimeout.Seconds()), adapter.NewUpstreamHandler(adapter.InboundContext{}, inbound.newConnection, inbound.newPacketConnection, inbound))
|
||||
case common.Contains(shadowaead.List, options.Method):
|
||||
inbound.service, err = shadowaead.NewService(options.Method, nil, options.Password, int64(udpTimeout.Seconds()), adapter.NewUpstreamHandler(adapter.InboundContext{}, inbound.newConnection, inbound.newPacketConnection, inbound))
|
||||
case common.Contains(shadowaead_2022.List, options.Method):
|
||||
inbound.service, err = shadowaead_2022.NewServiceWithPassword(options.Method, options.Password, int64(udpTimeout.Seconds()), adapter.NewUpstreamHandler(adapter.InboundContext{}, inbound.newConnection, inbound.newPacketConnection, inbound), ntp.TimeFuncFromContext(ctx))
|
||||
default:
|
||||
err = E.New("unsupported method: ", options.Method)
|
||||
}
|
||||
inbound.listener = listener.New(listener.Options{
|
||||
Context: ctx,
|
||||
Logger: logger,
|
||||
Network: options.Network.Build(),
|
||||
Listen: options.ListenOptions,
|
||||
ConnectionHandler: inbound,
|
||||
PacketHandler: inbound,
|
||||
ThreadUnsafePacketWriter: true,
|
||||
})
|
||||
return inbound, err
|
||||
}
|
||||
|
||||
func (h *Inbound) Start() error {
|
||||
return h.listener.Start()
|
||||
}
|
||||
|
||||
func (h *Inbound) Close() error {
|
||||
return h.listener.Close()
|
||||
}
|
||||
|
||||
func (h *Inbound) NewConnectionEx(ctx context.Context, conn net.Conn, metadata adapter.InboundContext, onClose N.CloseHandlerFunc) {
|
||||
err := h.service.NewConnection(ctx, conn, adapter.UpstreamMetadata(metadata))
|
||||
N.CloseOnHandshakeFailure(conn, onClose, err)
|
||||
if err != nil {
|
||||
h.logger.ErrorContext(ctx, E.Cause(err, "process connection from ", metadata.Source))
|
||||
}
|
||||
}
|
||||
|
||||
func (h *Inbound) NewPacketEx(buffer *buf.Buffer, source M.Socksaddr) {
|
||||
err := h.service.NewPacket(h.ctx, &stubPacketConn{h.listener.PacketWriter()}, buffer, M.Metadata{Source: source})
|
||||
if err != nil {
|
||||
h.logger.Error(E.Cause(err, "process packet from ", source))
|
||||
}
|
||||
}
|
||||
|
||||
func (h *Inbound) newConnection(ctx context.Context, conn net.Conn, metadata adapter.InboundContext) error {
|
||||
h.logger.InfoContext(ctx, "inbound connection to ", metadata.Destination)
|
||||
metadata.Inbound = h.Tag()
|
||||
metadata.InboundType = h.Type()
|
||||
return h.router.RouteConnection(ctx, conn, metadata)
|
||||
}
|
||||
|
||||
func (h *Inbound) newPacketConnection(ctx context.Context, conn N.PacketConn, metadata adapter.InboundContext) error {
|
||||
ctx = log.ContextWithNewID(ctx)
|
||||
h.logger.InfoContext(ctx, "inbound packet connection from ", metadata.Source)
|
||||
h.logger.InfoContext(ctx, "inbound packet connection to ", metadata.Destination)
|
||||
metadata.Inbound = h.Tag()
|
||||
metadata.InboundType = h.Type()
|
||||
metadata.InboundDetour = h.listener.ListenOptions().Detour
|
||||
metadata.InboundOptions = h.listener.ListenOptions().InboundOptions
|
||||
return h.router.RoutePacketConnection(ctx, conn, metadata)
|
||||
}
|
||||
|
||||
var _ N.PacketConn = (*stubPacketConn)(nil)
|
||||
|
||||
type stubPacketConn struct {
|
||||
N.PacketWriter
|
||||
}
|
||||
|
||||
func (c *stubPacketConn) ReadPacket(buffer *buf.Buffer) (destination M.Socksaddr, err error) {
|
||||
panic("stub!")
|
||||
}
|
||||
|
||||
func (c *stubPacketConn) Close() error {
|
||||
return nil
|
||||
}
|
||||
|
||||
func (c *stubPacketConn) LocalAddr() net.Addr {
|
||||
panic("stub!")
|
||||
}
|
||||
|
||||
func (c *stubPacketConn) SetDeadline(t time.Time) error {
|
||||
panic("stub!")
|
||||
}
|
||||
|
||||
func (c *stubPacketConn) SetReadDeadline(t time.Time) error {
|
||||
panic("stub!")
|
||||
}
|
||||
|
||||
func (c *stubPacketConn) SetWriteDeadline(t time.Time) error {
|
||||
panic("stub!")
|
||||
}
|
||||
|
||||
func (h *Inbound) NewError(ctx context.Context, err error) {
|
||||
NewError(h.logger, ctx, err)
|
||||
}
|
||||
|
||||
// Deprecated: remove
|
||||
func NewError(logger logger.ContextLogger, ctx context.Context, err error) {
|
||||
common.Close(err)
|
||||
if E.IsClosedOrCanceled(err) {
|
||||
logger.DebugContext(ctx, "connection closed: ", err)
|
||||
return
|
||||
}
|
||||
logger.ErrorContext(ctx, err)
|
||||
}
|
167
protocol/shadowsocks/inbound_multi.go
Normal file
167
protocol/shadowsocks/inbound_multi.go
Normal file
|
@ -0,0 +1,167 @@
|
|||
package shadowsocks
|
||||
|
||||
import (
|
||||
"context"
|
||||
"net"
|
||||
"os"
|
||||
"time"
|
||||
|
||||
"github.com/sagernet/sing-box/adapter"
|
||||
"github.com/sagernet/sing-box/adapter/inbound"
|
||||
"github.com/sagernet/sing-box/common/listener"
|
||||
"github.com/sagernet/sing-box/common/mux"
|
||||
"github.com/sagernet/sing-box/common/uot"
|
||||
C "github.com/sagernet/sing-box/constant"
|
||||
"github.com/sagernet/sing-box/log"
|
||||
"github.com/sagernet/sing-box/option"
|
||||
"github.com/sagernet/sing-shadowsocks"
|
||||
"github.com/sagernet/sing-shadowsocks/shadowaead"
|
||||
"github.com/sagernet/sing-shadowsocks/shadowaead_2022"
|
||||
"github.com/sagernet/sing/common"
|
||||
"github.com/sagernet/sing/common/auth"
|
||||
"github.com/sagernet/sing/common/buf"
|
||||
E "github.com/sagernet/sing/common/exceptions"
|
||||
F "github.com/sagernet/sing/common/format"
|
||||
"github.com/sagernet/sing/common/logger"
|
||||
M "github.com/sagernet/sing/common/metadata"
|
||||
N "github.com/sagernet/sing/common/network"
|
||||
"github.com/sagernet/sing/common/ntp"
|
||||
)
|
||||
|
||||
var _ adapter.TCPInjectableInbound = (*MultiInbound)(nil)
|
||||
|
||||
type MultiInbound struct {
|
||||
inbound.Adapter
|
||||
ctx context.Context
|
||||
router adapter.ConnectionRouterEx
|
||||
logger logger.ContextLogger
|
||||
listener *listener.Listener
|
||||
service shadowsocks.MultiService[int]
|
||||
users []option.ShadowsocksUser
|
||||
}
|
||||
|
||||
func newMultiInbound(ctx context.Context, router adapter.Router, logger log.ContextLogger, tag string, options option.ShadowsocksInboundOptions) (*MultiInbound, error) {
|
||||
inbound := &MultiInbound{
|
||||
Adapter: inbound.NewAdapter(C.TypeShadowsocks, tag),
|
||||
ctx: ctx,
|
||||
router: uot.NewRouter(router, logger),
|
||||
logger: logger,
|
||||
}
|
||||
var err error
|
||||
inbound.router, err = mux.NewRouterWithOptions(inbound.router, logger, common.PtrValueOrDefault(options.Multiplex))
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
var udpTimeout time.Duration
|
||||
if options.UDPTimeout != 0 {
|
||||
udpTimeout = time.Duration(options.UDPTimeout)
|
||||
} else {
|
||||
udpTimeout = C.UDPTimeout
|
||||
}
|
||||
var service shadowsocks.MultiService[int]
|
||||
if common.Contains(shadowaead_2022.List, options.Method) {
|
||||
service, err = shadowaead_2022.NewMultiServiceWithPassword[int](
|
||||
options.Method,
|
||||
options.Password,
|
||||
int64(udpTimeout.Seconds()),
|
||||
adapter.NewUpstreamHandler(adapter.InboundContext{}, inbound.newConnection, inbound.newPacketConnection, inbound),
|
||||
ntp.TimeFuncFromContext(ctx),
|
||||
)
|
||||
} else if common.Contains(shadowaead.List, options.Method) {
|
||||
service, err = shadowaead.NewMultiService[int](
|
||||
options.Method,
|
||||
int64(udpTimeout.Seconds()),
|
||||
adapter.NewUpstreamHandler(adapter.InboundContext{}, inbound.newConnection, inbound.newPacketConnection, inbound),
|
||||
)
|
||||
} else {
|
||||
return nil, E.New("unsupported method: " + options.Method)
|
||||
}
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
err = service.UpdateUsersWithPasswords(common.MapIndexed(options.Users, func(index int, user option.ShadowsocksUser) int {
|
||||
return index
|
||||
}), common.Map(options.Users, func(user option.ShadowsocksUser) string {
|
||||
return user.Password
|
||||
}))
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
inbound.service = service
|
||||
inbound.users = options.Users
|
||||
inbound.listener = listener.New(listener.Options{
|
||||
Context: ctx,
|
||||
Logger: logger,
|
||||
Network: options.Network.Build(),
|
||||
Listen: options.ListenOptions,
|
||||
ConnectionHandler: inbound,
|
||||
PacketHandler: inbound,
|
||||
ThreadUnsafePacketWriter: true,
|
||||
})
|
||||
return inbound, err
|
||||
}
|
||||
|
||||
func (h *MultiInbound) Start() error {
|
||||
return h.listener.Start()
|
||||
}
|
||||
|
||||
func (h *MultiInbound) Close() error {
|
||||
return h.listener.Close()
|
||||
}
|
||||
|
||||
func (h *MultiInbound) NewConnectionEx(ctx context.Context, conn net.Conn, metadata adapter.InboundContext, onClose N.CloseHandlerFunc) {
|
||||
err := h.service.NewConnection(ctx, conn, adapter.UpstreamMetadata(metadata))
|
||||
N.CloseOnHandshakeFailure(conn, onClose, err)
|
||||
if err != nil {
|
||||
h.logger.ErrorContext(ctx, E.Cause(err, "process connection from ", metadata.Source))
|
||||
}
|
||||
}
|
||||
|
||||
func (h *MultiInbound) NewPacketEx(buffer *buf.Buffer, source M.Socksaddr) {
|
||||
err := h.service.NewPacket(h.ctx, &stubPacketConn{h.listener.PacketWriter()}, buffer, M.Metadata{Source: source})
|
||||
if err != nil {
|
||||
h.logger.Error(E.Cause(err, "process packet from ", source))
|
||||
}
|
||||
}
|
||||
|
||||
func (h *MultiInbound) newConnection(ctx context.Context, conn net.Conn, metadata adapter.InboundContext) error {
|
||||
userIndex, loaded := auth.UserFromContext[int](ctx)
|
||||
if !loaded {
|
||||
return os.ErrInvalid
|
||||
}
|
||||
user := h.users[userIndex].Name
|
||||
if user == "" {
|
||||
user = F.ToString(userIndex)
|
||||
} else {
|
||||
metadata.User = user
|
||||
}
|
||||
h.logger.InfoContext(ctx, "[", user, "] inbound connection to ", metadata.Destination)
|
||||
metadata.Inbound = h.Tag()
|
||||
metadata.InboundType = h.Type()
|
||||
return h.router.RouteConnection(ctx, conn, metadata)
|
||||
}
|
||||
|
||||
func (h *MultiInbound) newPacketConnection(ctx context.Context, conn N.PacketConn, metadata adapter.InboundContext) error {
|
||||
userIndex, loaded := auth.UserFromContext[int](ctx)
|
||||
if !loaded {
|
||||
return os.ErrInvalid
|
||||
}
|
||||
user := h.users[userIndex].Name
|
||||
if user == "" {
|
||||
user = F.ToString(userIndex)
|
||||
} else {
|
||||
metadata.User = user
|
||||
}
|
||||
ctx = log.ContextWithNewID(ctx)
|
||||
h.logger.InfoContext(ctx, "[", user, "] inbound packet connection from ", metadata.Source)
|
||||
h.logger.InfoContext(ctx, "[", user, "] inbound packet connection to ", metadata.Destination)
|
||||
metadata.Inbound = h.Tag()
|
||||
metadata.InboundType = h.Type()
|
||||
metadata.InboundDetour = h.listener.ListenOptions().Detour
|
||||
metadata.InboundOptions = h.listener.ListenOptions().InboundOptions
|
||||
return h.router.RoutePacketConnection(ctx, conn, metadata)
|
||||
}
|
||||
|
||||
func (h *MultiInbound) NewError(ctx context.Context, err error) {
|
||||
NewError(h.logger, ctx, err)
|
||||
}
|
152
protocol/shadowsocks/inbound_relay.go
Normal file
152
protocol/shadowsocks/inbound_relay.go
Normal file
|
@ -0,0 +1,152 @@
|
|||
package shadowsocks
|
||||
|
||||
import (
|
||||
"context"
|
||||
"net"
|
||||
"os"
|
||||
"time"
|
||||
|
||||
"github.com/sagernet/sing-box/adapter"
|
||||
"github.com/sagernet/sing-box/adapter/inbound"
|
||||
"github.com/sagernet/sing-box/common/listener"
|
||||
"github.com/sagernet/sing-box/common/mux"
|
||||
"github.com/sagernet/sing-box/common/uot"
|
||||
C "github.com/sagernet/sing-box/constant"
|
||||
"github.com/sagernet/sing-box/log"
|
||||
"github.com/sagernet/sing-box/option"
|
||||
"github.com/sagernet/sing-shadowsocks/shadowaead_2022"
|
||||
"github.com/sagernet/sing/common"
|
||||
"github.com/sagernet/sing/common/auth"
|
||||
"github.com/sagernet/sing/common/buf"
|
||||
E "github.com/sagernet/sing/common/exceptions"
|
||||
F "github.com/sagernet/sing/common/format"
|
||||
"github.com/sagernet/sing/common/logger"
|
||||
M "github.com/sagernet/sing/common/metadata"
|
||||
N "github.com/sagernet/sing/common/network"
|
||||
)
|
||||
|
||||
var _ adapter.TCPInjectableInbound = (*RelayInbound)(nil)
|
||||
|
||||
type RelayInbound struct {
|
||||
inbound.Adapter
|
||||
ctx context.Context
|
||||
router adapter.ConnectionRouterEx
|
||||
logger logger.ContextLogger
|
||||
listener *listener.Listener
|
||||
service *shadowaead_2022.RelayService[int]
|
||||
destinations []option.ShadowsocksDestination
|
||||
}
|
||||
|
||||
func newRelayInbound(ctx context.Context, router adapter.Router, logger log.ContextLogger, tag string, options option.ShadowsocksInboundOptions) (*RelayInbound, error) {
|
||||
inbound := &RelayInbound{
|
||||
Adapter: inbound.NewAdapter(C.TypeShadowsocks, tag),
|
||||
ctx: ctx,
|
||||
router: uot.NewRouter(router, logger),
|
||||
logger: logger,
|
||||
destinations: options.Destinations,
|
||||
}
|
||||
var err error
|
||||
inbound.router, err = mux.NewRouterWithOptions(inbound.router, logger, common.PtrValueOrDefault(options.Multiplex))
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
var udpTimeout time.Duration
|
||||
if options.UDPTimeout != 0 {
|
||||
udpTimeout = time.Duration(options.UDPTimeout)
|
||||
} else {
|
||||
udpTimeout = C.UDPTimeout
|
||||
}
|
||||
service, err := shadowaead_2022.NewRelayServiceWithPassword[int](
|
||||
options.Method,
|
||||
options.Password,
|
||||
int64(udpTimeout.Seconds()),
|
||||
adapter.NewUpstreamHandler(adapter.InboundContext{}, inbound.newConnection, inbound.newPacketConnection, inbound),
|
||||
)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
err = service.UpdateUsersWithPasswords(common.MapIndexed(options.Destinations, func(index int, user option.ShadowsocksDestination) int {
|
||||
return index
|
||||
}), common.Map(options.Destinations, func(user option.ShadowsocksDestination) string {
|
||||
return user.Password
|
||||
}), common.Map(options.Destinations, option.ShadowsocksDestination.Build))
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
inbound.service = service
|
||||
inbound.listener = listener.New(listener.Options{
|
||||
Context: ctx,
|
||||
Logger: logger,
|
||||
Network: options.Network.Build(),
|
||||
Listen: options.ListenOptions,
|
||||
ConnectionHandler: inbound,
|
||||
PacketHandler: inbound,
|
||||
ThreadUnsafePacketWriter: true,
|
||||
})
|
||||
return inbound, err
|
||||
}
|
||||
|
||||
func (h *RelayInbound) Start() error {
|
||||
return h.listener.Start()
|
||||
}
|
||||
|
||||
func (h *RelayInbound) Close() error {
|
||||
return h.listener.Close()
|
||||
}
|
||||
|
||||
func (h *RelayInbound) NewConnectionEx(ctx context.Context, conn net.Conn, metadata adapter.InboundContext, onClose N.CloseHandlerFunc) {
|
||||
err := h.service.NewConnection(ctx, conn, adapter.UpstreamMetadata(metadata))
|
||||
N.CloseOnHandshakeFailure(conn, onClose, err)
|
||||
if err != nil {
|
||||
h.logger.ErrorContext(ctx, E.Cause(err, "process connection from ", metadata.Source))
|
||||
}
|
||||
}
|
||||
|
||||
func (h *RelayInbound) NewPacketEx(buffer *buf.Buffer, source M.Socksaddr) {
|
||||
err := h.service.NewPacket(h.ctx, &stubPacketConn{h.listener.PacketWriter()}, buffer, M.Metadata{Source: source})
|
||||
if err != nil {
|
||||
h.logger.Error(E.Cause(err, "process packet from ", source))
|
||||
}
|
||||
}
|
||||
|
||||
func (h *RelayInbound) newConnection(ctx context.Context, conn net.Conn, metadata adapter.InboundContext) error {
|
||||
destinationIndex, loaded := auth.UserFromContext[int](ctx)
|
||||
if !loaded {
|
||||
return os.ErrInvalid
|
||||
}
|
||||
destination := h.destinations[destinationIndex].Name
|
||||
if destination == "" {
|
||||
destination = F.ToString(destinationIndex)
|
||||
} else {
|
||||
metadata.User = destination
|
||||
}
|
||||
h.logger.InfoContext(ctx, "[", destination, "] inbound connection to ", metadata.Destination)
|
||||
metadata.Inbound = h.Tag()
|
||||
metadata.InboundType = h.Type()
|
||||
return h.router.RouteConnection(ctx, conn, metadata)
|
||||
}
|
||||
|
||||
func (h *RelayInbound) newPacketConnection(ctx context.Context, conn N.PacketConn, metadata adapter.InboundContext) error {
|
||||
destinationIndex, loaded := auth.UserFromContext[int](ctx)
|
||||
if !loaded {
|
||||
return os.ErrInvalid
|
||||
}
|
||||
destination := h.destinations[destinationIndex].Name
|
||||
if destination == "" {
|
||||
destination = F.ToString(destinationIndex)
|
||||
} else {
|
||||
metadata.User = destination
|
||||
}
|
||||
ctx = log.ContextWithNewID(ctx)
|
||||
h.logger.InfoContext(ctx, "[", destination, "] inbound packet connection from ", metadata.Source)
|
||||
h.logger.InfoContext(ctx, "[", destination, "] inbound packet connection to ", metadata.Destination)
|
||||
metadata.Inbound = h.Tag()
|
||||
metadata.InboundType = h.Type()
|
||||
metadata.InboundDetour = h.listener.ListenOptions().Detour
|
||||
metadata.InboundOptions = h.listener.ListenOptions().InboundOptions
|
||||
return h.router.RoutePacketConnection(ctx, conn, metadata)
|
||||
}
|
||||
|
||||
func (h *RelayInbound) NewError(ctx context.Context, err error) {
|
||||
NewError(h.logger, ctx, err)
|
||||
}
|
179
protocol/shadowsocks/outbound.go
Normal file
179
protocol/shadowsocks/outbound.go
Normal file
|
@ -0,0 +1,179 @@
|
|||
package shadowsocks
|
||||
|
||||
import (
|
||||
"context"
|
||||
"net"
|
||||
|
||||
"github.com/sagernet/sing-box/adapter"
|
||||
"github.com/sagernet/sing-box/adapter/outbound"
|
||||
"github.com/sagernet/sing-box/common/dialer"
|
||||
"github.com/sagernet/sing-box/common/mux"
|
||||
C "github.com/sagernet/sing-box/constant"
|
||||
"github.com/sagernet/sing-box/log"
|
||||
"github.com/sagernet/sing-box/option"
|
||||
"github.com/sagernet/sing-box/transport/sip003"
|
||||
"github.com/sagernet/sing-shadowsocks2"
|
||||
"github.com/sagernet/sing/common"
|
||||
"github.com/sagernet/sing/common/bufio"
|
||||
E "github.com/sagernet/sing/common/exceptions"
|
||||
"github.com/sagernet/sing/common/logger"
|
||||
M "github.com/sagernet/sing/common/metadata"
|
||||
N "github.com/sagernet/sing/common/network"
|
||||
"github.com/sagernet/sing/common/uot"
|
||||
)
|
||||
|
||||
func RegisterOutbound(registry *outbound.Registry) {
|
||||
outbound.Register[option.ShadowsocksOutboundOptions](registry, C.TypeShadowsocks, NewOutbound)
|
||||
}
|
||||
|
||||
type Outbound struct {
|
||||
outbound.Adapter
|
||||
logger logger.ContextLogger
|
||||
dialer N.Dialer
|
||||
method shadowsocks.Method
|
||||
serverAddr M.Socksaddr
|
||||
plugin sip003.Plugin
|
||||
uotClient *uot.Client
|
||||
multiplexDialer *mux.Client
|
||||
}
|
||||
|
||||
func NewOutbound(ctx context.Context, router adapter.Router, logger log.ContextLogger, tag string, options option.ShadowsocksOutboundOptions) (adapter.Outbound, error) {
|
||||
method, err := shadowsocks.CreateMethod(ctx, options.Method, shadowsocks.MethodOptions{
|
||||
Password: options.Password,
|
||||
})
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
outboundDialer, err := dialer.New(router, options.DialerOptions)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
outbound := &Outbound{
|
||||
Adapter: outbound.NewAdapterWithDialerOptions(C.TypeShadowsocks, options.Network.Build(), tag, options.DialerOptions),
|
||||
logger: logger,
|
||||
dialer: outboundDialer,
|
||||
method: method,
|
||||
serverAddr: options.ServerOptions.Build(),
|
||||
}
|
||||
if options.Plugin != "" {
|
||||
outbound.plugin, err = sip003.CreatePlugin(ctx, options.Plugin, options.PluginOptions, router, outbound.dialer, outbound.serverAddr)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
}
|
||||
uotOptions := common.PtrValueOrDefault(options.UDPOverTCP)
|
||||
if !uotOptions.Enabled {
|
||||
outbound.multiplexDialer, err = mux.NewClientWithOptions((*shadowsocksDialer)(outbound), logger, common.PtrValueOrDefault(options.Multiplex))
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
}
|
||||
if uotOptions.Enabled {
|
||||
outbound.uotClient = &uot.Client{
|
||||
Dialer: (*shadowsocksDialer)(outbound),
|
||||
Version: uotOptions.Version,
|
||||
}
|
||||
}
|
||||
return outbound, nil
|
||||
}
|
||||
|
||||
func (h *Outbound) DialContext(ctx context.Context, network string, destination M.Socksaddr) (net.Conn, error) {
|
||||
ctx, metadata := adapter.ExtendContext(ctx)
|
||||
metadata.Outbound = h.Tag()
|
||||
metadata.Destination = destination
|
||||
if h.multiplexDialer == nil {
|
||||
switch N.NetworkName(network) {
|
||||
case N.NetworkTCP:
|
||||
h.logger.InfoContext(ctx, "outbound connection to ", destination)
|
||||
case N.NetworkUDP:
|
||||
if h.uotClient != nil {
|
||||
h.logger.InfoContext(ctx, "outbound UoT connect packet connection to ", destination)
|
||||
return h.uotClient.DialContext(ctx, network, destination)
|
||||
} else {
|
||||
h.logger.InfoContext(ctx, "outbound packet connection to ", destination)
|
||||
}
|
||||
}
|
||||
return (*shadowsocksDialer)(h).DialContext(ctx, network, destination)
|
||||
} else {
|
||||
switch N.NetworkName(network) {
|
||||
case N.NetworkTCP:
|
||||
h.logger.InfoContext(ctx, "outbound multiplex connection to ", destination)
|
||||
case N.NetworkUDP:
|
||||
h.logger.InfoContext(ctx, "outbound multiplex packet connection to ", destination)
|
||||
}
|
||||
return h.multiplexDialer.DialContext(ctx, network, destination)
|
||||
}
|
||||
}
|
||||
|
||||
func (h *Outbound) ListenPacket(ctx context.Context, destination M.Socksaddr) (net.PacketConn, error) {
|
||||
ctx, metadata := adapter.ExtendContext(ctx)
|
||||
metadata.Outbound = h.Tag()
|
||||
metadata.Destination = destination
|
||||
if h.multiplexDialer == nil {
|
||||
if h.uotClient != nil {
|
||||
h.logger.InfoContext(ctx, "outbound UoT packet connection to ", destination)
|
||||
return h.uotClient.ListenPacket(ctx, destination)
|
||||
} else {
|
||||
h.logger.InfoContext(ctx, "outbound packet connection to ", destination)
|
||||
}
|
||||
h.logger.InfoContext(ctx, "outbound packet connection to ", destination)
|
||||
return (*shadowsocksDialer)(h).ListenPacket(ctx, destination)
|
||||
} else {
|
||||
h.logger.InfoContext(ctx, "outbound multiplex packet connection to ", destination)
|
||||
return h.multiplexDialer.ListenPacket(ctx, destination)
|
||||
}
|
||||
}
|
||||
|
||||
func (h *Outbound) InterfaceUpdated() {
|
||||
if h.multiplexDialer != nil {
|
||||
h.multiplexDialer.Reset()
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
func (h *Outbound) Close() error {
|
||||
return common.Close(common.PtrOrNil(h.multiplexDialer))
|
||||
}
|
||||
|
||||
var _ N.Dialer = (*shadowsocksDialer)(nil)
|
||||
|
||||
type shadowsocksDialer Outbound
|
||||
|
||||
func (h *shadowsocksDialer) DialContext(ctx context.Context, network string, destination M.Socksaddr) (net.Conn, error) {
|
||||
ctx, metadata := adapter.ExtendContext(ctx)
|
||||
metadata.Outbound = h.Tag()
|
||||
metadata.Destination = destination
|
||||
switch N.NetworkName(network) {
|
||||
case N.NetworkTCP:
|
||||
var outConn net.Conn
|
||||
var err error
|
||||
if h.plugin != nil {
|
||||
outConn, err = h.plugin.DialContext(ctx)
|
||||
} else {
|
||||
outConn, err = h.dialer.DialContext(ctx, N.NetworkTCP, h.serverAddr)
|
||||
}
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return h.method.DialEarlyConn(outConn, destination), nil
|
||||
case N.NetworkUDP:
|
||||
outConn, err := h.dialer.DialContext(ctx, N.NetworkUDP, h.serverAddr)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return bufio.NewBindPacketConn(h.method.DialPacketConn(outConn), destination), nil
|
||||
default:
|
||||
return nil, E.Extend(N.ErrUnknownNetwork, network)
|
||||
}
|
||||
}
|
||||
|
||||
func (h *shadowsocksDialer) ListenPacket(ctx context.Context, destination M.Socksaddr) (net.PacketConn, error) {
|
||||
ctx, metadata := adapter.ExtendContext(ctx)
|
||||
metadata.Outbound = h.Tag()
|
||||
metadata.Destination = destination
|
||||
outConn, err := h.dialer.DialContext(ctx, N.NetworkUDP, h.serverAddr)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return h.method.DialPacketConn(outConn), nil
|
||||
}
|
120
protocol/shadowtls/inbound.go
Normal file
120
protocol/shadowtls/inbound.go
Normal file
|
@ -0,0 +1,120 @@
|
|||
package shadowtls
|
||||
|
||||
import (
|
||||
"context"
|
||||
"net"
|
||||
|
||||
"github.com/sagernet/sing-box/adapter"
|
||||
"github.com/sagernet/sing-box/adapter/inbound"
|
||||
"github.com/sagernet/sing-box/common/dialer"
|
||||
"github.com/sagernet/sing-box/common/listener"
|
||||
C "github.com/sagernet/sing-box/constant"
|
||||
"github.com/sagernet/sing-box/log"
|
||||
"github.com/sagernet/sing-box/option"
|
||||
"github.com/sagernet/sing-shadowtls"
|
||||
"github.com/sagernet/sing/common"
|
||||
"github.com/sagernet/sing/common/auth"
|
||||
E "github.com/sagernet/sing/common/exceptions"
|
||||
"github.com/sagernet/sing/common/logger"
|
||||
N "github.com/sagernet/sing/common/network"
|
||||
)
|
||||
|
||||
func RegisterInbound(registry *inbound.Registry) {
|
||||
inbound.Register[option.ShadowTLSInboundOptions](registry, C.TypeShadowTLS, NewInbound)
|
||||
}
|
||||
|
||||
type Inbound struct {
|
||||
inbound.Adapter
|
||||
router adapter.Router
|
||||
logger logger.ContextLogger
|
||||
listener *listener.Listener
|
||||
service *shadowtls.Service
|
||||
}
|
||||
|
||||
func NewInbound(ctx context.Context, router adapter.Router, logger log.ContextLogger, tag string, options option.ShadowTLSInboundOptions) (adapter.Inbound, error) {
|
||||
inbound := &Inbound{
|
||||
Adapter: inbound.NewAdapter(C.TypeShadowTLS, tag),
|
||||
router: router,
|
||||
logger: logger,
|
||||
}
|
||||
|
||||
if options.Version == 0 {
|
||||
options.Version = 1
|
||||
}
|
||||
|
||||
var handshakeForServerName map[string]shadowtls.HandshakeConfig
|
||||
if options.Version > 1 {
|
||||
handshakeForServerName = make(map[string]shadowtls.HandshakeConfig)
|
||||
for serverName, serverOptions := range options.HandshakeForServerName {
|
||||
handshakeDialer, err := dialer.New(router, serverOptions.DialerOptions)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
handshakeForServerName[serverName] = shadowtls.HandshakeConfig{
|
||||
Server: serverOptions.ServerOptions.Build(),
|
||||
Dialer: handshakeDialer,
|
||||
}
|
||||
}
|
||||
}
|
||||
handshakeDialer, err := dialer.New(router, options.Handshake.DialerOptions)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
service, err := shadowtls.NewService(shadowtls.ServiceConfig{
|
||||
Version: options.Version,
|
||||
Password: options.Password,
|
||||
Users: common.Map(options.Users, func(it option.ShadowTLSUser) shadowtls.User {
|
||||
return (shadowtls.User)(it)
|
||||
}),
|
||||
Handshake: shadowtls.HandshakeConfig{
|
||||
Server: options.Handshake.ServerOptions.Build(),
|
||||
Dialer: handshakeDialer,
|
||||
},
|
||||
HandshakeForServerName: handshakeForServerName,
|
||||
StrictMode: options.StrictMode,
|
||||
Handler: adapter.NewUpstreamContextHandler(inbound.newConnection, nil, nil),
|
||||
Logger: logger,
|
||||
})
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
inbound.service = service
|
||||
inbound.listener = listener.New(listener.Options{
|
||||
Context: ctx,
|
||||
Logger: logger,
|
||||
Network: []string{N.NetworkTCP},
|
||||
Listen: options.ListenOptions,
|
||||
ConnectionHandler: inbound,
|
||||
})
|
||||
return inbound, nil
|
||||
}
|
||||
|
||||
func (h *Inbound) Start() error {
|
||||
return h.listener.Start()
|
||||
}
|
||||
|
||||
func (h *Inbound) Close() error {
|
||||
return h.listener.Close()
|
||||
}
|
||||
|
||||
func (h *Inbound) NewConnection(ctx context.Context, conn net.Conn, metadata adapter.InboundContext) error {
|
||||
return h.service.NewConnection(adapter.WithContext(log.ContextWithNewID(ctx), &metadata), conn, adapter.UpstreamMetadata(metadata))
|
||||
}
|
||||
|
||||
func (h *Inbound) newConnection(ctx context.Context, conn net.Conn, metadata adapter.InboundContext) error {
|
||||
if userName, _ := auth.UserFromContext[string](ctx); userName != "" {
|
||||
metadata.User = userName
|
||||
h.logger.InfoContext(ctx, "[", userName, "] inbound connection to ", metadata.Destination)
|
||||
} else {
|
||||
h.logger.InfoContext(ctx, "inbound connection to ", metadata.Destination)
|
||||
}
|
||||
return h.router.RouteConnection(ctx, conn, metadata)
|
||||
}
|
||||
|
||||
func (h *Inbound) NewConnectionEx(ctx context.Context, conn net.Conn, metadata adapter.InboundContext, onClose N.CloseHandlerFunc) {
|
||||
err := h.NewConnection(ctx, conn, metadata)
|
||||
N.CloseOnHandshakeFailure(conn, onClose, err)
|
||||
if err != nil {
|
||||
h.logger.ErrorContext(ctx, E.Cause(err, "process connection from ", metadata.Source))
|
||||
}
|
||||
}
|
104
protocol/shadowtls/outbound.go
Normal file
104
protocol/shadowtls/outbound.go
Normal file
|
@ -0,0 +1,104 @@
|
|||
package shadowtls
|
||||
|
||||
import (
|
||||
"context"
|
||||
"net"
|
||||
"os"
|
||||
|
||||
"github.com/sagernet/sing-box/adapter"
|
||||
"github.com/sagernet/sing-box/adapter/outbound"
|
||||
"github.com/sagernet/sing-box/common/dialer"
|
||||
"github.com/sagernet/sing-box/common/tls"
|
||||
C "github.com/sagernet/sing-box/constant"
|
||||
"github.com/sagernet/sing-box/log"
|
||||
"github.com/sagernet/sing-box/option"
|
||||
"github.com/sagernet/sing-shadowtls"
|
||||
"github.com/sagernet/sing/common"
|
||||
M "github.com/sagernet/sing/common/metadata"
|
||||
N "github.com/sagernet/sing/common/network"
|
||||
)
|
||||
|
||||
func RegisterOutbound(registry *outbound.Registry) {
|
||||
outbound.Register[option.ShadowTLSOutboundOptions](registry, C.TypeShadowTLS, NewOutbound)
|
||||
}
|
||||
|
||||
type Outbound struct {
|
||||
outbound.Adapter
|
||||
client *shadowtls.Client
|
||||
}
|
||||
|
||||
func NewOutbound(ctx context.Context, router adapter.Router, logger log.ContextLogger, tag string, options option.ShadowTLSOutboundOptions) (adapter.Outbound, error) {
|
||||
outbound := &Outbound{
|
||||
Adapter: outbound.NewAdapterWithDialerOptions(C.TypeShadowTLS, []string{N.NetworkTCP}, tag, options.DialerOptions),
|
||||
}
|
||||
if options.TLS == nil || !options.TLS.Enabled {
|
||||
return nil, C.ErrTLSRequired
|
||||
}
|
||||
|
||||
if options.Version == 0 {
|
||||
options.Version = 1
|
||||
}
|
||||
|
||||
if options.Version == 1 {
|
||||
options.TLS.MinVersion = "1.2"
|
||||
options.TLS.MaxVersion = "1.2"
|
||||
}
|
||||
tlsConfig, err := tls.NewClient(ctx, options.Server, common.PtrValueOrDefault(options.TLS))
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
var tlsHandshakeFunc shadowtls.TLSHandshakeFunc
|
||||
switch options.Version {
|
||||
case 1, 2:
|
||||
tlsHandshakeFunc = func(ctx context.Context, conn net.Conn, _ shadowtls.TLSSessionIDGeneratorFunc) error {
|
||||
return common.Error(tls.ClientHandshake(ctx, conn, tlsConfig))
|
||||
}
|
||||
case 3:
|
||||
if idConfig, loaded := tlsConfig.(tls.WithSessionIDGenerator); loaded {
|
||||
tlsHandshakeFunc = func(ctx context.Context, conn net.Conn, sessionIDGenerator shadowtls.TLSSessionIDGeneratorFunc) error {
|
||||
idConfig.SetSessionIDGenerator(sessionIDGenerator)
|
||||
return common.Error(tls.ClientHandshake(ctx, conn, tlsConfig))
|
||||
}
|
||||
} else {
|
||||
stdTLSConfig, err := tlsConfig.Config()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
tlsHandshakeFunc = shadowtls.DefaultTLSHandshakeFunc(options.Password, stdTLSConfig)
|
||||
}
|
||||
}
|
||||
outboundDialer, err := dialer.New(router, options.DialerOptions)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
client, err := shadowtls.NewClient(shadowtls.ClientConfig{
|
||||
Version: options.Version,
|
||||
Password: options.Password,
|
||||
Server: options.ServerOptions.Build(),
|
||||
Dialer: outboundDialer,
|
||||
TLSHandshake: tlsHandshakeFunc,
|
||||
Logger: logger,
|
||||
})
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
outbound.client = client
|
||||
return outbound, nil
|
||||
}
|
||||
|
||||
func (h *Outbound) DialContext(ctx context.Context, network string, destination M.Socksaddr) (net.Conn, error) {
|
||||
ctx, metadata := adapter.ExtendContext(ctx)
|
||||
metadata.Outbound = h.Tag()
|
||||
metadata.Destination = destination
|
||||
switch N.NetworkName(network) {
|
||||
case N.NetworkTCP:
|
||||
return h.client.DialContext(ctx)
|
||||
default:
|
||||
return nil, os.ErrInvalid
|
||||
}
|
||||
}
|
||||
|
||||
func (h *Outbound) ListenPacket(ctx context.Context, destination M.Socksaddr) (net.PacketConn, error) {
|
||||
return nil, os.ErrInvalid
|
||||
}
|
91
protocol/socks/inbound.go
Normal file
91
protocol/socks/inbound.go
Normal file
|
@ -0,0 +1,91 @@
|
|||
package socks
|
||||
|
||||
import (
|
||||
std_bufio "bufio"
|
||||
"context"
|
||||
"net"
|
||||
|
||||
"github.com/sagernet/sing-box/adapter"
|
||||
"github.com/sagernet/sing-box/adapter/inbound"
|
||||
"github.com/sagernet/sing-box/common/listener"
|
||||
"github.com/sagernet/sing-box/common/uot"
|
||||
C "github.com/sagernet/sing-box/constant"
|
||||
"github.com/sagernet/sing-box/log"
|
||||
"github.com/sagernet/sing-box/option"
|
||||
"github.com/sagernet/sing/common/auth"
|
||||
E "github.com/sagernet/sing/common/exceptions"
|
||||
"github.com/sagernet/sing/common/logger"
|
||||
N "github.com/sagernet/sing/common/network"
|
||||
"github.com/sagernet/sing/protocol/socks"
|
||||
)
|
||||
|
||||
func RegisterInbound(registry *inbound.Registry) {
|
||||
inbound.Register[option.SocksInboundOptions](registry, C.TypeSOCKS, NewInbound)
|
||||
}
|
||||
|
||||
var _ adapter.TCPInjectableInbound = (*Inbound)(nil)
|
||||
|
||||
type Inbound struct {
|
||||
inbound.Adapter
|
||||
router adapter.ConnectionRouterEx
|
||||
logger logger.ContextLogger
|
||||
listener *listener.Listener
|
||||
authenticator *auth.Authenticator
|
||||
}
|
||||
|
||||
func NewInbound(ctx context.Context, router adapter.Router, logger log.ContextLogger, tag string, options option.SocksInboundOptions) (adapter.Inbound, error) {
|
||||
inbound := &Inbound{
|
||||
Adapter: inbound.NewAdapter(C.TypeSOCKS, tag),
|
||||
router: uot.NewRouter(router, logger),
|
||||
logger: logger,
|
||||
authenticator: auth.NewAuthenticator(options.Users),
|
||||
}
|
||||
inbound.listener = listener.New(listener.Options{
|
||||
Context: ctx,
|
||||
Logger: logger,
|
||||
Network: []string{N.NetworkTCP},
|
||||
Listen: options.ListenOptions,
|
||||
ConnectionHandler: inbound,
|
||||
})
|
||||
return inbound, nil
|
||||
}
|
||||
|
||||
func (h *Inbound) Start() error {
|
||||
return h.listener.Start()
|
||||
}
|
||||
|
||||
func (h *Inbound) Close() error {
|
||||
return h.listener.Close()
|
||||
}
|
||||
|
||||
func (h *Inbound) NewConnectionEx(ctx context.Context, conn net.Conn, metadata adapter.InboundContext, onClose N.CloseHandlerFunc) {
|
||||
err := socks.HandleConnectionEx(ctx, conn, std_bufio.NewReader(conn), h.authenticator, nil, adapter.NewUpstreamHandlerEx(metadata, h.newUserConnection, h.streamUserPacketConnection), metadata.Source, metadata.Destination, onClose)
|
||||
N.CloseOnHandshakeFailure(conn, onClose, err)
|
||||
if err != nil {
|
||||
h.logger.ErrorContext(ctx, E.Cause(err, "process connection from ", metadata.Source))
|
||||
}
|
||||
}
|
||||
|
||||
func (h *Inbound) newUserConnection(ctx context.Context, conn net.Conn, metadata adapter.InboundContext, onClose N.CloseHandlerFunc) {
|
||||
user, loaded := auth.UserFromContext[string](ctx)
|
||||
if !loaded {
|
||||
h.logger.InfoContext(ctx, "inbound connection to ", metadata.Destination)
|
||||
h.router.RouteConnectionEx(ctx, conn, metadata, onClose)
|
||||
return
|
||||
}
|
||||
metadata.User = user
|
||||
h.logger.InfoContext(ctx, "[", user, "] inbound connection to ", metadata.Destination)
|
||||
h.router.RouteConnectionEx(ctx, conn, metadata, onClose)
|
||||
}
|
||||
|
||||
func (h *Inbound) streamUserPacketConnection(ctx context.Context, conn N.PacketConn, metadata adapter.InboundContext, onClose N.CloseHandlerFunc) {
|
||||
user, loaded := auth.UserFromContext[string](ctx)
|
||||
if !loaded {
|
||||
h.logger.InfoContext(ctx, "inbound packet connection to ", metadata.Destination)
|
||||
h.router.RoutePacketConnectionEx(ctx, conn, metadata, onClose)
|
||||
return
|
||||
}
|
||||
metadata.User = user
|
||||
h.logger.InfoContext(ctx, "[", user, "] inbound packet connection to ", metadata.Destination)
|
||||
h.router.RoutePacketConnectionEx(ctx, conn, metadata, onClose)
|
||||
}
|
137
protocol/socks/outbound.go
Normal file
137
protocol/socks/outbound.go
Normal file
|
@ -0,0 +1,137 @@
|
|||
package socks
|
||||
|
||||
import (
|
||||
"context"
|
||||
"net"
|
||||
|
||||
"github.com/sagernet/sing-box/adapter"
|
||||
"github.com/sagernet/sing-box/adapter/outbound"
|
||||
"github.com/sagernet/sing-box/common/dialer"
|
||||
C "github.com/sagernet/sing-box/constant"
|
||||
"github.com/sagernet/sing-box/log"
|
||||
"github.com/sagernet/sing-box/option"
|
||||
"github.com/sagernet/sing-dns"
|
||||
"github.com/sagernet/sing/common"
|
||||
E "github.com/sagernet/sing/common/exceptions"
|
||||
"github.com/sagernet/sing/common/logger"
|
||||
M "github.com/sagernet/sing/common/metadata"
|
||||
N "github.com/sagernet/sing/common/network"
|
||||
"github.com/sagernet/sing/common/uot"
|
||||
"github.com/sagernet/sing/protocol/socks"
|
||||
)
|
||||
|
||||
func RegisterOutbound(registry *outbound.Registry) {
|
||||
outbound.Register[option.SOCKSOutboundOptions](registry, C.TypeSOCKS, NewOutbound)
|
||||
}
|
||||
|
||||
var _ adapter.Outbound = (*Outbound)(nil)
|
||||
|
||||
type Outbound struct {
|
||||
outbound.Adapter
|
||||
router adapter.Router
|
||||
logger logger.ContextLogger
|
||||
client *socks.Client
|
||||
resolve bool
|
||||
uotClient *uot.Client
|
||||
}
|
||||
|
||||
func NewOutbound(ctx context.Context, router adapter.Router, logger log.ContextLogger, tag string, options option.SOCKSOutboundOptions) (adapter.Outbound, error) {
|
||||
var version socks.Version
|
||||
var err error
|
||||
if options.Version != "" {
|
||||
version, err = socks.ParseVersion(options.Version)
|
||||
} else {
|
||||
version = socks.Version5
|
||||
}
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
outboundDialer, err := dialer.New(router, options.DialerOptions)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
outbound := &Outbound{
|
||||
Adapter: outbound.NewAdapterWithDialerOptions(C.TypeSOCKS, options.Network.Build(), tag, options.DialerOptions),
|
||||
router: router,
|
||||
logger: logger,
|
||||
client: socks.NewClient(outboundDialer, options.ServerOptions.Build(), version, options.Username, options.Password),
|
||||
resolve: version == socks.Version4,
|
||||
}
|
||||
uotOptions := common.PtrValueOrDefault(options.UDPOverTCP)
|
||||
if uotOptions.Enabled {
|
||||
outbound.uotClient = &uot.Client{
|
||||
Dialer: outbound.client,
|
||||
Version: uotOptions.Version,
|
||||
}
|
||||
}
|
||||
return outbound, nil
|
||||
}
|
||||
|
||||
func (h *Outbound) DialContext(ctx context.Context, network string, destination M.Socksaddr) (net.Conn, error) {
|
||||
ctx, metadata := adapter.ExtendContext(ctx)
|
||||
metadata.Outbound = h.Tag()
|
||||
metadata.Destination = destination
|
||||
switch N.NetworkName(network) {
|
||||
case N.NetworkTCP:
|
||||
h.logger.InfoContext(ctx, "outbound connection to ", destination)
|
||||
case N.NetworkUDP:
|
||||
if h.uotClient != nil {
|
||||
h.logger.InfoContext(ctx, "outbound UoT connect packet connection to ", destination)
|
||||
return h.uotClient.DialContext(ctx, network, destination)
|
||||
}
|
||||
h.logger.InfoContext(ctx, "outbound packet connection to ", destination)
|
||||
default:
|
||||
return nil, E.Extend(N.ErrUnknownNetwork, network)
|
||||
}
|
||||
if h.resolve && destination.IsFqdn() {
|
||||
destinationAddresses, err := h.router.LookupDefault(ctx, destination.Fqdn)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return N.DialSerial(ctx, h.client, network, destination, destinationAddresses)
|
||||
}
|
||||
return h.client.DialContext(ctx, network, destination)
|
||||
}
|
||||
|
||||
func (h *Outbound) ListenPacket(ctx context.Context, destination M.Socksaddr) (net.PacketConn, error) {
|
||||
ctx, metadata := adapter.ExtendContext(ctx)
|
||||
metadata.Outbound = h.Tag()
|
||||
metadata.Destination = destination
|
||||
if h.uotClient != nil {
|
||||
h.logger.InfoContext(ctx, "outbound UoT packet connection to ", destination)
|
||||
return h.uotClient.ListenPacket(ctx, destination)
|
||||
}
|
||||
if h.resolve && destination.IsFqdn() {
|
||||
destinationAddresses, err := h.router.LookupDefault(ctx, destination.Fqdn)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
packetConn, _, err := N.ListenSerial(ctx, h.client, destination, destinationAddresses)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return packetConn, nil
|
||||
}
|
||||
h.logger.InfoContext(ctx, "outbound packet connection to ", destination)
|
||||
return h.client.ListenPacket(ctx, destination)
|
||||
}
|
||||
|
||||
// TODO
|
||||
// Deprecated
|
||||
func (h *Outbound) NewConnection(ctx context.Context, conn net.Conn, metadata adapter.InboundContext) error {
|
||||
if h.resolve {
|
||||
return outbound.NewDirectConnection(ctx, h.router, h, conn, metadata, dns.DomainStrategyUseIPv4)
|
||||
} else {
|
||||
return outbound.NewConnection(ctx, h, conn, metadata)
|
||||
}
|
||||
}
|
||||
|
||||
// TODO
|
||||
// Deprecated
|
||||
func (h *Outbound) NewPacketConnection(ctx context.Context, conn N.PacketConn, metadata adapter.InboundContext) error {
|
||||
if h.resolve {
|
||||
return outbound.NewDirectPacketConnection(ctx, h.router, h, conn, metadata, dns.DomainStrategyUseIPv4)
|
||||
} else {
|
||||
return outbound.NewPacketConnection(ctx, h, conn, metadata)
|
||||
}
|
||||
}
|
199
protocol/ssh/outbound.go
Normal file
199
protocol/ssh/outbound.go
Normal file
|
@ -0,0 +1,199 @@
|
|||
package ssh
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"context"
|
||||
"encoding/base64"
|
||||
"math/rand"
|
||||
"net"
|
||||
"os"
|
||||
"strconv"
|
||||
"strings"
|
||||
"sync"
|
||||
|
||||
"github.com/sagernet/sing-box/adapter"
|
||||
"github.com/sagernet/sing-box/adapter/outbound"
|
||||
"github.com/sagernet/sing-box/common/dialer"
|
||||
C "github.com/sagernet/sing-box/constant"
|
||||
"github.com/sagernet/sing-box/log"
|
||||
"github.com/sagernet/sing-box/option"
|
||||
"github.com/sagernet/sing/common"
|
||||
E "github.com/sagernet/sing/common/exceptions"
|
||||
"github.com/sagernet/sing/common/logger"
|
||||
M "github.com/sagernet/sing/common/metadata"
|
||||
N "github.com/sagernet/sing/common/network"
|
||||
|
||||
"golang.org/x/crypto/ssh"
|
||||
)
|
||||
|
||||
func RegisterOutbound(registry *outbound.Registry) {
|
||||
outbound.Register[option.SSHOutboundOptions](registry, C.TypeSSH, NewOutbound)
|
||||
}
|
||||
|
||||
var _ adapter.InterfaceUpdateListener = (*Outbound)(nil)
|
||||
|
||||
type Outbound struct {
|
||||
outbound.Adapter
|
||||
ctx context.Context
|
||||
logger logger.ContextLogger
|
||||
dialer N.Dialer
|
||||
serverAddr M.Socksaddr
|
||||
user string
|
||||
hostKey []ssh.PublicKey
|
||||
hostKeyAlgorithms []string
|
||||
clientVersion string
|
||||
authMethod []ssh.AuthMethod
|
||||
clientAccess sync.Mutex
|
||||
clientConn net.Conn
|
||||
client *ssh.Client
|
||||
}
|
||||
|
||||
func NewOutbound(ctx context.Context, router adapter.Router, logger log.ContextLogger, tag string, options option.SSHOutboundOptions) (adapter.Outbound, error) {
|
||||
outboundDialer, err := dialer.New(router, options.DialerOptions)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
outbound := &Outbound{
|
||||
Adapter: outbound.NewAdapterWithDialerOptions(C.TypeSSH, []string{N.NetworkTCP}, tag, options.DialerOptions),
|
||||
ctx: ctx,
|
||||
logger: logger,
|
||||
dialer: outboundDialer,
|
||||
serverAddr: options.ServerOptions.Build(),
|
||||
user: options.User,
|
||||
hostKeyAlgorithms: options.HostKeyAlgorithms,
|
||||
clientVersion: options.ClientVersion,
|
||||
}
|
||||
if outbound.serverAddr.Port == 0 {
|
||||
outbound.serverAddr.Port = 22
|
||||
}
|
||||
if outbound.user == "" {
|
||||
outbound.user = "root"
|
||||
}
|
||||
if outbound.clientVersion == "" {
|
||||
outbound.clientVersion = randomVersion()
|
||||
}
|
||||
if options.Password != "" {
|
||||
outbound.authMethod = append(outbound.authMethod, ssh.Password(options.Password))
|
||||
}
|
||||
if len(options.PrivateKey) > 0 || options.PrivateKeyPath != "" {
|
||||
var privateKey []byte
|
||||
if len(options.PrivateKey) > 0 {
|
||||
privateKey = []byte(strings.Join(options.PrivateKey, "\n"))
|
||||
} else {
|
||||
var err error
|
||||
privateKey, err = os.ReadFile(os.ExpandEnv(options.PrivateKeyPath))
|
||||
if err != nil {
|
||||
return nil, E.Cause(err, "read private key")
|
||||
}
|
||||
}
|
||||
var signer ssh.Signer
|
||||
var err error
|
||||
if options.PrivateKeyPassphrase == "" {
|
||||
signer, err = ssh.ParsePrivateKey(privateKey)
|
||||
} else {
|
||||
signer, err = ssh.ParsePrivateKeyWithPassphrase(privateKey, []byte(options.PrivateKeyPassphrase))
|
||||
}
|
||||
if err != nil {
|
||||
return nil, E.Cause(err, "parse private key")
|
||||
}
|
||||
outbound.authMethod = append(outbound.authMethod, ssh.PublicKeys(signer))
|
||||
}
|
||||
if len(options.HostKey) > 0 {
|
||||
for _, hostKey := range options.HostKey {
|
||||
key, _, _, _, err := ssh.ParseAuthorizedKey([]byte(hostKey))
|
||||
if err != nil {
|
||||
return nil, E.New("parse host key ", key)
|
||||
}
|
||||
outbound.hostKey = append(outbound.hostKey, key)
|
||||
}
|
||||
}
|
||||
return outbound, nil
|
||||
}
|
||||
|
||||
func randomVersion() string {
|
||||
version := "SSH-2.0-OpenSSH_"
|
||||
if rand.Intn(2) == 0 {
|
||||
version += "7." + strconv.Itoa(rand.Intn(10))
|
||||
} else {
|
||||
version += "8." + strconv.Itoa(rand.Intn(9))
|
||||
}
|
||||
return version
|
||||
}
|
||||
|
||||
func (s *Outbound) connect() (*ssh.Client, error) {
|
||||
if s.client != nil {
|
||||
return s.client, nil
|
||||
}
|
||||
|
||||
s.clientAccess.Lock()
|
||||
defer s.clientAccess.Unlock()
|
||||
|
||||
if s.client != nil {
|
||||
return s.client, nil
|
||||
}
|
||||
|
||||
conn, err := s.dialer.DialContext(s.ctx, N.NetworkTCP, s.serverAddr)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
config := &ssh.ClientConfig{
|
||||
User: s.user,
|
||||
Auth: s.authMethod,
|
||||
ClientVersion: s.clientVersion,
|
||||
HostKeyAlgorithms: s.hostKeyAlgorithms,
|
||||
HostKeyCallback: func(hostname string, remote net.Addr, key ssh.PublicKey) error {
|
||||
if len(s.hostKey) == 0 {
|
||||
return nil
|
||||
}
|
||||
serverKey := key.Marshal()
|
||||
for _, hostKey := range s.hostKey {
|
||||
if bytes.Equal(serverKey, hostKey.Marshal()) {
|
||||
return nil
|
||||
}
|
||||
}
|
||||
return E.New("host key mismatch, server send ", key.Type(), " ", base64.StdEncoding.EncodeToString(serverKey))
|
||||
},
|
||||
}
|
||||
clientConn, chans, reqs, err := ssh.NewClientConn(conn, s.serverAddr.Addr.String(), config)
|
||||
if err != nil {
|
||||
conn.Close()
|
||||
return nil, E.Cause(err, "connect to ssh server")
|
||||
}
|
||||
|
||||
client := ssh.NewClient(clientConn, chans, reqs)
|
||||
|
||||
s.clientConn = conn
|
||||
s.client = client
|
||||
|
||||
go func() {
|
||||
client.Wait()
|
||||
conn.Close()
|
||||
s.clientAccess.Lock()
|
||||
s.client = nil
|
||||
s.clientConn = nil
|
||||
s.clientAccess.Unlock()
|
||||
}()
|
||||
|
||||
return client, nil
|
||||
}
|
||||
|
||||
func (s *Outbound) InterfaceUpdated() {
|
||||
common.Close(s.clientConn)
|
||||
return
|
||||
}
|
||||
|
||||
func (s *Outbound) Close() error {
|
||||
return common.Close(s.clientConn)
|
||||
}
|
||||
|
||||
func (s *Outbound) DialContext(ctx context.Context, network string, destination M.Socksaddr) (net.Conn, error) {
|
||||
client, err := s.connect()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return client.Dial(network, destination.String())
|
||||
}
|
||||
|
||||
func (s *Outbound) ListenPacket(ctx context.Context, destination M.Socksaddr) (net.PacketConn, error) {
|
||||
return nil, os.ErrInvalid
|
||||
}
|
212
protocol/tor/outbound.go
Normal file
212
protocol/tor/outbound.go
Normal file
|
@ -0,0 +1,212 @@
|
|||
package tor
|
||||
|
||||
import (
|
||||
"context"
|
||||
"net"
|
||||
"os"
|
||||
"path/filepath"
|
||||
"strings"
|
||||
|
||||
"github.com/sagernet/sing-box/adapter"
|
||||
"github.com/sagernet/sing-box/adapter/outbound"
|
||||
"github.com/sagernet/sing-box/common/dialer"
|
||||
C "github.com/sagernet/sing-box/constant"
|
||||
"github.com/sagernet/sing-box/log"
|
||||
"github.com/sagernet/sing-box/option"
|
||||
"github.com/sagernet/sing/common"
|
||||
E "github.com/sagernet/sing/common/exceptions"
|
||||
F "github.com/sagernet/sing/common/format"
|
||||
"github.com/sagernet/sing/common/logger"
|
||||
M "github.com/sagernet/sing/common/metadata"
|
||||
N "github.com/sagernet/sing/common/network"
|
||||
"github.com/sagernet/sing/common/rw"
|
||||
"github.com/sagernet/sing/protocol/socks"
|
||||
|
||||
"github.com/cretz/bine/control"
|
||||
"github.com/cretz/bine/tor"
|
||||
)
|
||||
|
||||
func RegisterOutbound(registry *outbound.Registry) {
|
||||
outbound.Register[option.TorOutboundOptions](registry, C.TypeTor, NewOutbound)
|
||||
}
|
||||
|
||||
type Outbound struct {
|
||||
outbound.Adapter
|
||||
ctx context.Context
|
||||
logger logger.ContextLogger
|
||||
proxy *ProxyListener
|
||||
startConf *tor.StartConf
|
||||
options map[string]string
|
||||
events chan control.Event
|
||||
instance *tor.Tor
|
||||
socksClient *socks.Client
|
||||
}
|
||||
|
||||
func NewOutbound(ctx context.Context, router adapter.Router, logger log.ContextLogger, tag string, options option.TorOutboundOptions) (adapter.Outbound, error) {
|
||||
var startConf tor.StartConf
|
||||
startConf.DataDir = os.ExpandEnv(options.DataDirectory)
|
||||
startConf.TempDataDirBase = os.TempDir()
|
||||
startConf.ExtraArgs = options.ExtraArgs
|
||||
if options.DataDirectory != "" {
|
||||
dataDirAbs, _ := filepath.Abs(startConf.DataDir)
|
||||
if geoIPPath := filepath.Join(dataDirAbs, "geoip"); rw.IsFile(geoIPPath) && !common.Contains(options.ExtraArgs, "--GeoIPFile") {
|
||||
options.ExtraArgs = append(options.ExtraArgs, "--GeoIPFile", geoIPPath)
|
||||
}
|
||||
if geoIP6Path := filepath.Join(dataDirAbs, "geoip6"); rw.IsFile(geoIP6Path) && !common.Contains(options.ExtraArgs, "--GeoIPv6File") {
|
||||
options.ExtraArgs = append(options.ExtraArgs, "--GeoIPv6File", geoIP6Path)
|
||||
}
|
||||
}
|
||||
if options.ExecutablePath != "" {
|
||||
startConf.ExePath = options.ExecutablePath
|
||||
startConf.ProcessCreator = nil
|
||||
startConf.UseEmbeddedControlConn = false
|
||||
}
|
||||
if startConf.DataDir != "" {
|
||||
torrcFile := filepath.Join(startConf.DataDir, "torrc")
|
||||
err := rw.MkdirParent(torrcFile)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if !rw.IsFile(torrcFile) {
|
||||
err := os.WriteFile(torrcFile, []byte(""), 0o600)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
}
|
||||
startConf.TorrcFile = torrcFile
|
||||
}
|
||||
outboundDialer, err := dialer.New(router, options.DialerOptions)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return &Outbound{
|
||||
Adapter: outbound.NewAdapterWithDialerOptions(C.TypeTor, []string{N.NetworkTCP}, tag, options.DialerOptions),
|
||||
ctx: ctx,
|
||||
logger: logger,
|
||||
proxy: NewProxyListener(ctx, logger, outboundDialer),
|
||||
startConf: &startConf,
|
||||
options: options.Options,
|
||||
}, nil
|
||||
}
|
||||
|
||||
func (t *Outbound) Start() error {
|
||||
err := t.start()
|
||||
if err != nil {
|
||||
t.Close()
|
||||
}
|
||||
return err
|
||||
}
|
||||
|
||||
var torLogEvents = []control.EventCode{
|
||||
control.EventCodeLogDebug,
|
||||
control.EventCodeLogErr,
|
||||
control.EventCodeLogInfo,
|
||||
control.EventCodeLogNotice,
|
||||
control.EventCodeLogWarn,
|
||||
}
|
||||
|
||||
func (t *Outbound) start() error {
|
||||
torInstance, err := tor.Start(t.ctx, t.startConf)
|
||||
if err != nil {
|
||||
return E.New(strings.ToLower(err.Error()))
|
||||
}
|
||||
t.instance = torInstance
|
||||
t.events = make(chan control.Event, 8)
|
||||
err = torInstance.Control.AddEventListener(t.events, torLogEvents...)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
go t.recvLoop()
|
||||
err = t.proxy.Start()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
proxyPort := "127.0.0.1:" + F.ToString(t.proxy.Port())
|
||||
proxyUsername := t.proxy.Username()
|
||||
proxyPassword := t.proxy.Password()
|
||||
t.logger.Trace("created upstream proxy at ", proxyPort)
|
||||
t.logger.Trace("upstream proxy username ", proxyUsername)
|
||||
t.logger.Trace("upstream proxy password ", proxyPassword)
|
||||
confOptions := []*control.KeyVal{
|
||||
control.NewKeyVal("Socks5Proxy", proxyPort),
|
||||
control.NewKeyVal("Socks5ProxyUsername", proxyUsername),
|
||||
control.NewKeyVal("Socks5ProxyPassword", proxyPassword),
|
||||
}
|
||||
err = torInstance.Control.ResetConf(confOptions...)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if len(t.options) > 0 {
|
||||
for key, value := range t.options {
|
||||
switch key {
|
||||
case "Socks5Proxy",
|
||||
"Socks5ProxyUsername",
|
||||
"Socks5ProxyPassword":
|
||||
continue
|
||||
}
|
||||
err = torInstance.Control.SetConf(control.NewKeyVal(key, value))
|
||||
if err != nil {
|
||||
return E.Cause(err, "set ", key, "=", value)
|
||||
}
|
||||
}
|
||||
}
|
||||
err = torInstance.EnableNetwork(t.ctx, true)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
info, err := torInstance.Control.GetInfo("net/listeners/socks")
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if len(info) != 1 || info[0].Key != "net/listeners/socks" {
|
||||
return E.New("get socks proxy address")
|
||||
}
|
||||
t.logger.Trace("obtained tor socks5 address ", info[0].Val)
|
||||
// TODO: set password for tor socks5 server if supported
|
||||
t.socksClient = socks.NewClient(N.SystemDialer, M.ParseSocksaddr(info[0].Val), socks.Version5, "", "")
|
||||
return nil
|
||||
}
|
||||
|
||||
func (t *Outbound) recvLoop() {
|
||||
for rawEvent := range t.events {
|
||||
switch event := rawEvent.(type) {
|
||||
case *control.LogEvent:
|
||||
event.Raw = strings.ToLower(event.Raw)
|
||||
switch event.Severity {
|
||||
case control.EventCodeLogDebug, control.EventCodeLogInfo:
|
||||
t.logger.Trace(event.Raw)
|
||||
case control.EventCodeLogNotice:
|
||||
if strings.Contains(event.Raw, "disablenetwork") || strings.Contains(event.Raw, "socks listener") {
|
||||
t.logger.Trace(event.Raw)
|
||||
continue
|
||||
}
|
||||
t.logger.Info(event.Raw)
|
||||
case control.EventCodeLogWarn:
|
||||
t.logger.Warn(event.Raw)
|
||||
case control.EventCodeLogErr:
|
||||
t.logger.Error(event.Raw)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (t *Outbound) Close() error {
|
||||
err := common.Close(
|
||||
common.PtrOrNil(t.proxy),
|
||||
common.PtrOrNil(t.instance),
|
||||
)
|
||||
if t.events != nil {
|
||||
close(t.events)
|
||||
t.events = nil
|
||||
}
|
||||
return err
|
||||
}
|
||||
|
||||
func (t *Outbound) DialContext(ctx context.Context, network string, destination M.Socksaddr) (net.Conn, error) {
|
||||
t.logger.InfoContext(ctx, "outbound connection to ", destination)
|
||||
return t.socksClient.DialContext(ctx, network, destination)
|
||||
}
|
||||
|
||||
func (t *Outbound) ListenPacket(ctx context.Context, destination M.Socksaddr) (net.PacketConn, error) {
|
||||
return nil, os.ErrInvalid
|
||||
}
|
119
protocol/tor/proxy.go
Normal file
119
protocol/tor/proxy.go
Normal file
|
@ -0,0 +1,119 @@
|
|||
package tor
|
||||
|
||||
import (
|
||||
"context"
|
||||
"crypto/rand"
|
||||
"encoding/hex"
|
||||
"net"
|
||||
|
||||
"github.com/sagernet/sing-box/adapter"
|
||||
"github.com/sagernet/sing-box/adapter/outbound"
|
||||
"github.com/sagernet/sing-box/log"
|
||||
"github.com/sagernet/sing/common"
|
||||
"github.com/sagernet/sing/common/auth"
|
||||
E "github.com/sagernet/sing/common/exceptions"
|
||||
M "github.com/sagernet/sing/common/metadata"
|
||||
N "github.com/sagernet/sing/common/network"
|
||||
"github.com/sagernet/sing/protocol/socks"
|
||||
)
|
||||
|
||||
type ProxyListener struct {
|
||||
ctx context.Context
|
||||
logger log.ContextLogger
|
||||
dialer N.Dialer
|
||||
tcpListener *net.TCPListener
|
||||
username string
|
||||
password string
|
||||
authenticator *auth.Authenticator
|
||||
}
|
||||
|
||||
func NewProxyListener(ctx context.Context, logger log.ContextLogger, dialer N.Dialer) *ProxyListener {
|
||||
var usernameB [64]byte
|
||||
var passwordB [64]byte
|
||||
rand.Read(usernameB[:])
|
||||
rand.Read(passwordB[:])
|
||||
username := hex.EncodeToString(usernameB[:])
|
||||
password := hex.EncodeToString(passwordB[:])
|
||||
return &ProxyListener{
|
||||
ctx: ctx,
|
||||
logger: logger,
|
||||
dialer: dialer,
|
||||
authenticator: auth.NewAuthenticator([]auth.User{{Username: username, Password: password}}),
|
||||
username: username,
|
||||
password: password,
|
||||
}
|
||||
}
|
||||
|
||||
func (l *ProxyListener) Start() error {
|
||||
tcpListener, err := net.ListenTCP("tcp", &net.TCPAddr{
|
||||
IP: net.IPv4(127, 0, 0, 1),
|
||||
})
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
l.tcpListener = tcpListener
|
||||
go l.acceptLoop()
|
||||
return nil
|
||||
}
|
||||
|
||||
func (l *ProxyListener) Port() uint16 {
|
||||
if l.tcpListener == nil {
|
||||
panic("start listener first")
|
||||
}
|
||||
return M.SocksaddrFromNet(l.tcpListener.Addr()).Port
|
||||
}
|
||||
|
||||
func (l *ProxyListener) Username() string {
|
||||
return l.username
|
||||
}
|
||||
|
||||
func (l *ProxyListener) Password() string {
|
||||
return l.password
|
||||
}
|
||||
|
||||
func (l *ProxyListener) Close() error {
|
||||
return common.Close(l.tcpListener)
|
||||
}
|
||||
|
||||
func (l *ProxyListener) acceptLoop() {
|
||||
for {
|
||||
tcpConn, err := l.tcpListener.AcceptTCP()
|
||||
if err != nil {
|
||||
return
|
||||
}
|
||||
ctx := log.ContextWithNewID(l.ctx)
|
||||
go func() {
|
||||
hErr := l.accept(ctx, tcpConn)
|
||||
if hErr != nil {
|
||||
if E.IsClosedOrCanceled(hErr) {
|
||||
l.logger.DebugContext(ctx, E.Cause(hErr, "proxy connection closed"))
|
||||
return
|
||||
}
|
||||
l.logger.ErrorContext(ctx, E.Cause(hErr, "proxy"))
|
||||
}
|
||||
}()
|
||||
}
|
||||
}
|
||||
|
||||
// TODO: migrate to new api
|
||||
//
|
||||
//nolint:staticcheck
|
||||
func (l *ProxyListener) accept(ctx context.Context, conn *net.TCPConn) error {
|
||||
return socks.HandleConnection(ctx, conn, l.authenticator, l, M.Metadata{})
|
||||
}
|
||||
|
||||
func (l *ProxyListener) NewConnection(ctx context.Context, conn net.Conn, upstreamMetadata M.Metadata) error {
|
||||
var metadata adapter.InboundContext
|
||||
metadata.Network = N.NetworkTCP
|
||||
metadata.Destination = upstreamMetadata.Destination
|
||||
l.logger.InfoContext(ctx, "proxy connection to ", metadata.Destination)
|
||||
return outbound.NewConnection(ctx, l.dialer, conn, metadata)
|
||||
}
|
||||
|
||||
func (l *ProxyListener) NewPacketConnection(ctx context.Context, conn N.PacketConn, upstreamMetadata M.Metadata) error {
|
||||
var metadata adapter.InboundContext
|
||||
metadata.Network = N.NetworkUDP
|
||||
metadata.Destination = upstreamMetadata.Destination
|
||||
l.logger.InfoContext(ctx, "proxy packet connection to ", metadata.Destination)
|
||||
return outbound.NewPacketConnection(ctx, l.dialer, conn, metadata)
|
||||
}
|
244
protocol/trojan/inbound.go
Normal file
244
protocol/trojan/inbound.go
Normal file
|
@ -0,0 +1,244 @@
|
|||
package trojan
|
||||
|
||||
import (
|
||||
"context"
|
||||
"net"
|
||||
"os"
|
||||
|
||||
"github.com/sagernet/sing-box/adapter"
|
||||
"github.com/sagernet/sing-box/adapter/inbound"
|
||||
"github.com/sagernet/sing-box/common/listener"
|
||||
"github.com/sagernet/sing-box/common/mux"
|
||||
"github.com/sagernet/sing-box/common/tls"
|
||||
C "github.com/sagernet/sing-box/constant"
|
||||
"github.com/sagernet/sing-box/log"
|
||||
"github.com/sagernet/sing-box/option"
|
||||
"github.com/sagernet/sing-box/transport/trojan"
|
||||
"github.com/sagernet/sing-box/transport/v2ray"
|
||||
"github.com/sagernet/sing/common"
|
||||
"github.com/sagernet/sing/common/auth"
|
||||
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"
|
||||
)
|
||||
|
||||
func RegisterInbound(registry *inbound.Registry) {
|
||||
inbound.Register[option.TrojanInboundOptions](registry, C.TypeTrojan, NewInbound)
|
||||
}
|
||||
|
||||
var _ adapter.TCPInjectableInbound = (*Inbound)(nil)
|
||||
|
||||
type Inbound struct {
|
||||
inbound.Adapter
|
||||
router adapter.ConnectionRouterEx
|
||||
logger log.ContextLogger
|
||||
listener *listener.Listener
|
||||
service *trojan.Service[int]
|
||||
users []option.TrojanUser
|
||||
tlsConfig tls.ServerConfig
|
||||
fallbackAddr M.Socksaddr
|
||||
fallbackAddrTLSNextProto map[string]M.Socksaddr
|
||||
transport adapter.V2RayServerTransport
|
||||
}
|
||||
|
||||
func NewInbound(ctx context.Context, router adapter.Router, logger log.ContextLogger, tag string, options option.TrojanInboundOptions) (adapter.Inbound, error) {
|
||||
inbound := &Inbound{
|
||||
Adapter: inbound.NewAdapter(C.TypeTrojan, tag),
|
||||
router: router,
|
||||
logger: logger,
|
||||
users: options.Users,
|
||||
}
|
||||
if options.TLS != nil {
|
||||
tlsConfig, err := tls.NewServer(ctx, logger, common.PtrValueOrDefault(options.TLS))
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
inbound.tlsConfig = tlsConfig
|
||||
}
|
||||
var fallbackHandler N.TCPConnectionHandler
|
||||
if options.Fallback != nil && options.Fallback.Server != "" || len(options.FallbackForALPN) > 0 {
|
||||
if options.Fallback != nil && options.Fallback.Server != "" {
|
||||
inbound.fallbackAddr = options.Fallback.Build()
|
||||
if !inbound.fallbackAddr.IsValid() {
|
||||
return nil, E.New("invalid fallback address: ", inbound.fallbackAddr)
|
||||
}
|
||||
}
|
||||
if len(options.FallbackForALPN) > 0 {
|
||||
if inbound.tlsConfig == nil {
|
||||
return nil, E.New("fallback for ALPN is not supported without TLS")
|
||||
}
|
||||
fallbackAddrNextProto := make(map[string]M.Socksaddr)
|
||||
for nextProto, destination := range options.FallbackForALPN {
|
||||
fallbackAddr := destination.Build()
|
||||
if !fallbackAddr.IsValid() {
|
||||
return nil, E.New("invalid fallback address for ALPN ", nextProto, ": ", fallbackAddr)
|
||||
}
|
||||
fallbackAddrNextProto[nextProto] = fallbackAddr
|
||||
}
|
||||
inbound.fallbackAddrTLSNextProto = fallbackAddrNextProto
|
||||
}
|
||||
fallbackHandler = adapter.NewUpstreamContextHandler(inbound.fallbackConnection, nil, nil)
|
||||
}
|
||||
service := trojan.NewService[int](adapter.NewUpstreamContextHandler(inbound.newConnection, inbound.newPacketConnection, nil), fallbackHandler, logger)
|
||||
err := service.UpdateUsers(common.MapIndexed(options.Users, func(index int, it option.TrojanUser) int {
|
||||
return index
|
||||
}), common.Map(options.Users, func(it option.TrojanUser) string {
|
||||
return it.Password
|
||||
}))
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if options.Transport != nil {
|
||||
inbound.transport, err = v2ray.NewServerTransport(ctx, logger, common.PtrValueOrDefault(options.Transport), inbound.tlsConfig, (*inboundTransportHandler)(inbound))
|
||||
if err != nil {
|
||||
return nil, E.Cause(err, "create server transport: ", options.Transport.Type)
|
||||
}
|
||||
}
|
||||
inbound.router, err = mux.NewRouterWithOptions(inbound.router, logger, common.PtrValueOrDefault(options.Multiplex))
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
inbound.service = service
|
||||
inbound.listener = listener.New(listener.Options{
|
||||
Context: ctx,
|
||||
Logger: logger,
|
||||
Network: []string{N.NetworkTCP},
|
||||
Listen: options.ListenOptions,
|
||||
ConnectionHandler: inbound,
|
||||
})
|
||||
return inbound, nil
|
||||
}
|
||||
|
||||
func (h *Inbound) Start() error {
|
||||
if h.tlsConfig != nil {
|
||||
err := h.tlsConfig.Start()
|
||||
if err != nil {
|
||||
return E.Cause(err, "create TLS config")
|
||||
}
|
||||
}
|
||||
if h.transport == nil {
|
||||
return h.listener.Start()
|
||||
}
|
||||
if common.Contains(h.transport.Network(), N.NetworkTCP) {
|
||||
tcpListener, err := h.listener.ListenTCP()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
go func() {
|
||||
sErr := h.transport.Serve(tcpListener)
|
||||
if sErr != nil && !E.IsClosed(sErr) {
|
||||
h.logger.Error("transport serve error: ", sErr)
|
||||
}
|
||||
}()
|
||||
}
|
||||
if common.Contains(h.transport.Network(), N.NetworkUDP) {
|
||||
udpConn, err := h.listener.ListenUDP()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
go func() {
|
||||
sErr := h.transport.ServePacket(udpConn)
|
||||
if sErr != nil && !E.IsClosed(sErr) {
|
||||
h.logger.Error("transport serve error: ", sErr)
|
||||
}
|
||||
}()
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (h *Inbound) Close() error {
|
||||
return common.Close(
|
||||
&h.listener,
|
||||
h.tlsConfig,
|
||||
h.transport,
|
||||
)
|
||||
}
|
||||
|
||||
func (h *Inbound) NewConnection(ctx context.Context, conn net.Conn, metadata adapter.InboundContext) error {
|
||||
var err error
|
||||
if h.tlsConfig != nil && h.transport == nil {
|
||||
conn, err = tls.ServerHandshake(ctx, conn, h.tlsConfig)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
return h.service.NewConnection(adapter.WithContext(ctx, &metadata), conn, adapter.UpstreamMetadata(metadata))
|
||||
}
|
||||
|
||||
func (h *Inbound) NewConnectionEx(ctx context.Context, conn net.Conn, metadata adapter.InboundContext, onClose N.CloseHandlerFunc) {
|
||||
err := h.NewConnection(ctx, conn, metadata)
|
||||
N.CloseOnHandshakeFailure(conn, onClose, err)
|
||||
if err != nil {
|
||||
h.logger.ErrorContext(ctx, E.Cause(err, "process connection from ", metadata.Source))
|
||||
}
|
||||
}
|
||||
|
||||
func (h *Inbound) newConnection(ctx context.Context, conn net.Conn, metadata adapter.InboundContext) error {
|
||||
userIndex, loaded := auth.UserFromContext[int](ctx)
|
||||
if !loaded {
|
||||
return os.ErrInvalid
|
||||
}
|
||||
user := h.users[userIndex].Name
|
||||
if user == "" {
|
||||
user = F.ToString(userIndex)
|
||||
} else {
|
||||
metadata.User = user
|
||||
}
|
||||
h.logger.InfoContext(ctx, "[", user, "] inbound connection to ", metadata.Destination)
|
||||
return h.router.RouteConnection(ctx, conn, metadata)
|
||||
}
|
||||
|
||||
func (h *Inbound) fallbackConnection(ctx context.Context, conn net.Conn, metadata adapter.InboundContext) error {
|
||||
var fallbackAddr M.Socksaddr
|
||||
if len(h.fallbackAddrTLSNextProto) > 0 {
|
||||
if tlsConn, loaded := common.Cast[tls.Conn](conn); loaded {
|
||||
connectionState := tlsConn.ConnectionState()
|
||||
if connectionState.NegotiatedProtocol != "" {
|
||||
if fallbackAddr, loaded = h.fallbackAddrTLSNextProto[connectionState.NegotiatedProtocol]; !loaded {
|
||||
return E.New("fallback disabled for ALPN: ", connectionState.NegotiatedProtocol)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
if !fallbackAddr.IsValid() {
|
||||
if !h.fallbackAddr.IsValid() {
|
||||
return E.New("fallback disabled by default")
|
||||
}
|
||||
fallbackAddr = h.fallbackAddr
|
||||
}
|
||||
h.logger.InfoContext(ctx, "fallback connection to ", fallbackAddr)
|
||||
metadata.Destination = fallbackAddr
|
||||
return h.router.RouteConnection(ctx, conn, metadata)
|
||||
}
|
||||
|
||||
func (h *Inbound) newPacketConnection(ctx context.Context, conn N.PacketConn, metadata adapter.InboundContext) error {
|
||||
userIndex, loaded := auth.UserFromContext[int](ctx)
|
||||
if !loaded {
|
||||
return os.ErrInvalid
|
||||
}
|
||||
user := h.users[userIndex].Name
|
||||
if user == "" {
|
||||
user = F.ToString(userIndex)
|
||||
} else {
|
||||
metadata.User = user
|
||||
}
|
||||
h.logger.InfoContext(ctx, "[", user, "] inbound packet connection to ", metadata.Destination)
|
||||
return h.router.RoutePacketConnection(ctx, conn, metadata)
|
||||
}
|
||||
|
||||
var _ adapter.V2RayServerTransportHandler = (*inboundTransportHandler)(nil)
|
||||
|
||||
type inboundTransportHandler Inbound
|
||||
|
||||
func (h *inboundTransportHandler) NewConnectionEx(ctx context.Context, conn net.Conn, source M.Socksaddr, destination M.Socksaddr, onClose N.CloseHandlerFunc) {
|
||||
var metadata adapter.InboundContext
|
||||
metadata.Inbound = h.Tag()
|
||||
metadata.InboundType = h.Type()
|
||||
metadata.InboundDetour = h.listener.ListenOptions().Detour
|
||||
metadata.InboundOptions = h.listener.ListenOptions().InboundOptions
|
||||
metadata.Source = source
|
||||
metadata.Destination = destination
|
||||
h.logger.InfoContext(ctx, "inbound connection from ", metadata.Source)
|
||||
(*Inbound)(h).NewConnectionEx(ctx, conn, metadata, onClose)
|
||||
}
|
151
protocol/trojan/outbound.go
Normal file
151
protocol/trojan/outbound.go
Normal file
|
@ -0,0 +1,151 @@
|
|||
package trojan
|
||||
|
||||
import (
|
||||
"context"
|
||||
"net"
|
||||
|
||||
"github.com/sagernet/sing-box/adapter"
|
||||
"github.com/sagernet/sing-box/adapter/outbound"
|
||||
"github.com/sagernet/sing-box/common/dialer"
|
||||
"github.com/sagernet/sing-box/common/mux"
|
||||
"github.com/sagernet/sing-box/common/tls"
|
||||
C "github.com/sagernet/sing-box/constant"
|
||||
"github.com/sagernet/sing-box/log"
|
||||
"github.com/sagernet/sing-box/option"
|
||||
"github.com/sagernet/sing-box/transport/trojan"
|
||||
"github.com/sagernet/sing-box/transport/v2ray"
|
||||
"github.com/sagernet/sing/common"
|
||||
"github.com/sagernet/sing/common/bufio"
|
||||
E "github.com/sagernet/sing/common/exceptions"
|
||||
"github.com/sagernet/sing/common/logger"
|
||||
M "github.com/sagernet/sing/common/metadata"
|
||||
N "github.com/sagernet/sing/common/network"
|
||||
)
|
||||
|
||||
func RegisterOutbound(registry *outbound.Registry) {
|
||||
outbound.Register[option.TrojanOutboundOptions](registry, C.TypeTrojan, NewOutbound)
|
||||
}
|
||||
|
||||
type Outbound struct {
|
||||
outbound.Adapter
|
||||
logger logger.ContextLogger
|
||||
dialer N.Dialer
|
||||
serverAddr M.Socksaddr
|
||||
key [56]byte
|
||||
multiplexDialer *mux.Client
|
||||
tlsConfig tls.Config
|
||||
transport adapter.V2RayClientTransport
|
||||
}
|
||||
|
||||
func NewOutbound(ctx context.Context, router adapter.Router, logger log.ContextLogger, tag string, options option.TrojanOutboundOptions) (adapter.Outbound, error) {
|
||||
outboundDialer, err := dialer.New(router, options.DialerOptions)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
outbound := &Outbound{
|
||||
Adapter: outbound.NewAdapterWithDialerOptions(C.TypeTrojan, options.Network.Build(), tag, options.DialerOptions),
|
||||
logger: logger,
|
||||
dialer: outboundDialer,
|
||||
serverAddr: options.ServerOptions.Build(),
|
||||
key: trojan.Key(options.Password),
|
||||
}
|
||||
if options.TLS != nil {
|
||||
outbound.tlsConfig, err = tls.NewClient(ctx, options.Server, common.PtrValueOrDefault(options.TLS))
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
}
|
||||
if options.Transport != nil {
|
||||
outbound.transport, err = v2ray.NewClientTransport(ctx, outbound.dialer, outbound.serverAddr, common.PtrValueOrDefault(options.Transport), outbound.tlsConfig)
|
||||
if err != nil {
|
||||
return nil, E.Cause(err, "create client transport: ", options.Transport.Type)
|
||||
}
|
||||
}
|
||||
outbound.multiplexDialer, err = mux.NewClientWithOptions((*trojanDialer)(outbound), logger, common.PtrValueOrDefault(options.Multiplex))
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return outbound, nil
|
||||
}
|
||||
|
||||
func (h *Outbound) DialContext(ctx context.Context, network string, destination M.Socksaddr) (net.Conn, error) {
|
||||
if h.multiplexDialer == nil {
|
||||
switch N.NetworkName(network) {
|
||||
case N.NetworkTCP:
|
||||
h.logger.InfoContext(ctx, "outbound connection to ", destination)
|
||||
case N.NetworkUDP:
|
||||
h.logger.InfoContext(ctx, "outbound packet connection to ", destination)
|
||||
}
|
||||
return (*trojanDialer)(h).DialContext(ctx, network, destination)
|
||||
} else {
|
||||
switch N.NetworkName(network) {
|
||||
case N.NetworkTCP:
|
||||
h.logger.InfoContext(ctx, "outbound multiplex connection to ", destination)
|
||||
case N.NetworkUDP:
|
||||
h.logger.InfoContext(ctx, "outbound multiplex packet connection to ", destination)
|
||||
}
|
||||
return h.multiplexDialer.DialContext(ctx, network, destination)
|
||||
}
|
||||
}
|
||||
|
||||
func (h *Outbound) ListenPacket(ctx context.Context, destination M.Socksaddr) (net.PacketConn, error) {
|
||||
if h.multiplexDialer == nil {
|
||||
h.logger.InfoContext(ctx, "outbound packet connection to ", destination)
|
||||
return (*trojanDialer)(h).ListenPacket(ctx, destination)
|
||||
} else {
|
||||
h.logger.InfoContext(ctx, "outbound multiplex packet connection to ", destination)
|
||||
return h.multiplexDialer.ListenPacket(ctx, destination)
|
||||
}
|
||||
}
|
||||
|
||||
func (h *Outbound) InterfaceUpdated() {
|
||||
if h.transport != nil {
|
||||
h.transport.Close()
|
||||
}
|
||||
if h.multiplexDialer != nil {
|
||||
h.multiplexDialer.Reset()
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
func (h *Outbound) Close() error {
|
||||
return common.Close(common.PtrOrNil(h.multiplexDialer), h.transport)
|
||||
}
|
||||
|
||||
type trojanDialer Outbound
|
||||
|
||||
func (h *trojanDialer) DialContext(ctx context.Context, network string, destination M.Socksaddr) (net.Conn, error) {
|
||||
ctx, metadata := adapter.ExtendContext(ctx)
|
||||
metadata.Outbound = h.Tag()
|
||||
metadata.Destination = destination
|
||||
var conn net.Conn
|
||||
var err error
|
||||
if h.transport != nil {
|
||||
conn, err = h.transport.DialContext(ctx)
|
||||
} else {
|
||||
conn, err = h.dialer.DialContext(ctx, N.NetworkTCP, h.serverAddr)
|
||||
if err == nil && h.tlsConfig != nil {
|
||||
conn, err = tls.ClientHandshake(ctx, conn, h.tlsConfig)
|
||||
}
|
||||
}
|
||||
if err != nil {
|
||||
common.Close(conn)
|
||||
return nil, err
|
||||
}
|
||||
switch N.NetworkName(network) {
|
||||
case N.NetworkTCP:
|
||||
return trojan.NewClientConn(conn, h.key, destination), nil
|
||||
case N.NetworkUDP:
|
||||
return bufio.NewBindPacketConn(trojan.NewClientPacketConn(conn, h.key), destination), nil
|
||||
default:
|
||||
return nil, E.Extend(N.ErrUnknownNetwork, network)
|
||||
}
|
||||
}
|
||||
|
||||
func (h *trojanDialer) ListenPacket(ctx context.Context, destination M.Socksaddr) (net.PacketConn, error) {
|
||||
conn, err := h.DialContext(ctx, N.NetworkUDP, destination)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return conn.(net.PacketConn), nil
|
||||
}
|
157
protocol/tuic/inbound.go
Normal file
157
protocol/tuic/inbound.go
Normal file
|
@ -0,0 +1,157 @@
|
|||
package tuic
|
||||
|
||||
import (
|
||||
"context"
|
||||
"net"
|
||||
"time"
|
||||
|
||||
"github.com/sagernet/sing-box/adapter"
|
||||
"github.com/sagernet/sing-box/adapter/inbound"
|
||||
"github.com/sagernet/sing-box/common/listener"
|
||||
"github.com/sagernet/sing-box/common/tls"
|
||||
"github.com/sagernet/sing-box/common/uot"
|
||||
C "github.com/sagernet/sing-box/constant"
|
||||
"github.com/sagernet/sing-box/log"
|
||||
"github.com/sagernet/sing-box/option"
|
||||
"github.com/sagernet/sing-quic/tuic"
|
||||
"github.com/sagernet/sing/common"
|
||||
"github.com/sagernet/sing/common/auth"
|
||||
E "github.com/sagernet/sing/common/exceptions"
|
||||
N "github.com/sagernet/sing/common/network"
|
||||
|
||||
"github.com/gofrs/uuid/v5"
|
||||
)
|
||||
|
||||
func RegisterInbound(registry *inbound.Registry) {
|
||||
inbound.Register[option.TUICInboundOptions](registry, C.TypeTUIC, NewInbound)
|
||||
}
|
||||
|
||||
type Inbound struct {
|
||||
inbound.Adapter
|
||||
router adapter.ConnectionRouterEx
|
||||
logger log.ContextLogger
|
||||
listener *listener.Listener
|
||||
tlsConfig tls.ServerConfig
|
||||
server *tuic.Service[int]
|
||||
userNameList []string
|
||||
}
|
||||
|
||||
func NewInbound(ctx context.Context, router adapter.Router, logger log.ContextLogger, tag string, options option.TUICInboundOptions) (adapter.Inbound, error) {
|
||||
options.UDPFragmentDefault = true
|
||||
if options.TLS == nil || !options.TLS.Enabled {
|
||||
return nil, C.ErrTLSRequired
|
||||
}
|
||||
tlsConfig, err := tls.NewServer(ctx, logger, common.PtrValueOrDefault(options.TLS))
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
inbound := &Inbound{
|
||||
Adapter: inbound.NewAdapter(C.TypeTUIC, tag),
|
||||
router: uot.NewRouter(router, logger),
|
||||
logger: logger,
|
||||
listener: listener.New(listener.Options{
|
||||
Context: ctx,
|
||||
Logger: logger,
|
||||
Listen: options.ListenOptions,
|
||||
}),
|
||||
tlsConfig: tlsConfig,
|
||||
}
|
||||
var udpTimeout time.Duration
|
||||
if options.UDPTimeout != 0 {
|
||||
udpTimeout = time.Duration(options.UDPTimeout)
|
||||
} else {
|
||||
udpTimeout = C.UDPTimeout
|
||||
}
|
||||
service, err := tuic.NewService[int](tuic.ServiceOptions{
|
||||
Context: ctx,
|
||||
Logger: logger,
|
||||
TLSConfig: tlsConfig,
|
||||
CongestionControl: options.CongestionControl,
|
||||
AuthTimeout: time.Duration(options.AuthTimeout),
|
||||
ZeroRTTHandshake: options.ZeroRTTHandshake,
|
||||
Heartbeat: time.Duration(options.Heartbeat),
|
||||
UDPTimeout: udpTimeout,
|
||||
Handler: adapter.NewUpstreamHandler(adapter.InboundContext{}, inbound.newConnection, inbound.newPacketConnection, nil),
|
||||
})
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
var userList []int
|
||||
var userNameList []string
|
||||
var userUUIDList [][16]byte
|
||||
var userPasswordList []string
|
||||
for index, user := range options.Users {
|
||||
if user.UUID == "" {
|
||||
return nil, E.New("missing uuid for user ", index)
|
||||
}
|
||||
userUUID, err := uuid.FromString(user.UUID)
|
||||
if err != nil {
|
||||
return nil, E.Cause(err, "invalid uuid for user ", index)
|
||||
}
|
||||
userList = append(userList, index)
|
||||
userNameList = append(userNameList, user.Name)
|
||||
userUUIDList = append(userUUIDList, userUUID)
|
||||
userPasswordList = append(userPasswordList, user.Password)
|
||||
}
|
||||
service.UpdateUsers(userList, userUUIDList, userPasswordList)
|
||||
inbound.server = service
|
||||
inbound.userNameList = userNameList
|
||||
return inbound, nil
|
||||
}
|
||||
|
||||
func (h *Inbound) newConnection(ctx context.Context, conn net.Conn, metadata adapter.InboundContext) error {
|
||||
ctx = log.ContextWithNewID(ctx)
|
||||
metadata.Inbound = h.Tag()
|
||||
metadata.InboundType = h.Type()
|
||||
metadata.InboundDetour = h.listener.ListenOptions().Detour
|
||||
metadata.InboundOptions = h.listener.ListenOptions().InboundOptions
|
||||
h.logger.InfoContext(ctx, "inbound connection from ", metadata.Source)
|
||||
userID, _ := auth.UserFromContext[int](ctx)
|
||||
if userName := h.userNameList[userID]; userName != "" {
|
||||
metadata.User = userName
|
||||
h.logger.InfoContext(ctx, "[", userName, "] inbound connection to ", metadata.Destination)
|
||||
} else {
|
||||
h.logger.InfoContext(ctx, "inbound connection to ", metadata.Destination)
|
||||
}
|
||||
return h.router.RouteConnection(ctx, conn, metadata)
|
||||
}
|
||||
|
||||
func (h *Inbound) newPacketConnection(ctx context.Context, conn N.PacketConn, metadata adapter.InboundContext) error {
|
||||
ctx = log.ContextWithNewID(ctx)
|
||||
metadata.Inbound = h.Tag()
|
||||
metadata.InboundType = h.Type()
|
||||
metadata.InboundDetour = h.listener.ListenOptions().Detour
|
||||
metadata.InboundOptions = h.listener.ListenOptions().InboundOptions
|
||||
metadata.OriginDestination = h.listener.UDPAddr()
|
||||
h.logger.InfoContext(ctx, "inbound packet connection from ", metadata.Source)
|
||||
userID, _ := auth.UserFromContext[int](ctx)
|
||||
if userName := h.userNameList[userID]; userName != "" {
|
||||
metadata.User = userName
|
||||
h.logger.InfoContext(ctx, "[", userName, "] inbound packet connection to ", metadata.Destination)
|
||||
} else {
|
||||
h.logger.InfoContext(ctx, "inbound packet connection to ", metadata.Destination)
|
||||
}
|
||||
return h.router.RoutePacketConnection(ctx, conn, metadata)
|
||||
}
|
||||
|
||||
func (h *Inbound) Start() error {
|
||||
if h.tlsConfig != nil {
|
||||
err := h.tlsConfig.Start()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
packetConn, err := h.listener.ListenUDP()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
return h.server.Start(packetConn)
|
||||
}
|
||||
|
||||
func (h *Inbound) Close() error {
|
||||
return common.Close(
|
||||
&h.listener,
|
||||
h.tlsConfig,
|
||||
common.PtrOrNil(h.server),
|
||||
)
|
||||
}
|
141
protocol/tuic/outbound.go
Normal file
141
protocol/tuic/outbound.go
Normal file
|
@ -0,0 +1,141 @@
|
|||
package tuic
|
||||
|
||||
import (
|
||||
"context"
|
||||
"net"
|
||||
"os"
|
||||
"time"
|
||||
|
||||
"github.com/sagernet/sing-box/adapter"
|
||||
"github.com/sagernet/sing-box/adapter/outbound"
|
||||
"github.com/sagernet/sing-box/common/dialer"
|
||||
"github.com/sagernet/sing-box/common/tls"
|
||||
C "github.com/sagernet/sing-box/constant"
|
||||
"github.com/sagernet/sing-box/log"
|
||||
"github.com/sagernet/sing-box/option"
|
||||
"github.com/sagernet/sing-quic/tuic"
|
||||
"github.com/sagernet/sing/common"
|
||||
"github.com/sagernet/sing/common/bufio"
|
||||
E "github.com/sagernet/sing/common/exceptions"
|
||||
"github.com/sagernet/sing/common/logger"
|
||||
M "github.com/sagernet/sing/common/metadata"
|
||||
N "github.com/sagernet/sing/common/network"
|
||||
"github.com/sagernet/sing/common/uot"
|
||||
|
||||
"github.com/gofrs/uuid/v5"
|
||||
)
|
||||
|
||||
func RegisterOutbound(registry *outbound.Registry) {
|
||||
outbound.Register[option.TUICOutboundOptions](registry, C.TypeTUIC, NewOutbound)
|
||||
}
|
||||
|
||||
var _ adapter.InterfaceUpdateListener = (*Outbound)(nil)
|
||||
|
||||
type Outbound struct {
|
||||
outbound.Adapter
|
||||
logger logger.ContextLogger
|
||||
client *tuic.Client
|
||||
udpStream bool
|
||||
}
|
||||
|
||||
func NewOutbound(ctx context.Context, router adapter.Router, logger log.ContextLogger, tag string, options option.TUICOutboundOptions) (adapter.Outbound, error) {
|
||||
options.UDPFragmentDefault = true
|
||||
if options.TLS == nil || !options.TLS.Enabled {
|
||||
return nil, C.ErrTLSRequired
|
||||
}
|
||||
tlsConfig, err := tls.NewClient(ctx, options.Server, common.PtrValueOrDefault(options.TLS))
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
userUUID, err := uuid.FromString(options.UUID)
|
||||
if err != nil {
|
||||
return nil, E.Cause(err, "invalid uuid")
|
||||
}
|
||||
var tuicUDPStream bool
|
||||
if options.UDPOverStream && options.UDPRelayMode != "" {
|
||||
return nil, E.New("udp_over_stream is conflict with udp_relay_mode")
|
||||
}
|
||||
switch options.UDPRelayMode {
|
||||
case "native":
|
||||
case "quic":
|
||||
tuicUDPStream = true
|
||||
}
|
||||
outboundDialer, err := dialer.New(router, options.DialerOptions)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
client, err := tuic.NewClient(tuic.ClientOptions{
|
||||
Context: ctx,
|
||||
Dialer: outboundDialer,
|
||||
ServerAddress: options.ServerOptions.Build(),
|
||||
TLSConfig: tlsConfig,
|
||||
UUID: userUUID,
|
||||
Password: options.Password,
|
||||
CongestionControl: options.CongestionControl,
|
||||
UDPStream: tuicUDPStream,
|
||||
ZeroRTTHandshake: options.ZeroRTTHandshake,
|
||||
Heartbeat: time.Duration(options.Heartbeat),
|
||||
})
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return &Outbound{
|
||||
Adapter: outbound.NewAdapterWithDialerOptions(C.TypeTUIC, options.Network.Build(), tag, options.DialerOptions),
|
||||
logger: logger,
|
||||
client: client,
|
||||
udpStream: options.UDPOverStream,
|
||||
}, nil
|
||||
}
|
||||
|
||||
func (h *Outbound) DialContext(ctx context.Context, network string, destination M.Socksaddr) (net.Conn, error) {
|
||||
switch N.NetworkName(network) {
|
||||
case N.NetworkTCP:
|
||||
h.logger.InfoContext(ctx, "outbound connection to ", destination)
|
||||
return h.client.DialConn(ctx, destination)
|
||||
case N.NetworkUDP:
|
||||
if h.udpStream {
|
||||
h.logger.InfoContext(ctx, "outbound stream packet connection to ", destination)
|
||||
streamConn, err := h.client.DialConn(ctx, uot.RequestDestination(uot.Version))
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return uot.NewLazyConn(streamConn, uot.Request{
|
||||
IsConnect: true,
|
||||
Destination: destination,
|
||||
}), nil
|
||||
} else {
|
||||
conn, err := h.ListenPacket(ctx, destination)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return bufio.NewBindPacketConn(conn, destination), nil
|
||||
}
|
||||
default:
|
||||
return nil, E.New("unsupported network: ", network)
|
||||
}
|
||||
}
|
||||
|
||||
func (h *Outbound) ListenPacket(ctx context.Context, destination M.Socksaddr) (net.PacketConn, error) {
|
||||
if h.udpStream {
|
||||
h.logger.InfoContext(ctx, "outbound stream packet connection to ", destination)
|
||||
streamConn, err := h.client.DialConn(ctx, uot.RequestDestination(uot.Version))
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return uot.NewLazyConn(streamConn, uot.Request{
|
||||
IsConnect: false,
|
||||
Destination: destination,
|
||||
}), nil
|
||||
} else {
|
||||
h.logger.InfoContext(ctx, "outbound packet connection to ", destination)
|
||||
return h.client.ListenPacket(ctx)
|
||||
}
|
||||
}
|
||||
|
||||
func (h *Outbound) InterfaceUpdated() {
|
||||
_ = h.client.CloseWithError(E.New("network changed"))
|
||||
}
|
||||
|
||||
func (h *Outbound) Close() error {
|
||||
return h.client.CloseWithError(os.ErrClosed)
|
||||
}
|
462
protocol/tun/inbound.go
Normal file
462
protocol/tun/inbound.go
Normal file
|
@ -0,0 +1,462 @@
|
|||
package tun
|
||||
|
||||
import (
|
||||
"context"
|
||||
"net"
|
||||
"net/netip"
|
||||
"os"
|
||||
"runtime"
|
||||
"strconv"
|
||||
"strings"
|
||||
"time"
|
||||
|
||||
"github.com/sagernet/sing-box/adapter"
|
||||
"github.com/sagernet/sing-box/adapter/inbound"
|
||||
"github.com/sagernet/sing-box/common/taskmonitor"
|
||||
C "github.com/sagernet/sing-box/constant"
|
||||
"github.com/sagernet/sing-box/experimental/deprecated"
|
||||
"github.com/sagernet/sing-box/experimental/libbox/platform"
|
||||
"github.com/sagernet/sing-box/log"
|
||||
"github.com/sagernet/sing-box/option"
|
||||
"github.com/sagernet/sing-tun"
|
||||
"github.com/sagernet/sing/common"
|
||||
E "github.com/sagernet/sing/common/exceptions"
|
||||
M "github.com/sagernet/sing/common/metadata"
|
||||
N "github.com/sagernet/sing/common/network"
|
||||
"github.com/sagernet/sing/common/ranges"
|
||||
"github.com/sagernet/sing/common/x/list"
|
||||
"github.com/sagernet/sing/service"
|
||||
|
||||
"go4.org/netipx"
|
||||
)
|
||||
|
||||
func RegisterInbound(registry *inbound.Registry) {
|
||||
inbound.Register[option.TunInboundOptions](registry, C.TypeTun, NewInbound)
|
||||
}
|
||||
|
||||
type Inbound struct {
|
||||
tag string
|
||||
ctx context.Context
|
||||
router adapter.Router
|
||||
logger log.ContextLogger
|
||||
// Deprecated
|
||||
inboundOptions option.InboundOptions
|
||||
tunOptions tun.Options
|
||||
// Deprecated
|
||||
endpointIndependentNat bool
|
||||
udpTimeout time.Duration
|
||||
stack string
|
||||
tunIf tun.Tun
|
||||
tunStack tun.Stack
|
||||
platformInterface platform.Interface
|
||||
platformOptions option.TunPlatformOptions
|
||||
autoRedirect tun.AutoRedirect
|
||||
routeRuleSet []adapter.RuleSet
|
||||
routeRuleSetCallback []*list.Element[adapter.RuleSetUpdateCallback]
|
||||
routeExcludeRuleSet []adapter.RuleSet
|
||||
routeExcludeRuleSetCallback []*list.Element[adapter.RuleSetUpdateCallback]
|
||||
routeAddressSet []*netipx.IPSet
|
||||
routeExcludeAddressSet []*netipx.IPSet
|
||||
}
|
||||
|
||||
func NewInbound(ctx context.Context, router adapter.Router, logger log.ContextLogger, tag string, options option.TunInboundOptions) (adapter.Inbound, error) {
|
||||
address := options.Address
|
||||
var deprecatedAddressUsed bool
|
||||
//nolint:staticcheck
|
||||
//goland:noinspection GoDeprecation
|
||||
if len(options.Inet4Address) > 0 {
|
||||
address = append(address, options.Inet4Address...)
|
||||
deprecatedAddressUsed = true
|
||||
}
|
||||
//nolint:staticcheck
|
||||
//goland:noinspection GoDeprecation
|
||||
if len(options.Inet6Address) > 0 {
|
||||
address = append(address, options.Inet6Address...)
|
||||
deprecatedAddressUsed = true
|
||||
}
|
||||
inet4Address := common.Filter(address, func(it netip.Prefix) bool {
|
||||
return it.Addr().Is4()
|
||||
})
|
||||
inet6Address := common.Filter(address, func(it netip.Prefix) bool {
|
||||
return it.Addr().Is6()
|
||||
})
|
||||
|
||||
routeAddress := options.RouteAddress
|
||||
//nolint:staticcheck
|
||||
//goland:noinspection GoDeprecation
|
||||
if len(options.Inet4RouteAddress) > 0 {
|
||||
routeAddress = append(routeAddress, options.Inet4RouteAddress...)
|
||||
deprecatedAddressUsed = true
|
||||
}
|
||||
//nolint:staticcheck
|
||||
//goland:noinspection GoDeprecation
|
||||
if len(options.Inet6RouteAddress) > 0 {
|
||||
routeAddress = append(routeAddress, options.Inet6RouteAddress...)
|
||||
deprecatedAddressUsed = true
|
||||
}
|
||||
inet4RouteAddress := common.Filter(routeAddress, func(it netip.Prefix) bool {
|
||||
return it.Addr().Is4()
|
||||
})
|
||||
inet6RouteAddress := common.Filter(routeAddress, func(it netip.Prefix) bool {
|
||||
return it.Addr().Is6()
|
||||
})
|
||||
|
||||
routeExcludeAddress := options.RouteExcludeAddress
|
||||
//nolint:staticcheck
|
||||
//goland:noinspection GoDeprecation
|
||||
if len(options.Inet4RouteExcludeAddress) > 0 {
|
||||
routeExcludeAddress = append(routeExcludeAddress, options.Inet4RouteExcludeAddress...)
|
||||
deprecatedAddressUsed = true
|
||||
}
|
||||
//nolint:staticcheck
|
||||
//goland:noinspection GoDeprecation
|
||||
if len(options.Inet6RouteExcludeAddress) > 0 {
|
||||
routeExcludeAddress = append(routeExcludeAddress, options.Inet6RouteExcludeAddress...)
|
||||
deprecatedAddressUsed = true
|
||||
}
|
||||
inet4RouteExcludeAddress := common.Filter(routeExcludeAddress, func(it netip.Prefix) bool {
|
||||
return it.Addr().Is4()
|
||||
})
|
||||
inet6RouteExcludeAddress := common.Filter(routeExcludeAddress, func(it netip.Prefix) bool {
|
||||
return it.Addr().Is6()
|
||||
})
|
||||
|
||||
if deprecatedAddressUsed {
|
||||
deprecated.Report(ctx, deprecated.OptionTUNAddressX)
|
||||
}
|
||||
|
||||
tunMTU := options.MTU
|
||||
if tunMTU == 0 {
|
||||
tunMTU = 9000
|
||||
}
|
||||
var udpTimeout time.Duration
|
||||
if options.UDPTimeout != 0 {
|
||||
udpTimeout = time.Duration(options.UDPTimeout)
|
||||
} else {
|
||||
udpTimeout = C.UDPTimeout
|
||||
}
|
||||
var err error
|
||||
includeUID := uidToRange(options.IncludeUID)
|
||||
if len(options.IncludeUIDRange) > 0 {
|
||||
includeUID, err = parseRange(includeUID, options.IncludeUIDRange)
|
||||
if err != nil {
|
||||
return nil, E.Cause(err, "parse include_uid_range")
|
||||
}
|
||||
}
|
||||
excludeUID := uidToRange(options.ExcludeUID)
|
||||
if len(options.ExcludeUIDRange) > 0 {
|
||||
excludeUID, err = parseRange(excludeUID, options.ExcludeUIDRange)
|
||||
if err != nil {
|
||||
return nil, E.Cause(err, "parse exclude_uid_range")
|
||||
}
|
||||
}
|
||||
|
||||
tableIndex := options.IPRoute2TableIndex
|
||||
if tableIndex == 0 {
|
||||
tableIndex = tun.DefaultIPRoute2TableIndex
|
||||
}
|
||||
ruleIndex := options.IPRoute2RuleIndex
|
||||
if ruleIndex == 0 {
|
||||
ruleIndex = tun.DefaultIPRoute2RuleIndex
|
||||
}
|
||||
inputMark := uint32(options.AutoRedirectInputMark)
|
||||
if inputMark == 0 {
|
||||
inputMark = tun.DefaultAutoRedirectInputMark
|
||||
}
|
||||
outputMark := uint32(options.AutoRedirectOutputMark)
|
||||
if outputMark == 0 {
|
||||
outputMark = tun.DefaultAutoRedirectOutputMark
|
||||
}
|
||||
|
||||
inbound := &Inbound{
|
||||
tag: tag,
|
||||
ctx: ctx,
|
||||
router: router,
|
||||
logger: logger,
|
||||
inboundOptions: options.InboundOptions,
|
||||
tunOptions: tun.Options{
|
||||
Name: options.InterfaceName,
|
||||
MTU: tunMTU,
|
||||
GSO: options.GSO,
|
||||
Inet4Address: inet4Address,
|
||||
Inet6Address: inet6Address,
|
||||
AutoRoute: options.AutoRoute,
|
||||
IPRoute2TableIndex: tableIndex,
|
||||
IPRoute2RuleIndex: ruleIndex,
|
||||
AutoRedirectInputMark: inputMark,
|
||||
AutoRedirectOutputMark: outputMark,
|
||||
StrictRoute: options.StrictRoute,
|
||||
IncludeInterface: options.IncludeInterface,
|
||||
ExcludeInterface: options.ExcludeInterface,
|
||||
Inet4RouteAddress: inet4RouteAddress,
|
||||
Inet6RouteAddress: inet6RouteAddress,
|
||||
Inet4RouteExcludeAddress: inet4RouteExcludeAddress,
|
||||
Inet6RouteExcludeAddress: inet6RouteExcludeAddress,
|
||||
IncludeUID: includeUID,
|
||||
ExcludeUID: excludeUID,
|
||||
IncludeAndroidUser: options.IncludeAndroidUser,
|
||||
IncludePackage: options.IncludePackage,
|
||||
ExcludePackage: options.ExcludePackage,
|
||||
InterfaceMonitor: router.InterfaceMonitor(),
|
||||
},
|
||||
endpointIndependentNat: options.EndpointIndependentNat,
|
||||
udpTimeout: udpTimeout,
|
||||
stack: options.Stack,
|
||||
platformInterface: service.FromContext[platform.Interface](ctx),
|
||||
platformOptions: common.PtrValueOrDefault(options.Platform),
|
||||
}
|
||||
if options.AutoRedirect {
|
||||
if !options.AutoRoute {
|
||||
return nil, E.New("`auto_route` is required by `auto_redirect`")
|
||||
}
|
||||
disableNFTables, dErr := strconv.ParseBool(os.Getenv("DISABLE_NFTABLES"))
|
||||
inbound.autoRedirect, err = tun.NewAutoRedirect(tun.AutoRedirectOptions{
|
||||
TunOptions: &inbound.tunOptions,
|
||||
Context: ctx,
|
||||
Handler: (*autoRedirectHandler)(inbound),
|
||||
Logger: logger,
|
||||
NetworkMonitor: router.NetworkMonitor(),
|
||||
InterfaceFinder: router.InterfaceFinder(),
|
||||
TableName: "sing-box",
|
||||
DisableNFTables: dErr == nil && disableNFTables,
|
||||
RouteAddressSet: &inbound.routeAddressSet,
|
||||
RouteExcludeAddressSet: &inbound.routeExcludeAddressSet,
|
||||
})
|
||||
if err != nil {
|
||||
return nil, E.Cause(err, "initialize auto-redirect")
|
||||
}
|
||||
if runtime.GOOS != "android" {
|
||||
var markMode bool
|
||||
for _, routeAddressSet := range options.RouteAddressSet {
|
||||
ruleSet, loaded := router.RuleSet(routeAddressSet)
|
||||
if !loaded {
|
||||
return nil, E.New("parse route_address_set: rule-set not found: ", routeAddressSet)
|
||||
}
|
||||
ruleSet.IncRef()
|
||||
inbound.routeRuleSet = append(inbound.routeRuleSet, ruleSet)
|
||||
markMode = true
|
||||
}
|
||||
for _, routeExcludeAddressSet := range options.RouteExcludeAddressSet {
|
||||
ruleSet, loaded := router.RuleSet(routeExcludeAddressSet)
|
||||
if !loaded {
|
||||
return nil, E.New("parse route_exclude_address_set: rule-set not found: ", routeExcludeAddressSet)
|
||||
}
|
||||
ruleSet.IncRef()
|
||||
inbound.routeExcludeRuleSet = append(inbound.routeExcludeRuleSet, ruleSet)
|
||||
markMode = true
|
||||
}
|
||||
if markMode {
|
||||
inbound.tunOptions.AutoRedirectMarkMode = true
|
||||
err = router.RegisterAutoRedirectOutputMark(inbound.tunOptions.AutoRedirectOutputMark)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
return inbound, nil
|
||||
}
|
||||
|
||||
func uidToRange(uidList option.Listable[uint32]) []ranges.Range[uint32] {
|
||||
return common.Map(uidList, func(uid uint32) ranges.Range[uint32] {
|
||||
return ranges.NewSingle(uid)
|
||||
})
|
||||
}
|
||||
|
||||
func parseRange(uidRanges []ranges.Range[uint32], rangeList []string) ([]ranges.Range[uint32], error) {
|
||||
for _, uidRange := range rangeList {
|
||||
if !strings.Contains(uidRange, ":") {
|
||||
return nil, E.New("missing ':' in range: ", uidRange)
|
||||
}
|
||||
subIndex := strings.Index(uidRange, ":")
|
||||
if subIndex == 0 {
|
||||
return nil, E.New("missing range start: ", uidRange)
|
||||
} else if subIndex == len(uidRange)-1 {
|
||||
return nil, E.New("missing range end: ", uidRange)
|
||||
}
|
||||
var start, end uint64
|
||||
var err error
|
||||
start, err = strconv.ParseUint(uidRange[:subIndex], 0, 32)
|
||||
if err != nil {
|
||||
return nil, E.Cause(err, "parse range start")
|
||||
}
|
||||
end, err = strconv.ParseUint(uidRange[subIndex+1:], 0, 32)
|
||||
if err != nil {
|
||||
return nil, E.Cause(err, "parse range end")
|
||||
}
|
||||
uidRanges = append(uidRanges, ranges.New(uint32(start), uint32(end)))
|
||||
}
|
||||
return uidRanges, nil
|
||||
}
|
||||
|
||||
func (t *Inbound) Type() string {
|
||||
return C.TypeTun
|
||||
}
|
||||
|
||||
func (t *Inbound) Tag() string {
|
||||
return t.tag
|
||||
}
|
||||
|
||||
func (t *Inbound) Start() error {
|
||||
if C.IsAndroid && t.platformInterface == nil {
|
||||
t.tunOptions.BuildAndroidRules(t.router.PackageManager())
|
||||
}
|
||||
if t.tunOptions.Name == "" {
|
||||
t.tunOptions.Name = tun.CalculateInterfaceName("")
|
||||
}
|
||||
var (
|
||||
tunInterface tun.Tun
|
||||
err error
|
||||
)
|
||||
monitor := taskmonitor.New(t.logger, C.StartTimeout)
|
||||
monitor.Start("open tun interface")
|
||||
if t.platformInterface != nil {
|
||||
tunInterface, err = t.platformInterface.OpenTun(&t.tunOptions, t.platformOptions)
|
||||
} else {
|
||||
tunInterface, err = tun.New(t.tunOptions)
|
||||
}
|
||||
monitor.Finish()
|
||||
if err != nil {
|
||||
return E.Cause(err, "configure tun interface")
|
||||
}
|
||||
t.logger.Trace("creating stack")
|
||||
t.tunIf = tunInterface
|
||||
var (
|
||||
forwarderBindInterface bool
|
||||
includeAllNetworks bool
|
||||
)
|
||||
if t.platformInterface != nil {
|
||||
forwarderBindInterface = true
|
||||
includeAllNetworks = t.platformInterface.IncludeAllNetworks()
|
||||
}
|
||||
tunStack, err := tun.NewStack(t.stack, tun.StackOptions{
|
||||
Context: t.ctx,
|
||||
Tun: tunInterface,
|
||||
TunOptions: t.tunOptions,
|
||||
UDPTimeout: t.udpTimeout,
|
||||
Handler: t,
|
||||
Logger: t.logger,
|
||||
ForwarderBindInterface: forwarderBindInterface,
|
||||
InterfaceFinder: t.router.InterfaceFinder(),
|
||||
IncludeAllNetworks: includeAllNetworks,
|
||||
})
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
monitor.Start("initiating tun stack")
|
||||
err = tunStack.Start()
|
||||
monitor.Finish()
|
||||
t.tunStack = tunStack
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
t.logger.Info("started at ", t.tunOptions.Name)
|
||||
return nil
|
||||
}
|
||||
|
||||
func (t *Inbound) PostStart() error {
|
||||
monitor := taskmonitor.New(t.logger, C.StartTimeout)
|
||||
if t.autoRedirect != nil {
|
||||
t.routeAddressSet = common.FlatMap(t.routeRuleSet, adapter.RuleSet.ExtractIPSet)
|
||||
for _, routeRuleSet := range t.routeRuleSet {
|
||||
ipSets := routeRuleSet.ExtractIPSet()
|
||||
if len(ipSets) == 0 {
|
||||
t.logger.Warn("route_address_set: no destination IP CIDR rules found in rule-set: ", routeRuleSet.Name())
|
||||
}
|
||||
t.routeAddressSet = append(t.routeAddressSet, ipSets...)
|
||||
}
|
||||
t.routeExcludeAddressSet = common.FlatMap(t.routeExcludeRuleSet, adapter.RuleSet.ExtractIPSet)
|
||||
for _, routeExcludeRuleSet := range t.routeExcludeRuleSet {
|
||||
ipSets := routeExcludeRuleSet.ExtractIPSet()
|
||||
if len(ipSets) == 0 {
|
||||
t.logger.Warn("route_address_set: no destination IP CIDR rules found in rule-set: ", routeExcludeRuleSet.Name())
|
||||
}
|
||||
t.routeExcludeAddressSet = append(t.routeExcludeAddressSet, ipSets...)
|
||||
}
|
||||
monitor.Start("initialize auto-redirect")
|
||||
err := t.autoRedirect.Start()
|
||||
monitor.Finish()
|
||||
if err != nil {
|
||||
return E.Cause(err, "auto-redirect")
|
||||
}
|
||||
for _, routeRuleSet := range t.routeRuleSet {
|
||||
t.routeRuleSetCallback = append(t.routeRuleSetCallback, routeRuleSet.RegisterCallback(t.updateRouteAddressSet))
|
||||
routeRuleSet.DecRef()
|
||||
}
|
||||
for _, routeExcludeRuleSet := range t.routeExcludeRuleSet {
|
||||
t.routeExcludeRuleSetCallback = append(t.routeExcludeRuleSetCallback, routeExcludeRuleSet.RegisterCallback(t.updateRouteAddressSet))
|
||||
routeExcludeRuleSet.DecRef()
|
||||
}
|
||||
t.routeAddressSet = nil
|
||||
t.routeExcludeAddressSet = nil
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (t *Inbound) updateRouteAddressSet(it adapter.RuleSet) {
|
||||
t.routeAddressSet = common.FlatMap(t.routeRuleSet, adapter.RuleSet.ExtractIPSet)
|
||||
t.routeExcludeAddressSet = common.FlatMap(t.routeExcludeRuleSet, adapter.RuleSet.ExtractIPSet)
|
||||
t.autoRedirect.UpdateRouteAddressSet()
|
||||
t.routeAddressSet = nil
|
||||
t.routeExcludeAddressSet = nil
|
||||
}
|
||||
|
||||
func (t *Inbound) Close() error {
|
||||
return common.Close(
|
||||
t.tunStack,
|
||||
t.tunIf,
|
||||
t.autoRedirect,
|
||||
)
|
||||
}
|
||||
|
||||
func (t *Inbound) PrepareConnection(network string, source M.Socksaddr, destination M.Socksaddr) error {
|
||||
return t.router.PreMatch(adapter.InboundContext{
|
||||
Inbound: t.tag,
|
||||
InboundType: C.TypeTun,
|
||||
Network: network,
|
||||
Source: source,
|
||||
Destination: destination,
|
||||
InboundOptions: t.inboundOptions,
|
||||
})
|
||||
}
|
||||
|
||||
func (t *Inbound) NewConnectionEx(ctx context.Context, conn net.Conn, source M.Socksaddr, destination M.Socksaddr, onClose N.CloseHandlerFunc) {
|
||||
ctx = log.ContextWithNewID(ctx)
|
||||
var metadata adapter.InboundContext
|
||||
metadata.Inbound = t.tag
|
||||
metadata.InboundType = C.TypeTun
|
||||
metadata.Source = source
|
||||
metadata.Destination = destination
|
||||
metadata.InboundOptions = t.inboundOptions
|
||||
t.logger.InfoContext(ctx, "inbound connection from ", metadata.Source)
|
||||
t.logger.InfoContext(ctx, "inbound connection to ", metadata.Destination)
|
||||
t.router.RouteConnectionEx(ctx, conn, metadata, onClose)
|
||||
}
|
||||
|
||||
func (t *Inbound) NewPacketConnectionEx(ctx context.Context, conn N.PacketConn, source M.Socksaddr, destination M.Socksaddr, onClose N.CloseHandlerFunc) {
|
||||
ctx = log.ContextWithNewID(ctx)
|
||||
var metadata adapter.InboundContext
|
||||
metadata.Inbound = t.tag
|
||||
metadata.InboundType = C.TypeTun
|
||||
metadata.Source = source
|
||||
metadata.Destination = destination
|
||||
metadata.InboundOptions = t.inboundOptions
|
||||
t.logger.InfoContext(ctx, "inbound packet connection from ", metadata.Source)
|
||||
t.logger.InfoContext(ctx, "inbound packet connection to ", metadata.Destination)
|
||||
t.router.RoutePacketConnectionEx(ctx, conn, metadata, onClose)
|
||||
}
|
||||
|
||||
type autoRedirectHandler Inbound
|
||||
|
||||
func (t *autoRedirectHandler) NewConnectionEx(ctx context.Context, conn net.Conn, source M.Socksaddr, destination M.Socksaddr, onClose N.CloseHandlerFunc) {
|
||||
ctx = log.ContextWithNewID(ctx)
|
||||
var metadata adapter.InboundContext
|
||||
metadata.Inbound = t.tag
|
||||
metadata.InboundType = C.TypeTun
|
||||
metadata.Source = source
|
||||
metadata.Destination = destination
|
||||
metadata.InboundOptions = t.inboundOptions
|
||||
t.logger.InfoContext(ctx, "inbound redirect connection from ", metadata.Source)
|
||||
t.logger.InfoContext(ctx, "inbound connection to ", metadata.Destination)
|
||||
t.router.RouteConnectionEx(ctx, conn, metadata, onClose)
|
||||
}
|
221
protocol/vless/inbound.go
Normal file
221
protocol/vless/inbound.go
Normal file
|
@ -0,0 +1,221 @@
|
|||
package vless
|
||||
|
||||
import (
|
||||
"context"
|
||||
"net"
|
||||
"os"
|
||||
|
||||
"github.com/sagernet/sing-box/adapter"
|
||||
"github.com/sagernet/sing-box/adapter/inbound"
|
||||
"github.com/sagernet/sing-box/common/listener"
|
||||
"github.com/sagernet/sing-box/common/mux"
|
||||
"github.com/sagernet/sing-box/common/tls"
|
||||
"github.com/sagernet/sing-box/common/uot"
|
||||
C "github.com/sagernet/sing-box/constant"
|
||||
"github.com/sagernet/sing-box/log"
|
||||
"github.com/sagernet/sing-box/option"
|
||||
"github.com/sagernet/sing-box/transport/v2ray"
|
||||
"github.com/sagernet/sing-vmess"
|
||||
"github.com/sagernet/sing-vmess/packetaddr"
|
||||
"github.com/sagernet/sing-vmess/vless"
|
||||
"github.com/sagernet/sing/common"
|
||||
"github.com/sagernet/sing/common/auth"
|
||||
E "github.com/sagernet/sing/common/exceptions"
|
||||
F "github.com/sagernet/sing/common/format"
|
||||
"github.com/sagernet/sing/common/logger"
|
||||
M "github.com/sagernet/sing/common/metadata"
|
||||
N "github.com/sagernet/sing/common/network"
|
||||
)
|
||||
|
||||
func RegisterInbound(registry *inbound.Registry) {
|
||||
inbound.Register[option.VLESSInboundOptions](registry, C.TypeVLESS, NewInbound)
|
||||
}
|
||||
|
||||
var _ adapter.TCPInjectableInbound = (*Inbound)(nil)
|
||||
|
||||
type Inbound struct {
|
||||
inbound.Adapter
|
||||
ctx context.Context
|
||||
router adapter.ConnectionRouterEx
|
||||
logger logger.ContextLogger
|
||||
listener *listener.Listener
|
||||
users []option.VLESSUser
|
||||
service *vless.Service[int]
|
||||
tlsConfig tls.ServerConfig
|
||||
transport adapter.V2RayServerTransport
|
||||
}
|
||||
|
||||
func NewInbound(ctx context.Context, router adapter.Router, logger log.ContextLogger, tag string, options option.VLESSInboundOptions) (adapter.Inbound, error) {
|
||||
inbound := &Inbound{
|
||||
Adapter: inbound.NewAdapter(C.TypeVLESS, tag),
|
||||
ctx: ctx,
|
||||
router: uot.NewRouter(router, logger),
|
||||
logger: logger,
|
||||
users: options.Users,
|
||||
}
|
||||
var err error
|
||||
inbound.router, err = mux.NewRouterWithOptions(inbound.router, logger, common.PtrValueOrDefault(options.Multiplex))
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
service := vless.NewService[int](logger, adapter.NewUpstreamContextHandler(inbound.newConnection, inbound.newPacketConnection, inbound))
|
||||
service.UpdateUsers(common.MapIndexed(inbound.users, func(index int, _ option.VLESSUser) int {
|
||||
return index
|
||||
}), common.Map(inbound.users, func(it option.VLESSUser) string {
|
||||
return it.UUID
|
||||
}), common.Map(inbound.users, func(it option.VLESSUser) string {
|
||||
return it.Flow
|
||||
}))
|
||||
inbound.service = service
|
||||
if options.TLS != nil {
|
||||
inbound.tlsConfig, err = tls.NewServer(ctx, logger, common.PtrValueOrDefault(options.TLS))
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
}
|
||||
if options.Transport != nil {
|
||||
inbound.transport, err = v2ray.NewServerTransport(ctx, logger, common.PtrValueOrDefault(options.Transport), inbound.tlsConfig, (*inboundTransportHandler)(inbound))
|
||||
if err != nil {
|
||||
return nil, E.Cause(err, "create server transport: ", options.Transport.Type)
|
||||
}
|
||||
}
|
||||
inbound.listener = listener.New(listener.Options{
|
||||
Context: ctx,
|
||||
Logger: logger,
|
||||
Network: []string{N.NetworkTCP},
|
||||
Listen: options.ListenOptions,
|
||||
ConnectionHandler: inbound,
|
||||
})
|
||||
return inbound, nil
|
||||
}
|
||||
|
||||
func (h *Inbound) Start() error {
|
||||
if h.tlsConfig != nil {
|
||||
err := h.tlsConfig.Start()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
if h.transport == nil {
|
||||
return h.listener.Start()
|
||||
}
|
||||
if common.Contains(h.transport.Network(), N.NetworkTCP) {
|
||||
tcpListener, err := h.listener.ListenTCP()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
go func() {
|
||||
sErr := h.transport.Serve(tcpListener)
|
||||
if sErr != nil && !E.IsClosed(sErr) {
|
||||
h.logger.Error("transport serve error: ", sErr)
|
||||
}
|
||||
}()
|
||||
}
|
||||
if common.Contains(h.transport.Network(), N.NetworkUDP) {
|
||||
udpConn, err := h.listener.ListenUDP()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
go func() {
|
||||
sErr := h.transport.ServePacket(udpConn)
|
||||
if sErr != nil && !E.IsClosed(sErr) {
|
||||
h.logger.Error("transport serve error: ", sErr)
|
||||
}
|
||||
}()
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (h *Inbound) Close() error {
|
||||
return common.Close(
|
||||
h.service,
|
||||
&h.listener,
|
||||
h.tlsConfig,
|
||||
h.transport,
|
||||
)
|
||||
}
|
||||
|
||||
func (h *Inbound) NewConnection(ctx context.Context, conn net.Conn, metadata adapter.InboundContext) error {
|
||||
var err error
|
||||
if h.tlsConfig != nil && h.transport == nil {
|
||||
conn, err = tls.ServerHandshake(ctx, conn, h.tlsConfig)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
return h.service.NewConnection(adapter.WithContext(log.ContextWithNewID(ctx), &metadata), conn, adapter.UpstreamMetadata(metadata))
|
||||
}
|
||||
|
||||
func (h *Inbound) NewConnectionEx(ctx context.Context, conn net.Conn, metadata adapter.InboundContext, onClose N.CloseHandlerFunc) {
|
||||
err := h.NewConnection(ctx, conn, metadata)
|
||||
N.CloseOnHandshakeFailure(conn, onClose, err)
|
||||
if err != nil {
|
||||
h.logger.ErrorContext(ctx, E.Cause(err, "process connection from ", metadata.Source))
|
||||
}
|
||||
}
|
||||
|
||||
func (h *Inbound) newConnection(ctx context.Context, conn net.Conn, metadata adapter.InboundContext) error {
|
||||
userIndex, loaded := auth.UserFromContext[int](ctx)
|
||||
if !loaded {
|
||||
return os.ErrInvalid
|
||||
}
|
||||
user := h.users[userIndex].Name
|
||||
if user == "" {
|
||||
user = F.ToString(userIndex)
|
||||
} else {
|
||||
metadata.User = user
|
||||
}
|
||||
h.logger.InfoContext(ctx, "[", user, "] inbound connection to ", metadata.Destination)
|
||||
return h.router.RouteConnection(ctx, conn, metadata)
|
||||
}
|
||||
|
||||
func (h *Inbound) newPacketConnection(ctx context.Context, conn N.PacketConn, metadata adapter.InboundContext) error {
|
||||
userIndex, loaded := auth.UserFromContext[int](ctx)
|
||||
if !loaded {
|
||||
return os.ErrInvalid
|
||||
}
|
||||
user := h.users[userIndex].Name
|
||||
if user == "" {
|
||||
user = F.ToString(userIndex)
|
||||
} else {
|
||||
metadata.User = user
|
||||
}
|
||||
if metadata.Destination.Fqdn == packetaddr.SeqPacketMagicAddress {
|
||||
metadata.Destination = M.Socksaddr{}
|
||||
conn = packetaddr.NewConn(conn.(vmess.PacketConn), metadata.Destination)
|
||||
h.logger.InfoContext(ctx, "[", user, "] inbound packet addr connection")
|
||||
} else {
|
||||
h.logger.InfoContext(ctx, "[", user, "] inbound packet connection to ", metadata.Destination)
|
||||
}
|
||||
return h.router.RoutePacketConnection(ctx, conn, metadata)
|
||||
}
|
||||
|
||||
var _ adapter.V2RayServerTransportHandler = (*inboundTransportHandler)(nil)
|
||||
|
||||
type inboundTransportHandler Inbound
|
||||
|
||||
func (h *inboundTransportHandler) NewConnectionEx(ctx context.Context, conn net.Conn, source M.Socksaddr, destination M.Socksaddr, onClose N.CloseHandlerFunc) {
|
||||
var metadata adapter.InboundContext
|
||||
metadata.Inbound = h.Tag()
|
||||
metadata.InboundType = h.Type()
|
||||
metadata.InboundDetour = h.listener.ListenOptions().Detour
|
||||
metadata.InboundOptions = h.listener.ListenOptions().InboundOptions
|
||||
metadata.Source = source
|
||||
metadata.Destination = destination
|
||||
h.logger.InfoContext(ctx, "inbound connection from ", metadata.Source)
|
||||
(*Inbound)(h).NewConnectionEx(ctx, conn, metadata, onClose)
|
||||
}
|
||||
|
||||
func (h *Inbound) NewError(ctx context.Context, err error) {
|
||||
NewError(h.logger, ctx, err)
|
||||
}
|
||||
|
||||
// Deprecated: remove
|
||||
func NewError(logger logger.ContextLogger, ctx context.Context, err error) {
|
||||
common.Close(err)
|
||||
if E.IsClosedOrCanceled(err) {
|
||||
logger.DebugContext(ctx, "connection closed: ", err)
|
||||
return
|
||||
}
|
||||
logger.ErrorContext(ctx, err)
|
||||
}
|
211
protocol/vless/outbound.go
Normal file
211
protocol/vless/outbound.go
Normal file
|
@ -0,0 +1,211 @@
|
|||
package vless
|
||||
|
||||
import (
|
||||
"context"
|
||||
"net"
|
||||
|
||||
"github.com/sagernet/sing-box/adapter"
|
||||
"github.com/sagernet/sing-box/adapter/outbound"
|
||||
"github.com/sagernet/sing-box/common/dialer"
|
||||
"github.com/sagernet/sing-box/common/mux"
|
||||
"github.com/sagernet/sing-box/common/tls"
|
||||
C "github.com/sagernet/sing-box/constant"
|
||||
"github.com/sagernet/sing-box/log"
|
||||
"github.com/sagernet/sing-box/option"
|
||||
"github.com/sagernet/sing-box/transport/v2ray"
|
||||
"github.com/sagernet/sing-vmess/packetaddr"
|
||||
"github.com/sagernet/sing-vmess/vless"
|
||||
"github.com/sagernet/sing/common"
|
||||
"github.com/sagernet/sing/common/bufio"
|
||||
E "github.com/sagernet/sing/common/exceptions"
|
||||
"github.com/sagernet/sing/common/logger"
|
||||
M "github.com/sagernet/sing/common/metadata"
|
||||
N "github.com/sagernet/sing/common/network"
|
||||
)
|
||||
|
||||
func RegisterOutbound(registry *outbound.Registry) {
|
||||
outbound.Register[option.VLESSOutboundOptions](registry, C.TypeVLESS, NewOutbound)
|
||||
}
|
||||
|
||||
type Outbound struct {
|
||||
outbound.Adapter
|
||||
logger logger.ContextLogger
|
||||
dialer N.Dialer
|
||||
client *vless.Client
|
||||
serverAddr M.Socksaddr
|
||||
multiplexDialer *mux.Client
|
||||
tlsConfig tls.Config
|
||||
transport adapter.V2RayClientTransport
|
||||
packetAddr bool
|
||||
xudp bool
|
||||
}
|
||||
|
||||
func NewOutbound(ctx context.Context, router adapter.Router, logger log.ContextLogger, tag string, options option.VLESSOutboundOptions) (adapter.Outbound, error) {
|
||||
outboundDialer, err := dialer.New(router, options.DialerOptions)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
outbound := &Outbound{
|
||||
Adapter: outbound.NewAdapterWithDialerOptions(C.TypeVLESS, options.Network.Build(), tag, options.DialerOptions),
|
||||
logger: logger,
|
||||
dialer: outboundDialer,
|
||||
serverAddr: options.ServerOptions.Build(),
|
||||
}
|
||||
if options.TLS != nil {
|
||||
outbound.tlsConfig, err = tls.NewClient(ctx, options.Server, common.PtrValueOrDefault(options.TLS))
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
}
|
||||
if options.Transport != nil {
|
||||
outbound.transport, err = v2ray.NewClientTransport(ctx, outbound.dialer, outbound.serverAddr, common.PtrValueOrDefault(options.Transport), outbound.tlsConfig)
|
||||
if err != nil {
|
||||
return nil, E.Cause(err, "create client transport: ", options.Transport.Type)
|
||||
}
|
||||
}
|
||||
if options.PacketEncoding == nil {
|
||||
outbound.xudp = true
|
||||
} else {
|
||||
switch *options.PacketEncoding {
|
||||
case "":
|
||||
case "packetaddr":
|
||||
outbound.packetAddr = true
|
||||
case "xudp":
|
||||
outbound.xudp = true
|
||||
default:
|
||||
return nil, E.New("unknown packet encoding: ", options.PacketEncoding)
|
||||
}
|
||||
}
|
||||
outbound.client, err = vless.NewClient(options.UUID, options.Flow, logger)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
outbound.multiplexDialer, err = mux.NewClientWithOptions((*vlessDialer)(outbound), logger, common.PtrValueOrDefault(options.Multiplex))
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return outbound, nil
|
||||
}
|
||||
|
||||
func (h *Outbound) DialContext(ctx context.Context, network string, destination M.Socksaddr) (net.Conn, error) {
|
||||
if h.multiplexDialer == nil {
|
||||
switch N.NetworkName(network) {
|
||||
case N.NetworkTCP:
|
||||
h.logger.InfoContext(ctx, "outbound connection to ", destination)
|
||||
case N.NetworkUDP:
|
||||
h.logger.InfoContext(ctx, "outbound packet connection to ", destination)
|
||||
}
|
||||
return (*vlessDialer)(h).DialContext(ctx, network, destination)
|
||||
} else {
|
||||
switch N.NetworkName(network) {
|
||||
case N.NetworkTCP:
|
||||
h.logger.InfoContext(ctx, "outbound multiplex connection to ", destination)
|
||||
case N.NetworkUDP:
|
||||
h.logger.InfoContext(ctx, "outbound multiplex packet connection to ", destination)
|
||||
}
|
||||
return h.multiplexDialer.DialContext(ctx, network, destination)
|
||||
}
|
||||
}
|
||||
|
||||
func (h *Outbound) ListenPacket(ctx context.Context, destination M.Socksaddr) (net.PacketConn, error) {
|
||||
if h.multiplexDialer == nil {
|
||||
h.logger.InfoContext(ctx, "outbound packet connection to ", destination)
|
||||
return (*vlessDialer)(h).ListenPacket(ctx, destination)
|
||||
} else {
|
||||
h.logger.InfoContext(ctx, "outbound multiplex packet connection to ", destination)
|
||||
return h.multiplexDialer.ListenPacket(ctx, destination)
|
||||
}
|
||||
}
|
||||
|
||||
func (h *Outbound) InterfaceUpdated() {
|
||||
if h.transport != nil {
|
||||
h.transport.Close()
|
||||
}
|
||||
if h.multiplexDialer != nil {
|
||||
h.multiplexDialer.Reset()
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
func (h *Outbound) Close() error {
|
||||
return common.Close(common.PtrOrNil(h.multiplexDialer), h.transport)
|
||||
}
|
||||
|
||||
type vlessDialer Outbound
|
||||
|
||||
func (h *vlessDialer) DialContext(ctx context.Context, network string, destination M.Socksaddr) (net.Conn, error) {
|
||||
ctx, metadata := adapter.ExtendContext(ctx)
|
||||
metadata.Outbound = h.Tag()
|
||||
metadata.Destination = destination
|
||||
var conn net.Conn
|
||||
var err error
|
||||
if h.transport != nil {
|
||||
conn, err = h.transport.DialContext(ctx)
|
||||
} else {
|
||||
conn, err = h.dialer.DialContext(ctx, N.NetworkTCP, h.serverAddr)
|
||||
if err == nil && h.tlsConfig != nil {
|
||||
conn, err = tls.ClientHandshake(ctx, conn, h.tlsConfig)
|
||||
}
|
||||
}
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
switch N.NetworkName(network) {
|
||||
case N.NetworkTCP:
|
||||
h.logger.InfoContext(ctx, "outbound connection to ", destination)
|
||||
return h.client.DialEarlyConn(conn, destination)
|
||||
case N.NetworkUDP:
|
||||
h.logger.InfoContext(ctx, "outbound packet connection to ", destination)
|
||||
if h.xudp {
|
||||
return h.client.DialEarlyXUDPPacketConn(conn, destination)
|
||||
} else if h.packetAddr {
|
||||
if destination.IsFqdn() {
|
||||
return nil, E.New("packetaddr: domain destination is not supported")
|
||||
}
|
||||
packetConn, err := h.client.DialEarlyPacketConn(conn, M.Socksaddr{Fqdn: packetaddr.SeqPacketMagicAddress})
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return bufio.NewBindPacketConn(packetaddr.NewConn(packetConn, destination), destination), nil
|
||||
} else {
|
||||
return h.client.DialEarlyPacketConn(conn, destination)
|
||||
}
|
||||
default:
|
||||
return nil, E.Extend(N.ErrUnknownNetwork, network)
|
||||
}
|
||||
}
|
||||
|
||||
func (h *vlessDialer) ListenPacket(ctx context.Context, destination M.Socksaddr) (net.PacketConn, error) {
|
||||
h.logger.InfoContext(ctx, "outbound packet connection to ", destination)
|
||||
ctx, metadata := adapter.ExtendContext(ctx)
|
||||
metadata.Outbound = h.Tag()
|
||||
metadata.Destination = destination
|
||||
var conn net.Conn
|
||||
var err error
|
||||
if h.transport != nil {
|
||||
conn, err = h.transport.DialContext(ctx)
|
||||
} else {
|
||||
conn, err = h.dialer.DialContext(ctx, N.NetworkTCP, h.serverAddr)
|
||||
if err == nil && h.tlsConfig != nil {
|
||||
conn, err = tls.ClientHandshake(ctx, conn, h.tlsConfig)
|
||||
}
|
||||
}
|
||||
if err != nil {
|
||||
common.Close(conn)
|
||||
return nil, err
|
||||
}
|
||||
if h.xudp {
|
||||
return h.client.DialEarlyXUDPPacketConn(conn, destination)
|
||||
} else if h.packetAddr {
|
||||
if destination.IsFqdn() {
|
||||
return nil, E.New("packetaddr: domain destination is not supported")
|
||||
}
|
||||
conn, err := h.client.DialEarlyPacketConn(conn, M.Socksaddr{Fqdn: packetaddr.SeqPacketMagicAddress})
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return packetaddr.NewConn(conn, destination), nil
|
||||
} else {
|
||||
return h.client.DialEarlyPacketConn(conn, destination)
|
||||
}
|
||||
}
|
235
protocol/vmess/inbound.go
Normal file
235
protocol/vmess/inbound.go
Normal file
|
@ -0,0 +1,235 @@
|
|||
package vmess
|
||||
|
||||
import (
|
||||
"context"
|
||||
"net"
|
||||
"os"
|
||||
|
||||
"github.com/sagernet/sing-box/adapter"
|
||||
"github.com/sagernet/sing-box/adapter/inbound"
|
||||
"github.com/sagernet/sing-box/common/listener"
|
||||
"github.com/sagernet/sing-box/common/mux"
|
||||
"github.com/sagernet/sing-box/common/tls"
|
||||
"github.com/sagernet/sing-box/common/uot"
|
||||
C "github.com/sagernet/sing-box/constant"
|
||||
"github.com/sagernet/sing-box/log"
|
||||
"github.com/sagernet/sing-box/option"
|
||||
"github.com/sagernet/sing-box/transport/v2ray"
|
||||
"github.com/sagernet/sing-vmess"
|
||||
"github.com/sagernet/sing-vmess/packetaddr"
|
||||
"github.com/sagernet/sing/common"
|
||||
"github.com/sagernet/sing/common/auth"
|
||||
E "github.com/sagernet/sing/common/exceptions"
|
||||
F "github.com/sagernet/sing/common/format"
|
||||
"github.com/sagernet/sing/common/logger"
|
||||
M "github.com/sagernet/sing/common/metadata"
|
||||
N "github.com/sagernet/sing/common/network"
|
||||
"github.com/sagernet/sing/common/ntp"
|
||||
)
|
||||
|
||||
func RegisterInbound(registry *inbound.Registry) {
|
||||
inbound.Register[option.VMessInboundOptions](registry, C.TypeVMess, NewInbound)
|
||||
}
|
||||
|
||||
var _ adapter.TCPInjectableInbound = (*Inbound)(nil)
|
||||
|
||||
type Inbound struct {
|
||||
inbound.Adapter
|
||||
ctx context.Context
|
||||
router adapter.ConnectionRouterEx
|
||||
logger logger.ContextLogger
|
||||
listener *listener.Listener
|
||||
service *vmess.Service[int]
|
||||
users []option.VMessUser
|
||||
tlsConfig tls.ServerConfig
|
||||
transport adapter.V2RayServerTransport
|
||||
}
|
||||
|
||||
func NewInbound(ctx context.Context, router adapter.Router, logger log.ContextLogger, tag string, options option.VMessInboundOptions) (adapter.Inbound, error) {
|
||||
inbound := &Inbound{
|
||||
Adapter: inbound.NewAdapter(C.TypeVMess, tag),
|
||||
ctx: ctx,
|
||||
router: uot.NewRouter(router, logger),
|
||||
logger: logger,
|
||||
users: options.Users,
|
||||
}
|
||||
var err error
|
||||
inbound.router, err = mux.NewRouterWithOptions(inbound.router, logger, common.PtrValueOrDefault(options.Multiplex))
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
var serviceOptions []vmess.ServiceOption
|
||||
if timeFunc := ntp.TimeFuncFromContext(ctx); timeFunc != nil {
|
||||
serviceOptions = append(serviceOptions, vmess.ServiceWithTimeFunc(timeFunc))
|
||||
}
|
||||
if options.Transport != nil && options.Transport.Type != "" {
|
||||
serviceOptions = append(serviceOptions, vmess.ServiceWithDisableHeaderProtection())
|
||||
}
|
||||
service := vmess.NewService[int](adapter.NewUpstreamContextHandler(inbound.newConnection, inbound.newPacketConnection, inbound), serviceOptions...)
|
||||
inbound.service = service
|
||||
err = service.UpdateUsers(common.MapIndexed(options.Users, func(index int, it option.VMessUser) int {
|
||||
return index
|
||||
}), common.Map(options.Users, func(it option.VMessUser) string {
|
||||
return it.UUID
|
||||
}), common.Map(options.Users, func(it option.VMessUser) int {
|
||||
return it.AlterId
|
||||
}))
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if options.TLS != nil {
|
||||
inbound.tlsConfig, err = tls.NewServer(ctx, logger, common.PtrValueOrDefault(options.TLS))
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
}
|
||||
if options.Transport != nil {
|
||||
inbound.transport, err = v2ray.NewServerTransport(ctx, logger, common.PtrValueOrDefault(options.Transport), inbound.tlsConfig, (*inboundTransportHandler)(inbound))
|
||||
if err != nil {
|
||||
return nil, E.Cause(err, "create server transport: ", options.Transport.Type)
|
||||
}
|
||||
}
|
||||
inbound.listener = listener.New(listener.Options{
|
||||
Context: ctx,
|
||||
Logger: logger,
|
||||
Network: []string{N.NetworkTCP},
|
||||
Listen: options.ListenOptions,
|
||||
ConnectionHandler: inbound,
|
||||
})
|
||||
return inbound, nil
|
||||
}
|
||||
|
||||
func (h *Inbound) Start() error {
|
||||
err := h.service.Start()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if h.tlsConfig != nil {
|
||||
err = h.tlsConfig.Start()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
if h.transport == nil {
|
||||
return h.listener.Start()
|
||||
}
|
||||
if common.Contains(h.transport.Network(), N.NetworkTCP) {
|
||||
tcpListener, err := h.listener.ListenTCP()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
go func() {
|
||||
sErr := h.transport.Serve(tcpListener)
|
||||
if sErr != nil && !E.IsClosed(sErr) {
|
||||
h.logger.Error("transport serve error: ", sErr)
|
||||
}
|
||||
}()
|
||||
}
|
||||
if common.Contains(h.transport.Network(), N.NetworkUDP) {
|
||||
udpConn, err := h.listener.ListenUDP()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
go func() {
|
||||
sErr := h.transport.ServePacket(udpConn)
|
||||
if sErr != nil && !E.IsClosed(sErr) {
|
||||
h.logger.Error("transport serve error: ", sErr)
|
||||
}
|
||||
}()
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (h *Inbound) Close() error {
|
||||
return common.Close(
|
||||
h.service,
|
||||
&h.listener,
|
||||
h.tlsConfig,
|
||||
h.transport,
|
||||
)
|
||||
}
|
||||
|
||||
func (h *Inbound) NewConnection(ctx context.Context, conn net.Conn, metadata adapter.InboundContext) error {
|
||||
var err error
|
||||
if h.tlsConfig != nil && h.transport == nil {
|
||||
conn, err = tls.ServerHandshake(ctx, conn, h.tlsConfig)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
return h.service.NewConnection(adapter.WithContext(log.ContextWithNewID(ctx), &metadata), conn, adapter.UpstreamMetadata(metadata))
|
||||
}
|
||||
|
||||
func (h *Inbound) NewConnectionEx(ctx context.Context, conn net.Conn, metadata adapter.InboundContext, onClose N.CloseHandlerFunc) {
|
||||
err := h.NewConnection(ctx, conn, metadata)
|
||||
N.CloseOnHandshakeFailure(conn, onClose, err)
|
||||
if err != nil {
|
||||
h.logger.ErrorContext(ctx, E.Cause(err, "process connection from ", metadata.Source))
|
||||
}
|
||||
}
|
||||
|
||||
func (h *Inbound) newConnection(ctx context.Context, conn net.Conn, metadata adapter.InboundContext) error {
|
||||
userIndex, loaded := auth.UserFromContext[int](ctx)
|
||||
if !loaded {
|
||||
return os.ErrInvalid
|
||||
}
|
||||
user := h.users[userIndex].Name
|
||||
if user == "" {
|
||||
user = F.ToString(userIndex)
|
||||
} else {
|
||||
metadata.User = user
|
||||
}
|
||||
h.logger.InfoContext(ctx, "[", user, "] inbound connection to ", metadata.Destination)
|
||||
return h.router.RouteConnection(ctx, conn, metadata)
|
||||
}
|
||||
|
||||
func (h *Inbound) newPacketConnection(ctx context.Context, conn N.PacketConn, metadata adapter.InboundContext) error {
|
||||
userIndex, loaded := auth.UserFromContext[int](ctx)
|
||||
if !loaded {
|
||||
return os.ErrInvalid
|
||||
}
|
||||
user := h.users[userIndex].Name
|
||||
if user == "" {
|
||||
user = F.ToString(userIndex)
|
||||
} else {
|
||||
metadata.User = user
|
||||
}
|
||||
if metadata.Destination.Fqdn == packetaddr.SeqPacketMagicAddress {
|
||||
metadata.Destination = M.Socksaddr{}
|
||||
conn = packetaddr.NewConn(conn.(vmess.PacketConn), metadata.Destination)
|
||||
h.logger.InfoContext(ctx, "[", user, "] inbound packet addr connection")
|
||||
} else {
|
||||
h.logger.InfoContext(ctx, "[", user, "] inbound packet connection to ", metadata.Destination)
|
||||
}
|
||||
return h.router.RoutePacketConnection(ctx, conn, metadata)
|
||||
}
|
||||
|
||||
var _ adapter.V2RayServerTransportHandler = (*inboundTransportHandler)(nil)
|
||||
|
||||
type inboundTransportHandler Inbound
|
||||
|
||||
func (h *inboundTransportHandler) NewConnectionEx(ctx context.Context, conn net.Conn, source M.Socksaddr, destination M.Socksaddr, onClose N.CloseHandlerFunc) {
|
||||
var metadata adapter.InboundContext
|
||||
metadata.Inbound = h.Tag()
|
||||
metadata.InboundType = h.Type()
|
||||
metadata.InboundDetour = h.listener.ListenOptions().Detour
|
||||
metadata.InboundOptions = h.listener.ListenOptions().InboundOptions
|
||||
metadata.Source = source
|
||||
metadata.Destination = destination
|
||||
h.logger.InfoContext(ctx, "inbound connection from ", metadata.Source)
|
||||
(*Inbound)(h).NewConnectionEx(ctx, conn, metadata, onClose)
|
||||
}
|
||||
|
||||
func (h *Inbound) NewError(ctx context.Context, err error) {
|
||||
NewError(h.logger, ctx, err)
|
||||
}
|
||||
|
||||
// Deprecated: remove
|
||||
func NewError(logger logger.ContextLogger, ctx context.Context, err error) {
|
||||
common.Close(err)
|
||||
if E.IsClosedOrCanceled(err) {
|
||||
logger.DebugContext(ctx, "connection closed: ", err)
|
||||
return
|
||||
}
|
||||
logger.ErrorContext(ctx, err)
|
||||
}
|
205
protocol/vmess/outbound.go
Normal file
205
protocol/vmess/outbound.go
Normal file
|
@ -0,0 +1,205 @@
|
|||
package vmess
|
||||
|
||||
import (
|
||||
"context"
|
||||
"net"
|
||||
|
||||
"github.com/sagernet/sing-box/adapter"
|
||||
"github.com/sagernet/sing-box/adapter/outbound"
|
||||
"github.com/sagernet/sing-box/common/dialer"
|
||||
"github.com/sagernet/sing-box/common/mux"
|
||||
"github.com/sagernet/sing-box/common/tls"
|
||||
C "github.com/sagernet/sing-box/constant"
|
||||
"github.com/sagernet/sing-box/log"
|
||||
"github.com/sagernet/sing-box/option"
|
||||
"github.com/sagernet/sing-box/transport/v2ray"
|
||||
"github.com/sagernet/sing-vmess"
|
||||
"github.com/sagernet/sing-vmess/packetaddr"
|
||||
"github.com/sagernet/sing/common"
|
||||
E "github.com/sagernet/sing/common/exceptions"
|
||||
"github.com/sagernet/sing/common/logger"
|
||||
M "github.com/sagernet/sing/common/metadata"
|
||||
N "github.com/sagernet/sing/common/network"
|
||||
"github.com/sagernet/sing/common/ntp"
|
||||
)
|
||||
|
||||
func RegisterOutbound(registry *outbound.Registry) {
|
||||
outbound.Register[option.VMessOutboundOptions](registry, C.TypeVMess, NewOutbound)
|
||||
}
|
||||
|
||||
type Outbound struct {
|
||||
outbound.Adapter
|
||||
logger logger.ContextLogger
|
||||
dialer N.Dialer
|
||||
client *vmess.Client
|
||||
serverAddr M.Socksaddr
|
||||
multiplexDialer *mux.Client
|
||||
tlsConfig tls.Config
|
||||
transport adapter.V2RayClientTransport
|
||||
packetAddr bool
|
||||
xudp bool
|
||||
}
|
||||
|
||||
func NewOutbound(ctx context.Context, router adapter.Router, logger log.ContextLogger, tag string, options option.VMessOutboundOptions) (adapter.Outbound, error) {
|
||||
outboundDialer, err := dialer.New(router, options.DialerOptions)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
outbound := &Outbound{
|
||||
Adapter: outbound.NewAdapterWithDialerOptions(C.TypeVMess, options.Network.Build(), tag, options.DialerOptions),
|
||||
logger: logger,
|
||||
dialer: outboundDialer,
|
||||
serverAddr: options.ServerOptions.Build(),
|
||||
}
|
||||
if options.TLS != nil {
|
||||
outbound.tlsConfig, err = tls.NewClient(ctx, options.Server, common.PtrValueOrDefault(options.TLS))
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
}
|
||||
if options.Transport != nil {
|
||||
outbound.transport, err = v2ray.NewClientTransport(ctx, outbound.dialer, outbound.serverAddr, common.PtrValueOrDefault(options.Transport), outbound.tlsConfig)
|
||||
if err != nil {
|
||||
return nil, E.Cause(err, "create client transport: ", options.Transport.Type)
|
||||
}
|
||||
}
|
||||
outbound.multiplexDialer, err = mux.NewClientWithOptions((*vmessDialer)(outbound), logger, common.PtrValueOrDefault(options.Multiplex))
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
switch options.PacketEncoding {
|
||||
case "":
|
||||
case "packetaddr":
|
||||
outbound.packetAddr = true
|
||||
case "xudp":
|
||||
outbound.xudp = true
|
||||
default:
|
||||
return nil, E.New("unknown packet encoding: ", options.PacketEncoding)
|
||||
}
|
||||
var clientOptions []vmess.ClientOption
|
||||
if timeFunc := ntp.TimeFuncFromContext(ctx); timeFunc != nil {
|
||||
clientOptions = append(clientOptions, vmess.ClientWithTimeFunc(timeFunc))
|
||||
}
|
||||
if options.GlobalPadding {
|
||||
clientOptions = append(clientOptions, vmess.ClientWithGlobalPadding())
|
||||
}
|
||||
if options.AuthenticatedLength {
|
||||
clientOptions = append(clientOptions, vmess.ClientWithAuthenticatedLength())
|
||||
}
|
||||
security := options.Security
|
||||
if security == "" {
|
||||
security = "auto"
|
||||
}
|
||||
if security == "auto" && outbound.tlsConfig != nil {
|
||||
security = "zero"
|
||||
}
|
||||
client, err := vmess.NewClient(options.UUID, security, options.AlterId, clientOptions...)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
outbound.client = client
|
||||
return outbound, nil
|
||||
}
|
||||
|
||||
func (h *Outbound) InterfaceUpdated() {
|
||||
if h.transport != nil {
|
||||
h.transport.Close()
|
||||
}
|
||||
if h.multiplexDialer != nil {
|
||||
h.multiplexDialer.Reset()
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
func (h *Outbound) Close() error {
|
||||
return common.Close(common.PtrOrNil(h.multiplexDialer), h.transport)
|
||||
}
|
||||
|
||||
func (h *Outbound) DialContext(ctx context.Context, network string, destination M.Socksaddr) (net.Conn, error) {
|
||||
if h.multiplexDialer == nil {
|
||||
switch N.NetworkName(network) {
|
||||
case N.NetworkTCP:
|
||||
h.logger.InfoContext(ctx, "outbound connection to ", destination)
|
||||
case N.NetworkUDP:
|
||||
h.logger.InfoContext(ctx, "outbound packet connection to ", destination)
|
||||
}
|
||||
return (*vmessDialer)(h).DialContext(ctx, network, destination)
|
||||
} else {
|
||||
switch N.NetworkName(network) {
|
||||
case N.NetworkTCP:
|
||||
h.logger.InfoContext(ctx, "outbound multiplex connection to ", destination)
|
||||
case N.NetworkUDP:
|
||||
h.logger.InfoContext(ctx, "outbound multiplex packet connection to ", destination)
|
||||
}
|
||||
return h.multiplexDialer.DialContext(ctx, network, destination)
|
||||
}
|
||||
}
|
||||
|
||||
func (h *Outbound) ListenPacket(ctx context.Context, destination M.Socksaddr) (net.PacketConn, error) {
|
||||
if h.multiplexDialer == nil {
|
||||
h.logger.InfoContext(ctx, "outbound packet connection to ", destination)
|
||||
return (*vmessDialer)(h).ListenPacket(ctx, destination)
|
||||
} else {
|
||||
h.logger.InfoContext(ctx, "outbound multiplex packet connection to ", destination)
|
||||
return h.multiplexDialer.ListenPacket(ctx, destination)
|
||||
}
|
||||
}
|
||||
|
||||
type vmessDialer Outbound
|
||||
|
||||
func (h *vmessDialer) DialContext(ctx context.Context, network string, destination M.Socksaddr) (net.Conn, error) {
|
||||
ctx, metadata := adapter.ExtendContext(ctx)
|
||||
metadata.Outbound = h.Tag()
|
||||
metadata.Destination = destination
|
||||
var conn net.Conn
|
||||
var err error
|
||||
if h.transport != nil {
|
||||
conn, err = h.transport.DialContext(ctx)
|
||||
} else {
|
||||
conn, err = h.dialer.DialContext(ctx, N.NetworkTCP, h.serverAddr)
|
||||
if err == nil && h.tlsConfig != nil {
|
||||
conn, err = tls.ClientHandshake(ctx, conn, h.tlsConfig)
|
||||
}
|
||||
}
|
||||
if err != nil {
|
||||
common.Close(conn)
|
||||
return nil, err
|
||||
}
|
||||
switch N.NetworkName(network) {
|
||||
case N.NetworkTCP:
|
||||
return h.client.DialEarlyConn(conn, destination), nil
|
||||
case N.NetworkUDP:
|
||||
return h.client.DialEarlyPacketConn(conn, destination), nil
|
||||
default:
|
||||
return nil, E.Extend(N.ErrUnknownNetwork, network)
|
||||
}
|
||||
}
|
||||
|
||||
func (h *vmessDialer) ListenPacket(ctx context.Context, destination M.Socksaddr) (net.PacketConn, error) {
|
||||
ctx, metadata := adapter.ExtendContext(ctx)
|
||||
metadata.Outbound = h.Tag()
|
||||
metadata.Destination = destination
|
||||
var conn net.Conn
|
||||
var err error
|
||||
if h.transport != nil {
|
||||
conn, err = h.transport.DialContext(ctx)
|
||||
} else {
|
||||
conn, err = h.dialer.DialContext(ctx, N.NetworkTCP, h.serverAddr)
|
||||
if err == nil && h.tlsConfig != nil {
|
||||
conn, err = tls.ClientHandshake(ctx, conn, h.tlsConfig)
|
||||
}
|
||||
}
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if h.packetAddr {
|
||||
if destination.IsFqdn() {
|
||||
return nil, E.New("packetaddr: domain destination is not supported")
|
||||
}
|
||||
return packetaddr.NewConn(h.client.DialEarlyPacketConn(conn, M.Socksaddr{Fqdn: packetaddr.SeqPacketMagicAddress}), destination), nil
|
||||
} else if h.xudp {
|
||||
return h.client.DialEarlyXUDPPacketConn(conn, destination), nil
|
||||
} else {
|
||||
return h.client.DialEarlyPacketConn(conn, destination), nil
|
||||
}
|
||||
}
|
10
protocol/wireguard/init.go
Normal file
10
protocol/wireguard/init.go
Normal file
|
@ -0,0 +1,10 @@
|
|||
package wireguard
|
||||
|
||||
import (
|
||||
"github.com/sagernet/sing-box/common/dialer"
|
||||
"github.com/sagernet/wireguard-go/conn"
|
||||
)
|
||||
|
||||
func init() {
|
||||
dialer.WgControlFns = conn.ControlFns
|
||||
}
|
252
protocol/wireguard/outbound.go
Normal file
252
protocol/wireguard/outbound.go
Normal file
|
@ -0,0 +1,252 @@
|
|||
package wireguard
|
||||
|
||||
import (
|
||||
"context"
|
||||
"encoding/base64"
|
||||
"encoding/hex"
|
||||
"fmt"
|
||||
"net"
|
||||
"net/netip"
|
||||
"strings"
|
||||
|
||||
"github.com/sagernet/sing-box/adapter"
|
||||
"github.com/sagernet/sing-box/adapter/outbound"
|
||||
"github.com/sagernet/sing-box/common/dialer"
|
||||
C "github.com/sagernet/sing-box/constant"
|
||||
"github.com/sagernet/sing-box/log"
|
||||
"github.com/sagernet/sing-box/option"
|
||||
"github.com/sagernet/sing-box/transport/wireguard"
|
||||
"github.com/sagernet/sing-dns"
|
||||
"github.com/sagernet/sing-tun"
|
||||
"github.com/sagernet/sing/common"
|
||||
E "github.com/sagernet/sing/common/exceptions"
|
||||
"github.com/sagernet/sing/common/logger"
|
||||
M "github.com/sagernet/sing/common/metadata"
|
||||
N "github.com/sagernet/sing/common/network"
|
||||
"github.com/sagernet/sing/common/x/list"
|
||||
"github.com/sagernet/sing/service"
|
||||
"github.com/sagernet/sing/service/pause"
|
||||
"github.com/sagernet/wireguard-go/conn"
|
||||
"github.com/sagernet/wireguard-go/device"
|
||||
)
|
||||
|
||||
func RegisterOutbound(registry *outbound.Registry) {
|
||||
outbound.Register[option.WireGuardOutboundOptions](registry, C.TypeWireGuard, NewOutbound)
|
||||
}
|
||||
|
||||
var _ adapter.InterfaceUpdateListener = (*Outbound)(nil)
|
||||
|
||||
type Outbound struct {
|
||||
outbound.Adapter
|
||||
ctx context.Context
|
||||
router adapter.Router
|
||||
logger logger.ContextLogger
|
||||
workers int
|
||||
peers []wireguard.PeerConfig
|
||||
useStdNetBind bool
|
||||
listener N.Dialer
|
||||
ipcConf string
|
||||
|
||||
pauseManager pause.Manager
|
||||
pauseCallback *list.Element[pause.Callback]
|
||||
bind conn.Bind
|
||||
device *device.Device
|
||||
tunDevice wireguard.Device
|
||||
}
|
||||
|
||||
func NewOutbound(ctx context.Context, router adapter.Router, logger log.ContextLogger, tag string, options option.WireGuardOutboundOptions) (adapter.Outbound, error) {
|
||||
outbound := &Outbound{
|
||||
Adapter: outbound.NewAdapterWithDialerOptions(C.TypeWireGuard, options.Network.Build(), tag, options.DialerOptions),
|
||||
ctx: ctx,
|
||||
router: router,
|
||||
logger: logger,
|
||||
workers: options.Workers,
|
||||
pauseManager: service.FromContext[pause.Manager](ctx),
|
||||
}
|
||||
peers, err := wireguard.ParsePeers(options)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
outbound.peers = peers
|
||||
if len(options.LocalAddress) == 0 {
|
||||
return nil, E.New("missing local address")
|
||||
}
|
||||
if options.GSO {
|
||||
if options.GSO && options.Detour != "" {
|
||||
return nil, E.New("gso is conflict with detour")
|
||||
}
|
||||
options.IsWireGuardListener = true
|
||||
outbound.useStdNetBind = true
|
||||
}
|
||||
listener, err := dialer.New(router, options.DialerOptions)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
outbound.listener = listener
|
||||
var privateKey string
|
||||
{
|
||||
bytes, err := base64.StdEncoding.DecodeString(options.PrivateKey)
|
||||
if err != nil {
|
||||
return nil, E.Cause(err, "decode private key")
|
||||
}
|
||||
privateKey = hex.EncodeToString(bytes)
|
||||
}
|
||||
outbound.ipcConf = "private_key=" + privateKey
|
||||
mtu := options.MTU
|
||||
if mtu == 0 {
|
||||
mtu = 1408
|
||||
}
|
||||
var wireTunDevice wireguard.Device
|
||||
if !options.SystemInterface && tun.WithGVisor {
|
||||
wireTunDevice, err = wireguard.NewStackDevice(options.LocalAddress, mtu)
|
||||
} else {
|
||||
wireTunDevice, err = wireguard.NewSystemDevice(router, options.InterfaceName, options.LocalAddress, mtu, options.GSO)
|
||||
}
|
||||
if err != nil {
|
||||
return nil, E.Cause(err, "create WireGuard device")
|
||||
}
|
||||
outbound.tunDevice = wireTunDevice
|
||||
return outbound, nil
|
||||
}
|
||||
|
||||
func (w *Outbound) Start() error {
|
||||
if common.Any(w.peers, func(peer wireguard.PeerConfig) bool {
|
||||
return !peer.Endpoint.IsValid()
|
||||
}) {
|
||||
// wait for all outbounds to be started and continue in PortStart
|
||||
return nil
|
||||
}
|
||||
return w.start()
|
||||
}
|
||||
|
||||
func (w *Outbound) PostStart() error {
|
||||
if common.All(w.peers, func(peer wireguard.PeerConfig) bool {
|
||||
return peer.Endpoint.IsValid()
|
||||
}) {
|
||||
return nil
|
||||
}
|
||||
return w.start()
|
||||
}
|
||||
|
||||
func (w *Outbound) start() error {
|
||||
err := wireguard.ResolvePeers(w.ctx, w.router, w.peers)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
var bind conn.Bind
|
||||
if w.useStdNetBind {
|
||||
bind = conn.NewStdNetBind(w.listener.(dialer.WireGuardListener))
|
||||
} else {
|
||||
var (
|
||||
isConnect bool
|
||||
connectAddr netip.AddrPort
|
||||
reserved [3]uint8
|
||||
)
|
||||
peerLen := len(w.peers)
|
||||
if peerLen == 1 {
|
||||
isConnect = true
|
||||
connectAddr = w.peers[0].Endpoint
|
||||
reserved = w.peers[0].Reserved
|
||||
}
|
||||
bind = wireguard.NewClientBind(w.ctx, w.logger, w.listener, isConnect, connectAddr, reserved)
|
||||
}
|
||||
if w.useStdNetBind || len(w.peers) > 1 {
|
||||
for _, peer := range w.peers {
|
||||
if peer.Reserved != [3]uint8{} {
|
||||
bind.SetReservedForEndpoint(peer.Endpoint, peer.Reserved)
|
||||
}
|
||||
}
|
||||
}
|
||||
err = w.tunDevice.Start()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
wgDevice := device.NewDevice(w.ctx, w.tunDevice, bind, &device.Logger{
|
||||
Verbosef: func(format string, args ...interface{}) {
|
||||
w.logger.Debug(fmt.Sprintf(strings.ToLower(format), args...))
|
||||
},
|
||||
Errorf: func(format string, args ...interface{}) {
|
||||
w.logger.Error(fmt.Sprintf(strings.ToLower(format), args...))
|
||||
},
|
||||
}, w.workers)
|
||||
ipcConf := w.ipcConf
|
||||
for _, peer := range w.peers {
|
||||
ipcConf += peer.GenerateIpcLines()
|
||||
}
|
||||
err = wgDevice.IpcSet(ipcConf)
|
||||
if err != nil {
|
||||
return E.Cause(err, "setup wireguard: \n", ipcConf)
|
||||
}
|
||||
w.device = wgDevice
|
||||
w.pauseCallback = w.pauseManager.RegisterCallback(w.onPauseUpdated)
|
||||
return nil
|
||||
}
|
||||
|
||||
func (w *Outbound) Close() error {
|
||||
if w.device != nil {
|
||||
w.device.Close()
|
||||
}
|
||||
if w.pauseCallback != nil {
|
||||
w.pauseManager.UnregisterCallback(w.pauseCallback)
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (w *Outbound) InterfaceUpdated() {
|
||||
w.device.BindUpdate()
|
||||
return
|
||||
}
|
||||
|
||||
func (w *Outbound) onPauseUpdated(event int) {
|
||||
switch event {
|
||||
case pause.EventDevicePaused:
|
||||
w.device.Down()
|
||||
case pause.EventDeviceWake:
|
||||
w.device.Up()
|
||||
}
|
||||
}
|
||||
|
||||
func (w *Outbound) DialContext(ctx context.Context, network string, destination M.Socksaddr) (net.Conn, error) {
|
||||
switch network {
|
||||
case N.NetworkTCP:
|
||||
w.logger.InfoContext(ctx, "outbound connection to ", destination)
|
||||
case N.NetworkUDP:
|
||||
w.logger.InfoContext(ctx, "outbound packet connection to ", destination)
|
||||
}
|
||||
if destination.IsFqdn() {
|
||||
destinationAddresses, err := w.router.LookupDefault(ctx, destination.Fqdn)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return N.DialSerial(ctx, w.tunDevice, network, destination, destinationAddresses)
|
||||
}
|
||||
return w.tunDevice.DialContext(ctx, network, destination)
|
||||
}
|
||||
|
||||
func (w *Outbound) ListenPacket(ctx context.Context, destination M.Socksaddr) (net.PacketConn, error) {
|
||||
w.logger.InfoContext(ctx, "outbound packet connection to ", destination)
|
||||
if destination.IsFqdn() {
|
||||
destinationAddresses, err := w.router.LookupDefault(ctx, destination.Fqdn)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
packetConn, _, err := N.ListenSerial(ctx, w.tunDevice, destination, destinationAddresses)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return packetConn, err
|
||||
}
|
||||
return w.tunDevice.ListenPacket(ctx, destination)
|
||||
}
|
||||
|
||||
// TODO
|
||||
// Deprecated
|
||||
func (w *Outbound) NewConnection(ctx context.Context, conn net.Conn, metadata adapter.InboundContext) error {
|
||||
return outbound.NewDirectConnection(ctx, w.router, w, conn, metadata, dns.DomainStrategyAsIS)
|
||||
}
|
||||
|
||||
// TODO
|
||||
// Deprecated
|
||||
func (w *Outbound) NewPacketConnection(ctx context.Context, conn N.PacketConn, metadata adapter.InboundContext) error {
|
||||
return outbound.NewDirectPacketConnection(ctx, w.router, w, conn, metadata, dns.DomainStrategyAsIS)
|
||||
}
|
Loading…
Add table
Add a link
Reference in a new issue