feat: point write requests have metrics (#22910)
* fix: remove dead subscriptions code * feat: storage_writer metricspull/22912/head
parent
3a81166812
commit
0a740856c7
|
@ -128,7 +128,7 @@ func NewEngine(path string, c Config, options ...Option) *Engine {
|
|||
e.tsdbStore.EngineOptions.EngineVersion = c.Data.Engine
|
||||
e.tsdbStore.EngineOptions.IndexVersion = c.Data.Index
|
||||
|
||||
pw := coordinator.NewPointsWriter(c.WriteTimeout)
|
||||
pw := coordinator.NewPointsWriter(c.WriteTimeout, path)
|
||||
pw.TSDBStore = e.tsdbStore
|
||||
pw.MetaClient = e.metaClient
|
||||
e.pointsWriter = pw
|
||||
|
@ -166,6 +166,7 @@ func (e *Engine) WithLogger(log *zap.Logger) {
|
|||
func (e *Engine) PrometheusCollectors() []prometheus.Collector {
|
||||
var metrics []prometheus.Collector
|
||||
metrics = append(metrics, tsm1.PrometheusCollectors()...)
|
||||
metrics = append(metrics, coordinator.PrometheusCollectors()...)
|
||||
return metrics
|
||||
}
|
||||
|
||||
|
|
|
@ -6,37 +6,20 @@ import (
|
|||
"fmt"
|
||||
"sort"
|
||||
"sync"
|
||||
"sync/atomic"
|
||||
"time"
|
||||
|
||||
"github.com/influxdata/influxdb/v2/models"
|
||||
"github.com/influxdata/influxdb/v2/tsdb"
|
||||
influxdb "github.com/influxdata/influxdb/v2/v1"
|
||||
"github.com/influxdata/influxdb/v2/v1/services/meta"
|
||||
"github.com/prometheus/client_golang/prometheus"
|
||||
"go.uber.org/zap"
|
||||
)
|
||||
|
||||
// The keys for statistics generated by the "write" module.
|
||||
const (
|
||||
statWriteReq = "req"
|
||||
statPointWriteReq = "pointReq"
|
||||
statPointWriteReqLocal = "pointReqLocal"
|
||||
statWriteOK = "writeOk"
|
||||
statWriteDrop = "writeDrop"
|
||||
statWriteTimeout = "writeTimeout"
|
||||
statWriteErr = "writeError"
|
||||
statSubWriteOK = "subWriteOk"
|
||||
statSubWriteDrop = "subWriteDrop"
|
||||
)
|
||||
|
||||
var (
|
||||
// ErrTimeout is returned when a write times out.
|
||||
ErrTimeout = errors.New("timeout")
|
||||
|
||||
// ErrPartialWrite is returned when a write partially succeeds but does
|
||||
// not meet the requested consistency level.
|
||||
ErrPartialWrite = errors.New("partial write")
|
||||
|
||||
// ErrWriteFailed is returned when no writes succeeded.
|
||||
ErrWriteFailed = errors.New("write failed")
|
||||
)
|
||||
|
@ -61,9 +44,7 @@ type PointsWriter struct {
|
|||
WriteToShard(ctx context.Context, shardID uint64, points []models.Point) error
|
||||
}
|
||||
|
||||
subPoints []chan<- *WritePointsRequest
|
||||
|
||||
stats *WriteStatistics
|
||||
stats *engineWriteMetrics
|
||||
}
|
||||
|
||||
// WritePointsRequest represents a request to write point data to the cluster.
|
||||
|
@ -85,12 +66,12 @@ func (w *WritePointsRequest) AddPoint(name string, value interface{}, timestamp
|
|||
}
|
||||
|
||||
// NewPointsWriter returns a new instance of PointsWriter for a node.
|
||||
func NewPointsWriter(writeTimeout time.Duration) *PointsWriter {
|
||||
func NewPointsWriter(writeTimeout time.Duration, path string) *PointsWriter {
|
||||
return &PointsWriter{
|
||||
closing: make(chan struct{}),
|
||||
WriteTimeout: writeTimeout,
|
||||
Logger: zap.NewNop(),
|
||||
stats: &WriteStatistics{},
|
||||
stats: newEngineWriteMetrics(path),
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -135,54 +116,97 @@ func (w *PointsWriter) Close() error {
|
|||
if w.closing != nil {
|
||||
close(w.closing)
|
||||
}
|
||||
if w.subPoints != nil {
|
||||
// 'nil' channels always block so this makes the
|
||||
// select statement in WritePoints hit its default case
|
||||
// dropping any in-flight writes.
|
||||
w.subPoints = nil
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (w *PointsWriter) AddWriteSubscriber(c chan<- *WritePointsRequest) {
|
||||
w.subPoints = append(w.subPoints, c)
|
||||
}
|
||||
|
||||
// WithLogger sets the Logger on w.
|
||||
func (w *PointsWriter) WithLogger(log *zap.Logger) {
|
||||
w.Logger = log.With(zap.String("service", "write"))
|
||||
}
|
||||
|
||||
// WriteStatistics keeps statistics related to the PointsWriter.
|
||||
type WriteStatistics struct {
|
||||
WriteReq int64
|
||||
PointWriteReq int64
|
||||
PointWriteReqLocal int64
|
||||
WriteOK int64
|
||||
WriteDropped int64
|
||||
WriteTimeout int64
|
||||
WriteErr int64
|
||||
SubWriteOK int64
|
||||
SubWriteDrop int64
|
||||
var globalPointsWriteMetrics *writeMetrics = newWriteMetrics()
|
||||
|
||||
type writeMetrics struct {
|
||||
// labels: type: requested,ok,dropped,err
|
||||
pointsWriteRequested *prometheus.HistogramVec
|
||||
pointsWriteOk *prometheus.HistogramVec
|
||||
pointsWriteDropped *prometheus.HistogramVec
|
||||
pointsWriteErr *prometheus.HistogramVec
|
||||
timeout *prometheus.CounterVec
|
||||
}
|
||||
|
||||
// Statistics returns statistics for periodic monitoring.
|
||||
func (w *PointsWriter) Statistics(tags map[string]string) []models.Statistic {
|
||||
return []models.Statistic{{
|
||||
Name: "write",
|
||||
Tags: tags,
|
||||
Values: map[string]interface{}{
|
||||
statWriteReq: atomic.LoadInt64(&w.stats.WriteReq),
|
||||
statPointWriteReq: atomic.LoadInt64(&w.stats.PointWriteReq),
|
||||
statPointWriteReqLocal: atomic.LoadInt64(&w.stats.PointWriteReqLocal),
|
||||
statWriteOK: atomic.LoadInt64(&w.stats.WriteOK),
|
||||
statWriteDrop: atomic.LoadInt64(&w.stats.WriteDropped),
|
||||
statWriteTimeout: atomic.LoadInt64(&w.stats.WriteTimeout),
|
||||
statWriteErr: atomic.LoadInt64(&w.stats.WriteErr),
|
||||
statSubWriteOK: atomic.LoadInt64(&w.stats.SubWriteOK),
|
||||
statSubWriteDrop: atomic.LoadInt64(&w.stats.SubWriteDrop),
|
||||
},
|
||||
}}
|
||||
// PrometheusCollectors returns all prometheus metrics for the tsm1 package.
|
||||
func PrometheusCollectors() []prometheus.Collector {
|
||||
return []prometheus.Collector{
|
||||
globalPointsWriteMetrics.pointsWriteRequested,
|
||||
globalPointsWriteMetrics.pointsWriteOk,
|
||||
globalPointsWriteMetrics.pointsWriteDropped,
|
||||
globalPointsWriteMetrics.pointsWriteErr,
|
||||
globalPointsWriteMetrics.timeout,
|
||||
}
|
||||
}
|
||||
|
||||
const namespace = "storage"
|
||||
const writerSubsystem = "writer"
|
||||
|
||||
func newWriteMetrics() *writeMetrics {
|
||||
labels := []string{"path"}
|
||||
writeBuckets := []float64{10, 100, 1000, 10000, 100000}
|
||||
return &writeMetrics{
|
||||
pointsWriteRequested: prometheus.NewHistogramVec(prometheus.HistogramOpts{
|
||||
Namespace: namespace,
|
||||
Subsystem: writerSubsystem,
|
||||
Name: "req_points",
|
||||
Help: "Histogram of number of points requested to be written",
|
||||
Buckets: writeBuckets,
|
||||
}, labels),
|
||||
pointsWriteOk: prometheus.NewHistogramVec(prometheus.HistogramOpts{
|
||||
Namespace: namespace,
|
||||
Subsystem: writerSubsystem,
|
||||
Name: "ok_points",
|
||||
Help: "Histogram of number of points in successful shard write requests",
|
||||
Buckets: writeBuckets,
|
||||
}, labels),
|
||||
pointsWriteDropped: prometheus.NewHistogramVec(prometheus.HistogramOpts{
|
||||
Namespace: namespace,
|
||||
Subsystem: writerSubsystem,
|
||||
Name: "dropped_points",
|
||||
Help: "Histogram of number of points dropped due to partial writes",
|
||||
Buckets: writeBuckets,
|
||||
}, labels),
|
||||
pointsWriteErr: prometheus.NewHistogramVec(prometheus.HistogramOpts{
|
||||
Namespace: namespace,
|
||||
Subsystem: writerSubsystem,
|
||||
Name: "err_points",
|
||||
Help: "Histogram of number of points in errored shard write requests",
|
||||
Buckets: writeBuckets,
|
||||
}, labels),
|
||||
timeout: prometheus.NewCounterVec(prometheus.CounterOpts{
|
||||
Namespace: namespace,
|
||||
Subsystem: writerSubsystem,
|
||||
Name: "timeouts",
|
||||
Help: "Number of shard write request timeouts",
|
||||
ConstLabels: nil,
|
||||
}, labels),
|
||||
}
|
||||
}
|
||||
|
||||
type engineWriteMetrics struct {
|
||||
pointsWriteRequested prometheus.Observer
|
||||
pointsWriteOk prometheus.Observer
|
||||
pointsWriteDropped prometheus.Observer
|
||||
pointsWriteErr prometheus.Observer
|
||||
timeout prometheus.Counter
|
||||
}
|
||||
|
||||
func newEngineWriteMetrics(path string) *engineWriteMetrics {
|
||||
return &engineWriteMetrics{
|
||||
pointsWriteRequested: globalPointsWriteMetrics.pointsWriteRequested.With(prometheus.Labels{"path": path}),
|
||||
pointsWriteOk: globalPointsWriteMetrics.pointsWriteOk.With(prometheus.Labels{"path": path}),
|
||||
pointsWriteDropped: globalPointsWriteMetrics.pointsWriteDropped.With(prometheus.Labels{"path": path}),
|
||||
pointsWriteErr: globalPointsWriteMetrics.pointsWriteErr.With(prometheus.Labels{"path": path}),
|
||||
timeout: globalPointsWriteMetrics.timeout.With(prometheus.Labels{"path": path}),
|
||||
}
|
||||
}
|
||||
|
||||
// MapShards maps the points contained in wp to a ShardMapping. If a point
|
||||
|
@ -230,13 +254,13 @@ func (w *PointsWriter) MapShards(wp *WritePointsRequest) (*ShardMapping, error)
|
|||
// We didn't create a shard group because the point was outside the
|
||||
// scope of the RP.
|
||||
mapping.Dropped = append(mapping.Dropped, p)
|
||||
atomic.AddInt64(&w.stats.WriteDropped, 1)
|
||||
continue
|
||||
}
|
||||
|
||||
sh := sg.ShardFor(p)
|
||||
mapping.MapPoint(&sh, p)
|
||||
}
|
||||
|
||||
return mapping, nil
|
||||
}
|
||||
|
||||
|
@ -347,8 +371,7 @@ func (w *PointsWriter) WritePointsPrivileged(
|
|||
consistencyLevel models.ConsistencyLevel,
|
||||
points []models.Point,
|
||||
) error {
|
||||
atomic.AddInt64(&w.stats.WriteReq, 1)
|
||||
atomic.AddInt64(&w.stats.PointWriteReq, int64(len(points)))
|
||||
w.stats.pointsWriteRequested.Observe(float64(len(points)))
|
||||
|
||||
if retentionPolicy == "" {
|
||||
db := w.MetaClient.Database(database)
|
||||
|
@ -368,6 +391,11 @@ func (w *PointsWriter) WritePointsPrivileged(
|
|||
for shardID, points := range shardMappings.Points {
|
||||
go func(shard *meta.ShardInfo, database, retentionPolicy string, points []models.Point) {
|
||||
err := w.writeToShard(ctx, shard, database, retentionPolicy, points)
|
||||
if err == nil {
|
||||
w.stats.pointsWriteOk.Observe(float64(len(points)))
|
||||
} else {
|
||||
w.stats.pointsWriteErr.Observe(float64(len(points)))
|
||||
}
|
||||
if err == tsdb.ErrShardDeletion {
|
||||
err = tsdb.PartialWriteError{Reason: fmt.Sprintf("shard %d is pending deletion", shard.ID), Dropped: len(points)}
|
||||
}
|
||||
|
@ -375,30 +403,8 @@ func (w *PointsWriter) WritePointsPrivileged(
|
|||
}(shardMappings.Shards[shardID], database, retentionPolicy, points)
|
||||
}
|
||||
|
||||
// Send points to subscriptions if possible.
|
||||
var ok, dropped int64
|
||||
pts := &WritePointsRequest{Database: database, RetentionPolicy: retentionPolicy, Points: points}
|
||||
// We need to lock just in case the channel is about to be nil'ed
|
||||
w.mu.RLock()
|
||||
for _, ch := range w.subPoints {
|
||||
select {
|
||||
case ch <- pts:
|
||||
ok++
|
||||
default:
|
||||
dropped++
|
||||
}
|
||||
}
|
||||
w.mu.RUnlock()
|
||||
|
||||
if ok > 0 {
|
||||
atomic.AddInt64(&w.stats.SubWriteOK, ok)
|
||||
}
|
||||
|
||||
if dropped > 0 {
|
||||
atomic.AddInt64(&w.stats.SubWriteDrop, dropped)
|
||||
}
|
||||
|
||||
if err == nil && len(shardMappings.Dropped) > 0 {
|
||||
if len(shardMappings.Dropped) > 0 {
|
||||
w.stats.pointsWriteDropped.Observe(float64(len(shardMappings.Dropped)))
|
||||
err = tsdb.PartialWriteError{Reason: "points beyond retention policy", Dropped: len(shardMappings.Dropped)}
|
||||
}
|
||||
timeout := time.NewTimer(w.WriteTimeout)
|
||||
|
@ -408,7 +414,7 @@ func (w *PointsWriter) WritePointsPrivileged(
|
|||
case <-w.closing:
|
||||
return ErrWriteFailed
|
||||
case <-timeout.C:
|
||||
atomic.AddInt64(&w.stats.WriteTimeout, 1)
|
||||
w.stats.timeout.Inc()
|
||||
// return timeout error to caller
|
||||
return ErrTimeout
|
||||
case err := <-ch:
|
||||
|
@ -422,17 +428,13 @@ func (w *PointsWriter) WritePointsPrivileged(
|
|||
|
||||
// writeToShards writes points to a shard.
|
||||
func (w *PointsWriter) writeToShard(ctx context.Context, shard *meta.ShardInfo, database, retentionPolicy string, points []models.Point) error {
|
||||
atomic.AddInt64(&w.stats.PointWriteReqLocal, int64(len(points)))
|
||||
|
||||
err := w.TSDBStore.WriteToShard(ctx, shard.ID, points)
|
||||
if err == nil {
|
||||
atomic.AddInt64(&w.stats.WriteOK, 1)
|
||||
return nil
|
||||
}
|
||||
|
||||
// Except tsdb.ErrShardNotFound no error can be handled here
|
||||
if err != tsdb.ErrShardNotFound {
|
||||
atomic.AddInt64(&w.stats.WriteErr, 1)
|
||||
return err
|
||||
}
|
||||
|
||||
|
@ -440,16 +442,13 @@ func (w *PointsWriter) writeToShard(ctx context.Context, shard *meta.ShardInfo,
|
|||
// not actually created this shard, tell it to create it and retry the write
|
||||
if err = w.TSDBStore.CreateShard(ctx, database, retentionPolicy, shard.ID, true); err != nil {
|
||||
w.Logger.Info("Write failed", zap.Uint64("shard", shard.ID), zap.Error(err))
|
||||
atomic.AddInt64(&w.stats.WriteErr, 1)
|
||||
return err
|
||||
}
|
||||
|
||||
if err = w.TSDBStore.WriteToShard(ctx, shard.ID, points); err != nil {
|
||||
w.Logger.Info("Write failed", zap.Uint64("shard", shard.ID), zap.Error(err))
|
||||
atomic.AddInt64(&w.stats.WriteErr, 1)
|
||||
return err
|
||||
}
|
||||
|
||||
atomic.AddInt64(&w.stats.WriteOK, 1)
|
||||
return nil
|
||||
}
|
||||
|
|
|
@ -3,7 +3,6 @@ package coordinator_test
|
|||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"reflect"
|
||||
"sync"
|
||||
"sync/atomic"
|
||||
"testing"
|
||||
|
@ -83,7 +82,7 @@ func TestPointsWriter_MapShards_AlterShardDuration(t *testing.T) {
|
|||
return &sg, nil
|
||||
}
|
||||
|
||||
c := coordinator.NewPointsWriter(time.Second)
|
||||
c := coordinator.NewPointsWriter(time.Second, "")
|
||||
c.MetaClient = ms
|
||||
|
||||
pr := &coordinator.WritePointsRequest{
|
||||
|
@ -161,7 +160,7 @@ func TestPointsWriter_MapShards_Multiple(t *testing.T) {
|
|||
panic("should not get here")
|
||||
}
|
||||
|
||||
c := coordinator.NewPointsWriter(time.Second)
|
||||
c := coordinator.NewPointsWriter(time.Second, "")
|
||||
c.MetaClient = ms
|
||||
defer c.Close()
|
||||
pr := &coordinator.WritePointsRequest{
|
||||
|
@ -217,7 +216,7 @@ func TestPointsWriter_MapShards_Invalid(t *testing.T) {
|
|||
return &rp.ShardGroups[0], nil
|
||||
}
|
||||
|
||||
c := coordinator.NewPointsWriter(time.Second)
|
||||
c := coordinator.NewPointsWriter(time.Second, "")
|
||||
c.MetaClient = ms
|
||||
defer c.Close()
|
||||
pr := &coordinator.WritePointsRequest{
|
||||
|
@ -259,7 +258,6 @@ func TestPointsWriter_WritePoints(t *testing.T) {
|
|||
name: "write one success",
|
||||
database: "mydb",
|
||||
retentionPolicy: "myrp",
|
||||
err: []error{nil, nil, nil},
|
||||
expErr: nil,
|
||||
},
|
||||
|
||||
|
@ -268,7 +266,6 @@ func TestPointsWriter_WritePoints(t *testing.T) {
|
|||
name: "write to non-existent database",
|
||||
database: "doesnt_exist",
|
||||
retentionPolicy: "",
|
||||
err: []error{nil, nil, nil},
|
||||
expErr: fmt.Errorf("database not found: doesnt_exist"),
|
||||
},
|
||||
}
|
||||
|
@ -291,7 +288,6 @@ func TestPointsWriter_WritePoints(t *testing.T) {
|
|||
pr.AddPoint("cpu", 3.0, time.Now().Add(time.Hour+time.Second), nil)
|
||||
|
||||
// copy to prevent data race
|
||||
theTest := test
|
||||
sm := coordinator.NewShardMapping(16)
|
||||
sm.MapPoint(
|
||||
&meta.ShardInfo{ID: uint64(1), Owners: []meta.ShardOwner{
|
||||
|
@ -323,7 +319,7 @@ func TestPointsWriter_WritePoints(t *testing.T) {
|
|||
WriteFn: func(_ context.Context, shardID uint64, points []models.Point) error {
|
||||
mu.Lock()
|
||||
defer mu.Unlock()
|
||||
return theTest.err[0]
|
||||
return nil
|
||||
},
|
||||
}
|
||||
|
||||
|
@ -332,16 +328,9 @@ func TestPointsWriter_WritePoints(t *testing.T) {
|
|||
}
|
||||
ms.NodeIDFn = func() uint64 { return 1 }
|
||||
|
||||
subPoints := make(chan *coordinator.WritePointsRequest, 1)
|
||||
sub := Subscriber{}
|
||||
sub.PointsFn = func() chan<- *coordinator.WritePointsRequest {
|
||||
return subPoints
|
||||
}
|
||||
|
||||
c := coordinator.NewPointsWriter(time.Second)
|
||||
c := coordinator.NewPointsWriter(time.Second, "")
|
||||
c.MetaClient = ms
|
||||
c.TSDBStore = store
|
||||
c.AddWriteSubscriber(sub.Points())
|
||||
c.Node = &influxdb.Node{ID: 1}
|
||||
|
||||
c.Open()
|
||||
|
@ -358,16 +347,6 @@ func TestPointsWriter_WritePoints(t *testing.T) {
|
|||
if err != nil && test.expErr != nil && err.Error() != test.expErr.Error() {
|
||||
t.Errorf("PointsWriter.WritePointsPrivileged(): '%s' error: got %v, exp %v", test.name, err, test.expErr)
|
||||
}
|
||||
if test.expErr == nil {
|
||||
select {
|
||||
case p := <-subPoints:
|
||||
if !reflect.DeepEqual(p, pr) {
|
||||
t.Errorf("PointsWriter.WritePointsPrivileged(): '%s' error: unexpected WritePointsRequest got %v, exp %v", test.name, p, pr)
|
||||
}
|
||||
default:
|
||||
t.Errorf("PointsWriter.WritePointsPrivileged(): '%s' error: Subscriber.Points not called", test.name)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -408,16 +387,9 @@ func TestPointsWriter_WritePoints_Dropped(t *testing.T) {
|
|||
}
|
||||
ms.NodeIDFn = func() uint64 { return 1 }
|
||||
|
||||
subPoints := make(chan *coordinator.WritePointsRequest, 1)
|
||||
sub := Subscriber{}
|
||||
sub.PointsFn = func() chan<- *coordinator.WritePointsRequest {
|
||||
return subPoints
|
||||
}
|
||||
|
||||
c := coordinator.NewPointsWriter(time.Second)
|
||||
c := coordinator.NewPointsWriter(time.Second, "")
|
||||
c.MetaClient = ms
|
||||
c.TSDBStore = store
|
||||
c.AddWriteSubscriber(sub.Points())
|
||||
c.Node = &influxdb.Node{ID: 1}
|
||||
|
||||
c.Open()
|
||||
|
|
Loading…
Reference in New Issue