2017-04-10 11:59:45 -07: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 04:16:11 -08:00
|
|
|
|
package tsdb
|
|
|
|
|
|
|
|
|
|
import (
|
2019-05-24 11:33:28 -07:00
|
|
|
|
"fmt"
|
2017-01-04 05:06:40 -08:00
|
|
|
|
"math"
|
2017-10-07 06:55:11 -07:00
|
|
|
|
"runtime"
|
2016-12-14 09:38:46 -08:00
|
|
|
|
"sort"
|
2017-11-30 06:34:49 -08:00
|
|
|
|
"strings"
|
2016-12-04 04:16:11 -08:00
|
|
|
|
"sync"
|
2017-02-04 02:53:52 -08:00
|
|
|
|
"sync/atomic"
|
2017-08-30 09:34:54 -07:00
|
|
|
|
"time"
|
2017-05-17 07:43:01 -07:00
|
|
|
|
|
2017-01-06 06:18:06 -08:00
|
|
|
|
"github.com/go-kit/kit/log"
|
2017-09-28 00:19:34 -07:00
|
|
|
|
"github.com/go-kit/kit/log/level"
|
2019-07-23 01:04:48 -07:00
|
|
|
|
"github.com/oklog/ulid"
|
2017-01-19 02:22:47 -08:00
|
|
|
|
"github.com/pkg/errors"
|
2017-08-30 09:34:54 -07:00
|
|
|
|
"github.com/prometheus/client_golang/prometheus"
|
2019-11-18 11:53:33 -08:00
|
|
|
|
"github.com/prometheus/prometheus/pkg/labels"
|
2019-08-13 01:34:14 -07:00
|
|
|
|
"github.com/prometheus/prometheus/tsdb/chunkenc"
|
|
|
|
|
"github.com/prometheus/prometheus/tsdb/chunks"
|
|
|
|
|
"github.com/prometheus/prometheus/tsdb/encoding"
|
|
|
|
|
"github.com/prometheus/prometheus/tsdb/index"
|
2019-09-19 02:15:41 -07:00
|
|
|
|
"github.com/prometheus/prometheus/tsdb/record"
|
|
|
|
|
"github.com/prometheus/prometheus/tsdb/tombstones"
|
2019-08-13 01:34:14 -07:00
|
|
|
|
"github.com/prometheus/prometheus/tsdb/wal"
|
2016-12-04 04:16:11 -08:00
|
|
|
|
)
|
|
|
|
|
|
2017-01-17 07:33:58 -08:00
|
|
|
|
var (
|
|
|
|
|
// ErrNotFound is returned if a looked up resource was not found.
|
2017-03-21 02:11:23 -07:00
|
|
|
|
ErrNotFound = errors.Errorf("not found")
|
2017-01-17 07:33:58 -08:00
|
|
|
|
|
|
|
|
|
// ErrOutOfOrderSample is returned if an appended sample has a
|
2019-03-21 01:53:39 -07:00
|
|
|
|
// timestamp smaller than the most recent sample.
|
2017-01-17 07:33:58 -08:00
|
|
|
|
ErrOutOfOrderSample = errors.New("out of order sample")
|
|
|
|
|
|
|
|
|
|
// ErrAmendSample is returned if an appended sample has the same timestamp
|
|
|
|
|
// as the most recent sample but a different value.
|
|
|
|
|
ErrAmendSample = errors.New("amending sample")
|
|
|
|
|
|
|
|
|
|
// ErrOutOfBounds is returned if an appended sample is out of the
|
|
|
|
|
// writable time range.
|
|
|
|
|
ErrOutOfBounds = errors.New("out of bounds")
|
2019-01-08 09:08:41 -08:00
|
|
|
|
|
|
|
|
|
// emptyTombstoneReader is a no-op Tombstone Reader.
|
|
|
|
|
// This is used by head to satisfy the Tombstones() function call.
|
2019-09-19 02:15:41 -07:00
|
|
|
|
emptyTombstoneReader = tombstones.NewMemTombstones()
|
2017-01-17 07:33:58 -08:00
|
|
|
|
)
|
|
|
|
|
|
2017-08-28 15:39:17 -07:00
|
|
|
|
// Head handles reads and writes of time series data within a time window.
|
|
|
|
|
type Head struct {
|
2019-10-09 08:41:46 -07:00
|
|
|
|
// Keep all 64bit atomically accessed variables at the top of this struct.
|
|
|
|
|
// See https://golang.org/pkg/sync/atomic/#pkg-note-BUG for more info.
|
|
|
|
|
chunkRange int64
|
|
|
|
|
numSeries uint64
|
|
|
|
|
minTime, maxTime int64 // Current min and max of the samples included in the head.
|
|
|
|
|
minValidTime int64 // Mint allowed to be added to the head. It shouldn't be lower than the maxt of the last persisted block.
|
|
|
|
|
lastSeriesID uint64
|
|
|
|
|
|
2017-08-30 09:34:54 -07:00
|
|
|
|
metrics *headMetrics
|
2018-05-17 06:04:32 -07:00
|
|
|
|
wal *wal.WAL
|
2017-08-30 09:34:54 -07:00
|
|
|
|
logger log.Logger
|
|
|
|
|
appendPool sync.Pool
|
2019-09-19 02:15:41 -07:00
|
|
|
|
seriesPool sync.Pool
|
2018-05-17 06:04:32 -07:00
|
|
|
|
bytesPool sync.Pool
|
2017-02-04 02:53:52 -08:00
|
|
|
|
|
2017-09-05 02:45:18 -07:00
|
|
|
|
// All series addressable by their ID or hash.
|
|
|
|
|
series *stripeSeries
|
2016-12-21 16:12:28 -08:00
|
|
|
|
|
2017-09-05 02:45:18 -07:00
|
|
|
|
symMtx sync.RWMutex
|
|
|
|
|
symbols map[string]struct{}
|
|
|
|
|
values map[string]stringset // label names to possible values
|
|
|
|
|
|
2019-04-09 06:16:24 -07:00
|
|
|
|
deletedMtx sync.Mutex
|
|
|
|
|
deleted map[uint64]int // Deleted series, and what WAL segment they must be kept until.
|
|
|
|
|
|
2017-11-30 06:34:49 -08:00
|
|
|
|
postings *index.MemPostings // postings lists for terms
|
2019-11-04 18:06:13 -08:00
|
|
|
|
|
|
|
|
|
cardinalityMutex sync.Mutex
|
|
|
|
|
cardinalityCache *index.PostingsStats // posting stats cache which will expire after 30sec
|
|
|
|
|
lastPostingsStatsCall time.Duration // last posting stats call (PostgingsCardinalityStats()) time for caching
|
2016-12-04 04:16:11 -08:00
|
|
|
|
}
|
|
|
|
|
|
2017-08-30 09:34:54 -07:00
|
|
|
|
type headMetrics struct {
|
2018-09-25 06:48:33 -07:00
|
|
|
|
activeAppenders prometheus.Gauge
|
2019-07-23 01:04:48 -07:00
|
|
|
|
series prometheus.GaugeFunc
|
2018-09-25 06:48:33 -07:00
|
|
|
|
seriesCreated prometheus.Counter
|
|
|
|
|
seriesRemoved prometheus.Counter
|
|
|
|
|
seriesNotFound prometheus.Counter
|
|
|
|
|
chunks prometheus.Gauge
|
|
|
|
|
chunksCreated prometheus.Counter
|
|
|
|
|
chunksRemoved prometheus.Counter
|
|
|
|
|
gcDuration prometheus.Summary
|
|
|
|
|
minTime prometheus.GaugeFunc
|
|
|
|
|
maxTime prometheus.GaugeFunc
|
|
|
|
|
samplesAppended prometheus.Counter
|
|
|
|
|
walTruncateDuration prometheus.Summary
|
2018-12-18 02:24:56 -08:00
|
|
|
|
walCorruptionsTotal prometheus.Counter
|
2018-09-25 06:48:33 -07:00
|
|
|
|
headTruncateFail prometheus.Counter
|
|
|
|
|
headTruncateTotal prometheus.Counter
|
|
|
|
|
checkpointDeleteFail prometheus.Counter
|
|
|
|
|
checkpointDeleteTotal prometheus.Counter
|
|
|
|
|
checkpointCreationFail prometheus.Counter
|
|
|
|
|
checkpointCreationTotal prometheus.Counter
|
2017-08-30 09:34:54 -07:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
func newHeadMetrics(h *Head, r prometheus.Registerer) *headMetrics {
|
|
|
|
|
m := &headMetrics{}
|
|
|
|
|
|
|
|
|
|
m.activeAppenders = prometheus.NewGauge(prometheus.GaugeOpts{
|
2018-09-18 10:17:41 -07:00
|
|
|
|
Name: "prometheus_tsdb_head_active_appenders",
|
2017-08-30 09:34:54 -07:00
|
|
|
|
Help: "Number of currently active appender transactions",
|
|
|
|
|
})
|
2019-07-23 01:04:48 -07:00
|
|
|
|
m.series = prometheus.NewGaugeFunc(prometheus.GaugeOpts{
|
2018-09-18 10:17:41 -07:00
|
|
|
|
Name: "prometheus_tsdb_head_series",
|
2017-08-30 09:34:54 -07:00
|
|
|
|
Help: "Total number of series in the head block.",
|
2019-07-23 01:04:48 -07:00
|
|
|
|
}, func() float64 {
|
|
|
|
|
return float64(h.NumSeries())
|
2017-08-30 09:34:54 -07:00
|
|
|
|
})
|
2018-09-12 02:09:02 -07:00
|
|
|
|
m.seriesCreated = prometheus.NewCounter(prometheus.CounterOpts{
|
2018-09-18 10:17:41 -07:00
|
|
|
|
Name: "prometheus_tsdb_head_series_created_total",
|
2017-08-30 09:34:54 -07:00
|
|
|
|
Help: "Total number of series created in the head",
|
|
|
|
|
})
|
2018-09-12 02:09:02 -07:00
|
|
|
|
m.seriesRemoved = prometheus.NewCounter(prometheus.CounterOpts{
|
2018-09-18 10:17:41 -07:00
|
|
|
|
Name: "prometheus_tsdb_head_series_removed_total",
|
2017-08-30 09:34:54 -07:00
|
|
|
|
Help: "Total number of series removed in the head",
|
|
|
|
|
})
|
2017-10-12 06:25:12 -07:00
|
|
|
|
m.seriesNotFound = prometheus.NewCounter(prometheus.CounterOpts{
|
2018-09-18 10:17:41 -07:00
|
|
|
|
Name: "prometheus_tsdb_head_series_not_found_total",
|
2017-10-12 06:25:12 -07:00
|
|
|
|
Help: "Total number of requests for series that were not found.",
|
|
|
|
|
})
|
2017-08-30 09:34:54 -07:00
|
|
|
|
m.chunks = prometheus.NewGauge(prometheus.GaugeOpts{
|
2018-09-18 10:17:41 -07:00
|
|
|
|
Name: "prometheus_tsdb_head_chunks",
|
2017-08-30 09:34:54 -07:00
|
|
|
|
Help: "Total number of chunks in the head block.",
|
|
|
|
|
})
|
2018-09-12 02:09:02 -07:00
|
|
|
|
m.chunksCreated = prometheus.NewCounter(prometheus.CounterOpts{
|
2018-09-18 10:17:41 -07:00
|
|
|
|
Name: "prometheus_tsdb_head_chunks_created_total",
|
2017-08-30 09:34:54 -07:00
|
|
|
|
Help: "Total number of chunks created in the head",
|
|
|
|
|
})
|
2018-09-12 02:09:02 -07:00
|
|
|
|
m.chunksRemoved = prometheus.NewCounter(prometheus.CounterOpts{
|
2018-09-18 10:17:41 -07:00
|
|
|
|
Name: "prometheus_tsdb_head_chunks_removed_total",
|
2017-08-30 09:34:54 -07:00
|
|
|
|
Help: "Total number of chunks removed in the head",
|
|
|
|
|
})
|
|
|
|
|
m.gcDuration = prometheus.NewSummary(prometheus.SummaryOpts{
|
2019-06-11 17:12:32 -07:00
|
|
|
|
Name: "prometheus_tsdb_head_gc_duration_seconds",
|
|
|
|
|
Help: "Runtime of garbage collection in the head block.",
|
|
|
|
|
Objectives: map[float64]float64{},
|
2017-08-30 09:34:54 -07:00
|
|
|
|
})
|
2017-10-09 03:24:53 -07:00
|
|
|
|
m.maxTime = prometheus.NewGaugeFunc(prometheus.GaugeOpts{
|
2018-09-18 10:17:41 -07:00
|
|
|
|
Name: "prometheus_tsdb_head_max_time",
|
2018-11-30 10:18:12 -08:00
|
|
|
|
Help: "Maximum timestamp of the head block. The unit is decided by the library consumer.",
|
2017-08-30 09:34:54 -07:00
|
|
|
|
}, func() float64 {
|
|
|
|
|
return float64(h.MaxTime())
|
|
|
|
|
})
|
2017-10-09 03:24:53 -07:00
|
|
|
|
m.minTime = prometheus.NewGaugeFunc(prometheus.GaugeOpts{
|
2018-09-18 10:17:41 -07:00
|
|
|
|
Name: "prometheus_tsdb_head_min_time",
|
2018-11-30 10:18:12 -08:00
|
|
|
|
Help: "Minimum time bound of the head block. The unit is decided by the library consumer.",
|
2017-08-30 09:34:54 -07:00
|
|
|
|
}, func() float64 {
|
|
|
|
|
return float64(h.MinTime())
|
|
|
|
|
})
|
|
|
|
|
m.walTruncateDuration = prometheus.NewSummary(prometheus.SummaryOpts{
|
2019-06-11 17:12:32 -07:00
|
|
|
|
Name: "prometheus_tsdb_wal_truncate_duration_seconds",
|
|
|
|
|
Help: "Duration of WAL truncation.",
|
|
|
|
|
Objectives: map[float64]float64{},
|
2017-08-30 09:34:54 -07:00
|
|
|
|
})
|
2018-12-18 02:24:56 -08:00
|
|
|
|
m.walCorruptionsTotal = prometheus.NewCounter(prometheus.CounterOpts{
|
|
|
|
|
Name: "prometheus_tsdb_wal_corruptions_total",
|
|
|
|
|
Help: "Total number of WAL corruptions.",
|
|
|
|
|
})
|
2017-08-30 09:34:54 -07:00
|
|
|
|
m.samplesAppended = prometheus.NewCounter(prometheus.CounterOpts{
|
2018-09-18 10:17:41 -07:00
|
|
|
|
Name: "prometheus_tsdb_head_samples_appended_total",
|
2017-10-25 11:12:18 -07:00
|
|
|
|
Help: "Total number of appended samples.",
|
2017-08-30 09:34:54 -07:00
|
|
|
|
})
|
2018-09-25 06:48:33 -07:00
|
|
|
|
m.headTruncateFail = prometheus.NewCounter(prometheus.CounterOpts{
|
|
|
|
|
Name: "prometheus_tsdb_head_truncations_failed_total",
|
|
|
|
|
Help: "Total number of head truncations that failed.",
|
|
|
|
|
})
|
|
|
|
|
m.headTruncateTotal = prometheus.NewCounter(prometheus.CounterOpts{
|
|
|
|
|
Name: "prometheus_tsdb_head_truncations_total",
|
|
|
|
|
Help: "Total number of head truncations attempted.",
|
|
|
|
|
})
|
2018-09-25 04:49:09 -07:00
|
|
|
|
m.checkpointDeleteFail = prometheus.NewCounter(prometheus.CounterOpts{
|
2018-09-25 06:48:33 -07:00
|
|
|
|
Name: "prometheus_tsdb_checkpoint_deletions_failed_total",
|
|
|
|
|
Help: "Total number of checkpoint deletions that failed.",
|
|
|
|
|
})
|
|
|
|
|
m.checkpointDeleteTotal = prometheus.NewCounter(prometheus.CounterOpts{
|
|
|
|
|
Name: "prometheus_tsdb_checkpoint_deletions_total",
|
|
|
|
|
Help: "Total number of checkpoint deletions attempted.",
|
|
|
|
|
})
|
|
|
|
|
m.checkpointCreationFail = prometheus.NewCounter(prometheus.CounterOpts{
|
|
|
|
|
Name: "prometheus_tsdb_checkpoint_creations_failed_total",
|
|
|
|
|
Help: "Total number of checkpoint creations that failed.",
|
|
|
|
|
})
|
|
|
|
|
m.checkpointCreationTotal = prometheus.NewCounter(prometheus.CounterOpts{
|
|
|
|
|
Name: "prometheus_tsdb_checkpoint_creations_total",
|
|
|
|
|
Help: "Total number of checkpoint creations attempted.",
|
2018-09-25 04:49:09 -07:00
|
|
|
|
})
|
2017-08-30 09:34:54 -07:00
|
|
|
|
|
|
|
|
|
if r != nil {
|
|
|
|
|
r.MustRegister(
|
|
|
|
|
m.activeAppenders,
|
|
|
|
|
m.chunks,
|
|
|
|
|
m.chunksCreated,
|
|
|
|
|
m.chunksRemoved,
|
|
|
|
|
m.series,
|
|
|
|
|
m.seriesCreated,
|
|
|
|
|
m.seriesRemoved,
|
2017-10-12 06:25:12 -07:00
|
|
|
|
m.seriesNotFound,
|
2017-08-30 09:34:54 -07:00
|
|
|
|
m.minTime,
|
|
|
|
|
m.maxTime,
|
|
|
|
|
m.gcDuration,
|
|
|
|
|
m.walTruncateDuration,
|
2018-12-18 02:24:56 -08:00
|
|
|
|
m.walCorruptionsTotal,
|
2017-08-30 09:34:54 -07:00
|
|
|
|
m.samplesAppended,
|
2018-09-25 06:48:33 -07:00
|
|
|
|
m.headTruncateFail,
|
|
|
|
|
m.headTruncateTotal,
|
2018-09-25 04:49:09 -07:00
|
|
|
|
m.checkpointDeleteFail,
|
2018-09-25 06:48:33 -07:00
|
|
|
|
m.checkpointDeleteTotal,
|
|
|
|
|
m.checkpointCreationFail,
|
|
|
|
|
m.checkpointCreationTotal,
|
2017-08-30 09:34:54 -07:00
|
|
|
|
)
|
|
|
|
|
}
|
|
|
|
|
return m
|
|
|
|
|
}
|
|
|
|
|
|
2019-11-04 18:06:13 -08:00
|
|
|
|
const cardinalityCacheExpirationTime = time.Duration(30) * time.Second
|
|
|
|
|
|
|
|
|
|
// 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
|
|
|
|
|
}
|
|
|
|
|
|
2017-08-28 15:39:17 -07:00
|
|
|
|
// NewHead opens the head block in dir.
|
2018-05-17 06:04:32 -07:00
|
|
|
|
func NewHead(r prometheus.Registerer, l log.Logger, wal *wal.WAL, chunkRange int64) (*Head, error) {
|
2017-08-30 09:34:54 -07:00
|
|
|
|
if l == nil {
|
|
|
|
|
l = log.NewNopLogger()
|
|
|
|
|
}
|
|
|
|
|
if chunkRange < 1 {
|
|
|
|
|
return nil, errors.Errorf("invalid chunk range %d", chunkRange)
|
|
|
|
|
}
|
2017-08-28 15:39:17 -07:00
|
|
|
|
h := &Head{
|
2017-08-30 09:34:54 -07:00
|
|
|
|
wal: wal,
|
|
|
|
|
logger: l,
|
2017-08-28 15:39:17 -07:00
|
|
|
|
chunkRange: chunkRange,
|
2018-05-25 14:19:32 -07:00
|
|
|
|
minTime: math.MaxInt64,
|
2017-08-28 15:39:17 -07:00
|
|
|
|
maxTime: math.MinInt64,
|
2017-09-05 02:45:18 -07:00
|
|
|
|
series: newStripeSeries(),
|
2017-05-15 10:58:14 -07:00
|
|
|
|
values: map[string]stringset{},
|
2017-08-05 04:31:48 -07:00
|
|
|
|
symbols: map[string]struct{}{},
|
2017-11-30 06:34:49 -08:00
|
|
|
|
postings: index.NewUnorderedMemPostings(),
|
2019-04-09 06:16:24 -07:00
|
|
|
|
deleted: map[uint64]int{},
|
2017-01-07 07:20:32 -08:00
|
|
|
|
}
|
2017-08-30 09:34:54 -07:00
|
|
|
|
h.metrics = newHeadMetrics(h, r)
|
|
|
|
|
|
2017-09-06 07:20:37 -07:00
|
|
|
|
return h, nil
|
2017-08-30 09:34:54 -07:00
|
|
|
|
}
|
|
|
|
|
|
2017-10-07 06:55:11 -07:00
|
|
|
|
// processWALSamples adds a partition of samples it receives to the head and passes
|
|
|
|
|
// them on to other workers.
|
|
|
|
|
// Samples before the mint timestamp are discarded.
|
|
|
|
|
func (h *Head) processWALSamples(
|
2018-05-25 14:19:32 -07:00
|
|
|
|
minValidTime int64,
|
2019-09-19 02:15:41 -07:00
|
|
|
|
input <-chan []record.RefSample, output chan<- []record.RefSample,
|
2017-10-07 06:55:11 -07:00
|
|
|
|
) (unknownRefs uint64) {
|
|
|
|
|
defer close(output)
|
|
|
|
|
|
2018-10-31 05:51:21 -07:00
|
|
|
|
// Mitigate lock contention in getByID.
|
|
|
|
|
refSeries := map[uint64]*memSeries{}
|
|
|
|
|
|
2018-05-25 14:19:32 -07:00
|
|
|
|
mint, maxt := int64(math.MaxInt64), int64(math.MinInt64)
|
2018-05-17 06:04:32 -07:00
|
|
|
|
|
2017-10-07 06:55:11 -07:00
|
|
|
|
for samples := range input {
|
|
|
|
|
for _, s := range samples {
|
2018-10-31 15:52:26 -07:00
|
|
|
|
if s.T < minValidTime {
|
2017-10-07 06:55:11 -07:00
|
|
|
|
continue
|
|
|
|
|
}
|
2018-10-31 05:51:21 -07:00
|
|
|
|
ms := refSeries[s.Ref]
|
2017-10-07 06:55:11 -07:00
|
|
|
|
if ms == nil {
|
2018-10-31 05:51:21 -07:00
|
|
|
|
ms = h.series.getByID(s.Ref)
|
|
|
|
|
if ms == nil {
|
|
|
|
|
unknownRefs++
|
|
|
|
|
continue
|
|
|
|
|
}
|
|
|
|
|
refSeries[s.Ref] = ms
|
2017-10-07 06:55:11 -07:00
|
|
|
|
}
|
|
|
|
|
_, chunkCreated := ms.append(s.T, s.V)
|
|
|
|
|
if chunkCreated {
|
|
|
|
|
h.metrics.chunksCreated.Inc()
|
|
|
|
|
h.metrics.chunks.Inc()
|
|
|
|
|
}
|
2018-05-17 06:04:32 -07:00
|
|
|
|
if s.T > maxt {
|
|
|
|
|
maxt = s.T
|
|
|
|
|
}
|
2018-05-25 14:19:32 -07:00
|
|
|
|
if s.T < mint {
|
|
|
|
|
mint = s.T
|
|
|
|
|
}
|
2017-10-07 06:55:11 -07:00
|
|
|
|
}
|
|
|
|
|
output <- samples
|
|
|
|
|
}
|
2018-05-25 14:19:32 -07:00
|
|
|
|
h.updateMinMaxTime(mint, maxt)
|
|
|
|
|
|
|
|
|
|
return unknownRefs
|
|
|
|
|
}
|
2018-05-17 06:04:32 -07:00
|
|
|
|
|
2018-05-25 14:19:32 -07:00
|
|
|
|
func (h *Head) updateMinMaxTime(mint, maxt int64) {
|
|
|
|
|
for {
|
|
|
|
|
lt := h.MinTime()
|
|
|
|
|
if mint >= lt {
|
|
|
|
|
break
|
|
|
|
|
}
|
|
|
|
|
if atomic.CompareAndSwapInt64(&h.minTime, lt, mint) {
|
|
|
|
|
break
|
|
|
|
|
}
|
|
|
|
|
}
|
2018-05-17 06:04:32 -07:00
|
|
|
|
for {
|
|
|
|
|
ht := h.MaxTime()
|
|
|
|
|
if maxt <= ht {
|
|
|
|
|
break
|
|
|
|
|
}
|
|
|
|
|
if atomic.CompareAndSwapInt64(&h.maxTime, ht, maxt) {
|
|
|
|
|
break
|
|
|
|
|
}
|
|
|
|
|
}
|
2017-10-07 06:55:11 -07:00
|
|
|
|
}
|
|
|
|
|
|
2019-06-14 08:39:22 -07:00
|
|
|
|
func (h *Head) loadWAL(r *wal.Reader, multiRef map[uint64]uint64) (err error) {
|
2017-09-21 02:02:30 -07:00
|
|
|
|
// Track number of samples that referenced a series we don't know about
|
|
|
|
|
// for error reporting.
|
2017-10-07 06:55:11 -07:00
|
|
|
|
var unknownRefs uint64
|
|
|
|
|
|
|
|
|
|
// Start workers that each process samples for a partition of the series ID space.
|
|
|
|
|
// They are connected through a ring of channels which ensures that all sample batches
|
|
|
|
|
// read from the WAL are processed in order.
|
|
|
|
|
var (
|
2019-10-07 22:47:40 -07:00
|
|
|
|
wg sync.WaitGroup
|
|
|
|
|
n = runtime.GOMAXPROCS(0)
|
|
|
|
|
inputs = make([]chan []record.RefSample, n)
|
|
|
|
|
outputs = make([]chan []record.RefSample, n)
|
2017-10-07 06:55:11 -07:00
|
|
|
|
)
|
2017-10-11 01:12:29 -07:00
|
|
|
|
wg.Add(n)
|
|
|
|
|
|
2019-06-14 08:39:22 -07:00
|
|
|
|
defer func() {
|
|
|
|
|
// For CorruptionErr ensure to terminate all workers before exiting.
|
|
|
|
|
if _, ok := err.(*wal.CorruptionErr); ok {
|
|
|
|
|
for i := 0; i < n; i++ {
|
|
|
|
|
close(inputs[i])
|
|
|
|
|
for range outputs[i] {
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
wg.Wait()
|
|
|
|
|
}
|
|
|
|
|
}()
|
|
|
|
|
|
2017-10-07 06:55:11 -07:00
|
|
|
|
for i := 0; i < n; i++ {
|
2019-09-19 02:15:41 -07:00
|
|
|
|
outputs[i] = make(chan []record.RefSample, 300)
|
|
|
|
|
inputs[i] = make(chan []record.RefSample, 300)
|
2017-10-07 06:55:11 -07:00
|
|
|
|
|
2019-09-19 02:15:41 -07:00
|
|
|
|
go func(input <-chan []record.RefSample, output chan<- []record.RefSample) {
|
2018-12-04 02:30:49 -08:00
|
|
|
|
unknown := h.processWALSamples(h.minValidTime, input, output)
|
2017-10-07 06:55:11 -07:00
|
|
|
|
atomic.AddUint64(&unknownRefs, unknown)
|
2017-10-11 01:12:29 -07:00
|
|
|
|
wg.Done()
|
2018-10-31 15:52:26 -07:00
|
|
|
|
}(inputs[i], outputs[i])
|
2017-10-07 06:55:11 -07:00
|
|
|
|
}
|
2017-09-21 02:02:30 -07:00
|
|
|
|
|
2018-05-17 06:04:32 -07:00
|
|
|
|
var (
|
2019-09-19 02:15:41 -07:00
|
|
|
|
dec record.Decoder
|
|
|
|
|
allStones = tombstones.NewMemTombstones()
|
2019-10-03 02:51:39 -07:00
|
|
|
|
shards = make([][]record.RefSample, n)
|
2018-05-17 06:04:32 -07:00
|
|
|
|
)
|
2019-06-14 08:39:22 -07:00
|
|
|
|
defer func() {
|
|
|
|
|
if err := allStones.Close(); err != nil {
|
|
|
|
|
level.Warn(h.logger).Log("msg", "closing memTombstones during wal read", "err", err)
|
|
|
|
|
}
|
|
|
|
|
}()
|
2019-11-07 08:26:45 -08:00
|
|
|
|
|
|
|
|
|
var (
|
|
|
|
|
decoded = make(chan interface{}, 10)
|
|
|
|
|
errCh = make(chan error, 1)
|
|
|
|
|
seriesPool = sync.Pool{
|
|
|
|
|
New: func() interface{} {
|
|
|
|
|
return []record.RefSeries{}
|
|
|
|
|
},
|
|
|
|
|
}
|
|
|
|
|
samplesPool = sync.Pool{
|
|
|
|
|
New: func() interface{} {
|
|
|
|
|
return []record.RefSample{}
|
|
|
|
|
},
|
|
|
|
|
}
|
|
|
|
|
tstonesPool = sync.Pool{
|
|
|
|
|
New: func() interface{} {
|
|
|
|
|
return []tombstones.Stone{}
|
|
|
|
|
},
|
|
|
|
|
}
|
|
|
|
|
)
|
|
|
|
|
go func() {
|
|
|
|
|
defer close(decoded)
|
|
|
|
|
for r.Next() {
|
|
|
|
|
rec := r.Record()
|
|
|
|
|
switch dec.Type(rec) {
|
|
|
|
|
case record.Series:
|
|
|
|
|
series := seriesPool.Get().([]record.RefSeries)[:0]
|
|
|
|
|
series, err = dec.Series(rec, series)
|
|
|
|
|
if err != nil {
|
|
|
|
|
errCh <- &wal.CorruptionErr{
|
|
|
|
|
Err: errors.Wrap(err, "decode series"),
|
|
|
|
|
Segment: r.Segment(),
|
|
|
|
|
Offset: r.Offset(),
|
|
|
|
|
}
|
|
|
|
|
return
|
|
|
|
|
}
|
|
|
|
|
decoded <- series
|
|
|
|
|
case record.Samples:
|
|
|
|
|
samples := samplesPool.Get().([]record.RefSample)[:0]
|
|
|
|
|
samples, err = dec.Samples(rec, samples)
|
|
|
|
|
if err != nil {
|
|
|
|
|
errCh <- &wal.CorruptionErr{
|
|
|
|
|
Err: errors.Wrap(err, "decode samples"),
|
|
|
|
|
Segment: r.Segment(),
|
|
|
|
|
Offset: r.Offset(),
|
|
|
|
|
}
|
|
|
|
|
return
|
|
|
|
|
}
|
|
|
|
|
decoded <- samples
|
|
|
|
|
case record.Tombstones:
|
|
|
|
|
tstones := tstonesPool.Get().([]tombstones.Stone)[:0]
|
|
|
|
|
tstones, err = dec.Tombstones(rec, tstones)
|
|
|
|
|
if err != nil {
|
|
|
|
|
errCh <- &wal.CorruptionErr{
|
|
|
|
|
Err: errors.Wrap(err, "decode tombstones"),
|
|
|
|
|
Segment: r.Segment(),
|
|
|
|
|
Offset: r.Offset(),
|
|
|
|
|
}
|
|
|
|
|
return
|
|
|
|
|
}
|
|
|
|
|
decoded <- tstones
|
|
|
|
|
default:
|
|
|
|
|
errCh <- &wal.CorruptionErr{
|
|
|
|
|
Err: errors.Errorf("invalid record type %v", dec.Type(rec)),
|
2018-11-30 03:37:04 -08:00
|
|
|
|
Segment: r.Segment(),
|
|
|
|
|
Offset: r.Offset(),
|
|
|
|
|
}
|
2019-11-07 08:26:45 -08:00
|
|
|
|
return
|
2018-05-17 06:04:32 -07:00
|
|
|
|
}
|
2019-11-07 08:26:45 -08:00
|
|
|
|
}
|
|
|
|
|
}()
|
|
|
|
|
|
|
|
|
|
for d := range decoded {
|
|
|
|
|
switch v := d.(type) {
|
|
|
|
|
case []record.RefSeries:
|
|
|
|
|
for _, s := range v {
|
2019-06-06 06:28:54 -07:00
|
|
|
|
series, created := h.getOrCreateWithID(s.Ref, s.Labels.Hash(), s.Labels)
|
|
|
|
|
|
|
|
|
|
if !created {
|
|
|
|
|
// There's already a different ref for this series.
|
|
|
|
|
multiRef[s.Ref] = series.ref
|
|
|
|
|
}
|
2017-09-19 01:20:19 -07:00
|
|
|
|
|
2018-05-17 06:04:32 -07:00
|
|
|
|
if h.lastSeriesID < s.Ref {
|
|
|
|
|
h.lastSeriesID = s.Ref
|
|
|
|
|
}
|
2017-09-19 01:20:19 -07:00
|
|
|
|
}
|
2019-11-07 08:26:45 -08:00
|
|
|
|
//lint:ignore SA6002 relax staticcheck verification.
|
|
|
|
|
seriesPool.Put(v)
|
|
|
|
|
case []record.RefSample:
|
|
|
|
|
samples := v
|
2018-05-17 06:04:32 -07:00
|
|
|
|
// We split up the samples into chunks of 5000 samples or less.
|
|
|
|
|
// With O(300 * #cores) in-flight sample batches, large scrapes could otherwise
|
|
|
|
|
// cause thousands of very large in flight buffers occupying large amounts
|
|
|
|
|
// of unused memory.
|
|
|
|
|
for len(samples) > 0 {
|
2018-10-31 15:52:26 -07:00
|
|
|
|
m := 5000
|
|
|
|
|
if len(samples) < m {
|
|
|
|
|
m = len(samples)
|
|
|
|
|
}
|
|
|
|
|
for i := 0; i < n; i++ {
|
2019-09-19 02:15:41 -07:00
|
|
|
|
var buf []record.RefSample
|
2018-10-31 15:52:26 -07:00
|
|
|
|
select {
|
|
|
|
|
case buf = <-outputs[i]:
|
|
|
|
|
default:
|
|
|
|
|
}
|
|
|
|
|
shards[i] = buf[:0]
|
2018-05-17 06:04:32 -07:00
|
|
|
|
}
|
2018-10-31 15:52:26 -07:00
|
|
|
|
for _, sam := range samples[:m] {
|
2019-06-06 06:28:54 -07:00
|
|
|
|
if r, ok := multiRef[sam.Ref]; ok {
|
|
|
|
|
sam.Ref = r
|
|
|
|
|
}
|
2018-10-31 15:52:26 -07:00
|
|
|
|
mod := sam.Ref % uint64(n)
|
|
|
|
|
shards[mod] = append(shards[mod], sam)
|
2018-05-17 06:04:32 -07:00
|
|
|
|
}
|
2018-10-31 15:52:26 -07:00
|
|
|
|
for i := 0; i < n; i++ {
|
|
|
|
|
inputs[i] <- shards[i]
|
2018-05-17 06:04:32 -07:00
|
|
|
|
}
|
2018-10-31 15:52:26 -07:00
|
|
|
|
samples = samples[m:]
|
2017-10-23 07:22:24 -07:00
|
|
|
|
}
|
2019-11-07 08:26:45 -08:00
|
|
|
|
//lint:ignore SA6002 relax staticcheck verification.
|
|
|
|
|
samplesPool.Put(v)
|
|
|
|
|
case []tombstones.Stone:
|
|
|
|
|
for _, s := range v {
|
2019-09-19 02:15:41 -07:00
|
|
|
|
for _, itv := range s.Intervals {
|
2018-12-04 02:30:49 -08:00
|
|
|
|
if itv.Maxt < h.minValidTime {
|
2018-05-17 06:04:32 -07:00
|
|
|
|
continue
|
|
|
|
|
}
|
2019-09-19 02:15:41 -07:00
|
|
|
|
if m := h.series.getByID(s.Ref); m == nil {
|
2019-05-16 06:36:44 -07:00
|
|
|
|
unknownRefs++
|
|
|
|
|
continue
|
|
|
|
|
}
|
2019-09-19 02:15:41 -07:00
|
|
|
|
allStones.AddInterval(s.Ref, itv)
|
2017-09-06 07:20:37 -07:00
|
|
|
|
}
|
2017-08-30 09:34:54 -07:00
|
|
|
|
}
|
2019-11-07 08:26:45 -08:00
|
|
|
|
//lint:ignore SA6002 relax staticcheck verification.
|
|
|
|
|
tstonesPool.Put(v)
|
2018-05-17 06:04:32 -07:00
|
|
|
|
default:
|
2019-11-07 08:26:45 -08:00
|
|
|
|
panic(fmt.Errorf("unexpected decoded type: %T", d))
|
2017-08-30 09:34:54 -07:00
|
|
|
|
}
|
|
|
|
|
}
|
2017-09-21 02:02:30 -07:00
|
|
|
|
|
2019-11-07 08:26:45 -08:00
|
|
|
|
select {
|
|
|
|
|
case err := <-errCh:
|
|
|
|
|
return err
|
|
|
|
|
default:
|
|
|
|
|
}
|
|
|
|
|
|
2018-10-31 15:52:26 -07:00
|
|
|
|
// Signal termination to each worker and wait for it to close its output channel.
|
|
|
|
|
for i := 0; i < n; i++ {
|
|
|
|
|
close(inputs[i])
|
|
|
|
|
for range outputs[i] {
|
|
|
|
|
}
|
2017-10-07 06:55:11 -07:00
|
|
|
|
}
|
2017-10-11 01:12:29 -07:00
|
|
|
|
wg.Wait()
|
|
|
|
|
|
2019-06-14 08:39:22 -07:00
|
|
|
|
if r.Err() != nil {
|
|
|
|
|
return errors.Wrap(r.Err(), "read records")
|
|
|
|
|
}
|
|
|
|
|
|
2019-09-19 02:15:41 -07:00
|
|
|
|
if err := allStones.Iter(func(ref uint64, dranges tombstones.Intervals) error {
|
2019-01-08 09:08:41 -08:00
|
|
|
|
return h.chunkRewrite(ref, dranges)
|
|
|
|
|
}); err != nil {
|
|
|
|
|
return errors.Wrap(r.Err(), "deleting samples from tombstones")
|
|
|
|
|
}
|
|
|
|
|
|
2018-05-17 06:04:32 -07:00
|
|
|
|
if unknownRefs > 0 {
|
|
|
|
|
level.Warn(h.logger).Log("msg", "unknown series references", "count", unknownRefs)
|
|
|
|
|
}
|
|
|
|
|
return nil
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
// Init loads data from the write ahead log and prepares the head for writes.
|
2018-12-04 02:30:49 -08:00
|
|
|
|
// It should be called before using an appender so that
|
|
|
|
|
// limits the ingested samples to the head min valid time.
|
|
|
|
|
func (h *Head) Init(minValidTime int64) error {
|
|
|
|
|
h.minValidTime = minValidTime
|
2018-05-17 06:04:32 -07:00
|
|
|
|
defer h.postings.EnsureOrder()
|
2018-12-04 02:30:49 -08:00
|
|
|
|
defer h.gc() // After loading the wal remove the obsolete data from the head.
|
2018-05-17 06:04:32 -07:00
|
|
|
|
|
|
|
|
|
if h.wal == nil {
|
|
|
|
|
return nil
|
|
|
|
|
}
|
|
|
|
|
|
2019-07-13 10:10:44 -07:00
|
|
|
|
level.Info(h.logger).Log("msg", "replaying WAL, this may take awhile")
|
2018-05-17 06:04:32 -07:00
|
|
|
|
// Backfill the checkpoint first if it exists.
|
2019-09-19 02:15:41 -07:00
|
|
|
|
dir, startFrom, err := wal.LastCheckpoint(h.wal.Dir())
|
|
|
|
|
if err != nil && err != record.ErrNotFound {
|
2018-05-17 06:04:32 -07:00
|
|
|
|
return errors.Wrap(err, "find last checkpoint")
|
|
|
|
|
}
|
2019-06-06 06:28:54 -07:00
|
|
|
|
multiRef := map[uint64]uint64{}
|
2018-05-17 06:04:32 -07:00
|
|
|
|
if err == nil {
|
2018-11-30 06:46:16 -08:00
|
|
|
|
sr, err := wal.NewSegmentsReader(dir)
|
2018-05-17 06:04:32 -07:00
|
|
|
|
if err != nil {
|
|
|
|
|
return errors.Wrap(err, "open checkpoint")
|
|
|
|
|
}
|
2019-06-14 08:39:22 -07:00
|
|
|
|
defer func() {
|
|
|
|
|
if err := sr.Close(); err != nil {
|
|
|
|
|
level.Warn(h.logger).Log("msg", "error while closing the wal segments reader", "err", err)
|
|
|
|
|
}
|
|
|
|
|
}()
|
2018-05-17 06:04:32 -07:00
|
|
|
|
|
|
|
|
|
// A corrupted checkpoint is a hard error for now and requires user
|
|
|
|
|
// intervention. There's likely little data that can be recovered anyway.
|
2019-06-06 06:28:54 -07:00
|
|
|
|
if err := h.loadWAL(wal.NewReader(sr), multiRef); err != nil {
|
2018-05-17 06:04:32 -07:00
|
|
|
|
return errors.Wrap(err, "backfill checkpoint")
|
|
|
|
|
}
|
2018-10-11 08:23:52 -07:00
|
|
|
|
startFrom++
|
2019-07-13 10:10:44 -07:00
|
|
|
|
level.Info(h.logger).Log("msg", "WAL checkpoint loaded")
|
2018-05-17 06:04:32 -07:00
|
|
|
|
}
|
|
|
|
|
|
2019-05-24 11:33:28 -07:00
|
|
|
|
// Find the last segment.
|
|
|
|
|
_, last, err := h.wal.Segments()
|
2017-10-07 06:55:11 -07:00
|
|
|
|
if err != nil {
|
2019-05-24 11:33:28 -07:00
|
|
|
|
return errors.Wrap(err, "finding WAL segments")
|
2017-08-30 09:34:54 -07:00
|
|
|
|
}
|
2018-05-17 06:04:32 -07:00
|
|
|
|
|
2019-05-24 11:33:28 -07:00
|
|
|
|
// Backfill segments from the most recent checkpoint onwards.
|
|
|
|
|
for i := startFrom; i <= last; i++ {
|
|
|
|
|
s, err := wal.OpenReadSegment(wal.SegmentName(h.wal.Dir(), i))
|
|
|
|
|
if err != nil {
|
|
|
|
|
return errors.Wrap(err, fmt.Sprintf("open WAL segment: %d", i))
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
sr := wal.NewSegmentBufReader(s)
|
2019-06-06 06:28:54 -07:00
|
|
|
|
err = h.loadWAL(wal.NewReader(sr), multiRef)
|
2019-06-14 08:39:22 -07:00
|
|
|
|
if err := sr.Close(); err != nil {
|
|
|
|
|
level.Warn(h.logger).Log("msg", "error while closing the wal segments reader", "err", err)
|
2019-05-24 11:33:28 -07:00
|
|
|
|
}
|
2019-06-14 08:39:22 -07:00
|
|
|
|
if err != nil {
|
|
|
|
|
return err
|
2019-05-24 11:33:28 -07:00
|
|
|
|
}
|
2019-07-13 10:10:44 -07:00
|
|
|
|
level.Info(h.logger).Log("msg", "WAL segment loaded", "segment", i, "maxSegment", last)
|
2017-10-07 06:55:11 -07:00
|
|
|
|
}
|
2019-05-24 11:33:28 -07:00
|
|
|
|
|
2017-08-30 09:34:54 -07:00
|
|
|
|
return nil
|
2017-05-13 09:14:18 -07:00
|
|
|
|
}
|
2017-01-06 03:37:28 -08:00
|
|
|
|
|
2018-05-17 06:04:32 -07:00
|
|
|
|
// Truncate removes old data before mint from the head.
|
2018-09-25 06:48:33 -07:00
|
|
|
|
func (h *Head) Truncate(mint int64) (err error) {
|
|
|
|
|
defer func() {
|
|
|
|
|
if err != nil {
|
|
|
|
|
h.metrics.headTruncateFail.Inc()
|
|
|
|
|
}
|
|
|
|
|
}()
|
2018-05-25 14:19:32 -07:00
|
|
|
|
initialize := h.MinTime() == math.MaxInt64
|
2017-09-06 07:20:37 -07:00
|
|
|
|
|
2018-05-25 14:19:32 -07:00
|
|
|
|
if h.MinTime() >= mint && !initialize {
|
2017-09-01 05:38:49 -07:00
|
|
|
|
return nil
|
2017-08-30 09:34:54 -07:00
|
|
|
|
}
|
|
|
|
|
atomic.StoreInt64(&h.minTime, mint)
|
2018-12-14 03:42:07 -08:00
|
|
|
|
atomic.StoreInt64(&h.minValidTime, mint)
|
2017-08-30 09:34:54 -07:00
|
|
|
|
|
2017-09-07 04:04:02 -07:00
|
|
|
|
// Ensure that max time is at least as high as min time.
|
|
|
|
|
for h.MaxTime() < mint {
|
|
|
|
|
atomic.CompareAndSwapInt64(&h.maxTime, h.MaxTime(), mint)
|
|
|
|
|
}
|
|
|
|
|
|
2017-09-06 07:20:37 -07: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 06:48:33 -07:00
|
|
|
|
h.metrics.headTruncateTotal.Inc()
|
2017-08-30 09:34:54 -07:00
|
|
|
|
start := time.Now()
|
|
|
|
|
|
|
|
|
|
h.gc()
|
2017-09-28 00:19:34 -07:00
|
|
|
|
level.Info(h.logger).Log("msg", "head GC completed", "duration", time.Since(start))
|
2017-08-30 09:34:54 -07:00
|
|
|
|
h.metrics.gcDuration.Observe(time.Since(start).Seconds())
|
|
|
|
|
|
2018-05-17 06:04:32 -07:00
|
|
|
|
if h.wal == nil {
|
|
|
|
|
return nil
|
|
|
|
|
}
|
2017-08-30 09:34:54 -07:00
|
|
|
|
start = time.Now()
|
|
|
|
|
|
2018-10-11 08:23:52 -07:00
|
|
|
|
first, last, err := h.wal.Segments()
|
2018-05-17 06:04:32 -07:00
|
|
|
|
if err != nil {
|
|
|
|
|
return errors.Wrap(err, "get segment range")
|
|
|
|
|
}
|
2019-06-07 03:35:02 -07:00
|
|
|
|
// Start a new segment, so low ingestion volume TSDB don't have more WAL than
|
|
|
|
|
// needed.
|
|
|
|
|
err = h.wal.NextSegment()
|
|
|
|
|
if err != nil {
|
|
|
|
|
return errors.Wrap(err, "next segment")
|
|
|
|
|
}
|
2018-10-11 08:23:52 -07:00
|
|
|
|
last-- // Never consider last segment for checkpoint.
|
|
|
|
|
if last < 0 {
|
2018-05-17 06:04:32 -07:00
|
|
|
|
return nil // no segments yet.
|
|
|
|
|
}
|
|
|
|
|
// The lower third of segments should contain mostly obsolete samples.
|
2018-06-18 04:52:57 -07:00
|
|
|
|
// If we have less than three segments, it's not worth checkpointing yet.
|
2018-10-11 08:23:52 -07:00
|
|
|
|
last = first + (last-first)/3
|
|
|
|
|
if last <= first {
|
2018-05-17 06:04:32 -07:00
|
|
|
|
return nil
|
|
|
|
|
}
|
|
|
|
|
|
2017-09-21 02:02:30 -07:00
|
|
|
|
keep := func(id uint64) bool {
|
2019-04-09 06:16:24 -07:00
|
|
|
|
if h.series.getByID(id) != nil {
|
|
|
|
|
return true
|
|
|
|
|
}
|
|
|
|
|
h.deletedMtx.Lock()
|
|
|
|
|
_, ok := h.deleted[id]
|
|
|
|
|
h.deletedMtx.Unlock()
|
|
|
|
|
return ok
|
2017-08-31 02:39:22 -07:00
|
|
|
|
}
|
2018-09-25 06:48:33 -07:00
|
|
|
|
h.metrics.checkpointCreationTotal.Inc()
|
2019-09-19 02:15:41 -07:00
|
|
|
|
if _, err = wal.Checkpoint(h.wal, first, last, keep, mint); err != nil {
|
2018-09-25 06:48:33 -07:00
|
|
|
|
h.metrics.checkpointCreationFail.Inc()
|
2018-05-17 06:04:32 -07:00
|
|
|
|
return errors.Wrap(err, "create checkpoint")
|
2017-08-30 09:34:54 -07:00
|
|
|
|
}
|
2018-10-11 08:23:52 -07:00
|
|
|
|
if err := h.wal.Truncate(last + 1); err != nil {
|
2018-09-25 06:48:33 -07: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 06:16:24 -07: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 06:48:33 -07:00
|
|
|
|
h.metrics.checkpointDeleteTotal.Inc()
|
2019-09-19 02:15:41 -07:00
|
|
|
|
if err := wal.DeleteCheckpoints(h.wal.Dir(), last); err != nil {
|
2018-09-25 06:48:33 -07: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 09:34:54 -07:00
|
|
|
|
h.metrics.walTruncateDuration.Observe(time.Since(start).Seconds())
|
2017-09-01 05:38:49 -07:00
|
|
|
|
|
2018-05-17 06:04:32 -07:00
|
|
|
|
level.Info(h.logger).Log("msg", "WAL checkpoint complete",
|
2018-10-11 08:23:52 -07:00
|
|
|
|
"first", first, "last", last, "duration", time.Since(start))
|
2018-05-17 06:04:32 -07:00
|
|
|
|
|
2017-09-01 05:38:49 -07:00
|
|
|
|
return nil
|
2017-08-30 09:34:54 -07:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
// initTime initializes a head with the first timestamp. This only needs to be called
|
2018-12-11 12:09:17 -08:00
|
|
|
|
// for a completely fresh head with an empty WAL.
|
2017-08-30 09:34:54 -07:00
|
|
|
|
// Returns true if the initialization took an effect.
|
|
|
|
|
func (h *Head) initTime(t int64) (initialized bool) {
|
2018-05-25 14:19:32 -07:00
|
|
|
|
if !atomic.CompareAndSwapInt64(&h.minTime, math.MaxInt64, t) {
|
2017-08-30 09:34:54 -07:00
|
|
|
|
return false
|
|
|
|
|
}
|
2017-09-07 04:04:02 -07:00
|
|
|
|
// Ensure that max time is initialized to at least the min time we just set.
|
|
|
|
|
// Concurrent appenders may already have set it to a higher value.
|
|
|
|
|
atomic.CompareAndSwapInt64(&h.maxTime, math.MinInt64, t)
|
|
|
|
|
|
2017-08-30 09:34:54 -07:00
|
|
|
|
return true
|
|
|
|
|
}
|
|
|
|
|
|
2017-10-09 06:21:46 -07:00
|
|
|
|
type rangeHead struct {
|
|
|
|
|
head *Head
|
|
|
|
|
mint, maxt int64
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
func (h *rangeHead) Index() (IndexReader, error) {
|
|
|
|
|
return h.head.indexRange(h.mint, h.maxt), nil
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
func (h *rangeHead) Chunks() (ChunkReader, error) {
|
|
|
|
|
return h.head.chunksRange(h.mint, h.maxt), nil
|
|
|
|
|
}
|
|
|
|
|
|
2019-09-19 02:15:41 -07:00
|
|
|
|
func (h *rangeHead) Tombstones() (tombstones.Reader, error) {
|
2019-01-08 09:08:41 -08:00
|
|
|
|
return emptyTombstoneReader, nil
|
2017-10-09 06:21:46 -07:00
|
|
|
|
}
|
|
|
|
|
|
2019-02-14 05:29:41 -08:00
|
|
|
|
func (h *rangeHead) MinTime() int64 {
|
|
|
|
|
return h.mint
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
func (h *rangeHead) MaxTime() int64 {
|
|
|
|
|
return h.maxt
|
|
|
|
|
}
|
|
|
|
|
|
2019-07-23 01:04:48 -07:00
|
|
|
|
func (h *rangeHead) NumSeries() uint64 {
|
|
|
|
|
return h.head.NumSeries()
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
func (h *rangeHead) Meta() BlockMeta {
|
|
|
|
|
return BlockMeta{
|
|
|
|
|
MinTime: h.MinTime(),
|
|
|
|
|
MaxTime: h.MaxTime(),
|
|
|
|
|
ULID: h.head.Meta().ULID,
|
|
|
|
|
Stats: BlockStats{
|
|
|
|
|
NumSeries: h.NumSeries(),
|
|
|
|
|
},
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2018-01-13 09:51:50 -08:00
|
|
|
|
// initAppender is a helper to initialize the time bounds of the head
|
2017-08-30 09:34:54 -07:00
|
|
|
|
// upon the first sample it receives.
|
|
|
|
|
type initAppender struct {
|
|
|
|
|
app Appender
|
|
|
|
|
head *Head
|
|
|
|
|
}
|
|
|
|
|
|
2017-09-05 02:45:18 -07:00
|
|
|
|
func (a *initAppender) Add(lset labels.Labels, t int64, v float64) (uint64, error) {
|
2017-08-30 09:34:54 -07:00
|
|
|
|
if a.app != nil {
|
|
|
|
|
return a.app.Add(lset, t, v)
|
|
|
|
|
}
|
2017-09-01 03:09:29 -07:00
|
|
|
|
a.head.initTime(t)
|
|
|
|
|
a.app = a.head.appender()
|
|
|
|
|
|
2017-08-30 09:34:54 -07:00
|
|
|
|
return a.app.Add(lset, t, v)
|
|
|
|
|
}
|
|
|
|
|
|
2017-09-05 02:45:18 -07:00
|
|
|
|
func (a *initAppender) AddFast(ref uint64, t int64, v float64) error {
|
2017-08-30 09:34:54 -07:00
|
|
|
|
if a.app == nil {
|
|
|
|
|
return ErrNotFound
|
|
|
|
|
}
|
|
|
|
|
return a.app.AddFast(ref, t, v)
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
func (a *initAppender) Commit() error {
|
|
|
|
|
if a.app == nil {
|
|
|
|
|
return nil
|
|
|
|
|
}
|
|
|
|
|
return a.app.Commit()
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
func (a *initAppender) Rollback() error {
|
|
|
|
|
if a.app == nil {
|
|
|
|
|
return nil
|
|
|
|
|
}
|
|
|
|
|
return a.app.Rollback()
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
// Appender returns a new Appender on the database.
|
|
|
|
|
func (h *Head) Appender() Appender {
|
|
|
|
|
h.metrics.activeAppenders.Inc()
|
|
|
|
|
|
|
|
|
|
// The head cache might not have a starting point yet. The init appender
|
|
|
|
|
// picks up the first appended timestamp as the base.
|
2018-05-25 14:19:32 -07:00
|
|
|
|
if h.MinTime() == math.MaxInt64 {
|
2017-08-30 09:34:54 -07:00
|
|
|
|
return &initAppender{head: h}
|
|
|
|
|
}
|
|
|
|
|
return h.appender()
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
func (h *Head) appender() *headAppender {
|
|
|
|
|
return &headAppender{
|
2018-12-04 02:30:49 -08:00
|
|
|
|
head: h,
|
|
|
|
|
// Set the minimum valid time to whichever is greater the head min valid time or the compaciton window.
|
|
|
|
|
// This ensures that no samples will be added within the compaction window to avoid races.
|
2018-12-14 03:42:07 -08:00
|
|
|
|
minValidTime: max(atomic.LoadInt64(&h.minValidTime), h.MaxTime()-h.chunkRange/2),
|
2018-05-25 14:19:32 -07:00
|
|
|
|
mint: math.MaxInt64,
|
|
|
|
|
maxt: math.MinInt64,
|
|
|
|
|
samples: h.getAppendBuffer(),
|
2019-09-19 02:15:41 -07:00
|
|
|
|
sampleSeries: h.getSeriesBuffer(),
|
2017-08-30 09:34:54 -07:00
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2018-12-04 02:30:49 -08:00
|
|
|
|
func max(a, b int64) int64 {
|
|
|
|
|
if a > b {
|
|
|
|
|
return a
|
|
|
|
|
}
|
|
|
|
|
return b
|
|
|
|
|
}
|
|
|
|
|
|
2019-09-19 02:15:41 -07:00
|
|
|
|
func (h *Head) getAppendBuffer() []record.RefSample {
|
2017-08-30 09:34:54 -07:00
|
|
|
|
b := h.appendPool.Get()
|
|
|
|
|
if b == nil {
|
2019-09-19 02:15:41 -07:00
|
|
|
|
return make([]record.RefSample, 0, 512)
|
2017-08-30 09:34:54 -07:00
|
|
|
|
}
|
2019-09-19 02:15:41 -07:00
|
|
|
|
return b.([]record.RefSample)
|
2017-08-30 09:34:54 -07:00
|
|
|
|
}
|
|
|
|
|
|
2019-09-19 02:15:41 -07:00
|
|
|
|
func (h *Head) putAppendBuffer(b []record.RefSample) {
|
2019-01-02 08:48:42 -08:00
|
|
|
|
//lint:ignore SA6002 safe to ignore and actually fixing it has some performance penalty.
|
2017-08-30 09:34:54 -07:00
|
|
|
|
h.appendPool.Put(b[:0])
|
|
|
|
|
}
|
|
|
|
|
|
2019-09-19 02:15:41 -07:00
|
|
|
|
func (h *Head) getSeriesBuffer() []*memSeries {
|
|
|
|
|
b := h.seriesPool.Get()
|
|
|
|
|
if b == nil {
|
|
|
|
|
return make([]*memSeries, 0, 512)
|
|
|
|
|
}
|
|
|
|
|
return b.([]*memSeries)
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
func (h *Head) putSeriesBuffer(b []*memSeries) {
|
|
|
|
|
//lint:ignore SA6002 safe to ignore and actually fixing it has some performance penalty.
|
|
|
|
|
h.seriesPool.Put(b[:0])
|
|
|
|
|
}
|
|
|
|
|
|
2018-05-17 06:04:32 -07:00
|
|
|
|
func (h *Head) getBytesBuffer() []byte {
|
|
|
|
|
b := h.bytesPool.Get()
|
|
|
|
|
if b == nil {
|
|
|
|
|
return make([]byte, 0, 1024)
|
|
|
|
|
}
|
|
|
|
|
return b.([]byte)
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
func (h *Head) putBytesBuffer(b []byte) {
|
2019-01-02 08:48:42 -08:00
|
|
|
|
//lint:ignore SA6002 safe to ignore and actually fixing it has some performance penalty.
|
2018-05-17 06:04:32 -07:00
|
|
|
|
h.bytesPool.Put(b[:0])
|
|
|
|
|
}
|
|
|
|
|
|
2017-08-30 09:34:54 -07:00
|
|
|
|
type headAppender struct {
|
2018-05-25 14:19:32 -07:00
|
|
|
|
head *Head
|
|
|
|
|
minValidTime int64 // No samples below this timestamp are allowed.
|
|
|
|
|
mint, maxt int64
|
2017-08-30 09:34:54 -07:00
|
|
|
|
|
2019-09-19 02:15:41 -07:00
|
|
|
|
series []record.RefSeries
|
|
|
|
|
samples []record.RefSample
|
|
|
|
|
sampleSeries []*memSeries
|
2017-08-30 09:34:54 -07:00
|
|
|
|
}
|
|
|
|
|
|
2017-09-05 02:45:18 -07:00
|
|
|
|
func (a *headAppender) Add(lset labels.Labels, t int64, v float64) (uint64, error) {
|
2018-05-25 14:19:32 -07:00
|
|
|
|
if t < a.minValidTime {
|
2017-09-05 02:45:18 -07:00
|
|
|
|
return 0, ErrOutOfBounds
|
2017-08-30 09:34:54 -07:00
|
|
|
|
}
|
|
|
|
|
|
2019-05-07 03:00:16 -07:00
|
|
|
|
// Ensure no empty labels have gotten through.
|
|
|
|
|
lset = lset.WithoutEmpty()
|
|
|
|
|
|
2017-09-18 03:28:56 -07:00
|
|
|
|
s, created := a.head.getOrCreate(lset.Hash(), lset)
|
|
|
|
|
if created {
|
2019-09-19 02:15:41 -07:00
|
|
|
|
a.series = append(a.series, record.RefSeries{
|
2017-09-05 02:45:18 -07:00
|
|
|
|
Ref: s.ref,
|
|
|
|
|
Labels: lset,
|
|
|
|
|
})
|
|
|
|
|
}
|
|
|
|
|
return s.ref, a.AddFast(s.ref, t, v)
|
2017-08-30 09:34:54 -07:00
|
|
|
|
}
|
|
|
|
|
|
2017-09-05 02:45:18 -07:00
|
|
|
|
func (a *headAppender) AddFast(ref uint64, t int64, v float64) error {
|
2018-09-17 09:58:42 -07:00
|
|
|
|
if t < a.minValidTime {
|
|
|
|
|
return ErrOutOfBounds
|
|
|
|
|
}
|
2017-09-05 02:45:18 -07:00
|
|
|
|
|
2018-09-17 09:58:42 -07:00
|
|
|
|
s := a.head.series.getByID(ref)
|
2017-09-05 02:45:18 -07:00
|
|
|
|
if s == nil {
|
|
|
|
|
return errors.Wrap(ErrNotFound, "unknown series")
|
2017-08-30 09:34:54 -07:00
|
|
|
|
}
|
2017-09-07 23:48:19 -07:00
|
|
|
|
s.Lock()
|
2018-09-17 09:58:42 -07:00
|
|
|
|
if err := s.appendable(t, v); err != nil {
|
|
|
|
|
s.Unlock()
|
2017-09-08 01:12:28 -07:00
|
|
|
|
return err
|
|
|
|
|
}
|
2018-09-17 09:58:42 -07:00
|
|
|
|
s.pendingCommit = true
|
|
|
|
|
s.Unlock()
|
|
|
|
|
|
2018-05-25 14:19:32 -07:00
|
|
|
|
if t < a.mint {
|
|
|
|
|
a.mint = t
|
|
|
|
|
}
|
2018-02-21 08:01:12 -08:00
|
|
|
|
if t > a.maxt {
|
|
|
|
|
a.maxt = t
|
2017-08-30 09:34:54 -07:00
|
|
|
|
}
|
|
|
|
|
|
2019-09-19 02:15:41 -07:00
|
|
|
|
a.samples = append(a.samples, record.RefSample{
|
|
|
|
|
Ref: ref,
|
|
|
|
|
T: t,
|
|
|
|
|
V: v,
|
2017-08-30 09:34:54 -07:00
|
|
|
|
})
|
2019-09-19 02:15:41 -07:00
|
|
|
|
a.sampleSeries = append(a.sampleSeries, s)
|
2017-08-30 09:34:54 -07:00
|
|
|
|
return nil
|
|
|
|
|
}
|
|
|
|
|
|
2018-05-17 06:04:32 -07:00
|
|
|
|
func (a *headAppender) log() error {
|
|
|
|
|
if a.head.wal == nil {
|
|
|
|
|
return nil
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
buf := a.head.getBytesBuffer()
|
|
|
|
|
defer func() { a.head.putBytesBuffer(buf) }()
|
|
|
|
|
|
|
|
|
|
var rec []byte
|
2019-09-19 02:15:41 -07:00
|
|
|
|
var enc record.Encoder
|
2018-05-17 06:04:32 -07:00
|
|
|
|
|
|
|
|
|
if len(a.series) > 0 {
|
|
|
|
|
rec = enc.Series(a.series, buf)
|
|
|
|
|
buf = rec[:0]
|
|
|
|
|
|
|
|
|
|
if err := a.head.wal.Log(rec); err != nil {
|
|
|
|
|
return errors.Wrap(err, "log series")
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
if len(a.samples) > 0 {
|
|
|
|
|
rec = enc.Samples(a.samples, buf)
|
|
|
|
|
buf = rec[:0]
|
|
|
|
|
|
|
|
|
|
if err := a.head.wal.Log(rec); err != nil {
|
|
|
|
|
return errors.Wrap(err, "log samples")
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
return nil
|
|
|
|
|
}
|
|
|
|
|
|
2017-08-30 09:34:54 -07:00
|
|
|
|
func (a *headAppender) Commit() error {
|
2018-06-28 06:04:07 -07:00
|
|
|
|
defer a.head.metrics.activeAppenders.Dec()
|
|
|
|
|
defer a.head.putAppendBuffer(a.samples)
|
2019-09-19 02:15:41 -07:00
|
|
|
|
defer a.head.putSeriesBuffer(a.sampleSeries)
|
2017-08-30 09:34:54 -07:00
|
|
|
|
|
2018-05-17 06:04:32 -07:00
|
|
|
|
if err := a.log(); err != nil {
|
|
|
|
|
return errors.Wrap(err, "write to WAL")
|
2017-08-30 09:34:54 -07:00
|
|
|
|
}
|
|
|
|
|
|
2017-09-05 02:45:18 -07:00
|
|
|
|
total := len(a.samples)
|
2019-09-19 02:15:41 -07:00
|
|
|
|
var series *memSeries
|
|
|
|
|
for i, s := range a.samples {
|
|
|
|
|
series = a.sampleSeries[i]
|
|
|
|
|
series.Lock()
|
|
|
|
|
ok, chunkCreated := series.append(s.T, s.V)
|
|
|
|
|
series.pendingCommit = false
|
|
|
|
|
series.Unlock()
|
2017-09-07 23:48:19 -07:00
|
|
|
|
|
2017-08-30 09:34:54 -07:00
|
|
|
|
if !ok {
|
|
|
|
|
total--
|
|
|
|
|
}
|
|
|
|
|
if chunkCreated {
|
|
|
|
|
a.head.metrics.chunks.Inc()
|
|
|
|
|
a.head.metrics.chunksCreated.Inc()
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
a.head.metrics.samplesAppended.Add(float64(total))
|
2018-05-25 14:19:32 -07:00
|
|
|
|
a.head.updateMinMaxTime(a.mint, a.maxt)
|
2017-08-30 09:34:54 -07:00
|
|
|
|
|
|
|
|
|
return nil
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
func (a *headAppender) Rollback() error {
|
|
|
|
|
a.head.metrics.activeAppenders.Dec()
|
2019-09-19 02:15:41 -07:00
|
|
|
|
var series *memSeries
|
|
|
|
|
for i := range a.samples {
|
|
|
|
|
series = a.sampleSeries[i]
|
|
|
|
|
series.Lock()
|
|
|
|
|
series.pendingCommit = false
|
|
|
|
|
series.Unlock()
|
2018-09-17 09:58:42 -07:00
|
|
|
|
}
|
2017-08-30 09:34:54 -07:00
|
|
|
|
a.head.putAppendBuffer(a.samples)
|
|
|
|
|
|
2018-06-28 06:04:07 -07:00
|
|
|
|
// Series are created in the head memory regardless of rollback. Thus we have
|
|
|
|
|
// to log them to the WAL in any case.
|
2018-05-17 06:04:32 -07:00
|
|
|
|
a.samples = nil
|
|
|
|
|
return a.log()
|
2017-08-30 09:34:54 -07:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
// Delete all samples in the range of [mint, maxt] for series that satisfy the given
|
|
|
|
|
// label matchers.
|
2019-11-18 11:53:33 -08:00
|
|
|
|
func (h *Head) Delete(mint, maxt int64, ms ...*labels.Matcher) error {
|
2017-08-30 09:34:54 -07: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 10:08:21 -08:00
|
|
|
|
p, err := PostingsForMatchers(ir, ms...)
|
2017-11-13 03:16:58 -08:00
|
|
|
|
if err != nil {
|
|
|
|
|
return errors.Wrap(err, "select series")
|
|
|
|
|
}
|
2017-08-30 09:34:54 -07:00
|
|
|
|
|
2019-09-19 02:15:41 -07:00
|
|
|
|
var stones []tombstones.Stone
|
2019-01-08 09:08:41 -08:00
|
|
|
|
dirty := false
|
2017-08-30 09:34:54 -07:00
|
|
|
|
for p.Next() {
|
2017-09-05 02:45:18 -07:00
|
|
|
|
series := h.series.getByID(p.At())
|
2017-08-30 09:34:54 -07:00
|
|
|
|
|
2018-02-07 05:43:21 -08:00
|
|
|
|
t0, t1 := series.minTime(), series.maxTime()
|
|
|
|
|
if t0 == math.MinInt64 || t1 == math.MinInt64 {
|
|
|
|
|
continue
|
|
|
|
|
}
|
2017-08-30 09:34:54 -07:00
|
|
|
|
// Delete only until the current values and not beyond.
|
2018-02-07 05:43:21 -08:00
|
|
|
|
t0, t1 = clampInterval(mint, maxt, t0, t1)
|
2019-01-08 09:08:41 -08:00
|
|
|
|
if h.wal != nil {
|
2019-09-19 02:15:41 -07:00
|
|
|
|
stones = append(stones, tombstones.Stone{Ref: p.At(), Intervals: tombstones.Intervals{{Mint: t0, Maxt: t1}}})
|
2019-01-08 09:08:41 -08:00
|
|
|
|
}
|
2019-09-19 02:15:41 -07:00
|
|
|
|
if err := h.chunkRewrite(p.At(), tombstones.Intervals{{Mint: t0, Maxt: t1}}); err != nil {
|
2019-01-08 09:08:41 -08:00
|
|
|
|
return errors.Wrap(err, "delete samples")
|
|
|
|
|
}
|
|
|
|
|
dirty = true
|
2017-08-30 09:34:54 -07:00
|
|
|
|
}
|
|
|
|
|
if p.Err() != nil {
|
|
|
|
|
return p.Err()
|
|
|
|
|
}
|
2019-09-19 02:15:41 -07:00
|
|
|
|
var enc record.Encoder
|
2018-05-17 06:04:32 -07:00
|
|
|
|
if h.wal != nil {
|
2019-01-08 09:08:41 -08:00
|
|
|
|
// Although we don't store the stones in the head
|
2019-03-18 01:37:20 -07:00
|
|
|
|
// we need to write them to the WAL to mark these as deleted
|
|
|
|
|
// after a restart while loading the WAL.
|
2018-05-17 06:04:32 -07:00
|
|
|
|
if err := h.wal.Log(enc.Tombstones(stones, nil)); err != nil {
|
|
|
|
|
return err
|
|
|
|
|
}
|
2017-08-30 09:34:54 -07:00
|
|
|
|
}
|
2019-01-08 09:08:41 -08:00
|
|
|
|
if dirty {
|
|
|
|
|
h.gc()
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
return nil
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
// chunkRewrite re-writes the chunks which overlaps with deleted ranges
|
|
|
|
|
// and removes the samples in the deleted ranges.
|
|
|
|
|
// Chunks is deleted if no samples are left at the end.
|
2019-09-19 02:15:41 -07:00
|
|
|
|
func (h *Head) chunkRewrite(ref uint64, dranges tombstones.Intervals) (err error) {
|
2019-01-08 09:08:41 -08:00
|
|
|
|
if len(dranges) == 0 {
|
|
|
|
|
return nil
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
ms := h.series.getByID(ref)
|
|
|
|
|
ms.Lock()
|
|
|
|
|
defer ms.Unlock()
|
|
|
|
|
if len(ms.chunks) == 0 {
|
|
|
|
|
return nil
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
metas := ms.chunksMetas()
|
|
|
|
|
mint, maxt := metas[0].MinTime, metas[len(metas)-1].MaxTime
|
|
|
|
|
it := newChunkSeriesIterator(metas, dranges, mint, maxt)
|
|
|
|
|
|
|
|
|
|
ms.reset()
|
|
|
|
|
for it.Next() {
|
|
|
|
|
t, v := it.At()
|
|
|
|
|
ok, _ := ms.append(t, v)
|
|
|
|
|
if !ok {
|
|
|
|
|
level.Warn(h.logger).Log("msg", "failed to add sample during delete")
|
|
|
|
|
}
|
2017-08-30 09:34:54 -07:00
|
|
|
|
}
|
2019-01-08 09:08:41 -08:00
|
|
|
|
|
2017-08-30 09:34:54 -07:00
|
|
|
|
return nil
|
|
|
|
|
}
|
|
|
|
|
|
2018-01-13 09:51:50 -08:00
|
|
|
|
// gc removes data before the minimum timestamp from the head.
|
2017-08-30 09:34:54 -07:00
|
|
|
|
func (h *Head) gc() {
|
2017-08-28 15:39:17 -07:00
|
|
|
|
// Only data strictly lower than this timestamp must be deleted.
|
|
|
|
|
mint := h.MinTime()
|
2017-01-19 05:01:38 -08:00
|
|
|
|
|
2017-09-05 02:45:18 -07:00
|
|
|
|
// Drop old chunks and remember series IDs and hashes if they can be
|
|
|
|
|
// deleted entirely.
|
|
|
|
|
deleted, chunksRemoved := h.series.gc(mint)
|
|
|
|
|
seriesRemoved := len(deleted)
|
2017-03-20 02:41:43 -07:00
|
|
|
|
|
2017-09-05 02:45:18 -07:00
|
|
|
|
h.metrics.seriesRemoved.Add(float64(seriesRemoved))
|
|
|
|
|
h.metrics.chunksRemoved.Add(float64(chunksRemoved))
|
|
|
|
|
h.metrics.chunks.Sub(float64(chunksRemoved))
|
2019-08-27 10:33:40 -07:00
|
|
|
|
// Using AddUint64 to subtract series removed.
|
2019-07-23 01:04:48 -07:00
|
|
|
|
// See: https://golang.org/pkg/sync/atomic/#AddUint64.
|
|
|
|
|
atomic.AddUint64(&h.numSeries, ^uint64(seriesRemoved-1))
|
2017-03-04 07:50:48 -08:00
|
|
|
|
|
2017-11-30 06:34:49 -08:00
|
|
|
|
// Remove deleted series IDs from the postings lists.
|
|
|
|
|
h.postings.Delete(deleted)
|
2017-09-05 02:45:18 -07:00
|
|
|
|
|
2019-04-09 06:16:24 -07:00
|
|
|
|
if h.wal != nil {
|
|
|
|
|
_, last, _ := h.wal.Segments()
|
|
|
|
|
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()
|
|
|
|
|
}
|
|
|
|
|
|
2017-09-05 02:45:18 -07:00
|
|
|
|
// Rebuild symbols and label value indices from what is left in the postings terms.
|
2019-03-20 01:43:07 -07:00
|
|
|
|
symbols := make(map[string]struct{}, len(h.symbols))
|
2017-08-28 15:39:17 -07:00
|
|
|
|
values := make(map[string]stringset, len(h.values))
|
2017-06-05 01:18:31 -07:00
|
|
|
|
|
2018-09-20 01:33:52 -07:00
|
|
|
|
if err := h.postings.Iter(func(t labels.Label, _ index.Postings) error {
|
2017-09-05 02:45:18 -07:00
|
|
|
|
symbols[t.Name] = struct{}{}
|
|
|
|
|
symbols[t.Value] = struct{}{}
|
2017-08-05 04:31:48 -07:00
|
|
|
|
|
2017-09-05 02:45:18 -07:00
|
|
|
|
ss, ok := values[t.Name]
|
2017-08-28 15:39:17 -07:00
|
|
|
|
if !ok {
|
|
|
|
|
ss = stringset{}
|
2017-09-05 02:45:18 -07:00
|
|
|
|
values[t.Name] = ss
|
2017-08-28 15:39:17 -07:00
|
|
|
|
}
|
2017-09-05 02:45:18 -07:00
|
|
|
|
ss.set(t.Value)
|
2017-11-30 06:34:49 -08:00
|
|
|
|
return nil
|
2018-09-20 01:33:52 -07:00
|
|
|
|
}); err != nil {
|
|
|
|
|
// This should never happen, as the iteration function only returns nil.
|
|
|
|
|
panic(err)
|
|
|
|
|
}
|
2017-09-05 02:45:18 -07:00
|
|
|
|
|
|
|
|
|
h.symMtx.Lock()
|
|
|
|
|
|
2017-08-28 15:39:17 -07:00
|
|
|
|
h.symbols = symbols
|
|
|
|
|
h.values = values
|
2017-08-30 08:38:25 -07:00
|
|
|
|
|
2017-09-05 02:45:18 -07:00
|
|
|
|
h.symMtx.Unlock()
|
2017-03-20 02:21:21 -07:00
|
|
|
|
}
|
|
|
|
|
|
2017-10-09 06:21:46 -07:00
|
|
|
|
// Tombstones returns a new reader over the head's tombstones
|
2019-09-19 02:15:41 -07:00
|
|
|
|
func (h *Head) Tombstones() (tombstones.Reader, error) {
|
2019-01-08 09:08:41 -08:00
|
|
|
|
return emptyTombstoneReader, nil
|
2017-06-25 10:02:02 -07:00
|
|
|
|
}
|
|
|
|
|
|
2017-08-28 15:39:17 -07:00
|
|
|
|
// Index returns an IndexReader against the block.
|
2017-10-09 06:21:46 -07:00
|
|
|
|
func (h *Head) Index() (IndexReader, error) {
|
|
|
|
|
return h.indexRange(math.MinInt64, math.MaxInt64), nil
|
2017-03-20 00:41:56 -07:00
|
|
|
|
}
|
|
|
|
|
|
2017-08-28 15:39:17 -07:00
|
|
|
|
func (h *Head) indexRange(mint, maxt int64) *headIndexReader {
|
|
|
|
|
if hmin := h.MinTime(); hmin > mint {
|
|
|
|
|
mint = hmin
|
2017-01-12 11:00:36 -08:00
|
|
|
|
}
|
2017-08-28 15:39:17 -07:00
|
|
|
|
return &headIndexReader{head: h, mint: mint, maxt: maxt}
|
2017-01-12 10:18:51 -08:00
|
|
|
|
}
|
|
|
|
|
|
2017-08-28 15:39:17 -07:00
|
|
|
|
// Chunks returns a ChunkReader against the block.
|
2017-10-09 06:21:46 -07:00
|
|
|
|
func (h *Head) Chunks() (ChunkReader, error) {
|
|
|
|
|
return h.chunksRange(math.MinInt64, math.MaxInt64), nil
|
2017-01-12 10:18:51 -08:00
|
|
|
|
}
|
|
|
|
|
|
2017-08-28 15:39:17 -07:00
|
|
|
|
func (h *Head) chunksRange(mint, maxt int64) *headChunkReader {
|
|
|
|
|
if hmin := h.MinTime(); hmin > mint {
|
|
|
|
|
mint = hmin
|
2017-01-12 10:18:51 -08:00
|
|
|
|
}
|
2017-08-28 15:39:17 -07:00
|
|
|
|
return &headChunkReader{head: h, mint: mint, maxt: maxt}
|
2017-01-12 10:18:51 -08:00
|
|
|
|
}
|
|
|
|
|
|
2019-07-23 01:04:48 -07:00
|
|
|
|
// NumSeries returns the number of active series in the head.
|
|
|
|
|
func (h *Head) NumSeries() uint64 {
|
|
|
|
|
return atomic.LoadUint64(&h.numSeries)
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
// 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 15:39:17 -07:00
|
|
|
|
// MinTime returns the lowest time bound on visible data in the head.
|
|
|
|
|
func (h *Head) MinTime() int64 {
|
|
|
|
|
return atomic.LoadInt64(&h.minTime)
|
2017-01-12 10:18:51 -08:00
|
|
|
|
}
|
|
|
|
|
|
2017-08-28 15:39:17 -07:00
|
|
|
|
// MaxTime returns the highest timestamp seen in data of the head.
|
|
|
|
|
func (h *Head) MaxTime() int64 {
|
|
|
|
|
return atomic.LoadInt64(&h.maxTime)
|
2017-01-12 10:18:51 -08:00
|
|
|
|
}
|
|
|
|
|
|
2019-04-01 01:19:06 -07: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 {
|
|
|
|
|
return h.MaxTime()-h.MinTime() > h.chunkRange/2*3
|
|
|
|
|
}
|
|
|
|
|
|
2017-11-10 12:19:39 -08:00
|
|
|
|
// Close flushes the WAL and closes the head.
|
|
|
|
|
func (h *Head) Close() error {
|
2018-05-17 06:04:32 -07:00
|
|
|
|
if h.wal == nil {
|
|
|
|
|
return nil
|
|
|
|
|
}
|
2017-11-10 12:19:39 -08:00
|
|
|
|
return h.wal.Close()
|
|
|
|
|
}
|
|
|
|
|
|
2017-08-28 15:39:17 -07:00
|
|
|
|
type headChunkReader struct {
|
|
|
|
|
head *Head
|
|
|
|
|
mint, maxt int64
|
2017-01-12 10:18:51 -08:00
|
|
|
|
}
|
|
|
|
|
|
2017-08-28 15:39:17 -07:00
|
|
|
|
func (h *headChunkReader) Close() error {
|
2017-01-12 10:18:51 -08:00
|
|
|
|
return nil
|
|
|
|
|
}
|
|
|
|
|
|
2017-09-04 07:08:38 -07:00
|
|
|
|
// packChunkID packs a seriesID and a chunkID within it into a global 8 byte ID.
|
|
|
|
|
// It panicks if the seriesID exceeds 5 bytes or the chunk ID 3 bytes.
|
|
|
|
|
func packChunkID(seriesID, chunkID uint64) uint64 {
|
|
|
|
|
if seriesID > (1<<40)-1 {
|
|
|
|
|
panic("series ID exceeds 5 bytes")
|
|
|
|
|
}
|
|
|
|
|
if chunkID > (1<<24)-1 {
|
|
|
|
|
panic("chunk ID exceeds 3 bytes")
|
|
|
|
|
}
|
|
|
|
|
return (seriesID << 24) | chunkID
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
func unpackChunkID(id uint64) (seriesID, chunkID uint64) {
|
|
|
|
|
return id >> 24, (id << 40) >> 40
|
|
|
|
|
}
|
|
|
|
|
|
2016-12-15 07:14:33 -08:00
|
|
|
|
// Chunk returns the chunk for the reference number.
|
2017-11-30 06:34:49 -08:00
|
|
|
|
func (h *headChunkReader) Chunk(ref uint64) (chunkenc.Chunk, error) {
|
2017-09-04 07:08:38 -07:00
|
|
|
|
sid, cid := unpackChunkID(ref)
|
2017-09-05 02:45:18 -07:00
|
|
|
|
|
|
|
|
|
s := h.head.series.getByID(sid)
|
2017-12-13 12:58:21 -08:00
|
|
|
|
// This means that the series has been garbage collected.
|
|
|
|
|
if s == nil {
|
|
|
|
|
return nil, ErrNotFound
|
|
|
|
|
}
|
2017-01-06 08:23:12 -08:00
|
|
|
|
|
2017-09-07 23:48:19 -07:00
|
|
|
|
s.Lock()
|
2017-09-04 07:08:38 -07:00
|
|
|
|
c := s.chunk(int(cid))
|
2017-12-13 12:58:21 -08:00
|
|
|
|
|
2018-07-02 01:23:36 -07:00
|
|
|
|
// This means that the chunk has been garbage collected or is outside
|
|
|
|
|
// the specified range.
|
|
|
|
|
if c == nil || !c.OverlapsClosedInterval(h.mint, h.maxt) {
|
2017-12-13 12:58:21 -08:00
|
|
|
|
s.Unlock()
|
|
|
|
|
return nil, ErrNotFound
|
|
|
|
|
}
|
2017-09-07 23:48:19 -07:00
|
|
|
|
s.Unlock()
|
2017-02-18 08:33:20 -08:00
|
|
|
|
|
2017-08-28 15:39:17 -07:00
|
|
|
|
return &safeChunk{
|
|
|
|
|
Chunk: c.chunk,
|
|
|
|
|
s: s,
|
2017-09-04 07:08:38 -07:00
|
|
|
|
cid: int(cid),
|
2017-08-28 15:39:17 -07:00
|
|
|
|
}, nil
|
2016-12-14 09:38:46 -08:00
|
|
|
|
}
|
|
|
|
|
|
2017-01-09 07:51:39 -08:00
|
|
|
|
type safeChunk struct {
|
2017-11-30 06:34:49 -08:00
|
|
|
|
chunkenc.Chunk
|
2017-08-28 15:39:17 -07:00
|
|
|
|
s *memSeries
|
|
|
|
|
cid int
|
2017-01-09 07:51:39 -08:00
|
|
|
|
}
|
|
|
|
|
|
2019-07-09 02:49:34 -07:00
|
|
|
|
func (c *safeChunk) Iterator(reuseIter chunkenc.Iterator) chunkenc.Iterator {
|
2017-09-07 23:48:19 -07:00
|
|
|
|
c.s.Lock()
|
2019-07-09 02:49:34 -07:00
|
|
|
|
it := c.s.iterator(c.cid, reuseIter)
|
2017-09-07 23:48:19 -07:00
|
|
|
|
c.s.Unlock()
|
|
|
|
|
return it
|
2017-01-09 07:51:39 -08:00
|
|
|
|
}
|
|
|
|
|
|
2017-01-10 06:28:22 -08:00
|
|
|
|
type headIndexReader struct {
|
2017-08-28 15:39:17 -07:00
|
|
|
|
head *Head
|
|
|
|
|
mint, maxt int64
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
func (h *headIndexReader) Close() error {
|
|
|
|
|
return nil
|
2017-08-05 04:31:48 -07:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
func (h *headIndexReader) Symbols() (map[string]struct{}, error) {
|
2017-09-05 02:45:18 -07:00
|
|
|
|
h.head.symMtx.RLock()
|
|
|
|
|
defer h.head.symMtx.RUnlock()
|
|
|
|
|
|
|
|
|
|
res := make(map[string]struct{}, len(h.head.symbols))
|
|
|
|
|
|
|
|
|
|
for s := range h.head.symbols {
|
|
|
|
|
res[s] = struct{}{}
|
|
|
|
|
}
|
|
|
|
|
return res, nil
|
2016-12-14 09:38:46 -08:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
// LabelValues returns the possible label values
|
2017-11-30 06:34:49 -08:00
|
|
|
|
func (h *headIndexReader) LabelValues(names ...string) (index.StringTuples, error) {
|
2016-12-14 09:38:46 -08:00
|
|
|
|
if len(names) != 1 {
|
2019-02-22 09:11:11 -08:00
|
|
|
|
return nil, encoding.ErrInvalidSize
|
2016-12-14 09:38:46 -08:00
|
|
|
|
}
|
|
|
|
|
|
2017-09-05 02:45:18 -07:00
|
|
|
|
h.head.symMtx.RLock()
|
2018-10-22 03:52:02 -07:00
|
|
|
|
sl := make([]string, 0, len(h.head.values[names[0]]))
|
2017-08-28 15:39:17 -07:00
|
|
|
|
for s := range h.head.values[names[0]] {
|
2016-12-14 09:38:46 -08:00
|
|
|
|
sl = append(sl, s)
|
|
|
|
|
}
|
2018-10-22 03:52:02 -07:00
|
|
|
|
h.head.symMtx.RUnlock()
|
2016-12-14 09:38:46 -08:00
|
|
|
|
sort.Strings(sl)
|
|
|
|
|
|
2017-11-30 06:34:49 -08:00
|
|
|
|
return index.NewStringTuples(sl, len(names))
|
2016-12-14 09:38:46 -08:00
|
|
|
|
}
|
|
|
|
|
|
2018-11-07 07:52:41 -08:00
|
|
|
|
// LabelNames returns all the unique label names present in the head.
|
|
|
|
|
func (h *headIndexReader) LabelNames() ([]string, error) {
|
|
|
|
|
h.head.symMtx.RLock()
|
|
|
|
|
defer h.head.symMtx.RUnlock()
|
|
|
|
|
labelNames := make([]string, 0, len(h.head.values))
|
|
|
|
|
for name := range h.head.values {
|
|
|
|
|
if name == "" {
|
|
|
|
|
continue
|
|
|
|
|
}
|
|
|
|
|
labelNames = append(labelNames, name)
|
|
|
|
|
}
|
|
|
|
|
sort.Strings(labelNames)
|
|
|
|
|
return labelNames, nil
|
|
|
|
|
}
|
|
|
|
|
|
2016-12-14 09:38:46 -08:00
|
|
|
|
// Postings returns the postings list iterator for the label pair.
|
2017-11-30 06:34:49 -08:00
|
|
|
|
func (h *headIndexReader) Postings(name, value string) (index.Postings, error) {
|
|
|
|
|
return h.head.postings.Get(name, value), nil
|
2016-12-14 09:38:46 -08:00
|
|
|
|
}
|
|
|
|
|
|
2017-11-30 06:34:49 -08:00
|
|
|
|
func (h *headIndexReader) SortedPostings(p index.Postings) index.Postings {
|
2019-01-03 02:35:10 -08:00
|
|
|
|
series := make([]*memSeries, 0, 128)
|
2017-08-05 04:31:48 -07:00
|
|
|
|
|
2019-01-03 02:35:10 -08:00
|
|
|
|
// Fetch all the series only once.
|
2017-08-05 04:31:48 -07:00
|
|
|
|
for p.Next() {
|
2019-01-03 02:35:10 -08:00
|
|
|
|
s := h.head.series.getByID(p.At())
|
|
|
|
|
if s == nil {
|
|
|
|
|
level.Debug(h.head.logger).Log("msg", "looked up series not found")
|
|
|
|
|
} else {
|
|
|
|
|
series = append(series, s)
|
|
|
|
|
}
|
2017-08-05 04:31:48 -07:00
|
|
|
|
}
|
|
|
|
|
if err := p.Err(); err != nil {
|
2017-11-30 06:34:49 -08:00
|
|
|
|
return index.ErrPostings(errors.Wrap(err, "expand postings"))
|
2017-08-05 04:31:48 -07:00
|
|
|
|
}
|
|
|
|
|
|
2019-01-03 02:35:10 -08:00
|
|
|
|
sort.Slice(series, func(i, j int) bool {
|
|
|
|
|
return labels.Compare(series[i].lset, series[j].lset) < 0
|
2017-08-05 04:31:48 -07:00
|
|
|
|
})
|
2019-01-03 02:35:10 -08:00
|
|
|
|
|
|
|
|
|
// Convert back to list.
|
|
|
|
|
ep := make([]uint64, 0, len(series))
|
|
|
|
|
for _, p := range series {
|
|
|
|
|
ep = append(ep, p.ref)
|
|
|
|
|
}
|
2017-11-30 06:34:49 -08:00
|
|
|
|
return index.NewListPostings(ep)
|
2017-08-05 04:31:48 -07:00
|
|
|
|
}
|
|
|
|
|
|
2016-12-14 09:38:46 -08:00
|
|
|
|
// Series returns the series for the given reference.
|
2017-11-30 06:34:49 -08:00
|
|
|
|
func (h *headIndexReader) Series(ref uint64, lbls *labels.Labels, chks *[]chunks.Meta) error {
|
2017-09-05 02:45:18 -07:00
|
|
|
|
s := h.head.series.getByID(ref)
|
2017-01-06 08:23:12 -08:00
|
|
|
|
|
2017-05-17 07:43:01 -07:00
|
|
|
|
if s == nil {
|
2017-10-12 06:25:12 -07:00
|
|
|
|
h.head.metrics.seriesNotFound.Inc()
|
2017-08-05 04:31:48 -07:00
|
|
|
|
return ErrNotFound
|
2017-05-17 07:43:01 -07:00
|
|
|
|
}
|
2017-08-05 04:31:48 -07:00
|
|
|
|
*lbls = append((*lbls)[:0], s.lset...)
|
2017-01-11 04:02:38 -08:00
|
|
|
|
|
2017-09-07 23:48:19 -07:00
|
|
|
|
s.Lock()
|
|
|
|
|
defer s.Unlock()
|
2017-01-11 04:02:38 -08:00
|
|
|
|
|
2017-08-05 04:31:48 -07:00
|
|
|
|
*chks = (*chks)[:0]
|
|
|
|
|
|
2017-01-11 04:02:38 -08:00
|
|
|
|
for i, c := range s.chunks {
|
2017-08-28 15:39:17 -07:00
|
|
|
|
// Do not expose chunks that are outside of the specified range.
|
2018-07-02 01:23:36 -07:00
|
|
|
|
if !c.OverlapsClosedInterval(h.mint, h.maxt) {
|
2017-08-28 15:39:17 -07:00
|
|
|
|
continue
|
|
|
|
|
}
|
2019-07-03 03:47:31 -07:00
|
|
|
|
// Set the head chunks as open (being appended to).
|
|
|
|
|
maxTime := c.maxTime
|
|
|
|
|
if s.headChunk == c {
|
|
|
|
|
maxTime = math.MaxInt64
|
|
|
|
|
}
|
|
|
|
|
|
2017-11-30 06:34:49 -08:00
|
|
|
|
*chks = append(*chks, chunks.Meta{
|
2017-01-11 04:02:38 -08:00
|
|
|
|
MinTime: c.minTime,
|
2019-07-03 03:47:31 -07:00
|
|
|
|
MaxTime: maxTime,
|
2017-09-04 07:08:38 -07:00
|
|
|
|
Ref: packChunkID(s.ref, uint64(s.chunkID(i))),
|
2017-01-11 04:02:38 -08:00
|
|
|
|
})
|
2017-01-03 06:43:26 -08:00
|
|
|
|
}
|
2017-01-11 04:02:38 -08:00
|
|
|
|
|
2017-08-05 04:31:48 -07:00
|
|
|
|
return nil
|
2016-12-31 06:35:08 -08:00
|
|
|
|
}
|
|
|
|
|
|
2017-01-10 06:28:22 -08:00
|
|
|
|
func (h *headIndexReader) LabelIndices() ([][]string, error) {
|
2017-09-05 02:45:18 -07:00
|
|
|
|
h.head.symMtx.RLock()
|
|
|
|
|
defer h.head.symMtx.RUnlock()
|
2016-12-31 06:35:08 -08:00
|
|
|
|
res := [][]string{}
|
2017-08-28 15:39:17 -07:00
|
|
|
|
for s := range h.head.values {
|
2016-12-31 06:35:08 -08:00
|
|
|
|
res = append(res, []string{s})
|
2016-12-14 09:38:46 -08:00
|
|
|
|
}
|
2016-12-31 06:35:08 -08:00
|
|
|
|
return res, nil
|
2016-12-14 09:38:46 -08:00
|
|
|
|
}
|
|
|
|
|
|
2017-09-18 03:28:56 -07:00
|
|
|
|
func (h *Head) getOrCreate(hash uint64, lset labels.Labels) (*memSeries, bool) {
|
|
|
|
|
// Just using `getOrSet` below would be semantically sufficient, but we'd create
|
|
|
|
|
// 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 {
|
|
|
|
|
return s, false
|
|
|
|
|
}
|
2017-08-30 09:34:54 -07:00
|
|
|
|
|
2017-09-05 02:45:18 -07:00
|
|
|
|
// Optimistically assume that we are the first one to create the series.
|
2017-09-04 07:08:38 -07:00
|
|
|
|
id := atomic.AddUint64(&h.lastSeriesID, 1)
|
2017-09-19 01:20:19 -07:00
|
|
|
|
|
|
|
|
|
return h.getOrCreateWithID(id, hash, lset)
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
func (h *Head) getOrCreateWithID(id, hash uint64, lset labels.Labels) (*memSeries, bool) {
|
|
|
|
|
s := newMemSeries(lset, id, h.chunkRange)
|
2016-12-21 16:12:28 -08:00
|
|
|
|
|
2017-09-05 02:45:18 -07:00
|
|
|
|
s, created := h.series.getOrSet(hash, s)
|
|
|
|
|
if !created {
|
2017-09-18 03:28:56 -07:00
|
|
|
|
return s, false
|
2017-09-05 02:45:18 -07:00
|
|
|
|
}
|
|
|
|
|
|
2017-09-18 03:28:56 -07:00
|
|
|
|
h.metrics.seriesCreated.Inc()
|
2019-07-23 01:04:48 -07:00
|
|
|
|
atomic.AddUint64(&h.numSeries, 1)
|
2017-09-18 03:28:56 -07:00
|
|
|
|
|
2017-11-30 06:34:49 -08:00
|
|
|
|
h.postings.Add(id, lset)
|
2017-09-05 02:45:18 -07:00
|
|
|
|
|
|
|
|
|
h.symMtx.Lock()
|
|
|
|
|
defer h.symMtx.Unlock()
|
2016-12-21 16:12:28 -08:00
|
|
|
|
|
|
|
|
|
for _, l := range lset {
|
|
|
|
|
valset, ok := h.values[l.Name]
|
|
|
|
|
if !ok {
|
|
|
|
|
valset = stringset{}
|
|
|
|
|
h.values[l.Name] = valset
|
|
|
|
|
}
|
|
|
|
|
valset.set(l.Value)
|
2017-01-03 06:43:26 -08:00
|
|
|
|
|
2017-08-05 04:31:48 -07:00
|
|
|
|
h.symbols[l.Name] = struct{}{}
|
|
|
|
|
h.symbols[l.Value] = struct{}{}
|
2016-12-21 16:12:28 -08:00
|
|
|
|
}
|
2017-01-03 06:43:26 -08:00
|
|
|
|
|
2017-09-18 03:28:56 -07:00
|
|
|
|
return s, true
|
2017-09-05 02:45:18 -07: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 04:16:11 -08:00
|
|
|
|
|
2017-09-05 02:45:18 -07:00
|
|
|
|
func (m seriesHashmap) get(hash uint64, lset labels.Labels) *memSeries {
|
|
|
|
|
for _, s := range m[hash] {
|
2019-11-18 11:53:33 -08:00
|
|
|
|
if labels.Equal(s.lset, lset) {
|
2017-09-05 02:45:18 -07:00
|
|
|
|
return s
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
return nil
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
func (m seriesHashmap) set(hash uint64, s *memSeries) {
|
|
|
|
|
l := m[hash]
|
|
|
|
|
for i, prev := range l {
|
2019-11-18 11:53:33 -08:00
|
|
|
|
if labels.Equal(prev.lset, s.lset) {
|
2017-09-05 02:45:18 -07: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 11:53:33 -08:00
|
|
|
|
if !labels.Equal(s.lset, lset) {
|
2017-09-05 02:45:18 -07:00
|
|
|
|
rem = append(rem, s)
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
if len(rem) == 0 {
|
|
|
|
|
delete(m, hash)
|
|
|
|
|
} else {
|
|
|
|
|
m[hash] = rem
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
// stripeSeries locks modulo ranges of IDs and hashes to reduce lock contention.
|
2018-01-13 09:51:50 -08:00
|
|
|
|
// The locks are padded to not be on the same cache line. Filling the padded space
|
2017-09-05 02:45:18 -07:00
|
|
|
|
// with the maps was profiled to be slower – likely due to the additional pointer
|
|
|
|
|
// dereferences.
|
|
|
|
|
type stripeSeries struct {
|
|
|
|
|
series [stripeSize]map[uint64]*memSeries
|
|
|
|
|
hashes [stripeSize]seriesHashmap
|
|
|
|
|
locks [stripeSize]stripeLock
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
const (
|
|
|
|
|
stripeSize = 1 << 14
|
|
|
|
|
stripeMask = stripeSize - 1
|
|
|
|
|
)
|
|
|
|
|
|
|
|
|
|
type stripeLock struct {
|
|
|
|
|
sync.RWMutex
|
|
|
|
|
// Padding to avoid multiple locks being on the same cache line.
|
|
|
|
|
_ [40]byte
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
func newStripeSeries() *stripeSeries {
|
|
|
|
|
s := &stripeSeries{}
|
|
|
|
|
|
|
|
|
|
for i := range s.series {
|
|
|
|
|
s.series[i] = map[uint64]*memSeries{}
|
|
|
|
|
}
|
|
|
|
|
for i := range s.hashes {
|
|
|
|
|
s.hashes[i] = seriesHashmap{}
|
|
|
|
|
}
|
2017-01-11 04:02:38 -08:00
|
|
|
|
return s
|
2016-12-04 04:16:11 -08:00
|
|
|
|
}
|
|
|
|
|
|
2017-09-05 02:45:18 -07:00
|
|
|
|
// gc garbage collects old chunks that are strictly before mint and removes
|
|
|
|
|
// series entirely that have no chunks left.
|
|
|
|
|
func (s *stripeSeries) gc(mint int64) (map[uint64]struct{}, int) {
|
|
|
|
|
var (
|
|
|
|
|
deleted = map[uint64]struct{}{}
|
|
|
|
|
rmChunks = 0
|
|
|
|
|
)
|
|
|
|
|
// Run through all series and truncate old chunks. Mark those with no
|
2017-09-06 07:20:37 -07:00
|
|
|
|
// chunks left as deleted and store their ID.
|
2017-09-05 02:45:18 -07:00
|
|
|
|
for i := 0; i < stripeSize; i++ {
|
|
|
|
|
s.locks[i].Lock()
|
|
|
|
|
|
|
|
|
|
for hash, all := range s.hashes[i] {
|
|
|
|
|
for _, series := range all {
|
2017-09-07 23:48:19 -07:00
|
|
|
|
series.Lock()
|
2017-09-05 02:45:18 -07:00
|
|
|
|
rmChunks += series.truncateChunksBefore(mint)
|
|
|
|
|
|
2018-09-17 09:58:42 -07:00
|
|
|
|
if len(series.chunks) > 0 || series.pendingCommit {
|
2017-09-07 23:48:19 -07:00
|
|
|
|
series.Unlock()
|
2017-09-05 02:45:18 -07: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.
|
|
|
|
|
j := int(series.ref & stripeMask)
|
|
|
|
|
|
|
|
|
|
if i != j {
|
|
|
|
|
s.locks[j].Lock()
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
deleted[series.ref] = struct{}{}
|
|
|
|
|
s.hashes[i].del(hash, series.lset)
|
|
|
|
|
delete(s.series[j], series.ref)
|
|
|
|
|
|
|
|
|
|
if i != j {
|
|
|
|
|
s.locks[j].Unlock()
|
|
|
|
|
}
|
|
|
|
|
|
2017-09-07 23:48:19 -07:00
|
|
|
|
series.Unlock()
|
2017-09-05 02:45:18 -07:00
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
s.locks[i].Unlock()
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
return deleted, rmChunks
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
func (s *stripeSeries) getByID(id uint64) *memSeries {
|
|
|
|
|
i := id & stripeMask
|
|
|
|
|
|
|
|
|
|
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 {
|
|
|
|
|
i := hash & stripeMask
|
|
|
|
|
|
|
|
|
|
s.locks[i].RLock()
|
|
|
|
|
series := s.hashes[i].get(hash, lset)
|
|
|
|
|
s.locks[i].RUnlock()
|
|
|
|
|
|
|
|
|
|
return series
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
func (s *stripeSeries) getOrSet(hash uint64, series *memSeries) (*memSeries, bool) {
|
|
|
|
|
i := hash & stripeMask
|
|
|
|
|
|
|
|
|
|
s.locks[i].Lock()
|
|
|
|
|
|
|
|
|
|
if prev := s.hashes[i].get(hash, series.lset); prev != nil {
|
2017-09-18 02:23:22 -07:00
|
|
|
|
s.locks[i].Unlock()
|
2017-09-05 02:45:18 -07:00
|
|
|
|
return prev, false
|
|
|
|
|
}
|
|
|
|
|
s.hashes[i].set(hash, series)
|
|
|
|
|
s.locks[i].Unlock()
|
|
|
|
|
|
|
|
|
|
i = series.ref & stripeMask
|
|
|
|
|
|
|
|
|
|
s.locks[i].Lock()
|
|
|
|
|
s.series[i][series.ref] = series
|
|
|
|
|
s.locks[i].Unlock()
|
|
|
|
|
|
|
|
|
|
return series, true
|
|
|
|
|
}
|
|
|
|
|
|
2017-03-24 02:20:39 -07:00
|
|
|
|
type sample struct {
|
|
|
|
|
t int64
|
|
|
|
|
v float64
|
|
|
|
|
}
|
|
|
|
|
|
2018-10-25 13:06:19 -07:00
|
|
|
|
func (s sample) T() int64 {
|
|
|
|
|
return s.t
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
func (s sample) V() float64 {
|
|
|
|
|
return s.v
|
|
|
|
|
}
|
|
|
|
|
|
2017-09-07 23:48:19 -07:00
|
|
|
|
// memSeries is the in-memory representation of a series. None of its methods
|
2018-01-13 09:51:50 -08:00
|
|
|
|
// are goroutine safe and it is the caller's responsibility to lock it.
|
2017-01-11 04:02:38 -08:00
|
|
|
|
type memSeries struct {
|
2017-09-07 23:48:19 -07:00
|
|
|
|
sync.Mutex
|
2017-01-09 07:51:39 -08:00
|
|
|
|
|
2017-09-04 07:08:38 -07:00
|
|
|
|
ref uint64
|
2017-08-28 15:39:17 -07:00
|
|
|
|
lset labels.Labels
|
|
|
|
|
chunks []*memChunk
|
2018-10-31 06:28:56 -07:00
|
|
|
|
headChunk *memChunk
|
2017-08-28 15:39:17 -07:00
|
|
|
|
chunkRange int64
|
|
|
|
|
firstChunkID int
|
2017-01-09 07:51:39 -08:00
|
|
|
|
|
2018-09-17 09:58:42 -07: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 07:51:39 -08:00
|
|
|
|
|
2017-11-30 06:34:49 -08:00
|
|
|
|
app chunkenc.Appender // Current appender for the chunk.
|
2017-01-09 07:51:39 -08:00
|
|
|
|
}
|
|
|
|
|
|
2019-01-08 09:08:41 -08:00
|
|
|
|
func newMemSeries(lset labels.Labels, id uint64, chunkRange int64) *memSeries {
|
|
|
|
|
s := &memSeries{
|
|
|
|
|
lset: lset,
|
|
|
|
|
ref: id,
|
|
|
|
|
chunkRange: chunkRange,
|
|
|
|
|
nextAt: math.MinInt64,
|
|
|
|
|
}
|
|
|
|
|
return s
|
|
|
|
|
}
|
|
|
|
|
|
2017-08-28 15:39:17 -07:00
|
|
|
|
func (s *memSeries) minTime() int64 {
|
2018-02-07 05:43:21 -08:00
|
|
|
|
if len(s.chunks) == 0 {
|
|
|
|
|
return math.MinInt64
|
|
|
|
|
}
|
2017-08-28 15:39:17 -07:00
|
|
|
|
return s.chunks[0].minTime
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
func (s *memSeries) maxTime() int64 {
|
2018-02-07 05:43:21 -08:00
|
|
|
|
c := s.head()
|
|
|
|
|
if c == nil {
|
|
|
|
|
return math.MinInt64
|
|
|
|
|
}
|
|
|
|
|
return c.maxTime
|
2017-08-28 15:39:17 -07:00
|
|
|
|
}
|
|
|
|
|
|
2017-06-07 04:42:53 -07:00
|
|
|
|
func (s *memSeries) cut(mint int64) *memChunk {
|
2017-01-11 04:02:38 -08:00
|
|
|
|
c := &memChunk{
|
2017-11-30 06:34:49 -08:00
|
|
|
|
chunk: chunkenc.NewXORChunk(),
|
2017-06-07 04:42:53 -07:00
|
|
|
|
minTime: mint,
|
2017-01-11 04:02:38 -08:00
|
|
|
|
maxTime: math.MinInt64,
|
|
|
|
|
}
|
|
|
|
|
s.chunks = append(s.chunks, c)
|
2018-10-31 06:28:56 -07:00
|
|
|
|
s.headChunk = c
|
2017-01-11 04:02:38 -08:00
|
|
|
|
|
2017-10-25 00:32:06 -07:00
|
|
|
|
// Set upper bound on when the next chunk must be started. An earlier timestamp
|
|
|
|
|
// may be chosen dynamically at a later point.
|
2018-12-04 02:30:49 -08:00
|
|
|
|
s.nextAt = rangeForTimestamp(mint, s.chunkRange)
|
2017-10-25 00:32:06 -07:00
|
|
|
|
|
2017-01-11 04:02:38 -08:00
|
|
|
|
app, err := c.chunk.Appender()
|
|
|
|
|
if err != nil {
|
|
|
|
|
panic(err)
|
|
|
|
|
}
|
|
|
|
|
s.app = app
|
|
|
|
|
return c
|
|
|
|
|
}
|
|
|
|
|
|
2019-01-08 09:08:41 -08:00
|
|
|
|
func (s *memSeries) chunksMetas() []chunks.Meta {
|
|
|
|
|
metas := make([]chunks.Meta, 0, len(s.chunks))
|
|
|
|
|
for _, chk := range s.chunks {
|
|
|
|
|
metas = append(metas, chunks.Meta{Chunk: chk.chunk, MinTime: chk.minTime, MaxTime: chk.maxTime})
|
2017-06-07 04:42:53 -07:00
|
|
|
|
}
|
2019-01-08 09:08:41 -08:00
|
|
|
|
return metas
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
// reset re-initialises all the variable in the memSeries except 'lset', 'ref',
|
|
|
|
|
// and 'chunkRange', like how it would appear after 'newMemSeries(...)'.
|
|
|
|
|
func (s *memSeries) reset() {
|
|
|
|
|
s.chunks = nil
|
|
|
|
|
s.headChunk = nil
|
|
|
|
|
s.firstChunkID = 0
|
|
|
|
|
s.nextAt = math.MinInt64
|
|
|
|
|
s.sampleBuf = [4]sample{}
|
|
|
|
|
s.pendingCommit = false
|
|
|
|
|
s.app = nil
|
2017-06-07 04:42:53 -07:00
|
|
|
|
}
|
|
|
|
|
|
2017-08-28 15:39:17 -07:00
|
|
|
|
// appendable checks whether the given sample is valid for appending to the series.
|
|
|
|
|
func (s *memSeries) appendable(t int64, v float64) error {
|
2017-09-01 05:38:49 -07:00
|
|
|
|
c := s.head()
|
|
|
|
|
if c == nil {
|
2017-08-28 15:39:17 -07:00
|
|
|
|
return nil
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
if t > c.maxTime {
|
|
|
|
|
return nil
|
|
|
|
|
}
|
|
|
|
|
if t < c.maxTime {
|
|
|
|
|
return ErrOutOfOrderSample
|
|
|
|
|
}
|
|
|
|
|
// We are allowing exact duplicates as we can encounter them in valid cases
|
|
|
|
|
// like federation and erroring out at that time would be extremely noisy.
|
2018-11-14 06:02:32 -08:00
|
|
|
|
if math.Float64bits(s.sampleBuf[3].v) != math.Float64bits(v) {
|
2017-08-28 15:39:17 -07:00
|
|
|
|
return ErrAmendSample
|
|
|
|
|
}
|
|
|
|
|
return nil
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
func (s *memSeries) chunk(id int) *memChunk {
|
2017-09-01 05:38:49 -07:00
|
|
|
|
ix := id - s.firstChunkID
|
|
|
|
|
if ix < 0 || ix >= len(s.chunks) {
|
|
|
|
|
return nil
|
|
|
|
|
}
|
|
|
|
|
return s.chunks[ix]
|
2017-08-28 15:39:17 -07:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
func (s *memSeries) chunkID(pos int) int {
|
|
|
|
|
return pos + s.firstChunkID
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
// truncateChunksBefore removes all chunks from the series that have not timestamp
|
|
|
|
|
// at or after mint. Chunk IDs remain unchanged.
|
2017-08-30 08:38:25 -07:00
|
|
|
|
func (s *memSeries) truncateChunksBefore(mint int64) (removed int) {
|
2017-08-28 15:39:17 -07:00
|
|
|
|
var k int
|
|
|
|
|
for i, c := range s.chunks {
|
|
|
|
|
if c.maxTime >= mint {
|
|
|
|
|
break
|
|
|
|
|
}
|
|
|
|
|
k = i + 1
|
|
|
|
|
}
|
|
|
|
|
s.chunks = append(s.chunks[:0], s.chunks[k:]...)
|
|
|
|
|
s.firstChunkID += k
|
2018-10-31 06:28:56 -07:00
|
|
|
|
if len(s.chunks) == 0 {
|
|
|
|
|
s.headChunk = nil
|
|
|
|
|
} else {
|
|
|
|
|
s.headChunk = s.chunks[len(s.chunks)-1]
|
|
|
|
|
}
|
2017-08-30 08:38:25 -07:00
|
|
|
|
|
|
|
|
|
return k
|
2017-08-28 15:39:17 -07:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
// append adds the sample (t, v) to the series.
|
2017-08-30 08:38:25 -07:00
|
|
|
|
func (s *memSeries) append(t int64, v float64) (success, chunkCreated bool) {
|
2018-10-23 03:43:06 -07:00
|
|
|
|
// Based on Gorilla white papers this offers near-optimal compression ratio
|
|
|
|
|
// so anything bigger that this has diminishing returns and increases
|
|
|
|
|
// the time range within which we have to decompress all samples.
|
2017-06-07 04:42:53 -07:00
|
|
|
|
const samplesPerChunk = 120
|
|
|
|
|
|
2017-09-01 05:38:49 -07:00
|
|
|
|
c := s.head()
|
2017-01-11 04:02:38 -08:00
|
|
|
|
|
2017-09-01 05:38:49 -07:00
|
|
|
|
if c == nil {
|
2017-06-07 04:42:53 -07:00
|
|
|
|
c = s.cut(t)
|
2017-08-30 08:38:25 -07:00
|
|
|
|
chunkCreated = true
|
2017-06-07 04:42:53 -07:00
|
|
|
|
}
|
2017-10-07 06:55:11 -07:00
|
|
|
|
numSamples := c.chunk.NumSamples()
|
|
|
|
|
|
2017-10-25 00:32:06 -07:00
|
|
|
|
// Out of order sample.
|
2017-06-07 04:42:53 -07:00
|
|
|
|
if c.maxTime >= t {
|
2017-08-30 08:38:25 -07:00
|
|
|
|
return false, chunkCreated
|
2017-06-07 04:42:53 -07:00
|
|
|
|
}
|
2017-10-25 00:32:06 -07:00
|
|
|
|
// If we reach 25% of a chunk's desired sample count, set a definitive time
|
|
|
|
|
// at which to start the next chunk.
|
|
|
|
|
// At latest it must happen at the timestamp set when the chunk was cut.
|
|
|
|
|
if numSamples == samplesPerChunk/4 {
|
|
|
|
|
s.nextAt = computeChunkEndTime(c.minTime, c.maxTime, s.nextAt)
|
|
|
|
|
}
|
|
|
|
|
if t >= s.nextAt {
|
2017-06-07 04:42:53 -07:00
|
|
|
|
c = s.cut(t)
|
2017-08-30 09:34:54 -07:00
|
|
|
|
chunkCreated = true
|
2017-01-09 07:51:39 -08:00
|
|
|
|
}
|
2017-01-11 04:02:38 -08:00
|
|
|
|
s.app.Append(t, v)
|
|
|
|
|
|
|
|
|
|
c.maxTime = t
|
|
|
|
|
|
|
|
|
|
s.sampleBuf[0] = s.sampleBuf[1]
|
|
|
|
|
s.sampleBuf[1] = s.sampleBuf[2]
|
|
|
|
|
s.sampleBuf[2] = s.sampleBuf[3]
|
|
|
|
|
s.sampleBuf[3] = sample{t: t, v: v}
|
2017-01-09 07:51:39 -08:00
|
|
|
|
|
2017-08-30 08:38:25 -07:00
|
|
|
|
return true, chunkCreated
|
2017-01-09 07:51:39 -08:00
|
|
|
|
}
|
|
|
|
|
|
2017-06-07 04:42:53 -07:00
|
|
|
|
// computeChunkEndTime estimates the end timestamp based the beginning of a chunk,
|
|
|
|
|
// its current timestamp and the upper bound up to which we insert data.
|
|
|
|
|
// It assumes that the time range is 1/4 full.
|
|
|
|
|
func computeChunkEndTime(start, cur, max int64) int64 {
|
|
|
|
|
a := (max - start) / ((cur - start + 1) * 4)
|
|
|
|
|
if a == 0 {
|
|
|
|
|
return max
|
|
|
|
|
}
|
|
|
|
|
return start + (max-start)/a
|
|
|
|
|
}
|
|
|
|
|
|
2019-07-09 02:49:34 -07:00
|
|
|
|
func (s *memSeries) iterator(id int, it chunkenc.Iterator) chunkenc.Iterator {
|
2017-08-30 09:34:54 -07:00
|
|
|
|
c := s.chunk(id)
|
2017-10-20 00:43:52 -07:00
|
|
|
|
// TODO(fabxc): Work around! A querier may have retrieved a pointer to a series' chunk,
|
|
|
|
|
// which got then garbage collected before it got accessed.
|
|
|
|
|
// We must ensure to not garbage collect as long as any readers still hold a reference.
|
|
|
|
|
if c == nil {
|
2017-11-30 06:34:49 -08:00
|
|
|
|
return chunkenc.NewNopIterator()
|
2017-10-20 00:43:52 -07:00
|
|
|
|
}
|
2017-01-11 04:02:38 -08:00
|
|
|
|
|
2017-08-30 09:34:54 -07:00
|
|
|
|
if id-s.firstChunkID < len(s.chunks)-1 {
|
2019-07-09 02:49:34 -07:00
|
|
|
|
return c.chunk.Iterator(it)
|
2017-01-11 04:02:38 -08:00
|
|
|
|
}
|
2018-02-21 07:38:59 -08:00
|
|
|
|
// Serve the last 4 samples for the last chunk from the sample buffer
|
2017-08-30 09:34:54 -07:00
|
|
|
|
// as their compressed bytes may be mutated by added samples.
|
2019-07-09 02:49:34 -07:00
|
|
|
|
if msIter, ok := it.(*memSafeIterator); ok {
|
|
|
|
|
msIter.Iterator = c.chunk.Iterator(msIter.Iterator)
|
|
|
|
|
msIter.i = -1
|
|
|
|
|
msIter.total = c.chunk.NumSamples()
|
|
|
|
|
msIter.buf = s.sampleBuf
|
|
|
|
|
return msIter
|
|
|
|
|
}
|
|
|
|
|
return &memSafeIterator{
|
|
|
|
|
Iterator: c.chunk.Iterator(it),
|
2017-01-09 07:51:39 -08:00
|
|
|
|
i: -1,
|
2017-09-01 05:38:49 -07:00
|
|
|
|
total: c.chunk.NumSamples(),
|
2017-01-11 04:02:38 -08:00
|
|
|
|
buf: s.sampleBuf,
|
2017-01-09 07:51:39 -08:00
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2017-01-11 04:02:38 -08:00
|
|
|
|
func (s *memSeries) head() *memChunk {
|
2018-10-31 06:28:56 -07:00
|
|
|
|
return s.headChunk
|
2017-01-11 04:02:38 -08:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
type memChunk struct {
|
2017-11-30 06:34:49 -08:00
|
|
|
|
chunk chunkenc.Chunk
|
2017-01-11 04:02:38 -08:00
|
|
|
|
minTime, maxTime int64
|
|
|
|
|
}
|
|
|
|
|
|
2018-07-02 01:23:36 -07:00
|
|
|
|
// Returns true if the chunk overlaps [mint, maxt].
|
|
|
|
|
func (mc *memChunk) OverlapsClosedInterval(mint, maxt int64) bool {
|
|
|
|
|
return mc.minTime <= maxt && mint <= mc.maxTime
|
|
|
|
|
}
|
|
|
|
|
|
2017-01-09 07:51:39 -08:00
|
|
|
|
type memSafeIterator struct {
|
2017-11-30 06:34:49 -08:00
|
|
|
|
chunkenc.Iterator
|
2017-01-09 07:51:39 -08:00
|
|
|
|
|
|
|
|
|
i int
|
|
|
|
|
total int
|
|
|
|
|
buf [4]sample
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
func (it *memSafeIterator) Next() bool {
|
|
|
|
|
if it.i+1 >= it.total {
|
|
|
|
|
return false
|
|
|
|
|
}
|
|
|
|
|
it.i++
|
|
|
|
|
if it.total-it.i > 4 {
|
|
|
|
|
return it.Iterator.Next()
|
|
|
|
|
}
|
|
|
|
|
return true
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
func (it *memSafeIterator) At() (int64, float64) {
|
|
|
|
|
if it.total-it.i > 4 {
|
|
|
|
|
return it.Iterator.At()
|
|
|
|
|
}
|
|
|
|
|
s := it.buf[4-(it.total-it.i)]
|
|
|
|
|
return s.t, s.v
|
|
|
|
|
}
|
2017-11-30 06:34:49 -08:00
|
|
|
|
|
|
|
|
|
type stringset map[string]struct{}
|
|
|
|
|
|
|
|
|
|
func (ss stringset) set(s string) {
|
|
|
|
|
ss[s] = struct{}{}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
func (ss stringset) String() string {
|
|
|
|
|
return strings.Join(ss.slice(), ",")
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
func (ss stringset) slice() []string {
|
|
|
|
|
slice := make([]string, 0, len(ss))
|
|
|
|
|
for k := range ss {
|
|
|
|
|
slice = append(slice, k)
|
|
|
|
|
}
|
|
|
|
|
sort.Strings(slice)
|
|
|
|
|
return slice
|
|
|
|
|
}
|