Move WAL watcher code to tsdb/wal package. (#5999)

* Move WAL watcher code to tsdb/wal package.

Signed-off-by: Callum Styan <callumstyan@gmail.com>

* Fix tests after moving WAL watcher code.

Signed-off-by: Callum Styan <callumstyan@gmail.com>

* Lint fixes.

Signed-off-by: Callum Styan <callumstyan@gmail.com>
This commit is contained in:
Callum Styan 2019-09-19 02:15:41 -07:00 committed by Ganesh Vernekar
parent 8744afdd1e
commit 3344bb5c33
23 changed files with 734 additions and 640 deletions

View file

@ -32,8 +32,9 @@ import (
"github.com/prometheus/prometheus/pkg/labels" "github.com/prometheus/prometheus/pkg/labels"
"github.com/prometheus/prometheus/pkg/relabel" "github.com/prometheus/prometheus/pkg/relabel"
"github.com/prometheus/prometheus/prompb" "github.com/prometheus/prometheus/prompb"
"github.com/prometheus/prometheus/tsdb"
tsdbLabels "github.com/prometheus/prometheus/tsdb/labels" tsdbLabels "github.com/prometheus/prometheus/tsdb/labels"
"github.com/prometheus/prometheus/tsdb/record"
"github.com/prometheus/prometheus/tsdb/wal"
) )
// String constants for instrumentation. // String constants for instrumentation.
@ -191,7 +192,7 @@ type QueueManager struct {
externalLabels labels.Labels externalLabels labels.Labels
relabelConfigs []*relabel.Config relabelConfigs []*relabel.Config
client StorageClient client StorageClient
watcher *WALWatcher watcher *wal.Watcher
seriesLabels map[uint64]labels.Labels seriesLabels map[uint64]labels.Labels
seriesSegmentIndexes map[uint64]int seriesSegmentIndexes map[uint64]int
@ -223,7 +224,7 @@ type QueueManager struct {
} }
// NewQueueManager builds a new QueueManager. // NewQueueManager builds a new QueueManager.
func NewQueueManager(logger log.Logger, walDir string, samplesIn *ewmaRate, cfg config.QueueConfig, externalLabels labels.Labels, relabelConfigs []*relabel.Config, client StorageClient, flushDeadline time.Duration) *QueueManager { func NewQueueManager(reg prometheus.Registerer, logger log.Logger, walDir string, samplesIn *ewmaRate, cfg config.QueueConfig, externalLabels labels.Labels, relabelConfigs []*relabel.Config, client StorageClient, flushDeadline time.Duration) *QueueManager {
if logger == nil { if logger == nil {
logger = log.NewNopLogger() logger = log.NewNopLogger()
} }
@ -252,7 +253,7 @@ func NewQueueManager(logger log.Logger, walDir string, samplesIn *ewmaRate, cfg
samplesOutDuration: newEWMARate(ewmaWeight, shardUpdateDuration), samplesOutDuration: newEWMARate(ewmaWeight, shardUpdateDuration),
} }
t.watcher = NewWALWatcher(logger, name, t, walDir) t.watcher = wal.NewWatcher(reg, wal.NewWatcherMetrics(reg), logger, name, t, walDir)
t.shards = t.newShards() t.shards = t.newShards()
return t return t
@ -260,7 +261,7 @@ func NewQueueManager(logger log.Logger, walDir string, samplesIn *ewmaRate, cfg
// Append queues a sample to be sent to the remote storage. Blocks until all samples are // Append queues a sample to be sent to the remote storage. Blocks until all samples are
// enqueued on their shards or a shutdown signal is received. // enqueued on their shards or a shutdown signal is received.
func (t *QueueManager) Append(samples []tsdb.RefSample) bool { func (t *QueueManager) Append(samples []record.RefSample) bool {
outer: outer:
for _, s := range samples { for _, s := range samples {
lbls, ok := t.seriesLabels[s.Ref] lbls, ok := t.seriesLabels[s.Ref]
@ -376,7 +377,7 @@ func (t *QueueManager) Stop() {
} }
// StoreSeries keeps track of which series we know about for lookups when sending samples to remote. // StoreSeries keeps track of which series we know about for lookups when sending samples to remote.
func (t *QueueManager) StoreSeries(series []tsdb.RefSeries, index int) { func (t *QueueManager) StoreSeries(series []record.RefSeries, index int) {
for _, s := range series { for _, s := range series {
ls := processExternalLabels(s.Labels, t.externalLabels) ls := processExternalLabels(s.Labels, t.externalLabels)
lbls := relabel.Process(ls, t.relabelConfigs...) lbls := relabel.Process(ls, t.relabelConfigs...)

View file

@ -36,8 +36,8 @@ import (
"github.com/prometheus/prometheus/config" "github.com/prometheus/prometheus/config"
"github.com/prometheus/prometheus/pkg/labels" "github.com/prometheus/prometheus/pkg/labels"
"github.com/prometheus/prometheus/prompb" "github.com/prometheus/prometheus/prompb"
"github.com/prometheus/prometheus/tsdb"
tsdbLabels "github.com/prometheus/prometheus/tsdb/labels" tsdbLabels "github.com/prometheus/prometheus/tsdb/labels"
"github.com/prometheus/prometheus/tsdb/record"
"github.com/prometheus/prometheus/util/testutil" "github.com/prometheus/prometheus/util/testutil"
) )
@ -60,7 +60,7 @@ func TestSampleDelivery(t *testing.T) {
testutil.Ok(t, err) testutil.Ok(t, err)
defer os.RemoveAll(dir) defer os.RemoveAll(dir)
m := NewQueueManager(nil, dir, newEWMARate(ewmaWeight, shardUpdateDuration), cfg, nil, nil, c, defaultFlushDeadline) m := NewQueueManager(nil, nil, dir, newEWMARate(ewmaWeight, shardUpdateDuration), cfg, nil, nil, c, defaultFlushDeadline)
m.StoreSeries(series, 0) m.StoreSeries(series, 0)
// These should be received by the client. // These should be received by the client.
@ -88,7 +88,7 @@ func TestSampleDeliveryTimeout(t *testing.T) {
testutil.Ok(t, err) testutil.Ok(t, err)
defer os.RemoveAll(dir) defer os.RemoveAll(dir)
m := NewQueueManager(nil, dir, newEWMARate(ewmaWeight, shardUpdateDuration), cfg, nil, nil, c, defaultFlushDeadline) m := NewQueueManager(nil, nil, dir, newEWMARate(ewmaWeight, shardUpdateDuration), cfg, nil, nil, c, defaultFlushDeadline)
m.StoreSeries(series, 0) m.StoreSeries(series, 0)
m.Start() m.Start()
defer m.Stop() defer m.Stop()
@ -106,16 +106,16 @@ func TestSampleDeliveryTimeout(t *testing.T) {
func TestSampleDeliveryOrder(t *testing.T) { func TestSampleDeliveryOrder(t *testing.T) {
ts := 10 ts := 10
n := config.DefaultQueueConfig.MaxSamplesPerSend * ts n := config.DefaultQueueConfig.MaxSamplesPerSend * ts
samples := make([]tsdb.RefSample, 0, n) samples := make([]record.RefSample, 0, n)
series := make([]tsdb.RefSeries, 0, n) series := make([]record.RefSeries, 0, n)
for i := 0; i < n; i++ { for i := 0; i < n; i++ {
name := fmt.Sprintf("test_metric_%d", i%ts) name := fmt.Sprintf("test_metric_%d", i%ts)
samples = append(samples, tsdb.RefSample{ samples = append(samples, record.RefSample{
Ref: uint64(i), Ref: uint64(i),
T: int64(i), T: int64(i),
V: float64(i), V: float64(i),
}) })
series = append(series, tsdb.RefSeries{ series = append(series, record.RefSeries{
Ref: uint64(i), Ref: uint64(i),
Labels: tsdbLabels.Labels{tsdbLabels.Label{Name: "__name__", Value: name}}, Labels: tsdbLabels.Labels{tsdbLabels.Label{Name: "__name__", Value: name}},
}) })
@ -128,7 +128,7 @@ func TestSampleDeliveryOrder(t *testing.T) {
testutil.Ok(t, err) testutil.Ok(t, err)
defer os.RemoveAll(dir) defer os.RemoveAll(dir)
m := NewQueueManager(nil, dir, newEWMARate(ewmaWeight, shardUpdateDuration), config.DefaultQueueConfig, nil, nil, c, defaultFlushDeadline) m := NewQueueManager(nil, nil, dir, newEWMARate(ewmaWeight, shardUpdateDuration), config.DefaultQueueConfig, nil, nil, c, defaultFlushDeadline)
m.StoreSeries(series, 0) m.StoreSeries(series, 0)
m.Start() m.Start()
@ -146,7 +146,7 @@ func TestShutdown(t *testing.T) {
testutil.Ok(t, err) testutil.Ok(t, err)
defer os.RemoveAll(dir) defer os.RemoveAll(dir)
m := NewQueueManager(nil, dir, newEWMARate(ewmaWeight, shardUpdateDuration), config.DefaultQueueConfig, nil, nil, c, deadline) m := NewQueueManager(nil, nil, dir, newEWMARate(ewmaWeight, shardUpdateDuration), config.DefaultQueueConfig, nil, nil, c, deadline)
samples, series := createTimeseries(2 * config.DefaultQueueConfig.MaxSamplesPerSend) samples, series := createTimeseries(2 * config.DefaultQueueConfig.MaxSamplesPerSend)
m.StoreSeries(series, 0) m.StoreSeries(series, 0)
m.Start() m.Start()
@ -182,11 +182,11 @@ func TestSeriesReset(t *testing.T) {
testutil.Ok(t, err) testutil.Ok(t, err)
defer os.RemoveAll(dir) defer os.RemoveAll(dir)
m := NewQueueManager(nil, dir, newEWMARate(ewmaWeight, shardUpdateDuration), config.DefaultQueueConfig, nil, nil, c, deadline) m := NewQueueManager(nil, nil, dir, newEWMARate(ewmaWeight, shardUpdateDuration), config.DefaultQueueConfig, nil, nil, c, deadline)
for i := 0; i < numSegments; i++ { for i := 0; i < numSegments; i++ {
series := []tsdb.RefSeries{} series := []record.RefSeries{}
for j := 0; j < numSeries; j++ { for j := 0; j < numSeries; j++ {
series = append(series, tsdb.RefSeries{Ref: uint64((i * 100) + j), Labels: tsdbLabels.Labels{{Name: "a", Value: "a"}}}) series = append(series, record.RefSeries{Ref: uint64((i * 100) + j), Labels: tsdbLabels.Labels{{Name: "a", Value: "a"}}})
} }
m.StoreSeries(series, i) m.StoreSeries(series, i)
} }
@ -210,7 +210,7 @@ func TestReshard(t *testing.T) {
testutil.Ok(t, err) testutil.Ok(t, err)
defer os.RemoveAll(dir) defer os.RemoveAll(dir)
m := NewQueueManager(nil, dir, newEWMARate(ewmaWeight, shardUpdateDuration), cfg, nil, nil, c, defaultFlushDeadline) m := NewQueueManager(nil, nil, dir, newEWMARate(ewmaWeight, shardUpdateDuration), cfg, nil, nil, c, defaultFlushDeadline)
m.StoreSeries(series, 0) m.StoreSeries(series, 0)
m.Start() m.Start()
@ -242,7 +242,7 @@ func TestReshardRaceWithStop(t *testing.T) {
go func() { go func() {
for { for {
m = NewQueueManager(nil, "", newEWMARate(ewmaWeight, shardUpdateDuration), config.DefaultQueueConfig, nil, nil, c, defaultFlushDeadline) m = NewQueueManager(nil, nil, "", newEWMARate(ewmaWeight, shardUpdateDuration), config.DefaultQueueConfig, nil, nil, c, defaultFlushDeadline)
m.Start() m.Start()
h.Unlock() h.Unlock()
h.Lock() h.Lock()
@ -259,11 +259,11 @@ func TestReshardRaceWithStop(t *testing.T) {
func TestReleaseNoninternedString(t *testing.T) { func TestReleaseNoninternedString(t *testing.T) {
c := NewTestStorageClient() c := NewTestStorageClient()
m := NewQueueManager(nil, "", newEWMARate(ewmaWeight, shardUpdateDuration), config.DefaultQueueConfig, nil, nil, c, defaultFlushDeadline) m := NewQueueManager(nil, nil, "", newEWMARate(ewmaWeight, shardUpdateDuration), config.DefaultQueueConfig, nil, nil, c, defaultFlushDeadline)
m.Start() m.Start()
for i := 1; i < 1000; i++ { for i := 1; i < 1000; i++ {
m.StoreSeries([]tsdb.RefSeries{ m.StoreSeries([]record.RefSeries{
{ {
Ref: uint64(i), Ref: uint64(i),
Labels: tsdbLabels.Labels{ Labels: tsdbLabels.Labels{
@ -281,17 +281,17 @@ func TestReleaseNoninternedString(t *testing.T) {
testutil.Assert(t, metric == 0, "expected there to be no calls to release for strings that were not already interned: %d", int(metric)) testutil.Assert(t, metric == 0, "expected there to be no calls to release for strings that were not already interned: %d", int(metric))
} }
func createTimeseries(n int) ([]tsdb.RefSample, []tsdb.RefSeries) { func createTimeseries(n int) ([]record.RefSample, []record.RefSeries) {
samples := make([]tsdb.RefSample, 0, n) samples := make([]record.RefSample, 0, n)
series := make([]tsdb.RefSeries, 0, n) series := make([]record.RefSeries, 0, n)
for i := 0; i < n; i++ { for i := 0; i < n; i++ {
name := fmt.Sprintf("test_metric_%d", i) name := fmt.Sprintf("test_metric_%d", i)
samples = append(samples, tsdb.RefSample{ samples = append(samples, record.RefSample{
Ref: uint64(i), Ref: uint64(i),
T: int64(i), T: int64(i),
V: float64(i), V: float64(i),
}) })
series = append(series, tsdb.RefSeries{ series = append(series, record.RefSeries{
Ref: uint64(i), Ref: uint64(i),
Labels: tsdbLabels.Labels{{Name: "__name__", Value: name}}, Labels: tsdbLabels.Labels{{Name: "__name__", Value: name}},
}) })
@ -299,7 +299,7 @@ func createTimeseries(n int) ([]tsdb.RefSample, []tsdb.RefSeries) {
return samples, series return samples, series
} }
func getSeriesNameFromRef(r tsdb.RefSeries) string { func getSeriesNameFromRef(r record.RefSeries) string {
for _, l := range r.Labels { for _, l := range r.Labels {
if l.Name == "__name__" { if l.Name == "__name__" {
return l.Value return l.Value
@ -323,7 +323,7 @@ func NewTestStorageClient() *TestStorageClient {
} }
} }
func (c *TestStorageClient) expectSamples(ss []tsdb.RefSample, series []tsdb.RefSeries) { func (c *TestStorageClient) expectSamples(ss []record.RefSample, series []record.RefSeries) {
c.mtx.Lock() c.mtx.Lock()
defer c.mtx.Unlock() defer c.mtx.Unlock()
@ -351,7 +351,7 @@ func (c *TestStorageClient) waitForExpectedSamples(tb testing.TB) {
} }
} }
func (c *TestStorageClient) expectSampleCount(ss []tsdb.RefSample) { func (c *TestStorageClient) expectSampleCount(ss []record.RefSample) {
c.mtx.Lock() c.mtx.Lock()
defer c.mtx.Unlock() defer c.mtx.Unlock()
c.wg.Add(len(ss)) c.wg.Add(len(ss))
@ -443,7 +443,7 @@ func BenchmarkSampleDelivery(b *testing.B) {
testutil.Ok(b, err) testutil.Ok(b, err)
defer os.RemoveAll(dir) defer os.RemoveAll(dir)
m := NewQueueManager(nil, dir, newEWMARate(ewmaWeight, shardUpdateDuration), cfg, nil, nil, c, defaultFlushDeadline) m := NewQueueManager(nil, nil, dir, newEWMARate(ewmaWeight, shardUpdateDuration), cfg, nil, nil, c, defaultFlushDeadline)
m.StoreSeries(series, 0) m.StoreSeries(series, 0)
// These should be received by the client. // These should be received by the client.
@ -484,12 +484,12 @@ func BenchmarkStartup(b *testing.B) {
for n := 0; n < b.N; n++ { for n := 0; n < b.N; n++ {
c := NewTestBlockedStorageClient() c := NewTestBlockedStorageClient()
m := NewQueueManager(logger, dir, m := NewQueueManager(nil, logger, dir,
newEWMARate(ewmaWeight, shardUpdateDuration), newEWMARate(ewmaWeight, shardUpdateDuration),
config.DefaultQueueConfig, nil, nil, c, 1*time.Minute) config.DefaultQueueConfig, nil, nil, c, 1*time.Minute)
m.watcher.startTime = math.MaxInt64 m.watcher.StartTime = math.MaxInt64
m.watcher.maxSegment = segments[len(segments)-2] m.watcher.MaxSegment = segments[len(segments)-2]
err := m.watcher.run() err := m.watcher.Run()
testutil.Ok(b, err) testutil.Ok(b, err)
} }
} }

View file

@ -141,6 +141,7 @@ func (rws *WriteStorage) ApplyConfig(conf *config.Config) error {
return err return err
} }
newQueues = append(newQueues, NewQueueManager( newQueues = append(newQueues, NewQueueManager(
prometheus.DefaultRegisterer,
rws.logger, rws.logger,
rws.walDir, rws.walDir,
rws.samplesIn, rws.samplesIn,

View file

@ -32,6 +32,7 @@ import (
"github.com/prometheus/prometheus/tsdb/fileutil" "github.com/prometheus/prometheus/tsdb/fileutil"
"github.com/prometheus/prometheus/tsdb/index" "github.com/prometheus/prometheus/tsdb/index"
"github.com/prometheus/prometheus/tsdb/labels" "github.com/prometheus/prometheus/tsdb/labels"
"github.com/prometheus/prometheus/tsdb/tombstones"
) )
// IndexWriter serializes the index for a block of series data. // IndexWriter serializes the index for a block of series data.
@ -135,8 +136,8 @@ type BlockReader interface {
// Chunks returns a ChunkReader over the block's data. // Chunks returns a ChunkReader over the block's data.
Chunks() (ChunkReader, error) Chunks() (ChunkReader, error)
// Tombstones returns a TombstoneReader over the block's deleted data. // Tombstones returns a tombstones.Reader over the block's deleted data.
Tombstones() (TombstoneReader, error) Tombstones() (tombstones.Reader, error)
// Meta provides meta information about the block reader. // Meta provides meta information about the block reader.
Meta() BlockMeta Meta() BlockMeta
@ -279,7 +280,7 @@ type Block struct {
chunkr ChunkReader chunkr ChunkReader
indexr IndexReader indexr IndexReader
tombstones TombstoneReader tombstones tombstones.Reader
logger log.Logger logger log.Logger
@ -321,7 +322,7 @@ func OpenBlock(logger log.Logger, dir string, pool chunkenc.Pool) (pb *Block, er
} }
closers = append(closers, ir) closers = append(closers, ir)
tr, sizeTomb, err := readTombstones(dir) tr, sizeTomb, err := tombstones.ReadTombstones(dir)
if err != nil { if err != nil {
return nil, err return nil, err
} }
@ -412,11 +413,11 @@ func (pb *Block) Chunks() (ChunkReader, error) {
} }
// Tombstones returns a new TombstoneReader against the block data. // Tombstones returns a new TombstoneReader against the block data.
func (pb *Block) Tombstones() (TombstoneReader, error) { func (pb *Block) Tombstones() (tombstones.Reader, error) {
if err := pb.startRead(); err != nil { if err := pb.startRead(); err != nil {
return nil, err return nil, err
} }
return blockTombstoneReader{TombstoneReader: pb.tombstones, b: pb}, nil return blockTombstoneReader{Reader: pb.tombstones, b: pb}, nil
} }
// GetSymbolTableSize returns the Symbol Table Size in the index of this block. // GetSymbolTableSize returns the Symbol Table Size in the index of this block.
@ -483,7 +484,7 @@ func (r blockIndexReader) Close() error {
} }
type blockTombstoneReader struct { type blockTombstoneReader struct {
TombstoneReader tombstones.Reader
b *Block b *Block
} }
@ -519,7 +520,7 @@ func (pb *Block) Delete(mint, maxt int64, ms ...labels.Matcher) error {
ir := pb.indexr ir := pb.indexr
// Choose only valid postings which have chunks in the time-range. // Choose only valid postings which have chunks in the time-range.
stones := newMemTombstones() stones := tombstones.NewMemTombstones()
var lset labels.Labels var lset labels.Labels
var chks []chunks.Meta var chks []chunks.Meta
@ -535,7 +536,7 @@ Outer:
if chk.OverlapsClosedInterval(mint, maxt) { if chk.OverlapsClosedInterval(mint, maxt) {
// Delete only until the current values and not beyond. // Delete only until the current values and not beyond.
tmin, tmax := clampInterval(mint, maxt, chks[0].MinTime, chks[len(chks)-1].MaxTime) tmin, tmax := clampInterval(mint, maxt, chks[0].MinTime, chks[len(chks)-1].MaxTime)
stones.addInterval(p.At(), Interval{tmin, tmax}) stones.AddInterval(p.At(), tombstones.Interval{Mint: tmin, Maxt: tmax})
continue Outer continue Outer
} }
} }
@ -545,9 +546,9 @@ Outer:
return p.Err() return p.Err()
} }
err = pb.tombstones.Iter(func(id uint64, ivs Intervals) error { err = pb.tombstones.Iter(func(id uint64, ivs tombstones.Intervals) error {
for _, iv := range ivs { for _, iv := range ivs {
stones.addInterval(id, iv) stones.AddInterval(id, iv)
} }
return nil return nil
}) })
@ -557,7 +558,7 @@ Outer:
pb.tombstones = stones pb.tombstones = stones
pb.meta.Stats.NumTombstones = pb.tombstones.Total() pb.meta.Stats.NumTombstones = pb.tombstones.Total()
n, err := writeTombstoneFile(pb.logger, pb.dir, pb.tombstones) n, err := tombstones.WriteFile(pb.logger, pb.dir, pb.tombstones)
if err != nil { if err != nil {
return err return err
} }
@ -575,7 +576,7 @@ Outer:
func (pb *Block) CleanTombstones(dest string, c Compactor) (*ulid.ULID, error) { func (pb *Block) CleanTombstones(dest string, c Compactor) (*ulid.ULID, error) {
numStones := 0 numStones := 0
if err := pb.tombstones.Iter(func(id uint64, ivs Intervals) error { if err := pb.tombstones.Iter(func(id uint64, ivs tombstones.Intervals) error {
numStones += len(ivs) numStones += len(ivs)
return nil return nil
}); err != nil { }); err != nil {
@ -610,7 +611,7 @@ func (pb *Block) Snapshot(dir string) error {
for _, fname := range []string{ for _, fname := range []string{
metaFilename, metaFilename,
indexFilename, indexFilename,
tombstoneFilename, tombstones.TombstonesFilename,
} { } {
if err := os.Link(filepath.Join(pb.dir, fname), filepath.Join(blockDir, fname)); err != nil { if err := os.Link(filepath.Join(pb.dir, fname), filepath.Join(blockDir, fname)); err != nil {
return errors.Wrapf(err, "create snapshot %s", fname) return errors.Wrapf(err, "create snapshot %s", fname)

View file

@ -35,6 +35,7 @@ import (
"github.com/prometheus/prometheus/tsdb/fileutil" "github.com/prometheus/prometheus/tsdb/fileutil"
"github.com/prometheus/prometheus/tsdb/index" "github.com/prometheus/prometheus/tsdb/index"
"github.com/prometheus/prometheus/tsdb/labels" "github.com/prometheus/prometheus/tsdb/labels"
"github.com/prometheus/prometheus/tsdb/tombstones"
) )
// ExponentialBlockRanges returns the time ranges based on the stepSize. // ExponentialBlockRanges returns the time ranges based on the stepSize.
@ -607,7 +608,7 @@ func (c *LeveledCompactor) write(dest string, meta *BlockMeta, blocks ...BlockRe
} }
// Create an empty tombstones file. // Create an empty tombstones file.
if _, err := writeTombstoneFile(c.logger, tmp, newMemTombstones()); err != nil { if _, err := tombstones.WriteFile(c.logger, tmp, tombstones.NewMemTombstones()); err != nil {
return errors.Wrap(err, "write new tombstones file") return errors.Wrap(err, "write new tombstones file")
} }
@ -768,7 +769,7 @@ func (c *LeveledCompactor) populateBlock(blocks []BlockReader, meta *BlockMeta,
// //
// TODO think how to avoid the typecasting to verify when it is head block. // TODO think how to avoid the typecasting to verify when it is head block.
if _, isHeadChunk := chk.Chunk.(*safeChunk); isHeadChunk && chk.MaxTime >= meta.MaxTime { if _, isHeadChunk := chk.Chunk.(*safeChunk); isHeadChunk && chk.MaxTime >= meta.MaxTime {
dranges = append(dranges, Interval{Mint: meta.MaxTime, Maxt: math.MaxInt64}) dranges = append(dranges, tombstones.Interval{Mint: meta.MaxTime, Maxt: math.MaxInt64})
} else } else
// Sanity check for disk blocks. // Sanity check for disk blocks.
@ -876,15 +877,15 @@ type compactionSeriesSet struct {
p index.Postings p index.Postings
index IndexReader index IndexReader
chunks ChunkReader chunks ChunkReader
tombstones TombstoneReader tombstones tombstones.Reader
l labels.Labels l labels.Labels
c []chunks.Meta c []chunks.Meta
intervals Intervals intervals tombstones.Intervals
err error err error
} }
func newCompactionSeriesSet(i IndexReader, c ChunkReader, t TombstoneReader, p index.Postings) *compactionSeriesSet { func newCompactionSeriesSet(i IndexReader, c ChunkReader, t tombstones.Reader, p index.Postings) *compactionSeriesSet {
return &compactionSeriesSet{ return &compactionSeriesSet{
index: i, index: i,
chunks: c, chunks: c,
@ -914,7 +915,7 @@ func (c *compactionSeriesSet) Next() bool {
if len(c.intervals) > 0 { if len(c.intervals) > 0 {
chks := make([]chunks.Meta, 0, len(c.c)) chks := make([]chunks.Meta, 0, len(c.c))
for _, chk := range c.c { for _, chk := range c.c {
if !(Interval{chk.MinTime, chk.MaxTime}.isSubrange(c.intervals)) { if !(tombstones.Interval{Mint: chk.MinTime, Maxt: chk.MaxTime}.IsSubrange(c.intervals)) {
chks = append(chks, chk) chks = append(chks, chk)
} }
} }
@ -942,7 +943,7 @@ func (c *compactionSeriesSet) Err() error {
return c.p.Err() return c.p.Err()
} }
func (c *compactionSeriesSet) At() (labels.Labels, []chunks.Meta, Intervals) { func (c *compactionSeriesSet) At() (labels.Labels, []chunks.Meta, tombstones.Intervals) {
return c.l, c.c, c.intervals return c.l, c.c, c.intervals
} }
@ -952,7 +953,7 @@ type compactionMerger struct {
aok, bok bool aok, bok bool
l labels.Labels l labels.Labels
c []chunks.Meta c []chunks.Meta
intervals Intervals intervals tombstones.Intervals
} }
func newCompactionMerger(a, b ChunkSeriesSet) (*compactionMerger, error) { func newCompactionMerger(a, b ChunkSeriesSet) (*compactionMerger, error) {
@ -1008,7 +1009,7 @@ func (c *compactionMerger) Next() bool {
_, cb, rb := c.b.At() _, cb, rb := c.b.At()
for _, r := range rb { for _, r := range rb {
ra = ra.add(r) ra = ra.Add(r)
} }
c.l = append(c.l[:0], l...) c.l = append(c.l[:0], l...)
@ -1029,6 +1030,6 @@ func (c *compactionMerger) Err() error {
return c.b.Err() return c.b.Err()
} }
func (c *compactionMerger) At() (labels.Labels, []chunks.Meta, Intervals) { func (c *compactionMerger) At() (labels.Labels, []chunks.Meta, tombstones.Intervals) {
return c.l, c.c, c.intervals return c.l, c.c, c.intervals
} }

View file

@ -30,6 +30,7 @@ import (
"github.com/prometheus/prometheus/tsdb/chunks" "github.com/prometheus/prometheus/tsdb/chunks"
"github.com/prometheus/prometheus/tsdb/fileutil" "github.com/prometheus/prometheus/tsdb/fileutil"
"github.com/prometheus/prometheus/tsdb/labels" "github.com/prometheus/prometheus/tsdb/labels"
"github.com/prometheus/prometheus/tsdb/tombstones"
"github.com/prometheus/prometheus/util/testutil" "github.com/prometheus/prometheus/util/testutil"
) )
@ -455,10 +456,10 @@ func metaRange(name string, mint, maxt int64, stats *BlockStats) dirMeta {
type erringBReader struct{} type erringBReader struct{}
func (erringBReader) Index() (IndexReader, error) { return nil, errors.New("index") } func (erringBReader) Index() (IndexReader, error) { return nil, errors.New("index") }
func (erringBReader) Chunks() (ChunkReader, error) { return nil, errors.New("chunks") } func (erringBReader) Chunks() (ChunkReader, error) { return nil, errors.New("chunks") }
func (erringBReader) Tombstones() (TombstoneReader, error) { return nil, errors.New("tombstones") } func (erringBReader) Tombstones() (tombstones.Reader, error) { return nil, errors.New("tombstones") }
func (erringBReader) Meta() BlockMeta { return BlockMeta{} } func (erringBReader) Meta() BlockMeta { return BlockMeta{} }
type nopChunkWriter struct{} type nopChunkWriter struct{}

View file

@ -33,6 +33,8 @@ import (
"github.com/prometheus/prometheus/tsdb/chunks" "github.com/prometheus/prometheus/tsdb/chunks"
"github.com/prometheus/prometheus/tsdb/index" "github.com/prometheus/prometheus/tsdb/index"
"github.com/prometheus/prometheus/tsdb/labels" "github.com/prometheus/prometheus/tsdb/labels"
"github.com/prometheus/prometheus/tsdb/record"
"github.com/prometheus/prometheus/tsdb/tombstones"
"github.com/prometheus/prometheus/tsdb/tsdbutil" "github.com/prometheus/prometheus/tsdb/tsdbutil"
"github.com/prometheus/prometheus/tsdb/wal" "github.com/prometheus/prometheus/tsdb/wal"
"github.com/prometheus/prometheus/util/testutil" "github.com/prometheus/prometheus/util/testutil"
@ -243,27 +245,27 @@ func TestDeleteSimple(t *testing.T) {
numSamples := int64(10) numSamples := int64(10)
cases := []struct { cases := []struct {
intervals Intervals Intervals tombstones.Intervals
remaint []int64 remaint []int64
}{ }{
{ {
intervals: Intervals{{0, 3}}, Intervals: tombstones.Intervals{{Mint: 0, Maxt: 3}},
remaint: []int64{4, 5, 6, 7, 8, 9}, remaint: []int64{4, 5, 6, 7, 8, 9},
}, },
{ {
intervals: Intervals{{1, 3}}, Intervals: tombstones.Intervals{{Mint: 1, Maxt: 3}},
remaint: []int64{0, 4, 5, 6, 7, 8, 9}, remaint: []int64{0, 4, 5, 6, 7, 8, 9},
}, },
{ {
intervals: Intervals{{1, 3}, {4, 7}}, Intervals: tombstones.Intervals{{Mint: 1, Maxt: 3}, {Mint: 4, Maxt: 7}},
remaint: []int64{0, 8, 9}, remaint: []int64{0, 8, 9},
}, },
{ {
intervals: Intervals{{1, 3}, {4, 700}}, Intervals: tombstones.Intervals{{Mint: 1, Maxt: 3}, {Mint: 4, Maxt: 700}},
remaint: []int64{0}, remaint: []int64{0},
}, },
{ // This case is to ensure that labels and symbols are deleted. { // This case is to ensure that labels and symbols are deleted.
intervals: Intervals{{0, 9}}, Intervals: tombstones.Intervals{{Mint: 0, Maxt: 9}},
remaint: []int64{}, remaint: []int64{},
}, },
} }
@ -288,7 +290,7 @@ Outer:
// TODO(gouthamve): Reset the tombstones somehow. // TODO(gouthamve): Reset the tombstones somehow.
// Delete the ranges. // Delete the ranges.
for _, r := range c.intervals { for _, r := range c.Intervals {
testutil.Ok(t, db.Delete(r.Mint, r.Maxt, labels.NewEqualMatcher("a", "b"))) testutil.Ok(t, db.Delete(r.Mint, r.Maxt, labels.NewEqualMatcher("a", "b")))
} }
@ -561,11 +563,11 @@ func TestDB_SnapshotWithDelete(t *testing.T) {
testutil.Ok(t, app.Commit()) testutil.Ok(t, app.Commit())
cases := []struct { cases := []struct {
intervals Intervals intervals tombstones.Intervals
remaint []int64 remaint []int64
}{ }{
{ {
intervals: Intervals{{1, 3}, {4, 7}}, intervals: tombstones.Intervals{{Mint: 1, Maxt: 3}, {Mint: 4, Maxt: 7}},
remaint: []int64{0, 8, 9}, remaint: []int64{0, 8, 9},
}, },
} }
@ -888,11 +890,11 @@ func TestTombstoneClean(t *testing.T) {
testutil.Ok(t, app.Commit()) testutil.Ok(t, app.Commit())
cases := []struct { cases := []struct {
intervals Intervals intervals tombstones.Intervals
remaint []int64 remaint []int64
}{ }{
{ {
intervals: Intervals{{1, 3}, {4, 7}}, intervals: tombstones.Intervals{{Mint: 1, Maxt: 3}, {Mint: 4, Maxt: 7}},
remaint: []int64{0, 8, 9}, remaint: []int64{0, 8, 9},
}, },
} }
@ -964,7 +966,7 @@ func TestTombstoneClean(t *testing.T) {
} }
for _, b := range db.Blocks() { for _, b := range db.Blocks() {
testutil.Equals(t, newMemTombstones(), b.tombstones) testutil.Equals(t, tombstones.NewMemTombstones(), b.tombstones)
} }
} }
} }
@ -990,8 +992,8 @@ func TestTombstoneCleanFail(t *testing.T) {
block, err := OpenBlock(nil, blockDir, nil) block, err := OpenBlock(nil, blockDir, nil)
testutil.Ok(t, err) testutil.Ok(t, err)
// Add some some fake tombstones to trigger the compaction. // Add some some fake tombstones to trigger the compaction.
tomb := newMemTombstones() tomb := tombstones.NewMemTombstones()
tomb.addInterval(0, Interval{0, 1}) tomb.AddInterval(0, tombstones.Interval{Mint: 0, Maxt: 1})
block.tombstones = tomb block.tombstones = tomb
db.blocks = append(db.blocks, block) db.blocks = append(db.blocks, block)
@ -1470,13 +1472,13 @@ func TestInitializeHeadTimestamp(t *testing.T) {
w, err := wal.New(nil, nil, path.Join(dir, "wal"), false) w, err := wal.New(nil, nil, path.Join(dir, "wal"), false)
testutil.Ok(t, err) testutil.Ok(t, err)
var enc RecordEncoder var enc record.Encoder
err = w.Log( err = w.Log(
enc.Series([]RefSeries{ enc.Series([]record.RefSeries{
{Ref: 123, Labels: labels.FromStrings("a", "1")}, {Ref: 123, Labels: labels.FromStrings("a", "1")},
{Ref: 124, Labels: labels.FromStrings("a", "2")}, {Ref: 124, Labels: labels.FromStrings("a", "2")},
}, nil), }, nil),
enc.Samples([]RefSample{ enc.Samples([]record.RefSample{
{Ref: 123, T: 5000, V: 1}, {Ref: 123, T: 5000, V: 1},
{Ref: 124, T: 15000, V: 1}, {Ref: 124, T: 15000, V: 1},
}, nil), }, nil),
@ -1520,13 +1522,13 @@ func TestInitializeHeadTimestamp(t *testing.T) {
w, err := wal.New(nil, nil, path.Join(dir, "wal"), false) w, err := wal.New(nil, nil, path.Join(dir, "wal"), false)
testutil.Ok(t, err) testutil.Ok(t, err)
var enc RecordEncoder var enc record.Encoder
err = w.Log( err = w.Log(
enc.Series([]RefSeries{ enc.Series([]record.RefSeries{
{Ref: 123, Labels: labels.FromStrings("a", "1")}, {Ref: 123, Labels: labels.FromStrings("a", "1")},
{Ref: 124, Labels: labels.FromStrings("a", "2")}, {Ref: 124, Labels: labels.FromStrings("a", "2")},
}, nil), }, nil),
enc.Samples([]RefSample{ enc.Samples([]record.RefSample{
{Ref: 123, T: 5000, V: 1}, {Ref: 123, T: 5000, V: 1},
{Ref: 124, T: 15000, V: 1}, {Ref: 124, T: 15000, V: 1},
}, nil), }, nil),

View file

@ -33,6 +33,8 @@ import (
"github.com/prometheus/prometheus/tsdb/encoding" "github.com/prometheus/prometheus/tsdb/encoding"
"github.com/prometheus/prometheus/tsdb/index" "github.com/prometheus/prometheus/tsdb/index"
"github.com/prometheus/prometheus/tsdb/labels" "github.com/prometheus/prometheus/tsdb/labels"
"github.com/prometheus/prometheus/tsdb/record"
"github.com/prometheus/prometheus/tsdb/tombstones"
"github.com/prometheus/prometheus/tsdb/wal" "github.com/prometheus/prometheus/tsdb/wal"
) )
@ -54,7 +56,7 @@ var (
// emptyTombstoneReader is a no-op Tombstone Reader. // emptyTombstoneReader is a no-op Tombstone Reader.
// This is used by head to satisfy the Tombstones() function call. // This is used by head to satisfy the Tombstones() function call.
emptyTombstoneReader = newMemTombstones() emptyTombstoneReader = tombstones.NewMemTombstones()
) )
// Head handles reads and writes of time series data within a time window. // Head handles reads and writes of time series data within a time window.
@ -64,6 +66,7 @@ type Head struct {
wal *wal.WAL wal *wal.WAL
logger log.Logger logger log.Logger
appendPool sync.Pool appendPool sync.Pool
seriesPool sync.Pool
bytesPool sync.Pool bytesPool sync.Pool
numSeries uint64 numSeries uint64
@ -256,7 +259,7 @@ func NewHead(r prometheus.Registerer, l log.Logger, wal *wal.WAL, chunkRange int
// Samples before the mint timestamp are discarded. // Samples before the mint timestamp are discarded.
func (h *Head) processWALSamples( func (h *Head) processWALSamples(
minValidTime int64, minValidTime int64,
input <-chan []RefSample, output chan<- []RefSample, input <-chan []record.RefSample, output chan<- []record.RefSample,
) (unknownRefs uint64) { ) (unknownRefs uint64) {
defer close(output) defer close(output)
@ -331,8 +334,8 @@ func (h *Head) loadWAL(r *wal.Reader, multiRef map[uint64]uint64) (err error) {
wg sync.WaitGroup wg sync.WaitGroup
multiRefLock sync.Mutex multiRefLock sync.Mutex
n = runtime.GOMAXPROCS(0) n = runtime.GOMAXPROCS(0)
inputs = make([]chan []RefSample, n) inputs = make([]chan []record.RefSample, n)
outputs = make([]chan []RefSample, n) outputs = make([]chan []record.RefSample, n)
) )
wg.Add(n) wg.Add(n)
@ -349,10 +352,10 @@ func (h *Head) loadWAL(r *wal.Reader, multiRef map[uint64]uint64) (err error) {
}() }()
for i := 0; i < n; i++ { for i := 0; i < n; i++ {
outputs[i] = make(chan []RefSample, 300) outputs[i] = make(chan []record.RefSample, 300)
inputs[i] = make(chan []RefSample, 300) inputs[i] = make(chan []record.RefSample, 300)
go func(input <-chan []RefSample, output chan<- []RefSample) { go func(input <-chan []record.RefSample, output chan<- []record.RefSample) {
unknown := h.processWALSamples(h.minValidTime, input, output) unknown := h.processWALSamples(h.minValidTime, input, output)
atomic.AddUint64(&unknownRefs, unknown) atomic.AddUint64(&unknownRefs, unknown)
wg.Done() wg.Done()
@ -360,11 +363,11 @@ func (h *Head) loadWAL(r *wal.Reader, multiRef map[uint64]uint64) (err error) {
} }
var ( var (
dec RecordDecoder dec record.Decoder
series []RefSeries series []record.RefSeries
samples []RefSample samples []record.RefSample
tstones []Stone tstones []tombstones.Stone
allStones = newMemTombstones() allStones = tombstones.NewMemTombstones()
) )
defer func() { defer func() {
if err := allStones.Close(); err != nil { if err := allStones.Close(); err != nil {
@ -376,7 +379,7 @@ func (h *Head) loadWAL(r *wal.Reader, multiRef map[uint64]uint64) (err error) {
rec := r.Record() rec := r.Record()
switch dec.Type(rec) { switch dec.Type(rec) {
case RecordSeries: case record.Series:
series, err = dec.Series(rec, series) series, err = dec.Series(rec, series)
if err != nil { if err != nil {
return &wal.CorruptionErr{ return &wal.CorruptionErr{
@ -399,7 +402,7 @@ func (h *Head) loadWAL(r *wal.Reader, multiRef map[uint64]uint64) (err error) {
h.lastSeriesID = s.Ref h.lastSeriesID = s.Ref
} }
} }
case RecordSamples: case record.Samples:
samples, err = dec.Samples(rec, samples) samples, err = dec.Samples(rec, samples)
s := samples s := samples
if err != nil { if err != nil {
@ -418,9 +421,9 @@ func (h *Head) loadWAL(r *wal.Reader, multiRef map[uint64]uint64) (err error) {
if len(samples) < m { if len(samples) < m {
m = len(samples) m = len(samples)
} }
shards := make([][]RefSample, n) shards := make([][]record.RefSample, n)
for i := 0; i < n; i++ { for i := 0; i < n; i++ {
var buf []RefSample var buf []record.RefSample
select { select {
case buf = <-outputs[i]: case buf = <-outputs[i]:
default: default:
@ -440,7 +443,7 @@ func (h *Head) loadWAL(r *wal.Reader, multiRef map[uint64]uint64) (err error) {
samples = samples[m:] samples = samples[m:]
} }
samples = s // Keep whole slice for reuse. samples = s // Keep whole slice for reuse.
case RecordTombstones: case record.Tombstones:
tstones, err = dec.Tombstones(rec, tstones) tstones, err = dec.Tombstones(rec, tstones)
if err != nil { if err != nil {
return &wal.CorruptionErr{ return &wal.CorruptionErr{
@ -450,15 +453,15 @@ func (h *Head) loadWAL(r *wal.Reader, multiRef map[uint64]uint64) (err error) {
} }
} }
for _, s := range tstones { for _, s := range tstones {
for _, itv := range s.intervals { for _, itv := range s.Intervals {
if itv.Maxt < h.minValidTime { if itv.Maxt < h.minValidTime {
continue continue
} }
if m := h.series.getByID(s.ref); m == nil { if m := h.series.getByID(s.Ref); m == nil {
unknownRefs++ unknownRefs++
continue continue
} }
allStones.addInterval(s.ref, itv) allStones.AddInterval(s.Ref, itv)
} }
} }
default: default:
@ -482,7 +485,7 @@ func (h *Head) loadWAL(r *wal.Reader, multiRef map[uint64]uint64) (err error) {
return errors.Wrap(r.Err(), "read records") return errors.Wrap(r.Err(), "read records")
} }
if err := allStones.Iter(func(ref uint64, dranges Intervals) error { if err := allStones.Iter(func(ref uint64, dranges tombstones.Intervals) error {
return h.chunkRewrite(ref, dranges) return h.chunkRewrite(ref, dranges)
}); err != nil { }); err != nil {
return errors.Wrap(r.Err(), "deleting samples from tombstones") return errors.Wrap(r.Err(), "deleting samples from tombstones")
@ -508,8 +511,8 @@ func (h *Head) Init(minValidTime int64) error {
level.Info(h.logger).Log("msg", "replaying WAL, this may take awhile") level.Info(h.logger).Log("msg", "replaying WAL, this may take awhile")
// Backfill the checkpoint first if it exists. // Backfill the checkpoint first if it exists.
dir, startFrom, err := LastCheckpoint(h.wal.Dir()) dir, startFrom, err := wal.LastCheckpoint(h.wal.Dir())
if err != nil && err != ErrNotFound { if err != nil && err != record.ErrNotFound {
return errors.Wrap(err, "find last checkpoint") return errors.Wrap(err, "find last checkpoint")
} }
multiRef := map[uint64]uint64{} multiRef := map[uint64]uint64{}
@ -629,7 +632,7 @@ func (h *Head) Truncate(mint int64) (err error) {
return ok return ok
} }
h.metrics.checkpointCreationTotal.Inc() h.metrics.checkpointCreationTotal.Inc()
if _, err = Checkpoint(h.wal, first, last, keep, mint); err != nil { if _, err = wal.Checkpoint(h.wal, first, last, keep, mint); err != nil {
h.metrics.checkpointCreationFail.Inc() h.metrics.checkpointCreationFail.Inc()
return errors.Wrap(err, "create checkpoint") return errors.Wrap(err, "create checkpoint")
} }
@ -651,7 +654,7 @@ func (h *Head) Truncate(mint int64) (err error) {
h.deletedMtx.Unlock() h.deletedMtx.Unlock()
h.metrics.checkpointDeleteTotal.Inc() h.metrics.checkpointDeleteTotal.Inc()
if err := DeleteCheckpoints(h.wal.Dir(), last); err != nil { if err := wal.DeleteCheckpoints(h.wal.Dir(), last); err != nil {
// Leftover old checkpoints do not cause problems down the line beyond // Leftover old checkpoints do not cause problems down the line beyond
// occupying disk space. // occupying disk space.
// They will just be ignored since a higher checkpoint exists. // They will just be ignored since a higher checkpoint exists.
@ -693,7 +696,7 @@ func (h *rangeHead) Chunks() (ChunkReader, error) {
return h.head.chunksRange(h.mint, h.maxt), nil return h.head.chunksRange(h.mint, h.maxt), nil
} }
func (h *rangeHead) Tombstones() (TombstoneReader, error) { func (h *rangeHead) Tombstones() (tombstones.Reader, error) {
return emptyTombstoneReader, nil return emptyTombstoneReader, nil
} }
@ -779,6 +782,7 @@ func (h *Head) appender() *headAppender {
mint: math.MaxInt64, mint: math.MaxInt64,
maxt: math.MinInt64, maxt: math.MinInt64,
samples: h.getAppendBuffer(), samples: h.getAppendBuffer(),
sampleSeries: h.getSeriesBuffer(),
} }
} }
@ -789,19 +793,32 @@ func max(a, b int64) int64 {
return b return b
} }
func (h *Head) getAppendBuffer() []RefSample { func (h *Head) getAppendBuffer() []record.RefSample {
b := h.appendPool.Get() b := h.appendPool.Get()
if b == nil { if b == nil {
return make([]RefSample, 0, 512) return make([]record.RefSample, 0, 512)
} }
return b.([]RefSample) return b.([]record.RefSample)
} }
func (h *Head) putAppendBuffer(b []RefSample) { func (h *Head) putAppendBuffer(b []record.RefSample) {
//lint:ignore SA6002 safe to ignore and actually fixing it has some performance penalty. //lint:ignore SA6002 safe to ignore and actually fixing it has some performance penalty.
h.appendPool.Put(b[:0]) h.appendPool.Put(b[:0])
} }
func (h *Head) getSeriesBuffer() []*memSeries {
b := h.seriesPool.Get()
if b == nil {
return make([]*memSeries, 0, 512)
}
return b.([]*memSeries)
}
func (h *Head) putSeriesBuffer(b []*memSeries) {
//lint:ignore SA6002 safe to ignore and actually fixing it has some performance penalty.
h.seriesPool.Put(b[:0])
}
func (h *Head) getBytesBuffer() []byte { func (h *Head) getBytesBuffer() []byte {
b := h.bytesPool.Get() b := h.bytesPool.Get()
if b == nil { if b == nil {
@ -820,8 +837,9 @@ type headAppender struct {
minValidTime int64 // No samples below this timestamp are allowed. minValidTime int64 // No samples below this timestamp are allowed.
mint, maxt int64 mint, maxt int64
series []RefSeries series []record.RefSeries
samples []RefSample samples []record.RefSample
sampleSeries []*memSeries
} }
func (a *headAppender) Add(lset labels.Labels, t int64, v float64) (uint64, error) { func (a *headAppender) Add(lset labels.Labels, t int64, v float64) (uint64, error) {
@ -834,7 +852,7 @@ func (a *headAppender) Add(lset labels.Labels, t int64, v float64) (uint64, erro
s, created := a.head.getOrCreate(lset.Hash(), lset) s, created := a.head.getOrCreate(lset.Hash(), lset)
if created { if created {
a.series = append(a.series, RefSeries{ a.series = append(a.series, record.RefSeries{
Ref: s.ref, Ref: s.ref,
Labels: lset, Labels: lset,
}) })
@ -866,12 +884,12 @@ func (a *headAppender) AddFast(ref uint64, t int64, v float64) error {
a.maxt = t a.maxt = t
} }
a.samples = append(a.samples, RefSample{ a.samples = append(a.samples, record.RefSample{
Ref: ref, Ref: ref,
T: t, T: t,
V: v, V: v,
series: s,
}) })
a.sampleSeries = append(a.sampleSeries, s)
return nil return nil
} }
@ -884,7 +902,7 @@ func (a *headAppender) log() error {
defer func() { a.head.putBytesBuffer(buf) }() defer func() { a.head.putBytesBuffer(buf) }()
var rec []byte var rec []byte
var enc RecordEncoder var enc record.Encoder
if len(a.series) > 0 { if len(a.series) > 0 {
rec = enc.Series(a.series, buf) rec = enc.Series(a.series, buf)
@ -908,18 +926,20 @@ func (a *headAppender) log() error {
func (a *headAppender) Commit() error { func (a *headAppender) Commit() error {
defer a.head.metrics.activeAppenders.Dec() defer a.head.metrics.activeAppenders.Dec()
defer a.head.putAppendBuffer(a.samples) defer a.head.putAppendBuffer(a.samples)
defer a.head.putSeriesBuffer(a.sampleSeries)
if err := a.log(); err != nil { if err := a.log(); err != nil {
return errors.Wrap(err, "write to WAL") return errors.Wrap(err, "write to WAL")
} }
total := len(a.samples) total := len(a.samples)
var series *memSeries
for _, s := range a.samples { for i, s := range a.samples {
s.series.Lock() series = a.sampleSeries[i]
ok, chunkCreated := s.series.append(s.T, s.V) series.Lock()
s.series.pendingCommit = false ok, chunkCreated := series.append(s.T, s.V)
s.series.Unlock() series.pendingCommit = false
series.Unlock()
if !ok { if !ok {
total-- total--
@ -938,10 +958,12 @@ func (a *headAppender) Commit() error {
func (a *headAppender) Rollback() error { func (a *headAppender) Rollback() error {
a.head.metrics.activeAppenders.Dec() a.head.metrics.activeAppenders.Dec()
for _, s := range a.samples { var series *memSeries
s.series.Lock() for i := range a.samples {
s.series.pendingCommit = false series = a.sampleSeries[i]
s.series.Unlock() series.Lock()
series.pendingCommit = false
series.Unlock()
} }
a.head.putAppendBuffer(a.samples) a.head.putAppendBuffer(a.samples)
@ -964,7 +986,7 @@ func (h *Head) Delete(mint, maxt int64, ms ...labels.Matcher) error {
return errors.Wrap(err, "select series") return errors.Wrap(err, "select series")
} }
var stones []Stone var stones []tombstones.Stone
dirty := false dirty := false
for p.Next() { for p.Next() {
series := h.series.getByID(p.At()) series := h.series.getByID(p.At())
@ -976,9 +998,9 @@ func (h *Head) Delete(mint, maxt int64, ms ...labels.Matcher) error {
// Delete only until the current values and not beyond. // Delete only until the current values and not beyond.
t0, t1 = clampInterval(mint, maxt, t0, t1) t0, t1 = clampInterval(mint, maxt, t0, t1)
if h.wal != nil { if h.wal != nil {
stones = append(stones, Stone{p.At(), Intervals{{t0, t1}}}) stones = append(stones, tombstones.Stone{Ref: p.At(), Intervals: tombstones.Intervals{{Mint: t0, Maxt: t1}}})
} }
if err := h.chunkRewrite(p.At(), Intervals{{t0, t1}}); err != nil { if err := h.chunkRewrite(p.At(), tombstones.Intervals{{Mint: t0, Maxt: t1}}); err != nil {
return errors.Wrap(err, "delete samples") return errors.Wrap(err, "delete samples")
} }
dirty = true dirty = true
@ -986,7 +1008,7 @@ func (h *Head) Delete(mint, maxt int64, ms ...labels.Matcher) error {
if p.Err() != nil { if p.Err() != nil {
return p.Err() return p.Err()
} }
var enc RecordEncoder var enc record.Encoder
if h.wal != nil { if h.wal != nil {
// Although we don't store the stones in the head // Although we don't store the stones in the head
// we need to write them to the WAL to mark these as deleted // we need to write them to the WAL to mark these as deleted
@ -1005,7 +1027,7 @@ func (h *Head) Delete(mint, maxt int64, ms ...labels.Matcher) error {
// chunkRewrite re-writes the chunks which overlaps with deleted ranges // chunkRewrite re-writes the chunks which overlaps with deleted ranges
// and removes the samples in the deleted ranges. // and removes the samples in the deleted ranges.
// Chunks is deleted if no samples are left at the end. // Chunks is deleted if no samples are left at the end.
func (h *Head) chunkRewrite(ref uint64, dranges Intervals) (err error) { func (h *Head) chunkRewrite(ref uint64, dranges tombstones.Intervals) (err error) {
if len(dranges) == 0 { if len(dranges) == 0 {
return nil return nil
} }
@ -1097,7 +1119,7 @@ func (h *Head) gc() {
} }
// Tombstones returns a new reader over the head's tombstones // Tombstones returns a new reader over the head's tombstones
func (h *Head) Tombstones() (TombstoneReader, error) { func (h *Head) Tombstones() (tombstones.Reader, error) {
return emptyTombstoneReader, nil return emptyTombstoneReader, nil
} }

View file

@ -30,6 +30,8 @@ import (
"github.com/prometheus/prometheus/tsdb/chunks" "github.com/prometheus/prometheus/tsdb/chunks"
"github.com/prometheus/prometheus/tsdb/index" "github.com/prometheus/prometheus/tsdb/index"
"github.com/prometheus/prometheus/tsdb/labels" "github.com/prometheus/prometheus/tsdb/labels"
"github.com/prometheus/prometheus/tsdb/record"
"github.com/prometheus/prometheus/tsdb/tombstones"
"github.com/prometheus/prometheus/tsdb/tsdbutil" "github.com/prometheus/prometheus/tsdb/tsdbutil"
"github.com/prometheus/prometheus/tsdb/wal" "github.com/prometheus/prometheus/tsdb/wal"
"github.com/prometheus/prometheus/util/testutil" "github.com/prometheus/prometheus/util/testutil"
@ -51,14 +53,14 @@ func BenchmarkCreateSeries(b *testing.B) {
} }
func populateTestWAL(t testing.TB, w *wal.WAL, recs []interface{}) { func populateTestWAL(t testing.TB, w *wal.WAL, recs []interface{}) {
var enc RecordEncoder var enc record.Encoder
for _, r := range recs { for _, r := range recs {
switch v := r.(type) { switch v := r.(type) {
case []RefSeries: case []record.RefSeries:
testutil.Ok(t, w.Log(enc.Series(v, nil))) testutil.Ok(t, w.Log(enc.Series(v, nil)))
case []RefSample: case []record.RefSample:
testutil.Ok(t, w.Log(enc.Samples(v, nil))) testutil.Ok(t, w.Log(enc.Samples(v, nil)))
case []Stone: case []tombstones.Stone:
testutil.Ok(t, w.Log(enc.Tombstones(v, nil))) testutil.Ok(t, w.Log(enc.Tombstones(v, nil)))
} }
} }
@ -69,22 +71,22 @@ func readTestWAL(t testing.TB, dir string) (recs []interface{}) {
testutil.Ok(t, err) testutil.Ok(t, err)
defer sr.Close() defer sr.Close()
var dec RecordDecoder var dec record.Decoder
r := wal.NewReader(sr) r := wal.NewReader(sr)
for r.Next() { for r.Next() {
rec := r.Record() rec := r.Record()
switch dec.Type(rec) { switch dec.Type(rec) {
case RecordSeries: case record.Series:
series, err := dec.Series(rec, nil) series, err := dec.Series(rec, nil)
testutil.Ok(t, err) testutil.Ok(t, err)
recs = append(recs, series) recs = append(recs, series)
case RecordSamples: case record.Samples:
samples, err := dec.Samples(rec, nil) samples, err := dec.Samples(rec, nil)
testutil.Ok(t, err) testutil.Ok(t, err)
recs = append(recs, samples) recs = append(recs, samples)
case RecordTombstones: case record.Tombstones:
tstones, err := dec.Tombstones(rec, nil) tstones, err := dec.Tombstones(rec, nil)
testutil.Ok(t, err) testutil.Ok(t, err)
recs = append(recs, tstones) recs = append(recs, tstones)
@ -100,28 +102,28 @@ func TestHead_ReadWAL(t *testing.T) {
for _, compress := range []bool{false, true} { for _, compress := range []bool{false, true} {
t.Run(fmt.Sprintf("compress=%t", compress), func(t *testing.T) { t.Run(fmt.Sprintf("compress=%t", compress), func(t *testing.T) {
entries := []interface{}{ entries := []interface{}{
[]RefSeries{ []record.RefSeries{
{Ref: 10, Labels: labels.FromStrings("a", "1")}, {Ref: 10, Labels: labels.FromStrings("a", "1")},
{Ref: 11, Labels: labels.FromStrings("a", "2")}, {Ref: 11, Labels: labels.FromStrings("a", "2")},
{Ref: 100, Labels: labels.FromStrings("a", "3")}, {Ref: 100, Labels: labels.FromStrings("a", "3")},
}, },
[]RefSample{ []record.RefSample{
{Ref: 0, T: 99, V: 1}, {Ref: 0, T: 99, V: 1},
{Ref: 10, T: 100, V: 2}, {Ref: 10, T: 100, V: 2},
{Ref: 100, T: 100, V: 3}, {Ref: 100, T: 100, V: 3},
}, },
[]RefSeries{ []record.RefSeries{
{Ref: 50, Labels: labels.FromStrings("a", "4")}, {Ref: 50, Labels: labels.FromStrings("a", "4")},
// This series has two refs pointing to it. // This series has two refs pointing to it.
{Ref: 101, Labels: labels.FromStrings("a", "3")}, {Ref: 101, Labels: labels.FromStrings("a", "3")},
}, },
[]RefSample{ []record.RefSample{
{Ref: 10, T: 101, V: 5}, {Ref: 10, T: 101, V: 5},
{Ref: 50, T: 101, V: 6}, {Ref: 50, T: 101, V: 6},
{Ref: 101, T: 101, V: 7}, {Ref: 101, T: 101, V: 7},
}, },
[]Stone{ []tombstones.Stone{
{ref: 0, intervals: []Interval{{Mint: 99, Maxt: 101}}}, {Ref: 0, Intervals: []tombstones.Interval{{Mint: 99, Maxt: 101}}},
}, },
} }
dir, err := ioutil.TempDir("", "test_read_wal") dir, err := ioutil.TempDir("", "test_read_wal")
@ -326,14 +328,14 @@ func TestHeadDeleteSeriesWithoutSamples(t *testing.T) {
for _, compress := range []bool{false, true} { for _, compress := range []bool{false, true} {
t.Run(fmt.Sprintf("compress=%t", compress), func(t *testing.T) { t.Run(fmt.Sprintf("compress=%t", compress), func(t *testing.T) {
entries := []interface{}{ entries := []interface{}{
[]RefSeries{ []record.RefSeries{
{Ref: 10, Labels: labels.FromStrings("a", "1")}, {Ref: 10, Labels: labels.FromStrings("a", "1")},
}, },
[]RefSample{}, []record.RefSample{},
[]RefSeries{ []record.RefSeries{
{Ref: 50, Labels: labels.FromStrings("a", "2")}, {Ref: 50, Labels: labels.FromStrings("a", "2")},
}, },
[]RefSample{ []record.RefSample{
{Ref: 50, T: 80, V: 1}, {Ref: 50, T: 80, V: 1},
{Ref: 50, T: 90, V: 1}, {Ref: 50, T: 90, V: 1},
}, },
@ -371,27 +373,27 @@ func TestHeadDeleteSimple(t *testing.T) {
lblDefault := labels.Label{Name: "a", Value: "b"} lblDefault := labels.Label{Name: "a", Value: "b"}
cases := []struct { cases := []struct {
dranges Intervals dranges tombstones.Intervals
smplsExp []sample smplsExp []sample
}{ }{
{ {
dranges: Intervals{{0, 3}}, dranges: tombstones.Intervals{{Mint: 0, Maxt: 3}},
smplsExp: buildSmpls([]int64{4, 5, 6, 7, 8, 9}), smplsExp: buildSmpls([]int64{4, 5, 6, 7, 8, 9}),
}, },
{ {
dranges: Intervals{{1, 3}}, dranges: tombstones.Intervals{{Mint: 1, Maxt: 3}},
smplsExp: buildSmpls([]int64{0, 4, 5, 6, 7, 8, 9}), smplsExp: buildSmpls([]int64{0, 4, 5, 6, 7, 8, 9}),
}, },
{ {
dranges: Intervals{{1, 3}, {4, 7}}, dranges: tombstones.Intervals{{Mint: 1, Maxt: 3}, {Mint: 4, Maxt: 7}},
smplsExp: buildSmpls([]int64{0, 8, 9}), smplsExp: buildSmpls([]int64{0, 8, 9}),
}, },
{ {
dranges: Intervals{{1, 3}, {4, 700}}, dranges: tombstones.Intervals{{Mint: 1, Maxt: 3}, {Mint: 4, Maxt: 700}},
smplsExp: buildSmpls([]int64{0}), smplsExp: buildSmpls([]int64{0}),
}, },
{ // This case is to ensure that labels and symbols are deleted. { // This case is to ensure that labels and symbols are deleted.
dranges: Intervals{{0, 9}}, dranges: tombstones.Intervals{{Mint: 0, Maxt: 9}},
smplsExp: buildSmpls([]int64{}), smplsExp: buildSmpls([]int64{}),
}, },
} }
@ -591,7 +593,7 @@ func TestDeletedSamplesAndSeriesStillInWALAfterCheckpoint(t *testing.T) {
testutil.Ok(t, hb.Close()) testutil.Ok(t, hb.Close())
// Confirm there's been a checkpoint. // Confirm there's been a checkpoint.
cdir, _, err := LastCheckpoint(dir) cdir, _, err := wal.LastCheckpoint(dir)
testutil.Ok(t, err) testutil.Ok(t, err)
// Read in checkpoint and WAL. // Read in checkpoint and WAL.
recs := readTestWAL(t, cdir) recs := readTestWAL(t, cdir)
@ -600,11 +602,11 @@ func TestDeletedSamplesAndSeriesStillInWALAfterCheckpoint(t *testing.T) {
var series, samples, stones int var series, samples, stones int
for _, rec := range recs { for _, rec := range recs {
switch rec.(type) { switch rec.(type) {
case []RefSeries: case []record.RefSeries:
series++ series++
case []RefSample: case []record.RefSample:
samples++ samples++
case []Stone: case []tombstones.Stone:
stones++ stones++
default: default:
t.Fatalf("unknown record type") t.Fatalf("unknown record type")
@ -692,18 +694,18 @@ func TestDelete_e2e(t *testing.T) {
// Delete a time-range from each-selector. // Delete a time-range from each-selector.
dels := []struct { dels := []struct {
ms []labels.Matcher ms []labels.Matcher
drange Intervals drange tombstones.Intervals
}{ }{
{ {
ms: []labels.Matcher{labels.NewEqualMatcher("a", "b")}, ms: []labels.Matcher{labels.NewEqualMatcher("a", "b")},
drange: Intervals{{300, 500}, {600, 670}}, drange: tombstones.Intervals{{Mint: 300, Maxt: 500}, {Mint: 600, Maxt: 670}},
}, },
{ {
ms: []labels.Matcher{ ms: []labels.Matcher{
labels.NewEqualMatcher("a", "b"), labels.NewEqualMatcher("a", "b"),
labels.NewEqualMatcher("job", "prom-k8s"), labels.NewEqualMatcher("job", "prom-k8s"),
}, },
drange: Intervals{{300, 500}, {100, 670}}, drange: tombstones.Intervals{{Mint: 300, Maxt: 500}, {Mint: 100, Maxt: 670}},
}, },
{ {
ms: []labels.Matcher{ ms: []labels.Matcher{
@ -711,7 +713,7 @@ func TestDelete_e2e(t *testing.T) {
labels.NewEqualMatcher("instance", "localhost:9090"), labels.NewEqualMatcher("instance", "localhost:9090"),
labels.NewEqualMatcher("job", "prometheus"), labels.NewEqualMatcher("job", "prometheus"),
}, },
drange: Intervals{{300, 400}, {100, 6700}}, drange: tombstones.Intervals{{Mint: 300, Maxt: 400}, {Mint: 100, Maxt: 6700}},
}, },
// TODO: Add Regexp Matchers. // TODO: Add Regexp Matchers.
} }
@ -794,12 +796,12 @@ func boundedSamples(full []tsdbutil.Sample, mint, maxt int64) []tsdbutil.Sample
return full return full
} }
func deletedSamples(full []tsdbutil.Sample, dranges Intervals) []tsdbutil.Sample { func deletedSamples(full []tsdbutil.Sample, dranges tombstones.Intervals) []tsdbutil.Sample {
ds := make([]tsdbutil.Sample, 0, len(full)) ds := make([]tsdbutil.Sample, 0, len(full))
Outer: Outer:
for _, s := range full { for _, s := range full {
for _, r := range dranges { for _, r := range dranges {
if r.inBounds(s.T()) { if r.InBounds(s.T()) {
continue Outer continue Outer
} }
} }
@ -1055,9 +1057,9 @@ func TestHead_LogRollback(t *testing.T) {
testutil.Equals(t, 1, len(recs)) testutil.Equals(t, 1, len(recs))
series, ok := recs[0].([]RefSeries) series, ok := recs[0].([]record.RefSeries)
testutil.Assert(t, ok, "expected series record but got %+v", recs[0]) testutil.Assert(t, ok, "expected series record but got %+v", recs[0])
testutil.Equals(t, []RefSeries{{Ref: 1, Labels: labels.FromStrings("a", "b")}}, series) testutil.Equals(t, []record.RefSeries{{Ref: 1, Labels: labels.FromStrings("a", "b")}}, series)
}) })
} }
} }
@ -1065,7 +1067,7 @@ func TestHead_LogRollback(t *testing.T) {
// TestWalRepair_DecodingError ensures that a repair is run for an error // TestWalRepair_DecodingError ensures that a repair is run for an error
// when decoding a record. // when decoding a record.
func TestWalRepair_DecodingError(t *testing.T) { func TestWalRepair_DecodingError(t *testing.T) {
var enc RecordEncoder var enc record.Encoder
for name, test := range map[string]struct { for name, test := range map[string]struct {
corrFunc func(rec []byte) []byte // Func that applies the corruption to a record. corrFunc func(rec []byte) []byte // Func that applies the corruption to a record.
rec []byte rec []byte
@ -1077,10 +1079,10 @@ func TestWalRepair_DecodingError(t *testing.T) {
// Do not modify the base record because it is Logged multiple times. // Do not modify the base record because it is Logged multiple times.
res := make([]byte, len(rec)) res := make([]byte, len(rec))
copy(res, rec) copy(res, rec)
res[0] = byte(RecordInvalid) res[0] = byte(record.Invalid)
return res return res
}, },
enc.Series([]RefSeries{{Ref: 1, Labels: labels.FromStrings("a", "b")}}, []byte{}), enc.Series([]record.RefSeries{{Ref: 1, Labels: labels.FromStrings("a", "b")}}, []byte{}),
9, 9,
5, 5,
}, },
@ -1088,7 +1090,7 @@ func TestWalRepair_DecodingError(t *testing.T) {
func(rec []byte) []byte { func(rec []byte) []byte {
return rec[:3] return rec[:3]
}, },
enc.Series([]RefSeries{{Ref: 1, Labels: labels.FromStrings("a", "b")}}, []byte{}), enc.Series([]record.RefSeries{{Ref: 1, Labels: labels.FromStrings("a", "b")}}, []byte{}),
9, 9,
5, 5,
}, },
@ -1096,7 +1098,7 @@ func TestWalRepair_DecodingError(t *testing.T) {
func(rec []byte) []byte { func(rec []byte) []byte {
return rec[:3] return rec[:3]
}, },
enc.Samples([]RefSample{{Ref: 0, T: 99, V: 1}}, []byte{}), enc.Samples([]record.RefSample{{Ref: 0, T: 99, V: 1}}, []byte{}),
9, 9,
5, 5,
}, },
@ -1104,7 +1106,7 @@ func TestWalRepair_DecodingError(t *testing.T) {
func(rec []byte) []byte { func(rec []byte) []byte {
return rec[:3] return rec[:3]
}, },
enc.Tombstones([]Stone{{ref: 1, intervals: Intervals{}}}, []byte{}), enc.Tombstones([]tombstones.Stone{{Ref: 1, Intervals: tombstones.Intervals{}}}, []byte{}),
9, 9,
5, 5,
}, },

View file

@ -18,6 +18,7 @@ import (
"github.com/prometheus/prometheus/tsdb/chunks" "github.com/prometheus/prometheus/tsdb/chunks"
"github.com/prometheus/prometheus/tsdb/index" "github.com/prometheus/prometheus/tsdb/index"
"github.com/prometheus/prometheus/tsdb/labels" "github.com/prometheus/prometheus/tsdb/labels"
"github.com/prometheus/prometheus/tsdb/tombstones"
) )
type mockIndexWriter struct { type mockIndexWriter struct {
@ -72,7 +73,9 @@ type mockBReader struct {
maxt int64 maxt int64
} }
func (r *mockBReader) Index() (IndexReader, error) { return r.ir, nil } func (r *mockBReader) Index() (IndexReader, error) { return r.ir, nil }
func (r *mockBReader) Chunks() (ChunkReader, error) { return r.cr, nil } func (r *mockBReader) Chunks() (ChunkReader, error) { return r.cr, nil }
func (r *mockBReader) Tombstones() (TombstoneReader, error) { return newMemTombstones(), nil } func (r *mockBReader) Tombstones() (tombstones.Reader, error) {
func (r *mockBReader) Meta() BlockMeta { return BlockMeta{MinTime: r.mint, MaxTime: r.maxt} } return tombstones.NewMemTombstones(), nil
}
func (r *mockBReader) Meta() BlockMeta { return BlockMeta{MinTime: r.mint, MaxTime: r.maxt} }

View file

@ -25,6 +25,7 @@ import (
tsdb_errors "github.com/prometheus/prometheus/tsdb/errors" tsdb_errors "github.com/prometheus/prometheus/tsdb/errors"
"github.com/prometheus/prometheus/tsdb/index" "github.com/prometheus/prometheus/tsdb/index"
"github.com/prometheus/prometheus/tsdb/labels" "github.com/prometheus/prometheus/tsdb/labels"
"github.com/prometheus/prometheus/tsdb/tombstones"
) )
// Querier provides querying access over time series data of a fixed // Querier provides querying access over time series data of a fixed
@ -204,7 +205,7 @@ func NewBlockQuerier(b BlockReader, mint, maxt int64) (Querier, error) {
type blockQuerier struct { type blockQuerier struct {
index IndexReader index IndexReader
chunks ChunkReader chunks ChunkReader
tombstones TombstoneReader tombstones tombstones.Reader
closed bool closed bool
@ -671,7 +672,7 @@ func (s *mergedVerticalSeriesSet) Next() bool {
// actual series itself. // actual series itself.
type ChunkSeriesSet interface { type ChunkSeriesSet interface {
Next() bool Next() bool
At() (labels.Labels, []chunks.Meta, Intervals) At() (labels.Labels, []chunks.Meta, tombstones.Intervals)
Err() error Err() error
} }
@ -680,19 +681,19 @@ type ChunkSeriesSet interface {
type baseChunkSeries struct { type baseChunkSeries struct {
p index.Postings p index.Postings
index IndexReader index IndexReader
tombstones TombstoneReader tombstones tombstones.Reader
lset labels.Labels lset labels.Labels
chks []chunks.Meta chks []chunks.Meta
intervals Intervals intervals tombstones.Intervals
err error err error
} }
// LookupChunkSeries retrieves all series for the given matchers and returns a ChunkSeriesSet // LookupChunkSeries retrieves all series for the given matchers and returns a ChunkSeriesSet
// over them. It drops chunks based on tombstones in the given reader. // over them. It drops chunks based on tombstones in the given reader.
func LookupChunkSeries(ir IndexReader, tr TombstoneReader, ms ...labels.Matcher) (ChunkSeriesSet, error) { func LookupChunkSeries(ir IndexReader, tr tombstones.Reader, ms ...labels.Matcher) (ChunkSeriesSet, error) {
if tr == nil { if tr == nil {
tr = newMemTombstones() tr = tombstones.NewMemTombstones()
} }
p, err := PostingsForMatchers(ir, ms...) p, err := PostingsForMatchers(ir, ms...)
if err != nil { if err != nil {
@ -705,7 +706,7 @@ func LookupChunkSeries(ir IndexReader, tr TombstoneReader, ms ...labels.Matcher)
}, nil }, nil
} }
func (s *baseChunkSeries) At() (labels.Labels, []chunks.Meta, Intervals) { func (s *baseChunkSeries) At() (labels.Labels, []chunks.Meta, tombstones.Intervals) {
return s.lset, s.chks, s.intervals return s.lset, s.chks, s.intervals
} }
@ -741,7 +742,7 @@ func (s *baseChunkSeries) Next() bool {
// Only those chunks that are not entirely deleted. // Only those chunks that are not entirely deleted.
chks := make([]chunks.Meta, 0, len(s.chks)) chks := make([]chunks.Meta, 0, len(s.chks))
for _, chk := range s.chks { for _, chk := range s.chks {
if !(Interval{chk.MinTime, chk.MaxTime}.isSubrange(s.intervals)) { if !(tombstones.Interval{Mint: chk.MinTime, Maxt: chk.MaxTime}.IsSubrange(s.intervals)) {
chks = append(chks, chk) chks = append(chks, chk)
} }
} }
@ -768,10 +769,10 @@ type populatedChunkSeries struct {
err error err error
chks []chunks.Meta chks []chunks.Meta
lset labels.Labels lset labels.Labels
intervals Intervals intervals tombstones.Intervals
} }
func (s *populatedChunkSeries) At() (labels.Labels, []chunks.Meta, Intervals) { func (s *populatedChunkSeries) At() (labels.Labels, []chunks.Meta, tombstones.Intervals) {
return s.lset, s.chks, s.intervals return s.lset, s.chks, s.intervals
} }
@ -866,7 +867,7 @@ type chunkSeries struct {
mint, maxt int64 mint, maxt int64
intervals Intervals intervals tombstones.Intervals
} }
func (s *chunkSeries) Labels() labels.Labels { func (s *chunkSeries) Labels() labels.Labels {
@ -1067,10 +1068,10 @@ type chunkSeriesIterator struct {
maxt, mint int64 maxt, mint int64
intervals Intervals intervals tombstones.Intervals
} }
func newChunkSeriesIterator(cs []chunks.Meta, dranges Intervals, mint, maxt int64) *chunkSeriesIterator { func newChunkSeriesIterator(cs []chunks.Meta, dranges tombstones.Intervals, mint, maxt int64) *chunkSeriesIterator {
csi := &chunkSeriesIterator{ csi := &chunkSeriesIterator{
chunks: cs, chunks: cs,
i: 0, i: 0,
@ -1169,7 +1170,7 @@ func (it *chunkSeriesIterator) Err() error {
type deletedIterator struct { type deletedIterator struct {
it chunkenc.Iterator it chunkenc.Iterator
intervals Intervals intervals tombstones.Intervals
} }
func (it *deletedIterator) At() (int64, float64) { func (it *deletedIterator) At() (int64, float64) {
@ -1182,7 +1183,7 @@ Outer:
ts, _ := it.it.At() ts, _ := it.it.At()
for _, tr := range it.intervals { for _, tr := range it.intervals {
if tr.inBounds(ts) { if tr.InBounds(ts) {
continue Outer continue Outer
} }

View file

@ -29,6 +29,7 @@ import (
"github.com/prometheus/prometheus/tsdb/chunks" "github.com/prometheus/prometheus/tsdb/chunks"
"github.com/prometheus/prometheus/tsdb/index" "github.com/prometheus/prometheus/tsdb/index"
"github.com/prometheus/prometheus/tsdb/labels" "github.com/prometheus/prometheus/tsdb/labels"
"github.com/prometheus/prometheus/tsdb/tombstones"
"github.com/prometheus/prometheus/tsdb/tsdbutil" "github.com/prometheus/prometheus/tsdb/tsdbutil"
"github.com/prometheus/prometheus/util/testutil" "github.com/prometheus/prometheus/util/testutil"
) )
@ -368,7 +369,7 @@ Outer:
querier := &blockQuerier{ querier := &blockQuerier{
index: ir, index: ir,
chunks: cr, chunks: cr,
tombstones: newMemTombstones(), tombstones: tombstones.NewMemTombstones(),
mint: c.mint, mint: c.mint,
maxt: c.maxt, maxt: c.maxt,
@ -415,7 +416,7 @@ func TestBlockQuerierDelete(t *testing.T) {
cases := struct { cases := struct {
data []seriesSamples data []seriesSamples
tombstones TombstoneReader tombstones tombstones.Reader
queries []query queries []query
}{ }{
data: []seriesSamples{ data: []seriesSamples{
@ -460,11 +461,11 @@ func TestBlockQuerierDelete(t *testing.T) {
}, },
}, },
}, },
tombstones: &memTombstones{intvlGroups: map[uint64]Intervals{ tombstones: tombstones.NewTestMemTombstones([]tombstones.Intervals{
1: {{1, 3}}, tombstones.Intervals{{Mint: 1, Maxt: 3}},
2: {{1, 3}, {6, 10}}, tombstones.Intervals{{Mint: 1, Maxt: 3}, {Mint: 6, Maxt: 10}},
3: {{6, 10}}, tombstones.Intervals{{Mint: 6, Maxt: 10}},
}}, }),
queries: []query{ queries: []query{
{ {
mint: 2, mint: 2,
@ -524,6 +525,7 @@ func TestBlockQuerierDelete(t *testing.T) {
}, },
} }
fmt.Println("tombstones", cases.tombstones)
Outer: Outer:
for _, c := range cases.queries { for _, c := range cases.queries {
ir, cr, _, _ := createIdxChkReaders(t, cases.data) ir, cr, _, _ := createIdxChkReaders(t, cases.data)
@ -637,7 +639,7 @@ func TestBaseChunkSeries(t *testing.T) {
bcs := &baseChunkSeries{ bcs := &baseChunkSeries{
p: index.NewListPostings(tc.postings), p: index.NewListPostings(tc.postings),
index: mi, index: mi,
tombstones: newMemTombstones(), tombstones: tombstones.NewMemTombstones(),
} }
i := 0 i := 0
@ -1159,7 +1161,7 @@ func (m *mockChunkSeriesSet) Next() bool {
return m.i < len(m.l) return m.i < len(m.l)
} }
func (m *mockChunkSeriesSet) At() (labels.Labels, []chunks.Meta, Intervals) { func (m *mockChunkSeriesSet) At() (labels.Labels, []chunks.Meta, tombstones.Intervals) {
return m.l[m.i], m.cm[m.i], nil return m.l[m.i], m.cm[m.i], nil
} }
@ -1254,18 +1256,18 @@ func TestDeletedIterator(t *testing.T) {
} }
cases := []struct { cases := []struct {
r Intervals r tombstones.Intervals
}{ }{
{r: Intervals{{1, 20}}}, {r: tombstones.Intervals{{Mint: 1, Maxt: 20}}},
{r: Intervals{{1, 10}, {12, 20}, {21, 23}, {25, 30}}}, {r: tombstones.Intervals{{Mint: 1, Maxt: 10}, {Mint: 12, Maxt: 20}, {Mint: 21, Maxt: 23}, {Mint: 25, Maxt: 30}}},
{r: Intervals{{1, 10}, {12, 20}, {20, 30}}}, {r: tombstones.Intervals{{Mint: 1, Maxt: 10}, {Mint: 12, Maxt: 20}, {Mint: 20, Maxt: 30}}},
{r: Intervals{{1, 10}, {12, 23}, {25, 30}}}, {r: tombstones.Intervals{{Mint: 1, Maxt: 10}, {Mint: 12, Maxt: 23}, {Mint: 25, Maxt: 30}}},
{r: Intervals{{1, 23}, {12, 20}, {25, 30}}}, {r: tombstones.Intervals{{Mint: 1, Maxt: 23}, {Mint: 12, Maxt: 20}, {Mint: 25, Maxt: 30}}},
{r: Intervals{{1, 23}, {12, 20}, {25, 3000}}}, {r: tombstones.Intervals{{Mint: 1, Maxt: 23}, {Mint: 12, Maxt: 20}, {Mint: 25, Maxt: 3000}}},
{r: Intervals{{0, 2000}}}, {r: tombstones.Intervals{{Mint: 0, Maxt: 2000}}},
{r: Intervals{{500, 2000}}}, {r: tombstones.Intervals{{Mint: 500, Maxt: 2000}}},
{r: Intervals{{0, 200}}}, {r: tombstones.Intervals{{Mint: 0, Maxt: 200}}},
{r: Intervals{{1000, 20000}}}, {r: tombstones.Intervals{{Mint: 1000, Maxt: 20000}}},
} }
for _, c := range cases { for _, c := range cases {
@ -1275,7 +1277,7 @@ func TestDeletedIterator(t *testing.T) {
for it.Next() { for it.Next() {
i++ i++
for _, tr := range ranges { for _, tr := range ranges {
if tr.inBounds(i) { if tr.InBounds(i) {
i = tr.Maxt + 1 i = tr.Maxt + 1
ranges = ranges[1:] ranges = ranges[1:]
} }
@ -1290,7 +1292,7 @@ func TestDeletedIterator(t *testing.T) {
// There has been an extra call to Next(). // There has been an extra call to Next().
i++ i++
for _, tr := range ranges { for _, tr := range ranges {
if tr.inBounds(i) { if tr.InBounds(i) {
i = tr.Maxt + 1 i = tr.Maxt + 1
ranges = ranges[1:] ranges = ranges[1:]
} }

View file

@ -12,7 +12,7 @@
// See the License for the specific language governing permissions and // See the License for the specific language governing permissions and
// limitations under the License. // limitations under the License.
package tsdb package record
import ( import (
"math" "math"
@ -21,45 +21,64 @@ import (
"github.com/pkg/errors" "github.com/pkg/errors"
"github.com/prometheus/prometheus/tsdb/encoding" "github.com/prometheus/prometheus/tsdb/encoding"
"github.com/prometheus/prometheus/tsdb/labels" "github.com/prometheus/prometheus/tsdb/labels"
"github.com/prometheus/prometheus/tsdb/tombstones"
) )
// RecordType represents the data type of a record. // Type represents the data type of a record.
type RecordType uint8 type Type uint8
const ( const (
// RecordInvalid is returned for unrecognised WAL record types. // Invalid is returned for unrecognised WAL record types.
RecordInvalid RecordType = 255 Invalid Type = 255
// RecordSeries is used to match WAL records of type Series. // Series is used to match WAL records of type Series.
RecordSeries RecordType = 1 Series Type = 1
// RecordSamples is used to match WAL records of type Samples. // Samples is used to match WAL records of type Samples.
RecordSamples RecordType = 2 Samples Type = 2
// RecordTombstones is used to match WAL records of type Tombstones. // Tombstones is used to match WAL records of type Tombstones.
RecordTombstones RecordType = 3 Tombstones Type = 3
) )
// RecordDecoder decodes series, sample, and tombstone records. var (
// ErrNotFound is returned if a looked up resource was not found. Duplicate ErrNotFound from head.go.
ErrNotFound = errors.New("not found")
)
// RefSeries is the series labels with the series ID.
type RefSeries struct {
Ref uint64
Labels labels.Labels
}
// RefSample is a timestamp/value pair associated with a reference to a series.
type RefSample struct {
Ref uint64
T int64
V float64
}
// Decoder decodes series, sample, and tombstone records.
// The zero value is ready to use. // The zero value is ready to use.
type RecordDecoder struct { type Decoder struct {
} }
// Type returns the type of the record. // Type returns the type of the record.
// Return RecordInvalid if no valid record type is found. // Returns RecordInvalid if no valid record type is found.
func (d *RecordDecoder) Type(rec []byte) RecordType { func (d *Decoder) Type(rec []byte) Type {
if len(rec) < 1 { if len(rec) < 1 {
return RecordInvalid return Invalid
} }
switch t := RecordType(rec[0]); t { switch t := Type(rec[0]); t {
case RecordSeries, RecordSamples, RecordTombstones: case Series, Samples, Tombstones:
return t return t
} }
return RecordInvalid return Invalid
} }
// Series appends series in rec to the given slice. // Series appends series in rec to the given slice.
func (d *RecordDecoder) Series(rec []byte, series []RefSeries) ([]RefSeries, error) { func (d *Decoder) Series(rec []byte, series []RefSeries) ([]RefSeries, error) {
dec := encoding.Decbuf{B: rec} dec := encoding.Decbuf{B: rec}
if RecordType(dec.Byte()) != RecordSeries { if Type(dec.Byte()) != Series {
return nil, errors.New("invalid record type") return nil, errors.New("invalid record type")
} }
for len(dec.B) > 0 && dec.Err() == nil { for len(dec.B) > 0 && dec.Err() == nil {
@ -88,10 +107,10 @@ func (d *RecordDecoder) Series(rec []byte, series []RefSeries) ([]RefSeries, err
} }
// Samples appends samples in rec to the given slice. // Samples appends samples in rec to the given slice.
func (d *RecordDecoder) Samples(rec []byte, samples []RefSample) ([]RefSample, error) { func (d *Decoder) Samples(rec []byte, samples []RefSample) ([]RefSample, error) {
dec := encoding.Decbuf{B: rec} dec := encoding.Decbuf{B: rec}
if RecordType(dec.Byte()) != RecordSamples { if Type(dec.Byte()) != Samples {
return nil, errors.New("invalid record type") return nil, errors.New("invalid record type")
} }
if dec.Len() == 0 { if dec.Len() == 0 {
@ -123,16 +142,16 @@ func (d *RecordDecoder) Samples(rec []byte, samples []RefSample) ([]RefSample, e
} }
// Tombstones appends tombstones in rec to the given slice. // Tombstones appends tombstones in rec to the given slice.
func (d *RecordDecoder) Tombstones(rec []byte, tstones []Stone) ([]Stone, error) { func (d *Decoder) Tombstones(rec []byte, tstones []tombstones.Stone) ([]tombstones.Stone, error) {
dec := encoding.Decbuf{B: rec} dec := encoding.Decbuf{B: rec}
if RecordType(dec.Byte()) != RecordTombstones { if Type(dec.Byte()) != Tombstones {
return nil, errors.New("invalid record type") return nil, errors.New("invalid record type")
} }
for dec.Len() > 0 && dec.Err() == nil { for dec.Len() > 0 && dec.Err() == nil {
tstones = append(tstones, Stone{ tstones = append(tstones, tombstones.Stone{
ref: dec.Be64(), Ref: dec.Be64(),
intervals: Intervals{ Intervals: tombstones.Intervals{
{Mint: dec.Varint64(), Maxt: dec.Varint64()}, {Mint: dec.Varint64(), Maxt: dec.Varint64()},
}, },
}) })
@ -146,15 +165,15 @@ func (d *RecordDecoder) Tombstones(rec []byte, tstones []Stone) ([]Stone, error)
return tstones, nil return tstones, nil
} }
// RecordEncoder encodes series, sample, and tombstones records. // Encoder encodes series, sample, and tombstones records.
// The zero value is ready to use. // The zero value is ready to use.
type RecordEncoder struct { type Encoder struct {
} }
// Series appends the encoded series to b and returns the resulting slice. // Series appends the encoded series to b and returns the resulting slice.
func (e *RecordEncoder) Series(series []RefSeries, b []byte) []byte { func (e *Encoder) Series(series []RefSeries, b []byte) []byte {
buf := encoding.Encbuf{B: b} buf := encoding.Encbuf{B: b}
buf.PutByte(byte(RecordSeries)) buf.PutByte(byte(Series))
for _, s := range series { for _, s := range series {
buf.PutBE64(s.Ref) buf.PutBE64(s.Ref)
@ -169,9 +188,9 @@ func (e *RecordEncoder) Series(series []RefSeries, b []byte) []byte {
} }
// Samples appends the encoded samples to b and returns the resulting slice. // Samples appends the encoded samples to b and returns the resulting slice.
func (e *RecordEncoder) Samples(samples []RefSample, b []byte) []byte { func (e *Encoder) Samples(samples []RefSample, b []byte) []byte {
buf := encoding.Encbuf{B: b} buf := encoding.Encbuf{B: b}
buf.PutByte(byte(RecordSamples)) buf.PutByte(byte(Samples))
if len(samples) == 0 { if len(samples) == 0 {
return buf.Get() return buf.Get()
@ -193,13 +212,13 @@ func (e *RecordEncoder) Samples(samples []RefSample, b []byte) []byte {
} }
// Tombstones appends the encoded tombstones to b and returns the resulting slice. // Tombstones appends the encoded tombstones to b and returns the resulting slice.
func (e *RecordEncoder) Tombstones(tstones []Stone, b []byte) []byte { func (e *Encoder) Tombstones(tstones []tombstones.Stone, b []byte) []byte {
buf := encoding.Encbuf{B: b} buf := encoding.Encbuf{B: b}
buf.PutByte(byte(RecordTombstones)) buf.PutByte(byte(Tombstones))
for _, s := range tstones { for _, s := range tstones {
for _, iv := range s.intervals { for _, iv := range s.Intervals {
buf.PutBE64(s.ref) buf.PutBE64(s.Ref)
buf.PutVarint64(iv.Mint) buf.PutVarint64(iv.Mint)
buf.PutVarint64(iv.Maxt) buf.PutVarint64(iv.Maxt)
} }

View file

@ -12,7 +12,7 @@
// See the License for the specific language governing permissions and // See the License for the specific language governing permissions and
// limitations under the License. // limitations under the License.
package tsdb package record
import ( import (
"testing" "testing"
@ -20,12 +20,13 @@ import (
"github.com/pkg/errors" "github.com/pkg/errors"
"github.com/prometheus/prometheus/tsdb/encoding" "github.com/prometheus/prometheus/tsdb/encoding"
"github.com/prometheus/prometheus/tsdb/labels" "github.com/prometheus/prometheus/tsdb/labels"
"github.com/prometheus/prometheus/tsdb/tombstones"
"github.com/prometheus/prometheus/util/testutil" "github.com/prometheus/prometheus/util/testutil"
) )
func TestRecord_EncodeDecode(t *testing.T) { func TestRecord_EncodeDecode(t *testing.T) {
var enc RecordEncoder var enc Encoder
var dec RecordDecoder var dec Decoder
series := []RefSeries{ series := []RefSeries{
{ {
@ -54,31 +55,31 @@ func TestRecord_EncodeDecode(t *testing.T) {
// Intervals get split up into single entries. So we don't get back exactly // Intervals get split up into single entries. So we don't get back exactly
// what we put in. // what we put in.
tstones := []Stone{ tstones := []tombstones.Stone{
{ref: 123, intervals: Intervals{ {Ref: 123, Intervals: tombstones.Intervals{
{Mint: -1000, Maxt: 1231231}, {Mint: -1000, Maxt: 1231231},
{Mint: 5000, Maxt: 0}, {Mint: 5000, Maxt: 0},
}}, }},
{ref: 13, intervals: Intervals{ {Ref: 13, Intervals: tombstones.Intervals{
{Mint: -1000, Maxt: -11}, {Mint: -1000, Maxt: -11},
{Mint: 5000, Maxt: 1000}, {Mint: 5000, Maxt: 1000},
}}, }},
} }
decTstones, err := dec.Tombstones(enc.Tombstones(tstones, nil), nil) decTstones, err := dec.Tombstones(enc.Tombstones(tstones, nil), nil)
testutil.Ok(t, err) testutil.Ok(t, err)
testutil.Equals(t, []Stone{ testutil.Equals(t, []tombstones.Stone{
{ref: 123, intervals: Intervals{{Mint: -1000, Maxt: 1231231}}}, {Ref: 123, Intervals: tombstones.Intervals{{Mint: -1000, Maxt: 1231231}}},
{ref: 123, intervals: Intervals{{Mint: 5000, Maxt: 0}}}, {Ref: 123, Intervals: tombstones.Intervals{{Mint: 5000, Maxt: 0}}},
{ref: 13, intervals: Intervals{{Mint: -1000, Maxt: -11}}}, {Ref: 13, Intervals: tombstones.Intervals{{Mint: -1000, Maxt: -11}}},
{ref: 13, intervals: Intervals{{Mint: 5000, Maxt: 1000}}}, {Ref: 13, Intervals: tombstones.Intervals{{Mint: 5000, Maxt: 1000}}},
}, decTstones) }, decTstones)
} }
// TestRecord_Corruputed ensures that corrupted records return the correct error. // TestRecord_Corruputed ensures that corrupted records return the correct error.
// Bugfix check for pull/521 and pull/523. // Bugfix check for pull/521 and pull/523.
func TestRecord_Corruputed(t *testing.T) { func TestRecord_Corruputed(t *testing.T) {
var enc RecordEncoder var enc Encoder
var dec RecordDecoder var dec Decoder
t.Run("Test corrupted series record", func(t *testing.T) { t.Run("Test corrupted series record", func(t *testing.T) {
series := []RefSeries{ series := []RefSeries{
@ -104,8 +105,8 @@ func TestRecord_Corruputed(t *testing.T) {
}) })
t.Run("Test corrupted tombstone record", func(t *testing.T) { t.Run("Test corrupted tombstone record", func(t *testing.T) {
tstones := []Stone{ tstones := []tombstones.Stone{
{ref: 123, intervals: Intervals{ {Ref: 123, Intervals: tombstones.Intervals{
{Mint: -1000, Maxt: 1231231}, {Mint: -1000, Maxt: 1231231},
{Mint: 5000, Maxt: 0}, {Mint: 5000, Maxt: 0},
}}, }},

View file

@ -11,11 +11,13 @@
// See the License for the specific language governing permissions and // See the License for the specific language governing permissions and
// limitations under the License. // limitations under the License.
package tsdb package tombstones
import ( import (
"encoding/binary" "encoding/binary"
"fmt" "fmt"
"hash"
"hash/crc32"
"io" "io"
"io/ioutil" "io/ioutil"
"os" "os"
@ -30,7 +32,7 @@ import (
"github.com/prometheus/prometheus/tsdb/fileutil" "github.com/prometheus/prometheus/tsdb/fileutil"
) )
const tombstoneFilename = "tombstones" const TombstonesFilename = "tombstones"
const ( const (
// MagicTombstone is 4 bytes at the head of a tombstone file. // MagicTombstone is 4 bytes at the head of a tombstone file.
@ -39,8 +41,23 @@ const (
tombstoneFormatV1 = 1 tombstoneFormatV1 = 1
) )
// TombstoneReader gives access to tombstone intervals by series reference. // The table gets initialized with sync.Once but may still cause a race
type TombstoneReader interface { // with any other use of the crc32 package anywhere. Thus we initialize it
// before.
var castagnoliTable *crc32.Table
func init() {
castagnoliTable = crc32.MakeTable(crc32.Castagnoli)
}
// newCRC32 initializes a CRC32 hash with a preconfigured polynomial, so the
// polynomial may be easily changed in one location at a later time, if necessary.
func newCRC32() hash.Hash32 {
return crc32.New(castagnoliTable)
}
// Reader gives access to tombstone intervals by series reference.
type Reader interface {
// Get returns deletion intervals for the series with the given reference. // Get returns deletion intervals for the series with the given reference.
Get(ref uint64) (Intervals, error) Get(ref uint64) (Intervals, error)
@ -54,8 +71,8 @@ type TombstoneReader interface {
Close() error Close() error
} }
func writeTombstoneFile(logger log.Logger, dir string, tr TombstoneReader) (int64, error) { func WriteFile(logger log.Logger, dir string, tr Reader) (int64, error) {
path := filepath.Join(dir, tombstoneFilename) path := filepath.Join(dir, TombstonesFilename)
tmp := path + ".tmp" tmp := path + ".tmp"
hash := newCRC32() hash := newCRC32()
var size int var size int
@ -129,14 +146,14 @@ func writeTombstoneFile(logger log.Logger, dir string, tr TombstoneReader) (int6
// Stone holds the information on the posting and time-range // Stone holds the information on the posting and time-range
// that is deleted. // that is deleted.
type Stone struct { type Stone struct {
ref uint64 Ref uint64
intervals Intervals Intervals Intervals
} }
func readTombstones(dir string) (TombstoneReader, int64, error) { func ReadTombstones(dir string) (Reader, int64, error) {
b, err := ioutil.ReadFile(filepath.Join(dir, tombstoneFilename)) b, err := ioutil.ReadFile(filepath.Join(dir, TombstonesFilename))
if os.IsNotExist(err) { if os.IsNotExist(err) {
return newMemTombstones(), 0, nil return NewMemTombstones(), 0, nil
} else if err != nil { } else if err != nil {
return nil, 0, err return nil, 0, err
} }
@ -166,7 +183,7 @@ func readTombstones(dir string) (TombstoneReader, int64, error) {
return nil, 0, errors.New("checksum did not match") return nil, 0, errors.New("checksum did not match")
} }
stonesMap := newMemTombstones() stonesMap := NewMemTombstones()
for d.Len() > 0 { for d.Len() > 0 {
k := d.Uvarint64() k := d.Uvarint64()
@ -176,7 +193,7 @@ func readTombstones(dir string) (TombstoneReader, int64, error) {
return nil, 0, d.Err() return nil, 0, d.Err()
} }
stonesMap.addInterval(k, Interval{mint, maxt}) stonesMap.AddInterval(k, Interval{mint, maxt})
} }
return stonesMap, int64(len(b)), nil return stonesMap, int64(len(b)), nil
@ -187,12 +204,22 @@ type memTombstones struct {
mtx sync.RWMutex mtx sync.RWMutex
} }
// newMemTombstones creates new in memory TombstoneReader // NewMemTombstones creates new in memory Tombstone Reader
// that allows adding new intervals. // that allows adding new intervals.
func newMemTombstones() *memTombstones { func NewMemTombstones() *memTombstones {
return &memTombstones{intvlGroups: make(map[uint64]Intervals)} return &memTombstones{intvlGroups: make(map[uint64]Intervals)}
} }
func NewTestMemTombstones(intervals []Intervals) *memTombstones {
ret := NewMemTombstones()
for i, intervalsGroup := range intervals {
for _, interval := range intervalsGroup {
ret.AddInterval(uint64(i+1), interval)
}
}
return ret
}
func (t *memTombstones) Get(ref uint64) (Intervals, error) { func (t *memTombstones) Get(ref uint64) (Intervals, error) {
t.mtx.RLock() t.mtx.RLock()
defer t.mtx.RUnlock() defer t.mtx.RUnlock()
@ -221,12 +248,13 @@ func (t *memTombstones) Total() uint64 {
return total return total
} }
// addInterval to an existing memTombstones // AddInterval to an existing memTombstones.
func (t *memTombstones) addInterval(ref uint64, itvs ...Interval) { func (t *memTombstones) AddInterval(ref uint64, itvs ...Interval) {
t.mtx.Lock() t.mtx.Lock()
defer t.mtx.Unlock() defer t.mtx.Unlock()
for _, itv := range itvs { for _, itv := range itvs {
t.intvlGroups[ref] = t.intvlGroups[ref].add(itv) fmt.Println("adding interval to ref: ", ref)
t.intvlGroups[ref] = t.intvlGroups[ref].Add(itv)
} }
} }
@ -239,13 +267,13 @@ type Interval struct {
Mint, Maxt int64 Mint, Maxt int64
} }
func (tr Interval) inBounds(t int64) bool { func (tr Interval) InBounds(t int64) bool {
return t >= tr.Mint && t <= tr.Maxt return t >= tr.Mint && t <= tr.Maxt
} }
func (tr Interval) isSubrange(dranges Intervals) bool { func (tr Interval) IsSubrange(dranges Intervals) bool {
for _, r := range dranges { for _, r := range dranges {
if r.inBounds(tr.Mint) && r.inBounds(tr.Maxt) { if r.InBounds(tr.Mint) && r.InBounds(tr.Maxt) {
return true return true
} }
} }
@ -256,12 +284,12 @@ func (tr Interval) isSubrange(dranges Intervals) bool {
// Intervals represents a set of increasing and non-overlapping time-intervals. // Intervals represents a set of increasing and non-overlapping time-intervals.
type Intervals []Interval type Intervals []Interval
// add the new time-range to the existing ones. // Add the new time-range to the existing ones.
// The existing ones must be sorted. // The existing ones must be sorted.
func (itvs Intervals) add(n Interval) Intervals { func (itvs Intervals) Add(n Interval) Intervals {
for i, r := range itvs { for i, r := range itvs {
// TODO(gouthamve): Make this codepath easier to digest. // TODO(gouthamve): Make this codepath easier to digest.
if r.inBounds(n.Mint-1) || r.inBounds(n.Mint) { if r.InBounds(n.Mint-1) || r.InBounds(n.Mint) {
if n.Maxt > r.Maxt { if n.Maxt > r.Maxt {
itvs[i].Maxt = n.Maxt itvs[i].Maxt = n.Maxt
} }
@ -282,7 +310,7 @@ func (itvs Intervals) add(n Interval) Intervals {
return itvs return itvs
} }
if r.inBounds(n.Maxt+1) || r.inBounds(n.Maxt) { if r.InBounds(n.Maxt+1) || r.InBounds(n.Maxt) {
if n.Mint < r.Maxt { if n.Mint < r.Maxt {
itvs[i].Mint = n.Mint itvs[i].Mint = n.Mint
} }

View file

@ -11,7 +11,7 @@
// See the License for the specific language governing permissions and // See the License for the specific language governing permissions and
// limitations under the License. // limitations under the License.
package tsdb package tombstones
import ( import (
"io/ioutil" "io/ioutil"
@ -33,7 +33,7 @@ func TestWriteAndReadbackTombStones(t *testing.T) {
ref := uint64(0) ref := uint64(0)
stones := newMemTombstones() stones := NewMemTombstones()
// Generate the tombstones. // Generate the tombstones.
for i := 0; i < 100; i++ { for i := 0; i < 100; i++ {
ref += uint64(rand.Int31n(10)) + 1 ref += uint64(rand.Int31n(10)) + 1
@ -41,16 +41,16 @@ func TestWriteAndReadbackTombStones(t *testing.T) {
dranges := make(Intervals, 0, numRanges) dranges := make(Intervals, 0, numRanges)
mint := rand.Int63n(time.Now().UnixNano()) mint := rand.Int63n(time.Now().UnixNano())
for j := 0; j < numRanges; j++ { for j := 0; j < numRanges; j++ {
dranges = dranges.add(Interval{mint, mint + rand.Int63n(1000)}) dranges = dranges.Add(Interval{mint, mint + rand.Int63n(1000)})
mint += rand.Int63n(1000) + 1 mint += rand.Int63n(1000) + 1
} }
stones.addInterval(ref, dranges...) stones.AddInterval(ref, dranges...)
} }
_, err := writeTombstoneFile(log.NewNopLogger(), tmpdir, stones) _, err := WriteFile(log.NewNopLogger(), tmpdir, stones)
testutil.Ok(t, err) testutil.Ok(t, err)
restr, _, err := readTombstones(tmpdir) restr, _, err := ReadTombstones(tmpdir)
testutil.Ok(t, err) testutil.Ok(t, err)
// Compare the two readers. // Compare the two readers.
@ -122,20 +122,20 @@ func TestAddingNewIntervals(t *testing.T) {
for _, c := range cases { for _, c := range cases {
testutil.Equals(t, c.exp, c.exist.add(c.new)) testutil.Equals(t, c.exp, c.exist.Add(c.new))
} }
} }
// TestMemTombstonesConcurrency to make sure they are safe to access from different goroutines. // TestMemTombstonesConcurrency to make sure they are safe to access from different goroutines.
func TestMemTombstonesConcurrency(t *testing.T) { func TestMemTombstonesConcurrency(t *testing.T) {
tomb := newMemTombstones() tomb := NewMemTombstones()
totalRuns := 100 totalRuns := 100
var wg sync.WaitGroup var wg sync.WaitGroup
wg.Add(2) wg.Add(2)
go func() { go func() {
for x := 0; x < totalRuns; x++ { for x := 0; x < totalRuns; x++ {
tomb.addInterval(uint64(x), Interval{int64(x), int64(x)}) tomb.AddInterval(uint64(x), Interval{int64(x), int64(x)})
} }
wg.Done() wg.Done()
}() }()

View file

@ -34,6 +34,8 @@ import (
"github.com/prometheus/prometheus/tsdb/encoding" "github.com/prometheus/prometheus/tsdb/encoding"
"github.com/prometheus/prometheus/tsdb/fileutil" "github.com/prometheus/prometheus/tsdb/fileutil"
"github.com/prometheus/prometheus/tsdb/labels" "github.com/prometheus/prometheus/tsdb/labels"
"github.com/prometheus/prometheus/tsdb/record"
"github.com/prometheus/prometheus/tsdb/tombstones"
"github.com/prometheus/prometheus/tsdb/wal" "github.com/prometheus/prometheus/tsdb/wal"
) )
@ -89,9 +91,9 @@ func newWalMetrics(wal *SegmentWAL, r prometheus.Registerer) *walMetrics {
// DEPRECATED: use wal pkg combined with the record codex instead. // DEPRECATED: use wal pkg combined with the record codex instead.
type WAL interface { type WAL interface {
Reader() WALReader Reader() WALReader
LogSeries([]RefSeries) error LogSeries([]record.RefSeries) error
LogSamples([]RefSample) error LogSamples([]record.RefSample) error
LogDeletes([]Stone) error LogDeletes([]tombstones.Stone) error
Truncate(mint int64, keep func(uint64) bool) error Truncate(mint int64, keep func(uint64) bool) error
Close() error Close() error
} }
@ -99,27 +101,12 @@ type WAL interface {
// WALReader reads entries from a WAL. // WALReader reads entries from a WAL.
type WALReader interface { type WALReader interface {
Read( Read(
seriesf func([]RefSeries), seriesf func([]record.RefSeries),
samplesf func([]RefSample), samplesf func([]record.RefSample),
deletesf func([]Stone), deletesf func([]tombstones.Stone),
) error ) error
} }
// RefSeries is the series labels with the series ID.
type RefSeries struct {
Ref uint64
Labels labels.Labels
}
// RefSample is a timestamp/value pair associated with a reference to a series.
type RefSample struct {
Ref uint64
T int64
V float64
series *memSeries
}
// segmentFile wraps a file object of a segment and tracks the highest timestamp // segmentFile wraps a file object of a segment and tracks the highest timestamp
// it contains. During WAL truncating, all segments with no higher timestamp than // it contains. During WAL truncating, all segments with no higher timestamp than
// the truncation threshold can be compacted. // the truncation threshold can be compacted.
@ -240,9 +227,9 @@ type repairingWALReader struct {
} }
func (r *repairingWALReader) Read( func (r *repairingWALReader) Read(
seriesf func([]RefSeries), seriesf func([]record.RefSeries),
samplesf func([]RefSample), samplesf func([]record.RefSample),
deletesf func([]Stone), deletesf func([]tombstones.Stone),
) error { ) error {
err := r.r.Read(seriesf, samplesf, deletesf) err := r.r.Read(seriesf, samplesf, deletesf)
if err == nil { if err == nil {
@ -348,8 +335,8 @@ func (w *SegmentWAL) Truncate(mint int64, keep func(uint64) bool) error {
var ( var (
csf = newSegmentFile(f) csf = newSegmentFile(f)
crc32 = newCRC32() crc32 = newCRC32()
decSeries = []RefSeries{} decSeries = []record.RefSeries{}
activeSeries = []RefSeries{} activeSeries = []record.RefSeries{}
) )
for r.next() { for r.next() {
@ -433,7 +420,7 @@ func (w *SegmentWAL) Truncate(mint int64, keep func(uint64) bool) error {
// LogSeries writes a batch of new series labels to the log. // LogSeries writes a batch of new series labels to the log.
// The series have to be ordered. // The series have to be ordered.
func (w *SegmentWAL) LogSeries(series []RefSeries) error { func (w *SegmentWAL) LogSeries(series []record.RefSeries) error {
buf := w.getBuffer() buf := w.getBuffer()
flag := w.encodeSeries(buf, series) flag := w.encodeSeries(buf, series)
@ -460,7 +447,7 @@ func (w *SegmentWAL) LogSeries(series []RefSeries) error {
} }
// LogSamples writes a batch of new samples to the log. // LogSamples writes a batch of new samples to the log.
func (w *SegmentWAL) LogSamples(samples []RefSample) error { func (w *SegmentWAL) LogSamples(samples []record.RefSample) error {
buf := w.getBuffer() buf := w.getBuffer()
flag := w.encodeSamples(buf, samples) flag := w.encodeSamples(buf, samples)
@ -486,7 +473,7 @@ func (w *SegmentWAL) LogSamples(samples []RefSample) error {
} }
// LogDeletes write a batch of new deletes to the log. // LogDeletes write a batch of new deletes to the log.
func (w *SegmentWAL) LogDeletes(stones []Stone) error { func (w *SegmentWAL) LogDeletes(stones []tombstones.Stone) error {
buf := w.getBuffer() buf := w.getBuffer()
flag := w.encodeDeletes(buf, stones) flag := w.encodeDeletes(buf, stones)
@ -504,7 +491,7 @@ func (w *SegmentWAL) LogDeletes(stones []Stone) error {
tf := w.head() tf := w.head()
for _, s := range stones { for _, s := range stones {
for _, iv := range s.intervals { for _, iv := range s.Intervals {
if tf.maxTime < iv.Maxt { if tf.maxTime < iv.Maxt {
tf.maxTime = iv.Maxt tf.maxTime = iv.Maxt
} }
@ -797,7 +784,7 @@ const (
walDeletesSimple = 1 walDeletesSimple = 1
) )
func (w *SegmentWAL) encodeSeries(buf *encoding.Encbuf, series []RefSeries) uint8 { func (w *SegmentWAL) encodeSeries(buf *encoding.Encbuf, series []record.RefSeries) uint8 {
for _, s := range series { for _, s := range series {
buf.PutBE64(s.Ref) buf.PutBE64(s.Ref)
buf.PutUvarint(len(s.Labels)) buf.PutUvarint(len(s.Labels))
@ -810,7 +797,7 @@ func (w *SegmentWAL) encodeSeries(buf *encoding.Encbuf, series []RefSeries) uint
return walSeriesSimple return walSeriesSimple
} }
func (w *SegmentWAL) encodeSamples(buf *encoding.Encbuf, samples []RefSample) uint8 { func (w *SegmentWAL) encodeSamples(buf *encoding.Encbuf, samples []record.RefSample) uint8 {
if len(samples) == 0 { if len(samples) == 0 {
return walSamplesSimple return walSamplesSimple
} }
@ -831,10 +818,10 @@ func (w *SegmentWAL) encodeSamples(buf *encoding.Encbuf, samples []RefSample) ui
return walSamplesSimple return walSamplesSimple
} }
func (w *SegmentWAL) encodeDeletes(buf *encoding.Encbuf, stones []Stone) uint8 { func (w *SegmentWAL) encodeDeletes(buf *encoding.Encbuf, stones []tombstones.Stone) uint8 {
for _, s := range stones { for _, s := range stones {
for _, iv := range s.intervals { for _, iv := range s.Intervals {
buf.PutBE64(s.ref) buf.PutBE64(s.Ref)
buf.PutVarint64(iv.Mint) buf.PutVarint64(iv.Mint)
buf.PutVarint64(iv.Maxt) buf.PutVarint64(iv.Maxt)
} }
@ -877,9 +864,9 @@ func (r *walReader) Err() error {
} }
func (r *walReader) Read( func (r *walReader) Read(
seriesf func([]RefSeries), seriesf func([]record.RefSeries),
samplesf func([]RefSample), samplesf func([]record.RefSample),
deletesf func([]Stone), deletesf func([]tombstones.Stone),
) error { ) error {
// Concurrency for replaying the WAL is very limited. We at least split out decoding and // Concurrency for replaying the WAL is very limited. We at least split out decoding and
// processing into separate threads. // processing into separate threads.
@ -898,19 +885,19 @@ func (r *walReader) Read(
for x := range datac { for x := range datac {
switch v := x.(type) { switch v := x.(type) {
case []RefSeries: case []record.RefSeries:
if seriesf != nil { if seriesf != nil {
seriesf(v) seriesf(v)
} }
//lint:ignore SA6002 safe to ignore and actually fixing it has some performance penalty. //lint:ignore SA6002 safe to ignore and actually fixing it has some performance penalty.
seriesPool.Put(v[:0]) seriesPool.Put(v[:0])
case []RefSample: case []record.RefSample:
if samplesf != nil { if samplesf != nil {
samplesf(v) samplesf(v)
} }
//lint:ignore SA6002 safe to ignore and actually fixing it has some performance penalty. //lint:ignore SA6002 safe to ignore and actually fixing it has some performance penalty.
samplePool.Put(v[:0]) samplePool.Put(v[:0])
case []Stone: case []tombstones.Stone:
if deletesf != nil { if deletesf != nil {
deletesf(v) deletesf(v)
} }
@ -931,11 +918,11 @@ func (r *walReader) Read(
// Those should generally be catched by entry decoding before. // Those should generally be catched by entry decoding before.
switch et { switch et {
case WALEntrySeries: case WALEntrySeries:
var series []RefSeries var series []record.RefSeries
if v := seriesPool.Get(); v == nil { if v := seriesPool.Get(); v == nil {
series = make([]RefSeries, 0, 512) series = make([]record.RefSeries, 0, 512)
} else { } else {
series = v.([]RefSeries) series = v.([]record.RefSeries)
} }
err = r.decodeSeries(flag, b, &series) err = r.decodeSeries(flag, b, &series)
@ -952,11 +939,11 @@ func (r *walReader) Read(
} }
} }
case WALEntrySamples: case WALEntrySamples:
var samples []RefSample var samples []record.RefSample
if v := samplePool.Get(); v == nil { if v := samplePool.Get(); v == nil {
samples = make([]RefSample, 0, 512) samples = make([]record.RefSample, 0, 512)
} else { } else {
samples = v.([]RefSample) samples = v.([]record.RefSample)
} }
err = r.decodeSamples(flag, b, &samples) err = r.decodeSamples(flag, b, &samples)
@ -974,11 +961,11 @@ func (r *walReader) Read(
} }
} }
case WALEntryDeletes: case WALEntryDeletes:
var deletes []Stone var deletes []tombstones.Stone
if v := deletePool.Get(); v == nil { if v := deletePool.Get(); v == nil {
deletes = make([]Stone, 0, 512) deletes = make([]tombstones.Stone, 0, 512)
} else { } else {
deletes = v.([]Stone) deletes = v.([]tombstones.Stone)
} }
err = r.decodeDeletes(flag, b, &deletes) err = r.decodeDeletes(flag, b, &deletes)
@ -991,7 +978,7 @@ func (r *walReader) Read(
// Update the times for the WAL segment file. // Update the times for the WAL segment file.
cf := r.current() cf := r.current()
for _, s := range deletes { for _, s := range deletes {
for _, iv := range s.intervals { for _, iv := range s.Intervals {
if cf.maxTime < iv.Maxt { if cf.maxTime < iv.Maxt {
cf.maxTime = iv.Maxt cf.maxTime = iv.Maxt
} }
@ -1128,7 +1115,7 @@ func (r *walReader) entry(cr io.Reader) (WALEntryType, byte, []byte, error) {
return etype, flag, buf, nil return etype, flag, buf, nil
} }
func (r *walReader) decodeSeries(flag byte, b []byte, res *[]RefSeries) error { func (r *walReader) decodeSeries(flag byte, b []byte, res *[]record.RefSeries) error {
dec := encoding.Decbuf{B: b} dec := encoding.Decbuf{B: b}
for len(dec.B) > 0 && dec.Err() == nil { for len(dec.B) > 0 && dec.Err() == nil {
@ -1142,7 +1129,7 @@ func (r *walReader) decodeSeries(flag byte, b []byte, res *[]RefSeries) error {
} }
sort.Sort(lset) sort.Sort(lset)
*res = append(*res, RefSeries{ *res = append(*res, record.RefSeries{
Ref: ref, Ref: ref,
Labels: lset, Labels: lset,
}) })
@ -1156,7 +1143,7 @@ func (r *walReader) decodeSeries(flag byte, b []byte, res *[]RefSeries) error {
return nil return nil
} }
func (r *walReader) decodeSamples(flag byte, b []byte, res *[]RefSample) error { func (r *walReader) decodeSamples(flag byte, b []byte, res *[]record.RefSample) error {
if len(b) == 0 { if len(b) == 0 {
return nil return nil
} }
@ -1172,7 +1159,7 @@ func (r *walReader) decodeSamples(flag byte, b []byte, res *[]RefSample) error {
dtime := dec.Varint64() dtime := dec.Varint64()
val := dec.Be64() val := dec.Be64()
*res = append(*res, RefSample{ *res = append(*res, record.RefSample{
Ref: uint64(int64(baseRef) + dref), Ref: uint64(int64(baseRef) + dref),
T: baseTime + dtime, T: baseTime + dtime,
V: math.Float64frombits(val), V: math.Float64frombits(val),
@ -1188,13 +1175,13 @@ func (r *walReader) decodeSamples(flag byte, b []byte, res *[]RefSample) error {
return nil return nil
} }
func (r *walReader) decodeDeletes(flag byte, b []byte, res *[]Stone) error { func (r *walReader) decodeDeletes(flag byte, b []byte, res *[]tombstones.Stone) error {
dec := &encoding.Decbuf{B: b} dec := &encoding.Decbuf{B: b}
for dec.Len() > 0 && dec.Err() == nil { for dec.Len() > 0 && dec.Err() == nil {
*res = append(*res, Stone{ *res = append(*res, tombstones.Stone{
ref: dec.Be64(), Ref: dec.Be64(),
intervals: Intervals{ Intervals: tombstones.Intervals{
{Mint: dec.Varint64(), Maxt: dec.Varint64()}, {Mint: dec.Varint64(), Maxt: dec.Varint64()},
}, },
}) })
@ -1274,23 +1261,23 @@ func MigrateWAL(logger log.Logger, dir string) (err error) {
rdr := w.Reader() rdr := w.Reader()
var ( var (
enc RecordEncoder enc record.Encoder
b []byte b []byte
) )
decErr := rdr.Read( decErr := rdr.Read(
func(s []RefSeries) { func(s []record.RefSeries) {
if err != nil { if err != nil {
return return
} }
err = repl.Log(enc.Series(s, b[:0])) err = repl.Log(enc.Series(s, b[:0]))
}, },
func(s []RefSample) { func(s []record.RefSample) {
if err != nil { if err != nil {
return return
} }
err = repl.Log(enc.Samples(s, b[:0])) err = repl.Log(enc.Samples(s, b[:0]))
}, },
func(s []Stone) { func(s []tombstones.Stone) {
if err != nil { if err != nil {
return return
} }

View file

@ -12,7 +12,7 @@
// See the License for the specific language governing permissions and // See the License for the specific language governing permissions and
// limitations under the License. // limitations under the License.
package tsdb package wal
import ( import (
"fmt" "fmt"
@ -27,7 +27,8 @@ import (
"github.com/pkg/errors" "github.com/pkg/errors"
tsdb_errors "github.com/prometheus/prometheus/tsdb/errors" tsdb_errors "github.com/prometheus/prometheus/tsdb/errors"
"github.com/prometheus/prometheus/tsdb/fileutil" "github.com/prometheus/prometheus/tsdb/fileutil"
"github.com/prometheus/prometheus/tsdb/wal" "github.com/prometheus/prometheus/tsdb/record"
"github.com/prometheus/prometheus/tsdb/tombstones"
) )
// CheckpointStats returns stats about a created checkpoint. // CheckpointStats returns stats about a created checkpoint.
@ -63,7 +64,7 @@ func LastCheckpoint(dir string) (string, int, error) {
} }
return filepath.Join(dir, fi.Name()), idx, nil return filepath.Join(dir, fi.Name()), idx, nil
} }
return "", 0, ErrNotFound return "", 0, record.ErrNotFound
} }
// DeleteCheckpoints deletes all checkpoints in a directory below a given index. // DeleteCheckpoints deletes all checkpoints in a directory below a given index.
@ -99,15 +100,15 @@ const checkpointPrefix = "checkpoint."
// segmented format as the original WAL itself. // segmented format as the original WAL itself.
// This makes it easy to read it through the WAL package and concatenate // This makes it easy to read it through the WAL package and concatenate
// it with the original WAL. // it with the original WAL.
func Checkpoint(w *wal.WAL, from, to int, keep func(id uint64) bool, mint int64) (*CheckpointStats, error) { func Checkpoint(w *WAL, from, to int, keep func(id uint64) bool, mint int64) (*CheckpointStats, error) {
stats := &CheckpointStats{} stats := &CheckpointStats{}
var sgmReader io.ReadCloser var sgmReader io.ReadCloser
{ {
var sgmRange []wal.SegmentRange var sgmRange []SegmentRange
dir, idx, err := LastCheckpoint(w.Dir()) dir, idx, err := LastCheckpoint(w.Dir())
if err != nil && err != ErrNotFound { if err != nil && err != record.ErrNotFound {
return nil, errors.Wrap(err, "find last checkpoint") return nil, errors.Wrap(err, "find last checkpoint")
} }
last := idx + 1 last := idx + 1
@ -118,11 +119,11 @@ func Checkpoint(w *wal.WAL, from, to int, keep func(id uint64) bool, mint int64)
// Ignore WAL files below the checkpoint. They shouldn't exist to begin with. // Ignore WAL files below the checkpoint. They shouldn't exist to begin with.
from = last from = last
sgmRange = append(sgmRange, wal.SegmentRange{Dir: dir, Last: math.MaxInt32}) sgmRange = append(sgmRange, SegmentRange{Dir: dir, Last: math.MaxInt32})
} }
sgmRange = append(sgmRange, wal.SegmentRange{Dir: w.Dir(), First: from, Last: to}) sgmRange = append(sgmRange, SegmentRange{Dir: w.Dir(), First: from, Last: to})
sgmReader, err = wal.NewSegmentsRangeReader(sgmRange...) sgmReader, err = NewSegmentsRangeReader(sgmRange...)
if err != nil { if err != nil {
return nil, errors.Wrap(err, "create segment reader") return nil, errors.Wrap(err, "create segment reader")
} }
@ -135,7 +136,7 @@ func Checkpoint(w *wal.WAL, from, to int, keep func(id uint64) bool, mint int64)
if err := os.MkdirAll(cpdirtmp, 0777); err != nil { if err := os.MkdirAll(cpdirtmp, 0777); err != nil {
return nil, errors.Wrap(err, "create checkpoint dir") return nil, errors.Wrap(err, "create checkpoint dir")
} }
cp, err := wal.New(nil, nil, cpdirtmp, w.CompressionEnabled()) cp, err := New(nil, nil, cpdirtmp, w.CompressionEnabled())
if err != nil { if err != nil {
return nil, errors.Wrap(err, "open checkpoint") return nil, errors.Wrap(err, "open checkpoint")
} }
@ -146,14 +147,14 @@ func Checkpoint(w *wal.WAL, from, to int, keep func(id uint64) bool, mint int64)
os.RemoveAll(cpdirtmp) os.RemoveAll(cpdirtmp)
}() }()
r := wal.NewReader(sgmReader) r := NewReader(sgmReader)
var ( var (
series []RefSeries series []record.RefSeries
samples []RefSample samples []record.RefSample
tstones []Stone tstones []tombstones.Stone
dec RecordDecoder dec record.Decoder
enc RecordEncoder enc record.Encoder
buf []byte buf []byte
recs [][]byte recs [][]byte
) )
@ -167,7 +168,7 @@ func Checkpoint(w *wal.WAL, from, to int, keep func(id uint64) bool, mint int64)
rec := r.Record() rec := r.Record()
switch dec.Type(rec) { switch dec.Type(rec) {
case RecordSeries: case record.Series:
series, err = dec.Series(rec, series) series, err = dec.Series(rec, series)
if err != nil { if err != nil {
return nil, errors.Wrap(err, "decode series") return nil, errors.Wrap(err, "decode series")
@ -185,7 +186,7 @@ func Checkpoint(w *wal.WAL, from, to int, keep func(id uint64) bool, mint int64)
stats.TotalSeries += len(series) stats.TotalSeries += len(series)
stats.DroppedSeries += len(series) - len(repl) stats.DroppedSeries += len(series) - len(repl)
case RecordSamples: case record.Samples:
samples, err = dec.Samples(rec, samples) samples, err = dec.Samples(rec, samples)
if err != nil { if err != nil {
return nil, errors.Wrap(err, "decode samples") return nil, errors.Wrap(err, "decode samples")
@ -203,7 +204,7 @@ func Checkpoint(w *wal.WAL, from, to int, keep func(id uint64) bool, mint int64)
stats.TotalSamples += len(samples) stats.TotalSamples += len(samples)
stats.DroppedSamples += len(samples) - len(repl) stats.DroppedSamples += len(samples) - len(repl)
case RecordTombstones: case record.Tombstones:
tstones, err = dec.Tombstones(rec, tstones) tstones, err = dec.Tombstones(rec, tstones)
if err != nil { if err != nil {
return nil, errors.Wrap(err, "decode deletes") return nil, errors.Wrap(err, "decode deletes")
@ -211,7 +212,7 @@ func Checkpoint(w *wal.WAL, from, to int, keep func(id uint64) bool, mint int64)
// Drop irrelevant tombstones in place. // Drop irrelevant tombstones in place.
repl := tstones[:0] repl := tstones[:0]
for _, s := range tstones { for _, s := range tstones {
for _, iv := range s.intervals { for _, iv := range s.Intervals {
if iv.Maxt >= mint { if iv.Maxt >= mint {
repl = append(repl, s) repl = append(repl, s)
break break

View file

@ -12,7 +12,7 @@
// See the License for the specific language governing permissions and // See the License for the specific language governing permissions and
// limitations under the License. // limitations under the License.
package tsdb package wal
import ( import (
"fmt" "fmt"
@ -25,7 +25,7 @@ import (
"github.com/pkg/errors" "github.com/pkg/errors"
"github.com/prometheus/prometheus/tsdb/fileutil" "github.com/prometheus/prometheus/tsdb/fileutil"
"github.com/prometheus/prometheus/tsdb/labels" "github.com/prometheus/prometheus/tsdb/labels"
"github.com/prometheus/prometheus/tsdb/wal" "github.com/prometheus/prometheus/tsdb/record"
"github.com/prometheus/prometheus/util/testutil" "github.com/prometheus/prometheus/util/testutil"
) )
@ -37,7 +37,7 @@ func TestLastCheckpoint(t *testing.T) {
}() }()
_, _, err = LastCheckpoint(dir) _, _, err = LastCheckpoint(dir)
testutil.Equals(t, ErrNotFound, err) testutil.Equals(t, record.ErrNotFound, err)
testutil.Ok(t, os.MkdirAll(filepath.Join(dir, "checkpoint.0000"), 0777)) testutil.Ok(t, os.MkdirAll(filepath.Join(dir, "checkpoint.0000"), 0777))
s, k, err := LastCheckpoint(dir) s, k, err := LastCheckpoint(dir)
@ -94,18 +94,18 @@ func TestCheckpoint(t *testing.T) {
testutil.Ok(t, os.RemoveAll(dir)) testutil.Ok(t, os.RemoveAll(dir))
}() }()
var enc RecordEncoder var enc record.Encoder
// Create a dummy segment to bump the initial number. // Create a dummy segment to bump the initial number.
seg, err := wal.CreateSegment(dir, 100) seg, err := CreateSegment(dir, 100)
testutil.Ok(t, err) testutil.Ok(t, err)
testutil.Ok(t, seg.Close()) testutil.Ok(t, seg.Close())
// Manually create checkpoint for 99 and earlier. // Manually create checkpoint for 99 and earlier.
w, err := wal.New(nil, nil, filepath.Join(dir, "checkpoint.0099"), compress) w, err := New(nil, nil, filepath.Join(dir, "checkpoint.0099"), compress)
testutil.Ok(t, err) testutil.Ok(t, err)
// Add some data we expect to be around later. // Add some data we expect to be around later.
err = w.Log(enc.Series([]RefSeries{ err = w.Log(enc.Series([]record.RefSeries{
{Ref: 0, Labels: labels.FromStrings("a", "b", "c", "0")}, {Ref: 0, Labels: labels.FromStrings("a", "b", "c", "0")},
{Ref: 1, Labels: labels.FromStrings("a", "b", "c", "1")}, {Ref: 1, Labels: labels.FromStrings("a", "b", "c", "1")},
}, nil)) }, nil))
@ -113,7 +113,7 @@ func TestCheckpoint(t *testing.T) {
testutil.Ok(t, w.Close()) testutil.Ok(t, w.Close())
// Start a WAL and write records to it as usual. // Start a WAL and write records to it as usual.
w, err = wal.NewSize(nil, nil, dir, 64*1024, compress) w, err = NewSize(nil, nil, dir, 64*1024, compress)
testutil.Ok(t, err) testutil.Ok(t, err)
var last int64 var last int64
@ -125,7 +125,7 @@ func TestCheckpoint(t *testing.T) {
} }
// Write some series initially. // Write some series initially.
if i == 0 { if i == 0 {
b := enc.Series([]RefSeries{ b := enc.Series([]record.RefSeries{
{Ref: 2, Labels: labels.FromStrings("a", "b", "c", "2")}, {Ref: 2, Labels: labels.FromStrings("a", "b", "c", "2")},
{Ref: 3, Labels: labels.FromStrings("a", "b", "c", "3")}, {Ref: 3, Labels: labels.FromStrings("a", "b", "c", "3")},
{Ref: 4, Labels: labels.FromStrings("a", "b", "c", "4")}, {Ref: 4, Labels: labels.FromStrings("a", "b", "c", "4")},
@ -136,7 +136,7 @@ func TestCheckpoint(t *testing.T) {
// Write samples until the WAL has enough segments. // Write samples until the WAL has enough segments.
// Make them have drifting timestamps within a record to see that they // Make them have drifting timestamps within a record to see that they
// get filtered properly. // get filtered properly.
b := enc.Samples([]RefSample{ b := enc.Samples([]record.RefSample{
{Ref: 0, T: last, V: float64(i)}, {Ref: 0, T: last, V: float64(i)},
{Ref: 1, T: last + 10000, V: float64(i)}, {Ref: 1, T: last + 10000, V: float64(i)},
{Ref: 2, T: last + 20000, V: float64(i)}, {Ref: 2, T: last + 20000, V: float64(i)},
@ -161,22 +161,22 @@ func TestCheckpoint(t *testing.T) {
testutil.Equals(t, 1, len(files)) testutil.Equals(t, 1, len(files))
testutil.Equals(t, "checkpoint.000106", files[0]) testutil.Equals(t, "checkpoint.000106", files[0])
sr, err := wal.NewSegmentsReader(filepath.Join(dir, "checkpoint.000106")) sr, err := NewSegmentsReader(filepath.Join(dir, "checkpoint.000106"))
testutil.Ok(t, err) testutil.Ok(t, err)
defer sr.Close() defer sr.Close()
var dec RecordDecoder var dec record.Decoder
var series []RefSeries var series []record.RefSeries
r := wal.NewReader(sr) r := NewReader(sr)
for r.Next() { for r.Next() {
rec := r.Record() rec := r.Record()
switch dec.Type(rec) { switch dec.Type(rec) {
case RecordSeries: case record.Series:
series, err = dec.Series(rec, series) series, err = dec.Series(rec, series)
testutil.Ok(t, err) testutil.Ok(t, err)
case RecordSamples: case record.Samples:
samples, err := dec.Samples(rec, nil) samples, err := dec.Samples(rec, nil)
testutil.Ok(t, err) testutil.Ok(t, err)
for _, s := range samples { for _, s := range samples {
@ -185,7 +185,7 @@ func TestCheckpoint(t *testing.T) {
} }
} }
testutil.Ok(t, r.Err()) testutil.Ok(t, r.Err())
testutil.Equals(t, []RefSeries{ testutil.Equals(t, []record.RefSeries{
{Ref: 0, Labels: labels.FromStrings("a", "b", "c", "0")}, {Ref: 0, Labels: labels.FromStrings("a", "b", "c", "0")},
{Ref: 2, Labels: labels.FromStrings("a", "b", "c", "2")}, {Ref: 2, Labels: labels.FromStrings("a", "b", "c", "2")},
{Ref: 4, Labels: labels.FromStrings("a", "b", "c", "4")}, {Ref: 4, Labels: labels.FromStrings("a", "b", "c", "4")},
@ -201,7 +201,7 @@ func TestCheckpointNoTmpFolderAfterError(t *testing.T) {
defer func() { defer func() {
testutil.Ok(t, os.RemoveAll(dir)) testutil.Ok(t, os.RemoveAll(dir))
}() }()
w, err := wal.NewSize(nil, nil, dir, 64*1024, false) w, err := NewSize(nil, nil, dir, 64*1024, false)
testutil.Ok(t, err) testutil.Ok(t, err)
testutil.Ok(t, w.Log([]byte{99})) testutil.Ok(t, w.Log([]byte{99}))
w.Close() w.Close()

View file

@ -41,7 +41,7 @@ type reader interface {
Offset() int64 Offset() int64
} }
type record struct { type rec struct {
t recType t recType
b []byte b []byte
} }
@ -59,13 +59,13 @@ var readerConstructors = map[string]func(io.Reader) reader{
var data = make([]byte, 100000) var data = make([]byte, 100000)
var testReaderCases = []struct { var testReaderCases = []struct {
t []record t []rec
exp [][]byte exp [][]byte
fail bool fail bool
}{ }{
// Sequence of valid records. // Sequence of valid records.
{ {
t: []record{ t: []rec{
{recFull, data[0:200]}, {recFull, data[0:200]},
{recFirst, data[200:300]}, {recFirst, data[200:300]},
{recLast, data[300:400]}, {recLast, data[300:400]},
@ -89,7 +89,7 @@ var testReaderCases = []struct {
}, },
// Exactly at the limit of one page minus the header size // Exactly at the limit of one page minus the header size
{ {
t: []record{ t: []rec{
{recFull, data[0 : pageSize-recordHeaderSize]}, {recFull, data[0 : pageSize-recordHeaderSize]},
}, },
exp: [][]byte{ exp: [][]byte{
@ -99,7 +99,7 @@ var testReaderCases = []struct {
// More than a full page, this exceeds our buffer and can never happen // More than a full page, this exceeds our buffer and can never happen
// when written by the WAL. // when written by the WAL.
{ {
t: []record{ t: []rec{
{recFull, data[0 : pageSize+1]}, {recFull, data[0 : pageSize+1]},
}, },
fail: true, fail: true,
@ -108,7 +108,7 @@ var testReaderCases = []struct {
// NB currently the non-live reader succeeds on this. I think this is a bug. // NB currently the non-live reader succeeds on this. I think this is a bug.
// but we've seen it in production. // but we've seen it in production.
{ {
t: []record{ t: []rec{
{recFull, data[:pageSize/2]}, {recFull, data[:pageSize/2]},
{recFull, data[:pageSize/2]}, {recFull, data[:pageSize/2]},
}, },
@ -119,22 +119,22 @@ var testReaderCases = []struct {
}, },
// Invalid orders of record types. // Invalid orders of record types.
{ {
t: []record{{recMiddle, data[:200]}}, t: []rec{{recMiddle, data[:200]}},
fail: true, fail: true,
}, },
{ {
t: []record{{recLast, data[:200]}}, t: []rec{{recLast, data[:200]}},
fail: true, fail: true,
}, },
{ {
t: []record{ t: []rec{
{recFirst, data[:200]}, {recFirst, data[:200]},
{recFull, data[200:400]}, {recFull, data[200:400]},
}, },
fail: true, fail: true,
}, },
{ {
t: []record{ t: []rec{
{recFirst, data[:100]}, {recFirst, data[:100]},
{recMiddle, data[100:200]}, {recMiddle, data[100:200]},
{recFull, data[200:400]}, {recFull, data[200:400]},
@ -143,7 +143,7 @@ var testReaderCases = []struct {
}, },
// Non-zero data after page termination. // Non-zero data after page termination.
{ {
t: []record{ t: []rec{
{recFull, data[:100]}, {recFull, data[:100]},
{recPageTerm, append(make([]byte, pageSize-recordHeaderSize-102), 1)}, {recPageTerm, append(make([]byte, pageSize-recordHeaderSize-102), 1)},
}, },

View file

@ -11,7 +11,7 @@
// See the License for the specific language governing permissions and // See the License for the specific language governing permissions and
// limitations under the License. // limitations under the License.
package remote package wal
import ( import (
"fmt" "fmt"
@ -28,81 +28,44 @@ import (
"github.com/go-kit/kit/log/level" "github.com/go-kit/kit/log/level"
"github.com/pkg/errors" "github.com/pkg/errors"
"github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus"
"github.com/prometheus/prometheus/tsdb"
"github.com/prometheus/prometheus/tsdb/fileutil"
"github.com/prometheus/prometheus/tsdb/wal"
"github.com/prometheus/prometheus/pkg/timestamp" "github.com/prometheus/prometheus/pkg/timestamp"
"github.com/prometheus/prometheus/tsdb/fileutil"
"github.com/prometheus/prometheus/tsdb/record"
) )
const ( const (
readPeriod = 10 * time.Millisecond readPeriod = 10 * time.Millisecond
checkpointPeriod = 5 * time.Second checkpointPeriod = 5 * time.Second
segmentCheckPeriod = 100 * time.Millisecond segmentCheckPeriod = 100 * time.Millisecond
consumer = "consumer"
) )
var ( // WriteTo is an interface used by the Watcher to send the samples it's read
watcherRecordsRead = prometheus.NewCounterVec( // from the WAL on to somewhere else.
prometheus.CounterOpts{ type WriteTo interface {
Namespace: "prometheus", Append([]record.RefSample) bool
Subsystem: "wal_watcher", StoreSeries([]record.RefSeries, int)
Name: "records_read_total",
Help: "Number of records read by the WAL watcher from the WAL.",
},
[]string{queue, "type"},
)
watcherRecordDecodeFails = prometheus.NewCounterVec(
prometheus.CounterOpts{
Namespace: "prometheus",
Subsystem: "wal_watcher",
Name: "record_decode_failures_total",
Help: "Number of records read by the WAL watcher that resulted in an error when decoding.",
},
[]string{queue},
)
watcherSamplesSentPreTailing = prometheus.NewCounterVec(
prometheus.CounterOpts{
Namespace: "prometheus",
Subsystem: "wal_watcher",
Name: "samples_sent_pre_tailing_total",
Help: "Number of sample records read by the WAL watcher and sent to remote write during replay of existing WAL.",
},
[]string{queue},
)
watcherCurrentSegment = prometheus.NewGaugeVec(
prometheus.GaugeOpts{
Namespace: "prometheus",
Subsystem: "wal_watcher",
Name: "current_segment",
Help: "Current segment the WAL watcher is reading records from.",
},
[]string{queue},
)
liveReaderMetrics = wal.NewLiveReaderMetrics(prometheus.DefaultRegisterer)
)
func init() {
prometheus.MustRegister(watcherRecordsRead)
prometheus.MustRegister(watcherRecordDecodeFails)
prometheus.MustRegister(watcherSamplesSentPreTailing)
prometheus.MustRegister(watcherCurrentSegment)
}
type writeTo interface {
Append([]tsdb.RefSample) bool
StoreSeries([]tsdb.RefSeries, int)
SeriesReset(int) SeriesReset(int)
} }
// WALWatcher watches the TSDB WAL for a given WriteTo. type WatcherMetrics struct {
type WALWatcher struct { recordsRead *prometheus.CounterVec
recordDecodeFails *prometheus.CounterVec
samplesSentPreTailing *prometheus.CounterVec
currentSegment *prometheus.GaugeVec
}
// Watcher watches the TSDB WAL for a given WriteTo.
type Watcher struct {
name string name string
writer writeTo writer WriteTo
logger log.Logger logger log.Logger
walDir string walDir string
lastCheckpoint string lastCheckpoint string
metrics *WatcherMetrics
readerMetrics *liveReaderMetrics
startTime int64 StartTime int64
recordsReadMetric *prometheus.CounterVec recordsReadMetric *prometheus.CounterVec
recordDecodeFailsMetric prometheus.Counter recordDecodeFailsMetric prometheus.Counter
@ -113,66 +76,120 @@ type WALWatcher struct {
done chan struct{} done chan struct{}
// For testing, stop when we hit this segment. // For testing, stop when we hit this segment.
maxSegment int MaxSegment int
} }
// NewWALWatcher creates a new WAL watcher for a given WriteTo. func NewWatcherMetrics(reg prometheus.Registerer) *WatcherMetrics {
func NewWALWatcher(logger log.Logger, name string, writer writeTo, walDir string) *WALWatcher { m := &WatcherMetrics{
recordsRead: prometheus.NewCounterVec(
prometheus.CounterOpts{
Namespace: "prometheus",
Subsystem: "wal_watcher",
Name: "records_read_total",
Help: "Number of records read by the WAL watcher from the WAL.",
},
[]string{consumer, "type"},
),
recordDecodeFails: prometheus.NewCounterVec(
prometheus.CounterOpts{
Namespace: "prometheus",
Subsystem: "wal_watcher",
Name: "record_decode_failures_total",
Help: "Number of records read by the WAL watcher that resulted in an error when decoding.",
},
[]string{consumer},
),
samplesSentPreTailing: prometheus.NewCounterVec(
prometheus.CounterOpts{
Namespace: "prometheus",
Subsystem: "wal_watcher",
Name: "samples_sent_pre_tailing_total",
Help: "Number of sample records read by the WAL watcher and sent to remote write during replay of existing WAL.",
},
[]string{consumer},
),
currentSegment: prometheus.NewGaugeVec(
prometheus.GaugeOpts{
Namespace: "prometheus",
Subsystem: "wal_watcher",
Name: "current_segment",
Help: "Current segment the WAL watcher is reading records from.",
},
[]string{consumer},
),
}
if reg != nil {
_ = reg.Register(m.recordsRead)
_ = reg.Register(m.recordDecodeFails)
_ = reg.Register(m.samplesSentPreTailing)
_ = reg.Register(m.currentSegment)
}
return m
}
// NewWatcher creates a new WAL watcher for a given WriteTo.
func NewWatcher(reg prometheus.Registerer, metrics *WatcherMetrics, logger log.Logger, name string, writer WriteTo, walDir string) *Watcher {
if logger == nil { if logger == nil {
logger = log.NewNopLogger() logger = log.NewNopLogger()
} }
return &WALWatcher{ return &Watcher{
logger: logger, logger: logger,
writer: writer, writer: writer,
walDir: path.Join(walDir, "wal"), metrics: metrics,
name: name, readerMetrics: NewLiveReaderMetrics(reg),
quit: make(chan struct{}), walDir: path.Join(walDir, "wal"),
done: make(chan struct{}), name: name,
quit: make(chan struct{}),
done: make(chan struct{}),
maxSegment: -1, MaxSegment: -1,
} }
} }
func (w *WALWatcher) setMetrics() { func (w *Watcher) setMetrics() {
// Setup the WAL Watchers metrics. We do this here rather than in the // Setup the WAL Watchers metrics. We do this here rather than in the
// constructor because of the ordering of creating Queue Managers's, // constructor because of the ordering of creating Queue Managers's,
// stopping them, and then starting new ones in storage/remote/storage.go ApplyConfig. // stopping them, and then starting new ones in storage/remote/storage.go ApplyConfig.
w.recordsReadMetric = watcherRecordsRead.MustCurryWith(prometheus.Labels{queue: w.name}) if w.metrics != nil {
w.recordDecodeFailsMetric = watcherRecordDecodeFails.WithLabelValues(w.name) w.recordsReadMetric = w.metrics.recordsRead.MustCurryWith(prometheus.Labels{consumer: w.name})
w.samplesSentPreTailing = watcherSamplesSentPreTailing.WithLabelValues(w.name) w.recordDecodeFailsMetric = w.metrics.recordDecodeFails.WithLabelValues(w.name)
w.currentSegmentMetric = watcherCurrentSegment.WithLabelValues(w.name) w.samplesSentPreTailing = w.metrics.samplesSentPreTailing.WithLabelValues(w.name)
w.currentSegmentMetric = w.metrics.currentSegment.WithLabelValues(w.name)
}
} }
// Start the WALWatcher. // Start the Watcher.
func (w *WALWatcher) Start() { func (w *Watcher) Start() {
w.setMetrics() w.setMetrics()
level.Info(w.logger).Log("msg", "starting WAL watcher", "queue", w.name) level.Info(w.logger).Log("msg", "starting WAL watcher", "queue", w.name)
go w.loop() go w.loop()
} }
// Stop the WALWatcher. // Stop the Watcher.
func (w *WALWatcher) Stop() { func (w *Watcher) Stop() {
close(w.quit) close(w.quit)
<-w.done <-w.done
// Records read metric has series and samples. // Records read metric has series and samples.
watcherRecordsRead.DeleteLabelValues(w.name, "series") w.metrics.recordsRead.DeleteLabelValues(w.name, "series")
watcherRecordsRead.DeleteLabelValues(w.name, "samples") w.metrics.recordsRead.DeleteLabelValues(w.name, "samples")
watcherRecordDecodeFails.DeleteLabelValues(w.name) w.metrics.recordDecodeFails.DeleteLabelValues(w.name)
watcherSamplesSentPreTailing.DeleteLabelValues(w.name) w.metrics.samplesSentPreTailing.DeleteLabelValues(w.name)
watcherCurrentSegment.DeleteLabelValues(w.name) w.metrics.currentSegment.DeleteLabelValues(w.name)
level.Info(w.logger).Log("msg", "WAL watcher stopped", "queue", w.name) level.Info(w.logger).Log("msg", "WAL watcher stopped", "queue", w.name)
} }
func (w *WALWatcher) loop() { func (w *Watcher) loop() {
defer close(w.done) defer close(w.done)
// We may encounter failures processing the WAL; we should wait and retry. // We may encounter failures processing the WAL; we should wait and retry.
for !isClosed(w.quit) { for !isClosed(w.quit) {
w.startTime = timestamp.FromTime(time.Now()) w.StartTime = timestamp.FromTime(time.Now())
if err := w.run(); err != nil { if err := w.Run(); err != nil {
level.Error(w.logger).Log("msg", "error tailing WAL", "err", err) level.Error(w.logger).Log("msg", "error tailing WAL", "err", err)
} }
@ -184,15 +201,17 @@ func (w *WALWatcher) loop() {
} }
} }
func (w *WALWatcher) run() error { // Run the watcher, which will tail the WAL until the quit channel is closed
// or an error case is hit.
func (w *Watcher) Run() error {
_, lastSegment, err := w.firstAndLast() _, lastSegment, err := w.firstAndLast()
if err != nil { if err != nil {
return errors.Wrap(err, "wal.Segments") return errors.Wrap(err, "wal.Segments")
} }
// Backfill from the checkpoint first if it exists. // Backfill from the checkpoint first if it exists.
lastCheckpoint, checkpointIndex, err := tsdb.LastCheckpoint(w.walDir) lastCheckpoint, checkpointIndex, err := LastCheckpoint(w.walDir)
if err != nil && err != tsdb.ErrNotFound { if err != nil && err != record.ErrNotFound {
return errors.Wrap(err, "tsdb.LastCheckpoint") return errors.Wrap(err, "tsdb.LastCheckpoint")
} }
@ -220,7 +239,7 @@ func (w *WALWatcher) run() error {
} }
// For testing: stop when you hit a specific segment. // For testing: stop when you hit a specific segment.
if currentSegment == w.maxSegment { if currentSegment == w.MaxSegment {
return nil return nil
} }
@ -231,7 +250,7 @@ func (w *WALWatcher) run() error {
} }
// findSegmentForIndex finds the first segment greater than or equal to index. // findSegmentForIndex finds the first segment greater than or equal to index.
func (w *WALWatcher) findSegmentForIndex(index int) (int, error) { func (w *Watcher) findSegmentForIndex(index int) (int, error) {
refs, err := w.segments(w.walDir) refs, err := w.segments(w.walDir)
if err != nil { if err != nil {
return -1, err return -1, err
@ -246,7 +265,7 @@ func (w *WALWatcher) findSegmentForIndex(index int) (int, error) {
return -1, errors.New("failed to find segment for index") return -1, errors.New("failed to find segment for index")
} }
func (w *WALWatcher) firstAndLast() (int, int, error) { func (w *Watcher) firstAndLast() (int, int, error) {
refs, err := w.segments(w.walDir) refs, err := w.segments(w.walDir)
if err != nil { if err != nil {
return -1, -1, err return -1, -1, err
@ -260,7 +279,7 @@ func (w *WALWatcher) firstAndLast() (int, int, error) {
// Copied from tsdb/wal/wal.go so we do not have to open a WAL. // Copied from tsdb/wal/wal.go so we do not have to open a WAL.
// Plan is to move WAL watcher to TSDB and dedupe these implementations. // Plan is to move WAL watcher to TSDB and dedupe these implementations.
func (w *WALWatcher) segments(dir string) ([]int, error) { func (w *Watcher) segments(dir string) ([]int, error) {
files, err := fileutil.ReadDir(dir) files, err := fileutil.ReadDir(dir)
if err != nil { if err != nil {
return nil, err return nil, err
@ -287,14 +306,14 @@ func (w *WALWatcher) segments(dir string) ([]int, error) {
// Use tail true to indicate that the reader is currently on a segment that is // Use tail true to indicate that the reader is currently on a segment that is
// actively being written to. If false, assume it's a full segment and we're // actively being written to. If false, assume it's a full segment and we're
// replaying it on start to cache the series records. // replaying it on start to cache the series records.
func (w *WALWatcher) watch(segmentNum int, tail bool) error { func (w *Watcher) watch(segmentNum int, tail bool) error {
segment, err := wal.OpenReadSegment(wal.SegmentName(w.walDir, segmentNum)) segment, err := OpenReadSegment(SegmentName(w.walDir, segmentNum))
if err != nil { if err != nil {
return err return err
} }
defer segment.Close() defer segment.Close()
reader := wal.NewLiveReader(w.logger, liveReaderMetrics, segment) reader := NewLiveReader(w.logger, w.readerMetrics, segment)
readTicker := time.NewTicker(readPeriod) readTicker := time.NewTicker(readPeriod)
defer readTicker.Stop() defer readTicker.Stop()
@ -382,9 +401,9 @@ func (w *WALWatcher) watch(segmentNum int, tail bool) error {
} }
} }
func (w *WALWatcher) garbageCollectSeries(segmentNum int) error { func (w *Watcher) garbageCollectSeries(segmentNum int) error {
dir, _, err := tsdb.LastCheckpoint(w.walDir) dir, _, err := LastCheckpoint(w.walDir)
if err != nil && err != tsdb.ErrNotFound { if err != nil && err != record.ErrNotFound {
return errors.Wrap(err, "tsdb.LastCheckpoint") return errors.Wrap(err, "tsdb.LastCheckpoint")
} }
@ -414,12 +433,12 @@ func (w *WALWatcher) garbageCollectSeries(segmentNum int) error {
return nil return nil
} }
func (w *WALWatcher) readSegment(r *wal.LiveReader, segmentNum int, tail bool) error { func (w *Watcher) readSegment(r *LiveReader, segmentNum int, tail bool) error {
var ( var (
dec tsdb.RecordDecoder dec record.Decoder
series []tsdb.RefSeries series []record.RefSeries
samples []tsdb.RefSample samples []record.RefSample
send []tsdb.RefSample send []record.RefSample
) )
for r.Next() && !isClosed(w.quit) { for r.Next() && !isClosed(w.quit) {
@ -427,7 +446,7 @@ func (w *WALWatcher) readSegment(r *wal.LiveReader, segmentNum int, tail bool) e
w.recordsReadMetric.WithLabelValues(recordType(dec.Type(rec))).Inc() w.recordsReadMetric.WithLabelValues(recordType(dec.Type(rec))).Inc()
switch dec.Type(rec) { switch dec.Type(rec) {
case tsdb.RecordSeries: case record.Series:
series, err := dec.Series(rec, series[:0]) series, err := dec.Series(rec, series[:0])
if err != nil { if err != nil {
w.recordDecodeFailsMetric.Inc() w.recordDecodeFailsMetric.Inc()
@ -435,7 +454,7 @@ func (w *WALWatcher) readSegment(r *wal.LiveReader, segmentNum int, tail bool) e
} }
w.writer.StoreSeries(series, segmentNum) w.writer.StoreSeries(series, segmentNum)
case tsdb.RecordSamples: case record.Samples:
// If we're not tailing a segment we can ignore any samples records we see. // If we're not tailing a segment we can ignore any samples records we see.
// This speeds up replay of the WAL by > 10x. // This speeds up replay of the WAL by > 10x.
if !tail { if !tail {
@ -447,7 +466,7 @@ func (w *WALWatcher) readSegment(r *wal.LiveReader, segmentNum int, tail bool) e
return err return err
} }
for _, s := range samples { for _, s := range samples {
if s.T > w.startTime { if s.T > w.StartTime {
send = append(send, s) send = append(send, s)
} }
} }
@ -457,9 +476,9 @@ func (w *WALWatcher) readSegment(r *wal.LiveReader, segmentNum int, tail bool) e
send = send[:0] send = send[:0]
} }
case tsdb.RecordTombstones: case record.Tombstones:
// noop // noop
case tsdb.RecordInvalid: case record.Invalid:
return errors.New("invalid record") return errors.New("invalid record")
default: default:
@ -470,15 +489,15 @@ func (w *WALWatcher) readSegment(r *wal.LiveReader, segmentNum int, tail bool) e
return r.Err() return r.Err()
} }
func recordType(rt tsdb.RecordType) string { func recordType(rt record.Type) string {
switch rt { switch rt {
case tsdb.RecordInvalid: case record.Invalid:
return "invalid" return "invalid"
case tsdb.RecordSeries: case record.Series:
return "series" return "series"
case tsdb.RecordSamples: case record.Samples:
return "samples" return "samples"
case tsdb.RecordTombstones: case record.Tombstones:
return "tombstones" return "tombstones"
default: default:
return "unknown" return "unknown"
@ -486,7 +505,7 @@ func recordType(rt tsdb.RecordType) string {
} }
// Read all the series records from a Checkpoint directory. // Read all the series records from a Checkpoint directory.
func (w *WALWatcher) readCheckpoint(checkpointDir string) error { func (w *Watcher) readCheckpoint(checkpointDir string) error {
level.Debug(w.logger).Log("msg", "reading checkpoint", "dir", checkpointDir) level.Debug(w.logger).Log("msg", "reading checkpoint", "dir", checkpointDir)
index, err := checkpointNum(checkpointDir) index, err := checkpointNum(checkpointDir)
if err != nil { if err != nil {
@ -504,13 +523,13 @@ func (w *WALWatcher) readCheckpoint(checkpointDir string) error {
return errors.Wrap(err, "getSegmentSize") return errors.Wrap(err, "getSegmentSize")
} }
sr, err := wal.OpenReadSegment(wal.SegmentName(checkpointDir, seg)) sr, err := OpenReadSegment(SegmentName(checkpointDir, seg))
if err != nil { if err != nil {
return errors.Wrap(err, "unable to open segment") return errors.Wrap(err, "unable to open segment")
} }
defer sr.Close() defer sr.Close()
r := wal.NewLiveReader(w.logger, liveReaderMetrics, sr) r := NewLiveReader(w.logger, w.readerMetrics, sr)
if err := w.readSegment(r, index, false); err != io.EOF && err != nil { if err := w.readSegment(r, index, false); err != io.EOF && err != nil {
return errors.Wrap(err, "readSegment") return errors.Wrap(err, "readSegment")
} }
@ -543,7 +562,7 @@ func checkpointNum(dir string) (int, error) {
// Get size of segment. // Get size of segment.
func getSegmentSize(dir string, index int) (int64, error) { func getSegmentSize(dir string, index int) (int64, error) {
i := int64(-1) i := int64(-1)
fi, err := os.Stat(wal.SegmentName(dir, index)) fi, err := os.Stat(SegmentName(dir, index))
if err == nil { if err == nil {
i = fi.Size() i = fi.Size()
} }

View file

@ -10,7 +10,7 @@
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and // See the License for the specific language governing permissions and
// limitations under the License. // limitations under the License.
package remote package wal
import ( import (
"fmt" "fmt"
@ -22,14 +22,15 @@ import (
"testing" "testing"
"time" "time"
"github.com/prometheus/prometheus/tsdb" "github.com/prometheus/client_golang/prometheus"
"github.com/prometheus/prometheus/tsdb/labels" "github.com/prometheus/prometheus/tsdb/labels"
"github.com/prometheus/prometheus/tsdb/wal" "github.com/prometheus/prometheus/tsdb/record"
"github.com/prometheus/prometheus/util/testutil" "github.com/prometheus/prometheus/util/testutil"
) )
var defaultRetryInterval = 100 * time.Millisecond var defaultRetryInterval = 100 * time.Millisecond
var defaultRetries = 100 var defaultRetries = 100
var wMetrics = NewWatcherMetrics(prometheus.DefaultRegisterer)
// retry executes f() n times at each interval until it returns true. // retry executes f() n times at each interval until it returns true.
func retry(t *testing.T, interval time.Duration, n int, f func() bool) { func retry(t *testing.T, interval time.Duration, n int, f func() bool) {
@ -51,12 +52,12 @@ type writeToMock struct {
seriesSegmentIndexes map[uint64]int seriesSegmentIndexes map[uint64]int
} }
func (wtm *writeToMock) Append(s []tsdb.RefSample) bool { func (wtm *writeToMock) Append(s []record.RefSample) bool {
wtm.samplesAppended += len(s) wtm.samplesAppended += len(s)
return true return true
} }
func (wtm *writeToMock) StoreSeries(series []tsdb.RefSeries, index int) { func (wtm *writeToMock) StoreSeries(series []record.RefSeries, index int) {
wtm.seriesLock.Lock() wtm.seriesLock.Lock()
defer wtm.seriesLock.Unlock() defer wtm.seriesLock.Unlock()
for _, s := range series { for _, s := range series {
@ -104,14 +105,14 @@ func TestTailSamples(t *testing.T) {
err = os.Mkdir(wdir, 0777) err = os.Mkdir(wdir, 0777)
testutil.Ok(t, err) testutil.Ok(t, err)
enc := tsdb.RecordEncoder{} enc := record.Encoder{}
w, err := wal.NewSize(nil, nil, wdir, 128*pageSize, compress) w, err := NewSize(nil, nil, wdir, 128*pageSize, compress)
testutil.Ok(t, err) testutil.Ok(t, err)
// Write to the initial segment then checkpoint. // Write to the initial segment then checkpoint.
for i := 0; i < seriesCount; i++ { for i := 0; i < seriesCount; i++ {
ref := i + 100 ref := i + 100
series := enc.Series([]tsdb.RefSeries{ series := enc.Series([]record.RefSeries{
{ {
Ref: uint64(ref), Ref: uint64(ref),
Labels: labels.Labels{labels.Label{Name: "__name__", Value: fmt.Sprintf("metric_%d", i)}}, Labels: labels.Labels{labels.Label{Name: "__name__", Value: fmt.Sprintf("metric_%d", i)}},
@ -121,7 +122,7 @@ func TestTailSamples(t *testing.T) {
for j := 0; j < samplesCount; j++ { for j := 0; j < samplesCount; j++ {
inner := rand.Intn(ref + 1) inner := rand.Intn(ref + 1)
sample := enc.Samples([]tsdb.RefSample{ sample := enc.Samples([]record.RefSample{
{ {
Ref: uint64(inner), Ref: uint64(inner),
T: int64(now.UnixNano()) + 1, T: int64(now.UnixNano()) + 1,
@ -137,17 +138,17 @@ func TestTailSamples(t *testing.T) {
testutil.Ok(t, err) testutil.Ok(t, err)
wt := newWriteToMock() wt := newWriteToMock()
watcher := NewWALWatcher(nil, "", wt, dir) watcher := NewWatcher(nil, wMetrics, nil, "", wt, dir)
watcher.startTime = now.UnixNano() watcher.StartTime = now.UnixNano()
// Set the Watcher's metrics so they're not nil pointers. // Set the Watcher's metrics so they're not nil pointers.
watcher.setMetrics() watcher.setMetrics()
for i := first; i <= last; i++ { for i := first; i <= last; i++ {
segment, err := wal.OpenReadSegment(wal.SegmentName(watcher.walDir, i)) segment, err := OpenReadSegment(SegmentName(watcher.walDir, i))
testutil.Ok(t, err) testutil.Ok(t, err)
defer segment.Close() defer segment.Close()
reader := wal.NewLiveReader(nil, liveReaderMetrics, segment) reader := NewLiveReader(nil, NewLiveReaderMetrics(prometheus.DefaultRegisterer), segment)
// Use tail true so we can ensure we got the right number of samples. // Use tail true so we can ensure we got the right number of samples.
watcher.readSegment(reader, i, true) watcher.readSegment(reader, i, true)
} }
@ -177,15 +178,15 @@ func TestReadToEndNoCheckpoint(t *testing.T) {
err = os.Mkdir(wdir, 0777) err = os.Mkdir(wdir, 0777)
testutil.Ok(t, err) testutil.Ok(t, err)
w, err := wal.NewSize(nil, nil, wdir, 128*pageSize, compress) w, err := NewSize(nil, nil, wdir, 128*pageSize, compress)
testutil.Ok(t, err) testutil.Ok(t, err)
var recs [][]byte var recs [][]byte
enc := tsdb.RecordEncoder{} enc := record.Encoder{}
for i := 0; i < seriesCount; i++ { for i := 0; i < seriesCount; i++ {
series := enc.Series([]tsdb.RefSeries{ series := enc.Series([]record.RefSeries{
{ {
Ref: uint64(i), Ref: uint64(i),
Labels: labels.Labels{labels.Label{Name: "__name__", Value: fmt.Sprintf("metric_%d", i)}}, Labels: labels.Labels{labels.Label{Name: "__name__", Value: fmt.Sprintf("metric_%d", i)}},
@ -193,7 +194,7 @@ func TestReadToEndNoCheckpoint(t *testing.T) {
}, nil) }, nil)
recs = append(recs, series) recs = append(recs, series)
for j := 0; j < samplesCount; j++ { for j := 0; j < samplesCount; j++ {
sample := enc.Samples([]tsdb.RefSample{ sample := enc.Samples([]record.RefSample{
{ {
Ref: uint64(j), Ref: uint64(j),
T: int64(i), T: int64(i),
@ -216,7 +217,7 @@ func TestReadToEndNoCheckpoint(t *testing.T) {
testutil.Ok(t, err) testutil.Ok(t, err)
wt := newWriteToMock() wt := newWriteToMock()
watcher := NewWALWatcher(nil, "", wt, dir) watcher := NewWatcher(nil, wMetrics, nil, "", wt, dir)
go watcher.Start() go watcher.Start()
expected := seriesCount expected := seriesCount
@ -246,14 +247,14 @@ func TestReadToEndWithCheckpoint(t *testing.T) {
err = os.Mkdir(wdir, 0777) err = os.Mkdir(wdir, 0777)
testutil.Ok(t, err) testutil.Ok(t, err)
enc := tsdb.RecordEncoder{} enc := record.Encoder{}
w, err := wal.NewSize(nil, nil, wdir, segmentSize, compress) w, err := NewSize(nil, nil, wdir, segmentSize, compress)
testutil.Ok(t, err) testutil.Ok(t, err)
// Write to the initial segment then checkpoint. // Write to the initial segment then checkpoint.
for i := 0; i < seriesCount; i++ { for i := 0; i < seriesCount; i++ {
ref := i + 100 ref := i + 100
series := enc.Series([]tsdb.RefSeries{ series := enc.Series([]record.RefSeries{
{ {
Ref: uint64(ref), Ref: uint64(ref),
Labels: labels.Labels{labels.Label{Name: "__name__", Value: fmt.Sprintf("metric_%d", i)}}, Labels: labels.Labels{labels.Label{Name: "__name__", Value: fmt.Sprintf("metric_%d", i)}},
@ -263,7 +264,7 @@ func TestReadToEndWithCheckpoint(t *testing.T) {
for j := 0; j < samplesCount; j++ { for j := 0; j < samplesCount; j++ {
inner := rand.Intn(ref + 1) inner := rand.Intn(ref + 1)
sample := enc.Samples([]tsdb.RefSample{ sample := enc.Samples([]record.RefSample{
{ {
Ref: uint64(inner), Ref: uint64(inner),
T: int64(i), T: int64(i),
@ -274,12 +275,12 @@ func TestReadToEndWithCheckpoint(t *testing.T) {
} }
} }
tsdb.Checkpoint(w, 0, 1, func(x uint64) bool { return true }, 0) Checkpoint(w, 0, 1, func(x uint64) bool { return true }, 0)
w.Truncate(1) w.Truncate(1)
// Write more records after checkpointing. // Write more records after checkpointing.
for i := 0; i < seriesCount; i++ { for i := 0; i < seriesCount; i++ {
series := enc.Series([]tsdb.RefSeries{ series := enc.Series([]record.RefSeries{
{ {
Ref: uint64(i), Ref: uint64(i),
Labels: labels.Labels{labels.Label{Name: "__name__", Value: fmt.Sprintf("metric_%d", i)}}, Labels: labels.Labels{labels.Label{Name: "__name__", Value: fmt.Sprintf("metric_%d", i)}},
@ -288,7 +289,7 @@ func TestReadToEndWithCheckpoint(t *testing.T) {
testutil.Ok(t, w.Log(series)) testutil.Ok(t, w.Log(series))
for j := 0; j < samplesCount; j++ { for j := 0; j < samplesCount; j++ {
sample := enc.Samples([]tsdb.RefSample{ sample := enc.Samples([]record.RefSample{
{ {
Ref: uint64(j), Ref: uint64(j),
T: int64(i), T: int64(i),
@ -302,7 +303,7 @@ func TestReadToEndWithCheckpoint(t *testing.T) {
_, _, err = w.Segments() _, _, err = w.Segments()
testutil.Ok(t, err) testutil.Ok(t, err)
wt := newWriteToMock() wt := newWriteToMock()
watcher := NewWALWatcher(nil, "", wt, dir) watcher := NewWatcher(nil, wMetrics, nil, "", wt, dir)
go watcher.Start() go watcher.Start()
expected := seriesCount * 2 expected := seriesCount * 2
@ -330,16 +331,16 @@ func TestReadCheckpoint(t *testing.T) {
err = os.Mkdir(wdir, 0777) err = os.Mkdir(wdir, 0777)
testutil.Ok(t, err) testutil.Ok(t, err)
os.Create(wal.SegmentName(wdir, 30)) os.Create(SegmentName(wdir, 30))
enc := tsdb.RecordEncoder{} enc := record.Encoder{}
w, err := wal.NewSize(nil, nil, wdir, 128*pageSize, compress) w, err := NewSize(nil, nil, wdir, 128*pageSize, compress)
testutil.Ok(t, err) testutil.Ok(t, err)
// Write to the initial segment then checkpoint. // Write to the initial segment then checkpoint.
for i := 0; i < seriesCount; i++ { for i := 0; i < seriesCount; i++ {
ref := i + 100 ref := i + 100
series := enc.Series([]tsdb.RefSeries{ series := enc.Series([]record.RefSeries{
{ {
Ref: uint64(ref), Ref: uint64(ref),
Labels: labels.Labels{labels.Label{Name: "__name__", Value: fmt.Sprintf("metric_%d", i)}}, Labels: labels.Labels{labels.Label{Name: "__name__", Value: fmt.Sprintf("metric_%d", i)}},
@ -349,7 +350,7 @@ func TestReadCheckpoint(t *testing.T) {
for j := 0; j < samplesCount; j++ { for j := 0; j < samplesCount; j++ {
inner := rand.Intn(ref + 1) inner := rand.Intn(ref + 1)
sample := enc.Samples([]tsdb.RefSample{ sample := enc.Samples([]record.RefSample{
{ {
Ref: uint64(inner), Ref: uint64(inner),
T: int64(i), T: int64(i),
@ -359,7 +360,7 @@ func TestReadCheckpoint(t *testing.T) {
testutil.Ok(t, w.Log(sample)) testutil.Ok(t, w.Log(sample))
} }
} }
tsdb.Checkpoint(w, 30, 31, func(x uint64) bool { return true }, 0) Checkpoint(w, 30, 31, func(x uint64) bool { return true }, 0)
w.Truncate(32) w.Truncate(32)
// Start read after checkpoint, no more data written. // Start read after checkpoint, no more data written.
@ -367,8 +368,7 @@ func TestReadCheckpoint(t *testing.T) {
testutil.Ok(t, err) testutil.Ok(t, err)
wt := newWriteToMock() wt := newWriteToMock()
watcher := NewWALWatcher(nil, "", wt, dir) watcher := NewWatcher(nil, wMetrics, nil, "", wt, dir)
// watcher.
go watcher.Start() go watcher.Start()
expectedSeries := seriesCount expectedSeries := seriesCount
@ -398,15 +398,15 @@ func TestReadCheckpointMultipleSegments(t *testing.T) {
err = os.Mkdir(wdir, 0777) err = os.Mkdir(wdir, 0777)
testutil.Ok(t, err) testutil.Ok(t, err)
enc := tsdb.RecordEncoder{} enc := record.Encoder{}
w, err := wal.NewSize(nil, nil, wdir, pageSize, compress) w, err := NewSize(nil, nil, wdir, pageSize, compress)
testutil.Ok(t, err) testutil.Ok(t, err)
// Write a bunch of data. // Write a bunch of data.
for i := 0; i < segments; i++ { for i := 0; i < segments; i++ {
for j := 0; j < seriesCount; j++ { for j := 0; j < seriesCount; j++ {
ref := j + (i * 100) ref := j + (i * 100)
series := enc.Series([]tsdb.RefSeries{ series := enc.Series([]record.RefSeries{
{ {
Ref: uint64(ref), Ref: uint64(ref),
Labels: labels.Labels{labels.Label{Name: "__name__", Value: fmt.Sprintf("metric_%d", j)}}, Labels: labels.Labels{labels.Label{Name: "__name__", Value: fmt.Sprintf("metric_%d", j)}},
@ -416,7 +416,7 @@ func TestReadCheckpointMultipleSegments(t *testing.T) {
for k := 0; k < samplesCount; k++ { for k := 0; k < samplesCount; k++ {
inner := rand.Intn(ref + 1) inner := rand.Intn(ref + 1)
sample := enc.Samples([]tsdb.RefSample{ sample := enc.Samples([]record.RefSample{
{ {
Ref: uint64(inner), Ref: uint64(inner),
T: int64(i), T: int64(i),
@ -433,18 +433,18 @@ func TestReadCheckpointMultipleSegments(t *testing.T) {
err = os.Mkdir(checkpointDir, 0777) err = os.Mkdir(checkpointDir, 0777)
testutil.Ok(t, err) testutil.Ok(t, err)
for i := 0; i <= 4; i++ { for i := 0; i <= 4; i++ {
err := os.Rename(wal.SegmentName(dir+"/wal", i), wal.SegmentName(checkpointDir, i)) err := os.Rename(SegmentName(dir+"/wal", i), SegmentName(checkpointDir, i))
testutil.Ok(t, err) testutil.Ok(t, err)
} }
wt := newWriteToMock() wt := newWriteToMock()
watcher := NewWALWatcher(nil, "", wt, dir) watcher := NewWatcher(nil, wMetrics, nil, "", wt, dir)
watcher.maxSegment = -1 watcher.MaxSegment = -1
// Set the Watcher's metrics so they're not nil pointers. // Set the Watcher's metrics so they're not nil pointers.
watcher.setMetrics() watcher.setMetrics()
lastCheckpoint, _, err := tsdb.LastCheckpoint(watcher.walDir) lastCheckpoint, _, err := LastCheckpoint(watcher.walDir)
testutil.Ok(t, err) testutil.Ok(t, err)
err = watcher.readCheckpoint(lastCheckpoint) err = watcher.readCheckpoint(lastCheckpoint)
@ -477,14 +477,14 @@ func TestCheckpointSeriesReset(t *testing.T) {
err = os.Mkdir(wdir, 0777) err = os.Mkdir(wdir, 0777)
testutil.Ok(t, err) testutil.Ok(t, err)
enc := tsdb.RecordEncoder{} enc := record.Encoder{}
w, err := wal.NewSize(nil, nil, wdir, segmentSize, tc.compress) w, err := NewSize(nil, nil, wdir, segmentSize, tc.compress)
testutil.Ok(t, err) testutil.Ok(t, err)
// Write to the initial segment, then checkpoint later. // Write to the initial segment, then checkpoint later.
for i := 0; i < seriesCount; i++ { for i := 0; i < seriesCount; i++ {
ref := i + 100 ref := i + 100
series := enc.Series([]tsdb.RefSeries{ series := enc.Series([]record.RefSeries{
{ {
Ref: uint64(ref), Ref: uint64(ref),
Labels: labels.Labels{labels.Label{Name: "__name__", Value: fmt.Sprintf("metric_%d", i)}}, Labels: labels.Labels{labels.Label{Name: "__name__", Value: fmt.Sprintf("metric_%d", i)}},
@ -494,7 +494,7 @@ func TestCheckpointSeriesReset(t *testing.T) {
for j := 0; j < samplesCount; j++ { for j := 0; j < samplesCount; j++ {
inner := rand.Intn(ref + 1) inner := rand.Intn(ref + 1)
sample := enc.Samples([]tsdb.RefSample{ sample := enc.Samples([]record.RefSample{
{ {
Ref: uint64(inner), Ref: uint64(inner),
T: int64(i), T: int64(i),
@ -509,8 +509,8 @@ func TestCheckpointSeriesReset(t *testing.T) {
testutil.Ok(t, err) testutil.Ok(t, err)
wt := newWriteToMock() wt := newWriteToMock()
watcher := NewWALWatcher(nil, "", wt, dir) watcher := NewWatcher(nil, wMetrics, nil, "", wt, dir)
watcher.maxSegment = -1 watcher.MaxSegment = -1
go watcher.Start() go watcher.Start()
expected := seriesCount expected := seriesCount
@ -519,13 +519,13 @@ func TestCheckpointSeriesReset(t *testing.T) {
}) })
testutil.Equals(t, seriesCount, wt.checkNumLabels()) testutil.Equals(t, seriesCount, wt.checkNumLabels())
_, err = tsdb.Checkpoint(w, 2, 4, func(x uint64) bool { return true }, 0) _, err = Checkpoint(w, 2, 4, func(x uint64) bool { return true }, 0)
testutil.Ok(t, err) testutil.Ok(t, err)
err = w.Truncate(5) err = w.Truncate(5)
testutil.Ok(t, err) testutil.Ok(t, err)
_, cpi, err := tsdb.LastCheckpoint(path.Join(dir, "wal")) _, cpi, err := LastCheckpoint(path.Join(dir, "wal"))
testutil.Ok(t, err) testutil.Ok(t, err)
err = watcher.garbageCollectSeries(cpi + 1) err = watcher.garbageCollectSeries(cpi + 1)
testutil.Ok(t, err) testutil.Ok(t, err)

View file

@ -29,6 +29,8 @@ import (
"github.com/go-kit/kit/log" "github.com/go-kit/kit/log"
"github.com/prometheus/prometheus/tsdb/fileutil" "github.com/prometheus/prometheus/tsdb/fileutil"
"github.com/prometheus/prometheus/tsdb/labels" "github.com/prometheus/prometheus/tsdb/labels"
"github.com/prometheus/prometheus/tsdb/record"
"github.com/prometheus/prometheus/tsdb/tombstones"
"github.com/prometheus/prometheus/tsdb/wal" "github.com/prometheus/prometheus/tsdb/wal"
"github.com/prometheus/prometheus/util/testutil" "github.com/prometheus/prometheus/util/testutil"
) )
@ -95,10 +97,10 @@ func TestSegmentWAL_Truncate(t *testing.T) {
w.segmentSize = 10000 w.segmentSize = 10000
for i := 0; i < numMetrics; i += batch { for i := 0; i < numMetrics; i += batch {
var rs []RefSeries var rs []record.RefSeries
for j, s := range series[i : i+batch] { for j, s := range series[i : i+batch] {
rs = append(rs, RefSeries{Labels: s, Ref: uint64(i+j) + 1}) rs = append(rs, record.RefSeries{Labels: s, Ref: uint64(i+j) + 1})
} }
err := w.LogSeries(rs) err := w.LogSeries(rs)
testutil.Ok(t, err) testutil.Ok(t, err)
@ -125,11 +127,11 @@ func TestSegmentWAL_Truncate(t *testing.T) {
err = w.Truncate(1000, keepf) err = w.Truncate(1000, keepf)
testutil.Ok(t, err) testutil.Ok(t, err)
var expected []RefSeries var expected []record.RefSeries
for i := 1; i <= numMetrics; i++ { for i := 1; i <= numMetrics; i++ {
if i%2 == 1 || uint64(i) >= boundarySeries { if i%2 == 1 || uint64(i) >= boundarySeries {
expected = append(expected, RefSeries{Ref: uint64(i), Labels: series[i-1]}) expected = append(expected, record.RefSeries{Ref: uint64(i), Labels: series[i-1]})
} }
} }
@ -143,10 +145,10 @@ func TestSegmentWAL_Truncate(t *testing.T) {
w, err = OpenSegmentWAL(dir, nil, 0, nil) w, err = OpenSegmentWAL(dir, nil, 0, nil)
testutil.Ok(t, err) testutil.Ok(t, err)
var readSeries []RefSeries var readSeries []record.RefSeries
r := w.Reader() r := w.Reader()
testutil.Ok(t, r.Read(func(s []RefSeries) { testutil.Ok(t, r.Read(func(s []record.RefSeries) {
readSeries = append(readSeries, s...) readSeries = append(readSeries, s...)
}, nil, nil)) }, nil, nil))
@ -172,9 +174,9 @@ func TestSegmentWAL_Log_Restore(t *testing.T) {
}() }()
var ( var (
recordedSeries [][]RefSeries recordedSeries [][]record.RefSeries
recordedSamples [][]RefSample recordedSamples [][]record.RefSample
recordedDeletes [][]Stone recordedDeletes [][]tombstones.Stone
) )
var totalSamples int var totalSamples int
@ -190,29 +192,29 @@ func TestSegmentWAL_Log_Restore(t *testing.T) {
r := w.Reader() r := w.Reader()
var ( var (
resultSeries [][]RefSeries resultSeries [][]record.RefSeries
resultSamples [][]RefSample resultSamples [][]record.RefSample
resultDeletes [][]Stone resultDeletes [][]tombstones.Stone
) )
serf := func(series []RefSeries) { serf := func(series []record.RefSeries) {
if len(series) > 0 { if len(series) > 0 {
clsets := make([]RefSeries, len(series)) clsets := make([]record.RefSeries, len(series))
copy(clsets, series) copy(clsets, series)
resultSeries = append(resultSeries, clsets) resultSeries = append(resultSeries, clsets)
} }
} }
smplf := func(smpls []RefSample) { smplf := func(smpls []record.RefSample) {
if len(smpls) > 0 { if len(smpls) > 0 {
csmpls := make([]RefSample, len(smpls)) csmpls := make([]record.RefSample, len(smpls))
copy(csmpls, smpls) copy(csmpls, smpls)
resultSamples = append(resultSamples, csmpls) resultSamples = append(resultSamples, csmpls)
} }
} }
delf := func(stones []Stone) { delf := func(stones []tombstones.Stone) {
if len(stones) > 0 { if len(stones) > 0 {
cst := make([]Stone, len(stones)) cst := make([]tombstones.Stone, len(stones))
copy(cst, stones) copy(cst, stones)
resultDeletes = append(resultDeletes, cst) resultDeletes = append(resultDeletes, cst)
} }
@ -228,11 +230,11 @@ func TestSegmentWAL_Log_Restore(t *testing.T) {
// Insert in batches and generate different amounts of samples for each. // Insert in batches and generate different amounts of samples for each.
for i := 0; i < len(series); i += stepSize { for i := 0; i < len(series); i += stepSize {
var samples []RefSample var samples []record.RefSample
var stones []Stone var stones []tombstones.Stone
for j := 0; j < i*10; j++ { for j := 0; j < i*10; j++ {
samples = append(samples, RefSample{ samples = append(samples, record.RefSample{
Ref: uint64(j % 10000), Ref: uint64(j % 10000),
T: int64(j * 2), T: int64(j * 2),
V: rand.Float64(), V: rand.Float64(),
@ -241,13 +243,13 @@ func TestSegmentWAL_Log_Restore(t *testing.T) {
for j := 0; j < i*20; j++ { for j := 0; j < i*20; j++ {
ts := rand.Int63() ts := rand.Int63()
stones = append(stones, Stone{rand.Uint64(), Intervals{{ts, ts + rand.Int63n(10000)}}}) stones = append(stones, tombstones.Stone{Ref: rand.Uint64(), Intervals: tombstones.Intervals{{Mint: ts, Maxt: ts + rand.Int63n(10000)}}})
} }
lbls := series[i : i+stepSize] lbls := series[i : i+stepSize]
series := make([]RefSeries, 0, len(series)) series := make([]record.RefSeries, 0, len(series))
for j, l := range lbls { for j, l := range lbls {
series = append(series, RefSeries{ series = append(series, record.RefSeries{
Ref: uint64(i + j), Ref: uint64(i + j),
Labels: l, Labels: l,
}) })
@ -382,8 +384,8 @@ func TestWALRestoreCorrupted(t *testing.T) {
w, err := OpenSegmentWAL(dir, nil, 0, nil) w, err := OpenSegmentWAL(dir, nil, 0, nil)
testutil.Ok(t, err) testutil.Ok(t, err)
testutil.Ok(t, w.LogSamples([]RefSample{{T: 1, V: 2}})) testutil.Ok(t, w.LogSamples([]record.RefSample{{T: 1, V: 2}}))
testutil.Ok(t, w.LogSamples([]RefSample{{T: 2, V: 3}})) testutil.Ok(t, w.LogSamples([]record.RefSample{{T: 2, V: 3}}))
testutil.Ok(t, w.cut()) testutil.Ok(t, w.cut())
@ -392,8 +394,8 @@ func TestWALRestoreCorrupted(t *testing.T) {
// Hopefully cut will complete by 2 seconds. // Hopefully cut will complete by 2 seconds.
time.Sleep(2 * time.Second) time.Sleep(2 * time.Second)
testutil.Ok(t, w.LogSamples([]RefSample{{T: 3, V: 4}})) testutil.Ok(t, w.LogSamples([]record.RefSample{{T: 3, V: 4}}))
testutil.Ok(t, w.LogSamples([]RefSample{{T: 5, V: 6}})) testutil.Ok(t, w.LogSamples([]record.RefSample{{T: 5, V: 6}}))
testutil.Ok(t, w.Close()) testutil.Ok(t, w.Close())
@ -414,24 +416,24 @@ func TestWALRestoreCorrupted(t *testing.T) {
r := w2.Reader() r := w2.Reader()
serf := func(l []RefSeries) { serf := func(l []record.RefSeries) {
testutil.Equals(t, 0, len(l)) testutil.Equals(t, 0, len(l))
} }
// Weird hack to check order of reads. // Weird hack to check order of reads.
i := 0 i := 0
samplf := func(s []RefSample) { samplf := func(s []record.RefSample) {
if i == 0 { if i == 0 {
testutil.Equals(t, []RefSample{{T: 1, V: 2}}, s) testutil.Equals(t, []record.RefSample{{T: 1, V: 2}}, s)
i++ i++
} else { } else {
testutil.Equals(t, []RefSample{{T: 99, V: 100}}, s) testutil.Equals(t, []record.RefSample{{T: 99, V: 100}}, s)
} }
} }
testutil.Ok(t, r.Read(serf, samplf, nil)) testutil.Ok(t, r.Read(serf, samplf, nil))
testutil.Ok(t, w2.LogSamples([]RefSample{{T: 99, V: 100}})) testutil.Ok(t, w2.LogSamples([]record.RefSample{{T: 99, V: 100}}))
testutil.Ok(t, w2.Close()) testutil.Ok(t, w2.Close())
// We should see the first valid entry and the new one, everything after // We should see the first valid entry and the new one, everything after
@ -482,23 +484,23 @@ func TestMigrateWAL_Fuzz(t *testing.T) {
testutil.Ok(t, err) testutil.Ok(t, err)
// Write some data. // Write some data.
testutil.Ok(t, oldWAL.LogSeries([]RefSeries{ testutil.Ok(t, oldWAL.LogSeries([]record.RefSeries{
{Ref: 100, Labels: labels.FromStrings("abc", "def", "123", "456")}, {Ref: 100, Labels: labels.FromStrings("abc", "def", "123", "456")},
{Ref: 1, Labels: labels.FromStrings("abc", "def2", "1234", "4567")}, {Ref: 1, Labels: labels.FromStrings("abc", "def2", "1234", "4567")},
})) }))
testutil.Ok(t, oldWAL.LogSamples([]RefSample{ testutil.Ok(t, oldWAL.LogSamples([]record.RefSample{
{Ref: 1, T: 100, V: 200}, {Ref: 1, T: 100, V: 200},
{Ref: 2, T: 300, V: 400}, {Ref: 2, T: 300, V: 400},
})) }))
testutil.Ok(t, oldWAL.LogSeries([]RefSeries{ testutil.Ok(t, oldWAL.LogSeries([]record.RefSeries{
{Ref: 200, Labels: labels.FromStrings("xyz", "def", "foo", "bar")}, {Ref: 200, Labels: labels.FromStrings("xyz", "def", "foo", "bar")},
})) }))
testutil.Ok(t, oldWAL.LogSamples([]RefSample{ testutil.Ok(t, oldWAL.LogSamples([]record.RefSample{
{Ref: 3, T: 100, V: 200}, {Ref: 3, T: 100, V: 200},
{Ref: 4, T: 300, V: 400}, {Ref: 4, T: 300, V: 400},
})) }))
testutil.Ok(t, oldWAL.LogDeletes([]Stone{ testutil.Ok(t, oldWAL.LogDeletes([]tombstones.Stone{
{ref: 1, intervals: []Interval{{100, 200}}}, {Ref: 1, Intervals: []tombstones.Interval{{Mint: 100, Maxt: 200}}},
})) }))
testutil.Ok(t, oldWAL.Close()) testutil.Ok(t, oldWAL.Close())
@ -510,8 +512,8 @@ func TestMigrateWAL_Fuzz(t *testing.T) {
testutil.Ok(t, err) testutil.Ok(t, err)
// We can properly write some new data after migration. // We can properly write some new data after migration.
var enc RecordEncoder var enc record.Encoder
testutil.Ok(t, w.Log(enc.Samples([]RefSample{ testutil.Ok(t, w.Log(enc.Samples([]record.RefSample{
{Ref: 500, T: 1, V: 1}, {Ref: 500, T: 1, V: 1},
}, nil))) }, nil)))
@ -523,21 +525,21 @@ func TestMigrateWAL_Fuzz(t *testing.T) {
r := wal.NewReader(sr) r := wal.NewReader(sr)
var res []interface{} var res []interface{}
var dec RecordDecoder var dec record.Decoder
for r.Next() { for r.Next() {
rec := r.Record() rec := r.Record()
switch dec.Type(rec) { switch dec.Type(rec) {
case RecordSeries: case record.Series:
s, err := dec.Series(rec, nil) s, err := dec.Series(rec, nil)
testutil.Ok(t, err) testutil.Ok(t, err)
res = append(res, s) res = append(res, s)
case RecordSamples: case record.Samples:
s, err := dec.Samples(rec, nil) s, err := dec.Samples(rec, nil)
testutil.Ok(t, err) testutil.Ok(t, err)
res = append(res, s) res = append(res, s)
case RecordTombstones: case record.Tombstones:
s, err := dec.Tombstones(rec, nil) s, err := dec.Tombstones(rec, nil)
testutil.Ok(t, err) testutil.Ok(t, err)
res = append(res, s) res = append(res, s)
@ -548,17 +550,17 @@ func TestMigrateWAL_Fuzz(t *testing.T) {
testutil.Ok(t, r.Err()) testutil.Ok(t, r.Err())
testutil.Equals(t, []interface{}{ testutil.Equals(t, []interface{}{
[]RefSeries{ []record.RefSeries{
{Ref: 100, Labels: labels.FromStrings("abc", "def", "123", "456")}, {Ref: 100, Labels: labels.FromStrings("abc", "def", "123", "456")},
{Ref: 1, Labels: labels.FromStrings("abc", "def2", "1234", "4567")}, {Ref: 1, Labels: labels.FromStrings("abc", "def2", "1234", "4567")},
}, },
[]RefSample{{Ref: 1, T: 100, V: 200}, {Ref: 2, T: 300, V: 400}}, []record.RefSample{{Ref: 1, T: 100, V: 200}, {Ref: 2, T: 300, V: 400}},
[]RefSeries{ []record.RefSeries{
{Ref: 200, Labels: labels.FromStrings("xyz", "def", "foo", "bar")}, {Ref: 200, Labels: labels.FromStrings("xyz", "def", "foo", "bar")},
}, },
[]RefSample{{Ref: 3, T: 100, V: 200}, {Ref: 4, T: 300, V: 400}}, []record.RefSample{{Ref: 3, T: 100, V: 200}, {Ref: 4, T: 300, V: 400}},
[]Stone{{ref: 1, intervals: []Interval{{100, 200}}}}, []tombstones.Stone{{Ref: 1, Intervals: []tombstones.Interval{{Mint: 100, Maxt: 200}}}},
[]RefSample{{Ref: 500, T: 1, V: 1}}, []record.RefSample{{Ref: 500, T: 1, V: 1}},
}, res) }, res)
// Migrating an already migrated WAL shouldn't do anything. // Migrating an already migrated WAL shouldn't do anything.