feat: add HTTPS load balancing support

- Add HTTPS group controller for load balancing across multiple HTTPS backends
- Implement HTTPS reverse proxy with SNI-based routing
- Enhance HTTPS proxy with group registration and load balancing
- Add configuration examples for HTTPS load balancing
- Update documentation with HTTPS load balancing examples
- Support health checks for HTTPS load balancing groups

This feature enables high availability and horizontal scaling for HTTPS services
by distributing traffic across multiple backend HTTPS endpoints using round-robin
load balancing, similar to existing HTTP load balancing functionality.

Closes #[ISSUE_NUMBER]
pull/4939/head
SS 2025-08-15 11:16:47 +04:00
parent 024c334d9d
commit 5b4aea6454
8 changed files with 674 additions and 19 deletions

View File

@ -932,7 +932,7 @@ This feature is suitable for a large number of short connections.
Load balancing is supported by `group`.
This feature is only available for types `tcp`, `http`, `tcpmux` now.
This feature is available for types `tcp`, `http`, `https`, `tcpmux` now.
```toml
# frpc.toml
@ -954,6 +954,28 @@ loadBalancer.group = "web"
loadBalancer.groupKey = "123"
```
For HTTPS load balancing:
```toml
# frpc.toml
[[proxies]]
name = "web1"
type = "https"
localPort = 443
customDomains = ["example.com"]
loadBalancer.group = "web"
loadBalancer.groupKey = "123"
[[proxies]]
name = "web2"
type = "https"
localPort = 443
customDomains = ["example.com"]
loadBalancer.group = "web"
loadBalancer.groupKey = "123"
```
`loadBalancer.groupKey` is used for authentication.
Connections to port 80 will be dispatched to proxies in the same group randomly.
@ -962,6 +984,8 @@ For type `tcp`, `remotePort` in the same group should be the same.
For type `http`, `customDomains`, `subdomain`, `locations` should be the same.
For type `https`, `customDomains`, `subdomain` should be the same.
### Service Health Check
Health check feature can help you achieve high availability with load balancing.

View File

@ -247,6 +247,35 @@ customDomains = ["web02.yourdomain.com"]
# v1 or v2 or empty
transport.proxyProtocolVersion = "v2"
# HTTPS load balancing example
[[proxies]]
name = "web_lb_1"
type = "https"
localIP = "127.0.0.1"
localPort = 443
customDomains = ["app.yourdomain.com"]
loadBalancer.group = "web"
loadBalancer.groupKey = "123"
# Enable health check for load balancing
healthCheck.type = "tcp"
healthCheck.timeoutSeconds = 3
healthCheck.maxFailed = 3
healthCheck.intervalSeconds = 10
[[proxies]]
name = "web_lb_2"
type = "https"
localIP = "127.0.0.1"
localPort = 8443
customDomains = ["app.yourdomain.com"]
loadBalancer.group = "web"
loadBalancer.groupKey = "123"
# Enable health check for load balancing
healthCheck.type = "tcp"
healthCheck.timeoutSeconds = 3
healthCheck.maxFailed = 3
healthCheck.intervalSeconds = 10
[[proxies]]
name = "tcpmuxhttpconnect"
type = "tcpmux"

View File

@ -18,22 +18,174 @@ import (
"crypto/tls"
"io"
"net"
"strings"
"time"
"github.com/fatedier/golib/errors"
libio "github.com/fatedier/golib/io"
libnet "github.com/fatedier/golib/net"
httppkg "github.com/fatedier/frp/pkg/util/http"
"github.com/fatedier/frp/pkg/util/log"
"github.com/fatedier/frp/pkg/util/xlog"
)
type HTTPSMuxer struct {
*Muxer
httpsReverseProxy *HTTPSReverseProxy
}
func NewHTTPSMuxer(listener net.Listener, timeout time.Duration) (*HTTPSMuxer, error) {
mux, err := NewMuxer(listener, GetHTTPSHostname, timeout)
mux.SetFailHookFunc(vhostFailed)
if err != nil {
return nil, err
// Create muxer without auto-starting run method
mux := &Muxer{
listener: listener,
timeout: timeout,
vhostFunc: GetHTTPSHostname,
registryRouter: NewRouters(),
}
mux.SetFailHookFunc(vhostFailed)
httpsMux := &HTTPSMuxer{Muxer: mux}
// Start our custom handler
go httpsMux.run()
return httpsMux, nil
}
func (h *HTTPSMuxer) SetHTTPSReverseProxy(rp *HTTPSReverseProxy) {
h.httpsReverseProxy = rp
}
// Override the base muxer's run method to handle group routing
func (h *HTTPSMuxer) run() {
for {
conn, err := h.listener.Accept()
if err != nil {
return
}
go h.handleHTTPS(conn)
}
}
func (h *HTTPSMuxer) handleHTTPS(c net.Conn) {
if err := c.SetDeadline(time.Now().Add(h.timeout)); err != nil {
_ = c.Close()
return
}
sConn, reqInfoMap, err := h.vhostFunc(c)
if err != nil {
log.Debugf("get hostname from https request error: %v", err)
_ = c.Close()
return
}
hostname := strings.ToLower(reqInfoMap["Host"])
// Validate and canonicalize hostname for security
canonicalHostname, err := httppkg.CanonicalHost(hostname)
if err != nil {
log.Debugf("invalid hostname [%s]: %v", hostname, err)
h.failHook(sConn)
return
}
// First check if there's a group route for this domain
if h.httpsReverseProxy != nil {
if routeConfig := h.httpsReverseProxy.GetRouteConfig(canonicalHostname); routeConfig != nil {
log.Debugf("routing https request for host [%s] to group", hostname)
// SECURITY: Apply authentication check before group routing
if routeConfig.Username != "" && routeConfig.Password != "" {
if h.checkAuth != nil {
ok, err := h.checkAuth(c, routeConfig.Username, routeConfig.Password, reqInfoMap)
if !ok || err != nil {
log.Debugf("auth failed for group route user: %s", routeConfig.Username)
h.failHook(sConn)
return
}
}
}
// SECURITY: Apply success hook for group routing
if h.successHook != nil {
if err := h.successHook(c, reqInfoMap); err != nil {
log.Infof("success func failure on group vhost connection: %v", err)
_ = c.Close()
return
}
}
if err = sConn.SetDeadline(time.Time{}); err != nil {
_ = c.Close()
return
}
// Create connection to backend through group routing
remoteConn, err := h.httpsReverseProxy.CreateConnection(canonicalHostname)
if err != nil {
log.Debugf("failed to create connection through group: %v", err)
h.failHook(sConn)
return
}
// Start proxying data between client and remote
go func() {
defer func() {
if err := recover(); err != nil {
log.Warnf("panic in HTTPS proxy goroutine: %v", err)
}
}()
defer sConn.Close()
defer remoteConn.Close()
libio.Join(sConn, remoteConn)
}()
return
}
}
// Fall back to direct listener routing (existing behavior)
path := strings.ToLower(reqInfoMap["Path"])
httpUser := reqInfoMap["HTTPUser"]
l, ok := h.getListener(canonicalHostname, path, httpUser)
if !ok {
log.Debugf("https request for host [%s] path [%s] httpUser [%s] not found", hostname, path, httpUser)
h.failHook(sConn)
return
}
xl := xlog.FromContextSafe(l.ctx)
if h.successHook != nil {
if err := h.successHook(c, reqInfoMap); err != nil {
xl.Infof("success func failure on vhost connection: %v", err)
_ = c.Close()
return
}
}
// if checkAuth func is exist and username/password is set
// then verify user access
if l.mux.checkAuth != nil && l.username != "" {
ok, err := l.mux.checkAuth(c, l.username, l.password, reqInfoMap)
if !ok || err != nil {
xl.Debugf("auth failed for user: %s", l.username)
_ = c.Close()
return
}
}
if err = sConn.SetDeadline(time.Time{}); err != nil {
_ = c.Close()
return
}
c = sConn
xl.Debugf("new https request host [%s] path [%s] httpUser [%s]", hostname, path, httpUser)
err = errors.PanicToError(func() {
l.accept <- c
})
if err != nil {
xl.Warnf("listener is already closed, ignore this request")
}
return &HTTPSMuxer{mux}, err
}
func GetHTTPSHostname(c net.Conn) (_ net.Conn, _ map[string]string, err error) {

View File

@ -0,0 +1,143 @@
// Copyright 2024 fatedier, fatedier@gmail.com
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package vhost
import (
"fmt"
"net"
"strings"
libio "github.com/fatedier/golib/io"
httppkg "github.com/fatedier/frp/pkg/util/http"
"github.com/fatedier/frp/pkg/util/log"
)
type HTTPSReverseProxy struct {
vhostRouter *Routers
}
func NewHTTPSReverseProxy(vhostRouter *Routers) *HTTPSReverseProxy {
return &HTTPSReverseProxy{
vhostRouter: vhostRouter,
}
}
// Register register the route config to reverse proxy
// reverse proxy will use CreateConnFn from routeCfg to create a connection to the remote service
func (rp *HTTPSReverseProxy) Register(routeCfg RouteConfig) error {
err := rp.vhostRouter.Add(routeCfg.Domain, "", "", &routeCfg)
if err != nil {
return err
}
return nil
}
// UnRegister unregister route config by domain
func (rp *HTTPSReverseProxy) UnRegister(routeCfg RouteConfig) {
rp.vhostRouter.Del(routeCfg.Domain, "", "")
}
func (rp *HTTPSReverseProxy) GetRouteConfig(domain string) *RouteConfig {
// Validate and canonicalize hostname for security
canonicalDomain, err := httppkg.CanonicalHost(domain)
if err != nil {
log.Debugf("invalid hostname [%s]: %v", domain, err)
return nil
}
vr, ok := rp.getVhost(canonicalDomain)
if ok {
log.Debugf("get new https request host [%s]", canonicalDomain)
return vr.payload.(*RouteConfig)
}
return nil
}
// CreateConnection create a new connection by route config
func (rp *HTTPSReverseProxy) CreateConnection(domain string) (net.Conn, error) {
// Validate and canonicalize hostname for security
canonicalDomain, err := httppkg.CanonicalHost(domain)
if err != nil {
return nil, fmt.Errorf("invalid hostname: %v", err)
}
vr, ok := rp.getVhost(canonicalDomain)
if ok {
fn := vr.payload.(*RouteConfig).CreateConnFn
if fn != nil {
return fn("")
}
}
return nil, fmt.Errorf("%v: %s", ErrNoRouteFound, canonicalDomain)
}
// ProxyConn proxy connection for HTTPS
func (rp *HTTPSReverseProxy) ProxyConn(clientConn net.Conn, domain string) error {
remoteConn, err := rp.CreateConnection(domain)
if err != nil {
return err
}
// Start proxying data between client and remote
go func() {
defer clientConn.Close()
defer remoteConn.Close()
libio.Join(clientConn, remoteConn)
}()
return nil
}
// getVhost tries to get vhost router by domain.
func (rp *HTTPSReverseProxy) getVhost(domain string) (*Router, bool) {
findRouter := func(inDomain string) (*Router, bool) {
vr, ok := rp.vhostRouter.Get(inDomain, "", "")
if ok {
return vr, ok
}
return nil, false
}
domain = strings.ToLower(domain)
// First we check the full hostname
// if not exist, then check the wildcard_domain such as *.example.com
vr, ok := findRouter(domain)
if ok {
return vr, ok
}
// e.g. domain = test.example.com, try to match wildcard domains.
// *.example.com
// *.com
domainSplit := strings.Split(domain, ".")
for len(domainSplit) >= 3 {
domainSplit[0] = "*"
wildcardDomain := strings.Join(domainSplit, ".")
vr, ok = findRouter(wildcardDomain)
if ok {
return vr, true
}
domainSplit = domainSplit[1:]
}
// Finally, try to check if there is one proxy that domain is "*" means match all domains.
vr, ok = findRouter("*")
if ok {
return vr, true
}
return nil, false
}

View File

@ -35,6 +35,9 @@ type ResourceController struct {
// HTTP Group Controller
HTTPGroupCtl *group.HTTPGroupController
// HTTPS Group Controller
HTTPSGroupCtl *group.HTTPSGroupController
// TCP Mux Group Controller
TCPMuxGroupCtl *group.TCPMuxGroupCtl
@ -47,6 +50,9 @@ type ResourceController struct {
// For HTTP proxies, forwarding HTTP requests
HTTPReverseProxy *vhost.HTTPReverseProxy
// For HTTPS proxies, forwarding HTTPS requests (for group load balancing)
HTTPSReverseProxy *vhost.HTTPSReverseProxy
// For HTTPS proxies, route requests to different clients by hostname and other information
VhostHTTPSMuxer *vhost.HTTPSMuxer

189
server/group/https.go Normal file
View File

@ -0,0 +1,189 @@
package group
import (
"fmt"
"net"
"sync"
"sync/atomic"
"github.com/fatedier/frp/pkg/util/vhost"
)
type HTTPSGroupController struct {
// groups indexed by group name
groups map[string]*HTTPSGroup
// register createConn for each group to vhostRouter.
// createConn will get a connection from one proxy of the group
vhostRouter *vhost.Routers
mu sync.Mutex
}
func NewHTTPSGroupController(vhostRouter *vhost.Routers) *HTTPSGroupController {
return &HTTPSGroupController{
groups: make(map[string]*HTTPSGroup),
vhostRouter: vhostRouter,
}
}
func (ctl *HTTPSGroupController) Register(
proxyName, group, groupKey string,
routeConfig vhost.RouteConfig,
) (err error) {
indexKey := group
ctl.mu.Lock()
g, ok := ctl.groups[indexKey]
if !ok {
g = NewHTTPSGroup(ctl)
ctl.groups[indexKey] = g
}
ctl.mu.Unlock()
return g.Register(proxyName, group, groupKey, routeConfig)
}
func (ctl *HTTPSGroupController) UnRegister(proxyName, group string, _ vhost.RouteConfig) {
indexKey := group
ctl.mu.Lock()
defer ctl.mu.Unlock()
g, ok := ctl.groups[indexKey]
if !ok {
return
}
isEmpty := g.UnRegister(proxyName)
if isEmpty {
delete(ctl.groups, indexKey)
}
}
type HTTPSGroup struct {
group string
groupKey string
domain string
// CreateConnFuncs indexed by proxy name
createFuncs map[string]vhost.CreateConnFunc
pxyNames []string
index uint64
ctl *HTTPSGroupController
mu sync.RWMutex
}
func NewHTTPSGroup(ctl *HTTPSGroupController) *HTTPSGroup {
return &HTTPSGroup{
createFuncs: make(map[string]vhost.CreateConnFunc),
pxyNames: make([]string, 0),
ctl: ctl,
}
}
func (g *HTTPSGroup) Register(
proxyName, group, groupKey string,
routeConfig vhost.RouteConfig,
) (err error) {
g.mu.Lock()
defer g.mu.Unlock()
if len(g.createFuncs) == 0 {
// the first proxy in this group
tmp := routeConfig // copy object
tmp.CreateConnFn = g.createConn
tmp.ChooseEndpointFn = g.chooseEndpoint
tmp.CreateConnByEndpointFn = g.createConnByEndpoint
err = g.ctl.vhostRouter.Add(routeConfig.Domain, "", "", &tmp)
if err != nil {
return
}
g.group = group
g.groupKey = groupKey
g.domain = routeConfig.Domain
} else {
if g.group != group || g.domain != routeConfig.Domain {
err = ErrGroupParamsInvalid
return
}
if g.groupKey != groupKey {
err = ErrGroupAuthFailed
return
}
}
if _, ok := g.createFuncs[proxyName]; ok {
err = ErrProxyRepeated
return
}
g.createFuncs[proxyName] = routeConfig.CreateConnFn
g.pxyNames = append(g.pxyNames, proxyName)
return nil
}
func (g *HTTPSGroup) UnRegister(proxyName string) (isEmpty bool) {
g.mu.Lock()
defer g.mu.Unlock()
delete(g.createFuncs, proxyName)
for i, name := range g.pxyNames {
if name == proxyName {
g.pxyNames = append(g.pxyNames[:i], g.pxyNames[i+1:]...)
break
}
}
if len(g.createFuncs) == 0 {
isEmpty = true
g.ctl.vhostRouter.Del(g.domain, "", "")
}
return
}
func (g *HTTPSGroup) createConn(remoteAddr string) (net.Conn, error) {
var f vhost.CreateConnFunc
newIndex := atomic.AddUint64(&g.index, 1)
g.mu.RLock()
group := g.group
domain := g.domain
if len(g.pxyNames) > 0 {
name := g.pxyNames[int(newIndex)%len(g.pxyNames)]
f = g.createFuncs[name]
}
g.mu.RUnlock()
if f == nil {
return nil, fmt.Errorf("no CreateConnFunc for https group [%s], domain [%s]",
group, domain)
}
return f(remoteAddr)
}
func (g *HTTPSGroup) chooseEndpoint() (string, error) {
newIndex := atomic.AddUint64(&g.index, 1)
name := ""
g.mu.RLock()
group := g.group
domain := g.domain
if len(g.pxyNames) > 0 {
name = g.pxyNames[int(newIndex)%len(g.pxyNames)]
}
g.mu.RUnlock()
if name == "" {
return "", fmt.Errorf("no healthy endpoint for https group [%s], domain [%s]",
group, domain)
}
return name, nil
}
func (g *HTTPSGroup) createConnByEndpoint(endpoint, remoteAddr string) (net.Conn, error) {
var f vhost.CreateConnFunc
g.mu.RLock()
f = g.createFuncs[endpoint]
g.mu.RUnlock()
if f == nil {
return nil, fmt.Errorf("no CreateConnFunc for endpoint [%s] in group [%s]", endpoint, g.group)
}
return f(remoteAddr)
}

View File

@ -15,12 +15,19 @@
package proxy
import (
"io"
"net"
"reflect"
"strings"
libio "github.com/fatedier/golib/io"
v1 "github.com/fatedier/frp/pkg/config/v1"
"github.com/fatedier/frp/pkg/util/limit"
netpkg "github.com/fatedier/frp/pkg/util/net"
"github.com/fatedier/frp/pkg/util/util"
"github.com/fatedier/frp/pkg/util/vhost"
"github.com/fatedier/frp/server/metrics"
)
func init() {
@ -30,6 +37,8 @@ func init() {
type HTTPSProxy struct {
*BaseProxy
cfg *v1.HTTPSProxyConfig
closeFuncs []func()
}
func NewHTTPSProxy(baseProxy *BaseProxy) Proxy {
@ -45,13 +54,16 @@ func NewHTTPSProxy(baseProxy *BaseProxy) Proxy {
func (pxy *HTTPSProxy) Run() (remoteAddr string, err error) {
xl := pxy.xl
routeConfig := &vhost.RouteConfig{}
routeConfig := vhost.RouteConfig{
CreateConnFn: pxy.GetRealConn,
}
defer func() {
if err != nil {
pxy.Close()
}
}()
addrs := make([]string, 0)
for _, domain := range pxy.cfg.CustomDomains {
if domain == "" {
@ -59,26 +71,63 @@ func (pxy *HTTPSProxy) Run() (remoteAddr string, err error) {
}
routeConfig.Domain = domain
l, errRet := pxy.rc.VhostHTTPSMuxer.Listen(pxy.ctx, routeConfig)
if errRet != nil {
err = errRet
return
tmpRouteConfig := routeConfig
// handle group
if pxy.cfg.LoadBalancer.Group != "" {
err = pxy.rc.HTTPSGroupCtl.Register(pxy.name, pxy.cfg.LoadBalancer.Group, pxy.cfg.LoadBalancer.GroupKey, routeConfig)
if err != nil {
return
}
pxy.closeFuncs = append(pxy.closeFuncs, func() {
pxy.rc.HTTPSGroupCtl.UnRegister(pxy.name, pxy.cfg.LoadBalancer.Group, tmpRouteConfig)
})
} else {
// no group - use direct muxer
l, errRet := pxy.rc.VhostHTTPSMuxer.Listen(pxy.ctx, &routeConfig)
if errRet != nil {
err = errRet
return
}
xl.Infof("https proxy listen for host [%s]", routeConfig.Domain)
pxy.listeners = append(pxy.listeners, l)
}
xl.Infof("https proxy listen for host [%s]", routeConfig.Domain)
pxy.listeners = append(pxy.listeners, l)
addrs = append(addrs, util.CanonicalAddr(routeConfig.Domain, pxy.serverCfg.VhostHTTPSPort))
xl.Infof("https proxy listen for host [%s] group [%s]",
routeConfig.Domain, pxy.cfg.LoadBalancer.Group)
}
if pxy.cfg.SubDomain != "" {
routeConfig.Domain = pxy.cfg.SubDomain + "." + pxy.serverCfg.SubDomainHost
l, errRet := pxy.rc.VhostHTTPSMuxer.Listen(pxy.ctx, routeConfig)
if errRet != nil {
err = errRet
return
tmpRouteConfig := routeConfig
// handle group
if pxy.cfg.LoadBalancer.Group != "" {
err = pxy.rc.HTTPSGroupCtl.Register(pxy.name, pxy.cfg.LoadBalancer.Group, pxy.cfg.LoadBalancer.GroupKey, routeConfig)
if err != nil {
return
}
pxy.closeFuncs = append(pxy.closeFuncs, func() {
pxy.rc.HTTPSGroupCtl.UnRegister(pxy.name, pxy.cfg.LoadBalancer.Group, tmpRouteConfig)
})
} else {
// no group - use direct muxer
l, errRet := pxy.rc.VhostHTTPSMuxer.Listen(pxy.ctx, &routeConfig)
if errRet != nil {
err = errRet
return
}
xl.Infof("https proxy listen for host [%s]", routeConfig.Domain)
pxy.listeners = append(pxy.listeners, l)
}
xl.Infof("https proxy listen for host [%s]", routeConfig.Domain)
pxy.listeners = append(pxy.listeners, l)
addrs = append(addrs, util.CanonicalAddr(routeConfig.Domain, pxy.serverCfg.VhostHTTPSPort))
xl.Infof("https proxy listen for host [%s] group [%s]",
routeConfig.Domain, pxy.cfg.LoadBalancer.Group)
}
pxy.startCommonTCPListenersHandler()
@ -86,6 +135,55 @@ func (pxy *HTTPSProxy) Run() (remoteAddr string, err error) {
return
}
func (pxy *HTTPSProxy) GetRealConn(remoteAddr string) (workConn net.Conn, err error) {
xl := pxy.xl
rAddr, errRet := net.ResolveTCPAddr("tcp", remoteAddr)
if errRet != nil {
xl.Warnf("resolve TCP addr [%s] error: %v", remoteAddr, errRet)
// we do not return error here since remoteAddr is not necessary for proxies without proxy protocol enabled
}
tmpConn, errRet := pxy.GetWorkConnFromPool(rAddr, nil)
if errRet != nil {
err = errRet
return
}
var rwc io.ReadWriteCloser = tmpConn
if pxy.cfg.Transport.UseEncryption {
rwc, err = libio.WithEncryption(rwc, []byte(pxy.serverCfg.Auth.Token))
if err != nil {
xl.Errorf("create encryption stream error: %v", err)
return
}
}
if pxy.cfg.Transport.UseCompression {
rwc = libio.WithCompression(rwc)
}
if pxy.GetLimiter() != nil {
rwc = libio.WrapReadWriteCloser(limit.NewReader(rwc, pxy.GetLimiter()), limit.NewWriter(rwc, pxy.GetLimiter()), func() error {
return rwc.Close()
})
}
workConn = netpkg.WrapReadWriteCloserToConn(rwc, tmpConn)
workConn = netpkg.WrapStatsConn(workConn, pxy.updateStatsAfterClosedConn)
metrics.Server.OpenConnection(pxy.GetName(), pxy.GetConfigurer().GetBaseConfig().Type)
return
}
func (pxy *HTTPSProxy) updateStatsAfterClosedConn(totalRead, totalWrite int64) {
name := pxy.GetName()
proxyType := pxy.GetConfigurer().GetBaseConfig().Type
metrics.Server.CloseConnection(name, proxyType)
metrics.Server.AddTrafficIn(name, proxyType, totalWrite)
metrics.Server.AddTrafficOut(name, proxyType, totalRead)
}
func (pxy *HTTPSProxy) Close() {
pxy.BaseProxy.Close()
for _, closeFn := range pxy.closeFuncs {
closeFn()
}
}

View File

@ -106,6 +106,9 @@ type Service struct {
// HTTP vhost router
httpVhostRouter *vhost.Routers
// HTTPS vhost router
httpsVhostRouter *vhost.Routers
// All resource managers and controllers
rc *controller.ResourceController
@ -161,6 +164,7 @@ func NewService(cfg *v1.ServerConfig) (*Service, error) {
},
sshTunnelListener: netpkg.NewInternalListener(),
httpVhostRouter: vhost.NewRouters(),
httpsVhostRouter: vhost.NewRouters(),
authVerifier: auth.NewAuthVerifier(cfg.Auth),
webServer: webServer,
tlsConfig: tlsConfig,
@ -200,6 +204,9 @@ func NewService(cfg *v1.ServerConfig) (*Service, error) {
// Init HTTP group controller
svr.rc.HTTPGroupCtl = group.NewHTTPGroupController(svr.httpVhostRouter)
// Init HTTPS group controller
svr.rc.HTTPSGroupCtl = group.NewHTTPSGroupController(svr.httpsVhostRouter)
// Init TCP mux group controller
svr.rc.TCPMuxGroupCtl = group.NewTCPMuxGroupCtl(svr.rc.TCPMuxHTTPConnectMuxer)
@ -323,6 +330,13 @@ func NewService(cfg *v1.ServerConfig) (*Service, error) {
if err != nil {
return nil, fmt.Errorf("create vhost httpsMuxer error, %v", err)
}
// Init HTTPS reverse proxy for group routing
httpsReverseProxy := vhost.NewHTTPSReverseProxy(svr.httpsVhostRouter)
svr.rc.HTTPSReverseProxy = httpsReverseProxy
// Set the reverse proxy on the muxer for group routing
svr.rc.VhostHTTPSMuxer.SetHTTPSReverseProxy(httpsReverseProxy)
}
// frp tls listener