Switch to sequential block names
This changes block directory names from the int64 timestamp to sequential numbering.
This commit is contained in:
parent
4590b61343
commit
937cdb579c
50
block.go
50
block.go
|
@ -1,14 +1,10 @@
|
||||||
package tsdb
|
package tsdb
|
||||||
|
|
||||||
import (
|
import (
|
||||||
"fmt"
|
|
||||||
"io/ioutil"
|
|
||||||
"os"
|
"os"
|
||||||
"path/filepath"
|
"path/filepath"
|
||||||
"sort"
|
"sort"
|
||||||
"strconv"
|
|
||||||
|
|
||||||
"github.com/bradfitz/slice"
|
|
||||||
"github.com/coreos/etcd/pkg/fileutil"
|
"github.com/coreos/etcd/pkg/fileutil"
|
||||||
"github.com/pkg/errors"
|
"github.com/pkg/errors"
|
||||||
)
|
)
|
||||||
|
@ -104,54 +100,12 @@ func (pb *persistedBlock) interval() (int64, int64) {
|
||||||
return pb.bstats.MinTime, pb.bstats.MaxTime
|
return pb.bstats.MinTime, pb.bstats.MaxTime
|
||||||
}
|
}
|
||||||
|
|
||||||
// findBlocks finds time-ordered persisted blocks within a directory.
|
|
||||||
func findBlocks(path string) ([]*persistedBlock, []*HeadBlock, error) {
|
|
||||||
var (
|
|
||||||
pbs []*persistedBlock
|
|
||||||
heads []*HeadBlock
|
|
||||||
)
|
|
||||||
|
|
||||||
files, err := ioutil.ReadDir(path)
|
|
||||||
if err != nil {
|
|
||||||
return nil, nil, err
|
|
||||||
}
|
|
||||||
|
|
||||||
for _, fi := range files {
|
|
||||||
p := filepath.Join(path, fi.Name())
|
|
||||||
|
|
||||||
if _, err := os.Stat(chunksFileName(p)); os.IsNotExist(err) {
|
|
||||||
ts, err := strconv.Atoi(filepath.Base(p))
|
|
||||||
if err != nil {
|
|
||||||
return nil, nil, errors.Errorf("invalid directory name")
|
|
||||||
}
|
|
||||||
head, err := OpenHeadBlock(p, int64(ts))
|
|
||||||
if err != nil {
|
|
||||||
return nil, nil, err
|
|
||||||
}
|
|
||||||
heads = append(heads, head)
|
|
||||||
continue
|
|
||||||
}
|
|
||||||
|
|
||||||
pb, err := newPersistedBlock(p)
|
|
||||||
if err != nil {
|
|
||||||
return nil, nil, fmt.Errorf("error initializing block %q: %s", p, err)
|
|
||||||
}
|
|
||||||
pbs = append(pbs, pb)
|
|
||||||
}
|
|
||||||
|
|
||||||
// Order blocks by their base time so they represent a continous
|
|
||||||
// range of time.
|
|
||||||
slice.Sort(pbs, func(i, j int) bool { return pbs[i].bstats.MinTime < pbs[j].bstats.MinTime })
|
|
||||||
|
|
||||||
return pbs, heads, nil
|
|
||||||
}
|
|
||||||
|
|
||||||
func chunksFileName(path string) string {
|
func chunksFileName(path string) string {
|
||||||
return filepath.Join(path, "series")
|
return filepath.Join(path, "000-series")
|
||||||
}
|
}
|
||||||
|
|
||||||
func indexFileName(path string) string {
|
func indexFileName(path string) string {
|
||||||
return filepath.Join(path, "index")
|
return filepath.Join(path, "000-index")
|
||||||
}
|
}
|
||||||
|
|
||||||
type mmapFile struct {
|
type mmapFile struct {
|
||||||
|
|
222
db.go
222
db.go
|
@ -4,12 +4,13 @@ package tsdb
|
||||||
import (
|
import (
|
||||||
"bytes"
|
"bytes"
|
||||||
"fmt"
|
"fmt"
|
||||||
|
"io/ioutil"
|
||||||
|
"math"
|
||||||
"os"
|
"os"
|
||||||
"path/filepath"
|
"path/filepath"
|
||||||
"reflect"
|
"reflect"
|
||||||
"strconv"
|
"strconv"
|
||||||
"sync"
|
"sync"
|
||||||
"time"
|
|
||||||
"unsafe"
|
"unsafe"
|
||||||
|
|
||||||
"golang.org/x/sync/errgroup"
|
"golang.org/x/sync/errgroup"
|
||||||
|
@ -177,7 +178,7 @@ const sep = '\xff'
|
||||||
// Partition handles reads and writes of time series falling into
|
// Partition handles reads and writes of time series falling into
|
||||||
// a hashed partition of a series.
|
// a hashed partition of a series.
|
||||||
type Partition struct {
|
type Partition struct {
|
||||||
path string
|
dir string
|
||||||
logger log.Logger
|
logger log.Logger
|
||||||
metrics *partitionMetrics
|
metrics *partitionMetrics
|
||||||
|
|
||||||
|
@ -185,9 +186,6 @@ type Partition struct {
|
||||||
persisted []*persistedBlock
|
persisted []*persistedBlock
|
||||||
heads []*HeadBlock
|
heads []*HeadBlock
|
||||||
compactor *compactor
|
compactor *compactor
|
||||||
|
|
||||||
donec chan struct{}
|
|
||||||
cutc chan struct{}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
type partitionMetrics struct {
|
type partitionMetrics struct {
|
||||||
|
@ -231,86 +229,109 @@ func newPartitionMetrics(r prometheus.Registerer, i int) *partitionMetrics {
|
||||||
}
|
}
|
||||||
|
|
||||||
// OpenPartition returns a new Partition.
|
// OpenPartition returns a new Partition.
|
||||||
func OpenPartition(path string, i int, logger log.Logger) (*Partition, error) {
|
func OpenPartition(dir string, i int, logger log.Logger) (p *Partition, err error) {
|
||||||
// Create directory if partition is new.
|
// Create directory if partition is new.
|
||||||
if _, err := os.Stat(path); os.IsNotExist(err) {
|
if !fileutil.Exist(dir) {
|
||||||
if err := os.MkdirAll(path, 0777); err != nil {
|
if err := os.MkdirAll(dir, 0777); err != nil {
|
||||||
return nil, err
|
return nil, err
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
// Initialize previously persisted blocks.
|
p = &Partition{
|
||||||
persisted, heads, err := findBlocks(path)
|
dir: dir,
|
||||||
if err != nil {
|
logger: logger,
|
||||||
|
metrics: newPartitionMetrics(nil, i),
|
||||||
|
}
|
||||||
|
if err := p.initBlocks(); err != nil {
|
||||||
|
return nil, err
|
||||||
|
}
|
||||||
|
if p.compactor, err = newCompactor(i, p, logger); err != nil {
|
||||||
return nil, err
|
return nil, err
|
||||||
}
|
}
|
||||||
|
|
||||||
// TODO(fabxc): get time from client-defined `now` function.
|
return p, nil
|
||||||
baset := time.Unix(0, 0).UnixNano() / int64(time.Millisecond)
|
|
||||||
if len(persisted) > 0 {
|
|
||||||
baset = persisted[len(persisted)-1].bstats.MaxTime
|
|
||||||
}
|
|
||||||
if len(heads) == 0 {
|
|
||||||
head, err := OpenHeadBlock(filepath.Join(path, fmt.Sprintf("%d", baset)), baset)
|
|
||||||
if err != nil {
|
|
||||||
return nil, err
|
|
||||||
}
|
|
||||||
heads = []*HeadBlock{head}
|
|
||||||
}
|
|
||||||
|
|
||||||
s := &Partition{
|
|
||||||
path: path,
|
|
||||||
logger: logger,
|
|
||||||
metrics: newPartitionMetrics(nil, i),
|
|
||||||
heads: heads,
|
|
||||||
persisted: persisted,
|
|
||||||
cutc: make(chan struct{}, 1),
|
|
||||||
donec: make(chan struct{}),
|
|
||||||
}
|
|
||||||
if s.compactor, err = newCompactor(i, s, logger); err != nil {
|
|
||||||
return nil, err
|
|
||||||
}
|
|
||||||
go s.run()
|
|
||||||
|
|
||||||
return s, nil
|
|
||||||
}
|
}
|
||||||
|
|
||||||
func (s *Partition) run() {
|
func isBlockDir(fi os.FileInfo) bool {
|
||||||
for range s.cutc {
|
if !fi.IsDir() {
|
||||||
// if err := s.cut(); err != nil {
|
return false
|
||||||
// s.logger.Log("msg", "cut error", "err", err)
|
|
||||||
// }
|
|
||||||
// select {
|
|
||||||
// case <-s.cutc:
|
|
||||||
// default:
|
|
||||||
// }
|
|
||||||
// start := time.Now()
|
|
||||||
|
|
||||||
// if err := s.persist(); err != nil {
|
|
||||||
// s.logger.Log("msg", "persistence error", "err", err)
|
|
||||||
// }
|
|
||||||
|
|
||||||
// s.metrics.persistenceDuration.Observe(time.Since(start).Seconds())
|
|
||||||
// s.metrics.persistences.Inc()
|
|
||||||
}
|
}
|
||||||
close(s.donec)
|
if _, err := strconv.ParseUint(fi.Name(), 10, 32); err != nil {
|
||||||
|
return false
|
||||||
|
}
|
||||||
|
return true
|
||||||
|
}
|
||||||
|
|
||||||
|
func (p *Partition) initBlocks() error {
|
||||||
|
var (
|
||||||
|
pbs []*persistedBlock
|
||||||
|
heads []*HeadBlock
|
||||||
|
)
|
||||||
|
|
||||||
|
files, err := ioutil.ReadDir(p.dir)
|
||||||
|
if err != nil {
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
|
||||||
|
for _, fi := range files {
|
||||||
|
if !isBlockDir(fi) {
|
||||||
|
continue
|
||||||
|
}
|
||||||
|
dir := filepath.Join(p.dir, fi.Name())
|
||||||
|
|
||||||
|
if fileutil.Exist(filepath.Join(dir, walFileName)) {
|
||||||
|
h, err := OpenHeadBlock(dir)
|
||||||
|
if err != nil {
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
heads = append(heads, h)
|
||||||
|
continue
|
||||||
|
}
|
||||||
|
|
||||||
|
b, err := newPersistedBlock(dir)
|
||||||
|
if err != nil {
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
pbs = append(pbs, b)
|
||||||
|
}
|
||||||
|
|
||||||
|
// Validate that blocks are sequential in time.
|
||||||
|
lastTime := int64(math.MinInt64)
|
||||||
|
|
||||||
|
for _, b := range pbs {
|
||||||
|
if b.stats().MinTime < lastTime {
|
||||||
|
return errors.Errorf("illegal order for block at %q", b.dir())
|
||||||
|
}
|
||||||
|
lastTime = b.stats().MaxTime
|
||||||
|
}
|
||||||
|
for _, b := range heads {
|
||||||
|
if b.stats().MinTime < lastTime {
|
||||||
|
return errors.Errorf("illegal order for block at %q", b.dir())
|
||||||
|
}
|
||||||
|
lastTime = b.stats().MaxTime
|
||||||
|
}
|
||||||
|
|
||||||
|
p.persisted = pbs
|
||||||
|
p.heads = heads
|
||||||
|
|
||||||
|
if len(heads) == 0 {
|
||||||
|
return p.cut()
|
||||||
|
}
|
||||||
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
// Close the partition.
|
// Close the partition.
|
||||||
func (s *Partition) Close() error {
|
func (p *Partition) Close() error {
|
||||||
close(s.cutc)
|
|
||||||
<-s.donec
|
|
||||||
|
|
||||||
var merr MultiError
|
var merr MultiError
|
||||||
merr.Add(s.compactor.Close())
|
merr.Add(p.compactor.Close())
|
||||||
|
|
||||||
s.mtx.Lock()
|
p.mtx.Lock()
|
||||||
defer s.mtx.Unlock()
|
defer p.mtx.Unlock()
|
||||||
|
|
||||||
for _, pb := range s.persisted {
|
for _, pb := range p.persisted {
|
||||||
merr.Add(pb.Close())
|
merr.Add(pb.Close())
|
||||||
}
|
}
|
||||||
for _, hb := range s.heads {
|
for _, hb := range p.heads {
|
||||||
merr.Add(hb.Close())
|
merr.Add(hb.Close())
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -338,6 +359,8 @@ func (s *Partition) appendBatch(samples []hashedSample) error {
|
||||||
if head.bstats.SampleCount/(uint64(head.bstats.ChunkCount)+1) > 250 {
|
if head.bstats.SampleCount/(uint64(head.bstats.ChunkCount)+1) > 250 {
|
||||||
if err := s.cut(); err != nil {
|
if err := s.cut(); err != nil {
|
||||||
s.logger.Log("msg", "cut failed", "err", err)
|
s.logger.Log("msg", "cut failed", "err", err)
|
||||||
|
} else {
|
||||||
|
s.compactor.trigger()
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -444,7 +467,7 @@ func intervalContains(min, max, t int64) bool {
|
||||||
return t >= min && t <= max
|
return t >= min && t <= max
|
||||||
}
|
}
|
||||||
|
|
||||||
// blocksForRange returns all blocks within the partition that may contain
|
// blocksForInterval returns all blocks within the partition that may contain
|
||||||
// data for the given time range.
|
// data for the given time range.
|
||||||
func (s *Partition) blocksForInterval(mint, maxt int64) []block {
|
func (s *Partition) blocksForInterval(mint, maxt int64) []block {
|
||||||
var bs []block
|
var bs []block
|
||||||
|
@ -472,58 +495,33 @@ const headGracePeriod = 60 * 1000 // 60 seconds for millisecond scale
|
||||||
|
|
||||||
// cut starts a new head block to append to. The completed head block
|
// cut starts a new head block to append to. The completed head block
|
||||||
// will still be appendable for the configured grace period.
|
// will still be appendable for the configured grace period.
|
||||||
func (s *Partition) cut() error {
|
func (p *Partition) cut() error {
|
||||||
// Set new head block.
|
dir, err := p.nextBlockDir()
|
||||||
head := s.heads[len(s.heads)-1]
|
|
||||||
|
|
||||||
newHead, err := OpenHeadBlock(filepath.Join(s.path, fmt.Sprintf("%d", head.bstats.MaxTime)), head.bstats.MaxTime)
|
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
s.heads = append(s.heads, newHead)
|
newHead, err := OpenHeadBlock(dir)
|
||||||
|
if err != nil {
|
||||||
s.compactor.trigger()
|
return err
|
||||||
|
}
|
||||||
|
p.heads = append(p.heads, newHead)
|
||||||
|
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
// func (s *Partition) persist() error {
|
func (p *Partition) nextBlockDir() (string, error) {
|
||||||
// s.mtx.Lock()
|
names, err := fileutil.ReadDir(p.dir)
|
||||||
|
if err != nil {
|
||||||
// // Set new head block.
|
return "", err
|
||||||
// head := s.head
|
}
|
||||||
// newHead, err := OpenHeadBlock(filepath.Join(s.path, fmt.Sprintf("%d", head.bstats.MaxTime)), head.bstats.MaxTime)
|
i := uint64(0)
|
||||||
// if err != nil {
|
if len(names) > 0 {
|
||||||
// s.mtx.Unlock()
|
if i, err = strconv.ParseUint(names[len(names)-1], 10, 32); err != nil {
|
||||||
// return err
|
return "", err
|
||||||
// }
|
}
|
||||||
// s.head = newHead
|
}
|
||||||
|
return filepath.Join(p.dir, fmt.Sprintf("%0.6d", i+1)), nil
|
||||||
// s.mtx.Unlock()
|
}
|
||||||
|
|
||||||
// // TODO(fabxc): add grace period where we can still append to old head partition
|
|
||||||
// // before actually persisting it.
|
|
||||||
// dir := filepath.Join(s.path, fmt.Sprintf("%d", head.stats.MinTime))
|
|
||||||
|
|
||||||
// if err := persist(dir, head.persist); err != nil {
|
|
||||||
// return err
|
|
||||||
// }
|
|
||||||
// s.logger.Log("samples", head.stats.SampleCount, "chunks", head.stats.ChunkCount, "msg", "persisted head")
|
|
||||||
|
|
||||||
// // Reopen block as persisted block for querying.
|
|
||||||
// pb, err := newPersistedBlock(dir)
|
|
||||||
// if err != nil {
|
|
||||||
// return err
|
|
||||||
// }
|
|
||||||
|
|
||||||
// s.mtx.Lock()
|
|
||||||
// s.persisted = append(s.persisted, pb)
|
|
||||||
// s.mtx.Unlock()
|
|
||||||
|
|
||||||
// s.compactor.trigger()
|
|
||||||
|
|
||||||
// return nil
|
|
||||||
// }
|
|
||||||
|
|
||||||
// chunkDesc wraps a plain data chunk and provides cached meta data about it.
|
// chunkDesc wraps a plain data chunk and provides cached meta data about it.
|
||||||
type chunkDesc struct {
|
type chunkDesc struct {
|
||||||
|
|
24
head.go
24
head.go
|
@ -35,7 +35,7 @@ type HeadBlock struct {
|
||||||
}
|
}
|
||||||
|
|
||||||
// OpenHeadBlock creates a new empty head block.
|
// OpenHeadBlock creates a new empty head block.
|
||||||
func OpenHeadBlock(dir string, baseTime int64) (*HeadBlock, error) {
|
func OpenHeadBlock(dir string) (*HeadBlock, error) {
|
||||||
wal, err := OpenWAL(dir)
|
wal, err := OpenWAL(dir)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return nil, err
|
return nil, err
|
||||||
|
@ -49,14 +49,26 @@ func OpenHeadBlock(dir string, baseTime int64) (*HeadBlock, error) {
|
||||||
postings: &memPostings{m: make(map[term][]uint32)},
|
postings: &memPostings{m: make(map[term][]uint32)},
|
||||||
wal: wal,
|
wal: wal,
|
||||||
}
|
}
|
||||||
b.bstats.MinTime = baseTime
|
|
||||||
|
|
||||||
err = wal.ReadAll(&walHandler{
|
err = wal.ReadAll(&walHandler{
|
||||||
series: func(lset labels.Labels) {
|
series: func(lset labels.Labels) {
|
||||||
b.create(lset.Hash(), lset)
|
b.create(lset.Hash(), lset)
|
||||||
},
|
},
|
||||||
sample: func(s hashedSample) {
|
sample: func(s hashedSample) {
|
||||||
b.descs[s.ref].append(s.t, s.v)
|
cd := b.descs[s.ref]
|
||||||
|
|
||||||
|
// Duplicated from appendBatch – TODO(fabxc): deduplicate?
|
||||||
|
if cd.lastTimestamp == s.t && cd.lastValue != s.v {
|
||||||
|
return
|
||||||
|
}
|
||||||
|
cd.append(s.t, s.v)
|
||||||
|
|
||||||
|
if s.t > b.bstats.MaxTime {
|
||||||
|
b.bstats.MaxTime = s.t
|
||||||
|
}
|
||||||
|
if s.t < b.bstats.MinTime {
|
||||||
|
b.bstats.MinTime = s.t
|
||||||
|
}
|
||||||
b.bstats.SampleCount++
|
b.bstats.SampleCount++
|
||||||
},
|
},
|
||||||
})
|
})
|
||||||
|
@ -172,14 +184,13 @@ func (h *HeadBlock) get(hash uint64, lset labels.Labels) *chunkDesc {
|
||||||
}
|
}
|
||||||
|
|
||||||
func (h *HeadBlock) create(hash uint64, lset labels.Labels) *chunkDesc {
|
func (h *HeadBlock) create(hash uint64, lset labels.Labels) *chunkDesc {
|
||||||
var err error
|
|
||||||
|
|
||||||
cd := &chunkDesc{
|
cd := &chunkDesc{
|
||||||
lset: lset,
|
lset: lset,
|
||||||
chunk: chunks.NewXORChunk(),
|
chunk: chunks.NewXORChunk(),
|
||||||
lastTimestamp: math.MinInt64,
|
lastTimestamp: math.MinInt64,
|
||||||
}
|
}
|
||||||
|
|
||||||
|
var err error
|
||||||
cd.app, err = cd.chunk.Appender()
|
cd.app, err = cd.chunk.Appender()
|
||||||
if err != nil {
|
if err != nil {
|
||||||
// Getting an Appender for a new chunk must not panic.
|
// Getting an Appender for a new chunk must not panic.
|
||||||
|
@ -289,6 +300,9 @@ func (h *HeadBlock) appendBatch(samples []hashedSample) error {
|
||||||
if s.t > h.bstats.MaxTime {
|
if s.t > h.bstats.MaxTime {
|
||||||
h.bstats.MaxTime = s.t
|
h.bstats.MaxTime = s.t
|
||||||
}
|
}
|
||||||
|
if s.t < h.bstats.MinTime {
|
||||||
|
h.bstats.MinTime = s.t
|
||||||
|
}
|
||||||
h.bstats.SampleCount++
|
h.bstats.SampleCount++
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
4
wal.go
4
wal.go
|
@ -33,6 +33,8 @@ type WAL struct {
|
||||||
symbols map[string]uint32
|
symbols map[string]uint32
|
||||||
}
|
}
|
||||||
|
|
||||||
|
const walFileName = "000-wal"
|
||||||
|
|
||||||
// OpenWAL opens or creates a write ahead log in the given directory.
|
// OpenWAL opens or creates a write ahead log in the given directory.
|
||||||
// The WAL must be read completely before new data is written.
|
// The WAL must be read completely before new data is written.
|
||||||
func OpenWAL(dir string) (*WAL, error) {
|
func OpenWAL(dir string) (*WAL, error) {
|
||||||
|
@ -40,7 +42,7 @@ func OpenWAL(dir string) (*WAL, error) {
|
||||||
return nil, err
|
return nil, err
|
||||||
}
|
}
|
||||||
|
|
||||||
p := filepath.Join(dir, "wal")
|
p := filepath.Join(dir, walFileName)
|
||||||
|
|
||||||
f, err := fileutil.TryLockFile(p, os.O_RDWR, 0666)
|
f, err := fileutil.TryLockFile(p, os.O_RDWR, 0666)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
|
|
|
@ -177,7 +177,7 @@ func BenchmarkWALReadIntoHead(b *testing.B) {
|
||||||
|
|
||||||
b.ResetTimer()
|
b.ResetTimer()
|
||||||
|
|
||||||
_, err = OpenHeadBlock(d, 0)
|
_, err = OpenHeadBlock(d)
|
||||||
require.NoError(b, err)
|
require.NoError(b, err)
|
||||||
|
|
||||||
// stat, _ := head.wal.f.Stat()
|
// stat, _ := head.wal.f.Stat()
|
||||||
|
|
Loading…
Reference in New Issue