|
|
|
@ -16,19 +16,15 @@ package tsdb |
|
|
|
|
|
|
|
|
|
import ( |
|
|
|
|
"bytes" |
|
|
|
|
"encoding/binary" |
|
|
|
|
"fmt" |
|
|
|
|
"io" |
|
|
|
|
"io/ioutil" |
|
|
|
|
"math" |
|
|
|
|
"os" |
|
|
|
|
"path/filepath" |
|
|
|
|
"runtime" |
|
|
|
|
"sort" |
|
|
|
|
"strconv" |
|
|
|
|
"strings" |
|
|
|
|
"sync" |
|
|
|
|
"sync/atomic" |
|
|
|
|
"time" |
|
|
|
|
"unsafe" |
|
|
|
|
|
|
|
|
@ -99,13 +95,12 @@ type DB struct { |
|
|
|
|
dir string |
|
|
|
|
lockf *lockfile.Lockfile |
|
|
|
|
|
|
|
|
|
logger log.Logger |
|
|
|
|
metrics *dbMetrics |
|
|
|
|
opts *Options |
|
|
|
|
chunkPool chunks.Pool |
|
|
|
|
appendPool sync.Pool |
|
|
|
|
compactor Compactor |
|
|
|
|
wal WAL |
|
|
|
|
logger log.Logger |
|
|
|
|
metrics *dbMetrics |
|
|
|
|
opts *Options |
|
|
|
|
chunkPool chunks.Pool |
|
|
|
|
compactor Compactor |
|
|
|
|
wal WAL |
|
|
|
|
|
|
|
|
|
// Mutex for that must be held when modifying the general block layout.
|
|
|
|
|
mtx sync.RWMutex |
|
|
|
@ -123,33 +118,15 @@ type DB struct { |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
type dbMetrics struct { |
|
|
|
|
activeAppenders prometheus.Gauge |
|
|
|
|
loadedBlocks prometheus.GaugeFunc |
|
|
|
|
reloads prometheus.Counter |
|
|
|
|
reloadsFailed prometheus.Counter |
|
|
|
|
walTruncateDuration prometheus.Summary |
|
|
|
|
samplesAppended prometheus.Counter |
|
|
|
|
|
|
|
|
|
headSeries prometheus.Gauge |
|
|
|
|
headSeriesCreated prometheus.Counter |
|
|
|
|
headSeriesRemoved prometheus.Counter |
|
|
|
|
headChunks prometheus.Gauge |
|
|
|
|
headChunksCreated prometheus.Gauge |
|
|
|
|
headChunksRemoved prometheus.Gauge |
|
|
|
|
headGCDuration prometheus.Summary |
|
|
|
|
headMinTime prometheus.GaugeFunc |
|
|
|
|
headMaxTime prometheus.GaugeFunc |
|
|
|
|
|
|
|
|
|
loadedBlocks prometheus.GaugeFunc |
|
|
|
|
reloads prometheus.Counter |
|
|
|
|
reloadsFailed prometheus.Counter |
|
|
|
|
compactionsTriggered prometheus.Counter |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
func newDBMetrics(db *DB, r prometheus.Registerer) *dbMetrics { |
|
|
|
|
m := &dbMetrics{} |
|
|
|
|
|
|
|
|
|
m.activeAppenders = prometheus.NewGauge(prometheus.GaugeOpts{ |
|
|
|
|
Name: "tsdb_active_appenders", |
|
|
|
|
Help: "Number of currently active appender transactions", |
|
|
|
|
}) |
|
|
|
|
m.loadedBlocks = prometheus.NewGaugeFunc(prometheus.GaugeOpts{ |
|
|
|
|
Name: "tsdb_blocks_loaded", |
|
|
|
|
Help: "Number of currently loaded data blocks", |
|
|
|
@ -166,57 +143,6 @@ func newDBMetrics(db *DB, r prometheus.Registerer) *dbMetrics { |
|
|
|
|
Name: "tsdb_reloads_failures_total", |
|
|
|
|
Help: "Number of times the database failed to reload black data from disk.", |
|
|
|
|
}) |
|
|
|
|
|
|
|
|
|
m.walTruncateDuration = prometheus.NewSummary(prometheus.SummaryOpts{ |
|
|
|
|
Name: "tsdb_wal_truncate_duration_seconds", |
|
|
|
|
Help: "Duration of WAL truncation.", |
|
|
|
|
}) |
|
|
|
|
|
|
|
|
|
m.headSeries = prometheus.NewGauge(prometheus.GaugeOpts{ |
|
|
|
|
Name: "tsdb_head_series", |
|
|
|
|
Help: "Total number of series in the head block.", |
|
|
|
|
}) |
|
|
|
|
m.headSeriesCreated = prometheus.NewGauge(prometheus.GaugeOpts{ |
|
|
|
|
Name: "tsdb_head_series_created_total", |
|
|
|
|
Help: "Total number of series created in the head", |
|
|
|
|
}) |
|
|
|
|
m.headSeriesRemoved = prometheus.NewGauge(prometheus.GaugeOpts{ |
|
|
|
|
Name: "tsdb_head_series_removed_total", |
|
|
|
|
Help: "Total number of series removed in the head", |
|
|
|
|
}) |
|
|
|
|
m.headChunks = prometheus.NewGauge(prometheus.GaugeOpts{ |
|
|
|
|
Name: "tsdb_head_chunks", |
|
|
|
|
Help: "Total number of chunks in the head block.", |
|
|
|
|
}) |
|
|
|
|
m.headChunksCreated = prometheus.NewGauge(prometheus.GaugeOpts{ |
|
|
|
|
Name: "tsdb_head_chunks_created_total", |
|
|
|
|
Help: "Total number of chunks created in the head", |
|
|
|
|
}) |
|
|
|
|
m.headChunksRemoved = prometheus.NewGauge(prometheus.GaugeOpts{ |
|
|
|
|
Name: "tsdb_head_chunks_removed_total", |
|
|
|
|
Help: "Total number of chunks removed in the head", |
|
|
|
|
}) |
|
|
|
|
m.headGCDuration = prometheus.NewSummary(prometheus.SummaryOpts{ |
|
|
|
|
Name: "tsdb_head_gc_duration_seconds", |
|
|
|
|
Help: "Runtime of garbage collection in the head block.", |
|
|
|
|
}) |
|
|
|
|
m.headMinTime = prometheus.NewGaugeFunc(prometheus.GaugeOpts{ |
|
|
|
|
Name: "tsdb_head_max_time", |
|
|
|
|
Help: "Maximum timestamp of the head block.", |
|
|
|
|
}, func() float64 { |
|
|
|
|
return float64(db.head.MaxTime()) |
|
|
|
|
}) |
|
|
|
|
m.headMaxTime = prometheus.NewGaugeFunc(prometheus.GaugeOpts{ |
|
|
|
|
Name: "tsdb_head_min_time", |
|
|
|
|
Help: "Minimum time bound of the head block.", |
|
|
|
|
}, func() float64 { |
|
|
|
|
return float64(db.head.MinTime()) |
|
|
|
|
}) |
|
|
|
|
|
|
|
|
|
m.samplesAppended = prometheus.NewCounter(prometheus.CounterOpts{ |
|
|
|
|
Name: "tsdb_samples_appended_total", |
|
|
|
|
Help: "Total number of appended sampledb.", |
|
|
|
|
}) |
|
|
|
|
m.compactionsTriggered = prometheus.NewCounter(prometheus.CounterOpts{ |
|
|
|
|
Name: "tsdb_compactions_triggered_total", |
|
|
|
|
Help: "Total number of triggered compactions for the partition.", |
|
|
|
@ -224,23 +150,9 @@ func newDBMetrics(db *DB, r prometheus.Registerer) *dbMetrics { |
|
|
|
|
|
|
|
|
|
if r != nil { |
|
|
|
|
r.MustRegister( |
|
|
|
|
m.activeAppenders, |
|
|
|
|
m.loadedBlocks, |
|
|
|
|
m.reloads, |
|
|
|
|
m.reloadsFailed, |
|
|
|
|
m.walTruncateDuration, |
|
|
|
|
|
|
|
|
|
m.headChunks, |
|
|
|
|
m.headChunksCreated, |
|
|
|
|
m.headChunksRemoved, |
|
|
|
|
m.headSeries, |
|
|
|
|
m.headSeriesCreated, |
|
|
|
|
m.headSeriesRemoved, |
|
|
|
|
m.headMinTime, |
|
|
|
|
m.headMaxTime, |
|
|
|
|
m.headGCDuration, |
|
|
|
|
|
|
|
|
|
m.samplesAppended, |
|
|
|
|
m.compactionsTriggered, |
|
|
|
|
) |
|
|
|
|
} |
|
|
|
@ -260,16 +172,10 @@ func Open(dir string, l log.Logger, r prometheus.Registerer, opts *Options) (db |
|
|
|
|
opts = DefaultOptions |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
wal, err := OpenSegmentWAL(filepath.Join(dir, "wal"), l, 10*time.Second) |
|
|
|
|
if err != nil { |
|
|
|
|
return nil, err |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
db = &DB{ |
|
|
|
|
dir: dir, |
|
|
|
|
logger: l, |
|
|
|
|
opts: opts, |
|
|
|
|
wal: wal, |
|
|
|
|
compactc: make(chan struct{}, 1), |
|
|
|
|
donec: make(chan struct{}), |
|
|
|
|
stopc: make(chan struct{}), |
|
|
|
@ -312,14 +218,15 @@ func Open(dir string, l log.Logger, r prometheus.Registerer, opts *Options) (db |
|
|
|
|
|
|
|
|
|
db.compactor = NewLeveledCompactor(r, l, copts) |
|
|
|
|
|
|
|
|
|
db.head, err = NewHead(l, copts.blockRanges[0]) |
|
|
|
|
wal, err := OpenSegmentWAL(filepath.Join(dir, "wal"), l, 10*time.Second) |
|
|
|
|
if err != nil { |
|
|
|
|
return nil, err |
|
|
|
|
} |
|
|
|
|
if err := db.readWAL(db.wal.Reader()); err != nil { |
|
|
|
|
db.head, err = NewHead(r, l, wal, copts.blockRanges[0]) |
|
|
|
|
if err != nil { |
|
|
|
|
return nil, err |
|
|
|
|
} |
|
|
|
|
if err := db.reloadBlocks(); err != nil { |
|
|
|
|
if err := db.reload(); err != nil { |
|
|
|
|
return nil, err |
|
|
|
|
} |
|
|
|
|
|
|
|
|
@ -341,6 +248,7 @@ func (db *DB) run() { |
|
|
|
|
for { |
|
|
|
|
select { |
|
|
|
|
case <-db.stopc: |
|
|
|
|
return |
|
|
|
|
case <-time.After(backoff): |
|
|
|
|
} |
|
|
|
|
|
|
|
|
@ -364,7 +272,9 @@ func (db *DB) run() { |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
if err1 != nil || err2 != nil { |
|
|
|
|
exponential(backoff, 1*time.Second, 1*time.Minute) |
|
|
|
|
backoff = exponential(backoff, 1*time.Second, 1*time.Minute) |
|
|
|
|
} else { |
|
|
|
|
backoff = 0 |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
case <-db.stopc: |
|
|
|
@ -391,6 +301,30 @@ func (db *DB) retentionCutoff() (bool, error) { |
|
|
|
|
return retentionCutoff(db.dir, mint) |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
// Appender opens a new appender against the database.
|
|
|
|
|
func (db *DB) Appender() Appender { |
|
|
|
|
return dbAppender{db: db, Appender: db.head.Appender()} |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
// dbAppender wraps the DB's head appender and triggers compactions on commit
|
|
|
|
|
// if necessary.
|
|
|
|
|
type dbAppender struct { |
|
|
|
|
Appender |
|
|
|
|
db *DB |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
func (a dbAppender) Commit() error { |
|
|
|
|
err := a.Appender.Commit() |
|
|
|
|
|
|
|
|
|
if a.db.head.MaxTime()-a.db.head.MinTime() > a.db.head.chunkRange/2*3 { |
|
|
|
|
select { |
|
|
|
|
case a.db.compactc <- struct{}{}: |
|
|
|
|
default: |
|
|
|
|
} |
|
|
|
|
} |
|
|
|
|
return err |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
func (db *DB) compact() (changes bool, err error) { |
|
|
|
|
db.cmtx.Lock() |
|
|
|
|
defer db.cmtx.Unlock() |
|
|
|
@ -425,7 +359,7 @@ func (db *DB) compact() (changes bool, err error) { |
|
|
|
|
} |
|
|
|
|
changes = true |
|
|
|
|
|
|
|
|
|
if err := db.reloadBlocks(); err != nil { |
|
|
|
|
if err := db.reload(); err != nil { |
|
|
|
|
return changes, errors.Wrap(err, "reload blocks") |
|
|
|
|
} |
|
|
|
|
runtime.GC() |
|
|
|
@ -458,7 +392,7 @@ func (db *DB) compact() (changes bool, err error) { |
|
|
|
|
} |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
if err := db.reloadBlocks(); err != nil { |
|
|
|
|
if err := db.reload(); err != nil { |
|
|
|
|
return changes, errors.Wrap(err, "reload blocks") |
|
|
|
|
} |
|
|
|
|
runtime.GC() |
|
|
|
@ -512,50 +446,7 @@ func (db *DB) getBlock(id ulid.ULID) (DiskBlock, bool) { |
|
|
|
|
return nil, false |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
func (db *DB) readWAL(r WALReader) error { |
|
|
|
|
|
|
|
|
|
seriesFunc := func(series []labels.Labels) error { |
|
|
|
|
for _, lset := range series { |
|
|
|
|
db.head.create(lset.Hash(), lset) |
|
|
|
|
db.metrics.headSeries.Inc() |
|
|
|
|
db.metrics.headSeriesCreated.Inc() |
|
|
|
|
} |
|
|
|
|
return nil |
|
|
|
|
} |
|
|
|
|
samplesFunc := func(samples []RefSample) error { |
|
|
|
|
for _, s := range samples { |
|
|
|
|
ms, ok := db.head.series[uint32(s.Ref)] |
|
|
|
|
if !ok { |
|
|
|
|
return errors.Errorf("unknown series reference %d; abort WAL restore", s.Ref) |
|
|
|
|
} |
|
|
|
|
_, chunkCreated := ms.append(s.T, s.V) |
|
|
|
|
if chunkCreated { |
|
|
|
|
db.metrics.headChunksCreated.Inc() |
|
|
|
|
db.metrics.headChunks.Inc() |
|
|
|
|
} |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
return nil |
|
|
|
|
} |
|
|
|
|
deletesFunc := func(stones []Stone) error { |
|
|
|
|
for _, s := range stones { |
|
|
|
|
for _, itv := range s.intervals { |
|
|
|
|
db.head.tombstones.add(s.ref, itv) |
|
|
|
|
} |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
return nil |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
if err := r.Read(seriesFunc, samplesFunc, deletesFunc); err != nil { |
|
|
|
|
return errors.Wrap(err, "consume WAL") |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
return nil |
|
|
|
|
|
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
func (db *DB) reloadBlocks() (err error) { |
|
|
|
|
func (db *DB) reload() (err error) { |
|
|
|
|
defer func() { |
|
|
|
|
if err != nil { |
|
|
|
|
db.metrics.reloadsFailed.Inc() |
|
|
|
@ -613,29 +504,11 @@ func (db *DB) reloadBlocks() (err error) { |
|
|
|
|
// Garbage collect data in the head if the most recent persisted block
|
|
|
|
|
// covers data of its current time range.
|
|
|
|
|
if len(blocks) == 0 { |
|
|
|
|
return |
|
|
|
|
} |
|
|
|
|
maxt := blocks[len(db.blocks)-1].Meta().MaxTime |
|
|
|
|
if maxt <= db.head.MinTime() { |
|
|
|
|
return |
|
|
|
|
return nil |
|
|
|
|
} |
|
|
|
|
start := time.Now() |
|
|
|
|
atomic.StoreInt64(&db.head.minTime, maxt) |
|
|
|
|
|
|
|
|
|
series, chunks := db.head.gc() |
|
|
|
|
db.metrics.headSeriesRemoved.Add(float64(series)) |
|
|
|
|
db.metrics.headSeries.Sub(float64(series)) |
|
|
|
|
db.metrics.headChunksRemoved.Add(float64(chunks)) |
|
|
|
|
db.metrics.headChunks.Sub(float64(chunks)) |
|
|
|
|
|
|
|
|
|
db.logger.Log("msg", "head GC completed", "duration", time.Since(start)) |
|
|
|
|
|
|
|
|
|
start = time.Now() |
|
|
|
|
|
|
|
|
|
if err := db.wal.Truncate(maxt); err != nil { |
|
|
|
|
return errors.Wrapf(err, "truncate WAL at %d", maxt) |
|
|
|
|
} |
|
|
|
|
db.metrics.walTruncateDuration.Observe(time.Since(start).Seconds()) |
|
|
|
|
maxt := blocks[len(db.blocks)-1].Meta().MaxTime |
|
|
|
|
db.head.Truncate(maxt) |
|
|
|
|
|
|
|
|
|
return nil |
|
|
|
|
} |
|
|
|
@ -701,24 +574,28 @@ func (db *DB) EnableCompactions() { |
|
|
|
|
|
|
|
|
|
// Snapshot writes the current data to the directory.
|
|
|
|
|
func (db *DB) Snapshot(dir string) error { |
|
|
|
|
// if dir == db.dir {
|
|
|
|
|
// return errors.Errorf("cannot snapshot into base directory")
|
|
|
|
|
// }
|
|
|
|
|
// db.cmtx.Lock()
|
|
|
|
|
// defer db.cmtx.Unlock()
|
|
|
|
|
|
|
|
|
|
// db.mtx.Lock() // To block any appenders.
|
|
|
|
|
// defer db.mtx.Unlock()
|
|
|
|
|
|
|
|
|
|
// blocks := db.blocks[:]
|
|
|
|
|
// for _, b := range blocks {
|
|
|
|
|
// db.logger.Log("msg", "snapshotting block", "block", b)
|
|
|
|
|
// if err := b.Snapshot(dir); err != nil {
|
|
|
|
|
// return errors.Wrap(err, "error snapshotting headblock")
|
|
|
|
|
// }
|
|
|
|
|
// }
|
|
|
|
|
if dir == db.dir { |
|
|
|
|
return errors.Errorf("cannot snapshot into base directory") |
|
|
|
|
} |
|
|
|
|
if _, err := ulid.Parse(dir); err == nil { |
|
|
|
|
return errors.Errorf("dir must not be a valid ULID") |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
return nil |
|
|
|
|
db.cmtx.Lock() |
|
|
|
|
defer db.cmtx.Unlock() |
|
|
|
|
|
|
|
|
|
db.mtx.RLock() |
|
|
|
|
defer db.mtx.RUnlock() |
|
|
|
|
|
|
|
|
|
for _, b := range db.blocks { |
|
|
|
|
db.logger.Log("msg", "snapshotting block", "block", b) |
|
|
|
|
|
|
|
|
|
if err := b.Snapshot(dir); err != nil { |
|
|
|
|
return errors.Wrap(err, "error snapshotting headblock") |
|
|
|
|
} |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
return db.compactor.Write(dir, db.head, db.head.MinTime(), db.head.MaxTime()) |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
// Querier returns a new querier over the data partition for the given time range.
|
|
|
|
@ -741,320 +618,9 @@ func (db *DB) Querier(mint, maxt int64) Querier { |
|
|
|
|
tombstones: b.Tombstones(), |
|
|
|
|
}) |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
return sq |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
// initAppender is a helper to initialize the time bounds of a the head
|
|
|
|
|
// upon the first sample it receives.
|
|
|
|
|
type initAppender struct { |
|
|
|
|
app Appender |
|
|
|
|
db *DB |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
func (a *initAppender) Add(lset labels.Labels, t int64, v float64) (string, error) { |
|
|
|
|
if a.app != nil { |
|
|
|
|
return a.app.Add(lset, t, v) |
|
|
|
|
} |
|
|
|
|
for { |
|
|
|
|
// In the init state, the head has a high timestamp of math.MinInt64.
|
|
|
|
|
ht := a.db.head.MaxTime() |
|
|
|
|
if ht != math.MinInt64 { |
|
|
|
|
break |
|
|
|
|
} |
|
|
|
|
cr := a.db.opts.BlockRanges[0] |
|
|
|
|
mint, _ := rangeForTimestamp(t, cr) |
|
|
|
|
|
|
|
|
|
atomic.CompareAndSwapInt64(&a.db.head.maxTime, ht, t) |
|
|
|
|
atomic.StoreInt64(&a.db.head.minTime, mint-cr) |
|
|
|
|
} |
|
|
|
|
a.app = a.db.appender() |
|
|
|
|
|
|
|
|
|
return a.app.Add(lset, t, v) |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
func (a *initAppender) AddFast(ref string, t int64, v float64) error { |
|
|
|
|
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 (db *DB) Appender() Appender { |
|
|
|
|
db.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.
|
|
|
|
|
if db.head.MaxTime() == math.MinInt64 { |
|
|
|
|
return &initAppender{db: db} |
|
|
|
|
} |
|
|
|
|
return db.appender() |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
func (db *DB) appender() *dbAppender { |
|
|
|
|
db.head.mtx.RLock() |
|
|
|
|
|
|
|
|
|
return &dbAppender{ |
|
|
|
|
db: db, |
|
|
|
|
head: db.head, |
|
|
|
|
wal: db.wal, |
|
|
|
|
mint: db.head.MaxTime() - db.opts.BlockRanges[0]/2, |
|
|
|
|
samples: db.getAppendBuffer(), |
|
|
|
|
highTimestamp: math.MinInt64, |
|
|
|
|
lowTimestamp: math.MaxInt64, |
|
|
|
|
} |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
func (db *DB) getAppendBuffer() []RefSample { |
|
|
|
|
b := db.appendPool.Get() |
|
|
|
|
if b == nil { |
|
|
|
|
return make([]RefSample, 0, 512) |
|
|
|
|
} |
|
|
|
|
return b.([]RefSample) |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
func (db *DB) putAppendBuffer(b []RefSample) { |
|
|
|
|
db.appendPool.Put(b[:0]) |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
type dbAppender struct { |
|
|
|
|
db *DB |
|
|
|
|
head *Head |
|
|
|
|
wal WAL |
|
|
|
|
mint int64 |
|
|
|
|
|
|
|
|
|
newSeries []*hashedLabels |
|
|
|
|
newLabels []labels.Labels |
|
|
|
|
newHashes map[uint64]uint64 |
|
|
|
|
|
|
|
|
|
samples []RefSample |
|
|
|
|
highTimestamp int64 |
|
|
|
|
lowTimestamp int64 |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
type hashedLabels struct { |
|
|
|
|
ref uint64 |
|
|
|
|
hash uint64 |
|
|
|
|
labels labels.Labels |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
func (a *dbAppender) Add(lset labels.Labels, t int64, v float64) (string, error) { |
|
|
|
|
if t < a.mint { |
|
|
|
|
return "", ErrOutOfBounds |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
hash := lset.Hash() |
|
|
|
|
refb := make([]byte, 8) |
|
|
|
|
|
|
|
|
|
// Series exists already in the block.
|
|
|
|
|
if ms := a.head.get(hash, lset); ms != nil { |
|
|
|
|
binary.BigEndian.PutUint64(refb, uint64(ms.ref)) |
|
|
|
|
return string(refb), a.AddFast(string(refb), t, v) |
|
|
|
|
} |
|
|
|
|
// Series was added in this transaction previously.
|
|
|
|
|
if ref, ok := a.newHashes[hash]; ok { |
|
|
|
|
binary.BigEndian.PutUint64(refb, ref) |
|
|
|
|
// XXX(fabxc): there's no fast path for multiple samples for the same new series
|
|
|
|
|
// in the same transaction. We always return the invalid empty ref. It's has not
|
|
|
|
|
// been a relevant use case so far and is not worth the trouble.
|
|
|
|
|
return "", a.AddFast(string(refb), t, v) |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
// The series is completely new.
|
|
|
|
|
if a.newSeries == nil { |
|
|
|
|
a.newHashes = map[uint64]uint64{} |
|
|
|
|
} |
|
|
|
|
// First sample for new series.
|
|
|
|
|
ref := uint64(len(a.newSeries)) |
|
|
|
|
|
|
|
|
|
a.newSeries = append(a.newSeries, &hashedLabels{ |
|
|
|
|
ref: ref, |
|
|
|
|
hash: hash, |
|
|
|
|
labels: lset, |
|
|
|
|
}) |
|
|
|
|
// First bit indicates its a series created in this transaction.
|
|
|
|
|
ref |= (1 << 63) |
|
|
|
|
|
|
|
|
|
a.newHashes[hash] = ref |
|
|
|
|
binary.BigEndian.PutUint64(refb, ref) |
|
|
|
|
|
|
|
|
|
return "", a.AddFast(string(refb), t, v) |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
func (a *dbAppender) AddFast(ref string, t int64, v float64) error { |
|
|
|
|
if len(ref) != 8 { |
|
|
|
|
return errors.Wrap(ErrNotFound, "invalid ref length") |
|
|
|
|
} |
|
|
|
|
var ( |
|
|
|
|
refn = binary.BigEndian.Uint64(yoloBytes(ref)) |
|
|
|
|
id = uint32(refn) |
|
|
|
|
inTx = refn&(1<<63) != 0 |
|
|
|
|
) |
|
|
|
|
// Distinguish between existing series and series created in
|
|
|
|
|
// this transaction.
|
|
|
|
|
if inTx { |
|
|
|
|
if id > uint32(len(a.newSeries)-1) { |
|
|
|
|
return errors.Wrap(ErrNotFound, "transaction series ID too high") |
|
|
|
|
} |
|
|
|
|
// TODO(fabxc): we also have to validate here that the
|
|
|
|
|
// sample sequence is valid.
|
|
|
|
|
// We also have to revalidate it as we switch locks and create
|
|
|
|
|
// the new series.
|
|
|
|
|
} else { |
|
|
|
|
ms, ok := a.head.series[id] |
|
|
|
|
if !ok { |
|
|
|
|
return errors.Wrap(ErrNotFound, "unknown series") |
|
|
|
|
} |
|
|
|
|
if err := ms.appendable(t, v); err != nil { |
|
|
|
|
return err |
|
|
|
|
} |
|
|
|
|
} |
|
|
|
|
if t < a.mint { |
|
|
|
|
return ErrOutOfBounds |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
if t > a.highTimestamp { |
|
|
|
|
a.highTimestamp = t |
|
|
|
|
} |
|
|
|
|
// if t < a.lowTimestamp {
|
|
|
|
|
// a.lowTimestamp = t
|
|
|
|
|
// }
|
|
|
|
|
|
|
|
|
|
a.samples = append(a.samples, RefSample{ |
|
|
|
|
Ref: refn, |
|
|
|
|
T: t, |
|
|
|
|
V: v, |
|
|
|
|
}) |
|
|
|
|
return nil |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
func (a *dbAppender) createSeries() error { |
|
|
|
|
if len(a.newSeries) == 0 { |
|
|
|
|
return nil |
|
|
|
|
} |
|
|
|
|
a.newLabels = make([]labels.Labels, 0, len(a.newSeries)) |
|
|
|
|
base0 := len(a.head.series) |
|
|
|
|
|
|
|
|
|
a.head.mtx.RUnlock() |
|
|
|
|
defer a.head.mtx.RLock() |
|
|
|
|
a.head.mtx.Lock() |
|
|
|
|
defer a.head.mtx.Unlock() |
|
|
|
|
|
|
|
|
|
base1 := len(a.head.series) |
|
|
|
|
|
|
|
|
|
for _, l := range a.newSeries { |
|
|
|
|
// We switched locks and have to re-validate that the series were not
|
|
|
|
|
// created by another goroutine in the meantime.
|
|
|
|
|
if base1 > base0 { |
|
|
|
|
if ms := a.head.get(l.hash, l.labels); ms != nil { |
|
|
|
|
l.ref = uint64(ms.ref) |
|
|
|
|
continue |
|
|
|
|
} |
|
|
|
|
} |
|
|
|
|
// Series is still new.
|
|
|
|
|
a.newLabels = append(a.newLabels, l.labels) |
|
|
|
|
|
|
|
|
|
s := a.head.create(l.hash, l.labels) |
|
|
|
|
l.ref = uint64(s.ref) |
|
|
|
|
|
|
|
|
|
a.db.metrics.headSeriesCreated.Inc() |
|
|
|
|
a.db.metrics.headSeries.Inc() |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
// Write all new series to the WAL.
|
|
|
|
|
if err := a.wal.LogSeries(a.newLabels); err != nil { |
|
|
|
|
return errors.Wrap(err, "WAL log series") |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
return nil |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
func (a *dbAppender) Commit() error { |
|
|
|
|
defer a.head.mtx.RUnlock() |
|
|
|
|
|
|
|
|
|
defer a.db.metrics.activeAppenders.Dec() |
|
|
|
|
defer a.db.putAppendBuffer(a.samples) |
|
|
|
|
|
|
|
|
|
if err := a.createSeries(); err != nil { |
|
|
|
|
return err |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
// We have to update the refs of samples for series we just created.
|
|
|
|
|
for i := range a.samples { |
|
|
|
|
s := &a.samples[i] |
|
|
|
|
if s.Ref&(1<<63) != 0 { |
|
|
|
|
s.Ref = a.newSeries[(s.Ref<<1)>>1].ref |
|
|
|
|
} |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
// Write all new samples to the WAL and add them to the
|
|
|
|
|
// in-mem database on success.
|
|
|
|
|
if err := a.wal.LogSamples(a.samples); err != nil { |
|
|
|
|
return errors.Wrap(err, "WAL log samples") |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
total := uint64(len(a.samples)) |
|
|
|
|
|
|
|
|
|
for _, s := range a.samples { |
|
|
|
|
series, ok := a.head.series[uint32(s.Ref)] |
|
|
|
|
if !ok { |
|
|
|
|
return errors.Errorf("series with ID %d not found", s.Ref) |
|
|
|
|
} |
|
|
|
|
ok, chunkCreated := series.append(s.T, s.V) |
|
|
|
|
if !ok { |
|
|
|
|
total-- |
|
|
|
|
} |
|
|
|
|
if chunkCreated { |
|
|
|
|
a.db.metrics.headChunks.Inc() |
|
|
|
|
a.db.metrics.headChunksCreated.Inc() |
|
|
|
|
} |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
a.db.metrics.samplesAppended.Add(float64(total)) |
|
|
|
|
|
|
|
|
|
for { |
|
|
|
|
ht := a.head.MaxTime() |
|
|
|
|
if a.highTimestamp <= ht { |
|
|
|
|
break |
|
|
|
|
} |
|
|
|
|
if a.highTimestamp-a.head.MinTime() > a.head.chunkRange/2*3 { |
|
|
|
|
select { |
|
|
|
|
case a.db.compactc <- struct{}{}: |
|
|
|
|
default: |
|
|
|
|
} |
|
|
|
|
} |
|
|
|
|
if atomic.CompareAndSwapInt64(&a.head.maxTime, ht, a.highTimestamp) { |
|
|
|
|
break |
|
|
|
|
} |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
return nil |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
func (a *dbAppender) Rollback() error { |
|
|
|
|
a.head.mtx.RUnlock() |
|
|
|
|
|
|
|
|
|
a.db.metrics.activeAppenders.Dec() |
|
|
|
|
a.db.putAppendBuffer(a.samples) |
|
|
|
|
|
|
|
|
|
return nil |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
func rangeForTimestamp(t int64, width int64) (mint, maxt int64) { |
|
|
|
|
mint = (t / width) * width |
|
|
|
|
return mint, mint + width |
|
|
|
@ -1078,41 +644,10 @@ func (db *DB) Delete(mint, maxt int64, ms ...labels.Matcher) error { |
|
|
|
|
}(b)) |
|
|
|
|
} |
|
|
|
|
} |
|
|
|
|
if err := g.Wait(); err != nil { |
|
|
|
|
return err |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
ir := db.head.Index() |
|
|
|
|
|
|
|
|
|
pr := newPostingsReader(ir) |
|
|
|
|
p, absent := pr.Select(ms...) |
|
|
|
|
|
|
|
|
|
var stones []Stone |
|
|
|
|
|
|
|
|
|
Outer: |
|
|
|
|
for p.Next() { |
|
|
|
|
series := db.head.series[p.At()] |
|
|
|
|
|
|
|
|
|
for _, abs := range absent { |
|
|
|
|
if series.lset.Get(abs) != "" { |
|
|
|
|
continue Outer |
|
|
|
|
} |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
// Delete only until the current values and not beyond.
|
|
|
|
|
t0, t1 := clampInterval(mint, maxt, series.minTime(), series.maxTime()) |
|
|
|
|
stones = append(stones, Stone{p.At(), Intervals{{t0, t1}}}) |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
if p.Err() != nil { |
|
|
|
|
return p.Err() |
|
|
|
|
} |
|
|
|
|
if err := db.wal.LogDeletes(stones); err != nil { |
|
|
|
|
return err |
|
|
|
|
} |
|
|
|
|
for _, s := range stones { |
|
|
|
|
db.head.tombstones.add(s.ref, s.intervals[0]) |
|
|
|
|
} |
|
|
|
|
g.Go(func() error { |
|
|
|
|
return db.head.Delete(mint, maxt, ms...) |
|
|
|
|
}) |
|
|
|
|
|
|
|
|
|
if err := g.Wait(); err != nil { |
|
|
|
|
return err |
|
|
|
@ -1171,7 +706,7 @@ func blockDirs(dir string) ([]string, error) { |
|
|
|
|
return dirs, nil |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
func sequenceFiles(dir, prefix string) ([]string, error) { |
|
|
|
|
func sequenceFiles(dir string) ([]string, error) { |
|
|
|
|
files, err := ioutil.ReadDir(dir) |
|
|
|
|
if err != nil { |
|
|
|
|
return nil, err |
|
|
|
@ -1179,24 +714,15 @@ func sequenceFiles(dir, prefix string) ([]string, error) { |
|
|
|
|
var res []string |
|
|
|
|
|
|
|
|
|
for _, fi := range files { |
|
|
|
|
if isSequenceFile(fi, prefix) { |
|
|
|
|
res = append(res, filepath.Join(dir, fi.Name())) |
|
|
|
|
if _, err := strconv.ParseUint(fi.Name(), 10, 64); err != nil { |
|
|
|
|
continue |
|
|
|
|
} |
|
|
|
|
res = append(res, filepath.Join(dir, fi.Name())) |
|
|
|
|
} |
|
|
|
|
return res, nil |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
func isSequenceFile(fi os.FileInfo, prefix string) bool { |
|
|
|
|
if !strings.HasPrefix(fi.Name(), prefix) { |
|
|
|
|
return false |
|
|
|
|
} |
|
|
|
|
if _, err := strconv.ParseUint(fi.Name()[len(prefix):], 10, 32); err != nil { |
|
|
|
|
return false |
|
|
|
|
} |
|
|
|
|
return true |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
func nextSequenceFile(dir, prefix string) (string, int, error) { |
|
|
|
|
func nextSequenceFile(dir string) (string, int, error) { |
|
|
|
|
names, err := fileutil.ReadDir(dir) |
|
|
|
|
if err != nil { |
|
|
|
|
return "", 0, err |
|
|
|
@ -1204,16 +730,13 @@ func nextSequenceFile(dir, prefix string) (string, int, error) { |
|
|
|
|
|
|
|
|
|
i := uint64(0) |
|
|
|
|
for _, n := range names { |
|
|
|
|
if !strings.HasPrefix(n, prefix) { |
|
|
|
|
continue |
|
|
|
|
} |
|
|
|
|
j, err := strconv.ParseUint(n[len(prefix):], 10, 32) |
|
|
|
|
j, err := strconv.ParseUint(n, 10, 64) |
|
|
|
|
if err != nil { |
|
|
|
|
continue |
|
|
|
|
} |
|
|
|
|
i = j |
|
|
|
|
} |
|
|
|
|
return filepath.Join(dir, fmt.Sprintf("%s%0.6d", prefix, i+1)), int(i + 1), nil |
|
|
|
|
return filepath.Join(dir, fmt.Sprintf("%0.6d", i+1)), int(i + 1), nil |
|
|
|
|
} |
|
|
|
|
|
|
|
|
|
// The MultiError type implements the error interface, and contains the
|
|
|
|
|