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-11-15 01:34:25 -08:00
|
|
|
// Package tsdb implements a time series storage for float64 sample data.
|
|
|
|
package tsdb
|
|
|
|
|
|
|
|
import (
|
2016-12-10 09:08:50 -08:00
|
|
|
"bytes"
|
2017-08-28 15:39:17 -07:00
|
|
|
"encoding/binary"
|
2016-12-04 04:16:11 -08:00
|
|
|
"fmt"
|
2017-02-27 01:46:15 -08:00
|
|
|
"io"
|
2017-01-06 00:26:39 -08:00
|
|
|
"io/ioutil"
|
2017-08-28 15:39:17 -07:00
|
|
|
"math"
|
2016-12-04 04:16:11 -08:00
|
|
|
"os"
|
2016-12-08 08:43:10 -08:00
|
|
|
"path/filepath"
|
2017-03-21 04:21:02 -07:00
|
|
|
"runtime"
|
2017-05-18 07:09:30 -07:00
|
|
|
"sort"
|
2016-12-14 23:31:26 -08:00
|
|
|
"strconv"
|
2017-01-06 04:13:22 -08:00
|
|
|
"strings"
|
2016-12-08 08:43:10 -08:00
|
|
|
"sync"
|
2017-08-28 15:39:17 -07:00
|
|
|
"sync/atomic"
|
2017-01-06 06:18:06 -08:00
|
|
|
"time"
|
2016-12-15 02:56:41 -08:00
|
|
|
"unsafe"
|
2016-11-15 01:34:25 -08:00
|
|
|
|
2016-12-14 23:31:26 -08:00
|
|
|
"golang.org/x/sync/errgroup"
|
|
|
|
|
2017-01-03 06:43:26 -08:00
|
|
|
"github.com/coreos/etcd/pkg/fileutil"
|
2016-12-14 09:38:46 -08:00
|
|
|
"github.com/go-kit/kit/log"
|
2017-02-19 04:01:19 -08:00
|
|
|
"github.com/nightlyone/lockfile"
|
2017-05-18 07:09:30 -07:00
|
|
|
"github.com/oklog/ulid"
|
2017-01-03 06:43:26 -08:00
|
|
|
"github.com/pkg/errors"
|
2016-12-31 00:48:49 -08:00
|
|
|
"github.com/prometheus/client_golang/prometheus"
|
2017-08-08 08:35:34 -07:00
|
|
|
"github.com/prometheus/tsdb/chunks"
|
2017-04-04 02:27:26 -07:00
|
|
|
"github.com/prometheus/tsdb/labels"
|
2016-11-15 01:34:25 -08:00
|
|
|
)
|
|
|
|
|
2016-12-09 01:00:14 -08:00
|
|
|
// DefaultOptions used for the DB. They are sane for setups using
|
2017-01-06 02:40:09 -08:00
|
|
|
// millisecond precision timestampdb.
|
2016-11-15 01:34:25 -08:00
|
|
|
var DefaultOptions = &Options{
|
2017-02-09 17:54:26 -08:00
|
|
|
WALFlushInterval: 5 * time.Second,
|
|
|
|
RetentionDuration: 15 * 24 * 60 * 60 * 1000, // 15 days in milliseconds
|
2017-07-18 02:17:17 -07:00
|
|
|
BlockRanges: ExponentialBlockRanges(int64(2*time.Hour)/1e6, 3, 5),
|
2017-05-09 03:52:47 -07:00
|
|
|
NoLockfile: false,
|
2016-11-15 01:34:25 -08:00
|
|
|
}
|
|
|
|
|
|
|
|
// Options of the DB storage.
|
|
|
|
type Options struct {
|
2017-01-28 23:11:47 -08:00
|
|
|
// The interval at which the write ahead log is flushed to disc.
|
2017-01-06 06:18:06 -08:00
|
|
|
WALFlushInterval time.Duration
|
2017-01-28 23:11:47 -08:00
|
|
|
|
2017-02-09 17:54:26 -08:00
|
|
|
// Duration of persisted data to keep.
|
|
|
|
RetentionDuration uint64
|
|
|
|
|
2017-07-07 04:46:41 -07:00
|
|
|
// The sizes of the Blocks.
|
|
|
|
BlockRanges []int64
|
2017-01-28 23:11:47 -08:00
|
|
|
|
2017-05-09 03:52:47 -07:00
|
|
|
// NoLockfile disables creation and consideration of a lock file.
|
|
|
|
NoLockfile bool
|
2016-11-15 01:34:25 -08:00
|
|
|
}
|
|
|
|
|
2017-01-12 11:17:49 -08:00
|
|
|
// Appender allows appending a batch of data. It must be completed with a
|
|
|
|
// call to Commit or Rollback and must not be reused afterwards.
|
2017-04-14 13:37:28 -07:00
|
|
|
//
|
|
|
|
// Operations on the Appender interface are not goroutine-safe.
|
2016-12-10 09:08:50 -08:00
|
|
|
type Appender interface {
|
2017-02-01 06:29:48 -08:00
|
|
|
// Add adds a sample pair for the given series. A reference number is
|
|
|
|
// returned which can be used to add further samples in the same or later
|
|
|
|
// transactions.
|
|
|
|
// Returned reference numbers are ephemeral and may be rejected in calls
|
|
|
|
// to AddFast() at any point. Adding the sample via Add() returns a new
|
|
|
|
// reference number.
|
2017-06-26 05:58:00 -07:00
|
|
|
// If the reference is the empty string it must not be used for caching.
|
2017-05-17 07:43:01 -07:00
|
|
|
Add(l labels.Labels, t int64, v float64) (string, error)
|
2017-02-01 06:29:48 -08:00
|
|
|
|
|
|
|
// Add adds a sample pair for the referenced series. It is generally faster
|
|
|
|
// than adding a sample by providing its full label set.
|
2017-05-17 07:43:01 -07:00
|
|
|
AddFast(ref string, t int64, v float64) error
|
2016-12-20 15:02:37 -08:00
|
|
|
|
|
|
|
// Commit submits the collected samples and purges the batch.
|
2016-12-10 09:08:50 -08:00
|
|
|
Commit() error
|
2017-01-12 11:17:49 -08:00
|
|
|
|
|
|
|
// Rollback rolls back all modifications made in the appender so far.
|
|
|
|
Rollback() error
|
2016-12-10 09:08:50 -08:00
|
|
|
}
|
|
|
|
|
2017-01-06 02:40:09 -08:00
|
|
|
// DB handles reads and writes of time series falling into
|
|
|
|
// a hashed partition of a seriedb.
|
|
|
|
type DB struct {
|
2017-03-04 07:50:48 -08:00
|
|
|
dir string
|
2017-05-09 03:52:47 -07:00
|
|
|
lockf *lockfile.Lockfile
|
2017-03-04 07:50:48 -08:00
|
|
|
|
2017-08-28 15:39:17 -07:00
|
|
|
logger log.Logger
|
|
|
|
metrics *dbMetrics
|
|
|
|
opts *Options
|
|
|
|
chunkPool chunks.Pool
|
|
|
|
appendPool sync.Pool
|
|
|
|
compactor Compactor
|
|
|
|
wal WAL
|
2016-12-09 01:00:14 -08:00
|
|
|
|
2017-07-14 00:00:22 -07:00
|
|
|
// Mutex for that must be held when modifying the general block layout.
|
2017-03-20 00:41:56 -07:00
|
|
|
mtx sync.RWMutex
|
2017-08-28 15:39:17 -07:00
|
|
|
blocks []DiskBlock
|
2017-03-04 07:50:48 -08:00
|
|
|
|
2017-08-28 15:39:17 -07:00
|
|
|
head *Head
|
2017-01-06 03:37:28 -08:00
|
|
|
|
|
|
|
compactc chan struct{}
|
|
|
|
donec chan struct{}
|
|
|
|
stopc chan struct{}
|
2017-05-20 00:51:10 -07:00
|
|
|
|
2017-05-23 05:43:30 -07:00
|
|
|
// cmtx is used to control compactions and deletions.
|
2017-07-14 01:06:07 -07:00
|
|
|
cmtx sync.Mutex
|
|
|
|
compactionsEnabled bool
|
2016-12-09 01:00:14 -08:00
|
|
|
}
|
|
|
|
|
2017-01-06 02:40:09 -08:00
|
|
|
type dbMetrics struct {
|
2017-08-30 08:38:25 -07:00
|
|
|
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
|
|
|
|
|
2017-01-06 03:37:28 -08:00
|
|
|
compactionsTriggered prometheus.Counter
|
2016-12-31 00:48:49 -08:00
|
|
|
}
|
|
|
|
|
2017-05-26 06:13:03 -07:00
|
|
|
func newDBMetrics(db *DB, r prometheus.Registerer) *dbMetrics {
|
2017-01-06 02:40:09 -08:00
|
|
|
m := &dbMetrics{}
|
2017-01-03 06:43:26 -08:00
|
|
|
|
2017-05-26 06:13:03 -07:00
|
|
|
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",
|
|
|
|
}, func() float64 {
|
|
|
|
db.mtx.RLock()
|
|
|
|
defer db.mtx.RUnlock()
|
|
|
|
return float64(len(db.blocks))
|
|
|
|
})
|
|
|
|
m.reloads = prometheus.NewCounter(prometheus.CounterOpts{
|
|
|
|
Name: "tsdb_reloads_total",
|
|
|
|
Help: "Number of times the database reloaded block data from disk.",
|
|
|
|
})
|
|
|
|
m.reloadsFailed = prometheus.NewCounter(prometheus.CounterOpts{
|
|
|
|
Name: "tsdb_reloads_failures_total",
|
|
|
|
Help: "Number of times the database failed to reload black data from disk.",
|
|
|
|
})
|
2017-08-30 08:38:25 -07:00
|
|
|
|
|
|
|
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())
|
2017-05-26 06:13:03 -07:00
|
|
|
})
|
2017-08-30 08:38:25 -07:00
|
|
|
|
2017-01-03 06:43:26 -08:00
|
|
|
m.samplesAppended = prometheus.NewCounter(prometheus.CounterOpts{
|
2017-01-06 02:40:09 -08:00
|
|
|
Name: "tsdb_samples_appended_total",
|
|
|
|
Help: "Total number of appended sampledb.",
|
2017-01-03 06:43:26 -08:00
|
|
|
})
|
2017-01-06 03:37:28 -08:00
|
|
|
m.compactionsTriggered = prometheus.NewCounter(prometheus.CounterOpts{
|
|
|
|
Name: "tsdb_compactions_triggered_total",
|
|
|
|
Help: "Total number of triggered compactions for the partition.",
|
|
|
|
})
|
2016-12-31 00:48:49 -08:00
|
|
|
|
|
|
|
if r != nil {
|
|
|
|
r.MustRegister(
|
2017-05-26 06:13:03 -07:00
|
|
|
m.activeAppenders,
|
|
|
|
m.loadedBlocks,
|
|
|
|
m.reloads,
|
|
|
|
m.reloadsFailed,
|
2017-08-30 08:38:25 -07:00
|
|
|
m.walTruncateDuration,
|
|
|
|
|
|
|
|
m.headChunks,
|
|
|
|
m.headChunksCreated,
|
|
|
|
m.headChunksRemoved,
|
|
|
|
m.headSeries,
|
|
|
|
m.headSeriesCreated,
|
|
|
|
m.headSeriesRemoved,
|
|
|
|
m.headMinTime,
|
|
|
|
m.headMaxTime,
|
|
|
|
m.headGCDuration,
|
|
|
|
|
2016-12-31 00:48:49 -08:00
|
|
|
m.samplesAppended,
|
2017-01-09 10:14:21 -08:00
|
|
|
m.compactionsTriggered,
|
2016-12-31 00:48:49 -08:00
|
|
|
)
|
|
|
|
}
|
|
|
|
return m
|
|
|
|
}
|
|
|
|
|
2017-01-06 02:40:09 -08:00
|
|
|
// Open returns a new DB in the given directory.
|
2017-02-27 22:17:01 -08:00
|
|
|
func Open(dir string, l log.Logger, r prometheus.Registerer, opts *Options) (db *DB, err error) {
|
2017-02-19 04:01:19 -08:00
|
|
|
if err := os.MkdirAll(dir, 0777); err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
2017-02-19 07:04:37 -08:00
|
|
|
if l == nil {
|
|
|
|
l = log.NewLogfmtLogger(os.Stdout)
|
2017-03-19 08:33:09 -07:00
|
|
|
l = log.With(l, "ts", log.DefaultTimestampUTC, "caller", log.DefaultCaller)
|
2017-02-19 07:04:37 -08:00
|
|
|
}
|
2017-01-17 21:18:32 -08:00
|
|
|
if opts == nil {
|
|
|
|
opts = DefaultOptions
|
|
|
|
}
|
|
|
|
|
2017-08-28 15:39:17 -07:00
|
|
|
wal, err := OpenSegmentWAL(filepath.Join(dir, "wal"), l, 10*time.Second)
|
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
|
2017-01-06 03:37:28 -08:00
|
|
|
db = &DB{
|
2017-07-14 01:06:07 -07:00
|
|
|
dir: dir,
|
|
|
|
logger: l,
|
|
|
|
opts: opts,
|
2017-08-28 15:39:17 -07:00
|
|
|
wal: wal,
|
2017-07-14 01:06:07 -07:00
|
|
|
compactc: make(chan struct{}, 1),
|
|
|
|
donec: make(chan struct{}),
|
|
|
|
stopc: make(chan struct{}),
|
|
|
|
compactionsEnabled: true,
|
2017-08-08 08:35:34 -07:00
|
|
|
chunkPool: chunks.NewPool(),
|
2017-01-06 00:26:39 -08:00
|
|
|
}
|
2017-05-26 06:13:03 -07:00
|
|
|
db.metrics = newDBMetrics(db, r)
|
|
|
|
|
2017-05-09 03:52:47 -07:00
|
|
|
if !opts.NoLockfile {
|
2017-05-18 07:09:30 -07:00
|
|
|
absdir, err := filepath.Abs(dir)
|
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
2017-05-09 03:52:47 -07:00
|
|
|
lockf, err := lockfile.New(filepath.Join(absdir, "lock"))
|
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
if err := lockf.TryLock(); err != nil {
|
|
|
|
return nil, errors.Wrapf(err, "open DB in %s", dir)
|
|
|
|
}
|
|
|
|
db.lockf = &lockf
|
|
|
|
}
|
|
|
|
|
2017-08-09 02:10:29 -07:00
|
|
|
copts := &LeveledCompactorOptions{
|
2017-07-07 04:46:41 -07:00
|
|
|
blockRanges: opts.BlockRanges,
|
2017-08-08 08:35:34 -07:00
|
|
|
chunkPool: db.chunkPool,
|
2017-07-07 04:46:41 -07:00
|
|
|
}
|
|
|
|
|
|
|
|
if len(copts.blockRanges) == 0 {
|
|
|
|
return nil, errors.New("at least one block-range must exist")
|
|
|
|
}
|
|
|
|
|
|
|
|
for float64(copts.blockRanges[len(copts.blockRanges)-1])/float64(opts.RetentionDuration) > 0.2 {
|
|
|
|
if len(copts.blockRanges) == 1 {
|
|
|
|
break
|
|
|
|
}
|
|
|
|
// Max overflow is restricted to 20%.
|
|
|
|
copts.blockRanges = copts.blockRanges[:len(copts.blockRanges)-1]
|
|
|
|
}
|
|
|
|
|
2017-08-09 02:10:29 -07:00
|
|
|
db.compactor = NewLeveledCompactor(r, l, copts)
|
2017-01-06 03:37:28 -08:00
|
|
|
|
2017-08-30 08:38:25 -07:00
|
|
|
db.head, err = NewHead(l, copts.blockRanges[0])
|
2017-08-28 15:39:17 -07:00
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
2017-08-30 08:38:25 -07:00
|
|
|
if err := db.readWAL(db.wal.Reader()); err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
2017-03-02 00:13:29 -08:00
|
|
|
if err := db.reloadBlocks(); err != nil {
|
2016-12-14 23:31:26 -08:00
|
|
|
return nil, err
|
|
|
|
}
|
2017-08-28 15:39:17 -07:00
|
|
|
|
2017-01-06 03:37:28 -08:00
|
|
|
go db.run()
|
|
|
|
|
|
|
|
return db, nil
|
|
|
|
}
|
|
|
|
|
2017-06-08 03:14:13 -07:00
|
|
|
// Dir returns the directory of the database.
|
|
|
|
func (db *DB) Dir() string {
|
|
|
|
return db.dir
|
|
|
|
}
|
|
|
|
|
2017-01-06 03:37:28 -08:00
|
|
|
func (db *DB) run() {
|
|
|
|
defer close(db.donec)
|
|
|
|
|
2017-08-28 15:39:17 -07:00
|
|
|
backoff := time.Duration(0)
|
2017-02-28 06:08:52 -08:00
|
|
|
|
2017-01-19 22:58:19 -08:00
|
|
|
for {
|
|
|
|
select {
|
2017-08-28 15:39:17 -07:00
|
|
|
case <-db.stopc:
|
|
|
|
case <-time.After(backoff):
|
|
|
|
}
|
|
|
|
|
|
|
|
select {
|
|
|
|
case <-time.After(1 * time.Minute):
|
2017-02-28 06:08:52 -08:00
|
|
|
select {
|
|
|
|
case db.compactc <- struct{}{}:
|
|
|
|
default:
|
|
|
|
}
|
2017-01-06 03:37:28 -08:00
|
|
|
case <-db.compactc:
|
|
|
|
db.metrics.compactionsTriggered.Inc()
|
|
|
|
|
2017-08-28 15:39:17 -07:00
|
|
|
_, err1 := db.retentionCutoff()
|
|
|
|
if err1 != nil {
|
|
|
|
db.logger.Log("msg", "retention cutoff failed", "err", err1)
|
2017-03-19 05:50:35 -07:00
|
|
|
}
|
2017-03-04 07:50:48 -08:00
|
|
|
|
2017-08-28 15:39:17 -07:00
|
|
|
_, err2 := db.compact()
|
|
|
|
if err2 != nil {
|
|
|
|
db.logger.Log("msg", "compaction failed", "err", err2)
|
2017-03-19 05:50:35 -07:00
|
|
|
}
|
2017-03-17 07:30:05 -07:00
|
|
|
|
2017-08-28 15:39:17 -07:00
|
|
|
if err1 != nil || err2 != nil {
|
|
|
|
exponential(backoff, 1*time.Second, 1*time.Minute)
|
2017-01-06 03:37:28 -08:00
|
|
|
}
|
2017-01-06 06:18:06 -08:00
|
|
|
|
2017-01-06 03:37:28 -08:00
|
|
|
case <-db.stopc:
|
|
|
|
return
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2017-03-17 07:30:05 -07:00
|
|
|
func (db *DB) retentionCutoff() (bool, error) {
|
|
|
|
if db.opts.RetentionDuration == 0 {
|
|
|
|
return false, nil
|
|
|
|
}
|
2017-03-17 07:56:19 -07:00
|
|
|
|
|
|
|
db.mtx.RLock()
|
|
|
|
defer db.mtx.RUnlock()
|
|
|
|
|
2017-08-28 15:39:17 -07:00
|
|
|
if len(db.blocks) == 0 {
|
2017-03-17 07:30:05 -07:00
|
|
|
return false, nil
|
|
|
|
}
|
|
|
|
|
2017-08-28 15:39:17 -07:00
|
|
|
last := db.blocks[len(db.blocks)-1]
|
2017-03-17 07:30:05 -07:00
|
|
|
mint := last.Meta().MaxTime - int64(db.opts.RetentionDuration)
|
|
|
|
|
|
|
|
return retentionCutoff(db.dir, mint)
|
|
|
|
}
|
|
|
|
|
2017-07-13 07:15:13 -07:00
|
|
|
func (db *DB) compact() (changes bool, err error) {
|
|
|
|
db.cmtx.Lock()
|
|
|
|
defer db.cmtx.Unlock()
|
2017-01-17 21:18:32 -08:00
|
|
|
|
2017-07-14 01:06:07 -07:00
|
|
|
if !db.compactionsEnabled {
|
|
|
|
return false, nil
|
|
|
|
}
|
|
|
|
|
2017-07-13 07:15:13 -07:00
|
|
|
// Check whether we have pending head blocks that are ready to be persisted.
|
|
|
|
// They have the highest priority.
|
2017-08-28 15:39:17 -07:00
|
|
|
for {
|
2017-03-04 07:50:48 -08:00
|
|
|
select {
|
|
|
|
case <-db.stopc:
|
2017-03-20 02:41:43 -07:00
|
|
|
return changes, nil
|
2017-03-04 07:50:48 -08:00
|
|
|
default:
|
2017-02-02 00:32:06 -08:00
|
|
|
}
|
2017-08-28 15:39:17 -07:00
|
|
|
// The head has a compactable range if 1.5 level 0 ranges are between the oldest
|
|
|
|
// and newest timestamp. The 0.5 acts as a buffer of the appendable window.
|
|
|
|
if db.head.MaxTime()-db.head.MinTime() <= db.opts.BlockRanges[0]/2*3 {
|
|
|
|
break
|
|
|
|
}
|
|
|
|
mint, maxt := rangeForTimestamp(db.head.MinTime(), db.opts.BlockRanges[0])
|
2017-02-02 00:32:06 -08:00
|
|
|
|
2017-08-28 15:39:17 -07:00
|
|
|
// Wrap head into a range that bounds all reads to it.
|
|
|
|
head := &rangeHead{
|
|
|
|
head: db.head,
|
|
|
|
mint: mint,
|
|
|
|
maxt: maxt,
|
|
|
|
}
|
|
|
|
if err = db.compactor.Write(db.dir, head, mint, maxt); err != nil {
|
2017-03-04 07:50:48 -08:00
|
|
|
return changes, errors.Wrap(err, "persist head block")
|
|
|
|
}
|
|
|
|
changes = true
|
2017-08-09 02:10:29 -07:00
|
|
|
|
2017-08-28 15:39:17 -07:00
|
|
|
if err := db.reloadBlocks(); err != nil {
|
|
|
|
return changes, errors.Wrap(err, "reload blocks")
|
2017-08-09 02:10:29 -07:00
|
|
|
}
|
2017-03-21 04:21:02 -07:00
|
|
|
runtime.GC()
|
2017-03-04 07:50:48 -08:00
|
|
|
}
|
2017-01-06 03:37:28 -08:00
|
|
|
|
2017-03-02 00:13:29 -08:00
|
|
|
// Check for compactions of multiple blocks.
|
|
|
|
for {
|
2017-08-09 02:10:29 -07:00
|
|
|
plan, err := db.compactor.Plan(db.dir)
|
2017-03-02 00:13:29 -08:00
|
|
|
if err != nil {
|
2017-03-04 07:50:48 -08:00
|
|
|
return changes, errors.Wrap(err, "plan compaction")
|
2017-03-02 00:13:29 -08:00
|
|
|
}
|
2017-08-09 02:10:29 -07:00
|
|
|
if len(plan) == 0 {
|
2017-03-21 04:21:02 -07:00
|
|
|
break
|
|
|
|
}
|
2017-01-06 03:37:28 -08:00
|
|
|
|
2017-03-02 00:13:29 -08:00
|
|
|
select {
|
|
|
|
case <-db.stopc:
|
2017-03-20 02:41:43 -07:00
|
|
|
return changes, nil
|
2017-03-02 00:13:29 -08:00
|
|
|
default:
|
|
|
|
}
|
2017-03-20 02:41:43 -07:00
|
|
|
|
2017-08-09 02:10:29 -07:00
|
|
|
if err := db.compactor.Compact(db.dir, plan...); err != nil {
|
|
|
|
return changes, errors.Wrapf(err, "compact %s", plan)
|
|
|
|
}
|
|
|
|
changes = true
|
|
|
|
|
|
|
|
for _, pd := range plan {
|
|
|
|
if err := os.RemoveAll(pd); err != nil {
|
|
|
|
return changes, errors.Wrap(err, "delete compacted block")
|
2017-03-02 00:13:29 -08:00
|
|
|
}
|
2017-01-17 21:18:32 -08:00
|
|
|
}
|
2017-08-28 15:39:17 -07:00
|
|
|
|
|
|
|
if err := db.reloadBlocks(); err != nil {
|
|
|
|
return changes, errors.Wrap(err, "reload blocks")
|
|
|
|
}
|
2017-08-09 02:10:29 -07:00
|
|
|
runtime.GC()
|
2017-02-23 01:50:22 -08:00
|
|
|
}
|
|
|
|
|
2017-03-04 07:50:48 -08:00
|
|
|
return changes, nil
|
2017-02-09 17:54:26 -08:00
|
|
|
}
|
|
|
|
|
2017-03-17 07:30:05 -07:00
|
|
|
// retentionCutoff deletes all directories of blocks in dir that are strictly
|
|
|
|
// before mint.
|
|
|
|
func retentionCutoff(dir string, mint int64) (bool, error) {
|
2017-03-19 05:50:35 -07:00
|
|
|
df, err := fileutil.OpenDir(dir)
|
|
|
|
if err != nil {
|
|
|
|
return false, errors.Wrapf(err, "open directory")
|
|
|
|
}
|
2017-06-11 15:05:04 -07:00
|
|
|
defer df.Close()
|
|
|
|
|
2017-03-17 07:30:05 -07:00
|
|
|
dirs, err := blockDirs(dir)
|
|
|
|
if err != nil {
|
|
|
|
return false, errors.Wrapf(err, "list block dirs %s", dir)
|
|
|
|
}
|
|
|
|
|
|
|
|
changes := false
|
|
|
|
|
|
|
|
for _, dir := range dirs {
|
|
|
|
meta, err := readMetaFile(dir)
|
|
|
|
if err != nil {
|
|
|
|
return changes, errors.Wrapf(err, "read block meta %s", dir)
|
|
|
|
}
|
|
|
|
// The first block we encounter marks that we crossed the boundary
|
|
|
|
// of deletable blocks.
|
|
|
|
if meta.MaxTime >= mint {
|
|
|
|
break
|
|
|
|
}
|
|
|
|
changes = true
|
|
|
|
|
|
|
|
if err := os.RemoveAll(dir); err != nil {
|
|
|
|
return changes, err
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2017-03-19 05:50:35 -07:00
|
|
|
return changes, fileutil.Fsync(df)
|
2017-03-17 07:30:05 -07:00
|
|
|
}
|
2017-03-02 00:13:29 -08:00
|
|
|
|
2017-08-28 15:39:17 -07:00
|
|
|
func (db *DB) getBlock(id ulid.ULID) (DiskBlock, bool) {
|
2017-03-20 00:41:56 -07:00
|
|
|
for _, b := range db.blocks {
|
2017-05-18 07:09:30 -07:00
|
|
|
if b.Meta().ULID == id {
|
2017-03-20 00:41:56 -07:00
|
|
|
return b, true
|
|
|
|
}
|
|
|
|
}
|
|
|
|
return nil, false
|
|
|
|
}
|
|
|
|
|
2017-08-30 08:38:25 -07:00
|
|
|
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
|
|
|
|
|
|
|
|
}
|
|
|
|
|
2017-05-26 06:13:03 -07:00
|
|
|
func (db *DB) reloadBlocks() (err error) {
|
2017-08-30 08:38:25 -07:00
|
|
|
defer func() {
|
2017-05-26 06:13:03 -07:00
|
|
|
if err != nil {
|
|
|
|
db.metrics.reloadsFailed.Inc()
|
|
|
|
}
|
|
|
|
db.metrics.reloads.Inc()
|
2017-08-30 08:38:25 -07:00
|
|
|
}()
|
2017-05-26 06:13:03 -07:00
|
|
|
|
2017-03-17 06:10:18 -07:00
|
|
|
var cs []io.Closer
|
2017-03-23 10:27:20 -07:00
|
|
|
defer func() { closeAll(cs...) }()
|
2017-03-17 06:10:18 -07:00
|
|
|
|
2017-03-02 00:13:29 -08:00
|
|
|
dirs, err := blockDirs(db.dir)
|
|
|
|
if err != nil {
|
|
|
|
return errors.Wrap(err, "find blocks")
|
|
|
|
}
|
2017-02-09 17:54:26 -08:00
|
|
|
var (
|
2017-08-28 15:39:17 -07:00
|
|
|
blocks []DiskBlock
|
2017-05-18 07:09:30 -07:00
|
|
|
exist = map[ulid.ULID]struct{}{}
|
2017-02-09 17:54:26 -08:00
|
|
|
)
|
2017-03-02 00:13:29 -08:00
|
|
|
|
|
|
|
for _, dir := range dirs {
|
|
|
|
meta, err := readMetaFile(dir)
|
|
|
|
if err != nil {
|
|
|
|
return errors.Wrapf(err, "read meta information %s", dir)
|
2017-02-09 17:54:26 -08:00
|
|
|
}
|
2016-12-14 23:31:26 -08:00
|
|
|
|
2017-05-18 07:09:30 -07:00
|
|
|
b, ok := db.getBlock(meta.ULID)
|
|
|
|
if !ok {
|
2017-08-28 15:39:17 -07:00
|
|
|
b, err = newPersistedBlock(dir, db.chunkPool)
|
2017-05-18 07:09:30 -07:00
|
|
|
if err != nil {
|
|
|
|
return errors.Wrapf(err, "open block %s", dir)
|
2017-03-02 00:13:29 -08:00
|
|
|
}
|
|
|
|
}
|
2016-12-22 03:05:24 -08:00
|
|
|
|
2017-03-20 00:41:56 -07:00
|
|
|
blocks = append(blocks, b)
|
2017-05-18 07:09:30 -07:00
|
|
|
exist[meta.ULID] = struct{}{}
|
2016-12-09 01:00:14 -08:00
|
|
|
}
|
2017-01-06 00:26:39 -08:00
|
|
|
|
2017-05-18 07:09:30 -07:00
|
|
|
if err := validateBlockSequence(blocks); err != nil {
|
|
|
|
return errors.Wrap(err, "invalid block sequence")
|
|
|
|
}
|
2017-05-26 04:01:45 -07:00
|
|
|
|
2017-05-18 07:09:30 -07:00
|
|
|
// Close all opened blocks that no longer exist after we returned all locks.
|
2017-08-28 15:39:17 -07:00
|
|
|
// TODO(fabxc: probably races with querier still reading from them. Can
|
|
|
|
// we just abandon them and have the open FDs be GC'd automatically eventually?
|
2017-03-20 00:41:56 -07:00
|
|
|
for _, b := range db.blocks {
|
2017-05-18 07:09:30 -07:00
|
|
|
if _, ok := exist[b.Meta().ULID]; !ok {
|
2017-03-17 06:10:18 -07:00
|
|
|
cs = append(cs, b)
|
2017-01-06 00:26:39 -08:00
|
|
|
}
|
2017-01-02 01:34:55 -08:00
|
|
|
}
|
|
|
|
|
2017-08-28 15:39:17 -07:00
|
|
|
db.mtx.Lock()
|
2017-03-20 00:41:56 -07:00
|
|
|
db.blocks = blocks
|
2017-08-28 15:39:17 -07:00
|
|
|
db.mtx.Unlock()
|
2017-05-18 07:09:30 -07:00
|
|
|
|
2017-08-28 15:39:17 -07:00
|
|
|
// 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
|
|
|
|
}
|
|
|
|
start := time.Now()
|
|
|
|
atomic.StoreInt64(&db.head.minTime, maxt)
|
2017-08-30 08:38:25 -07:00
|
|
|
|
|
|
|
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))
|
|
|
|
|
2017-08-28 15:39:17 -07:00
|
|
|
db.logger.Log("msg", "head GC completed", "duration", time.Since(start))
|
|
|
|
|
2017-08-30 08:38:25 -07:00
|
|
|
start = time.Now()
|
|
|
|
|
2017-08-28 15:39:17 -07:00
|
|
|
if err := db.wal.Truncate(maxt); err != nil {
|
|
|
|
return errors.Wrapf(err, "truncate WAL at %d", maxt)
|
2017-05-18 07:09:30 -07:00
|
|
|
}
|
2017-08-30 08:38:25 -07:00
|
|
|
db.metrics.walTruncateDuration.Observe(time.Since(start).Seconds())
|
2017-05-18 07:09:30 -07:00
|
|
|
|
|
|
|
return nil
|
|
|
|
}
|
2017-01-03 06:43:26 -08:00
|
|
|
|
2017-08-28 15:39:17 -07:00
|
|
|
func validateBlockSequence(bs []DiskBlock) error {
|
2017-05-18 07:09:30 -07:00
|
|
|
if len(bs) == 0 {
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
sort.Slice(bs, func(i, j int) bool {
|
|
|
|
return bs[i].Meta().MinTime < bs[j].Meta().MinTime
|
|
|
|
})
|
|
|
|
prev := bs[0]
|
|
|
|
for _, b := range bs[1:] {
|
|
|
|
if b.Meta().MinTime < prev.Meta().MaxTime {
|
2017-05-26 06:13:03 -07:00
|
|
|
return errors.Errorf("block time ranges overlap (%d, %d)", b.Meta().MinTime, prev.Meta().MaxTime)
|
2017-05-18 07:09:30 -07:00
|
|
|
}
|
|
|
|
}
|
2017-02-01 06:29:48 -08:00
|
|
|
return nil
|
2017-01-02 13:24:35 -08:00
|
|
|
}
|
|
|
|
|
2017-01-05 23:08:02 -08:00
|
|
|
// Close the partition.
|
2017-01-06 02:40:09 -08:00
|
|
|
func (db *DB) Close() error {
|
2017-01-06 03:37:28 -08:00
|
|
|
close(db.stopc)
|
|
|
|
<-db.donec
|
|
|
|
|
2017-01-06 02:40:09 -08:00
|
|
|
db.mtx.Lock()
|
2017-03-17 04:12:50 -07:00
|
|
|
defer db.mtx.Unlock()
|
2017-03-04 07:50:48 -08:00
|
|
|
|
2017-03-06 03:13:15 -08:00
|
|
|
var g errgroup.Group
|
2017-01-02 01:34:55 -08:00
|
|
|
|
2017-03-20 00:41:56 -07:00
|
|
|
// blocks also contains all head blocks.
|
|
|
|
for _, pb := range db.blocks {
|
2017-03-06 03:13:15 -08:00
|
|
|
g.Go(pb.Close)
|
2016-12-14 23:31:26 -08:00
|
|
|
}
|
|
|
|
|
2017-03-06 03:13:15 -08:00
|
|
|
var merr MultiError
|
|
|
|
|
|
|
|
merr.Add(g.Wait())
|
2017-08-28 15:39:17 -07:00
|
|
|
|
2017-05-09 03:52:47 -07:00
|
|
|
if db.lockf != nil {
|
|
|
|
merr.Add(db.lockf.Unlock())
|
|
|
|
}
|
2017-01-02 13:24:35 -08:00
|
|
|
return merr.Err()
|
2016-12-09 01:00:14 -08:00
|
|
|
}
|
|
|
|
|
2017-06-06 07:53:20 -07:00
|
|
|
// DisableCompactions disables compactions.
|
2017-06-06 11:15:23 -07:00
|
|
|
func (db *DB) DisableCompactions() {
|
2017-07-14 01:06:07 -07:00
|
|
|
db.cmtx.Lock()
|
|
|
|
defer db.cmtx.Unlock()
|
|
|
|
|
|
|
|
db.compactionsEnabled = false
|
|
|
|
db.logger.Log("msg", "compactions disabled")
|
2017-06-06 07:53:20 -07:00
|
|
|
}
|
|
|
|
|
|
|
|
// EnableCompactions enables compactions.
|
2017-06-06 11:15:23 -07:00
|
|
|
func (db *DB) EnableCompactions() {
|
2017-07-14 01:06:07 -07:00
|
|
|
db.cmtx.Lock()
|
|
|
|
defer db.cmtx.Unlock()
|
|
|
|
|
|
|
|
db.compactionsEnabled = true
|
|
|
|
db.logger.Log("msg", "compactions enabled")
|
2017-06-05 01:18:31 -07:00
|
|
|
}
|
|
|
|
|
2017-06-06 07:53:20 -07:00
|
|
|
// Snapshot writes the current data to the directory.
|
2017-06-05 01:18:31 -07:00
|
|
|
func (db *DB) Snapshot(dir string) error {
|
2017-08-28 15:39:17 -07:00
|
|
|
// 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")
|
|
|
|
// }
|
|
|
|
// }
|
2017-06-05 01:18:31 -07:00
|
|
|
|
2017-08-28 15:39:17 -07:00
|
|
|
return nil
|
|
|
|
}
|
2017-07-14 00:00:22 -07:00
|
|
|
|
2017-08-28 15:39:17 -07:00
|
|
|
// Querier returns a new querier over the data partition for the given time range.
|
|
|
|
// A goroutine must not handle more than one open Querier.
|
|
|
|
func (db *DB) Querier(mint, maxt int64) Querier {
|
|
|
|
db.mtx.RLock()
|
|
|
|
|
|
|
|
blocks := db.blocksForInterval(mint, maxt)
|
|
|
|
|
|
|
|
sq := &querier{
|
|
|
|
blocks: make([]Querier, 0, len(blocks)),
|
|
|
|
db: db,
|
|
|
|
}
|
2017-06-06 05:45:54 -07:00
|
|
|
for _, b := range blocks {
|
2017-08-28 15:39:17 -07:00
|
|
|
sq.blocks = append(sq.blocks, &blockQuerier{
|
|
|
|
mint: mint,
|
|
|
|
maxt: maxt,
|
|
|
|
index: b.Index(),
|
|
|
|
chunks: b.Chunks(),
|
|
|
|
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
|
2017-06-05 01:18:31 -07:00
|
|
|
}
|
2017-08-28 15:39:17 -07:00
|
|
|
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)
|
2017-06-05 01:18:31 -07:00
|
|
|
}
|
2017-08-28 15:39:17 -07:00
|
|
|
a.app = a.db.appender()
|
2017-06-05 01:18:31 -07:00
|
|
|
|
2017-08-28 15:39:17 -07:00
|
|
|
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()
|
2017-06-05 01:18:31 -07:00
|
|
|
}
|
|
|
|
|
2017-01-17 21:18:32 -08:00
|
|
|
// Appender returns a new Appender on the database.
|
2017-01-09 11:04:16 -08:00
|
|
|
func (db *DB) Appender() Appender {
|
2017-05-26 06:13:03 -07:00
|
|
|
db.metrics.activeAppenders.Inc()
|
|
|
|
|
2017-08-28 15:39:17 -07:00
|
|
|
// 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()
|
2017-01-09 11:04:16 -08:00
|
|
|
}
|
|
|
|
|
2017-08-28 15:39:17 -07:00
|
|
|
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)
|
|
|
|
}
|
2017-03-20 00:41:56 -07:00
|
|
|
|
2017-08-28 15:39:17 -07:00
|
|
|
func (db *DB) putAppendBuffer(b []RefSample) {
|
|
|
|
db.appendPool.Put(b[:0])
|
2017-01-09 11:04:16 -08:00
|
|
|
}
|
|
|
|
|
2017-08-28 15:39:17 -07:00
|
|
|
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
|
2017-03-20 00:41:56 -07:00
|
|
|
}
|
|
|
|
|
2017-05-17 07:43:01 -07:00
|
|
|
func (a *dbAppender) Add(lset labels.Labels, t int64, v float64) (string, error) {
|
2017-08-28 15:39:17 -07:00
|
|
|
if t < a.mint {
|
|
|
|
return "", ErrOutOfBounds
|
2017-01-12 10:18:51 -08:00
|
|
|
}
|
2017-08-28 15:39:17 -07:00
|
|
|
|
|
|
|
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)
|
2017-02-01 06:29:48 -08:00
|
|
|
}
|
2017-05-17 07:43:01 -07:00
|
|
|
|
2017-08-28 15:39:17 -07:00
|
|
|
// The series is completely new.
|
|
|
|
if a.newSeries == nil {
|
|
|
|
a.newHashes = map[uint64]uint64{}
|
2017-06-26 05:58:00 -07:00
|
|
|
}
|
2017-08-28 15:39:17 -07:00
|
|
|
// 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)
|
2017-01-09 11:04:16 -08:00
|
|
|
}
|
|
|
|
|
2017-05-17 07:43:01 -07:00
|
|
|
func (a *dbAppender) AddFast(ref string, t int64, v float64) error {
|
2017-08-28 15:39:17 -07:00
|
|
|
if len(ref) != 8 {
|
2017-05-18 07:09:30 -07:00
|
|
|
return errors.Wrap(ErrNotFound, "invalid ref length")
|
2017-02-01 06:29:48 -08:00
|
|
|
}
|
2017-08-28 15:39:17 -07:00
|
|
|
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
|
|
|
|
}
|
2017-05-18 08:17:56 -07:00
|
|
|
}
|
2017-08-28 15:39:17 -07:00
|
|
|
if t < a.mint {
|
|
|
|
return ErrOutOfBounds
|
2016-12-31 06:35:08 -08:00
|
|
|
}
|
2017-08-28 15:39:17 -07:00
|
|
|
|
|
|
|
if t > a.highTimestamp {
|
|
|
|
a.highTimestamp = t
|
2017-02-08 16:13:16 -08:00
|
|
|
}
|
2017-08-28 15:39:17 -07:00
|
|
|
// if t < a.lowTimestamp {
|
|
|
|
// a.lowTimestamp = t
|
|
|
|
// }
|
2017-02-08 16:13:16 -08:00
|
|
|
|
2017-08-28 15:39:17 -07:00
|
|
|
a.samples = append(a.samples, RefSample{
|
|
|
|
Ref: refn,
|
|
|
|
T: t,
|
|
|
|
V: v,
|
|
|
|
})
|
2017-02-08 16:13:16 -08:00
|
|
|
return nil
|
2017-01-12 10:18:51 -08:00
|
|
|
}
|
2016-12-09 04:41:38 -08:00
|
|
|
|
2017-08-28 15:39:17 -07:00
|
|
|
func (a *dbAppender) createSeries() error {
|
|
|
|
if len(a.newSeries) == 0 {
|
|
|
|
return nil
|
2017-05-18 07:09:30 -07:00
|
|
|
}
|
2017-08-28 15:39:17 -07:00
|
|
|
a.newLabels = make([]labels.Labels, 0, len(a.newSeries))
|
|
|
|
base0 := len(a.head.series)
|
2017-03-04 07:50:48 -08:00
|
|
|
|
2017-08-28 15:39:17 -07:00
|
|
|
a.head.mtx.RUnlock()
|
|
|
|
defer a.head.mtx.RLock()
|
|
|
|
a.head.mtx.Lock()
|
|
|
|
defer a.head.mtx.Unlock()
|
2017-05-18 07:09:30 -07:00
|
|
|
|
2017-08-28 15:39:17 -07:00
|
|
|
base1 := len(a.head.series)
|
2017-03-17 06:10:18 -07:00
|
|
|
|
2017-08-28 15:39:17 -07:00
|
|
|
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
|
|
|
|
}
|
2017-04-04 02:21:19 -07:00
|
|
|
}
|
2017-08-28 15:39:17 -07:00
|
|
|
// Series is still new.
|
|
|
|
a.newLabels = append(a.newLabels, l.labels)
|
2017-05-18 07:09:30 -07:00
|
|
|
|
2017-08-28 15:39:17 -07:00
|
|
|
s := a.head.create(l.hash, l.labels)
|
|
|
|
l.ref = uint64(s.ref)
|
2017-08-30 08:38:25 -07:00
|
|
|
|
|
|
|
a.db.metrics.headSeriesCreated.Inc()
|
|
|
|
a.db.metrics.headSeries.Inc()
|
2017-05-18 07:09:30 -07:00
|
|
|
}
|
2017-08-28 15:39:17 -07:00
|
|
|
|
|
|
|
// Write all new series to the WAL.
|
|
|
|
if err := a.wal.LogSeries(a.newLabels); err != nil {
|
|
|
|
return errors.Wrap(err, "WAL log series")
|
2017-02-01 06:29:48 -08:00
|
|
|
}
|
2017-01-03 06:43:26 -08:00
|
|
|
|
2017-08-28 15:39:17 -07:00
|
|
|
return nil
|
2017-05-18 07:09:30 -07:00
|
|
|
}
|
|
|
|
|
2017-08-28 15:39:17 -07:00
|
|
|
func (a *dbAppender) Commit() error {
|
|
|
|
defer a.head.mtx.RUnlock()
|
2016-12-09 04:41:38 -08:00
|
|
|
|
2017-08-28 15:39:17 -07:00
|
|
|
defer a.db.metrics.activeAppenders.Dec()
|
|
|
|
defer a.db.putAppendBuffer(a.samples)
|
2017-02-01 06:29:48 -08:00
|
|
|
|
2017-08-28 15:39:17 -07:00
|
|
|
if err := a.createSeries(); err != nil {
|
|
|
|
return err
|
2017-05-18 08:17:56 -07:00
|
|
|
}
|
2017-08-28 15:39:17 -07:00
|
|
|
|
|
|
|
// 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
|
2017-02-01 12:31:35 -08:00
|
|
|
}
|
2017-05-18 07:09:30 -07:00
|
|
|
}
|
2017-08-28 15:39:17 -07:00
|
|
|
|
|
|
|
// 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")
|
2017-02-01 06:29:48 -08:00
|
|
|
}
|
2017-05-18 07:09:30 -07:00
|
|
|
|
2017-08-28 15:39:17 -07:00
|
|
|
total := uint64(len(a.samples))
|
2016-12-09 04:41:38 -08:00
|
|
|
|
2017-08-28 15:39:17 -07:00
|
|
|
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)
|
|
|
|
}
|
2017-08-30 08:38:25 -07:00
|
|
|
ok, chunkCreated := series.append(s.T, s.V)
|
|
|
|
if !ok {
|
2017-08-28 15:39:17 -07:00
|
|
|
total--
|
2017-05-18 08:17:56 -07:00
|
|
|
}
|
2017-08-30 08:38:25 -07:00
|
|
|
if chunkCreated {
|
|
|
|
a.db.metrics.headChunks.Inc()
|
|
|
|
a.db.metrics.headChunksCreated.Inc()
|
|
|
|
}
|
2017-02-01 06:29:48 -08:00
|
|
|
}
|
|
|
|
|
2017-08-30 08:38:25 -07:00
|
|
|
a.db.metrics.samplesAppended.Add(float64(total))
|
|
|
|
|
2017-08-28 15:39:17 -07:00
|
|
|
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
|
|
|
|
}
|
2017-02-08 16:13:16 -08:00
|
|
|
}
|
2017-04-04 02:21:19 -07:00
|
|
|
|
|
|
|
return nil
|
2017-02-01 06:29:48 -08:00
|
|
|
}
|
|
|
|
|
2017-01-12 11:17:49 -08:00
|
|
|
func (a *dbAppender) Rollback() error {
|
2017-08-28 15:39:17 -07:00
|
|
|
a.head.mtx.RUnlock()
|
2017-04-04 02:21:19 -07:00
|
|
|
|
2017-08-28 15:39:17 -07:00
|
|
|
a.db.metrics.activeAppenders.Dec()
|
|
|
|
a.db.putAppendBuffer(a.samples)
|
2017-02-01 06:29:48 -08:00
|
|
|
|
2017-08-28 15:39:17 -07:00
|
|
|
return nil
|
|
|
|
}
|
2017-02-01 06:29:48 -08:00
|
|
|
|
2017-08-28 15:39:17 -07:00
|
|
|
func rangeForTimestamp(t int64, width int64) (mint, maxt int64) {
|
|
|
|
mint = (t / width) * width
|
|
|
|
return mint, mint + width
|
2017-02-01 06:29:48 -08:00
|
|
|
}
|
|
|
|
|
2017-08-28 15:39:17 -07:00
|
|
|
// Delete implements deletion of metrics. It only has atomicity guarantees on a per-block basis.
|
2017-05-19 12:05:50 -07:00
|
|
|
func (db *DB) Delete(mint, maxt int64, ms ...labels.Matcher) error {
|
2017-05-20 00:51:10 -07:00
|
|
|
db.cmtx.Lock()
|
|
|
|
defer db.cmtx.Unlock()
|
2017-07-14 00:00:22 -07:00
|
|
|
|
2017-05-26 04:01:45 -07:00
|
|
|
db.mtx.Lock()
|
|
|
|
defer db.mtx.Unlock()
|
2017-05-19 12:05:50 -07:00
|
|
|
|
|
|
|
var g errgroup.Group
|
|
|
|
|
2017-08-28 15:39:17 -07:00
|
|
|
for _, b := range db.blocks {
|
|
|
|
m := b.Meta()
|
|
|
|
if intervalOverlap(mint, maxt, m.MinTime, m.MaxTime) {
|
|
|
|
g.Go(func(b DiskBlock) func() error {
|
|
|
|
return func() error { return b.Delete(mint, maxt, ms...) }
|
|
|
|
}(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])
|
2017-05-19 12:05:50 -07:00
|
|
|
}
|
|
|
|
|
|
|
|
if err := g.Wait(); err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
|
2017-05-26 04:01:45 -07:00
|
|
|
return nil
|
2017-05-19 12:05:50 -07:00
|
|
|
}
|
|
|
|
|
2016-12-15 07:14:33 -08:00
|
|
|
func intervalOverlap(amin, amax, bmin, bmax int64) bool {
|
2017-05-19 12:05:50 -07:00
|
|
|
// Checks Overlap: http://stackoverflow.com/questions/3269434/
|
2017-05-23 05:43:30 -07:00
|
|
|
return amin <= bmax && bmin <= amax
|
2016-12-15 07:14:33 -08:00
|
|
|
}
|
|
|
|
|
2016-12-16 03:13:17 -08:00
|
|
|
func intervalContains(min, max, t int64) bool {
|
2017-03-07 03:01:25 -08:00
|
|
|
return t >= min && t <= max
|
2016-12-16 03:13:17 -08:00
|
|
|
}
|
|
|
|
|
2017-01-06 00:26:39 -08:00
|
|
|
// blocksForInterval returns all blocks within the partition that may contain
|
2016-12-13 06:26:58 -08:00
|
|
|
// data for the given time range.
|
2017-08-28 15:39:17 -07:00
|
|
|
func (db *DB) blocksForInterval(mint, maxt int64) []BlockReader {
|
|
|
|
var bs []BlockReader
|
2016-12-15 07:14:33 -08:00
|
|
|
|
2017-03-20 00:41:56 -07:00
|
|
|
for _, b := range db.blocks {
|
2017-02-01 06:29:48 -08:00
|
|
|
m := b.Meta()
|
|
|
|
if intervalOverlap(mint, maxt, m.MinTime, m.MaxTime) {
|
2017-01-03 06:43:26 -08:00
|
|
|
bs = append(bs, b)
|
|
|
|
}
|
2016-12-15 07:14:33 -08:00
|
|
|
}
|
2017-08-28 15:39:17 -07:00
|
|
|
if maxt >= db.head.MinTime() {
|
|
|
|
bs = append(bs, db.head)
|
2017-02-01 06:29:48 -08:00
|
|
|
}
|
|
|
|
|
2017-08-28 15:39:17 -07:00
|
|
|
return bs
|
2017-01-19 02:22:47 -08:00
|
|
|
}
|
|
|
|
|
|
|
|
func isBlockDir(fi os.FileInfo) bool {
|
|
|
|
if !fi.IsDir() {
|
|
|
|
return false
|
|
|
|
}
|
2017-05-18 07:09:30 -07:00
|
|
|
_, err := ulid.Parse(fi.Name())
|
|
|
|
return err == nil
|
2017-01-19 02:22:47 -08:00
|
|
|
}
|
|
|
|
|
|
|
|
func blockDirs(dir string) ([]string, error) {
|
|
|
|
files, err := ioutil.ReadDir(dir)
|
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
var dirs []string
|
|
|
|
|
|
|
|
for _, fi := range files {
|
|
|
|
if isBlockDir(fi) {
|
|
|
|
dirs = append(dirs, filepath.Join(dir, fi.Name()))
|
|
|
|
}
|
|
|
|
}
|
|
|
|
return dirs, nil
|
2017-01-03 06:43:26 -08:00
|
|
|
}
|
2016-12-09 04:41:38 -08:00
|
|
|
|
2017-02-13 23:53:19 -08:00
|
|
|
func sequenceFiles(dir, prefix string) ([]string, error) {
|
|
|
|
files, err := ioutil.ReadDir(dir)
|
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
var res []string
|
|
|
|
|
|
|
|
for _, fi := range files {
|
|
|
|
if isSequenceFile(fi, prefix) {
|
|
|
|
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) {
|
2017-01-19 02:22:47 -08:00
|
|
|
names, err := fileutil.ReadDir(dir)
|
2017-01-06 00:26:39 -08:00
|
|
|
if err != nil {
|
2017-01-28 23:11:47 -08:00
|
|
|
return "", 0, err
|
2017-01-06 00:26:39 -08:00
|
|
|
}
|
2017-01-06 04:13:22 -08:00
|
|
|
|
2017-01-06 00:26:39 -08:00
|
|
|
i := uint64(0)
|
2017-01-06 04:13:22 -08:00
|
|
|
for _, n := range names {
|
2017-02-09 17:54:26 -08:00
|
|
|
if !strings.HasPrefix(n, prefix) {
|
2017-01-06 04:13:22 -08:00
|
|
|
continue
|
|
|
|
}
|
2017-02-09 17:54:26 -08:00
|
|
|
j, err := strconv.ParseUint(n[len(prefix):], 10, 32)
|
2017-01-06 04:13:22 -08:00
|
|
|
if err != nil {
|
|
|
|
continue
|
2017-01-06 00:26:39 -08:00
|
|
|
}
|
2017-01-06 04:13:22 -08:00
|
|
|
i = j
|
2017-01-06 00:26:39 -08:00
|
|
|
}
|
2017-02-09 17:54:26 -08:00
|
|
|
return filepath.Join(dir, fmt.Sprintf("%s%0.6d", prefix, i+1)), int(i + 1), nil
|
2017-01-06 00:26:39 -08:00
|
|
|
}
|
2016-12-09 04:41:38 -08:00
|
|
|
|
2016-12-10 09:08:50 -08:00
|
|
|
// The MultiError type implements the error interface, and contains the
|
|
|
|
// Errors used to construct it.
|
|
|
|
type MultiError []error
|
2016-12-07 08:10:49 -08:00
|
|
|
|
2016-12-10 09:08:50 -08:00
|
|
|
// Returns a concatenated string of the contained errors
|
|
|
|
func (es MultiError) Error() string {
|
|
|
|
var buf bytes.Buffer
|
2016-12-07 08:10:49 -08:00
|
|
|
|
2017-01-04 12:11:15 -08:00
|
|
|
if len(es) > 1 {
|
2016-12-10 09:08:50 -08:00
|
|
|
fmt.Fprintf(&buf, "%d errors: ", len(es))
|
2016-12-08 01:04:24 -08:00
|
|
|
}
|
2016-12-07 08:10:49 -08:00
|
|
|
|
2016-12-10 09:08:50 -08:00
|
|
|
for i, err := range es {
|
|
|
|
if i != 0 {
|
|
|
|
buf.WriteString("; ")
|
|
|
|
}
|
|
|
|
buf.WriteString(err.Error())
|
|
|
|
}
|
2016-12-07 08:10:49 -08:00
|
|
|
|
2016-12-10 09:08:50 -08:00
|
|
|
return buf.String()
|
2016-11-15 01:34:25 -08:00
|
|
|
}
|
2016-12-14 23:31:26 -08:00
|
|
|
|
2016-12-15 02:56:41 -08:00
|
|
|
// Add adds the error to the error list if it is not nil.
|
2016-12-31 06:35:08 -08:00
|
|
|
func (es *MultiError) Add(err error) {
|
|
|
|
if err == nil {
|
|
|
|
return
|
|
|
|
}
|
|
|
|
if merr, ok := err.(MultiError); ok {
|
|
|
|
*es = append(*es, merr...)
|
|
|
|
} else {
|
|
|
|
*es = append(*es, err)
|
2016-12-14 23:31:26 -08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2016-12-15 02:56:41 -08:00
|
|
|
// Err returns the error list as an error or nil if it is empty.
|
2016-12-14 23:31:26 -08:00
|
|
|
func (es MultiError) Err() error {
|
|
|
|
if len(es) == 0 {
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
return es
|
|
|
|
}
|
2016-12-15 02:56:41 -08:00
|
|
|
|
2017-05-17 07:43:01 -07:00
|
|
|
func yoloString(b []byte) string { return *((*string)(unsafe.Pointer(&b))) }
|
|
|
|
func yoloBytes(s string) []byte { return *((*[]byte)(unsafe.Pointer(&s))) }
|
2017-02-27 01:46:15 -08:00
|
|
|
|
|
|
|
func closeAll(cs ...io.Closer) error {
|
|
|
|
var merr MultiError
|
|
|
|
|
|
|
|
for _, c := range cs {
|
|
|
|
merr.Add(c.Close())
|
|
|
|
}
|
|
|
|
return merr.Err()
|
|
|
|
}
|
2017-08-28 15:39:17 -07:00
|
|
|
|
|
|
|
func exponential(d, min, max time.Duration) time.Duration {
|
|
|
|
d *= 2
|
|
|
|
if d < min {
|
|
|
|
d = min
|
|
|
|
}
|
|
|
|
if d > max {
|
|
|
|
d = max
|
|
|
|
}
|
|
|
|
return d
|
|
|
|
}
|