2017-04-10 18:59:45 +00:00
|
|
|
// Copyright 2017 The Prometheus Authors
|
|
|
|
// Licensed under the Apache License, Version 2.0 (the "License");
|
|
|
|
// you may not use this file except in compliance with the License.
|
|
|
|
// You may obtain a copy of the License at
|
|
|
|
//
|
|
|
|
// http://www.apache.org/licenses/LICENSE-2.0
|
|
|
|
//
|
|
|
|
// Unless required by applicable law or agreed to in writing, software
|
|
|
|
// distributed under the License is distributed on an "AS IS" BASIS,
|
|
|
|
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
|
|
|
// See the License for the specific language governing permissions and
|
|
|
|
// limitations under the License.
|
|
|
|
|
2016-11-15 09:34:25 +00:00
|
|
|
// Package tsdb implements a time series storage for float64 sample data.
|
|
|
|
package tsdb
|
|
|
|
|
|
|
|
import (
|
2016-12-10 17:08:50 +00:00
|
|
|
"bytes"
|
2016-12-04 12:16:11 +00:00
|
|
|
"fmt"
|
2017-02-27 09:46:15 +00:00
|
|
|
"io"
|
2017-01-06 08:26:39 +00:00
|
|
|
"io/ioutil"
|
2016-12-04 12:16:11 +00:00
|
|
|
"os"
|
2016-12-08 16:43:10 +00:00
|
|
|
"path/filepath"
|
2017-03-21 11:21:02 +00:00
|
|
|
"runtime"
|
2017-05-18 14:09:30 +00:00
|
|
|
"sort"
|
2016-12-15 07:31:26 +00:00
|
|
|
"strconv"
|
2017-01-06 12:13:22 +00:00
|
|
|
"strings"
|
2016-12-08 16:43:10 +00:00
|
|
|
"sync"
|
2017-01-06 14:18:06 +00:00
|
|
|
"time"
|
2016-12-15 10:56:41 +00:00
|
|
|
"unsafe"
|
2016-11-15 09:34:25 +00:00
|
|
|
|
2016-12-15 07:31:26 +00:00
|
|
|
"golang.org/x/sync/errgroup"
|
|
|
|
|
2017-01-03 14:43:26 +00:00
|
|
|
"github.com/coreos/etcd/pkg/fileutil"
|
2016-12-14 17:38:46 +00:00
|
|
|
"github.com/go-kit/kit/log"
|
2017-02-19 12:01:19 +00:00
|
|
|
"github.com/nightlyone/lockfile"
|
2017-05-18 14:09:30 +00:00
|
|
|
"github.com/oklog/ulid"
|
2017-01-03 14:43:26 +00:00
|
|
|
"github.com/pkg/errors"
|
2016-12-31 08:48:49 +00:00
|
|
|
"github.com/prometheus/client_golang/prometheus"
|
2017-04-04 09:27:26 +00:00
|
|
|
"github.com/prometheus/tsdb/labels"
|
2016-11-15 09:34:25 +00:00
|
|
|
)
|
|
|
|
|
2016-12-09 09:00:14 +00:00
|
|
|
// DefaultOptions used for the DB. They are sane for setups using
|
2017-01-06 10:40:09 +00:00
|
|
|
// millisecond precision timestampdb.
|
2016-11-15 09:34:25 +00:00
|
|
|
var DefaultOptions = &Options{
|
2017-02-10 01:54:26 +00:00
|
|
|
WALFlushInterval: 5 * time.Second,
|
|
|
|
RetentionDuration: 15 * 24 * 60 * 60 * 1000, // 15 days in milliseconds
|
|
|
|
MinBlockDuration: 3 * 60 * 60 * 1000, // 2 hours in milliseconds
|
|
|
|
MaxBlockDuration: 24 * 60 * 60 * 1000, // 1 days in milliseconds
|
2017-05-09 10:52:47 +00:00
|
|
|
NoLockfile: false,
|
2016-11-15 09:34:25 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
// Options of the DB storage.
|
|
|
|
type Options struct {
|
2017-01-29 07:11:47 +00:00
|
|
|
// The interval at which the write ahead log is flushed to disc.
|
2017-01-06 14:18:06 +00:00
|
|
|
WALFlushInterval time.Duration
|
2017-01-29 07:11:47 +00:00
|
|
|
|
2017-02-10 01:54:26 +00:00
|
|
|
// Duration of persisted data to keep.
|
|
|
|
RetentionDuration uint64
|
|
|
|
|
2017-02-01 14:29:48 +00:00
|
|
|
// The timestamp range of head blocks after which they get persisted.
|
|
|
|
// It's the minimum duration of any persisted block.
|
|
|
|
MinBlockDuration uint64
|
|
|
|
|
2017-01-29 07:11:47 +00:00
|
|
|
// The maximum timestamp range of compacted blocks.
|
2017-02-01 14:29:48 +00:00
|
|
|
MaxBlockDuration uint64
|
2017-01-29 07:11:47 +00:00
|
|
|
|
2017-05-09 10:52:47 +00:00
|
|
|
// NoLockfile disables creation and consideration of a lock file.
|
|
|
|
NoLockfile bool
|
2016-11-15 09:34:25 +00:00
|
|
|
}
|
|
|
|
|
2017-01-12 19:17:49 +00: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 20:37:28 +00:00
|
|
|
//
|
|
|
|
// Operations on the Appender interface are not goroutine-safe.
|
2016-12-10 17:08:50 +00:00
|
|
|
type Appender interface {
|
2017-02-01 14:29:48 +00: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-05-17 14:43:01 +00:00
|
|
|
Add(l labels.Labels, t int64, v float64) (string, error)
|
2017-02-01 14:29:48 +00: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 14:43:01 +00:00
|
|
|
AddFast(ref string, t int64, v float64) error
|
2016-12-20 23:02:37 +00:00
|
|
|
|
|
|
|
// Commit submits the collected samples and purges the batch.
|
2016-12-10 17:08:50 +00:00
|
|
|
Commit() error
|
2017-01-12 19:17:49 +00:00
|
|
|
|
|
|
|
// Rollback rolls back all modifications made in the appender so far.
|
|
|
|
Rollback() error
|
2016-12-10 17:08:50 +00:00
|
|
|
}
|
|
|
|
|
2017-01-06 10:40:09 +00:00
|
|
|
// DB handles reads and writes of time series falling into
|
|
|
|
// a hashed partition of a seriedb.
|
|
|
|
type DB struct {
|
2017-03-04 15:50:48 +00:00
|
|
|
dir string
|
2017-05-09 10:52:47 +00:00
|
|
|
lockf *lockfile.Lockfile
|
2017-03-04 15:50:48 +00:00
|
|
|
|
2017-01-02 21:24:35 +00:00
|
|
|
logger log.Logger
|
2017-01-06 10:40:09 +00:00
|
|
|
metrics *dbMetrics
|
2017-01-18 05:18:32 +00:00
|
|
|
opts *Options
|
2016-12-09 09:00:14 +00:00
|
|
|
|
2017-03-04 15:50:48 +00:00
|
|
|
// Mutex for that must be held when modifying the general
|
|
|
|
// block layout.
|
2017-03-20 07:41:56 +00:00
|
|
|
mtx sync.RWMutex
|
|
|
|
blocks []Block
|
2017-03-04 15:50:48 +00:00
|
|
|
|
|
|
|
// Mutex that must be held when modifying just the head blocks
|
|
|
|
// or the general layout.
|
2017-03-21 09:11:23 +00:00
|
|
|
// Must never be held when acquiring a blocks's mutex!
|
2017-03-04 15:50:48 +00:00
|
|
|
headmtx sync.RWMutex
|
2017-05-12 14:34:41 +00:00
|
|
|
heads []headBlock
|
2017-01-12 18:18:51 +00:00
|
|
|
|
2017-03-02 08:13:29 +00:00
|
|
|
compactor Compactor
|
2017-01-06 11:37:28 +00:00
|
|
|
|
|
|
|
compactc chan struct{}
|
|
|
|
donec chan struct{}
|
|
|
|
stopc chan struct{}
|
2016-12-09 09:00:14 +00:00
|
|
|
}
|
|
|
|
|
2017-01-06 10:40:09 +00:00
|
|
|
type dbMetrics struct {
|
2017-01-06 11:37:28 +00:00
|
|
|
samplesAppended prometheus.Counter
|
|
|
|
compactionsTriggered prometheus.Counter
|
2016-12-31 08:48:49 +00:00
|
|
|
}
|
|
|
|
|
2017-01-06 10:40:09 +00:00
|
|
|
func newDBMetrics(r prometheus.Registerer) *dbMetrics {
|
|
|
|
m := &dbMetrics{}
|
2017-01-03 14:43:26 +00:00
|
|
|
|
|
|
|
m.samplesAppended = prometheus.NewCounter(prometheus.CounterOpts{
|
2017-01-06 10:40:09 +00:00
|
|
|
Name: "tsdb_samples_appended_total",
|
|
|
|
Help: "Total number of appended sampledb.",
|
2017-01-03 14:43:26 +00:00
|
|
|
})
|
2017-01-06 11:37:28 +00:00
|
|
|
m.compactionsTriggered = prometheus.NewCounter(prometheus.CounterOpts{
|
|
|
|
Name: "tsdb_compactions_triggered_total",
|
|
|
|
Help: "Total number of triggered compactions for the partition.",
|
|
|
|
})
|
2016-12-31 08:48:49 +00:00
|
|
|
|
|
|
|
if r != nil {
|
|
|
|
r.MustRegister(
|
|
|
|
m.samplesAppended,
|
2017-01-09 18:14:21 +00:00
|
|
|
m.compactionsTriggered,
|
2016-12-31 08:48:49 +00:00
|
|
|
)
|
|
|
|
}
|
|
|
|
return m
|
|
|
|
}
|
|
|
|
|
2017-01-06 10:40:09 +00:00
|
|
|
// Open returns a new DB in the given directory.
|
2017-02-28 06:17:01 +00:00
|
|
|
func Open(dir string, l log.Logger, r prometheus.Registerer, opts *Options) (db *DB, err error) {
|
2017-02-19 12:01:19 +00:00
|
|
|
if err := os.MkdirAll(dir, 0777); err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
|
2017-02-19 15:04:37 +00:00
|
|
|
if l == nil {
|
|
|
|
l = log.NewLogfmtLogger(os.Stdout)
|
2017-03-19 15:33:09 +00:00
|
|
|
l = log.With(l, "ts", log.DefaultTimestampUTC, "caller", log.DefaultCaller)
|
2017-02-19 15:04:37 +00:00
|
|
|
}
|
|
|
|
|
2017-01-18 05:18:32 +00:00
|
|
|
if opts == nil {
|
|
|
|
opts = DefaultOptions
|
|
|
|
}
|
|
|
|
|
2017-01-06 11:37:28 +00:00
|
|
|
db = &DB{
|
|
|
|
dir: dir,
|
2017-02-19 15:04:37 +00:00
|
|
|
logger: l,
|
2017-01-09 18:14:21 +00:00
|
|
|
metrics: newDBMetrics(r),
|
2017-01-18 05:18:32 +00:00
|
|
|
opts: opts,
|
2017-01-06 11:37:28 +00:00
|
|
|
compactc: make(chan struct{}, 1),
|
|
|
|
donec: make(chan struct{}),
|
|
|
|
stopc: make(chan struct{}),
|
2017-01-06 08:26:39 +00:00
|
|
|
}
|
2017-05-09 10:52:47 +00:00
|
|
|
if !opts.NoLockfile {
|
2017-05-18 14:09:30 +00:00
|
|
|
absdir, err := filepath.Abs(dir)
|
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
2017-05-09 10:52:47 +00: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-05-18 14:09:30 +00:00
|
|
|
db.compactor = newCompactor(dir, r, l, &compactorOptions{
|
2017-02-01 14:29:48 +00:00
|
|
|
maxBlockRange: opts.MaxBlockDuration,
|
2017-01-18 05:18:32 +00:00
|
|
|
})
|
2017-01-06 11:37:28 +00:00
|
|
|
|
2017-03-02 08:13:29 +00:00
|
|
|
if err := db.reloadBlocks(); err != nil {
|
2016-12-15 07:31:26 +00:00
|
|
|
return nil, err
|
|
|
|
}
|
2017-01-06 11:37:28 +00:00
|
|
|
go db.run()
|
|
|
|
|
|
|
|
return db, nil
|
|
|
|
}
|
|
|
|
|
|
|
|
func (db *DB) run() {
|
|
|
|
defer close(db.donec)
|
|
|
|
|
2017-02-28 14:08:52 +00:00
|
|
|
tick := time.NewTicker(30 * time.Second)
|
|
|
|
defer tick.Stop()
|
|
|
|
|
2017-01-20 06:58:19 +00:00
|
|
|
for {
|
|
|
|
select {
|
2017-02-28 14:08:52 +00:00
|
|
|
case <-tick.C:
|
|
|
|
select {
|
|
|
|
case db.compactc <- struct{}{}:
|
|
|
|
default:
|
|
|
|
}
|
2017-01-06 11:37:28 +00:00
|
|
|
case <-db.compactc:
|
|
|
|
db.metrics.compactionsTriggered.Inc()
|
|
|
|
|
2017-03-17 14:30:05 +00:00
|
|
|
changes1, err := db.retentionCutoff()
|
2017-03-19 12:50:35 +00:00
|
|
|
if err != nil {
|
|
|
|
db.logger.Log("msg", "retention cutoff failed", "err", err)
|
|
|
|
}
|
2017-03-04 15:50:48 +00:00
|
|
|
|
2017-03-17 14:30:05 +00:00
|
|
|
changes2, err := db.compact()
|
2017-03-19 12:50:35 +00:00
|
|
|
if err != nil {
|
|
|
|
db.logger.Log("msg", "compaction failed", "err", err)
|
|
|
|
}
|
2017-03-17 14:30:05 +00:00
|
|
|
|
|
|
|
if changes1 || changes2 {
|
2017-03-19 12:50:35 +00:00
|
|
|
if err := db.reloadBlocks(); err != nil {
|
|
|
|
db.logger.Log("msg", "reloading blocks failed", "err", err)
|
|
|
|
}
|
2017-01-06 11:37:28 +00:00
|
|
|
}
|
2017-01-06 14:18:06 +00:00
|
|
|
|
2017-01-06 11:37:28 +00:00
|
|
|
case <-db.stopc:
|
|
|
|
return
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2017-03-17 14:30:05 +00:00
|
|
|
func (db *DB) retentionCutoff() (bool, error) {
|
|
|
|
if db.opts.RetentionDuration == 0 {
|
|
|
|
return false, nil
|
|
|
|
}
|
2017-03-17 14:56:19 +00:00
|
|
|
|
|
|
|
db.mtx.RLock()
|
|
|
|
defer db.mtx.RUnlock()
|
|
|
|
|
2017-03-21 09:11:23 +00:00
|
|
|
// We only consider the already persisted blocks. Head blocks generally
|
|
|
|
// only account for a fraction of the total data.
|
2017-03-20 13:45:27 +00:00
|
|
|
db.headmtx.RLock()
|
2017-03-21 09:11:23 +00:00
|
|
|
lenp := len(db.blocks) - len(db.heads)
|
|
|
|
db.headmtx.RUnlock()
|
2017-03-20 13:45:27 +00:00
|
|
|
|
2017-03-21 09:11:23 +00:00
|
|
|
if lenp == 0 {
|
2017-03-17 14:30:05 +00:00
|
|
|
return false, nil
|
|
|
|
}
|
|
|
|
|
2017-03-21 09:11:23 +00:00
|
|
|
last := db.blocks[lenp-1]
|
2017-03-17 14:30:05 +00:00
|
|
|
mint := last.Meta().MaxTime - int64(db.opts.RetentionDuration)
|
|
|
|
|
|
|
|
return retentionCutoff(db.dir, mint)
|
|
|
|
}
|
|
|
|
|
2017-03-04 15:50:48 +00:00
|
|
|
func (db *DB) compact() (changes bool, err error) {
|
|
|
|
db.headmtx.RLock()
|
|
|
|
|
2017-03-17 14:30:05 +00:00
|
|
|
// Check whether we have pending head blocks that are ready to be persisted.
|
|
|
|
// They have the highest priority.
|
2017-03-20 07:41:56 +00:00
|
|
|
var singles []Block
|
2017-01-06 15:27:50 +00:00
|
|
|
|
2017-03-04 15:50:48 +00:00
|
|
|
// Collect head blocks that are ready for compaction. Write them after
|
|
|
|
// returning the lock to not block Appenders.
|
|
|
|
// Selected blocks are semantically ensured to not be written to afterwards
|
|
|
|
// by appendable().
|
2017-05-18 14:09:30 +00:00
|
|
|
if len(db.heads) > 2 {
|
|
|
|
for _, h := range db.heads[:len(db.heads)-2] {
|
2017-03-02 08:13:29 +00:00
|
|
|
// Blocks that won't be appendable when instantiating a new appender
|
|
|
|
// might still have active appenders on them.
|
|
|
|
// Abort at the first one we encounter.
|
2017-03-20 07:41:56 +00:00
|
|
|
if h.Busy() {
|
2017-03-02 08:13:29 +00:00
|
|
|
break
|
|
|
|
}
|
2017-03-04 15:50:48 +00:00
|
|
|
singles = append(singles, h)
|
|
|
|
}
|
|
|
|
}
|
2017-01-06 11:37:28 +00:00
|
|
|
|
2017-03-04 15:50:48 +00:00
|
|
|
db.headmtx.RUnlock()
|
2017-01-18 05:18:32 +00:00
|
|
|
|
2017-03-04 15:50:48 +00:00
|
|
|
for _, h := range singles {
|
|
|
|
select {
|
|
|
|
case <-db.stopc:
|
2017-03-20 09:41:43 +00:00
|
|
|
return changes, nil
|
2017-03-04 15:50:48 +00:00
|
|
|
default:
|
2017-02-02 08:32:06 +00:00
|
|
|
}
|
|
|
|
|
2017-05-18 14:09:30 +00:00
|
|
|
if err = db.compactor.Write(h); err != nil {
|
2017-03-04 15:50:48 +00:00
|
|
|
return changes, errors.Wrap(err, "persist head block")
|
|
|
|
}
|
|
|
|
changes = true
|
2017-03-21 11:21:02 +00:00
|
|
|
runtime.GC()
|
2017-03-04 15:50:48 +00:00
|
|
|
}
|
2017-01-06 11:37:28 +00:00
|
|
|
|
2017-03-02 08:13:29 +00:00
|
|
|
// Check for compactions of multiple blocks.
|
|
|
|
for {
|
2017-05-18 14:09:30 +00:00
|
|
|
plans, err := db.compactor.Plan()
|
2017-03-02 08:13:29 +00:00
|
|
|
if err != nil {
|
2017-03-04 15:50:48 +00:00
|
|
|
return changes, errors.Wrap(err, "plan compaction")
|
2017-03-02 08:13:29 +00:00
|
|
|
}
|
2017-03-21 11:21:02 +00:00
|
|
|
if len(plans) == 0 {
|
|
|
|
break
|
|
|
|
}
|
2017-01-06 11:37:28 +00:00
|
|
|
|
2017-03-02 08:13:29 +00:00
|
|
|
select {
|
|
|
|
case <-db.stopc:
|
2017-03-20 09:41:43 +00:00
|
|
|
return changes, nil
|
2017-03-02 08:13:29 +00:00
|
|
|
default:
|
|
|
|
}
|
2017-03-20 09:41:43 +00:00
|
|
|
|
2017-03-02 08:13:29 +00:00
|
|
|
// We just execute compactions sequentially to not cause too extreme
|
|
|
|
// CPU and memory spikes.
|
|
|
|
// TODO(fabxc): return more descriptive plans in the future that allow
|
|
|
|
// estimation of resource usage and conditional parallelization?
|
|
|
|
for _, p := range plans {
|
|
|
|
if err := db.compactor.Compact(p...); err != nil {
|
2017-03-04 15:50:48 +00:00
|
|
|
return changes, errors.Wrapf(err, "compact %s", p)
|
2017-03-02 08:13:29 +00:00
|
|
|
}
|
|
|
|
changes = true
|
2017-03-21 11:21:02 +00:00
|
|
|
runtime.GC()
|
2017-01-18 05:18:32 +00:00
|
|
|
}
|
2017-02-23 09:50:22 +00:00
|
|
|
}
|
|
|
|
|
2017-03-04 15:50:48 +00:00
|
|
|
return changes, nil
|
2017-02-10 01:54:26 +00:00
|
|
|
}
|
|
|
|
|
2017-03-17 14:30:05 +00: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 12:50:35 +00:00
|
|
|
df, err := fileutil.OpenDir(dir)
|
|
|
|
if err != nil {
|
|
|
|
return false, errors.Wrapf(err, "open directory")
|
|
|
|
}
|
2017-03-17 14:30:05 +00: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 12:50:35 +00:00
|
|
|
return changes, fileutil.Fsync(df)
|
2017-03-17 14:30:05 +00:00
|
|
|
}
|
2017-03-02 08:13:29 +00:00
|
|
|
|
2017-05-18 14:09:30 +00:00
|
|
|
func (db *DB) getBlock(id ulid.ULID) (Block, bool) {
|
2017-03-20 07:41:56 +00:00
|
|
|
for _, b := range db.blocks {
|
2017-05-18 14:09:30 +00:00
|
|
|
if b.Meta().ULID == id {
|
2017-03-20 07:41:56 +00:00
|
|
|
return b, true
|
|
|
|
}
|
|
|
|
}
|
|
|
|
return nil, false
|
|
|
|
}
|
|
|
|
|
2017-03-02 08:13:29 +00:00
|
|
|
func (db *DB) reloadBlocks() error {
|
2017-03-17 13:10:18 +00:00
|
|
|
var cs []io.Closer
|
2017-03-23 17:27:20 +00:00
|
|
|
defer func() { closeAll(cs...) }()
|
2017-03-17 13:10:18 +00:00
|
|
|
|
2017-03-02 08:13:29 +00:00
|
|
|
db.mtx.Lock()
|
|
|
|
defer db.mtx.Unlock()
|
2017-02-10 01:54:26 +00:00
|
|
|
|
2017-03-06 08:33:55 +00:00
|
|
|
db.headmtx.Lock()
|
|
|
|
defer db.headmtx.Unlock()
|
|
|
|
|
2017-03-02 08:13:29 +00:00
|
|
|
dirs, err := blockDirs(db.dir)
|
|
|
|
if err != nil {
|
|
|
|
return errors.Wrap(err, "find blocks")
|
|
|
|
}
|
2017-02-10 01:54:26 +00:00
|
|
|
var (
|
2017-05-18 14:09:30 +00:00
|
|
|
blocks []Block
|
|
|
|
exist = map[ulid.ULID]struct{}{}
|
2017-02-10 01:54:26 +00:00
|
|
|
)
|
2017-03-02 08:13:29 +00:00
|
|
|
|
|
|
|
for _, dir := range dirs {
|
|
|
|
meta, err := readMetaFile(dir)
|
|
|
|
if err != nil {
|
|
|
|
return errors.Wrapf(err, "read meta information %s", dir)
|
2017-02-10 01:54:26 +00:00
|
|
|
}
|
2016-12-15 07:31:26 +00:00
|
|
|
|
2017-05-18 14:09:30 +00:00
|
|
|
b, ok := db.getBlock(meta.ULID)
|
|
|
|
if !ok {
|
|
|
|
if meta.Compaction.Generation == 0 {
|
|
|
|
b, err = db.openHeadBlock(dir)
|
|
|
|
} else {
|
|
|
|
b, err = newPersistedBlock(dir)
|
2017-03-02 08:13:29 +00:00
|
|
|
}
|
2017-05-18 14:09:30 +00:00
|
|
|
if err != nil {
|
|
|
|
return errors.Wrapf(err, "open block %s", dir)
|
2017-03-02 08:13:29 +00:00
|
|
|
}
|
|
|
|
}
|
2016-12-22 11:05:24 +00:00
|
|
|
|
2017-03-20 07:41:56 +00:00
|
|
|
blocks = append(blocks, b)
|
2017-05-18 14:09:30 +00:00
|
|
|
exist[meta.ULID] = struct{}{}
|
2016-12-09 09:00:14 +00:00
|
|
|
}
|
2017-01-06 08:26:39 +00:00
|
|
|
|
2017-05-18 14:09:30 +00:00
|
|
|
if err := validateBlockSequence(blocks); err != nil {
|
|
|
|
return errors.Wrap(err, "invalid block sequence")
|
|
|
|
}
|
|
|
|
// Close all opened blocks that no longer exist after we returned all locks.
|
2017-03-20 07:41:56 +00:00
|
|
|
for _, b := range db.blocks {
|
2017-05-18 14:09:30 +00:00
|
|
|
if _, ok := exist[b.Meta().ULID]; !ok {
|
2017-03-17 13:10:18 +00:00
|
|
|
cs = append(cs, b)
|
2017-01-06 08:26:39 +00:00
|
|
|
}
|
2017-01-02 09:34:55 +00:00
|
|
|
}
|
|
|
|
|
2017-03-20 07:41:56 +00:00
|
|
|
db.blocks = blocks
|
2017-05-18 14:09:30 +00:00
|
|
|
db.heads = nil
|
|
|
|
|
|
|
|
for _, b := range blocks {
|
|
|
|
if b.Meta().Compaction.Generation == 0 {
|
|
|
|
db.heads = append(db.heads, b.(*HeadBlock))
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
return nil
|
|
|
|
}
|
2017-01-03 14:43:26 +00:00
|
|
|
|
2017-05-18 14:09:30 +00:00
|
|
|
func validateBlockSequence(bs []Block) error {
|
|
|
|
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 {
|
|
|
|
return errors.Errorf("block time ranges overlap", b.Meta().MinTime, prev.Meta().MaxTime)
|
|
|
|
}
|
|
|
|
}
|
2017-02-01 14:29:48 +00:00
|
|
|
return nil
|
2017-01-02 21:24:35 +00:00
|
|
|
}
|
|
|
|
|
2017-01-06 07:08:02 +00:00
|
|
|
// Close the partition.
|
2017-01-06 10:40:09 +00:00
|
|
|
func (db *DB) Close() error {
|
2017-01-06 11:37:28 +00:00
|
|
|
close(db.stopc)
|
|
|
|
<-db.donec
|
|
|
|
|
2017-01-06 10:40:09 +00:00
|
|
|
db.mtx.Lock()
|
2017-03-17 11:12:50 +00:00
|
|
|
defer db.mtx.Unlock()
|
2017-03-04 15:50:48 +00:00
|
|
|
|
2017-03-06 11:13:15 +00:00
|
|
|
var g errgroup.Group
|
2017-01-02 09:34:55 +00:00
|
|
|
|
2017-03-20 07:41:56 +00:00
|
|
|
// blocks also contains all head blocks.
|
|
|
|
for _, pb := range db.blocks {
|
2017-03-06 11:13:15 +00:00
|
|
|
g.Go(pb.Close)
|
2016-12-15 07:31:26 +00:00
|
|
|
}
|
|
|
|
|
2017-03-06 11:13:15 +00:00
|
|
|
var merr MultiError
|
|
|
|
|
|
|
|
merr.Add(g.Wait())
|
2017-05-09 10:52:47 +00:00
|
|
|
if db.lockf != nil {
|
|
|
|
merr.Add(db.lockf.Unlock())
|
|
|
|
}
|
2017-02-19 12:01:19 +00:00
|
|
|
|
2017-01-02 21:24:35 +00:00
|
|
|
return merr.Err()
|
2016-12-09 09:00:14 +00:00
|
|
|
}
|
|
|
|
|
2017-01-18 05:18:32 +00:00
|
|
|
// Appender returns a new Appender on the database.
|
2017-01-09 19:04:16 +00:00
|
|
|
func (db *DB) Appender() Appender {
|
2017-01-12 18:18:51 +00:00
|
|
|
db.mtx.RLock()
|
2017-05-18 14:09:30 +00:00
|
|
|
return &dbAppender{db: db}
|
2017-01-09 19:04:16 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
type dbAppender struct {
|
2017-03-20 07:41:56 +00:00
|
|
|
db *DB
|
|
|
|
heads []*metaAppender
|
|
|
|
|
2017-02-09 00:13:16 +00:00
|
|
|
samples int
|
2017-01-09 19:04:16 +00:00
|
|
|
}
|
|
|
|
|
2017-03-20 07:41:56 +00:00
|
|
|
type metaAppender struct {
|
|
|
|
meta BlockMeta
|
|
|
|
app Appender
|
|
|
|
}
|
|
|
|
|
2017-05-17 14:43:01 +00:00
|
|
|
func (a *dbAppender) Add(lset labels.Labels, t int64, v float64) (string, error) {
|
2017-05-18 15:17:56 +00:00
|
|
|
h, err := a.appenderAt(t)
|
2017-01-12 18:18:51 +00:00
|
|
|
if err != nil {
|
2017-05-17 14:43:01 +00:00
|
|
|
return "", err
|
2017-01-12 18:18:51 +00:00
|
|
|
}
|
2017-03-20 07:41:56 +00:00
|
|
|
ref, err := h.app.Add(lset, t, v)
|
2017-02-01 14:29:48 +00:00
|
|
|
if err != nil {
|
2017-05-17 14:43:01 +00:00
|
|
|
return "", err
|
2017-02-01 14:29:48 +00:00
|
|
|
}
|
2017-02-09 00:13:16 +00:00
|
|
|
a.samples++
|
2017-05-17 14:43:01 +00:00
|
|
|
|
|
|
|
return string(append(h.meta.ULID[:], ref...)), nil
|
2017-01-09 19:04:16 +00:00
|
|
|
}
|
|
|
|
|
2017-05-17 14:43:01 +00:00
|
|
|
func (a *dbAppender) AddFast(ref string, t int64, v float64) error {
|
2017-05-18 14:09:30 +00:00
|
|
|
if len(ref) < 16 {
|
|
|
|
return errors.Wrap(ErrNotFound, "invalid ref length")
|
2017-02-01 14:29:48 +00:00
|
|
|
}
|
2017-05-18 14:09:30 +00:00
|
|
|
// The first 16 bytes a ref hold the ULID of the head block.
|
2017-05-18 15:17:56 +00:00
|
|
|
h, err := a.appenderAt(t)
|
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
// Validate the ref points to the same block we got for t.
|
|
|
|
if string(h.meta.ULID[:]) != ref[:16] {
|
|
|
|
return ErrNotFound
|
2016-12-31 14:35:08 +00:00
|
|
|
}
|
2017-05-17 14:43:01 +00:00
|
|
|
if err := h.app.AddFast(ref[16:], t, v); err != nil {
|
2017-05-18 14:09:30 +00:00
|
|
|
// The block the ref points to might fit the given timestamp.
|
|
|
|
// We mask the error to stick with our contract.
|
|
|
|
if errors.Cause(err) == ErrOutOfBounds {
|
|
|
|
err = ErrNotFound
|
|
|
|
}
|
2017-02-09 00:13:16 +00:00
|
|
|
return err
|
|
|
|
}
|
|
|
|
|
|
|
|
a.samples++
|
|
|
|
return nil
|
2017-01-12 18:18:51 +00:00
|
|
|
}
|
2016-12-09 12:41:38 +00:00
|
|
|
|
2017-02-01 14:29:48 +00:00
|
|
|
// appenderFor gets the appender for the head containing timestamp t.
|
|
|
|
// If the head block doesn't exist yet, it gets created.
|
2017-05-18 15:17:56 +00:00
|
|
|
func (a *dbAppender) appenderAt(t int64) (*metaAppender, error) {
|
2017-05-18 14:09:30 +00:00
|
|
|
for _, h := range a.heads {
|
|
|
|
if intervalContains(h.meta.MinTime, h.meta.MaxTime-1, t) {
|
|
|
|
return h, nil
|
2017-02-01 14:29:48 +00:00
|
|
|
}
|
2017-05-18 14:09:30 +00:00
|
|
|
}
|
|
|
|
// Currently opened appenders do not cover t. Ensure the head block is
|
|
|
|
// created and add missing appenders.
|
|
|
|
a.db.headmtx.Lock()
|
2017-03-04 15:50:48 +00:00
|
|
|
|
2017-05-18 14:09:30 +00:00
|
|
|
if err := a.db.ensureHead(t); err != nil {
|
2017-03-04 15:50:48 +00:00
|
|
|
a.db.headmtx.Unlock()
|
2017-05-18 14:09:30 +00:00
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
|
|
|
|
var hb headBlock
|
|
|
|
for _, h := range a.db.appendable() {
|
|
|
|
m := h.Meta()
|
2017-03-17 13:10:18 +00:00
|
|
|
|
2017-05-18 14:09:30 +00:00
|
|
|
if intervalContains(m.MinTime, m.MaxTime-1, t) {
|
|
|
|
hb = h
|
|
|
|
break
|
2017-04-04 09:21:19 +00:00
|
|
|
}
|
2017-02-01 20:31:35 +00:00
|
|
|
}
|
2017-05-18 14:09:30 +00:00
|
|
|
a.db.headmtx.Unlock()
|
|
|
|
|
|
|
|
if hb == nil {
|
|
|
|
return nil, ErrOutOfBounds
|
|
|
|
}
|
|
|
|
// Instantiate appender after returning headmtx!
|
|
|
|
app := &metaAppender{
|
|
|
|
meta: hb.Meta(),
|
|
|
|
app: hb.Appender(),
|
2017-02-01 14:29:48 +00:00
|
|
|
}
|
2017-05-18 14:09:30 +00:00
|
|
|
a.heads = append(a.heads, app)
|
2017-01-03 14:43:26 +00:00
|
|
|
|
2017-05-18 14:09:30 +00:00
|
|
|
return app, nil
|
|
|
|
}
|
|
|
|
|
|
|
|
func rangeForTimestamp(t int64, width int64) (mint, maxt int64) {
|
|
|
|
mint = (t / width) * width
|
|
|
|
return mint, mint + width
|
2017-02-01 14:29:48 +00:00
|
|
|
}
|
2016-12-09 12:41:38 +00:00
|
|
|
|
2017-03-06 11:13:15 +00:00
|
|
|
// ensureHead makes sure that there is a head block for the timestamp t if
|
|
|
|
// it is within or after the currently appendable window.
|
2017-02-01 20:31:35 +00:00
|
|
|
func (db *DB) ensureHead(t int64) error {
|
2017-05-18 15:17:56 +00:00
|
|
|
var (
|
|
|
|
mint, maxt = rangeForTimestamp(t, int64(db.opts.MinBlockDuration))
|
|
|
|
addBuffer = len(db.blocks) == 0
|
|
|
|
last BlockMeta
|
|
|
|
)
|
2017-02-01 14:29:48 +00:00
|
|
|
|
2017-05-18 15:17:56 +00:00
|
|
|
if !addBuffer {
|
|
|
|
last = db.blocks[len(db.blocks)-1].Meta()
|
|
|
|
addBuffer = last.MaxTime <= mint-int64(db.opts.MinBlockDuration)
|
|
|
|
}
|
2017-05-18 14:31:02 +00:00
|
|
|
// Create another block of buffer in front if the DB is initialized or retrieving
|
|
|
|
// new data after a long gap.
|
|
|
|
// This ensures we always have a full block width if append window.
|
2017-05-18 15:17:56 +00:00
|
|
|
if addBuffer {
|
2017-05-18 14:09:30 +00:00
|
|
|
if _, err := db.createHeadBlock(mint-int64(db.opts.MinBlockDuration), mint); err != nil {
|
2017-02-01 20:31:35 +00:00
|
|
|
return err
|
|
|
|
}
|
2017-05-18 14:09:30 +00:00
|
|
|
// If the previous block reaches into our new window, make it smaller.
|
2017-05-18 14:31:02 +00:00
|
|
|
} else if mt := last.MaxTime; mt > mint {
|
2017-05-18 14:09:30 +00:00
|
|
|
mint = mt
|
|
|
|
}
|
|
|
|
if mint >= maxt {
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
// Error if the requested time for a head is before the appendable window.
|
|
|
|
if len(db.heads) > 0 && t < db.heads[0].Meta().MinTime {
|
|
|
|
return ErrOutOfBounds
|
2017-02-01 14:29:48 +00:00
|
|
|
}
|
2017-05-18 14:09:30 +00:00
|
|
|
|
|
|
|
_, err := db.createHeadBlock(mint, maxt)
|
|
|
|
return err
|
2016-12-09 12:41:38 +00:00
|
|
|
}
|
|
|
|
|
2017-02-01 14:29:48 +00:00
|
|
|
func (a *dbAppender) Commit() error {
|
2017-04-04 09:21:19 +00:00
|
|
|
defer a.db.mtx.RUnlock()
|
|
|
|
|
|
|
|
// Commits to partial appenders must be concurrent as concurrent appenders
|
|
|
|
// may have conflicting locks on head appenders.
|
2017-05-18 15:17:56 +00:00
|
|
|
// For high-throughput use cases the errgroup causes significant blocking. Typically,
|
|
|
|
// we just deal with a single appender and special case it.
|
|
|
|
var err error
|
2017-02-01 14:29:48 +00:00
|
|
|
|
2017-05-18 15:17:56 +00:00
|
|
|
switch len(a.heads) {
|
|
|
|
case 1:
|
|
|
|
err = a.heads[0].app.Commit()
|
|
|
|
default:
|
|
|
|
var g errgroup.Group
|
|
|
|
for _, h := range a.heads {
|
|
|
|
g.Go(h.app.Commit)
|
|
|
|
}
|
|
|
|
err = g.Wait()
|
2017-02-01 14:29:48 +00:00
|
|
|
}
|
|
|
|
|
2017-05-18 15:17:56 +00:00
|
|
|
if err != nil {
|
2017-04-04 09:21:19 +00:00
|
|
|
return err
|
2017-02-09 00:13:16 +00:00
|
|
|
}
|
2017-04-04 09:21:19 +00:00
|
|
|
// XXX(fabxc): Push the metric down into head block to account properly
|
|
|
|
// for partial appends?
|
|
|
|
a.db.metrics.samplesAppended.Add(float64(a.samples))
|
|
|
|
|
|
|
|
return nil
|
2017-02-01 14:29:48 +00:00
|
|
|
}
|
|
|
|
|
2017-01-12 19:17:49 +00:00
|
|
|
func (a *dbAppender) Rollback() error {
|
2017-04-04 09:21:19 +00:00
|
|
|
defer a.db.mtx.RUnlock()
|
|
|
|
|
|
|
|
var g errgroup.Group
|
2017-02-01 14:29:48 +00:00
|
|
|
|
|
|
|
for _, h := range a.heads {
|
2017-04-18 16:22:13 +00:00
|
|
|
g.Go(h.app.Rollback)
|
2017-02-01 14:29:48 +00:00
|
|
|
}
|
|
|
|
|
2017-04-04 09:21:19 +00:00
|
|
|
return g.Wait()
|
2017-02-01 14:29:48 +00:00
|
|
|
}
|
|
|
|
|
2017-03-20 07:41:56 +00:00
|
|
|
// appendable returns a copy of a slice of HeadBlocks that can still be appended to.
|
2017-05-18 14:09:30 +00:00
|
|
|
func (db *DB) appendable() (r []headBlock) {
|
|
|
|
switch len(db.heads) {
|
|
|
|
case 0:
|
|
|
|
case 1:
|
|
|
|
r = append(r, db.heads[0])
|
|
|
|
default:
|
|
|
|
r = append(r, db.heads[len(db.heads)-2:]...)
|
2017-02-01 14:29:48 +00:00
|
|
|
}
|
2017-05-18 14:09:30 +00:00
|
|
|
return r
|
2017-01-12 19:17:49 +00:00
|
|
|
}
|
|
|
|
|
2016-12-15 15:14:33 +00:00
|
|
|
func intervalOverlap(amin, amax, bmin, bmax int64) bool {
|
2017-03-07 11:01:25 +00:00
|
|
|
if bmin >= amin && bmin <= amax {
|
2016-12-15 15:14:33 +00:00
|
|
|
return true
|
|
|
|
}
|
2017-03-07 11:01:25 +00:00
|
|
|
if amin >= bmin && amin <= bmax {
|
2016-12-15 15:14:33 +00:00
|
|
|
return true
|
|
|
|
}
|
|
|
|
return false
|
|
|
|
}
|
|
|
|
|
2016-12-16 11:13:17 +00:00
|
|
|
func intervalContains(min, max, t int64) bool {
|
2017-03-07 11:01:25 +00:00
|
|
|
return t >= min && t <= max
|
2016-12-16 11:13:17 +00:00
|
|
|
}
|
|
|
|
|
2017-01-06 08:26:39 +00:00
|
|
|
// blocksForInterval returns all blocks within the partition that may contain
|
2016-12-13 14:26:58 +00:00
|
|
|
// data for the given time range.
|
2017-01-10 14:28:22 +00:00
|
|
|
func (db *DB) blocksForInterval(mint, maxt int64) []Block {
|
|
|
|
var bs []Block
|
2016-12-15 15:14:33 +00:00
|
|
|
|
2017-03-20 07:41:56 +00:00
|
|
|
for _, b := range db.blocks {
|
2017-02-01 14:29:48 +00:00
|
|
|
m := b.Meta()
|
|
|
|
if intervalOverlap(mint, maxt, m.MinTime, m.MaxTime) {
|
2017-01-03 14:43:26 +00:00
|
|
|
bs = append(bs, b)
|
|
|
|
}
|
2016-12-15 15:14:33 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
return bs
|
2016-12-13 14:26:58 +00:00
|
|
|
}
|
|
|
|
|
2017-05-13 16:14:18 +00:00
|
|
|
// openHeadBlock opens the head block at dir.
|
|
|
|
func (db *DB) openHeadBlock(dir string) (*HeadBlock, error) {
|
|
|
|
var (
|
2017-05-18 14:09:30 +00:00
|
|
|
wdir = walDir(dir)
|
2017-05-13 16:14:18 +00:00
|
|
|
l = log.With(db.logger, "wal", wdir)
|
|
|
|
)
|
|
|
|
wal, err := OpenSegmentWAL(wdir, l, 5*time.Second)
|
|
|
|
if err != nil {
|
|
|
|
return nil, errors.Wrap(err, "open WAL %s")
|
|
|
|
}
|
|
|
|
|
|
|
|
h, err := OpenHeadBlock(dir, log.With(db.logger, "block", dir), wal)
|
|
|
|
if err != nil {
|
|
|
|
return nil, errors.Wrapf(err, "open head block %s", dir)
|
|
|
|
}
|
|
|
|
return h, nil
|
|
|
|
}
|
|
|
|
|
2017-05-18 14:09:30 +00:00
|
|
|
// createHeadBlock starts a new head block to append to.
|
|
|
|
func (db *DB) createHeadBlock(mint, maxt int64) (headBlock, error) {
|
|
|
|
dir, err := TouchHeadBlock(db.dir, mint, maxt)
|
2017-01-19 13:01:38 +00:00
|
|
|
if err != nil {
|
2017-05-13 16:14:18 +00:00
|
|
|
return nil, errors.Wrapf(err, "touch head block %s", dir)
|
|
|
|
}
|
|
|
|
newHead, err := db.openHeadBlock(dir)
|
2017-01-06 08:26:39 +00:00
|
|
|
if err != nil {
|
2017-01-19 10:22:47 +00:00
|
|
|
return nil, err
|
2017-01-06 08:26:39 +00:00
|
|
|
}
|
2017-02-01 14:29:48 +00:00
|
|
|
|
2017-03-21 09:11:23 +00:00
|
|
|
db.blocks = append(db.blocks, newHead) // TODO(fabxc): this is a race!
|
2017-01-06 14:18:06 +00:00
|
|
|
db.heads = append(db.heads, newHead)
|
2017-02-01 14:29:48 +00:00
|
|
|
|
|
|
|
select {
|
|
|
|
case db.compactc <- struct{}{}:
|
|
|
|
default:
|
|
|
|
}
|
|
|
|
|
2017-01-19 10:22:47 +00:00
|
|
|
return newHead, nil
|
|
|
|
}
|
|
|
|
|
|
|
|
func isBlockDir(fi os.FileInfo) bool {
|
|
|
|
if !fi.IsDir() {
|
|
|
|
return false
|
|
|
|
}
|
2017-05-18 14:09:30 +00:00
|
|
|
_, err := ulid.Parse(fi.Name())
|
|
|
|
return err == nil
|
2017-01-19 10:22:47 +00: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 14:43:26 +00:00
|
|
|
}
|
2016-12-09 12:41:38 +00:00
|
|
|
|
2017-02-14 07:53:19 +00: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 10:22:47 +00:00
|
|
|
names, err := fileutil.ReadDir(dir)
|
2017-01-06 08:26:39 +00:00
|
|
|
if err != nil {
|
2017-01-29 07:11:47 +00:00
|
|
|
return "", 0, err
|
2017-01-06 08:26:39 +00:00
|
|
|
}
|
2017-01-06 12:13:22 +00:00
|
|
|
|
2017-01-06 08:26:39 +00:00
|
|
|
i := uint64(0)
|
2017-01-06 12:13:22 +00:00
|
|
|
for _, n := range names {
|
2017-02-10 01:54:26 +00:00
|
|
|
if !strings.HasPrefix(n, prefix) {
|
2017-01-06 12:13:22 +00:00
|
|
|
continue
|
|
|
|
}
|
2017-02-10 01:54:26 +00:00
|
|
|
j, err := strconv.ParseUint(n[len(prefix):], 10, 32)
|
2017-01-06 12:13:22 +00:00
|
|
|
if err != nil {
|
|
|
|
continue
|
2017-01-06 08:26:39 +00:00
|
|
|
}
|
2017-01-06 12:13:22 +00:00
|
|
|
i = j
|
2017-01-06 08:26:39 +00:00
|
|
|
}
|
2017-02-10 01:54:26 +00:00
|
|
|
return filepath.Join(dir, fmt.Sprintf("%s%0.6d", prefix, i+1)), int(i + 1), nil
|
2017-01-06 08:26:39 +00:00
|
|
|
}
|
2016-12-09 12:41:38 +00:00
|
|
|
|
2016-12-10 17:08:50 +00:00
|
|
|
// The MultiError type implements the error interface, and contains the
|
|
|
|
// Errors used to construct it.
|
|
|
|
type MultiError []error
|
2016-12-07 16:10:49 +00:00
|
|
|
|
2016-12-10 17:08:50 +00:00
|
|
|
// Returns a concatenated string of the contained errors
|
|
|
|
func (es MultiError) Error() string {
|
|
|
|
var buf bytes.Buffer
|
2016-12-07 16:10:49 +00:00
|
|
|
|
2017-01-04 20:11:15 +00:00
|
|
|
if len(es) > 1 {
|
2016-12-10 17:08:50 +00:00
|
|
|
fmt.Fprintf(&buf, "%d errors: ", len(es))
|
2016-12-08 09:04:24 +00:00
|
|
|
}
|
2016-12-07 16:10:49 +00:00
|
|
|
|
2016-12-10 17:08:50 +00:00
|
|
|
for i, err := range es {
|
|
|
|
if i != 0 {
|
|
|
|
buf.WriteString("; ")
|
|
|
|
}
|
|
|
|
buf.WriteString(err.Error())
|
|
|
|
}
|
2016-12-07 16:10:49 +00:00
|
|
|
|
2016-12-10 17:08:50 +00:00
|
|
|
return buf.String()
|
2016-11-15 09:34:25 +00:00
|
|
|
}
|
2016-12-15 07:31:26 +00:00
|
|
|
|
2016-12-15 10:56:41 +00:00
|
|
|
// Add adds the error to the error list if it is not nil.
|
2016-12-31 14:35:08 +00: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-15 07:31:26 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2016-12-15 10:56:41 +00:00
|
|
|
// Err returns the error list as an error or nil if it is empty.
|
2016-12-15 07:31:26 +00:00
|
|
|
func (es MultiError) Err() error {
|
|
|
|
if len(es) == 0 {
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
return es
|
|
|
|
}
|
2016-12-15 10:56:41 +00:00
|
|
|
|
2017-05-17 14:43:01 +00:00
|
|
|
func yoloString(b []byte) string { return *((*string)(unsafe.Pointer(&b))) }
|
|
|
|
func yoloBytes(s string) []byte { return *((*[]byte)(unsafe.Pointer(&s))) }
|
2017-02-27 09:46:15 +00:00
|
|
|
|
|
|
|
func closeAll(cs ...io.Closer) error {
|
|
|
|
var merr MultiError
|
|
|
|
|
|
|
|
for _, c := range cs {
|
|
|
|
merr.Add(c.Close())
|
|
|
|
}
|
|
|
|
return merr.Err()
|
|
|
|
}
|