Add internal provider
Co-authored-by: Julien Salleyron <julien.salleyron@gmail.com>
This commit is contained in:
parent
2ee2e29262
commit
424e2a9439
71 changed files with 2523 additions and 1469 deletions
|
@ -67,7 +67,7 @@ func mergeConfiguration(configurations dynamic.Configurations) dynamic.Configura
|
|||
}
|
||||
|
||||
if len(defaultTLSOptionProviders) == 0 {
|
||||
conf.TLS.Options["default"] = tls.Options{}
|
||||
conf.TLS.Options["default"] = tls.DefaultTLSOptions
|
||||
} else if len(defaultTLSOptionProviders) > 1 {
|
||||
log.WithoutContext().Errorf("Default TLS Options defined multiple times in %v", defaultTLSOptionProviders)
|
||||
// We do not set an empty tls.TLS{} as above so that we actually get a "cascading failure" later on,
|
||||
|
|
238
pkg/server/configurationwatcher.go
Normal file
238
pkg/server/configurationwatcher.go
Normal file
|
@ -0,0 +1,238 @@
|
|||
package server
|
||||
|
||||
import (
|
||||
"encoding/json"
|
||||
"reflect"
|
||||
"time"
|
||||
|
||||
"github.com/containous/traefik/v2/pkg/config/dynamic"
|
||||
"github.com/containous/traefik/v2/pkg/log"
|
||||
"github.com/containous/traefik/v2/pkg/provider"
|
||||
"github.com/containous/traefik/v2/pkg/safe"
|
||||
"github.com/eapache/channels"
|
||||
"github.com/sirupsen/logrus"
|
||||
)
|
||||
|
||||
// ConfigurationWatcher watches configuration changes.
|
||||
type ConfigurationWatcher struct {
|
||||
provider provider.Provider
|
||||
|
||||
providersThrottleDuration time.Duration
|
||||
|
||||
currentConfigurations safe.Safe
|
||||
|
||||
configurationChan chan dynamic.Message
|
||||
configurationValidatedChan chan dynamic.Message
|
||||
providerConfigUpdateMap map[string]chan dynamic.Message
|
||||
|
||||
configurationListeners []func(dynamic.Configuration)
|
||||
|
||||
routinesPool *safe.Pool
|
||||
}
|
||||
|
||||
// NewConfigurationWatcher creates a new ConfigurationWatcher.
|
||||
func NewConfigurationWatcher(routinesPool *safe.Pool, pvd provider.Provider, providersThrottleDuration time.Duration) *ConfigurationWatcher {
|
||||
watcher := &ConfigurationWatcher{
|
||||
provider: pvd,
|
||||
configurationChan: make(chan dynamic.Message, 100),
|
||||
configurationValidatedChan: make(chan dynamic.Message, 100),
|
||||
providerConfigUpdateMap: make(map[string]chan dynamic.Message),
|
||||
providersThrottleDuration: providersThrottleDuration,
|
||||
routinesPool: routinesPool,
|
||||
}
|
||||
|
||||
currentConfigurations := make(dynamic.Configurations)
|
||||
watcher.currentConfigurations.Set(currentConfigurations)
|
||||
|
||||
return watcher
|
||||
}
|
||||
|
||||
// Start the configuration watcher.
|
||||
func (c *ConfigurationWatcher) Start() {
|
||||
c.routinesPool.Go(func(stop chan bool) { c.listenProviders(stop) })
|
||||
c.routinesPool.Go(func(stop chan bool) { c.listenConfigurations(stop) })
|
||||
c.startProvider()
|
||||
}
|
||||
|
||||
// Stop the configuration watcher.
|
||||
func (c *ConfigurationWatcher) Stop() {
|
||||
close(c.configurationChan)
|
||||
close(c.configurationValidatedChan)
|
||||
}
|
||||
|
||||
// AddListener adds a new listener function used when new configuration is provided
|
||||
func (c *ConfigurationWatcher) AddListener(listener func(dynamic.Configuration)) {
|
||||
if c.configurationListeners == nil {
|
||||
c.configurationListeners = make([]func(dynamic.Configuration), 0)
|
||||
}
|
||||
c.configurationListeners = append(c.configurationListeners, listener)
|
||||
}
|
||||
|
||||
func (c *ConfigurationWatcher) startProvider() {
|
||||
logger := log.WithoutContext()
|
||||
|
||||
jsonConf, err := json.Marshal(c.provider)
|
||||
if err != nil {
|
||||
logger.Debugf("Unable to marshal provider configuration %T: %v", c.provider, err)
|
||||
}
|
||||
|
||||
logger.Infof("Starting provider %T %s", c.provider, jsonConf)
|
||||
currentProvider := c.provider
|
||||
|
||||
safe.Go(func() {
|
||||
err := currentProvider.Provide(c.configurationChan, c.routinesPool)
|
||||
if err != nil {
|
||||
logger.Errorf("Error starting provider %T: %s", currentProvider, err)
|
||||
}
|
||||
})
|
||||
}
|
||||
|
||||
// listenProviders receives configuration changes from the providers.
|
||||
// The configuration message then gets passed along a series of check
|
||||
// to finally end up in a throttler that sends it to listenConfigurations (through c. configurationValidatedChan).
|
||||
func (c *ConfigurationWatcher) listenProviders(stop chan bool) {
|
||||
for {
|
||||
select {
|
||||
case <-stop:
|
||||
return
|
||||
case configMsg, ok := <-c.configurationChan:
|
||||
if !ok {
|
||||
return
|
||||
}
|
||||
|
||||
if configMsg.Configuration == nil {
|
||||
log.WithoutContext().WithField(log.ProviderName, configMsg.ProviderName).
|
||||
Debug("Received nil configuration from provider, skipping.")
|
||||
return
|
||||
}
|
||||
|
||||
c.preLoadConfiguration(configMsg)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (c *ConfigurationWatcher) listenConfigurations(stop chan bool) {
|
||||
for {
|
||||
select {
|
||||
case <-stop:
|
||||
return
|
||||
case configMsg, ok := <-c.configurationValidatedChan:
|
||||
if !ok || configMsg.Configuration == nil {
|
||||
return
|
||||
}
|
||||
c.loadMessage(configMsg)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (c *ConfigurationWatcher) loadMessage(configMsg dynamic.Message) {
|
||||
currentConfigurations := c.currentConfigurations.Get().(dynamic.Configurations)
|
||||
|
||||
// Copy configurations to new map so we don't change current if LoadConfig fails
|
||||
newConfigurations := currentConfigurations.DeepCopy()
|
||||
newConfigurations[configMsg.ProviderName] = configMsg.Configuration
|
||||
|
||||
c.currentConfigurations.Set(newConfigurations)
|
||||
|
||||
conf := mergeConfiguration(newConfigurations)
|
||||
|
||||
for _, listener := range c.configurationListeners {
|
||||
listener(conf)
|
||||
}
|
||||
}
|
||||
|
||||
func (c *ConfigurationWatcher) preLoadConfiguration(configMsg dynamic.Message) {
|
||||
currentConfigurations := c.currentConfigurations.Get().(dynamic.Configurations)
|
||||
|
||||
logger := log.WithoutContext().WithField(log.ProviderName, configMsg.ProviderName)
|
||||
if log.GetLevel() == logrus.DebugLevel {
|
||||
copyConf := configMsg.Configuration.DeepCopy()
|
||||
if copyConf.TLS != nil {
|
||||
copyConf.TLS.Certificates = nil
|
||||
|
||||
for _, v := range copyConf.TLS.Stores {
|
||||
v.DefaultCertificate = nil
|
||||
}
|
||||
}
|
||||
|
||||
jsonConf, err := json.Marshal(copyConf)
|
||||
if err != nil {
|
||||
logger.Errorf("Could not marshal dynamic configuration: %v", err)
|
||||
logger.Debugf("Configuration received from provider %s: [struct] %#v", configMsg.ProviderName, copyConf)
|
||||
} else {
|
||||
logger.Debugf("Configuration received from provider %s: %s", configMsg.ProviderName, string(jsonConf))
|
||||
}
|
||||
}
|
||||
|
||||
if isEmptyConfiguration(configMsg.Configuration) {
|
||||
logger.Infof("Skipping empty Configuration for provider %s", configMsg.ProviderName)
|
||||
return
|
||||
}
|
||||
|
||||
if reflect.DeepEqual(currentConfigurations[configMsg.ProviderName], configMsg.Configuration) {
|
||||
logger.Infof("Skipping same configuration for provider %s", configMsg.ProviderName)
|
||||
return
|
||||
}
|
||||
|
||||
providerConfigUpdateCh, ok := c.providerConfigUpdateMap[configMsg.ProviderName]
|
||||
if !ok {
|
||||
providerConfigUpdateCh = make(chan dynamic.Message)
|
||||
c.providerConfigUpdateMap[configMsg.ProviderName] = providerConfigUpdateCh
|
||||
c.routinesPool.Go(func(stop chan bool) {
|
||||
c.throttleProviderConfigReload(c.providersThrottleDuration, c.configurationValidatedChan, providerConfigUpdateCh, stop)
|
||||
})
|
||||
}
|
||||
|
||||
providerConfigUpdateCh <- configMsg
|
||||
}
|
||||
|
||||
// throttleProviderConfigReload throttles the configuration reload speed for a single provider.
|
||||
// It will immediately publish a new configuration and then only publish the next configuration after the throttle duration.
|
||||
// Note that in the case it receives N new configs in the timeframe of the throttle duration after publishing,
|
||||
// it will publish the last of the newly received configurations.
|
||||
func (c *ConfigurationWatcher) throttleProviderConfigReload(throttle time.Duration, publish chan<- dynamic.Message, in <-chan dynamic.Message, stop chan bool) {
|
||||
ring := channels.NewRingChannel(1)
|
||||
defer ring.Close()
|
||||
|
||||
c.routinesPool.Go(func(stop chan bool) {
|
||||
for {
|
||||
select {
|
||||
case <-stop:
|
||||
return
|
||||
case nextConfig := <-ring.Out():
|
||||
if config, ok := nextConfig.(dynamic.Message); ok {
|
||||
publish <- config
|
||||
time.Sleep(throttle)
|
||||
}
|
||||
}
|
||||
}
|
||||
})
|
||||
|
||||
for {
|
||||
select {
|
||||
case <-stop:
|
||||
return
|
||||
case nextConfig := <-in:
|
||||
ring.In() <- nextConfig
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func isEmptyConfiguration(conf *dynamic.Configuration) bool {
|
||||
if conf == nil {
|
||||
return true
|
||||
}
|
||||
|
||||
if conf.TCP == nil {
|
||||
conf.TCP = &dynamic.TCPConfiguration{}
|
||||
}
|
||||
if conf.HTTP == nil {
|
||||
conf.HTTP = &dynamic.HTTPConfiguration{}
|
||||
}
|
||||
|
||||
httpEmpty := conf.HTTP.Routers == nil && conf.HTTP.Services == nil && conf.HTTP.Middlewares == nil
|
||||
tlsEmpty := conf.TLS == nil || conf.TLS.Certificates == nil && conf.TLS.Stores == nil && conf.TLS.Options == nil
|
||||
tcpEmpty := conf.TCP.Routers == nil && conf.TCP.Services == nil
|
||||
|
||||
return httpEmpty && tlsEmpty && tcpEmpty
|
||||
}
|
228
pkg/server/configurationwatcher_test.go
Normal file
228
pkg/server/configurationwatcher_test.go
Normal file
|
@ -0,0 +1,228 @@
|
|||
package server
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"strconv"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/containous/traefik/v2/pkg/config/dynamic"
|
||||
"github.com/containous/traefik/v2/pkg/safe"
|
||||
th "github.com/containous/traefik/v2/pkg/testhelpers"
|
||||
"github.com/containous/traefik/v2/pkg/tls"
|
||||
"github.com/stretchr/testify/assert"
|
||||
)
|
||||
|
||||
type mockProvider struct {
|
||||
messages []dynamic.Message
|
||||
wait time.Duration
|
||||
}
|
||||
|
||||
func (p *mockProvider) Provide(configurationChan chan<- dynamic.Message, pool *safe.Pool) error {
|
||||
for _, message := range p.messages {
|
||||
configurationChan <- message
|
||||
|
||||
wait := p.wait
|
||||
if wait == 0 {
|
||||
wait = 20 * time.Millisecond
|
||||
}
|
||||
|
||||
fmt.Println("wait", wait, time.Now().Nanosecond())
|
||||
time.Sleep(wait)
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (p *mockProvider) Init() error {
|
||||
panic("implement me")
|
||||
}
|
||||
|
||||
func TestNewConfigurationWatcher(t *testing.T) {
|
||||
routinesPool := safe.NewPool(context.Background())
|
||||
pvd := &mockProvider{
|
||||
messages: []dynamic.Message{{
|
||||
ProviderName: "mock",
|
||||
Configuration: &dynamic.Configuration{
|
||||
HTTP: th.BuildConfiguration(
|
||||
th.WithRouters(
|
||||
th.WithRouter("test",
|
||||
th.WithEntryPoints("e"),
|
||||
th.WithServiceName("scv"))),
|
||||
),
|
||||
},
|
||||
}},
|
||||
}
|
||||
|
||||
watcher := NewConfigurationWatcher(routinesPool, pvd, time.Second)
|
||||
|
||||
run := make(chan struct{})
|
||||
|
||||
watcher.AddListener(func(conf dynamic.Configuration) {
|
||||
expected := dynamic.Configuration{
|
||||
HTTP: th.BuildConfiguration(
|
||||
th.WithRouters(
|
||||
th.WithRouter("test@mock",
|
||||
th.WithEntryPoints("e"),
|
||||
th.WithServiceName("scv"))),
|
||||
th.WithMiddlewares(),
|
||||
th.WithLoadBalancerServices(),
|
||||
),
|
||||
TCP: &dynamic.TCPConfiguration{
|
||||
Routers: map[string]*dynamic.TCPRouter{},
|
||||
Services: map[string]*dynamic.TCPService{},
|
||||
},
|
||||
TLS: &dynamic.TLSConfiguration{
|
||||
Options: map[string]tls.Options{
|
||||
"default": {},
|
||||
},
|
||||
Stores: map[string]tls.Store{},
|
||||
},
|
||||
}
|
||||
|
||||
assert.Equal(t, expected, conf)
|
||||
close(run)
|
||||
})
|
||||
|
||||
watcher.Start()
|
||||
<-run
|
||||
}
|
||||
|
||||
func TestListenProvidersThrottleProviderConfigReload(t *testing.T) {
|
||||
routinesPool := safe.NewPool(context.Background())
|
||||
|
||||
pvd := &mockProvider{
|
||||
wait: 10 * time.Millisecond,
|
||||
}
|
||||
|
||||
for i := 0; i < 5; i++ {
|
||||
pvd.messages = append(pvd.messages, dynamic.Message{
|
||||
ProviderName: "mock",
|
||||
Configuration: &dynamic.Configuration{
|
||||
HTTP: th.BuildConfiguration(
|
||||
th.WithRouters(th.WithRouter("foo"+strconv.Itoa(i))),
|
||||
th.WithLoadBalancerServices(th.WithService("bar")),
|
||||
),
|
||||
},
|
||||
})
|
||||
}
|
||||
|
||||
watcher := NewConfigurationWatcher(routinesPool, pvd, 30*time.Millisecond)
|
||||
|
||||
publishedConfigCount := 0
|
||||
watcher.AddListener(func(_ dynamic.Configuration) {
|
||||
publishedConfigCount++
|
||||
})
|
||||
|
||||
watcher.Start()
|
||||
defer watcher.Stop()
|
||||
|
||||
// give some time so that the configuration can be processed
|
||||
time.Sleep(100 * time.Millisecond)
|
||||
|
||||
// after 50 milliseconds 5 new configs were published
|
||||
// with a throttle duration of 30 milliseconds this means, we should have received 3 new configs
|
||||
assert.Equal(t, 3, publishedConfigCount, "times configs were published")
|
||||
}
|
||||
|
||||
func TestListenProvidersSkipsEmptyConfigs(t *testing.T) {
|
||||
routinesPool := safe.NewPool(context.Background())
|
||||
pvd := &mockProvider{
|
||||
messages: []dynamic.Message{{ProviderName: "mock"}},
|
||||
}
|
||||
|
||||
watcher := NewConfigurationWatcher(routinesPool, pvd, time.Second)
|
||||
watcher.AddListener(func(_ dynamic.Configuration) {
|
||||
t.Error("An empty configuration was published but it should not")
|
||||
})
|
||||
watcher.Start()
|
||||
defer watcher.Stop()
|
||||
|
||||
// give some time so that the configuration can be processed
|
||||
time.Sleep(100 * time.Millisecond)
|
||||
}
|
||||
|
||||
func TestListenProvidersSkipsSameConfigurationForProvider(t *testing.T) {
|
||||
routinesPool := safe.NewPool(context.Background())
|
||||
message := dynamic.Message{
|
||||
ProviderName: "mock",
|
||||
Configuration: &dynamic.Configuration{
|
||||
HTTP: th.BuildConfiguration(
|
||||
th.WithRouters(th.WithRouter("foo")),
|
||||
th.WithLoadBalancerServices(th.WithService("bar")),
|
||||
),
|
||||
},
|
||||
}
|
||||
pvd := &mockProvider{
|
||||
messages: []dynamic.Message{message, message},
|
||||
}
|
||||
|
||||
watcher := NewConfigurationWatcher(routinesPool, pvd, 0)
|
||||
|
||||
alreadyCalled := false
|
||||
watcher.AddListener(func(_ dynamic.Configuration) {
|
||||
if alreadyCalled {
|
||||
t.Error("Same configuration should not be published multiple times")
|
||||
}
|
||||
alreadyCalled = true
|
||||
})
|
||||
|
||||
watcher.Start()
|
||||
defer watcher.Stop()
|
||||
|
||||
// give some time so that the configuration can be processed
|
||||
time.Sleep(100 * time.Millisecond)
|
||||
}
|
||||
|
||||
func TestListenProvidersPublishesConfigForEachProvider(t *testing.T) {
|
||||
routinesPool := safe.NewPool(context.Background())
|
||||
|
||||
configuration := &dynamic.Configuration{
|
||||
HTTP: th.BuildConfiguration(
|
||||
th.WithRouters(th.WithRouter("foo")),
|
||||
th.WithLoadBalancerServices(th.WithService("bar")),
|
||||
),
|
||||
}
|
||||
|
||||
pvd := &mockProvider{
|
||||
messages: []dynamic.Message{
|
||||
{ProviderName: "mock", Configuration: configuration},
|
||||
{ProviderName: "mock2", Configuration: configuration},
|
||||
},
|
||||
}
|
||||
|
||||
watcher := NewConfigurationWatcher(routinesPool, pvd, 0)
|
||||
|
||||
var publishedProviderConfig dynamic.Configuration
|
||||
|
||||
watcher.AddListener(func(conf dynamic.Configuration) {
|
||||
publishedProviderConfig = conf
|
||||
})
|
||||
|
||||
watcher.Start()
|
||||
defer watcher.Stop()
|
||||
|
||||
// give some time so that the configuration can be processed
|
||||
time.Sleep(100 * time.Millisecond)
|
||||
|
||||
expected := dynamic.Configuration{
|
||||
HTTP: th.BuildConfiguration(
|
||||
th.WithRouters(th.WithRouter("foo@mock"), th.WithRouter("foo@mock2")),
|
||||
th.WithLoadBalancerServices(th.WithService("bar@mock"), th.WithService("bar@mock2")),
|
||||
th.WithMiddlewares(),
|
||||
),
|
||||
TCP: &dynamic.TCPConfiguration{
|
||||
Routers: map[string]*dynamic.TCPRouter{},
|
||||
Services: map[string]*dynamic.TCPService{},
|
||||
},
|
||||
TLS: &dynamic.TLSConfiguration{
|
||||
Options: map[string]tls.Options{
|
||||
"default": {},
|
||||
},
|
||||
Stores: map[string]tls.Store{},
|
||||
},
|
||||
}
|
||||
|
||||
assert.Equal(t, expected, publishedProviderConfig)
|
||||
}
|
115
pkg/server/internal/provider_test.go
Normal file
115
pkg/server/internal/provider_test.go
Normal file
|
@ -0,0 +1,115 @@
|
|||
package internal
|
||||
|
||||
import (
|
||||
"context"
|
||||
"testing"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
)
|
||||
|
||||
func TestAddProviderInContext(t *testing.T) {
|
||||
testCases := []struct {
|
||||
desc string
|
||||
ctx context.Context
|
||||
name string
|
||||
expected string
|
||||
}{
|
||||
{
|
||||
desc: "without provider information",
|
||||
ctx: context.Background(),
|
||||
name: "test",
|
||||
expected: "",
|
||||
},
|
||||
{
|
||||
desc: "provider name embedded in element name",
|
||||
ctx: context.Background(),
|
||||
name: "test@foo",
|
||||
expected: "foo",
|
||||
},
|
||||
{
|
||||
desc: "provider name in context",
|
||||
ctx: context.WithValue(context.Background(), providerKey, "foo"),
|
||||
name: "test",
|
||||
expected: "foo",
|
||||
},
|
||||
{
|
||||
desc: "provider name in context and different provider name embedded in element name",
|
||||
ctx: context.WithValue(context.Background(), providerKey, "foo"),
|
||||
name: "test@fii",
|
||||
expected: "fii",
|
||||
},
|
||||
{
|
||||
desc: "provider name in context and same provider name embedded in element name",
|
||||
ctx: context.WithValue(context.Background(), providerKey, "foo"),
|
||||
name: "test@foo",
|
||||
expected: "foo",
|
||||
},
|
||||
}
|
||||
|
||||
for _, test := range testCases {
|
||||
test := test
|
||||
t.Run(test.desc, func(t *testing.T) {
|
||||
t.Parallel()
|
||||
|
||||
newCtx := AddProviderInContext(test.ctx, test.name)
|
||||
|
||||
var providerName string
|
||||
if name, ok := newCtx.Value(providerKey).(string); ok {
|
||||
providerName = name
|
||||
}
|
||||
|
||||
assert.Equal(t, test.expected, providerName)
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func TestGetQualifiedName(t *testing.T) {
|
||||
testCases := []struct {
|
||||
desc string
|
||||
ctx context.Context
|
||||
name string
|
||||
expected string
|
||||
}{
|
||||
{
|
||||
desc: "empty name",
|
||||
ctx: context.Background(),
|
||||
name: "",
|
||||
expected: "",
|
||||
},
|
||||
{
|
||||
desc: "without provider",
|
||||
ctx: context.Background(),
|
||||
name: "test",
|
||||
expected: "test",
|
||||
},
|
||||
{
|
||||
desc: "with explicit provider",
|
||||
ctx: context.Background(),
|
||||
name: "test@foo",
|
||||
expected: "test@foo",
|
||||
},
|
||||
{
|
||||
desc: "with provider in context",
|
||||
ctx: context.WithValue(context.Background(), providerKey, "foo"),
|
||||
name: "test",
|
||||
expected: "test@foo",
|
||||
},
|
||||
{
|
||||
desc: "with provider in context and explicit name",
|
||||
ctx: context.WithValue(context.Background(), providerKey, "foo"),
|
||||
name: "test@fii",
|
||||
expected: "test@fii",
|
||||
},
|
||||
}
|
||||
|
||||
for _, test := range testCases {
|
||||
test := test
|
||||
t.Run(test.desc, func(t *testing.T) {
|
||||
t.Parallel()
|
||||
|
||||
qualifiedName := GetQualifiedName(test.ctx, test.name)
|
||||
|
||||
assert.Equal(t, test.expected, qualifiedName)
|
||||
})
|
||||
}
|
||||
}
|
124
pkg/server/middleware/chainbuilder.go
Normal file
124
pkg/server/middleware/chainbuilder.go
Normal file
|
@ -0,0 +1,124 @@
|
|||
package middleware
|
||||
|
||||
import (
|
||||
"context"
|
||||
|
||||
"github.com/containous/alice"
|
||||
"github.com/containous/traefik/v2/pkg/config/static"
|
||||
"github.com/containous/traefik/v2/pkg/log"
|
||||
"github.com/containous/traefik/v2/pkg/metrics"
|
||||
"github.com/containous/traefik/v2/pkg/middlewares/accesslog"
|
||||
metricsmiddleware "github.com/containous/traefik/v2/pkg/middlewares/metrics"
|
||||
"github.com/containous/traefik/v2/pkg/middlewares/requestdecorator"
|
||||
mTracing "github.com/containous/traefik/v2/pkg/middlewares/tracing"
|
||||
"github.com/containous/traefik/v2/pkg/tracing"
|
||||
"github.com/containous/traefik/v2/pkg/tracing/jaeger"
|
||||
)
|
||||
|
||||
// ChainBuilder Creates a middleware chain by entry point. It is used for middlewares that are created almost systematically and that need to be created before all others.
|
||||
type ChainBuilder struct {
|
||||
metricsRegistry metrics.Registry
|
||||
accessLoggerMiddleware *accesslog.Handler
|
||||
tracer *tracing.Tracing
|
||||
requestDecorator *requestdecorator.RequestDecorator
|
||||
}
|
||||
|
||||
// NewChainBuilder Creates a new ChainBuilder.
|
||||
func NewChainBuilder(staticConfiguration static.Configuration, metricsRegistry metrics.Registry, accessLoggerMiddleware *accesslog.Handler) *ChainBuilder {
|
||||
return &ChainBuilder{
|
||||
metricsRegistry: metricsRegistry,
|
||||
accessLoggerMiddleware: accessLoggerMiddleware,
|
||||
tracer: setupTracing(staticConfiguration.Tracing),
|
||||
requestDecorator: requestdecorator.New(staticConfiguration.HostResolver),
|
||||
}
|
||||
}
|
||||
|
||||
// Build a middleware chain by entry point.
|
||||
func (c *ChainBuilder) Build(ctx context.Context, entryPointName string) alice.Chain {
|
||||
chain := alice.New()
|
||||
|
||||
if c.accessLoggerMiddleware != nil {
|
||||
chain = chain.Append(accesslog.WrapHandler(c.accessLoggerMiddleware))
|
||||
}
|
||||
|
||||
if c.tracer != nil {
|
||||
chain = chain.Append(mTracing.WrapEntryPointHandler(ctx, c.tracer, entryPointName))
|
||||
}
|
||||
|
||||
if c.metricsRegistry != nil && c.metricsRegistry.IsEpEnabled() {
|
||||
chain = chain.Append(metricsmiddleware.WrapEntryPointHandler(ctx, c.metricsRegistry, entryPointName))
|
||||
}
|
||||
|
||||
return chain.Append(requestdecorator.WrapHandler(c.requestDecorator))
|
||||
}
|
||||
|
||||
// Close accessLogger and tracer.
|
||||
func (c *ChainBuilder) Close() {
|
||||
if c.accessLoggerMiddleware != nil {
|
||||
if err := c.accessLoggerMiddleware.Close(); err != nil {
|
||||
log.WithoutContext().Errorf("Could not close the access log file: %s", err)
|
||||
}
|
||||
}
|
||||
|
||||
if c.tracer != nil {
|
||||
c.tracer.Close()
|
||||
}
|
||||
}
|
||||
|
||||
func setupTracing(conf *static.Tracing) *tracing.Tracing {
|
||||
if conf == nil {
|
||||
return nil
|
||||
}
|
||||
|
||||
var backend tracing.Backend
|
||||
|
||||
if conf.Jaeger != nil {
|
||||
backend = conf.Jaeger
|
||||
}
|
||||
|
||||
if conf.Zipkin != nil {
|
||||
if backend != nil {
|
||||
log.WithoutContext().Error("Multiple tracing backend are not supported: cannot create Zipkin backend.")
|
||||
} else {
|
||||
backend = conf.Zipkin
|
||||
}
|
||||
}
|
||||
|
||||
if conf.Datadog != nil {
|
||||
if backend != nil {
|
||||
log.WithoutContext().Error("Multiple tracing backend are not supported: cannot create Datadog backend.")
|
||||
} else {
|
||||
backend = conf.Datadog
|
||||
}
|
||||
}
|
||||
|
||||
if conf.Instana != nil {
|
||||
if backend != nil {
|
||||
log.WithoutContext().Error("Multiple tracing backend are not supported: cannot create Instana backend.")
|
||||
} else {
|
||||
backend = conf.Instana
|
||||
}
|
||||
}
|
||||
|
||||
if conf.Haystack != nil {
|
||||
if backend != nil {
|
||||
log.WithoutContext().Error("Multiple tracing backend are not supported: cannot create Haystack backend.")
|
||||
} else {
|
||||
backend = conf.Haystack
|
||||
}
|
||||
}
|
||||
|
||||
if backend == nil {
|
||||
log.WithoutContext().Debug("Could not initialize tracing, using Jaeger by default")
|
||||
defaultBackend := &jaeger.Config{}
|
||||
defaultBackend.SetDefaults()
|
||||
backend = defaultBackend
|
||||
}
|
||||
|
||||
tracer, err := tracing.NewTracing(conf.ServiceName, conf.SpanNameLimit, backend)
|
||||
if err != nil {
|
||||
log.WithoutContext().Warnf("Unable to create tracer: %v", err)
|
||||
return nil
|
||||
}
|
||||
return tracer
|
||||
}
|
|
@ -1,89 +0,0 @@
|
|||
package router
|
||||
|
||||
import (
|
||||
"context"
|
||||
|
||||
"github.com/containous/alice"
|
||||
"github.com/containous/traefik/v2/pkg/api"
|
||||
"github.com/containous/traefik/v2/pkg/config/runtime"
|
||||
"github.com/containous/traefik/v2/pkg/config/static"
|
||||
"github.com/containous/traefik/v2/pkg/log"
|
||||
"github.com/containous/traefik/v2/pkg/metrics"
|
||||
"github.com/containous/traefik/v2/pkg/types"
|
||||
"github.com/gorilla/mux"
|
||||
)
|
||||
|
||||
// NewRouteAppenderAggregator Creates a new RouteAppenderAggregator
|
||||
func NewRouteAppenderAggregator(ctx context.Context, conf static.Configuration,
|
||||
entryPointName string, runtimeConfiguration *runtime.Configuration) *RouteAppenderAggregator {
|
||||
aggregator := &RouteAppenderAggregator{}
|
||||
|
||||
if conf.Ping != nil && conf.Ping.EntryPoint == entryPointName {
|
||||
aggregator.AddAppender(conf.Ping)
|
||||
}
|
||||
|
||||
if conf.Metrics != nil && conf.Metrics.Prometheus != nil && conf.Metrics.Prometheus.EntryPoint == entryPointName {
|
||||
aggregator.AddAppender(metrics.PrometheusHandler{})
|
||||
}
|
||||
|
||||
if entryPointName != "traefik" {
|
||||
return aggregator
|
||||
}
|
||||
|
||||
if conf.Providers != nil && conf.Providers.Rest != nil && conf.Providers.Rest.Insecure {
|
||||
aggregator.AddAppender(conf.Providers.Rest)
|
||||
}
|
||||
|
||||
if conf.API != nil && conf.API.Insecure {
|
||||
aggregator.AddAppender(api.New(conf, runtimeConfiguration))
|
||||
}
|
||||
|
||||
return aggregator
|
||||
}
|
||||
|
||||
// RouteAppenderAggregator RouteAppender that aggregate other RouteAppender
|
||||
type RouteAppenderAggregator struct {
|
||||
appenders []types.RouteAppender
|
||||
}
|
||||
|
||||
// Append Adds routes to the router
|
||||
func (r *RouteAppenderAggregator) Append(systemRouter *mux.Router) {
|
||||
for _, router := range r.appenders {
|
||||
router.Append(systemRouter)
|
||||
}
|
||||
}
|
||||
|
||||
// AddAppender adds a router in the aggregator
|
||||
func (r *RouteAppenderAggregator) AddAppender(router types.RouteAppender) {
|
||||
r.appenders = append(r.appenders, router)
|
||||
}
|
||||
|
||||
// WithMiddleware router with internal middleware
|
||||
type WithMiddleware struct {
|
||||
appender types.RouteAppender
|
||||
routerMiddlewares *alice.Chain
|
||||
}
|
||||
|
||||
// Append Adds routes to the router
|
||||
func (wm *WithMiddleware) Append(systemRouter *mux.Router) {
|
||||
realRouter := systemRouter.PathPrefix("/").Subrouter()
|
||||
|
||||
wm.appender.Append(realRouter)
|
||||
|
||||
if err := realRouter.Walk(wrapRoute(wm.routerMiddlewares)); err != nil {
|
||||
log.WithoutContext().Error(err)
|
||||
}
|
||||
}
|
||||
|
||||
// wrapRoute with middlewares
|
||||
func wrapRoute(middlewares *alice.Chain) func(*mux.Route, *mux.Router, []*mux.Route) error {
|
||||
return func(route *mux.Route, router *mux.Router, ancestors []*mux.Route) error {
|
||||
handler, err := middlewares.Then(route.GetHandler())
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
route.Handler(handler)
|
||||
return nil
|
||||
}
|
||||
}
|
|
@ -1,79 +0,0 @@
|
|||
package router
|
||||
|
||||
import (
|
||||
"context"
|
||||
"net/http"
|
||||
"net/http/httptest"
|
||||
"testing"
|
||||
|
||||
"github.com/containous/traefik/v2/pkg/config/static"
|
||||
"github.com/gorilla/mux"
|
||||
"github.com/stretchr/testify/assert"
|
||||
)
|
||||
|
||||
func TestNewRouteAppenderAggregator(t *testing.T) {
|
||||
testCases := []struct {
|
||||
desc string
|
||||
staticConf static.Configuration
|
||||
expected map[string]int
|
||||
}{
|
||||
{
|
||||
desc: "Secure API",
|
||||
staticConf: static.Configuration{
|
||||
Global: &static.Global{},
|
||||
API: &static.API{
|
||||
Insecure: false,
|
||||
},
|
||||
EntryPoints: static.EntryPoints{
|
||||
"traefik": {},
|
||||
},
|
||||
},
|
||||
expected: map[string]int{
|
||||
"/api/providers": http.StatusBadGateway,
|
||||
},
|
||||
},
|
||||
{
|
||||
desc: "Insecure API",
|
||||
staticConf: static.Configuration{
|
||||
Global: &static.Global{},
|
||||
API: &static.API{
|
||||
Insecure: true,
|
||||
},
|
||||
EntryPoints: static.EntryPoints{
|
||||
"traefik": {},
|
||||
},
|
||||
},
|
||||
expected: map[string]int{
|
||||
"/api/rawdata": http.StatusOK,
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
for _, test := range testCases {
|
||||
test := test
|
||||
t.Run(test.desc, func(t *testing.T) {
|
||||
t.Parallel()
|
||||
|
||||
ctx := context.Background()
|
||||
|
||||
router := NewRouteAppenderAggregator(ctx, test.staticConf, "traefik", nil)
|
||||
|
||||
internalMuxRouter := mux.NewRouter()
|
||||
router.Append(internalMuxRouter)
|
||||
|
||||
internalMuxRouter.NotFoundHandler = http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) {
|
||||
w.WriteHeader(http.StatusBadGateway)
|
||||
})
|
||||
|
||||
actual := make(map[string]int)
|
||||
for calledURL := range test.expected {
|
||||
recorder := httptest.NewRecorder()
|
||||
request := httptest.NewRequest(http.MethodGet, calledURL, nil)
|
||||
internalMuxRouter.ServeHTTP(recorder, request)
|
||||
actual[calledURL] = recorder.Code
|
||||
}
|
||||
|
||||
assert.Equal(t, test.expected, actual)
|
||||
})
|
||||
}
|
||||
}
|
|
@ -1,40 +0,0 @@
|
|||
package router
|
||||
|
||||
import (
|
||||
"context"
|
||||
|
||||
"github.com/containous/traefik/v2/pkg/config/runtime"
|
||||
"github.com/containous/traefik/v2/pkg/config/static"
|
||||
"github.com/containous/traefik/v2/pkg/provider/acme"
|
||||
"github.com/containous/traefik/v2/pkg/types"
|
||||
)
|
||||
|
||||
// NewRouteAppenderFactory Creates a new RouteAppenderFactory
|
||||
func NewRouteAppenderFactory(staticConfiguration static.Configuration, entryPointName string, acmeProvider []*acme.Provider) *RouteAppenderFactory {
|
||||
return &RouteAppenderFactory{
|
||||
staticConfiguration: staticConfiguration,
|
||||
entryPointName: entryPointName,
|
||||
acmeProvider: acmeProvider,
|
||||
}
|
||||
}
|
||||
|
||||
// RouteAppenderFactory A factory of RouteAppender
|
||||
type RouteAppenderFactory struct {
|
||||
staticConfiguration static.Configuration
|
||||
entryPointName string
|
||||
acmeProvider []*acme.Provider
|
||||
}
|
||||
|
||||
// NewAppender Creates a new RouteAppender
|
||||
func (r *RouteAppenderFactory) NewAppender(ctx context.Context, runtimeConfiguration *runtime.Configuration) types.RouteAppender {
|
||||
aggregator := NewRouteAppenderAggregator(ctx, r.staticConfiguration, r.entryPointName, runtimeConfiguration)
|
||||
|
||||
for _, p := range r.acmeProvider {
|
||||
if p != nil && p.HTTPChallenge != nil && p.HTTPChallenge.EntryPoint == r.entryPointName {
|
||||
aggregator.AddAppender(p)
|
||||
break
|
||||
}
|
||||
}
|
||||
|
||||
return aggregator
|
||||
}
|
|
@ -11,41 +11,55 @@ import (
|
|||
"github.com/containous/traefik/v2/pkg/middlewares/accesslog"
|
||||
"github.com/containous/traefik/v2/pkg/middlewares/recovery"
|
||||
"github.com/containous/traefik/v2/pkg/middlewares/tracing"
|
||||
"github.com/containous/traefik/v2/pkg/responsemodifiers"
|
||||
"github.com/containous/traefik/v2/pkg/rules"
|
||||
"github.com/containous/traefik/v2/pkg/server/internal"
|
||||
"github.com/containous/traefik/v2/pkg/server/middleware"
|
||||
"github.com/containous/traefik/v2/pkg/server/service"
|
||||
)
|
||||
|
||||
const (
|
||||
recoveryMiddlewareName = "traefik-internal-recovery"
|
||||
)
|
||||
|
||||
type middlewareBuilder interface {
|
||||
BuildChain(ctx context.Context, names []string) *alice.Chain
|
||||
}
|
||||
|
||||
type responseModifierBuilder interface {
|
||||
Build(ctx context.Context, names []string) func(*http.Response) error
|
||||
}
|
||||
|
||||
type serviceManager interface {
|
||||
BuildHTTP(rootCtx context.Context, serviceName string, responseModifier func(*http.Response) error) (http.Handler, error)
|
||||
LaunchHealthCheck()
|
||||
}
|
||||
|
||||
// Manager A route/router manager
|
||||
type Manager struct {
|
||||
routerHandlers map[string]http.Handler
|
||||
serviceManager serviceManager
|
||||
middlewaresBuilder middlewareBuilder
|
||||
chainBuilder *middleware.ChainBuilder
|
||||
modifierBuilder responseModifierBuilder
|
||||
conf *runtime.Configuration
|
||||
}
|
||||
|
||||
// NewManager Creates a new Manager
|
||||
func NewManager(conf *runtime.Configuration,
|
||||
serviceManager *service.Manager,
|
||||
middlewaresBuilder *middleware.Builder,
|
||||
modifierBuilder *responsemodifiers.Builder,
|
||||
serviceManager serviceManager,
|
||||
middlewaresBuilder middlewareBuilder,
|
||||
modifierBuilder responseModifierBuilder,
|
||||
chainBuilder *middleware.ChainBuilder,
|
||||
) *Manager {
|
||||
return &Manager{
|
||||
routerHandlers: make(map[string]http.Handler),
|
||||
serviceManager: serviceManager,
|
||||
middlewaresBuilder: middlewaresBuilder,
|
||||
modifierBuilder: modifierBuilder,
|
||||
chainBuilder: chainBuilder,
|
||||
conf: conf,
|
||||
}
|
||||
}
|
||||
|
||||
// Manager A route/router manager
|
||||
type Manager struct {
|
||||
routerHandlers map[string]http.Handler
|
||||
serviceManager *service.Manager
|
||||
middlewaresBuilder *middleware.Builder
|
||||
modifierBuilder *responsemodifiers.Builder
|
||||
conf *runtime.Configuration
|
||||
}
|
||||
|
||||
func (m *Manager) getHTTPRouters(ctx context.Context, entryPoints []string, tls bool) map[string]map[string]*runtime.RouterInfo {
|
||||
if m.conf != nil {
|
||||
return m.conf.GetRoutersByEntryPoints(ctx, entryPoints, tls)
|
||||
|
@ -79,6 +93,22 @@ func (m *Manager) BuildHandlers(rootCtx context.Context, entryPoints []string, t
|
|||
}
|
||||
}
|
||||
|
||||
for _, entryPointName := range entryPoints {
|
||||
ctx := log.With(rootCtx, log.Str(log.EntryPointName, entryPointName))
|
||||
|
||||
handler, ok := entryPointHandlers[entryPointName]
|
||||
if !ok || handler == nil {
|
||||
handler = BuildDefaultHTTPRouter()
|
||||
}
|
||||
|
||||
handlerWithMiddlewares, err := m.chainBuilder.Build(ctx, entryPointName).Then(handler)
|
||||
if err != nil {
|
||||
log.FromContext(ctx).Error(err)
|
||||
continue
|
||||
}
|
||||
entryPointHandlers[entryPointName] = handlerWithMiddlewares
|
||||
}
|
||||
|
||||
m.serviceManager.LaunchHealthCheck()
|
||||
|
||||
return entryPointHandlers
|
||||
|
@ -167,3 +197,8 @@ func (m *Manager) buildHTTPHandler(ctx context.Context, router *runtime.RouterIn
|
|||
|
||||
return alice.New().Extend(*mHandler).Append(tHandler).Then(sHandler)
|
||||
}
|
||||
|
||||
// BuildDefaultHTTPRouter creates a default HTTP router.
|
||||
func BuildDefaultHTTPRouter() http.Handler {
|
||||
return http.NotFoundHandler()
|
||||
}
|
||||
|
|
|
@ -10,6 +10,7 @@ import (
|
|||
|
||||
"github.com/containous/traefik/v2/pkg/config/dynamic"
|
||||
"github.com/containous/traefik/v2/pkg/config/runtime"
|
||||
"github.com/containous/traefik/v2/pkg/config/static"
|
||||
"github.com/containous/traefik/v2/pkg/middlewares/accesslog"
|
||||
"github.com/containous/traefik/v2/pkg/middlewares/requestdecorator"
|
||||
"github.com/containous/traefik/v2/pkg/responsemodifiers"
|
||||
|
@ -24,7 +25,7 @@ import (
|
|||
func TestRouterManager_Get(t *testing.T) {
|
||||
server := httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) {}))
|
||||
|
||||
type ExpectedResult struct {
|
||||
type expectedResult struct {
|
||||
StatusCode int
|
||||
RequestHeaders map[string]string
|
||||
}
|
||||
|
@ -35,7 +36,7 @@ func TestRouterManager_Get(t *testing.T) {
|
|||
serviceConfig map[string]*dynamic.Service
|
||||
middlewaresConfig map[string]*dynamic.Middleware
|
||||
entryPoints []string
|
||||
expected ExpectedResult
|
||||
expected expectedResult
|
||||
}{
|
||||
{
|
||||
desc: "no middleware",
|
||||
|
@ -58,7 +59,7 @@ func TestRouterManager_Get(t *testing.T) {
|
|||
},
|
||||
},
|
||||
entryPoints: []string{"web"},
|
||||
expected: ExpectedResult{StatusCode: http.StatusOK},
|
||||
expected: expectedResult{StatusCode: http.StatusOK},
|
||||
},
|
||||
{
|
||||
desc: "no load balancer",
|
||||
|
@ -73,7 +74,7 @@ func TestRouterManager_Get(t *testing.T) {
|
|||
"foo-service": {},
|
||||
},
|
||||
entryPoints: []string{"web"},
|
||||
expected: ExpectedResult{StatusCode: http.StatusNotFound},
|
||||
expected: expectedResult{StatusCode: http.StatusNotFound},
|
||||
},
|
||||
{
|
||||
desc: "no middleware, default entry point",
|
||||
|
@ -95,7 +96,7 @@ func TestRouterManager_Get(t *testing.T) {
|
|||
},
|
||||
},
|
||||
entryPoints: []string{"web"},
|
||||
expected: ExpectedResult{StatusCode: http.StatusOK},
|
||||
expected: expectedResult{StatusCode: http.StatusOK},
|
||||
},
|
||||
{
|
||||
desc: "no middleware, no matching",
|
||||
|
@ -118,7 +119,7 @@ func TestRouterManager_Get(t *testing.T) {
|
|||
},
|
||||
},
|
||||
entryPoints: []string{"web"},
|
||||
expected: ExpectedResult{StatusCode: http.StatusNotFound},
|
||||
expected: expectedResult{StatusCode: http.StatusNotFound},
|
||||
},
|
||||
{
|
||||
desc: "middleware: headers > auth",
|
||||
|
@ -154,7 +155,7 @@ func TestRouterManager_Get(t *testing.T) {
|
|||
},
|
||||
},
|
||||
entryPoints: []string{"web"},
|
||||
expected: ExpectedResult{
|
||||
expected: expectedResult{
|
||||
StatusCode: http.StatusUnauthorized,
|
||||
RequestHeaders: map[string]string{
|
||||
"X-Apero": "beer",
|
||||
|
@ -195,7 +196,7 @@ func TestRouterManager_Get(t *testing.T) {
|
|||
},
|
||||
},
|
||||
entryPoints: []string{"web"},
|
||||
expected: ExpectedResult{
|
||||
expected: expectedResult{
|
||||
StatusCode: http.StatusUnauthorized,
|
||||
RequestHeaders: map[string]string{
|
||||
"X-Apero": "",
|
||||
|
@ -223,7 +224,7 @@ func TestRouterManager_Get(t *testing.T) {
|
|||
},
|
||||
},
|
||||
entryPoints: []string{"web"},
|
||||
expected: ExpectedResult{StatusCode: http.StatusOK},
|
||||
expected: expectedResult{StatusCode: http.StatusOK},
|
||||
},
|
||||
{
|
||||
desc: "no middleware with specified provider name",
|
||||
|
@ -246,7 +247,7 @@ func TestRouterManager_Get(t *testing.T) {
|
|||
},
|
||||
},
|
||||
entryPoints: []string{"web"},
|
||||
expected: ExpectedResult{StatusCode: http.StatusOK},
|
||||
expected: expectedResult{StatusCode: http.StatusOK},
|
||||
},
|
||||
{
|
||||
desc: "middleware: chain with provider name",
|
||||
|
@ -285,7 +286,7 @@ func TestRouterManager_Get(t *testing.T) {
|
|||
},
|
||||
},
|
||||
entryPoints: []string{"web"},
|
||||
expected: ExpectedResult{
|
||||
expected: expectedResult{
|
||||
StatusCode: http.StatusUnauthorized,
|
||||
RequestHeaders: map[string]string{
|
||||
"X-Apero": "",
|
||||
|
@ -306,10 +307,13 @@ func TestRouterManager_Get(t *testing.T) {
|
|||
Middlewares: test.middlewaresConfig,
|
||||
},
|
||||
})
|
||||
serviceManager := service.NewManager(rtConf.Services, http.DefaultTransport, nil, nil, nil, nil)
|
||||
|
||||
serviceManager := service.NewManager(rtConf.Services, http.DefaultTransport, nil, nil)
|
||||
middlewaresBuilder := middleware.NewBuilder(rtConf.Middlewares, serviceManager)
|
||||
responseModifierFactory := responsemodifiers.NewBuilder(rtConf.Middlewares)
|
||||
routerManager := NewManager(rtConf, serviceManager, middlewaresBuilder, responseModifierFactory)
|
||||
chainBuilder := middleware.NewChainBuilder(static.Configuration{}, nil, nil)
|
||||
|
||||
routerManager := NewManager(rtConf, serviceManager, middlewaresBuilder, responseModifierFactory, chainBuilder)
|
||||
|
||||
handlers := routerManager.BuildHandlers(context.Background(), test.entryPoints, false)
|
||||
|
||||
|
@ -407,10 +411,12 @@ func TestAccessLog(t *testing.T) {
|
|||
},
|
||||
})
|
||||
|
||||
serviceManager := service.NewManager(rtConf.Services, http.DefaultTransport, nil, nil, nil, nil)
|
||||
serviceManager := service.NewManager(rtConf.Services, http.DefaultTransport, nil, nil)
|
||||
middlewaresBuilder := middleware.NewBuilder(rtConf.Middlewares, serviceManager)
|
||||
responseModifierFactory := responsemodifiers.NewBuilder(rtConf.Middlewares)
|
||||
routerManager := NewManager(rtConf, serviceManager, middlewaresBuilder, responseModifierFactory)
|
||||
chainBuilder := middleware.NewChainBuilder(static.Configuration{}, nil, nil)
|
||||
|
||||
routerManager := NewManager(rtConf, serviceManager, middlewaresBuilder, responseModifierFactory, chainBuilder)
|
||||
|
||||
handlers := routerManager.BuildHandlers(context.Background(), test.entryPoints, false)
|
||||
|
||||
|
@ -680,7 +686,6 @@ func TestRuntimeConfiguration(t *testing.T) {
|
|||
|
||||
for _, test := range testCases {
|
||||
test := test
|
||||
|
||||
t.Run(test.desc, func(t *testing.T) {
|
||||
t.Parallel()
|
||||
|
||||
|
@ -693,10 +698,13 @@ func TestRuntimeConfiguration(t *testing.T) {
|
|||
Middlewares: test.middlewareConfig,
|
||||
},
|
||||
})
|
||||
serviceManager := service.NewManager(rtConf.Services, http.DefaultTransport, nil, nil, nil, nil)
|
||||
|
||||
serviceManager := service.NewManager(rtConf.Services, http.DefaultTransport, nil, nil)
|
||||
middlewaresBuilder := middleware.NewBuilder(rtConf.Middlewares, serviceManager)
|
||||
responseModifierFactory := responsemodifiers.NewBuilder(map[string]*runtime.MiddlewareInfo{})
|
||||
routerManager := NewManager(rtConf, serviceManager, middlewaresBuilder, responseModifierFactory)
|
||||
chainBuilder := middleware.NewChainBuilder(static.Configuration{}, nil, nil)
|
||||
|
||||
routerManager := NewManager(rtConf, serviceManager, middlewaresBuilder, responseModifierFactory, chainBuilder)
|
||||
|
||||
_ = routerManager.BuildHandlers(context.Background(), entryPoints, false)
|
||||
|
||||
|
@ -762,10 +770,13 @@ func TestProviderOnMiddlewares(t *testing.T) {
|
|||
},
|
||||
},
|
||||
})
|
||||
serviceManager := service.NewManager(rtConf.Services, http.DefaultTransport, nil, nil, nil, nil)
|
||||
|
||||
serviceManager := service.NewManager(rtConf.Services, http.DefaultTransport, nil, nil)
|
||||
middlewaresBuilder := middleware.NewBuilder(rtConf.Middlewares, serviceManager)
|
||||
responseModifierFactory := responsemodifiers.NewBuilder(map[string]*runtime.MiddlewareInfo{})
|
||||
routerManager := NewManager(rtConf, serviceManager, middlewaresBuilder, responseModifierFactory)
|
||||
chainBuilder := middleware.NewChainBuilder(static.Configuration{}, nil, nil)
|
||||
|
||||
routerManager := NewManager(rtConf, serviceManager, middlewaresBuilder, responseModifierFactory, chainBuilder)
|
||||
|
||||
_ = routerManager.BuildHandlers(context.Background(), entryPoints, false)
|
||||
|
||||
|
@ -790,6 +801,7 @@ func BenchmarkRouterServe(b *testing.B) {
|
|||
StatusCode: 200,
|
||||
Body: ioutil.NopCloser(strings.NewReader("")),
|
||||
}
|
||||
|
||||
routersConfig := map[string]*dynamic.Router{
|
||||
"foo": {
|
||||
EntryPoints: []string{"web"},
|
||||
|
@ -817,10 +829,13 @@ func BenchmarkRouterServe(b *testing.B) {
|
|||
Middlewares: map[string]*dynamic.Middleware{},
|
||||
},
|
||||
})
|
||||
serviceManager := service.NewManager(rtConf.Services, &staticTransport{res}, nil, nil, nil, nil)
|
||||
|
||||
serviceManager := service.NewManager(rtConf.Services, &staticTransport{res}, nil, nil)
|
||||
middlewaresBuilder := middleware.NewBuilder(rtConf.Middlewares, serviceManager)
|
||||
responseModifierFactory := responsemodifiers.NewBuilder(rtConf.Middlewares)
|
||||
routerManager := NewManager(rtConf, serviceManager, middlewaresBuilder, responseModifierFactory)
|
||||
chainBuilder := middleware.NewChainBuilder(static.Configuration{}, nil, nil)
|
||||
|
||||
routerManager := NewManager(rtConf, serviceManager, middlewaresBuilder, responseModifierFactory, chainBuilder)
|
||||
|
||||
handlers := routerManager.BuildHandlers(context.Background(), entryPoints, false)
|
||||
|
||||
|
@ -857,7 +872,8 @@ func BenchmarkService(b *testing.B) {
|
|||
Services: serviceConfig,
|
||||
},
|
||||
})
|
||||
serviceManager := service.NewManager(rtConf.Services, &staticTransport{res}, nil, nil, nil, nil)
|
||||
|
||||
serviceManager := service.NewManager(rtConf.Services, &staticTransport{res}, nil, nil)
|
||||
w := httptest.NewRecorder()
|
||||
req := testhelpers.MustNewRequest(http.MethodGet, "http://foo.bar/", nil)
|
||||
|
||||
|
|
|
@ -16,6 +16,11 @@ import (
|
|||
traefiktls "github.com/containous/traefik/v2/pkg/tls"
|
||||
)
|
||||
|
||||
const (
|
||||
defaultTLSConfigName = "default"
|
||||
defaultTLSStoreName = "default"
|
||||
)
|
||||
|
||||
// NewManager Creates a new Manager
|
||||
func NewManager(conf *runtime.Configuration,
|
||||
serviceManager *tcpservice.Manager,
|
||||
|
@ -88,15 +93,18 @@ type nameAndConfig struct {
|
|||
func (m *Manager) buildEntryPointHandler(ctx context.Context, configs map[string]*runtime.TCPRouterInfo, configsHTTP map[string]*runtime.RouterInfo, handlerHTTP http.Handler, handlerHTTPS http.Handler) (*tcp.Router, error) {
|
||||
router := &tcp.Router{}
|
||||
router.HTTPHandler(handlerHTTP)
|
||||
const defaultTLSConfigName = "default"
|
||||
|
||||
defaultTLSConf, err := m.tlsManager.Get("default", defaultTLSConfigName)
|
||||
defaultTLSConf, err := m.tlsManager.Get(defaultTLSStoreName, defaultTLSConfigName)
|
||||
if err != nil {
|
||||
log.FromContext(ctx).Errorf("Error during the build of the default TLS configuration: %v", err)
|
||||
}
|
||||
|
||||
router.HTTPSHandler(handlerHTTPS, defaultTLSConf)
|
||||
|
||||
if len(configsHTTP) > 0 {
|
||||
router.AddRouteHTTPTLS("*", defaultTLSConf)
|
||||
}
|
||||
|
||||
// Keyed by domain, then by options reference.
|
||||
tlsOptionsForHostSNI := map[string]map[string]nameAndConfig{}
|
||||
for routerHTTPName, routerHTTPConfig := range configsHTTP {
|
||||
|
@ -126,12 +134,13 @@ func (m *Manager) buildEntryPointHandler(ctx context.Context, configs map[string
|
|||
tlsOptionsName = internal.GetQualifiedName(ctxRouter, routerHTTPConfig.TLS.Options)
|
||||
}
|
||||
|
||||
tlsConf, err := m.tlsManager.Get("default", tlsOptionsName)
|
||||
tlsConf, err := m.tlsManager.Get(defaultTLSStoreName, tlsOptionsName)
|
||||
if err != nil {
|
||||
routerHTTPConfig.AddError(err, true)
|
||||
logger.Debug(err)
|
||||
continue
|
||||
}
|
||||
|
||||
if tlsOptionsForHostSNI[domain] == nil {
|
||||
tlsOptionsForHostSNI[domain] = make(map[string]nameAndConfig)
|
||||
}
|
||||
|
@ -153,7 +162,9 @@ func (m *Manager) buildEntryPointHandler(ctx context.Context, configs map[string
|
|||
config = v.TLSConfig
|
||||
break
|
||||
}
|
||||
|
||||
logger.Debugf("Adding route for %s with TLS options %s", hostSNI, optionsName)
|
||||
|
||||
router.AddRouteHTTPTLS(hostSNI, config)
|
||||
} else {
|
||||
routers := make([]string, 0, len(tlsConfigs))
|
||||
|
@ -161,7 +172,9 @@ func (m *Manager) buildEntryPointHandler(ctx context.Context, configs map[string
|
|||
configsHTTP[v.routerName].AddError(fmt.Errorf("found different TLS options for routers on the same host %v, so using the default TLS options instead", hostSNI), false)
|
||||
routers = append(routers, v.routerName)
|
||||
}
|
||||
|
||||
logger.Warnf("Found different TLS options for routers on the same host %v, so using the default TLS options instead for these routers: %#v", hostSNI, routers)
|
||||
|
||||
router.AddRouteHTTPTLS(hostSNI, defaultTLSConf)
|
||||
}
|
||||
}
|
||||
|
@ -216,7 +229,7 @@ func (m *Manager) buildEntryPointHandler(ctx context.Context, configs map[string
|
|||
tlsOptionsName = internal.GetQualifiedName(ctxRouter, tlsOptionsName)
|
||||
}
|
||||
|
||||
tlsConf, err := m.tlsManager.Get("default", tlsOptionsName)
|
||||
tlsConf, err := m.tlsManager.Get(defaultTLSStoreName, tlsOptionsName)
|
||||
if err != nil {
|
||||
routerConfig.AddError(err, true)
|
||||
logger.Debug(err)
|
||||
|
|
|
@ -2,166 +2,47 @@ package server
|
|||
|
||||
import (
|
||||
"context"
|
||||
"encoding/json"
|
||||
"net/http"
|
||||
"os"
|
||||
"os/signal"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/containous/traefik/v2/pkg/api"
|
||||
"github.com/containous/traefik/v2/pkg/config/dynamic"
|
||||
"github.com/containous/traefik/v2/pkg/config/runtime"
|
||||
"github.com/containous/traefik/v2/pkg/config/static"
|
||||
"github.com/containous/traefik/v2/pkg/log"
|
||||
"github.com/containous/traefik/v2/pkg/metrics"
|
||||
"github.com/containous/traefik/v2/pkg/middlewares/accesslog"
|
||||
"github.com/containous/traefik/v2/pkg/middlewares/requestdecorator"
|
||||
"github.com/containous/traefik/v2/pkg/provider"
|
||||
"github.com/containous/traefik/v2/pkg/safe"
|
||||
"github.com/containous/traefik/v2/pkg/tls"
|
||||
"github.com/containous/traefik/v2/pkg/tracing"
|
||||
"github.com/containous/traefik/v2/pkg/tracing/jaeger"
|
||||
"github.com/containous/traefik/v2/pkg/types"
|
||||
"github.com/containous/traefik/v2/pkg/server/middleware"
|
||||
)
|
||||
|
||||
// Server is the reverse-proxy/load-balancer engine
|
||||
type Server struct {
|
||||
entryPointsTCP TCPEntryPoints
|
||||
configurationChan chan dynamic.Message
|
||||
configurationValidatedChan chan dynamic.Message
|
||||
signals chan os.Signal
|
||||
stopChan chan bool
|
||||
currentConfigurations safe.Safe
|
||||
providerConfigUpdateMap map[string]chan dynamic.Message
|
||||
accessLoggerMiddleware *accesslog.Handler
|
||||
tracer *tracing.Tracing
|
||||
routinesPool *safe.Pool
|
||||
defaultRoundTripper http.RoundTripper
|
||||
metricsRegistry metrics.Registry
|
||||
provider provider.Provider
|
||||
configurationListeners []func(dynamic.Configuration)
|
||||
requestDecorator *requestdecorator.RequestDecorator
|
||||
providersThrottleDuration time.Duration
|
||||
tlsManager *tls.Manager
|
||||
api func(configuration *runtime.Configuration) http.Handler
|
||||
restHandler http.Handler
|
||||
}
|
||||
watcher *ConfigurationWatcher
|
||||
tcpEntryPoints TCPEntryPoints
|
||||
chainBuilder *middleware.ChainBuilder
|
||||
|
||||
// RouteAppenderFactory the route appender factory interface
|
||||
type RouteAppenderFactory interface {
|
||||
NewAppender(ctx context.Context, runtimeConfiguration *runtime.Configuration) types.RouteAppender
|
||||
}
|
||||
accessLoggerMiddleware *accesslog.Handler
|
||||
|
||||
func setupTracing(conf *static.Tracing) tracing.Backend {
|
||||
var backend tracing.Backend
|
||||
signals chan os.Signal
|
||||
stopChan chan bool
|
||||
|
||||
if conf.Jaeger != nil {
|
||||
backend = conf.Jaeger
|
||||
}
|
||||
|
||||
if conf.Zipkin != nil {
|
||||
if backend != nil {
|
||||
log.WithoutContext().Error("Multiple tracing backend are not supported: cannot create Zipkin backend.")
|
||||
} else {
|
||||
backend = conf.Zipkin
|
||||
}
|
||||
}
|
||||
|
||||
if conf.Datadog != nil {
|
||||
if backend != nil {
|
||||
log.WithoutContext().Error("Multiple tracing backend are not supported: cannot create Datadog backend.")
|
||||
} else {
|
||||
backend = conf.Datadog
|
||||
}
|
||||
}
|
||||
|
||||
if conf.Instana != nil {
|
||||
if backend != nil {
|
||||
log.WithoutContext().Error("Multiple tracing backend are not supported: cannot create Instana backend.")
|
||||
} else {
|
||||
backend = conf.Instana
|
||||
}
|
||||
}
|
||||
|
||||
if conf.Haystack != nil {
|
||||
if backend != nil {
|
||||
log.WithoutContext().Error("Multiple tracing backend are not supported: cannot create Haystack backend.")
|
||||
} else {
|
||||
backend = conf.Haystack
|
||||
}
|
||||
}
|
||||
|
||||
if backend == nil {
|
||||
log.WithoutContext().Debug("Could not initialize tracing, use Jaeger by default")
|
||||
bcd := &jaeger.Config{}
|
||||
bcd.SetDefaults()
|
||||
backend = bcd
|
||||
}
|
||||
|
||||
return backend
|
||||
routinesPool *safe.Pool
|
||||
}
|
||||
|
||||
// NewServer returns an initialized Server.
|
||||
func NewServer(staticConfiguration static.Configuration, provider provider.Provider, entryPoints TCPEntryPoints, tlsManager *tls.Manager) *Server {
|
||||
server := &Server{}
|
||||
|
||||
if staticConfiguration.API != nil {
|
||||
server.api = api.NewBuilder(staticConfiguration)
|
||||
func NewServer(routinesPool *safe.Pool, entryPoints TCPEntryPoints, watcher *ConfigurationWatcher,
|
||||
chainBuilder *middleware.ChainBuilder, accessLoggerMiddleware *accesslog.Handler) *Server {
|
||||
srv := &Server{
|
||||
watcher: watcher,
|
||||
tcpEntryPoints: entryPoints,
|
||||
chainBuilder: chainBuilder,
|
||||
accessLoggerMiddleware: accessLoggerMiddleware,
|
||||
signals: make(chan os.Signal, 1),
|
||||
stopChan: make(chan bool, 1),
|
||||
routinesPool: routinesPool,
|
||||
}
|
||||
|
||||
if staticConfiguration.Providers != nil && staticConfiguration.Providers.Rest != nil {
|
||||
server.restHandler = staticConfiguration.Providers.Rest.Handler()
|
||||
}
|
||||
srv.configureSignals()
|
||||
|
||||
server.provider = provider
|
||||
server.entryPointsTCP = entryPoints
|
||||
server.configurationChan = make(chan dynamic.Message, 100)
|
||||
server.configurationValidatedChan = make(chan dynamic.Message, 100)
|
||||
server.signals = make(chan os.Signal, 1)
|
||||
server.stopChan = make(chan bool, 1)
|
||||
server.configureSignals()
|
||||
currentConfigurations := make(dynamic.Configurations)
|
||||
server.currentConfigurations.Set(currentConfigurations)
|
||||
server.providerConfigUpdateMap = make(map[string]chan dynamic.Message)
|
||||
server.tlsManager = tlsManager
|
||||
|
||||
if staticConfiguration.Providers != nil {
|
||||
server.providersThrottleDuration = time.Duration(staticConfiguration.Providers.ProvidersThrottleDuration)
|
||||
}
|
||||
|
||||
transport, err := createHTTPTransport(staticConfiguration.ServersTransport)
|
||||
if err != nil {
|
||||
log.WithoutContext().Errorf("Could not configure HTTP Transport, fallbacking on default transport: %v", err)
|
||||
server.defaultRoundTripper = http.DefaultTransport
|
||||
} else {
|
||||
server.defaultRoundTripper = transport
|
||||
}
|
||||
|
||||
server.routinesPool = safe.NewPool(context.Background())
|
||||
|
||||
if staticConfiguration.Tracing != nil {
|
||||
tracingBackend := setupTracing(staticConfiguration.Tracing)
|
||||
if tracingBackend != nil {
|
||||
server.tracer, err = tracing.NewTracing(staticConfiguration.Tracing.ServiceName, staticConfiguration.Tracing.SpanNameLimit, tracingBackend)
|
||||
if err != nil {
|
||||
log.WithoutContext().Warnf("Unable to create tracer: %v", err)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
server.requestDecorator = requestdecorator.New(staticConfiguration.HostResolver)
|
||||
|
||||
server.metricsRegistry = registerMetricClients(staticConfiguration.Metrics)
|
||||
|
||||
if staticConfiguration.AccessLog != nil {
|
||||
var err error
|
||||
server.accessLoggerMiddleware, err = accesslog.NewHandler(staticConfiguration.AccessLog)
|
||||
if err != nil {
|
||||
log.WithoutContext().Warnf("Unable to create access logger : %v", err)
|
||||
}
|
||||
}
|
||||
return server
|
||||
return srv
|
||||
}
|
||||
|
||||
// Start starts the server and Stop/Close it when context is Done
|
||||
|
@ -175,20 +56,15 @@ func (s *Server) Start(ctx context.Context) {
|
|||
s.Stop()
|
||||
}()
|
||||
|
||||
s.startTCPServers()
|
||||
s.routinesPool.Go(func(stop chan bool) {
|
||||
s.listenProviders(stop)
|
||||
})
|
||||
s.routinesPool.Go(func(stop chan bool) {
|
||||
s.listenConfigurations(stop)
|
||||
})
|
||||
s.startProvider()
|
||||
s.tcpEntryPoints.Start()
|
||||
s.watcher.Start()
|
||||
|
||||
s.routinesPool.Go(func(stop chan bool) {
|
||||
s.listenSignals(stop)
|
||||
})
|
||||
}
|
||||
|
||||
// Wait blocks until server is shutted down.
|
||||
// Wait blocks until the server shutdown.
|
||||
func (s *Server) Wait() {
|
||||
<-s.stopChan
|
||||
}
|
||||
|
@ -197,25 +73,15 @@ func (s *Server) Wait() {
|
|||
func (s *Server) Stop() {
|
||||
defer log.WithoutContext().Info("Server stopped")
|
||||
|
||||
var wg sync.WaitGroup
|
||||
for epn, ep := range s.entryPointsTCP {
|
||||
wg.Add(1)
|
||||
go func(entryPointName string, entryPoint *TCPEntryPoint) {
|
||||
ctx := log.With(context.Background(), log.Str(log.EntryPointName, entryPointName))
|
||||
defer wg.Done()
|
||||
s.tcpEntryPoints.Stop()
|
||||
|
||||
entryPoint.Shutdown(ctx)
|
||||
|
||||
log.FromContext(ctx).Debugf("Entry point %s closed", entryPointName)
|
||||
}(epn, ep)
|
||||
}
|
||||
wg.Wait()
|
||||
s.stopChan <- true
|
||||
}
|
||||
|
||||
// Close destroys the server
|
||||
func (s *Server) Close() {
|
||||
ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second)
|
||||
|
||||
go func(ctx context.Context) {
|
||||
<-ctx.Done()
|
||||
if ctx.Err() == context.Canceled {
|
||||
|
@ -226,125 +92,19 @@ func (s *Server) Close() {
|
|||
}(ctx)
|
||||
|
||||
stopMetricsClients()
|
||||
|
||||
s.routinesPool.Cleanup()
|
||||
close(s.configurationChan)
|
||||
close(s.configurationValidatedChan)
|
||||
|
||||
signal.Stop(s.signals)
|
||||
close(s.signals)
|
||||
|
||||
close(s.stopChan)
|
||||
|
||||
if s.accessLoggerMiddleware != nil {
|
||||
if err := s.accessLoggerMiddleware.Close(); err != nil {
|
||||
log.WithoutContext().Errorf("Could not close the access log file: %s", err)
|
||||
}
|
||||
}
|
||||
|
||||
if s.tracer != nil {
|
||||
s.tracer.Close()
|
||||
}
|
||||
s.chainBuilder.Close()
|
||||
|
||||
cancel()
|
||||
}
|
||||
|
||||
func (s *Server) startTCPServers() {
|
||||
// Use an empty configuration in order to initialize the default handlers with internal routes
|
||||
routers := s.loadConfigurationTCP(dynamic.Configurations{})
|
||||
for entryPointName, router := range routers {
|
||||
s.entryPointsTCP[entryPointName].switchRouter(router)
|
||||
}
|
||||
|
||||
for entryPointName, serverEntryPoint := range s.entryPointsTCP {
|
||||
ctx := log.With(context.Background(), log.Str(log.EntryPointName, entryPointName))
|
||||
go serverEntryPoint.startTCP(ctx)
|
||||
}
|
||||
}
|
||||
|
||||
func (s *Server) listenProviders(stop chan bool) {
|
||||
for {
|
||||
select {
|
||||
case <-stop:
|
||||
return
|
||||
case configMsg, ok := <-s.configurationChan:
|
||||
if !ok {
|
||||
return
|
||||
}
|
||||
if configMsg.Configuration != nil {
|
||||
s.preLoadConfiguration(configMsg)
|
||||
} else {
|
||||
log.WithoutContext().WithField(log.ProviderName, configMsg.ProviderName).
|
||||
Debug("Received nil configuration from provider, skipping.")
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// AddListener adds a new listener function used when new configuration is provided
|
||||
func (s *Server) AddListener(listener func(dynamic.Configuration)) {
|
||||
if s.configurationListeners == nil {
|
||||
s.configurationListeners = make([]func(dynamic.Configuration), 0)
|
||||
}
|
||||
s.configurationListeners = append(s.configurationListeners, listener)
|
||||
}
|
||||
|
||||
func (s *Server) startProvider() {
|
||||
logger := log.WithoutContext()
|
||||
|
||||
jsonConf, err := json.Marshal(s.provider)
|
||||
if err != nil {
|
||||
logger.Debugf("Unable to marshal provider configuration %T: %v", s.provider, err)
|
||||
}
|
||||
|
||||
logger.Infof("Starting provider %T %s", s.provider, jsonConf)
|
||||
currentProvider := s.provider
|
||||
|
||||
safe.Go(func() {
|
||||
err := currentProvider.Provide(s.configurationChan, s.routinesPool)
|
||||
if err != nil {
|
||||
logger.Errorf("Error starting provider %T: %s", s.provider, err)
|
||||
}
|
||||
})
|
||||
}
|
||||
|
||||
func registerMetricClients(metricsConfig *types.Metrics) metrics.Registry {
|
||||
if metricsConfig == nil {
|
||||
return metrics.NewVoidRegistry()
|
||||
}
|
||||
|
||||
var registries []metrics.Registry
|
||||
|
||||
if metricsConfig.Prometheus != nil {
|
||||
ctx := log.With(context.Background(), log.Str(log.MetricsProviderName, "prometheus"))
|
||||
prometheusRegister := metrics.RegisterPrometheus(ctx, metricsConfig.Prometheus)
|
||||
if prometheusRegister != nil {
|
||||
registries = append(registries, prometheusRegister)
|
||||
log.FromContext(ctx).Debug("Configured Prometheus metrics")
|
||||
}
|
||||
}
|
||||
|
||||
if metricsConfig.Datadog != nil {
|
||||
ctx := log.With(context.Background(), log.Str(log.MetricsProviderName, "datadog"))
|
||||
registries = append(registries, metrics.RegisterDatadog(ctx, metricsConfig.Datadog))
|
||||
log.FromContext(ctx).Debugf("Configured Datadog metrics: pushing to %s once every %s",
|
||||
metricsConfig.Datadog.Address, metricsConfig.Datadog.PushInterval)
|
||||
}
|
||||
|
||||
if metricsConfig.StatsD != nil {
|
||||
ctx := log.With(context.Background(), log.Str(log.MetricsProviderName, "statsd"))
|
||||
registries = append(registries, metrics.RegisterStatsd(ctx, metricsConfig.StatsD))
|
||||
log.FromContext(ctx).Debugf("Configured StatsD metrics: pushing to %s once every %s",
|
||||
metricsConfig.StatsD.Address, metricsConfig.StatsD.PushInterval)
|
||||
}
|
||||
|
||||
if metricsConfig.InfluxDB != nil {
|
||||
ctx := log.With(context.Background(), log.Str(log.MetricsProviderName, "influxdb"))
|
||||
registries = append(registries, metrics.RegisterInfluxDB(ctx, metricsConfig.InfluxDB))
|
||||
log.FromContext(ctx).Debugf("Configured InfluxDB metrics: pushing to %s once every %s",
|
||||
metricsConfig.InfluxDB.Address, metricsConfig.InfluxDB.PushInterval)
|
||||
}
|
||||
|
||||
return metrics.NewMultiRegistry(registries)
|
||||
}
|
||||
|
||||
func stopMetricsClients() {
|
||||
metrics.StopDatadog()
|
||||
metrics.StopStatsd()
|
||||
|
|
|
@ -1,291 +0,0 @@
|
|||
package server
|
||||
|
||||
import (
|
||||
"context"
|
||||
"encoding/json"
|
||||
"net/http"
|
||||
"reflect"
|
||||
"time"
|
||||
|
||||
"github.com/containous/alice"
|
||||
"github.com/containous/traefik/v2/pkg/config/dynamic"
|
||||
"github.com/containous/traefik/v2/pkg/config/runtime"
|
||||
"github.com/containous/traefik/v2/pkg/log"
|
||||
"github.com/containous/traefik/v2/pkg/metrics"
|
||||
"github.com/containous/traefik/v2/pkg/middlewares/accesslog"
|
||||
metricsmiddleware "github.com/containous/traefik/v2/pkg/middlewares/metrics"
|
||||
"github.com/containous/traefik/v2/pkg/middlewares/requestdecorator"
|
||||
"github.com/containous/traefik/v2/pkg/middlewares/tracing"
|
||||
"github.com/containous/traefik/v2/pkg/responsemodifiers"
|
||||
"github.com/containous/traefik/v2/pkg/server/middleware"
|
||||
"github.com/containous/traefik/v2/pkg/server/router"
|
||||
routertcp "github.com/containous/traefik/v2/pkg/server/router/tcp"
|
||||
"github.com/containous/traefik/v2/pkg/server/service"
|
||||
"github.com/containous/traefik/v2/pkg/server/service/tcp"
|
||||
tcpCore "github.com/containous/traefik/v2/pkg/tcp"
|
||||
"github.com/eapache/channels"
|
||||
"github.com/gorilla/mux"
|
||||
"github.com/sirupsen/logrus"
|
||||
)
|
||||
|
||||
// loadConfiguration manages dynamically routers, middlewares, servers and TLS configurations
|
||||
func (s *Server) loadConfiguration(configMsg dynamic.Message) {
|
||||
currentConfigurations := s.currentConfigurations.Get().(dynamic.Configurations)
|
||||
|
||||
// Copy configurations to new map so we don't change current if LoadConfig fails
|
||||
newConfigurations := currentConfigurations.DeepCopy()
|
||||
newConfigurations[configMsg.ProviderName] = configMsg.Configuration
|
||||
|
||||
s.metricsRegistry.ConfigReloadsCounter().Add(1)
|
||||
|
||||
handlersTCP := s.loadConfigurationTCP(newConfigurations)
|
||||
for entryPointName, router := range handlersTCP {
|
||||
s.entryPointsTCP[entryPointName].switchRouter(router)
|
||||
}
|
||||
|
||||
s.metricsRegistry.LastConfigReloadSuccessGauge().Set(float64(time.Now().Unix()))
|
||||
|
||||
s.currentConfigurations.Set(newConfigurations)
|
||||
|
||||
for _, listener := range s.configurationListeners {
|
||||
listener(*configMsg.Configuration)
|
||||
}
|
||||
|
||||
if s.metricsRegistry.IsEpEnabled() || s.metricsRegistry.IsSvcEnabled() {
|
||||
var entrypoints []string
|
||||
for key := range s.entryPointsTCP {
|
||||
entrypoints = append(entrypoints, key)
|
||||
}
|
||||
metrics.OnConfigurationUpdate(newConfigurations, entrypoints)
|
||||
}
|
||||
}
|
||||
|
||||
// loadConfigurationTCP returns a new gorilla.mux Route from the specified global configuration and the dynamic
|
||||
// provider configurations.
|
||||
func (s *Server) loadConfigurationTCP(configurations dynamic.Configurations) map[string]*tcpCore.Router {
|
||||
ctx := context.Background()
|
||||
|
||||
var entryPoints []string
|
||||
for entryPointName := range s.entryPointsTCP {
|
||||
entryPoints = append(entryPoints, entryPointName)
|
||||
}
|
||||
|
||||
conf := mergeConfiguration(configurations)
|
||||
|
||||
s.tlsManager.UpdateConfigs(ctx, conf.TLS.Stores, conf.TLS.Options, conf.TLS.Certificates)
|
||||
|
||||
rtConf := runtime.NewConfig(conf)
|
||||
handlersNonTLS, handlersTLS := s.createHTTPHandlers(ctx, rtConf, entryPoints)
|
||||
routersTCP := s.createTCPRouters(ctx, rtConf, entryPoints, handlersNonTLS, handlersTLS)
|
||||
rtConf.PopulateUsedBy()
|
||||
|
||||
return routersTCP
|
||||
}
|
||||
|
||||
// the given configuration must not be nil. its fields will get mutated.
|
||||
func (s *Server) createTCPRouters(ctx context.Context, configuration *runtime.Configuration, entryPoints []string, handlers map[string]http.Handler, handlersTLS map[string]http.Handler) map[string]*tcpCore.Router {
|
||||
if configuration == nil {
|
||||
return make(map[string]*tcpCore.Router)
|
||||
}
|
||||
|
||||
serviceManager := tcp.NewManager(configuration)
|
||||
|
||||
routerManager := routertcp.NewManager(configuration, serviceManager, handlers, handlersTLS, s.tlsManager)
|
||||
|
||||
return routerManager.BuildHandlers(ctx, entryPoints)
|
||||
}
|
||||
|
||||
// createHTTPHandlers returns, for the given configuration and entryPoints, the HTTP handlers for non-TLS connections, and for the TLS ones. the given configuration must not be nil. its fields will get mutated.
|
||||
func (s *Server) createHTTPHandlers(ctx context.Context, configuration *runtime.Configuration, entryPoints []string) (map[string]http.Handler, map[string]http.Handler) {
|
||||
var apiHandler http.Handler
|
||||
if s.api != nil {
|
||||
apiHandler = s.api(configuration)
|
||||
}
|
||||
|
||||
serviceManager := service.NewManager(configuration.Services, s.defaultRoundTripper, s.metricsRegistry, s.routinesPool, apiHandler, s.restHandler)
|
||||
middlewaresBuilder := middleware.NewBuilder(configuration.Middlewares, serviceManager)
|
||||
responseModifierFactory := responsemodifiers.NewBuilder(configuration.Middlewares)
|
||||
routerManager := router.NewManager(configuration, serviceManager, middlewaresBuilder, responseModifierFactory)
|
||||
|
||||
handlersNonTLS := routerManager.BuildHandlers(ctx, entryPoints, false)
|
||||
handlersTLS := routerManager.BuildHandlers(ctx, entryPoints, true)
|
||||
|
||||
routerHandlers := make(map[string]http.Handler)
|
||||
for _, entryPointName := range entryPoints {
|
||||
internalMuxRouter := mux.NewRouter().SkipClean(true)
|
||||
|
||||
ctx = log.With(ctx, log.Str(log.EntryPointName, entryPointName))
|
||||
|
||||
factory := s.entryPointsTCP[entryPointName].RouteAppenderFactory
|
||||
if factory != nil {
|
||||
// FIXME remove currentConfigurations
|
||||
appender := factory.NewAppender(ctx, configuration)
|
||||
appender.Append(internalMuxRouter)
|
||||
}
|
||||
|
||||
if h, ok := handlersNonTLS[entryPointName]; ok {
|
||||
internalMuxRouter.NotFoundHandler = h
|
||||
} else {
|
||||
internalMuxRouter.NotFoundHandler = buildDefaultHTTPRouter()
|
||||
}
|
||||
|
||||
routerHandlers[entryPointName] = internalMuxRouter
|
||||
|
||||
chain := alice.New()
|
||||
|
||||
if s.accessLoggerMiddleware != nil {
|
||||
chain = chain.Append(accesslog.WrapHandler(s.accessLoggerMiddleware))
|
||||
}
|
||||
|
||||
if s.tracer != nil {
|
||||
chain = chain.Append(tracing.WrapEntryPointHandler(ctx, s.tracer, entryPointName))
|
||||
}
|
||||
|
||||
if s.metricsRegistry.IsEpEnabled() {
|
||||
chain = chain.Append(metricsmiddleware.WrapEntryPointHandler(ctx, s.metricsRegistry, entryPointName))
|
||||
}
|
||||
|
||||
chain = chain.Append(requestdecorator.WrapHandler(s.requestDecorator))
|
||||
|
||||
handler, err := chain.Then(internalMuxRouter.NotFoundHandler)
|
||||
if err != nil {
|
||||
log.FromContext(ctx).Error(err)
|
||||
continue
|
||||
}
|
||||
internalMuxRouter.NotFoundHandler = handler
|
||||
|
||||
handlerTLS, ok := handlersTLS[entryPointName]
|
||||
if ok {
|
||||
handlerTLSWithMiddlewares, err := chain.Then(handlerTLS)
|
||||
if err != nil {
|
||||
log.FromContext(ctx).Error(err)
|
||||
continue
|
||||
}
|
||||
handlersTLS[entryPointName] = handlerTLSWithMiddlewares
|
||||
}
|
||||
}
|
||||
|
||||
return routerHandlers, handlersTLS
|
||||
}
|
||||
|
||||
func isEmptyConfiguration(conf *dynamic.Configuration) bool {
|
||||
if conf == nil {
|
||||
return true
|
||||
}
|
||||
if conf.TCP == nil {
|
||||
conf.TCP = &dynamic.TCPConfiguration{}
|
||||
}
|
||||
if conf.HTTP == nil {
|
||||
conf.HTTP = &dynamic.HTTPConfiguration{}
|
||||
}
|
||||
|
||||
return conf.HTTP.Routers == nil &&
|
||||
conf.HTTP.Services == nil &&
|
||||
conf.HTTP.Middlewares == nil &&
|
||||
(conf.TLS == nil || conf.TLS.Certificates == nil && conf.TLS.Stores == nil && conf.TLS.Options == nil) &&
|
||||
conf.TCP.Routers == nil &&
|
||||
conf.TCP.Services == nil
|
||||
}
|
||||
|
||||
func (s *Server) preLoadConfiguration(configMsg dynamic.Message) {
|
||||
s.defaultConfigurationValues(configMsg.Configuration.HTTP)
|
||||
currentConfigurations := s.currentConfigurations.Get().(dynamic.Configurations)
|
||||
|
||||
logger := log.WithoutContext().WithField(log.ProviderName, configMsg.ProviderName)
|
||||
if log.GetLevel() == logrus.DebugLevel {
|
||||
copyConf := configMsg.Configuration.DeepCopy()
|
||||
if copyConf.TLS != nil {
|
||||
copyConf.TLS.Certificates = nil
|
||||
|
||||
for _, v := range copyConf.TLS.Stores {
|
||||
v.DefaultCertificate = nil
|
||||
}
|
||||
}
|
||||
|
||||
jsonConf, err := json.Marshal(copyConf)
|
||||
if err != nil {
|
||||
logger.Errorf("Could not marshal dynamic configuration: %v", err)
|
||||
logger.Debugf("Configuration received from provider %s: [struct] %#v", configMsg.ProviderName, copyConf)
|
||||
} else {
|
||||
logger.Debugf("Configuration received from provider %s: %s", configMsg.ProviderName, string(jsonConf))
|
||||
}
|
||||
}
|
||||
|
||||
if isEmptyConfiguration(configMsg.Configuration) {
|
||||
logger.Infof("Skipping empty Configuration for provider %s", configMsg.ProviderName)
|
||||
return
|
||||
}
|
||||
|
||||
if reflect.DeepEqual(currentConfigurations[configMsg.ProviderName], configMsg.Configuration) {
|
||||
logger.Infof("Skipping same configuration for provider %s", configMsg.ProviderName)
|
||||
return
|
||||
}
|
||||
|
||||
providerConfigUpdateCh, ok := s.providerConfigUpdateMap[configMsg.ProviderName]
|
||||
if !ok {
|
||||
providerConfigUpdateCh = make(chan dynamic.Message)
|
||||
s.providerConfigUpdateMap[configMsg.ProviderName] = providerConfigUpdateCh
|
||||
s.routinesPool.Go(func(stop chan bool) {
|
||||
s.throttleProviderConfigReload(s.providersThrottleDuration, s.configurationValidatedChan, providerConfigUpdateCh, stop)
|
||||
})
|
||||
}
|
||||
|
||||
providerConfigUpdateCh <- configMsg
|
||||
}
|
||||
|
||||
func (s *Server) defaultConfigurationValues(configuration *dynamic.HTTPConfiguration) {
|
||||
// FIXME create a config hook
|
||||
}
|
||||
|
||||
func (s *Server) listenConfigurations(stop chan bool) {
|
||||
for {
|
||||
select {
|
||||
case <-stop:
|
||||
return
|
||||
case configMsg, ok := <-s.configurationValidatedChan:
|
||||
if !ok || configMsg.Configuration == nil {
|
||||
return
|
||||
}
|
||||
s.loadConfiguration(configMsg)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// throttleProviderConfigReload throttles the configuration reload speed for a single provider.
|
||||
// It will immediately publish a new configuration and then only publish the next configuration after the throttle duration.
|
||||
// Note that in the case it receives N new configs in the timeframe of the throttle duration after publishing,
|
||||
// it will publish the last of the newly received configurations.
|
||||
func (s *Server) throttleProviderConfigReload(throttle time.Duration, publish chan<- dynamic.Message, in <-chan dynamic.Message, stop chan bool) {
|
||||
ring := channels.NewRingChannel(1)
|
||||
defer ring.Close()
|
||||
|
||||
s.routinesPool.Go(func(stop chan bool) {
|
||||
for {
|
||||
select {
|
||||
case <-stop:
|
||||
return
|
||||
case nextConfig := <-ring.Out():
|
||||
if config, ok := nextConfig.(dynamic.Message); ok {
|
||||
publish <- config
|
||||
time.Sleep(throttle)
|
||||
}
|
||||
}
|
||||
}
|
||||
})
|
||||
|
||||
for {
|
||||
select {
|
||||
case <-stop:
|
||||
return
|
||||
case nextConfig := <-in:
|
||||
ring.In() <- nextConfig
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func buildDefaultHTTPRouter() *mux.Router {
|
||||
rt := mux.NewRouter()
|
||||
rt.NotFoundHandler = http.HandlerFunc(http.NotFound)
|
||||
rt.SkipClean(true)
|
||||
return rt
|
||||
}
|
|
@ -1,122 +0,0 @@
|
|||
package server
|
||||
|
||||
import (
|
||||
"context"
|
||||
"net/http"
|
||||
"net/http/httptest"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/containous/traefik/v2/pkg/config/dynamic"
|
||||
"github.com/containous/traefik/v2/pkg/config/runtime"
|
||||
"github.com/containous/traefik/v2/pkg/config/static"
|
||||
th "github.com/containous/traefik/v2/pkg/testhelpers"
|
||||
"github.com/stretchr/testify/assert"
|
||||
)
|
||||
|
||||
func TestReuseService(t *testing.T) {
|
||||
testServer := httptest.NewServer(http.HandlerFunc(func(rw http.ResponseWriter, req *http.Request) {
|
||||
rw.WriteHeader(http.StatusOK)
|
||||
}))
|
||||
defer testServer.Close()
|
||||
|
||||
entryPoints := TCPEntryPoints{
|
||||
"http": &TCPEntryPoint{},
|
||||
}
|
||||
|
||||
staticConfig := static.Configuration{}
|
||||
|
||||
dynamicConfigs := th.BuildConfiguration(
|
||||
th.WithRouters(
|
||||
th.WithRouter("foo",
|
||||
th.WithServiceName("bar"),
|
||||
th.WithRule("Path(`/ok`)")),
|
||||
th.WithRouter("foo2",
|
||||
th.WithEntryPoints("http"),
|
||||
th.WithRule("Path(`/unauthorized`)"),
|
||||
th.WithServiceName("bar"),
|
||||
th.WithRouterMiddlewares("basicauth")),
|
||||
),
|
||||
th.WithMiddlewares(th.WithMiddleware("basicauth",
|
||||
th.WithBasicAuth(&dynamic.BasicAuth{Users: []string{"foo:bar"}}),
|
||||
)),
|
||||
th.WithLoadBalancerServices(th.WithService("bar",
|
||||
th.WithServers(th.WithServer(testServer.URL))),
|
||||
),
|
||||
)
|
||||
|
||||
srv := NewServer(staticConfig, nil, entryPoints, nil)
|
||||
|
||||
rtConf := runtime.NewConfig(dynamic.Configuration{HTTP: dynamicConfigs})
|
||||
entrypointsHandlers, _ := srv.createHTTPHandlers(context.Background(), rtConf, []string{"http"})
|
||||
|
||||
// Test that the /ok path returns a status 200.
|
||||
responseRecorderOk := &httptest.ResponseRecorder{}
|
||||
requestOk := httptest.NewRequest(http.MethodGet, testServer.URL+"/ok", nil)
|
||||
entrypointsHandlers["http"].ServeHTTP(responseRecorderOk, requestOk)
|
||||
|
||||
assert.Equal(t, http.StatusOK, responseRecorderOk.Result().StatusCode, "status code")
|
||||
|
||||
// Test that the /unauthorized path returns a 401 because of
|
||||
// the basic authentication defined on the frontend.
|
||||
responseRecorderUnauthorized := &httptest.ResponseRecorder{}
|
||||
requestUnauthorized := httptest.NewRequest(http.MethodGet, testServer.URL+"/unauthorized", nil)
|
||||
entrypointsHandlers["http"].ServeHTTP(responseRecorderUnauthorized, requestUnauthorized)
|
||||
|
||||
assert.Equal(t, http.StatusUnauthorized, responseRecorderUnauthorized.Result().StatusCode, "status code")
|
||||
}
|
||||
|
||||
func TestThrottleProviderConfigReload(t *testing.T) {
|
||||
throttleDuration := 30 * time.Millisecond
|
||||
publishConfig := make(chan dynamic.Message)
|
||||
providerConfig := make(chan dynamic.Message)
|
||||
stop := make(chan bool)
|
||||
defer func() {
|
||||
stop <- true
|
||||
}()
|
||||
|
||||
staticConfiguration := static.Configuration{}
|
||||
server := NewServer(staticConfiguration, nil, nil, nil)
|
||||
|
||||
go server.throttleProviderConfigReload(throttleDuration, publishConfig, providerConfig, stop)
|
||||
|
||||
publishedConfigCount := 0
|
||||
stopConsumeConfigs := make(chan bool)
|
||||
go func() {
|
||||
for {
|
||||
select {
|
||||
case <-stop:
|
||||
return
|
||||
case <-stopConsumeConfigs:
|
||||
return
|
||||
case <-publishConfig:
|
||||
publishedConfigCount++
|
||||
}
|
||||
}
|
||||
}()
|
||||
|
||||
// publish 5 new configs, one new config each 10 milliseconds
|
||||
for i := 0; i < 5; i++ {
|
||||
providerConfig <- dynamic.Message{}
|
||||
time.Sleep(10 * time.Millisecond)
|
||||
}
|
||||
|
||||
// after 50 milliseconds 5 new configs were published
|
||||
// with a throttle duration of 30 milliseconds this means, we should have received 2 new configs
|
||||
assert.Equal(t, 2, publishedConfigCount, "times configs were published")
|
||||
|
||||
stopConsumeConfigs <- true
|
||||
|
||||
select {
|
||||
case <-publishConfig:
|
||||
// There should be exactly one more message that we receive after ~60 milliseconds since the start of the test.
|
||||
select {
|
||||
case <-publishConfig:
|
||||
t.Error("extra config publication found")
|
||||
case <-time.After(100 * time.Millisecond):
|
||||
return
|
||||
}
|
||||
case <-time.After(100 * time.Millisecond):
|
||||
t.Error("Last config was not published in time")
|
||||
}
|
||||
}
|
|
@ -16,6 +16,7 @@ import (
|
|||
"github.com/containous/traefik/v2/pkg/middlewares"
|
||||
"github.com/containous/traefik/v2/pkg/middlewares/forwardedheaders"
|
||||
"github.com/containous/traefik/v2/pkg/safe"
|
||||
"github.com/containous/traefik/v2/pkg/server/router"
|
||||
"github.com/containous/traefik/v2/pkg/tcp"
|
||||
"github.com/sirupsen/logrus"
|
||||
"golang.org/x/net/http2"
|
||||
|
@ -50,11 +51,60 @@ func (h *httpForwarder) Accept() (net.Conn, error) {
|
|||
// TCPEntryPoints holds a map of TCPEntryPoint (the entrypoint names being the keys)
|
||||
type TCPEntryPoints map[string]*TCPEntryPoint
|
||||
|
||||
// NewTCPEntryPoints creates a new TCPEntryPoints.
|
||||
func NewTCPEntryPoints(staticConfiguration static.Configuration) (TCPEntryPoints, error) {
|
||||
serverEntryPointsTCP := make(TCPEntryPoints)
|
||||
for entryPointName, config := range staticConfiguration.EntryPoints {
|
||||
ctx := log.With(context.Background(), log.Str(log.EntryPointName, entryPointName))
|
||||
|
||||
var err error
|
||||
serverEntryPointsTCP[entryPointName], err = NewTCPEntryPoint(ctx, config)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("error while building entryPoint %s: %v", entryPointName, err)
|
||||
}
|
||||
}
|
||||
return serverEntryPointsTCP, nil
|
||||
}
|
||||
|
||||
// Start the server entry points.
|
||||
func (eps TCPEntryPoints) Start() {
|
||||
for entryPointName, serverEntryPoint := range eps {
|
||||
ctx := log.With(context.Background(), log.Str(log.EntryPointName, entryPointName))
|
||||
go serverEntryPoint.StartTCP(ctx)
|
||||
}
|
||||
}
|
||||
|
||||
// Stop the server entry points.
|
||||
func (eps TCPEntryPoints) Stop() {
|
||||
var wg sync.WaitGroup
|
||||
|
||||
for epn, ep := range eps {
|
||||
wg.Add(1)
|
||||
|
||||
go func(entryPointName string, entryPoint *TCPEntryPoint) {
|
||||
defer wg.Done()
|
||||
|
||||
ctx := log.With(context.Background(), log.Str(log.EntryPointName, entryPointName))
|
||||
entryPoint.Shutdown(ctx)
|
||||
|
||||
log.FromContext(ctx).Debugf("Entry point %s closed", entryPointName)
|
||||
}(epn, ep)
|
||||
}
|
||||
|
||||
wg.Wait()
|
||||
}
|
||||
|
||||
// Switch the TCP routers.
|
||||
func (eps TCPEntryPoints) Switch(routersTCP map[string]*tcp.Router) {
|
||||
for entryPointName, rt := range routersTCP {
|
||||
eps[entryPointName].SwitchRouter(rt)
|
||||
}
|
||||
}
|
||||
|
||||
// TCPEntryPoint is the TCP server
|
||||
type TCPEntryPoint struct {
|
||||
listener net.Listener
|
||||
switcher *tcp.HandlerSwitcher
|
||||
RouteAppenderFactory RouteAppenderFactory
|
||||
transportConfiguration *static.EntryPointsTransport
|
||||
tracker *connectionTracker
|
||||
httpServer *httpServer
|
||||
|
@ -99,35 +149,8 @@ func NewTCPEntryPoint(ctx context.Context, configuration *static.EntryPoint) (*T
|
|||
}, nil
|
||||
}
|
||||
|
||||
// writeCloserWrapper wraps together a connection, and the concrete underlying
|
||||
// connection type that was found to satisfy WriteCloser.
|
||||
type writeCloserWrapper struct {
|
||||
net.Conn
|
||||
writeCloser tcp.WriteCloser
|
||||
}
|
||||
|
||||
func (c *writeCloserWrapper) CloseWrite() error {
|
||||
return c.writeCloser.CloseWrite()
|
||||
}
|
||||
|
||||
// writeCloser returns the given connection, augmented with the WriteCloser
|
||||
// implementation, if any was found within the underlying conn.
|
||||
func writeCloser(conn net.Conn) (tcp.WriteCloser, error) {
|
||||
switch typedConn := conn.(type) {
|
||||
case *proxyprotocol.Conn:
|
||||
underlying, err := writeCloser(typedConn.Conn)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return &writeCloserWrapper{writeCloser: underlying, Conn: typedConn}, nil
|
||||
case *net.TCPConn:
|
||||
return typedConn, nil
|
||||
default:
|
||||
return nil, fmt.Errorf("unknown connection type %T", typedConn)
|
||||
}
|
||||
}
|
||||
|
||||
func (e *TCPEntryPoint) startTCP(ctx context.Context) {
|
||||
// StartTCP starts the TCP server.
|
||||
func (e *TCPEntryPoint) StartTCP(ctx context.Context) {
|
||||
logger := log.FromContext(ctx)
|
||||
logger.Debugf("Start TCP Server")
|
||||
|
||||
|
@ -213,22 +236,55 @@ func (e *TCPEntryPoint) Shutdown(ctx context.Context) {
|
|||
cancel()
|
||||
}
|
||||
|
||||
func (e *TCPEntryPoint) switchRouter(router *tcp.Router) {
|
||||
router.HTTPForwarder(e.httpServer.Forwarder)
|
||||
router.HTTPSForwarder(e.httpsServer.Forwarder)
|
||||
// SwitchRouter switches the TCP router handler.
|
||||
func (e *TCPEntryPoint) SwitchRouter(rt *tcp.Router) {
|
||||
rt.HTTPForwarder(e.httpServer.Forwarder)
|
||||
|
||||
httpHandler := router.GetHTTPHandler()
|
||||
httpsHandler := router.GetHTTPSHandler()
|
||||
httpHandler := rt.GetHTTPHandler()
|
||||
if httpHandler == nil {
|
||||
httpHandler = buildDefaultHTTPRouter()
|
||||
}
|
||||
if httpsHandler == nil {
|
||||
httpsHandler = buildDefaultHTTPRouter()
|
||||
httpHandler = router.BuildDefaultHTTPRouter()
|
||||
}
|
||||
|
||||
e.httpServer.Switcher.UpdateHandler(httpHandler)
|
||||
|
||||
rt.HTTPSForwarder(e.httpsServer.Forwarder)
|
||||
|
||||
httpsHandler := rt.GetHTTPSHandler()
|
||||
if httpsHandler == nil {
|
||||
httpsHandler = router.BuildDefaultHTTPRouter()
|
||||
}
|
||||
|
||||
e.httpsServer.Switcher.UpdateHandler(httpsHandler)
|
||||
e.switcher.Switch(router)
|
||||
|
||||
e.switcher.Switch(rt)
|
||||
}
|
||||
|
||||
// writeCloserWrapper wraps together a connection, and the concrete underlying
|
||||
// connection type that was found to satisfy WriteCloser.
|
||||
type writeCloserWrapper struct {
|
||||
net.Conn
|
||||
writeCloser tcp.WriteCloser
|
||||
}
|
||||
|
||||
func (c *writeCloserWrapper) CloseWrite() error {
|
||||
return c.writeCloser.CloseWrite()
|
||||
}
|
||||
|
||||
// writeCloser returns the given connection, augmented with the WriteCloser
|
||||
// implementation, if any was found within the underlying conn.
|
||||
func writeCloser(conn net.Conn) (tcp.WriteCloser, error) {
|
||||
switch typedConn := conn.(type) {
|
||||
case *proxyprotocol.Conn:
|
||||
underlying, err := writeCloser(typedConn.Conn)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return &writeCloserWrapper{writeCloser: underlying, Conn: typedConn}, nil
|
||||
case *net.TCPConn:
|
||||
return typedConn, nil
|
||||
default:
|
||||
return nil, fmt.Errorf("unknown connection type %T", typedConn)
|
||||
}
|
||||
}
|
||||
|
||||
// tcpKeepAliveListener sets TCP keep-alive timeouts on accepted
|
||||
|
@ -382,7 +438,7 @@ type httpServer struct {
|
|||
}
|
||||
|
||||
func createHTTPServer(ctx context.Context, ln net.Listener, configuration *static.EntryPoint, withH2c bool) (*httpServer, error) {
|
||||
httpSwitcher := middlewares.NewHandlerSwitcher(buildDefaultHTTPRouter())
|
||||
httpSwitcher := middlewares.NewHandlerSwitcher(router.BuildDefaultHTTPRouter())
|
||||
|
||||
var handler http.Handler
|
||||
var err error
|
||||
|
|
|
@ -28,14 +28,14 @@ func TestShutdownHTTP(t *testing.T) {
|
|||
})
|
||||
require.NoError(t, err)
|
||||
|
||||
go entryPoint.startTCP(context.Background())
|
||||
go entryPoint.StartTCP(context.Background())
|
||||
|
||||
router := &tcp.Router{}
|
||||
router.HTTPHandler(http.HandlerFunc(func(rw http.ResponseWriter, req *http.Request) {
|
||||
time.Sleep(1 * time.Second)
|
||||
rw.WriteHeader(http.StatusOK)
|
||||
}))
|
||||
entryPoint.switchRouter(router)
|
||||
entryPoint.SwitchRouter(router)
|
||||
|
||||
conn, err := net.Dial("tcp", entryPoint.listener.Addr().String())
|
||||
require.NoError(t, err)
|
||||
|
@ -66,7 +66,7 @@ func TestShutdownHTTPHijacked(t *testing.T) {
|
|||
})
|
||||
require.NoError(t, err)
|
||||
|
||||
go entryPoint.startTCP(context.Background())
|
||||
go entryPoint.StartTCP(context.Background())
|
||||
|
||||
router := &tcp.Router{}
|
||||
router.HTTPHandler(http.HandlerFunc(func(rw http.ResponseWriter, req *http.Request) {
|
||||
|
@ -79,7 +79,7 @@ func TestShutdownHTTPHijacked(t *testing.T) {
|
|||
require.NoError(t, err)
|
||||
}))
|
||||
|
||||
entryPoint.switchRouter(router)
|
||||
entryPoint.SwitchRouter(router)
|
||||
|
||||
conn, err := net.Dial("tcp", entryPoint.listener.Addr().String())
|
||||
require.NoError(t, err)
|
||||
|
@ -110,7 +110,7 @@ func TestShutdownTCPConn(t *testing.T) {
|
|||
})
|
||||
require.NoError(t, err)
|
||||
|
||||
go entryPoint.startTCP(context.Background())
|
||||
go entryPoint.StartTCP(context.Background())
|
||||
|
||||
router := &tcp.Router{}
|
||||
router.AddCatchAllNoTLS(tcp.HandlerFunc(func(conn tcp.WriteCloser) {
|
||||
|
@ -123,7 +123,7 @@ func TestShutdownTCPConn(t *testing.T) {
|
|||
require.NoError(t, err)
|
||||
}))
|
||||
|
||||
entryPoint.switchRouter(router)
|
||||
entryPoint.SwitchRouter(router)
|
||||
|
||||
conn, err := net.Dial("tcp", entryPoint.listener.Addr().String())
|
||||
require.NoError(t, err)
|
||||
|
|
|
@ -1,268 +0,0 @@
|
|||
package server
|
||||
|
||||
import (
|
||||
"context"
|
||||
"net/http"
|
||||
"net/http/httptest"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/containous/traefik/v2/pkg/config/dynamic"
|
||||
"github.com/containous/traefik/v2/pkg/config/runtime"
|
||||
"github.com/containous/traefik/v2/pkg/config/static"
|
||||
th "github.com/containous/traefik/v2/pkg/testhelpers"
|
||||
"github.com/containous/traefik/v2/pkg/types"
|
||||
"github.com/stretchr/testify/assert"
|
||||
)
|
||||
|
||||
func TestListenProvidersSkipsEmptyConfigs(t *testing.T) {
|
||||
server, stop, invokeStopChan := setupListenProvider(10 * time.Millisecond)
|
||||
defer invokeStopChan()
|
||||
|
||||
go func() {
|
||||
for {
|
||||
select {
|
||||
case <-stop:
|
||||
return
|
||||
case <-server.configurationValidatedChan:
|
||||
t.Error("An empty configuration was published but it should not")
|
||||
}
|
||||
}
|
||||
}()
|
||||
|
||||
server.configurationChan <- dynamic.Message{ProviderName: "kubernetes"}
|
||||
|
||||
// give some time so that the configuration can be processed
|
||||
time.Sleep(100 * time.Millisecond)
|
||||
}
|
||||
|
||||
func TestListenProvidersSkipsSameConfigurationForProvider(t *testing.T) {
|
||||
server, stop, invokeStopChan := setupListenProvider(10 * time.Millisecond)
|
||||
defer invokeStopChan()
|
||||
|
||||
publishedConfigCount := 0
|
||||
go func() {
|
||||
for {
|
||||
select {
|
||||
case <-stop:
|
||||
return
|
||||
case conf := <-server.configurationValidatedChan:
|
||||
// set the current configuration
|
||||
// this is usually done in the processing part of the published configuration
|
||||
// so we have to emulate the behavior here
|
||||
currentConfigurations := server.currentConfigurations.Get().(dynamic.Configurations)
|
||||
currentConfigurations[conf.ProviderName] = conf.Configuration
|
||||
server.currentConfigurations.Set(currentConfigurations)
|
||||
|
||||
publishedConfigCount++
|
||||
if publishedConfigCount > 1 {
|
||||
t.Error("Same configuration should not be published multiple times")
|
||||
}
|
||||
}
|
||||
}
|
||||
}()
|
||||
conf := &dynamic.Configuration{}
|
||||
conf.HTTP = th.BuildConfiguration(
|
||||
th.WithRouters(th.WithRouter("foo")),
|
||||
th.WithLoadBalancerServices(th.WithService("bar")),
|
||||
)
|
||||
|
||||
// provide a configuration
|
||||
server.configurationChan <- dynamic.Message{ProviderName: "kubernetes", Configuration: conf}
|
||||
|
||||
// give some time so that the configuration can be processed
|
||||
time.Sleep(20 * time.Millisecond)
|
||||
|
||||
// provide the same configuration a second time
|
||||
server.configurationChan <- dynamic.Message{ProviderName: "kubernetes", Configuration: conf}
|
||||
|
||||
// give some time so that the configuration can be processed
|
||||
time.Sleep(100 * time.Millisecond)
|
||||
}
|
||||
|
||||
func TestListenProvidersPublishesConfigForEachProvider(t *testing.T) {
|
||||
server, stop, invokeStopChan := setupListenProvider(10 * time.Millisecond)
|
||||
defer invokeStopChan()
|
||||
|
||||
publishedProviderConfigCount := map[string]int{}
|
||||
publishedConfigCount := 0
|
||||
consumePublishedConfigsDone := make(chan bool)
|
||||
go func() {
|
||||
for {
|
||||
select {
|
||||
case <-stop:
|
||||
return
|
||||
case newConfig := <-server.configurationValidatedChan:
|
||||
publishedProviderConfigCount[newConfig.ProviderName]++
|
||||
publishedConfigCount++
|
||||
if publishedConfigCount == 2 {
|
||||
consumePublishedConfigsDone <- true
|
||||
return
|
||||
}
|
||||
}
|
||||
}
|
||||
}()
|
||||
|
||||
conf := &dynamic.Configuration{}
|
||||
conf.HTTP = th.BuildConfiguration(
|
||||
th.WithRouters(th.WithRouter("foo")),
|
||||
th.WithLoadBalancerServices(th.WithService("bar")),
|
||||
)
|
||||
server.configurationChan <- dynamic.Message{ProviderName: "kubernetes", Configuration: conf}
|
||||
server.configurationChan <- dynamic.Message{ProviderName: "marathon", Configuration: conf}
|
||||
|
||||
select {
|
||||
case <-consumePublishedConfigsDone:
|
||||
if val := publishedProviderConfigCount["kubernetes"]; val != 1 {
|
||||
t.Errorf("Got %d configuration publication(s) for provider %q, want 1", val, "kubernetes")
|
||||
}
|
||||
if val := publishedProviderConfigCount["marathon"]; val != 1 {
|
||||
t.Errorf("Got %d configuration publication(s) for provider %q, want 1", val, "marathon")
|
||||
}
|
||||
case <-time.After(100 * time.Millisecond):
|
||||
t.Errorf("Published configurations were not consumed in time")
|
||||
}
|
||||
}
|
||||
|
||||
// setupListenProvider configures the Server and starts listenProviders
|
||||
func setupListenProvider(throttleDuration time.Duration) (server *Server, stop chan bool, invokeStopChan func()) {
|
||||
stop = make(chan bool)
|
||||
invokeStopChan = func() {
|
||||
stop <- true
|
||||
}
|
||||
|
||||
staticConfiguration := static.Configuration{
|
||||
Providers: &static.Providers{
|
||||
ProvidersThrottleDuration: types.Duration(throttleDuration),
|
||||
},
|
||||
}
|
||||
|
||||
server = NewServer(staticConfiguration, nil, nil, nil)
|
||||
go server.listenProviders(stop)
|
||||
|
||||
return server, stop, invokeStopChan
|
||||
}
|
||||
|
||||
func TestServerResponseEmptyBackend(t *testing.T) {
|
||||
const requestPath = "/path"
|
||||
const routeRule = "Path(`" + requestPath + "`)"
|
||||
|
||||
testCases := []struct {
|
||||
desc string
|
||||
config func(testServerURL string) *dynamic.HTTPConfiguration
|
||||
expectedStatusCode int
|
||||
}{
|
||||
{
|
||||
desc: "Ok",
|
||||
config: func(testServerURL string) *dynamic.HTTPConfiguration {
|
||||
return th.BuildConfiguration(
|
||||
th.WithRouters(th.WithRouter("foo",
|
||||
th.WithEntryPoints("http"),
|
||||
th.WithServiceName("bar"),
|
||||
th.WithRule(routeRule)),
|
||||
),
|
||||
th.WithLoadBalancerServices(th.WithService("bar",
|
||||
th.WithServers(th.WithServer(testServerURL))),
|
||||
),
|
||||
)
|
||||
},
|
||||
expectedStatusCode: http.StatusOK,
|
||||
},
|
||||
{
|
||||
desc: "No Frontend",
|
||||
config: func(testServerURL string) *dynamic.HTTPConfiguration {
|
||||
return th.BuildConfiguration()
|
||||
},
|
||||
expectedStatusCode: http.StatusNotFound,
|
||||
},
|
||||
{
|
||||
desc: "Empty Backend LB",
|
||||
config: func(testServerURL string) *dynamic.HTTPConfiguration {
|
||||
return th.BuildConfiguration(
|
||||
th.WithRouters(th.WithRouter("foo",
|
||||
th.WithEntryPoints("http"),
|
||||
th.WithServiceName("bar"),
|
||||
th.WithRule(routeRule)),
|
||||
),
|
||||
th.WithLoadBalancerServices(th.WithService("bar")),
|
||||
)
|
||||
},
|
||||
expectedStatusCode: http.StatusServiceUnavailable,
|
||||
},
|
||||
{
|
||||
desc: "Empty Backend LB Sticky",
|
||||
config: func(testServerURL string) *dynamic.HTTPConfiguration {
|
||||
return th.BuildConfiguration(
|
||||
th.WithRouters(th.WithRouter("foo",
|
||||
th.WithEntryPoints("http"),
|
||||
th.WithServiceName("bar"),
|
||||
th.WithRule(routeRule)),
|
||||
),
|
||||
th.WithLoadBalancerServices(th.WithService("bar",
|
||||
th.WithSticky("test")),
|
||||
),
|
||||
)
|
||||
},
|
||||
expectedStatusCode: http.StatusServiceUnavailable,
|
||||
},
|
||||
{
|
||||
desc: "Empty Backend LB",
|
||||
config: func(testServerURL string) *dynamic.HTTPConfiguration {
|
||||
return th.BuildConfiguration(
|
||||
th.WithRouters(th.WithRouter("foo",
|
||||
th.WithEntryPoints("http"),
|
||||
th.WithServiceName("bar"),
|
||||
th.WithRule(routeRule)),
|
||||
),
|
||||
th.WithLoadBalancerServices(th.WithService("bar")),
|
||||
)
|
||||
},
|
||||
expectedStatusCode: http.StatusServiceUnavailable,
|
||||
},
|
||||
{
|
||||
desc: "Empty Backend LB Sticky",
|
||||
config: func(testServerURL string) *dynamic.HTTPConfiguration {
|
||||
return th.BuildConfiguration(
|
||||
th.WithRouters(th.WithRouter("foo",
|
||||
th.WithEntryPoints("http"),
|
||||
th.WithServiceName("bar"),
|
||||
th.WithRule(routeRule)),
|
||||
),
|
||||
th.WithLoadBalancerServices(th.WithService("bar",
|
||||
th.WithSticky("test")),
|
||||
),
|
||||
)
|
||||
},
|
||||
expectedStatusCode: http.StatusServiceUnavailable,
|
||||
},
|
||||
}
|
||||
|
||||
for _, test := range testCases {
|
||||
test := test
|
||||
|
||||
t.Run(test.desc, func(t *testing.T) {
|
||||
t.Parallel()
|
||||
|
||||
testServer := httptest.NewServer(http.HandlerFunc(func(rw http.ResponseWriter, req *http.Request) {
|
||||
rw.WriteHeader(http.StatusOK)
|
||||
}))
|
||||
defer testServer.Close()
|
||||
|
||||
globalConfig := static.Configuration{}
|
||||
entryPointsConfig := TCPEntryPoints{
|
||||
"http": &TCPEntryPoint{},
|
||||
}
|
||||
|
||||
srv := NewServer(globalConfig, nil, entryPointsConfig, nil)
|
||||
rtConf := runtime.NewConfig(dynamic.Configuration{HTTP: test.config(testServer.URL)})
|
||||
entryPoints, _ := srv.createHTTPHandlers(context.Background(), rtConf, []string{"http"})
|
||||
|
||||
responseRecorder := &httptest.ResponseRecorder{}
|
||||
request := httptest.NewRequest(http.MethodGet, testServer.URL+requestPath, nil)
|
||||
|
||||
entryPoints["http"].ServeHTTP(responseRecorder, request)
|
||||
|
||||
assert.Equal(t, test.expectedStatusCode, responseRecorder.Result().StatusCode, "status code")
|
||||
})
|
||||
}
|
||||
}
|
89
pkg/server/service/internalhandler.go
Normal file
89
pkg/server/service/internalhandler.go
Normal file
|
@ -0,0 +1,89 @@
|
|||
package service
|
||||
|
||||
import (
|
||||
"context"
|
||||
"errors"
|
||||
"fmt"
|
||||
"net/http"
|
||||
"strings"
|
||||
|
||||
"github.com/containous/traefik/v2/pkg/config/runtime"
|
||||
)
|
||||
|
||||
type serviceManager interface {
|
||||
BuildHTTP(rootCtx context.Context, serviceName string, responseModifier func(*http.Response) error) (http.Handler, error)
|
||||
LaunchHealthCheck()
|
||||
}
|
||||
|
||||
// InternalHandlers is the internal HTTP handlers builder.
|
||||
type InternalHandlers struct {
|
||||
api http.Handler
|
||||
dashboard http.Handler
|
||||
rest http.Handler
|
||||
prometheus http.Handler
|
||||
ping http.Handler
|
||||
serviceManager
|
||||
}
|
||||
|
||||
// NewInternalHandlers creates a new InternalHandlers.
|
||||
func NewInternalHandlers(api func(configuration *runtime.Configuration) http.Handler, configuration *runtime.Configuration, rest http.Handler, metricsHandler http.Handler, pingHandler http.Handler, dashboard http.Handler, next serviceManager) *InternalHandlers {
|
||||
var apiHandler http.Handler
|
||||
if api != nil {
|
||||
apiHandler = api(configuration)
|
||||
}
|
||||
|
||||
return &InternalHandlers{
|
||||
api: apiHandler,
|
||||
dashboard: dashboard,
|
||||
rest: rest,
|
||||
prometheus: metricsHandler,
|
||||
ping: pingHandler,
|
||||
serviceManager: next,
|
||||
}
|
||||
}
|
||||
|
||||
// BuildHTTP builds an HTTP handler.
|
||||
func (m *InternalHandlers) BuildHTTP(rootCtx context.Context, serviceName string, responseModifier func(*http.Response) error) (http.Handler, error) {
|
||||
if strings.HasSuffix(serviceName, "@internal") {
|
||||
return m.get(serviceName)
|
||||
}
|
||||
|
||||
return m.serviceManager.BuildHTTP(rootCtx, serviceName, responseModifier)
|
||||
}
|
||||
|
||||
func (m *InternalHandlers) get(serviceName string) (http.Handler, error) {
|
||||
switch serviceName {
|
||||
case "api@internal":
|
||||
if m.api == nil {
|
||||
return nil, errors.New("api is not enabled")
|
||||
}
|
||||
return m.api, nil
|
||||
|
||||
case "dashboard@internal":
|
||||
if m.dashboard == nil {
|
||||
return nil, errors.New("dashboard is not enabled")
|
||||
}
|
||||
return m.dashboard, nil
|
||||
|
||||
case "rest@internal":
|
||||
if m.rest == nil {
|
||||
return nil, errors.New("rest is not enabled")
|
||||
}
|
||||
return m.rest, nil
|
||||
|
||||
case "ping@internal":
|
||||
if m.ping == nil {
|
||||
return nil, errors.New("ping is not enabled")
|
||||
}
|
||||
return m.ping, nil
|
||||
|
||||
case "prometheus@internal":
|
||||
if m.prometheus == nil {
|
||||
return nil, errors.New("prometheus is not enabled")
|
||||
}
|
||||
return m.prometheus, nil
|
||||
|
||||
default:
|
||||
return nil, fmt.Errorf("unknown internal service %s", serviceName)
|
||||
}
|
||||
}
|
61
pkg/server/service/managerfactory.go
Normal file
61
pkg/server/service/managerfactory.go
Normal file
|
@ -0,0 +1,61 @@
|
|||
package service
|
||||
|
||||
import (
|
||||
"net/http"
|
||||
|
||||
"github.com/containous/traefik/v2/pkg/api"
|
||||
"github.com/containous/traefik/v2/pkg/config/runtime"
|
||||
"github.com/containous/traefik/v2/pkg/config/static"
|
||||
"github.com/containous/traefik/v2/pkg/metrics"
|
||||
"github.com/containous/traefik/v2/pkg/safe"
|
||||
)
|
||||
|
||||
// ManagerFactory a factory of service manager.
|
||||
type ManagerFactory struct {
|
||||
metricsRegistry metrics.Registry
|
||||
|
||||
defaultRoundTripper http.RoundTripper
|
||||
|
||||
api func(configuration *runtime.Configuration) http.Handler
|
||||
restHandler http.Handler
|
||||
dashboardHandler http.Handler
|
||||
metricsHandler http.Handler
|
||||
pingHandler http.Handler
|
||||
|
||||
routinesPool *safe.Pool
|
||||
}
|
||||
|
||||
// NewManagerFactory creates a new ManagerFactory.
|
||||
func NewManagerFactory(staticConfiguration static.Configuration, routinesPool *safe.Pool, metricsRegistry metrics.Registry) *ManagerFactory {
|
||||
factory := &ManagerFactory{
|
||||
metricsRegistry: metricsRegistry,
|
||||
defaultRoundTripper: setupDefaultRoundTripper(staticConfiguration.ServersTransport),
|
||||
routinesPool: routinesPool,
|
||||
}
|
||||
|
||||
if staticConfiguration.API != nil {
|
||||
factory.api = api.NewBuilder(staticConfiguration)
|
||||
|
||||
if staticConfiguration.API.Dashboard {
|
||||
factory.dashboardHandler = http.FileServer(staticConfiguration.API.DashboardAssets)
|
||||
}
|
||||
}
|
||||
|
||||
if staticConfiguration.Providers != nil && staticConfiguration.Providers.Rest != nil {
|
||||
factory.restHandler = staticConfiguration.Providers.Rest.CreateRouter()
|
||||
}
|
||||
|
||||
if staticConfiguration.Metrics != nil && staticConfiguration.Metrics.Prometheus != nil {
|
||||
factory.metricsHandler = metrics.PrometheusHandler()
|
||||
}
|
||||
|
||||
factory.pingHandler = staticConfiguration.Ping
|
||||
|
||||
return factory
|
||||
}
|
||||
|
||||
// Build creates a service manager.
|
||||
func (f *ManagerFactory) Build(configuration *runtime.Configuration) *InternalHandlers {
|
||||
svcManager := NewManager(configuration.Services, f.defaultRoundTripper, f.metricsRegistry, f.routinesPool)
|
||||
return NewInternalHandlers(f.api, configuration, f.restHandler, f.metricsHandler, f.pingHandler, f.dashboardHandler, svcManager)
|
||||
}
|
|
@ -1,4 +1,4 @@
|
|||
package server
|
||||
package service
|
||||
|
||||
import (
|
||||
"crypto/tls"
|
||||
|
@ -98,3 +98,13 @@ func createRootCACertPool(rootCAs []traefiktls.FileOrContent) *x509.CertPool {
|
|||
|
||||
return roots
|
||||
}
|
||||
|
||||
func setupDefaultRoundTripper(conf *static.ServersTransport) http.RoundTripper {
|
||||
transport, err := createHTTPTransport(conf)
|
||||
if err != nil {
|
||||
log.WithoutContext().Errorf("Could not configure HTTP Transport, fallbacking on default transport: %v", err)
|
||||
return http.DefaultTransport
|
||||
}
|
||||
|
||||
return transport
|
||||
}
|
|
@ -34,7 +34,7 @@ const (
|
|||
)
|
||||
|
||||
// NewManager creates a new Manager
|
||||
func NewManager(configs map[string]*runtime.ServiceInfo, defaultRoundTripper http.RoundTripper, metricsRegistry metrics.Registry, routinePool *safe.Pool, api http.Handler, rest http.Handler) *Manager {
|
||||
func NewManager(configs map[string]*runtime.ServiceInfo, defaultRoundTripper http.RoundTripper, metricsRegistry metrics.Registry, routinePool *safe.Pool) *Manager {
|
||||
return &Manager{
|
||||
routinePool: routinePool,
|
||||
metricsRegistry: metricsRegistry,
|
||||
|
@ -42,8 +42,6 @@ func NewManager(configs map[string]*runtime.ServiceInfo, defaultRoundTripper htt
|
|||
defaultRoundTripper: defaultRoundTripper,
|
||||
balancers: make(map[string][]healthcheck.BalancerHandler),
|
||||
configs: configs,
|
||||
api: api,
|
||||
rest: rest,
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -55,26 +53,10 @@ type Manager struct {
|
|||
defaultRoundTripper http.RoundTripper
|
||||
balancers map[string][]healthcheck.BalancerHandler
|
||||
configs map[string]*runtime.ServiceInfo
|
||||
api http.Handler
|
||||
rest http.Handler
|
||||
}
|
||||
|
||||
// BuildHTTP Creates a http.Handler for a service configuration.
|
||||
func (m *Manager) BuildHTTP(rootCtx context.Context, serviceName string, responseModifier func(*http.Response) error) (http.Handler, error) {
|
||||
if serviceName == "api@internal" {
|
||||
if m.api == nil {
|
||||
return nil, errors.New("api is not enabled")
|
||||
}
|
||||
return m.api, nil
|
||||
}
|
||||
|
||||
if serviceName == "rest@internal" {
|
||||
if m.rest == nil {
|
||||
return nil, errors.New("rest is not enabled")
|
||||
}
|
||||
return m.rest, nil
|
||||
}
|
||||
|
||||
ctx := log.With(rootCtx, log.Str(log.ServiceName, serviceName))
|
||||
|
||||
serviceName = internal.GetQualifiedName(ctx, serviceName)
|
||||
|
|
|
@ -80,7 +80,7 @@ func TestGetLoadBalancer(t *testing.T) {
|
|||
}
|
||||
|
||||
func TestGetLoadBalancerServiceHandler(t *testing.T) {
|
||||
sm := NewManager(nil, http.DefaultTransport, nil, nil, nil, nil)
|
||||
sm := NewManager(nil, http.DefaultTransport, nil, nil)
|
||||
|
||||
server1 := httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) {
|
||||
w.Header().Set("X-From", "first")
|
||||
|
@ -332,7 +332,7 @@ func TestManager_Build(t *testing.T) {
|
|||
t.Run(test.desc, func(t *testing.T) {
|
||||
t.Parallel()
|
||||
|
||||
manager := NewManager(test.configs, http.DefaultTransport, nil, nil, nil, nil)
|
||||
manager := NewManager(test.configs, http.DefaultTransport, nil, nil)
|
||||
|
||||
ctx := context.Background()
|
||||
if len(test.providerName) > 0 {
|
||||
|
@ -346,14 +346,16 @@ func TestManager_Build(t *testing.T) {
|
|||
}
|
||||
|
||||
func TestMultipleTypeOnBuildHTTP(t *testing.T) {
|
||||
manager := NewManager(map[string]*runtime.ServiceInfo{
|
||||
services := map[string]*runtime.ServiceInfo{
|
||||
"test@file": {
|
||||
Service: &dynamic.Service{
|
||||
LoadBalancer: &dynamic.ServersLoadBalancer{},
|
||||
Weighted: &dynamic.WeightedRoundRobin{},
|
||||
},
|
||||
},
|
||||
}, http.DefaultTransport, nil, nil, nil, nil)
|
||||
}
|
||||
|
||||
manager := NewManager(services, http.DefaultTransport, nil, nil)
|
||||
|
||||
_, err := manager.BuildHTTP(context.Background(), "test@file", nil)
|
||||
assert.Error(t, err, "cannot create service: multi-types service not supported, consider declaring two different pieces of service instead")
|
||||
|
|
70
pkg/server/tcprouterfactory.go
Normal file
70
pkg/server/tcprouterfactory.go
Normal file
|
@ -0,0 +1,70 @@
|
|||
package server
|
||||
|
||||
import (
|
||||
"context"
|
||||
|
||||
"github.com/containous/traefik/v2/pkg/config/dynamic"
|
||||
"github.com/containous/traefik/v2/pkg/config/runtime"
|
||||
"github.com/containous/traefik/v2/pkg/config/static"
|
||||
"github.com/containous/traefik/v2/pkg/responsemodifiers"
|
||||
"github.com/containous/traefik/v2/pkg/server/middleware"
|
||||
"github.com/containous/traefik/v2/pkg/server/router"
|
||||
routertcp "github.com/containous/traefik/v2/pkg/server/router/tcp"
|
||||
"github.com/containous/traefik/v2/pkg/server/service"
|
||||
"github.com/containous/traefik/v2/pkg/server/service/tcp"
|
||||
tcpCore "github.com/containous/traefik/v2/pkg/tcp"
|
||||
"github.com/containous/traefik/v2/pkg/tls"
|
||||
)
|
||||
|
||||
// TCPRouterFactory the factory of TCP routers.
|
||||
type TCPRouterFactory struct {
|
||||
entryPoints []string
|
||||
|
||||
managerFactory *service.ManagerFactory
|
||||
|
||||
chainBuilder *middleware.ChainBuilder
|
||||
tlsManager *tls.Manager
|
||||
}
|
||||
|
||||
// NewTCPRouterFactory creates a new TCPRouterFactory
|
||||
func NewTCPRouterFactory(staticConfiguration static.Configuration, managerFactory *service.ManagerFactory, tlsManager *tls.Manager, chainBuilder *middleware.ChainBuilder) *TCPRouterFactory {
|
||||
var entryPoints []string
|
||||
for name := range staticConfiguration.EntryPoints {
|
||||
entryPoints = append(entryPoints, name)
|
||||
}
|
||||
|
||||
return &TCPRouterFactory{
|
||||
entryPoints: entryPoints,
|
||||
managerFactory: managerFactory,
|
||||
tlsManager: tlsManager,
|
||||
chainBuilder: chainBuilder,
|
||||
}
|
||||
}
|
||||
|
||||
// CreateTCPRouters creates new TCPRouters
|
||||
func (f *TCPRouterFactory) CreateTCPRouters(conf dynamic.Configuration) map[string]*tcpCore.Router {
|
||||
ctx := context.Background()
|
||||
|
||||
rtConf := runtime.NewConfig(conf)
|
||||
|
||||
// HTTP
|
||||
serviceManager := f.managerFactory.Build(rtConf)
|
||||
|
||||
middlewaresBuilder := middleware.NewBuilder(rtConf.Middlewares, serviceManager)
|
||||
responseModifierFactory := responsemodifiers.NewBuilder(rtConf.Middlewares)
|
||||
|
||||
routerManager := router.NewManager(rtConf, serviceManager, middlewaresBuilder, responseModifierFactory, f.chainBuilder)
|
||||
|
||||
handlersNonTLS := routerManager.BuildHandlers(ctx, f.entryPoints, false)
|
||||
handlersTLS := routerManager.BuildHandlers(ctx, f.entryPoints, true)
|
||||
|
||||
// TCP
|
||||
svcTCPManager := tcp.NewManager(rtConf)
|
||||
|
||||
rtTCPManager := routertcp.NewManager(rtConf, svcTCPManager, handlersNonTLS, handlersTLS, f.tlsManager)
|
||||
routersTCP := rtTCPManager.BuildHandlers(ctx, f.entryPoints)
|
||||
|
||||
rtConf.PopulateUsedBy()
|
||||
|
||||
return routersTCP
|
||||
}
|
234
pkg/server/tcprouterfactory_test.go
Normal file
234
pkg/server/tcprouterfactory_test.go
Normal file
|
@ -0,0 +1,234 @@
|
|||
package server
|
||||
|
||||
import (
|
||||
"net/http"
|
||||
"net/http/httptest"
|
||||
"testing"
|
||||
|
||||
"github.com/containous/traefik/v2/pkg/config/dynamic"
|
||||
"github.com/containous/traefik/v2/pkg/config/static"
|
||||
"github.com/containous/traefik/v2/pkg/metrics"
|
||||
"github.com/containous/traefik/v2/pkg/server/middleware"
|
||||
"github.com/containous/traefik/v2/pkg/server/service"
|
||||
th "github.com/containous/traefik/v2/pkg/testhelpers"
|
||||
"github.com/containous/traefik/v2/pkg/tls"
|
||||
"github.com/stretchr/testify/assert"
|
||||
)
|
||||
|
||||
func TestReuseService(t *testing.T) {
|
||||
testServer := httptest.NewServer(http.HandlerFunc(func(rw http.ResponseWriter, req *http.Request) {
|
||||
rw.WriteHeader(http.StatusOK)
|
||||
}))
|
||||
defer testServer.Close()
|
||||
|
||||
staticConfig := static.Configuration{
|
||||
EntryPoints: map[string]*static.EntryPoint{
|
||||
"http": {},
|
||||
},
|
||||
}
|
||||
|
||||
dynamicConfigs := th.BuildConfiguration(
|
||||
th.WithRouters(
|
||||
th.WithRouter("foo",
|
||||
th.WithServiceName("bar"),
|
||||
th.WithRule("Path(`/ok`)")),
|
||||
th.WithRouter("foo2",
|
||||
th.WithEntryPoints("http"),
|
||||
th.WithRule("Path(`/unauthorized`)"),
|
||||
th.WithServiceName("bar"),
|
||||
th.WithRouterMiddlewares("basicauth")),
|
||||
),
|
||||
th.WithMiddlewares(th.WithMiddleware("basicauth",
|
||||
th.WithBasicAuth(&dynamic.BasicAuth{Users: []string{"foo:bar"}}),
|
||||
)),
|
||||
th.WithLoadBalancerServices(th.WithService("bar",
|
||||
th.WithServers(th.WithServer(testServer.URL))),
|
||||
),
|
||||
)
|
||||
|
||||
managerFactory := service.NewManagerFactory(staticConfig, nil, metrics.NewVoidRegistry())
|
||||
tlsManager := tls.NewManager()
|
||||
|
||||
factory := NewTCPRouterFactory(staticConfig, managerFactory, tlsManager, middleware.NewChainBuilder(staticConfig, metrics.NewVoidRegistry(), nil))
|
||||
|
||||
entryPointsHandlers := factory.CreateTCPRouters(dynamic.Configuration{HTTP: dynamicConfigs})
|
||||
|
||||
// Test that the /ok path returns a status 200.
|
||||
responseRecorderOk := &httptest.ResponseRecorder{}
|
||||
requestOk := httptest.NewRequest(http.MethodGet, testServer.URL+"/ok", nil)
|
||||
entryPointsHandlers["http"].GetHTTPHandler().ServeHTTP(responseRecorderOk, requestOk)
|
||||
|
||||
assert.Equal(t, http.StatusOK, responseRecorderOk.Result().StatusCode, "status code")
|
||||
|
||||
// Test that the /unauthorized path returns a 401 because of
|
||||
// the basic authentication defined on the frontend.
|
||||
responseRecorderUnauthorized := &httptest.ResponseRecorder{}
|
||||
requestUnauthorized := httptest.NewRequest(http.MethodGet, testServer.URL+"/unauthorized", nil)
|
||||
entryPointsHandlers["http"].GetHTTPHandler().ServeHTTP(responseRecorderUnauthorized, requestUnauthorized)
|
||||
|
||||
assert.Equal(t, http.StatusUnauthorized, responseRecorderUnauthorized.Result().StatusCode, "status code")
|
||||
}
|
||||
|
||||
func TestServerResponseEmptyBackend(t *testing.T) {
|
||||
const requestPath = "/path"
|
||||
const routeRule = "Path(`" + requestPath + "`)"
|
||||
|
||||
testCases := []struct {
|
||||
desc string
|
||||
config func(testServerURL string) *dynamic.HTTPConfiguration
|
||||
expectedStatusCode int
|
||||
}{
|
||||
{
|
||||
desc: "Ok",
|
||||
config: func(testServerURL string) *dynamic.HTTPConfiguration {
|
||||
return th.BuildConfiguration(
|
||||
th.WithRouters(th.WithRouter("foo",
|
||||
th.WithEntryPoints("http"),
|
||||
th.WithServiceName("bar"),
|
||||
th.WithRule(routeRule)),
|
||||
),
|
||||
th.WithLoadBalancerServices(th.WithService("bar",
|
||||
th.WithServers(th.WithServer(testServerURL))),
|
||||
),
|
||||
)
|
||||
},
|
||||
expectedStatusCode: http.StatusOK,
|
||||
},
|
||||
{
|
||||
desc: "No Frontend",
|
||||
config: func(testServerURL string) *dynamic.HTTPConfiguration {
|
||||
return th.BuildConfiguration()
|
||||
},
|
||||
expectedStatusCode: http.StatusNotFound,
|
||||
},
|
||||
{
|
||||
desc: "Empty Backend LB",
|
||||
config: func(testServerURL string) *dynamic.HTTPConfiguration {
|
||||
return th.BuildConfiguration(
|
||||
th.WithRouters(th.WithRouter("foo",
|
||||
th.WithEntryPoints("http"),
|
||||
th.WithServiceName("bar"),
|
||||
th.WithRule(routeRule)),
|
||||
),
|
||||
th.WithLoadBalancerServices(th.WithService("bar")),
|
||||
)
|
||||
},
|
||||
expectedStatusCode: http.StatusServiceUnavailable,
|
||||
},
|
||||
{
|
||||
desc: "Empty Backend LB Sticky",
|
||||
config: func(testServerURL string) *dynamic.HTTPConfiguration {
|
||||
return th.BuildConfiguration(
|
||||
th.WithRouters(th.WithRouter("foo",
|
||||
th.WithEntryPoints("http"),
|
||||
th.WithServiceName("bar"),
|
||||
th.WithRule(routeRule)),
|
||||
),
|
||||
th.WithLoadBalancerServices(th.WithService("bar",
|
||||
th.WithSticky("test")),
|
||||
),
|
||||
)
|
||||
},
|
||||
expectedStatusCode: http.StatusServiceUnavailable,
|
||||
},
|
||||
{
|
||||
desc: "Empty Backend LB",
|
||||
config: func(testServerURL string) *dynamic.HTTPConfiguration {
|
||||
return th.BuildConfiguration(
|
||||
th.WithRouters(th.WithRouter("foo",
|
||||
th.WithEntryPoints("http"),
|
||||
th.WithServiceName("bar"),
|
||||
th.WithRule(routeRule)),
|
||||
),
|
||||
th.WithLoadBalancerServices(th.WithService("bar")),
|
||||
)
|
||||
},
|
||||
expectedStatusCode: http.StatusServiceUnavailable,
|
||||
},
|
||||
{
|
||||
desc: "Empty Backend LB Sticky",
|
||||
config: func(testServerURL string) *dynamic.HTTPConfiguration {
|
||||
return th.BuildConfiguration(
|
||||
th.WithRouters(th.WithRouter("foo",
|
||||
th.WithEntryPoints("http"),
|
||||
th.WithServiceName("bar"),
|
||||
th.WithRule(routeRule)),
|
||||
),
|
||||
th.WithLoadBalancerServices(th.WithService("bar",
|
||||
th.WithSticky("test")),
|
||||
),
|
||||
)
|
||||
},
|
||||
expectedStatusCode: http.StatusServiceUnavailable,
|
||||
},
|
||||
}
|
||||
|
||||
for _, test := range testCases {
|
||||
test := test
|
||||
|
||||
t.Run(test.desc, func(t *testing.T) {
|
||||
t.Parallel()
|
||||
|
||||
testServer := httptest.NewServer(http.HandlerFunc(func(rw http.ResponseWriter, req *http.Request) {
|
||||
rw.WriteHeader(http.StatusOK)
|
||||
}))
|
||||
defer testServer.Close()
|
||||
|
||||
staticConfig := static.Configuration{
|
||||
EntryPoints: map[string]*static.EntryPoint{
|
||||
"http": {},
|
||||
},
|
||||
}
|
||||
|
||||
managerFactory := service.NewManagerFactory(staticConfig, nil, metrics.NewVoidRegistry())
|
||||
tlsManager := tls.NewManager()
|
||||
|
||||
factory := NewTCPRouterFactory(staticConfig, managerFactory, tlsManager, middleware.NewChainBuilder(staticConfig, metrics.NewVoidRegistry(), nil))
|
||||
|
||||
entryPointsHandlers := factory.CreateTCPRouters(dynamic.Configuration{HTTP: test.config(testServer.URL)})
|
||||
|
||||
responseRecorder := &httptest.ResponseRecorder{}
|
||||
request := httptest.NewRequest(http.MethodGet, testServer.URL+requestPath, nil)
|
||||
|
||||
entryPointsHandlers["http"].GetHTTPHandler().ServeHTTP(responseRecorder, request)
|
||||
|
||||
assert.Equal(t, test.expectedStatusCode, responseRecorder.Result().StatusCode, "status code")
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func TestInternalServices(t *testing.T) {
|
||||
testServer := httptest.NewServer(http.HandlerFunc(func(rw http.ResponseWriter, req *http.Request) {
|
||||
rw.WriteHeader(http.StatusOK)
|
||||
}))
|
||||
defer testServer.Close()
|
||||
|
||||
staticConfig := static.Configuration{
|
||||
API: &static.API{},
|
||||
EntryPoints: map[string]*static.EntryPoint{
|
||||
"http": {},
|
||||
},
|
||||
}
|
||||
|
||||
dynamicConfigs := th.BuildConfiguration(
|
||||
th.WithRouters(
|
||||
th.WithRouter("foo",
|
||||
th.WithServiceName("api@internal"),
|
||||
th.WithRule("PathPrefix(`/api`)")),
|
||||
),
|
||||
)
|
||||
|
||||
managerFactory := service.NewManagerFactory(staticConfig, nil, metrics.NewVoidRegistry())
|
||||
tlsManager := tls.NewManager()
|
||||
|
||||
factory := NewTCPRouterFactory(staticConfig, managerFactory, tlsManager, middleware.NewChainBuilder(staticConfig, metrics.NewVoidRegistry(), nil))
|
||||
|
||||
entryPointsHandlers := factory.CreateTCPRouters(dynamic.Configuration{HTTP: dynamicConfigs})
|
||||
|
||||
// Test that the /ok path returns a status 200.
|
||||
responseRecorderOk := &httptest.ResponseRecorder{}
|
||||
requestOk := httptest.NewRequest(http.MethodGet, testServer.URL+"/api/rawdata", nil)
|
||||
entryPointsHandlers["http"].GetHTTPHandler().ServeHTTP(responseRecorderOk, requestOk)
|
||||
|
||||
assert.Equal(t, http.StatusOK, responseRecorderOk.Result().StatusCode, "status code")
|
||||
}
|
Loading…
Add table
Add a link
Reference in a new issue