influxdb/tsdb/series_file.go

751 lines
19 KiB
Go
Raw Normal View History

2017-11-15 23:09:25 +00:00
package tsdb
2017-09-14 15:41:58 +00:00
import (
"bytes"
"encoding/binary"
2017-12-27 15:09:36 +00:00
"encoding/hex"
2017-09-14 15:41:58 +00:00
"fmt"
2017-11-22 15:30:02 +00:00
"io"
2017-12-29 18:57:30 +00:00
"io/ioutil"
2017-09-14 15:41:58 +00:00
"os"
2017-11-15 23:09:25 +00:00
"path/filepath"
2017-09-25 15:31:20 +00:00
"sync"
2017-09-14 15:41:58 +00:00
2017-11-22 15:30:02 +00:00
"github.com/cespare/xxhash"
2017-09-14 15:41:58 +00:00
"github.com/influxdata/influxdb/models"
"github.com/influxdata/influxdb/pkg/rhh"
)
2017-11-30 17:23:03 +00:00
// SeriesIDSize is the size in bytes of a series key ID.
2017-09-26 13:40:26 +00:00
const SeriesIDSize = 8
2017-09-14 15:41:58 +00:00
2017-11-30 17:23:03 +00:00
// SeriesMapThreshold is the number of series IDs to hold in the in-memory
// series map before compacting and rebuilding the on-disk representation.
2017-12-15 15:58:01 +00:00
const SeriesMapThreshold = 1 << 25 // ~33M ids * 8 bytes per id == 256MB
2017-10-02 14:07:11 +00:00
2017-09-14 15:41:58 +00:00
// SeriesFile represents the section of the index that holds series data.
type SeriesFile struct {
2017-09-25 15:31:20 +00:00
mu sync.RWMutex
2017-09-14 15:41:58 +00:00
path string
2017-12-29 18:57:30 +00:00
segments []*SeriesSegment
index *SeriesIndex
seq uint64 // series id sequence
2017-09-14 15:41:58 +00:00
}
// NewSeriesFile returns a new instance of SeriesFile.
func NewSeriesFile(path string) *SeriesFile {
return &SeriesFile{
2017-12-29 18:57:30 +00:00
path: path,
2017-09-14 15:41:58 +00:00
}
}
// Open memory maps the data file at the file's path.
func (f *SeriesFile) Open() error {
2017-12-29 18:57:30 +00:00
// Create path if it doesn't exist.
if err := os.MkdirAll(filepath.Join(f.path), 0777); err != nil {
2017-11-15 23:09:25 +00:00
return err
}
2017-12-19 17:31:33 +00:00
// Open components.
if err := func() (err error) {
2017-12-29 18:57:30 +00:00
if err := f.openSegments(); err != nil {
2017-12-19 17:31:33 +00:00
return err
}
2017-09-14 15:41:58 +00:00
2017-12-29 18:57:30 +00:00
// Init last segment for writes.
if err := f.activeSegment().InitForWrite(); err != nil {
2017-09-14 15:41:58 +00:00
return err
}
2017-12-29 18:57:30 +00:00
f.index = NewSeriesIndex(f.IndexPath())
if err := f.index.Open(); err != nil {
2017-12-19 17:31:33 +00:00
return err
}
2017-12-29 18:57:30 +00:00
// TODO: Replay new entries since index was built.
2017-09-14 15:41:58 +00:00
2017-12-29 18:57:30 +00:00
return nil
}(); err != nil {
f.Close()
return err
}
2017-12-20 22:13:34 +00:00
2017-12-29 18:57:30 +00:00
return nil
}
2017-12-20 22:13:34 +00:00
2017-12-29 18:57:30 +00:00
func (f *SeriesFile) openSegments() error {
fis, err := ioutil.ReadDir(f.path)
if err != nil {
return err
}
2017-12-20 22:13:34 +00:00
2017-12-29 18:57:30 +00:00
for _, fi := range fis {
if !IsValidSeriesSegmentFilename(fi.Name()) {
continue
}
2017-12-20 22:13:34 +00:00
2017-12-29 18:57:30 +00:00
segment := NewSeriesSegment(ParseSeriesSegmentFilename(fi.Name()), filepath.Join(f.path, fi.Name()))
if err := segment.Open(); err != nil {
return err
}
f.segments = append(f.segments, segment)
}
2017-12-20 22:13:34 +00:00
2017-12-29 18:57:30 +00:00
// Find max series id by searching segments in reverse order.
for i := len(f.segments) - 1; i >= 0; i-- {
if f.seq = f.segments[i].MaxSeriesID(); f.seq > 0 {
break
2017-12-20 22:13:34 +00:00
}
2017-12-29 18:57:30 +00:00
}
2017-12-19 17:31:33 +00:00
2017-12-29 18:57:30 +00:00
// Create initial segment if none exist.
if len(f.segments) == 0 {
segment, err := CreateSeriesSegment(0, filepath.Join(f.path, "0000"))
if err != nil {
return err
}
f.segments = append(f.segments, segment)
2017-09-14 15:41:58 +00:00
}
return nil
}
// Close unmaps the data file.
2017-12-29 18:57:30 +00:00
func (f *SeriesFile) Close() (err error) {
2017-09-25 15:31:20 +00:00
f.mu.Lock()
defer f.mu.Unlock()
2017-12-29 18:57:30 +00:00
for _, s := range f.segments {
if e := s.Close(); e != nil && err == nil {
err = e
}
2017-09-14 15:41:58 +00:00
}
2017-12-29 18:57:30 +00:00
f.segments = nil
2017-12-21 21:50:07 +00:00
2017-12-29 18:57:30 +00:00
if f.index != nil {
if e := f.index.Close(); e != nil && err == nil {
err = e
}
2017-12-21 21:50:07 +00:00
}
2017-12-29 18:57:30 +00:00
f.index = nil
2017-12-21 21:50:07 +00:00
2017-12-29 18:57:30 +00:00
return err
2017-09-14 15:41:58 +00:00
}
// Path returns the path to the file.
func (f *SeriesFile) Path() string { return f.path }
2017-12-29 18:57:30 +00:00
// Path returns the path to the series index.
func (f *SeriesFile) IndexPath() string { return filepath.Join(f.path, "index") }
// CreateSeriesListIfNotExists creates a list of series in bulk if they don't exist. Returns the offset of the series.
2017-12-19 17:31:33 +00:00
func (f *SeriesFile) CreateSeriesListIfNotExists(names [][]byte, tagsSlice []models.Tags, buf []byte) (ids []uint64, err error) {
2017-09-25 15:31:20 +00:00
f.mu.RLock()
2017-12-29 18:57:30 +00:00
ids, ok := f.index.FindIDListByNameTags(f.segments, names, tagsSlice, buf)
2017-12-19 17:31:33 +00:00
if ok {
f.mu.RUnlock()
return ids, nil
2017-09-25 15:31:20 +00:00
}
f.mu.RUnlock()
2017-12-19 17:31:33 +00:00
type keyRange struct {
id uint64
offset int64
2017-09-25 15:31:20 +00:00
}
2017-12-19 17:31:33 +00:00
newKeyRanges := make([]keyRange, 0, len(names))
2017-09-25 15:31:20 +00:00
// Obtain write lock to create new series.
f.mu.Lock()
defer f.mu.Unlock()
2017-10-17 19:58:07 +00:00
// Track offsets of duplicate series.
2017-12-19 17:31:33 +00:00
newIDs := make(map[string]uint64, len(ids))
2017-10-17 19:58:07 +00:00
2017-09-25 15:31:20 +00:00
for i := range names {
// Skip series that have already been created.
2017-12-19 17:31:33 +00:00
if id := ids[i]; id != 0 {
2017-09-25 15:31:20 +00:00
continue
}
2017-10-17 19:58:07 +00:00
// Generate series key.
buf = AppendSeriesKey(buf[:0], names[i], tagsSlice[i])
2017-09-25 15:31:20 +00:00
// Re-attempt lookup under write lock.
2017-12-19 17:31:33 +00:00
if ids[i] = newIDs[string(buf)]; ids[i] != 0 {
2017-10-17 19:58:07 +00:00
continue
2017-12-29 18:57:30 +00:00
} else if ids[i] = f.index.FindIDByNameTags(f.segments, names[i], tagsSlice[i], buf); ids[i] != 0 {
continue
}
2017-12-19 17:31:33 +00:00
// Write to series log and save offset.
id, offset, err := f.insert(buf)
if err != nil {
2017-10-26 19:55:00 +00:00
return nil, err
}
// Append new key to be added to hash map after flush.
2017-12-19 17:31:33 +00:00
ids[i] = id
2017-12-20 22:13:34 +00:00
newIDs[string(buf)] = id
2017-12-29 18:57:30 +00:00
newKeyRanges = append(newKeyRanges, keyRange{id, offset})
}
2017-12-29 18:57:30 +00:00
// Flush active segment writes so we can access data in mmap.
if segment := f.activeSegment(); segment != nil {
if err := segment.Flush(); err != nil {
return nil, err
}
}
2017-10-02 14:07:11 +00:00
// Add keys to hash map(s).
for _, keyRange := range newKeyRanges {
2017-12-29 18:57:30 +00:00
f.index.Insert(f.seriesKeyByOffset(keyRange.offset), keyRange.id, keyRange.offset)
2017-12-19 17:31:33 +00:00
}
2017-10-02 14:07:11 +00:00
2017-12-19 17:31:33 +00:00
return ids, nil
}
2017-10-02 14:07:11 +00:00
2017-10-26 19:55:00 +00:00
// DeleteSeriesID flags a series as permanently deleted.
2017-12-19 17:31:33 +00:00
// If the series is reintroduced later then it must create a new id.
func (f *SeriesFile) DeleteSeriesID(id uint64) error {
2017-10-26 19:55:00 +00:00
f.mu.Lock()
defer f.mu.Unlock()
// Already tombstoned, ignore.
2017-12-29 18:57:30 +00:00
if f.index.IsDeleted(id) {
2017-10-26 19:55:00 +00:00
return nil
}
// Write tombstone entry.
2017-12-29 18:57:30 +00:00
_, err := f.writeLogEntry(AppendSeriesEntry(nil, SeriesEntryTombstoneFlag, id, nil))
if err != nil {
2017-10-26 19:55:00 +00:00
return err
}
// Mark tombstone in memory.
2017-12-29 18:57:30 +00:00
f.index.Delete(id)
2017-10-26 19:55:00 +00:00
2017-10-25 13:29:44 +00:00
return nil
}
2017-10-26 19:55:00 +00:00
// IsDeleted returns true if the ID has been deleted before.
2017-12-19 17:31:33 +00:00
func (f *SeriesFile) IsDeleted(id uint64) bool {
2017-09-25 15:31:20 +00:00
f.mu.RLock()
2017-12-29 18:57:30 +00:00
v := f.index.IsDeleted(id)
2017-09-25 15:31:20 +00:00
f.mu.RUnlock()
2017-12-19 17:31:33 +00:00
return v
2017-09-25 15:31:20 +00:00
}
2017-12-19 17:31:33 +00:00
// SeriesKey returns the series key for a given id.
func (f *SeriesFile) SeriesKey(id uint64) []byte {
if id == 0 {
2017-09-14 15:41:58 +00:00
return nil
}
2017-12-19 17:31:33 +00:00
f.mu.RLock()
2017-12-29 18:57:30 +00:00
key := f.seriesKeyByOffset(f.index.FindOffsetByID(id))
2017-12-19 17:31:33 +00:00
f.mu.RUnlock()
return key
2017-09-17 18:06:37 +00:00
}
// Series returns the parsed series name and tags for an offset.
2017-12-19 17:31:33 +00:00
func (f *SeriesFile) Series(id uint64) ([]byte, models.Tags) {
key := f.SeriesKey(id)
2017-09-17 18:06:37 +00:00
if key == nil {
return nil, nil
}
return ParseSeriesKey(key)
2017-09-14 15:41:58 +00:00
}
2017-12-19 17:31:33 +00:00
// SeriesID return the series id for the series.
func (f *SeriesFile) SeriesID(name []byte, tags models.Tags, buf []byte) uint64 {
f.mu.RLock()
2017-12-29 18:57:30 +00:00
id := f.index.FindIDBySeriesKey(f.segments, AppendSeriesKey(buf[:0], name, tags))
2017-12-19 17:31:33 +00:00
f.mu.RUnlock()
return id
}
2017-09-14 15:41:58 +00:00
// HasSeries return true if the series exists.
func (f *SeriesFile) HasSeries(name []byte, tags models.Tags, buf []byte) bool {
2017-12-29 18:57:30 +00:00
return f.SeriesID(name, tags, buf) > 0
2017-09-14 15:41:58 +00:00
}
// SeriesCount returns the number of series.
2017-09-26 13:40:26 +00:00
func (f *SeriesFile) SeriesCount() uint64 {
2017-10-02 14:07:11 +00:00
f.mu.RLock()
2017-12-29 18:57:30 +00:00
n := f.index.Count()
2017-10-02 14:07:11 +00:00
f.mu.RUnlock()
return n
2017-09-14 15:41:58 +00:00
}
// SeriesIterator returns an iterator over all the series.
2017-11-15 23:09:25 +00:00
func (f *SeriesFile) SeriesIDIterator() SeriesIDIterator {
2017-12-20 22:13:34 +00:00
var ids []uint64
2017-12-29 18:57:30 +00:00
for _, segment := range f.segments {
ids = segment.AppendSeriesIDs(ids)
2017-11-22 15:30:02 +00:00
}
2017-12-29 18:57:30 +00:00
return NewSeriesIDSliceIterator(ids)
2017-12-19 17:31:33 +00:00
}
2017-11-22 15:30:02 +00:00
2017-12-29 18:57:30 +00:00
// activeSegment returns the last segment.
func (f *SeriesFile) activeSegment() *SeriesSegment {
if len(f.segments) == 0 {
2017-12-19 17:31:33 +00:00
return nil
2017-11-22 15:30:02 +00:00
}
2017-12-29 18:57:30 +00:00
return f.segments[len(f.segments)-1]
2017-12-20 22:13:34 +00:00
}
2017-12-29 18:57:30 +00:00
func (f *SeriesFile) insert(key []byte) (id uint64, offset int64, err error) {
id = f.seq + 1
2017-12-20 22:13:34 +00:00
2017-12-29 18:57:30 +00:00
offset, err = f.writeLogEntry(AppendSeriesEntry(nil, SeriesEntryInsertFlag, id, key))
if err != nil {
return 0, 0, err
2017-12-20 22:13:34 +00:00
}
2017-12-27 15:09:36 +00:00
2017-12-29 18:57:30 +00:00
f.seq++
return id, offset, nil
2017-12-27 15:09:36 +00:00
}
2017-12-29 18:57:30 +00:00
// writeLogEntry appends an entry to the end of the active segment.
// If there is no more room in the segment then a new segment is added.
func (f *SeriesFile) writeLogEntry(data []byte) (offset int64, err error) {
segment := f.activeSegment()
if segment == nil || !segment.CanWrite(data) {
if segment, err = f.createSegment(); err != nil {
return 0, err
2017-12-20 22:13:34 +00:00
}
}
2017-12-29 18:57:30 +00:00
return segment.WriteLogEntry(data)
2017-11-22 15:30:02 +00:00
}
2017-12-29 18:57:30 +00:00
// createSegment appends a new segment
func (f *SeriesFile) createSegment() (*SeriesSegment, error) {
// Close writer for active segment, if one exists.
if segment := f.activeSegment(); segment != nil {
if err := segment.CloseForWrite(); err != nil {
return nil, err
}
2017-11-22 15:30:02 +00:00
}
2017-12-19 17:31:33 +00:00
2017-12-29 18:57:30 +00:00
// Generate a new sequential segment identifier.
var id uint16
if len(f.segments) > 0 {
id = f.segments[len(f.segments)-1].ID() + 1
2017-11-22 15:30:02 +00:00
}
2017-12-29 18:57:30 +00:00
filename := fmt.Sprintf("%04x", id)
2017-11-22 15:30:02 +00:00
2017-12-29 18:57:30 +00:00
// Generate new empty segment.
segment, err := CreateSeriesSegment(id, filepath.Join(f.path, filename))
if err != nil {
return nil, err
2017-12-19 17:31:33 +00:00
}
2017-12-29 18:57:30 +00:00
f.segments = append(f.segments, segment)
2017-12-20 22:13:34 +00:00
2017-12-29 18:57:30 +00:00
// Allow segment to write.
if err := segment.InitForWrite(); err != nil {
return nil, err
2017-12-20 22:13:34 +00:00
}
2017-11-22 15:30:02 +00:00
2017-12-29 18:57:30 +00:00
return segment, nil
2017-12-19 17:31:33 +00:00
}
2017-10-02 14:07:11 +00:00
2017-12-29 18:57:30 +00:00
func (f *SeriesFile) seriesKeyByOffset(offset int64) []byte {
if offset == 0 {
return nil
2017-10-02 14:07:11 +00:00
}
2017-12-29 18:57:30 +00:00
segmentID, pos := SplitSeriesOffset(offset)
for _, segment := range f.segments {
if segment.ID() != segmentID {
continue
}
2017-10-02 14:07:11 +00:00
2017-12-29 18:57:30 +00:00
key, _ := ReadSeriesKey(segment.Slice(pos + SeriesEntryHeaderSize))
return key
2017-12-20 22:13:34 +00:00
}
2017-12-29 18:57:30 +00:00
return nil
2017-10-02 14:07:11 +00:00
}
2017-09-14 15:41:58 +00:00
// AppendSeriesKey serializes name and tags to a byte slice.
// The total length is prepended as a uvarint.
func AppendSeriesKey(dst []byte, name []byte, tags models.Tags) []byte {
2017-09-26 13:40:26 +00:00
buf := make([]byte, binary.MaxVarintLen64)
2017-09-14 15:41:58 +00:00
origLen := len(dst)
// The tag count is variable encoded, so we need to know ahead of time what
// the size of the tag count value will be.
2017-09-26 13:40:26 +00:00
tcBuf := make([]byte, binary.MaxVarintLen64)
2017-09-14 15:41:58 +00:00
tcSz := binary.PutUvarint(tcBuf, uint64(len(tags)))
// Size of name/tags. Does not include total length.
size := 0 + //
2 + // size of measurement
len(name) + // measurement
tcSz + // size of number of tags
(4 * len(tags)) + // length of each tag key and value
tags.Size() // size of tag keys/values
// Variable encode length.
totalSz := binary.PutUvarint(buf, uint64(size))
// If caller doesn't provide a buffer then pre-allocate an exact one.
if dst == nil {
dst = make([]byte, 0, size+totalSz)
}
// Append total length.
dst = append(dst, buf[:totalSz]...)
// Append name.
binary.BigEndian.PutUint16(buf, uint16(len(name)))
dst = append(dst, buf[:2]...)
dst = append(dst, name...)
// Append tag count.
dst = append(dst, tcBuf[:tcSz]...)
// Append tags.
for _, tag := range tags {
binary.BigEndian.PutUint16(buf, uint16(len(tag.Key)))
dst = append(dst, buf[:2]...)
dst = append(dst, tag.Key...)
binary.BigEndian.PutUint16(buf, uint16(len(tag.Value)))
dst = append(dst, buf[:2]...)
dst = append(dst, tag.Value...)
}
// Verify that the total length equals the encoded byte count.
if got, exp := len(dst)-origLen, size+totalSz; got != exp {
panic(fmt.Sprintf("series key encoding does not match calculated total length: actual=%d, exp=%d, key=%x", got, exp, dst))
}
return dst
}
// ReadSeriesKey returns the series key from the beginning of the buffer.
2017-09-18 19:03:47 +00:00
func ReadSeriesKey(data []byte) (key, remainder []byte) {
2017-09-14 15:41:58 +00:00
sz, n := binary.Uvarint(data)
2017-09-18 19:03:47 +00:00
return data[:int(sz)+n], data[int(sz)+n:]
2017-09-14 15:41:58 +00:00
}
func ReadSeriesKeyLen(data []byte) (sz int, remainder []byte) {
2017-09-17 18:06:37 +00:00
sz64, i := binary.Uvarint(data)
return int(sz64), data[i:]
2017-09-14 15:41:58 +00:00
}
func ReadSeriesKeyMeasurement(data []byte) (name, remainder []byte) {
2017-09-17 18:06:37 +00:00
n, data := binary.BigEndian.Uint16(data), data[2:]
2017-09-14 15:41:58 +00:00
return data[:n], data[n:]
}
func ReadSeriesKeyTagN(data []byte) (n int, remainder []byte) {
2017-09-17 18:06:37 +00:00
n64, i := binary.Uvarint(data)
return int(n64), data[i:]
2017-09-14 15:41:58 +00:00
}
func ReadSeriesKeyTag(data []byte) (key, value, remainder []byte) {
n, data := binary.BigEndian.Uint16(data), data[2:]
key, data = data[:n], data[n:]
n, data = binary.BigEndian.Uint16(data), data[2:]
value, data = data[:n], data[n:]
return key, value, data
}
2017-09-17 18:06:37 +00:00
// ParseSeriesKey extracts the name & tags from a series key.
func ParseSeriesKey(data []byte) (name []byte, tags models.Tags) {
2017-09-14 15:41:58 +00:00
_, data = ReadSeriesKeyLen(data)
2017-09-17 18:06:37 +00:00
name, data = ReadSeriesKeyMeasurement(data)
2017-09-14 15:41:58 +00:00
tagN, data := ReadSeriesKeyTagN(data)
2017-09-17 18:06:37 +00:00
tags = make(models.Tags, tagN)
2017-09-14 15:41:58 +00:00
for i := 0; i < tagN; i++ {
var key, value []byte
key, value, data = ReadSeriesKeyTag(data)
tags[i] = models.Tag{Key: key, Value: value}
}
2017-09-17 18:06:37 +00:00
return name, tags
2017-09-14 15:41:58 +00:00
}
func CompareSeriesKeys(a, b []byte) int {
// Handle 'nil' keys.
if len(a) == 0 && len(b) == 0 {
return 0
} else if len(a) == 0 {
return -1
} else if len(b) == 0 {
return 1
}
// Read total size.
2017-09-17 18:06:37 +00:00
_, a = ReadSeriesKeyLen(a)
_, b = ReadSeriesKeyLen(b)
2017-09-14 15:41:58 +00:00
// Read names.
name0, a := ReadSeriesKeyMeasurement(a)
name1, b := ReadSeriesKeyMeasurement(b)
// Compare names, return if not equal.
if cmp := bytes.Compare(name0, name1); cmp != 0 {
return cmp
}
// Read tag counts.
tagN0, a := ReadSeriesKeyTagN(a)
tagN1, b := ReadSeriesKeyTagN(b)
// Compare each tag in order.
2017-09-17 18:06:37 +00:00
for i := 0; ; i++ {
2017-09-14 15:41:58 +00:00
// Check for EOF.
if i == tagN0 && i == tagN1 {
return 0
} else if i == tagN0 {
return -1
} else if i == tagN1 {
return 1
}
// Read keys.
var key0, key1, value0, value1 []byte
key0, value0, a = ReadSeriesKeyTag(a)
key1, value1, b = ReadSeriesKeyTag(b)
// Compare keys & values.
if cmp := bytes.Compare(key0, key1); cmp != 0 {
return cmp
} else if cmp := bytes.Compare(value0, value1); cmp != 0 {
return cmp
}
}
}
type seriesKeys [][]byte
func (a seriesKeys) Len() int { return len(a) }
func (a seriesKeys) Swap(i, j int) { a[i], a[j] = a[j], a[i] }
func (a seriesKeys) Less(i, j int) bool {
return CompareSeriesKeys(a[i], a[j]) == -1
}
2017-10-02 14:07:11 +00:00
2017-12-29 18:57:30 +00:00
// SeriesFileCompactor represents an object reindexes a series file and optionally compacts segments.
type SeriesFileCompactor struct{}
2017-12-21 21:50:07 +00:00
2017-12-29 18:57:30 +00:00
// NewSeriesFileCompactor returns a new instance of SeriesFileCompactor.
func NewSeriesFileCompactor() *SeriesFileCompactor {
return &SeriesFileCompactor{}
2017-10-02 14:07:11 +00:00
}
2017-12-29 18:57:30 +00:00
// Compact rebuilds the series file index.
func (c *SeriesFileCompactor) Compact(f *SeriesFile) error {
// Snapshot the partitions and index so we can check tombstones and replay at the end under lock.
f.mu.RLock()
segments := CloneSeriesSegments(f.segments)
index := f.index.Clone()
seriesN := f.index.Count()
f.mu.RUnlock()
2017-12-19 17:31:33 +00:00
2017-12-29 18:57:30 +00:00
// Compact index to a temporary location.
indexPath := index.path + ".compacting"
if err := c.compactIndexTo(index, seriesN, segments, indexPath); err != nil {
return err
2017-10-02 14:07:11 +00:00
}
2017-12-29 18:57:30 +00:00
// Swap compacted index under lock & replay since compaction.
if err := func() error {
f.mu.Lock()
defer f.mu.Unlock()
2017-12-19 17:31:33 +00:00
2017-12-29 18:57:30 +00:00
// Reopen index with new file.
if err := index.Close(); err != nil {
return err
} else if err := os.Rename(indexPath, index.path); err != nil {
return err
} else if err := index.Open(); err != nil {
return err
2017-12-19 17:31:33 +00:00
}
2017-12-29 18:57:30 +00:00
// Replay new entries.
if err := index.Recover(f.segments); err != nil {
return err
2017-10-26 19:55:00 +00:00
}
2017-12-29 18:57:30 +00:00
return nil
}(); err != nil {
return err
2017-10-02 14:07:11 +00:00
}
2017-12-21 21:50:07 +00:00
return nil
}
2017-12-29 18:57:30 +00:00
func (c *SeriesFileCompactor) compactIndexTo(index *SeriesIndex, seriesN uint64, segments []*SeriesSegment, path string) error {
hdr := NewSeriesIndexHeader()
hdr.Count = seriesN
hdr.Capacity = pow2((int64(hdr.Count) * 100) / SeriesIndexLoadFactor)
2017-12-19 17:31:33 +00:00
2017-12-29 18:57:30 +00:00
// Allocate space for maps.
keyIDMap := make([]byte, (hdr.Capacity * SeriesIndexElemSize))
idOffsetMap := make([]byte, (hdr.Capacity * SeriesIndexElemSize))
2017-10-02 14:07:11 +00:00
2017-12-29 18:57:30 +00:00
// Reindex all partitions.
for _, segment := range segments {
if err := segment.ForEachEntry(func(flag uint8, id uint64, offset int64, key []byte) error {
// Only process insert entries.
switch flag {
case SeriesEntryInsertFlag: // fallthrough
case SeriesEntryTombstoneFlag:
return nil
default:
return fmt.Errorf("unexpected series file log entry flag: %d", flag)
}
2017-10-02 14:07:11 +00:00
2017-12-29 18:57:30 +00:00
// Ignore entry if tombstoned.
if index.IsDeleted(id) {
return nil
}
2017-10-02 14:07:11 +00:00
2017-12-29 18:57:30 +00:00
// Save highest id/offset to header.
hdr.MaxSeriesID, hdr.MaxOffset = id, offset
2017-12-19 17:31:33 +00:00
2017-12-29 18:57:30 +00:00
// Insert into maps.
c.insertIDOffsetMap(idOffsetMap, hdr.Capacity, id, offset)
return c.insertKeyIDMap(keyIDMap, hdr.Capacity, segments, key, offset, id)
}); err != nil {
return err
2017-10-02 14:07:11 +00:00
}
}
2017-12-27 15:09:36 +00:00
// Open file handler.
f, err := os.Create(path)
if err != nil {
return err
}
defer f.Close()
2017-12-29 18:57:30 +00:00
// Calculate map positions.
hdr.KeyIDMap.Offset, hdr.KeyIDMap.Size = SeriesIndexHeaderSize, int64(len(keyIDMap))
hdr.IDOffsetMap.Offset, hdr.IDOffsetMap.Size = hdr.KeyIDMap.Offset+hdr.KeyIDMap.Size, int64(len(idOffsetMap))
2017-12-27 15:09:36 +00:00
2017-12-29 18:57:30 +00:00
// Write header.
2017-12-27 15:09:36 +00:00
if _, err := hdr.WriteTo(f); err != nil {
return err
}
2017-12-29 18:57:30 +00:00
// Write maps.
2017-12-27 15:09:36 +00:00
if _, err := f.Write(keyIDMap); err != nil {
return err
2017-12-29 18:57:30 +00:00
} else if _, err := f.Write(idOffsetMap); err != nil {
2017-12-27 15:09:36 +00:00
return err
}
2017-12-29 18:57:30 +00:00
// Sync & close.
2017-12-27 15:09:36 +00:00
if err := f.Sync(); err != nil {
return err
} else if err := f.Close(); err != nil {
return err
}
return nil
}
2017-10-02 14:07:11 +00:00
2017-12-29 18:57:30 +00:00
func (c *SeriesFileCompactor) insertKeyIDMap(dst []byte, capacity int64, segments []*SeriesSegment, key []byte, offset int64, id uint64) error {
2017-12-27 15:09:36 +00:00
mask := capacity - 1
2017-10-02 14:07:11 +00:00
hash := rhh.HashKey(key)
// Continue searching until we find an empty slot or lower probe distance.
for dist, pos := int64(0), hash&mask; ; dist, pos = dist+1, (pos+1)&mask {
2017-12-29 18:57:30 +00:00
elem := dst[(pos * SeriesIndexElemSize):]
2017-12-27 15:09:36 +00:00
// If empty slot found or matching offset, insert and exit.
elemOffset := int64(binary.BigEndian.Uint64(elem[:8]))
elemID := binary.BigEndian.Uint64(elem[8:])
if elemOffset == 0 || elemOffset == offset {
binary.BigEndian.PutUint64(elem[:8], uint64(offset))
binary.BigEndian.PutUint64(elem[8:], id)
return nil
}
2017-12-29 18:57:30 +00:00
// Read key at position & hash.
elemKey := ReadSeriesKeyFromSegments(segments, elemOffset)
2017-12-27 15:09:36 +00:00
elemHash := rhh.HashKey(elemKey)
// If the existing elem has probed less than us, then swap places with
// existing elem, and keep going to find another slot for that elem.
if d := rhh.Dist(elemHash, pos, capacity); d < dist {
2017-10-02 14:07:11 +00:00
2017-12-27 15:09:36 +00:00
// Insert current values.
binary.BigEndian.PutUint64(elem[:8], uint64(offset))
binary.BigEndian.PutUint64(elem[8:], id)
2017-10-02 14:07:11 +00:00
2017-12-27 15:09:36 +00:00
// Swap with values in that position.
hash, key, offset, id = elemHash, elemKey, elemOffset, elemID
// Update current distance.
dist = d
}
}
}
2017-12-29 18:57:30 +00:00
func (c *SeriesFileCompactor) insertIDOffsetMap(dst []byte, capacity int64, id uint64, offset int64) {
2017-12-27 15:09:36 +00:00
mask := capacity - 1
hash := rhh.HashUint64(id)
// Continue searching until we find an empty slot or lower probe distance.
2017-12-29 18:57:30 +00:00
for i, dist, pos := int64(0), int64(0), hash&mask; ; i, dist, pos = i+1, dist+1, (pos+1)&mask {
elem := dst[(pos * SeriesIndexElemSize):]
2017-12-27 15:09:36 +00:00
// If empty slot found or matching id, insert and exit.
elemID := binary.BigEndian.Uint64(elem[:8])
elemOffset := int64(binary.BigEndian.Uint64(elem[8:]))
if elemOffset == 0 || elemOffset == offset {
binary.BigEndian.PutUint64(elem[:8], id)
binary.BigEndian.PutUint64(elem[8:], uint64(offset))
2017-10-02 14:07:11 +00:00
return
}
2017-12-27 15:09:36 +00:00
// Hash key.
elemHash := rhh.HashUint64(elemID)
2017-10-02 14:07:11 +00:00
// If the existing elem has probed less than us, then swap places with
// existing elem, and keep going to find another slot for that elem.
2017-12-27 15:09:36 +00:00
if d := rhh.Dist(elemHash, pos, capacity); d < dist {
2017-10-02 14:07:11 +00:00
// Insert current values.
2017-12-27 15:09:36 +00:00
binary.BigEndian.PutUint64(elem[:8], id)
binary.BigEndian.PutUint64(elem[8:], uint64(offset))
2017-10-02 14:07:11 +00:00
// Swap with values in that position.
2017-12-27 15:09:36 +00:00
hash, id, offset = elemHash, elemID, elemOffset
2017-10-02 14:07:11 +00:00
// Update current distance.
dist = d
}
2017-12-29 18:57:30 +00:00
if i > capacity {
panic("rhh map full")
}
2017-10-02 14:07:11 +00:00
}
}
// pow2 returns the number that is the next highest power of 2.
// Returns v if it is a power of 2.
func pow2(v int64) int64 {
for i := int64(2); i < 1<<62; i *= 2 {
if i >= v {
return i
}
}
panic("unreachable")
}
2017-11-22 15:30:02 +00:00
// hashReader generates an xxhash from the contents of r.
func hashReader(r io.Reader) ([]byte, error) {
h := xxhash.New()
if _, err := io.Copy(h, r); err != nil {
return nil, err
}
return h.Sum(nil), nil
}
2017-12-27 15:09:36 +00:00
func hexdump(data []byte) { os.Stderr.Write([]byte(hex.Dump(data))) }