influxdb/tsdb/engine/tsm1/wal.go

837 lines
20 KiB
Go
Raw Normal View History

package tsm1
import (
"encoding/binary"
"errors"
"expvar"
"fmt"
"io"
"log"
2015-11-21 22:50:28 +00:00
"math"
"os"
"path/filepath"
"sort"
"strconv"
"strings"
"sync"
2015-11-21 22:50:28 +00:00
"time"
"github.com/golang/snappy"
"github.com/influxdata/influxdb"
"github.com/influxdata/influxdb/tsdb"
)
const (
2015-11-25 04:24:13 +00:00
// DefaultSegmentSize of 10MB is the size at which segment files will be rolled over
2015-11-21 22:50:28 +00:00
DefaultSegmentSize = 10 * 1024 * 1024
// FileExtension is the file extension we expect for wal segments
WALFileExtension = "wal"
WALFilePrefix = "_"
2015-11-19 06:15:44 +00:00
defaultBufLen = 1024 << 10 // 1MB (sized for batches of 5000 points)
2015-11-21 22:50:28 +00:00
// walEncodeBufSize is the size of the wal entry encoding buffer
walEncodeBufSize = 4 * 1024 * 1024
2015-11-21 22:50:28 +00:00
float64EntryType = 1
2016-01-18 22:48:49 +00:00
integerEntryType = 2
2015-11-04 21:06:06 +00:00
booleanEntryType = 3
2015-11-21 22:50:28 +00:00
stringEntryType = 4
)
// SegmentInfo represents metadata about a segment.
type SegmentInfo struct {
name string
id int
}
// WalEntryType is a byte written to a wal segment file that indicates what the following compressed block contains
type WalEntryType byte
const (
WriteWALEntryType WalEntryType = 0x01
DeleteWALEntryType WalEntryType = 0x02
)
var ErrWALClosed = fmt.Errorf("WAL closed")
// Statistics gathered by the WAL.
const (
statWALOldBytes = "oldSegmentsDiskBytes"
statWALCurrentBytes = "currentSegmentDiskBytes"
)
type WAL struct {
mu sync.RWMutex
lastWriteTime time.Time
2015-11-18 22:42:48 +00:00
path string
// write variables
2015-11-18 22:42:48 +00:00
currentSegmentID int
currentSegmentWriter *WALSegmentWriter
// cache and flush variables
closing chan struct{}
// WALOutput is the writer used by the logger.
LogOutput io.Writer
logger *log.Logger
// SegmentSize is the file size at which a segment file will be rotated
SegmentSize int
// LoggingEnabled specifies if detailed logs should be output
LoggingEnabled bool
statMap *expvar.Map
}
func NewWAL(path string) *WAL {
db, rp := tsdb.DecodeStorePath(path)
return &WAL{
path: path,
// these options should be overriden by any options in the config
2015-11-18 22:42:48 +00:00
LogOutput: os.Stderr,
SegmentSize: DefaultSegmentSize,
logger: log.New(os.Stderr, "[tsm1wal] ", log.LstdFlags),
2015-11-18 22:42:48 +00:00
closing: make(chan struct{}),
statMap: influxdb.NewStatistics(
"tsm1_wal:"+path,
"tsm1_wal",
map[string]string{"path": path, "database": db, "retentionPolicy": rp},
),
}
}
// Path returns the path the log was initialized with.
2015-11-18 22:42:48 +00:00
func (l *WAL) Path() string {
l.mu.RLock()
defer l.mu.RUnlock()
return l.path
}
// Open opens and initializes the Log. Will recover from previous unclosed shutdowns
func (l *WAL) Open() error {
2015-11-18 22:42:48 +00:00
l.mu.Lock()
defer l.mu.Unlock()
if l.LoggingEnabled {
l.logger.Printf("tsm1 WAL starting with %d segment size\n", l.SegmentSize)
l.logger.Printf("tsm1 WAL writing to %s\n", l.path)
}
if err := os.MkdirAll(l.path, 0777); err != nil {
return err
}
segments, err := segmentFileNames(l.path)
if err != nil {
return err
}
if len(segments) > 0 {
lastSegment := segments[len(segments)-1]
id, err := idFromFileName(lastSegment)
if err != nil {
return err
}
l.currentSegmentID = id
stat, err := os.Stat(lastSegment)
if err != nil {
return err
}
if stat.Size() == 0 {
os.Remove(lastSegment)
segments = segments[:len(segments)-1]
}
if err := l.newSegmentFile(); err != nil {
return err
}
}
var totalOldDiskSize int64
for _, seg := range segments {
stat, err := os.Stat(seg)
if err != nil {
return err
}
totalOldDiskSize += stat.Size()
}
sizeStat := new(expvar.Int)
sizeStat.Set(totalOldDiskSize)
l.statMap.Set(statWALOldBytes, sizeStat)
l.closing = make(chan struct{})
l.lastWriteTime = time.Now()
return nil
}
// WritePoints writes the given points to the WAL. Returns the WAL segment ID to
// which the points were written. If an error is returned the segment ID should
// be ignored.
func (l *WAL) WritePoints(values map[string][]Value) (int, error) {
entry := &WriteWALEntry{
2015-11-21 22:50:28 +00:00
Values: values,
}
id, err := l.writeToLog(entry)
if err != nil {
return -1, err
}
return id, nil
}
func (l *WAL) ClosedSegments() ([]string, error) {
2015-11-18 22:42:48 +00:00
l.mu.RLock()
defer l.mu.RUnlock()
2015-11-18 22:42:48 +00:00
// Not loading files from disk so nothing to do
if l.path == "" {
return nil, nil
}
var currentFile string
if l.currentSegmentWriter != nil {
currentFile = l.currentSegmentWriter.path()
}
files, err := segmentFileNames(l.path)
if err != nil {
return nil, err
}
var closedFiles []string
for _, fn := range files {
// Skip the current path
if fn == currentFile {
continue
}
closedFiles = append(closedFiles, fn)
}
return closedFiles, nil
2015-11-18 22:42:48 +00:00
}
func (l *WAL) Remove(files []string) error {
l.mu.Lock()
defer l.mu.Unlock()
for _, fn := range files {
os.RemoveAll(fn)
}
// Refresh the on-disk size stats
segments, err := segmentFileNames(l.path)
if err != nil {
return err
}
var totalOldDiskSize int64
for _, seg := range segments {
stat, err := os.Stat(seg)
if err != nil {
return err
}
totalOldDiskSize += stat.Size()
}
sizeStat := new(expvar.Int)
sizeStat.Set(totalOldDiskSize)
l.statMap.Set(statWALOldBytes, sizeStat)
return nil
}
// LastWriteTime is the last time anything was written to the WAL
func (l *WAL) LastWriteTime() time.Time {
2015-11-19 06:15:44 +00:00
l.mu.RLock()
defer l.mu.RUnlock()
return l.lastWriteTime
}
func (l *WAL) writeToLog(entry WALEntry) (int, error) {
// encode and compress the entry while we're not locked
bytes := getBuf(walEncodeBufSize)
defer putBuf(bytes)
b, err := entry.Encode(bytes)
if err != nil {
return -1, err
}
encBuf := getBuf(snappy.MaxEncodedLen(len(b)))
defer putBuf(encBuf)
compressed := snappy.Encode(encBuf, b)
l.mu.Lock()
defer l.mu.Unlock()
// Make sure the log has not been closed
select {
case <-l.closing:
return -1, ErrWALClosed
default:
}
// roll the segment file if needed
if err := l.rollSegment(); err != nil {
return -1, fmt.Errorf("error rolling WAL segment: %v", err)
}
// write and sync
if err := l.currentSegmentWriter.Write(entry.Type(), compressed); err != nil {
return -1, fmt.Errorf("error writing WAL entry: %v", err)
}
// Update stats for current segment size
curSize := new(expvar.Int)
curSize.Set(int64(l.currentSegmentWriter.size))
l.statMap.Set(statWALCurrentBytes, curSize)
l.lastWriteTime = time.Now()
return l.currentSegmentID, l.currentSegmentWriter.sync()
2015-11-18 22:42:48 +00:00
}
// rollSegment closes the current segment and opens a new one if the current segment is over
// the max segment size.
func (l *WAL) rollSegment() error {
if l.currentSegmentWriter == nil || l.currentSegmentWriter.size > DefaultSegmentSize {
if err := l.newSegmentFile(); err != nil {
// A drop database or RP call could trigger this error if writes were in-flight
// when the drop statement executes.
return fmt.Errorf("error opening new segment file for wal (2): %v", err)
}
2015-11-21 22:50:28 +00:00
return nil
}
return nil
}
// CloseSegment closes the current segment if it is non-empty and opens a new one.
func (l *WAL) CloseSegment() error {
l.mu.Lock()
defer l.mu.Unlock()
if l.currentSegmentWriter == nil || l.currentSegmentWriter.size > 0 {
if err := l.newSegmentFile(); err != nil {
// A drop database or RP call could trigger this error if writes were in-flight
// when the drop statement executes.
return fmt.Errorf("error opening new segment file for wal (1): %v", err)
}
return nil
}
return nil
}
// Delete deletes the given keys, returning the segment ID for the operation.
func (l *WAL) Delete(keys []string) (int, error) {
if len(keys) == 0 {
return 0, nil
}
2015-11-18 22:42:48 +00:00
entry := &DeleteWALEntry{
Keys: keys,
}
id, err := l.writeToLog(entry)
if err != nil {
return -1, err
}
return id, nil
}
// Close will finish any flush that is currently in process and close file handles
func (l *WAL) Close() error {
2015-11-18 22:42:48 +00:00
l.mu.Lock()
defer l.mu.Unlock()
// Close, but don't set to nil so future goroutines can still be signaled
close(l.closing)
2015-11-18 22:42:48 +00:00
if l.currentSegmentWriter != nil {
l.currentSegmentWriter.close()
l.currentSegmentWriter = nil
}
return nil
}
// segmentFileNames will return all files that are WAL segment files in sorted order by ascending ID
func segmentFileNames(dir string) ([]string, error) {
names, err := filepath.Glob(filepath.Join(dir, fmt.Sprintf("%s*.%s", WALFilePrefix, WALFileExtension)))
if err != nil {
return nil, err
}
sort.Strings(names)
return names, nil
}
// newSegmentFile will close the current segment file and open a new one, updating bookkeeping info on the log
func (l *WAL) newSegmentFile() error {
l.currentSegmentID++
2015-11-18 22:42:48 +00:00
if l.currentSegmentWriter != nil {
if err := l.currentSegmentWriter.close(); err != nil {
return err
}
l.statMap.Add(statWALOldBytes, int64(l.currentSegmentWriter.size))
}
fileName := filepath.Join(l.path, fmt.Sprintf("%s%05d.%s", WALFilePrefix, l.currentSegmentID, WALFileExtension))
2015-11-23 21:01:09 +00:00
fd, err := os.OpenFile(fileName, os.O_CREATE|os.O_RDWR, 0666)
if err != nil {
return err
}
2015-11-23 21:01:09 +00:00
l.currentSegmentWriter = NewWALSegmentWriter(fd)
// Reset the current segment size stat
curSize := new(expvar.Int)
curSize.Set(0)
l.statMap.Set(statWALCurrentBytes, curSize)
return nil
}
// WALEntry is record stored in each WAL segment. Each entry has a type
// and an opaque, type dependent byte slice data attribute.
type WALEntry interface {
Type() WalEntryType
Encode(dst []byte) ([]byte, error)
MarshalBinary() ([]byte, error)
2015-11-19 06:15:44 +00:00
UnmarshalBinary(b []byte) error
}
// WriteWALEntry represents a write of points.
type WriteWALEntry struct {
2015-11-21 22:50:28 +00:00
Values map[string][]Value
}
2015-11-24 16:44:37 +00:00
// Encode converts the WriteWALEntry into a byte stream using dst if it
// is large enough. If dst is too small, the slice will be grown to fit the
// encoded entry.
func (w *WriteWALEntry) Encode(dst []byte) ([]byte, error) {
2015-11-24 16:44:37 +00:00
// The entries values are encode as follows:
//
// For each key and slice of values, first a 1 byte type for the []Values
// slice is written. Following the type, the length and key bytes are written.
// Following the key, a 4 byte count followed by each value as a 8 byte time
// and N byte value. The value is dependent on the type being encoded. float64,
// int64, use 8 bytes, boolean uses 1 byte, and string is similar to the key encoding,
// except that string values have a 4-byte length, and keys only use 2 bytes.
2015-11-24 16:44:37 +00:00
//
// This structure is then repeated for each key an value slices.
//
// ┌────────────────────────────────────────────────────────────────────┐
// │ WriteWALEntry │
// ├──────┬─────────┬────────┬───────┬─────────┬─────────┬───┬──────┬───┤
// │ Type │ Key Len │ Key │ Count │ Time │ Value │...│ Type │...│
// │1 byte│ 2 bytes │ N bytes│4 bytes│ 8 bytes │ N bytes │ │1 byte│ │
2015-11-24 16:44:37 +00:00
// └──────┴─────────┴────────┴───────┴─────────┴─────────┴───┴──────┴───┘
2015-11-21 22:50:28 +00:00
encLen := 7 * len(w.Values) // Type (1), Key Length (2), and Count (4) for each key
// determine required length
2015-11-21 22:50:28 +00:00
for k, v := range w.Values {
encLen += len(k)
if len(v) == 0 {
return nil, errors.New("empty value slice in WAL entry")
}
encLen += 8 * len(v) // timestamps (8)
switch v[0].(type) {
case *FloatValue, *IntegerValue:
encLen += 8 * len(v)
case *BooleanValue:
encLen += 1 * len(v)
case *StringValue:
for _, vv := range v {
str, ok := vv.(*StringValue)
if !ok {
return nil, fmt.Errorf("non-string found in string value slice: %T", vv)
}
encLen += 4 + len(str.value)
}
default:
return nil, fmt.Errorf("unsupported value type: %T", v[0])
}
}
// allocate or re-slice to correct size
if len(dst) < encLen {
dst = make([]byte, encLen)
} else {
dst = dst[:encLen]
}
// Finally, encode the entry
var n int
var curType byte
for k, v := range w.Values {
switch v[0].(type) {
case *FloatValue:
curType = float64EntryType
case *IntegerValue:
curType = integerEntryType
case *BooleanValue:
curType = booleanEntryType
case *StringValue:
curType = stringEntryType
2015-11-21 22:50:28 +00:00
default:
return nil, fmt.Errorf("unsupported value type: %T", v[0])
}
dst[n] = curType
2015-11-21 22:50:28 +00:00
n++
binary.BigEndian.PutUint16(dst[n:n+2], uint16(len(k)))
n += 2
n += copy(dst[n:], k)
2015-11-21 22:50:28 +00:00
binary.BigEndian.PutUint32(dst[n:n+4], uint32(len(v)))
n += 4
2015-11-21 22:50:28 +00:00
for _, vv := range v {
2016-02-23 03:47:17 +00:00
binary.BigEndian.PutUint64(dst[n:n+8], uint64(vv.UnixNano()))
n += 8
switch vv := vv.(type) {
case *FloatValue:
if curType != float64EntryType {
return nil, fmt.Errorf("incorrect value found in %T slice: %T", v[0].Value(), vv)
}
binary.BigEndian.PutUint64(dst[n:n+8], math.Float64bits(vv.value))
n += 8
case *IntegerValue:
if curType != integerEntryType {
return nil, fmt.Errorf("incorrect value found in %T slice: %T", v[0].Value(), vv)
}
binary.BigEndian.PutUint64(dst[n:n+8], uint64(vv.value))
n += 8
case *BooleanValue:
if curType != booleanEntryType {
return nil, fmt.Errorf("incorrect value found in %T slice: %T", v[0].Value(), vv)
}
if vv.value {
dst[n] = 1
2015-11-21 22:50:28 +00:00
} else {
dst[n] = 0
2015-11-21 22:50:28 +00:00
}
n++
case *StringValue:
if curType != stringEntryType {
return nil, fmt.Errorf("incorrect value found in %T slice: %T", v[0].Value(), vv)
}
binary.BigEndian.PutUint32(dst[n:n+4], uint32(len(vv.value)))
n += 4
n += copy(dst[n:], vv.value)
default:
return nil, fmt.Errorf("unsupported value found in %T slice: %T", v[0].Value(), vv)
2015-11-21 22:50:28 +00:00
}
}
}
return dst[:n], nil
}
func (w *WriteWALEntry) MarshalBinary() ([]byte, error) {
// Temp buffer to write marshaled points into
2015-11-19 06:15:44 +00:00
b := make([]byte, defaultBufLen)
return w.Encode(b)
}
2015-11-19 06:15:44 +00:00
func (w *WriteWALEntry) UnmarshalBinary(b []byte) error {
var i int
for i < len(b) {
2015-11-21 22:50:28 +00:00
typ := b[i]
i++
length := int(binary.BigEndian.Uint16(b[i : i+2]))
2015-11-21 22:50:28 +00:00
i += 2
k := string(b[i : i+length])
i += length
nvals := int(binary.BigEndian.Uint32(b[i : i+4]))
i += 4
2015-11-21 22:50:28 +00:00
var values []Value
switch typ {
case float64EntryType:
values = getFloat64Values(nvals)
2016-01-18 22:48:49 +00:00
case integerEntryType:
values = getIntegerValues(nvals)
2015-11-04 21:06:06 +00:00
case booleanEntryType:
values = getBooleanValues(nvals)
2015-11-21 22:50:28 +00:00
case stringEntryType:
values = getStringValues(nvals)
default:
return fmt.Errorf("unsupported value type: %#v", typ)
}
2015-11-21 22:50:28 +00:00
for j := 0; j < nvals; j++ {
un := int64(binary.BigEndian.Uint64(b[i : i+8]))
2015-11-21 22:50:28 +00:00
i += 8
switch typ {
case float64EntryType:
v := math.Float64frombits((binary.BigEndian.Uint64(b[i : i+8])))
2015-11-21 22:50:28 +00:00
i += 8
if fv, ok := values[j].(*FloatValue); ok {
fv.unixnano = un
2015-11-21 22:50:28 +00:00
fv.value = v
}
2016-01-18 22:48:49 +00:00
case integerEntryType:
v := int64(binary.BigEndian.Uint64(b[i : i+8]))
2015-11-21 22:50:28 +00:00
i += 8
2016-01-18 22:48:49 +00:00
if fv, ok := values[j].(*IntegerValue); ok {
fv.unixnano = un
2015-11-21 22:50:28 +00:00
fv.value = v
}
2015-11-04 21:06:06 +00:00
case booleanEntryType:
2015-11-21 22:50:28 +00:00
v := b[i]
i += 1
2015-11-04 21:06:06 +00:00
if fv, ok := values[j].(*BooleanValue); ok {
fv.unixnano = un
2015-11-21 22:50:28 +00:00
if v == 1 {
fv.value = true
} else {
fv.value = false
}
}
case stringEntryType:
length := int(binary.BigEndian.Uint32(b[i : i+4]))
if i+length > int(uint32(len(b))) {
return fmt.Errorf("corrupted write wall entry")
}
2015-11-21 22:50:28 +00:00
i += 4
v := string(b[i : i+length])
i += length
if fv, ok := values[j].(*StringValue); ok {
fv.unixnano = un
2015-11-21 22:50:28 +00:00
fv.value = v
}
default:
return fmt.Errorf("unsupported value type: %#v", typ)
}
}
w.Values[k] = values
}
return nil
}
func (w *WriteWALEntry) Type() WalEntryType {
return WriteWALEntryType
}
// DeleteWALEntry represents the deletion of multiple series.
type DeleteWALEntry struct {
Keys []string
}
func (w *DeleteWALEntry) MarshalBinary() ([]byte, error) {
2015-11-19 06:15:44 +00:00
b := make([]byte, defaultBufLen)
return w.Encode(b)
}
2015-11-19 06:15:44 +00:00
func (w *DeleteWALEntry) UnmarshalBinary(b []byte) error {
w.Keys = strings.Split(string(b), "\n")
return nil
}
func (w *DeleteWALEntry) Encode(dst []byte) ([]byte, error) {
var n int
for _, k := range w.Keys {
if len(dst[:n])+1+len(k) > len(dst) {
grow := make([]byte, len(dst)*2)
dst = append(dst, grow...)
}
n += copy(dst[n:], k)
n += copy(dst[n:], "\n")
}
// We return n-1 to strip off the last newline so that unmarshalling the value
// does not produce an empty string
return []byte(dst[:n-1]), nil
}
func (w *DeleteWALEntry) Type() WalEntryType {
return DeleteWALEntryType
}
// WALSegmentWriter writes WAL segments.
type WALSegmentWriter struct {
w io.WriteCloser
size int
}
func NewWALSegmentWriter(w io.WriteCloser) *WALSegmentWriter {
return &WALSegmentWriter{
w: w,
}
}
func (w *WALSegmentWriter) path() string {
2015-11-18 22:42:48 +00:00
if f, ok := w.w.(*os.File); ok {
return f.Name()
}
return ""
}
func (w *WALSegmentWriter) Write(entryType WalEntryType, compressed []byte) error {
var buf [5]byte
buf[0] = byte(entryType)
binary.BigEndian.PutUint32(buf[1:5], uint32(len(compressed)))
if _, err := w.w.Write(buf[:]); err != nil {
2015-11-19 06:15:44 +00:00
return err
}
2015-11-21 22:50:28 +00:00
if _, err := w.w.Write(compressed); err != nil {
2015-11-19 06:15:44 +00:00
return err
}
w.size += len(buf) + len(compressed)
2015-11-18 22:42:48 +00:00
return nil
}
// Sync flushes the file systems in-memory copy of recently written data to disk.
func (w *WALSegmentWriter) sync() error {
if f, ok := w.w.(*os.File); ok {
return f.Sync()
}
return nil
}
func (w *WALSegmentWriter) close() error {
return w.w.Close()
}
// WALSegmentReader reads WAL segments.
type WALSegmentReader struct {
r io.ReadCloser
entry WALEntry
n int64
err error
}
func NewWALSegmentReader(r io.ReadCloser) *WALSegmentReader {
return &WALSegmentReader{
r: r,
}
}
// Next indicates if there is a value to read
func (r *WALSegmentReader) Next() bool {
2015-11-19 06:15:44 +00:00
b := getBuf(defaultBufLen)
defer putBuf(b)
var nReadOK int
// read the type and the length of the entry
n, err := io.ReadFull(r.r, b[:5])
if err == io.EOF {
return false
}
if err != nil {
r.err = err
// We return true here because we want the client code to call read which
// will return the this error to be handled.
return true
}
nReadOK += n
entryType := b[0]
length := binary.BigEndian.Uint32(b[1:5])
// read the compressed block and decompress it
if int(length) > len(b) {
b = make([]byte, length)
}
n, err = io.ReadFull(r.r, b[:length])
if err != nil {
r.err = err
return true
}
nReadOK += n
decLen, err := snappy.DecodedLen(b[:length])
if err != nil {
r.err = err
return true
}
decBuf := getBuf(decLen)
defer putBuf(decBuf)
data, err := snappy.Decode(decBuf, b[:length])
if err != nil {
r.err = err
return true
}
// and marshal it and send it to the cache
switch WalEntryType(entryType) {
case WriteWALEntryType:
2015-11-21 22:50:28 +00:00
r.entry = &WriteWALEntry{
Values: map[string][]Value{},
}
case DeleteWALEntryType:
r.entry = &DeleteWALEntry{}
default:
r.err = fmt.Errorf("unknown wal entry type: %v", entryType)
return true
}
2015-11-19 06:15:44 +00:00
r.err = r.entry.UnmarshalBinary(data)
if r.err == nil {
// Read and decode of this entry was successful.
r.n += int64(nReadOK)
}
return true
}
func (r *WALSegmentReader) Read() (WALEntry, error) {
if r.err != nil {
return nil, r.err
}
return r.entry, nil
}
// Count returns the total number of bytes read successfully from the segment, as
// of the last call to Read(). The segment is guaranteed to be valid up to and
// including this number of bytes.
func (r *WALSegmentReader) Count() int64 {
return r.n
}
func (r *WALSegmentReader) Error() error {
return r.err
}
2015-11-21 22:50:28 +00:00
func (r *WALSegmentReader) Close() error {
return r.r.Close()
}
// idFromFileName parses the segment file ID from its name
func idFromFileName(name string) (int, error) {
parts := strings.Split(filepath.Base(name), ".")
if len(parts) != 2 {
return 0, fmt.Errorf("file %s has wrong name format to have an id", name)
}
id, err := strconv.ParseUint(parts[0][1:], 10, 32)
return int(id), err
}