prometheus/index.go
2017-04-30 10:10:18 +02:00

865 lines
20 KiB
Go

// Copyright 2017 The Prometheus Authors
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package tsdb
import (
"bufio"
"encoding/binary"
"fmt"
"hash"
"hash/crc32"
"io"
"os"
"path/filepath"
"sort"
"strings"
"github.com/coreos/etcd/pkg/fileutil"
"github.com/pkg/errors"
"github.com/prometheus/tsdb/labels"
)
const (
// MagicIndex 4 bytes at the head of an index file.
MagicIndex = 0xBAAAD700
indexFormatV1 = 1
indexSeriesFormatV1 = 1
)
const compactionPageBytes = minSectorSize * 64
// IndexWriter serializes the index for a block of series data.
// The methods must generally be called in the order they are specified in.
type IndexWriter interface {
// AddSeries populates the index writer with a series and its offsets
// of chunks that the index can reference.
// The reference number is used to resolve a series against the postings
// list iterator. It only has to be available during the write processing.
AddSeries(ref uint32, l labels.Labels, chunks ...*ChunkMeta) 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 Postings) error
// Close writes any finalization and closes the resources associated with
// the underlying writer.
Close() error
}
type indexWriterSeries struct {
labels labels.Labels
chunks []*ChunkMeta // series file offset of chunks
offset uint32 // index file offset of series reference
}
// indexWriter implements the IndexWriter interface for the standard
// serialization format.
type indexWriter struct {
f *os.File
fbuf *bufio.Writer
pos int
started bool
// Reusable memory.
buf1 encbuf
buf2 encbuf
b []byte
uint32s []uint32
series map[uint32]*indexWriterSeries
symbols map[string]uint32 // symbol offsets
labelIndexes []hashEntry // label index offsets
postings []hashEntry // postings lists offsets
crc32 hash.Hash
}
func newIndexWriter(dir string) (*indexWriter, error) {
df, err := fileutil.OpenDir(dir)
if err != nil {
return nil, err
}
f, err := os.OpenFile(filepath.Join(dir, "index"), os.O_CREATE|os.O_WRONLY, 0666)
if err != nil {
return nil, err
}
if err := fileutil.Fsync(df); err != nil {
return nil, errors.Wrap(err, "sync dir")
}
iw := &indexWriter{
f: f,
fbuf: bufio.NewWriterSize(f, 1<<22),
pos: 0,
// Reusable memory.
buf1: encbuf{b: make([]byte, 0, 1<<22)},
buf2: encbuf{b: make([]byte, 0, 1<<22)},
b: make([]byte, 0, 1<<23),
uint32s: make([]uint32, 0, 1<<15),
// Caches.
symbols: make(map[string]uint32, 1<<13),
series: make(map[uint32]*indexWriterSeries, 1<<16),
crc32: crc32.New(crc32.MakeTable(crc32.Castagnoli)),
}
if err := iw.writeMeta(); err != nil {
return nil, err
}
return iw, nil
}
func (w *indexWriter) write(bufs ...[]byte) error {
for _, b := range bufs {
n, err := w.fbuf.Write(b)
w.pos += n
if err != nil {
return err
}
}
return nil
}
func (w *indexWriter) writeMeta() error {
b := [5]byte{}
binary.BigEndian.PutUint32(b[:4], MagicIndex)
b[4] = flagStd
return w.write(b[:])
}
func (w *indexWriter) AddSeries(ref uint32, lset labels.Labels, chunks ...*ChunkMeta) error {
if _, ok := w.series[ref]; ok {
return errors.Errorf("series with reference %d already added", ref)
}
// Populate the symbol table from all label sets we have to reference.
for _, l := range lset {
w.symbols[l.Name] = 0
w.symbols[l.Value] = 0
}
w.series[ref] = &indexWriterSeries{
labels: lset,
chunks: chunks,
}
return nil
}
func (w *indexWriter) writeSymbols() error {
// Generate sorted list of strings we will store as reference table.
symbols := make([]string, 0, len(w.symbols))
for s := range w.symbols {
symbols = append(symbols, s)
}
sort.Strings(symbols)
const headerSize = 8
w.buf1.reset()
w.buf2.reset()
for _, s := range symbols {
w.symbols[s] = uint32(w.pos + headerSize + w.buf2.len())
w.buf2.putUvarint(len(s))
w.buf2.putString(s)
}
w.buf1.putBE32int(len(symbols))
w.buf1.putBE32int(w.buf2.len())
w.buf2.putHash(w.crc32)
err := w.write(w.buf1.get(), w.buf2.get())
return errors.Wrap(err, "write symbols")
}
type indexWriterSeriesSlice []*indexWriterSeries
func (s indexWriterSeriesSlice) Len() int { return len(s) }
func (s indexWriterSeriesSlice) Swap(i, j int) { s[i], s[j] = s[j], s[i] }
func (s indexWriterSeriesSlice) Less(i, j int) bool {
return labels.Compare(s[i].labels, s[j].labels) < 0
}
func (w *indexWriter) writeSeries() error {
// Series must be stored sorted along their labels.
series := make(indexWriterSeriesSlice, 0, len(w.series))
for _, s := range w.series {
series = append(series, s)
}
sort.Sort(series)
// Header holds number of series.
w.buf1.reset()
w.buf1.putBE32int(len(series))
if err := w.write(w.buf1.get()); err != nil {
return errors.Wrap(err, "write series count")
}
for _, s := range series {
w.buf2.reset()
w.buf2.putUvarint(len(s.labels))
for _, l := range s.labels {
w.buf2.putUvarint32(w.symbols[l.Name])
w.buf2.putUvarint32(w.symbols[l.Value])
}
w.buf2.putUvarint(len(s.chunks))
for _, c := range s.chunks {
w.buf2.putVarint64(c.MinTime)
w.buf2.putVarint64(c.MaxTime)
w.buf2.putUvarint64(c.Ref)
w.crc32.Reset()
c.hash(w.crc32)
w.buf2.putBytes(w.crc32.Sum(nil))
}
s.offset = uint32(w.pos)
w.buf1.reset()
w.buf1.putUvarint(w.buf2.len())
w.buf2.putHash(w.crc32)
if err := w.write(w.buf1.get(), w.buf2.get()); err != nil {
return errors.Wrap(err, "write series data")
}
}
return nil
}
func (w *indexWriter) init() error {
if err := w.writeSymbols(); err != nil {
return err
}
if err := w.writeSeries(); err != nil {
return err
}
w.started = true
return nil
}
func (w *indexWriter) ensureStarted() error {
if w.started {
return nil
}
return w.init()
}
func (w *indexWriter) WriteLabelIndex(names []string, values []string) error {
if err := w.ensureStarted(); err != nil {
return errors.Wrap(err, "initialize")
}
valt, err := newStringTuples(values, len(names))
if err != nil {
return err
}
sort.Sort(valt)
w.labelIndexes = append(w.labelIndexes, hashEntry{
name: strings.Join(names, string(sep)),
offset: uint32(w.pos),
})
w.buf2.reset()
w.buf2.putUvarint(len(names))
for _, v := range valt.s {
w.buf2.putBE32(w.symbols[v])
}
w.buf1.reset()
w.buf1.putUvarint(w.buf2.len())
w.buf2.putHash(w.crc32)
err = w.write(w.buf1.get(), w.buf2.get())
return errors.Wrap(err, "write label index")
}
func (w *indexWriter) WritePostings(name, value string, it Postings) error {
if err := w.ensureStarted(); err != nil {
return errors.Wrap(err, "initialize")
}
key := name + string(sep) + value
w.postings = append(w.postings, hashEntry{
name: key,
offset: uint32(w.pos),
})
// Order of the references in the postings list does not imply order
// of the series references within the persisted block they are mapped to.
// We have to sort the new references again.
refs := w.uint32s[:0]
for it.Next() {
s, ok := w.series[it.At()]
if !ok {
return errors.Errorf("series for reference %d not found", it.At())
}
refs = append(refs, s.offset)
}
if err := it.Err(); err != nil {
return err
}
sort.Sort(uint32slice(refs))
w.buf2.reset()
for _, r := range refs {
w.buf2.putBE32(r)
}
w.buf1.reset()
w.buf1.putUvarint(w.buf2.len())
w.buf2.putHash(w.crc32)
err := w.write(w.buf1.get(), w.buf2.get())
return errors.Wrap(err, "write postings")
}
type uint32slice []uint32
func (s uint32slice) Len() int { return len(s) }
func (s uint32slice) Swap(i, j int) { s[i], s[j] = s[j], s[i] }
func (s uint32slice) Less(i, j int) bool { return s[i] < s[j] }
type hashEntry struct {
name string
offset uint32
}
func (w *indexWriter) writeHashmap(h []hashEntry) error {
w.b = append(w.b[:0], flagStd, 0, 0, 0, 0)
buf := [binary.MaxVarintLen32]byte{}
for _, e := range h {
n := binary.PutUvarint(buf[:], uint64(len(e.name)))
w.b = append(w.b, buf[:n]...)
w.b = append(w.b, e.name...)
n = binary.PutUvarint(buf[:], uint64(e.offset))
w.b = append(w.b, buf[:n]...)
}
binary.BigEndian.PutUint32(w.b[1:], uint32(len(w.b)-5))
w.crc32.Reset()
if _, err := w.crc32.Write(w.b[5:]); err != nil {
return errors.Wrap(err, "calculate label index CRC32 checksum")
}
w.b = w.crc32.Sum(w.b)
return w.write(w.b)
}
func (w *indexWriter) finalize() error {
// Write out hash maps to jump to correct label index and postings sections.
lo := uint32(w.pos)
if err := w.writeHashmap(w.labelIndexes); err != nil {
return err
}
po := uint32(w.pos)
if err := w.writeHashmap(w.postings); err != nil {
return err
}
// Terminate index file with offsets to hashmaps. This is the entry Pointer
// for any index query.
// TODO(fabxc): also store offset to series section to allow plain
// iteration over all existing series?
b := [8]byte{}
binary.BigEndian.PutUint32(b[:4], lo)
binary.BigEndian.PutUint32(b[4:], po)
return w.write(b[:])
}
func (w *indexWriter) Close() error {
if err := w.finalize(); err != nil {
return err
}
if err := w.fbuf.Flush(); err != nil {
return err
}
if err := fileutil.Fsync(w.f); err != nil {
return err
}
return w.f.Close()
}
// IndexReader provides reading access of serialized index data.
type IndexReader interface {
// LabelValues returns the possible label values
LabelValues(names ...string) (StringTuples, error)
// Postings returns the postings list iterator for the label pair.
// The Postings here contain the offsets to the series inside the index.
Postings(name, value string) (Postings, error)
// Series returns the series for the given reference.
Series(ref uint32) (labels.Labels, []*ChunkMeta, error)
// LabelIndices returns the label pairs for which indices exist.
LabelIndices() ([][]string, error)
// Close released 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)
}
type indexReader struct {
// The underlying byte slice holding the encoded series data.
b []byte
// Close that releases the underlying resources of the byte slice.
c io.Closer
// Cached hashmaps of section offsets.
labels map[string]uint32
postings map[string]uint32
}
var (
errInvalidSize = fmt.Errorf("invalid size")
errInvalidFlag = fmt.Errorf("invalid flag")
)
// newIndexReader returns a new indexReader on the given directory.
func newIndexReader(dir string) (*indexReader, error) {
f, err := openMmapFile(filepath.Join(dir, "index"))
if err != nil {
return nil, err
}
r := &indexReader{b: f.b, c: f}
// Verify magic number.
if len(f.b) < 4 {
return nil, errors.Wrap(errInvalidSize, "index header")
}
if m := binary.BigEndian.Uint32(r.b[:4]); m != MagicIndex {
return nil, errors.Errorf("invalid magic number %x", m)
}
// The last two 4 bytes hold the pointers to the hashmaps.
loff := binary.BigEndian.Uint32(r.b[len(r.b)-8 : len(r.b)-4])
poff := binary.BigEndian.Uint32(r.b[len(r.b)-4:])
flag, b, err := r.section(loff)
if err != nil {
return nil, errors.Wrapf(err, "label index hashmap section at %d", loff)
}
if r.labels, err = readHashmap(flag, b); err != nil {
return nil, errors.Wrap(err, "read label index hashmap")
}
flag, b, err = r.section(poff)
if err != nil {
return nil, errors.Wrapf(err, "postings hashmap section at %d", loff)
}
if r.postings, err = readHashmap(flag, b); err != nil {
return nil, errors.Wrap(err, "read postings hashmap")
}
return r, nil
}
func readHashmap(flag byte, b []byte) (map[string]uint32, error) {
if flag != flagStd {
return nil, errInvalidFlag
}
h := make(map[string]uint32, 512)
for len(b) > 0 {
l, n := binary.Uvarint(b)
if n < 1 {
return nil, errors.Wrap(errInvalidSize, "read key length")
}
b = b[n:]
if len(b) < int(l) {
return nil, errors.Wrap(errInvalidSize, "read key")
}
s := string(b[:l])
b = b[l:]
o, n := binary.Uvarint(b)
if n < 1 {
return nil, errors.Wrap(errInvalidSize, "read offset value")
}
b = b[n:]
h[s] = uint32(o)
}
return h, nil
}
func (r *indexReader) Close() error {
return r.c.Close()
}
func (r *indexReader) section(o uint32) (byte, []byte, error) {
b := r.b[o:]
if len(b) < 5 {
return 0, nil, errors.Wrap(errInvalidSize, "read header")
}
flag := b[0]
l := binary.BigEndian.Uint32(b[1:5])
b = b[5:]
// b must have the given length plus 4 bytes for the CRC32 checksum.
if len(b) < int(l)+4 {
return 0, nil, errors.Wrap(errInvalidSize, "section content")
}
return flag, b[:l], nil
}
func (r *indexReader) lookupSymbol(o uint32) (string, error) {
if int(o) > len(r.b) {
return "", errors.Errorf("invalid symbol offset %d", o)
}
l, n := binary.Uvarint(r.b[o:])
if n < 0 {
return "", errors.New("reading symbol length failed")
}
end := int(o) + n + int(l)
if end > len(r.b) {
return "", errors.Errorf("invalid length %d", l)
}
b := r.b[int(o)+n : end]
return yoloString(b), nil
}
func (r *indexReader) getSized(off uint32) ([]byte, error) {
if int(off) > len(r.b) {
return nil, errInvalidSize
}
b := r.b[off:]
l, n := binary.Uvarint(b)
if n < 1 {
return nil, errInvalidSize
}
if int(l) > len(b[n:]) {
return nil, errInvalidSize
}
return b[n : n+int(l)], nil
}
func (r *indexReader) LabelValues(names ...string) (StringTuples, error) {
key := strings.Join(names, string(sep))
off, ok := r.labels[key]
if !ok {
// XXX(fabxc): hot fix. Should return a partial data error and handle cases
// where the entire block has no data gracefully.
return emptyStringTuples{}, nil
//return nil, fmt.Errorf("label index doesn't exist")
}
b, err := r.getSized(off)
if err != nil {
return nil, errors.Wrapf(err, "get sized region at %d", off)
}
c, n := binary.Uvarint(b)
if n < 1 {
return nil, errors.Wrap(errInvalidSize, "read label index size")
}
st := &serializedStringTuples{
l: int(c),
b: b[n:],
lookup: r.lookupSymbol,
}
return st, nil
}
type emptyStringTuples struct{}
func (emptyStringTuples) At(i int) ([]string, error) { return nil, nil }
func (emptyStringTuples) Len() int { return 0 }
func (r *indexReader) LabelIndices() ([][]string, error) {
res := [][]string{}
for s := range r.labels {
res = append(res, strings.Split(s, string(sep)))
}
return res, nil
}
func (r *indexReader) Series(ref uint32) (labels.Labels, []*ChunkMeta, error) {
// Read away length of series data.
_, n := binary.Uvarint(r.b[ref:])
b := r.b[int(ref)+n:]
k, n := binary.Uvarint(b)
if n < 1 {
return nil, nil, errors.Wrap(errInvalidSize, "number of labels")
}
b = b[n:]
lbls := make(labels.Labels, 0, k)
for i := 0; i < 2*int(k); i += 2 {
o, m := binary.Uvarint(b)
if m < 1 {
return nil, nil, errors.Wrap(errInvalidSize, "symbol offset")
}
n, err := r.lookupSymbol(uint32(o))
if err != nil {
return nil, nil, errors.Wrap(err, "symbol lookup")
}
b = b[m:]
o, m = binary.Uvarint(b)
if m < 1 {
return nil, nil, errors.Wrap(errInvalidSize, "symbol offset")
}
v, err := r.lookupSymbol(uint32(o))
if err != nil {
return nil, nil, errors.Wrap(err, "symbol lookup")
}
b = b[m:]
lbls = append(lbls, labels.Label{
Name: n,
Value: v,
})
}
// Read the chunks meta data.
k, n = binary.Uvarint(b)
if n < 1 {
return nil, nil, errors.Wrap(errInvalidSize, "number of chunks")
}
b = b[n:]
chunks := make([]*ChunkMeta, 0, k)
for i := 0; i < int(k); i++ {
firstTime, n := binary.Varint(b)
if n < 1 {
return nil, nil, errors.Wrap(errInvalidSize, "first time")
}
b = b[n:]
lastTime, n := binary.Varint(b)
if n < 1 {
return nil, nil, errors.Wrap(errInvalidSize, "last time")
}
b = b[n:]
o, n := binary.Uvarint(b)
if n < 1 {
return nil, nil, errors.Wrap(errInvalidSize, "chunk offset")
}
b = b[n:]
// TODO(fabxc): read and potentially verify checksum.
b = b[4:]
chunks = append(chunks, &ChunkMeta{
Ref: o,
MinTime: firstTime,
MaxTime: lastTime,
})
}
// TODO(fabxc): read and potentially verify checksum.
return lbls, chunks, nil
}
func (r *indexReader) Postings(name, value string) (Postings, error) {
key := name + string(sep) + value
off, ok := r.postings[key]
if !ok {
return emptyPostings, nil
}
b, err := r.getSized(off)
if err != nil {
return nil, errors.Wrapf(err, "get sized region at %d", off)
}
// Add iterator over the bytes.
if len(b)%4 != 0 {
return nil, errors.Wrap(errInvalidSize, "plain postings entry")
}
return newBigEndianPostings(b), nil
}
type stringTuples struct {
l int // tuple length
s []string // flattened tuple entries
}
func newStringTuples(s []string, l int) (*stringTuples, error) {
if len(s)%l != 0 {
return nil, errors.Wrap(errInvalidSize, "string tuple list")
}
return &stringTuples{s: s, l: l}, nil
}
func (t *stringTuples) Len() int { return len(t.s) / t.l }
func (t *stringTuples) At(i int) ([]string, error) { return t.s[i : i+t.l], nil }
func (t *stringTuples) Swap(i, j int) {
c := make([]string, t.l)
copy(c, t.s[i:i+t.l])
for k := 0; k < t.l; k++ {
t.s[i+k] = t.s[j+k]
t.s[j+k] = c[k]
}
}
func (t *stringTuples) Less(i, j int) bool {
for k := 0; k < t.l; k++ {
d := strings.Compare(t.s[i+k], t.s[j+k])
if d < 0 {
return true
}
if d > 0 {
return false
}
}
return false
}
type serializedStringTuples struct {
l int
b []byte
lookup func(uint32) (string, error)
}
func (t *serializedStringTuples) Len() int {
// TODO(fabxc): Cache this?
return len(t.b) / (4 * t.l)
}
func (t *serializedStringTuples) At(i int) ([]string, error) {
if len(t.b) < (i+t.l)*4 {
return nil, errInvalidSize
}
res := make([]string, 0, t.l)
for k := 0; k < t.l; k++ {
offset := binary.BigEndian.Uint32(t.b[(i+k)*4:])
s, err := t.lookup(offset)
if err != nil {
return nil, errors.Wrap(err, "symbol lookup")
}
res = append(res, s)
}
return res, nil
}
type encbuf struct {
b []byte
c [binary.MaxVarintLen64]byte
}
func (e *encbuf) reset() {
e.b = e.b[:0]
}
func (e *encbuf) putBE32(x uint32) {
binary.BigEndian.PutUint32(e.c[:], x)
e.b = append(e.b, e.c[:4]...)
}
func (e *encbuf) putBE32int(x int) {
e.putBE32(uint32(x))
}
func (e *encbuf) putUvarint32(x uint32) {
e.putUvarint64(uint64(x))
}
func (e *encbuf) putUvarint(x int) {
e.putUvarint64(uint64(x))
}
func (e *encbuf) putUvarint64(x uint64) {
n := binary.PutUvarint(e.c[:], x)
e.b = append(e.b, e.c[:n]...)
}
func (e *encbuf) putVarint64(x int64) {
n := binary.PutVarint(e.c[:], x)
e.b = append(e.b, e.c[:n]...)
}
func (e *encbuf) putString(s string) {
e.b = append(e.b, s...)
}
func (e *encbuf) putBytes(b []byte) {
e.b = append(e.b, b...)
}
func (e *encbuf) putHash(h hash.Hash) {
h.Reset()
_, err := h.Write(e.b)
if err != nil {
panic(err) // The CRC32 implementation does not error
}
h.Sum(e.b)
}
func (e *encbuf) get() []byte {
return e.b
}
func (e *encbuf) len() int {
return len(e.b)
}