https: add load balancing group support (#5032)

This commit is contained in:
fatedier
2025-10-28 17:37:18 +08:00
committed by GitHub
parent 469097a549
commit 1cf325bb0c
9 changed files with 311 additions and 21 deletions

View File

@@ -23,7 +23,7 @@ an API that records Zoom, Google Meet, Microsoft Teams, in-person meetings, and
</div>
<p align="center">
<a href="https://app.requestly.io/api-client/?utm_source=github&utm_medium=partnered&utm_campaign=frp" target="_blank">
<a href="https://requestly.com/?utm_source=github&utm_medium=partnered&utm_campaign=frp" target="_blank">
<img width="480px" src="https://github.com/user-attachments/assets/24670320-997d-4d62-9bca-955c59fe883d">
<br>
<b>Requestly - Free & Open-Source alternative to Postman</b>

View File

@@ -25,7 +25,7 @@ an API that records Zoom, Google Meet, Microsoft Teams, in-person meetings, and
</div>
<p align="center">
<a href="https://app.requestly.io/api-client/?utm_source=github&utm_medium=partnered&utm_campaign=frp" target="_blank">
<a href="https://requestly.com/?utm_source=github&utm_medium=partnered&utm_campaign=frp" target="_blank">
<img width="480px" src="https://github.com/user-attachments/assets/24670320-997d-4d62-9bca-955c59fe883d">
<br>
<b>Requestly - Free & Open-Source alternative to Postman</b>

View File

@@ -1,5 +1,3 @@
## Features
* Add NAT traversal configuration options for XTCP proxies and visitors. Support disabling assisted addresses to avoid using slow VPN connections during NAT hole punching.
* Enhanced OIDC client configuration with support for custom TLS certificate verification and proxy settings. Added `trustedCaFile`, `insecureSkipVerify`, and `proxyURL` options for OIDC token endpoint connections.
* Added detailed Prometheus metrics with `proxy_counts_detailed` metric that includes both proxy type and proxy name labels, enabling monitoring of individual proxy connections instead of just aggregate counts.
* HTTPS proxies now support load balancing groups. Multiple HTTPS proxies can be configured with the same `loadBalancer.group` and `loadBalancer.groupKey` to share the same custom domain and distribute traffic across multiple backend services, similar to the existing TCP and HTTP load balancing capabilities.

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

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

@@ -0,0 +1,197 @@
// Copyright 2025 The frp Authors
//
// 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 group
import (
"context"
"net"
"sync"
gerr "github.com/fatedier/golib/errors"
"github.com/fatedier/frp/pkg/util/vhost"
)
type HTTPSGroupController struct {
groups map[string]*HTTPSGroup
httpsMuxer *vhost.HTTPSMuxer
mu sync.Mutex
}
func NewHTTPSGroupController(httpsMuxer *vhost.HTTPSMuxer) *HTTPSGroupController {
return &HTTPSGroupController{
groups: make(map[string]*HTTPSGroup),
httpsMuxer: httpsMuxer,
}
}
func (ctl *HTTPSGroupController) Listen(
ctx context.Context,
group, groupKey string,
routeConfig vhost.RouteConfig,
) (l net.Listener, 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.Listen(ctx, group, groupKey, routeConfig)
}
func (ctl *HTTPSGroupController) RemoveGroup(group string) {
ctl.mu.Lock()
defer ctl.mu.Unlock()
delete(ctl.groups, group)
}
type HTTPSGroup struct {
group string
groupKey string
domain string
acceptCh chan net.Conn
httpsLn *vhost.Listener
lns []*HTTPSGroupListener
ctl *HTTPSGroupController
mu sync.Mutex
}
func NewHTTPSGroup(ctl *HTTPSGroupController) *HTTPSGroup {
return &HTTPSGroup{
lns: make([]*HTTPSGroupListener, 0),
ctl: ctl,
acceptCh: make(chan net.Conn),
}
}
func (g *HTTPSGroup) Listen(
ctx context.Context,
group, groupKey string,
routeConfig vhost.RouteConfig,
) (ln *HTTPSGroupListener, err error) {
g.mu.Lock()
defer g.mu.Unlock()
if len(g.lns) == 0 {
// the first listener, listen on the real address
httpsLn, errRet := g.ctl.httpsMuxer.Listen(ctx, &routeConfig)
if errRet != nil {
return nil, errRet
}
ln = newHTTPSGroupListener(group, g, httpsLn.Addr())
g.group = group
g.groupKey = groupKey
g.domain = routeConfig.Domain
g.httpsLn = httpsLn
g.lns = append(g.lns, ln)
go g.worker()
} else {
// route config in the same group must be equal
if g.group != group || g.domain != routeConfig.Domain {
return nil, ErrGroupParamsInvalid
}
if g.groupKey != groupKey {
return nil, ErrGroupAuthFailed
}
ln = newHTTPSGroupListener(group, g, g.lns[0].Addr())
g.lns = append(g.lns, ln)
}
return
}
func (g *HTTPSGroup) worker() {
for {
c, err := g.httpsLn.Accept()
if err != nil {
return
}
err = gerr.PanicToError(func() {
g.acceptCh <- c
})
if err != nil {
return
}
}
}
func (g *HTTPSGroup) Accept() <-chan net.Conn {
return g.acceptCh
}
func (g *HTTPSGroup) CloseListener(ln *HTTPSGroupListener) {
g.mu.Lock()
defer g.mu.Unlock()
for i, tmpLn := range g.lns {
if tmpLn == ln {
g.lns = append(g.lns[:i], g.lns[i+1:]...)
break
}
}
if len(g.lns) == 0 {
close(g.acceptCh)
if g.httpsLn != nil {
g.httpsLn.Close()
}
g.ctl.RemoveGroup(g.group)
}
}
type HTTPSGroupListener struct {
groupName string
group *HTTPSGroup
addr net.Addr
closeCh chan struct{}
}
func newHTTPSGroupListener(name string, group *HTTPSGroup, addr net.Addr) *HTTPSGroupListener {
return &HTTPSGroupListener{
groupName: name,
group: group,
addr: addr,
closeCh: make(chan struct{}),
}
}
func (ln *HTTPSGroupListener) Accept() (c net.Conn, err error) {
var ok bool
select {
case <-ln.closeCh:
return nil, ErrListenerClosed
case c, ok = <-ln.group.Accept():
if !ok {
return nil, ErrListenerClosed
}
return c, nil
}
}
func (ln *HTTPSGroupListener) Addr() net.Addr {
return ln.addr
}
func (ln *HTTPSGroupListener) Close() (err error) {
close(ln.closeCh)
// remove self from HTTPSGroup
ln.group.CloseListener(ln)
return
}

View File

@@ -15,6 +15,7 @@
package proxy
import (
"net"
"reflect"
"strings"
@@ -58,27 +59,24 @@ func (pxy *HTTPSProxy) Run() (remoteAddr string, err error) {
continue
}
routeConfig.Domain = domain
l, errRet := pxy.rc.VhostHTTPSMuxer.Listen(pxy.ctx, routeConfig)
if errRet != nil {
err = errRet
return
l, err := pxy.listenForDomain(routeConfig, domain)
if err != nil {
return "", err
}
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))
addrs = append(addrs, util.CanonicalAddr(domain, pxy.serverCfg.VhostHTTPSPort))
xl.Infof("https proxy listen for host [%s] group [%s]", 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
domain := pxy.cfg.SubDomain + "." + pxy.serverCfg.SubDomainHost
l, err := pxy.listenForDomain(routeConfig, domain)
if err != nil {
return "", err
}
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))
addrs = append(addrs, util.CanonicalAddr(domain, pxy.serverCfg.VhostHTTPSPort))
xl.Infof("https proxy listen for host [%s] group [%s]", domain, pxy.cfg.LoadBalancer.Group)
}
pxy.startCommonTCPListenersHandler()
@@ -89,3 +87,18 @@ func (pxy *HTTPSProxy) Run() (remoteAddr string, err error) {
func (pxy *HTTPSProxy) Close() {
pxy.BaseProxy.Close()
}
func (pxy *HTTPSProxy) listenForDomain(routeConfig *vhost.RouteConfig, domain string) (net.Listener, error) {
tmpRouteConfig := *routeConfig
tmpRouteConfig.Domain = domain
if pxy.cfg.LoadBalancer.Group != "" {
return pxy.rc.HTTPSGroupCtl.Listen(
pxy.ctx,
pxy.cfg.LoadBalancer.Group,
pxy.cfg.LoadBalancer.GroupKey,
tmpRouteConfig,
)
}
return pxy.rc.VhostHTTPSMuxer.Listen(pxy.ctx, &tmpRouteConfig)
}

View File

@@ -322,6 +322,9 @@ func NewService(cfg *v1.ServerConfig) (*Service, error) {
if err != nil {
return nil, fmt.Errorf("create vhost httpsMuxer error, %v", err)
}
// Init HTTPS group controller after HTTPSMuxer is created
svr.rc.HTTPSGroupCtl = group.NewHTTPSGroupController(svr.rc.VhostHTTPSMuxer)
}
// frp tls listener

View File

@@ -75,8 +75,8 @@ func (f *Framework) RunFrps(args ...string) (*process.Process, string, error) {
if err != nil {
return p, p.StdOutput(), err
}
// sleep for a while to get std output
time.Sleep(2 * time.Second)
// Give frps extra time to finish binding ports before proceeding.
time.Sleep(4 * time.Second)
return p, p.StdOutput(), nil
}

View File

@@ -1,6 +1,7 @@
package features
import (
"crypto/tls"
"fmt"
"strconv"
"sync"
@@ -8,6 +9,7 @@ import (
"github.com/onsi/ginkgo/v2"
"github.com/fatedier/frp/pkg/transport"
"github.com/fatedier/frp/test/e2e/framework"
"github.com/fatedier/frp/test/e2e/framework/consts"
"github.com/fatedier/frp/test/e2e/mock/server/httpserver"
@@ -112,6 +114,80 @@ var _ = ginkgo.Describe("[Feature: Group]", func() {
framework.ExpectTrue(fooCount > 1 && barCount > 1, "fooCount: %d, barCount: %d", fooCount, barCount)
})
ginkgo.It("HTTPS", func() {
vhostHTTPSPort := f.AllocPort()
serverConf := consts.DefaultServerConfig + fmt.Sprintf(`
vhostHTTPSPort = %d
`, vhostHTTPSPort)
clientConf := consts.DefaultClientConfig
tlsConfig, err := transport.NewServerTLSConfig("", "", "")
framework.ExpectNoError(err)
fooPort := f.AllocPort()
fooServer := httpserver.New(
httpserver.WithBindPort(fooPort),
httpserver.WithHandler(framework.SpecifiedHTTPBodyHandler([]byte("foo"))),
httpserver.WithTLSConfig(tlsConfig),
)
f.RunServer("", fooServer)
barPort := f.AllocPort()
barServer := httpserver.New(
httpserver.WithBindPort(barPort),
httpserver.WithHandler(framework.SpecifiedHTTPBodyHandler([]byte("bar"))),
httpserver.WithTLSConfig(tlsConfig),
)
f.RunServer("", barServer)
clientConf += fmt.Sprintf(`
[[proxies]]
name = "foo"
type = "https"
localPort = %d
customDomains = ["example.com"]
loadBalancer.group = "test"
loadBalancer.groupKey = "123"
[[proxies]]
name = "bar"
type = "https"
localPort = %d
customDomains = ["example.com"]
loadBalancer.group = "test"
loadBalancer.groupKey = "123"
`, fooPort, barPort)
f.RunProcesses([]string{serverConf}, []string{clientConf})
fooCount := 0
barCount := 0
for i := 0; i < 10; i++ {
framework.NewRequestExpect(f).
Explain("times " + strconv.Itoa(i)).
Port(vhostHTTPSPort).
RequestModify(func(r *request.Request) {
r.HTTPS().HTTPHost("example.com").TLSConfig(&tls.Config{
ServerName: "example.com",
InsecureSkipVerify: true,
})
}).
Ensure(func(resp *request.Response) bool {
switch string(resp.Content) {
case "foo":
fooCount++
case "bar":
barCount++
default:
return false
}
return true
})
}
framework.ExpectTrue(fooCount > 1 && barCount > 1, "fooCount: %d, barCount: %d", fooCount, barCount)
})
})
ginkgo.Describe("Health Check", func() {