reporter: factorize how we use channels for healthchecking

Add two to the snmp component. Other components are not interesting.
This commit is contained in:
Vincent Bernat
2022-03-22 13:28:15 +01:00
parent 15483ee8f6
commit 5f5479524f
6 changed files with 97 additions and 52 deletions

View File

@@ -2,7 +2,6 @@
package core package core
import ( import (
"context"
"errors" "errors"
"fmt" "fmt"
"net" "net"
@@ -33,8 +32,8 @@ type Component struct {
metrics metrics metrics metrics
healthy chan chan<- bool // for healthcheck healthy chan reporter.ChannelHealthcheckFunc
httpFlowClients uint32 // for dumping flows httpFlowClients uint32 // for dumping flows
httpFlowChannel chan *flow.FlowMessage httpFlowChannel chan *flow.FlowMessage
httpFlowFlushDelay time.Duration httpFlowFlushDelay time.Duration
@@ -53,7 +52,7 @@ type Dependencies struct {
} }
// New creates a new core component. // New creates a new core component.
func New(reporter *reporter.Reporter, configuration Configuration, dependencies Dependencies) (*Component, error) { func New(r *reporter.Reporter, configuration Configuration, dependencies Dependencies) (*Component, error) {
cache, err := ristretto.NewCache(&ristretto.Config{ cache, err := ristretto.NewCache(&ristretto.Config{
NumCounters: int64(configuration.ClassifierCacheSize) * 10, NumCounters: int64(configuration.ClassifierCacheSize) * 10,
MaxCost: int64(configuration.ClassifierCacheSize), MaxCost: int64(configuration.ClassifierCacheSize),
@@ -64,11 +63,11 @@ func New(reporter *reporter.Reporter, configuration Configuration, dependencies
return nil, fmt.Errorf("cannot initialize classifier cache: %w", err) return nil, fmt.Errorf("cannot initialize classifier cache: %w", err)
} }
c := Component{ c := Component{
r: reporter, r: r,
d: &dependencies, d: &dependencies,
config: configuration, config: configuration,
healthy: make(chan chan<- bool), healthy: make(chan reporter.ChannelHealthcheckFunc),
httpFlowClients: 0, httpFlowClients: 0,
httpFlowChannel: make(chan *flow.FlowMessage, 10), httpFlowChannel: make(chan *flow.FlowMessage, 10),
httpFlowFlushDelay: time.Second, httpFlowFlushDelay: time.Second,
@@ -91,7 +90,7 @@ func (c *Component) Start() error {
}) })
} }
c.r.RegisterHealthcheck("core", c.runHealthcheck) c.r.RegisterHealthcheck("core", c.channelHealthcheck())
c.d.HTTP.AddHandler("/api/v0/flows", c.FlowsHTTPHandler()) c.d.HTTP.AddHandler("/api/v0/flows", c.FlowsHTTPHandler())
return nil return nil
} }
@@ -108,8 +107,10 @@ func (c *Component) runWorker(workerID int) error {
case <-c.t.Dying(): case <-c.t.Dying():
c.r.Debug().Int("worker", workerID).Msg("stopping core worker") c.r.Debug().Int("worker", workerID).Msg("stopping core worker")
return nil return nil
case answerChan := <-c.healthy: case cb := <-c.healthy:
answerChan <- true if cb != nil {
cb(reporter.HealthcheckOK, fmt.Sprintf("worker %d ok", workerID))
}
case flow := <-c.d.Flow.Flows(): case flow := <-c.d.Flow.Flows():
startBusy := time.Now() startBusy := time.Now()
if flow == nil { if flow == nil {
@@ -167,43 +168,6 @@ func (c *Component) Stop() error {
return c.t.Wait() return c.t.Wait()
} }
func (c *Component) runHealthcheck(ctx context.Context) reporter.HealthcheckResult { func (c *Component) channelHealthcheck() reporter.HealthcheckFunc {
say := func(reason string) reporter.HealthcheckResult { return reporter.ChannelHealthcheck(c.t.Context(nil), c.healthy)
if reason == "" {
return reporter.HealthcheckResult{
Status: reporter.HealthcheckOK,
Reason: "ok",
}
}
return reporter.HealthcheckResult{Status: reporter.HealthcheckError, Reason: reason}
}
if !c.t.Alive() {
return say("dead")
}
// Request a worker to answer
answerChan := make(chan bool)
defer close(answerChan)
select {
case <-c.t.Dying():
return say("dying")
case <-ctx.Done():
return say("timeout (no worker)")
case c.healthy <- answerChan:
}
// Wait for answer from worker
select {
case <-c.t.Dying():
return say("dying")
case <-ctx.Done():
return say("timeout (worker dead)")
case ok := <-answerChan:
if !ok {
// Cannot happen
return say("worker unwell")
}
return say("")
}
} }

View File

@@ -197,10 +197,10 @@ func TestCore(t *testing.T) {
// Test the healthcheck function // Test the healthcheck function
t.Run("healthcheck", func(t *testing.T) { t.Run("healthcheck", func(t *testing.T) {
got := c.runHealthcheck(context.Background()) _, got := r.RunHealthchecks(context.Background())
if diff := helpers.Diff(got, reporter.HealthcheckResult{ if diff := helpers.Diff(got["core"], reporter.HealthcheckResult{
reporter.HealthcheckOK, reporter.HealthcheckOK,
"ok", "worker 0 ok",
}); diff != "" { }); diff != "" {
t.Fatalf("runHealthcheck() (-got, +want):\n%s", diff) t.Fatalf("runHealthcheck() (-got, +want):\n%s", diff)
} }

View File

@@ -95,7 +95,15 @@ names with the module name.
It also exposes a simple way to report healthchecks from various It also exposes a simple way to report healthchecks from various
components. While it could be used to kill the application components. While it could be used to kill the application
proactively, currently, it is only exposed through HTTP. proactively, currently, it is only exposed through HTTP. Not all
components have healthchecks. For example, for the `flow` component,
it is difficult to read from UDP while watching for a check. For the
`http` component, the healthcheck would be too trivial (not in the
routine handling the heavy work). For `kafka`, the hard work is hidden
by the underlying library and we wouldn't want to be declared
unhealthy because of a transient problem by checking broker states
manually. The `daemon` component tracks the important goroutines, so it
is not vital.
The general idea is to give a good visibility to an operator. The general idea is to give a good visibility to an operator.
Everything that moves should get a counter, errors should either be Everything that moves should get a counter, errors should either be

View File

@@ -145,3 +145,43 @@ func (r *Reporter) HealthcheckHTTPHandler() http.Handler {
json.NewEncoder(w).Encode(results) json.NewEncoder(w).Encode(results)
}) })
} }
// ChannelHealthcheckFunc is the function sent over a channel to signal liveness
type ChannelHealthcheckFunc func(HealthcheckStatus, string)
// ChannelHealthcheck implements an HealthcheckFunc using a channel to
// verify a component liveness. The component should call the sent
// function received over the provided channel to tell its status.
func ChannelHealthcheck(ctx context.Context, contact chan<- ChannelHealthcheckFunc) HealthcheckFunc {
return func(healthcheckCtx context.Context) HealthcheckResult {
answerChan := make(chan HealthcheckResult)
defer close(answerChan)
signalFunc := func(status HealthcheckStatus, reason string) {
// The answer chan may be closed, because this
// function was called too late.
defer recover()
answerChan <- HealthcheckResult{status, reason}
}
// Send the signal function to contact.
select {
case <-ctx.Done():
return HealthcheckResult{HealthcheckError, "dead"}
case <-healthcheckCtx.Done():
return HealthcheckResult{HealthcheckError, "timeout"}
case contact <- signalFunc:
}
// Wait for answer from worker
select {
case <-ctx.Done():
return HealthcheckResult{HealthcheckError, "dead"}
case <-healthcheckCtx.Done():
return HealthcheckResult{HealthcheckError, "timeout"}
case result := <-answerChan:
return result
}
}
}

View File

@@ -79,6 +79,24 @@ func TestHealthcheckCancelContext(t *testing.T) {
}) })
} }
func TestChannelHealthcheck(t *testing.T) {
contact := make(chan reporter.ChannelHealthcheckFunc)
go func() {
select {
case f := <-contact:
f(reporter.HealthcheckOK, "all well, thank you!")
case <-time.After(50 * time.Millisecond):
}
}()
r := reporter.NewMock(t)
r.RegisterHealthcheck("hc1", reporter.ChannelHealthcheck(context.Background(), contact))
testHealthchecks(t, r, context.Background(),
reporter.HealthcheckOK, map[string]reporter.HealthcheckResult{
"hc1": {reporter.HealthcheckOK, "all well, thank you!"},
})
}
func TestHealthcheckHTTPHandler(t *testing.T) { func TestHealthcheckHTTPHandler(t *testing.T) {
r := reporter.NewMock(t) r := reporter.NewMock(t)
r.RegisterHealthcheck("hc1", func(ctx context.Context) reporter.HealthcheckResult { r.RegisterHealthcheck("hc1", func(ctx context.Context) reporter.HealthcheckResult {

View File

@@ -6,6 +6,7 @@ package snmp
import ( import (
"context" "context"
"errors" "errors"
"fmt"
"strconv" "strconv"
"time" "time"
@@ -91,10 +92,13 @@ func (c *Component) Start() error {
} }
// Goroutine to refresh the cache // Goroutine to refresh the cache
healthyTicker := make(chan reporter.ChannelHealthcheckFunc)
c.r.RegisterHealthcheck("snmp/ticker", reporter.ChannelHealthcheck(c.t.Context(nil), healthyTicker))
c.t.Go(func() error { c.t.Go(func() error {
c.r.Debug().Msg("starting SNMP ticker") c.r.Debug().Msg("starting SNMP ticker")
ticker := c.d.Clock.Ticker(c.config.CacheRefreshInterval) ticker := c.d.Clock.Ticker(c.config.CacheRefreshInterval)
defer ticker.Stop() defer ticker.Stop()
defer close(healthyTicker)
for { for {
select { select {
case <-c.t.Dying(): case <-c.t.Dying():
@@ -105,6 +109,10 @@ func (c *Component) Start() error {
} }
} }
return nil return nil
case cb := <-healthyTicker:
if cb != nil {
cb(reporter.HealthcheckOK, "ok")
}
case <-ticker.C: case <-ticker.C:
c.sc.Expire(c.config.CacheDuration) c.sc.Expire(c.config.CacheDuration)
if c.config.CacheRefresh > 0 { if c.config.CacheRefresh > 0 {
@@ -130,16 +138,23 @@ func (c *Component) Start() error {
}) })
// Goroutines to poll samplers // Goroutines to poll samplers
healthyWorkers := make(chan reporter.ChannelHealthcheckFunc)
c.r.RegisterHealthcheck("snmp/worker", reporter.ChannelHealthcheck(c.t.Context(nil), healthyWorkers))
for i := 0; i < c.config.Workers; i++ { for i := 0; i < c.config.Workers; i++ {
workerIDStr := strconv.Itoa(i) workerIDStr := strconv.Itoa(i)
c.t.Go(func() error { c.t.Go(func() error {
c.r.Debug().Str("worker", workerIDStr).Msg("starting SNMP poller") c.r.Debug().Str("worker", workerIDStr).Msg("starting SNMP poller")
defer close(healthyWorkers)
for { for {
startIdle := time.Now() startIdle := time.Now()
select { select {
case <-c.t.Dying(): case <-c.t.Dying():
c.r.Debug().Str("worker", workerIDStr).Msg("stopping SNMP poller") c.r.Debug().Str("worker", workerIDStr).Msg("stopping SNMP poller")
return nil return nil
case cb := <-healthyWorkers:
if cb != nil {
cb(reporter.HealthcheckOK, fmt.Sprintf("worker %s ok", workerIDStr))
}
case request := <-c.pollerChannel: case request := <-c.pollerChannel:
startBusy := time.Now() startBusy := time.Now()
samplerIP := request.SamplerIP samplerIP := request.SamplerIP