Like Prometheus, but for logs.
You can not select more than 25 topics Topics must start with a letter or number, can include dashes ('-') and can be up to 35 characters long.
loki/pkg/ingester/metrics.go

112 lines
4.5 KiB

Adds WAL support (experimental) (#2981) * marshalable chunks * wal record types custom serialization * proto types for wal checkpoints * byteswith output unaffected by buffer * wal & record pool ifcs * wal record can hold entries from multiple series * entry pool * ingester uses noopWal * removes duplicate argument passing in ingester code. adds ingester config validation & derives chunk encoding. * segment writing * [WIP] wal recovery from segments * replay uses sync.Maps & preserves WAL fingerprints * in memory wal recovery * wal segment recovery * ingester metrics struct * wal replay locks streamsMtx in instances, adds checkpoint codec * ingester metrics * checkpointer * WAL checkpoint writer * checkpointwriter can write multiple checkpoints * reorgs checkpointing * wires up checkpointwriter to wal * ingester SeriesIter impl * wires up ingesterRecoverer to consume checkpoints * generic recovery fn * generic recovery fn * recover from both wal types * cleans up old tmp checkpoints & allows aborting in flight checkpoints * wires up wal checkpointing * more granular wal logging * fixes off by 1 wal truncation & removes double logging * adds userID to wal records correctly * wire chunk encoding tests * more granular wal metrics * checkpoint encoding test * ignores debug bins * segment replay ignores out of orders * fixes bug between WAL reading []byte validity and proto unmarshalling refs * conf validations, removes comments * flush on shutdown config * POST /ingester/shutdown * renames flush on shutdown * wal & checkpoint use same segment size * writes entries to wal regardless of tailers * makes wal checkpoing duration default to 5m * recovery metrics * encodes headchunks separately for wal purposes * merge upstream * linting * addresses pr feedback uses entry pool in stream push/tailer removes unnecessary pool interaction checkpointbytes comment fillchunk helper, record resetting in tests via pool redundant comment defers wg done in recovery s/num/count/ checkpoint wal uses a logger encodeWithTypeHeader now creates its own []byte removes pool from decodeEntries wal stop can error * prevent shared access bug with tailers and entry pool * removes stream push entry pool optimization
5 years ago
package ingester
import (
"github.com/prometheus/client_golang/prometheus"
"github.com/prometheus/client_golang/prometheus/promauto"
)
type ingesterMetrics struct {
checkpointDeleteFail prometheus.Counter
checkpointDeleteTotal prometheus.Counter
checkpointCreationFail prometheus.Counter
checkpointCreationTotal prometheus.Counter
checkpointDuration prometheus.Summary
checkpointLoggedBytesTotal prometheus.Counter
walDiskFullFailures prometheus.Counter
Adds WAL support (experimental) (#2981) * marshalable chunks * wal record types custom serialization * proto types for wal checkpoints * byteswith output unaffected by buffer * wal & record pool ifcs * wal record can hold entries from multiple series * entry pool * ingester uses noopWal * removes duplicate argument passing in ingester code. adds ingester config validation & derives chunk encoding. * segment writing * [WIP] wal recovery from segments * replay uses sync.Maps & preserves WAL fingerprints * in memory wal recovery * wal segment recovery * ingester metrics struct * wal replay locks streamsMtx in instances, adds checkpoint codec * ingester metrics * checkpointer * WAL checkpoint writer * checkpointwriter can write multiple checkpoints * reorgs checkpointing * wires up checkpointwriter to wal * ingester SeriesIter impl * wires up ingesterRecoverer to consume checkpoints * generic recovery fn * generic recovery fn * recover from both wal types * cleans up old tmp checkpoints & allows aborting in flight checkpoints * wires up wal checkpointing * more granular wal logging * fixes off by 1 wal truncation & removes double logging * adds userID to wal records correctly * wire chunk encoding tests * more granular wal metrics * checkpoint encoding test * ignores debug bins * segment replay ignores out of orders * fixes bug between WAL reading []byte validity and proto unmarshalling refs * conf validations, removes comments * flush on shutdown config * POST /ingester/shutdown * renames flush on shutdown * wal & checkpoint use same segment size * writes entries to wal regardless of tailers * makes wal checkpoing duration default to 5m * recovery metrics * encodes headchunks separately for wal purposes * merge upstream * linting * addresses pr feedback uses entry pool in stream push/tailer removes unnecessary pool interaction checkpointbytes comment fillchunk helper, record resetting in tests via pool redundant comment defers wg done in recovery s/num/count/ checkpoint wal uses a logger encodeWithTypeHeader now creates its own []byte removes pool from decodeEntries wal stop can error * prevent shared access bug with tailers and entry pool * removes stream push entry pool optimization
5 years ago
walReplayDuration prometheus.Gauge
walCorruptionsTotal *prometheus.CounterVec
walLoggedBytesTotal prometheus.Counter
walRecordsLogged prometheus.Counter
recoveredStreamsTotal prometheus.Counter
recoveredChunksTotal prometheus.Counter
recoveredEntriesTotal prometheus.Counter
recoveredBytesTotal prometheus.Counter
recoveryBytesInUse prometheus.Gauge
}
// setRecoveryBytesInUse bounds the bytes reports to >= 0.
// TODO(owen-d): we can gain some efficiency by having the flusher never update this after recovery ends.
func (m *ingesterMetrics) setRecoveryBytesInUse(v int64) {
if v < 0 {
v = 0
}
m.recoveryBytesInUse.Set(float64(v))
Adds WAL support (experimental) (#2981) * marshalable chunks * wal record types custom serialization * proto types for wal checkpoints * byteswith output unaffected by buffer * wal & record pool ifcs * wal record can hold entries from multiple series * entry pool * ingester uses noopWal * removes duplicate argument passing in ingester code. adds ingester config validation & derives chunk encoding. * segment writing * [WIP] wal recovery from segments * replay uses sync.Maps & preserves WAL fingerprints * in memory wal recovery * wal segment recovery * ingester metrics struct * wal replay locks streamsMtx in instances, adds checkpoint codec * ingester metrics * checkpointer * WAL checkpoint writer * checkpointwriter can write multiple checkpoints * reorgs checkpointing * wires up checkpointwriter to wal * ingester SeriesIter impl * wires up ingesterRecoverer to consume checkpoints * generic recovery fn * generic recovery fn * recover from both wal types * cleans up old tmp checkpoints & allows aborting in flight checkpoints * wires up wal checkpointing * more granular wal logging * fixes off by 1 wal truncation & removes double logging * adds userID to wal records correctly * wire chunk encoding tests * more granular wal metrics * checkpoint encoding test * ignores debug bins * segment replay ignores out of orders * fixes bug between WAL reading []byte validity and proto unmarshalling refs * conf validations, removes comments * flush on shutdown config * POST /ingester/shutdown * renames flush on shutdown * wal & checkpoint use same segment size * writes entries to wal regardless of tailers * makes wal checkpoing duration default to 5m * recovery metrics * encodes headchunks separately for wal purposes * merge upstream * linting * addresses pr feedback uses entry pool in stream push/tailer removes unnecessary pool interaction checkpointbytes comment fillchunk helper, record resetting in tests via pool redundant comment defers wg done in recovery s/num/count/ checkpoint wal uses a logger encodeWithTypeHeader now creates its own []byte removes pool from decodeEntries wal stop can error * prevent shared access bug with tailers and entry pool * removes stream push entry pool optimization
5 years ago
}
const (
walTypeCheckpoint = "checkpoint"
walTypeSegment = "segment"
)
func newIngesterMetrics(r prometheus.Registerer) *ingesterMetrics {
return &ingesterMetrics{
walDiskFullFailures: promauto.With(r).NewCounter(prometheus.CounterOpts{
Name: "loki_ingester_wal_disk_full_failures_total",
Help: "Total number of wal write failures due to full disk.",
}),
Adds WAL support (experimental) (#2981) * marshalable chunks * wal record types custom serialization * proto types for wal checkpoints * byteswith output unaffected by buffer * wal & record pool ifcs * wal record can hold entries from multiple series * entry pool * ingester uses noopWal * removes duplicate argument passing in ingester code. adds ingester config validation & derives chunk encoding. * segment writing * [WIP] wal recovery from segments * replay uses sync.Maps & preserves WAL fingerprints * in memory wal recovery * wal segment recovery * ingester metrics struct * wal replay locks streamsMtx in instances, adds checkpoint codec * ingester metrics * checkpointer * WAL checkpoint writer * checkpointwriter can write multiple checkpoints * reorgs checkpointing * wires up checkpointwriter to wal * ingester SeriesIter impl * wires up ingesterRecoverer to consume checkpoints * generic recovery fn * generic recovery fn * recover from both wal types * cleans up old tmp checkpoints & allows aborting in flight checkpoints * wires up wal checkpointing * more granular wal logging * fixes off by 1 wal truncation & removes double logging * adds userID to wal records correctly * wire chunk encoding tests * more granular wal metrics * checkpoint encoding test * ignores debug bins * segment replay ignores out of orders * fixes bug between WAL reading []byte validity and proto unmarshalling refs * conf validations, removes comments * flush on shutdown config * POST /ingester/shutdown * renames flush on shutdown * wal & checkpoint use same segment size * writes entries to wal regardless of tailers * makes wal checkpoing duration default to 5m * recovery metrics * encodes headchunks separately for wal purposes * merge upstream * linting * addresses pr feedback uses entry pool in stream push/tailer removes unnecessary pool interaction checkpointbytes comment fillchunk helper, record resetting in tests via pool redundant comment defers wg done in recovery s/num/count/ checkpoint wal uses a logger encodeWithTypeHeader now creates its own []byte removes pool from decodeEntries wal stop can error * prevent shared access bug with tailers and entry pool * removes stream push entry pool optimization
5 years ago
walReplayDuration: promauto.With(r).NewGauge(prometheus.GaugeOpts{
Name: "loki_ingester_wal_replay_duration_seconds",
Help: "Time taken to replay the checkpoint and the WAL.",
}),
walCorruptionsTotal: promauto.With(r).NewCounterVec(prometheus.CounterOpts{
Name: "loki_ingester_wal_corruptions_total",
Help: "Total number of WAL corruptions encountered.",
}, []string{"type"}),
checkpointDeleteFail: promauto.With(r).NewCounter(prometheus.CounterOpts{
Name: "loki_ingester_checkpoint_deletions_failed_total",
Help: "Total number of checkpoint deletions that failed.",
}),
checkpointDeleteTotal: promauto.With(r).NewCounter(prometheus.CounterOpts{
Name: "loki_ingester_checkpoint_deletions_total",
Help: "Total number of checkpoint deletions attempted.",
}),
checkpointCreationFail: promauto.With(r).NewCounter(prometheus.CounterOpts{
Name: "loki_ingester_checkpoint_creations_failed_total",
Help: "Total number of checkpoint creations that failed.",
}),
checkpointCreationTotal: promauto.With(r).NewCounter(prometheus.CounterOpts{
Name: "loki_ingester_checkpoint_creations_total",
Help: "Total number of checkpoint creations attempted.",
}),
checkpointDuration: promauto.With(r).NewSummary(prometheus.SummaryOpts{
Name: "loki_ingester_checkpoint_duration_seconds",
Help: "Time taken to create a checkpoint.",
Objectives: map[float64]float64{0.5: 0.05, 0.9: 0.01, 0.99: 0.001},
}),
walRecordsLogged: promauto.With(r).NewCounter(prometheus.CounterOpts{
Name: "loki_ingester_wal_records_logged_total",
Help: "Total number of WAL records logged.",
}),
checkpointLoggedBytesTotal: promauto.With(r).NewCounter(prometheus.CounterOpts{
Name: "loki_ingester_checkpoint_logged_bytes_total",
Help: "Total number of bytes written to disk for checkpointing.",
}),
walLoggedBytesTotal: promauto.With(r).NewCounter(prometheus.CounterOpts{
Name: "loki_ingester_wal_logged_bytes_total",
Help: "Total number of bytes written to disk for WAL records.",
}),
recoveredStreamsTotal: promauto.With(r).NewCounter(prometheus.CounterOpts{
Name: "loki_ingester_wal_recovered_streams_total",
Help: "Total number of streams recovered from the WAL.",
}),
recoveredChunksTotal: promauto.With(r).NewCounter(prometheus.CounterOpts{
Name: "loki_ingester_wal_recovered_chunks_total",
Help: "Total number of chunks recovered from the WAL checkpoints.",
}),
recoveredEntriesTotal: promauto.With(r).NewCounter(prometheus.CounterOpts{
Name: "loki_ingester_wal_recovered_entries_total",
Help: "Total number of entries recovered from the WAL.",
}),
recoveredBytesTotal: promauto.With(r).NewCounter(prometheus.CounterOpts{
Name: "loki_ingester_wal_recovered_bytes_total",
Help: "Total number of bytes recovered from the WAL.",
}),
recoveryBytesInUse: promauto.With(r).NewGauge(prometheus.GaugeOpts{
Name: "loki_ingester_wal_bytes_in_use",
Help: "Total number of bytes in use by the WAL recovery process.",
}),
Adds WAL support (experimental) (#2981) * marshalable chunks * wal record types custom serialization * proto types for wal checkpoints * byteswith output unaffected by buffer * wal & record pool ifcs * wal record can hold entries from multiple series * entry pool * ingester uses noopWal * removes duplicate argument passing in ingester code. adds ingester config validation & derives chunk encoding. * segment writing * [WIP] wal recovery from segments * replay uses sync.Maps & preserves WAL fingerprints * in memory wal recovery * wal segment recovery * ingester metrics struct * wal replay locks streamsMtx in instances, adds checkpoint codec * ingester metrics * checkpointer * WAL checkpoint writer * checkpointwriter can write multiple checkpoints * reorgs checkpointing * wires up checkpointwriter to wal * ingester SeriesIter impl * wires up ingesterRecoverer to consume checkpoints * generic recovery fn * generic recovery fn * recover from both wal types * cleans up old tmp checkpoints & allows aborting in flight checkpoints * wires up wal checkpointing * more granular wal logging * fixes off by 1 wal truncation & removes double logging * adds userID to wal records correctly * wire chunk encoding tests * more granular wal metrics * checkpoint encoding test * ignores debug bins * segment replay ignores out of orders * fixes bug between WAL reading []byte validity and proto unmarshalling refs * conf validations, removes comments * flush on shutdown config * POST /ingester/shutdown * renames flush on shutdown * wal & checkpoint use same segment size * writes entries to wal regardless of tailers * makes wal checkpoing duration default to 5m * recovery metrics * encodes headchunks separately for wal purposes * merge upstream * linting * addresses pr feedback uses entry pool in stream push/tailer removes unnecessary pool interaction checkpointbytes comment fillchunk helper, record resetting in tests via pool redundant comment defers wg done in recovery s/num/count/ checkpoint wal uses a logger encodeWithTypeHeader now creates its own []byte removes pool from decodeEntries wal stop can error * prevent shared access bug with tailers and entry pool * removes stream push entry pool optimization
5 years ago
}
}