prometheus/block.go

660 lines
18 KiB
Go
Raw Normal View History

2017-04-10 11:59:45 -07:00
// Copyright 2017 The Prometheus Authors
2017-04-10 11:59:45 -07:00
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package tsdb
2016-12-14 23:31:26 -08:00
import (
"encoding/json"
"io"
"io/ioutil"
2016-12-14 23:31:26 -08:00
"os"
"path/filepath"
"sync"
2016-12-22 06:54:39 -08:00
"github.com/go-kit/kit/log"
"github.com/go-kit/kit/log/level"
2017-02-27 01:46:15 -08:00
"github.com/oklog/ulid"
2016-12-22 06:54:39 -08:00
"github.com/pkg/errors"
"github.com/prometheus/tsdb/chunkenc"
2017-08-08 08:35:34 -07:00
"github.com/prometheus/tsdb/chunks"
tsdb_errors "github.com/prometheus/tsdb/errors"
"github.com/prometheus/tsdb/fileutil"
"github.com/prometheus/tsdb/index"
"github.com/prometheus/tsdb/labels"
)
// IndexWriter serializes the index for a block of series data.
// The methods must be called in the order they are specified in.
type IndexWriter interface {
// AddSymbols registers all string symbols that are encountered in series
// and other indices.
AddSymbols(sym map[string]struct{}) error
// AddSeries populates the index writer with a series and its offsets
// of chunks that the index can reference.
// Implementations may require series to be insert in increasing order by
// their labels.
// The reference numbers are used to resolve entries in postings lists that
// are added later.
AddSeries(ref uint64, l labels.Labels, chunks ...chunks.Meta) error
// WriteLabelIndex serializes an index from label names to values.
// The passed in values chained tuples of strings of the length of names.
WriteLabelIndex(names []string, values []string) error
// WritePostings writes a postings list for a single label pair.
// The Postings here contain refs to the series that were added.
WritePostings(name, value string, it index.Postings) error
// Close writes any finalization and closes the resources associated with
// the underlying writer.
Close() error
}
// IndexReader provides reading access of serialized index data.
type IndexReader interface {
// Symbols returns a set of string symbols that may occur in series' labels
// and indices.
Symbols() (map[string]struct{}, error)
// LabelValues returns the possible label values.
LabelValues(names ...string) (index.StringTuples, error)
// Postings returns the postings list iterator for the label pair.
// The Postings here contain the offsets to the series inside the index.
// Found IDs are not strictly required to point to a valid Series, e.g. during
// background garbage collections.
Postings(name, value string) (index.Postings, error)
// SortedPostings returns a postings list that is reordered to be sorted
// by the label set of the underlying series.
SortedPostings(index.Postings) index.Postings
// Series populates the given labels and chunk metas for the series identified
// by the reference.
// Returns ErrNotFound if the ref does not resolve to a known series.
Series(ref uint64, lset *labels.Labels, chks *[]chunks.Meta) error
// LabelIndices returns a list of string tuples for which a label value index exists.
// NOTE: This is deprecated. Use `LabelNames()` instead.
LabelIndices() ([][]string, error)
// LabelNames returns all the unique label names present in the index in sorted order.
LabelNames() ([]string, error)
// Close releases the underlying resources of the reader.
Close() error
}
// StringTuples provides access to a sorted list of string tuples.
type StringTuples interface {
// Total number of tuples in the list.
Len() int
// At returns the tuple at position i.
At(i int) ([]string, error)
}
// ChunkWriter serializes a time block of chunked series data.
type ChunkWriter interface {
// WriteChunks writes several chunks. The Chunk field of the ChunkMetas
// must be populated.
// After returning successfully, the Ref fields in the ChunkMetas
// are set and can be used to retrieve the chunks from the written data.
WriteChunks(chunks ...chunks.Meta) error
// Close writes any required finalization and closes the resources
// associated with the underlying writer.
Close() error
}
// ChunkReader provides reading access of serialized time series data.
type ChunkReader interface {
// Chunk returns the series data chunk with the given reference.
Chunk(ref uint64) (chunkenc.Chunk, error)
// Close releases all underlying resources of the reader.
Close() error
}
// BlockReader provides reading access to a data block.
type BlockReader interface {
// Index returns an IndexReader over the block's data.
Index() (IndexReader, error)
2017-04-28 06:41:42 -07:00
// Chunks returns a ChunkReader over the block's data.
Chunks() (ChunkReader, error)
// Tombstones returns a TombstoneReader over the block's deleted data.
Tombstones() (TombstoneReader, error)
Vertical query merging and compaction (#370) * Vertical series iterator Signed-off-by: Ganesh Vernekar <cs15btech11018@iith.ac.in> * Select overlapped blocks first in compactor Plan() Signed-off-by: Ganesh Vernekar <cs15btech11018@iith.ac.in> * Added vertical compaction Signed-off-by: Ganesh Vernekar <cs15btech11018@iith.ac.in> * Code cleanup and comments Signed-off-by: Ganesh Vernekar <cs15btech11018@iith.ac.in> * Fix review comments Signed-off-by: Ganesh Vernekar <cs15btech11018@iith.ac.in> * Fix tests Signed-off-by: Ganesh Vernekar <cs15btech11018@iith.ac.in> * Add benchmark for compaction Signed-off-by: Ganesh Vernekar <cs15btech11018@iith.ac.in> * Perform vertical compaction only when blocks are overlapping. Actions for vertical compaction: * Sorting chunk metas * Calling chunks.MergeOverlappingChunks on the chunks Signed-off-by: Ganesh Vernekar <cs15btech11018@iith.ac.in> * Benchmark for vertical compaction * BenchmarkNormalCompaction => BenchmarkCompaction * Moved the benchmark from db_test.go to compact_test.go Signed-off-by: Ganesh Vernekar <cs15btech11018@iith.ac.in> * Benchmark for query iterator and seek for non overlapping blocks Signed-off-by: Ganesh Vernekar <cs15btech11018@iith.ac.in> * Vertical query merge only for overlapping blocks Signed-off-by: Ganesh Vernekar <cs15btech11018@iith.ac.in> * Simplify logging in Compact(...) Signed-off-by: Ganesh Vernekar <cs15btech11018@iith.ac.in> * Updated CHANGELOG.md Signed-off-by: Ganesh Vernekar <cs15btech11018@iith.ac.in> * Calculate overlapping inside populateBlock Signed-off-by: Ganesh Vernekar <cs15btech11018@iith.ac.in> * MinTime and MaxTime for BlockReader. Using this to find overlapping blocks in populateBlock() Signed-off-by: Ganesh Vernekar <cs15btech11018@iith.ac.in> * Sort blocks w.r.t. MinTime in reload() Signed-off-by: Ganesh Vernekar <cs15btech11018@iith.ac.in> * Log about overlapping in LeveledCompactor.write() instead of returning bool Signed-off-by: Ganesh Vernekar <cs15btech11018@iith.ac.in> * Log about overlapping inside LeveledCompactor.populateBlock() Signed-off-by: Ganesh Vernekar <cs15btech11018@iith.ac.in> * Fix review comments Signed-off-by: Ganesh Vernekar <cs15btech11018@iith.ac.in> * Refactor createBlock to take optional []Series Signed-off-by: Ganesh Vernekar <cs15btech11018@iith.ac.in> * review1 Signed-off-by: Krasi Georgiev <kgeorgie@redhat.com> * Updated CHANGELOG and minor nits Signed-off-by: Ganesh Vernekar <cs15btech11018@iith.ac.in> * nits Signed-off-by: Ganesh Vernekar <cs15btech11018@iith.ac.in> * Updated CHANGELOG Signed-off-by: Ganesh Vernekar <cs15btech11018@iith.ac.in> * Refactor iterator and seek benchmarks for Querier. Also has as overlapping blocks. Signed-off-by: Ganesh Vernekar <cs15btech11018@iith.ac.in> * Additional test case Signed-off-by: Ganesh Vernekar <cs15btech11018@iith.ac.in> * genSeries takes optional labels. Updated BenchmarkQueryIterator and BenchmarkQuerySeek. Signed-off-by: Ganesh Vernekar <cs15btech11018@iith.ac.in> * Split genSeries into genSeries and populateSeries Signed-off-by: Ganesh Vernekar <cs15btech11018@iith.ac.in> * Check error in benchmark Signed-off-by: Ganesh Vernekar <cs15btech11018@iith.ac.in> * Fix review comments Signed-off-by: Ganesh Vernekar <cs15btech11018@iith.ac.in> * Warn about overlapping blocks in reload() Signed-off-by: Ganesh Vernekar <cs15btech11018@iith.ac.in>
2019-02-14 05:29:41 -08:00
// MinTime returns the min time of the block.
MinTime() int64
// MaxTime returns the max time of the block.
MaxTime() int64
2016-12-13 06:26:58 -08:00
}
// Appendable defines an entity to which data can be appended.
type Appendable interface {
// Appender returns a new Appender against an underlying store.
Appender() Appender
}
// BlockMeta provides meta information about a block.
type BlockMeta struct {
2017-02-27 01:46:15 -08:00
// Unique identifier for the block and its contents. Changes on compaction.
ULID ulid.ULID `json:"ulid"`
2017-01-19 05:01:38 -08:00
// MinTime and MaxTime specify the time range all samples
// in the block are in.
MinTime int64 `json:"minTime"`
MaxTime int64 `json:"maxTime"`
2017-01-07 09:02:17 -08:00
// Stats about the contents of the block.
Stats BlockStats `json:"stats,omitempty"`
2017-01-19 10:45:52 -08:00
// Information on compactions the block was created from.
Compaction BlockMetaCompaction `json:"compaction"`
// Version of the index format.
Version int `json:"version"`
}
// BlockStats contains stats about contents of a block.
type BlockStats struct {
NumSamples uint64 `json:"numSamples,omitempty"`
NumSeries uint64 `json:"numSeries,omitempty"`
NumChunks uint64 `json:"numChunks,omitempty"`
NumTombstones uint64 `json:"numTombstones,omitempty"`
}
// BlockDesc describes a block by ULID and time range.
type BlockDesc struct {
ULID ulid.ULID `json:"ulid"`
MinTime int64 `json:"minTime"`
MaxTime int64 `json:"maxTime"`
}
// BlockMetaCompaction holds information about compactions a block went through.
type BlockMetaCompaction struct {
// Maximum number of compaction cycles any source block has
// gone through.
2017-08-09 02:10:29 -07:00
Level int `json:"level"`
// ULIDs of all source head blocks that went into the block.
Sources []ulid.ULID `json:"sources,omitempty"`
// Indicates that during compaction it resulted in a block without any samples
// so it should be deleted on the next reload.
Deletable bool `json:"deletable,omitempty"`
// Short descriptions of the direct blocks that were used to create
// this block.
Parents []BlockDesc `json:"parents,omitempty"`
Failed bool `json:"failed,omitempty"`
}
const indexFilename = "index"
const metaFilename = "meta.json"
func chunkDir(dir string) string { return filepath.Join(dir, "chunks") }
func readMetaFile(dir string) (*BlockMeta, int64, error) {
2017-01-19 05:01:38 -08:00
b, err := ioutil.ReadFile(filepath.Join(dir, metaFilename))
if err != nil {
return nil, 0, err
2017-01-19 05:01:38 -08:00
}
var m BlockMeta
2017-01-19 05:01:38 -08:00
if err := json.Unmarshal(b, &m); err != nil {
return nil, 0, err
2017-01-19 05:01:38 -08:00
}
2018-02-09 04:11:03 -08:00
if m.Version != 1 {
return nil, 0, errors.Errorf("unexpected meta file version %d", m.Version)
2017-01-19 05:01:38 -08:00
}
return &m, int64(len(b)), nil
2017-01-19 05:01:38 -08:00
}
func writeMetaFile(logger log.Logger, dir string, meta *BlockMeta) (int64, error) {
2018-02-09 04:11:03 -08:00
meta.Version = 1
2017-03-01 08:19:57 -08:00
// Make any changes to the file appear atomic.
path := filepath.Join(dir, metaFilename)
tmp := path + ".tmp"
defer func() {
if err := os.RemoveAll(tmp); err != nil {
level.Error(logger).Log("msg", "remove tmp file", "err", err.Error())
}
}()
2017-03-01 08:19:57 -08:00
f, err := os.Create(tmp)
2017-01-19 05:01:38 -08:00
if err != nil {
return 0, err
2017-01-19 05:01:38 -08:00
}
jsonMeta, err := json.MarshalIndent(meta, "", "\t")
if err != nil {
return 0, err
}
2017-01-19 05:01:38 -08:00
var merr tsdb_errors.MultiError
n, err := f.Write(jsonMeta)
if err != nil {
merr.Add(err)
merr.Add(f.Close())
return 0, merr.Err()
2017-01-19 05:01:38 -08:00
}
// Force the kernel to persist the file on disk to avoid data loss if the host crashes.
if err := f.Sync(); err != nil {
merr.Add(err)
merr.Add(f.Close())
return 0, merr.Err()
}
2017-01-19 05:01:38 -08:00
if err := f.Close(); err != nil {
return 0, err
2017-01-19 05:01:38 -08:00
}
return int64(n), fileutil.Replace(tmp, path)
2017-01-19 05:01:38 -08:00
}
2017-10-25 15:12:13 -07:00
// Block represents a directory of time series data covering a continuous time range.
type Block struct {
mtx sync.RWMutex
closing bool
pendingReaders sync.WaitGroup
dir string
meta BlockMeta
// Symbol Table Size in bytes.
// We maintain this variable to avoid recalculation everytime.
symbolTableSize uint64
2017-11-13 04:32:24 -08:00
chunkr ChunkReader
indexr IndexReader
tombstones TombstoneReader
logger log.Logger
numBytesChunks int64
numBytesIndex int64
numBytesTombstone int64
numBytesMeta int64
}
// OpenBlock opens the block in the directory. It can be passed a chunk pool, which is used
// to instantiate chunk structs.
func OpenBlock(logger log.Logger, dir string, pool chunkenc.Pool) (pb *Block, err error) {
if logger == nil {
logger = log.NewNopLogger()
}
var closers []io.Closer
defer func() {
if err != nil {
var merr tsdb_errors.MultiError
merr.Add(err)
merr.Add(closeAll(closers))
err = merr.Err()
}
}()
meta, sizeMeta, err := readMetaFile(dir)
2017-01-19 10:45:52 -08:00
if err != nil {
return nil, err
}
2016-12-14 23:31:26 -08:00
cr, err := chunks.NewDirReader(chunkDir(dir), pool)
2016-12-14 23:31:26 -08:00
if err != nil {
return nil, err
2016-12-14 23:31:26 -08:00
}
closers = append(closers, cr)
ir, err := index.NewFileReader(filepath.Join(dir, indexFilename))
2016-12-14 23:31:26 -08:00
if err != nil {
return nil, err
2016-12-14 23:31:26 -08:00
}
closers = append(closers, ir)
tr, sizeTomb, err := readTombstones(dir)
if err != nil {
return nil, err
}
closers = append(closers, tr)
pb = &Block{
dir: dir,
meta: *meta,
chunkr: cr,
indexr: ir,
tombstones: tr,
symbolTableSize: ir.SymbolTableSize(),
logger: logger,
numBytesChunks: cr.Size(),
numBytesIndex: ir.Size(),
numBytesTombstone: sizeTomb,
numBytesMeta: sizeMeta,
2016-12-14 23:31:26 -08:00
}
return pb, nil
}
// Close closes the on-disk block. It blocks as long as there are readers reading from the block.
func (pb *Block) Close() error {
pb.mtx.Lock()
pb.closing = true
pb.mtx.Unlock()
pb.pendingReaders.Wait()
var merr tsdb_errors.MultiError
2016-12-14 23:31:26 -08:00
2017-02-27 01:46:15 -08:00
merr.Add(pb.chunkr.Close())
merr.Add(pb.indexr.Close())
merr.Add(pb.tombstones.Close())
2017-02-27 01:46:15 -08:00
return merr.Err()
2016-12-14 23:31:26 -08:00
}
func (pb *Block) String() string {
return pb.meta.ULID.String()
}
// Dir returns the directory of the block.
func (pb *Block) Dir() string { return pb.dir }
// Meta returns meta information about the block.
func (pb *Block) Meta() BlockMeta { return pb.meta }
Vertical query merging and compaction (#370) * Vertical series iterator Signed-off-by: Ganesh Vernekar <cs15btech11018@iith.ac.in> * Select overlapped blocks first in compactor Plan() Signed-off-by: Ganesh Vernekar <cs15btech11018@iith.ac.in> * Added vertical compaction Signed-off-by: Ganesh Vernekar <cs15btech11018@iith.ac.in> * Code cleanup and comments Signed-off-by: Ganesh Vernekar <cs15btech11018@iith.ac.in> * Fix review comments Signed-off-by: Ganesh Vernekar <cs15btech11018@iith.ac.in> * Fix tests Signed-off-by: Ganesh Vernekar <cs15btech11018@iith.ac.in> * Add benchmark for compaction Signed-off-by: Ganesh Vernekar <cs15btech11018@iith.ac.in> * Perform vertical compaction only when blocks are overlapping. Actions for vertical compaction: * Sorting chunk metas * Calling chunks.MergeOverlappingChunks on the chunks Signed-off-by: Ganesh Vernekar <cs15btech11018@iith.ac.in> * Benchmark for vertical compaction * BenchmarkNormalCompaction => BenchmarkCompaction * Moved the benchmark from db_test.go to compact_test.go Signed-off-by: Ganesh Vernekar <cs15btech11018@iith.ac.in> * Benchmark for query iterator and seek for non overlapping blocks Signed-off-by: Ganesh Vernekar <cs15btech11018@iith.ac.in> * Vertical query merge only for overlapping blocks Signed-off-by: Ganesh Vernekar <cs15btech11018@iith.ac.in> * Simplify logging in Compact(...) Signed-off-by: Ganesh Vernekar <cs15btech11018@iith.ac.in> * Updated CHANGELOG.md Signed-off-by: Ganesh Vernekar <cs15btech11018@iith.ac.in> * Calculate overlapping inside populateBlock Signed-off-by: Ganesh Vernekar <cs15btech11018@iith.ac.in> * MinTime and MaxTime for BlockReader. Using this to find overlapping blocks in populateBlock() Signed-off-by: Ganesh Vernekar <cs15btech11018@iith.ac.in> * Sort blocks w.r.t. MinTime in reload() Signed-off-by: Ganesh Vernekar <cs15btech11018@iith.ac.in> * Log about overlapping in LeveledCompactor.write() instead of returning bool Signed-off-by: Ganesh Vernekar <cs15btech11018@iith.ac.in> * Log about overlapping inside LeveledCompactor.populateBlock() Signed-off-by: Ganesh Vernekar <cs15btech11018@iith.ac.in> * Fix review comments Signed-off-by: Ganesh Vernekar <cs15btech11018@iith.ac.in> * Refactor createBlock to take optional []Series Signed-off-by: Ganesh Vernekar <cs15btech11018@iith.ac.in> * review1 Signed-off-by: Krasi Georgiev <kgeorgie@redhat.com> * Updated CHANGELOG and minor nits Signed-off-by: Ganesh Vernekar <cs15btech11018@iith.ac.in> * nits Signed-off-by: Ganesh Vernekar <cs15btech11018@iith.ac.in> * Updated CHANGELOG Signed-off-by: Ganesh Vernekar <cs15btech11018@iith.ac.in> * Refactor iterator and seek benchmarks for Querier. Also has as overlapping blocks. Signed-off-by: Ganesh Vernekar <cs15btech11018@iith.ac.in> * Additional test case Signed-off-by: Ganesh Vernekar <cs15btech11018@iith.ac.in> * genSeries takes optional labels. Updated BenchmarkQueryIterator and BenchmarkQuerySeek. Signed-off-by: Ganesh Vernekar <cs15btech11018@iith.ac.in> * Split genSeries into genSeries and populateSeries Signed-off-by: Ganesh Vernekar <cs15btech11018@iith.ac.in> * Check error in benchmark Signed-off-by: Ganesh Vernekar <cs15btech11018@iith.ac.in> * Fix review comments Signed-off-by: Ganesh Vernekar <cs15btech11018@iith.ac.in> * Warn about overlapping blocks in reload() Signed-off-by: Ganesh Vernekar <cs15btech11018@iith.ac.in>
2019-02-14 05:29:41 -08:00
// MinTime returns the min time of the meta.
func (pb *Block) MinTime() int64 { return pb.meta.MinTime }
// MaxTime returns the max time of the meta.
func (pb *Block) MaxTime() int64 { return pb.meta.MaxTime }
// Size returns the number of bytes that the block takes up.
func (pb *Block) Size() int64 {
return pb.numBytesChunks + pb.numBytesIndex + pb.numBytesTombstone + pb.numBytesMeta
}
// ErrClosing is returned when a block is in the process of being closed.
var ErrClosing = errors.New("block is closing")
func (pb *Block) startRead() error {
pb.mtx.RLock()
defer pb.mtx.RUnlock()
if pb.closing {
return ErrClosing
}
pb.pendingReaders.Add(1)
return nil
}
// Index returns a new IndexReader against the block data.
func (pb *Block) Index() (IndexReader, error) {
if err := pb.startRead(); err != nil {
return nil, err
}
return blockIndexReader{ir: pb.indexr, b: pb}, nil
}
// Chunks returns a new ChunkReader against the block data.
func (pb *Block) Chunks() (ChunkReader, error) {
if err := pb.startRead(); err != nil {
return nil, err
}
return blockChunkReader{ChunkReader: pb.chunkr, b: pb}, nil
}
// Tombstones returns a new TombstoneReader against the block data.
func (pb *Block) Tombstones() (TombstoneReader, error) {
if err := pb.startRead(); err != nil {
return nil, err
}
return blockTombstoneReader{TombstoneReader: pb.tombstones, b: pb}, nil
}
// GetSymbolTableSize returns the Symbol Table Size in the index of this block.
func (pb *Block) GetSymbolTableSize() uint64 {
return pb.symbolTableSize
}
func (pb *Block) setCompactionFailed() error {
pb.meta.Compaction.Failed = true
n, err := writeMetaFile(pb.logger, pb.dir, &pb.meta)
if err != nil {
return err
}
pb.numBytesMeta = n
return nil
}
type blockIndexReader struct {
ir IndexReader
b *Block
}
func (r blockIndexReader) Symbols() (map[string]struct{}, error) {
s, err := r.ir.Symbols()
return s, errors.Wrapf(err, "block: %s", r.b.Meta().ULID)
}
func (r blockIndexReader) LabelValues(names ...string) (index.StringTuples, error) {
st, err := r.ir.LabelValues(names...)
return st, errors.Wrapf(err, "block: %s", r.b.Meta().ULID)
}
func (r blockIndexReader) Postings(name, value string) (index.Postings, error) {
p, err := r.ir.Postings(name, value)
if err != nil {
return p, errors.Wrapf(err, "block: %s", r.b.Meta().ULID)
}
return p, nil
}
func (r blockIndexReader) SortedPostings(p index.Postings) index.Postings {
return r.ir.SortedPostings(p)
}
func (r blockIndexReader) Series(ref uint64, lset *labels.Labels, chks *[]chunks.Meta) error {
if err := r.ir.Series(ref, lset, chks); err != nil {
return errors.Wrapf(err, "block: %s", r.b.Meta().ULID)
}
return nil
}
func (r blockIndexReader) LabelIndices() ([][]string, error) {
ss, err := r.ir.LabelIndices()
return ss, errors.Wrapf(err, "block: %s", r.b.Meta().ULID)
}
func (r blockIndexReader) LabelNames() ([]string, error) {
return r.b.LabelNames()
}
func (r blockIndexReader) Close() error {
r.b.pendingReaders.Done()
return nil
}
type blockTombstoneReader struct {
TombstoneReader
b *Block
}
func (r blockTombstoneReader) Close() error {
r.b.pendingReaders.Done()
return nil
}
type blockChunkReader struct {
ChunkReader
b *Block
}
func (r blockChunkReader) Close() error {
r.b.pendingReaders.Done()
return nil
}
// Delete matching series between mint and maxt in the block.
func (pb *Block) Delete(mint, maxt int64, ms ...labels.Matcher) error {
pb.mtx.Lock()
defer pb.mtx.Unlock()
if pb.closing {
return ErrClosing
}
p, err := PostingsForMatchers(pb.indexr, ms...)
if err != nil {
return errors.Wrap(err, "select series")
}
ir := pb.indexr
// Choose only valid postings which have chunks in the time-range.
stones := newMemTombstones()
var lset labels.Labels
var chks []chunks.Meta
Outer:
for p.Next() {
err := ir.Series(p.At(), &lset, &chks)
if err != nil {
return err
}
for _, chk := range chks {
if chk.OverlapsClosedInterval(mint, maxt) {
2018-02-07 05:43:21 -08:00
// Delete only until the current values and not beyond.
tmin, tmax := clampInterval(mint, maxt, chks[0].MinTime, chks[len(chks)-1].MaxTime)
stones.addInterval(p.At(), Interval{tmin, tmax})
continue Outer
}
}
}
if p.Err() != nil {
return p.Err()
}
2017-11-13 04:32:24 -08:00
err = pb.tombstones.Iter(func(id uint64, ivs Intervals) error {
for _, iv := range ivs {
stones.addInterval(id, iv)
2017-11-13 04:32:24 -08:00
}
return nil
})
if err != nil {
return err
}
2017-11-13 04:32:24 -08:00
pb.tombstones = stones
pb.meta.Stats.NumTombstones = pb.tombstones.Total()
n, err := writeTombstoneFile(pb.logger, pb.dir, pb.tombstones)
if err != nil {
return err
}
pb.numBytesTombstone = n
n, err = writeMetaFile(pb.logger, pb.dir, &pb.meta)
if err != nil {
return err
}
pb.numBytesMeta = n
return nil
}
// CleanTombstones will remove the tombstones and rewrite the block (only if there are any tombstones).
// If there was a rewrite, then it returns the ULID of the new block written, else nil.
func (pb *Block) CleanTombstones(dest string, c Compactor) (*ulid.ULID, error) {
numStones := 0
if err := pb.tombstones.Iter(func(id uint64, ivs Intervals) error {
numStones += len(ivs)
return nil
}); err != nil {
// This should never happen, as the iteration function only returns nil.
panic(err)
}
if numStones == 0 {
return nil, nil
}
meta := pb.Meta()
uid, err := c.Write(dest, pb, pb.meta.MinTime, pb.meta.MaxTime, &meta)
if err != nil {
return nil, err
}
return &uid, nil
}
// Snapshot creates snapshot of the block into dir.
func (pb *Block) Snapshot(dir string) error {
blockDir := filepath.Join(dir, pb.meta.ULID.String())
if err := os.MkdirAll(blockDir, 0777); err != nil {
return errors.Wrap(err, "create snapshot block dir")
}
chunksDir := chunkDir(blockDir)
if err := os.MkdirAll(chunksDir, 0777); err != nil {
return errors.Wrap(err, "create snapshot chunk dir")
}
// Hardlink meta, index and tombstones
for _, fname := range []string{
metaFilename,
indexFilename,
tombstoneFilename,
} {
if err := os.Link(filepath.Join(pb.dir, fname), filepath.Join(blockDir, fname)); err != nil {
return errors.Wrapf(err, "create snapshot %s", fname)
}
}
// Hardlink the chunks
curChunkDir := chunkDir(pb.dir)
files, err := ioutil.ReadDir(curChunkDir)
if err != nil {
return errors.Wrap(err, "ReadDir the current chunk dir")
}
for _, f := range files {
err := os.Link(filepath.Join(curChunkDir, f.Name()), filepath.Join(chunksDir, f.Name()))
if err != nil {
return errors.Wrap(err, "hardlink a chunk")
}
}
return nil
}
// OverlapsClosedInterval returns true if the block overlaps [mint, maxt].
func (pb *Block) OverlapsClosedInterval(mint, maxt int64) bool {
// The block itself is a half-open interval
// [pb.meta.MinTime, pb.meta.MaxTime).
return pb.meta.MinTime <= maxt && mint < pb.meta.MaxTime
}
// LabelNames returns all the unique label names present in the Block in sorted order.
func (pb *Block) LabelNames() ([]string, error) {
return pb.indexr.LabelNames()
}
func clampInterval(a, b, mint, maxt int64) (int64, int64) {
if a < mint {
a = mint
}
if b > maxt {
b = maxt
}
return a, b
}