2017-04-10 18:59:45 +00:00
|
|
|
|
// Copyright 2017 The Prometheus Authors
|
|
|
|
|
// Licensed under the Apache License, Version 2.0 (the "License");
|
|
|
|
|
// you may not use this file except in compliance with the License.
|
|
|
|
|
// You may obtain a copy of the License at
|
|
|
|
|
//
|
|
|
|
|
// http://www.apache.org/licenses/LICENSE-2.0
|
|
|
|
|
//
|
|
|
|
|
// Unless required by applicable law or agreed to in writing, software
|
|
|
|
|
// distributed under the License is distributed on an "AS IS" BASIS,
|
|
|
|
|
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
|
|
|
|
// See the License for the specific language governing permissions and
|
|
|
|
|
// limitations under the License.
|
|
|
|
|
|
2016-12-04 12:16:11 +00:00
|
|
|
|
package tsdb
|
|
|
|
|
|
|
|
|
|
import (
|
2019-05-24 18:33:28 +00:00
|
|
|
|
"fmt"
|
2021-08-17 17:08:16 +00:00
|
|
|
|
"io"
|
2017-01-04 13:06:40 +00:00
|
|
|
|
"math"
|
2020-05-06 15:30:00 +00:00
|
|
|
|
"path/filepath"
|
2016-12-04 12:16:11 +00:00
|
|
|
|
"sync"
|
2017-08-30 16:34:54 +00:00
|
|
|
|
"time"
|
2017-05-17 14:43:01 +00:00
|
|
|
|
|
2021-06-11 16:17:59 +00:00
|
|
|
|
"github.com/go-kit/log"
|
|
|
|
|
"github.com/go-kit/log/level"
|
2019-07-23 08:04:48 +00:00
|
|
|
|
"github.com/oklog/ulid"
|
2017-01-19 10:22:47 +00:00
|
|
|
|
"github.com/pkg/errors"
|
2017-08-30 16:34:54 +00:00
|
|
|
|
"github.com/prometheus/client_golang/prometheus"
|
2020-10-22 09:00:08 +00:00
|
|
|
|
"go.uber.org/atomic"
|
|
|
|
|
|
2021-07-20 04:52:57 +00:00
|
|
|
|
"github.com/prometheus/prometheus/config"
|
2021-03-16 09:47:45 +00:00
|
|
|
|
"github.com/prometheus/prometheus/pkg/exemplar"
|
2019-11-18 19:53:33 +00:00
|
|
|
|
"github.com/prometheus/prometheus/pkg/labels"
|
2020-02-06 15:58:38 +00:00
|
|
|
|
"github.com/prometheus/prometheus/storage"
|
2019-08-13 08:34:14 +00:00
|
|
|
|
"github.com/prometheus/prometheus/tsdb/chunkenc"
|
|
|
|
|
"github.com/prometheus/prometheus/tsdb/chunks"
|
2020-05-06 15:30:00 +00:00
|
|
|
|
tsdb_errors "github.com/prometheus/prometheus/tsdb/errors"
|
2019-08-13 08:34:14 +00:00
|
|
|
|
"github.com/prometheus/prometheus/tsdb/index"
|
2019-09-19 09:15:41 +00:00
|
|
|
|
"github.com/prometheus/prometheus/tsdb/record"
|
|
|
|
|
"github.com/prometheus/prometheus/tsdb/tombstones"
|
2020-07-31 15:03:02 +00:00
|
|
|
|
"github.com/prometheus/prometheus/tsdb/tsdbutil"
|
2019-08-13 08:34:14 +00:00
|
|
|
|
"github.com/prometheus/prometheus/tsdb/wal"
|
2016-12-04 12:16:11 +00:00
|
|
|
|
)
|
|
|
|
|
|
2017-01-17 15:33:58 +00:00
|
|
|
|
var (
|
2020-01-20 11:05:27 +00:00
|
|
|
|
// ErrInvalidSample is returned if an appended sample is not valid and can't
|
|
|
|
|
// be ingested.
|
|
|
|
|
ErrInvalidSample = errors.New("invalid sample")
|
2021-03-16 09:47:45 +00:00
|
|
|
|
// ErrInvalidExemplar is returned if an appended exemplar is not valid and can't
|
|
|
|
|
// be ingested.
|
|
|
|
|
ErrInvalidExemplar = errors.New("invalid exemplar")
|
2020-07-22 09:57:38 +00:00
|
|
|
|
// ErrAppenderClosed is returned if an appender has already be successfully
|
2020-08-07 07:57:25 +00:00
|
|
|
|
// rolled back or committed.
|
2020-07-22 09:57:38 +00:00
|
|
|
|
ErrAppenderClosed = errors.New("appender closed")
|
2017-01-17 15:33:58 +00:00
|
|
|
|
)
|
|
|
|
|
|
2017-08-28 22:39:17 +00:00
|
|
|
|
// Head handles reads and writes of time series data within a time window.
|
|
|
|
|
type Head struct {
|
2021-07-20 08:47:20 +00:00
|
|
|
|
chunkRange atomic.Int64
|
|
|
|
|
numSeries atomic.Uint64
|
|
|
|
|
minTime, maxTime atomic.Int64 // Current min and max of the samples included in the head.
|
|
|
|
|
minValidTime atomic.Int64 // Mint allowed to be added to the head. It shouldn't be lower than the maxt of the last persisted block.
|
|
|
|
|
lastWALTruncationTime atomic.Int64
|
|
|
|
|
lastMemoryTruncationTime atomic.Int64
|
|
|
|
|
lastSeriesID atomic.Uint64
|
2019-10-09 15:41:46 +00:00
|
|
|
|
|
2021-07-20 04:52:57 +00:00
|
|
|
|
metrics *headMetrics
|
|
|
|
|
opts *HeadOptions
|
|
|
|
|
wal *wal.WAL
|
|
|
|
|
exemplarMetrics *ExemplarMetrics
|
|
|
|
|
exemplars ExemplarStorage
|
|
|
|
|
logger log.Logger
|
|
|
|
|
appendPool sync.Pool
|
|
|
|
|
exemplarsPool sync.Pool
|
|
|
|
|
seriesPool sync.Pool
|
|
|
|
|
bytesPool sync.Pool
|
|
|
|
|
memChunkPool sync.Pool
|
2017-02-04 10:53:52 +00:00
|
|
|
|
|
2017-09-05 09:45:18 +00:00
|
|
|
|
// All series addressable by their ID or hash.
|
2021-02-09 14:12:48 +00:00
|
|
|
|
series *stripeSeries
|
2016-12-22 00:12:28 +00:00
|
|
|
|
|
2019-04-09 13:16:24 +00:00
|
|
|
|
deletedMtx sync.Mutex
|
|
|
|
|
deleted map[uint64]int // Deleted series, and what WAL segment they must be kept until.
|
|
|
|
|
|
2020-02-17 18:37:09 +00:00
|
|
|
|
postings *index.MemPostings // Postings lists for terms.
|
2019-11-05 02:06:13 +00:00
|
|
|
|
|
2020-01-20 15:38:00 +00:00
|
|
|
|
tombstones *tombstones.MemTombstones
|
|
|
|
|
|
2020-02-12 19:22:27 +00:00
|
|
|
|
iso *isolation
|
|
|
|
|
|
2019-11-05 02:06:13 +00:00
|
|
|
|
cardinalityMutex sync.Mutex
|
2020-02-17 18:37:09 +00:00
|
|
|
|
cardinalityCache *index.PostingsStats // Posting stats cache which will expire after 30sec.
|
|
|
|
|
lastPostingsStatsCall time.Duration // Last posting stats call (PostingsCardinalityStats()) time for caching.
|
2020-05-06 15:30:00 +00:00
|
|
|
|
|
|
|
|
|
// chunkDiskMapper is used to write and read Head chunks to/from disk.
|
|
|
|
|
chunkDiskMapper *chunks.ChunkDiskMapper
|
2020-05-22 09:03:23 +00:00
|
|
|
|
|
2021-08-06 16:51:01 +00:00
|
|
|
|
chunkSnapshotMtx sync.Mutex
|
|
|
|
|
|
2020-05-22 09:03:23 +00:00
|
|
|
|
closedMtx sync.Mutex
|
|
|
|
|
closed bool
|
2021-06-05 14:29:32 +00:00
|
|
|
|
|
|
|
|
|
stats *HeadStats
|
2021-07-20 04:52:57 +00:00
|
|
|
|
reg prometheus.Registerer
|
2021-07-20 08:47:20 +00:00
|
|
|
|
|
|
|
|
|
memTruncationInProcess atomic.Bool
|
2016-12-04 12:16:11 +00:00
|
|
|
|
}
|
|
|
|
|
|
2021-08-03 12:14:26 +00:00
|
|
|
|
type ExemplarStorage interface {
|
|
|
|
|
storage.ExemplarQueryable
|
|
|
|
|
AddExemplar(labels.Labels, exemplar.Exemplar) error
|
|
|
|
|
ValidateExemplar(labels.Labels, exemplar.Exemplar) error
|
2021-08-30 14:04:38 +00:00
|
|
|
|
IterateExemplars(f func(seriesLabels labels.Labels, e exemplar.Exemplar) error) error
|
2021-08-03 12:14:26 +00:00
|
|
|
|
}
|
|
|
|
|
|
2021-02-09 14:12:48 +00:00
|
|
|
|
// HeadOptions are parameters for the Head block.
|
|
|
|
|
type HeadOptions struct {
|
2021-08-12 16:38:06 +00:00
|
|
|
|
// Runtime reloadable option. At the top of the struct for 32 bit OS:
|
|
|
|
|
// https://pkg.go.dev/sync/atomic#pkg-note-BUG
|
|
|
|
|
MaxExemplars atomic.Int64
|
|
|
|
|
|
2021-02-09 14:12:48 +00:00
|
|
|
|
ChunkRange int64
|
|
|
|
|
// ChunkDirRoot is the parent directory of the chunks directory.
|
|
|
|
|
ChunkDirRoot string
|
|
|
|
|
ChunkPool chunkenc.Pool
|
|
|
|
|
ChunkWriteBufferSize int
|
|
|
|
|
// StripeSize sets the number of entries in the hash map, it must be a power of 2.
|
|
|
|
|
// A larger StripeSize will allocate more memory up-front, but will increase performance when handling a large number of series.
|
|
|
|
|
// A smaller StripeSize reduces the memory allocated, but can decrease performance with large number of series.
|
2021-08-06 16:51:01 +00:00
|
|
|
|
StripeSize int
|
|
|
|
|
SeriesCallback SeriesLifecycleCallback
|
|
|
|
|
EnableExemplarStorage bool
|
|
|
|
|
EnableMemorySnapshotOnShutdown bool
|
2021-02-09 14:12:48 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
func DefaultHeadOptions() *HeadOptions {
|
|
|
|
|
return &HeadOptions{
|
|
|
|
|
ChunkRange: DefaultBlockDuration,
|
|
|
|
|
ChunkDirRoot: "",
|
|
|
|
|
ChunkPool: chunkenc.NewPool(),
|
|
|
|
|
ChunkWriteBufferSize: chunks.DefaultWriteBufferSize,
|
|
|
|
|
StripeSize: DefaultStripeSize,
|
|
|
|
|
SeriesCallback: &noopSeriesLifecycleCallback{},
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2021-08-03 12:14:26 +00:00
|
|
|
|
// SeriesLifecycleCallback specifies a list of callbacks that will be called during a lifecycle of a series.
|
|
|
|
|
// It is always a no-op in Prometheus and mainly meant for external users who import TSDB.
|
|
|
|
|
// All the callbacks should be safe to be called concurrently.
|
|
|
|
|
// It is up to the user to implement soft or hard consistency by making the callbacks
|
|
|
|
|
// atomic or non-atomic. Atomic callbacks can cause degradation performance.
|
|
|
|
|
type SeriesLifecycleCallback interface {
|
|
|
|
|
// PreCreation is called before creating a series to indicate if the series can be created.
|
|
|
|
|
// A non nil error means the series should not be created.
|
|
|
|
|
PreCreation(labels.Labels) error
|
|
|
|
|
// PostCreation is called after creating a series to indicate a creation of series.
|
|
|
|
|
PostCreation(labels.Labels)
|
|
|
|
|
// PostDeletion is called after deletion of series.
|
|
|
|
|
PostDeletion(...labels.Labels)
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
// NewHead opens the head block in dir.
|
|
|
|
|
func NewHead(r prometheus.Registerer, l log.Logger, wal *wal.WAL, opts *HeadOptions, stats *HeadStats) (*Head, error) {
|
|
|
|
|
var err error
|
|
|
|
|
if l == nil {
|
|
|
|
|
l = log.NewNopLogger()
|
|
|
|
|
}
|
|
|
|
|
if opts.ChunkRange < 1 {
|
|
|
|
|
return nil, errors.Errorf("invalid chunk range %d", opts.ChunkRange)
|
|
|
|
|
}
|
|
|
|
|
if opts.SeriesCallback == nil {
|
|
|
|
|
opts.SeriesCallback = &noopSeriesLifecycleCallback{}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
if stats == nil {
|
|
|
|
|
stats = NewHeadStats()
|
|
|
|
|
}
|
|
|
|
|
|
2021-10-05 05:21:25 +00:00
|
|
|
|
if !opts.EnableExemplarStorage {
|
|
|
|
|
opts.MaxExemplars.Store(0)
|
|
|
|
|
}
|
|
|
|
|
|
2021-08-03 12:14:26 +00:00
|
|
|
|
h := &Head{
|
2021-08-17 17:08:16 +00:00
|
|
|
|
wal: wal,
|
|
|
|
|
logger: l,
|
|
|
|
|
opts: opts,
|
2021-08-03 12:14:26 +00:00
|
|
|
|
memChunkPool: sync.Pool{
|
|
|
|
|
New: func() interface{} {
|
|
|
|
|
return &memChunk{}
|
|
|
|
|
},
|
|
|
|
|
},
|
|
|
|
|
stats: stats,
|
|
|
|
|
reg: r,
|
|
|
|
|
}
|
2021-08-17 17:08:16 +00:00
|
|
|
|
if err := h.resetInMemoryState(); err != nil {
|
|
|
|
|
return nil, err
|
|
|
|
|
}
|
2021-08-03 12:14:26 +00:00
|
|
|
|
h.metrics = newHeadMetrics(h, r)
|
|
|
|
|
|
|
|
|
|
if opts.ChunkPool == nil {
|
|
|
|
|
opts.ChunkPool = chunkenc.NewPool()
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
h.chunkDiskMapper, err = chunks.NewChunkDiskMapper(
|
|
|
|
|
mmappedChunksDir(opts.ChunkDirRoot),
|
|
|
|
|
opts.ChunkPool,
|
|
|
|
|
opts.ChunkWriteBufferSize,
|
|
|
|
|
)
|
|
|
|
|
if err != nil {
|
|
|
|
|
return nil, err
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
return h, nil
|
|
|
|
|
}
|
|
|
|
|
|
2021-08-17 17:08:16 +00:00
|
|
|
|
func (h *Head) resetInMemoryState() error {
|
|
|
|
|
var err error
|
2021-10-05 05:21:25 +00:00
|
|
|
|
var em *ExemplarMetrics
|
|
|
|
|
if h.exemplars != nil {
|
|
|
|
|
ce, ok := h.exemplars.(*CircularExemplarStorage)
|
|
|
|
|
if ok {
|
|
|
|
|
em = ce.metrics
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
if em == nil {
|
|
|
|
|
em = NewExemplarMetrics(h.reg)
|
|
|
|
|
}
|
2021-08-17 17:08:16 +00:00
|
|
|
|
es, err := NewCircularExemplarStorage(h.opts.MaxExemplars.Load(), em)
|
|
|
|
|
if err != nil {
|
|
|
|
|
return err
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
h.exemplarMetrics = em
|
|
|
|
|
h.exemplars = es
|
|
|
|
|
h.series = newStripeSeries(h.opts.StripeSize, h.opts.SeriesCallback)
|
|
|
|
|
h.postings = index.NewUnorderedMemPostings()
|
|
|
|
|
h.tombstones = tombstones.NewMemTombstones()
|
|
|
|
|
h.iso = newIsolation()
|
|
|
|
|
h.deleted = map[uint64]int{}
|
|
|
|
|
h.chunkRange.Store(h.opts.ChunkRange)
|
|
|
|
|
h.minTime.Store(math.MaxInt64)
|
|
|
|
|
h.maxTime.Store(math.MinInt64)
|
|
|
|
|
h.lastWALTruncationTime.Store(math.MinInt64)
|
|
|
|
|
h.lastMemoryTruncationTime.Store(math.MinInt64)
|
|
|
|
|
return nil
|
|
|
|
|
}
|
|
|
|
|
|
2017-08-30 16:34:54 +00:00
|
|
|
|
type headMetrics struct {
|
2020-05-06 15:30:00 +00:00
|
|
|
|
activeAppenders prometheus.Gauge
|
|
|
|
|
series prometheus.GaugeFunc
|
|
|
|
|
seriesCreated prometheus.Counter
|
|
|
|
|
seriesRemoved prometheus.Counter
|
|
|
|
|
seriesNotFound prometheus.Counter
|
|
|
|
|
chunks prometheus.Gauge
|
|
|
|
|
chunksCreated prometheus.Counter
|
|
|
|
|
chunksRemoved prometheus.Counter
|
|
|
|
|
gcDuration prometheus.Summary
|
|
|
|
|
samplesAppended prometheus.Counter
|
|
|
|
|
outOfBoundSamples prometheus.Counter
|
|
|
|
|
outOfOrderSamples prometheus.Counter
|
|
|
|
|
walTruncateDuration prometheus.Summary
|
|
|
|
|
walCorruptionsTotal prometheus.Counter
|
2020-09-21 16:25:05 +00:00
|
|
|
|
walTotalReplayDuration prometheus.Gauge
|
2020-05-06 15:30:00 +00:00
|
|
|
|
headTruncateFail prometheus.Counter
|
|
|
|
|
headTruncateTotal prometheus.Counter
|
|
|
|
|
checkpointDeleteFail prometheus.Counter
|
|
|
|
|
checkpointDeleteTotal prometheus.Counter
|
|
|
|
|
checkpointCreationFail prometheus.Counter
|
|
|
|
|
checkpointCreationTotal prometheus.Counter
|
|
|
|
|
mmapChunkCorruptionTotal prometheus.Counter
|
2021-08-17 17:08:16 +00:00
|
|
|
|
snapshotReplayErrorTotal prometheus.Counter // Will be either 0 or 1.
|
2017-08-30 16:34:54 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
func newHeadMetrics(h *Head, r prometheus.Registerer) *headMetrics {
|
2020-02-12 19:22:27 +00:00
|
|
|
|
m := &headMetrics{
|
|
|
|
|
activeAppenders: prometheus.NewGauge(prometheus.GaugeOpts{
|
|
|
|
|
Name: "prometheus_tsdb_head_active_appenders",
|
|
|
|
|
Help: "Number of currently active appender transactions",
|
|
|
|
|
}),
|
|
|
|
|
series: prometheus.NewGaugeFunc(prometheus.GaugeOpts{
|
|
|
|
|
Name: "prometheus_tsdb_head_series",
|
|
|
|
|
Help: "Total number of series in the head block.",
|
|
|
|
|
}, func() float64 {
|
|
|
|
|
return float64(h.NumSeries())
|
|
|
|
|
}),
|
|
|
|
|
seriesCreated: prometheus.NewCounter(prometheus.CounterOpts{
|
|
|
|
|
Name: "prometheus_tsdb_head_series_created_total",
|
|
|
|
|
Help: "Total number of series created in the head",
|
|
|
|
|
}),
|
|
|
|
|
seriesRemoved: prometheus.NewCounter(prometheus.CounterOpts{
|
|
|
|
|
Name: "prometheus_tsdb_head_series_removed_total",
|
|
|
|
|
Help: "Total number of series removed in the head",
|
|
|
|
|
}),
|
|
|
|
|
seriesNotFound: prometheus.NewCounter(prometheus.CounterOpts{
|
|
|
|
|
Name: "prometheus_tsdb_head_series_not_found_total",
|
|
|
|
|
Help: "Total number of requests for series that were not found.",
|
|
|
|
|
}),
|
|
|
|
|
chunks: prometheus.NewGauge(prometheus.GaugeOpts{
|
|
|
|
|
Name: "prometheus_tsdb_head_chunks",
|
|
|
|
|
Help: "Total number of chunks in the head block.",
|
|
|
|
|
}),
|
|
|
|
|
chunksCreated: prometheus.NewCounter(prometheus.CounterOpts{
|
|
|
|
|
Name: "prometheus_tsdb_head_chunks_created_total",
|
|
|
|
|
Help: "Total number of chunks created in the head",
|
|
|
|
|
}),
|
|
|
|
|
chunksRemoved: prometheus.NewCounter(prometheus.CounterOpts{
|
|
|
|
|
Name: "prometheus_tsdb_head_chunks_removed_total",
|
|
|
|
|
Help: "Total number of chunks removed in the head",
|
|
|
|
|
}),
|
|
|
|
|
gcDuration: prometheus.NewSummary(prometheus.SummaryOpts{
|
|
|
|
|
Name: "prometheus_tsdb_head_gc_duration_seconds",
|
|
|
|
|
Help: "Runtime of garbage collection in the head block.",
|
|
|
|
|
}),
|
|
|
|
|
walTruncateDuration: prometheus.NewSummary(prometheus.SummaryOpts{
|
|
|
|
|
Name: "prometheus_tsdb_wal_truncate_duration_seconds",
|
|
|
|
|
Help: "Duration of WAL truncation.",
|
|
|
|
|
}),
|
|
|
|
|
walCorruptionsTotal: prometheus.NewCounter(prometheus.CounterOpts{
|
|
|
|
|
Name: "prometheus_tsdb_wal_corruptions_total",
|
|
|
|
|
Help: "Total number of WAL corruptions.",
|
|
|
|
|
}),
|
2020-09-21 16:25:05 +00:00
|
|
|
|
walTotalReplayDuration: prometheus.NewGauge(prometheus.GaugeOpts{
|
|
|
|
|
Name: "prometheus_tsdb_data_replay_duration_seconds",
|
|
|
|
|
Help: "Time taken to replay the data on disk.",
|
|
|
|
|
}),
|
2020-02-12 19:22:27 +00:00
|
|
|
|
samplesAppended: prometheus.NewCounter(prometheus.CounterOpts{
|
|
|
|
|
Name: "prometheus_tsdb_head_samples_appended_total",
|
|
|
|
|
Help: "Total number of appended samples.",
|
|
|
|
|
}),
|
2020-05-06 15:30:00 +00:00
|
|
|
|
outOfBoundSamples: prometheus.NewCounter(prometheus.CounterOpts{
|
|
|
|
|
Name: "prometheus_tsdb_out_of_bound_samples_total",
|
|
|
|
|
Help: "Total number of out of bound samples ingestion failed attempts.",
|
|
|
|
|
}),
|
|
|
|
|
outOfOrderSamples: prometheus.NewCounter(prometheus.CounterOpts{
|
|
|
|
|
Name: "prometheus_tsdb_out_of_order_samples_total",
|
|
|
|
|
Help: "Total number of out of order samples ingestion failed attempts.",
|
|
|
|
|
}),
|
2020-02-12 19:22:27 +00:00
|
|
|
|
headTruncateFail: prometheus.NewCounter(prometheus.CounterOpts{
|
|
|
|
|
Name: "prometheus_tsdb_head_truncations_failed_total",
|
|
|
|
|
Help: "Total number of head truncations that failed.",
|
|
|
|
|
}),
|
|
|
|
|
headTruncateTotal: prometheus.NewCounter(prometheus.CounterOpts{
|
|
|
|
|
Name: "prometheus_tsdb_head_truncations_total",
|
|
|
|
|
Help: "Total number of head truncations attempted.",
|
|
|
|
|
}),
|
|
|
|
|
checkpointDeleteFail: prometheus.NewCounter(prometheus.CounterOpts{
|
|
|
|
|
Name: "prometheus_tsdb_checkpoint_deletions_failed_total",
|
|
|
|
|
Help: "Total number of checkpoint deletions that failed.",
|
|
|
|
|
}),
|
|
|
|
|
checkpointDeleteTotal: prometheus.NewCounter(prometheus.CounterOpts{
|
|
|
|
|
Name: "prometheus_tsdb_checkpoint_deletions_total",
|
|
|
|
|
Help: "Total number of checkpoint deletions attempted.",
|
|
|
|
|
}),
|
|
|
|
|
checkpointCreationFail: prometheus.NewCounter(prometheus.CounterOpts{
|
|
|
|
|
Name: "prometheus_tsdb_checkpoint_creations_failed_total",
|
|
|
|
|
Help: "Total number of checkpoint creations that failed.",
|
|
|
|
|
}),
|
|
|
|
|
checkpointCreationTotal: prometheus.NewCounter(prometheus.CounterOpts{
|
|
|
|
|
Name: "prometheus_tsdb_checkpoint_creations_total",
|
|
|
|
|
Help: "Total number of checkpoint creations attempted.",
|
|
|
|
|
}),
|
2020-05-06 15:30:00 +00:00
|
|
|
|
mmapChunkCorruptionTotal: prometheus.NewCounter(prometheus.CounterOpts{
|
|
|
|
|
Name: "prometheus_tsdb_mmap_chunk_corruptions_total",
|
|
|
|
|
Help: "Total number of memory-mapped chunk corruptions.",
|
|
|
|
|
}),
|
2021-08-17 17:08:16 +00:00
|
|
|
|
snapshotReplayErrorTotal: prometheus.NewCounter(prometheus.CounterOpts{
|
|
|
|
|
Name: "prometheus_tsdb_snapshot_replay_error_total",
|
|
|
|
|
Help: "Total number snapshot replays that failed.",
|
|
|
|
|
}),
|
2020-02-12 19:22:27 +00:00
|
|
|
|
}
|
2017-08-30 16:34:54 +00:00
|
|
|
|
|
|
|
|
|
if r != nil {
|
|
|
|
|
r.MustRegister(
|
|
|
|
|
m.activeAppenders,
|
2020-02-12 19:22:27 +00:00
|
|
|
|
m.series,
|
2017-08-30 16:34:54 +00:00
|
|
|
|
m.chunks,
|
|
|
|
|
m.chunksCreated,
|
|
|
|
|
m.chunksRemoved,
|
|
|
|
|
m.seriesCreated,
|
|
|
|
|
m.seriesRemoved,
|
2017-10-12 13:25:12 +00:00
|
|
|
|
m.seriesNotFound,
|
2017-08-30 16:34:54 +00:00
|
|
|
|
m.gcDuration,
|
|
|
|
|
m.walTruncateDuration,
|
2018-12-18 10:24:56 +00:00
|
|
|
|
m.walCorruptionsTotal,
|
2020-09-21 16:25:05 +00:00
|
|
|
|
m.walTotalReplayDuration,
|
2017-08-30 16:34:54 +00:00
|
|
|
|
m.samplesAppended,
|
2020-05-06 15:30:00 +00:00
|
|
|
|
m.outOfBoundSamples,
|
|
|
|
|
m.outOfOrderSamples,
|
2018-09-25 13:48:33 +00:00
|
|
|
|
m.headTruncateFail,
|
|
|
|
|
m.headTruncateTotal,
|
2018-09-25 11:49:09 +00:00
|
|
|
|
m.checkpointDeleteFail,
|
2018-09-25 13:48:33 +00:00
|
|
|
|
m.checkpointDeleteTotal,
|
|
|
|
|
m.checkpointCreationFail,
|
|
|
|
|
m.checkpointCreationTotal,
|
2020-05-06 15:30:00 +00:00
|
|
|
|
m.mmapChunkCorruptionTotal,
|
2021-08-17 17:08:16 +00:00
|
|
|
|
m.snapshotReplayErrorTotal,
|
2020-02-12 19:22:27 +00:00
|
|
|
|
// Metrics bound to functions and not needed in tests
|
|
|
|
|
// can be created and registered on the spot.
|
|
|
|
|
prometheus.NewGaugeFunc(prometheus.GaugeOpts{
|
|
|
|
|
Name: "prometheus_tsdb_head_max_time",
|
|
|
|
|
Help: "Maximum timestamp of the head block. The unit is decided by the library consumer.",
|
|
|
|
|
}, func() float64 {
|
|
|
|
|
return float64(h.MaxTime())
|
|
|
|
|
}),
|
|
|
|
|
prometheus.NewGaugeFunc(prometheus.GaugeOpts{
|
|
|
|
|
Name: "prometheus_tsdb_head_min_time",
|
|
|
|
|
Help: "Minimum time bound of the head block. The unit is decided by the library consumer.",
|
|
|
|
|
}, func() float64 {
|
|
|
|
|
return float64(h.MinTime())
|
|
|
|
|
}),
|
|
|
|
|
prometheus.NewGaugeFunc(prometheus.GaugeOpts{
|
|
|
|
|
Name: "prometheus_tsdb_isolation_low_watermark",
|
|
|
|
|
Help: "The lowest TSDB append ID that is still referenced.",
|
|
|
|
|
}, func() float64 {
|
|
|
|
|
return float64(h.iso.lowWatermark())
|
|
|
|
|
}),
|
|
|
|
|
prometheus.NewGaugeFunc(prometheus.GaugeOpts{
|
|
|
|
|
Name: "prometheus_tsdb_isolation_high_watermark",
|
|
|
|
|
Help: "The highest TSDB append ID that has been given out.",
|
|
|
|
|
}, func() float64 {
|
2020-06-03 18:09:05 +00:00
|
|
|
|
return float64(h.iso.lastAppendID())
|
2020-02-12 19:22:27 +00:00
|
|
|
|
}),
|
2017-08-30 16:34:54 +00:00
|
|
|
|
)
|
|
|
|
|
}
|
|
|
|
|
return m
|
|
|
|
|
}
|
|
|
|
|
|
2021-08-03 12:14:26 +00:00
|
|
|
|
func mmappedChunksDir(dir string) string { return filepath.Join(dir, "chunks_head") }
|
|
|
|
|
|
2021-06-05 14:29:32 +00:00
|
|
|
|
// HeadStats are the statistics for the head component of the DB.
|
|
|
|
|
type HeadStats struct {
|
|
|
|
|
WALReplayStatus *WALReplayStatus
|
|
|
|
|
}
|
|
|
|
|
|
2021-08-03 12:14:26 +00:00
|
|
|
|
// NewHeadStats returns a new HeadStats object.
|
|
|
|
|
func NewHeadStats() *HeadStats {
|
|
|
|
|
return &HeadStats{
|
|
|
|
|
WALReplayStatus: &WALReplayStatus{},
|
2017-10-07 13:55:11 +00:00
|
|
|
|
}
|
2021-08-03 12:14:26 +00:00
|
|
|
|
}
|
2017-10-11 08:12:29 +00:00
|
|
|
|
|
2021-08-03 12:14:26 +00:00
|
|
|
|
// WALReplayStatus contains status information about the WAL replay.
|
|
|
|
|
type WALReplayStatus struct {
|
|
|
|
|
sync.RWMutex
|
|
|
|
|
Min int
|
|
|
|
|
Max int
|
|
|
|
|
Current int
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
// GetWALReplayStatus returns the WAL replay status information.
|
|
|
|
|
func (s *WALReplayStatus) GetWALReplayStatus() WALReplayStatus {
|
|
|
|
|
s.RLock()
|
|
|
|
|
defer s.RUnlock()
|
2019-06-14 15:39:22 +00:00
|
|
|
|
|
2021-08-03 12:14:26 +00:00
|
|
|
|
return WALReplayStatus{
|
|
|
|
|
Min: s.Min,
|
|
|
|
|
Max: s.Max,
|
|
|
|
|
Current: s.Current,
|
2018-05-17 13:04:32 +00:00
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2021-08-03 12:14:26 +00:00
|
|
|
|
const cardinalityCacheExpirationTime = time.Duration(30) * time.Second
|
|
|
|
|
|
2018-05-17 13:04:32 +00:00
|
|
|
|
// Init loads data from the write ahead log and prepares the head for writes.
|
2020-02-12 19:22:27 +00:00
|
|
|
|
// It should be called before using an appender so that it
|
2018-12-04 10:30:49 +00:00
|
|
|
|
// limits the ingested samples to the head min valid time.
|
2021-08-30 14:04:38 +00:00
|
|
|
|
func (h *Head) Init(minValidTime int64) error {
|
2020-07-28 04:42:42 +00:00
|
|
|
|
h.minValidTime.Store(minValidTime)
|
2018-05-17 13:04:32 +00:00
|
|
|
|
defer h.postings.EnsureOrder()
|
2018-12-04 10:30:49 +00:00
|
|
|
|
defer h.gc() // After loading the wal remove the obsolete data from the head.
|
2021-08-06 16:51:01 +00:00
|
|
|
|
defer func() {
|
|
|
|
|
// Loading of m-mapped chunks and snapshot can make the mint of the Head
|
|
|
|
|
// to go below minValidTime.
|
|
|
|
|
if h.MinTime() < h.minValidTime.Load() {
|
|
|
|
|
h.minTime.Store(h.minValidTime.Load())
|
|
|
|
|
}
|
|
|
|
|
}()
|
2018-05-17 13:04:32 +00:00
|
|
|
|
|
2020-07-16 13:04:08 +00:00
|
|
|
|
level.Info(h.logger).Log("msg", "Replaying on-disk memory mappable chunks if any")
|
2020-03-03 14:11:14 +00:00
|
|
|
|
start := time.Now()
|
2020-05-06 15:30:00 +00:00
|
|
|
|
|
2021-08-17 17:08:16 +00:00
|
|
|
|
snapIdx, snapOffset := -1, 0
|
|
|
|
|
refSeries := make(map[uint64]*memSeries)
|
|
|
|
|
|
|
|
|
|
if h.opts.EnableMemorySnapshotOnShutdown {
|
|
|
|
|
level.Info(h.logger).Log("msg", "Chunk snapshot is enabled, replaying from the snapshot")
|
2021-08-30 14:04:38 +00:00
|
|
|
|
var err error
|
2021-08-17 17:08:16 +00:00
|
|
|
|
snapIdx, snapOffset, refSeries, err = h.loadChunkSnapshot()
|
|
|
|
|
if err != nil {
|
|
|
|
|
snapIdx, snapOffset = -1, 0
|
|
|
|
|
h.metrics.snapshotReplayErrorTotal.Inc()
|
|
|
|
|
level.Error(h.logger).Log("msg", "Failed to load chunk snapshot", "err", err)
|
|
|
|
|
// We clear the partially loaded data to replay fresh from the WAL.
|
|
|
|
|
if err := h.resetInMemoryState(); err != nil {
|
|
|
|
|
return err
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
level.Info(h.logger).Log("msg", "Chunk snapshot loading time", "duration", time.Since(start).String())
|
2021-08-06 16:51:01 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
mmapChunkReplayStart := time.Now()
|
|
|
|
|
mmappedChunks, err := h.loadMmappedChunks(refSeries)
|
2020-05-06 15:30:00 +00:00
|
|
|
|
if err != nil {
|
|
|
|
|
level.Error(h.logger).Log("msg", "Loading on-disk chunks failed", "err", err)
|
|
|
|
|
if _, ok := errors.Cause(err).(*chunks.CorruptionErr); ok {
|
|
|
|
|
h.metrics.mmapChunkCorruptionTotal.Inc()
|
|
|
|
|
}
|
|
|
|
|
// If this fails, data will be recovered from WAL.
|
|
|
|
|
// Hence we wont lose any data (given WAL is not corrupt).
|
2021-08-06 16:51:01 +00:00
|
|
|
|
mmappedChunks = h.removeCorruptedMmappedChunks(err, refSeries)
|
2020-05-06 15:30:00 +00:00
|
|
|
|
}
|
|
|
|
|
|
2021-08-06 16:51:01 +00:00
|
|
|
|
level.Info(h.logger).Log("msg", "On-disk memory mappable chunks replay completed", "duration", time.Since(mmapChunkReplayStart).String())
|
2020-07-16 13:04:08 +00:00
|
|
|
|
if h.wal == nil {
|
|
|
|
|
level.Info(h.logger).Log("msg", "WAL not found")
|
|
|
|
|
return nil
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
level.Info(h.logger).Log("msg", "Replaying WAL, this may take a while")
|
|
|
|
|
|
|
|
|
|
checkpointReplayStart := time.Now()
|
2018-05-17 13:04:32 +00:00
|
|
|
|
// Backfill the checkpoint first if it exists.
|
2019-09-19 09:15:41 +00:00
|
|
|
|
dir, startFrom, err := wal.LastCheckpoint(h.wal.Dir())
|
|
|
|
|
if err != nil && err != record.ErrNotFound {
|
2018-05-17 13:04:32 +00:00
|
|
|
|
return errors.Wrap(err, "find last checkpoint")
|
|
|
|
|
}
|
2021-07-13 10:08:07 +00:00
|
|
|
|
|
|
|
|
|
// Find the last segment.
|
|
|
|
|
_, endAt, e := wal.Segments(h.wal.Dir())
|
|
|
|
|
if e != nil {
|
|
|
|
|
return errors.Wrap(e, "finding WAL segments")
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
h.startWALReplayStatus(startFrom, endAt)
|
|
|
|
|
|
2019-06-06 13:28:54 +00:00
|
|
|
|
multiRef := map[uint64]uint64{}
|
2021-08-25 16:18:55 +00:00
|
|
|
|
if err == nil && startFrom >= snapIdx {
|
2018-11-30 14:46:16 +00:00
|
|
|
|
sr, err := wal.NewSegmentsReader(dir)
|
2018-05-17 13:04:32 +00:00
|
|
|
|
if err != nil {
|
|
|
|
|
return errors.Wrap(err, "open checkpoint")
|
|
|
|
|
}
|
2019-06-14 15:39:22 +00:00
|
|
|
|
defer func() {
|
|
|
|
|
if err := sr.Close(); err != nil {
|
2020-04-11 08:22:18 +00:00
|
|
|
|
level.Warn(h.logger).Log("msg", "Error while closing the wal segments reader", "err", err)
|
2019-06-14 15:39:22 +00:00
|
|
|
|
}
|
|
|
|
|
}()
|
2018-05-17 13:04:32 +00:00
|
|
|
|
|
|
|
|
|
// A corrupted checkpoint is a hard error for now and requires user
|
|
|
|
|
// intervention. There's likely little data that can be recovered anyway.
|
2020-05-06 15:30:00 +00:00
|
|
|
|
if err := h.loadWAL(wal.NewReader(sr), multiRef, mmappedChunks); err != nil {
|
2018-05-17 13:04:32 +00:00
|
|
|
|
return errors.Wrap(err, "backfill checkpoint")
|
|
|
|
|
}
|
2021-07-13 10:08:07 +00:00
|
|
|
|
h.updateWALReplayStatusRead(startFrom)
|
2018-10-11 15:23:52 +00:00
|
|
|
|
startFrom++
|
2019-07-13 17:10:44 +00:00
|
|
|
|
level.Info(h.logger).Log("msg", "WAL checkpoint loaded")
|
2018-05-17 13:04:32 +00:00
|
|
|
|
}
|
2020-07-16 13:04:08 +00:00
|
|
|
|
checkpointReplayDuration := time.Since(checkpointReplayStart)
|
2018-05-17 13:04:32 +00:00
|
|
|
|
|
2020-07-16 13:04:08 +00:00
|
|
|
|
walReplayStart := time.Now()
|
2021-06-05 14:29:32 +00:00
|
|
|
|
|
2021-08-06 16:51:01 +00:00
|
|
|
|
if snapIdx > startFrom {
|
|
|
|
|
startFrom = snapIdx
|
|
|
|
|
}
|
2019-05-24 18:33:28 +00:00
|
|
|
|
// Backfill segments from the most recent checkpoint onwards.
|
2021-07-13 10:08:07 +00:00
|
|
|
|
for i := startFrom; i <= endAt; i++ {
|
2019-05-24 18:33:28 +00:00
|
|
|
|
s, err := wal.OpenReadSegment(wal.SegmentName(h.wal.Dir(), i))
|
|
|
|
|
if err != nil {
|
|
|
|
|
return errors.Wrap(err, fmt.Sprintf("open WAL segment: %d", i))
|
|
|
|
|
}
|
|
|
|
|
|
2021-08-06 16:51:01 +00:00
|
|
|
|
offset := 0
|
|
|
|
|
if i == snapIdx {
|
|
|
|
|
offset = snapOffset
|
|
|
|
|
}
|
|
|
|
|
sr, err := wal.NewSegmentBufReaderWithOffset(offset, s)
|
2021-08-17 17:08:16 +00:00
|
|
|
|
if errors.Cause(err) == io.EOF {
|
|
|
|
|
// File does not exist.
|
|
|
|
|
continue
|
|
|
|
|
}
|
2021-08-06 16:51:01 +00:00
|
|
|
|
if err != nil {
|
|
|
|
|
return errors.Wrapf(err, "segment reader (offset=%d)", offset)
|
|
|
|
|
}
|
2020-05-06 15:30:00 +00:00
|
|
|
|
err = h.loadWAL(wal.NewReader(sr), multiRef, mmappedChunks)
|
2019-06-14 15:39:22 +00:00
|
|
|
|
if err := sr.Close(); err != nil {
|
2020-04-11 08:22:18 +00:00
|
|
|
|
level.Warn(h.logger).Log("msg", "Error while closing the wal segments reader", "err", err)
|
2019-05-24 18:33:28 +00:00
|
|
|
|
}
|
2019-06-14 15:39:22 +00:00
|
|
|
|
if err != nil {
|
|
|
|
|
return err
|
2019-05-24 18:33:28 +00:00
|
|
|
|
}
|
2021-07-13 10:08:07 +00:00
|
|
|
|
level.Info(h.logger).Log("msg", "WAL segment loaded", "segment", i, "maxSegment", endAt)
|
2021-06-05 14:29:32 +00:00
|
|
|
|
h.updateWALReplayStatusRead(i)
|
2017-10-07 13:55:11 +00:00
|
|
|
|
}
|
2019-05-24 18:33:28 +00:00
|
|
|
|
|
2020-09-21 16:25:05 +00:00
|
|
|
|
walReplayDuration := time.Since(start)
|
|
|
|
|
h.metrics.walTotalReplayDuration.Set(walReplayDuration.Seconds())
|
2020-07-16 13:04:08 +00:00
|
|
|
|
level.Info(h.logger).Log(
|
|
|
|
|
"msg", "WAL replay completed",
|
|
|
|
|
"checkpoint_replay_duration", checkpointReplayDuration.String(),
|
|
|
|
|
"wal_replay_duration", time.Since(walReplayStart).String(),
|
2020-09-21 16:25:05 +00:00
|
|
|
|
"total_replay_duration", walReplayDuration.String(),
|
2020-07-16 13:04:08 +00:00
|
|
|
|
)
|
2020-03-03 14:11:14 +00:00
|
|
|
|
|
2017-08-30 16:34:54 +00:00
|
|
|
|
return nil
|
2017-05-13 16:14:18 +00:00
|
|
|
|
}
|
2017-01-06 11:37:28 +00:00
|
|
|
|
|
2021-08-06 16:51:01 +00:00
|
|
|
|
func (h *Head) loadMmappedChunks(refSeries map[uint64]*memSeries) (map[uint64][]*mmappedChunk, error) {
|
2020-05-06 15:30:00 +00:00
|
|
|
|
mmappedChunks := map[uint64][]*mmappedChunk{}
|
2021-10-13 12:14:32 +00:00
|
|
|
|
if err := h.chunkDiskMapper.IterateAllChunks(func(seriesRef uint64, chunkRef chunks.ChunkDiskMapperRef, mint, maxt int64, numSamples uint16) error {
|
2020-07-28 04:42:42 +00:00
|
|
|
|
if maxt < h.minValidTime.Load() {
|
2020-05-06 15:30:00 +00:00
|
|
|
|
return nil
|
|
|
|
|
}
|
2021-08-06 16:51:01 +00:00
|
|
|
|
ms, ok := refSeries[seriesRef]
|
|
|
|
|
if !ok {
|
|
|
|
|
slice := mmappedChunks[seriesRef]
|
|
|
|
|
if len(slice) > 0 && slice[len(slice)-1].maxTime >= mint {
|
|
|
|
|
return errors.Errorf("out of sequence m-mapped chunk for series ref %d", seriesRef)
|
2020-05-06 15:30:00 +00:00
|
|
|
|
}
|
2021-08-06 16:51:01 +00:00
|
|
|
|
|
|
|
|
|
slice = append(slice, &mmappedChunk{
|
|
|
|
|
ref: chunkRef,
|
|
|
|
|
minTime: mint,
|
|
|
|
|
maxTime: maxt,
|
|
|
|
|
numSamples: numSamples,
|
|
|
|
|
})
|
|
|
|
|
mmappedChunks[seriesRef] = slice
|
|
|
|
|
return nil
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
if len(ms.mmappedChunks) > 0 && ms.mmappedChunks[len(ms.mmappedChunks)-1].maxTime >= mint {
|
|
|
|
|
return errors.Errorf("out of sequence m-mapped chunk for series ref %d", seriesRef)
|
2020-05-06 15:30:00 +00:00
|
|
|
|
}
|
|
|
|
|
|
2021-08-06 16:51:01 +00:00
|
|
|
|
h.metrics.chunks.Inc()
|
|
|
|
|
h.metrics.chunksCreated.Inc()
|
|
|
|
|
ms.mmappedChunks = append(ms.mmappedChunks, &mmappedChunk{
|
2020-05-06 15:30:00 +00:00
|
|
|
|
ref: chunkRef,
|
|
|
|
|
minTime: mint,
|
|
|
|
|
maxTime: maxt,
|
|
|
|
|
numSamples: numSamples,
|
|
|
|
|
})
|
2021-08-06 16:51:01 +00:00
|
|
|
|
h.updateMinMaxTime(mint, maxt)
|
|
|
|
|
if ms.headChunk != nil && maxt >= ms.headChunk.minTime {
|
|
|
|
|
// The head chunk was completed and was m-mapped after taking the snapshot.
|
|
|
|
|
// Hence remove this chunk.
|
|
|
|
|
ms.nextAt = 0
|
|
|
|
|
ms.headChunk = nil
|
|
|
|
|
ms.app = nil
|
|
|
|
|
}
|
2020-05-06 15:30:00 +00:00
|
|
|
|
return nil
|
|
|
|
|
}); err != nil {
|
|
|
|
|
return nil, errors.Wrap(err, "iterate on on-disk chunks")
|
|
|
|
|
}
|
|
|
|
|
return mmappedChunks, nil
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
// removeCorruptedMmappedChunks attempts to delete the corrupted mmapped chunks and if it fails, it clears all the previously
|
|
|
|
|
// loaded mmapped chunks.
|
2021-08-06 16:51:01 +00:00
|
|
|
|
func (h *Head) removeCorruptedMmappedChunks(err error, refSeries map[uint64]*memSeries) map[uint64][]*mmappedChunk {
|
2020-05-06 15:30:00 +00:00
|
|
|
|
level.Info(h.logger).Log("msg", "Deleting mmapped chunk files")
|
|
|
|
|
|
|
|
|
|
if err := h.chunkDiskMapper.DeleteCorrupted(err); err != nil {
|
|
|
|
|
level.Info(h.logger).Log("msg", "Deletion of mmap chunk files failed, discarding chunk files completely", "err", err)
|
|
|
|
|
return map[uint64][]*mmappedChunk{}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
level.Info(h.logger).Log("msg", "Deletion of mmap chunk files successful, reattempting m-mapping the on-disk chunks")
|
2021-08-06 16:51:01 +00:00
|
|
|
|
mmappedChunks, err := h.loadMmappedChunks(refSeries)
|
2020-05-06 15:30:00 +00:00
|
|
|
|
if err != nil {
|
|
|
|
|
level.Error(h.logger).Log("msg", "Loading on-disk chunks failed, discarding chunk files completely", "err", err)
|
|
|
|
|
mmappedChunks = map[uint64][]*mmappedChunk{}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
return mmappedChunks
|
|
|
|
|
}
|
|
|
|
|
|
2021-08-03 12:14:26 +00:00
|
|
|
|
func (h *Head) ApplyConfig(cfg *config.Config) error {
|
|
|
|
|
if !h.opts.EnableExemplarStorage {
|
|
|
|
|
return nil
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
// Head uses opts.MaxExemplars in combination with opts.EnableExemplarStorage
|
|
|
|
|
// to decide if it should pass exemplars along to it's exemplar storage, so we
|
|
|
|
|
// need to update opts.MaxExemplars here.
|
|
|
|
|
prevSize := h.opts.MaxExemplars.Load()
|
|
|
|
|
h.opts.MaxExemplars.Store(cfg.StorageConfig.ExemplarsConfig.MaxExemplars)
|
|
|
|
|
|
|
|
|
|
if prevSize == h.opts.MaxExemplars.Load() {
|
|
|
|
|
return nil
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
migrated := h.exemplars.(*CircularExemplarStorage).Resize(h.opts.MaxExemplars.Load())
|
|
|
|
|
level.Info(h.logger).Log("msg", "Exemplar storage resized", "from", prevSize, "to", h.opts.MaxExemplars, "migrated", migrated)
|
|
|
|
|
return nil
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
// PostingsCardinalityStats returns top 10 highest cardinality stats By label and value names.
|
|
|
|
|
func (h *Head) PostingsCardinalityStats(statsByLabelName string) *index.PostingsStats {
|
|
|
|
|
h.cardinalityMutex.Lock()
|
|
|
|
|
defer h.cardinalityMutex.Unlock()
|
|
|
|
|
currentTime := time.Duration(time.Now().Unix()) * time.Second
|
|
|
|
|
seconds := currentTime - h.lastPostingsStatsCall
|
|
|
|
|
if seconds > cardinalityCacheExpirationTime {
|
|
|
|
|
h.cardinalityCache = nil
|
|
|
|
|
}
|
|
|
|
|
if h.cardinalityCache != nil {
|
|
|
|
|
return h.cardinalityCache
|
|
|
|
|
}
|
|
|
|
|
h.cardinalityCache = h.postings.Stats(statsByLabelName)
|
|
|
|
|
h.lastPostingsStatsCall = time.Duration(time.Now().Unix()) * time.Second
|
|
|
|
|
|
|
|
|
|
return h.cardinalityCache
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
func (h *Head) updateMinMaxTime(mint, maxt int64) {
|
|
|
|
|
for {
|
|
|
|
|
lt := h.MinTime()
|
|
|
|
|
if mint >= lt {
|
|
|
|
|
break
|
|
|
|
|
}
|
|
|
|
|
if h.minTime.CAS(lt, mint) {
|
|
|
|
|
break
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
for {
|
|
|
|
|
ht := h.MaxTime()
|
|
|
|
|
if maxt <= ht {
|
|
|
|
|
break
|
|
|
|
|
}
|
|
|
|
|
if h.maxTime.CAS(ht, maxt) {
|
|
|
|
|
break
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
// SetMinValidTime sets the minimum timestamp the head can ingest.
|
|
|
|
|
func (h *Head) SetMinValidTime(minValidTime int64) {
|
|
|
|
|
h.minValidTime.Store(minValidTime)
|
|
|
|
|
}
|
|
|
|
|
|
2020-10-19 15:27:08 +00:00
|
|
|
|
// Truncate removes old data before mint from the head and WAL.
|
2018-09-25 13:48:33 +00:00
|
|
|
|
func (h *Head) Truncate(mint int64) (err error) {
|
2020-10-19 15:27:08 +00:00
|
|
|
|
initialize := h.MinTime() == math.MaxInt64
|
|
|
|
|
if err := h.truncateMemory(mint); err != nil {
|
|
|
|
|
return err
|
|
|
|
|
}
|
|
|
|
|
if initialize {
|
|
|
|
|
return nil
|
|
|
|
|
}
|
|
|
|
|
return h.truncateWAL(mint)
|
|
|
|
|
}
|
|
|
|
|
|
2021-09-16 06:50:03 +00:00
|
|
|
|
// OverlapsClosedInterval returns true if the head overlaps [mint, maxt].
|
|
|
|
|
func (h *Head) OverlapsClosedInterval(mint, maxt int64) bool {
|
|
|
|
|
return h.MinTime() <= maxt && mint <= h.MaxTime()
|
|
|
|
|
}
|
|
|
|
|
|
2020-10-19 15:27:08 +00:00
|
|
|
|
// truncateMemory removes old data before mint from the head.
|
|
|
|
|
func (h *Head) truncateMemory(mint int64) (err error) {
|
2021-08-06 16:51:01 +00:00
|
|
|
|
h.chunkSnapshotMtx.Lock()
|
|
|
|
|
defer h.chunkSnapshotMtx.Unlock()
|
|
|
|
|
|
2018-09-25 13:48:33 +00:00
|
|
|
|
defer func() {
|
|
|
|
|
if err != nil {
|
|
|
|
|
h.metrics.headTruncateFail.Inc()
|
|
|
|
|
}
|
|
|
|
|
}()
|
2021-07-20 08:47:20 +00:00
|
|
|
|
|
2018-05-25 21:19:32 +00:00
|
|
|
|
initialize := h.MinTime() == math.MaxInt64
|
2017-09-06 14:20:37 +00:00
|
|
|
|
|
2018-05-25 21:19:32 +00:00
|
|
|
|
if h.MinTime() >= mint && !initialize {
|
2017-09-01 12:38:49 +00:00
|
|
|
|
return nil
|
2017-08-30 16:34:54 +00:00
|
|
|
|
}
|
2021-07-20 08:47:20 +00:00
|
|
|
|
|
|
|
|
|
// The order of these two Store() should not be changed,
|
|
|
|
|
// i.e. truncation time is set before in-process boolean.
|
|
|
|
|
h.lastMemoryTruncationTime.Store(mint)
|
|
|
|
|
h.memTruncationInProcess.Store(true)
|
|
|
|
|
defer h.memTruncationInProcess.Store(false)
|
|
|
|
|
|
|
|
|
|
// We wait for pending queries to end that overlap with this truncation.
|
|
|
|
|
if !initialize {
|
|
|
|
|
h.WaitForPendingReadersInTimeRange(h.MinTime(), mint)
|
|
|
|
|
}
|
|
|
|
|
|
2020-07-28 04:42:42 +00:00
|
|
|
|
h.minTime.Store(mint)
|
|
|
|
|
h.minValidTime.Store(mint)
|
2017-08-30 16:34:54 +00:00
|
|
|
|
|
2017-09-07 11:04:02 +00:00
|
|
|
|
// Ensure that max time is at least as high as min time.
|
|
|
|
|
for h.MaxTime() < mint {
|
2020-07-28 04:42:42 +00:00
|
|
|
|
h.maxTime.CAS(h.MaxTime(), mint)
|
2017-09-07 11:04:02 +00:00
|
|
|
|
}
|
|
|
|
|
|
2017-09-06 14:20:37 +00:00
|
|
|
|
// This was an initial call to Truncate after loading blocks on startup.
|
|
|
|
|
// We haven't read back the WAL yet, so do not attempt to truncate it.
|
|
|
|
|
if initialize {
|
|
|
|
|
return nil
|
|
|
|
|
}
|
|
|
|
|
|
2018-09-25 13:48:33 +00:00
|
|
|
|
h.metrics.headTruncateTotal.Inc()
|
2017-08-30 16:34:54 +00:00
|
|
|
|
start := time.Now()
|
|
|
|
|
|
2020-11-25 13:03:30 +00:00
|
|
|
|
actualMint := h.gc()
|
2020-04-11 08:22:18 +00:00
|
|
|
|
level.Info(h.logger).Log("msg", "Head GC completed", "duration", time.Since(start))
|
2017-08-30 16:34:54 +00:00
|
|
|
|
h.metrics.gcDuration.Observe(time.Since(start).Seconds())
|
2020-11-25 13:03:30 +00:00
|
|
|
|
if actualMint > h.minTime.Load() {
|
|
|
|
|
// The actual mint of the Head is higher than the one asked to truncate.
|
|
|
|
|
appendableMinValidTime := h.appendableMinValidTime()
|
|
|
|
|
if actualMint < appendableMinValidTime {
|
|
|
|
|
h.minTime.Store(actualMint)
|
|
|
|
|
h.minValidTime.Store(actualMint)
|
|
|
|
|
} else {
|
|
|
|
|
// The actual min time is in the appendable window.
|
|
|
|
|
// So we set the mint to the appendableMinValidTime.
|
|
|
|
|
h.minTime.Store(appendableMinValidTime)
|
|
|
|
|
h.minValidTime.Store(appendableMinValidTime)
|
|
|
|
|
}
|
|
|
|
|
}
|
2017-08-30 16:34:54 +00:00
|
|
|
|
|
2020-05-06 15:30:00 +00:00
|
|
|
|
// Truncate the chunk m-mapper.
|
|
|
|
|
if err := h.chunkDiskMapper.Truncate(mint); err != nil {
|
|
|
|
|
return errors.Wrap(err, "truncate chunks.HeadReadWriter")
|
|
|
|
|
}
|
2020-10-19 15:27:08 +00:00
|
|
|
|
return nil
|
|
|
|
|
}
|
2020-05-06 15:30:00 +00:00
|
|
|
|
|
2021-07-20 08:47:20 +00:00
|
|
|
|
// WaitForPendingReadersInTimeRange waits for queries overlapping with given range to finish querying.
|
|
|
|
|
// The query timeout limits the max wait time of this function implicitly.
|
|
|
|
|
// The mint is inclusive and maxt is the truncation time hence exclusive.
|
|
|
|
|
func (h *Head) WaitForPendingReadersInTimeRange(mint, maxt int64) {
|
|
|
|
|
maxt-- // Making it inclusive before checking overlaps.
|
|
|
|
|
overlaps := func() bool {
|
|
|
|
|
o := false
|
|
|
|
|
h.iso.TraverseOpenReads(func(s *isolationState) bool {
|
|
|
|
|
if s.mint <= maxt && mint <= s.maxt {
|
|
|
|
|
// Overlaps with the truncation range.
|
|
|
|
|
o = true
|
|
|
|
|
return false
|
|
|
|
|
}
|
|
|
|
|
return true
|
|
|
|
|
})
|
|
|
|
|
return o
|
|
|
|
|
}
|
|
|
|
|
for overlaps() {
|
|
|
|
|
time.Sleep(500 * time.Millisecond)
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
// IsQuerierCollidingWithTruncation returns if the current querier needs to be closed and if a new querier
|
|
|
|
|
// has to be created. In the latter case, the method also returns the new mint to be used for creating the
|
|
|
|
|
// new range head and the new querier. This methods helps preventing races with the truncation of in-memory data.
|
|
|
|
|
//
|
|
|
|
|
// NOTE: The querier should already be taken before calling this.
|
2021-10-22 08:06:44 +00:00
|
|
|
|
func (h *Head) IsQuerierCollidingWithTruncation(querierMint, querierMaxt int64) (shouldClose, getNew bool, newMint int64) {
|
2021-07-20 08:47:20 +00:00
|
|
|
|
if !h.memTruncationInProcess.Load() {
|
|
|
|
|
return false, false, 0
|
|
|
|
|
}
|
|
|
|
|
// Head truncation is in process. It also means that the block that was
|
|
|
|
|
// created for this truncation range is also available.
|
|
|
|
|
// Check if we took a querier that overlaps with this truncation.
|
|
|
|
|
memTruncTime := h.lastMemoryTruncationTime.Load()
|
|
|
|
|
if querierMaxt < memTruncTime {
|
|
|
|
|
// Head compaction has happened and this time range is being truncated.
|
|
|
|
|
// This query doesn't overlap with the Head any longer.
|
|
|
|
|
// We should close this querier to avoid races and the data would be
|
|
|
|
|
// available with the blocks below.
|
|
|
|
|
// Cases:
|
|
|
|
|
// 1. |------truncation------|
|
|
|
|
|
// |---query---|
|
|
|
|
|
// 2. |------truncation------|
|
|
|
|
|
// |---query---|
|
|
|
|
|
return true, false, 0
|
|
|
|
|
}
|
|
|
|
|
if querierMint < memTruncTime {
|
|
|
|
|
// The truncation time is not same as head mint that we saw above but the
|
|
|
|
|
// query still overlaps with the Head.
|
|
|
|
|
// The truncation started after we got the querier. So it is not safe
|
|
|
|
|
// to use this querier and/or might block truncation. We should get
|
|
|
|
|
// a new querier for the new Head range while remaining will be available
|
|
|
|
|
// in the blocks below.
|
|
|
|
|
// Case:
|
|
|
|
|
// |------truncation------|
|
|
|
|
|
// |----query----|
|
|
|
|
|
// Turns into
|
|
|
|
|
// |------truncation------|
|
|
|
|
|
// |---qu---|
|
|
|
|
|
return true, true, memTruncTime
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
// Other case is this, which is a no-op
|
|
|
|
|
// |------truncation------|
|
|
|
|
|
// |---query---|
|
|
|
|
|
return false, false, 0
|
|
|
|
|
}
|
|
|
|
|
|
2020-10-19 15:27:08 +00:00
|
|
|
|
// truncateWAL removes old data before mint from the WAL.
|
|
|
|
|
func (h *Head) truncateWAL(mint int64) error {
|
2021-08-06 16:51:01 +00:00
|
|
|
|
h.chunkSnapshotMtx.Lock()
|
|
|
|
|
defer h.chunkSnapshotMtx.Unlock()
|
|
|
|
|
|
2020-10-19 15:27:08 +00:00
|
|
|
|
if h.wal == nil || mint <= h.lastWALTruncationTime.Load() {
|
2018-05-17 13:04:32 +00:00
|
|
|
|
return nil
|
|
|
|
|
}
|
2020-10-19 15:27:08 +00:00
|
|
|
|
start := time.Now()
|
|
|
|
|
h.lastWALTruncationTime.Store(mint)
|
2017-08-30 16:34:54 +00:00
|
|
|
|
|
2020-09-01 09:16:57 +00:00
|
|
|
|
first, last, err := wal.Segments(h.wal.Dir())
|
2018-05-17 13:04:32 +00:00
|
|
|
|
if err != nil {
|
|
|
|
|
return errors.Wrap(err, "get segment range")
|
|
|
|
|
}
|
2019-06-07 10:35:02 +00:00
|
|
|
|
// Start a new segment, so low ingestion volume TSDB don't have more WAL than
|
|
|
|
|
// needed.
|
2020-10-19 15:27:08 +00:00
|
|
|
|
if err := h.wal.NextSegment(); err != nil {
|
2019-06-07 10:35:02 +00:00
|
|
|
|
return errors.Wrap(err, "next segment")
|
|
|
|
|
}
|
2018-10-11 15:23:52 +00:00
|
|
|
|
last-- // Never consider last segment for checkpoint.
|
|
|
|
|
if last < 0 {
|
2018-05-17 13:04:32 +00:00
|
|
|
|
return nil // no segments yet.
|
|
|
|
|
}
|
2020-04-07 10:25:57 +00:00
|
|
|
|
// The lower two thirds of segments should contain mostly obsolete samples.
|
|
|
|
|
// If we have less than two segments, it's not worth checkpointing yet.
|
|
|
|
|
// With the default 2h blocks, this will keeping up to around 3h worth
|
|
|
|
|
// of WAL segments.
|
|
|
|
|
last = first + (last-first)*2/3
|
2018-10-11 15:23:52 +00:00
|
|
|
|
if last <= first {
|
2018-05-17 13:04:32 +00:00
|
|
|
|
return nil
|
|
|
|
|
}
|
|
|
|
|
|
2017-09-21 09:02:30 +00:00
|
|
|
|
keep := func(id uint64) bool {
|
2019-04-09 13:16:24 +00:00
|
|
|
|
if h.series.getByID(id) != nil {
|
|
|
|
|
return true
|
|
|
|
|
}
|
|
|
|
|
h.deletedMtx.Lock()
|
|
|
|
|
_, ok := h.deleted[id]
|
|
|
|
|
h.deletedMtx.Unlock()
|
|
|
|
|
return ok
|
2017-08-31 09:39:22 +00:00
|
|
|
|
}
|
2018-09-25 13:48:33 +00:00
|
|
|
|
h.metrics.checkpointCreationTotal.Inc()
|
2020-07-15 13:45:37 +00:00
|
|
|
|
if _, err = wal.Checkpoint(h.logger, h.wal, first, last, keep, mint); err != nil {
|
2018-09-25 13:48:33 +00:00
|
|
|
|
h.metrics.checkpointCreationFail.Inc()
|
2020-07-05 05:55:42 +00:00
|
|
|
|
if _, ok := errors.Cause(err).(*wal.CorruptionErr); ok {
|
|
|
|
|
h.metrics.walCorruptionsTotal.Inc()
|
|
|
|
|
}
|
2018-05-17 13:04:32 +00:00
|
|
|
|
return errors.Wrap(err, "create checkpoint")
|
2017-08-30 16:34:54 +00:00
|
|
|
|
}
|
2018-10-11 15:23:52 +00:00
|
|
|
|
if err := h.wal.Truncate(last + 1); err != nil {
|
2018-09-25 13:48:33 +00:00
|
|
|
|
// If truncating fails, we'll just try again at the next checkpoint.
|
|
|
|
|
// Leftover segments will just be ignored in the future if there's a checkpoint
|
|
|
|
|
// that supersedes them.
|
|
|
|
|
level.Error(h.logger).Log("msg", "truncating segments failed", "err", err)
|
|
|
|
|
}
|
2019-04-09 13:16:24 +00:00
|
|
|
|
|
|
|
|
|
// The checkpoint is written and segments before it is truncated, so we no
|
|
|
|
|
// longer need to track deleted series that are before it.
|
|
|
|
|
h.deletedMtx.Lock()
|
|
|
|
|
for ref, segment := range h.deleted {
|
|
|
|
|
if segment < first {
|
|
|
|
|
delete(h.deleted, ref)
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
h.deletedMtx.Unlock()
|
|
|
|
|
|
2018-09-25 13:48:33 +00:00
|
|
|
|
h.metrics.checkpointDeleteTotal.Inc()
|
2019-09-19 09:15:41 +00:00
|
|
|
|
if err := wal.DeleteCheckpoints(h.wal.Dir(), last); err != nil {
|
2018-09-25 13:48:33 +00:00
|
|
|
|
// Leftover old checkpoints do not cause problems down the line beyond
|
|
|
|
|
// occupying disk space.
|
|
|
|
|
// They will just be ignored since a higher checkpoint exists.
|
|
|
|
|
level.Error(h.logger).Log("msg", "delete old checkpoints", "err", err)
|
|
|
|
|
h.metrics.checkpointDeleteFail.Inc()
|
|
|
|
|
}
|
2017-08-30 16:34:54 +00:00
|
|
|
|
h.metrics.walTruncateDuration.Observe(time.Since(start).Seconds())
|
2017-09-01 12:38:49 +00:00
|
|
|
|
|
2018-05-17 13:04:32 +00:00
|
|
|
|
level.Info(h.logger).Log("msg", "WAL checkpoint complete",
|
2018-10-11 15:23:52 +00:00
|
|
|
|
"first", first, "last", last, "duration", time.Since(start))
|
2018-05-17 13:04:32 +00:00
|
|
|
|
|
2017-09-01 12:38:49 +00:00
|
|
|
|
return nil
|
2017-08-30 16:34:54 +00:00
|
|
|
|
}
|
|
|
|
|
|
2020-04-29 16:16:14 +00:00
|
|
|
|
type Stats struct {
|
|
|
|
|
NumSeries uint64
|
|
|
|
|
MinTime, MaxTime int64
|
|
|
|
|
IndexPostingStats *index.PostingsStats
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
// Stats returns important current HEAD statistics. Note that it is expensive to
|
|
|
|
|
// calculate these.
|
|
|
|
|
func (h *Head) Stats(statsByLabelName string) *Stats {
|
|
|
|
|
return &Stats{
|
|
|
|
|
NumSeries: h.NumSeries(),
|
|
|
|
|
MaxTime: h.MaxTime(),
|
|
|
|
|
MinTime: h.MinTime(),
|
|
|
|
|
IndexPostingStats: h.PostingsCardinalityStats(statsByLabelName),
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2020-02-14 09:50:24 +00:00
|
|
|
|
type RangeHead struct {
|
2017-10-09 13:21:46 +00:00
|
|
|
|
head *Head
|
|
|
|
|
mint, maxt int64
|
|
|
|
|
}
|
|
|
|
|
|
2020-03-25 23:17:56 +00:00
|
|
|
|
// NewRangeHead returns a *RangeHead.
|
2020-02-14 09:50:24 +00:00
|
|
|
|
func NewRangeHead(head *Head, mint, maxt int64) *RangeHead {
|
|
|
|
|
return &RangeHead{
|
|
|
|
|
head: head,
|
|
|
|
|
mint: mint,
|
|
|
|
|
maxt: maxt,
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2020-03-25 19:13:47 +00:00
|
|
|
|
func (h *RangeHead) Index() (IndexReader, error) {
|
|
|
|
|
return h.head.indexRange(h.mint, h.maxt), nil
|
2017-10-09 13:21:46 +00:00
|
|
|
|
}
|
|
|
|
|
|
2020-02-14 09:50:24 +00:00
|
|
|
|
func (h *RangeHead) Chunks() (ChunkReader, error) {
|
2021-07-20 08:47:20 +00:00
|
|
|
|
return h.head.chunksRange(h.mint, h.maxt, h.head.iso.State(h.mint, h.maxt))
|
2017-10-09 13:21:46 +00:00
|
|
|
|
}
|
|
|
|
|
|
2020-02-14 09:50:24 +00:00
|
|
|
|
func (h *RangeHead) Tombstones() (tombstones.Reader, error) {
|
2020-01-20 15:38:00 +00:00
|
|
|
|
return h.head.tombstones, nil
|
2017-10-09 13:21:46 +00:00
|
|
|
|
}
|
|
|
|
|
|
2020-02-14 09:50:24 +00:00
|
|
|
|
func (h *RangeHead) MinTime() int64 {
|
2019-02-14 13:29:41 +00:00
|
|
|
|
return h.mint
|
|
|
|
|
}
|
|
|
|
|
|
2020-10-19 15:27:08 +00:00
|
|
|
|
// MaxTime returns the max time of actual data fetch-able from the head.
|
|
|
|
|
// This controls the chunks time range which is closed [b.MinTime, b.MaxTime].
|
2020-02-14 09:50:24 +00:00
|
|
|
|
func (h *RangeHead) MaxTime() int64 {
|
2019-02-14 13:29:41 +00:00
|
|
|
|
return h.maxt
|
|
|
|
|
}
|
|
|
|
|
|
2020-10-19 15:27:08 +00:00
|
|
|
|
// BlockMaxTime returns the max time of the potential block created from this head.
|
|
|
|
|
// It's different to MaxTime as we need to add +1 millisecond to block maxt because block
|
|
|
|
|
// intervals are half-open: [b.MinTime, b.MaxTime). Block intervals are always +1 than the total samples it includes.
|
|
|
|
|
func (h *RangeHead) BlockMaxTime() int64 {
|
|
|
|
|
return h.MaxTime() + 1
|
|
|
|
|
}
|
|
|
|
|
|
2020-02-14 09:50:24 +00:00
|
|
|
|
func (h *RangeHead) NumSeries() uint64 {
|
2019-07-23 08:04:48 +00:00
|
|
|
|
return h.head.NumSeries()
|
|
|
|
|
}
|
|
|
|
|
|
2020-02-14 09:50:24 +00:00
|
|
|
|
func (h *RangeHead) Meta() BlockMeta {
|
2019-07-23 08:04:48 +00:00
|
|
|
|
return BlockMeta{
|
|
|
|
|
MinTime: h.MinTime(),
|
|
|
|
|
MaxTime: h.MaxTime(),
|
|
|
|
|
ULID: h.head.Meta().ULID,
|
2021-08-03 12:14:26 +00:00
|
|
|
|
Stats: BlockStats{
|
|
|
|
|
NumSeries: h.NumSeries(),
|
|
|
|
|
},
|
2018-09-17 16:58:42 +00:00
|
|
|
|
}
|
2021-08-03 12:14:26 +00:00
|
|
|
|
}
|
2017-08-30 16:34:54 +00:00
|
|
|
|
|
2021-08-03 12:14:26 +00:00
|
|
|
|
// String returns an human readable representation of the range head. It's important to
|
|
|
|
|
// keep this function in order to avoid the struct dump when the head is stringified in
|
|
|
|
|
// errors or logs.
|
|
|
|
|
func (h *RangeHead) String() string {
|
|
|
|
|
return fmt.Sprintf("range head (mint: %d, maxt: %d)", h.MinTime(), h.MaxTime())
|
2017-08-30 16:34:54 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
// Delete all samples in the range of [mint, maxt] for series that satisfy the given
|
|
|
|
|
// label matchers.
|
2019-11-18 19:53:33 +00:00
|
|
|
|
func (h *Head) Delete(mint, maxt int64, ms ...*labels.Matcher) error {
|
2017-08-30 16:34:54 +00:00
|
|
|
|
// Do not delete anything beyond the currently valid range.
|
|
|
|
|
mint, maxt = clampInterval(mint, maxt, h.MinTime(), h.MaxTime())
|
|
|
|
|
|
|
|
|
|
ir := h.indexRange(mint, maxt)
|
|
|
|
|
|
2017-12-17 18:08:21 +00:00
|
|
|
|
p, err := PostingsForMatchers(ir, ms...)
|
2017-11-13 11:16:58 +00:00
|
|
|
|
if err != nil {
|
|
|
|
|
return errors.Wrap(err, "select series")
|
|
|
|
|
}
|
2017-08-30 16:34:54 +00:00
|
|
|
|
|
2019-09-19 09:15:41 +00:00
|
|
|
|
var stones []tombstones.Stone
|
2017-08-30 16:34:54 +00:00
|
|
|
|
for p.Next() {
|
2017-09-05 09:45:18 +00:00
|
|
|
|
series := h.series.getByID(p.At())
|
2017-08-30 16:34:54 +00:00
|
|
|
|
|
2020-03-16 12:59:22 +00:00
|
|
|
|
series.RLock()
|
2018-02-07 13:43:21 +00:00
|
|
|
|
t0, t1 := series.minTime(), series.maxTime()
|
2020-03-16 12:59:22 +00:00
|
|
|
|
series.RUnlock()
|
2018-02-07 13:43:21 +00:00
|
|
|
|
if t0 == math.MinInt64 || t1 == math.MinInt64 {
|
|
|
|
|
continue
|
|
|
|
|
}
|
2017-08-30 16:34:54 +00:00
|
|
|
|
// Delete only until the current values and not beyond.
|
2018-02-07 13:43:21 +00:00
|
|
|
|
t0, t1 = clampInterval(mint, maxt, t0, t1)
|
2020-01-20 15:38:00 +00:00
|
|
|
|
stones = append(stones, tombstones.Stone{Ref: p.At(), Intervals: tombstones.Intervals{{Mint: t0, Maxt: t1}}})
|
2017-08-30 16:34:54 +00:00
|
|
|
|
}
|
|
|
|
|
if p.Err() != nil {
|
|
|
|
|
return p.Err()
|
|
|
|
|
}
|
2018-05-17 13:04:32 +00:00
|
|
|
|
if h.wal != nil {
|
2020-01-20 15:38:00 +00:00
|
|
|
|
var enc record.Encoder
|
2018-05-17 13:04:32 +00:00
|
|
|
|
if err := h.wal.Log(enc.Tombstones(stones, nil)); err != nil {
|
|
|
|
|
return err
|
|
|
|
|
}
|
2017-08-30 16:34:54 +00:00
|
|
|
|
}
|
2020-01-20 15:38:00 +00:00
|
|
|
|
for _, s := range stones {
|
|
|
|
|
h.tombstones.AddInterval(s.Ref, s.Intervals[0])
|
2017-08-30 16:34:54 +00:00
|
|
|
|
}
|
2019-01-08 17:08:41 +00:00
|
|
|
|
|
2017-08-30 16:34:54 +00:00
|
|
|
|
return nil
|
|
|
|
|
}
|
|
|
|
|
|
2018-01-13 17:51:50 +00:00
|
|
|
|
// gc removes data before the minimum timestamp from the head.
|
2020-11-25 13:03:30 +00:00
|
|
|
|
// It returns the actual min times of the chunks present in the Head.
|
|
|
|
|
func (h *Head) gc() int64 {
|
2017-08-28 22:39:17 +00:00
|
|
|
|
// Only data strictly lower than this timestamp must be deleted.
|
|
|
|
|
mint := h.MinTime()
|
2017-01-19 13:01:38 +00:00
|
|
|
|
|
2017-09-05 09:45:18 +00:00
|
|
|
|
// Drop old chunks and remember series IDs and hashes if they can be
|
|
|
|
|
// deleted entirely.
|
2020-11-25 13:03:30 +00:00
|
|
|
|
deleted, chunksRemoved, actualMint := h.series.gc(mint)
|
2017-09-05 09:45:18 +00:00
|
|
|
|
seriesRemoved := len(deleted)
|
2017-03-20 09:41:43 +00:00
|
|
|
|
|
2017-09-05 09:45:18 +00:00
|
|
|
|
h.metrics.seriesRemoved.Add(float64(seriesRemoved))
|
|
|
|
|
h.metrics.chunksRemoved.Add(float64(chunksRemoved))
|
|
|
|
|
h.metrics.chunks.Sub(float64(chunksRemoved))
|
2020-07-28 04:42:42 +00:00
|
|
|
|
h.numSeries.Sub(uint64(seriesRemoved))
|
2017-03-04 15:50:48 +00:00
|
|
|
|
|
2017-11-30 14:34:49 +00:00
|
|
|
|
// Remove deleted series IDs from the postings lists.
|
|
|
|
|
h.postings.Delete(deleted)
|
2017-09-05 09:45:18 +00:00
|
|
|
|
|
2021-09-16 06:50:03 +00:00
|
|
|
|
// Remove tombstones referring to the deleted series.
|
|
|
|
|
h.tombstones.DeleteTombstones(deleted)
|
|
|
|
|
h.tombstones.TruncateBefore(mint)
|
|
|
|
|
|
2019-04-09 13:16:24 +00:00
|
|
|
|
if h.wal != nil {
|
2020-09-01 09:16:57 +00:00
|
|
|
|
_, last, _ := wal.Segments(h.wal.Dir())
|
2019-04-09 13:16:24 +00:00
|
|
|
|
h.deletedMtx.Lock()
|
|
|
|
|
// Keep series records until we're past segment 'last'
|
|
|
|
|
// because the WAL will still have samples records with
|
|
|
|
|
// this ref ID. If we didn't keep these series records then
|
|
|
|
|
// on start up when we replay the WAL, or any other code
|
|
|
|
|
// that reads the WAL, wouldn't be able to use those
|
|
|
|
|
// samples since we would have no labels for that ref ID.
|
|
|
|
|
for ref := range deleted {
|
|
|
|
|
h.deleted[ref] = last
|
|
|
|
|
}
|
|
|
|
|
h.deletedMtx.Unlock()
|
|
|
|
|
}
|
|
|
|
|
|
2020-11-25 13:03:30 +00:00
|
|
|
|
return actualMint
|
2017-03-20 09:21:21 +00:00
|
|
|
|
}
|
|
|
|
|
|
2017-10-09 13:21:46 +00:00
|
|
|
|
// Tombstones returns a new reader over the head's tombstones
|
2019-09-19 09:15:41 +00:00
|
|
|
|
func (h *Head) Tombstones() (tombstones.Reader, error) {
|
2020-01-20 15:38:00 +00:00
|
|
|
|
return h.tombstones, nil
|
2017-06-25 17:02:02 +00:00
|
|
|
|
}
|
|
|
|
|
|
2019-07-23 08:04:48 +00:00
|
|
|
|
// NumSeries returns the number of active series in the head.
|
|
|
|
|
func (h *Head) NumSeries() uint64 {
|
2020-07-28 04:42:42 +00:00
|
|
|
|
return h.numSeries.Load()
|
2019-07-23 08:04:48 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
// Meta returns meta information about the head.
|
|
|
|
|
// The head is dynamic so will return dynamic results.
|
|
|
|
|
func (h *Head) Meta() BlockMeta {
|
|
|
|
|
var id [16]byte
|
|
|
|
|
copy(id[:], "______head______")
|
|
|
|
|
return BlockMeta{
|
|
|
|
|
MinTime: h.MinTime(),
|
|
|
|
|
MaxTime: h.MaxTime(),
|
|
|
|
|
ULID: ulid.ULID(id),
|
|
|
|
|
Stats: BlockStats{
|
|
|
|
|
NumSeries: h.NumSeries(),
|
|
|
|
|
},
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2017-08-28 22:39:17 +00:00
|
|
|
|
// MinTime returns the lowest time bound on visible data in the head.
|
|
|
|
|
func (h *Head) MinTime() int64 {
|
2020-07-28 04:42:42 +00:00
|
|
|
|
return h.minTime.Load()
|
2017-01-12 18:18:51 +00:00
|
|
|
|
}
|
|
|
|
|
|
2017-08-28 22:39:17 +00:00
|
|
|
|
// MaxTime returns the highest timestamp seen in data of the head.
|
|
|
|
|
func (h *Head) MaxTime() int64 {
|
2020-07-28 04:42:42 +00:00
|
|
|
|
return h.maxTime.Load()
|
2017-01-12 18:18:51 +00:00
|
|
|
|
}
|
|
|
|
|
|
2019-04-01 08:19:06 +00:00
|
|
|
|
// compactable returns whether the head has a compactable range.
|
|
|
|
|
// The head has a compactable range when the head time range is 1.5 times the chunk range.
|
|
|
|
|
// The 0.5 acts as a buffer of the appendable window.
|
|
|
|
|
func (h *Head) compactable() bool {
|
2020-07-28 04:42:42 +00:00
|
|
|
|
return h.MaxTime()-h.MinTime() > h.chunkRange.Load()/2*3
|
2019-04-01 08:19:06 +00:00
|
|
|
|
}
|
|
|
|
|
|
2017-11-10 20:19:39 +00:00
|
|
|
|
// Close flushes the WAL and closes the head.
|
2021-08-06 16:51:01 +00:00
|
|
|
|
// It also takes a snapshot of in-memory chunks if enabled.
|
2017-11-10 20:19:39 +00:00
|
|
|
|
func (h *Head) Close() error {
|
2020-05-22 09:03:23 +00:00
|
|
|
|
h.closedMtx.Lock()
|
|
|
|
|
defer h.closedMtx.Unlock()
|
|
|
|
|
h.closed = true
|
2020-10-28 15:24:58 +00:00
|
|
|
|
errs := tsdb_errors.NewMulti(h.chunkDiskMapper.Close())
|
2020-05-06 15:30:00 +00:00
|
|
|
|
if h.wal != nil {
|
2020-10-28 15:24:58 +00:00
|
|
|
|
errs.Add(h.wal.Close())
|
2018-05-17 13:04:32 +00:00
|
|
|
|
}
|
2021-09-13 13:00:41 +00:00
|
|
|
|
if errs.Err() == nil && h.opts.EnableMemorySnapshotOnShutdown {
|
|
|
|
|
errs.Add(h.performChunkSnapshot())
|
|
|
|
|
}
|
2020-10-28 15:24:58 +00:00
|
|
|
|
return errs.Err()
|
2017-11-10 20:19:39 +00:00
|
|
|
|
}
|
|
|
|
|
|
2021-01-21 10:37:29 +00:00
|
|
|
|
// String returns an human readable representation of the TSDB head. It's important to
|
|
|
|
|
// keep this function in order to avoid the struct dump when the head is stringified in
|
|
|
|
|
// errors or logs.
|
|
|
|
|
func (h *Head) String() string {
|
|
|
|
|
return "head"
|
|
|
|
|
}
|
|
|
|
|
|
2020-05-20 13:22:08 +00:00
|
|
|
|
func (h *Head) getOrCreate(hash uint64, lset labels.Labels) (*memSeries, bool, error) {
|
2021-03-18 15:23:50 +00:00
|
|
|
|
// Just using `getOrCreateWithID` below would be semantically sufficient, but we'd create
|
2017-09-18 10:28:56 +00:00
|
|
|
|
// a new series on every sample inserted via Add(), which causes allocations
|
|
|
|
|
// and makes our series IDs rather random and harder to compress in postings.
|
|
|
|
|
s := h.series.getByHash(hash, lset)
|
|
|
|
|
if s != nil {
|
2020-05-20 13:22:08 +00:00
|
|
|
|
return s, false, nil
|
2017-09-18 10:28:56 +00:00
|
|
|
|
}
|
2017-08-30 16:34:54 +00:00
|
|
|
|
|
2017-09-05 09:45:18 +00:00
|
|
|
|
// Optimistically assume that we are the first one to create the series.
|
2020-07-28 04:42:42 +00:00
|
|
|
|
id := h.lastSeriesID.Inc()
|
2017-09-19 08:20:19 +00:00
|
|
|
|
|
|
|
|
|
return h.getOrCreateWithID(id, hash, lset)
|
|
|
|
|
}
|
|
|
|
|
|
2020-05-20 13:22:08 +00:00
|
|
|
|
func (h *Head) getOrCreateWithID(id, hash uint64, lset labels.Labels) (*memSeries, bool, error) {
|
2021-03-18 15:23:50 +00:00
|
|
|
|
s, created, err := h.series.getOrSet(hash, lset, func() *memSeries {
|
|
|
|
|
return newMemSeries(lset, id, h.chunkRange.Load(), &h.memChunkPool)
|
|
|
|
|
})
|
2020-05-20 13:22:08 +00:00
|
|
|
|
if err != nil {
|
|
|
|
|
return nil, false, err
|
|
|
|
|
}
|
2017-09-05 09:45:18 +00:00
|
|
|
|
if !created {
|
2020-05-20 13:22:08 +00:00
|
|
|
|
return s, false, nil
|
2017-09-05 09:45:18 +00:00
|
|
|
|
}
|
|
|
|
|
|
2017-09-18 10:28:56 +00:00
|
|
|
|
h.metrics.seriesCreated.Inc()
|
2020-07-28 04:42:42 +00:00
|
|
|
|
h.numSeries.Inc()
|
2017-09-18 10:28:56 +00:00
|
|
|
|
|
2020-07-14 08:36:22 +00:00
|
|
|
|
h.postings.Add(id, lset)
|
2020-05-20 13:22:08 +00:00
|
|
|
|
return s, true, nil
|
2017-09-05 09:45:18 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
// seriesHashmap is a simple hashmap for memSeries by their label set. It is built
|
|
|
|
|
// on top of a regular hashmap and holds a slice of series to resolve hash collisions.
|
|
|
|
|
// Its methods require the hash to be submitted with it to avoid re-computations throughout
|
|
|
|
|
// the code.
|
|
|
|
|
type seriesHashmap map[uint64][]*memSeries
|
2016-12-04 12:16:11 +00:00
|
|
|
|
|
2017-09-05 09:45:18 +00:00
|
|
|
|
func (m seriesHashmap) get(hash uint64, lset labels.Labels) *memSeries {
|
|
|
|
|
for _, s := range m[hash] {
|
2019-11-18 19:53:33 +00:00
|
|
|
|
if labels.Equal(s.lset, lset) {
|
2017-09-05 09:45:18 +00:00
|
|
|
|
return s
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
return nil
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
func (m seriesHashmap) set(hash uint64, s *memSeries) {
|
|
|
|
|
l := m[hash]
|
|
|
|
|
for i, prev := range l {
|
2019-11-18 19:53:33 +00:00
|
|
|
|
if labels.Equal(prev.lset, s.lset) {
|
2017-09-05 09:45:18 +00:00
|
|
|
|
l[i] = s
|
|
|
|
|
return
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
m[hash] = append(l, s)
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
func (m seriesHashmap) del(hash uint64, lset labels.Labels) {
|
|
|
|
|
var rem []*memSeries
|
|
|
|
|
for _, s := range m[hash] {
|
2019-11-18 19:53:33 +00:00
|
|
|
|
if !labels.Equal(s.lset, lset) {
|
2017-09-05 09:45:18 +00:00
|
|
|
|
rem = append(rem, s)
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
if len(rem) == 0 {
|
|
|
|
|
delete(m, hash)
|
|
|
|
|
} else {
|
|
|
|
|
m[hash] = rem
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2020-01-30 07:12:43 +00:00
|
|
|
|
const (
|
|
|
|
|
// DefaultStripeSize is the default number of entries to allocate in the stripeSeries hash map.
|
|
|
|
|
DefaultStripeSize = 1 << 14
|
|
|
|
|
)
|
|
|
|
|
|
2017-09-05 09:45:18 +00:00
|
|
|
|
// stripeSeries locks modulo ranges of IDs and hashes to reduce lock contention.
|
2018-01-13 17:51:50 +00:00
|
|
|
|
// The locks are padded to not be on the same cache line. Filling the padded space
|
2017-09-05 09:45:18 +00:00
|
|
|
|
// with the maps was profiled to be slower – likely due to the additional pointer
|
|
|
|
|
// dereferences.
|
|
|
|
|
type stripeSeries struct {
|
2020-05-20 13:22:08 +00:00
|
|
|
|
size int
|
|
|
|
|
series []map[uint64]*memSeries
|
|
|
|
|
hashes []seriesHashmap
|
|
|
|
|
locks []stripeLock
|
|
|
|
|
seriesLifecycleCallback SeriesLifecycleCallback
|
2017-09-05 09:45:18 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
type stripeLock struct {
|
|
|
|
|
sync.RWMutex
|
|
|
|
|
// Padding to avoid multiple locks being on the same cache line.
|
|
|
|
|
_ [40]byte
|
|
|
|
|
}
|
|
|
|
|
|
2020-05-20 13:22:08 +00:00
|
|
|
|
func newStripeSeries(stripeSize int, seriesCallback SeriesLifecycleCallback) *stripeSeries {
|
2020-01-30 07:12:43 +00:00
|
|
|
|
s := &stripeSeries{
|
2020-05-20 13:22:08 +00:00
|
|
|
|
size: stripeSize,
|
|
|
|
|
series: make([]map[uint64]*memSeries, stripeSize),
|
|
|
|
|
hashes: make([]seriesHashmap, stripeSize),
|
|
|
|
|
locks: make([]stripeLock, stripeSize),
|
|
|
|
|
seriesLifecycleCallback: seriesCallback,
|
2020-01-30 07:12:43 +00:00
|
|
|
|
}
|
2017-09-05 09:45:18 +00:00
|
|
|
|
|
|
|
|
|
for i := range s.series {
|
|
|
|
|
s.series[i] = map[uint64]*memSeries{}
|
|
|
|
|
}
|
|
|
|
|
for i := range s.hashes {
|
|
|
|
|
s.hashes[i] = seriesHashmap{}
|
|
|
|
|
}
|
2017-01-11 12:02:38 +00:00
|
|
|
|
return s
|
2016-12-04 12:16:11 +00:00
|
|
|
|
}
|
|
|
|
|
|
2017-09-05 09:45:18 +00:00
|
|
|
|
// gc garbage collects old chunks that are strictly before mint and removes
|
|
|
|
|
// series entirely that have no chunks left.
|
2020-11-25 13:03:30 +00:00
|
|
|
|
func (s *stripeSeries) gc(mint int64) (map[uint64]struct{}, int, int64) {
|
2017-09-05 09:45:18 +00:00
|
|
|
|
var (
|
2020-11-25 13:03:30 +00:00
|
|
|
|
deleted = map[uint64]struct{}{}
|
|
|
|
|
deletedForCallback = []labels.Labels{}
|
|
|
|
|
rmChunks = 0
|
|
|
|
|
actualMint int64 = math.MaxInt64
|
2017-09-05 09:45:18 +00:00
|
|
|
|
)
|
|
|
|
|
// Run through all series and truncate old chunks. Mark those with no
|
2017-09-06 14:20:37 +00:00
|
|
|
|
// chunks left as deleted and store their ID.
|
2020-01-30 07:12:43 +00:00
|
|
|
|
for i := 0; i < s.size; i++ {
|
2017-09-05 09:45:18 +00:00
|
|
|
|
s.locks[i].Lock()
|
|
|
|
|
|
|
|
|
|
for hash, all := range s.hashes[i] {
|
|
|
|
|
for _, series := range all {
|
2017-09-08 06:48:19 +00:00
|
|
|
|
series.Lock()
|
2017-09-05 09:45:18 +00:00
|
|
|
|
rmChunks += series.truncateChunksBefore(mint)
|
|
|
|
|
|
2020-05-06 15:30:00 +00:00
|
|
|
|
if len(series.mmappedChunks) > 0 || series.headChunk != nil || series.pendingCommit {
|
2020-11-25 13:03:30 +00:00
|
|
|
|
seriesMint := series.minTime()
|
|
|
|
|
if seriesMint < actualMint {
|
|
|
|
|
actualMint = seriesMint
|
|
|
|
|
}
|
2017-09-08 06:48:19 +00:00
|
|
|
|
series.Unlock()
|
2017-09-05 09:45:18 +00:00
|
|
|
|
continue
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
// The series is gone entirely. We need to keep the series lock
|
|
|
|
|
// and make sure we have acquired the stripe locks for hash and ID of the
|
|
|
|
|
// series alike.
|
|
|
|
|
// If we don't hold them all, there's a very small chance that a series receives
|
|
|
|
|
// samples again while we are half-way into deleting it.
|
2020-01-30 07:12:43 +00:00
|
|
|
|
j := int(series.ref) & (s.size - 1)
|
2017-09-05 09:45:18 +00:00
|
|
|
|
|
|
|
|
|
if i != j {
|
|
|
|
|
s.locks[j].Lock()
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
deleted[series.ref] = struct{}{}
|
|
|
|
|
s.hashes[i].del(hash, series.lset)
|
|
|
|
|
delete(s.series[j], series.ref)
|
2020-05-20 13:22:08 +00:00
|
|
|
|
deletedForCallback = append(deletedForCallback, series.lset)
|
2017-09-05 09:45:18 +00:00
|
|
|
|
|
|
|
|
|
if i != j {
|
|
|
|
|
s.locks[j].Unlock()
|
|
|
|
|
}
|
|
|
|
|
|
2017-09-08 06:48:19 +00:00
|
|
|
|
series.Unlock()
|
2017-09-05 09:45:18 +00:00
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
s.locks[i].Unlock()
|
2020-05-20 13:22:08 +00:00
|
|
|
|
|
|
|
|
|
s.seriesLifecycleCallback.PostDeletion(deletedForCallback...)
|
|
|
|
|
deletedForCallback = deletedForCallback[:0]
|
2017-09-05 09:45:18 +00:00
|
|
|
|
}
|
|
|
|
|
|
2020-11-25 13:03:30 +00:00
|
|
|
|
if actualMint == math.MaxInt64 {
|
|
|
|
|
actualMint = mint
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
return deleted, rmChunks, actualMint
|
2017-09-05 09:45:18 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
func (s *stripeSeries) getByID(id uint64) *memSeries {
|
2020-01-30 07:12:43 +00:00
|
|
|
|
i := id & uint64(s.size-1)
|
2017-09-05 09:45:18 +00:00
|
|
|
|
|
|
|
|
|
s.locks[i].RLock()
|
|
|
|
|
series := s.series[i][id]
|
|
|
|
|
s.locks[i].RUnlock()
|
|
|
|
|
|
|
|
|
|
return series
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
func (s *stripeSeries) getByHash(hash uint64, lset labels.Labels) *memSeries {
|
2020-01-30 07:12:43 +00:00
|
|
|
|
i := hash & uint64(s.size-1)
|
2017-09-05 09:45:18 +00:00
|
|
|
|
|
|
|
|
|
s.locks[i].RLock()
|
|
|
|
|
series := s.hashes[i].get(hash, lset)
|
|
|
|
|
s.locks[i].RUnlock()
|
|
|
|
|
|
|
|
|
|
return series
|
|
|
|
|
}
|
|
|
|
|
|
2021-03-18 15:23:50 +00:00
|
|
|
|
func (s *stripeSeries) getOrSet(hash uint64, lset labels.Labels, createSeries func() *memSeries) (*memSeries, bool, error) {
|
2020-05-20 13:22:08 +00:00
|
|
|
|
// PreCreation is called here to avoid calling it inside the lock.
|
|
|
|
|
// It is not necessary to call it just before creating a series,
|
|
|
|
|
// rather it gives a 'hint' whether to create a series or not.
|
2021-03-18 15:23:50 +00:00
|
|
|
|
preCreationErr := s.seriesLifecycleCallback.PreCreation(lset)
|
|
|
|
|
|
|
|
|
|
// Create the series, unless the PreCreation() callback as failed.
|
|
|
|
|
// If failed, we'll not allow to create a new series anyway.
|
|
|
|
|
var series *memSeries
|
|
|
|
|
if preCreationErr == nil {
|
|
|
|
|
series = createSeries()
|
|
|
|
|
}
|
2017-09-05 09:45:18 +00:00
|
|
|
|
|
2020-05-20 13:22:08 +00:00
|
|
|
|
i := hash & uint64(s.size-1)
|
2017-09-05 09:45:18 +00:00
|
|
|
|
s.locks[i].Lock()
|
|
|
|
|
|
2021-03-18 15:23:50 +00:00
|
|
|
|
if prev := s.hashes[i].get(hash, lset); prev != nil {
|
2017-09-18 09:23:22 +00:00
|
|
|
|
s.locks[i].Unlock()
|
2020-05-20 13:22:08 +00:00
|
|
|
|
return prev, false, nil
|
|
|
|
|
}
|
2021-03-18 15:23:50 +00:00
|
|
|
|
if preCreationErr == nil {
|
2020-05-20 13:22:08 +00:00
|
|
|
|
s.hashes[i].set(hash, series)
|
2017-09-05 09:45:18 +00:00
|
|
|
|
}
|
|
|
|
|
s.locks[i].Unlock()
|
|
|
|
|
|
2021-03-18 15:23:50 +00:00
|
|
|
|
if preCreationErr != nil {
|
2020-05-20 13:22:08 +00:00
|
|
|
|
// The callback prevented creation of series.
|
2021-03-18 15:23:50 +00:00
|
|
|
|
return nil, false, preCreationErr
|
2020-05-20 13:22:08 +00:00
|
|
|
|
}
|
|
|
|
|
// Setting the series in the s.hashes marks the creation of series
|
|
|
|
|
// as any further calls to this methods would return that series.
|
|
|
|
|
s.seriesLifecycleCallback.PostCreation(series.lset)
|
|
|
|
|
|
2020-01-30 07:12:43 +00:00
|
|
|
|
i = series.ref & uint64(s.size-1)
|
2017-09-05 09:45:18 +00:00
|
|
|
|
|
|
|
|
|
s.locks[i].Lock()
|
|
|
|
|
s.series[i][series.ref] = series
|
|
|
|
|
s.locks[i].Unlock()
|
|
|
|
|
|
2020-05-20 13:22:08 +00:00
|
|
|
|
return series, true, nil
|
2017-09-05 09:45:18 +00:00
|
|
|
|
}
|
|
|
|
|
|
2017-03-24 09:20:39 +00:00
|
|
|
|
type sample struct {
|
|
|
|
|
t int64
|
|
|
|
|
v float64
|
|
|
|
|
}
|
|
|
|
|
|
2020-07-31 15:03:02 +00:00
|
|
|
|
func newSample(t int64, v float64) tsdbutil.Sample { return sample{t, v} }
|
|
|
|
|
func (s sample) T() int64 { return s.t }
|
|
|
|
|
func (s sample) V() float64 { return s.v }
|
2018-10-25 20:06:19 +00:00
|
|
|
|
|
2017-09-08 06:48:19 +00:00
|
|
|
|
// memSeries is the in-memory representation of a series. None of its methods
|
2018-01-13 17:51:50 +00:00
|
|
|
|
// are goroutine safe and it is the caller's responsibility to lock it.
|
2017-01-11 12:02:38 +00:00
|
|
|
|
type memSeries struct {
|
2020-03-16 12:59:22 +00:00
|
|
|
|
sync.RWMutex
|
2017-01-09 15:51:39 +00:00
|
|
|
|
|
2020-05-06 15:30:00 +00:00
|
|
|
|
ref uint64
|
|
|
|
|
lset labels.Labels
|
|
|
|
|
mmappedChunks []*mmappedChunk
|
2021-08-10 09:23:31 +00:00
|
|
|
|
mmMaxTime int64 // Max time of any mmapped chunk, only used during WAL replay.
|
2020-05-06 15:30:00 +00:00
|
|
|
|
headChunk *memChunk
|
|
|
|
|
chunkRange int64
|
|
|
|
|
firstChunkID int
|
2017-01-09 15:51:39 +00:00
|
|
|
|
|
2018-09-17 16:58:42 +00:00
|
|
|
|
nextAt int64 // Timestamp at which to cut the next chunk.
|
|
|
|
|
sampleBuf [4]sample
|
|
|
|
|
pendingCommit bool // Whether there are samples waiting to be committed to this series.
|
2017-01-09 15:51:39 +00:00
|
|
|
|
|
2017-11-30 14:34:49 +00:00
|
|
|
|
app chunkenc.Appender // Current appender for the chunk.
|
2020-02-12 19:22:27 +00:00
|
|
|
|
|
2020-05-06 15:30:00 +00:00
|
|
|
|
memChunkPool *sync.Pool
|
|
|
|
|
|
2020-02-12 19:22:27 +00:00
|
|
|
|
txs *txRing
|
2017-01-09 15:51:39 +00:00
|
|
|
|
}
|
|
|
|
|
|
2020-05-06 15:30:00 +00:00
|
|
|
|
func newMemSeries(lset labels.Labels, id uint64, chunkRange int64, memChunkPool *sync.Pool) *memSeries {
|
2019-01-08 17:08:41 +00:00
|
|
|
|
s := &memSeries{
|
2020-05-06 15:30:00 +00:00
|
|
|
|
lset: lset,
|
|
|
|
|
ref: id,
|
|
|
|
|
chunkRange: chunkRange,
|
|
|
|
|
nextAt: math.MinInt64,
|
|
|
|
|
txs: newTxRing(4),
|
|
|
|
|
memChunkPool: memChunkPool,
|
2019-01-08 17:08:41 +00:00
|
|
|
|
}
|
|
|
|
|
return s
|
|
|
|
|
}
|
|
|
|
|
|
2017-08-28 22:39:17 +00:00
|
|
|
|
func (s *memSeries) minTime() int64 {
|
2020-05-06 15:30:00 +00:00
|
|
|
|
if len(s.mmappedChunks) > 0 {
|
|
|
|
|
return s.mmappedChunks[0].minTime
|
2018-02-07 13:43:21 +00:00
|
|
|
|
}
|
2020-05-06 15:30:00 +00:00
|
|
|
|
if s.headChunk != nil {
|
|
|
|
|
return s.headChunk.minTime
|
|
|
|
|
}
|
|
|
|
|
return math.MinInt64
|
2017-08-28 22:39:17 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
func (s *memSeries) maxTime() int64 {
|
2018-02-07 13:43:21 +00:00
|
|
|
|
c := s.head()
|
2021-08-17 17:08:16 +00:00
|
|
|
|
if c != nil {
|
|
|
|
|
return c.maxTime
|
|
|
|
|
}
|
|
|
|
|
if len(s.mmappedChunks) > 0 {
|
|
|
|
|
return s.mmappedChunks[len(s.mmappedChunks)-1].maxTime
|
2018-02-07 13:43:21 +00:00
|
|
|
|
}
|
2021-08-17 17:08:16 +00:00
|
|
|
|
return math.MinInt64
|
2017-08-28 22:39:17 +00:00
|
|
|
|
}
|
|
|
|
|
|
2020-07-21 07:02:13 +00:00
|
|
|
|
// truncateChunksBefore removes all chunks from the series that
|
|
|
|
|
// have no timestamp at or after mint.
|
|
|
|
|
// Chunk IDs remain unchanged.
|
2017-08-30 15:38:25 +00:00
|
|
|
|
func (s *memSeries) truncateChunksBefore(mint int64) (removed int) {
|
2020-05-06 15:30:00 +00:00
|
|
|
|
if s.headChunk != nil && s.headChunk.maxTime < mint {
|
|
|
|
|
// If head chunk is truncated, we can truncate all mmapped chunks.
|
2020-11-30 08:55:33 +00:00
|
|
|
|
removed = 1 + len(s.mmappedChunks)
|
|
|
|
|
s.firstChunkID += removed
|
2018-10-31 13:28:56 +00:00
|
|
|
|
s.headChunk = nil
|
2020-05-06 15:30:00 +00:00
|
|
|
|
s.mmappedChunks = nil
|
2020-11-30 08:55:33 +00:00
|
|
|
|
return removed
|
2020-05-06 15:30:00 +00:00
|
|
|
|
}
|
|
|
|
|
if len(s.mmappedChunks) > 0 {
|
|
|
|
|
for i, c := range s.mmappedChunks {
|
|
|
|
|
if c.maxTime >= mint {
|
|
|
|
|
break
|
|
|
|
|
}
|
2020-11-30 08:55:33 +00:00
|
|
|
|
removed = i + 1
|
2020-05-06 15:30:00 +00:00
|
|
|
|
}
|
2020-11-30 08:55:33 +00:00
|
|
|
|
s.mmappedChunks = append(s.mmappedChunks[:0], s.mmappedChunks[removed:]...)
|
|
|
|
|
s.firstChunkID += removed
|
2018-10-31 13:28:56 +00:00
|
|
|
|
}
|
2020-11-30 08:55:33 +00:00
|
|
|
|
return removed
|
2017-08-28 22:39:17 +00:00
|
|
|
|
}
|
|
|
|
|
|
2020-02-12 19:22:27 +00:00
|
|
|
|
// cleanupAppendIDsBelow cleans up older appendIDs. Has to be called after
|
|
|
|
|
// acquiring lock.
|
|
|
|
|
func (s *memSeries) cleanupAppendIDsBelow(bound uint64) {
|
|
|
|
|
s.txs.cleanupAppendIDsBelow(bound)
|
|
|
|
|
}
|
|
|
|
|
|
2017-01-11 12:02:38 +00:00
|
|
|
|
func (s *memSeries) head() *memChunk {
|
2018-10-31 13:28:56 +00:00
|
|
|
|
return s.headChunk
|
2017-01-11 12:02:38 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
type memChunk struct {
|
2017-11-30 14:34:49 +00:00
|
|
|
|
chunk chunkenc.Chunk
|
2017-01-11 12:02:38 +00:00
|
|
|
|
minTime, maxTime int64
|
|
|
|
|
}
|
|
|
|
|
|
2020-05-06 15:30:00 +00:00
|
|
|
|
// OverlapsClosedInterval returns true if the chunk overlaps [mint, maxt].
|
2018-07-02 08:23:36 +00:00
|
|
|
|
func (mc *memChunk) OverlapsClosedInterval(mint, maxt int64) bool {
|
2021-08-03 14:33:54 +00:00
|
|
|
|
return overlapsClosedInterval(mc.minTime, mc.maxTime, mint, maxt)
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
func overlapsClosedInterval(mint1, maxt1, mint2, maxt2 int64) bool {
|
|
|
|
|
return mint1 <= maxt2 && mint2 <= maxt1
|
2018-07-02 08:23:36 +00:00
|
|
|
|
}
|
|
|
|
|
|
2021-10-13 12:14:32 +00:00
|
|
|
|
// mappedChunks describes chunk data on disk that can be mmapped
|
2020-05-06 15:30:00 +00:00
|
|
|
|
type mmappedChunk struct {
|
2021-10-13 12:14:32 +00:00
|
|
|
|
ref chunks.ChunkDiskMapperRef
|
2020-05-06 15:30:00 +00:00
|
|
|
|
numSamples uint16
|
|
|
|
|
minTime, maxTime int64
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
// Returns true if the chunk overlaps [mint, maxt].
|
|
|
|
|
func (mc *mmappedChunk) OverlapsClosedInterval(mint, maxt int64) bool {
|
2021-08-03 14:33:54 +00:00
|
|
|
|
return overlapsClosedInterval(mc.minTime, mc.maxTime, mint, maxt)
|
2020-05-06 15:30:00 +00:00
|
|
|
|
}
|
2020-05-20 13:22:08 +00:00
|
|
|
|
|
|
|
|
|
type noopSeriesLifecycleCallback struct{}
|
|
|
|
|
|
|
|
|
|
func (noopSeriesLifecycleCallback) PreCreation(labels.Labels) error { return nil }
|
|
|
|
|
func (noopSeriesLifecycleCallback) PostCreation(labels.Labels) {}
|
|
|
|
|
func (noopSeriesLifecycleCallback) PostDeletion(...labels.Labels) {}
|
2020-10-12 21:15:40 +00:00
|
|
|
|
|
|
|
|
|
func (h *Head) Size() int64 {
|
|
|
|
|
var walSize int64
|
|
|
|
|
if h.wal != nil {
|
|
|
|
|
walSize, _ = h.wal.Size()
|
|
|
|
|
}
|
2020-11-03 10:04:59 +00:00
|
|
|
|
cdmSize, _ := h.chunkDiskMapper.Size()
|
|
|
|
|
return walSize + cdmSize
|
2020-10-12 21:15:40 +00:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
func (h *RangeHead) Size() int64 {
|
|
|
|
|
return h.head.Size()
|
|
|
|
|
}
|
2021-06-05 14:29:32 +00:00
|
|
|
|
|
|
|
|
|
func (h *Head) startWALReplayStatus(startFrom, last int) {
|
|
|
|
|
h.stats.WALReplayStatus.Lock()
|
|
|
|
|
defer h.stats.WALReplayStatus.Unlock()
|
|
|
|
|
|
|
|
|
|
h.stats.WALReplayStatus.Min = startFrom
|
|
|
|
|
h.stats.WALReplayStatus.Max = last
|
|
|
|
|
h.stats.WALReplayStatus.Current = startFrom
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
func (h *Head) updateWALReplayStatusRead(current int) {
|
|
|
|
|
h.stats.WALReplayStatus.Lock()
|
|
|
|
|
defer h.stats.WALReplayStatus.Unlock()
|
|
|
|
|
|
|
|
|
|
h.stats.WALReplayStatus.Current = current
|
|
|
|
|
}
|