Add Audit target metrics (#16044)

This commit is contained in:
Klaus Post 2022-11-10 19:20:21 +01:00 committed by GitHub
parent 34d28dd79f
commit 5b242f1d11
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
8 changed files with 233 additions and 32 deletions

View File

@ -21,6 +21,7 @@ import (
"container/ring" "container/ring"
"context" "context"
"sync" "sync"
"sync/atomic"
"github.com/minio/madmin-go" "github.com/minio/madmin-go"
"github.com/minio/minio/internal/logger" "github.com/minio/minio/internal/logger"
@ -36,6 +37,9 @@ const defaultLogBufferCount = 10000
// HTTPConsoleLoggerSys holds global console logger state // HTTPConsoleLoggerSys holds global console logger state
type HTTPConsoleLoggerSys struct { type HTTPConsoleLoggerSys struct {
totalMessages int64
failedMessages int64
sync.RWMutex sync.RWMutex
pubsub *pubsub.PubSub[log.Info, madmin.LogMask] pubsub *pubsub.PubSub[log.Info, madmin.LogMask]
console *console.Target console *console.Target
@ -133,6 +137,15 @@ func (sys *HTTPConsoleLoggerSys) String() string {
return logger.ConsoleLoggerTgt return logger.ConsoleLoggerTgt
} }
// Stats returns the target statistics.
func (sys *HTTPConsoleLoggerSys) Stats() types.TargetStats {
return types.TargetStats{
TotalMessages: atomic.LoadInt64(&sys.totalMessages),
FailedMessages: atomic.LoadInt64(&sys.failedMessages),
QueueLength: 0,
}
}
// Content returns the console stdout log // Content returns the console stdout log
func (sys *HTTPConsoleLoggerSys) Content() (logs []log.Entry) { func (sys *HTTPConsoleLoggerSys) Content() (logs []log.Entry) {
sys.RLock() sys.RLock()
@ -170,6 +183,7 @@ func (sys *HTTPConsoleLoggerSys) Send(entry interface{}) error {
case string: case string:
lg = log.Info{ConsoleMsg: e, NodeName: sys.nodeName} lg = log.Info{ConsoleMsg: e, NodeName: sys.nodeName}
} }
atomic.AddInt64(&sys.totalMessages, 1)
sys.pubsub.Publish(lg) sys.pubsub.Publish(lg)
sys.Lock() sys.Lock()
@ -177,6 +191,9 @@ func (sys *HTTPConsoleLoggerSys) Send(entry interface{}) error {
sys.logBuf.Value = lg sys.logBuf.Value = lg
sys.logBuf = sys.logBuf.Next() sys.logBuf = sys.logBuf.Next()
sys.Unlock() sys.Unlock()
err := sys.console.Send(entry, string(logger.All))
return sys.console.Send(entry, string(logger.All)) if err != nil {
atomic.AddInt64(&sys.failedMessages, 1)
}
return err
} }

View File

@ -62,6 +62,11 @@ func (t *testingLogger) Type() types.TargetType {
return types.TargetHTTP return types.TargetHTTP
} }
// Stats returns the target statistics.
func (t *testingLogger) Stats() types.TargetStats {
return types.TargetStats{}
}
func (t *testingLogger) Send(entry interface{}) error { func (t *testingLogger) Send(entry interface{}) error {
t.mu.Lock() t.mu.Lock()
defer t.mu.Unlock() defer t.mu.Unlock()

View File

@ -130,6 +130,7 @@ const (
iamSubsystem MetricSubsystem = "iam" iamSubsystem MetricSubsystem = "iam"
kmsSubsystem MetricSubsystem = "kms" kmsSubsystem MetricSubsystem = "kms"
notifySubsystem MetricSubsystem = "notify" notifySubsystem MetricSubsystem = "notify"
auditSubsystem MetricSubsystem = "audit"
) )
// MetricName are the individual names for the metric. // MetricName are the individual names for the metric.
@ -1567,6 +1568,43 @@ func getNotificationMetrics() *MetricsGroup {
Value: float64(st.CurrentQueue), Value: float64(st.CurrentQueue),
}) })
} }
// Audit and system:
audit := logger.CurrentStats()
for id, st := range audit {
metrics = append(metrics, Metric{
Description: MetricDescription{
Namespace: minioNamespace,
Subsystem: auditSubsystem,
Name: "target_queue_length",
Help: "Number of unsent messages in queue for target",
Type: gaugeMetric,
},
VariableLabels: map[string]string{"target_id": id},
Value: float64(st.QueueLength),
})
metrics = append(metrics, Metric{
Description: MetricDescription{
Namespace: minioNamespace,
Subsystem: auditSubsystem,
Name: "total_messages",
Help: "Total number of messages sent since start",
Type: counterMetric,
},
VariableLabels: map[string]string{"target_id": id},
Value: float64(st.TotalMessages),
})
metrics = append(metrics, Metric{
Description: MetricDescription{
Namespace: minioNamespace,
Subsystem: auditSubsystem,
Name: "failed_messages",
Help: "Total number of messages that failed to send since start",
Type: counterMetric,
},
VariableLabels: map[string]string{"target_id": id},
Value: float64(st.FailedMessages),
})
}
return metrics return metrics
}) })
return mg return mg

View File

@ -26,14 +26,20 @@ import (
"net/http" "net/http"
"strings" "strings"
"sync" "sync"
"sync/atomic"
"time" "time"
xhttp "github.com/minio/minio/internal/http" xhttp "github.com/minio/minio/internal/http"
"github.com/minio/minio/internal/logger/target/types" "github.com/minio/minio/internal/logger/target/types"
) )
// Timeout for the webhook http call const (
const webhookCallTimeout = 5 * time.Second // Timeout for the webhook http call
webhookCallTimeout = 5 * time.Second
// maxWorkers is the maximum number of concurrent operations.
maxWorkers = 8
)
// Config http logger target // Config http logger target
type Config struct { type Config struct {
@ -57,6 +63,14 @@ type Config struct {
// buffer is full, new logs are just ignored and an error // buffer is full, new logs are just ignored and an error
// is returned to the caller. // is returned to the caller.
type Target struct { type Target struct {
totalMessages int64
failedMessages int64
// Worker control
workers int64
workerStartMu sync.Mutex
lastStarted time.Time
wg sync.WaitGroup wg sync.WaitGroup
doneCh chan struct{} doneCh chan struct{}
@ -64,6 +78,7 @@ type Target struct {
logCh chan interface{} logCh chan interface{}
config Config config Config
client *http.Client
} }
// Endpoint returns the backend endpoint // Endpoint returns the backend endpoint
@ -75,6 +90,15 @@ func (h *Target) String() string {
return h.config.Name return h.config.Name
} }
// Stats returns the target statistics.
func (h *Target) Stats() types.TargetStats {
return types.TargetStats{
TotalMessages: atomic.LoadInt64(&h.totalMessages),
FailedMessages: atomic.LoadInt64(&h.failedMessages),
QueueLength: len(h.logCh),
}
}
// Init validate and initialize the http target // Init validate and initialize the http target
func (h *Target) Init() error { func (h *Target) Init() error {
ctx, cancel := context.WithTimeout(context.Background(), 2*webhookCallTimeout) ctx, cancel := context.WithTimeout(context.Background(), 2*webhookCallTimeout)
@ -100,6 +124,7 @@ func (h *Target) Init() error {
if err != nil { if err != nil {
return err return err
} }
h.client = &client
// Drain any response. // Drain any response.
xhttp.DrainBody(resp.Body) xhttp.DrainBody(resp.Body)
@ -114,6 +139,8 @@ func (h *Target) Init() error {
h.config.Endpoint, resp.Status) h.config.Endpoint, resp.Status)
} }
h.lastStarted = time.Now()
atomic.AddInt64(&h.workers, 1)
go h.startHTTPLogger() go h.startHTTPLogger()
return nil return nil
} }
@ -128,15 +155,17 @@ func acceptedResponseStatusCode(code int) bool {
func (h *Target) logEntry(entry interface{}) { func (h *Target) logEntry(entry interface{}) {
logJSON, err := json.Marshal(&entry) logJSON, err := json.Marshal(&entry)
if err != nil { if err != nil {
atomic.AddInt64(&h.failedMessages, 1)
return return
} }
ctx, cancel := context.WithTimeout(context.Background(), webhookCallTimeout) ctx, cancel := context.WithTimeout(context.Background(), webhookCallTimeout)
defer cancel()
req, err := http.NewRequestWithContext(ctx, http.MethodPost, req, err := http.NewRequestWithContext(ctx, http.MethodPost,
h.config.Endpoint, bytes.NewReader(logJSON)) h.config.Endpoint, bytes.NewReader(logJSON))
if err != nil { if err != nil {
h.config.LogOnce(ctx, fmt.Errorf("%s returned '%w', please check your endpoint configuration", h.config.Endpoint, err), h.config.Endpoint) h.config.LogOnce(ctx, fmt.Errorf("%s returned '%w', please check your endpoint configuration", h.config.Endpoint, err), h.config.Endpoint)
cancel() atomic.AddInt64(&h.failedMessages, 1)
return return
} }
req.Header.Set(xhttp.ContentType, "application/json") req.Header.Set(xhttp.ContentType, "application/json")
@ -151,10 +180,9 @@ func (h *Target) logEntry(entry interface{}) {
req.Header.Set("Authorization", h.config.AuthToken) req.Header.Set("Authorization", h.config.AuthToken)
} }
client := http.Client{Transport: h.config.Transport} resp, err := h.client.Do(req)
resp, err := client.Do(req)
cancel()
if err != nil { if err != nil {
atomic.AddInt64(&h.failedMessages, 1)
h.config.LogOnce(ctx, fmt.Errorf("%s returned '%w', please check your endpoint configuration", h.config.Endpoint, err), h.config.Endpoint) h.config.LogOnce(ctx, fmt.Errorf("%s returned '%w', please check your endpoint configuration", h.config.Endpoint, err), h.config.Endpoint)
return return
} }
@ -163,6 +191,7 @@ func (h *Target) logEntry(entry interface{}) {
xhttp.DrainBody(resp.Body) xhttp.DrainBody(resp.Body)
if !acceptedResponseStatusCode(resp.StatusCode) { if !acceptedResponseStatusCode(resp.StatusCode) {
atomic.AddInt64(&h.failedMessages, 1)
switch resp.StatusCode { switch resp.StatusCode {
case http.StatusForbidden: case http.StatusForbidden:
h.config.LogOnce(ctx, fmt.Errorf("%s returned '%s', please check if your auth token is correctly set", h.config.Endpoint, resp.Status), h.config.Endpoint) h.config.LogOnce(ctx, fmt.Errorf("%s returned '%s', please check if your auth token is correctly set", h.config.Endpoint, resp.Status), h.config.Endpoint)
@ -177,11 +206,15 @@ func (h *Target) startHTTPLogger() {
// from an internal channel. // from an internal channel.
h.wg.Add(1) h.wg.Add(1)
go func() { go func() {
defer h.wg.Done() defer func() {
h.wg.Done()
atomic.AddInt64(&h.workers, -1)
}()
for { for {
select { select {
case entry := <-h.logCh: case entry := <-h.logCh:
atomic.AddInt64(&h.totalMessages, 1)
h.logEntry(entry) h.logEntry(entry)
case <-h.doneCh: case <-h.doneCh:
return return
@ -214,8 +247,30 @@ func (h *Target) Send(entry interface{}) error {
case <-h.doneCh: case <-h.doneCh:
case h.logCh <- entry: case h.logCh <- entry:
default: default:
nWorkers := atomic.LoadInt64(&h.workers)
if nWorkers < maxWorkers {
// Only have one try to start at the same time.
h.workerStartMu.Lock()
defer h.workerStartMu.Unlock()
// Start one max every second.
if time.Since(h.lastStarted) > time.Second {
if atomic.CompareAndSwapInt64(&h.workers, nWorkers, nWorkers+1) {
// Start another logger.
h.lastStarted = time.Now()
go h.startHTTPLogger()
}
}
// Block to send
select {
case <-h.doneCh:
case h.logCh <- entry:
}
return nil
}
// log channel is full, do not wait and return // log channel is full, do not wait and return
// an error immediately to the caller // an error immediately to the caller
atomic.AddInt64(&h.totalMessages, 1)
atomic.AddInt64(&h.failedMessages, 1)
return errors.New("log buffer full") return errors.New("log buffer full")
} }

View File

@ -25,6 +25,7 @@ import (
"errors" "errors"
"net" "net"
"sync" "sync"
"sync/atomic"
"github.com/Shopify/sarama" "github.com/Shopify/sarama"
saramatls "github.com/Shopify/sarama/tools/tls" saramatls "github.com/Shopify/sarama/tools/tls"
@ -36,11 +37,14 @@ import (
// Target - Kafka target. // Target - Kafka target.
type Target struct { type Target struct {
totalMessages int64
failedMessages int64
wg sync.WaitGroup wg sync.WaitGroup
doneCh chan struct{} doneCh chan struct{}
// Channel of log entries // Channel of log entries
logCh chan interface{} logCh chan audit.Entry
producer sarama.SyncProducer producer sarama.SyncProducer
kconfig Config kconfig Config
@ -55,38 +59,41 @@ func (h *Target) Send(entry interface{}) error {
default: default:
} }
if e, ok := entry.(audit.Entry); ok {
select { select {
case <-h.doneCh: case <-h.doneCh:
case h.logCh <- entry: case h.logCh <- e:
default: default:
// log channel is full, do not wait and return // log channel is full, do not wait and return
// an error immediately to the caller // an error immediately to the caller
atomic.AddInt64(&h.totalMessages, 1)
atomic.AddInt64(&h.failedMessages, 1)
return errors.New("log buffer full") return errors.New("log buffer full")
} }
}
return nil return nil
} }
func (h *Target) logEntry(entry interface{}) { func (h *Target) logEntry(entry audit.Entry) {
atomic.AddInt64(&h.totalMessages, 1)
logJSON, err := json.Marshal(&entry) logJSON, err := json.Marshal(&entry)
if err != nil { if err != nil {
atomic.AddInt64(&h.failedMessages, 1)
return return
} }
ae, ok := entry.(audit.Entry)
if ok {
msg := sarama.ProducerMessage{ msg := sarama.ProducerMessage{
Topic: h.kconfig.Topic, Topic: h.kconfig.Topic,
Key: sarama.StringEncoder(ae.RequestID), Key: sarama.StringEncoder(entry.RequestID),
Value: sarama.ByteEncoder(logJSON), Value: sarama.ByteEncoder(logJSON),
} }
_, _, err = h.producer.SendMessage(&msg) _, _, err = h.producer.SendMessage(&msg)
if err != nil { if err != nil {
atomic.AddInt64(&h.failedMessages, 1)
h.kconfig.LogOnce(context.Background(), err, h.kconfig.Topic) h.kconfig.LogOnce(context.Background(), err, h.kconfig.Topic)
return return
} }
}
} }
func (h *Target) startKakfaLogger() { func (h *Target) startKakfaLogger() {
@ -147,6 +154,15 @@ func (k Config) pingBrokers() error {
return err return err
} }
// Stats returns the target statistics.
func (h *Target) Stats() types.TargetStats {
return types.TargetStats{
TotalMessages: atomic.LoadInt64(&h.totalMessages),
FailedMessages: atomic.LoadInt64(&h.failedMessages),
QueueLength: len(h.logCh),
}
}
// Endpoint - return kafka target // Endpoint - return kafka target
func (h *Target) Endpoint() string { func (h *Target) Endpoint() string {
return "kafka" return "kafka"
@ -231,7 +247,7 @@ func (h *Target) Cancel() {
// sends log over http to the specified endpoint // sends log over http to the specified endpoint
func New(config Config) *Target { func New(config Config) *Target {
target := &Target{ target := &Target{
logCh: make(chan interface{}, 10000), logCh: make(chan audit.Entry, 10000),
doneCh: make(chan struct{}), doneCh: make(chan struct{}),
kconfig: config, kconfig: config,
} }

View File

@ -0,0 +1,26 @@
// Code generated by "stringer -type=TargetType -trimprefix=Target types.go"; DO NOT EDIT.
package types
import "strconv"
func _() {
// An "invalid array index" compiler error signifies that the constant values have changed.
// Re-run the stringer command to generate them again.
var x [1]struct{}
_ = x[TargetConsole-1]
_ = x[TargetHTTP-2]
_ = x[TargetKafka-3]
}
const _TargetType_name = "ConsoleHTTPKafka"
var _TargetType_index = [...]uint8{0, 7, 11, 16}
func (i TargetType) String() string {
i -= 1
if i >= TargetType(len(_TargetType_index)-1) {
return "TargetType(" + strconv.FormatInt(int64(i+1), 10) + ")"
}
return _TargetType_name[_TargetType_index[i]:_TargetType_index[i+1]]
}

View File

@ -20,6 +20,8 @@ package types
// TargetType indicates type of the target e.g. console, http, kafka // TargetType indicates type of the target e.g. console, http, kafka
type TargetType uint8 type TargetType uint8
//go:generate stringer -type=TargetType -trimprefix=Target $GOFILE
// Constants for target types // Constants for target types
const ( const (
_ TargetType = iota _ TargetType = iota
@ -27,3 +29,15 @@ const (
TargetHTTP TargetHTTP
TargetKafka TargetKafka
) )
// TargetStats contains statistics for a target.
type TargetStats struct {
// QueueLength is the queue length if any messages are queued.
QueueLength int
// TotalMessages is the total number of messages sent in the lifetime of the target
TotalMessages int64
// FailedMessages should log message count that failed to send.
FailedMessages int64
}

View File

@ -18,6 +18,8 @@
package logger package logger
import ( import (
"fmt"
"strings"
"sync" "sync"
"github.com/minio/minio/internal/logger/target/http" "github.com/minio/minio/internal/logger/target/http"
@ -32,6 +34,7 @@ import (
type Target interface { type Target interface {
String() string String() string
Endpoint() string Endpoint() string
Stats() types.TargetStats
Init() error Init() error
Cancel() Cancel()
Send(entry interface{}) error Send(entry interface{}) error
@ -71,6 +74,33 @@ func AuditTargets() []Target {
return res return res
} }
// CurrentStats returns the current statistics.
func CurrentStats() map[string]types.TargetStats {
sys := SystemTargets()
audit := AuditTargets()
res := make(map[string]types.TargetStats, len(sys)+len(audit))
cnt := make(map[string]int, len(sys)+len(audit))
// Add system and audit.
for _, t := range sys {
key := strings.ToLower(t.Type().String())
n := cnt[key]
cnt[key]++
key = fmt.Sprintf("sys_%s_%d", key, n)
res[key] = t.Stats()
}
for _, t := range audit {
key := strings.ToLower(t.Type().String())
n := cnt[key]
cnt[key]++
key = fmt.Sprintf("audit_%s_%d", key, n)
res[key] = t.Stats()
}
return res
}
// auditTargets is the list of enabled audit loggers // auditTargets is the list of enabled audit loggers
// Must be immutable at all times. // Must be immutable at all times.
// Can be swapped to another while holding swapMu // Can be swapped to another while holding swapMu