From 3446c6024d17c51ceae92c1477ce7eab361bca7e Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Wed, 2 Sep 2015 18:05:07 -0400 Subject: [PATCH 001/139] WIP: skeleton for encoding for new engine --- tsdb/engine/pd1/encoding.go | 51 ++++++++++++++++++++++++++++++++ tsdb/engine/pd1/encoding_test.go | 38 ++++++++++++++++++++++++ 2 files changed, 89 insertions(+) create mode 100644 tsdb/engine/pd1/encoding.go create mode 100644 tsdb/engine/pd1/encoding_test.go diff --git a/tsdb/engine/pd1/encoding.go b/tsdb/engine/pd1/encoding.go new file mode 100644 index 00000000000..7739ce22341 --- /dev/null +++ b/tsdb/engine/pd1/encoding.go @@ -0,0 +1,51 @@ +package pd1 + +import () + +type FloatValue struct { + Time int64 + Value float64 +} + +func EncodeFloatBlock(buf []byte, values []FloatValue) []byte { + return nil +} + +func DecodeFloatBlock(block []byte) ([]FloatValue, error) { + return nil, nil +} + +type BoolValue struct { + Time int64 + Value bool +} + +func EncodeBoolBlock(buf []byte, values []BoolValue) []byte { + return nil +} + +func DecodeBoolBlock(block []byte) ([]BoolValue, error) { + return nil, nil +} + +type Int64Value struct { + Time int64 + Value int64 +} + +func EncodeInt64Block(buf []byte, values []Int64Value) []byte { + return nil +} + +func DecodeInt64Block(block []byte) ([]Int64Value, error) { + return nil, nil +} + +type StringValue struct { + Time int64 + Value string +} + +func EncodeStringBlock(values []StringValue) []byte { + return nil +} diff --git a/tsdb/engine/pd1/encoding_test.go b/tsdb/engine/pd1/encoding_test.go new file mode 100644 index 00000000000..077053449a0 --- /dev/null +++ b/tsdb/engine/pd1/encoding_test.go @@ -0,0 +1,38 @@ +package pd1_test + +import ( + "math/rand" + "reflect" + "testing" + "time" + + "github.com/influxdb/influxdb/tsdb/engine/pd1" +) + +func TestEncoding_FloatBlock(t *testing.T) { + valueCount := 100 + times := getTimes(valueCount, 60, time.Second) + values := make([]pd1.FloatValue, len(times)) + for i, t := range times { + values[i] = pd1.FloatValue{Time: t, Value: rand.Float64()} + } + + b := pd1.EncodeFloatBlock(nil, values) + + decodedValues, err := pd1.DecodeFloatBlock(b) + if err != nil { + t.Fatalf("error decoding: %s", err.Error) + } + if !reflect.DeepEqual(decodedValues, values) { + t.Fatalf("values not equal:\n\tgot: %s\n\texp: %s", values, decodedValues) + } +} + +func getTimes(n, step int, precision time.Duration) []int64 { + t := time.Now().Round(precision) + a := make([]int64, n) + for i := 0; i < n; i++ { + a[i] = t.Add(60 * precision).UnixNano() + } + return a +} From 8d16c96060f92abcae25d5ab28d7d2c7b831e07e Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Wed, 2 Sep 2015 18:45:32 -0400 Subject: [PATCH 002/139] Add comment about encoding float --- tsdb/engine/pd1/encoding.go | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/tsdb/engine/pd1/encoding.go b/tsdb/engine/pd1/encoding.go index 7739ce22341..b53f4f25635 100644 --- a/tsdb/engine/pd1/encoding.go +++ b/tsdb/engine/pd1/encoding.go @@ -1,12 +1,16 @@ package pd1 -import () +import ( + "time" +) type FloatValue struct { Time int64 Value float64 } +// First 8 bytes should be the timestamp, second 8 bytes should be +// the first float value func EncodeFloatBlock(buf []byte, values []FloatValue) []byte { return nil } From 041e7bead3b46904d35e80c10fd5b766aa47b0c0 Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Sun, 6 Sep 2015 15:49:15 -0700 Subject: [PATCH 003/139] WIP: engine work --- cmd/influxd/run/server.go | 2 + services/httpd/handler.go | 2 +- tsdb/engine.go | 17 +- tsdb/engine/b1/b1.go | 2 +- tsdb/engine/bz1/bz1.go | 2 +- tsdb/engine/bz1/bz1_test.go | 4 +- tsdb/engine/engine.go | 1 + tsdb/engine/pd1/encoding.go | 36 +- tsdb/engine/pd1/encoding_test.go | 9 +- tsdb/engine/pd1/pd1.go | 1255 ++++++++++++++++++++++++++++++ tsdb/engine/pd1/pd1_test.go | 196 +++++ tsdb/shard.go | 13 +- 12 files changed, 1517 insertions(+), 22 deletions(-) create mode 100644 tsdb/engine/pd1/pd1.go create mode 100644 tsdb/engine/pd1/pd1_test.go diff --git a/cmd/influxd/run/server.go b/cmd/influxd/run/server.go index 13aa35a97ce..64253f0f57e 100644 --- a/cmd/influxd/run/server.go +++ b/cmd/influxd/run/server.go @@ -83,6 +83,8 @@ func NewServer(c *Config, buildInfo *BuildInfo) (*Server, error) { tsdbStore := tsdb.NewStore(c.Data.Dir) tsdbStore.EngineOptions.Config = c.Data + runtime.GOMAXPROCS(runtime.NumCPU()) + s := &Server{ buildInfo: *buildInfo, err: make(chan error), diff --git a/services/httpd/handler.go b/services/httpd/handler.go index b3c136e794d..2fb96e99df7 100644 --- a/services/httpd/handler.go +++ b/services/httpd/handler.go @@ -83,7 +83,7 @@ func NewHandler(requireAuthentication, loggingEnabled, writeTrace bool, statMap mux: pat.New(), requireAuthentication: requireAuthentication, Logger: log.New(os.Stderr, "[http] ", log.LstdFlags), - loggingEnabled: loggingEnabled, + loggingEnabled: false, WriteTrace: writeTrace, statMap: statMap, } diff --git a/tsdb/engine.go b/tsdb/engine.go index c8d5946139f..d2862b54863 100644 --- a/tsdb/engine.go +++ b/tsdb/engine.go @@ -24,7 +24,7 @@ type Engine interface { Close() error SetLogOutput(io.Writer) - LoadMetadataIndex(index *DatabaseIndex, measurementFields map[string]*MeasurementFields) error + LoadMetadataIndex(shard *Shard, index *DatabaseIndex, measurementFields map[string]*MeasurementFields) error Begin(writable bool) (Tx, error) WritePoints(points []models.Point, measurementFieldsToSave map[string]*MeasurementFields, seriesToCreate []*SeriesCreate) error @@ -60,6 +60,21 @@ func NewEngine(path string, walPath string, options EngineOptions) (Engine, erro // Only bolt-based backends are currently supported so open it and check the format. var format string if err := func() error { + // if it's a dir then it's a pd1 engine + f, err := os.Open(path) + if err != nil { + return err + } + fi, err := f.Stat() + f.Close() + if err != nil { + return err + } + if fi.Mode().IsDir() { + format = "pd1" + return nil + } + db, err := bolt.Open(path, 0666, &bolt.Options{Timeout: 1 * time.Second}) if err != nil { return err diff --git a/tsdb/engine/b1/b1.go b/tsdb/engine/b1/b1.go index fdc337b2f7a..356b327fe95 100644 --- a/tsdb/engine/b1/b1.go +++ b/tsdb/engine/b1/b1.go @@ -174,7 +174,7 @@ func (e *Engine) close() error { func (e *Engine) SetLogOutput(w io.Writer) { e.LogOutput = w } // LoadMetadataIndex loads the shard metadata into memory. -func (e *Engine) LoadMetadataIndex(index *tsdb.DatabaseIndex, measurementFields map[string]*tsdb.MeasurementFields) error { +func (e *Engine) LoadMetadataIndex(shard *tsdb.Shard, index *tsdb.DatabaseIndex, measurementFields map[string]*tsdb.MeasurementFields) error { return e.db.View(func(tx *bolt.Tx) error { // load measurement metadata meta := tx.Bucket([]byte("fields")) diff --git a/tsdb/engine/bz1/bz1.go b/tsdb/engine/bz1/bz1.go index e4d5682ca47..aa38600b806 100644 --- a/tsdb/engine/bz1/bz1.go +++ b/tsdb/engine/bz1/bz1.go @@ -176,7 +176,7 @@ func (e *Engine) close() error { func (e *Engine) SetLogOutput(w io.Writer) {} // LoadMetadataIndex loads the shard metadata into memory. -func (e *Engine) LoadMetadataIndex(index *tsdb.DatabaseIndex, measurementFields map[string]*tsdb.MeasurementFields) error { +func (e *Engine) LoadMetadataIndex(shard *tsdb.Shard, index *tsdb.DatabaseIndex, measurementFields map[string]*tsdb.MeasurementFields) error { if err := e.db.View(func(tx *bolt.Tx) error { // Load measurement metadata fields, err := e.readFields(tx) diff --git a/tsdb/engine/bz1/bz1_test.go b/tsdb/engine/bz1/bz1_test.go index 97873afe3b2..0b0cb1e60cc 100644 --- a/tsdb/engine/bz1/bz1_test.go +++ b/tsdb/engine/bz1/bz1_test.go @@ -38,7 +38,7 @@ func TestEngine_LoadMetadataIndex_Series(t *testing.T) { // Load metadata index. index := tsdb.NewDatabaseIndex() - if err := e.LoadMetadataIndex(index, make(map[string]*tsdb.MeasurementFields)); err != nil { + if err := e.LoadMetadataIndex(nil, index, make(map[string]*tsdb.MeasurementFields)); err != nil { t.Fatal(err) } @@ -80,7 +80,7 @@ func TestEngine_LoadMetadataIndex_Fields(t *testing.T) { // Load metadata index. mfs := make(map[string]*tsdb.MeasurementFields) - if err := e.LoadMetadataIndex(tsdb.NewDatabaseIndex(), mfs); err != nil { + if err := e.LoadMetadataIndex(nil, tsdb.NewDatabaseIndex(), mfs); err != nil { t.Fatal(err) } diff --git a/tsdb/engine/engine.go b/tsdb/engine/engine.go index c5565ff06cf..03022f9d0e7 100644 --- a/tsdb/engine/engine.go +++ b/tsdb/engine/engine.go @@ -3,4 +3,5 @@ package engine import ( _ "github.com/influxdb/influxdb/tsdb/engine/b1" _ "github.com/influxdb/influxdb/tsdb/engine/bz1" + _ "github.com/influxdb/influxdb/tsdb/engine/pd1" ) diff --git a/tsdb/engine/pd1/encoding.go b/tsdb/engine/pd1/encoding.go index b53f4f25635..285c83f8c0b 100644 --- a/tsdb/engine/pd1/encoding.go +++ b/tsdb/engine/pd1/encoding.go @@ -2,25 +2,43 @@ package pd1 import ( "time" + + "github.com/dgryski/go-tsz" ) type FloatValue struct { - Time int64 + Time time.Time Value float64 } -// First 8 bytes should be the timestamp, second 8 bytes should be -// the first float value +type FloatValues []FloatValue + +func (a FloatValues) Len() int { return len(a) } +func (a FloatValues) Swap(i, j int) { a[i], a[j] = a[j], a[i] } +func (a FloatValues) Less(i, j int) bool { return a[i].Time.UnixNano() < a[j].Time.UnixNano() } + +// TODO: make this work with nanosecond timestamps func EncodeFloatBlock(buf []byte, values []FloatValue) []byte { - return nil + s := tsz.New(uint32(values[0].Time.Unix())) + for _, v := range values { + s.Push(uint32(v.Time.Unix()), v.Value) + } + s.Finish() + return s.Bytes() } func DecodeFloatBlock(block []byte) ([]FloatValue, error) { - return nil, nil + iter, _ := tsz.NewIterator(block) + a := make([]FloatValue, 0) + for iter.Next() { + t, f := iter.Values() + a = append(a, FloatValue{time.Unix(int64(t), 0), f}) + } + return a, nil } type BoolValue struct { - Time int64 + Time time.Time Value bool } @@ -33,7 +51,7 @@ func DecodeBoolBlock(block []byte) ([]BoolValue, error) { } type Int64Value struct { - Time int64 + Time time.Time Value int64 } @@ -46,10 +64,10 @@ func DecodeInt64Block(block []byte) ([]Int64Value, error) { } type StringValue struct { - Time int64 + Time time.Time Value string } -func EncodeStringBlock(values []StringValue) []byte { +func EncodeStringBlock(buf []byte, values []StringValue) []byte { return nil } diff --git a/tsdb/engine/pd1/encoding_test.go b/tsdb/engine/pd1/encoding_test.go index 077053449a0..26bb4c2e076 100644 --- a/tsdb/engine/pd1/encoding_test.go +++ b/tsdb/engine/pd1/encoding_test.go @@ -23,16 +23,17 @@ func TestEncoding_FloatBlock(t *testing.T) { if err != nil { t.Fatalf("error decoding: %s", err.Error) } + if !reflect.DeepEqual(decodedValues, values) { - t.Fatalf("values not equal:\n\tgot: %s\n\texp: %s", values, decodedValues) + t.Fatalf("unexpected results:\n\tgot: %v\n\texp: %v\n", decodedValues, values) } } -func getTimes(n, step int, precision time.Duration) []int64 { +func getTimes(n, step int, precision time.Duration) []time.Time { t := time.Now().Round(precision) - a := make([]int64, n) + a := make([]time.Time, n) for i := 0; i < n; i++ { - a[i] = t.Add(60 * precision).UnixNano() + a[i] = t.Add(60 * precision) } return a } diff --git a/tsdb/engine/pd1/pd1.go b/tsdb/engine/pd1/pd1.go new file mode 100644 index 00000000000..d94a7d7795c --- /dev/null +++ b/tsdb/engine/pd1/pd1.go @@ -0,0 +1,1255 @@ +package pd1 + +import ( + "bytes" + "encoding/binary" + "encoding/json" + "fmt" + "hash/fnv" + "io" + "io/ioutil" + "math" + "os" + "path/filepath" + "sort" + "strconv" + "strings" + "sync" + "syscall" + "time" + + "github.com/golang/snappy" + "github.com/influxdb/influxdb/influxql" + "github.com/influxdb/influxdb/tsdb" +) + +const ( + // Format is the file format name of this engine. + Format = "pd1" + + // FieldsFileExtension is the extension for the file that stores compressed field + // encoding data for this db + FieldsFileExtension = "fld" + + // SeriesFileExtension is the extension for the file that stores the compressed + // series metadata for series in this db + SeriesFileExtension = "srs" + + CollisionsFileExtension = "col" +) + +type TimePrecision uint8 + +const ( + Seconds TimePrecision = iota + Milliseconds + Microseconds + Nanoseconds +) + +func init() { + tsdb.RegisterEngine(Format, NewEngine) +} + +const ( + // DefaultBlockSize is the default size of uncompressed points blocks. + DefaultBlockSize = 512 * 1024 // 512KB + + DefaultMaxFileSize = 50 * 1024 * 1024 // 50MB + + DefaultMaxPointsPerBlock = 5000 + + // MAP_POPULATE is for the mmap syscall. For some reason this isn't defined in golang's syscall + MAP_POPULATE = 0x8000 +) + +// Ensure Engine implements the interface. +var _ tsdb.Engine = &Engine{} + +// Engine represents a storage engine with compressed blocks. +type Engine struct { + mu sync.Mutex + path string + + shard *tsdb.Shard + + // HashSeriesField is a function that takes a series key and a field name + // and returns a hash identifier. It's not guaranteed to be unique. + HashSeriesField func(key string) uint64 + + filesLock sync.RWMutex + files dataFiles + currentFileID int + queryLock sync.RWMutex +} + +// NewEngine returns a new instance of Engine. +func NewEngine(path string, walPath string, opt tsdb.EngineOptions) tsdb.Engine { + e := &Engine{ + path: path, + + HashSeriesField: hashSeriesField, + } + + return e +} + +// Path returns the path the engine was opened with. +func (e *Engine) Path() string { return e.path } + +// Open opens and initializes the engine. +func (e *Engine) Open() error { + if err := os.MkdirAll(e.path, 0777); err != nil { + return err + } + + // TODO: clean up previous series write + // TODO: clean up previous fields write + // TODO: clean up previous names write + // TODO: clean up any data files that didn't get cleaned up + + return nil +} + +// Close closes the engine. +func (e *Engine) Close() error { + return nil +} + +// SetLogOutput is a no-op. +func (e *Engine) SetLogOutput(w io.Writer) {} + +// LoadMetadataIndex loads the shard metadata into memory. +func (e *Engine) LoadMetadataIndex(shard *tsdb.Shard, index *tsdb.DatabaseIndex, measurementFields map[string]*tsdb.MeasurementFields) error { + e.shard = shard + // TODO: write the metadata from the WAL + + // Load measurement metadata + fields, err := e.readFields() + if err != nil { + return err + } + for k, mf := range fields { + m := index.CreateMeasurementIndexIfNotExists(string(k)) + for name, _ := range mf.Fields { + m.SetFieldName(name) + } + mf.Codec = tsdb.NewFieldCodec(mf.Fields) + measurementFields[m.Name] = mf + } + + // Load series metadata + series, err := e.readSeries() + if err != nil { + return err + } + + // Load the series into the in-memory index in sorted order to ensure + // it's always consistent for testing purposes + a := make([]string, 0, len(series)) + for k, _ := range series { + a = append(a, k) + } + sort.Strings(a) + for _, key := range a { + s := series[key] + s.InitializeShards() + index.CreateSeriesIndexIfNotExists(tsdb.MeasurementFromSeriesKey(string(key)), s) + } + + return nil +} + +// WritePoints writes metadata and point data into the engine. +// Returns an error if new points are added to an existing key. +func (e *Engine) WritePoints(points []tsdb.Point, measurementFieldsToSave map[string]*tsdb.MeasurementFields, seriesToCreate []*tsdb.SeriesCreate) error { + // TODO: Write points to the WAL + + return nil +} + +func (e *Engine) WriteAndCompact(points []tsdb.Point, measurementFieldsToSave map[string]*tsdb.MeasurementFields, seriesToCreate []*tsdb.SeriesCreate) error { + e.mu.Lock() + defer e.mu.Unlock() + + if err := e.writeNewFields(measurementFieldsToSave); err != nil { + return err + } + if err := e.writeNewSeries(seriesToCreate); err != nil { + return err + } + + if len(points) == 0 { + return nil + } + + b, err := e.readCompressedFile("names") + if err != nil { + return err + } + ids := make(map[uint64]string) + + var names []string + if b != nil { + if err := json.Unmarshal(b, &names); err != nil { + return err + } + + for _, n := range names { + ids[e.HashSeriesField(n)] = n + } + } + + fmt.Println("read names: ", len(names), len(ids)) + + // these are values that are newer than anything stored in the shard + valuesByID := make(map[uint64]*valueCollection) + // map the points to the data file they belong to if they overlap + dataFileToValues := make(map[*dataFile]map[uint64]*valueCollection) + + // we keep track of the newest data file and if it should be + // rewritten with new data. + var newestDataFile *dataFile + overwriteNewestFile := false + if len(e.files) > 0 { + newestDataFile = e.files[len(e.files)-1] + overwriteNewestFile = newestDataFile.size < DefaultMaxFileSize + } + + // compute ids of new keys and arrange for insertion + for _, p := range points { + for fn, val := range p.Fields() { + n := seriesFieldKey(string(p.Key()), fn) + id := e.HashSeriesField(n) + if series, ok := ids[id]; !ok { + names = append(names, n) + } else { // possible collision? + if n != series { + // TODO: implement collision detection + panic("name collision!") + } + } + + ids[id] = n + + vals := valuesByID[id] + if vals == nil { + // TODO: deal with situation where there are already files, + // but the user is inserting a bunch of data that predates + // any of them. It's ok to rewrite the first file, but + // only to max size. Then we should create a new one + + // points always come in time increasing order. This is + // the first point we've seen for this key. So it might + // need to get put into an older file instead of a new + // one. Check and set accordingly + var df *dataFile + for i := len(e.files) - 1; i >= 0; i-- { + if p.UnixNano() > e.files[i].MaxTime() { + break + } + df = e.files[i] + } + vals = &valueCollection{} + + if df == nil || (df == newestDataFile && overwriteNewestFile) { + // this point is newer than anything we have stored + // or it belongs in the most recent file, which should get + // rewritten + valuesByID[id] = vals + } else { + // it overlaps with another file so mark it and it can be compacted + dfm := dataFileToValues[df] + if dfm == nil { + dfm = make(map[uint64]*valueCollection) + dataFileToValues[df] = dfm + } + + if vc := dfm[id]; vc == nil { + dfm[id] = vals + } else { + vals = vc + } + } + } + + switch t := val.(type) { + case float64: + vals.floatValues = append(vals.floatValues, FloatValue{Time: p.Time(), Value: t}) + case int64: + vals.intValues = append(vals.intValues, Int64Value{Time: p.Time(), Value: t}) + case bool: + vals.boolValues = append(vals.boolValues, BoolValue{Time: p.Time(), Value: t}) + case string: + vals.stringValues = append(vals.stringValues, StringValue{Time: p.Time(), Value: t}) + default: + panic("unsupported type") + } + } + } + + fmt.Println("writing names:", len(names)) + b, err = json.Marshal(names) + if err != nil { + return err + } + + if err := e.replaceCompressedFile("names", b); err != nil { + return err + } + + // flush values by id to either a new file or rewrite the old one + if overwriteNewestFile { + if err := e.rewriteFile(newestDataFile, valuesByID); err != nil { + return err + } else if err := e.rewriteFile(nil, valuesByID); err != nil { + return err + } + } + + // flush each of the old ones + for df, vals := range dataFileToValues { + fmt.Println("writing vals to old file: ", df.f.Name()) + if err := e.rewriteFile(df, vals); err != nil { + return err + } + } + + return nil +} + +func (e *Engine) rewriteFile(oldDF *dataFile, values map[uint64]*valueCollection) error { + // we need the values in sorted order so that we can merge them into the + // new file as we read the old file + ids := make([]uint64, 0, len(values)) + for id, _ := range values { + ids = append(ids, id) + } + + minTime := int64(math.MaxInt64) + maxTime := int64(math.MinInt64) + + // read header of ids to starting positions and times + oldIDToPosition := make(map[uint64]uint32) + if oldDF != nil { + oldIDToPosition = oldDF.IDToPosition() + minTime = oldDF.MinTime() + maxTime = oldDF.MaxTime() + } + for _, v := range values { + if minTime > v.MinTime().UnixNano() { + minTime = v.MinTime().UnixNano() + } + if maxTime < v.MaxTime().UnixNano() { + maxTime = v.MaxTime().UnixNano() + } + } + + // add any ids that are in the file that aren't getting flushed here + for id, _ := range oldIDToPosition { + if _, ok := values[id]; !ok { + ids = append(ids, id) + } + } + + // always write in order by ID + sort.Sort(uint64slice(ids)) + + // TODO: add checkpoint file that indicates if this completed or not + f, err := os.OpenFile(e.nextFileName(), os.O_CREATE|os.O_RDWR, 0666) + if err != nil { + return err + } + + // write the header of the file and keep track of the current file position + currentPosition := uint32(4) + // series count + if _, err := f.Write(u32tob(uint32(len(ids)))); err != nil { + f.Close() + return err + } + // empty min time and max time + currentPosition += 16 + if _, err := f.Write([]byte{0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00}); err != nil { + f.Close() + return nil + } + // write the series ids and empty starting positions + for _, id := range ids { + if _, err := f.Write(append(u64tob(id), []byte{0x00, 0x00, 0x00, 0x00}...)); err != nil { + f.Close() + return err + } + currentPosition += 12 + } + + // now combine the old file data with the new values, keeping track of + // their positions + newPositions := make([]uint32, len(ids)) + buf := make([]byte, DefaultMaxPointsPerBlock*20) + for i, id := range ids { + // mark the position for this ID + newPositions[i] = currentPosition + + newVals := values[id] + + // if this id is only in the file and not in the new values, just copy over from old file + if newVals == nil { + fpos := oldIDToPosition[id] + + // write the blocks until we hit whatever the next id is + for { + fid := btou64(oldDF.mmap[fpos : fpos+8]) + if fid != id { + break + } + length := btou32(oldDF.mmap[fpos+8 : fpos+12]) + if _, err := f.Write(oldDF.mmap[fpos : fpos+12+length]); err != nil { + f.Close() + return err + } + fpos += (12 + length) + currentPosition += (12 + length) + + // make sure we're not at the end of the file + if fpos >= oldDF.size { + break + } + } + + continue + } + + // if the values are not in the file, just write the new ones + fpos, ok := oldIDToPosition[id] + if !ok { + block := newVals.Encode(buf) + if _, err := f.Write(append(u64tob(id), u32tob(uint32(len(block)))...)); err != nil { + f.Close() + return err + } + if _, err := f.Write(block); err != nil { + f.Close() + return err + } + currentPosition += uint32(12 + len(block)) + + continue + } + + // it's in the file and the new values, combine them and write out + for { + fid := btou64(oldDF.mmap[fpos : fpos+8]) + if fid != id { + break + } + length := btou32(oldDF.mmap[fpos+8 : fpos+12]) + block := oldDF.mmap[fpos : fpos+12+length] + fpos += (12 + length) + + // determine if there's a block after this with the same id and get its time + hasFutureBlock := false + nextTime := int64(0) + if fpos < oldDF.size { + nextID := btou64(oldDF.mmap[fpos : fpos+8]) + if nextID == id { + hasFutureBlock = true + nextTime = int64(btou64(oldDF.mmap[fpos+12 : fpos+20])) + } + } + + newBlock, err := newVals.DecodeAndCombine(block, buf[:0], nextTime, hasFutureBlock) + if err != nil { + return err + } + if _, err := f.Write(append(u64tob(id), u32tob(uint32(len(newBlock)))...)); err != nil { + f.Close() + return err + } + if _, err := f.Write(newBlock); err != nil { + f.Close() + return err + } + + currentPosition += uint32(12 + len(newBlock)) + + if fpos >= oldDF.size { + break + } + } + } + + // write out the times and positions + if _, err := f.Seek(4, 0); err != nil { + f.Close() + return err + } + if _, err := f.Write(append(u64tob(uint64(minTime)), u64tob(uint64(maxTime))...)); err != nil { + f.Close() + return err + } + for _, pos := range newPositions { + if _, err := f.Seek(8, 1); err != nil { + f.Close() + return err + } + + if _, err := f.Write(u32tob(pos)); err != nil { + return err + } + } + if err := f.Sync(); err != nil { + return err + } + if _, err := f.Seek(0, 0); err != nil { + f.Close() + return err + } + + // now open it as a memory mapped data file + newDF, err := NewDataFile(f) + if err != nil { + return err + } + + // update the engine to point at the new dataFiles + e.queryLock.Lock() + var files dataFiles + for _, df := range e.files { + if df != oldDF { + files = append(files, df) + } + } + files = append(files, newDF) + sort.Sort(files) + e.files = files + e.queryLock.Unlock() + + // remove the old data file. no need to block returning the write, + // but we need to let any running queries finish before deleting it + if oldDF != nil { + go func(df *dataFile) { + if err := oldDF.Delete(); err != nil { + // TODO: log this error + } + }(oldDF) + } + + return nil +} + +func (e *Engine) nextFileName() string { + e.currentFileID++ + return filepath.Join(e.path, fmt.Sprintf("%07d.%s", e.currentFileID, Format)) +} + +func (e *Engine) readCompressedFile(name string) ([]byte, error) { + f, err := os.OpenFile(filepath.Join(e.path, name), os.O_RDONLY, 0666) + if os.IsNotExist(err) { + return nil, nil + } else if err != nil { + return nil, err + } + b, err := ioutil.ReadAll(f) + if err != nil { + return nil, err + } + + data, err := snappy.Decode(nil, b) + if err != nil { + return nil, err + } + return data, nil +} + +func (e *Engine) replaceCompressedFile(name string, data []byte) error { + tmpName := filepath.Join(e.path, name+"tmp") + f, err := os.OpenFile(tmpName, os.O_CREATE|os.O_RDWR, 0666) + if err != nil { + return err + } + b := snappy.Encode(nil, data) + if _, err := f.Write(b); err != nil { + return err + } + fmt.Println("compressed: ", len(b)) + if err := f.Close(); err != nil { + return err + } + if err := os.Remove(name); err != nil && !os.IsNotExist(err) { + return err + } + return os.Rename(tmpName, filepath.Join(e.path, name)) +} + +// DeleteSeries deletes the series from the engine. +func (e *Engine) DeleteSeries(keys []string) error { + return nil +} + +// DeleteMeasurement deletes a measurement and all related series. +func (e *Engine) DeleteMeasurement(name string, seriesKeys []string) error { + return nil +} + +// SeriesCount returns the number of series buckets on the shard. +func (e *Engine) SeriesCount() (n int, err error) { + return 0, nil +} + +// Begin starts a new transaction on the engine. +func (e *Engine) Begin(writable bool) (tsdb.Tx, error) { + return e, nil +} + +// TODO: make the cursor take a field name +func (e *Engine) Cursor(series string, direction tsdb.Direction) tsdb.Cursor { + measurementName := tsdb.MeasurementFromSeriesKey(series) + codec := e.shard.FieldCodec(measurementName) + if codec == nil { + return &cursor{} + } + field := codec.FieldByName("value") + if field == nil { + panic("pd1 engine only supports one field with name of value") + } + + // TODO: ensure we map the collisions + id := hashSeriesField(seriesFieldKey(series, field.Name)) + return newCursor(id, field.Type, e.copyFilesCollection(), direction) +} + +func (e *Engine) copyFilesCollection() []*dataFile { + e.filesLock.RLock() + defer e.filesLock.RUnlock() + a := make([]*dataFile, len(e.files)) + copy(a, e.files) + return a +} + +func (e *Engine) Size() int64 { return 0 } +func (e *Engine) Commit() error { return nil } +func (e *Engine) Rollback() error { return nil } +func (e *Engine) WriteTo(w io.Writer) (n int64, err error) { return 0, nil } + +func (e *Engine) writeNewFields(measurementFieldsToSave map[string]*tsdb.MeasurementFields) error { + if len(measurementFieldsToSave) == 0 { + return nil + } + + // read in all the previously saved fields + fields, err := e.readFields() + if err != nil { + return err + } + + // add the new ones or overwrite old ones + for name, mf := range measurementFieldsToSave { + fields[name] = mf + } + + return e.writeFields(fields) +} + +func (e *Engine) writeFields(fields map[string]*tsdb.MeasurementFields) error { + // compress and save everything + data, err := json.Marshal(fields) + if err != nil { + return err + } + + fn := e.path + "." + FieldsFileExtension + "tmp" + ff, err := os.OpenFile(fn, os.O_CREATE|os.O_RDWR, 0666) + if err != nil { + return err + } + _, err = ff.Write(snappy.Encode(nil, data)) + if err != nil { + return err + } + if err := ff.Close(); err != nil { + return err + } + fieldsFileName := e.path + "." + FieldsFileExtension + + if _, err := os.Stat(fieldsFileName); !os.IsNotExist(err) { + if err := os.Remove(fieldsFileName); err != nil { + return err + } + } + + return os.Rename(fn, fieldsFileName) +} + +func (e *Engine) readFields() (map[string]*tsdb.MeasurementFields, error) { + fields := make(map[string]*tsdb.MeasurementFields) + + f, err := os.OpenFile(e.path+"."+FieldsFileExtension, os.O_RDONLY, 0666) + if os.IsNotExist(err) { + return fields, nil + } else if err != nil { + return nil, err + } + b, err := ioutil.ReadAll(f) + if err != nil { + return nil, err + } + + data, err := snappy.Decode(nil, b) + if err != nil { + return nil, err + } + + if err := json.Unmarshal(data, &fields); err != nil { + return nil, err + } + + return fields, nil +} + +func (e *Engine) writeNewSeries(seriesToCreate []*tsdb.SeriesCreate) error { + if len(seriesToCreate) == 0 { + return nil + } + + // read in previously saved series + series, err := e.readSeries() + if err != nil { + return err + } + + // add new ones, compress and save + for _, s := range seriesToCreate { + series[s.Series.Key] = s.Series + } + + return e.writeSeries(series) +} + +func (e *Engine) writeSeries(series map[string]*tsdb.Series) error { + data, err := json.Marshal(series) + if err != nil { + return err + } + + fn := e.path + "." + SeriesFileExtension + "tmp" + ff, err := os.OpenFile(fn, os.O_CREATE|os.O_RDWR, 0666) + if err != nil { + return err + } + _, err = ff.Write(snappy.Encode(nil, data)) + if err != nil { + return err + } + if err := ff.Close(); err != nil { + return err + } + seriesFileName := e.path + "." + SeriesFileExtension + + if _, err := os.Stat(seriesFileName); !os.IsNotExist(err) { + if err := os.Remove(seriesFileName); err != nil && err != os.ErrNotExist { + return err + } + } + + return os.Rename(fn, seriesFileName) +} + +func (e *Engine) readSeries() (map[string]*tsdb.Series, error) { + series := make(map[string]*tsdb.Series) + + f, err := os.OpenFile(e.path+"."+SeriesFileExtension, os.O_RDONLY, 0666) + if os.IsNotExist(err) { + return series, nil + } else if err != nil { + return nil, err + } + defer f.Close() + b, err := ioutil.ReadAll(f) + if err != nil { + return nil, err + } + + data, err := snappy.Decode(nil, b) + if err != nil { + return nil, err + } + + if err := json.Unmarshal(data, &series); err != nil { + return nil, err + } + + return series, nil +} + +type valueCollection struct { + floatValues []FloatValue + boolValues []BoolValue + intValues []Int64Value + stringValues []StringValue +} + +func (v *valueCollection) MinTime() time.Time { + if v.floatValues != nil { + return v.floatValues[0].Time + } else if v.boolValues != nil { + return v.boolValues[0].Time + } else if v.intValues != nil { + return v.intValues[0].Time + } else if v.stringValues != nil { + return v.stringValues[0].Time + } + + return time.Unix(0, 0) +} + +func (v *valueCollection) MaxTime() time.Time { + if v.floatValues != nil { + return v.floatValues[len(v.floatValues)-1].Time + } else if v.boolValues != nil { + return v.boolValues[len(v.boolValues)-1].Time + } else if v.intValues != nil { + return v.intValues[len(v.intValues)-1].Time + } else if v.stringValues != nil { + return v.stringValues[len(v.stringValues)-1].Time + } + + return time.Unix(0, 0) +} + +func (v *valueCollection) Encode(buf []byte) []byte { + if v.floatValues != nil { + return EncodeFloatBlock(buf, v.floatValues) + } else if v.boolValues != nil { + return EncodeBoolBlock(buf, v.boolValues) + } else if v.intValues != nil { + return EncodeInt64Block(buf, v.intValues) + } else if v.stringValues != nil { + return EncodeStringBlock(buf, v.stringValues) + } + + return nil +} + +// DecodeAndCombine take an encoded block from a file, decodes it and interleaves the file +// values with the values in this collection. nextTime and hasNext refer to if the file +// has future encoded blocks so that this method can know how much of its values can be +// combined and output in the resulting encoded block. +func (v *valueCollection) DecodeAndCombine(block, buf []byte, nextTime int64, hasFutureBlock bool) ([]byte, error) { + if v.floatValues != nil { + values, err := DecodeFloatBlock(block) + if err != nil { + return nil, err + } + + if hasFutureBlock { + for i, val := range v.floatValues { + if val.Time.UnixNano() > nextTime { + values = append(values, v.floatValues[:i]...) + v.floatValues = v.floatValues[i:] + } + } + } else { + values = append(values, v.floatValues...) + } + sort.Sort(FloatValues(values)) + // TODO: deduplicate values + + if len(values) > DefaultMaxPointsPerBlock { + v.floatValues = values[DefaultMaxPointsPerBlock:] + values = values[:DefaultMaxPointsPerBlock] + } + + return EncodeFloatBlock(buf, values), nil + } else if v.boolValues != nil { + // TODO: wire up the other value types + return nil, fmt.Errorf("not implemented") + } else if v.intValues != nil { + return nil, fmt.Errorf("not implemented") + } else if v.stringValues != nil { + return nil, fmt.Errorf("not implemented") + } + + return nil, nil +} + +type dataFile struct { + f *os.File + mu sync.RWMutex + size uint32 + mmap []byte +} + +// byte size constants for the data file +const ( + seriesCountSize = 4 + timeSize = 8 + fileHeaderSize = seriesCountSize + (2 * timeSize) + seriesIDSize = 8 + seriesPositionSize = 4 + seriesHeaderSize = seriesIDSize + seriesPositionSize +) + +func NewDataFile(f *os.File) (*dataFile, error) { + fInfo, err := f.Stat() + if err != nil { + return nil, err + } + mmap, err := syscall.Mmap(int(f.Fd()), 0, int(fInfo.Size()), syscall.PROT_READ, syscall.MAP_SHARED|MAP_POPULATE) + if err != nil { + f.Close() + return nil, err + } + + return &dataFile{ + f: f, + mmap: mmap, + size: uint32(fInfo.Size()), + }, nil +} + +func (d *dataFile) Close() error { + d.mu.Lock() + defer d.mu.Unlock() + return d.close() +} + +func (d *dataFile) Delete() error { + d.mu.Lock() + defer d.mu.Unlock() + if err := d.close(); err != nil { + return err + } + return os.Remove(d.f.Name()) +} + +func (d *dataFile) close() error { + if d.mmap == nil { + return nil + } + err := syscall.Munmap(d.mmap) + if err != nil { + return err + } + + d.mmap = nil + return d.f.Close() +} + +func (d *dataFile) MinTime() int64 { + return int64(btou64(d.mmap[4:12])) +} + +func (d *dataFile) MaxTime() int64 { + return int64(btou64(d.mmap[12:20])) +} + +func (d *dataFile) SeriesCount() uint32 { + return btou32(d.mmap[:4]) +} + +func (d *dataFile) IDToPosition() map[uint64]uint32 { + count := int(d.SeriesCount()) + m := make(map[uint64]uint32) + + for i := 0; i < count; i++ { + offset := 20 + (i * 12) + id := btou64(d.mmap[offset : offset+8]) + m[id] = btou32(d.mmap[offset+8 : offset+12]) + } + + return m +} + +// StartingPositionForID returns the position in the file of the +// first block for the given ID. If zero is returned the ID doesn't +// have any data in this file. +func (d *dataFile) StartingPositionForID(id uint64) uint32 { + seriesCount := d.SeriesCount() + + min := 0 + max := seriesCount + // // set the minimum position to the first after the file header + // posMin := fileHeaderSize + + // // set the maximum position to the end of the series header + // posMax := fileHeaderSize + (seriesCount * seriesHeaderSize) + + for min < max { + mid := (max-min)/2 + min + + offset := mid*seriesHeaderSize + fileHeaderSize + checkID := btou64(d.mmap[offset:8]) + + if checkID == id { + return btou32(d.mmap[offset+8 : offset+12]) + } else if checkID < id { + min = mid + 1 + } + max = mid + } + return uint32(0) +} + +type dataFiles []*dataFile + +func (a dataFiles) Len() int { return len(a) } +func (a dataFiles) Swap(i, j int) { a[i], a[j] = a[j], a[i] } +func (a dataFiles) Less(i, j int) bool { return a[i].MinTime() < a[j].MinTime() } + +type cursor struct { + id uint64 + dataType influxql.DataType + f *dataFile + dataFilePos int + pos uint32 + vals []FloatValues + + direction tsdb.Direction + + // time acending list of data files + files []*dataFile +} + +func newCursor(id uint64, dataType influxql.DataType, files []*dataFile, direction tsdb.Direction) *cursor { + return &cursor{ + ids: id, + types: dataType, + direction: direction, + files: files, + } +} + +func (c *cursor) Seek(seek []byte) (key, value []byte) { return nil, nil } + +func (c *cursor) Next() (key, value []byte) { + if vals == nil { + // loop until we find a file with some data + for dataFilePos < len(c.files) { + f = c.files[c.dataFilePos] + c.dataFilePos++ + + // startPosition := f + } + } + return nil, nil +} + +func (c *cursor) next(id uint64) (key, value []byte) + +func (c *cursor) Direction() tsdb.Direction { return c.direction } + +// u64tob converts a uint64 into an 8-byte slice. +func u64tob(v uint64) []byte { + b := make([]byte, 8) + binary.BigEndian.PutUint64(b, v) + return b +} + +func btou64(b []byte) uint64 { + return binary.BigEndian.Uint64(b) +} + +func u32tob(v uint32) []byte { + b := make([]byte, 4) + binary.BigEndian.PutUint32(b, v) + return b +} + +func btou32(b []byte) uint32 { + return uint32(binary.BigEndian.Uint32(b)) +} + +func hashSeriesField(key string) uint64 { + h := fnv.New64a() + h.Write([]byte(key)) + return h.Sum64() +} + +// seriesFieldKey combine a series key and field name for a unique string to be hashed to a numeric ID +func seriesFieldKey(seriesKey, field string) string { + return seriesKey + "#" + field +} + +type uint64slice []uint64 + +func (a uint64slice) Len() int { return len(a) } +func (a uint64slice) Swap(i, j int) { a[i], a[j] = a[j], a[i] } +func (a uint64slice) Less(i, j int) bool { return a[i] < a[j] } + +/* TODO: REMOVE THIS STUFF */ +func (e *Engine) pointsToBlocks(points [][]byte) []byte { + var b bytes.Buffer + block := make([]byte, 0) + for _, p := range points { + block = append(block, p[0:8]...) + block = append(block, u32tob(uint32(len(p)-8))...) + block = append(block, p[8:]...) + if len(block) > DefaultBlockSize { + e.writeBlockToBuffer(block, &b) + block = make([]byte, 0) + } + } + if len(block) > 0 { + e.writeBlockToBuffer(block, &b) + } + + return b.Bytes() +} + +func (e *Engine) writeBlockToBuffer(block []byte, b *bytes.Buffer) { + // write the min time + if _, err := b.Write(block[0:8]); err != nil { + panic(err) + } + + // write the length of the compressed data + data := snappy.Encode(nil, block) + if _, err := b.Write(u32tob(uint32(len(data)))); err != nil { + panic(err) + } + + // write the compressed data + if _, err := b.Write(data); err != nil { + panic(err) + } +} + +func (e *Engine) readPointsFromFile(f *os.File) (map[uint64][][]byte, error) { + buf := make([]byte, 8) + if _, err := io.ReadFull(f, buf); err != nil { + return nil, err + } + seriesCount := btou64(buf) + positions := make([]uint64, seriesCount, seriesCount) + ids := make([]uint64, seriesCount, seriesCount) + + // read the series index file header + position := uint64(8) + for i := 0; uint64(i) < seriesCount; i++ { + // read the id of the series + if _, err := io.ReadFull(f, buf); err != nil { + return nil, err + } + ids[i] = btou64(buf) + + // read the min time and ignore + if _, err := io.ReadFull(f, buf); err != nil { + return nil, err + } + if _, err := io.ReadFull(f, buf); err != nil { + return nil, err + } + + // read the starting position of this id + if _, err := io.ReadFull(f, buf); err != nil { + return nil, err + } + positions[i] = btou64(buf) + position += 32 + } + + if position != positions[0] { + panic("we aren't at the right place") + } + + // read the raw data + seriesData := make(map[uint64][][]byte) + compressedBuff := make([]byte, DefaultBlockSize) + seriesPosition := 0 + for { + // read the min time and ignore + if _, err := io.ReadFull(f, buf); err == io.EOF { + break + } else if err != nil { + return nil, err + } + + // read the length of the compressed block + if _, err := io.ReadFull(f, buf[:4]); err != nil { + return nil, err + } + length := btou32(buf) + + if length > uint32(len(compressedBuff)) { + compressedBuff = make([]byte, length) + } + if _, err := io.ReadFull(f, compressedBuff[:length]); err != nil { + return nil, err + } + + data, err := snappy.Decode(nil, compressedBuff[:length]) + if err != nil { + return nil, err + } + id := ids[seriesPosition] + seriesData[id] = append(seriesData[id], e.pointsFromDataBlock(data)...) + position += uint64(12 + length) + + if seriesPosition+1 >= len(positions) { + continue + } + if positions[seriesPosition+1] == position { + seriesPosition += 1 + } + } + + return seriesData, nil +} + +func (e *Engine) pointsFromDataBlock(data []byte) [][]byte { + a := make([][]byte, 0) + for { + length := entryDataSize(data) + p := append(data[:8], data[12:12+length]...) + a = append(a, p) + data = data[12+length:] + if len(data) == 0 { + break + } + } + return a +} + +func entryDataSize(v []byte) int { return int(binary.BigEndian.Uint32(v[8:12])) } + +func (e *Engine) lastFileAndNewFile() (*os.File, *os.File, error) { + files, err := filepath.Glob(filepath.Join(e.path, fmt.Sprintf("*.%s", Format))) + if err != nil { + return nil, nil, err + } + + if len(files) == 0 { + newFile, err := os.OpenFile(filepath.Join(e.path, fmt.Sprintf("%07d.%s", 1, Format)), os.O_CREATE|os.O_RDWR, 0666) + if err != nil { + return nil, nil, err + } + return nil, newFile, nil + } + + oldFile, err := os.OpenFile(files[len(files)-1], os.O_RDONLY, 0666) + if err != nil { + return nil, nil, err + } + + info, err := oldFile.Stat() + if err != nil { + _ = oldFile.Close() + return nil, nil, err + } + + num := strings.Split(filepath.Base(files[len(files)-1]), ".")[0] + n, err := strconv.ParseUint(num, 10, 32) + if err != nil { + return nil, nil, err + } + newFile, err := os.OpenFile(filepath.Join(e.path, fmt.Sprintf("%07d.%s", n+1, Format)), os.O_CREATE|os.O_RDWR, 0666) + if err != nil { + return nil, nil, err + } + if info.Size() >= DefaultMaxFileSize { + oldFile.Close() + return nil, newFile, nil + } + return oldFile, newFile, nil +} diff --git a/tsdb/engine/pd1/pd1_test.go b/tsdb/engine/pd1/pd1_test.go new file mode 100644 index 00000000000..48cdff79293 --- /dev/null +++ b/tsdb/engine/pd1/pd1_test.go @@ -0,0 +1,196 @@ +package pd1_test + +import ( + "encoding/binary" + "fmt" + "io/ioutil" + "os" + "reflect" + "testing" + "time" + + "github.com/influxdb/influxdb/influxql" + "github.com/influxdb/influxdb/tsdb" + "github.com/influxdb/influxdb/tsdb/engine/pd1" +) + +func TestEngine_WriteAndReadFloats(t *testing.T) { + e := OpenDefaultEngine() + defer e.Close() + + codec := tsdb.NewFieldCodec(map[string]*tsdb.Field{ + "value": { + ID: uint8(1), + Name: "value", + Type: influxql.Float, + }, + }) + + p1 := parsePoint("cpu,host=A value=1.1 1000000000", codec) + p2 := parsePoint("cpu,host=B value=1.2 1000000000", codec) + p3 := parsePoint("cpu,host=A value=2.1 2000000000", codec) + p4 := parsePoint("cpu,host=B value=2.2 2000000000", codec) + + if err := e.WriteAndCompact([]tsdb.Point{p1, p2, p3}, nil, nil); err != nil { + t.Fatalf("failed to write points: %s", err.Error()) + } + + verify := func() { + c := e.Cursor("cpu,host=A", tsdb.Forward) + k, v := c.Next() + if btou64(k) != uint64(p1.UnixNano()) { + t.Fatalf("p1 time wrong:\n\texp:%d\n\tgot:%d\n", p1.UnixNano(), btou64(k)) + } + if !reflect.DeepEqual(v, p1.Data()) { + t.Fatal("p1 data not equal") + } + k, v = c.Next() + if btou64(k) != uint64(p3.UnixNano()) { + t.Fatalf("p3 time wrong:\n\texp:%d\n\tgot:%d\n", p3.UnixNano(), btou64(k)) + } + if !reflect.DeepEqual(v, p3.Data()) { + t.Fatal("p3 data not equal") + } + k, v = c.Next() + if k != nil { + t.Fatal("expected nil") + } + + c = e.Cursor("cpu,host=B", tsdb.Forward) + k, v = c.Next() + if btou64(k) != uint64(p2.UnixNano()) { + t.Fatalf("p2 time wrong:\n\texp:%d\n\tgot:%d\n", p2.UnixNano(), btou64(k)) + } + if !reflect.DeepEqual(v, p2.Data()) { + t.Fatal("p2 data not equal") + } + k, v = c.Next() + if k != nil { + t.Fatal("expected nil") + } + } + verify() + + if err := e.WriteAndCompact([]tsdb.Point{p4}, nil, nil); err != nil { + t.Fatalf("failed to write points: %s", err.Error()) + } + verify() + + c := e.Cursor("cpu,host=B", tsdb.Forward) + k, v := c.Seek(u64tob(2000000000)) + if btou64(k) != uint64(p4.UnixNano()) { + t.Fatalf("p4 time wrong:\n\texp:%d\n\tgot:%d\n", p4.UnixNano(), btou64(k)) + } + if !reflect.DeepEqual(v, p4.Data()) { + t.Fatal("p4 data not equal") + } +} + +func TestEngine_WriteIndexWithCollision(t *testing.T) { +} + +func TestEngine_WriteIndexBenchmarkNames(t *testing.T) { + t.Skip("whatevs") + + e := OpenDefaultEngine() + defer e.Close() + + codec := tsdb.NewFieldCodec(map[string]*tsdb.Field{ + "value": { + ID: uint8(1), + Name: "value", + Type: influxql.Float, + }, + }) + + var points []tsdb.Point + for i := 0; i < 100000; i++ { + points = append(points, parsePoint(fmt.Sprintf("cpu%d value=22.1", i), codec)) + } + + st := time.Now() + if err := e.WriteAndCompact(points, nil, nil); err != nil { + t.Fatalf("failed to write points: %s", err.Error()) + } + fmt.Println("took: ", time.Since(st)) + + st = time.Now() + if err := e.WriteAndCompact(points, nil, nil); err != nil { + t.Fatalf("failed to write points: %s", err.Error()) + } + fmt.Println("took: ", time.Since(st)) +} + +// Engine represents a test wrapper for pd1.Engine. +type Engine struct { + *pd1.Engine +} + +// NewEngine returns a new instance of Engine. +func NewEngine(opt tsdb.EngineOptions) *Engine { + dir, err := ioutil.TempDir("", "pd1-test") + if err != nil { + panic("couldn't get temp dir") + } + + // Create test wrapper and attach mocks. + e := &Engine{ + Engine: pd1.NewEngine(dir, dir, opt).(*pd1.Engine), + } + + return e +} + +// OpenEngine returns an opened instance of Engine. Panic on error. +func OpenEngine(opt tsdb.EngineOptions) *Engine { + e := NewEngine(opt) + if err := e.Open(); err != nil { + panic(err) + } + return e +} + +// OpenDefaultEngine returns an open Engine with default options. +func OpenDefaultEngine() *Engine { return OpenEngine(tsdb.NewEngineOptions()) } + +// Close closes the engine and removes all data. +func (e *Engine) Close() error { + e.Engine.Close() + os.RemoveAll(e.Path()) + return nil +} + +func parsePoints(buf string, codec *tsdb.FieldCodec) []tsdb.Point { + points, err := tsdb.ParsePointsString(buf) + if err != nil { + panic(fmt.Sprintf("couldn't parse points: %s", err.Error())) + } + for _, p := range points { + b, err := codec.EncodeFields(p.Fields()) + if err != nil { + panic(fmt.Sprintf("couldn't encode fields: %s", err.Error())) + } + p.SetData(b) + } + return points +} + +func parsePoint(buf string, codec *tsdb.FieldCodec) tsdb.Point { + return parsePoints(buf, codec)[0] +} + +func inttob(v int) []byte { + b := make([]byte, 8) + binary.BigEndian.PutUint64(b, uint64(v)) + return b +} + +func btou64(b []byte) uint64 { + return binary.BigEndian.Uint64(b) +} + +func u64tob(v uint64) []byte { + b := make([]byte, 8) + binary.BigEndian.PutUint64(b, v) + return b +} diff --git a/tsdb/shard.go b/tsdb/shard.go index e355cc661de..a14822f367c 100644 --- a/tsdb/shard.go +++ b/tsdb/shard.go @@ -121,7 +121,7 @@ func (s *Shard) Open() error { } // Load metadata index. - if err := s.engine.LoadMetadataIndex(s.index, s.measurementFields); err != nil { + if err := s.engine.LoadMetadataIndex(s, s.index, s.measurementFields); err != nil { return fmt.Errorf("load metadata index: %s", err) } @@ -741,15 +741,22 @@ func (f *FieldCodec) DecodeByID(targetID uint8, b []byte) (interface{}, error) { // DecodeByName scans a byte slice for a field with the given name, converts it to its // expected type, and return that value. func (f *FieldCodec) DecodeByName(name string, b []byte) (interface{}, error) { - fi := f.fieldByName(name) + fi := f.FieldByName(name) if fi == nil { return 0, ErrFieldNotFound } return f.DecodeByID(fi.ID, b) } +func (f *FieldCodec) Fields() (a []*Field) { + for _, f := range f.fieldsByID { + a = append(a, f) + } + return +} + // FieldByName returns the field by its name. It will return a nil if not found -func (f *FieldCodec) fieldByName(name string) *Field { +func (f *FieldCodec) FieldByName(name string) *Field { return f.fieldsByName[name] } From 4e2dfd77c30010a0ae5d3145e10c66116c065976 Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Mon, 7 Sep 2015 15:56:21 -0700 Subject: [PATCH 004/139] WIP: finish basics of PD1. IT WORKS! (kind of) --- cmd/influx_stress/influx_stress.go | 1 + tsdb/engine.go | 3 + tsdb/engine/pd1/encoding.go | 22 ++- tsdb/engine/pd1/pd1.go | 248 +++++++++++++++++++++++------ tsdb/engine/pd1/pd1_test.go | 118 +++++++++----- tsdb/shard.go | 58 +++---- 6 files changed, 325 insertions(+), 125 deletions(-) diff --git a/cmd/influx_stress/influx_stress.go b/cmd/influx_stress/influx_stress.go index 9292fb02d90..9fe9e2af4b5 100644 --- a/cmd/influx_stress/influx_stress.go +++ b/cmd/influx_stress/influx_stress.go @@ -3,6 +3,7 @@ package main import ( "flag" "fmt" + "net/url" "runtime" "sort" "time" diff --git a/tsdb/engine.go b/tsdb/engine.go index d2862b54863..b0e9254d95a 100644 --- a/tsdb/engine.go +++ b/tsdb/engine.go @@ -18,6 +18,9 @@ var ( ErrFormatNotFound = errors.New("format not found") ) +// DefaultEngine is the default engine used by the shard when initializing. +const DefaultEngine = "pd1" + // Engine represents a swappable storage engine for the shard. type Engine interface { Open() error diff --git a/tsdb/engine/pd1/encoding.go b/tsdb/engine/pd1/encoding.go index 285c83f8c0b..60f72766d83 100644 --- a/tsdb/engine/pd1/encoding.go +++ b/tsdb/engine/pd1/encoding.go @@ -1,16 +1,34 @@ package pd1 import ( + "encoding/binary" + "math" "time" "github.com/dgryski/go-tsz" ) +type Value interface { + TimeBytes() []byte + ValueBytes() []byte + Time() time.Time +} + type FloatValue struct { Time time.Time Value float64 } +func (f *FloatValue) TimeBytes() []byte { + return u64tob(uint64(f.Time.UnixNano())) +} + +func (f *FloatValue) ValueBytes() []byte { + buf := make([]byte, 8) + binary.BigEndian.PutUint64(buf, math.Float64bits(f.Value)) + return buf +} + type FloatValues []FloatValue func (a FloatValues) Len() int { return len(a) } @@ -24,11 +42,11 @@ func EncodeFloatBlock(buf []byte, values []FloatValue) []byte { s.Push(uint32(v.Time.Unix()), v.Value) } s.Finish() - return s.Bytes() + return append(u64tob(uint64(values[0].Time.UnixNano())), s.Bytes()...) } func DecodeFloatBlock(block []byte) ([]FloatValue, error) { - iter, _ := tsz.NewIterator(block) + iter, _ := tsz.NewIterator(block[8:]) a := make([]FloatValue, 0) for iter.Next() { t, f := iter.Values() diff --git a/tsdb/engine/pd1/pd1.go b/tsdb/engine/pd1/pd1.go index d94a7d7795c..4d3c752ae2e 100644 --- a/tsdb/engine/pd1/pd1.go +++ b/tsdb/engine/pd1/pd1.go @@ -55,9 +55,9 @@ const ( // DefaultBlockSize is the default size of uncompressed points blocks. DefaultBlockSize = 512 * 1024 // 512KB - DefaultMaxFileSize = 50 * 1024 * 1024 // 50MB + DefaultMaxFileSize = 5 * 1024 * 1024 // 5MB - DefaultMaxPointsPerBlock = 5000 + DefaultMaxPointsPerBlock = 1000 // MAP_POPULATE is for the mmap syscall. For some reason this isn't defined in golang's syscall MAP_POPULATE = 0x8000 @@ -71,12 +71,15 @@ type Engine struct { mu sync.Mutex path string - shard *tsdb.Shard - // HashSeriesField is a function that takes a series key and a field name // and returns a hash identifier. It's not guaranteed to be unique. HashSeriesField func(key string) uint64 + // Shard is an interface that can pull back field type information based on measurement name + Shard interface { + FieldCodec(measurementName string) *tsdb.FieldCodec + } + filesLock sync.RWMutex files dataFiles currentFileID int @@ -108,11 +111,34 @@ func (e *Engine) Open() error { // TODO: clean up previous names write // TODO: clean up any data files that didn't get cleaned up + files, err := filepath.Glob(filepath.Join(e.path, fmt.Sprintf("*.%s", Format))) + if err != nil { + return err + } + for _, fn := range files { + f, err := os.OpenFile(fn, os.O_RDONLY, 0666) + if err != nil { + return fmt.Errorf("error opening file %s: %s", fn, err.Error()) + } + df, err := NewDataFile(f) + if err != nil { + return fmt.Errorf("error opening memory map for file %s: %s", fn, err.Error()) + } + e.files = append(e.files, df) + } + sort.Sort(e.files) + return nil } // Close closes the engine. func (e *Engine) Close() error { + e.queryLock.Lock() + defer e.queryLock.Unlock() + + for _, df := range e.files { + _ = df.Close() + } return nil } @@ -121,7 +147,7 @@ func (e *Engine) SetLogOutput(w io.Writer) {} // LoadMetadataIndex loads the shard metadata into memory. func (e *Engine) LoadMetadataIndex(shard *tsdb.Shard, index *tsdb.DatabaseIndex, measurementFields map[string]*tsdb.MeasurementFields) error { - e.shard = shard + e.Shard = shard // TODO: write the metadata from the WAL // Load measurement metadata @@ -165,7 +191,7 @@ func (e *Engine) LoadMetadataIndex(shard *tsdb.Shard, index *tsdb.DatabaseIndex, func (e *Engine) WritePoints(points []tsdb.Point, measurementFieldsToSave map[string]*tsdb.MeasurementFields, seriesToCreate []*tsdb.SeriesCreate) error { // TODO: Write points to the WAL - return nil + return e.WriteAndCompact(points, measurementFieldsToSave, seriesToCreate) } func (e *Engine) WriteAndCompact(points []tsdb.Point, measurementFieldsToSave map[string]*tsdb.MeasurementFields, seriesToCreate []*tsdb.SeriesCreate) error { @@ -200,8 +226,6 @@ func (e *Engine) WriteAndCompact(points []tsdb.Point, measurementFieldsToSave ma } } - fmt.Println("read names: ", len(names), len(ids)) - // these are values that are newer than anything stored in the shard valuesByID := make(map[uint64]*valueCollection) // map the points to the data file they belong to if they overlap @@ -288,7 +312,6 @@ func (e *Engine) WriteAndCompact(points []tsdb.Point, measurementFieldsToSave ma } } - fmt.Println("writing names:", len(names)) b, err = json.Marshal(names) if err != nil { return err @@ -302,14 +325,13 @@ func (e *Engine) WriteAndCompact(points []tsdb.Point, measurementFieldsToSave ma if overwriteNewestFile { if err := e.rewriteFile(newestDataFile, valuesByID); err != nil { return err - } else if err := e.rewriteFile(nil, valuesByID); err != nil { - return err } + } else if err := e.rewriteFile(nil, valuesByID); err != nil { + return err } // flush each of the old ones for df, vals := range dataFileToValues { - fmt.Println("writing vals to old file: ", df.f.Name()) if err := e.rewriteFile(df, vals); err != nil { return err } @@ -374,6 +396,7 @@ func (e *Engine) rewriteFile(oldDF *dataFile, values map[uint64]*valueCollection f.Close() return nil } + // write the series ids and empty starting positions for _, id := range ids { if _, err := f.Write(append(u64tob(id), []byte{0x00, 0x00, 0x00, 0x00}...)); err != nil { @@ -423,6 +446,7 @@ func (e *Engine) rewriteFile(oldDF *dataFile, values map[uint64]*valueCollection // if the values are not in the file, just write the new ones fpos, ok := oldIDToPosition[id] if !ok { + // TODO: ensure we encode only the amount in a block block := newVals.Encode(buf) if _, err := f.Write(append(u64tob(id), u32tob(uint32(len(block)))...)); err != nil { f.Close() @@ -444,7 +468,7 @@ func (e *Engine) rewriteFile(oldDF *dataFile, values map[uint64]*valueCollection break } length := btou32(oldDF.mmap[fpos+8 : fpos+12]) - block := oldDF.mmap[fpos : fpos+12+length] + block := oldDF.mmap[fpos+12 : fpos+12+length] fpos += (12 + length) // determine if there's a block after this with the same id and get its time @@ -477,6 +501,21 @@ func (e *Engine) rewriteFile(oldDF *dataFile, values map[uint64]*valueCollection break } } + + // TODO: ensure we encode only the amount in a block, refactor this wil line 450 into func + if len(newVals.floatValues) > 0 { + // TODO: ensure we encode only the amount in a block + block := newVals.Encode(buf) + if _, err := f.Write(append(u64tob(id), u32tob(uint32(len(block)))...)); err != nil { + f.Close() + return err + } + if _, err := f.Write(block); err != nil { + f.Close() + return err + } + currentPosition += uint32(12 + len(block)) + } } // write out the times and positions @@ -572,7 +611,6 @@ func (e *Engine) replaceCompressedFile(name string, data []byte) error { if _, err := f.Write(b); err != nil { return err } - fmt.Println("compressed: ", len(b)) if err := f.Close(); err != nil { return err } @@ -605,7 +643,7 @@ func (e *Engine) Begin(writable bool) (tsdb.Tx, error) { // TODO: make the cursor take a field name func (e *Engine) Cursor(series string, direction tsdb.Direction) tsdb.Cursor { measurementName := tsdb.MeasurementFromSeriesKey(series) - codec := e.shard.FieldCodec(measurementName) + codec := e.Shard.FieldCodec(measurementName) if codec == nil { return &cursor{} } @@ -658,7 +696,7 @@ func (e *Engine) writeFields(fields map[string]*tsdb.MeasurementFields) error { return err } - fn := e.path + "." + FieldsFileExtension + "tmp" + fn := filepath.Join(e.path, FieldsFileExtension+"tmp") ff, err := os.OpenFile(fn, os.O_CREATE|os.O_RDWR, 0666) if err != nil { return err @@ -670,7 +708,7 @@ func (e *Engine) writeFields(fields map[string]*tsdb.MeasurementFields) error { if err := ff.Close(); err != nil { return err } - fieldsFileName := e.path + "." + FieldsFileExtension + fieldsFileName := filepath.Join(e.path, FieldsFileExtension) if _, err := os.Stat(fieldsFileName); !os.IsNotExist(err) { if err := os.Remove(fieldsFileName); err != nil { @@ -684,7 +722,7 @@ func (e *Engine) writeFields(fields map[string]*tsdb.MeasurementFields) error { func (e *Engine) readFields() (map[string]*tsdb.MeasurementFields, error) { fields := make(map[string]*tsdb.MeasurementFields) - f, err := os.OpenFile(e.path+"."+FieldsFileExtension, os.O_RDONLY, 0666) + f, err := os.OpenFile(filepath.Join(e.path, FieldsFileExtension), os.O_RDONLY, 0666) if os.IsNotExist(err) { return fields, nil } else if err != nil { @@ -732,7 +770,7 @@ func (e *Engine) writeSeries(series map[string]*tsdb.Series) error { return err } - fn := e.path + "." + SeriesFileExtension + "tmp" + fn := filepath.Join(e.path, SeriesFileExtension+"tmp") ff, err := os.OpenFile(fn, os.O_CREATE|os.O_RDWR, 0666) if err != nil { return err @@ -744,7 +782,7 @@ func (e *Engine) writeSeries(series map[string]*tsdb.Series) error { if err := ff.Close(); err != nil { return err } - seriesFileName := e.path + "." + SeriesFileExtension + seriesFileName := filepath.Join(e.path, SeriesFileExtension) if _, err := os.Stat(seriesFileName); !os.IsNotExist(err) { if err := os.Remove(seriesFileName); err != nil && err != os.ErrNotExist { @@ -758,7 +796,7 @@ func (e *Engine) writeSeries(series map[string]*tsdb.Series) error { func (e *Engine) readSeries() (map[string]*tsdb.Series, error) { series := make(map[string]*tsdb.Series) - f, err := os.OpenFile(e.path+"."+SeriesFileExtension, os.O_RDONLY, 0666) + f, err := os.OpenFile(filepath.Join(e.path, SeriesFileExtension), os.O_RDONLY, 0666) if os.IsNotExist(err) { return series, nil } else if err != nil { @@ -843,14 +881,15 @@ func (v *valueCollection) DecodeAndCombine(block, buf []byte, nextTime int64, ha } if hasFutureBlock { - for i, val := range v.floatValues { - if val.Time.UnixNano() > nextTime { - values = append(values, v.floatValues[:i]...) - v.floatValues = v.floatValues[i:] - } - } + // take all values that have times less than the future block and update the vals array + pos := sort.Search(len(v.floatValues), func(i int) bool { + return v.floatValues[i].Time.UnixNano() >= nextTime + }) + values = append(values, v.floatValues[:pos]...) + v.floatValues = v.floatValues[pos:] } else { values = append(values, v.floatValues...) + v.floatValues = nil } sort.Sort(FloatValues(values)) // TODO: deduplicate values @@ -955,7 +994,8 @@ func (d *dataFile) IDToPosition() map[uint64]uint32 { for i := 0; i < count; i++ { offset := 20 + (i * 12) id := btou64(d.mmap[offset : offset+8]) - m[id] = btou32(d.mmap[offset+8 : offset+12]) + pos := btou32(d.mmap[offset+8 : offset+12]) + m[id] = pos } return m @@ -968,26 +1008,23 @@ func (d *dataFile) StartingPositionForID(id uint64) uint32 { seriesCount := d.SeriesCount() min := 0 - max := seriesCount - // // set the minimum position to the first after the file header - // posMin := fileHeaderSize - - // // set the maximum position to the end of the series header - // posMax := fileHeaderSize + (seriesCount * seriesHeaderSize) + max := int(seriesCount) for min < max { mid := (max-min)/2 + min offset := mid*seriesHeaderSize + fileHeaderSize - checkID := btou64(d.mmap[offset:8]) + checkID := btou64(d.mmap[offset : offset+8]) if checkID == id { return btou32(d.mmap[offset+8 : offset+12]) } else if checkID < id { min = mid + 1 + } else { + max = mid } - max = mid } + return uint32(0) } @@ -998,12 +1035,12 @@ func (a dataFiles) Swap(i, j int) { a[i], a[j] = a[j], a[i] } func (a dataFiles) Less(i, j int) bool { return a[i].MinTime() < a[j].MinTime() } type cursor struct { - id uint64 - dataType influxql.DataType - f *dataFile - dataFilePos int - pos uint32 - vals []FloatValues + id uint64 + dataType influxql.DataType + f *dataFile + filesPos int // the index in the files slice we're looking at + pos uint32 + vals FloatValues direction tsdb.Direction @@ -1013,29 +1050,136 @@ type cursor struct { func newCursor(id uint64, dataType influxql.DataType, files []*dataFile, direction tsdb.Direction) *cursor { return &cursor{ - ids: id, - types: dataType, + id: id, + dataType: dataType, direction: direction, files: files, } } -func (c *cursor) Seek(seek []byte) (key, value []byte) { return nil, nil } +func (c *cursor) Seek(seek []byte) (key, value []byte) { + t := int64(btou64(seek)) + + if t < c.files[0].MinTime() { + c.filesPos = 0 + c.f = c.files[0] + } else { + for i, f := range c.files { + if t >= f.MinTime() && t <= f.MaxTime() { + c.filesPos = i + c.f = f + break + } + } + } + + if c.f == nil { + return nil, nil + } + + // TODO: make this for the reverse direction cursor + + // now find the spot in the file we need to go + for { + pos := c.f.StartingPositionForID(c.id) + + // if this id isn't in this file, move to next one or return + if pos == 0 { + c.filesPos++ + if c.filesPos >= len(c.files) { + return nil, nil + } + c.f = c.files[c.filesPos] + continue + } + + // seek to the block and values we're looking for + for { + // if the time is between this block and the next, + // decode this block and go, otherwise seek to next block + length := btou32(c.f.mmap[pos+8 : pos+12]) + + // if the next block has a time less than what we're seeking to, + // skip decoding this block and continue on + nextBlockPos := pos + 12 + length + if nextBlockPos < c.f.size { + nextBlockID := btou64(c.f.mmap[nextBlockPos : nextBlockPos+8]) + if nextBlockID == c.id { + nextBlockTime := int64(btou64(c.f.mmap[nextBlockPos+12 : nextBlockPos+20])) + if nextBlockTime <= t { + pos = nextBlockPos + continue + } + } + } + + // it must be in this block or not at all + tb, vb := c.decodeBlockAndGetValues(pos) + if int64(btou64(tb)) >= t { + return tb, vb + } + + // wasn't in the first value popped out of the block, check the rest + for i, v := range c.vals { + if v.Time.UnixNano() >= t { + c.vals = c.vals[i+1:] + return v.TimeBytes(), v.ValueBytes() + } + } + + // not in this one, let the top loop look for it in the next file + break + } + } +} func (c *cursor) Next() (key, value []byte) { - if vals == nil { + if len(c.vals) == 0 { + // if we have a file set, see if the next block is for this ID + if c.f != nil && c.pos < c.f.size { + nextBlockID := btou64(c.f.mmap[c.pos : c.pos+8]) + if nextBlockID == c.id { + return c.decodeBlockAndGetValues(c.pos) + } + } + + // if the file is nil we hit the end of the previous file, advance the file cursor + if c.f != nil { + c.filesPos++ + } + // loop until we find a file with some data - for dataFilePos < len(c.files) { - f = c.files[c.dataFilePos] - c.dataFilePos++ + for c.filesPos < len(c.files) { + f := c.files[c.filesPos] - // startPosition := f + startingPos := f.StartingPositionForID(c.id) + if startingPos == 0 { + continue + } + c.f = f + return c.decodeBlockAndGetValues(startingPos) } + + // we didn't get to a file that had a next value + return nil, nil } - return nil, nil + + v := c.vals[0] + c.vals = c.vals[1:] + + return v.TimeBytes(), v.ValueBytes() } -func (c *cursor) next(id uint64) (key, value []byte) +func (c *cursor) decodeBlockAndGetValues(position uint32) ([]byte, []byte) { + length := btou32(c.f.mmap[position+8 : position+12]) + block := c.f.mmap[position+12 : position+12+length] + c.vals, _ = DecodeFloatBlock(block) + c.pos = position + 12 + length + + v := c.vals[0] + c.vals = c.vals[1:] + return v.TimeBytes(), v.ValueBytes() +} func (c *cursor) Direction() tsdb.Direction { return c.direction } diff --git a/tsdb/engine/pd1/pd1_test.go b/tsdb/engine/pd1/pd1_test.go index 48cdff79293..7d915aab69e 100644 --- a/tsdb/engine/pd1/pd1_test.go +++ b/tsdb/engine/pd1/pd1_test.go @@ -4,8 +4,8 @@ import ( "encoding/binary" "fmt" "io/ioutil" + "math" "os" - "reflect" "testing" "time" @@ -18,37 +18,31 @@ func TestEngine_WriteAndReadFloats(t *testing.T) { e := OpenDefaultEngine() defer e.Close() - codec := tsdb.NewFieldCodec(map[string]*tsdb.Field{ - "value": { - ID: uint8(1), - Name: "value", - Type: influxql.Float, - }, - }) + e.Shard = newFieldCodecMock(map[string]influxql.DataType{"value": influxql.Float}) - p1 := parsePoint("cpu,host=A value=1.1 1000000000", codec) - p2 := parsePoint("cpu,host=B value=1.2 1000000000", codec) - p3 := parsePoint("cpu,host=A value=2.1 2000000000", codec) - p4 := parsePoint("cpu,host=B value=2.2 2000000000", codec) + p1 := parsePoint("cpu,host=A value=1.1 1000000000") + p2 := parsePoint("cpu,host=B value=1.2 1000000000") + p3 := parsePoint("cpu,host=A value=2.1 2000000000") + p4 := parsePoint("cpu,host=B value=2.2 2000000000") if err := e.WriteAndCompact([]tsdb.Point{p1, p2, p3}, nil, nil); err != nil { t.Fatalf("failed to write points: %s", err.Error()) } - verify := func() { + verify := func(checkSingleBVal bool) { c := e.Cursor("cpu,host=A", tsdb.Forward) k, v := c.Next() if btou64(k) != uint64(p1.UnixNano()) { t.Fatalf("p1 time wrong:\n\texp:%d\n\tgot:%d\n", p1.UnixNano(), btou64(k)) } - if !reflect.DeepEqual(v, p1.Data()) { + if 1.1 != btof64(v) { t.Fatal("p1 data not equal") } k, v = c.Next() if btou64(k) != uint64(p3.UnixNano()) { t.Fatalf("p3 time wrong:\n\texp:%d\n\tgot:%d\n", p3.UnixNano(), btou64(k)) } - if !reflect.DeepEqual(v, p3.Data()) { + if 2.1 != btof64(v) { t.Fatal("p3 data not equal") } k, v = c.Next() @@ -61,28 +55,56 @@ func TestEngine_WriteAndReadFloats(t *testing.T) { if btou64(k) != uint64(p2.UnixNano()) { t.Fatalf("p2 time wrong:\n\texp:%d\n\tgot:%d\n", p2.UnixNano(), btou64(k)) } - if !reflect.DeepEqual(v, p2.Data()) { + if 1.2 != btof64(v) { t.Fatal("p2 data not equal") } - k, v = c.Next() - if k != nil { - t.Fatal("expected nil") + + if checkSingleBVal { + k, v = c.Next() + if k != nil { + t.Fatal("expected nil") + } } } - verify() + verify(true) if err := e.WriteAndCompact([]tsdb.Point{p4}, nil, nil); err != nil { t.Fatalf("failed to write points: %s", err.Error()) } - verify() + verify(false) c := e.Cursor("cpu,host=B", tsdb.Forward) - k, v := c.Seek(u64tob(2000000000)) + k, v := c.Next() + if btou64(k) != uint64(p2.UnixNano()) { + t.Fatalf("p2 time wrong:\n\texp:%d\n\tgot:%d\n", p2.UnixNano(), btou64(k)) + } + if 1.2 != btof64(v) { + t.Fatal("p2 data not equal") + } + k, v = c.Next() + if btou64(k) != uint64(p4.UnixNano()) { + t.Fatalf("p2 time wrong:\n\texp:%d\n\tgot:%d\n", p2.UnixNano(), btou64(k)) + } + if 2.2 != btof64(v) { + t.Fatal("p2 data not equal") + } + + // verify we can seek + k, v = c.Seek(u64tob(2000000000)) if btou64(k) != uint64(p4.UnixNano()) { - t.Fatalf("p4 time wrong:\n\texp:%d\n\tgot:%d\n", p4.UnixNano(), btou64(k)) + t.Fatalf("p2 time wrong:\n\texp:%d\n\tgot:%d\n", p2.UnixNano(), btou64(k)) } - if !reflect.DeepEqual(v, p4.Data()) { - t.Fatal("p4 data not equal") + if 2.2 != btof64(v) { + t.Fatal("p2 data not equal") + } + + c = e.Cursor("cpu,host=A", tsdb.Forward) + k, v = c.Seek(u64tob(0)) + if btou64(k) != uint64(p1.UnixNano()) { + t.Fatalf("p1 time wrong:\n\texp:%d\n\tgot:%d\n", p1.UnixNano(), btou64(k)) + } + if 1.1 != btof64(v) { + t.Fatal("p1 data not equal") } } @@ -95,17 +117,9 @@ func TestEngine_WriteIndexBenchmarkNames(t *testing.T) { e := OpenDefaultEngine() defer e.Close() - codec := tsdb.NewFieldCodec(map[string]*tsdb.Field{ - "value": { - ID: uint8(1), - Name: "value", - Type: influxql.Float, - }, - }) - var points []tsdb.Point for i := 0; i < 100000; i++ { - points = append(points, parsePoint(fmt.Sprintf("cpu%d value=22.1", i), codec)) + points = append(points, parsePoint(fmt.Sprintf("cpu%d value=22.1", i))) } st := time.Now() @@ -160,23 +174,35 @@ func (e *Engine) Close() error { return nil } -func parsePoints(buf string, codec *tsdb.FieldCodec) []tsdb.Point { +func newFieldCodecMock(fields map[string]influxql.DataType) *FieldCodeMock { + m := make(map[string]*tsdb.Field) + + for n, t := range fields { + m[n] = &tsdb.Field{Name: n, Type: t} + } + codec := tsdb.NewFieldCodec(m) + + return &FieldCodeMock{codec: codec} +} + +type FieldCodeMock struct { + codec *tsdb.FieldCodec +} + +func (f *FieldCodeMock) FieldCodec(m string) *tsdb.FieldCodec { + return f.codec +} + +func parsePoints(buf string) []tsdb.Point { points, err := tsdb.ParsePointsString(buf) if err != nil { panic(fmt.Sprintf("couldn't parse points: %s", err.Error())) } - for _, p := range points { - b, err := codec.EncodeFields(p.Fields()) - if err != nil { - panic(fmt.Sprintf("couldn't encode fields: %s", err.Error())) - } - p.SetData(b) - } return points } -func parsePoint(buf string, codec *tsdb.FieldCodec) tsdb.Point { - return parsePoints(buf, codec)[0] +func parsePoint(buf string) tsdb.Point { + return parsePoints(buf)[0] } func inttob(v int) []byte { @@ -194,3 +220,7 @@ func u64tob(v uint64) []byte { binary.BigEndian.PutUint64(b, v) return b } + +func btof64(b []byte) float64 { + return math.Float64frombits(binary.BigEndian.Uint64(b)) +} diff --git a/tsdb/shard.go b/tsdb/shard.go index a14822f367c..2e04735665f 100644 --- a/tsdb/shard.go +++ b/tsdb/shard.go @@ -229,28 +229,29 @@ func (s *Shard) WritePoints(points []models.Point) error { } // make sure all data is encoded before attempting to save to bolt - for _, p := range points { - // Ignore if raw data has already been marshaled. - if p.Data() != nil { - continue - } - - // This was populated earlier, don't need to validate that it's there. - s.mu.RLock() - mf := s.measurementFields[p.Name()] - s.mu.RUnlock() - - // If a measurement is dropped while writes for it are in progress, this could be nil - if mf == nil { - return ErrFieldNotFound - } - - data, err := mf.Codec.EncodeFields(p.Fields()) - if err != nil { - return err - } - p.SetData(data) - } + // TODO: make this only commented out for pd1 engine + // for _, p := range points { + // // Ignore if raw data has already been marshaled. + // if p.Data() != nil { + // continue + // } + + // // This was populated earlier, don't need to validate that it's there. + // s.mu.RLock() + // mf := s.measurementFields[p.Name()] + // s.mu.RUnlock() + + // // If a measurement is dropped while writes for it are in progress, this could be nil + // if mf == nil { + // return ErrFieldNotFound + // } + + // data, err := mf.Codec.EncodeFields(p.Fields()) + // if err != nil { + // return err + // } + // p.SetData(data) + // } // Write to the engine. if err := s.engine.WritePoints(points, measurementFieldsToSave, seriesToCreate); err != nil { @@ -741,11 +742,14 @@ func (f *FieldCodec) DecodeByID(targetID uint8, b []byte) (interface{}, error) { // DecodeByName scans a byte slice for a field with the given name, converts it to its // expected type, and return that value. func (f *FieldCodec) DecodeByName(name string, b []byte) (interface{}, error) { - fi := f.FieldByName(name) - if fi == nil { - return 0, ErrFieldNotFound - } - return f.DecodeByID(fi.ID, b) + // TODO: this is a hack for PD1 testing, please to remove + return math.Float64frombits(binary.BigEndian.Uint64(b)), nil + + // fi := f.FieldByName(name) + // if fi == nil { + // return 0, ErrFieldNotFound + // } + // return f.DecodeByID(fi.ID, b) } func (f *FieldCodec) Fields() (a []*Field) { From f37df1133920e54890a36e65c363caf8bd6b4949 Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Wed, 9 Sep 2015 11:29:50 -0700 Subject: [PATCH 005/139] WIP: more WAL work --- cmd/influx_stress/influx_stress.go | 1 + tsdb/config.go | 2 +- tsdb/engine/pd1/encoding.go | 119 ++++-- tsdb/engine/pd1/encoding_test.go | 34 +- tsdb/engine/pd1/pd1.go | 502 ++++++------------------- tsdb/engine/pd1/pd1_test.go | 9 +- tsdb/engine/pd1/wal.go | 581 +++++++++++++++++++++++++++++ 7 files changed, 820 insertions(+), 428 deletions(-) create mode 100644 tsdb/engine/pd1/wal.go diff --git a/cmd/influx_stress/influx_stress.go b/cmd/influx_stress/influx_stress.go index 9fe9e2af4b5..2247a5329b3 100644 --- a/cmd/influx_stress/influx_stress.go +++ b/cmd/influx_stress/influx_stress.go @@ -3,6 +3,7 @@ package main import ( "flag" "fmt" + "math/rand" "net/url" "runtime" "sort" diff --git a/tsdb/config.go b/tsdb/config.go index 9843541e296..dfd267d2c37 100644 --- a/tsdb/config.go +++ b/tsdb/config.go @@ -42,7 +42,7 @@ const ( // we'll need to create backpressure, otherwise we'll fill up the memory and die. // This number multiplied by the parition count is roughly the max possible memory // size for the in-memory WAL cache. - DefaultPartitionSizeThreshold = 20 * 1024 * 1024 // 20MB + DefaultPartitionSizeThreshold = 50 * 1024 * 1024 // 50MB ) type Config struct { diff --git a/tsdb/engine/pd1/encoding.go b/tsdb/engine/pd1/encoding.go index 60f72766d83..7262e8a6c36 100644 --- a/tsdb/engine/pd1/encoding.go +++ b/tsdb/engine/pd1/encoding.go @@ -12,52 +12,127 @@ type Value interface { TimeBytes() []byte ValueBytes() []byte Time() time.Time + Value() interface{} + Size() int } +func NewValue(t time.Time, value interface{}) Value { + switch v := value.(type) { + // case int64: + // return &Int64Value{time: t, value: v} + case float64: + return &FloatValue{time: t, value: v} + // case bool: + // return &BoolValue{time: t, value: v} + // case string: + // return &StringValue{time: t, value: v} + } + return &EmptyValue{} +} + +type EmptyValue struct { +} + +func (e *EmptyValue) TimeBytes() []byte { return nil } +func (e *EmptyValue) ValueBytes() []byte { return nil } +func (e *EmptyValue) Time() time.Time { return time.Unix(0, 0) } +func (e *EmptyValue) Value() interface{} { return nil } +func (e *EmptyValue) Size() int { return 0 } + +// Values represented a time ascending sorted collection of Value types. +// the underlying type should be the same across all values, but the interface +// makes the code cleaner. +type Values []Value + +func (v Values) MinTime() int64 { + return v[0].Time().UnixNano() +} + +func (v Values) MaxTime() int64 { + return v[len(v)-1].Time().UnixNano() +} + +func (v Values) Encode(buf []byte) []byte { + switch v[0].(type) { + case *FloatValue: + a := make([]*FloatValue, len(v)) + for i, vv := range v { + a[i] = vv.(*FloatValue) + } + return EncodeFloatBlock(buf, a) + + // TODO: add support for other types + } + + return nil +} + +func (v Values) DecodeSameTypeBlock(block []byte) Values { + switch v[0].(type) { + case *FloatValue: + a, _ := DecodeFloatBlock(block) + return a + + // TODO: add support for other types + } + return nil +} + +// Sort methods +func (a Values) Len() int { return len(a) } +func (a Values) Swap(i, j int) { a[i], a[j] = a[j], a[i] } +func (a Values) Less(i, j int) bool { return a[i].Time().UnixNano() < a[j].Time().UnixNano() } + type FloatValue struct { - Time time.Time - Value float64 + time time.Time + value float64 +} + +func (f *FloatValue) Time() time.Time { + return f.time +} + +func (f *FloatValue) Value() interface{} { + return f.value } func (f *FloatValue) TimeBytes() []byte { - return u64tob(uint64(f.Time.UnixNano())) + return u64tob(uint64(f.Time().UnixNano())) } func (f *FloatValue) ValueBytes() []byte { buf := make([]byte, 8) - binary.BigEndian.PutUint64(buf, math.Float64bits(f.Value)) + binary.BigEndian.PutUint64(buf, math.Float64bits(f.value)) return buf } -type FloatValues []FloatValue - -func (a FloatValues) Len() int { return len(a) } -func (a FloatValues) Swap(i, j int) { a[i], a[j] = a[j], a[i] } -func (a FloatValues) Less(i, j int) bool { return a[i].Time.UnixNano() < a[j].Time.UnixNano() } +func (f *FloatValue) Size() int { + return 16 +} // TODO: make this work with nanosecond timestamps -func EncodeFloatBlock(buf []byte, values []FloatValue) []byte { - s := tsz.New(uint32(values[0].Time.Unix())) +func EncodeFloatBlock(buf []byte, values []*FloatValue) []byte { + s := tsz.New(uint32(values[0].Time().Unix())) for _, v := range values { - s.Push(uint32(v.Time.Unix()), v.Value) + s.Push(uint32(v.Time().Unix()), v.value) } s.Finish() - return append(u64tob(uint64(values[0].Time.UnixNano())), s.Bytes()...) + return append(u64tob(uint64(values[0].Time().UnixNano())), s.Bytes()...) } -func DecodeFloatBlock(block []byte) ([]FloatValue, error) { +func DecodeFloatBlock(block []byte) ([]Value, error) { iter, _ := tsz.NewIterator(block[8:]) - a := make([]FloatValue, 0) + a := make([]Value, 0) for iter.Next() { t, f := iter.Values() - a = append(a, FloatValue{time.Unix(int64(t), 0), f}) + a = append(a, &FloatValue{time.Unix(int64(t), 0), f}) } return a, nil } type BoolValue struct { - Time time.Time - Value bool + time time.Time + value bool } func EncodeBoolBlock(buf []byte, values []BoolValue) []byte { @@ -69,8 +144,8 @@ func DecodeBoolBlock(block []byte) ([]BoolValue, error) { } type Int64Value struct { - Time time.Time - Value int64 + time time.Time + value int64 } func EncodeInt64Block(buf []byte, values []Int64Value) []byte { @@ -82,8 +157,8 @@ func DecodeInt64Block(block []byte) ([]Int64Value, error) { } type StringValue struct { - Time time.Time - Value string + time time.Time + value string } func EncodeStringBlock(buf []byte, values []StringValue) []byte { diff --git a/tsdb/engine/pd1/encoding_test.go b/tsdb/engine/pd1/encoding_test.go index 26bb4c2e076..aa5a4b15e73 100644 --- a/tsdb/engine/pd1/encoding_test.go +++ b/tsdb/engine/pd1/encoding_test.go @@ -1,32 +1,32 @@ package pd1_test import ( - "math/rand" - "reflect" + // "math/rand" + // "reflect" "testing" "time" - "github.com/influxdb/influxdb/tsdb/engine/pd1" + // "github.com/influxdb/influxdb/tsdb/engine/pd1" ) func TestEncoding_FloatBlock(t *testing.T) { - valueCount := 100 - times := getTimes(valueCount, 60, time.Second) - values := make([]pd1.FloatValue, len(times)) - for i, t := range times { - values[i] = pd1.FloatValue{Time: t, Value: rand.Float64()} - } + // valueCount := 100 + // times := getTimes(valueCount, 60, time.Second) + // values := make([]Value, len(times)) + // for i, t := range times { + // values[i] = pd1.NewValue(t, rand.Float64()) + // } - b := pd1.EncodeFloatBlock(nil, values) + // b := pd1.EncodeFloatBlock(nil, values) - decodedValues, err := pd1.DecodeFloatBlock(b) - if err != nil { - t.Fatalf("error decoding: %s", err.Error) - } + // decodedValues, err := pd1.DecodeFloatBlock(b) + // if err != nil { + // t.Fatalf("error decoding: %s", err.Error) + // } - if !reflect.DeepEqual(decodedValues, values) { - t.Fatalf("unexpected results:\n\tgot: %v\n\texp: %v\n", decodedValues, values) - } + // if !reflect.DeepEqual(decodedValues, values) { + // t.Fatalf("unexpected results:\n\tgot: %v\n\texp: %v\n", decodedValues, values) + // } } func getTimes(n, step int, precision time.Duration) []time.Time { diff --git a/tsdb/engine/pd1/pd1.go b/tsdb/engine/pd1/pd1.go index 4d3c752ae2e..ef91fb66e2d 100644 --- a/tsdb/engine/pd1/pd1.go +++ b/tsdb/engine/pd1/pd1.go @@ -1,7 +1,6 @@ package pd1 import ( - "bytes" "encoding/binary" "encoding/json" "fmt" @@ -12,8 +11,6 @@ import ( "os" "path/filepath" "sort" - "strconv" - "strings" "sync" "syscall" "time" @@ -29,13 +26,13 @@ const ( // FieldsFileExtension is the extension for the file that stores compressed field // encoding data for this db - FieldsFileExtension = "fld" + FieldsFileExtension = "fields" // SeriesFileExtension is the extension for the file that stores the compressed // series metadata for series in this db - SeriesFileExtension = "srs" + SeriesFileExtension = "series" - CollisionsFileExtension = "col" + CollisionsFileExtension = "collisions" ) type TimePrecision uint8 @@ -55,7 +52,7 @@ const ( // DefaultBlockSize is the default size of uncompressed points blocks. DefaultBlockSize = 512 * 1024 // 512KB - DefaultMaxFileSize = 5 * 1024 * 1024 // 5MB + DefaultMaxFileSize = 10 * 1024 * 1024 // 10MB DefaultMaxPointsPerBlock = 1000 @@ -80,6 +77,8 @@ type Engine struct { FieldCodec(measurementName string) *tsdb.FieldCodec } + WAL *Log + filesLock sync.RWMutex files dataFiles currentFileID int @@ -88,11 +87,19 @@ type Engine struct { // NewEngine returns a new instance of Engine. func NewEngine(path string, walPath string, opt tsdb.EngineOptions) tsdb.Engine { + w := NewLog(path) + w.FlushColdInterval = time.Duration(opt.Config.WALFlushColdInterval) + w.MemorySizeThreshold = int(opt.Config.WALPartitionSizeThreshold) + w.LoggingEnabled = opt.Config.WALLoggingEnabled + e := &Engine{ path: path, + // TODO: this is the function where we can inject a check against the in memory collisions HashSeriesField: hashSeriesField, + WAL: w, } + e.WAL.Index = e return e } @@ -116,6 +123,13 @@ func (e *Engine) Open() error { return err } for _, fn := range files { + id, err := idFromFileName(fn) + if err != nil { + return err + } + if id >= e.currentFileID { + e.currentFileID = id + 1 + } f, err := os.OpenFile(fn, os.O_RDONLY, 0666) if err != nil { return fmt.Errorf("error opening file %s: %s", fn, err.Error()) @@ -128,6 +142,10 @@ func (e *Engine) Open() error { } sort.Sort(e.files) + if err := e.WAL.Open(); err != nil { + return err + } + return nil } @@ -189,12 +207,10 @@ func (e *Engine) LoadMetadataIndex(shard *tsdb.Shard, index *tsdb.DatabaseIndex, // WritePoints writes metadata and point data into the engine. // Returns an error if new points are added to an existing key. func (e *Engine) WritePoints(points []tsdb.Point, measurementFieldsToSave map[string]*tsdb.MeasurementFields, seriesToCreate []*tsdb.SeriesCreate) error { - // TODO: Write points to the WAL - - return e.WriteAndCompact(points, measurementFieldsToSave, seriesToCreate) + return e.WAL.WritePoints(points, measurementFieldsToSave, seriesToCreate) } -func (e *Engine) WriteAndCompact(points []tsdb.Point, measurementFieldsToSave map[string]*tsdb.MeasurementFields, seriesToCreate []*tsdb.SeriesCreate) error { +func (e *Engine) WriteAndCompact(pointsByKey map[string]Values, measurementFieldsToSave map[string]*tsdb.MeasurementFields, seriesToCreate []*tsdb.SeriesCreate) error { e.mu.Lock() defer e.mu.Unlock() @@ -205,120 +221,78 @@ func (e *Engine) WriteAndCompact(points []tsdb.Point, measurementFieldsToSave ma return err } - if len(points) == 0 { + if len(pointsByKey) == 0 { return nil } - b, err := e.readCompressedFile("names") + // read in keys and assign any that aren't defined + b, err := e.readCompressedFile("ids") if err != nil { return err } - ids := make(map[uint64]string) - - var names []string + ids := make(map[string]uint64) if b != nil { - if err := json.Unmarshal(b, &names); err != nil { + if err := json.Unmarshal(b, &ids); err != nil { return err } - - for _, n := range names { - ids[e.HashSeriesField(n)] = n - } } // these are values that are newer than anything stored in the shard - valuesByID := make(map[uint64]*valueCollection) - // map the points to the data file they belong to if they overlap - dataFileToValues := make(map[*dataFile]map[uint64]*valueCollection) - - // we keep track of the newest data file and if it should be - // rewritten with new data. - var newestDataFile *dataFile - overwriteNewestFile := false - if len(e.files) > 0 { - newestDataFile = e.files[len(e.files)-1] - overwriteNewestFile = newestDataFile.size < DefaultMaxFileSize - } - - // compute ids of new keys and arrange for insertion - for _, p := range points { - for fn, val := range p.Fields() { - n := seriesFieldKey(string(p.Key()), fn) - id := e.HashSeriesField(n) - if series, ok := ids[id]; !ok { - names = append(names, n) - } else { // possible collision? - if n != series { - // TODO: implement collision detection - panic("name collision!") + valuesByID := make(map[uint64]Values) + + idToKey := make(map[uint64]string) // we only use this map if new ids are being created + newKeys := false + for k, values := range pointsByKey { + var id uint64 + var ok bool + if id, ok = ids[k]; !ok { + // populate the map if we haven't already + if len(idToKey) == 0 { + for n, id := range ids { + idToKey[id] = n } } - ids[id] = n - - vals := valuesByID[id] - if vals == nil { - // TODO: deal with situation where there are already files, - // but the user is inserting a bunch of data that predates - // any of them. It's ok to rewrite the first file, but - // only to max size. Then we should create a new one - - // points always come in time increasing order. This is - // the first point we've seen for this key. So it might - // need to get put into an older file instead of a new - // one. Check and set accordingly - var df *dataFile - for i := len(e.files) - 1; i >= 0; i-- { - if p.UnixNano() > e.files[i].MaxTime() { - break - } - df = e.files[i] - } - vals = &valueCollection{} - - if df == nil || (df == newestDataFile && overwriteNewestFile) { - // this point is newer than anything we have stored - // or it belongs in the most recent file, which should get - // rewritten - valuesByID[id] = vals - } else { - // it overlaps with another file so mark it and it can be compacted - dfm := dataFileToValues[df] - if dfm == nil { - dfm = make(map[uint64]*valueCollection) - dataFileToValues[df] = dfm - } - - if vc := dfm[id]; vc == nil { - dfm[id] = vals - } else { - vals = vc - } + // now see if the hash id collides with a different key + hashID := hashSeriesField(k) + existingKey, idInMap := idToKey[hashID] + if idInMap { + // we only care if the keys are different. if so, it's a hash collision we have to keep track of + if k != existingKey { + // we have a collision, give this new key a different id and move on + // TODO: handle collisions + panic("name collision, not implemented yet!") } - } - - switch t := val.(type) { - case float64: - vals.floatValues = append(vals.floatValues, FloatValue{Time: p.Time(), Value: t}) - case int64: - vals.intValues = append(vals.intValues, Int64Value{Time: p.Time(), Value: t}) - case bool: - vals.boolValues = append(vals.boolValues, BoolValue{Time: p.Time(), Value: t}) - case string: - vals.stringValues = append(vals.stringValues, StringValue{Time: p.Time(), Value: t}) - default: - panic("unsupported type") + } else { + newKeys = true + ids[k] = hashID + idToKey[id] = k + id = hashID } } + + valuesByID[id] = values } - b, err = json.Marshal(names) - if err != nil { - return err + if newKeys { + b, err := json.Marshal(ids) + if err != nil { + return err + } + if err := e.replaceCompressedFile("ids", b); err != nil { + return err + } } - if err := e.replaceCompressedFile("names", b); err != nil { - return err + // TODO: handle values written in the past that force an old data file to get rewritten + + // we keep track of the newest data file and if it should be + // rewritten with new data. + var newestDataFile *dataFile + overwriteNewestFile := false + if len(e.files) > 0 { + newestDataFile = e.files[len(e.files)-1] + overwriteNewestFile = newestDataFile.size < DefaultMaxFileSize } // flush values by id to either a new file or rewrite the old one @@ -330,21 +304,14 @@ func (e *Engine) WriteAndCompact(points []tsdb.Point, measurementFieldsToSave ma return err } - // flush each of the old ones - for df, vals := range dataFileToValues { - if err := e.rewriteFile(df, vals); err != nil { - return err - } - } - return nil } -func (e *Engine) rewriteFile(oldDF *dataFile, values map[uint64]*valueCollection) error { +func (e *Engine) rewriteFile(oldDF *dataFile, valuesByID map[uint64]Values) error { // we need the values in sorted order so that we can merge them into the // new file as we read the old file - ids := make([]uint64, 0, len(values)) - for id, _ := range values { + ids := make([]uint64, 0, len(valuesByID)) + for id, _ := range valuesByID { ids = append(ids, id) } @@ -358,18 +325,18 @@ func (e *Engine) rewriteFile(oldDF *dataFile, values map[uint64]*valueCollection minTime = oldDF.MinTime() maxTime = oldDF.MaxTime() } - for _, v := range values { - if minTime > v.MinTime().UnixNano() { - minTime = v.MinTime().UnixNano() + for _, v := range valuesByID { + if minTime > v.MinTime() { + minTime = v.MinTime() } - if maxTime < v.MaxTime().UnixNano() { - maxTime = v.MaxTime().UnixNano() + if maxTime < v.MaxTime() { + maxTime = v.MaxTime() } } // add any ids that are in the file that aren't getting flushed here for id, _ := range oldIDToPosition { - if _, ok := values[id]; !ok { + if _, ok := valuesByID[id]; !ok { ids = append(ids, id) } } @@ -414,10 +381,10 @@ func (e *Engine) rewriteFile(oldDF *dataFile, values map[uint64]*valueCollection // mark the position for this ID newPositions[i] = currentPosition - newVals := values[id] + newVals := valuesByID[id] // if this id is only in the file and not in the new values, just copy over from old file - if newVals == nil { + if len(newVals) == 0 { fpos := oldIDToPosition[id] // write the blocks until we hit whatever the next id is @@ -482,7 +449,8 @@ func (e *Engine) rewriteFile(oldDF *dataFile, values map[uint64]*valueCollection } } - newBlock, err := newVals.DecodeAndCombine(block, buf[:0], nextTime, hasFutureBlock) + nv, newBlock, err := e.DecodeAndCombine(newVals, block, buf[:0], nextTime, hasFutureBlock) + newVals = nv if err != nil { return err } @@ -503,7 +471,7 @@ func (e *Engine) rewriteFile(oldDF *dataFile, values map[uint64]*valueCollection } // TODO: ensure we encode only the amount in a block, refactor this wil line 450 into func - if len(newVals.floatValues) > 0 { + if len(newVals) > 0 { // TODO: ensure we encode only the amount in a block block := newVals.Encode(buf) if _, err := f.Write(append(u64tob(id), u32tob(uint32(len(block)))...)); err != nil { @@ -820,96 +788,39 @@ func (e *Engine) readSeries() (map[string]*tsdb.Series, error) { return series, nil } -type valueCollection struct { - floatValues []FloatValue - boolValues []BoolValue - intValues []Int64Value - stringValues []StringValue -} - -func (v *valueCollection) MinTime() time.Time { - if v.floatValues != nil { - return v.floatValues[0].Time - } else if v.boolValues != nil { - return v.boolValues[0].Time - } else if v.intValues != nil { - return v.intValues[0].Time - } else if v.stringValues != nil { - return v.stringValues[0].Time - } - - return time.Unix(0, 0) -} - -func (v *valueCollection) MaxTime() time.Time { - if v.floatValues != nil { - return v.floatValues[len(v.floatValues)-1].Time - } else if v.boolValues != nil { - return v.boolValues[len(v.boolValues)-1].Time - } else if v.intValues != nil { - return v.intValues[len(v.intValues)-1].Time - } else if v.stringValues != nil { - return v.stringValues[len(v.stringValues)-1].Time - } - - return time.Unix(0, 0) -} - -func (v *valueCollection) Encode(buf []byte) []byte { - if v.floatValues != nil { - return EncodeFloatBlock(buf, v.floatValues) - } else if v.boolValues != nil { - return EncodeBoolBlock(buf, v.boolValues) - } else if v.intValues != nil { - return EncodeInt64Block(buf, v.intValues) - } else if v.stringValues != nil { - return EncodeStringBlock(buf, v.stringValues) - } - - return nil -} - // DecodeAndCombine take an encoded block from a file, decodes it and interleaves the file -// values with the values in this collection. nextTime and hasNext refer to if the file +// values with the values passed in. nextTime and hasNext refer to if the file // has future encoded blocks so that this method can know how much of its values can be // combined and output in the resulting encoded block. -func (v *valueCollection) DecodeAndCombine(block, buf []byte, nextTime int64, hasFutureBlock bool) ([]byte, error) { - if v.floatValues != nil { - values, err := DecodeFloatBlock(block) - if err != nil { - return nil, err - } - - if hasFutureBlock { - // take all values that have times less than the future block and update the vals array - pos := sort.Search(len(v.floatValues), func(i int) bool { - return v.floatValues[i].Time.UnixNano() >= nextTime - }) - values = append(values, v.floatValues[:pos]...) - v.floatValues = v.floatValues[pos:] - } else { - values = append(values, v.floatValues...) - v.floatValues = nil +func (e *Engine) DecodeAndCombine(newValues Values, block, buf []byte, nextTime int64, hasFutureBlock bool) (Values, []byte, error) { + values := newValues.DecodeSameTypeBlock(block) + + var remainingValues Values + + if hasFutureBlock { + // take all values that have times less than the future block and update the vals array + pos := sort.Search(len(newValues), func(i int) bool { + return newValues[i].Time().UnixNano() >= nextTime + }) + values = append(values, newValues[:pos]...) + remainingValues = newValues[pos:] + sort.Sort(values) + } else { + requireSort := values.MaxTime() > newValues.MinTime() + values = append(values, newValues...) + if requireSort { + sort.Sort(values) } - sort.Sort(FloatValues(values)) - // TODO: deduplicate values + } - if len(values) > DefaultMaxPointsPerBlock { - v.floatValues = values[DefaultMaxPointsPerBlock:] - values = values[:DefaultMaxPointsPerBlock] - } + // TODO: deduplicate values - return EncodeFloatBlock(buf, values), nil - } else if v.boolValues != nil { - // TODO: wire up the other value types - return nil, fmt.Errorf("not implemented") - } else if v.intValues != nil { - return nil, fmt.Errorf("not implemented") - } else if v.stringValues != nil { - return nil, fmt.Errorf("not implemented") + if len(values) > DefaultMaxPointsPerBlock { + remainingValues = values[DefaultMaxPointsPerBlock:] + values = values[:DefaultMaxPointsPerBlock] } - return nil, nil + return remainingValues, values.Encode(buf), nil } type dataFile struct { @@ -1040,7 +951,7 @@ type cursor struct { f *dataFile filesPos int // the index in the files slice we're looking at pos uint32 - vals FloatValues + vals Values direction tsdb.Direction @@ -1121,7 +1032,7 @@ func (c *cursor) Seek(seek []byte) (key, value []byte) { // wasn't in the first value popped out of the block, check the rest for i, v := range c.vals { - if v.Time.UnixNano() >= t { + if v.Time().UnixNano() >= t { c.vals = c.vals[i+1:] return v.TimeBytes(), v.ValueBytes() } @@ -1220,180 +1131,3 @@ type uint64slice []uint64 func (a uint64slice) Len() int { return len(a) } func (a uint64slice) Swap(i, j int) { a[i], a[j] = a[j], a[i] } func (a uint64slice) Less(i, j int) bool { return a[i] < a[j] } - -/* TODO: REMOVE THIS STUFF */ -func (e *Engine) pointsToBlocks(points [][]byte) []byte { - var b bytes.Buffer - block := make([]byte, 0) - for _, p := range points { - block = append(block, p[0:8]...) - block = append(block, u32tob(uint32(len(p)-8))...) - block = append(block, p[8:]...) - if len(block) > DefaultBlockSize { - e.writeBlockToBuffer(block, &b) - block = make([]byte, 0) - } - } - if len(block) > 0 { - e.writeBlockToBuffer(block, &b) - } - - return b.Bytes() -} - -func (e *Engine) writeBlockToBuffer(block []byte, b *bytes.Buffer) { - // write the min time - if _, err := b.Write(block[0:8]); err != nil { - panic(err) - } - - // write the length of the compressed data - data := snappy.Encode(nil, block) - if _, err := b.Write(u32tob(uint32(len(data)))); err != nil { - panic(err) - } - - // write the compressed data - if _, err := b.Write(data); err != nil { - panic(err) - } -} - -func (e *Engine) readPointsFromFile(f *os.File) (map[uint64][][]byte, error) { - buf := make([]byte, 8) - if _, err := io.ReadFull(f, buf); err != nil { - return nil, err - } - seriesCount := btou64(buf) - positions := make([]uint64, seriesCount, seriesCount) - ids := make([]uint64, seriesCount, seriesCount) - - // read the series index file header - position := uint64(8) - for i := 0; uint64(i) < seriesCount; i++ { - // read the id of the series - if _, err := io.ReadFull(f, buf); err != nil { - return nil, err - } - ids[i] = btou64(buf) - - // read the min time and ignore - if _, err := io.ReadFull(f, buf); err != nil { - return nil, err - } - if _, err := io.ReadFull(f, buf); err != nil { - return nil, err - } - - // read the starting position of this id - if _, err := io.ReadFull(f, buf); err != nil { - return nil, err - } - positions[i] = btou64(buf) - position += 32 - } - - if position != positions[0] { - panic("we aren't at the right place") - } - - // read the raw data - seriesData := make(map[uint64][][]byte) - compressedBuff := make([]byte, DefaultBlockSize) - seriesPosition := 0 - for { - // read the min time and ignore - if _, err := io.ReadFull(f, buf); err == io.EOF { - break - } else if err != nil { - return nil, err - } - - // read the length of the compressed block - if _, err := io.ReadFull(f, buf[:4]); err != nil { - return nil, err - } - length := btou32(buf) - - if length > uint32(len(compressedBuff)) { - compressedBuff = make([]byte, length) - } - if _, err := io.ReadFull(f, compressedBuff[:length]); err != nil { - return nil, err - } - - data, err := snappy.Decode(nil, compressedBuff[:length]) - if err != nil { - return nil, err - } - id := ids[seriesPosition] - seriesData[id] = append(seriesData[id], e.pointsFromDataBlock(data)...) - position += uint64(12 + length) - - if seriesPosition+1 >= len(positions) { - continue - } - if positions[seriesPosition+1] == position { - seriesPosition += 1 - } - } - - return seriesData, nil -} - -func (e *Engine) pointsFromDataBlock(data []byte) [][]byte { - a := make([][]byte, 0) - for { - length := entryDataSize(data) - p := append(data[:8], data[12:12+length]...) - a = append(a, p) - data = data[12+length:] - if len(data) == 0 { - break - } - } - return a -} - -func entryDataSize(v []byte) int { return int(binary.BigEndian.Uint32(v[8:12])) } - -func (e *Engine) lastFileAndNewFile() (*os.File, *os.File, error) { - files, err := filepath.Glob(filepath.Join(e.path, fmt.Sprintf("*.%s", Format))) - if err != nil { - return nil, nil, err - } - - if len(files) == 0 { - newFile, err := os.OpenFile(filepath.Join(e.path, fmt.Sprintf("%07d.%s", 1, Format)), os.O_CREATE|os.O_RDWR, 0666) - if err != nil { - return nil, nil, err - } - return nil, newFile, nil - } - - oldFile, err := os.OpenFile(files[len(files)-1], os.O_RDONLY, 0666) - if err != nil { - return nil, nil, err - } - - info, err := oldFile.Stat() - if err != nil { - _ = oldFile.Close() - return nil, nil, err - } - - num := strings.Split(filepath.Base(files[len(files)-1]), ".")[0] - n, err := strconv.ParseUint(num, 10, 32) - if err != nil { - return nil, nil, err - } - newFile, err := os.OpenFile(filepath.Join(e.path, fmt.Sprintf("%07d.%s", n+1, Format)), os.O_CREATE|os.O_RDWR, 0666) - if err != nil { - return nil, nil, err - } - if info.Size() >= DefaultMaxFileSize { - oldFile.Close() - return nil, newFile, nil - } - return oldFile, newFile, nil -} diff --git a/tsdb/engine/pd1/pd1_test.go b/tsdb/engine/pd1/pd1_test.go index 7d915aab69e..79817eb531d 100644 --- a/tsdb/engine/pd1/pd1_test.go +++ b/tsdb/engine/pd1/pd1_test.go @@ -25,7 +25,7 @@ func TestEngine_WriteAndReadFloats(t *testing.T) { p3 := parsePoint("cpu,host=A value=2.1 2000000000") p4 := parsePoint("cpu,host=B value=2.2 2000000000") - if err := e.WriteAndCompact([]tsdb.Point{p1, p2, p3}, nil, nil); err != nil { + if err := e.WritePoints([]tsdb.Point{p1, p2, p3}, nil, nil); err != nil { t.Fatalf("failed to write points: %s", err.Error()) } @@ -68,7 +68,7 @@ func TestEngine_WriteAndReadFloats(t *testing.T) { } verify(true) - if err := e.WriteAndCompact([]tsdb.Point{p4}, nil, nil); err != nil { + if err := e.WritePoints([]tsdb.Point{p4}, nil, nil); err != nil { t.Fatalf("failed to write points: %s", err.Error()) } verify(false) @@ -123,13 +123,13 @@ func TestEngine_WriteIndexBenchmarkNames(t *testing.T) { } st := time.Now() - if err := e.WriteAndCompact(points, nil, nil); err != nil { + if err := e.WritePoints(points, nil, nil); err != nil { t.Fatalf("failed to write points: %s", err.Error()) } fmt.Println("took: ", time.Since(st)) st = time.Now() - if err := e.WriteAndCompact(points, nil, nil); err != nil { + if err := e.WritePoints(points, nil, nil); err != nil { t.Fatalf("failed to write points: %s", err.Error()) } fmt.Println("took: ", time.Since(st)) @@ -161,6 +161,7 @@ func OpenEngine(opt tsdb.EngineOptions) *Engine { if err := e.Open(); err != nil { panic(err) } + e.WAL.SkipCache = true return e } diff --git a/tsdb/engine/pd1/wal.go b/tsdb/engine/pd1/wal.go new file mode 100644 index 00000000000..26f2af48ffe --- /dev/null +++ b/tsdb/engine/pd1/wal.go @@ -0,0 +1,581 @@ +package pd1 + +import ( + "bytes" + "fmt" + "io" + "log" + "os" + "path/filepath" + "sort" + "strconv" + "strings" + "sync" + "time" + + "github.com/golang/snappy" + "github.com/influxdb/influxdb/tsdb" +) + +const ( + // DefaultSegmentSize of 2MB is the size at which segment files will be rolled over + DefaultSegmentSize = 2 * 1024 * 1024 + + // FileExtension is the file extension we expect for wal segments + WALFileExtension = "wal" + + WALFilePrefix = "_" + + // defaultFlushCheckInterval is how often flushes are triggered automatically by the flush criteria + defaultFlushCheckInterval = time.Second +) + +// flushType indiciates why a flush and compaction are being run so the partition can +// do the appropriate type of compaction +type flushType int + +const ( + // noFlush indicates that no flush or compaction are necesssary at this time + noFlush flushType = iota + // memoryFlush indicates that we should look for the series using the most + // memory to flush out and compact all others + memoryFlush + // idleFlush indicates that we should flush all series in the parition, + // delete all segment files and hold off on opening a new one + idleFlush + // deleteFlush indicates that we're flushing because series need to be removed from the WAL + deleteFlush + + writeBufLen = 32 << 10 // 32kb +) + +// walEntry is a byte written to a wal segment file that indicates what the following compressed block contains +type walEntryType byte + +const ( + pointsEntry walEntryType = 0x01 + fieldsEntry walEntryType = 0x02 + seriesEntry walEntryType = 0x03 +) + +type Log struct { + path string + + flushCheckTimer *time.Timer // check this often to see if a background flush should happen + flushCheckInterval time.Duration + + // write variables + writeLock sync.Mutex + currentSegmentID int + currentSegmentFile *os.File + currentSegmentSize int + lastWriteTime time.Time + flushRunning bool + + // cache variables + cacheLock sync.RWMutex + cache map[string]Values + cacheDirtySort map[string]bool // this map should be small, only for dirty vals + flushCache map[string]Values // temporary map while flushing + memorySize int + measurementFieldsCache map[string]*tsdb.MeasurementFields + seriesToCreateCache []*tsdb.SeriesCreate + + // These coordinate closing and waiting for running goroutines. + wg sync.WaitGroup + closing chan struct{} + + // LogOutput is the writer used by the logger. + LogOutput io.Writer + logger *log.Logger + + // FlushColdInterval is the period of time after which a partition will do a + // full flush and compaction if it has been cold for writes. + FlushColdInterval time.Duration + + // SegmentSize is the file size at which a segment file will be rotated + SegmentSize int + + // MemorySizeThreshold specifies when the log should be forced to be flushed. + MemorySizeThreshold int + + // Index is the database series will be flushed to + Index IndexWriter + + // LoggingEnabled specifies if detailed logs should be output + LoggingEnabled bool + + // SkipCache specifies if the wal should immediately write to the index instead of + // caching data in memory. False by default so we buffer in memory before flushing to index. + SkipCache bool + + // SkipDurability specifies if the wal should not write the wal entries to disk. + // False by default which means all writes are durable even when cached before flushing to index. + SkipDurability bool +} + +// IndexWriter is an interface for the indexed database the WAL flushes data to +type IndexWriter interface { + WriteAndCompact(valuesByKey map[string]Values, measurementFieldsToSave map[string]*tsdb.MeasurementFields, seriesToCreate []*tsdb.SeriesCreate) error +} + +func NewLog(path string) *Log { + return &Log{ + path: path, + + // these options should be overriden by any options in the config + LogOutput: os.Stderr, + FlushColdInterval: tsdb.DefaultFlushColdInterval, + SegmentSize: DefaultSegmentSize, + MemorySizeThreshold: tsdb.DefaultPartitionSizeThreshold, + flushCheckInterval: defaultFlushCheckInterval, + logger: log.New(os.Stderr, "[pwl] ", log.LstdFlags), + } +} + +// Open opens and initializes the Log. Will recover from previous unclosed shutdowns +func (l *Log) Open() error { + + if l.LoggingEnabled { + l.logger.Printf("PD1 WAL starting with %d memory size threshold\n", l.MemorySizeThreshold) + l.logger.Printf("WAL writing to %s\n", l.path) + } + if err := os.MkdirAll(l.path, 0777); err != nil { + return err + } + + l.cache = make(map[string]Values) + l.cacheDirtySort = make(map[string]bool) + l.measurementFieldsCache = make(map[string]*tsdb.MeasurementFields) + // TODO: read segment files and flush them all to disk + + l.flushCheckTimer = time.NewTimer(l.flushCheckInterval) + + // Start background goroutines. + l.wg.Add(1) + l.closing = make(chan struct{}) + go l.autoflusher(l.closing) + + return nil +} + +// Cursor will return a cursor object to Seek and iterate with Next for the WAL cache for the given +func (l *Log) Cursor(key string, direction tsdb.Direction) tsdb.Cursor { + l.cacheLock.RLock() + defer l.cacheLock.RUnlock() + + // TODO: make this work for other fields + ck := seriesFieldKey(key, "value") + values := l.cache[ck] + + // if we're in the middle of a flush, combine the previous cache + // with this one for the cursor + if l.flushCache != nil { + if fc, ok := l.flushCache[ck]; ok { + c := make([]Value, len(fc), len(fc)+len(values)) + copy(c, fc) + c = append(c, values...) + + return newWALCursor(c, direction) + } + } + + if l.cacheDirtySort[ck] { + sort.Sort(values) + delete(l.cacheDirtySort, ck) + } + + // build a copy so writes afterwards don't change the result set + a := make([]Value, len(values)) + copy(a, values) + return newWALCursor(a, direction) +} + +func (l *Log) WritePoints(points []tsdb.Point, fields map[string]*tsdb.MeasurementFields, series []*tsdb.SeriesCreate) error { + // make the write durable if specified + if !l.SkipDurability { + pointStrings := make([]string, len(points)) + for i, p := range points { + pointStrings[i] = p.String() + } + data := strings.Join(pointStrings, "\n") + compressed := snappy.Encode(nil, []byte(data)) + + if err := l.writeToLog(pointsEntry, compressed); err != nil { + return err + } + + // TODO: write the fields + + // TODO: write the series + } + + // convert to values that can be either cached in memory or flushed to the index + l.cacheLock.Lock() + for _, p := range points { + for name, value := range p.Fields() { + k := seriesFieldKey(string(p.Key()), name) + v := NewValue(p.Time(), value) + cacheValues := l.cache[k] + + // only mark it as dirty if it isn't already + if _, ok := l.cacheDirtySort[k]; !ok && len(cacheValues) > 0 { + dirty := cacheValues[len(cacheValues)-1].Time().UnixNano() > v.Time().UnixNano() + if dirty { + l.cacheDirtySort[k] = true + } + } + l.memorySize += v.Size() + l.cache[k] = append(cacheValues, v) + } + } + + for k, v := range fields { + l.measurementFieldsCache[k] = v + } + l.seriesToCreateCache = append(l.seriesToCreateCache, series...) + l.lastWriteTime = time.Now() + l.cacheLock.Unlock() + + // usually skipping the cache is only for testing purposes and this was the easiest + // way to represent the logic (to cache and then immediately flush) + if l.SkipCache { + l.flush(idleFlush) + } + + return nil +} + +func (l *Log) writeToLog(writeType walEntryType, data []byte) error { + l.writeLock.Lock() + defer l.writeLock.Unlock() + + if l.currentSegmentFile == nil { + l.newSegmentFile() + } + + if _, err := l.currentSegmentFile.Write([]byte{byte(writeType)}); err != nil { + panic(fmt.Sprintf("error writing type to wal: %s", err.Error())) + } + if _, err := l.currentSegmentFile.Write(u32tob(uint32(len(data)))); err != nil { + panic(fmt.Sprintf("error writing len to wal: %s", err.Error())) + } + if _, err := l.currentSegmentFile.Write(data); err != nil { + panic(fmt.Sprintf("error writing data to wal: %s", err.Error())) + } + + return l.currentSegmentFile.Sync() +} + +// Flush will force a flush of the WAL to the index +func (l *Log) Flush() error { + return l.flush(idleFlush) +} + +func (l *Log) DeleteSeries(keys []string) error { + panic("not implemented") +} + +// Close will finish any flush that is currently in process and close file handles +func (l *Log) Close() error { + // stop the autoflushing process so it doesn't try to kick another one off + l.writeLock.Lock() + l.cacheLock.Lock() + + if l.closing != nil { + close(l.closing) + l.closing = nil + } + l.writeLock.Unlock() + l.cacheLock.Unlock() + + // Allow goroutines to finish running. + l.wg.Wait() + + // Lock the remainder of the closing process. + l.writeLock.Lock() + l.cacheLock.Lock() + defer l.writeLock.Unlock() + defer l.cacheLock.Unlock() + + l.cache = nil + l.measurementFieldsCache = nil + l.seriesToCreateCache = nil + if l.currentSegmentFile == nil { + return nil + } + if err := l.currentSegmentFile.Close(); err != nil { + return err + } + l.currentSegmentFile = nil + + return nil +} + +// close all the open Log partitions and file handles +func (l *Log) close() error { + l.cache = nil + l.cacheDirtySort = nil + if l.currentSegmentFile == nil { + return nil + } + if err := l.currentSegmentFile.Close(); err != nil { + return err + } + l.currentSegmentFile = nil + + return nil +} + +// flush writes all wal data in memory to the index +func (l *Log) flush(flush flushType) error { + l.writeLock.Lock() + if l.flushRunning { + l.writeLock.Unlock() + return nil + } + + l.flushRunning = true + defer func() { + l.writeLock.Lock() + l.flushRunning = false + l.writeLock.Unlock() + }() + lastFileID := l.currentSegmentID + if err := l.newSegmentFile(); err != nil { + // there's no recovering from this, fail hard + panic(fmt.Sprintf("error creating new wal file: %s", err.Error())) + } + l.writeLock.Unlock() + + // copy the cache items to new maps so we can empty them out + l.cacheLock.Lock() + + // move over the flush cache and make a copy to write + l.flushCache = l.cache + l.cache = make(map[string]Values) + l.cacheDirtySort = make(map[string]bool) + valuesByKey := make(map[string]Values) + + valueCount := 0 + for key, v := range l.flushCache { + valuesByKey[key] = v + valueCount += len(v) + } + + if l.LoggingEnabled { + ftype := "idle" + if flush == memoryFlush { + ftype = "memory" + } + l.logger.Printf("%s flush of %d keys with %d values of %d bytes\n", ftype, len(valuesByKey), valueCount, l.memorySize) + } + + // reset the memory being used by the cache + l.memorySize = 0 + + // reset the measurements for flushing + mfc := l.measurementFieldsCache + l.measurementFieldsCache = make(map[string]*tsdb.MeasurementFields) + + // reset the series for flushing + scc := l.seriesToCreateCache + l.seriesToCreateCache = nil + + l.cacheLock.Unlock() + + startTime := time.Now() + if err := l.Index.WriteAndCompact(valuesByKey, mfc, scc); err != nil { + return err + } + if l.LoggingEnabled { + l.logger.Printf("flush to index took %s\n", time.Since(startTime)) + } + + l.cacheLock.Lock() + l.flushCache = nil + l.cacheLock.Unlock() + + // remove all the old segment files + fileNames, err := l.segmentFileNames() + if err != nil { + return err + } + for _, fn := range fileNames { + id, err := idFromFileName(fn) + if err != nil { + return err + } + if id <= lastFileID { + err := os.Remove(fn) + if err != nil { + return err + } + } + } + + return nil +} + +// triggerAutoFlush will flush and compact any partitions that have hit the thresholds for compaction +func (l *Log) triggerAutoFlush() { + if f := l.shouldFlush(); f != noFlush { + if err := l.flush(f); err != nil { + l.logger.Printf("error flushing wal: %s\n", err) + } + } +} + +// autoflusher waits for notification of a flush and kicks it off in the background. +// This method runs in a separate goroutine. +func (l *Log) autoflusher(closing chan struct{}) { + defer l.wg.Done() + + for { + // Wait for close or flush signal. + select { + case <-closing: + return + case <-l.flushCheckTimer.C: + l.triggerAutoFlush() + l.flushCheckTimer.Reset(l.flushCheckInterval) + } + } +} + +// segmentFileNames will return all files that are WAL segment files in sorted order by ascending ID +func (l *Log) segmentFileNames() ([]string, error) { + names, err := filepath.Glob(filepath.Join(l.path, 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 *Log) newSegmentFile() error { + l.currentSegmentID += 1 + if l.currentSegmentFile != nil { + if err := l.currentSegmentFile.Close(); err != nil { + return err + } + } + + fileName := filepath.Join(l.path, fmt.Sprintf("%s%05d.%s", WALFilePrefix, l.currentSegmentID, WALFileExtension)) + ff, err := os.OpenFile(fileName, os.O_CREATE|os.O_RDWR, 0666) + if err != nil { + return err + } + l.currentSegmentSize = 0 + l.currentSegmentFile = ff + + return nil +} + +// shouldFlush +func (l *Log) shouldFlush() flushType { + l.cacheLock.RLock() + defer l.cacheLock.RUnlock() + + if len(l.cache) == 0 { + return noFlush + } + + if l.memorySize > l.MemorySizeThreshold { + return memoryFlush + } + + if time.Since(l.lastWriteTime) > l.FlushColdInterval { + return idleFlush + } + + return noFlush +} + +// cursor is a unidirectional iterator for a given entry in the cache +type walCursor struct { + cache Values + position int + direction tsdb.Direction +} + +func newWALCursor(cache Values, direction tsdb.Direction) *walCursor { + // position is set such that a call to Next will successfully advance + // to the next postion and return the value. + c := &walCursor{cache: cache, direction: direction, position: -1} + if direction.Reverse() { + c.position = len(c.cache) + } + return c +} + +func (c *walCursor) Direction() tsdb.Direction { return c.direction } + +// Seek will point the cursor to the given time (or key) +func (c *walCursor) Seek(seek []byte) (key, value []byte) { + // Seek cache index + c.position = sort.Search(len(c.cache), func(i int) bool { + return bytes.Compare(c.cache[i].TimeBytes(), seek) != -1 + }) + + // If seek is not in the cache, return the last value in the cache + if c.direction.Reverse() && c.position >= len(c.cache) { + c.position = len(c.cache) + } + + // Make sure our position points to something in the cache + if c.position < 0 || c.position >= len(c.cache) { + return nil, nil + } + + v := c.cache[c.position] + + return v.TimeBytes(), v.ValueBytes() +} + +// Next moves the cursor to the next key/value. will return nil if at the end +func (c *walCursor) Next() (key, value []byte) { + var v Value + if c.direction.Forward() { + v = c.nextForward() + } else { + v = c.nextReverse() + } + + return v.TimeBytes(), v.ValueBytes() +} + +// nextForward advances the cursor forward returning the next value +func (c *walCursor) nextForward() Value { + c.position++ + + if c.position >= len(c.cache) { + return &EmptyValue{} + } + + return c.cache[c.position] +} + +// nextReverse advances the cursor backwards returning the next value +func (c *walCursor) nextReverse() Value { + c.position-- + + if c.position < 0 { + return &EmptyValue{} + } + + return c.cache[c.position] +} + +// 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 +} From 318bc7281d74772293038c7f3028ed4a9355980f Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Thu, 17 Sep 2015 11:23:27 -0400 Subject: [PATCH 006/139] Add full durability to WAL and flush on startup --- tsdb/engine/pd1/wal.go | 184 ++++++++++++++++++++++++++++++++---- tsdb/engine/pd1/wal_test.go | 170 +++++++++++++++++++++++++++++++++ 2 files changed, 333 insertions(+), 21 deletions(-) create mode 100644 tsdb/engine/pd1/wal_test.go diff --git a/tsdb/engine/pd1/wal.go b/tsdb/engine/pd1/wal.go index 26f2af48ffe..03548f062e7 100644 --- a/tsdb/engine/pd1/wal.go +++ b/tsdb/engine/pd1/wal.go @@ -2,6 +2,7 @@ package pd1 import ( "bytes" + "encoding/json" "fmt" "io" "log" @@ -28,6 +29,8 @@ const ( // defaultFlushCheckInterval is how often flushes are triggered automatically by the flush criteria defaultFlushCheckInterval = time.Second + + writeBufLen = 32 << 10 // 32kb ) // flushType indiciates why a flush and compaction are being run so the partition can @@ -45,8 +48,8 @@ const ( idleFlush // deleteFlush indicates that we're flushing because series need to be removed from the WAL deleteFlush - - writeBufLen = 32 << 10 // 32kb + // startupFlush indicates that we're flushing because the database is starting up + startupFlush ) // walEntry is a byte written to a wal segment file that indicates what the following compressed block contains @@ -129,7 +132,7 @@ func NewLog(path string) *Log { SegmentSize: DefaultSegmentSize, MemorySizeThreshold: tsdb.DefaultPartitionSizeThreshold, flushCheckInterval: defaultFlushCheckInterval, - logger: log.New(os.Stderr, "[pwl] ", log.LstdFlags), + logger: log.New(os.Stderr, "[pd1wal] ", log.LstdFlags), } } @@ -138,7 +141,7 @@ func (l *Log) Open() error { if l.LoggingEnabled { l.logger.Printf("PD1 WAL starting with %d memory size threshold\n", l.MemorySizeThreshold) - l.logger.Printf("WAL writing to %s\n", l.path) + l.logger.Printf("PD1 WAL writing to %s\n", l.path) } if err := os.MkdirAll(l.path, 0777); err != nil { return err @@ -147,7 +150,11 @@ func (l *Log) Open() error { l.cache = make(map[string]Values) l.cacheDirtySort = make(map[string]bool) l.measurementFieldsCache = make(map[string]*tsdb.MeasurementFields) - // TODO: read segment files and flush them all to disk + + // flush out any WAL entries that are there from before + if err := l.readAndFlushWAL(); err != nil { + return err + } l.flushCheckTimer = time.NewTimer(l.flushCheckInterval) @@ -194,6 +201,7 @@ func (l *Log) Cursor(key string, direction tsdb.Direction) tsdb.Cursor { func (l *Log) WritePoints(points []tsdb.Point, fields map[string]*tsdb.MeasurementFields, series []*tsdb.SeriesCreate) error { // make the write durable if specified if !l.SkipDurability { + // write the points pointStrings := make([]string, len(points)) for i, p := range points { pointStrings[i] = p.String() @@ -205,13 +213,47 @@ func (l *Log) WritePoints(points []tsdb.Point, fields map[string]*tsdb.Measureme return err } - // TODO: write the fields + // write the new fields + if len(fields) > 0 { + data, err := json.Marshal(fields) + if err != nil { + return err + } + compressed = snappy.Encode(compressed, data) + if err := l.writeToLog(fieldsEntry, compressed); err != nil { + return err + } + } - // TODO: write the series + // write the new series + if len(series) > 0 { + data, err := json.Marshal(series) + if err != nil { + return err + } + compressed = snappy.Encode(compressed, data) + if err := l.writeToLog(seriesEntry, compressed); err != nil { + return err + } + } + } + + // add everything to the cache + l.addToCache(points, fields, series) + + // usually skipping the cache is only for testing purposes and this was the easiest + // way to represent the logic (to cache and then immediately flush) + if l.SkipCache { + l.flush(idleFlush) } - // convert to values that can be either cached in memory or flushed to the index + return nil +} + +func (l *Log) addToCache(points []tsdb.Point, fields map[string]*tsdb.MeasurementFields, series []*tsdb.SeriesCreate) { l.cacheLock.Lock() + defer l.cacheLock.Unlock() + for _, p := range points { for name, value := range p.Fields() { k := seriesFieldKey(string(p.Key()), name) @@ -235,25 +277,114 @@ func (l *Log) WritePoints(points []tsdb.Point, fields map[string]*tsdb.Measureme } l.seriesToCreateCache = append(l.seriesToCreateCache, series...) l.lastWriteTime = time.Now() - l.cacheLock.Unlock() +} - // usually skipping the cache is only for testing purposes and this was the easiest - // way to represent the logic (to cache and then immediately flush) - if l.SkipCache { - l.flush(idleFlush) +// readAndFlushWAL is called on open and will read the segment files in, flushing whenever +// the memory gets over the limit. Once all files have been read it will flush and remove the files +func (l *Log) readAndFlushWAL() error { + files, err := l.segmentFileNames() + if err != nil { + return err + } + + // read all the segment files and cache them, flushing along the way if we + // hit memory limits + for _, fn := range files { + if err := l.readFileToCache(fn); err != nil { + return err + } + + if l.memorySize > l.MemorySizeThreshold { + if err := l.flush(memoryFlush); err != nil { + return err + } + } + } + + // now flush and remove all the old files + if err := l.flush(startupFlush); err != nil { + return err } return nil } +func (l *Log) readFileToCache(fileName string) error { + f, err := os.OpenFile(fileName, os.O_RDONLY, 0666) + if err != nil { + return err + } + defer f.Close() + + buf := make([]byte, writeBufLen) + data := make([]byte, writeBufLen) + for { + // read the type and the length of the entry + _, err := io.ReadFull(f, buf[0:5]) + if err == io.EOF { + return nil + } else if err != nil { + l.logger.Printf("error reading segment file %s: %s", fileName, err.Error()) + return err + } + entryType := buf[0] + length := btou32(buf[1:5]) + + // read the compressed block and decompress it + if int(length) > len(buf) { + buf = make([]byte, length) + } + _, err = io.ReadFull(f, buf[0:length]) + if err == io.EOF { + l.logger.Printf("hit end of file while reading compressed wal entry from %s", fileName) + return nil + } else if err != nil { + return err + } + data, err = snappy.Decode(data, buf[0:length]) + if err != nil { + l.logger.Printf("error decoding compressed entry from %s: %s", fileName, err.Error()) + return nil + } + + // and marshal it and send it to the cache + switch walEntryType(entryType) { + case pointsEntry: + points, err := tsdb.ParsePoints(data) + if err != nil { + return err + } + l.addToCache(points, nil, nil) + case fieldsEntry: + fields := make(map[string]*tsdb.MeasurementFields) + if err := json.Unmarshal(data, &fields); err != nil { + return err + } + l.addToCache(nil, fields, nil) + case seriesEntry: + series := make([]*tsdb.SeriesCreate, 0) + if err := json.Unmarshal(data, &series); err != nil { + return err + } + l.addToCache(nil, nil, series) + } + } +} + func (l *Log) writeToLog(writeType walEntryType, data []byte) error { l.writeLock.Lock() defer l.writeLock.Unlock() if l.currentSegmentFile == nil { - l.newSegmentFile() + if err := l.newSegmentFile(); err != nil { + // fail hard since we can't write data + panic(fmt.Sprintf("error opening new segment file for wal: %s", err.Error())) + } } + // The panics here are an intentional choice. Based on reports from users + // it's better to fail hard if the database can't take writes. Then they'll + // get alerted and fix whatever is broken. Remove these and face Paul's wrath. if _, err := l.currentSegmentFile.Write([]byte{byte(writeType)}); err != nil { panic(fmt.Sprintf("error writing type to wal: %s", err.Error())) } @@ -329,12 +460,14 @@ func (l *Log) close() error { // flush writes all wal data in memory to the index func (l *Log) flush(flush flushType) error { + // only flush if there isn't one already running l.writeLock.Lock() if l.flushRunning { l.writeLock.Unlock() return nil } + // only hold the lock while we rotate the segment file l.flushRunning = true defer func() { l.writeLock.Lock() @@ -363,13 +496,7 @@ func (l *Log) flush(flush flushType) error { valueCount += len(v) } - if l.LoggingEnabled { - ftype := "idle" - if flush == memoryFlush { - ftype = "memory" - } - l.logger.Printf("%s flush of %d keys with %d values of %d bytes\n", ftype, len(valuesByKey), valueCount, l.memorySize) - } + flushSize := l.memorySize // reset the memory being used by the cache l.memorySize = 0 @@ -384,6 +511,21 @@ func (l *Log) flush(flush flushType) error { l.cacheLock.Unlock() + // exit if there's nothing to flush to the index + if len(valuesByKey) == 0 && len(mfc) == 0 && len(scc) == 0 { + return nil + } + + if l.LoggingEnabled { + ftype := "idle" + if flush == memoryFlush { + ftype = "memory" + } else if flush == startupFlush { + ftype = "startup" + } + l.logger.Printf("%s flush of %d keys with %d values of %d bytes\n", ftype, len(valuesByKey), valueCount, flushSize) + } + startTime := time.Now() if err := l.Index.WriteAndCompact(valuesByKey, mfc, scc); err != nil { return err diff --git a/tsdb/engine/pd1/wal_test.go b/tsdb/engine/pd1/wal_test.go new file mode 100644 index 00000000000..c1ef614650c --- /dev/null +++ b/tsdb/engine/pd1/wal_test.go @@ -0,0 +1,170 @@ +package pd1_test + +import ( + "io/ioutil" + "os" + "reflect" + "testing" + + "github.com/influxdb/influxdb/tsdb" + "github.com/influxdb/influxdb/tsdb/engine/pd1" +) + +func TestWAL_TestWriteQueryOpen(t *testing.T) { + w := NewWAL() + defer w.Cleanup() + + var vals map[string]pd1.Values + var fields map[string]*tsdb.MeasurementFields + var series []*tsdb.SeriesCreate + + w.Index = &MockIndexWriter{ + fn: func(valuesByKey map[string]pd1.Values, measurementFieldsToSave map[string]*tsdb.MeasurementFields, seriesToCreate []*tsdb.SeriesCreate) error { + vals = valuesByKey + fields = measurementFieldsToSave + series = seriesToCreate + return nil + }, + } + + if err := w.Open(); err != nil { + t.Fatalf("error opening: %s", err.Error) + } + + p1 := parsePoint("cpu,host=A value=1.1 1000000000") + p2 := parsePoint("cpu,host=B value=1.2 1000000000") + p3 := parsePoint("cpu,host=A value=2.1 2000000000") + p4 := parsePoint("cpu,host=B value=2.2 2000000000") + fieldsToWrite := map[string]*tsdb.MeasurementFields{"foo": {Fields: map[string]*tsdb.Field{"bar": {Name: "value"}}}} + seriesToWrite := []*tsdb.SeriesCreate{{Measurement: "asdf"}} + + if err := w.WritePoints([]tsdb.Point{p1, p2}, fieldsToWrite, seriesToWrite); err != nil { + t.Fatalf("failed to write points: %s", err.Error()) + } + + c := w.Cursor("cpu,host=A", tsdb.Forward) + k, v := c.Next() + if btou64(k) != uint64(p1.UnixNano()) { + t.Fatalf("p1 time wrong:\n\texp:%d\n\tgot:%d\n", p1.UnixNano(), btou64(k)) + } + if 1.1 != btof64(v) { + t.Fatal("p1 data not equal") + } + c = w.Cursor("cpu,host=B", tsdb.Forward) + k, v = c.Next() + if btou64(k) != uint64(p2.UnixNano()) { + t.Fatalf("p2 time wrong:\n\texp:%d\n\tgot:%d\n", p2.UnixNano(), btou64(k)) + } + if 1.2 != btof64(v) { + t.Fatal("p2 data not equal") + } + + k, v = c.Next() + if k != nil { + t.Fatal("expected nil") + } + + // ensure we can do another write to the wal and get stuff + if err := w.WritePoints([]tsdb.Point{p3}, nil, nil); err != nil { + t.Fatalf("failed to write: %s", err.Error) + } + + c = w.Cursor("cpu,host=A", tsdb.Forward) + k, v = c.Next() + if btou64(k) != uint64(p1.UnixNano()) { + t.Fatalf("p1 time wrong:\n\texp:%d\n\tgot:%d\n", p1.UnixNano(), btou64(k)) + } + if 1.1 != btof64(v) { + t.Fatal("p1 data not equal") + } + k, v = c.Next() + if btou64(k) != uint64(p3.UnixNano()) { + t.Fatalf("p3 time wrong:\n\texp:%d\n\tgot:%d\n", p3.UnixNano(), btou64(k)) + } + if 2.1 != btof64(v) { + t.Fatal("p3 data not equal") + } + + // ensure we can seek + k, v = c.Seek(u64tob(2000000000)) + if btou64(k) != uint64(p3.UnixNano()) { + t.Fatalf("p3 time wrong:\n\texp:%d\n\tgot:%d\n", p3.UnixNano(), btou64(k)) + } + if 2.1 != btof64(v) { + t.Fatal("p3 data not equal") + } + k, v = c.Next() + if k != nil { + t.Fatal("expected nil") + } + + // ensure we close and after open it flushes to the index + if err := w.Close(); err != nil { + t.Fatalf("failed to close: %s", err.Error()) + } + + if err := w.Open(); err != nil { + t.Fatalf("failed to open: %s", err.Error()) + } + + if len(vals["cpu,host=A#value"]) != 2 { + t.Fatal("expected host A values to flush to index on open") + } + + if len(vals["cpu,host=B#value"]) != 1 { + t.Fatal("expected host B values to flush to index on open") + } + + if err := w.WritePoints([]tsdb.Point{p4}, nil, nil); err != nil { + t.Fatalf("failed to write: %s", err.Error) + } + c = w.Cursor("cpu,host=B", tsdb.Forward) + k, v = c.Next() + if btou64(k) != uint64(p4.UnixNano()) { + t.Fatalf("p4 time wrong:\n\texp:%d\n\tgot:%d\n", p4.UnixNano(), btou64(k)) + } + if 2.2 != btof64(v) { + t.Fatal("p4 data not equal") + } + + if !reflect.DeepEqual(fields, fieldsToWrite) { + t.Fatal("fields not flushed") + } + + if !reflect.DeepEqual(series, seriesToWrite) { + t.Fatal("series not flushed") + } +} + +type Log struct { + *pd1.Log + path string +} + +func NewWAL() *Log { + dir, err := ioutil.TempDir("", "pd1-test") + if err != nil { + panic("couldn't get temp dir") + } + + l := &Log{ + Log: pd1.NewLog(dir), + path: dir, + } + l.LoggingEnabled = true + return l +} + +func (l *Log) Cleanup() error { + l.Close() + os.RemoveAll(l.path) + return nil +} + +type MockIndexWriter struct { + fn func(valuesByKey map[string]pd1.Values, measurementFieldsToSave map[string]*tsdb.MeasurementFields, seriesToCreate []*tsdb.SeriesCreate) error +} + +func (m *MockIndexWriter) WriteAndCompact(valuesByKey map[string]pd1.Values, measurementFieldsToSave map[string]*tsdb.MeasurementFields, seriesToCreate []*tsdb.SeriesCreate) error { + return m.fn(valuesByKey, measurementFieldsToSave, seriesToCreate) +} From 95f9e1619e44e3025ac1be4e96580256fa5d7966 Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Thu, 17 Sep 2015 15:46:37 -0400 Subject: [PATCH 007/139] Update encoding test to work with new interface. --- tsdb/engine/pd1/encoding_test.go | 31 +++++++++++++++---------------- 1 file changed, 15 insertions(+), 16 deletions(-) diff --git a/tsdb/engine/pd1/encoding_test.go b/tsdb/engine/pd1/encoding_test.go index aa5a4b15e73..82968912b0a 100644 --- a/tsdb/engine/pd1/encoding_test.go +++ b/tsdb/engine/pd1/encoding_test.go @@ -2,31 +2,30 @@ package pd1_test import ( // "math/rand" - // "reflect" + "fmt" + "reflect" "testing" "time" - // "github.com/influxdb/influxdb/tsdb/engine/pd1" + "github.com/influxdb/influxdb/tsdb/engine/pd1" ) func TestEncoding_FloatBlock(t *testing.T) { - // valueCount := 100 - // times := getTimes(valueCount, 60, time.Second) - // values := make([]Value, len(times)) - // for i, t := range times { - // values[i] = pd1.NewValue(t, rand.Float64()) - // } + valueCount := 1000 + times := getTimes(valueCount, 60, time.Second) + values := make(pd1.Values, len(times)) + for i, t := range times { + values[i] = pd1.NewValue(t, float64(i)) + } - // b := pd1.EncodeFloatBlock(nil, values) + b := values.Encode(nil) + fmt.Println("**** ", len(b)) - // decodedValues, err := pd1.DecodeFloatBlock(b) - // if err != nil { - // t.Fatalf("error decoding: %s", err.Error) - // } + decodedValues := values.DecodeSameTypeBlock(b) - // if !reflect.DeepEqual(decodedValues, values) { - // t.Fatalf("unexpected results:\n\tgot: %v\n\texp: %v\n", decodedValues, values) - // } + if !reflect.DeepEqual(decodedValues, values) { + t.Fatalf("unexpected results:\n\tgot: %v\n\texp: %v\n", decodedValues, values) + } } func getTimes(n, step int, precision time.Duration) []time.Time { From 750856836e5cd9d76c2deadec67005c88f663245 Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Thu, 17 Sep 2015 15:46:53 -0400 Subject: [PATCH 008/139] Add memory settings and WAL backpressure --- tsdb/config.go | 22 +++++++--- tsdb/engine/pd1/pd1.go | 3 +- tsdb/engine/pd1/wal.go | 92 ++++++++++++++++++++++++++---------------- 3 files changed, 76 insertions(+), 41 deletions(-) diff --git a/tsdb/config.go b/tsdb/config.go index dfd267d2c37..b7bc409b4a9 100644 --- a/tsdb/config.go +++ b/tsdb/config.go @@ -43,6 +43,10 @@ const ( // This number multiplied by the parition count is roughly the max possible memory // size for the in-memory WAL cache. DefaultPartitionSizeThreshold = 50 * 1024 * 1024 // 50MB + + // Default WAL settings for the PD1 WAL + DefaultFlushMemorySizeThreshold = 50 * 1024 * 1024 // 50MB + DefaultMaxMemorySizeThreshold = 200 * 1024 * 1024 // 200MB ) type Config struct { @@ -63,6 +67,10 @@ type Config struct { WALFlushColdInterval toml.Duration `toml:"wal-flush-cold-interval"` WALPartitionSizeThreshold uint64 `toml:"wal-partition-size-threshold"` + // WAL configuration options for pd1 introduced in 0.9.5 + WALFlushMemorySizeThreshold int `toml:"wal-flush-memory-size-threshold"` + WALMaxMemorySizeThreshold int `toml:"wal-max-memory-size-threshold"` + // Query logging QueryLogEnabled bool `toml:"query-log-enabled"` } @@ -74,12 +82,14 @@ func NewConfig() Config { WALFlushInterval: toml.Duration(DefaultWALFlushInterval), WALPartitionFlushDelay: toml.Duration(DefaultWALPartitionFlushDelay), - WALLoggingEnabled: true, - WALReadySeriesSize: DefaultReadySeriesSize, - WALCompactionThreshold: DefaultCompactionThreshold, - WALMaxSeriesSize: DefaultMaxSeriesSize, - WALFlushColdInterval: toml.Duration(DefaultFlushColdInterval), - WALPartitionSizeThreshold: DefaultPartitionSizeThreshold, + WALLoggingEnabled: true, + WALReadySeriesSize: DefaultReadySeriesSize, + WALCompactionThreshold: DefaultCompactionThreshold, + WALMaxSeriesSize: DefaultMaxSeriesSize, + WALFlushColdInterval: toml.Duration(DefaultFlushColdInterval), + WALPartitionSizeThreshold: DefaultPartitionSizeThreshold, + WALFlushMemorySizeThreshold: DefaultFlushMemorySizeThreshold, + WALMaxMemorySizeThreshold: DefaultMaxMemorySizeThreshold, QueryLogEnabled: true, } diff --git a/tsdb/engine/pd1/pd1.go b/tsdb/engine/pd1/pd1.go index ef91fb66e2d..02708d7b45f 100644 --- a/tsdb/engine/pd1/pd1.go +++ b/tsdb/engine/pd1/pd1.go @@ -89,7 +89,8 @@ type Engine struct { func NewEngine(path string, walPath string, opt tsdb.EngineOptions) tsdb.Engine { w := NewLog(path) w.FlushColdInterval = time.Duration(opt.Config.WALFlushColdInterval) - w.MemorySizeThreshold = int(opt.Config.WALPartitionSizeThreshold) + w.FlushMemorySizeThreshold = opt.Config.WALFlushMemorySizeThreshold + w.MaxMemorySizeThreshold = opt.Config.WALMaxMemorySizeThreshold w.LoggingEnabled = opt.Config.WALLoggingEnabled e := &Engine{ diff --git a/tsdb/engine/pd1/wal.go b/tsdb/engine/pd1/wal.go index 03548f062e7..e3157a94b03 100644 --- a/tsdb/engine/pd1/wal.go +++ b/tsdb/engine/pd1/wal.go @@ -72,11 +72,11 @@ type Log struct { currentSegmentID int currentSegmentFile *os.File currentSegmentSize int - lastWriteTime time.Time - flushRunning bool - // cache variables + // cache and flush variables cacheLock sync.RWMutex + lastWriteTime time.Time + flushRunning bool cache map[string]Values cacheDirtySort map[string]bool // this map should be small, only for dirty vals flushCache map[string]Values // temporary map while flushing @@ -99,8 +99,11 @@ type Log struct { // SegmentSize is the file size at which a segment file will be rotated SegmentSize int - // MemorySizeThreshold specifies when the log should be forced to be flushed. - MemorySizeThreshold int + // FlushMemorySizeThreshold specifies when the log should be forced to be flushed + FlushMemorySizeThreshold int + + // MaxMemorySizeThreshold specifies the limit at which writes to the WAL should be rejected + MaxMemorySizeThreshold int // Index is the database series will be flushed to Index IndexWriter @@ -127,12 +130,13 @@ func NewLog(path string) *Log { path: path, // these options should be overriden by any options in the config - LogOutput: os.Stderr, - FlushColdInterval: tsdb.DefaultFlushColdInterval, - SegmentSize: DefaultSegmentSize, - MemorySizeThreshold: tsdb.DefaultPartitionSizeThreshold, - flushCheckInterval: defaultFlushCheckInterval, - logger: log.New(os.Stderr, "[pd1wal] ", log.LstdFlags), + LogOutput: os.Stderr, + FlushColdInterval: tsdb.DefaultFlushColdInterval, + SegmentSize: DefaultSegmentSize, + FlushMemorySizeThreshold: tsdb.DefaultFlushMemorySizeThreshold, + MaxMemorySizeThreshold: tsdb.DefaultMaxMemorySizeThreshold, + flushCheckInterval: defaultFlushCheckInterval, + logger: log.New(os.Stderr, "[pd1wal] ", log.LstdFlags), } } @@ -140,7 +144,7 @@ func NewLog(path string) *Log { func (l *Log) Open() error { if l.LoggingEnabled { - l.logger.Printf("PD1 WAL starting with %d memory size threshold\n", l.MemorySizeThreshold) + l.logger.Printf("PD1 WAL starting with %d flush memory size threshold and %d max memory size threshold\n", l.FlushMemorySizeThreshold, l.MaxMemorySizeThreshold) l.logger.Printf("PD1 WAL writing to %s\n", l.path) } if err := os.MkdirAll(l.path, 0777); err != nil { @@ -199,6 +203,11 @@ func (l *Log) Cursor(key string, direction tsdb.Direction) tsdb.Cursor { } func (l *Log) WritePoints(points []tsdb.Point, fields map[string]*tsdb.MeasurementFields, series []*tsdb.SeriesCreate) error { + // add everything to the cache, or return an error if we've hit our max memory + if addedToCache := l.addToCache(points, fields, series, true); !addedToCache { + return fmt.Errorf("WAL backed up flushing to index, hit max memory") + } + // make the write durable if specified if !l.SkipDurability { // write the points @@ -238,9 +247,6 @@ func (l *Log) WritePoints(points []tsdb.Point, fields map[string]*tsdb.Measureme } } - // add everything to the cache - l.addToCache(points, fields, series) - // usually skipping the cache is only for testing purposes and this was the easiest // way to represent the logic (to cache and then immediately flush) if l.SkipCache { @@ -250,10 +256,23 @@ func (l *Log) WritePoints(points []tsdb.Point, fields map[string]*tsdb.Measureme return nil } -func (l *Log) addToCache(points []tsdb.Point, fields map[string]*tsdb.MeasurementFields, series []*tsdb.SeriesCreate) { +// addToCache will add the points, measurements, and fields to the cache and return true if successful. They will be queryable +// immediately after return and will be flushed at the next flush cycle. Before adding to the cache we check if we're over the +// max memory threshold. If we are we request a flush in a new goroutine and return false, indicating we didn't add the values +// to the cache and that writes should return a failure. +func (l *Log) addToCache(points []tsdb.Point, fields map[string]*tsdb.MeasurementFields, series []*tsdb.SeriesCreate, checkMemory bool) bool { l.cacheLock.Lock() defer l.cacheLock.Unlock() + // if we should check memory and we're over the threshold, mark a flush as running and kick one off in a goroutine + if checkMemory && l.memorySize > l.MaxMemorySizeThreshold { + if !l.flushRunning { + l.flushRunning = true + go l.flush(memoryFlush) + } + return false + } + for _, p := range points { for name, value := range p.Fields() { k := seriesFieldKey(string(p.Key()), name) @@ -277,6 +296,8 @@ func (l *Log) addToCache(points []tsdb.Point, fields map[string]*tsdb.Measuremen } l.seriesToCreateCache = append(l.seriesToCreateCache, series...) l.lastWriteTime = time.Now() + + return true } // readAndFlushWAL is called on open and will read the segment files in, flushing whenever @@ -294,7 +315,7 @@ func (l *Log) readAndFlushWAL() error { return err } - if l.memorySize > l.MemorySizeThreshold { + if l.memorySize > l.MaxMemorySizeThreshold { if err := l.flush(memoryFlush); err != nil { return err } @@ -354,19 +375,19 @@ func (l *Log) readFileToCache(fileName string) error { if err != nil { return err } - l.addToCache(points, nil, nil) + l.addToCache(points, nil, nil, false) case fieldsEntry: fields := make(map[string]*tsdb.MeasurementFields) if err := json.Unmarshal(data, &fields); err != nil { return err } - l.addToCache(nil, fields, nil) + l.addToCache(nil, fields, nil, false) case seriesEntry: series := make([]*tsdb.SeriesCreate, 0) if err := json.Unmarshal(data, &series); err != nil { return err } - l.addToCache(nil, nil, series) + l.addToCache(nil, nil, series, false) } } } @@ -460,20 +481,24 @@ func (l *Log) close() error { // flush writes all wal data in memory to the index func (l *Log) flush(flush flushType) error { - // only flush if there isn't one already running - l.writeLock.Lock() - if l.flushRunning { - l.writeLock.Unlock() + // only flush if there isn't one already running. Memory flushes are only triggered + // by writes, which will mark the flush as running, so we can ignore it. + l.cacheLock.Lock() + if l.flushRunning && flush != memoryFlush { + l.cacheLock.Unlock() return nil } - // only hold the lock while we rotate the segment file + // mark the flush as running and ensure that it gets marked as not running when we return l.flushRunning = true defer func() { - l.writeLock.Lock() + l.cacheLock.Lock() l.flushRunning = false - l.writeLock.Unlock() + l.cacheLock.Unlock() }() + + // only hold the lock while we rotate the segment file + l.writeLock.Lock() lastFileID := l.currentSegmentID if err := l.newSegmentFile(); err != nil { // there's no recovering from this, fail hard @@ -482,9 +507,6 @@ func (l *Log) flush(flush flushType) error { l.writeLock.Unlock() // copy the cache items to new maps so we can empty them out - l.cacheLock.Lock() - - // move over the flush cache and make a copy to write l.flushCache = l.cache l.cache = make(map[string]Values) l.cacheDirtySort = make(map[string]bool) @@ -561,6 +583,7 @@ func (l *Log) flush(flush flushType) error { // triggerAutoFlush will flush and compact any partitions that have hit the thresholds for compaction func (l *Log) triggerAutoFlush() { + // if f := l.shouldFlush(); f != noFlush { if err := l.flush(f); err != nil { l.logger.Printf("error flushing wal: %s\n", err) @@ -615,17 +638,18 @@ func (l *Log) newSegmentFile() error { return nil } -// shouldFlush +// shouldFlush will return the flushType specifying whether we should flush. memoryFlush +// is never returned from this function since those can only be triggered by writes func (l *Log) shouldFlush() flushType { l.cacheLock.RLock() defer l.cacheLock.RUnlock() - if len(l.cache) == 0 { + if l.flushRunning { return noFlush } - if l.memorySize > l.MemorySizeThreshold { - return memoryFlush + if len(l.cache) == 0 { + return noFlush } if time.Since(l.lastWriteTime) > l.FlushColdInterval { From 38f9b2992553d4c744836476fec1bd34de13ca66 Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Thu, 17 Sep 2015 18:25:55 -0400 Subject: [PATCH 009/139] Update engine to put index at the end of data files --- tsdb/engine/pd1/pd1.go | 76 ++++++++++++++++++++---------------------- 1 file changed, 37 insertions(+), 39 deletions(-) diff --git a/tsdb/engine/pd1/pd1.go b/tsdb/engine/pd1/pd1.go index 02708d7b45f..b16561776b6 100644 --- a/tsdb/engine/pd1/pd1.go +++ b/tsdb/engine/pd1/pd1.go @@ -58,6 +58,8 @@ const ( // MAP_POPULATE is for the mmap syscall. For some reason this isn't defined in golang's syscall MAP_POPULATE = 0x8000 + + magicNumber uint32 = 0x16D116D1 ) // Ensure Engine implements the interface. @@ -308,6 +310,8 @@ func (e *Engine) WriteAndCompact(pointsByKey map[string]Values, measurementField return nil } +// rewriteFile will read in the old data file, if provided and merge the values +// in the passed map into a new data file func (e *Engine) rewriteFile(oldDF *dataFile, valuesByID map[uint64]Values) error { // we need the values in sorted order so that we can merge them into the // new file as we read the old file @@ -351,31 +355,15 @@ func (e *Engine) rewriteFile(oldDF *dataFile, valuesByID map[uint64]Values) erro return err } - // write the header of the file and keep track of the current file position - currentPosition := uint32(4) - // series count - if _, err := f.Write(u32tob(uint32(len(ids)))); err != nil { + // write the magic number + if _, err := f.Write(u32tob(magicNumber)); err != nil { f.Close() return err } - // empty min time and max time - currentPosition += 16 - if _, err := f.Write([]byte{0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00}); err != nil { - f.Close() - return nil - } - - // write the series ids and empty starting positions - for _, id := range ids { - if _, err := f.Write(append(u64tob(id), []byte{0x00, 0x00, 0x00, 0x00}...)); err != nil { - f.Close() - return err - } - currentPosition += 12 - } // now combine the old file data with the new values, keeping track of // their positions + currentPosition := uint32(4) newPositions := make([]uint32, len(ids)) buf := make([]byte, DefaultMaxPointsPerBlock*20) for i, id := range ids { @@ -487,25 +475,31 @@ func (e *Engine) rewriteFile(oldDF *dataFile, valuesByID map[uint64]Values) erro } } - // write out the times and positions - if _, err := f.Seek(4, 0); err != nil { - f.Close() - return err + // write the file index, starting with the series ids and their positions + for i, id := range ids { + if _, err := f.Write(u64tob(id)); err != nil { + f.Close() + return err + } + if _, err := f.Write(u32tob(newPositions[i])); err != nil { + f.Close() + return err + } } + + // write the min time, max time if _, err := f.Write(append(u64tob(uint64(minTime)), u64tob(uint64(maxTime))...)); err != nil { f.Close() return err } - for _, pos := range newPositions { - if _, err := f.Seek(8, 1); err != nil { - f.Close() - return err - } - if _, err := f.Write(u32tob(pos)); err != nil { - return err - } + // series count + if _, err := f.Write(u32tob(uint32(len(ids)))); err != nil { + f.Close() + return err } + + // sync it and see4k back to the beginning to hand off to the mmap if err := f.Sync(); err != nil { return err } @@ -888,23 +882,24 @@ func (d *dataFile) close() error { } func (d *dataFile) MinTime() int64 { - return int64(btou64(d.mmap[4:12])) + return int64(btou64(d.mmap[d.size-20 : d.size-12])) } func (d *dataFile) MaxTime() int64 { - return int64(btou64(d.mmap[12:20])) + return int64(btou64(d.mmap[d.size-12 : d.size-4])) } func (d *dataFile) SeriesCount() uint32 { - return btou32(d.mmap[:4]) + return btou32(d.mmap[d.size-4:]) } func (d *dataFile) IDToPosition() map[uint64]uint32 { count := int(d.SeriesCount()) m := make(map[uint64]uint32) + indexStart := d.size - uint32(count*12+20) for i := 0; i < count; i++ { - offset := 20 + (i * 12) + offset := indexStart + uint32(i*12) id := btou64(d.mmap[offset : offset+8]) pos := btou32(d.mmap[offset+8 : offset+12]) m[id] = pos @@ -917,15 +912,17 @@ func (d *dataFile) IDToPosition() map[uint64]uint32 { // first block for the given ID. If zero is returned the ID doesn't // have any data in this file. func (d *dataFile) StartingPositionForID(id uint64) uint32 { + seriesCount := d.SeriesCount() + indexStart := d.size - uint32(seriesCount*12+20) - min := 0 - max := int(seriesCount) + min := uint32(0) + max := uint32(seriesCount) for min < max { mid := (max-min)/2 + min - offset := mid*seriesHeaderSize + fileHeaderSize + offset := mid*seriesHeaderSize + indexStart checkID := btou64(d.mmap[offset : offset+8]) if checkID == id { @@ -1066,6 +1063,7 @@ func (c *cursor) Next() (key, value []byte) { startingPos := f.StartingPositionForID(c.id) if startingPos == 0 { + c.filesPos++ continue } c.f = f @@ -1119,7 +1117,7 @@ func btou32(b []byte) uint32 { func hashSeriesField(key string) uint64 { h := fnv.New64a() h.Write([]byte(key)) - return h.Sum64() + return h.Sum64() % 100 } // seriesFieldKey combine a series key and field name for a unique string to be hashed to a numeric ID From 1b57b80fdb78d971f9bb9caeb837833c78de8dc5 Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Fri, 18 Sep 2015 11:08:03 -0400 Subject: [PATCH 010/139] Add test for close and restart of engine and fix errors. --- tsdb/engine/pd1/pd1.go | 14 ++++++++++++-- tsdb/engine/pd1/pd1_test.go | 19 +++++++++++++++---- 2 files changed, 27 insertions(+), 6 deletions(-) diff --git a/tsdb/engine/pd1/pd1.go b/tsdb/engine/pd1/pd1.go index b16561776b6..df930a6124f 100644 --- a/tsdb/engine/pd1/pd1.go +++ b/tsdb/engine/pd1/pd1.go @@ -70,6 +70,10 @@ type Engine struct { mu sync.Mutex path string + // deletesPending mark how many old data files are waiting to be deleted. This will + // keep a close from returning until all deletes finish + deletesPending sync.WaitGroup + // HashSeriesField is a function that takes a series key and a field name // and returns a hash identifier. It's not guaranteed to be unique. HashSeriesField func(key string) uint64 @@ -157,9 +161,13 @@ func (e *Engine) Close() error { e.queryLock.Lock() defer e.queryLock.Unlock() + e.deletesPending.Wait() + for _, df := range e.files { _ = df.Close() } + e.files = nil + e.currentFileID = 0 return nil } @@ -530,11 +538,13 @@ func (e *Engine) rewriteFile(oldDF *dataFile, valuesByID map[uint64]Values) erro // remove the old data file. no need to block returning the write, // but we need to let any running queries finish before deleting it if oldDF != nil { - go func(df *dataFile) { + e.deletesPending.Add(1) + go func() { if err := oldDF.Delete(); err != nil { // TODO: log this error } - }(oldDF) + e.deletesPending.Done() + }() } return nil diff --git a/tsdb/engine/pd1/pd1_test.go b/tsdb/engine/pd1/pd1_test.go index 79817eb531d..0ed7c368c5d 100644 --- a/tsdb/engine/pd1/pd1_test.go +++ b/tsdb/engine/pd1/pd1_test.go @@ -16,7 +16,7 @@ import ( func TestEngine_WriteAndReadFloats(t *testing.T) { e := OpenDefaultEngine() - defer e.Close() + defer e.Cleanup() e.Shard = newFieldCodecMock(map[string]influxql.DataType{"value": influxql.Float}) @@ -47,6 +47,7 @@ func TestEngine_WriteAndReadFloats(t *testing.T) { } k, v = c.Next() if k != nil { + fmt.Println(btou64(k), btof64(v)) t.Fatal("expected nil") } @@ -106,6 +107,16 @@ func TestEngine_WriteAndReadFloats(t *testing.T) { if 1.1 != btof64(v) { t.Fatal("p1 data not equal") } + + if err := e.Close(); err != nil { + t.Fatalf("error closing: %s", err.Error()) + } + + if err := e.Open(); err != nil { + t.Fatalf("error opening: %s", err.Error()) + } + + verify(false) } func TestEngine_WriteIndexWithCollision(t *testing.T) { @@ -115,7 +126,7 @@ func TestEngine_WriteIndexBenchmarkNames(t *testing.T) { t.Skip("whatevs") e := OpenDefaultEngine() - defer e.Close() + defer e.Cleanup() var points []tsdb.Point for i := 0; i < 100000; i++ { @@ -168,8 +179,8 @@ func OpenEngine(opt tsdb.EngineOptions) *Engine { // OpenDefaultEngine returns an open Engine with default options. func OpenDefaultEngine() *Engine { return OpenEngine(tsdb.NewEngineOptions()) } -// Close closes the engine and removes all data. -func (e *Engine) Close() error { +// Cleanup closes the engine and removes all data. +func (e *Engine) Cleanup() error { e.Engine.Close() os.RemoveAll(e.Path()) return nil From ea85f8042d4410e65e995b6203e249b3c89f8eac Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Fri, 18 Sep 2015 15:18:05 -0400 Subject: [PATCH 011/139] Update wal to only open new segment file on flush if its not an idle flush --- tsdb/engine/pd1/wal.go | 17 ++++++++++++++--- 1 file changed, 14 insertions(+), 3 deletions(-) diff --git a/tsdb/engine/pd1/wal.go b/tsdb/engine/pd1/wal.go index e3157a94b03..610fe74d31c 100644 --- a/tsdb/engine/pd1/wal.go +++ b/tsdb/engine/pd1/wal.go @@ -500,9 +500,20 @@ func (l *Log) flush(flush flushType) error { // only hold the lock while we rotate the segment file l.writeLock.Lock() lastFileID := l.currentSegmentID - if err := l.newSegmentFile(); err != nil { - // there's no recovering from this, fail hard - panic(fmt.Sprintf("error creating new wal file: %s", err.Error())) + // if it's an idle flush, don't open a new segment file + if flush == idleFlush { + if l.currentSegmentFile != nil { + if err := l.currentSegmentFile.Close(); err != nil { + return err + } + l.currentSegmentFile = nil + l.currentSegmentSize = 0 + } + } else { + if err := l.newSegmentFile(); err != nil { + // there's no recovering from this, fail hard + panic(fmt.Sprintf("error creating new wal file: %s", err.Error())) + } } l.writeLock.Unlock() From ed7055146a0be60a9ff26d8a59fe864f2dc2225d Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Fri, 25 Sep 2015 10:49:26 -0400 Subject: [PATCH 012/139] Update to work with new cursor definitiono and Point in models --- tsdb/config.go | 2 +- tsdb/engine.go | 3 -- tsdb/engine/pd1/encoding.go | 3 +- tsdb/engine/pd1/pd1.go | 49 +++++++++++------------ tsdb/engine/pd1/pd1_test.go | 79 +++++++++++++++++++------------------ tsdb/engine/pd1/wal.go | 45 ++++++++++----------- tsdb/engine/pd1/wal_test.go | 64 ++++++++++++++++-------------- 7 files changed, 125 insertions(+), 120 deletions(-) diff --git a/tsdb/config.go b/tsdb/config.go index b7bc409b4a9..4aec4a14e52 100644 --- a/tsdb/config.go +++ b/tsdb/config.go @@ -8,7 +8,7 @@ import ( const ( // DefaultEngine is the default engine for new shards - DefaultEngine = "bz1" + DefaultEngine = "pd1" // DefaultMaxWALSize is the default size of the WAL before it is flushed. DefaultMaxWALSize = 100 * 1024 * 1024 // 100MB diff --git a/tsdb/engine.go b/tsdb/engine.go index b0e9254d95a..d2862b54863 100644 --- a/tsdb/engine.go +++ b/tsdb/engine.go @@ -18,9 +18,6 @@ var ( ErrFormatNotFound = errors.New("format not found") ) -// DefaultEngine is the default engine used by the shard when initializing. -const DefaultEngine = "pd1" - // Engine represents a swappable storage engine for the shard. type Engine interface { Open() error diff --git a/tsdb/engine/pd1/encoding.go b/tsdb/engine/pd1/encoding.go index 7262e8a6c36..cad991aa7a8 100644 --- a/tsdb/engine/pd1/encoding.go +++ b/tsdb/engine/pd1/encoding.go @@ -6,6 +6,7 @@ import ( "time" "github.com/dgryski/go-tsz" + "github.com/influxdb/influxdb/tsdb" ) type Value interface { @@ -35,7 +36,7 @@ type EmptyValue struct { func (e *EmptyValue) TimeBytes() []byte { return nil } func (e *EmptyValue) ValueBytes() []byte { return nil } -func (e *EmptyValue) Time() time.Time { return time.Unix(0, 0) } +func (e *EmptyValue) Time() time.Time { return time.Unix(0, tsdb.EOF) } func (e *EmptyValue) Value() interface{} { return nil } func (e *EmptyValue) Size() int { return 0 } diff --git a/tsdb/engine/pd1/pd1.go b/tsdb/engine/pd1/pd1.go index df930a6124f..1428f04b45d 100644 --- a/tsdb/engine/pd1/pd1.go +++ b/tsdb/engine/pd1/pd1.go @@ -17,6 +17,7 @@ import ( "github.com/golang/snappy" "github.com/influxdb/influxdb/influxql" + "github.com/influxdb/influxdb/models" "github.com/influxdb/influxdb/tsdb" ) @@ -217,7 +218,7 @@ func (e *Engine) LoadMetadataIndex(shard *tsdb.Shard, index *tsdb.DatabaseIndex, // WritePoints writes metadata and point data into the engine. // Returns an error if new points are added to an existing key. -func (e *Engine) WritePoints(points []tsdb.Point, measurementFieldsToSave map[string]*tsdb.MeasurementFields, seriesToCreate []*tsdb.SeriesCreate) error { +func (e *Engine) WritePoints(points []models.Point, measurementFieldsToSave map[string]*tsdb.MeasurementFields, seriesToCreate []*tsdb.SeriesCreate) error { return e.WAL.WritePoints(points, measurementFieldsToSave, seriesToCreate) } @@ -614,7 +615,7 @@ func (e *Engine) Begin(writable bool) (tsdb.Tx, error) { } // TODO: make the cursor take a field name -func (e *Engine) Cursor(series string, direction tsdb.Direction) tsdb.Cursor { +func (e *Engine) Cursor(series string, fields []string, dec *tsdb.FieldCodec, ascending bool) tsdb.Cursor { measurementName := tsdb.MeasurementFromSeriesKey(series) codec := e.Shard.FieldCodec(measurementName) if codec == nil { @@ -627,7 +628,7 @@ func (e *Engine) Cursor(series string, direction tsdb.Direction) tsdb.Cursor { // TODO: ensure we map the collisions id := hashSeriesField(seriesFieldKey(series, field.Name)) - return newCursor(id, field.Type, e.copyFilesCollection(), direction) + return newCursor(id, field.Type, e.copyFilesCollection(), ascending) } func (e *Engine) copyFilesCollection() []*dataFile { @@ -961,30 +962,28 @@ type cursor struct { pos uint32 vals Values - direction tsdb.Direction + ascending bool // time acending list of data files files []*dataFile } -func newCursor(id uint64, dataType influxql.DataType, files []*dataFile, direction tsdb.Direction) *cursor { +func newCursor(id uint64, dataType influxql.DataType, files []*dataFile, ascending bool) *cursor { return &cursor{ id: id, dataType: dataType, - direction: direction, + ascending: ascending, files: files, } } -func (c *cursor) Seek(seek []byte) (key, value []byte) { - t := int64(btou64(seek)) - - if t < c.files[0].MinTime() { +func (c *cursor) SeekTo(seek int64) (int64, interface{}) { + if seek < c.files[0].MinTime() { c.filesPos = 0 c.f = c.files[0] } else { for i, f := range c.files { - if t >= f.MinTime() && t <= f.MaxTime() { + if seek >= f.MinTime() && seek <= f.MaxTime() { c.filesPos = i c.f = f break @@ -993,7 +992,7 @@ func (c *cursor) Seek(seek []byte) (key, value []byte) { } if c.f == nil { - return nil, nil + return tsdb.EOF, nil } // TODO: make this for the reverse direction cursor @@ -1006,7 +1005,7 @@ func (c *cursor) Seek(seek []byte) (key, value []byte) { if pos == 0 { c.filesPos++ if c.filesPos >= len(c.files) { - return nil, nil + return tsdb.EOF, nil } c.f = c.files[c.filesPos] continue @@ -1025,7 +1024,7 @@ func (c *cursor) Seek(seek []byte) (key, value []byte) { nextBlockID := btou64(c.f.mmap[nextBlockPos : nextBlockPos+8]) if nextBlockID == c.id { nextBlockTime := int64(btou64(c.f.mmap[nextBlockPos+12 : nextBlockPos+20])) - if nextBlockTime <= t { + if nextBlockTime <= seek { pos = nextBlockPos continue } @@ -1033,16 +1032,16 @@ func (c *cursor) Seek(seek []byte) (key, value []byte) { } // it must be in this block or not at all - tb, vb := c.decodeBlockAndGetValues(pos) - if int64(btou64(tb)) >= t { - return tb, vb + t, v := c.decodeBlockAndGetValues(pos) + if t >= seek { + return t, v } // wasn't in the first value popped out of the block, check the rest for i, v := range c.vals { - if v.Time().UnixNano() >= t { + if v.Time().UnixNano() >= seek { c.vals = c.vals[i+1:] - return v.TimeBytes(), v.ValueBytes() + return v.Time().UnixNano(), v.Value() } } @@ -1052,7 +1051,7 @@ func (c *cursor) Seek(seek []byte) (key, value []byte) { } } -func (c *cursor) Next() (key, value []byte) { +func (c *cursor) Next() (int64, interface{}) { if len(c.vals) == 0 { // if we have a file set, see if the next block is for this ID if c.f != nil && c.pos < c.f.size { @@ -1081,16 +1080,16 @@ func (c *cursor) Next() (key, value []byte) { } // we didn't get to a file that had a next value - return nil, nil + return tsdb.EOF, nil } v := c.vals[0] c.vals = c.vals[1:] - return v.TimeBytes(), v.ValueBytes() + return v.Time().UnixNano(), v.Value() } -func (c *cursor) decodeBlockAndGetValues(position uint32) ([]byte, []byte) { +func (c *cursor) decodeBlockAndGetValues(position uint32) (int64, interface{}) { length := btou32(c.f.mmap[position+8 : position+12]) block := c.f.mmap[position+12 : position+12+length] c.vals, _ = DecodeFloatBlock(block) @@ -1098,10 +1097,10 @@ func (c *cursor) decodeBlockAndGetValues(position uint32) ([]byte, []byte) { v := c.vals[0] c.vals = c.vals[1:] - return v.TimeBytes(), v.ValueBytes() + return v.Time().UnixNano(), v.Value() } -func (c *cursor) Direction() tsdb.Direction { return c.direction } +func (c *cursor) Ascending() bool { return c.ascending } // u64tob converts a uint64 into an 8-byte slice. func u64tob(v uint64) []byte { diff --git a/tsdb/engine/pd1/pd1_test.go b/tsdb/engine/pd1/pd1_test.go index 0ed7c368c5d..1a8f8613fc3 100644 --- a/tsdb/engine/pd1/pd1_test.go +++ b/tsdb/engine/pd1/pd1_test.go @@ -10,6 +10,7 @@ import ( "time" "github.com/influxdb/influxdb/influxql" + "github.com/influxdb/influxdb/models" "github.com/influxdb/influxdb/tsdb" "github.com/influxdb/influxdb/tsdb/engine/pd1" ) @@ -25,86 +26,88 @@ func TestEngine_WriteAndReadFloats(t *testing.T) { p3 := parsePoint("cpu,host=A value=2.1 2000000000") p4 := parsePoint("cpu,host=B value=2.2 2000000000") - if err := e.WritePoints([]tsdb.Point{p1, p2, p3}, nil, nil); err != nil { + if err := e.WritePoints([]models.Point{p1, p2, p3}, nil, nil); err != nil { t.Fatalf("failed to write points: %s", err.Error()) } + fields := []string{"value"} + var codec *tsdb.FieldCodec + verify := func(checkSingleBVal bool) { - c := e.Cursor("cpu,host=A", tsdb.Forward) + c := e.Cursor("cpu,host=A", fields, codec, true) k, v := c.Next() - if btou64(k) != uint64(p1.UnixNano()) { - t.Fatalf("p1 time wrong:\n\texp:%d\n\tgot:%d\n", p1.UnixNano(), btou64(k)) + if k != p1.UnixNano() { + t.Fatalf("p1 time wrong:\n\texp:%d\n\tgot:%d\n", p1.UnixNano(), k) } - if 1.1 != btof64(v) { + if 1.1 != v { t.Fatal("p1 data not equal") } k, v = c.Next() - if btou64(k) != uint64(p3.UnixNano()) { - t.Fatalf("p3 time wrong:\n\texp:%d\n\tgot:%d\n", p3.UnixNano(), btou64(k)) + if k != p3.UnixNano() { + t.Fatalf("p3 time wrong:\n\texp:%d\n\tgot:%d\n", p3.UnixNano(), k) } - if 2.1 != btof64(v) { + if 2.1 != v { t.Fatal("p3 data not equal") } k, v = c.Next() - if k != nil { - fmt.Println(btou64(k), btof64(v)) - t.Fatal("expected nil") + if k != tsdb.EOF { + t.Fatal("expected EOF") } - c = e.Cursor("cpu,host=B", tsdb.Forward) + c = e.Cursor("cpu,host=B", fields, codec, true) k, v = c.Next() - if btou64(k) != uint64(p2.UnixNano()) { - t.Fatalf("p2 time wrong:\n\texp:%d\n\tgot:%d\n", p2.UnixNano(), btou64(k)) + if k != p2.UnixNano() { + t.Fatalf("p2 time wrong:\n\texp:%d\n\tgot:%d\n", p2.UnixNano(), k) } - if 1.2 != btof64(v) { + if 1.2 != v { t.Fatal("p2 data not equal") } if checkSingleBVal { k, v = c.Next() - if k != nil { - t.Fatal("expected nil") + if k != tsdb.EOF { + t.Fatal("expected EOF") } } } verify(true) - if err := e.WritePoints([]tsdb.Point{p4}, nil, nil); err != nil { + if err := e.WritePoints([]models.Point{p4}, nil, nil); err != nil { t.Fatalf("failed to write points: %s", err.Error()) } verify(false) - c := e.Cursor("cpu,host=B", tsdb.Forward) + c := e.Cursor("cpu,host=B", fields, codec, true) k, v := c.Next() - if btou64(k) != uint64(p2.UnixNano()) { - t.Fatalf("p2 time wrong:\n\texp:%d\n\tgot:%d\n", p2.UnixNano(), btou64(k)) + if k != p2.UnixNano() { + t.Fatalf("p2 time wrong:\n\texp:%d\n\tgot:%d\n", p2.UnixNano(), k) } - if 1.2 != btof64(v) { + if 1.2 != v { t.Fatal("p2 data not equal") } k, v = c.Next() - if btou64(k) != uint64(p4.UnixNano()) { - t.Fatalf("p2 time wrong:\n\texp:%d\n\tgot:%d\n", p2.UnixNano(), btou64(k)) + if k != p4.UnixNano() { + t.Fatalf("p2 time wrong:\n\texp:%d\n\tgot:%d\n", p2.UnixNano(), k) } - if 2.2 != btof64(v) { + if 2.2 != v { t.Fatal("p2 data not equal") } // verify we can seek - k, v = c.Seek(u64tob(2000000000)) - if btou64(k) != uint64(p4.UnixNano()) { - t.Fatalf("p2 time wrong:\n\texp:%d\n\tgot:%d\n", p2.UnixNano(), btou64(k)) + k, v = c.SeekTo(2000000000) + if k != p4.UnixNano() { + t.Fatalf("p2 time wrong:\n\texp:%d\n\tgot:%d\n", p2.UnixNano(), k) } - if 2.2 != btof64(v) { + if 2.2 != v { t.Fatal("p2 data not equal") } - c = e.Cursor("cpu,host=A", tsdb.Forward) - k, v = c.Seek(u64tob(0)) - if btou64(k) != uint64(p1.UnixNano()) { - t.Fatalf("p1 time wrong:\n\texp:%d\n\tgot:%d\n", p1.UnixNano(), btou64(k)) + c = e.Cursor("cpu,host=A", fields, codec, true) + k, v = c.SeekTo(0) + if k != p1.UnixNano() { + t.Fatalf("p1 time wrong:\n\texp:%d\n\tgot:%d\n", p1.UnixNano(), k) } - if 1.1 != btof64(v) { + if 1.1 != v { t.Fatal("p1 data not equal") } @@ -128,7 +131,7 @@ func TestEngine_WriteIndexBenchmarkNames(t *testing.T) { e := OpenDefaultEngine() defer e.Cleanup() - var points []tsdb.Point + var points []models.Point for i := 0; i < 100000; i++ { points = append(points, parsePoint(fmt.Sprintf("cpu%d value=22.1", i))) } @@ -205,15 +208,15 @@ func (f *FieldCodeMock) FieldCodec(m string) *tsdb.FieldCodec { return f.codec } -func parsePoints(buf string) []tsdb.Point { - points, err := tsdb.ParsePointsString(buf) +func parsePoints(buf string) []models.Point { + points, err := models.ParsePointsString(buf) if err != nil { panic(fmt.Sprintf("couldn't parse points: %s", err.Error())) } return points } -func parsePoint(buf string) tsdb.Point { +func parsePoint(buf string) models.Point { return parsePoints(buf)[0] } diff --git a/tsdb/engine/pd1/wal.go b/tsdb/engine/pd1/wal.go index 610fe74d31c..7b538fed3a0 100644 --- a/tsdb/engine/pd1/wal.go +++ b/tsdb/engine/pd1/wal.go @@ -1,7 +1,6 @@ package pd1 import ( - "bytes" "encoding/json" "fmt" "io" @@ -14,8 +13,10 @@ import ( "sync" "time" - "github.com/golang/snappy" + "github.com/influxdb/influxdb/models" "github.com/influxdb/influxdb/tsdb" + + "github.com/golang/snappy" ) const ( @@ -171,12 +172,12 @@ func (l *Log) Open() error { } // Cursor will return a cursor object to Seek and iterate with Next for the WAL cache for the given -func (l *Log) Cursor(key string, direction tsdb.Direction) tsdb.Cursor { +func (l *Log) Cursor(series string, fields []string, dec *tsdb.FieldCodec, ascending bool) tsdb.Cursor { l.cacheLock.RLock() defer l.cacheLock.RUnlock() // TODO: make this work for other fields - ck := seriesFieldKey(key, "value") + ck := seriesFieldKey(series, "value") values := l.cache[ck] // if we're in the middle of a flush, combine the previous cache @@ -187,7 +188,7 @@ func (l *Log) Cursor(key string, direction tsdb.Direction) tsdb.Cursor { copy(c, fc) c = append(c, values...) - return newWALCursor(c, direction) + return newWALCursor(c, ascending) } } @@ -199,10 +200,10 @@ func (l *Log) Cursor(key string, direction tsdb.Direction) tsdb.Cursor { // build a copy so writes afterwards don't change the result set a := make([]Value, len(values)) copy(a, values) - return newWALCursor(a, direction) + return newWALCursor(a, ascending) } -func (l *Log) WritePoints(points []tsdb.Point, fields map[string]*tsdb.MeasurementFields, series []*tsdb.SeriesCreate) error { +func (l *Log) WritePoints(points []models.Point, fields map[string]*tsdb.MeasurementFields, series []*tsdb.SeriesCreate) error { // add everything to the cache, or return an error if we've hit our max memory if addedToCache := l.addToCache(points, fields, series, true); !addedToCache { return fmt.Errorf("WAL backed up flushing to index, hit max memory") @@ -260,7 +261,7 @@ func (l *Log) WritePoints(points []tsdb.Point, fields map[string]*tsdb.Measureme // immediately after return and will be flushed at the next flush cycle. Before adding to the cache we check if we're over the // max memory threshold. If we are we request a flush in a new goroutine and return false, indicating we didn't add the values // to the cache and that writes should return a failure. -func (l *Log) addToCache(points []tsdb.Point, fields map[string]*tsdb.MeasurementFields, series []*tsdb.SeriesCreate, checkMemory bool) bool { +func (l *Log) addToCache(points []models.Point, fields map[string]*tsdb.MeasurementFields, series []*tsdb.SeriesCreate, checkMemory bool) bool { l.cacheLock.Lock() defer l.cacheLock.Unlock() @@ -371,7 +372,7 @@ func (l *Log) readFileToCache(fileName string) error { // and marshal it and send it to the cache switch walEntryType(entryType) { case pointsEntry: - points, err := tsdb.ParsePoints(data) + points, err := models.ParsePoints(data) if err != nil { return err } @@ -674,53 +675,53 @@ func (l *Log) shouldFlush() flushType { type walCursor struct { cache Values position int - direction tsdb.Direction + ascending bool } -func newWALCursor(cache Values, direction tsdb.Direction) *walCursor { +func newWALCursor(cache Values, ascending bool) *walCursor { // position is set such that a call to Next will successfully advance // to the next postion and return the value. - c := &walCursor{cache: cache, direction: direction, position: -1} - if direction.Reverse() { + c := &walCursor{cache: cache, ascending: ascending, position: -1} + if !ascending { c.position = len(c.cache) } return c } -func (c *walCursor) Direction() tsdb.Direction { return c.direction } +func (c *walCursor) Ascending() bool { return c.ascending } // Seek will point the cursor to the given time (or key) -func (c *walCursor) Seek(seek []byte) (key, value []byte) { +func (c *walCursor) SeekTo(seek int64) (int64, interface{}) { // Seek cache index c.position = sort.Search(len(c.cache), func(i int) bool { - return bytes.Compare(c.cache[i].TimeBytes(), seek) != -1 + return c.cache[i].Time().UnixNano() >= seek }) // If seek is not in the cache, return the last value in the cache - if c.direction.Reverse() && c.position >= len(c.cache) { + if !c.ascending && c.position >= len(c.cache) { c.position = len(c.cache) } // Make sure our position points to something in the cache if c.position < 0 || c.position >= len(c.cache) { - return nil, nil + return tsdb.EOF, nil } v := c.cache[c.position] - return v.TimeBytes(), v.ValueBytes() + return v.Time().UnixNano(), v.Value() } // Next moves the cursor to the next key/value. will return nil if at the end -func (c *walCursor) Next() (key, value []byte) { +func (c *walCursor) Next() (int64, interface{}) { var v Value - if c.direction.Forward() { + if c.ascending { v = c.nextForward() } else { v = c.nextReverse() } - return v.TimeBytes(), v.ValueBytes() + return v.Time().UnixNano(), v.Value() } // nextForward advances the cursor forward returning the next value diff --git a/tsdb/engine/pd1/wal_test.go b/tsdb/engine/pd1/wal_test.go index c1ef614650c..de03dfbd601 100644 --- a/tsdb/engine/pd1/wal_test.go +++ b/tsdb/engine/pd1/wal_test.go @@ -6,6 +6,7 @@ import ( "reflect" "testing" + "github.com/influxdb/influxdb/models" "github.com/influxdb/influxdb/tsdb" "github.com/influxdb/influxdb/tsdb/engine/pd1" ) @@ -38,64 +39,67 @@ func TestWAL_TestWriteQueryOpen(t *testing.T) { fieldsToWrite := map[string]*tsdb.MeasurementFields{"foo": {Fields: map[string]*tsdb.Field{"bar": {Name: "value"}}}} seriesToWrite := []*tsdb.SeriesCreate{{Measurement: "asdf"}} - if err := w.WritePoints([]tsdb.Point{p1, p2}, fieldsToWrite, seriesToWrite); err != nil { + if err := w.WritePoints([]models.Point{p1, p2}, fieldsToWrite, seriesToWrite); err != nil { t.Fatalf("failed to write points: %s", err.Error()) } - c := w.Cursor("cpu,host=A", tsdb.Forward) + fieldNames := []string{"value"} + var codec *tsdb.FieldCodec + + c := w.Cursor("cpu,host=A", fieldNames, codec, true) k, v := c.Next() - if btou64(k) != uint64(p1.UnixNano()) { - t.Fatalf("p1 time wrong:\n\texp:%d\n\tgot:%d\n", p1.UnixNano(), btou64(k)) + if k != p1.UnixNano() { + t.Fatalf("p1 time wrong:\n\texp:%d\n\tgot:%d\n", p1.UnixNano(), k) } - if 1.1 != btof64(v) { + if 1.1 != v { t.Fatal("p1 data not equal") } - c = w.Cursor("cpu,host=B", tsdb.Forward) + c = w.Cursor("cpu,host=B", fieldNames, codec, true) k, v = c.Next() - if btou64(k) != uint64(p2.UnixNano()) { - t.Fatalf("p2 time wrong:\n\texp:%d\n\tgot:%d\n", p2.UnixNano(), btou64(k)) + if k != p2.UnixNano() { + t.Fatalf("p2 time wrong:\n\texp:%d\n\tgot:%d\n", p2.UnixNano(), k) } - if 1.2 != btof64(v) { + if 1.2 != v { t.Fatal("p2 data not equal") } k, v = c.Next() - if k != nil { - t.Fatal("expected nil") + if k != tsdb.EOF { + t.Fatal("expected EOF", k, v) } // ensure we can do another write to the wal and get stuff - if err := w.WritePoints([]tsdb.Point{p3}, nil, nil); err != nil { + if err := w.WritePoints([]models.Point{p3}, nil, nil); err != nil { t.Fatalf("failed to write: %s", err.Error) } - c = w.Cursor("cpu,host=A", tsdb.Forward) + c = w.Cursor("cpu,host=A", fieldNames, codec, true) k, v = c.Next() - if btou64(k) != uint64(p1.UnixNano()) { - t.Fatalf("p1 time wrong:\n\texp:%d\n\tgot:%d\n", p1.UnixNano(), btou64(k)) + if k != p1.UnixNano() { + t.Fatalf("p1 time wrong:\n\texp:%d\n\tgot:%d\n", p1.UnixNano(), k) } - if 1.1 != btof64(v) { + if 1.1 != v { t.Fatal("p1 data not equal") } k, v = c.Next() - if btou64(k) != uint64(p3.UnixNano()) { - t.Fatalf("p3 time wrong:\n\texp:%d\n\tgot:%d\n", p3.UnixNano(), btou64(k)) + if k != p3.UnixNano() { + t.Fatalf("p3 time wrong:\n\texp:%d\n\tgot:%d\n", p3.UnixNano(), k) } - if 2.1 != btof64(v) { + if 2.1 != v { t.Fatal("p3 data not equal") } // ensure we can seek - k, v = c.Seek(u64tob(2000000000)) - if btou64(k) != uint64(p3.UnixNano()) { - t.Fatalf("p3 time wrong:\n\texp:%d\n\tgot:%d\n", p3.UnixNano(), btou64(k)) + k, v = c.SeekTo(2000000000) + if k != p3.UnixNano() { + t.Fatalf("p3 time wrong:\n\texp:%d\n\tgot:%d\n", p3.UnixNano(), k) } - if 2.1 != btof64(v) { + if 2.1 != v { t.Fatal("p3 data not equal") } k, v = c.Next() - if k != nil { - t.Fatal("expected nil") + if k != tsdb.EOF { + t.Fatal("expected EOF") } // ensure we close and after open it flushes to the index @@ -115,15 +119,15 @@ func TestWAL_TestWriteQueryOpen(t *testing.T) { t.Fatal("expected host B values to flush to index on open") } - if err := w.WritePoints([]tsdb.Point{p4}, nil, nil); err != nil { + if err := w.WritePoints([]models.Point{p4}, nil, nil); err != nil { t.Fatalf("failed to write: %s", err.Error) } - c = w.Cursor("cpu,host=B", tsdb.Forward) + c = w.Cursor("cpu,host=B", fieldNames, codec, true) k, v = c.Next() - if btou64(k) != uint64(p4.UnixNano()) { - t.Fatalf("p4 time wrong:\n\texp:%d\n\tgot:%d\n", p4.UnixNano(), btou64(k)) + if k != p4.UnixNano() { + t.Fatalf("p4 time wrong:\n\texp:%d\n\tgot:%d\n", p4.UnixNano(), k) } - if 2.2 != btof64(v) { + if 2.2 != v { t.Fatal("p4 data not equal") } From 0aa684728c9c581e08fe08ccf17ec05fc34b0df6 Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Fri, 25 Sep 2015 17:11:27 -0400 Subject: [PATCH 013/139] Make writes to historical areas possible --- tsdb/engine/pd1/encoding.go | 22 +++------ tsdb/engine/pd1/pd1.go | 99 +++++++++++++++++++++++++++++++------ tsdb/engine/pd1/pd1_test.go | 59 ++++++++++++++++++++++ tsdb/engine/pd1/wal.go | 3 ++ 4 files changed, 151 insertions(+), 32 deletions(-) diff --git a/tsdb/engine/pd1/encoding.go b/tsdb/engine/pd1/encoding.go index cad991aa7a8..5d1bee9a8fe 100644 --- a/tsdb/engine/pd1/encoding.go +++ b/tsdb/engine/pd1/encoding.go @@ -1,8 +1,6 @@ package pd1 import ( - "encoding/binary" - "math" "time" "github.com/dgryski/go-tsz" @@ -10,9 +8,8 @@ import ( ) type Value interface { - TimeBytes() []byte - ValueBytes() []byte Time() time.Time + UnixNano() int64 Value() interface{} Size() int } @@ -34,8 +31,7 @@ func NewValue(t time.Time, value interface{}) Value { type EmptyValue struct { } -func (e *EmptyValue) TimeBytes() []byte { return nil } -func (e *EmptyValue) ValueBytes() []byte { return nil } +func (e *EmptyValue) UnixNano() int64 { return tsdb.EOF } func (e *EmptyValue) Time() time.Time { return time.Unix(0, tsdb.EOF) } func (e *EmptyValue) Value() interface{} { return nil } func (e *EmptyValue) Size() int { return 0 } @@ -93,18 +89,12 @@ func (f *FloatValue) Time() time.Time { return f.time } -func (f *FloatValue) Value() interface{} { - return f.value -} - -func (f *FloatValue) TimeBytes() []byte { - return u64tob(uint64(f.Time().UnixNano())) +func (f *FloatValue) UnixNano() int64 { + return f.time.UnixNano() } -func (f *FloatValue) ValueBytes() []byte { - buf := make([]byte, 8) - binary.BigEndian.PutUint64(buf, math.Float64bits(f.value)) - return buf +func (f *FloatValue) Value() interface{} { + return f.value } func (f *FloatValue) Size() int { diff --git a/tsdb/engine/pd1/pd1.go b/tsdb/engine/pd1/pd1.go index 1428f04b45d..40aa7f8050d 100644 --- a/tsdb/engine/pd1/pd1.go +++ b/tsdb/engine/pd1/pd1.go @@ -53,7 +53,7 @@ const ( // DefaultBlockSize is the default size of uncompressed points blocks. DefaultBlockSize = 512 * 1024 // 512KB - DefaultMaxFileSize = 10 * 1024 * 1024 // 10MB + DefaultRotateFileSize = 10 * 1024 * 1024 // 10MB DefaultMaxPointsPerBlock = 1000 @@ -86,6 +86,8 @@ type Engine struct { WAL *Log + RotateFileSize uint32 + filesLock sync.RWMutex files dataFiles currentFileID int @@ -106,6 +108,7 @@ func NewEngine(path string, walPath string, opt tsdb.EngineOptions) tsdb.Engine // TODO: this is the function where we can inject a check against the in memory collisions HashSeriesField: hashSeriesField, WAL: w, + RotateFileSize: DefaultRotateFileSize, } e.WAL.Index = e @@ -172,6 +175,13 @@ func (e *Engine) Close() error { return nil } +// DataFileCount returns the number of data files in the database +func (e *Engine) DataFileCount() int { + e.filesLock.RLock() + defer e.filesLock.RUnlock() + return len(e.files) +} + // SetLogOutput is a no-op. func (e *Engine) SetLogOutput(w io.Writer) {} @@ -296,32 +306,89 @@ func (e *Engine) WriteAndCompact(pointsByKey map[string]Values, measurementField } } - // TODO: handle values written in the past that force an old data file to get rewritten - - // we keep track of the newest data file and if it should be - // rewritten with new data. - var newestDataFile *dataFile - overwriteNewestFile := false - if len(e.files) > 0 { - newestDataFile = e.files[len(e.files)-1] - overwriteNewestFile = newestDataFile.size < DefaultMaxFileSize + if len(e.files) == 0 { + return e.rewriteFile(nil, valuesByID) } - // flush values by id to either a new file or rewrite the old one - if overwriteNewestFile { - if err := e.rewriteFile(newestDataFile, valuesByID); err != nil { + maxTime := int64(math.MaxInt64) + // reverse through the data files and write in the data + for i := len(e.files) - 1; i >= 0; i-- { + f := e.files[i] + // max times are exclusive, so add 1 to it + fileMax := f.MaxTime() + 1 + fileMin := f.MinTime() + // if the file is < rotate, write all data between fileMin and maxTime + if f.size < e.RotateFileSize { + if err := e.rewriteFile(f, e.filterDataBetweenTimes(valuesByID, fileMin, maxTime)); err != nil { + return err + } + continue + } + // if the file is > rotate: + // write all data between fileMax and maxTime into new file + // write all data between fileMin and fileMax into old file + if err := e.rewriteFile(nil, e.filterDataBetweenTimes(valuesByID, fileMax, maxTime)); err != nil { return err } - } else if err := e.rewriteFile(nil, valuesByID); err != nil { - return err + if err := e.rewriteFile(f, e.filterDataBetweenTimes(valuesByID, fileMin, fileMax)); err != nil { + return err + } + maxTime = fileMin } + // for any data leftover, write into a new file since it's all older + // than any file we currently have + return e.rewriteFile(nil, valuesByID) +} - return nil +// filterDataBetweenTimes will create a new map with data between +// the minTime (inclusive) and maxTime (exclusive) while removing that +// data from the passed in map. It is assume that the Values arrays +// are sorted in time ascending order +func (e *Engine) filterDataBetweenTimes(valuesByID map[uint64]Values, minTime, maxTime int64) map[uint64]Values { + filteredValues := make(map[uint64]Values) + for id, values := range valuesByID { + maxIndex := len(values) + minIndex := 0 + // find the index of the first value in the range + for i, v := range values { + t := v.UnixNano() + if t >= minTime && t < maxTime { + minIndex = i + break + } + } + // go backwards to find the index of the last value in the range + for i := len(values) - 1; i >= 0; i-- { + t := values[i].UnixNano() + if t < maxTime { + maxIndex = i + 1 + break + } + } + + // write into the result map and filter the passed in map + filteredValues[id] = values[minIndex:maxIndex] + + // if we grabbed all the values, remove them from the passed in map + if minIndex == len(values) || (minIndex == 0 && maxIndex == len(values)) { + delete(valuesByID, id) + continue + } + + valuesByID[id] = values[0:minIndex] + if maxIndex < len(values) { + valuesByID[id] = append(valuesByID[id], values[maxIndex:]...) + } + } + return filteredValues } // rewriteFile will read in the old data file, if provided and merge the values // in the passed map into a new data file func (e *Engine) rewriteFile(oldDF *dataFile, valuesByID map[uint64]Values) error { + if len(valuesByID) == 0 { + return nil + } // we need the values in sorted order so that we can merge them into the // new file as we read the old file ids := make([]uint64, 0, len(valuesByID)) diff --git a/tsdb/engine/pd1/pd1_test.go b/tsdb/engine/pd1/pd1_test.go index 1a8f8613fc3..a20cc665ced 100644 --- a/tsdb/engine/pd1/pd1_test.go +++ b/tsdb/engine/pd1/pd1_test.go @@ -125,6 +125,65 @@ func TestEngine_WriteAndReadFloats(t *testing.T) { func TestEngine_WriteIndexWithCollision(t *testing.T) { } +func TestEngine_WriteIndexQueryAcrossDataFiles(t *testing.T) { + e := OpenDefaultEngine() + defer e.Cleanup() + + e.Shard = newFieldCodecMock(map[string]influxql.DataType{"value": influxql.Float}) + e.RotateFileSize = 10 + + p1 := parsePoint("cpu,host=A value=1.1 1000000000") + p2 := parsePoint("cpu,host=B value=1.1 1000000000") + p3 := parsePoint("cpu,host=A value=2.4 4000000000") + p4 := parsePoint("cpu,host=B value=2.4 4000000000") + + if err := e.WritePoints([]models.Point{p1, p2, p3, p4}, nil, nil); err != nil { + t.Fatalf("failed to write points: %s", err.Error()) + } + + p5 := parsePoint("cpu,host=A value=1.5 5000000000") + p6 := parsePoint("cpu,host=B value=2.5 5000000000") + p7 := parsePoint("cpu,host=A value=1.3 3000000000") + p8 := parsePoint("cpu,host=B value=2.3 3000000000") + + if err := e.WritePoints([]models.Point{p5, p6, p7, p8}, nil, nil); err != nil { + t.Fatalf("failed to write points: %s", err.Error()) + } + + if count := e.DataFileCount(); count != 2 { + t.Fatalf("expected 2 data files to exist but got %d", count) + } + + fields := []string{"value"} + var codec *tsdb.FieldCodec + + verify := func(series string, points []models.Point, seek int64) { + c := e.Cursor(series, fields, codec, true) + + // we we want to seek, do it and verify the first point matches + if seek != 0 { + k, v := c.SeekTo(seek) + p := points[0] + val := p.Fields()["value"] + if p.UnixNano() != k || val != v { + t.Fatalf("expected to seek to first point\n\texp: %d %f\n\tgot: %d %f", p.UnixNano(), val, k, v) + } + points = points[1:] + } + + for _, p := range points { + k, v := c.Next() + val := p.Fields()["value"] + if p.UnixNano() != k || val != v { + t.Fatalf("expected to seek to first point\n\texp: %d %f\n\tgot: %d %f", p.UnixNano(), val, k, v.(float64)) + } + } + } + + verify("cpu,host=A", []models.Point{p1, p7, p3, p5}, 0) + verify("cpu,host=B", []models.Point{p2, p8, p4, p6}, 0) +} + func TestEngine_WriteIndexBenchmarkNames(t *testing.T) { t.Skip("whatevs") diff --git a/tsdb/engine/pd1/wal.go b/tsdb/engine/pd1/wal.go index 7b538fed3a0..e7fb7fb39a4 100644 --- a/tsdb/engine/pd1/wal.go +++ b/tsdb/engine/pd1/wal.go @@ -521,6 +521,9 @@ func (l *Log) flush(flush flushType) error { // copy the cache items to new maps so we can empty them out l.flushCache = l.cache l.cache = make(map[string]Values) + for k, _ := range l.cacheDirtySort { + sort.Sort(l.flushCache[k]) + } l.cacheDirtySort = make(map[string]bool) valuesByKey := make(map[string]Values) From 33d28e18ecd5a0c997243214ff2226d8a97fcde1 Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Sat, 26 Sep 2015 09:02:14 -0400 Subject: [PATCH 014/139] Ensure we don't have duplicate values. Fix panic in compaction. --- tsdb/engine/pd1/encoding.go | 19 +++++++++++++++ tsdb/engine/pd1/pd1.go | 19 ++++++++------- tsdb/engine/pd1/pd1_test.go | 47 +++++++++++++++++++++++++++++++++++++ tsdb/engine/pd1/wal.go | 4 ++-- 4 files changed, 79 insertions(+), 10 deletions(-) diff --git a/tsdb/engine/pd1/encoding.go b/tsdb/engine/pd1/encoding.go index 5d1bee9a8fe..bf06fb98807 100644 --- a/tsdb/engine/pd1/encoding.go +++ b/tsdb/engine/pd1/encoding.go @@ -1,6 +1,7 @@ package pd1 import ( + "sort" "time" "github.com/dgryski/go-tsz" @@ -75,6 +76,24 @@ func (v Values) DecodeSameTypeBlock(block []byte) Values { return nil } +// Deduplicate returns a new Values slice with any values +// that have the same timestamp removed. The Value that appears +// last in the slice is the one that is kept. The returned slice is in ascending order +func (v Values) Deduplicate() Values { + m := make(map[int64]Value) + for _, val := range v { + m[val.UnixNano()] = val + } + + a := make([]Value, 0, len(m)) + for _, val := range m { + a = append(a, val) + } + sort.Sort(Values(a)) + + return a +} + // Sort methods func (a Values) Len() int { return len(a) } func (a Values) Swap(i, j int) { a[i], a[j] = a[j], a[i] } diff --git a/tsdb/engine/pd1/pd1.go b/tsdb/engine/pd1/pd1.go index 40aa7f8050d..5ae35e47980 100644 --- a/tsdb/engine/pd1/pd1.go +++ b/tsdb/engine/pd1/pd1.go @@ -389,6 +389,7 @@ func (e *Engine) rewriteFile(oldDF *dataFile, valuesByID map[uint64]Values) erro if len(valuesByID) == 0 { return nil } + // we need the values in sorted order so that we can merge them into the // new file as we read the old file ids := make([]uint64, 0, len(valuesByID)) @@ -506,7 +507,7 @@ func (e *Engine) rewriteFile(oldDF *dataFile, valuesByID map[uint64]Values) erro // determine if there's a block after this with the same id and get its time hasFutureBlock := false nextTime := int64(0) - if fpos < oldDF.size { + if fpos < oldDF.indexPosition() { nextID := btou64(oldDF.mmap[fpos : fpos+8]) if nextID == id { hasFutureBlock = true @@ -530,7 +531,7 @@ func (e *Engine) rewriteFile(oldDF *dataFile, valuesByID map[uint64]Values) erro currentPosition += uint32(12 + len(newBlock)) - if fpos >= oldDF.size { + if fpos >= oldDF.indexPosition() { break } } @@ -877,17 +878,15 @@ func (e *Engine) DecodeAndCombine(newValues Values, block, buf []byte, nextTime }) values = append(values, newValues[:pos]...) remainingValues = newValues[pos:] - sort.Sort(values) + values = values.Deduplicate() } else { - requireSort := values.MaxTime() > newValues.MinTime() + requireSort := values.MaxTime() >= newValues.MinTime() values = append(values, newValues...) if requireSort { - sort.Sort(values) + values = values.Deduplicate() } } - // TODO: deduplicate values - if len(values) > DefaultMaxPointsPerBlock { remainingValues = values[DefaultMaxPointsPerBlock:] values = values[:DefaultMaxPointsPerBlock] @@ -986,6 +985,10 @@ func (d *dataFile) IDToPosition() map[uint64]uint32 { return m } +func (d *dataFile) indexPosition() uint32 { + return d.size - uint32(d.SeriesCount()*12+20) +} + // StartingPositionForID returns the position in the file of the // first block for the given ID. If zero is returned the ID doesn't // have any data in this file. @@ -1123,7 +1126,7 @@ func (c *cursor) Next() (int64, interface{}) { // if we have a file set, see if the next block is for this ID if c.f != nil && c.pos < c.f.size { nextBlockID := btou64(c.f.mmap[c.pos : c.pos+8]) - if nextBlockID == c.id { + if nextBlockID == c.id && c.pos != c.f.indexPosition() { return c.decodeBlockAndGetValues(c.pos) } } diff --git a/tsdb/engine/pd1/pd1_test.go b/tsdb/engine/pd1/pd1_test.go index a20cc665ced..3be9bc63bba 100644 --- a/tsdb/engine/pd1/pd1_test.go +++ b/tsdb/engine/pd1/pd1_test.go @@ -184,6 +184,53 @@ func TestEngine_WriteIndexQueryAcrossDataFiles(t *testing.T) { verify("cpu,host=B", []models.Point{p2, p8, p4, p6}, 0) } +func TestEngine_WriteOverwritePreviousPoint(t *testing.T) { + e := OpenDefaultEngine() + defer e.Cleanup() + + e.Shard = newFieldCodecMock(map[string]influxql.DataType{"value": influxql.Float}) + fields := []string{"value"} + var codec *tsdb.FieldCodec + + p1 := parsePoint("cpu,host=A value=1.1 1000000000") + p2 := parsePoint("cpu,host=A value=1.2 1000000000") + p3 := parsePoint("cpu,host=A value=1.3 1000000000") + + if err := e.WritePoints([]models.Point{p1, p2}, nil, nil); err != nil { + t.Fatalf("failed to write points: %s", err.Error()) + } + + c := e.Cursor("cpu,host=A", fields, codec, true) + k, v := c.Next() + if k != p2.UnixNano() { + t.Fatalf("time wrong:\n\texp:%d\n\tgot:%d\n", p2.UnixNano(), k) + } + if 1.2 != v { + t.Fatalf("data wrong:\n\texp:%f\n\tgot:%f", 1.2, v.(float64)) + } + k, v = c.Next() + if k != tsdb.EOF { + t.Fatal("expected EOF") + } + + if err := e.WritePoints([]models.Point{p3}, nil, nil); err != nil { + t.Fatalf("failed to write points: %s", err.Error()) + } + + c = e.Cursor("cpu,host=A", fields, codec, true) + k, v = c.Next() + if k != p3.UnixNano() { + t.Fatalf("time wrong:\n\texp:%d\n\tgot:%d\n", p3.UnixNano(), k) + } + if 1.3 != v { + t.Fatalf("data wrong:\n\texp:%f\n\tgot:%f", 1.3, v.(float64)) + } + k, v = c.Next() + if k != tsdb.EOF { + t.Fatal("expected EOF") + } +} + func TestEngine_WriteIndexBenchmarkNames(t *testing.T) { t.Skip("whatevs") diff --git a/tsdb/engine/pd1/wal.go b/tsdb/engine/pd1/wal.go index e7fb7fb39a4..063e0d884b7 100644 --- a/tsdb/engine/pd1/wal.go +++ b/tsdb/engine/pd1/wal.go @@ -282,7 +282,7 @@ func (l *Log) addToCache(points []models.Point, fields map[string]*tsdb.Measurem // only mark it as dirty if it isn't already if _, ok := l.cacheDirtySort[k]; !ok && len(cacheValues) > 0 { - dirty := cacheValues[len(cacheValues)-1].Time().UnixNano() > v.Time().UnixNano() + dirty := cacheValues[len(cacheValues)-1].Time().UnixNano() >= v.Time().UnixNano() if dirty { l.cacheDirtySort[k] = true } @@ -522,7 +522,7 @@ func (l *Log) flush(flush flushType) error { l.flushCache = l.cache l.cache = make(map[string]Values) for k, _ := range l.cacheDirtySort { - sort.Sort(l.flushCache[k]) + l.flushCache[k] = l.flushCache[k].Deduplicate() } l.cacheDirtySort = make(map[string]bool) valuesByKey := make(map[string]Values) From 3aba709440ac638e0a9ae76664ffeb77c725ead3 Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Sat, 26 Sep 2015 10:47:36 -0400 Subject: [PATCH 015/139] Add multicursor to combine wal and index --- tsdb/engine/pd1/pd1.go | 24 +++----- tsdb/engine/pd1/pd1_test.go | 119 ++++++++++++++++++++++++------------ tsdb/engine/pd1/wal.go | 3 + 3 files changed, 89 insertions(+), 57 deletions(-) diff --git a/tsdb/engine/pd1/pd1.go b/tsdb/engine/pd1/pd1.go index 5ae35e47980..9ecb6fe5b03 100644 --- a/tsdb/engine/pd1/pd1.go +++ b/tsdb/engine/pd1/pd1.go @@ -79,11 +79,6 @@ type Engine struct { // and returns a hash identifier. It's not guaranteed to be unique. HashSeriesField func(key string) uint64 - // Shard is an interface that can pull back field type information based on measurement name - Shard interface { - FieldCodec(measurementName string) *tsdb.FieldCodec - } - WAL *Log RotateFileSize uint32 @@ -187,9 +182,6 @@ func (e *Engine) SetLogOutput(w io.Writer) {} // LoadMetadataIndex loads the shard metadata into memory. func (e *Engine) LoadMetadataIndex(shard *tsdb.Shard, index *tsdb.DatabaseIndex, measurementFields map[string]*tsdb.MeasurementFields) error { - e.Shard = shard - // TODO: write the metadata from the WAL - // Load measurement metadata fields, err := e.readFields() if err != nil { @@ -682,21 +674,18 @@ func (e *Engine) Begin(writable bool) (tsdb.Tx, error) { return e, nil } -// TODO: make the cursor take a field name +// TODO: handle multiple fields and descending func (e *Engine) Cursor(series string, fields []string, dec *tsdb.FieldCodec, ascending bool) tsdb.Cursor { - measurementName := tsdb.MeasurementFromSeriesKey(series) - codec := e.Shard.FieldCodec(measurementName) - if codec == nil { - return &cursor{} - } - field := codec.FieldByName("value") - if field == nil { + field := dec.FieldByName("value") + if field == nil || len(fields) > 1 { panic("pd1 engine only supports one field with name of value") } // TODO: ensure we map the collisions id := hashSeriesField(seriesFieldKey(series, field.Name)) - return newCursor(id, field.Type, e.copyFilesCollection(), ascending) + indexCursor := newCursor(id, field.Type, e.copyFilesCollection(), ascending) + wc := e.WAL.Cursor(series, fields, dec, ascending) + return tsdb.MultiCursor(wc, indexCursor) } func (e *Engine) copyFilesCollection() []*dataFile { @@ -1167,6 +1156,7 @@ func (c *cursor) decodeBlockAndGetValues(position uint32) (int64, interface{}) { v := c.vals[0] c.vals = c.vals[1:] + return v.Time().UnixNano(), v.Value() } diff --git a/tsdb/engine/pd1/pd1_test.go b/tsdb/engine/pd1/pd1_test.go index 3be9bc63bba..097c5155363 100644 --- a/tsdb/engine/pd1/pd1_test.go +++ b/tsdb/engine/pd1/pd1_test.go @@ -19,8 +19,6 @@ func TestEngine_WriteAndReadFloats(t *testing.T) { e := OpenDefaultEngine() defer e.Cleanup() - e.Shard = newFieldCodecMock(map[string]influxql.DataType{"value": influxql.Float}) - p1 := parsePoint("cpu,host=A value=1.1 1000000000") p2 := parsePoint("cpu,host=B value=1.2 1000000000") p3 := parsePoint("cpu,host=A value=2.1 2000000000") @@ -31,11 +29,17 @@ func TestEngine_WriteAndReadFloats(t *testing.T) { } fields := []string{"value"} - var codec *tsdb.FieldCodec + codec := tsdb.NewFieldCodec(map[string]*tsdb.Field{ + "value": { + ID: uint8(1), + Name: "value", + Type: influxql.Float, + }, + }) verify := func(checkSingleBVal bool) { c := e.Cursor("cpu,host=A", fields, codec, true) - k, v := c.Next() + k, v := c.SeekTo(0) if k != p1.UnixNano() { t.Fatalf("p1 time wrong:\n\texp:%d\n\tgot:%d\n", p1.UnixNano(), k) } @@ -55,7 +59,7 @@ func TestEngine_WriteAndReadFloats(t *testing.T) { } c = e.Cursor("cpu,host=B", fields, codec, true) - k, v = c.Next() + k, v = c.SeekTo(0) if k != p2.UnixNano() { t.Fatalf("p2 time wrong:\n\texp:%d\n\tgot:%d\n", p2.UnixNano(), k) } @@ -78,7 +82,7 @@ func TestEngine_WriteAndReadFloats(t *testing.T) { verify(false) c := e.Cursor("cpu,host=B", fields, codec, true) - k, v := c.Next() + k, v := c.SeekTo(0) if k != p2.UnixNano() { t.Fatalf("p2 time wrong:\n\texp:%d\n\tgot:%d\n", p2.UnixNano(), k) } @@ -129,7 +133,6 @@ func TestEngine_WriteIndexQueryAcrossDataFiles(t *testing.T) { e := OpenDefaultEngine() defer e.Cleanup() - e.Shard = newFieldCodecMock(map[string]influxql.DataType{"value": influxql.Float}) e.RotateFileSize = 10 p1 := parsePoint("cpu,host=A value=1.1 1000000000") @@ -155,21 +158,24 @@ func TestEngine_WriteIndexQueryAcrossDataFiles(t *testing.T) { } fields := []string{"value"} - var codec *tsdb.FieldCodec + codec := tsdb.NewFieldCodec(map[string]*tsdb.Field{ + "value": { + ID: uint8(1), + Name: "value", + Type: influxql.Float, + }, + }) verify := func(series string, points []models.Point, seek int64) { c := e.Cursor(series, fields, codec, true) - // we we want to seek, do it and verify the first point matches - if seek != 0 { - k, v := c.SeekTo(seek) - p := points[0] - val := p.Fields()["value"] - if p.UnixNano() != k || val != v { - t.Fatalf("expected to seek to first point\n\texp: %d %f\n\tgot: %d %f", p.UnixNano(), val, k, v) - } - points = points[1:] + k, v := c.SeekTo(seek) + p := points[0] + val := p.Fields()["value"] + if p.UnixNano() != k || val != v { + t.Fatalf("expected to seek to first point\n\texp: %d %f\n\tgot: %d %f", p.UnixNano(), val, k, v) } + points = points[1:] for _, p := range points { k, v := c.Next() @@ -182,15 +188,22 @@ func TestEngine_WriteIndexQueryAcrossDataFiles(t *testing.T) { verify("cpu,host=A", []models.Point{p1, p7, p3, p5}, 0) verify("cpu,host=B", []models.Point{p2, p8, p4, p6}, 0) + verify("cpu,host=A", []models.Point{p5}, 5000000000) + verify("cpu,host=B", []models.Point{p6}, 5000000000) } func TestEngine_WriteOverwritePreviousPoint(t *testing.T) { e := OpenDefaultEngine() defer e.Cleanup() - e.Shard = newFieldCodecMock(map[string]influxql.DataType{"value": influxql.Float}) fields := []string{"value"} - var codec *tsdb.FieldCodec + codec := tsdb.NewFieldCodec(map[string]*tsdb.Field{ + "value": { + ID: uint8(1), + Name: "value", + Type: influxql.Float, + }, + }) p1 := parsePoint("cpu,host=A value=1.1 1000000000") p2 := parsePoint("cpu,host=A value=1.2 1000000000") @@ -201,7 +214,7 @@ func TestEngine_WriteOverwritePreviousPoint(t *testing.T) { } c := e.Cursor("cpu,host=A", fields, codec, true) - k, v := c.Next() + k, v := c.SeekTo(0) if k != p2.UnixNano() { t.Fatalf("time wrong:\n\texp:%d\n\tgot:%d\n", p2.UnixNano(), k) } @@ -218,7 +231,7 @@ func TestEngine_WriteOverwritePreviousPoint(t *testing.T) { } c = e.Cursor("cpu,host=A", fields, codec, true) - k, v = c.Next() + k, v = c.SeekTo(0) if k != p3.UnixNano() { t.Fatalf("time wrong:\n\texp:%d\n\tgot:%d\n", p3.UnixNano(), k) } @@ -231,6 +244,51 @@ func TestEngine_WriteOverwritePreviousPoint(t *testing.T) { } } +func TestEngine_CursorCombinesWALAndIndex(t *testing.T) { + e := OpenDefaultEngine() + defer e.Cleanup() + + fields := []string{"value"} + codec := tsdb.NewFieldCodec(map[string]*tsdb.Field{ + "value": { + ID: uint8(1), + Name: "value", + Type: influxql.Float, + }, + }) + + p1 := parsePoint("cpu,host=A value=1.1 1000000000") + p2 := parsePoint("cpu,host=A value=1.2 2000000000") + + if err := e.WritePoints([]models.Point{p1}, nil, nil); err != nil { + t.Fatalf("failed to write points: %s", err.Error()) + } + e.WAL.SkipCache = false + if err := e.WritePoints([]models.Point{p2}, nil, nil); err != nil { + t.Fatalf("failed to write points: %s", err.Error()) + } + + c := e.Cursor("cpu,host=A", fields, codec, true) + k, v := c.SeekTo(0) + if k != p1.UnixNano() { + t.Fatalf("time wrong:\n\texp:%d\n\tgot:%d\n", p1.UnixNano(), k) + } + if 1.1 != v { + t.Fatalf("data wrong:\n\texp:%f\n\tgot:%f", 1.1, v.(float64)) + } + k, v = c.Next() + if k != p2.UnixNano() { + t.Fatalf("time wrong:\n\texp:%d\n\tgot:%d\n", p2.UnixNano(), k) + } + if 1.2 != v { + t.Fatalf("data wrong:\n\texp:%f\n\tgot:%f", 1.2, v.(float64)) + } + k, v = c.Next() + if k != tsdb.EOF { + t.Fatal("expected EOF") + } +} + func TestEngine_WriteIndexBenchmarkNames(t *testing.T) { t.Skip("whatevs") @@ -295,25 +353,6 @@ func (e *Engine) Cleanup() error { return nil } -func newFieldCodecMock(fields map[string]influxql.DataType) *FieldCodeMock { - m := make(map[string]*tsdb.Field) - - for n, t := range fields { - m[n] = &tsdb.Field{Name: n, Type: t} - } - codec := tsdb.NewFieldCodec(m) - - return &FieldCodeMock{codec: codec} -} - -type FieldCodeMock struct { - codec *tsdb.FieldCodec -} - -func (f *FieldCodeMock) FieldCodec(m string) *tsdb.FieldCodec { - return f.codec -} - func parsePoints(buf string) []models.Point { points, err := models.ParsePointsString(buf) if err != nil { diff --git a/tsdb/engine/pd1/wal.go b/tsdb/engine/pd1/wal.go index 063e0d884b7..be9dd48e71d 100644 --- a/tsdb/engine/pd1/wal.go +++ b/tsdb/engine/pd1/wal.go @@ -177,6 +177,9 @@ func (l *Log) Cursor(series string, fields []string, dec *tsdb.FieldCodec, ascen defer l.cacheLock.RUnlock() // TODO: make this work for other fields + if len(fields) != 1 || fields[0] != "value" { + panic("pd1 wal only supports 1 field with name value") + } ck := seriesFieldKey(series, "value") values := l.cache[ck] From 4db2e5c2d40f69d0f94bada1339c8b7c2acd4ba5 Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Sun, 27 Sep 2015 14:45:55 -0400 Subject: [PATCH 016/139] Add compaction and time range based write locks. --- tsdb/config.go | 16 +- tsdb/engine/pd1/encoding.go | 8 + tsdb/engine/pd1/pd1.go | 512 +++++++++++++++++++++++++--------- tsdb/engine/pd1/pd1_test.go | 84 ++++++ tsdb/engine/pd1/wal.go | 4 +- tsdb/engine/pd1/wal_test.go | 2 +- tsdb/engine/pd1/write_lock.go | 86 ++++++ 7 files changed, 582 insertions(+), 130 deletions(-) create mode 100644 tsdb/engine/pd1/write_lock.go diff --git a/tsdb/config.go b/tsdb/config.go index 4aec4a14e52..8716a455375 100644 --- a/tsdb/config.go +++ b/tsdb/config.go @@ -45,8 +45,10 @@ const ( DefaultPartitionSizeThreshold = 50 * 1024 * 1024 // 50MB // Default WAL settings for the PD1 WAL - DefaultFlushMemorySizeThreshold = 50 * 1024 * 1024 // 50MB + DefaultFlushMemorySizeThreshold = 10 * 1024 * 1024 // 10MB DefaultMaxMemorySizeThreshold = 200 * 1024 * 1024 // 200MB + DefaultIndexCompactionAge = 10 * time.Minute + DefaultIndexCompactionFileCount = 5 ) type Config struct { @@ -71,6 +73,16 @@ type Config struct { WALFlushMemorySizeThreshold int `toml:"wal-flush-memory-size-threshold"` WALMaxMemorySizeThreshold int `toml:"wal-max-memory-size-threshold"` + // compaction options for pd1 introduced in 0.9.5 + + // IndexCompactionAge specifies the duration after the data file creation time + // at which it is eligible to be compacted + IndexCompactionAge time.Duration `toml:"index-compaction-age"` + + // IndexCompactionFileCount specifies the minimum number of data files that + // must be eligible for compaction before actually running one + IndexCompactionFileCount int `toml:"index-compaction-file-count"` + // Query logging QueryLogEnabled bool `toml:"query-log-enabled"` } @@ -90,6 +102,8 @@ func NewConfig() Config { WALPartitionSizeThreshold: DefaultPartitionSizeThreshold, WALFlushMemorySizeThreshold: DefaultFlushMemorySizeThreshold, WALMaxMemorySizeThreshold: DefaultMaxMemorySizeThreshold, + IndexCompactionAge: DefaultIndexCompactionAge, + IndexCompactionFileCount: DefaultIndexCompactionFileCount, QueryLogEnabled: true, } diff --git a/tsdb/engine/pd1/encoding.go b/tsdb/engine/pd1/encoding.go index bf06fb98807..9a6d9fea7a6 100644 --- a/tsdb/engine/pd1/encoding.go +++ b/tsdb/engine/pd1/encoding.go @@ -76,6 +76,14 @@ func (v Values) DecodeSameTypeBlock(block []byte) Values { return nil } +// DecodeBlock takes a byte array and will decode into values of the appropriate type +// based on the block +func DecodeBlock(block []byte) Values { + // TODO: add support for other block types + a, _ := DecodeFloatBlock(block) + return a +} + // Deduplicate returns a new Values slice with any values // that have the same timestamp removed. The Value that appears // last in the slice is the one that is kept. The returned slice is in ascending order diff --git a/tsdb/engine/pd1/pd1.go b/tsdb/engine/pd1/pd1.go index 9ecb6fe5b03..01087194573 100644 --- a/tsdb/engine/pd1/pd1.go +++ b/tsdb/engine/pd1/pd1.go @@ -10,6 +10,7 @@ import ( "math" "os" "path/filepath" + "reflect" "sort" "sync" "syscall" @@ -50,8 +51,10 @@ func init() { } const ( - // DefaultBlockSize is the default size of uncompressed points blocks. - DefaultBlockSize = 512 * 1024 // 512KB + MaxDataFileSize = 1024 * 1024 * 1024 // 1GB + + // DefaultRotateBlockSize is the default size to rotate to a new compressed block + DefaultRotateBlockSize = 512 * 1024 // 512KB DefaultRotateFileSize = 10 * 1024 * 1024 // 10MB @@ -68,8 +71,9 @@ var _ tsdb.Engine = &Engine{} // Engine represents a storage engine with compressed blocks. type Engine struct { - mu sync.Mutex - path string + writeLock *writeLock + metaLock sync.Mutex + path string // deletesPending mark how many old data files are waiting to be deleted. This will // keep a close from returning until all deletes finish @@ -81,12 +85,19 @@ type Engine struct { WAL *Log - RotateFileSize uint32 + RotateFileSize uint32 + SkipCompaction bool + CompactionAge time.Duration + CompactionFileCount int + // filesLock is only for modifying and accessing the files slice filesLock sync.RWMutex files dataFiles currentFileID int - queryLock sync.RWMutex + + // queryLock keeps data files from being deleted or the store from + // being closed while queries are running + queryLock sync.RWMutex } // NewEngine returns a new instance of Engine. @@ -98,12 +109,15 @@ func NewEngine(path string, walPath string, opt tsdb.EngineOptions) tsdb.Engine w.LoggingEnabled = opt.Config.WALLoggingEnabled e := &Engine{ - path: path, + path: path, + writeLock: &writeLock{}, // TODO: this is the function where we can inject a check against the in memory collisions - HashSeriesField: hashSeriesField, - WAL: w, - RotateFileSize: DefaultRotateFileSize, + HashSeriesField: hashSeriesField, + WAL: w, + RotateFileSize: DefaultRotateFileSize, + CompactionAge: opt.Config.IndexCompactionAge, + CompactionFileCount: opt.Config.IndexCompactionFileCount, } e.WAL.Index = e @@ -157,9 +171,18 @@ func (e *Engine) Open() error { // Close closes the engine. func (e *Engine) Close() error { + // get all the locks so queries, writes, and compactions stop before closing e.queryLock.Lock() defer e.queryLock.Unlock() - + e.metaLock.Lock() + defer e.metaLock.Unlock() + min, max := int64(math.MinInt64), int64(math.MaxInt64) + e.writeLock.LockRange(min, max) + defer e.writeLock.UnlockRange(min, max) + e.filesLock.Lock() + defer e.filesLock.Unlock() + + // ensure all deletes have been processed e.deletesPending.Wait() for _, df := range e.files { @@ -224,38 +247,328 @@ func (e *Engine) WritePoints(points []models.Point, measurementFieldsToSave map[ return e.WAL.WritePoints(points, measurementFieldsToSave, seriesToCreate) } -func (e *Engine) WriteAndCompact(pointsByKey map[string]Values, measurementFieldsToSave map[string]*tsdb.MeasurementFields, seriesToCreate []*tsdb.SeriesCreate) error { - e.mu.Lock() - defer e.mu.Unlock() +func (e *Engine) Write(pointsByKey map[string]Values, measurementFieldsToSave map[string]*tsdb.MeasurementFields, seriesToCreate []*tsdb.SeriesCreate) error { + err, startTime, endTime, valuesByID := e.convertKeysAndWriteMetadata(pointsByKey, measurementFieldsToSave, seriesToCreate) + if err != nil { + return err + } + if len(valuesByID) == 0 { + return nil + } + + e.writeLock.LockRange(startTime, endTime) + defer e.writeLock.UnlockRange(startTime, endTime) - if err := e.writeNewFields(measurementFieldsToSave); err != nil { + if len(e.files) == 0 { + return e.rewriteFile(nil, valuesByID) + } + + maxTime := int64(math.MaxInt64) + // reverse through the data files and write in the data + files := e.copyFilesCollection() + for i := len(files) - 1; i >= 0; i-- { + f := files[i] + // max times are exclusive, so add 1 to it + fileMax := f.MaxTime() + 1 + fileMin := f.MinTime() + // if the file is < rotate, write all data between fileMin and maxTime + if f.size < e.RotateFileSize { + if err := e.rewriteFile(f, e.filterDataBetweenTimes(valuesByID, fileMin, maxTime)); err != nil { + return err + } + continue + } + // if the file is > rotate: + // write all data between fileMax and maxTime into new file + // write all data between fileMin and fileMax into old file + if err := e.rewriteFile(nil, e.filterDataBetweenTimes(valuesByID, fileMax, maxTime)); err != nil { + return err + } + if err := e.rewriteFile(f, e.filterDataBetweenTimes(valuesByID, fileMin, fileMax)); err != nil { + return err + } + maxTime = fileMin + } + // for any data leftover, write into a new file since it's all older + // than any file we currently have + err = e.rewriteFile(nil, valuesByID) + + if !e.SkipCompaction && e.shouldCompact() { + go e.Compact() + } + + return err +} + +func (e *Engine) Compact() error { + // we're looping here to ensure that the files we've marked to compact are + // still there after we've obtained the write lock + var minTime, maxTime int64 + var files dataFiles + for { + files = e.filesToCompact() + if len(files) < 2 { + return nil + } + minTime = files[0].MinTime() + maxTime = files[len(files)-1].MaxTime() + + e.writeLock.LockRange(minTime, maxTime) + + // if the files are different after obtaining the write lock, one or more + // was rewritten. Release the lock and try again. This shouldn't happen really. + if !reflect.DeepEqual(files, e.filesToCompact()) { + e.writeLock.UnlockRange(minTime, maxTime) + continue + } + + // we've got the write lock and the files are all there + break + } + defer e.writeLock.UnlockRange(minTime, maxTime) + + positions := make([]uint32, len(files)) + ids := make([]uint64, len(files)) + + // initilaize for writing + f, err := os.OpenFile(e.nextFileName(), os.O_CREATE|os.O_RDWR, 0666) + if err != nil { return err } - if err := e.writeNewSeries(seriesToCreate); err != nil { + + // write the magic number + if _, err := f.Write(u32tob(magicNumber)); err != nil { + f.Close() return err } + for i, df := range files { + ids[i] = btou64(df.mmap[4:12]) + positions[i] = 4 + } + currentPosition := uint32(fileHeaderSize) + newPositions := make([]uint32, 0) + newIDs := make([]uint64, 0) + buf := make([]byte, DefaultRotateBlockSize) + for { + // find the min ID so we can write it to the file + minID := uint64(math.MaxUint64) + for _, id := range ids { + if minID > id { + minID = id + } + } + if minID == 0 { // we've emptied all the files + break + } + + newIDs = append(newIDs, minID) + newPositions = append(newPositions, currentPosition) + + // write the blocks in order from the files with this id. as we + // go merge blocks together from one file to another, if the right size + var previousValues Values + for i, id := range ids { + if id != minID { + continue + } + df := files[i] + pos := positions[i] + fid, _, block := df.block(pos) + if fid != id { + panic("not possible") + } + newPos := pos + uint32(blockHeaderSize+len(block)) + positions[i] = newPos + + // write the blocks out to file that are already at their size limit + for { + // if the next block is the same ID, we don't need to decod this one + // so we can just write it out to the file + nextID, _, nextBlock := df.block(newPos) + newPos = newPos + uint32(blockHeaderSize+len(block)) + + if len(previousValues) > 0 { + previousValues = append(previousValues, previousValues.DecodeSameTypeBlock(block)...) + } else if len(block) > DefaultRotateBlockSize { + if _, err := f.Write(df.mmap[pos:newPos]); err != nil { + return err + } + currentPosition += uint32(newPos - pos) + } else { + previousValues = DecodeBlock(block) + } + + // write the previous values and clear if we've hit the limit + if len(previousValues) > DefaultMaxPointsPerBlock { + b := previousValues.Encode(buf) + if err := e.writeBlock(f, id, b); err != nil { + // fail hard. If we can't write a file someone needs to get woken up + panic(fmt.Sprintf("failure writing block: %s", err.Error())) + } + currentPosition += uint32(blockHeaderSize + len(b)) + previousValues = nil + } + + // move to the next block in this file only if the id is the same + if nextID != id { + ids[i] = nextID + break + } + positions[i] = newPos + block = nextBlock + newPos = newPos + uint32(blockHeaderSize+len(block)) + } + } + + if len(previousValues) > 0 { + b := previousValues.Encode(buf) + if err := e.writeBlock(f, minID, b); err != nil { + // fail hard. If we can't write a file someone needs to get woken up + panic(fmt.Sprintf("failure writing block: %s", err.Error())) + } + currentPosition += uint32(blockHeaderSize + len(b)) + } + } + + err, newDF := e.writeIndexAndGetDataFile(f, minTime, maxTime, newIDs, newPositions) + if err != nil { + return err + } + + // update engine with new file pointers + e.filesLock.Lock() + var newFiles dataFiles + for _, df := range e.files { + // exclude any files that were compacted + include := true + for _, f := range files { + if f == df { + include = false + break + } + } + if include { + newFiles = append(newFiles, df) + } + } + newFiles = append(newFiles, newDF) + sort.Sort(newFiles) + e.files = newFiles + e.filesLock.Unlock() + + // delete the old files in a goroutine so running queries won't block the write + // from completing + e.deletesPending.Add(1) + go func() { + for _, f := range files { + if err := f.Delete(); err != nil { + // TODO: log this error + } + } + e.deletesPending.Done() + }() + + return nil +} + +func (e *Engine) writeBlock(f *os.File, id uint64, block []byte) error { + if _, err := f.Write(append(u64tob(id), u32tob(uint32(len(block)))...)); err != nil { + return err + } + _, err := f.Write(block) + return err +} + +func (e *Engine) writeIndexAndGetDataFile(f *os.File, minTime, maxTime int64, ids []uint64, newPositions []uint32) (error, *dataFile) { + // write the file index, starting with the series ids and their positions + for i, id := range ids { + if _, err := f.Write(u64tob(id)); err != nil { + return err, nil + } + if _, err := f.Write(u32tob(newPositions[i])); err != nil { + return err, nil + } + } + + // write the min time, max time + if _, err := f.Write(append(u64tob(uint64(minTime)), u64tob(uint64(maxTime))...)); err != nil { + return err, nil + } + + // series count + if _, err := f.Write(u32tob(uint32(len(ids)))); err != nil { + return err, nil + } + + // sync it and see4k back to the beginning to hand off to the mmap + if err := f.Sync(); err != nil { + return err, nil + } + if _, err := f.Seek(0, 0); err != nil { + return err, nil + } + + // now open it as a memory mapped data file + newDF, err := NewDataFile(f) + if err != nil { + return err, nil + } + + return nil, newDF +} + +func (e *Engine) shouldCompact() bool { + return len(e.filesToCompact()) >= e.CompactionFileCount +} + +func (e *Engine) filesToCompact() dataFiles { + e.filesLock.RLock() + defer e.filesLock.RUnlock() + + a := make([]*dataFile, 0) + for _, df := range e.files { + if time.Since(df.modTime) > e.CompactionAge && df.size < MaxDataFileSize { + a = append(a, df) + } + } + return a +} + +func (e *Engine) convertKeysAndWriteMetadata(pointsByKey map[string]Values, measurementFieldsToSave map[string]*tsdb.MeasurementFields, seriesToCreate []*tsdb.SeriesCreate) (err error, minTime, maxTime int64, valuesByID map[uint64]Values) { + e.metaLock.Lock() + defer e.metaLock.Unlock() + + if err := e.writeNewFields(measurementFieldsToSave); err != nil { + return err, 0, 0, nil + } + if err := e.writeNewSeries(seriesToCreate); err != nil { + return err, 0, 0, nil + } if len(pointsByKey) == 0 { - return nil + return nil, 0, 0, nil } // read in keys and assign any that aren't defined b, err := e.readCompressedFile("ids") if err != nil { - return err + return err, 0, 0, nil } ids := make(map[string]uint64) if b != nil { if err := json.Unmarshal(b, &ids); err != nil { - return err + return err, 0, 0, nil } } // these are values that are newer than anything stored in the shard - valuesByID := make(map[uint64]Values) + valuesByID = make(map[uint64]Values) idToKey := make(map[uint64]string) // we only use this map if new ids are being created newKeys := false + // track the min and max time of values being inserted so we can lock that time range + minTime = int64(math.MaxInt64) + maxTime = int64(math.MinInt64) for k, values := range pointsByKey { var id uint64 var ok bool @@ -285,51 +598,27 @@ func (e *Engine) WriteAndCompact(pointsByKey map[string]Values, measurementField } } + if minTime > values.MinTime() { + minTime = values.MinTime() + } + if maxTime < values.MaxTime() { + maxTime = values.MaxTime() + } + valuesByID[id] = values } if newKeys { b, err := json.Marshal(ids) if err != nil { - return err + return err, 0, 0, nil } if err := e.replaceCompressedFile("ids", b); err != nil { - return err + return err, 0, 0, nil } } - if len(e.files) == 0 { - return e.rewriteFile(nil, valuesByID) - } - - maxTime := int64(math.MaxInt64) - // reverse through the data files and write in the data - for i := len(e.files) - 1; i >= 0; i-- { - f := e.files[i] - // max times are exclusive, so add 1 to it - fileMax := f.MaxTime() + 1 - fileMin := f.MinTime() - // if the file is < rotate, write all data between fileMin and maxTime - if f.size < e.RotateFileSize { - if err := e.rewriteFile(f, e.filterDataBetweenTimes(valuesByID, fileMin, maxTime)); err != nil { - return err - } - continue - } - // if the file is > rotate: - // write all data between fileMax and maxTime into new file - // write all data between fileMin and fileMax into old file - if err := e.rewriteFile(nil, e.filterDataBetweenTimes(valuesByID, fileMax, maxTime)); err != nil { - return err - } - if err := e.rewriteFile(f, e.filterDataBetweenTimes(valuesByID, fileMin, fileMax)); err != nil { - return err - } - maxTime = fileMin - } - // for any data leftover, write into a new file since it's all older - // than any file we currently have - return e.rewriteFile(nil, valuesByID) + return } // filterDataBetweenTimes will create a new map with data between @@ -432,7 +721,7 @@ func (e *Engine) rewriteFile(oldDF *dataFile, valuesByID map[uint64]Values) erro // now combine the old file data with the new values, keeping track of // their positions - currentPosition := uint32(4) + currentPosition := uint32(fileHeaderSize) newPositions := make([]uint32, len(ids)) buf := make([]byte, DefaultMaxPointsPerBlock*20) for i, id := range ids { @@ -473,39 +762,26 @@ func (e *Engine) rewriteFile(oldDF *dataFile, valuesByID map[uint64]Values) erro if !ok { // TODO: ensure we encode only the amount in a block block := newVals.Encode(buf) - if _, err := f.Write(append(u64tob(id), u32tob(uint32(len(block)))...)); err != nil { - f.Close() - return err - } - if _, err := f.Write(block); err != nil { + if err := e.writeBlock(f, id, block); err != nil { f.Close() return err } - currentPosition += uint32(12 + len(block)) + currentPosition += uint32(blockHeaderSize + len(block)) continue } // it's in the file and the new values, combine them and write out for { - fid := btou64(oldDF.mmap[fpos : fpos+8]) + fid, _, block := oldDF.block(fpos) if fid != id { break } - length := btou32(oldDF.mmap[fpos+8 : fpos+12]) - block := oldDF.mmap[fpos+12 : fpos+12+length] - fpos += (12 + length) + fpos += uint32(blockHeaderSize + len(block)) // determine if there's a block after this with the same id and get its time - hasFutureBlock := false - nextTime := int64(0) - if fpos < oldDF.indexPosition() { - nextID := btou64(oldDF.mmap[fpos : fpos+8]) - if nextID == id { - hasFutureBlock = true - nextTime = int64(btou64(oldDF.mmap[fpos+12 : fpos+20])) - } - } + nextID, nextTime, _ := oldDF.block(fpos) + hasFutureBlock := nextID == id nv, newBlock, err := e.DecodeAndCombine(newVals, block, buf[:0], nextTime, hasFutureBlock) newVals = nv @@ -521,7 +797,7 @@ func (e *Engine) rewriteFile(oldDF *dataFile, valuesByID map[uint64]Values) erro return err } - currentPosition += uint32(12 + len(newBlock)) + currentPosition += uint32(blockHeaderSize + len(newBlock)) if fpos >= oldDF.indexPosition() { break @@ -540,51 +816,18 @@ func (e *Engine) rewriteFile(oldDF *dataFile, valuesByID map[uint64]Values) erro f.Close() return err } - currentPosition += uint32(12 + len(block)) + currentPosition += uint32(blockHeaderSize + len(block)) } } - // write the file index, starting with the series ids and their positions - for i, id := range ids { - if _, err := f.Write(u64tob(id)); err != nil { - f.Close() - return err - } - if _, err := f.Write(u32tob(newPositions[i])); err != nil { - f.Close() - return err - } - } - - // write the min time, max time - if _, err := f.Write(append(u64tob(uint64(minTime)), u64tob(uint64(maxTime))...)); err != nil { - f.Close() - return err - } - - // series count - if _, err := f.Write(u32tob(uint32(len(ids)))); err != nil { - f.Close() - return err - } - - // sync it and see4k back to the beginning to hand off to the mmap - if err := f.Sync(); err != nil { - return err - } - if _, err := f.Seek(0, 0); err != nil { - f.Close() - return err - } - - // now open it as a memory mapped data file - newDF, err := NewDataFile(f) + err, newDF := e.writeIndexAndGetDataFile(f, minTime, maxTime, ids, newPositions) if err != nil { + f.Close() return err } // update the engine to point at the new dataFiles - e.queryLock.Lock() + e.filesLock.Lock() var files dataFiles for _, df := range e.files { if df != oldDF { @@ -594,7 +837,7 @@ func (e *Engine) rewriteFile(oldDF *dataFile, valuesByID map[uint64]Values) erro files = append(files, newDF) sort.Sort(files) e.files = files - e.queryLock.Unlock() + e.filesLock.Unlock() // remove the old data file. no need to block returning the write, // but we need to let any running queries finish before deleting it @@ -671,6 +914,7 @@ func (e *Engine) SeriesCount() (n int, err error) { // Begin starts a new transaction on the engine. func (e *Engine) Begin(writable bool) (tsdb.Tx, error) { + e.queryLock.RLock() return e, nil } @@ -696,10 +940,14 @@ func (e *Engine) copyFilesCollection() []*dataFile { return a } -func (e *Engine) Size() int64 { return 0 } -func (e *Engine) Commit() error { return nil } -func (e *Engine) Rollback() error { return nil } -func (e *Engine) WriteTo(w io.Writer) (n int64, err error) { return 0, nil } +// TODO: refactor the Tx interface to not have Size, Commit, or WriteTo since they're not used +func (e *Engine) Size() int64 { panic("not implemented") } +func (e *Engine) Commit() error { panic("not implemented") } +func (e *Engine) WriteTo(w io.Writer) (n int64, err error) { panic("not implemented") } +func (e *Engine) Rollback() error { + e.queryLock.RUnlock() + return nil +} func (e *Engine) writeNewFields(measurementFieldsToSave map[string]*tsdb.MeasurementFields) error { if len(measurementFieldsToSave) == 0 { @@ -885,17 +1133,19 @@ func (e *Engine) DecodeAndCombine(newValues Values, block, buf []byte, nextTime } type dataFile struct { - f *os.File - mu sync.RWMutex - size uint32 - mmap []byte + f *os.File + mu sync.RWMutex + size uint32 + modTime time.Time + mmap []byte } // byte size constants for the data file const ( + fileHeaderSize = 4 seriesCountSize = 4 timeSize = 8 - fileHeaderSize = seriesCountSize + (2 * timeSize) + blockHeaderSize = 12 seriesIDSize = 8 seriesPositionSize = 4 seriesHeaderSize = seriesIDSize + seriesPositionSize @@ -908,14 +1158,14 @@ func NewDataFile(f *os.File) (*dataFile, error) { } mmap, err := syscall.Mmap(int(f.Fd()), 0, int(fInfo.Size()), syscall.PROT_READ, syscall.MAP_SHARED|MAP_POPULATE) if err != nil { - f.Close() return nil, err } return &dataFile{ - f: f, - mmap: mmap, - size: uint32(fInfo.Size()), + f: f, + mmap: mmap, + size: uint32(fInfo.Size()), + modTime: fInfo.ModTime(), }, nil } @@ -1007,6 +1257,16 @@ func (d *dataFile) StartingPositionForID(id uint64) uint32 { return uint32(0) } +func (d *dataFile) block(pos uint32) (id uint64, t int64, block []byte) { + if pos < d.indexPosition() { + id = btou64(d.mmap[pos : pos+8]) + length := btou32(d.mmap[pos+8 : pos+12]) + block = d.mmap[pos+blockHeaderSize : pos+blockHeaderSize+length] + t = int64(btou64(d.mmap[pos+blockHeaderSize : pos+blockHeaderSize+8])) + } + return +} + type dataFiles []*dataFile func (a dataFiles) Len() int { return len(a) } diff --git a/tsdb/engine/pd1/pd1_test.go b/tsdb/engine/pd1/pd1_test.go index 097c5155363..db867e93039 100644 --- a/tsdb/engine/pd1/pd1_test.go +++ b/tsdb/engine/pd1/pd1_test.go @@ -289,6 +289,89 @@ func TestEngine_CursorCombinesWALAndIndex(t *testing.T) { } } +func TestEngine_Compaction(t *testing.T) { + e := OpenDefaultEngine() + defer e.Cleanup() + + e.RotateFileSize = 10 + + p1 := parsePoint("cpu,host=A value=1.1 1000000000") + p2 := parsePoint("cpu,host=B value=1.1 1000000000") + if err := e.WritePoints([]models.Point{p1, p2}, nil, nil); err != nil { + t.Fatalf("failed to write points: %s", err.Error()) + } + + p3 := parsePoint("cpu,host=A value=2.4 4000000000") + p4 := parsePoint("cpu,host=B value=2.4 4000000000") + if err := e.WritePoints([]models.Point{p3, p4}, nil, nil); err != nil { + t.Fatalf("failed to write points: %s", err.Error()) + } + + p5 := parsePoint("cpu,host=A value=1.5 5000000000") + p6 := parsePoint("cpu,host=B value=2.5 5000000000") + if err := e.WritePoints([]models.Point{p5, p6}, nil, nil); err != nil { + t.Fatalf("failed to write points: %s", err.Error()) + } + + p7 := parsePoint("cpu,host=A value=1.5 6000000000") + p8 := parsePoint("cpu,host=B value=2.5 6000000000") + if err := e.WritePoints([]models.Point{p7, p8}, nil, nil); err != nil { + t.Fatalf("failed to write points: %s", err.Error()) + } + + if count := e.DataFileCount(); count != 4 { + t.Fatalf("expected 3 data files to exist but got %d", count) + } + + fields := []string{"value"} + codec := tsdb.NewFieldCodec(map[string]*tsdb.Field{ + "value": { + ID: uint8(1), + Name: "value", + Type: influxql.Float, + }, + }) + + e.CompactionAge = time.Duration(0) + + if err := e.Compact(); err != nil { + t.Fatalf("error compacting: %s", err.Error()) + } + + if count := e.DataFileCount(); count != 1 { + t.Fatalf("expected compaction to reduce data file count to 1 but got %d", count) + } + + verify := func(series string, points []models.Point, seek int64) { + c := e.Cursor(series, fields, codec, true) + + k, v := c.SeekTo(seek) + p := points[0] + val := p.Fields()["value"] + if p.UnixNano() != k || val != v { + t.Fatalf("expected to seek to first point\n\texp: %d %f\n\tgot: %d %f", p.UnixNano(), val, k, v) + } + points = points[1:] + + for _, p := range points { + k, v := c.Next() + val := p.Fields()["value"] + if p.UnixNano() != k || val != v { + t.Fatalf("expected to seek to first point\n\texp: %d %f\n\tgot: %d %f", p.UnixNano(), val, k, v.(float64)) + } + } + } + + if err := e.Close(); err != nil { + t.Fatalf("error closing: %s", err.Error()) + } + if err := e.Open(); err != nil { + t.Fatalf("error opening: %s", err.Error()) + } + verify("cpu,host=A", []models.Point{p1, p3, p5, p7}, 0) + verify("cpu,host=B", []models.Point{p2, p4, p6, p8}, 0) +} + func TestEngine_WriteIndexBenchmarkNames(t *testing.T) { t.Skip("whatevs") @@ -340,6 +423,7 @@ func OpenEngine(opt tsdb.EngineOptions) *Engine { panic(err) } e.WAL.SkipCache = true + e.SkipCompaction = true return e } diff --git a/tsdb/engine/pd1/wal.go b/tsdb/engine/pd1/wal.go index be9dd48e71d..1447cb5f06a 100644 --- a/tsdb/engine/pd1/wal.go +++ b/tsdb/engine/pd1/wal.go @@ -123,7 +123,7 @@ type Log struct { // IndexWriter is an interface for the indexed database the WAL flushes data to type IndexWriter interface { - WriteAndCompact(valuesByKey map[string]Values, measurementFieldsToSave map[string]*tsdb.MeasurementFields, seriesToCreate []*tsdb.SeriesCreate) error + Write(valuesByKey map[string]Values, measurementFieldsToSave map[string]*tsdb.MeasurementFields, seriesToCreate []*tsdb.SeriesCreate) error } func NewLog(path string) *Log { @@ -567,7 +567,7 @@ func (l *Log) flush(flush flushType) error { } startTime := time.Now() - if err := l.Index.WriteAndCompact(valuesByKey, mfc, scc); err != nil { + if err := l.Index.Write(valuesByKey, mfc, scc); err != nil { return err } if l.LoggingEnabled { diff --git a/tsdb/engine/pd1/wal_test.go b/tsdb/engine/pd1/wal_test.go index de03dfbd601..509ed6984ca 100644 --- a/tsdb/engine/pd1/wal_test.go +++ b/tsdb/engine/pd1/wal_test.go @@ -169,6 +169,6 @@ type MockIndexWriter struct { fn func(valuesByKey map[string]pd1.Values, measurementFieldsToSave map[string]*tsdb.MeasurementFields, seriesToCreate []*tsdb.SeriesCreate) error } -func (m *MockIndexWriter) WriteAndCompact(valuesByKey map[string]pd1.Values, measurementFieldsToSave map[string]*tsdb.MeasurementFields, seriesToCreate []*tsdb.SeriesCreate) error { +func (m *MockIndexWriter) Write(valuesByKey map[string]pd1.Values, measurementFieldsToSave map[string]*tsdb.MeasurementFields, seriesToCreate []*tsdb.SeriesCreate) error { return m.fn(valuesByKey, measurementFieldsToSave, seriesToCreate) } diff --git a/tsdb/engine/pd1/write_lock.go b/tsdb/engine/pd1/write_lock.go new file mode 100644 index 00000000000..5f48fb9311a --- /dev/null +++ b/tsdb/engine/pd1/write_lock.go @@ -0,0 +1,86 @@ +package pd1 + +import ( + "sync" +) + +// writeLock is a lock that enables locking of ranges between a +// min and max value. We use this so that flushes from the WAL +// can occur concurrently along with compactions. +type writeLock struct { + mu sync.Mutex + + rangesLock sync.Mutex + ranges []*rangeLock +} + +// LockRange will ensure an exclusive lock between the min and +// max values inclusive. Any subsequent calls that have an +// an overlapping range will have to wait until the previous +// lock is released. A corresponding call to UnlockRange should +// be deferred. +func (w *writeLock) LockRange(min, max int64) { + w.mu.Lock() + defer w.mu.Unlock() + + r := &rangeLock{min: min, max: max} + ranges := w.currentlyLockedRanges() + + // ensure there are no currently locked ranges that overlap + for _, rr := range ranges { + if rr.overlaps(r) { + // wait until it gets unlocked + rr.mu.Lock() + // release the lock so the object can get GC'd + rr.mu.Unlock() + } + } + + // and lock the range + r.mu.Lock() + + // now that we know the range is free, add it to the locks + w.rangesLock.Lock() + w.ranges = append(w.ranges, r) + w.rangesLock.Unlock() +} + +// UnlockRange will release a previously locked range. +func (w *writeLock) UnlockRange(min, max int64) { + w.rangesLock.Lock() + defer w.rangesLock.Unlock() + + // take the range out of the slice and unlock it + a := make([]*rangeLock, 0) + for _, r := range w.ranges { + if r.min == min && r.max == max { + r.mu.Unlock() + continue + } + a = append(a, r) + } + w.ranges = a +} + +func (w *writeLock) currentlyLockedRanges() []*rangeLock { + w.rangesLock.Lock() + defer w.rangesLock.Unlock() + a := make([]*rangeLock, len(w.ranges)) + copy(a, w.ranges) + return a +} + +type rangeLock struct { + mu sync.Mutex + min int64 + max int64 +} + +func (r *rangeLock) overlaps(l *rangeLock) bool { + if l.min >= r.min && l.min <= r.max { + return true + } else if l.max >= r.min && l.max <= r.max { + return true + } + return false +} From a1034325983d5908cbbce9cb40aa976311e26718 Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Sun, 27 Sep 2015 16:44:57 -0400 Subject: [PATCH 017/139] Handle hash collisions on keys --- tsdb/engine/pd1/pd1.go | 89 ++++++++++++++++++++++++++++++------- tsdb/engine/pd1/pd1_test.go | 87 ++++++++++++++++++++++++++++++++++++ 2 files changed, 161 insertions(+), 15 deletions(-) diff --git a/tsdb/engine/pd1/pd1.go b/tsdb/engine/pd1/pd1.go index 01087194573..3496268bf9d 100644 --- a/tsdb/engine/pd1/pd1.go +++ b/tsdb/engine/pd1/pd1.go @@ -95,6 +95,9 @@ type Engine struct { files dataFiles currentFileID int + collisionsLock sync.RWMutex + collisions map[string]uint64 + // queryLock keeps data files from being deleted or the store from // being closed while queries are running queryLock sync.RWMutex @@ -137,6 +140,7 @@ func (e *Engine) Open() error { // TODO: clean up previous fields write // TODO: clean up previous names write // TODO: clean up any data files that didn't get cleaned up + // TODO: clean up previous collisions write files, err := filepath.Glob(filepath.Join(e.path, fmt.Sprintf("*.%s", Format))) if err != nil { @@ -166,6 +170,10 @@ func (e *Engine) Open() error { return err } + if err := e.readCollisions(); err != nil { + return err + } + return nil } @@ -190,6 +198,7 @@ func (e *Engine) Close() error { } e.files = nil e.currentFileID = 0 + e.collisions = nil return nil } @@ -564,7 +573,8 @@ func (e *Engine) convertKeysAndWriteMetadata(pointsByKey map[string]Values, meas // these are values that are newer than anything stored in the shard valuesByID = make(map[uint64]Values) - idToKey := make(map[uint64]string) // we only use this map if new ids are being created + idToKey := make(map[uint64]string) // we only use this map if new ids are being created + collisions := make(map[string]uint64) // we only use this if a collision is encountered newKeys := false // track the min and max time of values being inserted so we can lock that time range minTime = int64(math.MaxInt64) @@ -574,6 +584,7 @@ func (e *Engine) convertKeysAndWriteMetadata(pointsByKey map[string]Values, meas var ok bool if id, ok = ids[k]; !ok { // populate the map if we haven't already + if len(idToKey) == 0 { for n, id := range ids { idToKey[id] = n @@ -581,21 +592,26 @@ func (e *Engine) convertKeysAndWriteMetadata(pointsByKey map[string]Values, meas } // now see if the hash id collides with a different key - hashID := hashSeriesField(k) + hashID := e.HashSeriesField(k) existingKey, idInMap := idToKey[hashID] - if idInMap { - // we only care if the keys are different. if so, it's a hash collision we have to keep track of - if k != existingKey { - // we have a collision, give this new key a different id and move on - // TODO: handle collisions - panic("name collision, not implemented yet!") + // we only care if the keys are different. if so, it's a hash collision we have to keep track of + if idInMap && k != existingKey { + // we have a collision, find this new key the next available id + hashID = 0 + for { + hashID++ + if _, ok := idToKey[hashID]; !ok { + // next ID is available, use it + break + } } - } else { - newKeys = true - ids[k] = hashID - idToKey[id] = k - id = hashID + collisions[k] = hashID } + + newKeys = true + ids[k] = hashID + idToKey[hashID] = k + id = hashID } if minTime > values.MinTime() { @@ -618,9 +634,44 @@ func (e *Engine) convertKeysAndWriteMetadata(pointsByKey map[string]Values, meas } } + if len(collisions) > 0 { + e.saveNewCollisions(collisions) + } + return } +func (e *Engine) saveNewCollisions(collisions map[string]uint64) error { + e.collisionsLock.Lock() + defer e.collisionsLock.Unlock() + + for k, v := range collisions { + e.collisions[k] = v + } + + data, err := json.Marshal(e.collisions) + + if err != nil { + return err + } + + return e.replaceCompressedFile(CollisionsFileExtension, data) +} + +func (e *Engine) readCollisions() error { + e.collisions = make(map[string]uint64) + data, err := e.readCompressedFile(CollisionsFileExtension) + if err != nil { + return err + } + + if len(data) == 0 { + return nil + } + + return json.Unmarshal(data, &e.collisions) +} + // filterDataBetweenTimes will create a new map with data between // the minTime (inclusive) and maxTime (exclusive) while removing that // data from the passed in map. It is assume that the Values arrays @@ -925,8 +976,16 @@ func (e *Engine) Cursor(series string, fields []string, dec *tsdb.FieldCodec, as panic("pd1 engine only supports one field with name of value") } - // TODO: ensure we map the collisions - id := hashSeriesField(seriesFieldKey(series, field.Name)) + // get the ID for the key and be sure to check if it had hash collision before + key := seriesFieldKey(series, field.Name) + e.collisionsLock.RLock() + id, ok := e.collisions[key] + e.collisionsLock.RUnlock() + + if !ok { + id = e.HashSeriesField(key) + } + indexCursor := newCursor(id, field.Type, e.copyFilesCollection(), ascending) wc := e.WAL.Cursor(series, fields, dec, ascending) return tsdb.MultiCursor(wc, indexCursor) diff --git a/tsdb/engine/pd1/pd1_test.go b/tsdb/engine/pd1/pd1_test.go index db867e93039..35385ddfb66 100644 --- a/tsdb/engine/pd1/pd1_test.go +++ b/tsdb/engine/pd1/pd1_test.go @@ -372,6 +372,93 @@ func TestEngine_Compaction(t *testing.T) { verify("cpu,host=B", []models.Point{p2, p4, p6, p8}, 0) } +// Ensure that if two keys have the same fnv64-a id, we handle it +func TestEngine_KeyCollisionsAreHandled(t *testing.T) { + e := OpenDefaultEngine() + defer e.Cleanup() + + fields := []string{"value"} + codec := tsdb.NewFieldCodec(map[string]*tsdb.Field{ + "value": { + ID: uint8(1), + Name: "value", + Type: influxql.Float, + }, + }) + + // make sure two of these keys collide + e.HashSeriesField = func(key string) uint64 { + return 1 + } + p1 := parsePoint("cpu,host=A value=1.1 1000000000") + p2 := parsePoint("cpu,host=B value=1.2 1000000000") + p3 := parsePoint("cpu,host=C value=1.3 1000000000") + + if err := e.WritePoints([]models.Point{p1, p2, p3}, nil, nil); err != nil { + t.Fatalf("failed to write points: %s", err.Error()) + } + + verify := func(series string, points []models.Point, seek int64) { + c := e.Cursor(series, fields, codec, true) + + k, v := c.SeekTo(seek) + p := points[0] + val := p.Fields()["value"] + if p.UnixNano() != k || val != v { + t.Fatalf("expected to seek to first point\n\texp: %d %f\n\tgot: %d %f", p.UnixNano(), val, k, v) + } + points = points[1:] + + for _, p := range points { + k, v := c.Next() + val := p.Fields()["value"] + if p.UnixNano() != k || val != v { + t.Fatalf("expected to seek to first point\n\texp: %d %f\n\tgot: %d %f", p.UnixNano(), val, k, v.(float64)) + } + } + } + + verify("cpu,host=A", []models.Point{p1}, 0) + verify("cpu,host=B", []models.Point{p2}, 0) + verify("cpu,host=C", []models.Point{p3}, 0) + + p4 := parsePoint("cpu,host=A value=2.1 2000000000") + p5 := parsePoint("cpu,host=B value=2.2 2000000000") + p6 := parsePoint("cpu,host=C value=2.3 2000000000") + + if err := e.WritePoints([]models.Point{p4, p5, p6}, nil, nil); err != nil { + t.Fatalf("failed to write points: %s", err.Error()) + } + + verify("cpu,host=A", []models.Point{p1, p4}, 0) + verify("cpu,host=B", []models.Point{p2, p5}, 0) + verify("cpu,host=C", []models.Point{p3, p6}, 0) + + // verify collisions are handled after closing and reopening + if err := e.Close(); err != nil { + t.Fatalf("error closing: %s", err.Error()) + } + if err := e.Open(); err != nil { + t.Fatalf("error opening: %s", err.Error()) + } + + verify("cpu,host=A", []models.Point{p1, p4}, 0) + verify("cpu,host=B", []models.Point{p2, p5}, 0) + verify("cpu,host=C", []models.Point{p3, p6}, 0) + + p7 := parsePoint("cpu,host=A value=3.1 3000000000") + p8 := parsePoint("cpu,host=B value=3.2 3000000000") + p9 := parsePoint("cpu,host=C value=3.3 3000000000") + + if err := e.WritePoints([]models.Point{p7, p8, p9}, nil, nil); err != nil { + t.Fatalf("failed to write points: %s", err.Error()) + } + + verify("cpu,host=A", []models.Point{p1, p4, p7}, 0) + verify("cpu,host=B", []models.Point{p2, p5, p8}, 0) + verify("cpu,host=C", []models.Point{p3, p6, p9}, 0) +} + func TestEngine_WriteIndexBenchmarkNames(t *testing.T) { t.Skip("whatevs") From 17ed6932ae40f4a5f585a0ed9e45ace91e62664e Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Sun, 27 Sep 2015 19:37:03 -0400 Subject: [PATCH 018/139] Add support for multiple fields --- tsdb/engine/pd1/cursor.go | 148 ++++++++++++++++++++++++++++++++++ tsdb/engine/pd1/encoding.go | 5 +- tsdb/engine/pd1/pd1.go | 45 +++++++---- tsdb/engine/pd1/pd1_test.go | 156 ++++++++++++++++++++++++++++++++++++ tsdb/engine/pd1/wal.go | 11 +-- 5 files changed, 343 insertions(+), 22 deletions(-) create mode 100644 tsdb/engine/pd1/cursor.go diff --git a/tsdb/engine/pd1/cursor.go b/tsdb/engine/pd1/cursor.go new file mode 100644 index 00000000000..d31252aa32a --- /dev/null +++ b/tsdb/engine/pd1/cursor.go @@ -0,0 +1,148 @@ +package pd1 + +import ( + "math" + + "github.com/influxdb/influxdb/tsdb" +) + +type combinedEngineCursor struct { + walCursor tsdb.Cursor + engineCursor tsdb.Cursor + walKeyBuf int64 + walValueBuf interface{} + engineKeyBuf int64 + engineValueBuf interface{} + ascending bool +} + +func NewCombinedEngineCursor(wc, ec tsdb.Cursor, ascending bool) tsdb.Cursor { + return &combinedEngineCursor{ + walCursor: wc, + engineCursor: ec, + ascending: ascending, + } +} + +func (c *combinedEngineCursor) SeekTo(seek int64) (key int64, value interface{}) { + c.walKeyBuf, c.walValueBuf = c.walCursor.SeekTo(seek) + c.engineKeyBuf, c.engineValueBuf = c.engineCursor.SeekTo(seek) + return c.read() +} + +func (c *combinedEngineCursor) Next() (int64, interface{}) { + return c.read() +} + +func (c *combinedEngineCursor) Ascending() bool { + return c.ascending +} + +func (c *combinedEngineCursor) read() (key int64, value interface{}) { + key = tsdb.EOF + + // handle the case where they have the same point + if c.walKeyBuf != tsdb.EOF && c.walKeyBuf == c.engineKeyBuf { + // keep the wal value since it will overwrite the engine value + key = c.walKeyBuf + value = c.walValueBuf + c.walKeyBuf, c.walValueBuf = c.walCursor.Next() + // drop the engine value + _, _ = c.engineCursor.Next() + return + } + + // ascending order + if c.ascending { + if c.engineKeyBuf == tsdb.EOF || (c.walKeyBuf != tsdb.EOF && c.walKeyBuf < c.engineKeyBuf) { + key = c.walKeyBuf + value = c.walValueBuf + c.walKeyBuf, c.walValueBuf = c.walCursor.Next() + } else { + key = c.engineKeyBuf + value = c.engineValueBuf + c.engineKeyBuf, c.engineValueBuf = c.engineCursor.Next() + } + + return + } + + // descending order + if c.engineKeyBuf == tsdb.EOF || (c.walKeyBuf != tsdb.EOF && c.walKeyBuf > c.engineKeyBuf) { + key = c.walKeyBuf + value = c.walValueBuf + c.walKeyBuf, c.walValueBuf = c.walCursor.Next() + return + } + key = c.engineKeyBuf + value = c.engineValueBuf + c.engineKeyBuf, c.engineValueBuf = c.engineCursor.Next() + + return +} + +type multiFieldCursor struct { + fields []string + cursors []tsdb.Cursor + ascending bool + keyBuffer []int64 + valueBuffer []interface{} +} + +func NewMultiFieldCursor(fields []string, cursors []tsdb.Cursor, ascending bool) tsdb.Cursor { + return &multiFieldCursor{ + fields: fields, + cursors: cursors, + ascending: ascending, + keyBuffer: make([]int64, len(cursors)), + valueBuffer: make([]interface{}, len(cursors)), + } +} + +func (m *multiFieldCursor) SeekTo(seek int64) (key int64, value interface{}) { + for i, c := range m.cursors { + m.keyBuffer[i], m.valueBuffer[i] = c.SeekTo(seek) + } + return m.read() +} + +func (m *multiFieldCursor) Next() (int64, interface{}) { + return m.read() +} + +func (m *multiFieldCursor) Ascending() bool { + return m.ascending +} + +func (m *multiFieldCursor) read() (int64, interface{}) { + t := int64(math.MaxInt64) + if !m.ascending { + t = int64(math.MinInt64) + } + + // find the time we need to combine all fields + for _, k := range m.keyBuffer { + if k == tsdb.EOF { + continue + } + if m.ascending && t > k { + t = k + } else if !m.ascending && t < k { + t = k + } + } + + // get the value and advance each of the cursors that have the matching time + if t == math.MinInt64 || t == math.MaxInt64 { + return tsdb.EOF, nil + } + + mm := make(map[string]interface{}) + for i, k := range m.keyBuffer { + if k == t { + mm[m.fields[i]] = m.valueBuffer[i] + m.keyBuffer[i], m.valueBuffer[i] = m.cursors[i].Next() + } + } + return t, mm +} diff --git a/tsdb/engine/pd1/encoding.go b/tsdb/engine/pd1/encoding.go index 9a6d9fea7a6..b21d394af5d 100644 --- a/tsdb/engine/pd1/encoding.go +++ b/tsdb/engine/pd1/encoding.go @@ -78,10 +78,9 @@ func (v Values) DecodeSameTypeBlock(block []byte) Values { // DecodeBlock takes a byte array and will decode into values of the appropriate type // based on the block -func DecodeBlock(block []byte) Values { +func DecodeBlock(block []byte) (Values, error) { // TODO: add support for other block types - a, _ := DecodeFloatBlock(block) - return a + return DecodeFloatBlock(block) } // Deduplicate returns a new Values slice with any values diff --git a/tsdb/engine/pd1/pd1.go b/tsdb/engine/pd1/pd1.go index 3496268bf9d..a2d8687db9d 100644 --- a/tsdb/engine/pd1/pd1.go +++ b/tsdb/engine/pd1/pd1.go @@ -17,7 +17,6 @@ import ( "time" "github.com/golang/snappy" - "github.com/influxdb/influxdb/influxql" "github.com/influxdb/influxdb/models" "github.com/influxdb/influxdb/tsdb" ) @@ -404,7 +403,8 @@ func (e *Engine) Compact() error { } currentPosition += uint32(newPos - pos) } else { - previousValues = DecodeBlock(block) + // TODO: handle decode error + previousValues, _ = DecodeBlock(block) } // write the previous values and clear if we've hit the limit @@ -971,13 +971,35 @@ func (e *Engine) Begin(writable bool) (tsdb.Tx, error) { // TODO: handle multiple fields and descending func (e *Engine) Cursor(series string, fields []string, dec *tsdb.FieldCodec, ascending bool) tsdb.Cursor { - field := dec.FieldByName("value") - if field == nil || len(fields) > 1 { - panic("pd1 engine only supports one field with name of value") + files := e.copyFilesCollection() + + // don't add the overhead of the multifield cursor if we only have one field + if len(fields) == 1 { + id := e.keyAndFieldToID(series, fields[0]) + indexCursor := newCursor(id, files, ascending) + wc := e.WAL.Cursor(series, fields, dec, ascending) + return NewCombinedEngineCursor(wc, indexCursor, ascending) + } + + // multiple fields. use just the MultiFieldCursor, which also handles time collisions + // so we don't need to use the combined cursor + cursors := make([]tsdb.Cursor, 0) + cursorFields := make([]string, 0) + for _, field := range fields { + id := e.keyAndFieldToID(series, field) + indexCursor := newCursor(id, files, ascending) + wc := e.WAL.Cursor(series, []string{field}, dec, ascending) + // double up the fields since there's one for the wal and one for the index + cursorFields = append(cursorFields, field, field) + cursors = append(cursors, indexCursor, wc) } + return NewMultiFieldCursor(cursorFields, cursors, ascending) +} + +func (e *Engine) keyAndFieldToID(series, field string) uint64 { // get the ID for the key and be sure to check if it had hash collision before - key := seriesFieldKey(series, field.Name) + key := seriesFieldKey(series, field) e.collisionsLock.RLock() id, ok := e.collisions[key] e.collisionsLock.RUnlock() @@ -985,10 +1007,7 @@ func (e *Engine) Cursor(series string, fields []string, dec *tsdb.FieldCodec, as if !ok { id = e.HashSeriesField(key) } - - indexCursor := newCursor(id, field.Type, e.copyFilesCollection(), ascending) - wc := e.WAL.Cursor(series, fields, dec, ascending) - return tsdb.MultiCursor(wc, indexCursor) + return id } func (e *Engine) copyFilesCollection() []*dataFile { @@ -1334,7 +1353,6 @@ func (a dataFiles) Less(i, j int) bool { return a[i].MinTime() < a[j].MinTime() type cursor struct { id uint64 - dataType influxql.DataType f *dataFile filesPos int // the index in the files slice we're looking at pos uint32 @@ -1346,10 +1364,9 @@ type cursor struct { files []*dataFile } -func newCursor(id uint64, dataType influxql.DataType, files []*dataFile, ascending bool) *cursor { +func newCursor(id uint64, files []*dataFile, ascending bool) *cursor { return &cursor{ id: id, - dataType: dataType, ascending: ascending, files: files, } @@ -1470,7 +1487,7 @@ func (c *cursor) Next() (int64, interface{}) { func (c *cursor) decodeBlockAndGetValues(position uint32) (int64, interface{}) { length := btou32(c.f.mmap[position+8 : position+12]) block := c.f.mmap[position+12 : position+12+length] - c.vals, _ = DecodeFloatBlock(block) + c.vals, _ = DecodeBlock(block) c.pos = position + 12 + length v := c.vals[0] diff --git a/tsdb/engine/pd1/pd1_test.go b/tsdb/engine/pd1/pd1_test.go index 35385ddfb66..52cb9717f37 100644 --- a/tsdb/engine/pd1/pd1_test.go +++ b/tsdb/engine/pd1/pd1_test.go @@ -6,6 +6,7 @@ import ( "io/ioutil" "math" "os" + "reflect" "testing" "time" @@ -459,6 +460,161 @@ func TestEngine_KeyCollisionsAreHandled(t *testing.T) { verify("cpu,host=C", []models.Point{p3, p6, p9}, 0) } +func TestEngine_SupportMultipleFields(t *testing.T) { + e := OpenDefaultEngine() + defer e.Cleanup() + + fields := []string{"value", "foo"} + + p1 := parsePoint("cpu,host=A value=1.1 1000000000") + p2 := parsePoint("cpu,host=A value=1.2,foo=2.2 2000000000") + + if err := e.WritePoints([]models.Point{p1, p2}, nil, nil); err != nil { + t.Fatalf("failed to write points: %s", err.Error()) + } + c := e.Cursor("cpu,host=A", fields, nil, true) + k, v := c.SeekTo(0) + if k != p1.UnixNano() { + t.Fatalf("time wrong:\n\texp: %d\n\tgot: %d", p1.UnixNano(), k) + } + if !reflect.DeepEqual(v, map[string]interface{}{"value": 1.1}) { + t.Fatalf("value wrong: %v", v) + } + k, v = c.Next() + if k != p2.UnixNano() { + t.Fatalf("time wrong:\n\texp: %d\n\tgot: %d", p2.UnixNano(), k) + } + if !reflect.DeepEqual(v, map[string]interface{}{"value": 1.2, "foo": 2.2}) { + t.Fatalf("value wrong: %v", v) + } + k, _ = c.Next() + if k != tsdb.EOF { + t.Fatal("expected EOF") + } + + // verify we can update a field and it's still all good + p11 := parsePoint("cpu,host=A foo=2.1 1000000000") + if err := e.WritePoints([]models.Point{p11}, nil, nil); err != nil { + t.Fatalf("failed to write points: %s", err.Error()) + } + + c = e.Cursor("cpu,host=A", fields, nil, true) + k, v = c.SeekTo(0) + if k != p1.UnixNano() { + t.Fatalf("time wrong:\n\texp: %d\n\tgot: %d", p1.UnixNano(), k) + } + if !reflect.DeepEqual(v, map[string]interface{}{"value": 1.1, "foo": 2.1}) { + t.Fatalf("value wrong: %v", v) + } + k, v = c.Next() + if k != p2.UnixNano() { + t.Fatalf("time wrong:\n\texp: %d\n\tgot: %d", p2.UnixNano(), k) + } + if !reflect.DeepEqual(v, map[string]interface{}{"value": 1.2, "foo": 2.2}) { + t.Fatalf("value wrong: %v", v) + } + k, _ = c.Next() + if k != tsdb.EOF { + t.Fatal("expected EOF") + } + + // verify it's all good with the wal in the picture + e.WAL.SkipCache = false + + p3 := parsePoint("cpu,host=A value=1.3 3000000000") + p4 := parsePoint("cpu,host=A value=1.4,foo=2.4 4000000000") + if err := e.WritePoints([]models.Point{p3, p4}, nil, nil); err != nil { + t.Fatalf("failed to write points: %s", err.Error()) + } + + c = e.Cursor("cpu,host=A", fields, nil, true) + k, v = c.SeekTo(0) + if k != p1.UnixNano() { + t.Fatalf("time wrong:\n\texp: %d\n\tgot: %d", p1.UnixNano(), k) + } + if !reflect.DeepEqual(v, map[string]interface{}{"value": 1.1, "foo": 2.1}) { + t.Fatalf("value wrong: %v", v) + } + k, v = c.Next() + if k != p2.UnixNano() { + t.Fatalf("time wrong:\n\texp: %d\n\tgot: %d", p2.UnixNano(), k) + } + if !reflect.DeepEqual(v, map[string]interface{}{"value": 1.2, "foo": 2.2}) { + t.Fatalf("value wrong: %v", v) + } + k, v = c.Next() + if k != p3.UnixNano() { + t.Fatalf("time wrong:\n\texp: %d\n\tgot: %d", p3.UnixNano(), k) + } + if !reflect.DeepEqual(v, map[string]interface{}{"value": 1.3}) { + t.Fatalf("value wrong: %v", v) + } + k, v = c.Next() + if k != p4.UnixNano() { + t.Fatalf("time wrong:\n\texp: %d\n\tgot: %d", p2.UnixNano(), k) + } + if !reflect.DeepEqual(v, map[string]interface{}{"value": 1.4, "foo": 2.4}) { + t.Fatalf("value wrong: %v", v) + } + k, _ = c.Next() + if k != tsdb.EOF { + t.Fatal("expected EOF") + } + + p33 := parsePoint("cpu,host=A foo=2.3 3000000000") + if err := e.WritePoints([]models.Point{p33}, nil, nil); err != nil { + t.Fatalf("failed to write points: %s", err.Error()) + } + + c = e.Cursor("cpu,host=A", fields, nil, true) + k, v = c.SeekTo(0) + if k != p1.UnixNano() { + t.Fatalf("time wrong:\n\texp: %d\n\tgot: %d", p1.UnixNano(), k) + } + if !reflect.DeepEqual(v, map[string]interface{}{"value": 1.1, "foo": 2.1}) { + t.Fatalf("value wrong: %v", v) + } + k, v = c.Next() + if k != p2.UnixNano() { + t.Fatalf("time wrong:\n\texp: %d\n\tgot: %d", p2.UnixNano(), k) + } + if !reflect.DeepEqual(v, map[string]interface{}{"value": 1.2, "foo": 2.2}) { + t.Fatalf("value wrong: %v", v) + } + k, v = c.Next() + if k != p3.UnixNano() { + t.Fatalf("time wrong:\n\texp: %d\n\tgot: %d", p3.UnixNano(), k) + } + if !reflect.DeepEqual(v, map[string]interface{}{"value": 1.3, "foo": 2.3}) { + t.Fatalf("value wrong: %v", v) + } + k, v = c.Next() + if k != p4.UnixNano() { + t.Fatalf("time wrong:\n\texp: %d\n\tgot: %d", p2.UnixNano(), k) + } + if !reflect.DeepEqual(v, map[string]interface{}{"value": 1.4, "foo": 2.4}) { + t.Fatalf("value wrong: %v", v) + } + k, _ = c.Next() + if k != tsdb.EOF { + t.Fatal("expected EOF") + } + + // and ensure we can grab one of the fields + c = e.Cursor("cpu,host=A", []string{"value"}, nil, true) + k, v = c.SeekTo(4000000000) + if k != p4.UnixNano() { + t.Fatalf("time wrong:\n\texp: %d\n\tgot: %d", p4.UnixNano(), k) + } + if v != 1.4 { + t.Fatalf("value wrong: %v", v) + } + k, _ = c.Next() + if k != tsdb.EOF { + t.Fatal("expected EOF") + } +} + func TestEngine_WriteIndexBenchmarkNames(t *testing.T) { t.Skip("whatevs") diff --git a/tsdb/engine/pd1/wal.go b/tsdb/engine/pd1/wal.go index 1447cb5f06a..df1cbfe1b23 100644 --- a/tsdb/engine/pd1/wal.go +++ b/tsdb/engine/pd1/wal.go @@ -171,16 +171,17 @@ func (l *Log) Open() error { return nil } -// Cursor will return a cursor object to Seek and iterate with Next for the WAL cache for the given +// Cursor will return a cursor object to Seek and iterate with Next for the WAL cache for the given. +// This should only ever be called by the engine cursor method, which will always give it +// exactly one field. func (l *Log) Cursor(series string, fields []string, dec *tsdb.FieldCodec, ascending bool) tsdb.Cursor { l.cacheLock.RLock() defer l.cacheLock.RUnlock() - // TODO: make this work for other fields - if len(fields) != 1 || fields[0] != "value" { - panic("pd1 wal only supports 1 field with name value") + if len(fields) != 1 { + panic("wal cursor should only ever be called with 1 field") } - ck := seriesFieldKey(series, "value") + ck := seriesFieldKey(series, fields[0]) values := l.cache[ck] // if we're in the middle of a flush, combine the previous cache From f1ba618b2303310491393b9ee793e57549843817 Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Mon, 28 Sep 2015 21:04:02 -0400 Subject: [PATCH 019/139] Fix wal flushing, compacting, and write lock --- tsdb/config.go | 6 +- tsdb/engine/pd1/pd1.go | 157 ++++++++++++++++++++++++++++++------ tsdb/engine/pd1/pd1_test.go | 2 + tsdb/engine/pd1/wal.go | 26 +++--- 4 files changed, 153 insertions(+), 38 deletions(-) diff --git a/tsdb/config.go b/tsdb/config.go index 8716a455375..2039ff813ba 100644 --- a/tsdb/config.go +++ b/tsdb/config.go @@ -45,9 +45,9 @@ const ( DefaultPartitionSizeThreshold = 50 * 1024 * 1024 // 50MB // Default WAL settings for the PD1 WAL - DefaultFlushMemorySizeThreshold = 10 * 1024 * 1024 // 10MB - DefaultMaxMemorySizeThreshold = 200 * 1024 * 1024 // 200MB - DefaultIndexCompactionAge = 10 * time.Minute + DefaultFlushMemorySizeThreshold = 5 * 1024 * 1024 // 5MB + DefaultMaxMemorySizeThreshold = 100 * 1024 * 1024 // 100MB + DefaultIndexCompactionAge = time.Minute DefaultIndexCompactionFileCount = 5 ) diff --git a/tsdb/engine/pd1/pd1.go b/tsdb/engine/pd1/pd1.go index a2d8687db9d..54cb6d47505 100644 --- a/tsdb/engine/pd1/pd1.go +++ b/tsdb/engine/pd1/pd1.go @@ -55,7 +55,7 @@ const ( // DefaultRotateBlockSize is the default size to rotate to a new compressed block DefaultRotateBlockSize = 512 * 1024 // 512KB - DefaultRotateFileSize = 10 * 1024 * 1024 // 10MB + DefaultRotateFileSize = 5 * 1024 * 1024 // 5MB DefaultMaxPointsPerBlock = 1000 @@ -90,9 +90,10 @@ type Engine struct { CompactionFileCount int // filesLock is only for modifying and accessing the files slice - filesLock sync.RWMutex - files dataFiles - currentFileID int + filesLock sync.RWMutex + files dataFiles + currentFileID int + compactionRunning bool collisionsLock sync.RWMutex collisions map[string]uint64 @@ -264,16 +265,21 @@ func (e *Engine) Write(pointsByKey map[string]Values, measurementFieldsToSave ma return nil } - e.writeLock.LockRange(startTime, endTime) - defer e.writeLock.UnlockRange(startTime, endTime) + files, lockStart, lockEnd := e.filesAndLock(startTime, endTime) + defer e.writeLock.UnlockRange(lockStart, lockEnd) - if len(e.files) == 0 { + if len(files) == 0 { return e.rewriteFile(nil, valuesByID) } maxTime := int64(math.MaxInt64) + + // do the file rewrites in parallel + var mu sync.Mutex + var writes sync.WaitGroup + var errors []error + // reverse through the data files and write in the data - files := e.copyFilesCollection() for i := len(files) - 1; i >= 0; i-- { f := files[i] // max times are exclusive, so add 1 to it @@ -281,31 +287,105 @@ func (e *Engine) Write(pointsByKey map[string]Values, measurementFieldsToSave ma fileMin := f.MinTime() // if the file is < rotate, write all data between fileMin and maxTime if f.size < e.RotateFileSize { - if err := e.rewriteFile(f, e.filterDataBetweenTimes(valuesByID, fileMin, maxTime)); err != nil { - return err - } + writes.Add(1) + go func(df *dataFile, vals map[uint64]Values) { + if err := e.rewriteFile(df, vals); err != nil { + mu.Lock() + errors = append(errors, err) + mu.Unlock() + } + writes.Done() + }(f, e.filterDataBetweenTimes(valuesByID, fileMin, maxTime)) continue } // if the file is > rotate: // write all data between fileMax and maxTime into new file // write all data between fileMin and fileMax into old file - if err := e.rewriteFile(nil, e.filterDataBetweenTimes(valuesByID, fileMax, maxTime)); err != nil { - return err - } - if err := e.rewriteFile(f, e.filterDataBetweenTimes(valuesByID, fileMin, fileMax)); err != nil { - return err - } + writes.Add(1) + go func(vals map[uint64]Values) { + if err := e.rewriteFile(nil, vals); err != nil { + mu.Lock() + errors = append(errors, err) + mu.Unlock() + } + writes.Done() + }(e.filterDataBetweenTimes(valuesByID, fileMax, maxTime)) + writes.Add(1) + go func(df *dataFile, vals map[uint64]Values) { + if err := e.rewriteFile(df, vals); err != nil { + mu.Lock() + errors = append(errors, err) + mu.Unlock() + } + writes.Done() + }(f, e.filterDataBetweenTimes(valuesByID, fileMin, fileMax)) maxTime = fileMin } // for any data leftover, write into a new file since it's all older // than any file we currently have - err = e.rewriteFile(nil, valuesByID) + writes.Add(1) + go func() { + if err := e.rewriteFile(nil, valuesByID); err != nil { + mu.Lock() + errors = append(errors, err) + mu.Unlock() + } + writes.Done() + }() + + writes.Wait() + + if len(errors) > 0 { + // TODO: log errors + return errors[0] + } if !e.SkipCompaction && e.shouldCompact() { go e.Compact() } - return err + return nil +} + +// filesAndLock returns the data files that match the given range and +// ensures that the write lock will hold for the entire range +func (e *Engine) filesAndLock(min, max int64) (a dataFiles, lockStart, lockEnd int64) { + for { + a = make([]*dataFile, 0) + files := e.copyFilesCollection() + + for _, f := range e.files { + fmin, fmax := f.MinTime(), f.MaxTime() + if min < fmax && fmin >= fmin { + a = append(a, f) + } else if max >= fmin && max < fmax { + a = append(a, f) + } + } + + if len(a) > 0 { + lockStart = a[0].MinTime() + lockEnd = a[len(a)-1].MaxTime() + if max > lockEnd { + lockEnd = max + } + } else { + lockStart = min + lockEnd = max + } + + e.writeLock.LockRange(lockStart, lockEnd) + + // it's possible for compaction to change the files collection while we + // were waiting for a write lock on the range. Make sure the files are still the + // same after we got the lock, otherwise try again. This shouldn't happen often. + filesAfterLock := e.copyFilesCollection() + if reflect.DeepEqual(files, filesAfterLock) { + return + } + + e.writeLock.UnlockRange(lockStart, lockEnd) + } } func (e *Engine) Compact() error { @@ -333,7 +413,24 @@ func (e *Engine) Compact() error { // we've got the write lock and the files are all there break } - defer e.writeLock.UnlockRange(minTime, maxTime) + + // mark the compaction as running + e.filesLock.Lock() + e.compactionRunning = true + e.filesLock.Unlock() + defer func() { + //release the lock + e.writeLock.UnlockRange(minTime, maxTime) + + // see if we should run aonther compaction + if e.shouldCompact() { + go e.Compact() + } else { + e.filesLock.Lock() + e.compactionRunning = false + e.filesLock.Unlock() + } + }() positions := make([]uint32, len(files)) ids := make([]uint64, len(files)) @@ -472,6 +569,7 @@ func (e *Engine) Compact() error { for _, f := range files { if err := f.Delete(); err != nil { // TODO: log this error + fmt.Println("ERROR DELETING:", f.f.Name()) } } e.deletesPending.Done() @@ -527,6 +625,12 @@ func (e *Engine) writeIndexAndGetDataFile(f *os.File, minTime, maxTime int64, id } func (e *Engine) shouldCompact() bool { + e.filesLock.RLock() + running := e.compactionRunning + e.filesLock.RUnlock() + if running { + return false + } return len(e.filesToCompact()) >= e.CompactionFileCount } @@ -538,6 +642,10 @@ func (e *Engine) filesToCompact() dataFiles { for _, df := range e.files { if time.Since(df.modTime) > e.CompactionAge && df.size < MaxDataFileSize { a = append(a, df) + } else if len(a) > 0 { + // only compact contiguous ranges. If we hit the negative case and + // there are files to compact, stop here + break } } return a @@ -744,7 +852,8 @@ func (e *Engine) rewriteFile(oldDF *dataFile, valuesByID map[uint64]Values) erro minTime = v.MinTime() } if maxTime < v.MaxTime() { - maxTime = v.MaxTime() + // add 1 ns to the time since maxTime is exclusive + maxTime = v.MaxTime() + 1 } } @@ -896,7 +1005,7 @@ func (e *Engine) rewriteFile(oldDF *dataFile, valuesByID map[uint64]Values) erro e.deletesPending.Add(1) go func() { if err := oldDF.Delete(); err != nil { - // TODO: log this error + fmt.Println("ERROR DELETING FROM REWRITE:", oldDF.f.Name()) } e.deletesPending.Done() }() @@ -1312,7 +1421,7 @@ func (d *dataFile) indexPosition() uint32 { func (d *dataFile) StartingPositionForID(id uint64) uint32 { seriesCount := d.SeriesCount() - indexStart := d.size - uint32(seriesCount*12+20) + indexStart := d.indexPosition() min := uint32(0) max := uint32(seriesCount) @@ -1522,7 +1631,7 @@ func btou32(b []byte) uint32 { func hashSeriesField(key string) uint64 { h := fnv.New64a() h.Write([]byte(key)) - return h.Sum64() % 100 + return h.Sum64() } // seriesFieldKey combine a series key and field name for a unique string to be hashed to a numeric ID diff --git a/tsdb/engine/pd1/pd1_test.go b/tsdb/engine/pd1/pd1_test.go index 52cb9717f37..35ec6c720f7 100644 --- a/tsdb/engine/pd1/pd1_test.go +++ b/tsdb/engine/pd1/pd1_test.go @@ -363,6 +363,8 @@ func TestEngine_Compaction(t *testing.T) { } } + verify("cpu,host=A", []models.Point{p1, p3, p5, p7}, 0) + verify("cpu,host=B", []models.Point{p2, p4, p6, p8}, 0) if err := e.Close(); err != nil { t.Fatalf("error closing: %s", err.Error()) } diff --git a/tsdb/engine/pd1/wal.go b/tsdb/engine/pd1/wal.go index df1cbfe1b23..c4c6d23ebe8 100644 --- a/tsdb/engine/pd1/wal.go +++ b/tsdb/engine/pd1/wal.go @@ -270,12 +270,14 @@ func (l *Log) addToCache(points []models.Point, fields map[string]*tsdb.Measurem defer l.cacheLock.Unlock() // if we should check memory and we're over the threshold, mark a flush as running and kick one off in a goroutine - if checkMemory && l.memorySize > l.MaxMemorySizeThreshold { + if checkMemory && l.memorySize > l.FlushMemorySizeThreshold { if !l.flushRunning { l.flushRunning = true go l.flush(memoryFlush) } - return false + if l.memorySize > l.MaxMemorySizeThreshold { + return false + } } for _, p := range points { @@ -401,7 +403,7 @@ func (l *Log) writeToLog(writeType walEntryType, data []byte) error { l.writeLock.Lock() defer l.writeLock.Unlock() - if l.currentSegmentFile == nil { + if l.currentSegmentFile == nil || l.currentSegmentSize > DefaultSegmentSize { if err := l.newSegmentFile(); err != nil { // fail hard since we can't write data panic(fmt.Sprintf("error opening new segment file for wal: %s", err.Error())) @@ -421,6 +423,8 @@ func (l *Log) writeToLog(writeType walEntryType, data []byte) error { panic(fmt.Sprintf("error writing data to wal: %s", err.Error())) } + l.currentSegmentSize += 5 + len(data) + return l.currentSegmentFile.Sync() } @@ -489,6 +493,7 @@ func (l *Log) flush(flush flushType) error { // only flush if there isn't one already running. Memory flushes are only triggered // by writes, which will mark the flush as running, so we can ignore it. l.cacheLock.Lock() + if l.flushRunning && flush != memoryFlush { l.cacheLock.Unlock() return nil @@ -523,19 +528,18 @@ func (l *Log) flush(flush flushType) error { l.writeLock.Unlock() // copy the cache items to new maps so we can empty them out - l.flushCache = l.cache - l.cache = make(map[string]Values) + l.flushCache = make(map[string]Values) for k, _ := range l.cacheDirtySort { l.flushCache[k] = l.flushCache[k].Deduplicate() } l.cacheDirtySort = make(map[string]bool) - valuesByKey := make(map[string]Values) valueCount := 0 - for key, v := range l.flushCache { - valuesByKey[key] = v + for key, v := range l.cache { + l.flushCache[key] = v valueCount += len(v) } + l.cache = make(map[string]Values) flushSize := l.memorySize @@ -553,7 +557,7 @@ func (l *Log) flush(flush flushType) error { l.cacheLock.Unlock() // exit if there's nothing to flush to the index - if len(valuesByKey) == 0 && len(mfc) == 0 && len(scc) == 0 { + if len(l.flushCache) == 0 && len(mfc) == 0 && len(scc) == 0 { return nil } @@ -564,11 +568,11 @@ func (l *Log) flush(flush flushType) error { } else if flush == startupFlush { ftype = "startup" } - l.logger.Printf("%s flush of %d keys with %d values of %d bytes\n", ftype, len(valuesByKey), valueCount, flushSize) + l.logger.Printf("%s flush of %d keys with %d values of %d bytes\n", ftype, len(l.flushCache), valueCount, flushSize) } startTime := time.Now() - if err := l.Index.Write(valuesByKey, mfc, scc); err != nil { + if err := l.Index.Write(l.flushCache, mfc, scc); err != nil { return err } if l.LoggingEnabled { From 8444e0546de6652e54d877c5e10c123810889349 Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Mon, 28 Sep 2015 22:50:00 -0400 Subject: [PATCH 020/139] Add PerformMaintenance to store for flushes and compactions. Also fixed shard to work again with b1 and bz1 engines. --- tsdb/config.go | 40 +++++++++++------- tsdb/engine.go | 14 +++++++ tsdb/engine/b1/b1.go | 8 ++++ tsdb/engine/bz1/bz1.go | 8 ++++ tsdb/engine/pd1/encoding_test.go | 3 -- tsdb/engine/pd1/pd1.go | 48 +++++++++++++++------ tsdb/engine/pd1/pd1_test.go | 25 ++++------- tsdb/engine/pd1/wal.go | 72 +++++--------------------------- tsdb/shard.go | 67 +++++++++++++++-------------- tsdb/store.go | 42 ++++++++++++++++++- 10 files changed, 188 insertions(+), 139 deletions(-) diff --git a/tsdb/config.go b/tsdb/config.go index 2039ff813ba..1e7e29a0d86 100644 --- a/tsdb/config.go +++ b/tsdb/config.go @@ -45,10 +45,12 @@ const ( DefaultPartitionSizeThreshold = 50 * 1024 * 1024 // 50MB // Default WAL settings for the PD1 WAL - DefaultFlushMemorySizeThreshold = 5 * 1024 * 1024 // 5MB - DefaultMaxMemorySizeThreshold = 100 * 1024 * 1024 // 100MB - DefaultIndexCompactionAge = time.Minute - DefaultIndexCompactionFileCount = 5 + DefaultFlushMemorySizeThreshold = 5 * 1024 * 1024 // 5MB + DefaultMaxMemorySizeThreshold = 100 * 1024 * 1024 // 100MB + DefaultIndexCompactionAge = time.Minute + DefaultIndexMinimumCompactionInterval = time.Minute + DefaultIndexCompactionFileCount = 5 + DefaultIndexCompactionFullAge = time.Minute ) type Config struct { @@ -79,10 +81,18 @@ type Config struct { // at which it is eligible to be compacted IndexCompactionAge time.Duration `toml:"index-compaction-age"` + // IndexMinimumCompactionInterval specifies the minimum amount of time that must + // pass after a compaction before another compaction is run + IndexMinimumCompactionInterval time.Duration `toml:"index-minimum-compaction-interval"` + // IndexCompactionFileCount specifies the minimum number of data files that // must be eligible for compaction before actually running one IndexCompactionFileCount int `toml:"index-compaction-file-count"` + // IndexCompactionFullAge specifies how long after the last write was received + // in the WAL that a full compaction should be performed. + IndexCompactionFullAge time.Duration `toml:"index-compaction-full-age"` + // Query logging QueryLogEnabled bool `toml:"query-log-enabled"` } @@ -94,16 +104,18 @@ func NewConfig() Config { WALFlushInterval: toml.Duration(DefaultWALFlushInterval), WALPartitionFlushDelay: toml.Duration(DefaultWALPartitionFlushDelay), - WALLoggingEnabled: true, - WALReadySeriesSize: DefaultReadySeriesSize, - WALCompactionThreshold: DefaultCompactionThreshold, - WALMaxSeriesSize: DefaultMaxSeriesSize, - WALFlushColdInterval: toml.Duration(DefaultFlushColdInterval), - WALPartitionSizeThreshold: DefaultPartitionSizeThreshold, - WALFlushMemorySizeThreshold: DefaultFlushMemorySizeThreshold, - WALMaxMemorySizeThreshold: DefaultMaxMemorySizeThreshold, - IndexCompactionAge: DefaultIndexCompactionAge, - IndexCompactionFileCount: DefaultIndexCompactionFileCount, + WALLoggingEnabled: true, + WALReadySeriesSize: DefaultReadySeriesSize, + WALCompactionThreshold: DefaultCompactionThreshold, + WALMaxSeriesSize: DefaultMaxSeriesSize, + WALFlushColdInterval: toml.Duration(DefaultFlushColdInterval), + WALPartitionSizeThreshold: DefaultPartitionSizeThreshold, + WALFlushMemorySizeThreshold: DefaultFlushMemorySizeThreshold, + WALMaxMemorySizeThreshold: DefaultMaxMemorySizeThreshold, + IndexCompactionAge: DefaultIndexCompactionAge, + IndexCompactionFileCount: DefaultIndexCompactionFileCount, + IndexCompactionFullAge: DefaultIndexCompactionFullAge, + IndexMinimumCompactionInterval: DefaultIndexMinimumCompactionInterval, QueryLogEnabled: true, } diff --git a/tsdb/engine.go b/tsdb/engine.go index d2862b54863..407801d842e 100644 --- a/tsdb/engine.go +++ b/tsdb/engine.go @@ -32,9 +32,23 @@ type Engine interface { DeleteMeasurement(name string, seriesKeys []string) error SeriesCount() (n int, err error) + // PerformMaintenance will get called periodically by the store + PerformMaintenance() + + // Format will return the format for the engine + Format() EngineFormat + io.WriterTo } +type EngineFormat int + +const ( + B1Format EngineFormat = iota + BZ1Format + PD1Format +) + // NewEngineFunc creates a new engine. type NewEngineFunc func(path string, walPath string, options EngineOptions) Engine diff --git a/tsdb/engine/b1/b1.go b/tsdb/engine/b1/b1.go index 356b327fe95..a3f63602cdc 100644 --- a/tsdb/engine/b1/b1.go +++ b/tsdb/engine/b1/b1.go @@ -91,6 +91,14 @@ func NewEngine(path string, walPath string, opt tsdb.EngineOptions) tsdb.Engine // Path returns the path the engine was initialized with. func (e *Engine) Path() string { return e.path } +// PerformMaintenance is for periodic maintenance of the store. A no-op for b1 +func (e *Engine) PerformMaintenance() {} + +// Format returns the format type of this engine +func (e *Engine) Format() tsdb.EngineFormat { + return tsdb.B1Format +} + // Open opens and initializes the engine. func (e *Engine) Open() error { if err := func() error { diff --git a/tsdb/engine/bz1/bz1.go b/tsdb/engine/bz1/bz1.go index aa38600b806..881b82dc431 100644 --- a/tsdb/engine/bz1/bz1.go +++ b/tsdb/engine/bz1/bz1.go @@ -114,6 +114,14 @@ func NewEngine(path string, walPath string, opt tsdb.EngineOptions) tsdb.Engine // Path returns the path the engine was opened with. func (e *Engine) Path() string { return e.path } +// PerformMaintenance is for periodic maintenance of the store. A no-op for bz1 +func (e *Engine) PerformMaintenance() {} + +// Format returns the format type of this engine +func (e *Engine) Format() tsdb.EngineFormat { + return tsdb.BZ1Format +} + // Open opens and initializes the engine. func (e *Engine) Open() error { if err := func() error { diff --git a/tsdb/engine/pd1/encoding_test.go b/tsdb/engine/pd1/encoding_test.go index 82968912b0a..02598a764da 100644 --- a/tsdb/engine/pd1/encoding_test.go +++ b/tsdb/engine/pd1/encoding_test.go @@ -1,8 +1,6 @@ package pd1_test import ( - // "math/rand" - "fmt" "reflect" "testing" "time" @@ -19,7 +17,6 @@ func TestEncoding_FloatBlock(t *testing.T) { } b := values.Encode(nil) - fmt.Println("**** ", len(b)) decodedValues := values.DecodeSameTypeBlock(b) diff --git a/tsdb/engine/pd1/pd1.go b/tsdb/engine/pd1/pd1.go index 54cb6d47505..1f56c066539 100644 --- a/tsdb/engine/pd1/pd1.go +++ b/tsdb/engine/pd1/pd1.go @@ -84,10 +84,11 @@ type Engine struct { WAL *Log - RotateFileSize uint32 - SkipCompaction bool - CompactionAge time.Duration - CompactionFileCount int + RotateFileSize uint32 + SkipCompaction bool + CompactionAge time.Duration + CompactionFileCount int + IndexCompactionFullAge time.Duration // filesLock is only for modifying and accessing the files slice filesLock sync.RWMutex @@ -116,11 +117,12 @@ func NewEngine(path string, walPath string, opt tsdb.EngineOptions) tsdb.Engine writeLock: &writeLock{}, // TODO: this is the function where we can inject a check against the in memory collisions - HashSeriesField: hashSeriesField, - WAL: w, - RotateFileSize: DefaultRotateFileSize, - CompactionAge: opt.Config.IndexCompactionAge, - CompactionFileCount: opt.Config.IndexCompactionFileCount, + HashSeriesField: hashSeriesField, + WAL: w, + RotateFileSize: DefaultRotateFileSize, + CompactionAge: opt.Config.IndexCompactionAge, + CompactionFileCount: opt.Config.IndexCompactionFileCount, + IndexCompactionFullAge: opt.Config.IndexCompactionFullAge, } e.WAL.Index = e @@ -130,6 +132,28 @@ func NewEngine(path string, walPath string, opt tsdb.EngineOptions) tsdb.Engine // Path returns the path the engine was opened with. func (e *Engine) Path() string { return e.path } +// PerformMaintenance is for periodic maintenance of the store. A no-op for b1 +func (e *Engine) PerformMaintenance() { + if f := e.WAL.shouldFlush(); f != noFlush { + go func() { + fmt.Println("maintenance autoflush") + e.WAL.flush(f) + if time.Since(e.WAL.lastWriteTime) > e.IndexCompactionFullAge { + fmt.Println("mainenance compact autoflush") + e.Compact(true) + } + }() + } else if time.Since(e.WAL.lastWriteTime) > e.IndexCompactionFullAge { + fmt.Println("compact full, suckas") + go e.Compact(true) + } +} + +// Format returns the format type of this engine +func (e *Engine) Format() tsdb.EngineFormat { + return tsdb.PD1Format +} + // Open opens and initializes the engine. func (e *Engine) Open() error { if err := os.MkdirAll(e.path, 0777); err != nil { @@ -341,7 +365,7 @@ func (e *Engine) Write(pointsByKey map[string]Values, measurementFieldsToSave ma } if !e.SkipCompaction && e.shouldCompact() { - go e.Compact() + go e.Compact(false) } return nil @@ -388,7 +412,7 @@ func (e *Engine) filesAndLock(min, max int64) (a dataFiles, lockStart, lockEnd i } } -func (e *Engine) Compact() error { +func (e *Engine) Compact(fullCompaction bool) error { // we're looping here to ensure that the files we've marked to compact are // still there after we've obtained the write lock var minTime, maxTime int64 @@ -424,7 +448,7 @@ func (e *Engine) Compact() error { // see if we should run aonther compaction if e.shouldCompact() { - go e.Compact() + go e.Compact(false) } else { e.filesLock.Lock() e.compactionRunning = false diff --git a/tsdb/engine/pd1/pd1_test.go b/tsdb/engine/pd1/pd1_test.go index 35ec6c720f7..491c7cd552b 100644 --- a/tsdb/engine/pd1/pd1_test.go +++ b/tsdb/engine/pd1/pd1_test.go @@ -137,18 +137,18 @@ func TestEngine_WriteIndexQueryAcrossDataFiles(t *testing.T) { e.RotateFileSize = 10 p1 := parsePoint("cpu,host=A value=1.1 1000000000") - p2 := parsePoint("cpu,host=B value=1.1 1000000000") - p3 := parsePoint("cpu,host=A value=2.4 4000000000") - p4 := parsePoint("cpu,host=B value=2.4 4000000000") + p2 := parsePoint("cpu,host=B value=1.2 1000000000") + p3 := parsePoint("cpu,host=A value=2.1 4000000000") + p4 := parsePoint("cpu,host=B value=2.2 4000000000") if err := e.WritePoints([]models.Point{p1, p2, p3, p4}, nil, nil); err != nil { t.Fatalf("failed to write points: %s", err.Error()) } - p5 := parsePoint("cpu,host=A value=1.5 5000000000") - p6 := parsePoint("cpu,host=B value=2.5 5000000000") - p7 := parsePoint("cpu,host=A value=1.3 3000000000") - p8 := parsePoint("cpu,host=B value=2.3 3000000000") + p5 := parsePoint("cpu,host=A value=3.1 5000000000") + p6 := parsePoint("cpu,host=B value=3.2 5000000000") + p7 := parsePoint("cpu,host=A value=4.1 3000000000") + p8 := parsePoint("cpu,host=B value=4.2 3000000000") if err := e.WritePoints([]models.Point{p5, p6, p7, p8}, nil, nil); err != nil { t.Fatalf("failed to write points: %s", err.Error()) @@ -159,16 +159,9 @@ func TestEngine_WriteIndexQueryAcrossDataFiles(t *testing.T) { } fields := []string{"value"} - codec := tsdb.NewFieldCodec(map[string]*tsdb.Field{ - "value": { - ID: uint8(1), - Name: "value", - Type: influxql.Float, - }, - }) verify := func(series string, points []models.Point, seek int64) { - c := e.Cursor(series, fields, codec, true) + c := e.Cursor(series, fields, nil, true) k, v := c.SeekTo(seek) p := points[0] @@ -335,7 +328,7 @@ func TestEngine_Compaction(t *testing.T) { e.CompactionAge = time.Duration(0) - if err := e.Compact(); err != nil { + if err := e.Compact(true); err != nil { t.Fatalf("error compacting: %s", err.Error()) } diff --git a/tsdb/engine/pd1/wal.go b/tsdb/engine/pd1/wal.go index c4c6d23ebe8..49b72b4a608 100644 --- a/tsdb/engine/pd1/wal.go +++ b/tsdb/engine/pd1/wal.go @@ -28,9 +28,6 @@ const ( WALFilePrefix = "_" - // defaultFlushCheckInterval is how often flushes are triggered automatically by the flush criteria - defaultFlushCheckInterval = time.Second - writeBufLen = 32 << 10 // 32kb ) @@ -85,10 +82,6 @@ type Log struct { measurementFieldsCache map[string]*tsdb.MeasurementFields seriesToCreateCache []*tsdb.SeriesCreate - // These coordinate closing and waiting for running goroutines. - wg sync.WaitGroup - closing chan struct{} - // LogOutput is the writer used by the logger. LogOutput io.Writer logger *log.Logger @@ -136,7 +129,6 @@ func NewLog(path string) *Log { SegmentSize: DefaultSegmentSize, FlushMemorySizeThreshold: tsdb.DefaultFlushMemorySizeThreshold, MaxMemorySizeThreshold: tsdb.DefaultMaxMemorySizeThreshold, - flushCheckInterval: defaultFlushCheckInterval, logger: log.New(os.Stderr, "[pd1wal] ", log.LstdFlags), } } @@ -161,13 +153,6 @@ func (l *Log) Open() error { return err } - l.flushCheckTimer = time.NewTimer(l.flushCheckInterval) - - // Start background goroutines. - l.wg.Add(1) - l.closing = make(chan struct{}) - go l.autoflusher(l.closing) - return nil } @@ -307,6 +292,12 @@ func (l *Log) addToCache(points []models.Point, fields map[string]*tsdb.Measurem return true } +func (l *Log) LastWriteTime() time.Time { + l.cacheLock.RLock() + defer l.cacheLock.RUnlock() + return l.lastWriteTime +} + // readAndFlushWAL is called on open and will read the segment files in, flushing whenever // the memory gets over the limit. Once all files have been read it will flush and remove the files func (l *Log) readAndFlushWAL() error { @@ -439,21 +430,6 @@ func (l *Log) DeleteSeries(keys []string) error { // Close will finish any flush that is currently in process and close file handles func (l *Log) Close() error { - // stop the autoflushing process so it doesn't try to kick another one off - l.writeLock.Lock() - l.cacheLock.Lock() - - if l.closing != nil { - close(l.closing) - l.closing = nil - } - l.writeLock.Unlock() - l.cacheLock.Unlock() - - // Allow goroutines to finish running. - l.wg.Wait() - - // Lock the remainder of the closing process. l.writeLock.Lock() l.cacheLock.Lock() defer l.writeLock.Unlock() @@ -529,17 +505,16 @@ func (l *Log) flush(flush flushType) error { // copy the cache items to new maps so we can empty them out l.flushCache = make(map[string]Values) - for k, _ := range l.cacheDirtySort { - l.flushCache[k] = l.flushCache[k].Deduplicate() - } - l.cacheDirtySort = make(map[string]bool) - valueCount := 0 for key, v := range l.cache { l.flushCache[key] = v valueCount += len(v) } l.cache = make(map[string]Values) + for k, _ := range l.cacheDirtySort { + l.flushCache[k] = l.flushCache[k].Deduplicate() + } + l.cacheDirtySort = make(map[string]bool) flushSize := l.memorySize @@ -604,33 +579,6 @@ func (l *Log) flush(flush flushType) error { return nil } -// triggerAutoFlush will flush and compact any partitions that have hit the thresholds for compaction -func (l *Log) triggerAutoFlush() { - // - if f := l.shouldFlush(); f != noFlush { - if err := l.flush(f); err != nil { - l.logger.Printf("error flushing wal: %s\n", err) - } - } -} - -// autoflusher waits for notification of a flush and kicks it off in the background. -// This method runs in a separate goroutine. -func (l *Log) autoflusher(closing chan struct{}) { - defer l.wg.Done() - - for { - // Wait for close or flush signal. - select { - case <-closing: - return - case <-l.flushCheckTimer.C: - l.triggerAutoFlush() - l.flushCheckTimer.Reset(l.flushCheckInterval) - } - } -} - // segmentFileNames will return all files that are WAL segment files in sorted order by ascending ID func (l *Log) segmentFileNames() ([]string, error) { names, err := filepath.Glob(filepath.Join(l.path, fmt.Sprintf("%s*.%s", WALFilePrefix, WALFileExtension))) diff --git a/tsdb/shard.go b/tsdb/shard.go index 2e04735665f..b1dbac32fcd 100644 --- a/tsdb/shard.go +++ b/tsdb/shard.go @@ -91,6 +91,12 @@ func NewShard(id uint64, index *DatabaseIndex, path string, walPath string, opti // Path returns the path set on the shard when it was created. func (s *Shard) Path() string { return s.path } +// PerformMaintenance gets called periodically to have the engine perform +// any maintenance tasks like WAL flushing and compaction +func (s *Shard) PerformMaintenance() { + s.engine.PerformMaintenance() +} + // open initializes and opens the shard's store. func (s *Shard) Open() error { if err := func() error { @@ -229,29 +235,31 @@ func (s *Shard) WritePoints(points []models.Point) error { } // make sure all data is encoded before attempting to save to bolt - // TODO: make this only commented out for pd1 engine - // for _, p := range points { - // // Ignore if raw data has already been marshaled. - // if p.Data() != nil { - // continue - // } - - // // This was populated earlier, don't need to validate that it's there. - // s.mu.RLock() - // mf := s.measurementFields[p.Name()] - // s.mu.RUnlock() - - // // If a measurement is dropped while writes for it are in progress, this could be nil - // if mf == nil { - // return ErrFieldNotFound - // } - - // data, err := mf.Codec.EncodeFields(p.Fields()) - // if err != nil { - // return err - // } - // p.SetData(data) - // } + // only required for the b1 and bz1 formats + if s.engine.Format() != PD1Format { + for _, p := range points { + // Ignore if raw data has already been marshaled. + if p.Data() != nil { + continue + } + + // This was populated earlier, don't need to validate that it's there. + s.mu.RLock() + mf := s.measurementFields[p.Name()] + s.mu.RUnlock() + + // If a measurement is dropped while writes for it are in progress, this could be nil + if mf == nil { + return ErrFieldNotFound + } + + data, err := mf.Codec.EncodeFields(p.Fields()) + if err != nil { + return err + } + p.SetData(data) + } + } // Write to the engine. if err := s.engine.WritePoints(points, measurementFieldsToSave, seriesToCreate); err != nil { @@ -742,14 +750,11 @@ func (f *FieldCodec) DecodeByID(targetID uint8, b []byte) (interface{}, error) { // DecodeByName scans a byte slice for a field with the given name, converts it to its // expected type, and return that value. func (f *FieldCodec) DecodeByName(name string, b []byte) (interface{}, error) { - // TODO: this is a hack for PD1 testing, please to remove - return math.Float64frombits(binary.BigEndian.Uint64(b)), nil - - // fi := f.FieldByName(name) - // if fi == nil { - // return 0, ErrFieldNotFound - // } - // return f.DecodeByID(fi.ID, b) + fi := f.FieldByName(name) + if fi == nil { + return 0, ErrFieldNotFound + } + return f.DecodeByID(fi.ID, b) } func (f *FieldCodec) Fields() (a []*Field) { diff --git a/tsdb/store.go b/tsdb/store.go index 13235a16844..62dab631172 100644 --- a/tsdb/store.go +++ b/tsdb/store.go @@ -9,6 +9,7 @@ import ( "strconv" "strings" "sync" + "time" "github.com/influxdb/influxdb/influxql" "github.com/influxdb/influxdb/models" @@ -29,6 +30,10 @@ var ( ErrShardNotFound = fmt.Errorf("shard not found") ) +const ( + MaintenanceCheckInterval = time.Minute +) + type Store struct { mu sync.RWMutex path string @@ -38,7 +43,9 @@ type Store struct { EngineOptions EngineOptions Logger *log.Logger - closing chan struct{} + + closing chan struct{} + wg sync.WaitGroup } // Path returns the store's root path. @@ -301,6 +308,32 @@ func (s *Store) loadShards() error { } +// periodicMaintenance is the method called in a goroutine on the opening of the store +// to perform periodic maintenance of the shards. +func (s *Store) periodicMaintenance() { + t := time.NewTicker(MaintenanceCheckInterval) + for { + select { + case <-t.C: + s.performMaintenance() + case <-s.closing: + t.Stop() + return + } + } +} + +// performMaintenance will loop through the shars and tell them +// to perform any maintenance tasks. Those tasks should kick off +// their own goroutines if it's anything that could take time. +func (s *Store) performMaintenance() { + s.mu.Lock() + defer s.mu.Unlock() + for _, sh := range s.shards { + sh.PerformMaintenance() + } +} + func (s *Store) Open() error { s.mu.Lock() defer s.mu.Unlock() @@ -326,6 +359,8 @@ func (s *Store) Open() error { return err } + go s.periodicMaintenance() + return nil } @@ -366,6 +401,11 @@ func (s *Store) CreateMapper(shardID uint64, stmt influxql.Statement, chunkSize func (s *Store) Close() error { s.mu.Lock() defer s.mu.Unlock() + if s.closing != nil { + close(s.closing) + s.closing = nil + } + s.wg.Wait() for _, sh := range s.shards { if err := sh.Close(); err != nil { From 68d03e8ae0d8b545714dd431be549f6e5db0732a Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Tue, 29 Sep 2015 08:35:05 -0400 Subject: [PATCH 021/139] Make compaction run at most at set duration. --- tsdb/engine/pd1/pd1.go | 77 ++++++++++++++++++++++++------------------ 1 file changed, 44 insertions(+), 33 deletions(-) diff --git a/tsdb/engine/pd1/pd1.go b/tsdb/engine/pd1/pd1.go index 1f56c066539..8be62d069d1 100644 --- a/tsdb/engine/pd1/pd1.go +++ b/tsdb/engine/pd1/pd1.go @@ -84,17 +84,19 @@ type Engine struct { WAL *Log - RotateFileSize uint32 - SkipCompaction bool - CompactionAge time.Duration - CompactionFileCount int - IndexCompactionFullAge time.Duration + RotateFileSize uint32 + SkipCompaction bool + CompactionAge time.Duration + CompactionFileCount int + IndexCompactionFullAge time.Duration + IndexMinimumCompactionInterval time.Duration // filesLock is only for modifying and accessing the files slice - filesLock sync.RWMutex - files dataFiles - currentFileID int - compactionRunning bool + filesLock sync.RWMutex + files dataFiles + currentFileID int + compactionRunning bool + lastCompactionTime time.Time collisionsLock sync.RWMutex collisions map[string]uint64 @@ -117,12 +119,13 @@ func NewEngine(path string, walPath string, opt tsdb.EngineOptions) tsdb.Engine writeLock: &writeLock{}, // TODO: this is the function where we can inject a check against the in memory collisions - HashSeriesField: hashSeriesField, - WAL: w, - RotateFileSize: DefaultRotateFileSize, - CompactionAge: opt.Config.IndexCompactionAge, - CompactionFileCount: opt.Config.IndexCompactionFileCount, - IndexCompactionFullAge: opt.Config.IndexCompactionFullAge, + HashSeriesField: hashSeriesField, + WAL: w, + RotateFileSize: DefaultRotateFileSize, + CompactionAge: opt.Config.IndexCompactionAge, + CompactionFileCount: opt.Config.IndexCompactionFileCount, + IndexCompactionFullAge: opt.Config.IndexCompactionFullAge, + IndexMinimumCompactionInterval: opt.Config.IndexMinimumCompactionInterval, } e.WAL.Index = e @@ -136,15 +139,12 @@ func (e *Engine) Path() string { return e.path } func (e *Engine) PerformMaintenance() { if f := e.WAL.shouldFlush(); f != noFlush { go func() { - fmt.Println("maintenance autoflush") e.WAL.flush(f) - if time.Since(e.WAL.lastWriteTime) > e.IndexCompactionFullAge { - fmt.Println("mainenance compact autoflush") + if e.shouldCompact() { e.Compact(true) } }() - } else if time.Since(e.WAL.lastWriteTime) > e.IndexCompactionFullAge { - fmt.Println("compact full, suckas") + } else if e.shouldCompact() { go e.Compact(true) } } @@ -418,7 +418,11 @@ func (e *Engine) Compact(fullCompaction bool) error { var minTime, maxTime int64 var files dataFiles for { - files = e.filesToCompact() + if fullCompaction { + files = e.copyFilesCollection() + } else { + files = e.filesToCompact() + } if len(files) < 2 { return nil } @@ -429,7 +433,13 @@ func (e *Engine) Compact(fullCompaction bool) error { // if the files are different after obtaining the write lock, one or more // was rewritten. Release the lock and try again. This shouldn't happen really. - if !reflect.DeepEqual(files, e.filesToCompact()) { + var filesAfterLock dataFiles + if fullCompaction { + filesAfterLock = e.copyFilesCollection() + } else { + filesAfterLock = e.filesToCompact() + } + if !reflect.DeepEqual(files, filesAfterLock) { e.writeLock.UnlockRange(minTime, maxTime) continue } @@ -438,6 +448,9 @@ func (e *Engine) Compact(fullCompaction bool) error { break } + fmt.Println("Starting compaction with files:", len(files)) + st := time.Now() + // mark the compaction as running e.filesLock.Lock() e.compactionRunning = true @@ -445,15 +458,10 @@ func (e *Engine) Compact(fullCompaction bool) error { defer func() { //release the lock e.writeLock.UnlockRange(minTime, maxTime) - - // see if we should run aonther compaction - if e.shouldCompact() { - go e.Compact(false) - } else { - e.filesLock.Lock() - e.compactionRunning = false - e.filesLock.Unlock() - } + e.filesLock.Lock() + e.lastCompactionTime = time.Now() + e.compactionRunning = false + e.filesLock.Unlock() }() positions := make([]uint32, len(files)) @@ -586,6 +594,8 @@ func (e *Engine) Compact(fullCompaction bool) error { e.files = newFiles e.filesLock.Unlock() + fmt.Println("Compaction took ", time.Since(st)) + // delete the old files in a goroutine so running queries won't block the write // from completing e.deletesPending.Add(1) @@ -651,8 +661,9 @@ func (e *Engine) writeIndexAndGetDataFile(f *os.File, minTime, maxTime int64, id func (e *Engine) shouldCompact() bool { e.filesLock.RLock() running := e.compactionRunning + since := time.Since(e.lastCompactionTime) e.filesLock.RUnlock() - if running { + if running || since < e.IndexMinimumCompactionInterval { return false } return len(e.filesToCompact()) >= e.CompactionFileCount @@ -662,7 +673,7 @@ func (e *Engine) filesToCompact() dataFiles { e.filesLock.RLock() defer e.filesLock.RUnlock() - a := make([]*dataFile, 0) + var a dataFiles for _, df := range e.files { if time.Since(df.modTime) > e.CompactionAge && df.size < MaxDataFileSize { a = append(a, df) From 9e630f8b8b647a0e1f7427b1628d430a807c2f31 Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Tue, 29 Sep 2015 08:59:25 -0400 Subject: [PATCH 022/139] Ensure data files can't be deleted while query is running. Also ensure that queries don't try to use files that have been deleted. --- tsdb/engine/pd1/pd1.go | 72 ++++++++++++++++++++++-------------------- tsdb/engine/pd1/tx.go | 52 ++++++++++++++++++++++++++++++ 2 files changed, 90 insertions(+), 34 deletions(-) create mode 100644 tsdb/engine/pd1/tx.go diff --git a/tsdb/engine/pd1/pd1.go b/tsdb/engine/pd1/pd1.go index 8be62d069d1..32edc84c7c3 100644 --- a/tsdb/engine/pd1/pd1.go +++ b/tsdb/engine/pd1/pd1.go @@ -1110,37 +1110,45 @@ func (e *Engine) SeriesCount() (n int, err error) { // Begin starts a new transaction on the engine. func (e *Engine) Begin(writable bool) (tsdb.Tx, error) { e.queryLock.RLock() - return e, nil -} -// TODO: handle multiple fields and descending -func (e *Engine) Cursor(series string, fields []string, dec *tsdb.FieldCodec, ascending bool) tsdb.Cursor { - files := e.copyFilesCollection() + var files dataFiles - // don't add the overhead of the multifield cursor if we only have one field - if len(fields) == 1 { - id := e.keyAndFieldToID(series, fields[0]) - indexCursor := newCursor(id, files, ascending) - wc := e.WAL.Cursor(series, fields, dec, ascending) - return NewCombinedEngineCursor(wc, indexCursor, ascending) - } + // we do this to ensure that the data files haven't been deleted from a compaction + // while we were waiting to get the query lock + for { + files = e.copyFilesCollection() + + // get the query lock + for _, f := range files { + f.mu.RLock() + } - // multiple fields. use just the MultiFieldCursor, which also handles time collisions - // so we don't need to use the combined cursor - cursors := make([]tsdb.Cursor, 0) - cursorFields := make([]string, 0) - for _, field := range fields { - id := e.keyAndFieldToID(series, field) - indexCursor := newCursor(id, files, ascending) - wc := e.WAL.Cursor(series, []string{field}, dec, ascending) - // double up the fields since there's one for the wal and one for the index - cursorFields = append(cursorFields, field, field) - cursors = append(cursors, indexCursor, wc) + // ensure they're all still open + reset := false + for _, f := range files { + if f.f == nil { + reset = true + break + } + } + + // if not, release and try again + if reset { + for _, f := range files { + f.mu.RUnlock() + } + continue + } + + // we're good to go + break } - return NewMultiFieldCursor(cursorFields, cursors, ascending) + return &tx{files: files, engine: e}, nil } +func (e *Engine) WriteTo(w io.Writer) (n int64, err error) { panic("not implemented") } + func (e *Engine) keyAndFieldToID(series, field string) uint64 { // get the ID for the key and be sure to check if it had hash collision before key := seriesFieldKey(series, field) @@ -1162,15 +1170,6 @@ func (e *Engine) copyFilesCollection() []*dataFile { return a } -// TODO: refactor the Tx interface to not have Size, Commit, or WriteTo since they're not used -func (e *Engine) Size() int64 { panic("not implemented") } -func (e *Engine) Commit() error { panic("not implemented") } -func (e *Engine) WriteTo(w io.Writer) (n int64, err error) { panic("not implemented") } -func (e *Engine) Rollback() error { - e.queryLock.RUnlock() - return nil -} - func (e *Engine) writeNewFields(measurementFieldsToSave map[string]*tsdb.MeasurementFields) error { if len(measurementFieldsToSave) == 0 { return nil @@ -1403,7 +1402,12 @@ func (d *dataFile) Delete() error { if err := d.close(); err != nil { return err } - return os.Remove(d.f.Name()) + err := os.Remove(d.f.Name()) + if err != nil { + return err + } + d.f = nil + return nil } func (d *dataFile) close() error { diff --git a/tsdb/engine/pd1/tx.go b/tsdb/engine/pd1/tx.go new file mode 100644 index 00000000000..16aac8b5e43 --- /dev/null +++ b/tsdb/engine/pd1/tx.go @@ -0,0 +1,52 @@ +package pd1 + +import ( + "io" + + "github.com/influxdb/influxdb/tsdb" +) + +type tx struct { + files dataFiles + engine *Engine +} + +// TODO: handle multiple fields and descending +func (t *tx) Cursor(series string, fields []string, dec *tsdb.FieldCodec, ascending bool) tsdb.Cursor { + // don't add the overhead of the multifield cursor if we only have one field + if len(fields) == 1 { + id := t.engine.keyAndFieldToID(series, fields[0]) + indexCursor := newCursor(id, t.files, ascending) + wc := t.engine.WAL.Cursor(series, fields, dec, ascending) + return NewCombinedEngineCursor(wc, indexCursor, ascending) + } + + // multiple fields. use just the MultiFieldCursor, which also handles time collisions + // so we don't need to use the combined cursor + cursors := make([]tsdb.Cursor, 0) + cursorFields := make([]string, 0) + for _, field := range fields { + id := t.engine.keyAndFieldToID(series, field) + indexCursor := newCursor(id, t.files, ascending) + wc := t.engine.WAL.Cursor(series, []string{field}, dec, ascending) + // double up the fields since there's one for the wal and one for the index + cursorFields = append(cursorFields, field, field) + cursors = append(cursors, indexCursor, wc) + } + + return NewMultiFieldCursor(cursorFields, cursors, ascending) +} + +func (t *tx) Rollback() error { + t.engine.queryLock.RUnlock() + for _, f := range t.files { + f.mu.RUnlock() + } + + return nil +} + +// TODO: refactor the Tx interface to not have Size, Commit, or WriteTo since they're not used +func (t *tx) Size() int64 { panic("not implemented") } +func (t *tx) Commit() error { panic("not implemented") } +func (t *tx) WriteTo(w io.Writer) (n int64, err error) { panic("not implemented") } From eb24c0daa626154d65c95bbb782b002c8ab3ccf8 Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Tue, 29 Sep 2015 10:56:11 -0400 Subject: [PATCH 023/139] Add recover to maintenance. Change snapshot writer to not use bolt on shard. --- tsdb/engine/pd1/pd1.go | 3 --- tsdb/shard.go | 2 -- tsdb/snapshot_writer.go | 5 ++--- tsdb/store.go | 11 ++++++++++- 4 files changed, 12 insertions(+), 9 deletions(-) diff --git a/tsdb/engine/pd1/pd1.go b/tsdb/engine/pd1/pd1.go index 32edc84c7c3..82df8c60e6d 100644 --- a/tsdb/engine/pd1/pd1.go +++ b/tsdb/engine/pd1/pd1.go @@ -140,9 +140,6 @@ func (e *Engine) PerformMaintenance() { if f := e.WAL.shouldFlush(); f != noFlush { go func() { e.WAL.flush(f) - if e.shouldCompact() { - e.Compact(true) - } }() } else if e.shouldCompact() { go e.Compact(true) diff --git a/tsdb/shard.go b/tsdb/shard.go index b1dbac32fcd..1f606b613b3 100644 --- a/tsdb/shard.go +++ b/tsdb/shard.go @@ -16,7 +16,6 @@ import ( "github.com/influxdb/influxdb/models" "github.com/influxdb/influxdb/tsdb/internal" - "github.com/boltdb/bolt" "github.com/gogo/protobuf/proto" ) @@ -49,7 +48,6 @@ var ( // Data can be split across many shards. The query engine in TSDB is responsible // for combining the output of many shards into a single query result. type Shard struct { - db *bolt.DB // underlying data store index *DatabaseIndex path string walPath string diff --git a/tsdb/snapshot_writer.go b/tsdb/snapshot_writer.go index 785ca13908c..4a0a2d3edef 100644 --- a/tsdb/snapshot_writer.go +++ b/tsdb/snapshot_writer.go @@ -8,7 +8,6 @@ import ( "path/filepath" "time" - "github.com/boltdb/bolt" "github.com/influxdb/influxdb/snapshot" ) @@ -83,7 +82,7 @@ func appendShardSnapshotFile(sw *snapshot.Writer, sh *Shard, name string) error } // Begin transaction. - tx, err := sh.db.Begin(false) + tx, err := sh.engine.Begin(false) if err != nil { return fmt.Errorf("begin: %s", err) } @@ -103,7 +102,7 @@ func appendShardSnapshotFile(sw *snapshot.Writer, sh *Shard, name string) error // boltTxCloser wraps a Bolt transaction to implement io.Closer. type boltTxCloser struct { - *bolt.Tx + Tx } // Close rolls back the transaction. diff --git a/tsdb/store.go b/tsdb/store.go index 62dab631172..1fb8560167e 100644 --- a/tsdb/store.go +++ b/tsdb/store.go @@ -330,10 +330,19 @@ func (s *Store) performMaintenance() { s.mu.Lock() defer s.mu.Unlock() for _, sh := range s.shards { - sh.PerformMaintenance() + s.performMaintenanceOnShard(sh) } } +func (s *Store) performMaintenanceOnShard(shard *Shard) { + defer func() { + if r := recover(); r != nil { + s.Logger.Printf("recovered eror in maintenance on shard %d", shard.id) + } + }() + shard.PerformMaintenance() +} + func (s *Store) Open() error { s.mu.Lock() defer s.mu.Unlock() From 9031804dcca4e8a45bc90fa0e163479e64e8bb54 Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Tue, 29 Sep 2015 11:27:02 -0400 Subject: [PATCH 024/139] Update tests to use transactions. Add test for single series 10k points. --- tsdb/engine/pd1/pd1_test.go | 126 +++++++++++++++++++++--------------- 1 file changed, 75 insertions(+), 51 deletions(-) diff --git a/tsdb/engine/pd1/pd1_test.go b/tsdb/engine/pd1/pd1_test.go index 491c7cd552b..e1abe87360c 100644 --- a/tsdb/engine/pd1/pd1_test.go +++ b/tsdb/engine/pd1/pd1_test.go @@ -10,7 +10,6 @@ import ( "testing" "time" - "github.com/influxdb/influxdb/influxql" "github.com/influxdb/influxdb/models" "github.com/influxdb/influxdb/tsdb" "github.com/influxdb/influxdb/tsdb/engine/pd1" @@ -30,16 +29,11 @@ func TestEngine_WriteAndReadFloats(t *testing.T) { } fields := []string{"value"} - codec := tsdb.NewFieldCodec(map[string]*tsdb.Field{ - "value": { - ID: uint8(1), - Name: "value", - Type: influxql.Float, - }, - }) verify := func(checkSingleBVal bool) { - c := e.Cursor("cpu,host=A", fields, codec, true) + tx, _ := e.Begin(false) + defer tx.Rollback() + c := tx.Cursor("cpu,host=A", fields, nil, true) k, v := c.SeekTo(0) if k != p1.UnixNano() { t.Fatalf("p1 time wrong:\n\texp:%d\n\tgot:%d\n", p1.UnixNano(), k) @@ -59,7 +53,7 @@ func TestEngine_WriteAndReadFloats(t *testing.T) { t.Fatal("expected EOF") } - c = e.Cursor("cpu,host=B", fields, codec, true) + c = tx.Cursor("cpu,host=B", fields, nil, true) k, v = c.SeekTo(0) if k != p2.UnixNano() { t.Fatalf("p2 time wrong:\n\texp:%d\n\tgot:%d\n", p2.UnixNano(), k) @@ -82,7 +76,9 @@ func TestEngine_WriteAndReadFloats(t *testing.T) { } verify(false) - c := e.Cursor("cpu,host=B", fields, codec, true) + tx, _ := e.Begin(false) + defer tx.Rollback() + c := tx.Cursor("cpu,host=B", fields, nil, true) k, v := c.SeekTo(0) if k != p2.UnixNano() { t.Fatalf("p2 time wrong:\n\texp:%d\n\tgot:%d\n", p2.UnixNano(), k) @@ -107,7 +103,7 @@ func TestEngine_WriteAndReadFloats(t *testing.T) { t.Fatal("p2 data not equal") } - c = e.Cursor("cpu,host=A", fields, codec, true) + c = tx.Cursor("cpu,host=A", fields, nil, true) k, v = c.SeekTo(0) if k != p1.UnixNano() { t.Fatalf("p1 time wrong:\n\texp:%d\n\tgot:%d\n", p1.UnixNano(), k) @@ -161,7 +157,9 @@ func TestEngine_WriteIndexQueryAcrossDataFiles(t *testing.T) { fields := []string{"value"} verify := func(series string, points []models.Point, seek int64) { - c := e.Cursor(series, fields, nil, true) + tx, _ := e.Begin(false) + defer tx.Rollback() + c := tx.Cursor(series, fields, nil, true) k, v := c.SeekTo(seek) p := points[0] @@ -191,13 +189,6 @@ func TestEngine_WriteOverwritePreviousPoint(t *testing.T) { defer e.Cleanup() fields := []string{"value"} - codec := tsdb.NewFieldCodec(map[string]*tsdb.Field{ - "value": { - ID: uint8(1), - Name: "value", - Type: influxql.Float, - }, - }) p1 := parsePoint("cpu,host=A value=1.1 1000000000") p2 := parsePoint("cpu,host=A value=1.2 1000000000") @@ -207,7 +198,9 @@ func TestEngine_WriteOverwritePreviousPoint(t *testing.T) { t.Fatalf("failed to write points: %s", err.Error()) } - c := e.Cursor("cpu,host=A", fields, codec, true) + tx, _ := e.Begin(false) + defer tx.Rollback() + c := tx.Cursor("cpu,host=A", fields, nil, true) k, v := c.SeekTo(0) if k != p2.UnixNano() { t.Fatalf("time wrong:\n\texp:%d\n\tgot:%d\n", p2.UnixNano(), k) @@ -224,7 +217,9 @@ func TestEngine_WriteOverwritePreviousPoint(t *testing.T) { t.Fatalf("failed to write points: %s", err.Error()) } - c = e.Cursor("cpu,host=A", fields, codec, true) + tx2, _ := e.Begin(false) + defer tx2.Rollback() + c = tx2.Cursor("cpu,host=A", fields, nil, true) k, v = c.SeekTo(0) if k != p3.UnixNano() { t.Fatalf("time wrong:\n\texp:%d\n\tgot:%d\n", p3.UnixNano(), k) @@ -243,13 +238,6 @@ func TestEngine_CursorCombinesWALAndIndex(t *testing.T) { defer e.Cleanup() fields := []string{"value"} - codec := tsdb.NewFieldCodec(map[string]*tsdb.Field{ - "value": { - ID: uint8(1), - Name: "value", - Type: influxql.Float, - }, - }) p1 := parsePoint("cpu,host=A value=1.1 1000000000") p2 := parsePoint("cpu,host=A value=1.2 2000000000") @@ -262,7 +250,9 @@ func TestEngine_CursorCombinesWALAndIndex(t *testing.T) { t.Fatalf("failed to write points: %s", err.Error()) } - c := e.Cursor("cpu,host=A", fields, codec, true) + tx, _ := e.Begin(false) + defer tx.Rollback() + c := tx.Cursor("cpu,host=A", fields, nil, true) k, v := c.SeekTo(0) if k != p1.UnixNano() { t.Fatalf("time wrong:\n\texp:%d\n\tgot:%d\n", p1.UnixNano(), k) @@ -318,13 +308,6 @@ func TestEngine_Compaction(t *testing.T) { } fields := []string{"value"} - codec := tsdb.NewFieldCodec(map[string]*tsdb.Field{ - "value": { - ID: uint8(1), - Name: "value", - Type: influxql.Float, - }, - }) e.CompactionAge = time.Duration(0) @@ -337,7 +320,9 @@ func TestEngine_Compaction(t *testing.T) { } verify := func(series string, points []models.Point, seek int64) { - c := e.Cursor(series, fields, codec, true) + tx, _ := e.Begin(false) + defer tx.Rollback() + c := tx.Cursor(series, fields, nil, true) k, v := c.SeekTo(seek) p := points[0] @@ -374,13 +359,6 @@ func TestEngine_KeyCollisionsAreHandled(t *testing.T) { defer e.Cleanup() fields := []string{"value"} - codec := tsdb.NewFieldCodec(map[string]*tsdb.Field{ - "value": { - ID: uint8(1), - Name: "value", - Type: influxql.Float, - }, - }) // make sure two of these keys collide e.HashSeriesField = func(key string) uint64 { @@ -395,7 +373,9 @@ func TestEngine_KeyCollisionsAreHandled(t *testing.T) { } verify := func(series string, points []models.Point, seek int64) { - c := e.Cursor(series, fields, codec, true) + tx, _ := e.Begin(false) + defer tx.Rollback() + c := tx.Cursor(series, fields, nil, true) k, v := c.SeekTo(seek) p := points[0] @@ -467,7 +447,9 @@ func TestEngine_SupportMultipleFields(t *testing.T) { if err := e.WritePoints([]models.Point{p1, p2}, nil, nil); err != nil { t.Fatalf("failed to write points: %s", err.Error()) } - c := e.Cursor("cpu,host=A", fields, nil, true) + tx, _ := e.Begin(false) + defer tx.Rollback() + c := tx.Cursor("cpu,host=A", fields, nil, true) k, v := c.SeekTo(0) if k != p1.UnixNano() { t.Fatalf("time wrong:\n\texp: %d\n\tgot: %d", p1.UnixNano(), k) @@ -493,7 +475,9 @@ func TestEngine_SupportMultipleFields(t *testing.T) { t.Fatalf("failed to write points: %s", err.Error()) } - c = e.Cursor("cpu,host=A", fields, nil, true) + tx2, _ := e.Begin(false) + defer tx2.Rollback() + c = tx2.Cursor("cpu,host=A", fields, nil, true) k, v = c.SeekTo(0) if k != p1.UnixNano() { t.Fatalf("time wrong:\n\texp: %d\n\tgot: %d", p1.UnixNano(), k) @@ -522,7 +506,9 @@ func TestEngine_SupportMultipleFields(t *testing.T) { t.Fatalf("failed to write points: %s", err.Error()) } - c = e.Cursor("cpu,host=A", fields, nil, true) + tx3, _ := e.Begin(false) + defer tx3.Rollback() + c = tx3.Cursor("cpu,host=A", fields, nil, true) k, v = c.SeekTo(0) if k != p1.UnixNano() { t.Fatalf("time wrong:\n\texp: %d\n\tgot: %d", p1.UnixNano(), k) @@ -561,7 +547,9 @@ func TestEngine_SupportMultipleFields(t *testing.T) { t.Fatalf("failed to write points: %s", err.Error()) } - c = e.Cursor("cpu,host=A", fields, nil, true) + tx4, _ := e.Begin(false) + defer tx4.Rollback() + c = tx4.Cursor("cpu,host=A", fields, nil, true) k, v = c.SeekTo(0) if k != p1.UnixNano() { t.Fatalf("time wrong:\n\texp: %d\n\tgot: %d", p1.UnixNano(), k) @@ -596,7 +584,7 @@ func TestEngine_SupportMultipleFields(t *testing.T) { } // and ensure we can grab one of the fields - c = e.Cursor("cpu,host=A", []string{"value"}, nil, true) + c = tx4.Cursor("cpu,host=A", []string{"value"}, nil, true) k, v = c.SeekTo(4000000000) if k != p4.UnixNano() { t.Fatalf("time wrong:\n\texp: %d\n\tgot: %d", p4.UnixNano(), k) @@ -610,6 +598,42 @@ func TestEngine_SupportMultipleFields(t *testing.T) { } } +func TestEngine_WriteManyPointsToSingleSeries(t *testing.T) { + e := OpenDefaultEngine() + defer e.Cleanup() + + fields := []string{"value"} + + var points []models.Point + for i := 1; i <= 10000; i++ { + points = append(points, parsePoint(fmt.Sprintf("cpu,host=A value=%d %d000000000", i, i))) + if i%500 == 0 { + if err := e.WritePoints(points, nil, nil); err != nil { + t.Fatalf("failed to write points: %s", err.Error()) + } + points = nil + } + } + + tx, _ := e.Begin(false) + defer tx.Rollback() + c := tx.Cursor("cpu,host=A", fields, nil, true) + k, v := c.SeekTo(0) + for i := 2; i <= 10000; i++ { + k, v = c.Next() + if k != int64(i)*1000000000 { + t.Fatalf("time wrong:\n\texp: %d\n\tgot: %d", i*1000000000, k) + } + if v != float64(i) { + t.Fatalf("value wrong:\n\texp:%v\n\tgot:%v", float64(i), v) + } + } + k, _ = c.Next() + if k != tsdb.EOF { + t.Fatal("expected EOF") + } +} + func TestEngine_WriteIndexBenchmarkNames(t *testing.T) { t.Skip("whatevs") From ca2a13c76ea4fdd6b32d4bd087d5632ae7146431 Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Tue, 29 Sep 2015 11:37:28 -0400 Subject: [PATCH 025/139] Update stress to use second timestamps and less random floats. --- cmd/influx_stress/influx_stress.go | 2 -- stress/runner.go | 10 +++++++--- 2 files changed, 7 insertions(+), 5 deletions(-) diff --git a/cmd/influx_stress/influx_stress.go b/cmd/influx_stress/influx_stress.go index 2247a5329b3..9292fb02d90 100644 --- a/cmd/influx_stress/influx_stress.go +++ b/cmd/influx_stress/influx_stress.go @@ -3,8 +3,6 @@ package main import ( "flag" "fmt" - "math/rand" - "net/url" "runtime" "sort" "time" diff --git a/stress/runner.go b/stress/runner.go index f098c1bdc8c..d968784b02c 100644 --- a/stress/runner.go +++ b/stress/runner.go @@ -157,17 +157,22 @@ func Run(cfg *Config) (totalPoints int, failedRequests int, responseTimes Respon batch := &client.BatchPoints{ Database: cfg.Database, WriteConsistency: "any", - Time: time.Now(), Precision: cfg.Precision, } + pointTime := time.Now().Unix() - int64(cfg.PointCount) + for i := 1; i <= cfg.PointCount; i++ { + pointTime++ + batchTime := time.Unix(pointTime, 0) + for j := 1; j <= cfg.SeriesCount; j++ { for _, m := range cfg.Measurements { p := client.Point{ Measurement: m, Tags: map[string]string{"region": "uswest", "host": fmt.Sprintf("host-%d", j)}, - Fields: map[string]interface{}{"value": rand.Float64()}, + Fields: map[string]interface{}{"value": float64(rand.Intn(1000))}, + Time: batchTime, } batch.Points = append(batch.Points, p) if len(batch.Points) >= cfg.BatchSize { @@ -206,7 +211,6 @@ func Run(cfg *Config) (totalPoints int, failedRequests int, responseTimes Respon Database: cfg.Database, WriteConsistency: "any", Precision: "n", - Time: time.Now(), } } } From 4fcc61c7667a38f8a3cdd1d1d88ab427e6a40b63 Mon Sep 17 00:00:00 2001 From: Jason Wilder Date: Mon, 21 Sep 2015 13:56:10 -0600 Subject: [PATCH 026/139] Ensure we have files when iterating in cursor Prevents index out of bounds panic --- tsdb/engine/pd1/pd1.go | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/tsdb/engine/pd1/pd1.go b/tsdb/engine/pd1/pd1.go index 82df8c60e6d..d707661fad7 100644 --- a/tsdb/engine/pd1/pd1.go +++ b/tsdb/engine/pd1/pd1.go @@ -1518,6 +1518,10 @@ func newCursor(id uint64, files []*dataFile, ascending bool) *cursor { } func (c *cursor) SeekTo(seek int64) (int64, interface{}) { + if len(c.files) == 0 { + return tsdb.EOF, nil + } + if seek < c.files[0].MinTime() { c.filesPos = 0 c.f = c.files[0] From 938bae97d4ec4d4c57bb85a112afd3ca1f141d91 Mon Sep 17 00:00:00 2001 From: Jason Wilder Date: Mon, 21 Sep 2015 13:57:03 -0600 Subject: [PATCH 027/139] Fix go vet errors --- tsdb/engine/pd1/wal_test.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tsdb/engine/pd1/wal_test.go b/tsdb/engine/pd1/wal_test.go index 509ed6984ca..034ad2dd3e5 100644 --- a/tsdb/engine/pd1/wal_test.go +++ b/tsdb/engine/pd1/wal_test.go @@ -29,7 +29,7 @@ func TestWAL_TestWriteQueryOpen(t *testing.T) { } if err := w.Open(); err != nil { - t.Fatalf("error opening: %s", err.Error) + t.Fatalf("error opening: %s", err.Error()) } p1 := parsePoint("cpu,host=A value=1.1 1000000000") @@ -70,7 +70,7 @@ func TestWAL_TestWriteQueryOpen(t *testing.T) { // ensure we can do another write to the wal and get stuff if err := w.WritePoints([]models.Point{p3}, nil, nil); err != nil { - t.Fatalf("failed to write: %s", err.Error) + t.Fatalf("failed to write: %s", err.Error()) } c = w.Cursor("cpu,host=A", fieldNames, codec, true) @@ -120,7 +120,7 @@ func TestWAL_TestWriteQueryOpen(t *testing.T) { } if err := w.WritePoints([]models.Point{p4}, nil, nil); err != nil { - t.Fatalf("failed to write: %s", err.Error) + t.Fatalf("failed to write: %s", err.Error()) } c = w.Cursor("cpu,host=B", fieldNames, codec, true) k, v = c.Next() From 57b0a276da87d0b90ed73c715d595bae22676f81 Mon Sep 17 00:00:00 2001 From: Jason Wilder Date: Mon, 21 Sep 2015 13:58:33 -0600 Subject: [PATCH 028/139] Allow influx_stress to use small data sizes Prevents a panics when response size is less than 100. Also allows data to be posted when it is less than the batch size. --- cmd/influx_stress/influx_stress.go | 16 +++++++++++++--- 1 file changed, 13 insertions(+), 3 deletions(-) diff --git a/cmd/influx_stress/influx_stress.go b/cmd/influx_stress/influx_stress.go index 9292fb02d90..fee98f776c4 100644 --- a/cmd/influx_stress/influx_stress.go +++ b/cmd/influx_stress/influx_stress.go @@ -35,6 +35,10 @@ func main() { ms = append(ms, "cpu") } + if *pointCount**seriesCount < *batchSize { + *batchSize = *pointCount * *seriesCount + } + cfg := &runner.Config{ BatchSize: *batchSize, Measurements: ms, @@ -51,17 +55,23 @@ func main() { sort.Sort(sort.Reverse(sort.Interface(responseTimes))) - total := int64(0) + var total, mean int64 for _, t := range responseTimes { total += int64(t.Value) } - mean := total / int64(len(responseTimes)) + if len(responseTimes) > 0 { + mean = total / int64(len(responseTimes)) + } fmt.Printf("Wrote %d points at average rate of %.0f\n", totalPoints, float64(totalPoints)/timer.Elapsed().Seconds()) fmt.Printf("%d requests failed for %d total points that didn't get posted.\n", failedRequests, failedRequests**batchSize) fmt.Println("Average response time: ", time.Duration(mean)) fmt.Println("Slowest response times:") - for _, r := range responseTimes[:100] { + + if len(responseTimes) > 100 { + responseTimes = responseTimes[:100] + } + for _, r := range responseTimes { fmt.Println(time.Duration(r.Value)) } } From 19877a6d85004a4df1f8ae5aefc25fc1ec827f53 Mon Sep 17 00:00:00 2001 From: Jason Wilder Date: Mon, 21 Sep 2015 14:52:41 -0600 Subject: [PATCH 029/139] Add time and float compression Time compression uses an adaptive approach using delta-encoding, frame-of-reference, run length encoding as well as compressed integer encoding. Float compression uses an implementation of the Gorilla paper encoding for timestamps based on XOR deltas and leading and trailing null suppression. --- tsdb/engine/pd1/encoding.go | 83 ++++++- tsdb/engine/pd1/float.go | 206 +++++++++++++++++ tsdb/engine/pd1/float_test.go | 149 +++++++++++++ tsdb/engine/pd1/timestamp.go | 286 ++++++++++++++++++++++++ tsdb/engine/pd1/timestamp_test.go | 353 ++++++++++++++++++++++++++++++ 5 files changed, 1066 insertions(+), 11 deletions(-) create mode 100644 tsdb/engine/pd1/float.go create mode 100644 tsdb/engine/pd1/float_test.go create mode 100644 tsdb/engine/pd1/timestamp.go create mode 100644 tsdb/engine/pd1/timestamp_test.go diff --git a/tsdb/engine/pd1/encoding.go b/tsdb/engine/pd1/encoding.go index b21d394af5d..6b29913e0e0 100644 --- a/tsdb/engine/pd1/encoding.go +++ b/tsdb/engine/pd1/encoding.go @@ -4,7 +4,6 @@ import ( "sort" "time" - "github.com/dgryski/go-tsz" "github.com/influxdb/influxdb/tsdb" ) @@ -127,23 +126,59 @@ func (f *FloatValue) Size() int { return 16 } -// TODO: make this work with nanosecond timestamps func EncodeFloatBlock(buf []byte, values []*FloatValue) []byte { - s := tsz.New(uint32(values[0].Time().Unix())) + if len(values) == 0 { + return []byte{} + } + + // A float block is encoded using different compression strategies + // for timestamps and values. + + // Encode values using Gorilla float compression + venc := NewFloatEncoder() + + // Encode timestamps using an adaptive encoder that uses delta-encoding, + // frame-or-reference and run length encoding. + tsenc := NewTimeEncoder() + for _, v := range values { - s.Push(uint32(v.Time().Unix()), v.value) + tsenc.Write(v.Time()) + venc.Push(v.value) } - s.Finish() - return append(u64tob(uint64(values[0].Time().UnixNano())), s.Bytes()...) + venc.Finish() + + // Encoded timestamp values + tb, err := tsenc.Bytes() + if err != nil { + panic(err.Error()) + } + // Encoded float values + vb := venc.Bytes() + + // Preprend the first timestamp of the block in the first 8 bytes + return append(u64tob(uint64(values[0].Time().UnixNano())), + packBlock(tb, vb)...) } func DecodeFloatBlock(block []byte) ([]Value, error) { - iter, _ := tsz.NewIterator(block[8:]) - a := make([]Value, 0) - for iter.Next() { - t, f := iter.Values() - a = append(a, &FloatValue{time.Unix(int64(t), 0), f}) + // The first 8 bytes is the minimum timestamp of the block + tb, vb := unpackBlock(block[8:]) + + // Setup our timestamp and value decoders + dec := NewTimeDecoder(tb) + iter, err := NewFloatDecoder(vb) + if err != nil { + return nil, err + } + + // Decode both a timestamp and value + var a []Value + for dec.Next() && iter.Next() { + ts := dec.Read() + v := iter.Values() + a = append(a, &FloatValue{ts, v}) } + return a, nil } @@ -181,3 +216,29 @@ type StringValue struct { func EncodeStringBlock(buf []byte, values []StringValue) []byte { return nil } + +func packBlock(ts []byte, values []byte) []byte { + // We encode the length of the timestamp block using a variable byte encoding. + // This allows small byte slices to take up 1 byte while larger ones use 2 or more. + b := make([]byte, 10) + i := binary.PutUvarint(b, uint64(len(ts))) + + // block is , , + block := append(b[:i], ts...) + + // We don't encode the value length because we know it's the rest of the block after + // the timestamp block. + return append(block, values...) +} + +func unpackBlock(buf []byte) (ts, values []byte) { + // Unpack the timestamp block length + tsLen, i := binary.Uvarint(buf) + + // Unpack the timestamp bytes + ts = buf[int(i) : int(i)+int(tsLen)] + + // Unpack the value bytes + values = buf[int(i)+int(tsLen):] + return +} diff --git a/tsdb/engine/pd1/float.go b/tsdb/engine/pd1/float.go new file mode 100644 index 00000000000..dddb9f39b24 --- /dev/null +++ b/tsdb/engine/pd1/float.go @@ -0,0 +1,206 @@ +package pd1 + +/* +This code is originally from: https://github.com/dgryski/go-tsz and has been modified to remove +the timestamp compression fuctionality. + +It implements the float compression as presented in: http://www.vldb.org/pvldb/vol8/p1816-teller.pdf. +This implementation uses a sentinel value of NaN which means that float64 NaN cannot be stored using +this version. +*/ + +import ( + "bytes" + "math" + + "github.com/dgryski/go-bits" + "github.com/dgryski/go-bitstream" +) + +type FloatEncoder struct { + val float64 + + leading uint64 + trailing uint64 + + buf bytes.Buffer + bw *bitstream.BitWriter + + first bool + finished bool +} + +func NewFloatEncoder() *FloatEncoder { + s := FloatEncoder{ + first: true, + leading: ^uint64(0), + } + + s.bw = bitstream.NewWriter(&s.buf) + + return &s + +} + +func (s *FloatEncoder) Bytes() []byte { + return s.buf.Bytes() +} + +func (s *FloatEncoder) Finish() { + + if !s.finished { + // // write an end-of-stream record + s.Push(math.NaN()) + s.bw.Flush(bitstream.Zero) + s.finished = true + } +} + +func (s *FloatEncoder) Push(v float64) { + + if s.first { + // first point + s.val = v + s.first = false + s.bw.WriteBits(math.Float64bits(v), 64) + return + } + + vDelta := math.Float64bits(v) ^ math.Float64bits(s.val) + + if vDelta == 0 { + s.bw.WriteBit(bitstream.Zero) + } else { + s.bw.WriteBit(bitstream.One) + + leading := bits.Clz(vDelta) + trailing := bits.Ctz(vDelta) + + // TODO(dgryski): check if it's 'cheaper' to reset the leading/trailing bits instead + if s.leading != ^uint64(0) && leading >= s.leading && trailing >= s.trailing { + s.bw.WriteBit(bitstream.Zero) + s.bw.WriteBits(vDelta>>s.trailing, 64-int(s.leading)-int(s.trailing)) + } else { + s.leading, s.trailing = leading, trailing + + s.bw.WriteBit(bitstream.One) + s.bw.WriteBits(leading, 5) + + sigbits := 64 - leading - trailing + s.bw.WriteBits(sigbits, 6) + s.bw.WriteBits(vDelta>>trailing, int(sigbits)) + } + } + + s.val = v +} + +func (s *FloatEncoder) FloatDecoder() *FloatDecoder { + iter, _ := NewFloatDecoder(s.buf.Bytes()) + return iter +} + +type FloatDecoder struct { + val float64 + + leading uint64 + trailing uint64 + + br *bitstream.BitReader + + b []byte + + first bool + finished bool + + err error +} + +func NewFloatDecoder(b []byte) (*FloatDecoder, error) { + br := bitstream.NewReader(bytes.NewReader(b)) + + v, err := br.ReadBits(64) + if err != nil { + return nil, err + } + + return &FloatDecoder{ + val: math.Float64frombits(v), + first: true, + br: br, + b: b, + }, nil +} + +func (it *FloatDecoder) Next() bool { + if it.err != nil || it.finished { + return false + } + + if it.first { + it.first = false + return true + } + + // read compressed value + bit, err := it.br.ReadBit() + if err != nil { + it.err = err + return false + } + + if bit == bitstream.Zero { + // it.val = it.val + } else { + bit, err := it.br.ReadBit() + if err != nil { + it.err = err + return false + } + if bit == bitstream.Zero { + // reuse leading/trailing zero bits + // it.leading, it.trailing = it.leading, it.trailing + } else { + bits, err := it.br.ReadBits(5) + if err != nil { + it.err = err + return false + } + it.leading = bits + + bits, err = it.br.ReadBits(6) + if err != nil { + it.err = err + return false + } + mbits := bits + it.trailing = 64 - it.leading - mbits + } + + mbits := int(64 - it.leading - it.trailing) + bits, err := it.br.ReadBits(mbits) + if err != nil { + it.err = err + return false + } + vbits := math.Float64bits(it.val) + vbits ^= (bits << it.trailing) + + val := math.Float64frombits(vbits) + if math.IsNaN(val) { + it.finished = true + return false + } + it.val = val + } + + return true +} + +func (it *FloatDecoder) Values() float64 { + return it.val +} + +func (it *FloatDecoder) Err() error { + return it.err +} diff --git a/tsdb/engine/pd1/float_test.go b/tsdb/engine/pd1/float_test.go new file mode 100644 index 00000000000..5bfa377ac1b --- /dev/null +++ b/tsdb/engine/pd1/float_test.go @@ -0,0 +1,149 @@ +package pd1_test + +import ( + "testing" + + "github.com/influxdb/influxdb/tsdb/engine/pd1" +) + +func TestExampleEncoding(t *testing.T) { + + // Example from the paper + s := pd1.NewFloatEncoder() + + s.Push(12) + s.Push(12) + s.Push(24) + + // extra tests + + // floating point masking/shifting bug + s.Push(13) + s.Push(24) + + // delta-of-delta sizes + s.Push(24) + s.Push(24) + s.Push(24) + + s.Finish() + + it := s.FloatDecoder() + + want := []float64{ + 12, + 12, + 24, + + 13, + 24, + + 24, + 24, + 24, + } + + for _, w := range want { + if !it.Next() { + t.Fatalf("Next()=false, want true") + } + vv := it.Values() + if w != vv { + t.Errorf("Values()=(%v), want (%v)\n", vv, w) + } + } + + if it.Next() { + t.Fatalf("Next()=true, want false") + } + + if err := it.Err(); err != nil { + t.Errorf("it.Err()=%v, want nil", err) + } +} + +var TwoHoursData = []struct { + v float64 +}{ + // 2h of data + {761}, {727}, {763}, {706}, {700}, + {679}, {757}, {708}, {739}, {707}, + {699}, {740}, {729}, {766}, {730}, + {715}, {705}, {693}, {765}, {724}, + {799}, {761}, {737}, {766}, {756}, + {719}, {722}, {801}, {747}, {731}, + {742}, {744}, {791}, {750}, {759}, + {809}, {751}, {705}, {770}, {792}, + {727}, {762}, {772}, {721}, {748}, + {753}, {744}, {716}, {776}, {659}, + {789}, {766}, {758}, {690}, {795}, + {770}, {758}, {723}, {767}, {765}, + {693}, {706}, {681}, {727}, {724}, + {780}, {678}, {696}, {758}, {740}, + {735}, {700}, {742}, {747}, {752}, + {734}, {743}, {732}, {746}, {770}, + {780}, {710}, {731}, {712}, {712}, + {741}, {770}, {770}, {754}, {718}, + {670}, {775}, {749}, {795}, {756}, + {741}, {787}, {721}, {745}, {782}, + {765}, {780}, {811}, {790}, {836}, + {743}, {858}, {739}, {762}, {770}, + {752}, {763}, {795}, {792}, {746}, + {786}, {785}, {774}, {786}, {718}, +} + +func TestRoundtrip(t *testing.T) { + + s := pd1.NewFloatEncoder() + for _, p := range TwoHoursData { + s.Push(p.v) + } + s.Finish() + + it := s.FloatDecoder() + for _, w := range TwoHoursData { + if !it.Next() { + t.Fatalf("Next()=false, want true") + } + vv := it.Values() + // t.Logf("it.Values()=(%+v, %+v)\n", time.Unix(int64(tt), 0), vv) + if w.v != vv { + t.Errorf("Values()=(%v), want (%v)\n", vv, w.v) + } + } + + if it.Next() { + t.Fatalf("Next()=true, want false") + } + + if err := it.Err(); err != nil { + t.Errorf("it.Err()=%v, want nil", err) + } +} + +func BenchmarkFloatEncoder(b *testing.B) { + for i := 0; i < b.N; i++ { + s := pd1.NewFloatEncoder() + for _, tt := range TwoHoursData { + s.Push(tt.v) + } + s.Finish() + } +} + +func BenchmarkFloatDecoder(b *testing.B) { + s := pd1.NewFloatEncoder() + for _, tt := range TwoHoursData { + s.Push(tt.v) + } + s.Finish() + + b.ResetTimer() + + for i := 0; i < b.N; i++ { + it := s.FloatDecoder() + for j := 0; j < len(TwoHoursData); it.Next() { + j++ + } + } +} diff --git a/tsdb/engine/pd1/timestamp.go b/tsdb/engine/pd1/timestamp.go new file mode 100644 index 00000000000..b83199ef043 --- /dev/null +++ b/tsdb/engine/pd1/timestamp.go @@ -0,0 +1,286 @@ +// Package timestamp provides structs and functions for converting streams of timestamps +// to byte slices. +// +// The encoding is adapative based on structure of the timestamps that are encoded. By default, +// a bit-packed format that compresses multiple 64bit timestamps into a single 64bit word is used. +// If the values are too large to be compressed using the bit-packed format, it will fall back to +// a raw 8byte per timestamp format. If the the values can be run-length encoded, based on the +// differences between consectutive values, a shorter, variable sized RLE format is used. +package pd1 + +import ( + "encoding/binary" + "math" + "time" + + "github.com/jwilder/encoding/simple8b" +) + +const ( + // EncodingPacked is a bit-packed format + EncodingPacked = 0 + // EncodingRLE is a run-length encoded format + EncodingRLE = 1 + // EncodingRAW is a non-compressed format + EncodingRaw = 2 +) + +// TimeEncoder encodes time.Time to byte slices. +type TimeEncoder interface { + Write(t time.Time) + Bytes() ([]byte, error) +} + +// TimeEncoder decodes byte slices to time.Time values. +type TimeDecoder interface { + Next() bool + Read() time.Time +} + +type encoder struct { + ts []int64 +} + +// NewTimeEncoder returns a TimeEncoder +func NewTimeEncoder() TimeEncoder { + return &encoder{} +} + +// Write adds a time.Time to the compressed stream. +func (e *encoder) Write(t time.Time) { + e.ts = append(e.ts, t.UnixNano()) +} + +func (e *encoder) reduce() (min, max, divisor int64, rle bool, deltas []int64) { + // We make a copy of the timestamps so that if we end up using using RAW encoding, + // we still have the original values to encode. + deltas = make([]int64, len(e.ts)) + copy(deltas, e.ts) + + // Starting values for a min, max and divisor + min, max, divisor = e.ts[0], 0, 1e12 + + // First differential encode the values in place + for i := len(deltas) - 1; i > 0; i-- { + deltas[i] = deltas[i] - deltas[i-1] + + // We also want to keep track of the min, max and divisor so we don't + // have to loop again + v := deltas[i] + if v < min { + min = v + } + + if v > max { + max = v + } + + for { + // If our value is divisible by 10, break. Otherwise, try the next smallest divisor. + if v%divisor == 0 { + break + } + divisor /= 10 + } + } + + // Are the deltas able to be run-length encoded? + rle = true + for i := 1; i < len(deltas); i++ { + deltas[i] = (deltas[i] - min) / divisor + // Skip the first value || see if prev = curr. The deltas can be RLE if the are all equal. + rle = i == 1 || rle && (deltas[i-1] == deltas[i]) + } + + // No point RLE encoding 1 value + rle = rle && len(deltas) > 1 + return +} + +// Bytes returns the encoded bytes of all written times. +func (e *encoder) Bytes() ([]byte, error) { + if len(e.ts) == 0 { + return []byte{}, nil + } + + // Minimum, maxim and largest common divisor. rle is true if dts (the delta timestamps), + // are all the same. + min, max, div, rle, dts := e.reduce() + + // The deltas are all the same, so we can run-length encode them + if rle && len(e.ts) > 60 { + return e.encodeRLE(e.ts[0], e.ts[1]-e.ts[0], div, len(e.ts)) + } + + // We can't compress this time-range, the deltas exceed 1 << 60. That would mean that two + // adjacent timestamps are nanosecond resolution and ~36.5yr apart. + if max > simple8b.MaxValue { + return e.encodeRaw() + } + + // Otherwise, encode them in a compressed format + return e.encodePacked(min, div, dts) +} + +func (e *encoder) encodePacked(min, div int64, dts []int64) ([]byte, error) { + enc := simple8b.NewEncoder() + for _, v := range dts[1:] { + enc.Write(uint64(v)) + } + + b := make([]byte, 8*2+1) + + // 4 high bits used for the encoding type + b[0] = byte(EncodingPacked) << 4 + // 4 low bits are the log10 divisor + b[0] |= byte(math.Log10(float64(div))) + + // The minimum timestamp value + binary.BigEndian.PutUint64(b[1:9], uint64(min)) + + // The first delta value + binary.BigEndian.PutUint64(b[9:17], uint64(dts[0])) + + // The compressed deltas + deltas, err := enc.Bytes() + if err != nil { + return nil, err + } + + return append(b, deltas...), nil +} + +func (e *encoder) encodeRaw() ([]byte, error) { + b := make([]byte, 1+len(e.ts)*8) + b[0] = byte(EncodingRaw) << 4 + for i, v := range e.ts { + binary.BigEndian.PutUint64(b[1+i*8:1+i*8+8], uint64(v)) + } + return b, nil +} + +func (e *encoder) encodeRLE(first, delta, div int64, n int) ([]byte, error) { + // Large varints can take up to 10 bytes + b := make([]byte, 1+10*3) + + // 4 high bits used for the encoding type + b[0] = byte(EncodingRLE) << 4 + // 4 low bits are the log10 divisor + b[0] |= byte(math.Log10(float64(div))) + + i := 1 + // The first timestamp + binary.BigEndian.PutUint64(b[i:], uint64(first)) + i += 8 + // The first delta + i += binary.PutUvarint(b[i:], uint64(delta/div)) + // The number of times the delta is repeated + i += binary.PutUvarint(b[i:], uint64(n)) + + return b[:i], nil +} + +type decoder struct { + v time.Time + ts []int64 +} + +func NewTimeDecoder(b []byte) TimeDecoder { + d := &decoder{} + d.decode(b) + return d +} + +func (d *decoder) Next() bool { + if len(d.ts) == 0 { + return false + } + d.v = time.Unix(0, d.ts[0]) + d.ts = d.ts[1:] + return true +} + +func (d *decoder) Read() time.Time { + return d.v +} + +func (d *decoder) decode(b []byte) { + if len(b) == 0 { + return + } + + // Encoding type is stored in the 4 high bits of the first byte + encoding := b[0] >> 4 + switch encoding { + case EncodingRaw: + d.decodeRaw(b[1:]) + case EncodingRLE: + d.decodeRLE(b) + default: + d.decodePacked(b) + } +} + +func (d *decoder) decodePacked(b []byte) { + div := int64(math.Pow10(int(b[0] & 0xF))) + min := int64(binary.BigEndian.Uint64(b[1:9])) + first := int64(binary.BigEndian.Uint64(b[9:17])) + + enc := simple8b.NewDecoder(b[17:]) + + deltas := []int64{first} + for enc.Next() { + deltas = append(deltas, int64(enc.Read())) + } + + // Compute the prefix sum and scale the deltas back up + for i := 1; i < len(deltas); i++ { + deltas[i] = (deltas[i] * div) + min + deltas[i] = deltas[i-1] + deltas[i] + } + + d.ts = deltas +} + +func (d *decoder) decodeRLE(b []byte) { + var i, n int + + // Lower 4 bits hold the 10 based exponent so we can scale the values back up + div := int64(math.Pow10(int(b[i] & 0xF))) + i += 1 + + // Next 8 bytes is the starting timestamp + first := binary.BigEndian.Uint64(b[i : i+8]) + i += 8 + + // Next 1-10 bytes is our (scaled down by factor of 10) run length values + value, n := binary.Uvarint(b[i:]) + + // Scale the value back up + value *= uint64(div) + i += n + + // Last 1-10 bytes is how many times the value repeats + count, n := binary.Uvarint(b[i:]) + + // Rebuild construct the original values now + deltas := make([]int64, count) + for i := range deltas { + deltas[i] = int64(value) + } + + // Reverse the delta-encoding + deltas[0] = int64(first) + for i := 1; i < len(deltas); i++ { + deltas[i] = deltas[i-1] + deltas[i] + } + + d.ts = deltas +} + +func (d *decoder) decodeRaw(b []byte) { + d.ts = make([]int64, len(b)/8) + for i := range d.ts { + d.ts[i] = int64(binary.BigEndian.Uint64(b[i*8 : i*8+8])) + } +} diff --git a/tsdb/engine/pd1/timestamp_test.go b/tsdb/engine/pd1/timestamp_test.go new file mode 100644 index 00000000000..806fba7c5eb --- /dev/null +++ b/tsdb/engine/pd1/timestamp_test.go @@ -0,0 +1,353 @@ +package pd1_test + +import ( + "testing" + "time" + + "github.com/influxdb/influxdb/tsdb/engine/pd1" +) + +func Test_TimeEncoder(t *testing.T) { + enc := pd1.NewTimeEncoder() + + x := []time.Time{} + now := time.Unix(0, 0) + x = append(x, now) + enc.Write(now) + for i := 1; i < 4; i++ { + x = append(x, now.Add(time.Duration(i)*time.Second)) + enc.Write(x[i]) + } + + b, err := enc.Bytes() + if err != nil { + t.Fatalf("unexpected error: %v", err) + } + + dec := pd1.NewTimeDecoder(b) + for i, v := range x { + if !dec.Next() { + t.Fatalf("Next == false, expected true") + } + + if v != dec.Read() { + t.Fatalf("Item %d mismatch, got %v, exp %v", i, dec.Read(), v) + } + } +} + +func Test_TimeEncoder_NoValues(t *testing.T) { + enc := pd1.NewTimeEncoder() + b, err := enc.Bytes() + if err != nil { + t.Fatalf("unexpected error: %v", err) + } + + dec := pd1.NewTimeDecoder(b) + if dec.Next() { + t.Fatalf("unexpected next value: got true, exp false") + } +} + +func Test_TimeEncoder_One(t *testing.T) { + enc := pd1.NewTimeEncoder() + tm := time.Unix(0, 0) + + enc.Write(tm) + b, err := enc.Bytes() + if err != nil { + t.Fatalf("unexpected error: %v", err) + } + + dec := pd1.NewTimeDecoder(b) + if !dec.Next() { + t.Fatalf("unexpected next value: got true, exp false") + } + + if tm != dec.Read() { + t.Fatalf("read value mismatch: got %v, exp %v", dec.Read(), tm) + } +} + +func Test_TimeEncoder_Two(t *testing.T) { + enc := pd1.NewTimeEncoder() + t1 := time.Unix(0, 0) + t2 := time.Unix(0, 1) + enc.Write(t1) + enc.Write(t2) + + b, err := enc.Bytes() + if err != nil { + t.Fatalf("unexpected error: %v", err) + } + + dec := pd1.NewTimeDecoder(b) + if !dec.Next() { + t.Fatalf("unexpected next value: got true, exp false") + } + + if t1 != dec.Read() { + t.Fatalf("read value mismatch: got %v, exp %v", dec.Read(), t1) + } + + if !dec.Next() { + t.Fatalf("unexpected next value: got true, exp false") + } + + if t2 != dec.Read() { + t.Fatalf("read value mismatch: got %v, exp %v", dec.Read(), t2) + } +} + +func Test_TimeEncoder_Three(t *testing.T) { + enc := pd1.NewTimeEncoder() + t1 := time.Unix(0, 0) + t2 := time.Unix(0, 1) + t3 := time.Unix(0, 2) + + enc.Write(t1) + enc.Write(t2) + enc.Write(t3) + + b, err := enc.Bytes() + if err != nil { + t.Fatalf("unexpected error: %v", err) + } + + dec := pd1.NewTimeDecoder(b) + if !dec.Next() { + t.Fatalf("unexpected next value: got true, exp false") + } + + if t1 != dec.Read() { + t.Fatalf("read value mismatch: got %v, exp %v", dec.Read(), t1) + } + + if !dec.Next() { + t.Fatalf("unexpected next value: got true, exp false") + } + + if t2 != dec.Read() { + t.Fatalf("read value mismatch: got %v, exp %v", dec.Read(), t2) + } + + if !dec.Next() { + t.Fatalf("unexpected next value: got true, exp false") + } + + if t3 != dec.Read() { + t.Fatalf("read value mismatch: got %v, exp %v", dec.Read(), t3) + } +} + +func Test_TimeEncoder_Large_Range(t *testing.T) { + enc := pd1.NewTimeEncoder() + t1 := time.Unix(0, 1442369134000000000) + t2 := time.Unix(0, 1442369135000000000) + enc.Write(t1) + enc.Write(t2) + b, err := enc.Bytes() + if err != nil { + t.Fatalf("unexpected error: %v", err) + } + + dec := pd1.NewTimeDecoder(b) + if !dec.Next() { + t.Fatalf("unexpected next value: got true, exp false") + } + + if t1 != dec.Read() { + t.Fatalf("read value mismatch: got %v, exp %v", dec.Read(), t1) + } + + if !dec.Next() { + t.Fatalf("unexpected next value: got true, exp false") + } + + if t2 != dec.Read() { + t.Fatalf("read value mismatch: got %v, exp %v", dec.Read(), t2) + } +} + +func Test_TimeEncoder_Raw(t *testing.T) { + enc := pd1.NewTimeEncoder() + t1 := time.Unix(0, 0) + t2 := time.Unix(1, 0) + + // about 36.5yrs in NS resolution is max range for compressed format + // This should cause the encoding to fallback to raw points + t3 := time.Unix(2, (2 << 59)) + enc.Write(t1) + enc.Write(t2) + enc.Write(t3) + + b, err := enc.Bytes() + if err != nil { + t.Fatalf("expected error: %v", err) + } + + if exp := 25; len(b) != exp { + t.Fatalf("length mismatch: got %v, exp %v", len(b), exp) + } + + dec := pd1.NewTimeDecoder(b) + if !dec.Next() { + t.Fatalf("unexpected next value: got true, exp false") + } + + if t1 != dec.Read() { + t.Fatalf("read value mismatch: got %v, exp %v", dec.Read(), t1) + } + + if !dec.Next() { + t.Fatalf("unexpected next value: got true, exp false") + } + + if t2 != dec.Read() { + t.Fatalf("read value mismatch: got %v, exp %v", dec.Read(), t2) + } + + if !dec.Next() { + t.Fatalf("unexpected next value: got true, exp false") + } + + if t3 != dec.Read() { + t.Fatalf("read value mismatch: got %v, exp %v", dec.Read(), t3) + } +} + +func Test_TimeEncoder_RLE(t *testing.T) { + enc := pd1.NewTimeEncoder() + var ts []time.Time + for i := 0; i < 500; i++ { + ts = append(ts, time.Unix(int64(i), 0)) + } + + for _, v := range ts { + enc.Write(v) + } + + b, err := enc.Bytes() + if exp := 12; len(b) != exp { + t.Fatalf("length mismatch: got %v, exp %v", len(b), exp) + } + + if err != nil { + t.Fatalf("unexpected error: %v", err) + } + + dec := pd1.NewTimeDecoder(b) + for i, v := range ts { + if !dec.Next() { + t.Fatalf("Next == false, expected true") + } + + if v != dec.Read() { + t.Fatalf("Item %d mismatch, got %v, exp %v", i, dec.Read(), v) + } + } + + if dec.Next() { + t.Fatalf("unexpected extra values") + } +} + +func Test_TimeEncoder_Reverse(t *testing.T) { + enc := pd1.NewTimeEncoder() + ts := []time.Time{ + time.Unix(0, 3), + time.Unix(0, 2), + time.Unix(0, 1), + } + + for _, v := range ts { + enc.Write(v) + } + + b, err := enc.Bytes() + if err != nil { + t.Fatalf("unexpected error: %v", err) + } + + dec := pd1.NewTimeDecoder(b) + i := 0 + for dec.Next() { + if ts[i] != dec.Read() { + t.Fatalf("read value %d mismatch: got %v, exp %v", i, dec.Read(), ts[i]) + } + i += 1 + } +} + +func Test_TimeEncoder_220SecondDelta(t *testing.T) { + enc := pd1.NewTimeEncoder() + var ts []time.Time + for i := 0; i < 220; i++ { + ts = append(ts, time.Unix(int64(i), 0)) + } + + for _, v := range ts { + enc.Write(v) + } + + b, err := enc.Bytes() + if err != nil { + t.Fatalf("unexpected error: %v", err) + } + + // Using RLE, should get 12 bytes + if exp := 12; len(b) != exp { + t.Fatalf("unexpected length: got %v, exp %v", len(b), exp) + } + + dec := pd1.NewTimeDecoder(b) + i := 0 + for dec.Next() { + if ts[i] != dec.Read() { + t.Fatalf("read value %d mismatch: got %v, exp %v", i, dec.Read(), ts[i]) + } + i += 1 + } + + if i != len(ts) { + t.Fatalf("Read too few values: exp %d, got %d", len(ts), i) + } + + if dec.Next() { + t.Fatalf("expecte Next() = false, got true") + } +} + +func BenchmarkTimeEncoder(b *testing.B) { + enc := pd1.NewTimeEncoder() + x := make([]time.Time, 1024) + for i := 0; i < len(x); i++ { + x[i] = time.Now() + enc.Write(x[i]) + } + + b.ResetTimer() + for i := 0; i < b.N; i++ { + enc.Bytes() + } +} + +func BenchmarkTimeDecoder(b *testing.B) { + x := make([]time.Time, 1024) + enc := pd1.NewTimeEncoder() + for i := 0; i < len(x); i++ { + x[i] = time.Now() + enc.Write(x[i]) + } + bytes, _ := enc.Bytes() + + b.ResetTimer() + + for i := 0; i < b.N; i++ { + b.StopTimer() + dec := pd1.NewTimeDecoder(bytes) + b.StartTimer() + for dec.Next() { + } + } +} From bf74c7cf8620a3bd1174cad284c3579943f849fd Mon Sep 17 00:00:00 2001 From: Jason Wilder Date: Thu, 24 Sep 2015 14:29:51 -0600 Subject: [PATCH 030/139] Add int64 compression This is using zig zag encoding to convert int64 to uint64s and then using simple8b to compress them, falling back to uncompressed if the value exceeds 1 << 60. A patched encoding scheme would likely be better in general but this provides decent compression for integers that are not at the ends of the int64 range. --- tsdb/engine/pd1/encoding.go | 86 ++++++++++- tsdb/engine/pd1/encoding_test.go | 40 +++++ tsdb/engine/pd1/int.go | 113 +++++++++++++++ tsdb/engine/pd1/int_test.go | 241 +++++++++++++++++++++++++++++++ tsdb/engine/pd1/timestamp.go | 18 +-- 5 files changed, 480 insertions(+), 18 deletions(-) create mode 100644 tsdb/engine/pd1/int.go create mode 100644 tsdb/engine/pd1/int_test.go diff --git a/tsdb/engine/pd1/encoding.go b/tsdb/engine/pd1/encoding.go index 6b29913e0e0..055048620ce 100644 --- a/tsdb/engine/pd1/encoding.go +++ b/tsdb/engine/pd1/encoding.go @@ -1,12 +1,23 @@ package pd1 import ( + "encoding/binary" + "fmt" "sort" "time" "github.com/influxdb/influxdb/tsdb" ) +const ( + // EncodingPacked is a bit-packed format + EncodingPacked = 0 + // EncodingRLE is a run-length encoded format + EncodingRLE = 1 + // EncodingUncompressed is a non-compressed format + EncodingUncompressed = 2 +) + type Value interface { Time() time.Time UnixNano() int64 @@ -16,8 +27,8 @@ type Value interface { func NewValue(t time.Time, value interface{}) Value { switch v := value.(type) { - // case int64: - // return &Int64Value{time: t, value: v} + case int64: + return &Int64Value{time: t, value: v} case float64: return &FloatValue{time: t, value: v} // case bool: @@ -58,6 +69,13 @@ func (v Values) Encode(buf []byte) []byte { } return EncodeFloatBlock(buf, a) + case *Int64Value: + a := make([]*Int64Value, len(v)) + for i, vv := range v { + a[i] = vv.(*Int64Value) + } + return EncodeInt64Block(buf, a) + // TODO: add support for other types } @@ -69,6 +87,9 @@ func (v Values) DecodeSameTypeBlock(block []byte) Values { case *FloatValue: a, _ := DecodeFloatBlock(block) return a + case *Int64Value: + a, _ := DecodeInt64Block(block) + return a // TODO: add support for other types } @@ -200,12 +221,65 @@ type Int64Value struct { value int64 } -func EncodeInt64Block(buf []byte, values []Int64Value) []byte { - return nil +func (v *Int64Value) Time() time.Time { + return v.time } -func DecodeInt64Block(block []byte) ([]Int64Value, error) { - return nil, nil +func (v *Int64Value) Value() interface{} { + return v.value +} + +func (f *Int64Value) UnixNano() int64 { + return f.time.UnixNano() +} + +func (v *Int64Value) Size() int { + return 16 +} + +func (v *Int64Value) String() string { return fmt.Sprintf("%v", v.value) } + +func EncodeInt64Block(buf []byte, values []*Int64Value) []byte { + tsEnc := NewTimeEncoder() + vEnc := NewInt64Encoder() + for _, v := range values { + tsEnc.Write(v.Time()) + vEnc.Write(v.value) + } + + // Encoded timestamp values + tb, err := tsEnc.Bytes() + if err != nil { + panic(err.Error()) + } + // Encoded int64 values + vb, err := vEnc.Bytes() + if err != nil { + panic(err.Error()) + } + + // Preprend the first timestamp of the block in the first 8 bytes + return append(u64tob(uint64(values[0].Time().UnixNano())), + packBlock(tb, vb)...) +} + +func DecodeInt64Block(block []byte) ([]Value, error) { + // The first 8 bytes is the minimum timestamp of the block + tb, vb := unpackBlock(block[8:]) + + // Setup our timestamp and value decoders + tsDec := NewTimeDecoder(tb) + vDec := NewInt64Decoder(vb) + + // Decode both a timestamp and value + var a []Value + for tsDec.Next() && vDec.Next() { + ts := tsDec.Read() + v := vDec.Read() + a = append(a, &Int64Value{ts, v}) + } + + return a, nil } type StringValue struct { diff --git a/tsdb/engine/pd1/encoding_test.go b/tsdb/engine/pd1/encoding_test.go index 02598a764da..49006085d7d 100644 --- a/tsdb/engine/pd1/encoding_test.go +++ b/tsdb/engine/pd1/encoding_test.go @@ -1,6 +1,8 @@ package pd1_test import ( + // "math/rand" + "reflect" "testing" "time" @@ -25,6 +27,44 @@ func TestEncoding_FloatBlock(t *testing.T) { } } +func TestEncoding_IntBlock(t *testing.T) { + valueCount := 1000 + times := getTimes(valueCount, 60, time.Second) + values := make(pd1.Values, len(times)) + for i, t := range times { + values[i] = pd1.NewValue(t, int64(i)) + } + + b := values.Encode(nil) + + decodedValues := values.DecodeSameTypeBlock(b) + + if !reflect.DeepEqual(decodedValues, values) { + t.Fatalf("unexpected results:\n\tgot: %v\n\texp: %v\n", decodedValues, values) + } +} + +func TestEncoding_IntBlock_Negatives(t *testing.T) { + valueCount := 1000 + times := getTimes(valueCount, 60, time.Second) + values := make(pd1.Values, len(times)) + for i, t := range times { + v := int64(i) + if i%2 == 0 { + v = -v + } + values[i] = pd1.NewValue(t, int64(v)) + } + + b := values.Encode(nil) + + decodedValues := values.DecodeSameTypeBlock(b) + + if !reflect.DeepEqual(decodedValues, values) { + t.Fatalf("unexpected results:\n\tgot: %v\n\texp: %v\n", decodedValues, values) + } +} + func getTimes(n, step int, precision time.Duration) []time.Time { t := time.Now().Round(precision) a := make([]time.Time, n) diff --git a/tsdb/engine/pd1/int.go b/tsdb/engine/pd1/int.go new file mode 100644 index 00000000000..04ddc1a0f4f --- /dev/null +++ b/tsdb/engine/pd1/int.go @@ -0,0 +1,113 @@ +package pd1 + +import ( + "encoding/binary" + "fmt" + + "github.com/jwilder/encoding/simple8b" +) + +type int64Encoder struct { + values []int64 +} + +func NewInt64Encoder() *int64Encoder { + return &int64Encoder{} +} + +func (e *int64Encoder) Write(v int64) { + e.values = append(e.values, v) +} + +func (e *int64Encoder) zigZagEncode(x int64) uint64 { + return uint64(uint64(x<<1) ^ uint64((int64(x) >> 63))) +} + +func (e *int64Encoder) Bytes() ([]byte, error) { + enc := simple8b.NewEncoder() + + for _, v := range e.values { + n := e.zigZagEncode(v) + // Value is too large to encode using packed format + if n > simple8b.MaxValue { + return e.encodeUncompressed() + } + enc.Write(n) + } + + b, err := enc.Bytes() + if err != nil { + return nil, err + } + + return append([]byte{EncodingPacked << 4}, b...), nil +} + +func (e *int64Encoder) encodeUncompressed() ([]byte, error) { + b := make([]byte, 1+len(e.values)*8) + // 4 high bits of first byte store the encoding type for the block + b[0] = byte(EncodingUncompressed) << 4 + for i, v := range e.values { + binary.BigEndian.PutUint64(b[1+i*8:1+i*8+8], uint64(v)) + } + return b, nil +} + +type int64Decoder struct { + values []int64 + v int64 +} + +func NewInt64Decoder(b []byte) *int64Decoder { + d := &int64Decoder{} + d.decode(b) + return d +} + +func (d *int64Decoder) zigZagDecode(v uint64) int64 { + return int64((v >> 1) ^ uint64((int64(v&1)<<63)>>63)) +} + +func (d *int64Decoder) Next() bool { + if len(d.values) == 0 { + return false + } + d.v = d.values[0] + d.values = d.values[1:] + return true +} + +func (d *int64Decoder) Read() int64 { + return d.v +} + +func (d *int64Decoder) decode(b []byte) { + if len(b) == 0 { + return + } + + // Encoding type is stored in the 4 high bits of the first byte + encoding := b[0] >> 4 + switch encoding { + case EncodingUncompressed: + d.decodeUncompressed(b[1:]) + case EncodingPacked: + d.decodePacked(b[1:]) + default: + panic(fmt.Sprintf("unknown encoding %v", encoding)) + } +} + +func (d *int64Decoder) decodePacked(b []byte) { + dec := simple8b.NewDecoder(b) + for dec.Next() { + d.values = append(d.values, d.zigZagDecode(dec.Read())) + } +} + +func (d *int64Decoder) decodeUncompressed(b []byte) { + d.values = make([]int64, len(b)/8) + for i := range d.values { + d.values[i] = int64(binary.BigEndian.Uint64(b[i*8 : i*8+8])) + } +} diff --git a/tsdb/engine/pd1/int_test.go b/tsdb/engine/pd1/int_test.go new file mode 100644 index 00000000000..82042f77bd4 --- /dev/null +++ b/tsdb/engine/pd1/int_test.go @@ -0,0 +1,241 @@ +package pd1_test + +import ( + "math" + "testing" + + "github.com/influxdb/influxdb/tsdb/engine/pd1" +) + +func Test_Int64Encoder_NoValues(t *testing.T) { + enc := pd1.NewInt64Encoder() + b, err := enc.Bytes() + if err != nil { + t.Fatalf("unexpected error: %v", err) + } + + dec := pd1.NewInt64Decoder(b) + if dec.Next() { + t.Fatalf("unexpected next value: got true, exp false") + } +} + +func Test_Int64Encoder_One(t *testing.T) { + enc := pd1.NewInt64Encoder() + v1 := int64(1) + + enc.Write(1) + b, err := enc.Bytes() + if err != nil { + t.Fatalf("unexpected error: %v", err) + } + + dec := pd1.NewInt64Decoder(b) + if !dec.Next() { + t.Fatalf("unexpected next value: got true, exp false") + } + + if v1 != dec.Read() { + t.Fatalf("read value mismatch: got %v, exp %v", dec.Read(), v1) + } +} + +func Test_Int64Encoder_Two(t *testing.T) { + enc := pd1.NewInt64Encoder() + var v1, v2 int64 = 1, 2 + + enc.Write(v1) + enc.Write(v2) + + b, err := enc.Bytes() + if err != nil { + t.Fatalf("unexpected error: %v", err) + } + + dec := pd1.NewInt64Decoder(b) + if !dec.Next() { + t.Fatalf("unexpected next value: got true, exp false") + } + + if v1 != dec.Read() { + t.Fatalf("read value mismatch: got %v, exp %v", dec.Read(), v1) + } + + if !dec.Next() { + t.Fatalf("unexpected next value: got true, exp false") + } + + if v2 != dec.Read() { + t.Fatalf("read value mismatch: got %v, exp %v", dec.Read(), v2) + } +} + +func Test_Int64Encoder_Negative(t *testing.T) { + enc := pd1.NewInt64Encoder() + var v1, v2, v3 int64 = -2, 0, 1 + + enc.Write(v1) + enc.Write(v2) + enc.Write(v3) + + b, err := enc.Bytes() + if err != nil { + t.Fatalf("unexpected error: %v", err) + } + + dec := pd1.NewInt64Decoder(b) + if !dec.Next() { + t.Fatalf("unexpected next value: got true, exp false") + } + + if v1 != dec.Read() { + t.Fatalf("read value mismatch: got %v, exp %v", dec.Read(), v1) + } + + if !dec.Next() { + t.Fatalf("unexpected next value: got true, exp false") + } + + if v2 != dec.Read() { + t.Fatalf("read value mismatch: got %v, exp %v", dec.Read(), v2) + } + + if !dec.Next() { + t.Fatalf("unexpected next value: got true, exp false") + } + + if v3 != dec.Read() { + t.Fatalf("read value mismatch: got %v, exp %v", dec.Read(), v3) + } +} + +func Test_Int64Encoder_Large_Range(t *testing.T) { + enc := pd1.NewInt64Encoder() + var v1, v2 int64 = math.MinInt64, math.MaxInt64 + enc.Write(v1) + enc.Write(v2) + b, err := enc.Bytes() + if err != nil { + t.Fatalf("unexpected error: %v", err) + } + + dec := pd1.NewInt64Decoder(b) + if !dec.Next() { + t.Fatalf("unexpected next value: got true, exp false") + } + + if v1 != dec.Read() { + t.Fatalf("read value mismatch: got %v, exp %v", dec.Read(), v1) + } + + if !dec.Next() { + t.Fatalf("unexpected next value: got true, exp false") + } + + if v2 != dec.Read() { + t.Fatalf("read value mismatch: got %v, exp %v", dec.Read(), v2) + } +} + +func Test_Int64Encoder_Uncompressed(t *testing.T) { + enc := pd1.NewInt64Encoder() + var v1, v2, v3 int64 = 0, 1, 1 << 60 + + enc.Write(v1) + enc.Write(v2) + enc.Write(v3) + + b, err := enc.Bytes() + if err != nil { + t.Fatalf("expected error: %v", err) + } + + // 1 byte header + 3 * 8 byte values + if exp := 25; len(b) != exp { + t.Fatalf("length mismatch: got %v, exp %v", len(b), exp) + } + + dec := pd1.NewInt64Decoder(b) + if !dec.Next() { + t.Fatalf("unexpected next value: got true, exp false") + } + + if v1 != dec.Read() { + t.Fatalf("read value mismatch: got %v, exp %v", dec.Read(), v1) + } + + if !dec.Next() { + t.Fatalf("unexpected next value: got true, exp false") + } + + if v2 != dec.Read() { + t.Fatalf("read value mismatch: got %v, exp %v", dec.Read(), v2) + } + + if !dec.Next() { + t.Fatalf("unexpected next value: got true, exp false") + } + + if v3 != dec.Read() { + t.Fatalf("read value mismatch: got %v, exp %v", dec.Read(), v3) + } +} + +func Test_Int64Encoder_AllNegative(t *testing.T) { + enc := pd1.NewInt64Encoder() + values := []int64{ + -10, -5, -1, + } + + for _, v := range values { + enc.Write(v) + } + + b, err := enc.Bytes() + if err != nil { + t.Fatalf("unexpected error: %v", err) + } + + dec := pd1.NewInt64Decoder(b) + i := 0 + for dec.Next() { + if values[i] != dec.Read() { + t.Fatalf("read value %d mismatch: got %v, exp %v", i, dec.Read(), values[i]) + } + i += 1 + } +} + +func BenchmarkInt64Encoder(b *testing.B) { + enc := pd1.NewInt64Encoder() + x := make([]int64, 1024) + for i := 0; i < len(x); i++ { + x[i] = int64(i) + enc.Write(x[i]) + } + + b.ResetTimer() + for i := 0; i < b.N; i++ { + enc.Bytes() + } +} + +func BenchmarkInt64Decoder(b *testing.B) { + x := make([]int64, 1024) + enc := pd1.NewInt64Encoder() + for i := 0; i < len(x); i++ { + x[i] = int64(i) + enc.Write(x[i]) + } + bytes, _ := enc.Bytes() + + b.ResetTimer() + + for i := 0; i < b.N; i++ { + b.StopTimer() + dec := pd1.NewInt64Decoder(bytes) + b.StartTimer() + for dec.Next() { + } + } +} diff --git a/tsdb/engine/pd1/timestamp.go b/tsdb/engine/pd1/timestamp.go index b83199ef043..64907bb7607 100644 --- a/tsdb/engine/pd1/timestamp.go +++ b/tsdb/engine/pd1/timestamp.go @@ -10,21 +10,13 @@ package pd1 import ( "encoding/binary" + "fmt" "math" "time" "github.com/jwilder/encoding/simple8b" ) -const ( - // EncodingPacked is a bit-packed format - EncodingPacked = 0 - // EncodingRLE is a run-length encoded format - EncodingRLE = 1 - // EncodingRAW is a non-compressed format - EncodingRaw = 2 -) - // TimeEncoder encodes time.Time to byte slices. type TimeEncoder interface { Write(t time.Time) @@ -152,7 +144,7 @@ func (e *encoder) encodePacked(min, div int64, dts []int64) ([]byte, error) { func (e *encoder) encodeRaw() ([]byte, error) { b := make([]byte, 1+len(e.ts)*8) - b[0] = byte(EncodingRaw) << 4 + b[0] = byte(EncodingUncompressed) << 4 for i, v := range e.ts { binary.BigEndian.PutUint64(b[1+i*8:1+i*8+8], uint64(v)) } @@ -212,12 +204,14 @@ func (d *decoder) decode(b []byte) { // Encoding type is stored in the 4 high bits of the first byte encoding := b[0] >> 4 switch encoding { - case EncodingRaw: + case EncodingUncompressed: d.decodeRaw(b[1:]) case EncodingRLE: d.decodeRLE(b) - default: + case EncodingPacked: d.decodePacked(b) + default: + panic(fmt.Sprintf("unknown encoding: %v", encoding)) } } From fce01a24665a3b64e171bcfaa6fe2e3c490bb082 Mon Sep 17 00:00:00 2001 From: Jason Wilder Date: Thu, 24 Sep 2015 16:42:48 -0600 Subject: [PATCH 031/139] Use zigzag encoding for timestamp deltas Previously were using a frame of reference approach where we would transform the (possibly negative) deltas into positive values from the minimum. That required an extra pass over the values as well as a large slice allocation so we could encode the originals in uncompressed form if they were too large. This switches the encoding to use zigzag encoding for the deltas which removes the extra slice allocation as well as the extra loops. Improves encoding performane by ~4x. --- tsdb/engine/pd1/encoding.go | 11 +++ tsdb/engine/pd1/encoding_test.go | 19 ++++-- tsdb/engine/pd1/int.go | 12 +--- tsdb/engine/pd1/timestamp.go | 109 ++++++++++++++---------------- tsdb/engine/pd1/timestamp_test.go | 3 +- 5 files changed, 82 insertions(+), 72 deletions(-) diff --git a/tsdb/engine/pd1/encoding.go b/tsdb/engine/pd1/encoding.go index 055048620ce..a2e16699456 100644 --- a/tsdb/engine/pd1/encoding.go +++ b/tsdb/engine/pd1/encoding.go @@ -316,3 +316,14 @@ func unpackBlock(buf []byte) (ts, values []byte) { values = buf[int(i)+int(tsLen):] return } + +// ZigZagEncode converts a int64 to a uint64 by zig zagging negative and positive values +// across even and odd numbers. Eg. [0,-1,1,-2] becomes [0, 1, 2, 3] +func ZigZagEncode(x int64) uint64 { + return uint64(uint64(x<<1) ^ uint64((int64(x) >> 63))) +} + +// ZigZagDecode converts a previously zigzag encoded uint64 back to a int64 +func ZigZagDecode(v uint64) int64 { + return int64((v >> 1) ^ uint64((int64(v&1)<<63)>>63)) +} diff --git a/tsdb/engine/pd1/encoding_test.go b/tsdb/engine/pd1/encoding_test.go index 49006085d7d..c249f1aa8ce 100644 --- a/tsdb/engine/pd1/encoding_test.go +++ b/tsdb/engine/pd1/encoding_test.go @@ -27,7 +27,7 @@ func TestEncoding_FloatBlock(t *testing.T) { } } -func TestEncoding_IntBlock(t *testing.T) { +func TestEncoding_IntBlock_Basic(t *testing.T) { valueCount := 1000 times := getTimes(valueCount, 60, time.Second) values := make(pd1.Values, len(times)) @@ -39,8 +39,19 @@ func TestEncoding_IntBlock(t *testing.T) { decodedValues := values.DecodeSameTypeBlock(b) - if !reflect.DeepEqual(decodedValues, values) { - t.Fatalf("unexpected results:\n\tgot: %v\n\texp: %v\n", decodedValues, values) + if len(decodedValues) != len(values) { + t.Fatalf("unexpected results length:\n\tgot: %v\n\texp: %v\n", len(decodedValues), len(values)) + } + + for i := 0; i < len(decodedValues); i++ { + + if decodedValues[i].Time() != values[i].Time() { + t.Fatalf("unexpected results:\n\tgot: %v\n\texp: %v\n", decodedValues[i].Time(), values[i].Time()) + } + + if decodedValues[i].Value() != values[i].Value() { + t.Fatalf("unexpected results:\n\tgot: %v\n\texp: %v\n", decodedValues[i].Value(), values[i].Value()) + } } } @@ -69,7 +80,7 @@ func getTimes(n, step int, precision time.Duration) []time.Time { t := time.Now().Round(precision) a := make([]time.Time, n) for i := 0; i < n; i++ { - a[i] = t.Add(60 * precision) + a[i] = t.Add(time.Duration(i*60) * precision) } return a } diff --git a/tsdb/engine/pd1/int.go b/tsdb/engine/pd1/int.go index 04ddc1a0f4f..c4c40b8778d 100644 --- a/tsdb/engine/pd1/int.go +++ b/tsdb/engine/pd1/int.go @@ -19,15 +19,11 @@ func (e *int64Encoder) Write(v int64) { e.values = append(e.values, v) } -func (e *int64Encoder) zigZagEncode(x int64) uint64 { - return uint64(uint64(x<<1) ^ uint64((int64(x) >> 63))) -} - func (e *int64Encoder) Bytes() ([]byte, error) { enc := simple8b.NewEncoder() for _, v := range e.values { - n := e.zigZagEncode(v) + n := ZigZagEncode(v) // Value is too large to encode using packed format if n > simple8b.MaxValue { return e.encodeUncompressed() @@ -64,10 +60,6 @@ func NewInt64Decoder(b []byte) *int64Decoder { return d } -func (d *int64Decoder) zigZagDecode(v uint64) int64 { - return int64((v >> 1) ^ uint64((int64(v&1)<<63)>>63)) -} - func (d *int64Decoder) Next() bool { if len(d.values) == 0 { return false @@ -101,7 +93,7 @@ func (d *int64Decoder) decode(b []byte) { func (d *int64Decoder) decodePacked(b []byte) { dec := simple8b.NewDecoder(b) for dec.Next() { - d.values = append(d.values, d.zigZagDecode(dec.Read())) + d.values = append(d.values, ZigZagDecode(dec.Read())) } } diff --git a/tsdb/engine/pd1/timestamp.go b/tsdb/engine/pd1/timestamp.go index 64907bb7607..9119dbd5182 100644 --- a/tsdb/engine/pd1/timestamp.go +++ b/tsdb/engine/pd1/timestamp.go @@ -30,7 +30,7 @@ type TimeDecoder interface { } type encoder struct { - ts []int64 + ts []uint64 } // NewTimeEncoder returns a TimeEncoder @@ -40,28 +40,29 @@ func NewTimeEncoder() TimeEncoder { // Write adds a time.Time to the compressed stream. func (e *encoder) Write(t time.Time) { - e.ts = append(e.ts, t.UnixNano()) + e.ts = append(e.ts, uint64(t.UnixNano())) } -func (e *encoder) reduce() (min, max, divisor int64, rle bool, deltas []int64) { - // We make a copy of the timestamps so that if we end up using using RAW encoding, - // we still have the original values to encode. - deltas = make([]int64, len(e.ts)) - copy(deltas, e.ts) +func (e *encoder) reduce() (max, divisor uint64, rle bool, deltas []uint64) { + // Compute the deltas in place to avoid allocating another slice + deltas = e.ts + // Starting values for a max and divisor + max, divisor = 0, 1e12 - // Starting values for a min, max and divisor - min, max, divisor = e.ts[0], 0, 1e12 + // Indicates whether the the deltas can be run-length encoded + rle = true - // First differential encode the values in place + // Interate in reverse so we can apply deltas in place for i := len(deltas) - 1; i > 0; i-- { - deltas[i] = deltas[i] - deltas[i-1] - // We also want to keep track of the min, max and divisor so we don't - // have to loop again + // First differential encode the values + delta := int64(deltas[i] - deltas[i-1]) + + // The delta may be negative so zigzag encode it into a postive value + deltas[i] = ZigZagEncode(delta) + + // We're also need to keep track of the max value and largest common divisor v := deltas[i] - if v < min { - min = v - } if v > max { max = v @@ -74,18 +75,10 @@ func (e *encoder) reduce() (min, max, divisor int64, rle bool, deltas []int64) { } divisor /= 10 } - } - // Are the deltas able to be run-length encoded? - rle = true - for i := 1; i < len(deltas); i++ { - deltas[i] = (deltas[i] - min) / divisor // Skip the first value || see if prev = curr. The deltas can be RLE if the are all equal. - rle = i == 1 || rle && (deltas[i-1] == deltas[i]) + rle = i != 0 || rle && (deltas[i-1] == deltas[i]) } - - // No point RLE encoding 1 value - rle = rle && len(deltas) > 1 return } @@ -95,43 +88,38 @@ func (e *encoder) Bytes() ([]byte, error) { return []byte{}, nil } - // Minimum, maxim and largest common divisor. rle is true if dts (the delta timestamps), + // Maximum and largest common divisor. rle is true if dts (the delta timestamps), // are all the same. - min, max, div, rle, dts := e.reduce() + max, mod, rle, dts := e.reduce() // The deltas are all the same, so we can run-length encode them if rle && len(e.ts) > 60 { - return e.encodeRLE(e.ts[0], e.ts[1]-e.ts[0], div, len(e.ts)) + return e.encodeRLE(e.ts[0], e.ts[1], mod, len(e.ts)) } - // We can't compress this time-range, the deltas exceed 1 << 60. That would mean that two - // adjacent timestamps are nanosecond resolution and ~36.5yr apart. + // We can't compress this time-range, the deltas exceed 1 << 60 if max > simple8b.MaxValue { return e.encodeRaw() } - // Otherwise, encode them in a compressed format - return e.encodePacked(min, div, dts) + return e.encodePacked(mod, dts) } -func (e *encoder) encodePacked(min, div int64, dts []int64) ([]byte, error) { +func (e *encoder) encodePacked(div uint64, dts []uint64) ([]byte, error) { enc := simple8b.NewEncoder() for _, v := range dts[1:] { - enc.Write(uint64(v)) + enc.Write(uint64(v) / div) } - b := make([]byte, 8*2+1) + b := make([]byte, 8+1) // 4 high bits used for the encoding type b[0] = byte(EncodingPacked) << 4 // 4 low bits are the log10 divisor b[0] |= byte(math.Log10(float64(div))) - // The minimum timestamp value - binary.BigEndian.PutUint64(b[1:9], uint64(min)) - // The first delta value - binary.BigEndian.PutUint64(b[9:17], uint64(dts[0])) + binary.BigEndian.PutUint64(b[1:9], uint64(dts[0])) // The compressed deltas deltas, err := enc.Bytes() @@ -151,7 +139,7 @@ func (e *encoder) encodeRaw() ([]byte, error) { return b, nil } -func (e *encoder) encodeRLE(first, delta, div int64, n int) ([]byte, error) { +func (e *encoder) encodeRLE(first, delta, div uint64, n int) ([]byte, error) { // Large varints can take up to 10 bytes b := make([]byte, 1+10*3) @@ -174,7 +162,7 @@ func (e *encoder) encodeRLE(first, delta, div int64, n int) ([]byte, error) { type decoder struct { v time.Time - ts []int64 + ts []uint64 } func NewTimeDecoder(b []byte) TimeDecoder { @@ -187,7 +175,7 @@ func (d *decoder) Next() bool { if len(d.ts) == 0 { return false } - d.v = time.Unix(0, d.ts[0]) + d.v = time.Unix(0, int64(d.ts[0])) d.ts = d.ts[1:] return true } @@ -216,21 +204,20 @@ func (d *decoder) decode(b []byte) { } func (d *decoder) decodePacked(b []byte) { - div := int64(math.Pow10(int(b[0] & 0xF))) - min := int64(binary.BigEndian.Uint64(b[1:9])) - first := int64(binary.BigEndian.Uint64(b[9:17])) + div := uint64(math.Pow10(int(b[0] & 0xF))) + first := uint64(binary.BigEndian.Uint64(b[1:9])) - enc := simple8b.NewDecoder(b[17:]) + enc := simple8b.NewDecoder(b[9:]) - deltas := []int64{first} + deltas := []uint64{first} for enc.Next() { - deltas = append(deltas, int64(enc.Read())) + deltas = append(deltas, enc.Read()) } // Compute the prefix sum and scale the deltas back up for i := 1; i < len(deltas); i++ { - deltas[i] = (deltas[i] * div) + min - deltas[i] = deltas[i-1] + deltas[i] + dgap := ZigZagDecode(deltas[i] * div) + deltas[i] = uint64(int64(deltas[i-1]) + dgap) } d.ts = deltas @@ -240,7 +227,7 @@ func (d *decoder) decodeRLE(b []byte) { var i, n int // Lower 4 bits hold the 10 based exponent so we can scale the values back up - div := int64(math.Pow10(int(b[i] & 0xF))) + mod := int64(math.Pow10(int(b[i] & 0xF))) i += 1 // Next 8 bytes is the starting timestamp @@ -250,21 +237,23 @@ func (d *decoder) decodeRLE(b []byte) { // Next 1-10 bytes is our (scaled down by factor of 10) run length values value, n := binary.Uvarint(b[i:]) + value = uint64(ZigZagDecode(value)) + // Scale the value back up - value *= uint64(div) + value *= uint64(mod) i += n // Last 1-10 bytes is how many times the value repeats count, n := binary.Uvarint(b[i:]) // Rebuild construct the original values now - deltas := make([]int64, count) + deltas := make([]uint64, count) for i := range deltas { - deltas[i] = int64(value) + deltas[i] = value } // Reverse the delta-encoding - deltas[0] = int64(first) + deltas[0] = first for i := 1; i < len(deltas); i++ { deltas[i] = deltas[i-1] + deltas[i] } @@ -273,8 +262,14 @@ func (d *decoder) decodeRLE(b []byte) { } func (d *decoder) decodeRaw(b []byte) { - d.ts = make([]int64, len(b)/8) + d.ts = make([]uint64, len(b)/8) for i := range d.ts { - d.ts[i] = int64(binary.BigEndian.Uint64(b[i*8 : i*8+8])) + d.ts[i] = binary.BigEndian.Uint64(b[i*8 : i*8+8]) + + delta := ZigZagDecode(d.ts[i]) + // Compute the prefix sum and scale the deltas back up + if i > 0 { + d.ts[i] = uint64(int64(d.ts[i-1]) + delta) + } } } diff --git a/tsdb/engine/pd1/timestamp_test.go b/tsdb/engine/pd1/timestamp_test.go index 806fba7c5eb..da78b2b6e85 100644 --- a/tsdb/engine/pd1/timestamp_test.go +++ b/tsdb/engine/pd1/timestamp_test.go @@ -282,8 +282,9 @@ func Test_TimeEncoder_Reverse(t *testing.T) { func Test_TimeEncoder_220SecondDelta(t *testing.T) { enc := pd1.NewTimeEncoder() var ts []time.Time + now := time.Now() for i := 0; i < 220; i++ { - ts = append(ts, time.Unix(int64(i), 0)) + ts = append(ts, now.Add(time.Duration(i*60)*time.Second)) } for _, v := range ts { From 8ce3d7564d594f7ac3e5276935955b85c7b7ace5 Mon Sep 17 00:00:00 2001 From: Jason Wilder Date: Fri, 25 Sep 2015 13:30:24 -0600 Subject: [PATCH 032/139] Reduce memory allocations Converting between different encoders is wasting a lot of memory allocating different typed slices. --- tsdb/engine/pd1/int.go | 82 ++++++++++++++++++++++++++++--------- tsdb/engine/pd1/int_test.go | 10 +++-- 2 files changed, 70 insertions(+), 22 deletions(-) diff --git a/tsdb/engine/pd1/int.go b/tsdb/engine/pd1/int.go index c4c40b8778d..706ca5fce8e 100644 --- a/tsdb/engine/pd1/int.go +++ b/tsdb/engine/pd1/int.go @@ -7,42 +7,62 @@ import ( "github.com/jwilder/encoding/simple8b" ) +type Int64Encoder interface { + Write(v int64) + Bytes() ([]byte, error) +} + +type Int64Decoder interface { + Next() bool + Read() int64 +} + type int64Encoder struct { - values []int64 + values []uint64 } -func NewInt64Encoder() *int64Encoder { +func NewInt64Encoder() Int64Encoder { return &int64Encoder{} } func (e *int64Encoder) Write(v int64) { - e.values = append(e.values, v) + e.values = append(e.values, ZigZagEncode(v)) } func (e *int64Encoder) Bytes() ([]byte, error) { - enc := simple8b.NewEncoder() - for _, v := range e.values { - n := ZigZagEncode(v) // Value is too large to encode using packed format - if n > simple8b.MaxValue { + if v > simple8b.MaxValue { return e.encodeUncompressed() } - enc.Write(n) } - b, err := enc.Bytes() + return e.encodePacked() +} + +func (e *int64Encoder) encodePacked() ([]byte, error) { + encoded, err := simple8b.Encode(e.values) if err != nil { return nil, err } - return append([]byte{EncodingPacked << 4}, b...), nil + b := make([]byte, 1+len(encoded)*8+4) + // 4 high bits of first byte store the encoding type for the block + b[0] = byte(EncodingPacked) << 4 + + binary.BigEndian.PutUint32(b[1:5], uint32(len(e.values))) + + for i, v := range encoded { + binary.BigEndian.PutUint64(b[5+i*8:5+i*8+8], v) + } + return b, nil } func (e *int64Encoder) encodeUncompressed() ([]byte, error) { b := make([]byte, 1+len(e.values)*8) // 4 high bits of first byte store the encoding type for the block b[0] = byte(EncodingUncompressed) << 4 + for i, v := range e.values { binary.BigEndian.PutUint64(b[1+i*8:1+i*8+8], uint64(v)) } @@ -50,21 +70,30 @@ func (e *int64Encoder) encodeUncompressed() ([]byte, error) { } type int64Decoder struct { - values []int64 + values []uint64 v int64 + buf []uint64 + vbuf []uint64 } -func NewInt64Decoder(b []byte) *int64Decoder { - d := &int64Decoder{} +func NewInt64Decoder(b []byte) Int64Decoder { + d := &int64Decoder{ + buf: make([]uint64, 240), + vbuf: make([]uint64, 1), + } d.decode(b) return d } +func (d *int64Decoder) SetBytes(b []byte) { + d.decode(b) +} + func (d *int64Decoder) Next() bool { if len(d.values) == 0 { return false } - d.v = d.values[0] + d.v = ZigZagDecode(d.values[0]) d.values = d.values[1:] return true } @@ -91,15 +120,30 @@ func (d *int64Decoder) decode(b []byte) { } func (d *int64Decoder) decodePacked(b []byte) { - dec := simple8b.NewDecoder(b) - for dec.Next() { - d.values = append(d.values, ZigZagDecode(dec.Read())) + if len(b) == 0 { + return + } + + count := binary.BigEndian.Uint32(b[:4]) + + if count == 0 { + return + } + + d.values = make([]uint64, count) + b = b[4:] + j := 0 + for i := 0; i < len(b); i += 8 { + d.vbuf[0] = binary.BigEndian.Uint64(b[i : i+8]) + n, _ := simple8b.Decode(d.buf, d.vbuf) + copy(d.values[j:], d.buf[:n]) + j += n } } func (d *int64Decoder) decodeUncompressed(b []byte) { - d.values = make([]int64, len(b)/8) + d.values = make([]uint64, len(b)/8) for i := range d.values { - d.values[i] = int64(binary.BigEndian.Uint64(b[i*8 : i*8+8])) + d.values[i] = binary.BigEndian.Uint64(b[i*8 : i*8+8]) } } diff --git a/tsdb/engine/pd1/int_test.go b/tsdb/engine/pd1/int_test.go index 82042f77bd4..063ae05c075 100644 --- a/tsdb/engine/pd1/int_test.go +++ b/tsdb/engine/pd1/int_test.go @@ -220,6 +220,10 @@ func BenchmarkInt64Encoder(b *testing.B) { } } +type byteSetter interface { + SetBytes(b []byte) +} + func BenchmarkInt64Decoder(b *testing.B) { x := make([]int64, 1024) enc := pd1.NewInt64Encoder() @@ -231,10 +235,10 @@ func BenchmarkInt64Decoder(b *testing.B) { b.ResetTimer() + dec := pd1.NewInt64Decoder(bytes) + for i := 0; i < b.N; i++ { - b.StopTimer() - dec := pd1.NewInt64Decoder(bytes) - b.StartTimer() + dec.(byteSetter).SetBytes(bytes) for dec.Next() { } } From 071739b960d03d876ca0c852d84dc95c8c9aec19 Mon Sep 17 00:00:00 2001 From: Jason Wilder Date: Fri, 25 Sep 2015 14:19:24 -0600 Subject: [PATCH 033/139] Fix run length encoding check Values were run length encoded even when they should not have been --- tsdb/engine/pd1/timestamp.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tsdb/engine/pd1/timestamp.go b/tsdb/engine/pd1/timestamp.go index 9119dbd5182..04de5b991fb 100644 --- a/tsdb/engine/pd1/timestamp.go +++ b/tsdb/engine/pd1/timestamp.go @@ -77,7 +77,7 @@ func (e *encoder) reduce() (max, divisor uint64, rle bool, deltas []uint64) { } // Skip the first value || see if prev = curr. The deltas can be RLE if the are all equal. - rle = i != 0 || rle && (deltas[i-1] == deltas[i]) + rle = i == len(deltas)-1 || rle && (deltas[i+1] == deltas[i]) } return } From 5a49e1a04b42aee0a70ac3e618894177c4260902 Mon Sep 17 00:00:00 2001 From: Jason Wilder Date: Fri, 25 Sep 2015 14:25:42 -0600 Subject: [PATCH 034/139] Add test assertions for time encoding type --- tsdb/engine/pd1/timestamp_test.go | 38 ++++++++++++++++++++++++++++++- 1 file changed, 37 insertions(+), 1 deletion(-) diff --git a/tsdb/engine/pd1/timestamp_test.go b/tsdb/engine/pd1/timestamp_test.go index da78b2b6e85..9eb3e0fe2c3 100644 --- a/tsdb/engine/pd1/timestamp_test.go +++ b/tsdb/engine/pd1/timestamp_test.go @@ -24,6 +24,10 @@ func Test_TimeEncoder(t *testing.T) { t.Fatalf("unexpected error: %v", err) } + if got := b[0] >> 4; got != pd1.EncodingPacked { + t.Fatalf("Wrong encoding used: expected uncompressed, got %v", got) + } + dec := pd1.NewTimeDecoder(b) for i, v := range x { if !dec.Next() { @@ -59,6 +63,10 @@ func Test_TimeEncoder_One(t *testing.T) { t.Fatalf("unexpected error: %v", err) } + if got := b[0] >> 4; got != pd1.EncodingPacked { + t.Fatalf("Wrong encoding used: expected uncompressed, got %v", got) + } + dec := pd1.NewTimeDecoder(b) if !dec.Next() { t.Fatalf("unexpected next value: got true, exp false") @@ -81,6 +89,10 @@ func Test_TimeEncoder_Two(t *testing.T) { t.Fatalf("unexpected error: %v", err) } + if got := b[0] >> 4; got != pd1.EncodingPacked { + t.Fatalf("Wrong encoding used: expected uncompressed, got %v", got) + } + dec := pd1.NewTimeDecoder(b) if !dec.Next() { t.Fatalf("unexpected next value: got true, exp false") @@ -114,6 +126,10 @@ func Test_TimeEncoder_Three(t *testing.T) { t.Fatalf("unexpected error: %v", err) } + if got := b[0] >> 4; got != pd1.EncodingPacked { + t.Fatalf("Wrong encoding used: expected uncompressed, got %v", got) + } + dec := pd1.NewTimeDecoder(b) if !dec.Next() { t.Fatalf("unexpected next value: got true, exp false") @@ -151,6 +167,10 @@ func Test_TimeEncoder_Large_Range(t *testing.T) { t.Fatalf("unexpected error: %v", err) } + if got := b[0] >> 4; got != pd1.EncodingPacked { + t.Fatalf("Wrong encoding used: expected uncompressed, got %v", got) + } + dec := pd1.NewTimeDecoder(b) if !dec.Next() { t.Fatalf("unexpected next value: got true, exp false") @@ -169,7 +189,7 @@ func Test_TimeEncoder_Large_Range(t *testing.T) { } } -func Test_TimeEncoder_Raw(t *testing.T) { +func Test_TimeEncoder_Uncompressed(t *testing.T) { enc := pd1.NewTimeEncoder() t1 := time.Unix(0, 0) t2 := time.Unix(1, 0) @@ -190,6 +210,10 @@ func Test_TimeEncoder_Raw(t *testing.T) { t.Fatalf("length mismatch: got %v, exp %v", len(b), exp) } + if got := b[0] >> 4; got != pd1.EncodingUncompressed { + t.Fatalf("Wrong encoding used: expected uncompressed, got %v", got) + } + dec := pd1.NewTimeDecoder(b) if !dec.Next() { t.Fatalf("unexpected next value: got true, exp false") @@ -232,6 +256,10 @@ func Test_TimeEncoder_RLE(t *testing.T) { t.Fatalf("length mismatch: got %v, exp %v", len(b), exp) } + if got := b[0] >> 4; got != pd1.EncodingRLE { + t.Fatalf("Wrong encoding used: expected uncompressed, got %v", got) + } + if err != nil { t.Fatalf("unexpected error: %v", err) } @@ -269,6 +297,10 @@ func Test_TimeEncoder_Reverse(t *testing.T) { t.Fatalf("unexpected error: %v", err) } + if got := b[0] >> 4; got != pd1.EncodingPacked { + t.Fatalf("Wrong encoding used: expected uncompressed, got %v", got) + } + dec := pd1.NewTimeDecoder(b) i := 0 for dec.Next() { @@ -301,6 +333,10 @@ func Test_TimeEncoder_220SecondDelta(t *testing.T) { t.Fatalf("unexpected length: got %v, exp %v", len(b), exp) } + if got := b[0] >> 4; got != pd1.EncodingRLE { + t.Fatalf("Wrong encoding used: expected uncompressed, got %v", got) + } + dec := pd1.NewTimeDecoder(b) i := 0 for dec.Next() { From 8d2ecb5df52228c61db01f011213014d9fe3e5af Mon Sep 17 00:00:00 2001 From: Jason Wilder Date: Fri, 25 Sep 2015 15:27:43 -0600 Subject: [PATCH 035/139] Remove unnecessary allocations from int64 decoder The decoder was creating a large slice and decoding all values when instead, it could decode one packed value as needed. --- tsdb/engine/pd1/int.go | 98 +++++++++++++++++-------------------- tsdb/engine/pd1/int_test.go | 4 ++ 2 files changed, 48 insertions(+), 54 deletions(-) diff --git a/tsdb/engine/pd1/int.go b/tsdb/engine/pd1/int.go index 706ca5fce8e..3c6cdf9c6ee 100644 --- a/tsdb/engine/pd1/int.go +++ b/tsdb/engine/pd1/int.go @@ -46,14 +46,12 @@ func (e *int64Encoder) encodePacked() ([]byte, error) { return nil, err } - b := make([]byte, 1+len(encoded)*8+4) + b := make([]byte, 1+len(encoded)*8) // 4 high bits of first byte store the encoding type for the block b[0] = byte(EncodingPacked) << 4 - binary.BigEndian.PutUint32(b[1:5], uint32(len(e.values))) - for i, v := range encoded { - binary.BigEndian.PutUint64(b[5+i*8:5+i*8+8], v) + binary.BigEndian.PutUint64(b[1+i*8:1+i*8+8], v) } return b, nil } @@ -64,86 +62,78 @@ func (e *int64Encoder) encodeUncompressed() ([]byte, error) { b[0] = byte(EncodingUncompressed) << 4 for i, v := range e.values { - binary.BigEndian.PutUint64(b[1+i*8:1+i*8+8], uint64(v)) + binary.BigEndian.PutUint64(b[1+i*8:1+i*8+8], v) } return b, nil } type int64Decoder struct { values []uint64 - v int64 - buf []uint64 - vbuf []uint64 + bytes []byte + i int + n int + + encoding byte } func NewInt64Decoder(b []byte) Int64Decoder { d := &int64Decoder{ - buf: make([]uint64, 240), - vbuf: make([]uint64, 1), + values: make([]uint64, 240), } - d.decode(b) + + d.SetBytes(b) return d } func (d *int64Decoder) SetBytes(b []byte) { - d.decode(b) + if len(b) > 0 { + d.encoding = b[0] >> 4 + d.bytes = b[1:] + } + d.i = 0 + d.n = 0 } func (d *int64Decoder) Next() bool { - if len(d.values) == 0 { + if d.i >= d.n && len(d.bytes) == 0 { return false } - d.v = ZigZagDecode(d.values[0]) - d.values = d.values[1:] - return true -} -func (d *int64Decoder) Read() int64 { - return d.v -} + d.i += 1 -func (d *int64Decoder) decode(b []byte) { - if len(b) == 0 { - return + if d.i >= d.n { + switch d.encoding { + case EncodingUncompressed: + d.decodeUncompressed() + case EncodingPacked: + d.decodePacked() + default: + panic(fmt.Sprintf("unknown encoding %v", d.encoding)) + } } + return d.i < d.n +} - // Encoding type is stored in the 4 high bits of the first byte - encoding := b[0] >> 4 - switch encoding { - case EncodingUncompressed: - d.decodeUncompressed(b[1:]) - case EncodingPacked: - d.decodePacked(b[1:]) - default: - panic(fmt.Sprintf("unknown encoding %v", encoding)) - } +func (d *int64Decoder) Read() int64 { + return ZigZagDecode(d.values[d.i]) } -func (d *int64Decoder) decodePacked(b []byte) { - if len(b) == 0 { +func (d *int64Decoder) decodePacked() { + if len(d.bytes) == 0 { return } - count := binary.BigEndian.Uint32(b[:4]) - - if count == 0 { - return - } + v := binary.BigEndian.Uint64(d.bytes[0:8]) + n, _ := simple8b.DecodeSingle(d.values, v) - d.values = make([]uint64, count) - b = b[4:] - j := 0 - for i := 0; i < len(b); i += 8 { - d.vbuf[0] = binary.BigEndian.Uint64(b[i : i+8]) - n, _ := simple8b.Decode(d.buf, d.vbuf) - copy(d.values[j:], d.buf[:n]) - j += n - } + d.n = n + d.i = 0 + d.bytes = d.bytes[8:] } -func (d *int64Decoder) decodeUncompressed(b []byte) { - d.values = make([]uint64, len(b)/8) - for i := range d.values { - d.values[i] = binary.BigEndian.Uint64(b[i*8 : i*8+8]) - } +func (d *int64Decoder) decodeUncompressed() { + d.values[0] = binary.BigEndian.Uint64(d.bytes[0:8]) + d.i = 0 + d.n = 1 + d.bytes = d.bytes[8:] } diff --git a/tsdb/engine/pd1/int_test.go b/tsdb/engine/pd1/int_test.go index 063ae05c075..7f1ba2eff74 100644 --- a/tsdb/engine/pd1/int_test.go +++ b/tsdb/engine/pd1/int_test.go @@ -199,6 +199,10 @@ func Test_Int64Encoder_AllNegative(t *testing.T) { dec := pd1.NewInt64Decoder(b) i := 0 for dec.Next() { + if i > len(values) { + t.Fatalf("read too many values: got %v, exp %v", i, len(values)) + } + if values[i] != dec.Read() { t.Fatalf("read value %d mismatch: got %v, exp %v", i, dec.Read(), values[i]) } From 7fe9673bbc23a591a661a5452ee7332045c41ba3 Mon Sep 17 00:00:00 2001 From: Jason Wilder Date: Mon, 28 Sep 2015 10:25:58 -0600 Subject: [PATCH 036/139] Keep track of the type of the block encoded Allowes decode to decode an arbitrary block correctly. --- tsdb/engine/pd1/encoding.go | 94 +++++++++++++++++++++++++++++-------- 1 file changed, 74 insertions(+), 20 deletions(-) diff --git a/tsdb/engine/pd1/encoding.go b/tsdb/engine/pd1/encoding.go index a2e16699456..227fe776123 100644 --- a/tsdb/engine/pd1/encoding.go +++ b/tsdb/engine/pd1/encoding.go @@ -16,6 +16,18 @@ const ( EncodingRLE = 1 // EncodingUncompressed is a non-compressed format EncodingUncompressed = 2 + + // BlockFloat64 designates a block encodes float64 values + BlockFloat64 = 0 + + // BlockInt64 designates a block encodes int64 values + BlockInt64 = 1 + + // BlockBool designates a block encodes bool values + BlockBool = 2 + + // BlockString designates a block encodes string values + BlockString = 3 ) type Value interface { @@ -67,14 +79,14 @@ func (v Values) Encode(buf []byte) []byte { for i, vv := range v { a[i] = vv.(*FloatValue) } - return EncodeFloatBlock(buf, a) + return encodeFloatBlock(buf, a) case *Int64Value: a := make([]*Int64Value, len(v)) for i, vv := range v { a[i] = vv.(*Int64Value) } - return EncodeInt64Block(buf, a) + return encodeInt64Block(buf, a) // TODO: add support for other types } @@ -85,10 +97,10 @@ func (v Values) Encode(buf []byte) []byte { func (v Values) DecodeSameTypeBlock(block []byte) Values { switch v[0].(type) { case *FloatValue: - a, _ := DecodeFloatBlock(block) + a, _ := decodeFloatBlock(block) return a case *Int64Value: - a, _ := DecodeInt64Block(block) + a, _ := decodeInt64Block(block) return a // TODO: add support for other types @@ -99,8 +111,25 @@ func (v Values) DecodeSameTypeBlock(block []byte) Values { // DecodeBlock takes a byte array and will decode into values of the appropriate type // based on the block func DecodeBlock(block []byte) (Values, error) { + if len(block) == 0 { + return Values{}, nil + } + + blockType := block[8] + switch blockType { + case BlockFloat64: + return decodeFloatBlock(block) + case BlockInt64: + return decodeInt64Block(block) + case BlockBool: + // return decodeBoolBlock(block) + case BlockString: + // return decodeStringBlock(block) + default: + } + // TODO: add support for other block types - return DecodeFloatBlock(block) + return nil, fmt.Errorf("unknown block type: %d", blockType) } // Deduplicate returns a new Values slice with any values @@ -147,9 +176,9 @@ func (f *FloatValue) Size() int { return 16 } -func EncodeFloatBlock(buf []byte, values []*FloatValue) []byte { +func encodeFloatBlock(buf []byte, values []*FloatValue) []byte { if len(values) == 0 { - return []byte{} + return nil } // A float block is encoded using different compression strategies @@ -176,14 +205,25 @@ func EncodeFloatBlock(buf []byte, values []*FloatValue) []byte { // Encoded float values vb := venc.Bytes() - // Preprend the first timestamp of the block in the first 8 bytes - return append(u64tob(uint64(values[0].Time().UnixNano())), - packBlock(tb, vb)...) + // Preprend the first timestamp of the block in the first 8 bytes and the block + // in the next byte, followed by the block + block := packBlockHeader(values[0].Time(), BlockFloat64) + block = append(block, packBlock(tb, vb)...) + return block } -func DecodeFloatBlock(block []byte) ([]Value, error) { +func decodeFloatBlock(block []byte) ([]Value, error) { // The first 8 bytes is the minimum timestamp of the block - tb, vb := unpackBlock(block[8:]) + block = block[8:] + + // Block type is the next block, make sure we actually have a float block + blockType := block[0] + if blockType != BlockFloat64 { + return nil, fmt.Errorf("invalid block type: exp %d, got %d", BlockFloat64, blockType) + } + block = block[1:] + + tb, vb := unpackBlock(block) // Setup our timestamp and value decoders dec := NewTimeDecoder(tb) @@ -208,11 +248,11 @@ type BoolValue struct { value bool } -func EncodeBoolBlock(buf []byte, values []BoolValue) []byte { +func encodeBoolBlock(buf []byte, values []BoolValue) []byte { return nil } -func DecodeBoolBlock(block []byte) ([]BoolValue, error) { +func eecodeBoolBlock(block []byte) ([]BoolValue, error) { return nil, nil } @@ -239,7 +279,7 @@ func (v *Int64Value) Size() int { func (v *Int64Value) String() string { return fmt.Sprintf("%v", v.value) } -func EncodeInt64Block(buf []byte, values []*Int64Value) []byte { +func encodeInt64Block(buf []byte, values []*Int64Value) []byte { tsEnc := NewTimeEncoder() vEnc := NewInt64Encoder() for _, v := range values { @@ -259,13 +299,23 @@ func EncodeInt64Block(buf []byte, values []*Int64Value) []byte { } // Preprend the first timestamp of the block in the first 8 bytes - return append(u64tob(uint64(values[0].Time().UnixNano())), - packBlock(tb, vb)...) + block := packBlockHeader(values[0].Time(), BlockInt64) + return append(block, packBlock(tb, vb)...) } -func DecodeInt64Block(block []byte) ([]Value, error) { +func decodeInt64Block(block []byte) ([]Value, error) { + // slice off the first 8 bytes (min timestmap for the block) + block = block[8:] + + blockType := block[0] + if blockType != BlockInt64 { + return nil, fmt.Errorf("invalid block type: exp %d, got %d", BlockInt64, blockType) + } + + block = block[1:] + // The first 8 bytes is the minimum timestamp of the block - tb, vb := unpackBlock(block[8:]) + tb, vb := unpackBlock(block) // Setup our timestamp and value decoders tsDec := NewTimeDecoder(tb) @@ -287,10 +337,14 @@ type StringValue struct { value string } -func EncodeStringBlock(buf []byte, values []StringValue) []byte { +func encodeStringBlock(buf []byte, blockType byte, values []StringValue) []byte { return nil } +func packBlockHeader(firstTime time.Time, blockType byte) []byte { + return append(u64tob(uint64(firstTime.UnixNano())), blockType) +} + func packBlock(ts []byte, values []byte) []byte { // We encode the length of the timestamp block using a variable byte encoding. // This allows small byte slices to take up 1 byte while larger ones use 2 or more. From 6c0f53dfe496cb47bb59d897e6d30e67d8fdba37 Mon Sep 17 00:00:00 2001 From: Jason Wilder Date: Mon, 28 Sep 2015 12:53:27 -0600 Subject: [PATCH 037/139] Add a compressed boolean encoding Packs booleans into bytes using 1 bit per value. --- tsdb/engine/pd1/bool.go | 118 ++++++++++++++++++++++++++++++ tsdb/engine/pd1/bool_test.go | 73 ++++++++++++++++++ tsdb/engine/pd1/encoding.go | 108 ++++++++++++++++++++++++--- tsdb/engine/pd1/encoding_test.go | 21 ++++++ tsdb/engine/pd1/int.go | 4 +- tsdb/engine/pd1/timestamp.go | 4 +- tsdb/engine/pd1/timestamp_test.go | 12 +-- 7 files changed, 319 insertions(+), 21 deletions(-) create mode 100644 tsdb/engine/pd1/bool.go create mode 100644 tsdb/engine/pd1/bool_test.go diff --git a/tsdb/engine/pd1/bool.go b/tsdb/engine/pd1/bool.go new file mode 100644 index 00000000000..065bd61c7f5 --- /dev/null +++ b/tsdb/engine/pd1/bool.go @@ -0,0 +1,118 @@ +package pd1 + +import "encoding/binary" + +type BoolEncoder interface { + Write(b bool) + Bytes() ([]byte, error) +} + +type BoolDecoder interface { + Next() bool + Read() bool +} + +type boolEncoder struct { + // The encoded bytes + bytes []byte + + // The current byte being encoded + b byte + + // The number of bools packed into b + i int + + // The total number of bools written + n int +} + +func NewBoolEncoder() BoolEncoder { + return &boolEncoder{} +} + +func (e *boolEncoder) Write(b bool) { + // If we have filled the current byte, flush it + if e.i >= 8 { + e.flush() + } + + // Use 1 bit for each boolen value, shift the current byte + // by 1 and set the least signficant bit acordingly + e.b = e.b << 1 + if b { + e.b |= 1 + } + + // Increment the current bool count + e.i += 1 + // Increment the total bool count + e.n += 1 +} + +func (e *boolEncoder) flush() { + // Pad remaining byte w/ 0s + for e.i < 8 { + e.b = e.b << 1 + e.i += 1 + } + + // If we have bits set, append them to the byte slice + if e.i > 0 { + e.bytes = append(e.bytes, e.b) + e.b = 0 + e.i = 0 + } +} + +func (e *boolEncoder) Bytes() ([]byte, error) { + // Ensure the current byte is flushed + e.flush() + b := make([]byte, 10+1) + + // Store the encoding type in the 4 high bits of the first byte + b[0] = byte(EncodingBitPacked) << 4 + + i := 1 + // Encode the number of bools written + i += binary.PutUvarint(b[i:], uint64(e.n)) + + // Append the packed booleans + return append(b[:i], e.bytes...), nil +} + +type boolDecoder struct { + b []byte + i int + n int +} + +func NewBoolDecoder(b []byte) BoolDecoder { + // First byte stores the encoding type, only have 1 bit-packet format + // currently ignore for now. + b = b[1:] + count, n := binary.Uvarint(b) + return &boolDecoder{b: b[n:], i: -1, n: int(count)} +} + +func (e *boolDecoder) Next() bool { + e.i += 1 + return e.i < e.n +} + +func (e *boolDecoder) Read() bool { + + // Index into the byte slice + idx := e.i / 8 + + // Bit position + pos := (8 - e.i%8) - 1 + + // The mask to select the bit + mask := byte(1 << uint(pos)) + + // The packed byte + v := e.b[idx] + + // Returns true if the bit is set + return v&mask == mask +} diff --git a/tsdb/engine/pd1/bool_test.go b/tsdb/engine/pd1/bool_test.go new file mode 100644 index 00000000000..ed642cff86e --- /dev/null +++ b/tsdb/engine/pd1/bool_test.go @@ -0,0 +1,73 @@ +package pd1_test + +import ( + "testing" + + "github.com/influxdb/influxdb/tsdb/engine/pd1" +) + +func Test_BoolEncoder_NoValues(t *testing.T) { + enc := pd1.NewBoolEncoder() + b, err := enc.Bytes() + if err != nil { + t.Fatalf("unexpected error: %v", err) + } + + dec := pd1.NewBoolDecoder(b) + if dec.Next() { + t.Fatalf("unexpected next value: got true, exp false") + } +} + +func Test_BoolEncoder_Single(t *testing.T) { + enc := pd1.NewBoolEncoder() + v1 := true + enc.Write(v1) + b, err := enc.Bytes() + if err != nil { + t.Fatalf("unexpected error: %v", err) + } + + dec := pd1.NewBoolDecoder(b) + if !dec.Next() { + t.Fatalf("unexpected next value: got false, exp true") + } + + if v1 != dec.Read() { + t.Fatalf("unexpected value: got %v, exp %v", dec.Read(), v1) + } +} + +func Test_BoolEncoder_Multi_Compressed(t *testing.T) { + enc := pd1.NewBoolEncoder() + + values := make([]bool, 10) + for i := range values { + values[i] = i%2 == 0 + enc.Write(values[i]) + } + + b, err := enc.Bytes() + if err != nil { + t.Fatalf("unexpected error: %v", err) + } + + if exp := 4; len(b) != exp { + t.Fatalf("unexpected length: got %v, exp %v", len(b), exp) + } + + dec := pd1.NewBoolDecoder(b) + + for i, v := range values { + if !dec.Next() { + t.Fatalf("unexpected next value: got false, exp true") + } + if v != dec.Read() { + t.Fatalf("unexpected value at pos %d: got %v, exp %v", i, dec.Read(), v) + } + } + + if dec.Next() { + t.Fatalf("unexpected next value: got true, exp false") + } +} diff --git a/tsdb/engine/pd1/encoding.go b/tsdb/engine/pd1/encoding.go index 227fe776123..d7abb195b5b 100644 --- a/tsdb/engine/pd1/encoding.go +++ b/tsdb/engine/pd1/encoding.go @@ -10,13 +10,18 @@ import ( ) const ( - // EncodingPacked is a bit-packed format - EncodingPacked = 0 + // EncodingPackedSimple is a bit-packed format + EncodingPackedSimple = 0 + // EncodingRLE is a run-length encoded format EncodingRLE = 1 + // EncodingUncompressed is a non-compressed format EncodingUncompressed = 2 + // EncodingBitPacked is a basic bit-packed format + EncodingBitPacked = 3 + // BlockFloat64 designates a block encodes float64 values BlockFloat64 = 0 @@ -43,8 +48,8 @@ func NewValue(t time.Time, value interface{}) Value { return &Int64Value{time: t, value: v} case float64: return &FloatValue{time: t, value: v} - // case bool: - // return &BoolValue{time: t, value: v} + case bool: + return &BoolValue{time: t, value: v} // case string: // return &StringValue{time: t, value: v} } @@ -88,6 +93,13 @@ func (v Values) Encode(buf []byte) []byte { } return encodeInt64Block(buf, a) + case *BoolValue: + a := make([]*BoolValue, len(v)) + for i, vv := range v { + a[i] = vv.(*BoolValue) + } + return encodeBoolBlock(buf, a) + // TODO: add support for other types } @@ -102,8 +114,9 @@ func (v Values) DecodeSameTypeBlock(block []byte) Values { case *Int64Value: a, _ := decodeInt64Block(block) return a - - // TODO: add support for other types + case *BoolValue: + a, _ := decodeBoolBlock(block) + return a } return nil } @@ -122,7 +135,7 @@ func DecodeBlock(block []byte) (Values, error) { case BlockInt64: return decodeInt64Block(block) case BlockBool: - // return decodeBoolBlock(block) + return decodeBoolBlock(block) case BlockString: // return decodeStringBlock(block) default: @@ -248,12 +261,85 @@ type BoolValue struct { value bool } -func encodeBoolBlock(buf []byte, values []BoolValue) []byte { - return nil +func (b *BoolValue) Time() time.Time { + return b.time +} + +func (b *BoolValue) Size() int { + return 9 +} + +func (b *BoolValue) UnixNano() int64 { + return b.time.UnixNano() +} + +func (b *BoolValue) Value() interface{} { + return b.value } -func eecodeBoolBlock(block []byte) ([]BoolValue, error) { - return nil, nil +func encodeBoolBlock(buf []byte, values []*BoolValue) []byte { + if len(values) == 0 { + return nil + } + + // A bool block is encoded using different compression strategies + // for timestamps and values. + + // Encode values using Gorilla float compression + venc := NewBoolEncoder() + + // Encode timestamps using an adaptive encoder + tsenc := NewTimeEncoder() + + for _, v := range values { + tsenc.Write(v.Time()) + venc.Write(v.value) + } + + // Encoded timestamp values + tb, err := tsenc.Bytes() + if err != nil { + panic(err.Error()) + } + // Encoded float values + vb, err := venc.Bytes() + if err != nil { + panic(err.Error()) + } + + // Preprend the first timestamp of the block in the first 8 bytes and the block + // in the next byte, followed by the block + block := packBlockHeader(values[0].Time(), BlockBool) + block = append(block, packBlock(tb, vb)...) + return block +} + +func decodeBoolBlock(block []byte) ([]Value, error) { + // The first 8 bytes is the minimum timestamp of the block + block = block[8:] + + // Block type is the next block, make sure we actually have a float block + blockType := block[0] + if blockType != BlockBool { + return nil, fmt.Errorf("invalid block type: exp %d, got %d", BlockBool, blockType) + } + block = block[1:] + + tb, vb := unpackBlock(block) + + // Setup our timestamp and value decoders + dec := NewTimeDecoder(tb) + vdec := NewBoolDecoder(vb) + + // Decode both a timestamp and value + var a []Value + for dec.Next() && vdec.Next() { + ts := dec.Read() + v := vdec.Read() + a = append(a, &BoolValue{ts, v}) + } + + return a, nil } type Int64Value struct { diff --git a/tsdb/engine/pd1/encoding_test.go b/tsdb/engine/pd1/encoding_test.go index c249f1aa8ce..aa5b67d6409 100644 --- a/tsdb/engine/pd1/encoding_test.go +++ b/tsdb/engine/pd1/encoding_test.go @@ -76,6 +76,27 @@ func TestEncoding_IntBlock_Negatives(t *testing.T) { } } +func TestEncoding_BoolBlock_Basic(t *testing.T) { + valueCount := 1000 + times := getTimes(valueCount, 60, time.Second) + values := make(pd1.Values, len(times)) + for i, t := range times { + v := true + if i%2 == 0 { + v = false + } + values[i] = pd1.NewValue(t, v) + } + + b := values.Encode(nil) + + decodedValues := values.DecodeSameTypeBlock(b) + + if !reflect.DeepEqual(decodedValues, values) { + t.Fatalf("unexpected results:\n\tgot: %v\n\texp: %v\n", decodedValues, values) + } +} + func getTimes(n, step int, precision time.Duration) []time.Time { t := time.Now().Round(precision) a := make([]time.Time, n) diff --git a/tsdb/engine/pd1/int.go b/tsdb/engine/pd1/int.go index 3c6cdf9c6ee..a7258e6d651 100644 --- a/tsdb/engine/pd1/int.go +++ b/tsdb/engine/pd1/int.go @@ -48,7 +48,7 @@ func (e *int64Encoder) encodePacked() ([]byte, error) { b := make([]byte, 1+len(encoded)*8) // 4 high bits of first byte store the encoding type for the block - b[0] = byte(EncodingPacked) << 4 + b[0] = byte(EncodingPackedSimple) << 4 for i, v := range encoded { binary.BigEndian.PutUint64(b[1+i*8:1+i*8+8], v) @@ -105,7 +105,7 @@ func (d *int64Decoder) Next() bool { switch d.encoding { case EncodingUncompressed: d.decodeUncompressed() - case EncodingPacked: + case EncodingPackedSimple: d.decodePacked() default: panic(fmt.Sprintf("unknown encoding %v", d.encoding)) diff --git a/tsdb/engine/pd1/timestamp.go b/tsdb/engine/pd1/timestamp.go index 04de5b991fb..6863a51afed 100644 --- a/tsdb/engine/pd1/timestamp.go +++ b/tsdb/engine/pd1/timestamp.go @@ -114,7 +114,7 @@ func (e *encoder) encodePacked(div uint64, dts []uint64) ([]byte, error) { b := make([]byte, 8+1) // 4 high bits used for the encoding type - b[0] = byte(EncodingPacked) << 4 + b[0] = byte(EncodingPackedSimple) << 4 // 4 low bits are the log10 divisor b[0] |= byte(math.Log10(float64(div))) @@ -196,7 +196,7 @@ func (d *decoder) decode(b []byte) { d.decodeRaw(b[1:]) case EncodingRLE: d.decodeRLE(b) - case EncodingPacked: + case EncodingPackedSimple: d.decodePacked(b) default: panic(fmt.Sprintf("unknown encoding: %v", encoding)) diff --git a/tsdb/engine/pd1/timestamp_test.go b/tsdb/engine/pd1/timestamp_test.go index 9eb3e0fe2c3..e0dd235d62a 100644 --- a/tsdb/engine/pd1/timestamp_test.go +++ b/tsdb/engine/pd1/timestamp_test.go @@ -24,7 +24,7 @@ func Test_TimeEncoder(t *testing.T) { t.Fatalf("unexpected error: %v", err) } - if got := b[0] >> 4; got != pd1.EncodingPacked { + if got := b[0] >> 4; got != pd1.EncodingPackedSimple { t.Fatalf("Wrong encoding used: expected uncompressed, got %v", got) } @@ -63,7 +63,7 @@ func Test_TimeEncoder_One(t *testing.T) { t.Fatalf("unexpected error: %v", err) } - if got := b[0] >> 4; got != pd1.EncodingPacked { + if got := b[0] >> 4; got != pd1.EncodingPackedSimple { t.Fatalf("Wrong encoding used: expected uncompressed, got %v", got) } @@ -89,7 +89,7 @@ func Test_TimeEncoder_Two(t *testing.T) { t.Fatalf("unexpected error: %v", err) } - if got := b[0] >> 4; got != pd1.EncodingPacked { + if got := b[0] >> 4; got != pd1.EncodingPackedSimple { t.Fatalf("Wrong encoding used: expected uncompressed, got %v", got) } @@ -126,7 +126,7 @@ func Test_TimeEncoder_Three(t *testing.T) { t.Fatalf("unexpected error: %v", err) } - if got := b[0] >> 4; got != pd1.EncodingPacked { + if got := b[0] >> 4; got != pd1.EncodingPackedSimple { t.Fatalf("Wrong encoding used: expected uncompressed, got %v", got) } @@ -167,7 +167,7 @@ func Test_TimeEncoder_Large_Range(t *testing.T) { t.Fatalf("unexpected error: %v", err) } - if got := b[0] >> 4; got != pd1.EncodingPacked { + if got := b[0] >> 4; got != pd1.EncodingPackedSimple { t.Fatalf("Wrong encoding used: expected uncompressed, got %v", got) } @@ -297,7 +297,7 @@ func Test_TimeEncoder_Reverse(t *testing.T) { t.Fatalf("unexpected error: %v", err) } - if got := b[0] >> 4; got != pd1.EncodingPacked { + if got := b[0] >> 4; got != pd1.EncodingPackedSimple { t.Fatalf("Wrong encoding used: expected uncompressed, got %v", got) } From 3cbd4b198f37e9d2f052666f4fd0ea734cda5488 Mon Sep 17 00:00:00 2001 From: Jason Wilder Date: Mon, 28 Sep 2015 23:06:17 -0600 Subject: [PATCH 038/139] Update simple8b api usage --- tsdb/engine/pd1/int.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tsdb/engine/pd1/int.go b/tsdb/engine/pd1/int.go index a7258e6d651..c8f42341c99 100644 --- a/tsdb/engine/pd1/int.go +++ b/tsdb/engine/pd1/int.go @@ -41,7 +41,7 @@ func (e *int64Encoder) Bytes() ([]byte, error) { } func (e *int64Encoder) encodePacked() ([]byte, error) { - encoded, err := simple8b.Encode(e.values) + encoded, err := simple8b.EncodeAll(e.values) if err != nil { return nil, err } @@ -124,7 +124,7 @@ func (d *int64Decoder) decodePacked() { } v := binary.BigEndian.Uint64(d.bytes[0:8]) - n, _ := simple8b.DecodeSingle(d.values, v) + n, _ := simple8b.Decode(d.values, v) d.n = n d.i = 0 From 3ea1b5e7955bc4c5e8ec739f939dc647174e3c8f Mon Sep 17 00:00:00 2001 From: Jason Wilder Date: Tue, 29 Sep 2015 10:44:24 -0600 Subject: [PATCH 039/139] Fix deadlock in pd1_test.go The defer tx.Rollback() tries to free the queryLock but the defer e.Cleanup() runs before it and tries to take a write lock on the query lock (which blocks) and prevents tx.Rollback() from acquring the read lock. --- tsdb/engine/pd1/pd1_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tsdb/engine/pd1/pd1_test.go b/tsdb/engine/pd1/pd1_test.go index e1abe87360c..2a59c9ac4e2 100644 --- a/tsdb/engine/pd1/pd1_test.go +++ b/tsdb/engine/pd1/pd1_test.go @@ -77,7 +77,6 @@ func TestEngine_WriteAndReadFloats(t *testing.T) { verify(false) tx, _ := e.Begin(false) - defer tx.Rollback() c := tx.Cursor("cpu,host=B", fields, nil, true) k, v := c.SeekTo(0) if k != p2.UnixNano() { @@ -111,6 +110,7 @@ func TestEngine_WriteAndReadFloats(t *testing.T) { if 1.1 != v { t.Fatal("p1 data not equal") } + tx.Rollback() if err := e.Close(); err != nil { t.Fatalf("error closing: %s", err.Error()) From 24922181c6faa2b8fa74b1a048b984f5d56552b7 Mon Sep 17 00:00:00 2001 From: Jason Wilder Date: Tue, 29 Sep 2015 10:46:19 -0600 Subject: [PATCH 040/139] Add documentation about compression --- tsdb/engine/pd1/bool.go | 5 +++++ tsdb/engine/pd1/int.go | 20 +++++++++++++++++ tsdb/engine/pd1/timestamp.go | 43 +++++++++++++++++++++++++++++------- 3 files changed, 60 insertions(+), 8 deletions(-) diff --git a/tsdb/engine/pd1/bool.go b/tsdb/engine/pd1/bool.go index 065bd61c7f5..4fad26586bb 100644 --- a/tsdb/engine/pd1/bool.go +++ b/tsdb/engine/pd1/bool.go @@ -1,5 +1,10 @@ package pd1 +// bool encoding uses 1 bit per value. Each compressed byte slice contains a 1 byte header +// indicating the compression type, followed by a variable byte encoded length indicating +// how many booleans are packed in the slice. The remaining bytes contains 1 byte for every +// 8 boolean values encoded. + import "encoding/binary" type BoolEncoder interface { diff --git a/tsdb/engine/pd1/int.go b/tsdb/engine/pd1/int.go index c8f42341c99..98cf717f303 100644 --- a/tsdb/engine/pd1/int.go +++ b/tsdb/engine/pd1/int.go @@ -1,5 +1,25 @@ package pd1 +// Int64 encoding uses two different strategies depending on the range of values in +// the uncompressed data. Encoded values are first encoding used zig zag encoding. +// This interleaves postiive and negative integers across a range of positive integers. +// +// For example, [-2,-1,0,1] becomes [3,1,0,2]. See +// https://developers.google.com/protocol-buffers/docs/encoding?hl=en#signed-integers +// for more information. +// +// If all the zig zag encoded values less than 1 << 60 - 1, they are compressed using +// simple8b encoding. If any values is larger than 1 << 60 - 1, the values are stored uncompressed. +// +// Each encoded byte slice, contains a 1 byte header followed by multiple 8 byte packed integers +// or 8 byte uncompressed integers. The 4 high bits of the first byte indicate the encoding type +// for the remaining bytes. +// +// There are currently two encoding types that can be used with room for 15 more. These additional +// encoding slots are reserved for future use. One improvement to to be made is to use a patched +// encoding such as PFOR if only a small number of values exceed the max compressed value range. This +// should improve compression ratios with very integers near the ends of the int64 range. + import ( "encoding/binary" "fmt" diff --git a/tsdb/engine/pd1/timestamp.go b/tsdb/engine/pd1/timestamp.go index 6863a51afed..88a2c2517bd 100644 --- a/tsdb/engine/pd1/timestamp.go +++ b/tsdb/engine/pd1/timestamp.go @@ -1,13 +1,40 @@ -// Package timestamp provides structs and functions for converting streams of timestamps -// to byte slices. -// -// The encoding is adapative based on structure of the timestamps that are encoded. By default, -// a bit-packed format that compresses multiple 64bit timestamps into a single 64bit word is used. -// If the values are too large to be compressed using the bit-packed format, it will fall back to -// a raw 8byte per timestamp format. If the the values can be run-length encoded, based on the -// differences between consectutive values, a shorter, variable sized RLE format is used. package pd1 +// Timestamp encoding is adapative and based on structure of the timestamps that are encoded. It +// uses a combination of delta encoding, zig zag encoding, scaling and compression using simple8b, +// run length encoding as well as falling back to no compression if needed. +// +// Timestamp values to be encoded should be sorted before encoding. When encoded, the values are +// first delta-encoded. The first value is the starting timestamp, subsequent values are the difference. +// from the prior value. +// +// Delta encoding can produce negative values. After delta encoding, the values are zig zag encoded +// to convert them to positive values. +// +// Timestamp resolution can also be in the nanosecond. Many timestamps are monotonically increasing +// and fall on even boundaries of time such as every 10s. When the timestamps have this structure, +// they are scaled by the largest common divisor that is also a factor of 10. This has the effect +// of converting very large integer deltas into very small one that can be reversed by multiplying them +// by the scaling factor. +// +// Using these adjusted values, if all the deltas are the same, the time range is stored using run +// length encoding. If run length encoding is not possible and all values are less than 1 << 60 - 1 +// (~36.5 yrs in nanosecond resolution), then the timestamps are encoded using simple8b encoding. If +// any value exceeds the maximum values, the deltas are stored uncompressed using 8b each. +// +// Each compressed byte slice has a 1 byte header indicating the compression type. The 4 high bits +// indicated the encoding type. The 4 low bits are using by the encoding type. +// +// For run length encoding, the 4 low bits store the log10 of the scaling factor. The next 8 bytes are +// the starting timestamp, next 1-10 bytes is the delta value using variable-length encoding, finally the +// next 1-10 bytes is the count of values. +// +// For simple8b encoding, the 4 low bits store the log10 of the scaling factor. The next 8 bytes is the +// first delta value stored uncompressed, the remaining bytes are 64bit words containg compressed delta +// values. +// +// For uncompressed encoding, the delta values are stored using 8 bytes each. + import ( "encoding/binary" "fmt" From c0eba04d9d86f37a736ffc6402fae828eda5efc8 Mon Sep 17 00:00:00 2001 From: Jason Wilder Date: Tue, 29 Sep 2015 10:47:35 -0600 Subject: [PATCH 041/139] Rename float encoding tests --- tsdb/engine/pd1/float_test.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tsdb/engine/pd1/float_test.go b/tsdb/engine/pd1/float_test.go index 5bfa377ac1b..60702938f5f 100644 --- a/tsdb/engine/pd1/float_test.go +++ b/tsdb/engine/pd1/float_test.go @@ -6,7 +6,7 @@ import ( "github.com/influxdb/influxdb/tsdb/engine/pd1" ) -func TestExampleEncoding(t *testing.T) { +func TestFloatEncoder_Simple(t *testing.T) { // Example from the paper s := pd1.NewFloatEncoder() @@ -92,7 +92,7 @@ var TwoHoursData = []struct { {786}, {785}, {774}, {786}, {718}, } -func TestRoundtrip(t *testing.T) { +func TestFloatEncoder_Roundtrip(t *testing.T) { s := pd1.NewFloatEncoder() for _, p := range TwoHoursData { From 72fa3dd5a4c3d5f0b838e3326d7365fdb835a354 Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Tue, 29 Sep 2015 16:17:58 -0400 Subject: [PATCH 042/139] Update WAL to deduplicate values on Cursor query. Added test and have failing section for single value encoding. --- tsdb/engine/pd1/pd1_test.go | 51 +++++++++++++++++++++++++++---------- tsdb/engine/pd1/wal.go | 5 ++-- 2 files changed, 39 insertions(+), 17 deletions(-) diff --git a/tsdb/engine/pd1/pd1_test.go b/tsdb/engine/pd1/pd1_test.go index 2a59c9ac4e2..e3e80aee3f6 100644 --- a/tsdb/engine/pd1/pd1_test.go +++ b/tsdb/engine/pd1/pd1_test.go @@ -634,28 +634,51 @@ func TestEngine_WriteManyPointsToSingleSeries(t *testing.T) { } } -func TestEngine_WriteIndexBenchmarkNames(t *testing.T) { - t.Skip("whatevs") - +func TestEngine_WritePointsInMultipleRequestsWithSameTime(t *testing.T) { e := OpenDefaultEngine() defer e.Cleanup() - var points []models.Point - for i := 0; i < 100000; i++ { - points = append(points, parsePoint(fmt.Sprintf("cpu%d value=22.1", i))) - } + fields := []string{"value"} - st := time.Now() - if err := e.WritePoints(points, nil, nil); err != nil { + e.WAL.SkipCache = false + + if err := e.WritePoints([]models.Point{parsePoint("foo value=1 0")}, nil, nil); err != nil { t.Fatalf("failed to write points: %s", err.Error()) } - fmt.Println("took: ", time.Since(st)) - - st = time.Now() - if err := e.WritePoints(points, nil, nil); err != nil { + if err := e.WritePoints([]models.Point{parsePoint("foo value=2 0")}, nil, nil); err != nil { t.Fatalf("failed to write points: %s", err.Error()) } - fmt.Println("took: ", time.Since(st)) + if err := e.WritePoints([]models.Point{parsePoint("foo value=3 0")}, nil, nil); err != nil { + t.Fatalf("failed to write points: %s", err.Error()) + } + + verify := func() { + tx, _ := e.Begin(false) + defer tx.Rollback() + c := tx.Cursor("foo", fields, nil, true) + k, v := c.SeekTo(0) + if k != 0 { + t.Fatalf("expected 0 time but got %d", k) + } + if v != float64(3) { + t.Fatalf("expected 3 for value but got %f", v.(float64)) + } + k, _ = c.Next() + if k != tsdb.EOF { + t.Fatal("expected EOF") + } + } + + verify() + + if err := e.Close(); err != nil { + t.Fatalf("error closing: %s", err.Error()) + } + if err := e.Open(); err != nil { + t.Fatalf("error opening: %s", err.Error()) + } + + verify() } // Engine represents a test wrapper for pd1.Engine. diff --git a/tsdb/engine/pd1/wal.go b/tsdb/engine/pd1/wal.go index 49b72b4a608..f44937ac674 100644 --- a/tsdb/engine/pd1/wal.go +++ b/tsdb/engine/pd1/wal.go @@ -177,13 +177,12 @@ func (l *Log) Cursor(series string, fields []string, dec *tsdb.FieldCodec, ascen copy(c, fc) c = append(c, values...) - return newWALCursor(c, ascending) + return newWALCursor(Values(c).Deduplicate(), ascending) } } if l.cacheDirtySort[ck] { - sort.Sort(values) - delete(l.cacheDirtySort, ck) + values = Values(values).Deduplicate() } // build a copy so writes afterwards don't change the result set From 6c2bef6073296c4cdb9831b462c9e6927b1330a5 Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Tue, 29 Sep 2015 20:03:37 -0400 Subject: [PATCH 043/139] Add logging to pd1 --- tsdb/engine/pd1/pd1.go | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/tsdb/engine/pd1/pd1.go b/tsdb/engine/pd1/pd1.go index d707661fad7..3834aad2261 100644 --- a/tsdb/engine/pd1/pd1.go +++ b/tsdb/engine/pd1/pd1.go @@ -7,6 +7,7 @@ import ( "hash/fnv" "io" "io/ioutil" + "log" "math" "os" "path/filepath" @@ -73,6 +74,7 @@ type Engine struct { writeLock *writeLock metaLock sync.Mutex path string + logger *log.Logger // deletesPending mark how many old data files are waiting to be deleted. This will // keep a close from returning until all deletes finish @@ -117,6 +119,7 @@ func NewEngine(path string, walPath string, opt tsdb.EngineOptions) tsdb.Engine e := &Engine{ path: path, writeLock: &writeLock{}, + logger: log.New(os.Stderr, "[pd1] ", log.LstdFlags), // TODO: this is the function where we can inject a check against the in memory collisions HashSeriesField: hashSeriesField, @@ -142,6 +145,7 @@ func (e *Engine) PerformMaintenance() { e.WAL.flush(f) }() } else if e.shouldCompact() { + e.logger.Println("compacting for maintenance") go e.Compact(true) } } @@ -445,7 +449,7 @@ func (e *Engine) Compact(fullCompaction bool) error { break } - fmt.Println("Starting compaction with files:", len(files)) + e.logger.Printf("Starting compaction in partition %s of %d files", e.path, len(files)) st := time.Now() // mark the compaction as running @@ -591,7 +595,7 @@ func (e *Engine) Compact(fullCompaction bool) error { e.files = newFiles e.filesLock.Unlock() - fmt.Println("Compaction took ", time.Since(st)) + e.logger.Println("Compaction took ", time.Since(st)) // delete the old files in a goroutine so running queries won't block the write // from completing From 4937ae8fbfa13f6da7e0fab26e985381629de507 Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Tue, 29 Sep 2015 21:51:16 -0400 Subject: [PATCH 044/139] Fix panic when data file has small index --- tsdb/engine/pd1/pd1.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tsdb/engine/pd1/pd1.go b/tsdb/engine/pd1/pd1.go index 3834aad2261..a6bd2113a16 100644 --- a/tsdb/engine/pd1/pd1.go +++ b/tsdb/engine/pd1/pd1.go @@ -520,7 +520,7 @@ func (e *Engine) Compact(fullCompaction bool) error { // write the blocks out to file that are already at their size limit for { - // if the next block is the same ID, we don't need to decod this one + // if the next block is the same ID, we don't need to decode this one // so we can just write it out to the file nextID, _, nextBlock := df.block(newPos) newPos = newPos + uint32(blockHeaderSize+len(block)) @@ -1568,7 +1568,7 @@ func (c *cursor) SeekTo(seek int64) (int64, interface{}) { // if the next block has a time less than what we're seeking to, // skip decoding this block and continue on nextBlockPos := pos + 12 + length - if nextBlockPos < c.f.size { + if nextBlockPos < c.f.indexPosition() { nextBlockID := btou64(c.f.mmap[nextBlockPos : nextBlockPos+8]) if nextBlockID == c.id { nextBlockTime := int64(btou64(c.f.mmap[nextBlockPos+12 : nextBlockPos+20])) From f29c4c8cf9ccfef5852b67295a43a14221a70f44 Mon Sep 17 00:00:00 2001 From: Jason Wilder Date: Tue, 29 Sep 2015 16:08:18 -0600 Subject: [PATCH 045/139] Add compressed string encoding Uses snappy to compress multiple strings into a block --- tsdb/engine/pd1/encoding.go | 90 +++++++++++++++++++++++++++++--- tsdb/engine/pd1/encoding_test.go | 18 +++++++ tsdb/engine/pd1/string.go | 82 +++++++++++++++++++++++++++++ tsdb/engine/pd1/string_test.go | 78 +++++++++++++++++++++++++++ 4 files changed, 262 insertions(+), 6 deletions(-) create mode 100644 tsdb/engine/pd1/string.go create mode 100644 tsdb/engine/pd1/string_test.go diff --git a/tsdb/engine/pd1/encoding.go b/tsdb/engine/pd1/encoding.go index d7abb195b5b..005692d2735 100644 --- a/tsdb/engine/pd1/encoding.go +++ b/tsdb/engine/pd1/encoding.go @@ -22,6 +22,9 @@ const ( // EncodingBitPacked is a basic bit-packed format EncodingBitPacked = 3 + // EncodingSnappy is a snappy encoded format + EncodingSnappy = 4 + // BlockFloat64 designates a block encodes float64 values BlockFloat64 = 0 @@ -50,8 +53,8 @@ func NewValue(t time.Time, value interface{}) Value { return &FloatValue{time: t, value: v} case bool: return &BoolValue{time: t, value: v} - // case string: - // return &StringValue{time: t, value: v} + case string: + return &StringValue{time: t, value: v} } return &EmptyValue{} } @@ -100,7 +103,12 @@ func (v Values) Encode(buf []byte) []byte { } return encodeBoolBlock(buf, a) - // TODO: add support for other types + case *StringValue: + a := make([]*StringValue, len(v)) + for i, vv := range v { + a[i] = vv.(*StringValue) + } + return encodeStringBlock(buf, a) } return nil @@ -117,6 +125,9 @@ func (v Values) DecodeSameTypeBlock(block []byte) Values { case *BoolValue: a, _ := decodeBoolBlock(block) return a + case *StringValue: + a, _ := decodeStringBlock(block) + return a } return nil } @@ -137,7 +148,7 @@ func DecodeBlock(block []byte) (Values, error) { case BlockBool: return decodeBoolBlock(block) case BlockString: - // return decodeStringBlock(block) + return decodeStringBlock(block) default: } @@ -423,8 +434,75 @@ type StringValue struct { value string } -func encodeStringBlock(buf []byte, blockType byte, values []StringValue) []byte { - return nil +func (v *StringValue) Time() time.Time { + return v.time +} + +func (v *StringValue) Value() interface{} { + return v.value +} + +func (v *StringValue) UnixNano() int64 { + return v.time.UnixNano() +} + +func (v *StringValue) Size() int { + return 8 + len(v.value) +} + +func (v *StringValue) String() string { return v.value } + +func encodeStringBlock(buf []byte, values []*StringValue) []byte { + tsEnc := NewTimeEncoder() + vEnc := NewStringEncoder() + for _, v := range values { + tsEnc.Write(v.Time()) + vEnc.Write(v.value) + } + + // Encoded timestamp values + tb, err := tsEnc.Bytes() + if err != nil { + panic(err.Error()) + } + // Encoded int64 values + vb, err := vEnc.Bytes() + if err != nil { + panic(err.Error()) + } + + // Preprend the first timestamp of the block in the first 8 bytes + block := packBlockHeader(values[0].Time(), BlockString) + return append(block, packBlock(tb, vb)...) +} + +func decodeStringBlock(block []byte) ([]Value, error) { + // slice off the first 8 bytes (min timestmap for the block) + block = block[8:] + + blockType := block[0] + if blockType != BlockString { + return nil, fmt.Errorf("invalid block type: exp %d, got %d", BlockString, blockType) + } + + block = block[1:] + + // The first 8 bytes is the minimum timestamp of the block + tb, vb := unpackBlock(block) + + // Setup our timestamp and value decoders + tsDec := NewTimeDecoder(tb) + vDec := NewStringDecoder(vb) + + // Decode both a timestamp and value + var a []Value + for tsDec.Next() && vDec.Next() { + ts := tsDec.Read() + v := vDec.Read() + a = append(a, &StringValue{ts, v}) + } + + return a, nil } func packBlockHeader(firstTime time.Time, blockType byte) []byte { diff --git a/tsdb/engine/pd1/encoding_test.go b/tsdb/engine/pd1/encoding_test.go index aa5b67d6409..306ad155767 100644 --- a/tsdb/engine/pd1/encoding_test.go +++ b/tsdb/engine/pd1/encoding_test.go @@ -3,6 +3,7 @@ package pd1_test import ( // "math/rand" + "fmt" "reflect" "testing" "time" @@ -97,6 +98,23 @@ func TestEncoding_BoolBlock_Basic(t *testing.T) { } } +func TestEncoding_StringBlock_Basic(t *testing.T) { + valueCount := 1000 + times := getTimes(valueCount, 60, time.Second) + values := make(pd1.Values, len(times)) + for i, t := range times { + values[i] = pd1.NewValue(t, fmt.Sprintf("value %d", i)) + } + + b := values.Encode(nil) + + decodedValues := values.DecodeSameTypeBlock(b) + + if !reflect.DeepEqual(decodedValues, values) { + t.Fatalf("unexpected results:\n\tgot: %v\n\texp: %v\n", decodedValues, values) + } +} + func getTimes(n, step int, precision time.Duration) []time.Time { t := time.Now().Round(precision) a := make([]time.Time, n) diff --git a/tsdb/engine/pd1/string.go b/tsdb/engine/pd1/string.go new file mode 100644 index 00000000000..e548b68c560 --- /dev/null +++ b/tsdb/engine/pd1/string.go @@ -0,0 +1,82 @@ +package pd1 + +// String encoding uses snappy compression to compress each string. Each string is +// appended to byte slice prefixed with a variable byte length followed by the string +// bytes. The bytes are compressed using snappy compressor and a 1 byte header is used +// to indicate the type of encoding. + +import ( + "encoding/binary" + "fmt" + + "github.com/golang/snappy" +) + +type StringEncoder interface { + Write(s string) + Bytes() ([]byte, error) +} + +type StringDecoder interface { + Next() bool + Read() string +} + +type stringEncoder struct { + // The encoded bytes + bytes []byte +} + +func NewStringEncoder() StringEncoder { + return &stringEncoder{} +} + +func (e *stringEncoder) Write(s string) { + b := make([]byte, 10) + // Append the length of the string using variable byte encoding + i := binary.PutUvarint(b, uint64(len(s))) + e.bytes = append(e.bytes, b[:i]...) + + // Append the string bytes + e.bytes = append(e.bytes, s...) +} + +func (e *stringEncoder) Bytes() ([]byte, error) { + // Compress the currently appended bytes using snappy and prefix with + // a 1 byte header for future extension + data := snappy.Encode(nil, e.bytes) + return append([]byte{EncodingSnappy << 4}, data...), nil +} + +type stringDecoder struct { + b []byte + l int + i int +} + +func NewStringDecoder(b []byte) StringDecoder { + // First byte stores the encoding type, only have snappy format + // currently so ignore for now. + data, err := snappy.Decode(nil, b[1:]) + if err != nil { + // TODO: Need to propogate errors up the call stack better + panic(fmt.Sprintf("failed to decode string block: %v", err.Error())) + } + + return &stringDecoder{b: data} +} + +func (e *stringDecoder) Next() bool { + e.i += e.l + return e.i < len(e.b) +} + +func (e *stringDecoder) Read() string { + // Read the length of the string + length, n := binary.Uvarint(e.b[e.i:]) + + // The length of this string plus the length of the variable byte encoded length + e.l = int(length) + n + + return string(e.b[e.i+n : e.i+n+int(length)]) +} diff --git a/tsdb/engine/pd1/string_test.go b/tsdb/engine/pd1/string_test.go new file mode 100644 index 00000000000..c457de8697d --- /dev/null +++ b/tsdb/engine/pd1/string_test.go @@ -0,0 +1,78 @@ +package pd1_test + +import ( + "fmt" + "testing" + + "github.com/influxdb/influxdb/tsdb/engine/pd1" +) + +func Test_StringEncoder_NoValues(t *testing.T) { + enc := pd1.NewStringEncoder() + b, err := enc.Bytes() + if err != nil { + t.Fatalf("unexpected error: %v", err) + } + + dec := pd1.NewStringDecoder(b) + if dec.Next() { + t.Fatalf("unexpected next value: got true, exp false") + } +} + +func Test_StringEncoder_Single(t *testing.T) { + enc := pd1.NewStringEncoder() + v1 := "v1" + enc.Write(v1) + b, err := enc.Bytes() + if err != nil { + t.Fatalf("unexpected error: %v", err) + } + + dec := pd1.NewStringDecoder(b) + if !dec.Next() { + t.Fatalf("unexpected next value: got false, exp true") + } + + if v1 != dec.Read() { + t.Fatalf("unexpected value: got %v, exp %v", dec.Read(), v1) + } +} + +func Test_StringEncoder_Multi_Compressed(t *testing.T) { + enc := pd1.NewStringEncoder() + + values := make([]string, 10) + for i := range values { + values[i] = fmt.Sprintf("value %d", i) + enc.Write(values[i]) + } + + b, err := enc.Bytes() + if err != nil { + t.Fatalf("unexpected error: %v", err) + } + + if b[0]>>4 != pd1.EncodingSnappy { + t.Fatalf("unexpected encoding: got %v, exp %v", b[0], pd1.EncodingSnappy) + } + + if exp := 47; len(b) != exp { + t.Fatalf("unexpected length: got %v, exp %v", len(b), exp) + } + + dec := pd1.NewStringDecoder(b) + + for i, v := range values { + if !dec.Next() { + t.Fatalf("unexpected next value: got false, exp true") + } + if v != dec.Read() { + t.Fatalf("unexpected value at pos %d: got %v, exp %v", i, dec.Read(), v) + } + } + + if dec.Next() { + t.Fatalf("unexpected next value: got true, exp false") + } +} From c27de6fbb877c092638cdc8befe053a770336fd2 Mon Sep 17 00:00:00 2001 From: Jason Wilder Date: Tue, 29 Sep 2015 20:01:43 -0600 Subject: [PATCH 046/139] Add test with duplicate timestamps Should not happen but makes sure that the same values are encoded and decoded correctly. --- tsdb/engine/pd1/encoding_test.go | 15 +++++++++++++++ 1 file changed, 15 insertions(+) diff --git a/tsdb/engine/pd1/encoding_test.go b/tsdb/engine/pd1/encoding_test.go index 306ad155767..7126eec3887 100644 --- a/tsdb/engine/pd1/encoding_test.go +++ b/tsdb/engine/pd1/encoding_test.go @@ -28,6 +28,21 @@ func TestEncoding_FloatBlock(t *testing.T) { } } +func TestEncoding_FloatBlock_ZeroTime(t *testing.T) { + values := make(pd1.Values, 3) + for i := 0; i < 3; i++ { + values[i] = pd1.NewValue(time.Unix(0, 0), float64(i)) + } + + b := values.Encode(nil) + + decodedValues := values.DecodeSameTypeBlock(b) + + if !reflect.DeepEqual(decodedValues, values) { + t.Fatalf("unexpected results:\n\tgot: %v\n\texp: %v\n", decodedValues, values) + } +} + func TestEncoding_IntBlock_Basic(t *testing.T) { valueCount := 1000 times := getTimes(valueCount, 60, time.Second) From 9b84a20dec53b9e89eb1f68d300fe8daef96b22e Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Wed, 30 Sep 2015 11:48:26 -0400 Subject: [PATCH 047/139] Fix bugs with writing old data and compaction. --- tsdb/engine/pd1/pd1.go | 69 ++++++++---- tsdb/engine/pd1/pd1_test.go | 213 ++++++++++++++++++++++++++++++++++++ 2 files changed, 261 insertions(+), 21 deletions(-) diff --git a/tsdb/engine/pd1/pd1.go b/tsdb/engine/pd1/pd1.go index a6bd2113a16..37e16c6d498 100644 --- a/tsdb/engine/pd1/pd1.go +++ b/tsdb/engine/pd1/pd1.go @@ -92,6 +92,8 @@ type Engine struct { CompactionFileCount int IndexCompactionFullAge time.Duration IndexMinimumCompactionInterval time.Duration + MaxPointsPerBlock int + RotateBlockSize int // filesLock is only for modifying and accessing the files slice filesLock sync.RWMutex @@ -129,6 +131,8 @@ func NewEngine(path string, walPath string, opt tsdb.EngineOptions) tsdb.Engine CompactionFileCount: opt.Config.IndexCompactionFileCount, IndexCompactionFullAge: opt.Config.IndexCompactionFullAge, IndexMinimumCompactionInterval: opt.Config.IndexMinimumCompactionInterval, + MaxPointsPerBlock: DefaultMaxPointsPerBlock, + RotateBlockSize: DefaultRotateBlockSize, } e.WAL.Index = e @@ -144,6 +148,7 @@ func (e *Engine) PerformMaintenance() { go func() { e.WAL.flush(f) }() + return } else if e.shouldCompact() { e.logger.Println("compacting for maintenance") go e.Compact(true) @@ -449,7 +454,11 @@ func (e *Engine) Compact(fullCompaction bool) error { break } - e.logger.Printf("Starting compaction in partition %s of %d files", e.path, len(files)) + var s string + if fullCompaction { + s = "FULL " + } + e.logger.Printf("Starting %scompaction in partition %s of %d files", s, e.path, len(files)) st := time.Now() // mark the compaction as running @@ -486,16 +495,16 @@ func (e *Engine) Compact(fullCompaction bool) error { currentPosition := uint32(fileHeaderSize) newPositions := make([]uint32, 0) newIDs := make([]uint64, 0) - buf := make([]byte, DefaultRotateBlockSize) + buf := make([]byte, e.RotateBlockSize) for { // find the min ID so we can write it to the file minID := uint64(math.MaxUint64) for _, id := range ids { - if minID > id { + if minID > id && id != 0 { minID = id } } - if minID == 0 { // we've emptied all the files + if minID == math.MaxUint64 { // we've emptied all the files break } @@ -520,14 +529,10 @@ func (e *Engine) Compact(fullCompaction bool) error { // write the blocks out to file that are already at their size limit for { - // if the next block is the same ID, we don't need to decode this one - // so we can just write it out to the file - nextID, _, nextBlock := df.block(newPos) - newPos = newPos + uint32(blockHeaderSize+len(block)) - + // write the values, the block or combine with previous if len(previousValues) > 0 { previousValues = append(previousValues, previousValues.DecodeSameTypeBlock(block)...) - } else if len(block) > DefaultRotateBlockSize { + } else if len(block) > e.RotateBlockSize { if _, err := f.Write(df.mmap[pos:newPos]); err != nil { return err } @@ -538,7 +543,7 @@ func (e *Engine) Compact(fullCompaction bool) error { } // write the previous values and clear if we've hit the limit - if len(previousValues) > DefaultMaxPointsPerBlock { + if len(previousValues) > e.MaxPointsPerBlock { b := previousValues.Encode(buf) if err := e.writeBlock(f, id, b); err != nil { // fail hard. If we can't write a file someone needs to get woken up @@ -548,14 +553,28 @@ func (e *Engine) Compact(fullCompaction bool) error { previousValues = nil } + // if the next block is the same ID, we don't need to decode this one + // so we can just write it out to the file + nextID, _, nextBlock := df.block(newPos) + // move to the next block in this file only if the id is the same if nextID != id { + // flush remaining values + if len(previousValues) > 0 { + b := previousValues.Encode(buf) + currentPosition += uint32(blockHeaderSize + len(b)) + previousValues = nil + if err := e.writeBlock(f, id, b); err != nil { + panic(fmt.Sprintf("error writing file %s: %s", f.Name(), err.Error())) + } + } ids[i] = nextID break } + pos = newPos + newPos = pos + uint32(blockHeaderSize+len(nextBlock)) positions[i] = newPos block = nextBlock - newPos = newPos + uint32(blockHeaderSize+len(block)) } } @@ -595,7 +614,7 @@ func (e *Engine) Compact(fullCompaction bool) error { e.files = newFiles e.filesLock.Unlock() - e.logger.Println("Compaction took ", time.Since(st)) + e.logger.Printf("Compaction of %s took %s", e.path, time.Since(st)) // delete the old files in a goroutine so running queries won't block the write // from completing @@ -603,8 +622,7 @@ func (e *Engine) Compact(fullCompaction bool) error { go func() { for _, f := range files { if err := f.Delete(); err != nil { - // TODO: log this error - fmt.Println("ERROR DELETING:", f.f.Name()) + e.logger.Println("ERROR DELETING:", f.f.Name()) } } e.deletesPending.Done() @@ -824,7 +842,7 @@ func (e *Engine) filterDataBetweenTimes(valuesByID map[uint64]Values, minTime, m filteredValues := make(map[uint64]Values) for id, values := range valuesByID { maxIndex := len(values) - minIndex := 0 + minIndex := -1 // find the index of the first value in the range for i, v := range values { t := v.UnixNano() @@ -833,6 +851,9 @@ func (e *Engine) filterDataBetweenTimes(valuesByID map[uint64]Values, minTime, m break } } + if minIndex == -1 { + continue + } // go backwards to find the index of the last value in the range for i := len(values) - 1; i >= 0; i-- { t := values[i].UnixNano() @@ -883,6 +904,7 @@ func (e *Engine) rewriteFile(oldDF *dataFile, valuesByID map[uint64]Values) erro minTime = oldDF.MinTime() maxTime = oldDF.MaxTime() } + for _, v := range valuesByID { if minTime > v.MinTime() { minTime = v.MinTime() @@ -919,7 +941,7 @@ func (e *Engine) rewriteFile(oldDF *dataFile, valuesByID map[uint64]Values) erro // their positions currentPosition := uint32(fileHeaderSize) newPositions := make([]uint32, len(ids)) - buf := make([]byte, DefaultMaxPointsPerBlock*20) + buf := make([]byte, e.MaxPointsPerBlock*20) for i, id := range ids { // mark the position for this ID newPositions[i] = currentPosition @@ -1041,7 +1063,7 @@ func (e *Engine) rewriteFile(oldDF *dataFile, valuesByID map[uint64]Values) erro e.deletesPending.Add(1) go func() { if err := oldDF.Delete(); err != nil { - fmt.Println("ERROR DELETING FROM REWRITE:", oldDF.f.Name()) + e.logger.Println("ERROR DELETING FROM REWRITE:", oldDF.f.Name()) } e.deletesPending.Done() }() @@ -1346,9 +1368,9 @@ func (e *Engine) DecodeAndCombine(newValues Values, block, buf []byte, nextTime } } - if len(values) > DefaultMaxPointsPerBlock { - remainingValues = values[DefaultMaxPointsPerBlock:] - values = values[:DefaultMaxPointsPerBlock] + if len(values) > e.MaxPointsPerBlock { + remainingValues = values[e.MaxPointsPerBlock:] + values = values[:e.MaxPointsPerBlock] } return remainingValues, values.Encode(buf), nil @@ -1485,6 +1507,11 @@ func (d *dataFile) StartingPositionForID(id uint64) uint32 { } func (d *dataFile) block(pos uint32) (id uint64, t int64, block []byte) { + defer func() { + if r := recover(); r != nil { + fmt.Println("FUCK: ", d.f.Name(), pos, id, t) + } + }() if pos < d.indexPosition() { id = btou64(d.mmap[pos : pos+8]) length := btou32(d.mmap[pos+8 : pos+12]) diff --git a/tsdb/engine/pd1/pd1_test.go b/tsdb/engine/pd1/pd1_test.go index e3e80aee3f6..9eda01fdc46 100644 --- a/tsdb/engine/pd1/pd1_test.go +++ b/tsdb/engine/pd1/pd1_test.go @@ -681,6 +681,219 @@ func TestEngine_WritePointsInMultipleRequestsWithSameTime(t *testing.T) { verify() } +func TestEngine_CompactWithSeriesInOneFile(t *testing.T) { + e := OpenDefaultEngine() + defer e.Cleanup() + + fields := []string{"value"} + + e.RotateFileSize = 10 + e.MaxPointsPerBlock = 1 + + p1 := parsePoint("cpu,host=A value=1.1 1000000000") + p2 := parsePoint("cpu,host=B value=1.2 2000000000") + p3 := parsePoint("cpu,host=A value=1.3 3000000000") + + if err := e.WritePoints([]models.Point{p1}, nil, nil); err != nil { + t.Fatalf("failed to write points: %s", err.Error()) + } + if err := e.WritePoints([]models.Point{p2}, nil, nil); err != nil { + t.Fatalf("failed to write points: %s", err.Error()) + } + if err := e.WritePoints([]models.Point{p3}, nil, nil); err != nil { + t.Fatalf("failed to write points: %s", err.Error()) + } + + if count := e.DataFileCount(); count != 3 { + t.Fatalf("expected 3 data files but got %d", count) + } + + verify := func() { + tx, _ := e.Begin(false) + defer tx.Rollback() + c := tx.Cursor("cpu,host=A", fields, nil, false) + k, v := c.SeekTo(0) + if k != 1000000000 { + t.Fatalf("expected time 1000000000 but got %d", k) + } + if v != 1.1 { + t.Fatalf("expected value 1.1 but got %f", v.(float64)) + } + k, v = c.Next() + if k != 3000000000 { + t.Fatalf("expected time 3000000000 but got %d", k) + } + c = tx.Cursor("cpu,host=B", fields, nil, false) + k, v = c.SeekTo(0) + if k != 2000000000 { + t.Fatalf("expected time 2000000000 but got %d", k) + } + if v != 1.2 { + t.Fatalf("expected value 1.2 but got %f", v.(float64)) + } + } + + fmt.Println("verify 1") + verify() + + if err := e.Compact(true); err != nil { + t.Fatalf("error compacting: %s", err.Error) + } + fmt.Println("verify 2") + verify() + + p4 := parsePoint("cpu,host=A value=1.4 4000000000") + if err := e.WritePoints([]models.Point{p4}, nil, nil); err != nil { + t.Fatalf("failed to write points: %s", err.Error()) + } + + if err := e.Compact(true); err != nil { + t.Fatalf("error compacting: %s", err.Error) + } + tx1, _ := e.Begin(false) + defer tx1.Rollback() + c := tx1.Cursor("cpu,host=A", fields, nil, false) + k, v := c.SeekTo(0) + if k != 1000000000 { + t.Fatalf("expected time 1000000000 but got %d", k) + } + if v != 1.1 { + t.Fatalf("expected value 1.1 but got %f", v.(float64)) + } + k, v = c.Next() + if k != 3000000000 { + t.Fatalf("expected time 3000000000 but got %d", k) + } + k, v = c.Next() + if k != 4000000000 { + t.Fatalf("expected time 3000000000 but got %d", k) + } +} + +// Ensure that compactions that happen where blocks from old data files +// skip decoding and just get copied over to the new data file works. +func TestEngine_CompactionWithCopiedBlocks(t *testing.T) { + e := OpenDefaultEngine() + defer e.Cleanup() + + fields := []string{"value"} + + e.RotateFileSize = 10 + e.MaxPointsPerBlock = 1 + e.RotateBlockSize = 10 + + p1 := parsePoint("cpu,host=A value=1.1 1000000000") + p2 := parsePoint("cpu,host=A value=1.2 2000000000") + p3 := parsePoint("cpu,host=A value=1.3 3000000000") + + if err := e.WritePoints([]models.Point{p1, p2}, nil, nil); err != nil { + t.Fatalf("failed to write points: %s", err.Error()) + } + if err := e.WritePoints([]models.Point{p3}, nil, nil); err != nil { + t.Fatalf("failed to write points: %s", err.Error()) + } + + verify := func() { + tx, _ := e.Begin(false) + defer tx.Rollback() + c := tx.Cursor("cpu,host=A", fields, nil, false) + k, _ := c.SeekTo(0) + if k != 1000000000 { + t.Fatalf("expected time 1000000000 but got %d", k) + } + k, _ = c.Next() + if k != 2000000000 { + t.Fatalf("expected time 2000000000 but got %d", k) + } + k, _ = c.Next() + if k != 3000000000 { + t.Fatalf("expected time 3000000000 but got %d", k) + } + } + + verify() + if err := e.Compact(true); err != nil { + t.Fatalf("error compacting: %s", err.Error) + } + fmt.Println("verify 2") + verify() + + p4 := parsePoint("cpu,host=B value=1.4 4000000000") + if err := e.WritePoints([]models.Point{p4}, nil, nil); err != nil { + t.Fatalf("failed to write points: %s", err.Error()) + } + + if err := e.Compact(true); err != nil { + t.Fatalf("error compacting: %s", err.Error) + } + fmt.Println("verify 3") + verify() + + p5 := parsePoint("cpu,host=A value=1.5 5000000000") + p6 := parsePoint("cpu,host=A value=1.6 6000000000") + p7 := parsePoint("cpu,host=B value=2.1 7000000000") + if err := e.WritePoints([]models.Point{p5, p6, p7}, nil, nil); err != nil { + t.Fatalf("failed to write points: %s", err.Error()) + } + + p8 := parsePoint("cpu,host=A value=1.5 7000000000") + p9 := parsePoint("cpu,host=A value=1.6 8000000000") + p10 := parsePoint("cpu,host=B value=2.1 8000000000") + if err := e.WritePoints([]models.Point{p8, p9, p10}, nil, nil); err != nil { + t.Fatalf("failed to write points: %s", err.Error()) + } + + if err := e.Compact(true); err != nil { + t.Fatalf("error compacting: %s", err.Error) + } + verify() + +} + +func TestEngine_RewritingOldBlocks(t *testing.T) { + e := OpenDefaultEngine() + defer e.Cleanup() + + fields := []string{"value"} + + e.MaxPointsPerBlock = 2 + + p1 := parsePoint("cpu,host=A value=1.1 1000000000") + p2 := parsePoint("cpu,host=A value=1.2 2000000000") + p3 := parsePoint("cpu,host=A value=1.3 3000000000") + p4 := parsePoint("cpu,host=A value=1.5 1500000000") + + if err := e.WritePoints([]models.Point{p1, p2}, nil, nil); err != nil { + t.Fatalf("failed to write points: %s", err.Error()) + } + if err := e.WritePoints([]models.Point{p3}, nil, nil); err != nil { + t.Fatalf("failed to write points: %s", err.Error()) + } + if err := e.WritePoints([]models.Point{p4}, nil, nil); err != nil { + t.Fatalf("failed to write points: %s", err.Error()) + } + + tx, _ := e.Begin(false) + defer tx.Rollback() + c := tx.Cursor("cpu,host=A", fields, nil, false) + k, _ := c.SeekTo(0) + if k != 1000000000 { + t.Fatalf("expected time 1000000000 but got %d", k) + } + k, _ = c.Next() + if k != 1500000000 { + t.Fatalf("expected time 1500000000 but got %d", k) + } + k, _ = c.Next() + if k != 2000000000 { + t.Fatalf("expected time 2000000000 but got %d", k) + } + k, _ = c.Next() + if k != 3000000000 { + t.Fatalf("expected time 3000000000 but got %d", k) + } +} + // Engine represents a test wrapper for pd1.Engine. type Engine struct { *pd1.Engine From c1accf7e591763ce910a26fac7561c71cb9ba9ca Mon Sep 17 00:00:00 2001 From: Jason Wilder Date: Wed, 30 Sep 2015 09:48:20 -0600 Subject: [PATCH 048/139] Handle partial reads when loading WAL If reading into fixed sized buffer using io.ReadFull, the func can return io.ErrUnexpectedEOF if the read was short. This was slipping through the error handling causing the shard to fail to load. --- tsdb/engine/pd1/wal.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tsdb/engine/pd1/wal.go b/tsdb/engine/pd1/wal.go index f44937ac674..6e51980f2b2 100644 --- a/tsdb/engine/pd1/wal.go +++ b/tsdb/engine/pd1/wal.go @@ -353,7 +353,7 @@ func (l *Log) readFileToCache(fileName string) error { buf = make([]byte, length) } _, err = io.ReadFull(f, buf[0:length]) - if err == io.EOF { + if err == io.EOF || err == io.ErrUnexpectedEOF { l.logger.Printf("hit end of file while reading compressed wal entry from %s", fileName) return nil } else if err != nil { From 01b5b9268e6661b6f51c9c3abf7d714cb07b12f8 Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Wed, 30 Sep 2015 16:38:55 -0400 Subject: [PATCH 049/139] Fix compaction and multi-write bugs. * Fix bug with locking when the interval completely covers or is totally inside another one. * Fix bug with full compactions running when the index is actively being written to. --- tsdb/engine/pd1/pd1.go | 59 ++++++++++--- tsdb/engine/pd1/pd1_test.go | 67 +++++++++++++++ tsdb/engine/pd1/wal.go | 4 +- tsdb/engine/pd1/write_lock.go | 58 +++++++------ tsdb/engine/pd1/write_lock_test.go | 131 +++++++++++++++++++++++++++++ 5 files changed, 279 insertions(+), 40 deletions(-) create mode 100644 tsdb/engine/pd1/write_lock_test.go diff --git a/tsdb/engine/pd1/pd1.go b/tsdb/engine/pd1/pd1.go index 37e16c6d498..5f64c6c2f5a 100644 --- a/tsdb/engine/pd1/pd1.go +++ b/tsdb/engine/pd1/pd1.go @@ -71,7 +71,7 @@ var _ tsdb.Engine = &Engine{} // Engine represents a storage engine with compressed blocks. type Engine struct { - writeLock *writeLock + writeLock *WriteLock metaLock sync.Mutex path string logger *log.Logger @@ -120,7 +120,7 @@ func NewEngine(path string, walPath string, opt tsdb.EngineOptions) tsdb.Engine e := &Engine{ path: path, - writeLock: &writeLock{}, + writeLock: &WriteLock{}, logger: log.New(os.Stderr, "[pd1] ", log.LstdFlags), // TODO: this is the function where we can inject a check against the in memory collisions @@ -149,10 +149,28 @@ func (e *Engine) PerformMaintenance() { e.WAL.flush(f) }() return - } else if e.shouldCompact() { - e.logger.Println("compacting for maintenance") - go e.Compact(true) } + + // don't do a full compaction if the WAL received writes in the time window + if time.Since(e.WAL.LastWriteTime()) < e.IndexCompactionFullAge { + return + } + + e.filesLock.RLock() + running := e.compactionRunning + e.filesLock.RUnlock() + if running { + return + } + + // do a full compaction if all the index files are older than the compaction time + for _, f := range e.copyFilesCollection() { + if time.Since(f.modTime) < e.IndexCompactionFullAge { + return + } + } + + go e.Compact(true) } // Format returns the format type of this engine @@ -204,6 +222,8 @@ func (e *Engine) Open() error { return err } + e.lastCompactionTime = time.Now() + return nil } @@ -454,15 +474,12 @@ func (e *Engine) Compact(fullCompaction bool) error { break } - var s string - if fullCompaction { - s = "FULL " - } - e.logger.Printf("Starting %scompaction in partition %s of %d files", s, e.path, len(files)) - st := time.Now() - // mark the compaction as running e.filesLock.Lock() + if e.compactionRunning { + e.filesLock.Unlock() + return nil + } e.compactionRunning = true e.filesLock.Unlock() defer func() { @@ -474,11 +491,19 @@ func (e *Engine) Compact(fullCompaction bool) error { e.filesLock.Unlock() }() + var s string + if fullCompaction { + s = "FULL " + } + fileName := e.nextFileName() + e.logger.Printf("Starting %scompaction in partition %s of %d files to new file %s", s, e.path, len(files), fileName) + st := time.Now() + positions := make([]uint32, len(files)) ids := make([]uint64, len(files)) // initilaize for writing - f, err := os.OpenFile(e.nextFileName(), os.O_CREATE|os.O_RDWR, 0666) + f, err := os.OpenFile(fileName, os.O_CREATE|os.O_RDWR, 0666) if err != nil { return err } @@ -931,6 +956,12 @@ func (e *Engine) rewriteFile(oldDF *dataFile, valuesByID map[uint64]Values) erro return err } + if oldDF == nil { + e.logger.Printf("writing new index file %s", f.Name()) + } else { + e.logger.Printf("rewriting index file %s with %s", oldDF.f.Name(), f.Name()) + } + // write the magic number if _, err := f.Write(u32tob(magicNumber)); err != nil { f.Close() @@ -1509,7 +1540,7 @@ func (d *dataFile) StartingPositionForID(id uint64) uint32 { func (d *dataFile) block(pos uint32) (id uint64, t int64, block []byte) { defer func() { if r := recover(); r != nil { - fmt.Println("FUCK: ", d.f.Name(), pos, id, t) + panic(fmt.Sprintf("panic decoding file: %s at position %d for id %d at time %d", d.f.Name(), pos, id, t)) } }() if pos < d.indexPosition() { diff --git a/tsdb/engine/pd1/pd1_test.go b/tsdb/engine/pd1/pd1_test.go index 9eda01fdc46..494c63524fe 100644 --- a/tsdb/engine/pd1/pd1_test.go +++ b/tsdb/engine/pd1/pd1_test.go @@ -894,6 +894,73 @@ func TestEngine_RewritingOldBlocks(t *testing.T) { } } +func TestEngine_WriteIntoCompactedFile(t *testing.T) { + e := OpenDefaultEngine() + defer e.Cleanup() + + fields := []string{"value"} + + e.MaxPointsPerBlock = 3 + e.RotateFileSize = 10 + + p1 := parsePoint("cpu,host=A value=1.1 1000000000") + p2 := parsePoint("cpu,host=A value=1.2 2000000000") + p3 := parsePoint("cpu,host=A value=1.3 3000000000") + p4 := parsePoint("cpu,host=A value=1.5 4000000000") + p5 := parsePoint("cpu,host=A value=1.6 2500000000") + + if err := e.WritePoints([]models.Point{p1, p2}, nil, nil); err != nil { + t.Fatalf("failed to write points: %s", err.Error()) + } + if err := e.WritePoints([]models.Point{p3}, nil, nil); err != nil { + t.Fatalf("failed to write points: %s", err.Error()) + } + + if err := e.Compact(true); err != nil { + t.Fatalf("error compacting: %s", err.Error) + } + + if err := e.WritePoints([]models.Point{p4}, nil, nil); err != nil { + t.Fatalf("failed to write points: %s", err.Error()) + } + + if err := e.Compact(true); err != nil { + t.Fatalf("error compacting: %s", err.Error) + } + + if err := e.WritePoints([]models.Point{p5}, nil, nil); err != nil { + t.Fatalf("failed to write points: %s", err.Error()) + } + + if count := e.DataFileCount(); count != 1 { + t.Fatalf("execpted 1 data file but got %d", count) + } + + tx, _ := e.Begin(false) + defer tx.Rollback() + c := tx.Cursor("cpu,host=A", fields, nil, true) + k, _ := c.SeekTo(0) + if k != 1000000000 { + t.Fatalf("wrong time: %d", k) + } + k, _ = c.Next() + if k != 2000000000 { + t.Fatalf("wrong time: %d", k) + } + k, _ = c.Next() + if k != 2500000000 { + t.Fatalf("wrong time: %d", k) + } + k, _ = c.Next() + if k != 3000000000 { + t.Fatalf("wrong time: %d", k) + } + k, _ = c.Next() + if k != 4000000000 { + t.Fatalf("wrong time: %d", k) + } +} + // Engine represents a test wrapper for pd1.Engine. type Engine struct { *pd1.Engine diff --git a/tsdb/engine/pd1/wal.go b/tsdb/engine/pd1/wal.go index 6e51980f2b2..ef4399912cd 100644 --- a/tsdb/engine/pd1/wal.go +++ b/tsdb/engine/pd1/wal.go @@ -542,7 +542,7 @@ func (l *Log) flush(flush flushType) error { } else if flush == startupFlush { ftype = "startup" } - l.logger.Printf("%s flush of %d keys with %d values of %d bytes\n", ftype, len(l.flushCache), valueCount, flushSize) + l.logger.Printf("%s flush of %s with %d keys and %d total values of %d bytes\n", ftype, l.path, len(l.flushCache), valueCount, flushSize) } startTime := time.Now() @@ -550,7 +550,7 @@ func (l *Log) flush(flush flushType) error { return err } if l.LoggingEnabled { - l.logger.Printf("flush to index took %s\n", time.Since(startTime)) + l.logger.Printf("%s flush to index took %s\n", l.path, time.Since(startTime)) } l.cacheLock.Lock() diff --git a/tsdb/engine/pd1/write_lock.go b/tsdb/engine/pd1/write_lock.go index 5f48fb9311a..a791b663abb 100644 --- a/tsdb/engine/pd1/write_lock.go +++ b/tsdb/engine/pd1/write_lock.go @@ -1,15 +1,14 @@ package pd1 import ( + "reflect" "sync" ) // writeLock is a lock that enables locking of ranges between a // min and max value. We use this so that flushes from the WAL // can occur concurrently along with compactions. -type writeLock struct { - mu sync.Mutex - +type WriteLock struct { rangesLock sync.Mutex ranges []*rangeLock } @@ -19,34 +18,41 @@ type writeLock struct { // an overlapping range will have to wait until the previous // lock is released. A corresponding call to UnlockRange should // be deferred. -func (w *writeLock) LockRange(min, max int64) { - w.mu.Lock() - defer w.mu.Unlock() - +func (w *WriteLock) LockRange(min, max int64) { r := &rangeLock{min: min, max: max} - ranges := w.currentlyLockedRanges() + for { + ranges := w.currentlyLockedRanges() - // ensure there are no currently locked ranges that overlap - for _, rr := range ranges { - if rr.overlaps(r) { - // wait until it gets unlocked - rr.mu.Lock() - // release the lock so the object can get GC'd - rr.mu.Unlock() + // ensure there are no currently locked ranges that overlap + for _, rr := range ranges { + if rr.overlaps(r) { + // wait until it gets unlocked + rr.mu.Lock() + // release the lock so the object can get GC'd + rr.mu.Unlock() + } } - } - // and lock the range - r.mu.Lock() + // ensure that no one else got a lock on the range while we + // were waiting + w.rangesLock.Lock() + if len(w.ranges) == 0 || reflect.DeepEqual(ranges, w.ranges) { + // and lock the range + r.mu.Lock() - // now that we know the range is free, add it to the locks - w.rangesLock.Lock() - w.ranges = append(w.ranges, r) - w.rangesLock.Unlock() + // now that we know the range is free, add it to the locks + w.ranges = append(w.ranges, r) + w.rangesLock.Unlock() + return + } + + // try again + w.rangesLock.Unlock() + } } // UnlockRange will release a previously locked range. -func (w *writeLock) UnlockRange(min, max int64) { +func (w *WriteLock) UnlockRange(min, max int64) { w.rangesLock.Lock() defer w.rangesLock.Unlock() @@ -62,7 +68,7 @@ func (w *writeLock) UnlockRange(min, max int64) { w.ranges = a } -func (w *writeLock) currentlyLockedRanges() []*rangeLock { +func (w *WriteLock) currentlyLockedRanges() []*rangeLock { w.rangesLock.Lock() defer w.rangesLock.Unlock() a := make([]*rangeLock, len(w.ranges)) @@ -81,6 +87,10 @@ func (r *rangeLock) overlaps(l *rangeLock) bool { return true } else if l.max >= r.min && l.max <= r.max { return true + } else if l.min <= r.min && l.max >= r.max { + return true + } else if l.min >= r.min && l.max <= r.max { + return true } return false } diff --git a/tsdb/engine/pd1/write_lock_test.go b/tsdb/engine/pd1/write_lock_test.go new file mode 100644 index 00000000000..4cdc88e816e --- /dev/null +++ b/tsdb/engine/pd1/write_lock_test.go @@ -0,0 +1,131 @@ +package pd1_test + +import ( + // "sync" + "testing" + "time" + + "github.com/influxdb/influxdb/tsdb/engine/pd1" +) + +func TestWriteLock_FullCover(t *testing.T) { + w := &pd1.WriteLock{} + w.LockRange(2, 10) + + lock := make(chan bool) + timeout := time.NewTimer(10 * time.Millisecond) + go func() { + w.LockRange(1, 11) + lock <- true + }() + select { + case <-lock: + t.Fatal("able to get lock when we shouldn't") + case <-timeout.C: + // we're all good + } +} + +func TestWriteLock_RightIntersect(t *testing.T) { + w := &pd1.WriteLock{} + w.LockRange(2, 10) + + lock := make(chan bool) + timeout := time.NewTimer(10 * time.Millisecond) + go func() { + w.LockRange(5, 15) + lock <- true + }() + select { + case <-lock: + t.Fatal("able to get lock when we shouldn't") + case <-timeout.C: + // we're all good + } +} + +func TestWriteLock_LeftIntersect(t *testing.T) { + w := &pd1.WriteLock{} + w.LockRange(1, 4) + + lock := make(chan bool) + timeout := time.NewTimer(10 * time.Millisecond) + go func() { + w.LockRange(1, 11) + lock <- true + }() + select { + case <-lock: + t.Fatal("able to get lock when we shouldn't") + case <-timeout.C: + // we're all good + } +} + +func TestWriteLock_Inside(t *testing.T) { + w := &pd1.WriteLock{} + w.LockRange(4, 8) + + lock := make(chan bool) + timeout := time.NewTimer(10 * time.Millisecond) + go func() { + w.LockRange(1, 11) + lock <- true + }() + select { + case <-lock: + t.Fatal("able to get lock when we shouldn't") + case <-timeout.C: + // we're all good + } +} + +func TestWriteLock_Same(t *testing.T) { + w := &pd1.WriteLock{} + w.LockRange(2, 10) + + lock := make(chan bool) + timeout := time.NewTimer(10 * time.Millisecond) + go func() { + w.LockRange(2, 10) + lock <- true + }() + select { + case <-lock: + t.Fatal("able to get lock when we shouldn't") + case <-timeout.C: + // we're all good + } +} + +// func TestWriteLock_FreeRangeWithContentionElsewhere(t *testing.T) { +// w := &pd1.WriteLock{} +// w.LockRange(2, 10) + +// lock := make(chan bool) +// freeRange := make(chan bool) +// timeout := time.NewTimer(10 * time.Millisecond) +// var wg sync.WaitGroup + +// wg.Add(1) +// go func() { +// wg.Done() +// w.LockRange(4, 12) +// lock <- true +// }() + +// // make sure the other go func has gotten to the point of requesting the lock +// wg.Wait() +// go func() { +// w.LockRange(15, 23) +// freeRange <- true +// }() +// select { +// case <-lock: +// t.Fatal("able to get lock when we shouldn't") +// case <-timeout.C: +// t.Fatal("unable to get lock of free range when contention exists elsewhere") +// case <-freeRange: +// // we're all good +// } +// } From b826f8d6ac19b4a645f3dd2ea60213e3942c1808 Mon Sep 17 00:00:00 2001 From: Jason Wilder Date: Wed, 30 Sep 2015 13:11:49 -0600 Subject: [PATCH 050/139] Remove zig zag encoding from timestamp encoder Not needed since all timestamps will be sorted in ascending order. Negatives are not possible. --- tsdb/engine/pd1/timestamp.go | 22 +++++++--------------- tsdb/engine/pd1/timestamp_test.go | 2 +- 2 files changed, 8 insertions(+), 16 deletions(-) diff --git a/tsdb/engine/pd1/timestamp.go b/tsdb/engine/pd1/timestamp.go index 88a2c2517bd..b6fc6fef411 100644 --- a/tsdb/engine/pd1/timestamp.go +++ b/tsdb/engine/pd1/timestamp.go @@ -1,16 +1,13 @@ package pd1 // Timestamp encoding is adapative and based on structure of the timestamps that are encoded. It -// uses a combination of delta encoding, zig zag encoding, scaling and compression using simple8b, -// run length encoding as well as falling back to no compression if needed. +// uses a combination of delta encoding, scaling and compression using simple8b, run length encoding +// as well as falling back to no compression if needed. // // Timestamp values to be encoded should be sorted before encoding. When encoded, the values are // first delta-encoded. The first value is the starting timestamp, subsequent values are the difference. // from the prior value. // -// Delta encoding can produce negative values. After delta encoding, the values are zig zag encoded -// to convert them to positive values. -// // Timestamp resolution can also be in the nanosecond. Many timestamps are monotonically increasing // and fall on even boundaries of time such as every 10s. When the timestamps have this structure, // they are scaled by the largest common divisor that is also a factor of 10. This has the effect @@ -83,10 +80,7 @@ func (e *encoder) reduce() (max, divisor uint64, rle bool, deltas []uint64) { for i := len(deltas) - 1; i > 0; i-- { // First differential encode the values - delta := int64(deltas[i] - deltas[i-1]) - - // The delta may be negative so zigzag encode it into a postive value - deltas[i] = ZigZagEncode(delta) + deltas[i] = deltas[i] - deltas[i-1] // We're also need to keep track of the max value and largest common divisor v := deltas[i] @@ -243,8 +237,8 @@ func (d *decoder) decodePacked(b []byte) { // Compute the prefix sum and scale the deltas back up for i := 1; i < len(deltas); i++ { - dgap := ZigZagDecode(deltas[i] * div) - deltas[i] = uint64(int64(deltas[i-1]) + dgap) + dgap := deltas[i] * div + deltas[i] = deltas[i-1] + dgap } d.ts = deltas @@ -264,8 +258,6 @@ func (d *decoder) decodeRLE(b []byte) { // Next 1-10 bytes is our (scaled down by factor of 10) run length values value, n := binary.Uvarint(b[i:]) - value = uint64(ZigZagDecode(value)) - // Scale the value back up value *= uint64(mod) i += n @@ -293,10 +285,10 @@ func (d *decoder) decodeRaw(b []byte) { for i := range d.ts { d.ts[i] = binary.BigEndian.Uint64(b[i*8 : i*8+8]) - delta := ZigZagDecode(d.ts[i]) + delta := d.ts[i] // Compute the prefix sum and scale the deltas back up if i > 0 { - d.ts[i] = uint64(int64(d.ts[i-1]) + delta) + d.ts[i] = d.ts[i-1] + delta } } } diff --git a/tsdb/engine/pd1/timestamp_test.go b/tsdb/engine/pd1/timestamp_test.go index e0dd235d62a..f7030db8a40 100644 --- a/tsdb/engine/pd1/timestamp_test.go +++ b/tsdb/engine/pd1/timestamp_test.go @@ -297,7 +297,7 @@ func Test_TimeEncoder_Reverse(t *testing.T) { t.Fatalf("unexpected error: %v", err) } - if got := b[0] >> 4; got != pd1.EncodingPackedSimple { + if got := b[0] >> 4; got != pd1.EncodingUncompressed { t.Fatalf("Wrong encoding used: expected uncompressed, got %v", got) } From 5326ac423928f4279359b8167a8233f6aadb359d Mon Sep 17 00:00:00 2001 From: Jason Wilder Date: Wed, 30 Sep 2015 13:12:53 -0600 Subject: [PATCH 051/139] Fix go vet --- tsdb/engine/pd1/pd1_test.go | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/tsdb/engine/pd1/pd1_test.go b/tsdb/engine/pd1/pd1_test.go index 494c63524fe..e11a9baf929 100644 --- a/tsdb/engine/pd1/pd1_test.go +++ b/tsdb/engine/pd1/pd1_test.go @@ -737,7 +737,7 @@ func TestEngine_CompactWithSeriesInOneFile(t *testing.T) { verify() if err := e.Compact(true); err != nil { - t.Fatalf("error compacting: %s", err.Error) + t.Fatalf("error compacting: %s", err.Error()) } fmt.Println("verify 2") verify() @@ -748,7 +748,7 @@ func TestEngine_CompactWithSeriesInOneFile(t *testing.T) { } if err := e.Compact(true); err != nil { - t.Fatalf("error compacting: %s", err.Error) + t.Fatalf("error compacting: %s", err.Error()) } tx1, _ := e.Begin(false) defer tx1.Rollback() @@ -813,7 +813,7 @@ func TestEngine_CompactionWithCopiedBlocks(t *testing.T) { verify() if err := e.Compact(true); err != nil { - t.Fatalf("error compacting: %s", err.Error) + t.Fatalf("error compacting: %s", err.Error()) } fmt.Println("verify 2") verify() @@ -824,7 +824,7 @@ func TestEngine_CompactionWithCopiedBlocks(t *testing.T) { } if err := e.Compact(true); err != nil { - t.Fatalf("error compacting: %s", err.Error) + t.Fatalf("error compacting: %s", err.Error()) } fmt.Println("verify 3") verify() @@ -844,7 +844,7 @@ func TestEngine_CompactionWithCopiedBlocks(t *testing.T) { } if err := e.Compact(true); err != nil { - t.Fatalf("error compacting: %s", err.Error) + t.Fatalf("error compacting: %s", err.Error()) } verify() From c0aa5f0f5688fb84a331153d207d66ce6404da40 Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Wed, 30 Sep 2015 20:41:50 -0400 Subject: [PATCH 052/139] Implement reverse cursor direction on pd1 --- tsdb/engine/pd1/cursor.go | 25 +-- tsdb/engine/pd1/pd1.go | 292 ++++++++++++++++++++++++++---------- tsdb/engine/pd1/pd1_test.go | 120 ++++++++++++++- tsdb/engine/pd1/wal.go | 2 +- 4 files changed, 344 insertions(+), 95 deletions(-) diff --git a/tsdb/engine/pd1/cursor.go b/tsdb/engine/pd1/cursor.go index d31252aa32a..8319992c25b 100644 --- a/tsdb/engine/pd1/cursor.go +++ b/tsdb/engine/pd1/cursor.go @@ -39,45 +39,48 @@ func (c *combinedEngineCursor) Ascending() bool { } func (c *combinedEngineCursor) read() (key int64, value interface{}) { - key = tsdb.EOF + if c.walKeyBuf == tsdb.EOF && c.engineKeyBuf == tsdb.EOF { + return tsdb.EOF, nil + } // handle the case where they have the same point - if c.walKeyBuf != tsdb.EOF && c.walKeyBuf == c.engineKeyBuf { + if c.walKeyBuf == c.engineKeyBuf { // keep the wal value since it will overwrite the engine value key = c.walKeyBuf value = c.walValueBuf c.walKeyBuf, c.walValueBuf = c.walCursor.Next() - // drop the engine value - _, _ = c.engineCursor.Next() + + // overwrite the buffered engine values + c.engineKeyBuf, c.engineValueBuf = c.engineCursor.Next() return } // ascending order if c.ascending { - if c.engineKeyBuf == tsdb.EOF || (c.walKeyBuf != tsdb.EOF && c.walKeyBuf < c.engineKeyBuf) { + if c.walKeyBuf != tsdb.EOF && (c.walKeyBuf < c.engineKeyBuf || c.engineKeyBuf == tsdb.EOF) { key = c.walKeyBuf value = c.walValueBuf c.walKeyBuf, c.walValueBuf = c.walCursor.Next() - } else { - key = c.engineKeyBuf - value = c.engineValueBuf - c.engineKeyBuf, c.engineValueBuf = c.engineCursor.Next() + return } + key = c.engineKeyBuf + value = c.engineValueBuf + c.engineKeyBuf, c.engineValueBuf = c.engineCursor.Next() return } // descending order - if c.engineKeyBuf == tsdb.EOF || (c.walKeyBuf != tsdb.EOF && c.walKeyBuf > c.engineKeyBuf) { + if c.walKeyBuf != tsdb.EOF && c.walKeyBuf > c.engineKeyBuf { key = c.walKeyBuf value = c.walValueBuf c.walKeyBuf, c.walValueBuf = c.walCursor.Next() return } + key = c.engineKeyBuf value = c.engineValueBuf c.engineKeyBuf, c.engineValueBuf = c.engineCursor.Next() - return } diff --git a/tsdb/engine/pd1/pd1.go b/tsdb/engine/pd1/pd1.go index 5f64c6c2f5a..44f8325277f 100644 --- a/tsdb/engine/pd1/pd1.go +++ b/tsdb/engine/pd1/pd1.go @@ -1567,6 +1567,8 @@ type cursor struct { ascending bool + blockPositions []uint32 // only used for descending queries + // time acending list of data files files []*dataFile } @@ -1584,15 +1586,32 @@ func (c *cursor) SeekTo(seek int64) (int64, interface{}) { return tsdb.EOF, nil } - if seek < c.files[0].MinTime() { - c.filesPos = 0 - c.f = c.files[0] + if c.ascending { + if seek <= c.files[0].MinTime() { + c.filesPos = 0 + c.f = c.files[0] + } else { + for i, f := range c.files { + if seek >= f.MinTime() && seek <= f.MaxTime() { + c.filesPos = i + c.f = f + break + } + } + } } else { - for i, f := range c.files { - if seek >= f.MinTime() && seek <= f.MaxTime() { - c.filesPos = i - c.f = f - break + if seek >= c.files[len(c.files)-1].MaxTime() { + c.filesPos = len(c.files) - 1 + c.f = c.files[c.filesPos] + } else if seek < c.files[0].MinTime() { + return tsdb.EOF, nil + } else { + for i, f := range c.files { + if seek >= f.MinTime() && seek <= f.MaxTime() { + c.filesPos = i + c.f = f + break + } } } } @@ -1601,110 +1620,227 @@ func (c *cursor) SeekTo(seek int64) (int64, interface{}) { return tsdb.EOF, nil } - // TODO: make this for the reverse direction cursor - - // now find the spot in the file we need to go + // find the first file we need to check in for { - pos := c.f.StartingPositionForID(c.id) + if c.filesPos < 0 || c.filesPos >= len(c.files) { + return tsdb.EOF, nil + } + c.f = c.files[c.filesPos] + + c.pos = c.f.StartingPositionForID(c.id) // if this id isn't in this file, move to next one or return - if pos == 0 { - c.filesPos++ - if c.filesPos >= len(c.files) { - return tsdb.EOF, nil + if c.pos == 0 { + if c.ascending { + c.filesPos++ + } else { + c.filesPos-- + c.blockPositions = nil } - c.f = c.files[c.filesPos] continue } - // seek to the block and values we're looking for - for { - // if the time is between this block and the next, - // decode this block and go, otherwise seek to next block - length := btou32(c.f.mmap[pos+8 : pos+12]) - - // if the next block has a time less than what we're seeking to, - // skip decoding this block and continue on - nextBlockPos := pos + 12 + length - if nextBlockPos < c.f.indexPosition() { - nextBlockID := btou64(c.f.mmap[nextBlockPos : nextBlockPos+8]) - if nextBlockID == c.id { - nextBlockTime := int64(btou64(c.f.mmap[nextBlockPos+12 : nextBlockPos+20])) - if nextBlockTime <= seek { - pos = nextBlockPos - continue - } + // handle seek for correct order + k := tsdb.EOF + var v interface{} + + if c.ascending { + k, v = c.seekAscending(seek) + } else { + k, v = c.seekDescending(seek) + } + + if k != tsdb.EOF { + return k, v + } + + if c.ascending { + c.filesPos++ + } else { + c.filesPos-- + c.blockPositions = nil + } + } +} + +func (c *cursor) seekAscending(seek int64) (int64, interface{}) { + // seek to the block and values we're looking for + for { + // if the time is between this block and the next, + // decode this block and go, otherwise seek to next block + length := c.blockLength(c.pos) + + // if the next block has a time less than what we're seeking to, + // skip decoding this block and continue on + nextBlockPos := c.pos + blockHeaderSize + length + if nextBlockPos < c.f.indexPosition() { + nextBlockID := btou64(c.f.mmap[nextBlockPos : nextBlockPos+8]) + if nextBlockID == c.id { + nextBlockTime := c.blockMinTime(nextBlockPos) + if nextBlockTime <= seek { + c.pos = nextBlockPos + continue } } + } - // it must be in this block or not at all - t, v := c.decodeBlockAndGetValues(pos) - if t >= seek { - return t, v + // it must be in this block or not at all + c.decodeBlock(c.pos) + + // see if we can find it in this block + for i, v := range c.vals { + if v.Time().UnixNano() >= seek { + c.vals = c.vals[i+1:] + return v.Time().UnixNano(), v.Value() } + } + } +} + +func (c *cursor) seekDescending(seek int64) (int64, interface{}) { + c.setBlockPositions() + if len(c.blockPositions) == 0 { + return tsdb.EOF, nil + } + + for i := len(c.blockPositions) - 1; i >= 0; i-- { + pos := c.blockPositions[i] + if c.blockMinTime(pos) > seek { + continue + } + + c.decodeBlock(pos) + c.blockPositions = c.blockPositions[:i] - // wasn't in the first value popped out of the block, check the rest - for i, v := range c.vals { - if v.Time().UnixNano() >= seek { - c.vals = c.vals[i+1:] - return v.Time().UnixNano(), v.Value() + for i := len(c.vals) - 1; i >= 0; i-- { + val := c.vals[i] + if seek >= val.UnixNano() { + c.vals = c.vals[:i] + return val.UnixNano(), val.Value() + } + if seek < val.UnixNano() { + // we need to move to the next block + if i == 0 { + break } + val := c.vals[i-1] + c.vals = c.vals[:i-1] + return val.UnixNano(), val.Value() } + } + c.blockPositions = c.blockPositions[:i] + } - // not in this one, let the top loop look for it in the next file - break + return tsdb.EOF, nil +} + +func (c *cursor) blockMinTime(pos uint32) int64 { + return int64(btou64(c.f.mmap[pos+12 : pos+20])) +} + +func (c *cursor) setBlockPositions() { + pos := c.pos + + for { + if pos >= c.f.indexPosition() { + return + } + + length := c.blockLength(pos) + id := btou64(c.f.mmap[pos : pos+8]) + + if id != c.id { + return } + + c.blockPositions = append(c.blockPositions, pos) + pos += blockHeaderSize + length } } func (c *cursor) Next() (int64, interface{}) { - if len(c.vals) == 0 { - // if we have a file set, see if the next block is for this ID - if c.f != nil && c.pos < c.f.size { - nextBlockID := btou64(c.f.mmap[c.pos : c.pos+8]) - if nextBlockID == c.id && c.pos != c.f.indexPosition() { - return c.decodeBlockAndGetValues(c.pos) - } + if c.ascending { + return c.nextAscending() + } + return c.nextDescending() +} + +func (c *cursor) nextAscending() (int64, interface{}) { + if len(c.vals) > 0 { + v := c.vals[0] + c.vals = c.vals[1:] + + return v.Time().UnixNano(), v.Value() + } + + // if we have a file set, see if the next block is for this ID + if c.f != nil && c.pos < c.f.indexPosition() { + nextBlockID := btou64(c.f.mmap[c.pos : c.pos+8]) + if nextBlockID == c.id { + c.decodeBlock(c.pos) + return c.nextAscending() } + } - // if the file is nil we hit the end of the previous file, advance the file cursor - if c.f != nil { - c.filesPos++ + // loop through the files until we hit the next one that has this id + for { + c.filesPos++ + if c.filesPos >= len(c.files) { + return tsdb.EOF, nil } + c.f = c.files[c.filesPos] - // loop until we find a file with some data - for c.filesPos < len(c.files) { - f := c.files[c.filesPos] + startingPos := c.f.StartingPositionForID(c.id) + if startingPos == 0 { + // move to next file because it isn't in this one + continue + } - startingPos := f.StartingPositionForID(c.id) - if startingPos == 0 { - c.filesPos++ - continue - } - c.f = f - return c.decodeBlockAndGetValues(startingPos) + // we have a block with this id, decode and return + c.decodeBlock(startingPos) + return c.nextAscending() + } +} + +func (c *cursor) nextDescending() (int64, interface{}) { + if len(c.vals) > 0 { + v := c.vals[len(c.vals)-1] + if len(c.vals) >= 1 { + c.vals = c.vals[:len(c.vals)-1] + } else { + c.vals = nil } + return v.UnixNano(), v.Value() + } - // we didn't get to a file that had a next value - return tsdb.EOF, nil + for i := len(c.blockPositions) - 1; i >= 0; i-- { + c.decodeBlock(c.blockPositions[i]) + c.blockPositions = c.blockPositions[:i] + if len(c.vals) == 0 { + continue + } + val := c.vals[len(c.vals)-1] + c.vals = c.vals[:len(c.vals)-1] + return val.UnixNano(), val.Value() } - v := c.vals[0] - c.vals = c.vals[1:] + return tsdb.EOF, nil +} - return v.Time().UnixNano(), v.Value() +func (c *cursor) blockLength(pos uint32) uint32 { + return btou32(c.f.mmap[pos+8 : pos+12]) } -func (c *cursor) decodeBlockAndGetValues(position uint32) (int64, interface{}) { - length := btou32(c.f.mmap[position+8 : position+12]) - block := c.f.mmap[position+12 : position+12+length] +func (c *cursor) decodeBlock(position uint32) { + length := c.blockLength(position) + block := c.f.mmap[position+blockHeaderSize : position+blockHeaderSize+length] c.vals, _ = DecodeBlock(block) - c.pos = position + 12 + length - v := c.vals[0] - c.vals = c.vals[1:] - - return v.Time().UnixNano(), v.Value() + // only adavance the position if we're asceending. + // Descending queries use the blockPositions + if c.ascending { + c.pos = position + blockHeaderSize + length + } } func (c *cursor) Ascending() bool { return c.ascending } diff --git a/tsdb/engine/pd1/pd1_test.go b/tsdb/engine/pd1/pd1_test.go index e11a9baf929..5c28572a846 100644 --- a/tsdb/engine/pd1/pd1_test.go +++ b/tsdb/engine/pd1/pd1_test.go @@ -170,6 +170,7 @@ func TestEngine_WriteIndexQueryAcrossDataFiles(t *testing.T) { points = points[1:] for _, p := range points { + fmt.Println("> ", p.Time()) k, v := c.Next() val := p.Fields()["value"] if p.UnixNano() != k || val != v { @@ -178,9 +179,13 @@ func TestEngine_WriteIndexQueryAcrossDataFiles(t *testing.T) { } } + fmt.Println("v1") verify("cpu,host=A", []models.Point{p1, p7, p3, p5}, 0) + fmt.Println("v2") verify("cpu,host=B", []models.Point{p2, p8, p4, p6}, 0) + fmt.Println("v3") verify("cpu,host=A", []models.Point{p5}, 5000000000) + fmt.Println("v4") verify("cpu,host=B", []models.Point{p6}, 5000000000) } @@ -681,6 +686,81 @@ func TestEngine_WritePointsInMultipleRequestsWithSameTime(t *testing.T) { verify() } +func TestEngine_CursorDescendingOrder(t *testing.T) { + e := OpenDefaultEngine() + defer e.Cleanup() + + fields := []string{"value"} + + p1 := parsePoint("foo value=1 1") + p2 := parsePoint("foo value=2 2") + + e.WAL.SkipCache = false + + if err := e.WritePoints([]models.Point{p1, p2}, nil, nil); err != nil { + t.Fatalf("failed to write points: %s", err.Error()) + } + + verify := func() { + tx, _ := e.Begin(false) + defer tx.Rollback() + c := tx.Cursor("foo", fields, nil, false) + fmt.Println("seek") + k, v := c.SeekTo(5000000) + if k != 2 { + t.Fatalf("expected 2 time but got %d", k) + } + if v != float64(2) { + t.Fatalf("expected 2 for value but got %f", v.(float64)) + } + fmt.Println("next1") + k, v = c.Next() + if k != 1 { + t.Fatalf("expected 1 time but got %d", k) + } + fmt.Println("next2") + if v != float64(1) { + t.Fatalf("expected 1 for value but got %f", v.(float64)) + } + k, _ = c.Next() + if k != tsdb.EOF { + t.Fatal("expected EOF", k) + } + } + fmt.Println("verify 1") + verify() + + if err := e.WAL.Flush(); err != nil { + t.Fatalf("error flushing WAL %s", err.Error) + } + + fmt.Println("verify 2") + verify() + + p3 := parsePoint("foo value=3 3") + + if err := e.WritePoints([]models.Point{p3}, nil, nil); err != nil { + t.Fatalf("failed to write points: %s", err.Error()) + } + + func() { + tx, _ := e.Begin(false) + defer tx.Rollback() + c := tx.Cursor("foo", fields, nil, false) + k, v := c.SeekTo(234232) + if k != 3 { + t.Fatalf("expected 3 time but got %d", k) + } + if v != float64(3) { + t.Fatalf("expected 3 for value but got %f", v.(float64)) + } + k, _ = c.Next() + if k != 2 { + t.Fatalf("expected 2 time but got %d", k) + } + }() +} + func TestEngine_CompactWithSeriesInOneFile(t *testing.T) { e := OpenDefaultEngine() defer e.Cleanup() @@ -711,7 +791,7 @@ func TestEngine_CompactWithSeriesInOneFile(t *testing.T) { verify := func() { tx, _ := e.Begin(false) defer tx.Rollback() - c := tx.Cursor("cpu,host=A", fields, nil, false) + c := tx.Cursor("cpu,host=A", fields, nil, true) k, v := c.SeekTo(0) if k != 1000000000 { t.Fatalf("expected time 1000000000 but got %d", k) @@ -723,7 +803,7 @@ func TestEngine_CompactWithSeriesInOneFile(t *testing.T) { if k != 3000000000 { t.Fatalf("expected time 3000000000 but got %d", k) } - c = tx.Cursor("cpu,host=B", fields, nil, false) + c = tx.Cursor("cpu,host=B", fields, nil, true) k, v = c.SeekTo(0) if k != 2000000000 { t.Fatalf("expected time 2000000000 but got %d", k) @@ -752,7 +832,7 @@ func TestEngine_CompactWithSeriesInOneFile(t *testing.T) { } tx1, _ := e.Begin(false) defer tx1.Rollback() - c := tx1.Cursor("cpu,host=A", fields, nil, false) + c := tx1.Cursor("cpu,host=A", fields, nil, true) k, v := c.SeekTo(0) if k != 1000000000 { t.Fatalf("expected time 1000000000 but got %d", k) @@ -796,7 +876,7 @@ func TestEngine_CompactionWithCopiedBlocks(t *testing.T) { verify := func() { tx, _ := e.Begin(false) defer tx.Rollback() - c := tx.Cursor("cpu,host=A", fields, nil, false) + c := tx.Cursor("cpu,host=A", fields, nil, true) k, _ := c.SeekTo(0) if k != 1000000000 { t.Fatalf("expected time 1000000000 but got %d", k) @@ -875,7 +955,7 @@ func TestEngine_RewritingOldBlocks(t *testing.T) { tx, _ := e.Begin(false) defer tx.Rollback() - c := tx.Cursor("cpu,host=A", fields, nil, false) + c := tx.Cursor("cpu,host=A", fields, nil, true) k, _ := c.SeekTo(0) if k != 1000000000 { t.Fatalf("expected time 1000000000 but got %d", k) @@ -961,6 +1041,36 @@ func TestEngine_WriteIntoCompactedFile(t *testing.T) { } } +func TestEngine_DuplicatePointsInWalAndIndex(t *testing.T) { + e := OpenDefaultEngine() + defer e.Cleanup() + + fields := []string{"value"} + p1 := parsePoint("cpu,host=A value=1.1 1000000000") + p2 := parsePoint("cpu,host=A value=1.2 1000000000") + if err := e.WritePoints([]models.Point{p1}, nil, nil); err != nil { + t.Fatalf("failed to write points: %s", err.Error()) + } + e.WAL.SkipCache = false + if err := e.WritePoints([]models.Point{p2}, nil, nil); err != nil { + t.Fatalf("failed to write points: %s", err.Error()) + } + tx, _ := e.Begin(false) + defer tx.Rollback() + c := tx.Cursor("cpu,host=A", fields, nil, true) + k, v := c.SeekTo(0) + if k != 1000000000 { + t.Fatalf("wrong time: %d", k) + } + if v != 1.2 { + t.Fatalf("wrong value: %f", v.(float64)) + } + k, _ = c.Next() + if k != tsdb.EOF { + t.Fatal("expected EOF", k) + } +} + // Engine represents a test wrapper for pd1.Engine. type Engine struct { *pd1.Engine diff --git a/tsdb/engine/pd1/wal.go b/tsdb/engine/pd1/wal.go index ef4399912cd..b6cc0cc2143 100644 --- a/tsdb/engine/pd1/wal.go +++ b/tsdb/engine/pd1/wal.go @@ -657,7 +657,7 @@ func (c *walCursor) SeekTo(seek int64) (int64, interface{}) { // If seek is not in the cache, return the last value in the cache if !c.ascending && c.position >= len(c.cache) { - c.position = len(c.cache) + c.position = len(c.cache) - 1 } // Make sure our position points to something in the cache From 8fa187ca5087f923fdd6551d935b2978741d155a Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Thu, 1 Oct 2015 15:16:28 -0400 Subject: [PATCH 053/139] Add deletes to new storage engine --- tsdb/engine/pd1/cursor.go | 295 +++++++++++++++++++++ tsdb/engine/pd1/pd1.go | 505 ++++++++++++++---------------------- tsdb/engine/pd1/pd1_test.go | 167 +++++++++++- tsdb/engine/pd1/tx.go | 21 +- tsdb/engine/pd1/wal.go | 31 ++- tsdb/engine/pd1/wal_test.go | 2 + 6 files changed, 711 insertions(+), 310 deletions(-) diff --git a/tsdb/engine/pd1/cursor.go b/tsdb/engine/pd1/cursor.go index 8319992c25b..4b49972131e 100644 --- a/tsdb/engine/pd1/cursor.go +++ b/tsdb/engine/pd1/cursor.go @@ -149,3 +149,298 @@ func (m *multiFieldCursor) read() (int64, interface{}) { } return t, mm } + +type emptyCursor struct { + ascending bool +} + +func (c *emptyCursor) Next() (int64, interface{}) { return tsdb.EOF, nil } +func (c *emptyCursor) SeekTo(key int64) (int64, interface{}) { return tsdb.EOF, nil } +func (c *emptyCursor) Ascending() bool { return c.ascending } + +type cursor struct { + id uint64 + f *dataFile + filesPos int // the index in the files slice we're looking at + pos uint32 + vals Values + + ascending bool + + blockPositions []uint32 // only used for descending queries + + // time acending list of data files + files []*dataFile +} + +func newCursor(id uint64, files []*dataFile, ascending bool) *cursor { + return &cursor{ + id: id, + ascending: ascending, + files: files, + } +} + +func (c *cursor) SeekTo(seek int64) (int64, interface{}) { + if len(c.files) == 0 { + return tsdb.EOF, nil + } + + if c.ascending { + if seek <= c.files[0].MinTime() { + c.filesPos = 0 + c.f = c.files[0] + } else { + for i, f := range c.files { + if seek >= f.MinTime() && seek <= f.MaxTime() { + c.filesPos = i + c.f = f + break + } + } + } + } else { + if seek >= c.files[len(c.files)-1].MaxTime() { + c.filesPos = len(c.files) - 1 + c.f = c.files[c.filesPos] + } else if seek < c.files[0].MinTime() { + return tsdb.EOF, nil + } else { + for i, f := range c.files { + if seek >= f.MinTime() && seek <= f.MaxTime() { + c.filesPos = i + c.f = f + break + } + } + } + } + + if c.f == nil { + return tsdb.EOF, nil + } + + // find the first file we need to check in + for { + if c.filesPos < 0 || c.filesPos >= len(c.files) { + return tsdb.EOF, nil + } + c.f = c.files[c.filesPos] + + c.pos = c.f.StartingPositionForID(c.id) + + // if this id isn't in this file, move to next one or return + if c.pos == 0 { + if c.ascending { + c.filesPos++ + } else { + c.filesPos-- + c.blockPositions = nil + } + continue + } + + // handle seek for correct order + k := tsdb.EOF + var v interface{} + + if c.ascending { + k, v = c.seekAscending(seek) + } else { + k, v = c.seekDescending(seek) + } + + if k != tsdb.EOF { + return k, v + } + + if c.ascending { + c.filesPos++ + } else { + c.filesPos-- + c.blockPositions = nil + } + } +} + +func (c *cursor) seekAscending(seek int64) (int64, interface{}) { + // seek to the block and values we're looking for + for { + // if the time is between this block and the next, + // decode this block and go, otherwise seek to next block + length := c.blockLength(c.pos) + + // if the next block has a time less than what we're seeking to, + // skip decoding this block and continue on + nextBlockPos := c.pos + blockHeaderSize + length + if nextBlockPos < c.f.indexPosition() { + nextBlockID := btou64(c.f.mmap[nextBlockPos : nextBlockPos+8]) + if nextBlockID == c.id { + nextBlockTime := c.blockMinTime(nextBlockPos) + if nextBlockTime <= seek { + c.pos = nextBlockPos + continue + } + } + } + + // it must be in this block or not at all + c.decodeBlock(c.pos) + + // see if we can find it in this block + for i, v := range c.vals { + if v.Time().UnixNano() >= seek { + c.vals = c.vals[i+1:] + return v.Time().UnixNano(), v.Value() + } + } + } +} + +func (c *cursor) seekDescending(seek int64) (int64, interface{}) { + c.setBlockPositions() + if len(c.blockPositions) == 0 { + return tsdb.EOF, nil + } + + for i := len(c.blockPositions) - 1; i >= 0; i-- { + pos := c.blockPositions[i] + if c.blockMinTime(pos) > seek { + continue + } + + c.decodeBlock(pos) + c.blockPositions = c.blockPositions[:i] + + for i := len(c.vals) - 1; i >= 0; i-- { + val := c.vals[i] + if seek >= val.UnixNano() { + c.vals = c.vals[:i] + return val.UnixNano(), val.Value() + } + if seek < val.UnixNano() { + // we need to move to the next block + if i == 0 { + break + } + val := c.vals[i-1] + c.vals = c.vals[:i-1] + return val.UnixNano(), val.Value() + } + } + c.blockPositions = c.blockPositions[:i] + } + + return tsdb.EOF, nil +} + +func (c *cursor) blockMinTime(pos uint32) int64 { + return int64(btou64(c.f.mmap[pos+12 : pos+20])) +} + +func (c *cursor) setBlockPositions() { + pos := c.pos + + for { + if pos >= c.f.indexPosition() { + return + } + + length := c.blockLength(pos) + id := btou64(c.f.mmap[pos : pos+8]) + + if id != c.id { + return + } + + c.blockPositions = append(c.blockPositions, pos) + pos += blockHeaderSize + length + } +} + +func (c *cursor) Next() (int64, interface{}) { + if c.ascending { + return c.nextAscending() + } + return c.nextDescending() +} + +func (c *cursor) nextAscending() (int64, interface{}) { + if len(c.vals) > 0 { + v := c.vals[0] + c.vals = c.vals[1:] + + return v.Time().UnixNano(), v.Value() + } + + // if we have a file set, see if the next block is for this ID + if c.f != nil && c.pos < c.f.indexPosition() { + nextBlockID := btou64(c.f.mmap[c.pos : c.pos+8]) + if nextBlockID == c.id { + c.decodeBlock(c.pos) + return c.nextAscending() + } + } + + // loop through the files until we hit the next one that has this id + for { + c.filesPos++ + if c.filesPos >= len(c.files) { + return tsdb.EOF, nil + } + c.f = c.files[c.filesPos] + + startingPos := c.f.StartingPositionForID(c.id) + if startingPos == 0 { + // move to next file because it isn't in this one + continue + } + + // we have a block with this id, decode and return + c.decodeBlock(startingPos) + return c.nextAscending() + } +} + +func (c *cursor) nextDescending() (int64, interface{}) { + if len(c.vals) > 0 { + v := c.vals[len(c.vals)-1] + if len(c.vals) >= 1 { + c.vals = c.vals[:len(c.vals)-1] + } else { + c.vals = nil + } + return v.UnixNano(), v.Value() + } + + for i := len(c.blockPositions) - 1; i >= 0; i-- { + c.decodeBlock(c.blockPositions[i]) + c.blockPositions = c.blockPositions[:i] + if len(c.vals) == 0 { + continue + } + val := c.vals[len(c.vals)-1] + c.vals = c.vals[:len(c.vals)-1] + return val.UnixNano(), val.Value() + } + + return tsdb.EOF, nil +} + +func (c *cursor) blockLength(pos uint32) uint32 { + return btou32(c.f.mmap[pos+8 : pos+12]) +} + +func (c *cursor) decodeBlock(position uint32) { + length := c.blockLength(position) + block := c.f.mmap[position+blockHeaderSize : position+blockHeaderSize+length] + c.vals, _ = DecodeBlock(block) + + // only adavance the position if we're asceending. + // Descending queries use the blockPositions + if c.ascending { + c.pos = position + blockHeaderSize + length + } +} + +func (c *cursor) Ascending() bool { return c.ascending } diff --git a/tsdb/engine/pd1/pd1.go b/tsdb/engine/pd1/pd1.go index 44f8325277f..c6a8e7c3c5f 100644 --- a/tsdb/engine/pd1/pd1.go +++ b/tsdb/engine/pd1/pd1.go @@ -102,6 +102,10 @@ type Engine struct { compactionRunning bool lastCompactionTime time.Time + // deletes is a map of keys that are deleted, but haven't yet been + // compacted and flushed + deletes map[uint64]bool + collisionsLock sync.RWMutex collisions map[string]uint64 @@ -158,8 +162,9 @@ func (e *Engine) PerformMaintenance() { e.filesLock.RLock() running := e.compactionRunning + deletesPending := len(e.deletes) > 0 e.filesLock.RUnlock() - if running { + if running || deletesPending { return } @@ -214,11 +219,17 @@ func (e *Engine) Open() error { } sort.Sort(e.files) - if err := e.WAL.Open(); err != nil { + if err := e.readCollisions(); err != nil { return err } - if err := e.readCollisions(); err != nil { + e.deletes = make(map[uint64]bool) + + // mark the last compaction as now so it doesn't try to compact while + // flushing the WAL on load + e.lastCompactionTime = time.Now() + + if err := e.WAL.Open(); err != nil { return err } @@ -249,6 +260,7 @@ func (e *Engine) Close() error { e.files = nil e.currentFileID = 0 e.collisions = nil + e.deletes = nil return nil } @@ -307,6 +319,14 @@ func (e *Engine) WritePoints(points []models.Point, measurementFieldsToSave map[ } func (e *Engine) Write(pointsByKey map[string]Values, measurementFieldsToSave map[string]*tsdb.MeasurementFields, seriesToCreate []*tsdb.SeriesCreate) error { + // Flush any deletes before writing new data from the WAL + e.filesLock.RLock() + hasDeletes := len(e.deletes) > 0 + e.filesLock.RUnlock() + if hasDeletes { + e.flushDeletes() + } + err, startTime, endTime, valuesByID := e.convertKeysAndWriteMetadata(pointsByKey, measurementFieldsToSave, seriesToCreate) if err != nil { return err @@ -397,6 +417,16 @@ func (e *Engine) Write(pointsByKey map[string]Values, measurementFieldsToSave ma return nil } +// MarkDeletes will mark the given keys for deletion in memory. They will be deleted from data +// files on the next flush. This mainly for the WAL to use on startup +func (e *Engine) MarkDeletes(keys []string) { + e.filesLock.Lock() + defer e.filesLock.Unlock() + for _, k := range keys { + e.deletes[e.keyToID(k)] = true + } +} + // filesAndLock returns the data files that match the given range and // ensures that the write lock will hold for the entire range func (e *Engine) filesAndLock(min, max int64) (a dataFiles, lockStart, lockEnd int64) { @@ -613,7 +643,7 @@ func (e *Engine) Compact(fullCompaction bool) error { } } - err, newDF := e.writeIndexAndGetDataFile(f, minTime, maxTime, newIDs, newPositions) + newDF, err := e.writeIndexAndGetDataFile(f, minTime, maxTime, newIDs, newPositions) if err != nil { return err } @@ -664,50 +694,51 @@ func (e *Engine) writeBlock(f *os.File, id uint64, block []byte) error { return err } -func (e *Engine) writeIndexAndGetDataFile(f *os.File, minTime, maxTime int64, ids []uint64, newPositions []uint32) (error, *dataFile) { +func (e *Engine) writeIndexAndGetDataFile(f *os.File, minTime, maxTime int64, ids []uint64, newPositions []uint32) (*dataFile, error) { // write the file index, starting with the series ids and their positions for i, id := range ids { if _, err := f.Write(u64tob(id)); err != nil { - return err, nil + return nil, err } if _, err := f.Write(u32tob(newPositions[i])); err != nil { - return err, nil + return nil, err } } // write the min time, max time if _, err := f.Write(append(u64tob(uint64(minTime)), u64tob(uint64(maxTime))...)); err != nil { - return err, nil + return nil, err } // series count if _, err := f.Write(u32tob(uint32(len(ids)))); err != nil { - return err, nil + return nil, err } // sync it and see4k back to the beginning to hand off to the mmap if err := f.Sync(); err != nil { - return err, nil + return nil, err } if _, err := f.Seek(0, 0); err != nil { - return err, nil + return nil, err } // now open it as a memory mapped data file newDF, err := NewDataFile(f) if err != nil { - return err, nil + return nil, err } - return nil, newDF + return newDF, nil } func (e *Engine) shouldCompact() bool { e.filesLock.RLock() running := e.compactionRunning since := time.Since(e.lastCompactionTime) + deletesPending := len(e.deletes) > 0 e.filesLock.RUnlock() - if running || since < e.IndexMinimumCompactionInterval { + if running || since < e.IndexMinimumCompactionInterval || deletesPending { return false } return len(e.filesToCompact()) >= e.CompactionFileCount @@ -1069,7 +1100,7 @@ func (e *Engine) rewriteFile(oldDF *dataFile, valuesByID map[uint64]Values) erro } } - err, newDF := e.writeIndexAndGetDataFile(f, minTime, maxTime, ids, newPositions) + newDF, err := e.writeIndexAndGetDataFile(f, minTime, maxTime, ids, newPositions) if err != nil { f.Close() return err @@ -1103,6 +1134,81 @@ func (e *Engine) rewriteFile(oldDF *dataFile, valuesByID map[uint64]Values) erro return nil } +// flushDeletes will lock the entire shard and rewrite all index files so they no +// longer contain the flushed IDs +func (e *Engine) flushDeletes() error { + e.writeLock.LockRange(math.MinInt64, math.MaxInt64) + defer e.writeLock.UnlockRange(math.MinInt64, math.MaxInt64) + + files := e.copyFilesCollection() + newFiles := make(dataFiles, 0, len(files)) + for _, f := range files { + newFiles = append(newFiles, e.writeNewFileExcludeDeletes(f)) + } + + e.filesLock.Lock() + defer e.filesLock.Unlock() + e.files = newFiles + e.deletes = make(map[uint64]bool) + + e.deletesPending.Add(1) + go func() { + for _, oldDF := range files { + if err := oldDF.Delete(); err != nil { + e.logger.Println("ERROR DELETING FROM REWRITE:", oldDF.f.Name()) + } + } + e.deletesPending.Done() + }() + return nil +} + +func (e *Engine) writeNewFileExcludeDeletes(oldDF *dataFile) *dataFile { + // TODO: add checkpoint file that indicates if this completed or not + f, err := os.OpenFile(e.nextFileName(), os.O_CREATE|os.O_RDWR, 0666) + if err != nil { + panic(fmt.Sprintf("error opening new index file: %s", err.Error())) + } + // write the magic number + if _, err := f.Write(u32tob(magicNumber)); err != nil { + panic(fmt.Sprintf("error writing new index file: %s", err.Error())) + } + + ids := make([]uint64, 0) + positions := make([]uint32, 0) + + indexPosition := oldDF.indexPosition() + currentPosition := uint32(fileHeaderSize) + currentID := uint64(0) + for currentPosition < indexPosition { + id := btou64(oldDF.mmap[currentPosition : currentPosition+8]) + length := btou32(oldDF.mmap[currentPosition+8 : currentPosition+blockHeaderSize]) + newPosition := currentPosition + blockHeaderSize + length + + if _, ok := e.deletes[id]; ok { + currentPosition = newPosition + continue + } + + if _, err := f.Write(oldDF.mmap[currentPosition:newPosition]); err != nil { + panic(fmt.Sprintf("error writing new index file: %s", err.Error())) + } + if id != currentID { + currentID = id + ids = append(ids, id) + positions = append(positions, currentPosition) + } + currentPosition = newPosition + } + + df, err := e.writeIndexAndGetDataFile(f, oldDF.MinTime(), oldDF.MaxTime(), ids, positions) + if err != nil { + panic(fmt.Sprintf("error writing new index file: %s", err.Error())) + } + + return df +} + func (e *Engine) nextFileName() string { e.currentFileID++ return filepath.Join(e.path, fmt.Sprintf("%07d.%s", e.currentFileID, Format)) @@ -1146,14 +1252,86 @@ func (e *Engine) replaceCompressedFile(name string, data []byte) error { return os.Rename(tmpName, filepath.Join(e.path, name)) } +// keysWithFields takes the map of measurements to their fields and a set of series keys +// and returns the columnar keys for the keys and fields +func (e *Engine) keysWithFields(fields map[string]*tsdb.MeasurementFields, keys []string) []string { + e.WAL.cacheLock.RLock() + defer e.WAL.cacheLock.RUnlock() + + a := make([]string, 0) + for _, k := range keys { + measurement := tsdb.MeasurementFromSeriesKey(k) + + // add the fields from the index + mf := fields[measurement] + if mf != nil { + for _, f := range mf.Fields { + a = append(a, seriesFieldKey(k, f.Name)) + } + } + + // now add any fields from the WAL that haven't been flushed yet + mf = e.WAL.measurementFieldsCache[measurement] + if mf != nil { + for _, f := range mf.Fields { + a = append(a, seriesFieldKey(k, f.Name)) + } + } + } + + return a +} + // DeleteSeries deletes the series from the engine. func (e *Engine) DeleteSeries(keys []string) error { + fields, err := e.readFields() + if err != nil { + return err + } + + keyFields := e.keysWithFields(fields, keys) + + return e.deleteKeyFields(keyFields) +} + +func (e *Engine) deleteKeyFields(keyFields []string) error { + err := e.WAL.DeleteSeries(keyFields) + if err != nil { + return err + } + e.filesLock.Lock() + defer e.filesLock.Unlock() + + for _, k := range keyFields { + e.deletes[e.keyToID(k)] = true + } + return nil } // DeleteMeasurement deletes a measurement and all related series. func (e *Engine) DeleteMeasurement(name string, seriesKeys []string) error { - return nil + e.metaLock.Lock() + defer e.metaLock.Unlock() + + // remove the field data from the index + fields, err := e.readFields() + if err != nil { + return err + } + + keyFields := e.keysWithFields(fields, seriesKeys) + + delete(fields, name) + + if err := e.writeFields(fields); err != nil { + return err + } + + e.WAL.DropMeasurementFields(name) + + // now delete all the measurement's series + return e.deleteKeyFields(keyFields) } // SeriesCount returns the number of series buckets on the shard. @@ -1203,9 +1381,8 @@ func (e *Engine) Begin(writable bool) (tsdb.Tx, error) { func (e *Engine) WriteTo(w io.Writer) (n int64, err error) { panic("not implemented") } -func (e *Engine) keyAndFieldToID(series, field string) uint64 { +func (e *Engine) keyToID(key string) uint64 { // get the ID for the key and be sure to check if it had hash collision before - key := seriesFieldKey(series, field) e.collisionsLock.RLock() id, ok := e.collisions[key] e.collisionsLock.RUnlock() @@ -1216,6 +1393,11 @@ func (e *Engine) keyAndFieldToID(series, field string) uint64 { return id } +func (e *Engine) keyAndFieldToID(series, field string) uint64 { + key := seriesFieldKey(series, field) + return e.keyToID(key) +} + func (e *Engine) copyFilesCollection() []*dataFile { e.filesLock.RLock() defer e.filesLock.RUnlock() @@ -1558,293 +1740,6 @@ func (a dataFiles) Len() int { return len(a) } func (a dataFiles) Swap(i, j int) { a[i], a[j] = a[j], a[i] } func (a dataFiles) Less(i, j int) bool { return a[i].MinTime() < a[j].MinTime() } -type cursor struct { - id uint64 - f *dataFile - filesPos int // the index in the files slice we're looking at - pos uint32 - vals Values - - ascending bool - - blockPositions []uint32 // only used for descending queries - - // time acending list of data files - files []*dataFile -} - -func newCursor(id uint64, files []*dataFile, ascending bool) *cursor { - return &cursor{ - id: id, - ascending: ascending, - files: files, - } -} - -func (c *cursor) SeekTo(seek int64) (int64, interface{}) { - if len(c.files) == 0 { - return tsdb.EOF, nil - } - - if c.ascending { - if seek <= c.files[0].MinTime() { - c.filesPos = 0 - c.f = c.files[0] - } else { - for i, f := range c.files { - if seek >= f.MinTime() && seek <= f.MaxTime() { - c.filesPos = i - c.f = f - break - } - } - } - } else { - if seek >= c.files[len(c.files)-1].MaxTime() { - c.filesPos = len(c.files) - 1 - c.f = c.files[c.filesPos] - } else if seek < c.files[0].MinTime() { - return tsdb.EOF, nil - } else { - for i, f := range c.files { - if seek >= f.MinTime() && seek <= f.MaxTime() { - c.filesPos = i - c.f = f - break - } - } - } - } - - if c.f == nil { - return tsdb.EOF, nil - } - - // find the first file we need to check in - for { - if c.filesPos < 0 || c.filesPos >= len(c.files) { - return tsdb.EOF, nil - } - c.f = c.files[c.filesPos] - - c.pos = c.f.StartingPositionForID(c.id) - - // if this id isn't in this file, move to next one or return - if c.pos == 0 { - if c.ascending { - c.filesPos++ - } else { - c.filesPos-- - c.blockPositions = nil - } - continue - } - - // handle seek for correct order - k := tsdb.EOF - var v interface{} - - if c.ascending { - k, v = c.seekAscending(seek) - } else { - k, v = c.seekDescending(seek) - } - - if k != tsdb.EOF { - return k, v - } - - if c.ascending { - c.filesPos++ - } else { - c.filesPos-- - c.blockPositions = nil - } - } -} - -func (c *cursor) seekAscending(seek int64) (int64, interface{}) { - // seek to the block and values we're looking for - for { - // if the time is between this block and the next, - // decode this block and go, otherwise seek to next block - length := c.blockLength(c.pos) - - // if the next block has a time less than what we're seeking to, - // skip decoding this block and continue on - nextBlockPos := c.pos + blockHeaderSize + length - if nextBlockPos < c.f.indexPosition() { - nextBlockID := btou64(c.f.mmap[nextBlockPos : nextBlockPos+8]) - if nextBlockID == c.id { - nextBlockTime := c.blockMinTime(nextBlockPos) - if nextBlockTime <= seek { - c.pos = nextBlockPos - continue - } - } - } - - // it must be in this block or not at all - c.decodeBlock(c.pos) - - // see if we can find it in this block - for i, v := range c.vals { - if v.Time().UnixNano() >= seek { - c.vals = c.vals[i+1:] - return v.Time().UnixNano(), v.Value() - } - } - } -} - -func (c *cursor) seekDescending(seek int64) (int64, interface{}) { - c.setBlockPositions() - if len(c.blockPositions) == 0 { - return tsdb.EOF, nil - } - - for i := len(c.blockPositions) - 1; i >= 0; i-- { - pos := c.blockPositions[i] - if c.blockMinTime(pos) > seek { - continue - } - - c.decodeBlock(pos) - c.blockPositions = c.blockPositions[:i] - - for i := len(c.vals) - 1; i >= 0; i-- { - val := c.vals[i] - if seek >= val.UnixNano() { - c.vals = c.vals[:i] - return val.UnixNano(), val.Value() - } - if seek < val.UnixNano() { - // we need to move to the next block - if i == 0 { - break - } - val := c.vals[i-1] - c.vals = c.vals[:i-1] - return val.UnixNano(), val.Value() - } - } - c.blockPositions = c.blockPositions[:i] - } - - return tsdb.EOF, nil -} - -func (c *cursor) blockMinTime(pos uint32) int64 { - return int64(btou64(c.f.mmap[pos+12 : pos+20])) -} - -func (c *cursor) setBlockPositions() { - pos := c.pos - - for { - if pos >= c.f.indexPosition() { - return - } - - length := c.blockLength(pos) - id := btou64(c.f.mmap[pos : pos+8]) - - if id != c.id { - return - } - - c.blockPositions = append(c.blockPositions, pos) - pos += blockHeaderSize + length - } -} - -func (c *cursor) Next() (int64, interface{}) { - if c.ascending { - return c.nextAscending() - } - return c.nextDescending() -} - -func (c *cursor) nextAscending() (int64, interface{}) { - if len(c.vals) > 0 { - v := c.vals[0] - c.vals = c.vals[1:] - - return v.Time().UnixNano(), v.Value() - } - - // if we have a file set, see if the next block is for this ID - if c.f != nil && c.pos < c.f.indexPosition() { - nextBlockID := btou64(c.f.mmap[c.pos : c.pos+8]) - if nextBlockID == c.id { - c.decodeBlock(c.pos) - return c.nextAscending() - } - } - - // loop through the files until we hit the next one that has this id - for { - c.filesPos++ - if c.filesPos >= len(c.files) { - return tsdb.EOF, nil - } - c.f = c.files[c.filesPos] - - startingPos := c.f.StartingPositionForID(c.id) - if startingPos == 0 { - // move to next file because it isn't in this one - continue - } - - // we have a block with this id, decode and return - c.decodeBlock(startingPos) - return c.nextAscending() - } -} - -func (c *cursor) nextDescending() (int64, interface{}) { - if len(c.vals) > 0 { - v := c.vals[len(c.vals)-1] - if len(c.vals) >= 1 { - c.vals = c.vals[:len(c.vals)-1] - } else { - c.vals = nil - } - return v.UnixNano(), v.Value() - } - - for i := len(c.blockPositions) - 1; i >= 0; i-- { - c.decodeBlock(c.blockPositions[i]) - c.blockPositions = c.blockPositions[:i] - if len(c.vals) == 0 { - continue - } - val := c.vals[len(c.vals)-1] - c.vals = c.vals[:len(c.vals)-1] - return val.UnixNano(), val.Value() - } - - return tsdb.EOF, nil -} - -func (c *cursor) blockLength(pos uint32) uint32 { - return btou32(c.f.mmap[pos+8 : pos+12]) -} - -func (c *cursor) decodeBlock(position uint32) { - length := c.blockLength(position) - block := c.f.mmap[position+blockHeaderSize : position+blockHeaderSize+length] - c.vals, _ = DecodeBlock(block) - - // only adavance the position if we're asceending. - // Descending queries use the blockPositions - if c.ascending { - c.pos = position + blockHeaderSize + length - } -} - -func (c *cursor) Ascending() bool { return c.ascending } - // u64tob converts a uint64 into an 8-byte slice. func u64tob(v uint64) []byte { b := make([]byte, 8) diff --git a/tsdb/engine/pd1/pd1_test.go b/tsdb/engine/pd1/pd1_test.go index 5c28572a846..02f1a576631 100644 --- a/tsdb/engine/pd1/pd1_test.go +++ b/tsdb/engine/pd1/pd1_test.go @@ -10,6 +10,7 @@ import ( "testing" "time" + "github.com/influxdb/influxdb/influxql" "github.com/influxdb/influxdb/models" "github.com/influxdb/influxdb/tsdb" "github.com/influxdb/influxdb/tsdb/engine/pd1" @@ -170,7 +171,6 @@ func TestEngine_WriteIndexQueryAcrossDataFiles(t *testing.T) { points = points[1:] for _, p := range points { - fmt.Println("> ", p.Time()) k, v := c.Next() val := p.Fields()["value"] if p.UnixNano() != k || val != v { @@ -1071,6 +1071,171 @@ func TestEngine_DuplicatePointsInWalAndIndex(t *testing.T) { } } +func TestEngine_Deletes(t *testing.T) { + e := OpenDefaultEngine() + defer e.Cleanup() + + fields := []string{"value"} + // Create metadata. + mf := &tsdb.MeasurementFields{Fields: make(map[string]*tsdb.Field)} + mf.CreateFieldIfNotExists("value", influxql.Float) + atag := map[string]string{"host": "A"} + btag := map[string]string{"host": "B"} + seriesToCreate := []*tsdb.SeriesCreate{ + {Series: tsdb.NewSeries(string(models.MakeKey([]byte("cpu"), atag)), atag)}, + {Series: tsdb.NewSeries(string(models.MakeKey([]byte("cpu"), btag)), btag)}, + } + + p1 := parsePoint("cpu,host=A value=1.1 1000000001") + p2 := parsePoint("cpu,host=A value=1.2 2000000001") + p3 := parsePoint("cpu,host=B value=2.1 1000000000") + p4 := parsePoint("cpu,host=B value=2.1 2000000000") + + e.SkipCompaction = true + e.WAL.SkipCache = false + + if err := e.WritePoints([]models.Point{p1, p3}, map[string]*tsdb.MeasurementFields{"cpu": mf}, seriesToCreate); err != nil { + t.Fatalf("failed to write points: %s", err.Error()) + } + + func() { + tx, _ := e.Begin(false) + defer tx.Rollback() + c := tx.Cursor("cpu,host=A", fields, nil, true) + k, _ := c.SeekTo(0) + if k != p1.UnixNano() { + t.Fatalf("time wrong:\n\texp:%d\n\tgot:%d\n", p1.UnixNano(), k) + } + }() + + if err := e.DeleteSeries([]string{"cpu,host=A"}); err != nil { + t.Fatalf("failed to delete series: %s", err.Error()) + } + + func() { + tx, _ := e.Begin(false) + defer tx.Rollback() + c := tx.Cursor("cpu,host=B", fields, nil, true) + k, _ := c.SeekTo(0) + if k != p3.UnixNano() { + t.Fatalf("time wrong:\n\texp:%d\n\tgot:%d\n", p1.UnixNano(), k) + } + c = tx.Cursor("cpu,host=A", fields, nil, true) + k, _ = c.SeekTo(0) + if k != tsdb.EOF { + t.Fatal("expected EOF", k) + } + }() + + if err := e.WritePoints([]models.Point{p2, p4}, nil, nil); err != nil { + t.Fatalf("failed to write points: %s", err.Error()) + } + + if err := e.WAL.Flush(); err != nil { + t.Fatalf("error flushing wal: %s", err.Error()) + } + + func() { + tx, _ := e.Begin(false) + defer tx.Rollback() + c := tx.Cursor("cpu,host=A", fields, nil, true) + k, _ := c.SeekTo(0) + if k != p2.UnixNano() { + t.Fatalf("time wrong:\n\texp:%d\n\tgot:%d\n", p1.UnixNano(), k) + } + }() + + if err := e.DeleteSeries([]string{"cpu,host=A"}); err != nil { + t.Fatalf("failed to delete series: %s", err.Error()) + } + + // we already know the delete on the wal works. open and close so + // the wal flushes to the index. To verify that the delete gets + // persisted and will go all the way through the index + + if err := e.Close(); err != nil { + t.Fatalf("error closing: %s", err.Error()) + } + if err := e.Open(); err != nil { + t.Fatalf("error opening: %s", err.Error()) + } + + verify := func() { + tx, _ := e.Begin(false) + defer tx.Rollback() + c := tx.Cursor("cpu,host=B", fields, nil, true) + k, _ := c.SeekTo(0) + if k != p3.UnixNano() { + t.Fatalf("time wrong:\n\texp:%d\n\tgot:%d\n", p1.UnixNano(), k) + } + c = tx.Cursor("cpu,host=A", fields, nil, true) + k, _ = c.SeekTo(0) + if k != tsdb.EOF { + t.Fatal("expected EOF") + } + } + + fmt.Println("verify 1") + verify() + + // open and close to verify thd delete was persisted + if err := e.Close(); err != nil { + t.Fatalf("error closing: %s", err.Error()) + } + if err := e.Open(); err != nil { + t.Fatalf("error opening: %s", err.Error()) + } + + fmt.Println("verify 2") + verify() + + if err := e.DeleteSeries([]string{"cpu,host=B"}); err != nil { + t.Fatalf("failed to delete series: %s", err.Error()) + } + + func() { + tx, _ := e.Begin(false) + defer tx.Rollback() + c := tx.Cursor("cpu,host=B", fields, nil, true) + k, _ := c.SeekTo(0) + if k != tsdb.EOF { + t.Fatal("expected EOF") + } + }() + + if err := e.WAL.Flush(); err != nil { + t.Fatalf("error flushing: %s", err.Error()) + } + + func() { + tx, _ := e.Begin(false) + defer tx.Rollback() + c := tx.Cursor("cpu,host=B", fields, nil, true) + k, _ := c.SeekTo(0) + if k != tsdb.EOF { + t.Fatal("expected EOF") + } + }() + + // open and close to verify thd delete was persisted + if err := e.Close(); err != nil { + t.Fatalf("error closing: %s", err.Error()) + } + if err := e.Open(); err != nil { + t.Fatalf("error opening: %s", err.Error()) + } + + func() { + tx, _ := e.Begin(false) + defer tx.Rollback() + c := tx.Cursor("cpu,host=B", fields, nil, true) + k, _ := c.SeekTo(0) + if k != tsdb.EOF { + t.Fatal("expected EOF") + } + }() +} + // Engine represents a test wrapper for pd1.Engine. type Engine struct { *pd1.Engine diff --git a/tsdb/engine/pd1/tx.go b/tsdb/engine/pd1/tx.go index 16aac8b5e43..89d62e9d88a 100644 --- a/tsdb/engine/pd1/tx.go +++ b/tsdb/engine/pd1/tx.go @@ -13,10 +13,20 @@ type tx struct { // TODO: handle multiple fields and descending func (t *tx) Cursor(series string, fields []string, dec *tsdb.FieldCodec, ascending bool) tsdb.Cursor { + t.engine.filesLock.RLock() + defer t.engine.filesLock.RUnlock() + // don't add the overhead of the multifield cursor if we only have one field if len(fields) == 1 { id := t.engine.keyAndFieldToID(series, fields[0]) - indexCursor := newCursor(id, t.files, ascending) + isDeleted := t.engine.deletes[id] + + var indexCursor tsdb.Cursor + if isDeleted { + indexCursor = &emptyCursor{ascending: ascending} + } else { + indexCursor = newCursor(id, t.files, ascending) + } wc := t.engine.WAL.Cursor(series, fields, dec, ascending) return NewCombinedEngineCursor(wc, indexCursor, ascending) } @@ -27,7 +37,14 @@ func (t *tx) Cursor(series string, fields []string, dec *tsdb.FieldCodec, ascend cursorFields := make([]string, 0) for _, field := range fields { id := t.engine.keyAndFieldToID(series, field) - indexCursor := newCursor(id, t.files, ascending) + isDeleted := t.engine.deletes[id] + + var indexCursor tsdb.Cursor + if isDeleted { + indexCursor = &emptyCursor{ascending: ascending} + } else { + indexCursor = newCursor(id, t.files, ascending) + } wc := t.engine.WAL.Cursor(series, []string{field}, dec, ascending) // double up the fields since there's one for the wal and one for the index cursorFields = append(cursorFields, field, field) diff --git a/tsdb/engine/pd1/wal.go b/tsdb/engine/pd1/wal.go index b6cc0cc2143..f253db33f51 100644 --- a/tsdb/engine/pd1/wal.go +++ b/tsdb/engine/pd1/wal.go @@ -57,6 +57,7 @@ const ( pointsEntry walEntryType = 0x01 fieldsEntry walEntryType = 0x02 seriesEntry walEntryType = 0x03 + deleteEntry walEntryType = 0x04 ) type Log struct { @@ -117,6 +118,7 @@ type Log struct { // IndexWriter is an interface for the indexed database the WAL flushes data to type IndexWriter interface { Write(valuesByKey map[string]Values, measurementFieldsToSave map[string]*tsdb.MeasurementFields, seriesToCreate []*tsdb.SeriesCreate) error + MarkDeletes(keys []string) } func NewLog(path string) *Log { @@ -385,6 +387,12 @@ func (l *Log) readFileToCache(fileName string) error { return err } l.addToCache(nil, nil, series, false) + case deleteEntry: + var keys []string + if err := json.Unmarshal(data, &keys); err != nil { + return err + } + l.Index.MarkDeletes(keys) } } } @@ -423,8 +431,27 @@ func (l *Log) Flush() error { return l.flush(idleFlush) } +func (l *Log) DropMeasurementFields(measurement string) { + l.cacheLock.Lock() + defer l.cacheLock.Unlock() + delete(l.measurementFieldsCache, measurement) +} + func (l *Log) DeleteSeries(keys []string) error { - panic("not implemented") + l.cacheLock.Lock() + for _, k := range keys { + delete(l.cache, k) + } + l.cacheLock.Unlock() + + b, err := json.Marshal(keys) + if err != nil { + return err + } + + cb := snappy.Encode(nil, b) + + return l.writeToLog(deleteEntry, cb) } // Close will finish any flush that is currently in process and close file handles @@ -531,7 +558,7 @@ func (l *Log) flush(flush flushType) error { l.cacheLock.Unlock() // exit if there's nothing to flush to the index - if len(l.flushCache) == 0 && len(mfc) == 0 && len(scc) == 0 { + if len(l.flushCache) == 0 && len(mfc) == 0 && len(scc) == 0 && flush != startupFlush { return nil } diff --git a/tsdb/engine/pd1/wal_test.go b/tsdb/engine/pd1/wal_test.go index 034ad2dd3e5..4718f4cebd2 100644 --- a/tsdb/engine/pd1/wal_test.go +++ b/tsdb/engine/pd1/wal_test.go @@ -172,3 +172,5 @@ type MockIndexWriter struct { func (m *MockIndexWriter) Write(valuesByKey map[string]pd1.Values, measurementFieldsToSave map[string]*tsdb.MeasurementFields, seriesToCreate []*tsdb.SeriesCreate) error { return m.fn(valuesByKey, measurementFieldsToSave, seriesToCreate) } + +func (m *MockIndexWriter) MarkDeletes(keys []string) {} From 5fe3c4e5804ae4fb265d3018f61927dd18b03f5d Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Thu, 1 Oct 2015 15:23:38 -0400 Subject: [PATCH 054/139] Rename storage engine to tsm1, for Time Structured Merge Tree! --- tsdb/engine/{pd1 => tsm1}/bool.go | 2 +- tsdb/engine/{pd1 => tsm1}/bool_test.go | 16 ++--- tsdb/engine/{pd1 => tsm1}/cursor.go | 2 +- tsdb/engine/{pd1 => tsm1}/encoding.go | 2 +- tsdb/engine/{pd1 => tsm1}/encoding_test.go | 28 ++++---- tsdb/engine/{pd1 => tsm1}/float.go | 2 +- tsdb/engine/{pd1 => tsm1}/float_test.go | 12 ++-- tsdb/engine/{pd1 => tsm1}/int.go | 2 +- tsdb/engine/{pd1 => tsm1}/int_test.go | 38 +++++------ tsdb/engine/{pd1 => tsm1}/string.go | 2 +- tsdb/engine/{pd1 => tsm1}/string_test.go | 20 +++--- tsdb/engine/{pd1 => tsm1}/timestamp.go | 2 +- tsdb/engine/{pd1 => tsm1}/timestamp_test.go | 68 +++++++++---------- tsdb/engine/{pd1/pd1.go => tsm1/tsm1.go} | 2 +- .../{pd1/pd1_test.go => tsm1/tsm1_test.go} | 12 ++-- tsdb/engine/{pd1 => tsm1}/tx.go | 2 +- tsdb/engine/{pd1 => tsm1}/wal.go | 8 +-- tsdb/engine/{pd1 => tsm1}/wal_test.go | 18 ++--- tsdb/engine/{pd1 => tsm1}/write_lock.go | 2 +- tsdb/engine/{pd1 => tsm1}/write_lock_test.go | 16 ++--- 20 files changed, 128 insertions(+), 128 deletions(-) rename tsdb/engine/{pd1 => tsm1}/bool.go (99%) rename tsdb/engine/{pd1 => tsm1}/bool_test.go (82%) rename tsdb/engine/{pd1 => tsm1}/cursor.go (99%) rename tsdb/engine/{pd1 => tsm1}/encoding.go (99%) rename tsdb/engine/{pd1 => tsm1}/encoding_test.go (83%) rename tsdb/engine/{pd1 => tsm1}/float.go (99%) rename tsdb/engine/{pd1 => tsm1}/float_test.go (93%) rename tsdb/engine/{pd1 => tsm1}/int.go (99%) rename tsdb/engine/{pd1 => tsm1}/int_test.go (87%) rename tsdb/engine/{pd1 => tsm1}/string.go (99%) rename tsdb/engine/{pd1 => tsm1}/string_test.go (76%) rename tsdb/engine/{pd1 => tsm1}/timestamp.go (99%) rename tsdb/engine/{pd1 => tsm1}/timestamp_test.go (84%) rename tsdb/engine/{pd1/pd1.go => tsm1/tsm1.go} (99%) rename tsdb/engine/{pd1/pd1_test.go => tsm1/tsm1_test.go} (99%) rename tsdb/engine/{pd1 => tsm1}/tx.go (99%) rename tsdb/engine/{pd1 => tsm1}/wal.go (98%) rename tsdb/engine/{pd1 => tsm1}/wal_test.go (85%) rename tsdb/engine/{pd1 => tsm1}/write_lock.go (99%) rename tsdb/engine/{pd1 => tsm1}/write_lock_test.go (91%) diff --git a/tsdb/engine/pd1/bool.go b/tsdb/engine/tsm1/bool.go similarity index 99% rename from tsdb/engine/pd1/bool.go rename to tsdb/engine/tsm1/bool.go index 4fad26586bb..8d9653d1991 100644 --- a/tsdb/engine/pd1/bool.go +++ b/tsdb/engine/tsm1/bool.go @@ -1,4 +1,4 @@ -package pd1 +package tsm1 // bool encoding uses 1 bit per value. Each compressed byte slice contains a 1 byte header // indicating the compression type, followed by a variable byte encoded length indicating diff --git a/tsdb/engine/pd1/bool_test.go b/tsdb/engine/tsm1/bool_test.go similarity index 82% rename from tsdb/engine/pd1/bool_test.go rename to tsdb/engine/tsm1/bool_test.go index ed642cff86e..ed68987afd1 100644 --- a/tsdb/engine/pd1/bool_test.go +++ b/tsdb/engine/tsm1/bool_test.go @@ -1,26 +1,26 @@ -package pd1_test +package tsm1_test import ( "testing" - "github.com/influxdb/influxdb/tsdb/engine/pd1" + "github.com/influxdb/influxdb/tsdb/engine/tsm1" ) func Test_BoolEncoder_NoValues(t *testing.T) { - enc := pd1.NewBoolEncoder() + enc := tsm1.NewBoolEncoder() b, err := enc.Bytes() if err != nil { t.Fatalf("unexpected error: %v", err) } - dec := pd1.NewBoolDecoder(b) + dec := tsm1.NewBoolDecoder(b) if dec.Next() { t.Fatalf("unexpected next value: got true, exp false") } } func Test_BoolEncoder_Single(t *testing.T) { - enc := pd1.NewBoolEncoder() + enc := tsm1.NewBoolEncoder() v1 := true enc.Write(v1) b, err := enc.Bytes() @@ -28,7 +28,7 @@ func Test_BoolEncoder_Single(t *testing.T) { t.Fatalf("unexpected error: %v", err) } - dec := pd1.NewBoolDecoder(b) + dec := tsm1.NewBoolDecoder(b) if !dec.Next() { t.Fatalf("unexpected next value: got false, exp true") } @@ -39,7 +39,7 @@ func Test_BoolEncoder_Single(t *testing.T) { } func Test_BoolEncoder_Multi_Compressed(t *testing.T) { - enc := pd1.NewBoolEncoder() + enc := tsm1.NewBoolEncoder() values := make([]bool, 10) for i := range values { @@ -56,7 +56,7 @@ func Test_BoolEncoder_Multi_Compressed(t *testing.T) { t.Fatalf("unexpected length: got %v, exp %v", len(b), exp) } - dec := pd1.NewBoolDecoder(b) + dec := tsm1.NewBoolDecoder(b) for i, v := range values { if !dec.Next() { diff --git a/tsdb/engine/pd1/cursor.go b/tsdb/engine/tsm1/cursor.go similarity index 99% rename from tsdb/engine/pd1/cursor.go rename to tsdb/engine/tsm1/cursor.go index 4b49972131e..06fd0bbf8f8 100644 --- a/tsdb/engine/pd1/cursor.go +++ b/tsdb/engine/tsm1/cursor.go @@ -1,4 +1,4 @@ -package pd1 +package tsm1 import ( "math" diff --git a/tsdb/engine/pd1/encoding.go b/tsdb/engine/tsm1/encoding.go similarity index 99% rename from tsdb/engine/pd1/encoding.go rename to tsdb/engine/tsm1/encoding.go index 005692d2735..4b6a112d568 100644 --- a/tsdb/engine/pd1/encoding.go +++ b/tsdb/engine/tsm1/encoding.go @@ -1,4 +1,4 @@ -package pd1 +package tsm1 import ( "encoding/binary" diff --git a/tsdb/engine/pd1/encoding_test.go b/tsdb/engine/tsm1/encoding_test.go similarity index 83% rename from tsdb/engine/pd1/encoding_test.go rename to tsdb/engine/tsm1/encoding_test.go index 7126eec3887..c4889f18e5d 100644 --- a/tsdb/engine/pd1/encoding_test.go +++ b/tsdb/engine/tsm1/encoding_test.go @@ -1,4 +1,4 @@ -package pd1_test +package tsm1_test import ( // "math/rand" @@ -8,15 +8,15 @@ import ( "testing" "time" - "github.com/influxdb/influxdb/tsdb/engine/pd1" + "github.com/influxdb/influxdb/tsdb/engine/tsm1" ) func TestEncoding_FloatBlock(t *testing.T) { valueCount := 1000 times := getTimes(valueCount, 60, time.Second) - values := make(pd1.Values, len(times)) + values := make(tsm1.Values, len(times)) for i, t := range times { - values[i] = pd1.NewValue(t, float64(i)) + values[i] = tsm1.NewValue(t, float64(i)) } b := values.Encode(nil) @@ -29,9 +29,9 @@ func TestEncoding_FloatBlock(t *testing.T) { } func TestEncoding_FloatBlock_ZeroTime(t *testing.T) { - values := make(pd1.Values, 3) + values := make(tsm1.Values, 3) for i := 0; i < 3; i++ { - values[i] = pd1.NewValue(time.Unix(0, 0), float64(i)) + values[i] = tsm1.NewValue(time.Unix(0, 0), float64(i)) } b := values.Encode(nil) @@ -46,9 +46,9 @@ func TestEncoding_FloatBlock_ZeroTime(t *testing.T) { func TestEncoding_IntBlock_Basic(t *testing.T) { valueCount := 1000 times := getTimes(valueCount, 60, time.Second) - values := make(pd1.Values, len(times)) + values := make(tsm1.Values, len(times)) for i, t := range times { - values[i] = pd1.NewValue(t, int64(i)) + values[i] = tsm1.NewValue(t, int64(i)) } b := values.Encode(nil) @@ -74,13 +74,13 @@ func TestEncoding_IntBlock_Basic(t *testing.T) { func TestEncoding_IntBlock_Negatives(t *testing.T) { valueCount := 1000 times := getTimes(valueCount, 60, time.Second) - values := make(pd1.Values, len(times)) + values := make(tsm1.Values, len(times)) for i, t := range times { v := int64(i) if i%2 == 0 { v = -v } - values[i] = pd1.NewValue(t, int64(v)) + values[i] = tsm1.NewValue(t, int64(v)) } b := values.Encode(nil) @@ -95,13 +95,13 @@ func TestEncoding_IntBlock_Negatives(t *testing.T) { func TestEncoding_BoolBlock_Basic(t *testing.T) { valueCount := 1000 times := getTimes(valueCount, 60, time.Second) - values := make(pd1.Values, len(times)) + values := make(tsm1.Values, len(times)) for i, t := range times { v := true if i%2 == 0 { v = false } - values[i] = pd1.NewValue(t, v) + values[i] = tsm1.NewValue(t, v) } b := values.Encode(nil) @@ -116,9 +116,9 @@ func TestEncoding_BoolBlock_Basic(t *testing.T) { func TestEncoding_StringBlock_Basic(t *testing.T) { valueCount := 1000 times := getTimes(valueCount, 60, time.Second) - values := make(pd1.Values, len(times)) + values := make(tsm1.Values, len(times)) for i, t := range times { - values[i] = pd1.NewValue(t, fmt.Sprintf("value %d", i)) + values[i] = tsm1.NewValue(t, fmt.Sprintf("value %d", i)) } b := values.Encode(nil) diff --git a/tsdb/engine/pd1/float.go b/tsdb/engine/tsm1/float.go similarity index 99% rename from tsdb/engine/pd1/float.go rename to tsdb/engine/tsm1/float.go index dddb9f39b24..ff6a61c505d 100644 --- a/tsdb/engine/pd1/float.go +++ b/tsdb/engine/tsm1/float.go @@ -1,4 +1,4 @@ -package pd1 +package tsm1 /* This code is originally from: https://github.com/dgryski/go-tsz and has been modified to remove diff --git a/tsdb/engine/pd1/float_test.go b/tsdb/engine/tsm1/float_test.go similarity index 93% rename from tsdb/engine/pd1/float_test.go rename to tsdb/engine/tsm1/float_test.go index 60702938f5f..00b259bf95d 100644 --- a/tsdb/engine/pd1/float_test.go +++ b/tsdb/engine/tsm1/float_test.go @@ -1,15 +1,15 @@ -package pd1_test +package tsm1_test import ( "testing" - "github.com/influxdb/influxdb/tsdb/engine/pd1" + "github.com/influxdb/influxdb/tsdb/engine/tsm1" ) func TestFloatEncoder_Simple(t *testing.T) { // Example from the paper - s := pd1.NewFloatEncoder() + s := tsm1.NewFloatEncoder() s.Push(12) s.Push(12) @@ -94,7 +94,7 @@ var TwoHoursData = []struct { func TestFloatEncoder_Roundtrip(t *testing.T) { - s := pd1.NewFloatEncoder() + s := tsm1.NewFloatEncoder() for _, p := range TwoHoursData { s.Push(p.v) } @@ -123,7 +123,7 @@ func TestFloatEncoder_Roundtrip(t *testing.T) { func BenchmarkFloatEncoder(b *testing.B) { for i := 0; i < b.N; i++ { - s := pd1.NewFloatEncoder() + s := tsm1.NewFloatEncoder() for _, tt := range TwoHoursData { s.Push(tt.v) } @@ -132,7 +132,7 @@ func BenchmarkFloatEncoder(b *testing.B) { } func BenchmarkFloatDecoder(b *testing.B) { - s := pd1.NewFloatEncoder() + s := tsm1.NewFloatEncoder() for _, tt := range TwoHoursData { s.Push(tt.v) } diff --git a/tsdb/engine/pd1/int.go b/tsdb/engine/tsm1/int.go similarity index 99% rename from tsdb/engine/pd1/int.go rename to tsdb/engine/tsm1/int.go index 98cf717f303..b178c503751 100644 --- a/tsdb/engine/pd1/int.go +++ b/tsdb/engine/tsm1/int.go @@ -1,4 +1,4 @@ -package pd1 +package tsm1 // Int64 encoding uses two different strategies depending on the range of values in // the uncompressed data. Encoded values are first encoding used zig zag encoding. diff --git a/tsdb/engine/pd1/int_test.go b/tsdb/engine/tsm1/int_test.go similarity index 87% rename from tsdb/engine/pd1/int_test.go rename to tsdb/engine/tsm1/int_test.go index 7f1ba2eff74..279b55e49bf 100644 --- a/tsdb/engine/pd1/int_test.go +++ b/tsdb/engine/tsm1/int_test.go @@ -1,27 +1,27 @@ -package pd1_test +package tsm1_test import ( "math" "testing" - "github.com/influxdb/influxdb/tsdb/engine/pd1" + "github.com/influxdb/influxdb/tsdb/engine/tsm1" ) func Test_Int64Encoder_NoValues(t *testing.T) { - enc := pd1.NewInt64Encoder() + enc := tsm1.NewInt64Encoder() b, err := enc.Bytes() if err != nil { t.Fatalf("unexpected error: %v", err) } - dec := pd1.NewInt64Decoder(b) + dec := tsm1.NewInt64Decoder(b) if dec.Next() { t.Fatalf("unexpected next value: got true, exp false") } } func Test_Int64Encoder_One(t *testing.T) { - enc := pd1.NewInt64Encoder() + enc := tsm1.NewInt64Encoder() v1 := int64(1) enc.Write(1) @@ -30,7 +30,7 @@ func Test_Int64Encoder_One(t *testing.T) { t.Fatalf("unexpected error: %v", err) } - dec := pd1.NewInt64Decoder(b) + dec := tsm1.NewInt64Decoder(b) if !dec.Next() { t.Fatalf("unexpected next value: got true, exp false") } @@ -41,7 +41,7 @@ func Test_Int64Encoder_One(t *testing.T) { } func Test_Int64Encoder_Two(t *testing.T) { - enc := pd1.NewInt64Encoder() + enc := tsm1.NewInt64Encoder() var v1, v2 int64 = 1, 2 enc.Write(v1) @@ -52,7 +52,7 @@ func Test_Int64Encoder_Two(t *testing.T) { t.Fatalf("unexpected error: %v", err) } - dec := pd1.NewInt64Decoder(b) + dec := tsm1.NewInt64Decoder(b) if !dec.Next() { t.Fatalf("unexpected next value: got true, exp false") } @@ -71,7 +71,7 @@ func Test_Int64Encoder_Two(t *testing.T) { } func Test_Int64Encoder_Negative(t *testing.T) { - enc := pd1.NewInt64Encoder() + enc := tsm1.NewInt64Encoder() var v1, v2, v3 int64 = -2, 0, 1 enc.Write(v1) @@ -83,7 +83,7 @@ func Test_Int64Encoder_Negative(t *testing.T) { t.Fatalf("unexpected error: %v", err) } - dec := pd1.NewInt64Decoder(b) + dec := tsm1.NewInt64Decoder(b) if !dec.Next() { t.Fatalf("unexpected next value: got true, exp false") } @@ -110,7 +110,7 @@ func Test_Int64Encoder_Negative(t *testing.T) { } func Test_Int64Encoder_Large_Range(t *testing.T) { - enc := pd1.NewInt64Encoder() + enc := tsm1.NewInt64Encoder() var v1, v2 int64 = math.MinInt64, math.MaxInt64 enc.Write(v1) enc.Write(v2) @@ -119,7 +119,7 @@ func Test_Int64Encoder_Large_Range(t *testing.T) { t.Fatalf("unexpected error: %v", err) } - dec := pd1.NewInt64Decoder(b) + dec := tsm1.NewInt64Decoder(b) if !dec.Next() { t.Fatalf("unexpected next value: got true, exp false") } @@ -138,7 +138,7 @@ func Test_Int64Encoder_Large_Range(t *testing.T) { } func Test_Int64Encoder_Uncompressed(t *testing.T) { - enc := pd1.NewInt64Encoder() + enc := tsm1.NewInt64Encoder() var v1, v2, v3 int64 = 0, 1, 1 << 60 enc.Write(v1) @@ -155,7 +155,7 @@ func Test_Int64Encoder_Uncompressed(t *testing.T) { t.Fatalf("length mismatch: got %v, exp %v", len(b), exp) } - dec := pd1.NewInt64Decoder(b) + dec := tsm1.NewInt64Decoder(b) if !dec.Next() { t.Fatalf("unexpected next value: got true, exp false") } @@ -182,7 +182,7 @@ func Test_Int64Encoder_Uncompressed(t *testing.T) { } func Test_Int64Encoder_AllNegative(t *testing.T) { - enc := pd1.NewInt64Encoder() + enc := tsm1.NewInt64Encoder() values := []int64{ -10, -5, -1, } @@ -196,7 +196,7 @@ func Test_Int64Encoder_AllNegative(t *testing.T) { t.Fatalf("unexpected error: %v", err) } - dec := pd1.NewInt64Decoder(b) + dec := tsm1.NewInt64Decoder(b) i := 0 for dec.Next() { if i > len(values) { @@ -211,7 +211,7 @@ func Test_Int64Encoder_AllNegative(t *testing.T) { } func BenchmarkInt64Encoder(b *testing.B) { - enc := pd1.NewInt64Encoder() + enc := tsm1.NewInt64Encoder() x := make([]int64, 1024) for i := 0; i < len(x); i++ { x[i] = int64(i) @@ -230,7 +230,7 @@ type byteSetter interface { func BenchmarkInt64Decoder(b *testing.B) { x := make([]int64, 1024) - enc := pd1.NewInt64Encoder() + enc := tsm1.NewInt64Encoder() for i := 0; i < len(x); i++ { x[i] = int64(i) enc.Write(x[i]) @@ -239,7 +239,7 @@ func BenchmarkInt64Decoder(b *testing.B) { b.ResetTimer() - dec := pd1.NewInt64Decoder(bytes) + dec := tsm1.NewInt64Decoder(bytes) for i := 0; i < b.N; i++ { dec.(byteSetter).SetBytes(bytes) diff --git a/tsdb/engine/pd1/string.go b/tsdb/engine/tsm1/string.go similarity index 99% rename from tsdb/engine/pd1/string.go rename to tsdb/engine/tsm1/string.go index e548b68c560..1b5dafac3f0 100644 --- a/tsdb/engine/pd1/string.go +++ b/tsdb/engine/tsm1/string.go @@ -1,4 +1,4 @@ -package pd1 +package tsm1 // String encoding uses snappy compression to compress each string. Each string is // appended to byte slice prefixed with a variable byte length followed by the string diff --git a/tsdb/engine/pd1/string_test.go b/tsdb/engine/tsm1/string_test.go similarity index 76% rename from tsdb/engine/pd1/string_test.go rename to tsdb/engine/tsm1/string_test.go index c457de8697d..8710a50b365 100644 --- a/tsdb/engine/pd1/string_test.go +++ b/tsdb/engine/tsm1/string_test.go @@ -1,27 +1,27 @@ -package pd1_test +package tsm1_test import ( "fmt" "testing" - "github.com/influxdb/influxdb/tsdb/engine/pd1" + "github.com/influxdb/influxdb/tsdb/engine/tsm1" ) func Test_StringEncoder_NoValues(t *testing.T) { - enc := pd1.NewStringEncoder() + enc := tsm1.NewStringEncoder() b, err := enc.Bytes() if err != nil { t.Fatalf("unexpected error: %v", err) } - dec := pd1.NewStringDecoder(b) + dec := tsm1.NewStringDecoder(b) if dec.Next() { t.Fatalf("unexpected next value: got true, exp false") } } func Test_StringEncoder_Single(t *testing.T) { - enc := pd1.NewStringEncoder() + enc := tsm1.NewStringEncoder() v1 := "v1" enc.Write(v1) b, err := enc.Bytes() @@ -29,7 +29,7 @@ func Test_StringEncoder_Single(t *testing.T) { t.Fatalf("unexpected error: %v", err) } - dec := pd1.NewStringDecoder(b) + dec := tsm1.NewStringDecoder(b) if !dec.Next() { t.Fatalf("unexpected next value: got false, exp true") } @@ -40,7 +40,7 @@ func Test_StringEncoder_Single(t *testing.T) { } func Test_StringEncoder_Multi_Compressed(t *testing.T) { - enc := pd1.NewStringEncoder() + enc := tsm1.NewStringEncoder() values := make([]string, 10) for i := range values { @@ -53,15 +53,15 @@ func Test_StringEncoder_Multi_Compressed(t *testing.T) { t.Fatalf("unexpected error: %v", err) } - if b[0]>>4 != pd1.EncodingSnappy { - t.Fatalf("unexpected encoding: got %v, exp %v", b[0], pd1.EncodingSnappy) + if b[0]>>4 != tsm1.EncodingSnappy { + t.Fatalf("unexpected encoding: got %v, exp %v", b[0], tsm1.EncodingSnappy) } if exp := 47; len(b) != exp { t.Fatalf("unexpected length: got %v, exp %v", len(b), exp) } - dec := pd1.NewStringDecoder(b) + dec := tsm1.NewStringDecoder(b) for i, v := range values { if !dec.Next() { diff --git a/tsdb/engine/pd1/timestamp.go b/tsdb/engine/tsm1/timestamp.go similarity index 99% rename from tsdb/engine/pd1/timestamp.go rename to tsdb/engine/tsm1/timestamp.go index b6fc6fef411..59990f5cb41 100644 --- a/tsdb/engine/pd1/timestamp.go +++ b/tsdb/engine/tsm1/timestamp.go @@ -1,4 +1,4 @@ -package pd1 +package tsm1 // Timestamp encoding is adapative and based on structure of the timestamps that are encoded. It // uses a combination of delta encoding, scaling and compression using simple8b, run length encoding diff --git a/tsdb/engine/pd1/timestamp_test.go b/tsdb/engine/tsm1/timestamp_test.go similarity index 84% rename from tsdb/engine/pd1/timestamp_test.go rename to tsdb/engine/tsm1/timestamp_test.go index f7030db8a40..dbb5a2341a6 100644 --- a/tsdb/engine/pd1/timestamp_test.go +++ b/tsdb/engine/tsm1/timestamp_test.go @@ -1,14 +1,14 @@ -package pd1_test +package tsm1_test import ( "testing" "time" - "github.com/influxdb/influxdb/tsdb/engine/pd1" + "github.com/influxdb/influxdb/tsdb/engine/tsm1" ) func Test_TimeEncoder(t *testing.T) { - enc := pd1.NewTimeEncoder() + enc := tsm1.NewTimeEncoder() x := []time.Time{} now := time.Unix(0, 0) @@ -24,11 +24,11 @@ func Test_TimeEncoder(t *testing.T) { t.Fatalf("unexpected error: %v", err) } - if got := b[0] >> 4; got != pd1.EncodingPackedSimple { + if got := b[0] >> 4; got != tsm1.EncodingPackedSimple { t.Fatalf("Wrong encoding used: expected uncompressed, got %v", got) } - dec := pd1.NewTimeDecoder(b) + dec := tsm1.NewTimeDecoder(b) for i, v := range x { if !dec.Next() { t.Fatalf("Next == false, expected true") @@ -41,20 +41,20 @@ func Test_TimeEncoder(t *testing.T) { } func Test_TimeEncoder_NoValues(t *testing.T) { - enc := pd1.NewTimeEncoder() + enc := tsm1.NewTimeEncoder() b, err := enc.Bytes() if err != nil { t.Fatalf("unexpected error: %v", err) } - dec := pd1.NewTimeDecoder(b) + dec := tsm1.NewTimeDecoder(b) if dec.Next() { t.Fatalf("unexpected next value: got true, exp false") } } func Test_TimeEncoder_One(t *testing.T) { - enc := pd1.NewTimeEncoder() + enc := tsm1.NewTimeEncoder() tm := time.Unix(0, 0) enc.Write(tm) @@ -63,11 +63,11 @@ func Test_TimeEncoder_One(t *testing.T) { t.Fatalf("unexpected error: %v", err) } - if got := b[0] >> 4; got != pd1.EncodingPackedSimple { + if got := b[0] >> 4; got != tsm1.EncodingPackedSimple { t.Fatalf("Wrong encoding used: expected uncompressed, got %v", got) } - dec := pd1.NewTimeDecoder(b) + dec := tsm1.NewTimeDecoder(b) if !dec.Next() { t.Fatalf("unexpected next value: got true, exp false") } @@ -78,7 +78,7 @@ func Test_TimeEncoder_One(t *testing.T) { } func Test_TimeEncoder_Two(t *testing.T) { - enc := pd1.NewTimeEncoder() + enc := tsm1.NewTimeEncoder() t1 := time.Unix(0, 0) t2 := time.Unix(0, 1) enc.Write(t1) @@ -89,11 +89,11 @@ func Test_TimeEncoder_Two(t *testing.T) { t.Fatalf("unexpected error: %v", err) } - if got := b[0] >> 4; got != pd1.EncodingPackedSimple { + if got := b[0] >> 4; got != tsm1.EncodingPackedSimple { t.Fatalf("Wrong encoding used: expected uncompressed, got %v", got) } - dec := pd1.NewTimeDecoder(b) + dec := tsm1.NewTimeDecoder(b) if !dec.Next() { t.Fatalf("unexpected next value: got true, exp false") } @@ -112,7 +112,7 @@ func Test_TimeEncoder_Two(t *testing.T) { } func Test_TimeEncoder_Three(t *testing.T) { - enc := pd1.NewTimeEncoder() + enc := tsm1.NewTimeEncoder() t1 := time.Unix(0, 0) t2 := time.Unix(0, 1) t3 := time.Unix(0, 2) @@ -126,11 +126,11 @@ func Test_TimeEncoder_Three(t *testing.T) { t.Fatalf("unexpected error: %v", err) } - if got := b[0] >> 4; got != pd1.EncodingPackedSimple { + if got := b[0] >> 4; got != tsm1.EncodingPackedSimple { t.Fatalf("Wrong encoding used: expected uncompressed, got %v", got) } - dec := pd1.NewTimeDecoder(b) + dec := tsm1.NewTimeDecoder(b) if !dec.Next() { t.Fatalf("unexpected next value: got true, exp false") } @@ -157,7 +157,7 @@ func Test_TimeEncoder_Three(t *testing.T) { } func Test_TimeEncoder_Large_Range(t *testing.T) { - enc := pd1.NewTimeEncoder() + enc := tsm1.NewTimeEncoder() t1 := time.Unix(0, 1442369134000000000) t2 := time.Unix(0, 1442369135000000000) enc.Write(t1) @@ -167,11 +167,11 @@ func Test_TimeEncoder_Large_Range(t *testing.T) { t.Fatalf("unexpected error: %v", err) } - if got := b[0] >> 4; got != pd1.EncodingPackedSimple { + if got := b[0] >> 4; got != tsm1.EncodingPackedSimple { t.Fatalf("Wrong encoding used: expected uncompressed, got %v", got) } - dec := pd1.NewTimeDecoder(b) + dec := tsm1.NewTimeDecoder(b) if !dec.Next() { t.Fatalf("unexpected next value: got true, exp false") } @@ -190,7 +190,7 @@ func Test_TimeEncoder_Large_Range(t *testing.T) { } func Test_TimeEncoder_Uncompressed(t *testing.T) { - enc := pd1.NewTimeEncoder() + enc := tsm1.NewTimeEncoder() t1 := time.Unix(0, 0) t2 := time.Unix(1, 0) @@ -210,11 +210,11 @@ func Test_TimeEncoder_Uncompressed(t *testing.T) { t.Fatalf("length mismatch: got %v, exp %v", len(b), exp) } - if got := b[0] >> 4; got != pd1.EncodingUncompressed { + if got := b[0] >> 4; got != tsm1.EncodingUncompressed { t.Fatalf("Wrong encoding used: expected uncompressed, got %v", got) } - dec := pd1.NewTimeDecoder(b) + dec := tsm1.NewTimeDecoder(b) if !dec.Next() { t.Fatalf("unexpected next value: got true, exp false") } @@ -241,7 +241,7 @@ func Test_TimeEncoder_Uncompressed(t *testing.T) { } func Test_TimeEncoder_RLE(t *testing.T) { - enc := pd1.NewTimeEncoder() + enc := tsm1.NewTimeEncoder() var ts []time.Time for i := 0; i < 500; i++ { ts = append(ts, time.Unix(int64(i), 0)) @@ -256,7 +256,7 @@ func Test_TimeEncoder_RLE(t *testing.T) { t.Fatalf("length mismatch: got %v, exp %v", len(b), exp) } - if got := b[0] >> 4; got != pd1.EncodingRLE { + if got := b[0] >> 4; got != tsm1.EncodingRLE { t.Fatalf("Wrong encoding used: expected uncompressed, got %v", got) } @@ -264,7 +264,7 @@ func Test_TimeEncoder_RLE(t *testing.T) { t.Fatalf("unexpected error: %v", err) } - dec := pd1.NewTimeDecoder(b) + dec := tsm1.NewTimeDecoder(b) for i, v := range ts { if !dec.Next() { t.Fatalf("Next == false, expected true") @@ -281,7 +281,7 @@ func Test_TimeEncoder_RLE(t *testing.T) { } func Test_TimeEncoder_Reverse(t *testing.T) { - enc := pd1.NewTimeEncoder() + enc := tsm1.NewTimeEncoder() ts := []time.Time{ time.Unix(0, 3), time.Unix(0, 2), @@ -297,11 +297,11 @@ func Test_TimeEncoder_Reverse(t *testing.T) { t.Fatalf("unexpected error: %v", err) } - if got := b[0] >> 4; got != pd1.EncodingUncompressed { + if got := b[0] >> 4; got != tsm1.EncodingUncompressed { t.Fatalf("Wrong encoding used: expected uncompressed, got %v", got) } - dec := pd1.NewTimeDecoder(b) + dec := tsm1.NewTimeDecoder(b) i := 0 for dec.Next() { if ts[i] != dec.Read() { @@ -312,7 +312,7 @@ func Test_TimeEncoder_Reverse(t *testing.T) { } func Test_TimeEncoder_220SecondDelta(t *testing.T) { - enc := pd1.NewTimeEncoder() + enc := tsm1.NewTimeEncoder() var ts []time.Time now := time.Now() for i := 0; i < 220; i++ { @@ -333,11 +333,11 @@ func Test_TimeEncoder_220SecondDelta(t *testing.T) { t.Fatalf("unexpected length: got %v, exp %v", len(b), exp) } - if got := b[0] >> 4; got != pd1.EncodingRLE { + if got := b[0] >> 4; got != tsm1.EncodingRLE { t.Fatalf("Wrong encoding used: expected uncompressed, got %v", got) } - dec := pd1.NewTimeDecoder(b) + dec := tsm1.NewTimeDecoder(b) i := 0 for dec.Next() { if ts[i] != dec.Read() { @@ -356,7 +356,7 @@ func Test_TimeEncoder_220SecondDelta(t *testing.T) { } func BenchmarkTimeEncoder(b *testing.B) { - enc := pd1.NewTimeEncoder() + enc := tsm1.NewTimeEncoder() x := make([]time.Time, 1024) for i := 0; i < len(x); i++ { x[i] = time.Now() @@ -371,7 +371,7 @@ func BenchmarkTimeEncoder(b *testing.B) { func BenchmarkTimeDecoder(b *testing.B) { x := make([]time.Time, 1024) - enc := pd1.NewTimeEncoder() + enc := tsm1.NewTimeEncoder() for i := 0; i < len(x); i++ { x[i] = time.Now() enc.Write(x[i]) @@ -382,7 +382,7 @@ func BenchmarkTimeDecoder(b *testing.B) { for i := 0; i < b.N; i++ { b.StopTimer() - dec := pd1.NewTimeDecoder(bytes) + dec := tsm1.NewTimeDecoder(bytes) b.StartTimer() for dec.Next() { } diff --git a/tsdb/engine/pd1/pd1.go b/tsdb/engine/tsm1/tsm1.go similarity index 99% rename from tsdb/engine/pd1/pd1.go rename to tsdb/engine/tsm1/tsm1.go index c6a8e7c3c5f..d569d5a1528 100644 --- a/tsdb/engine/pd1/pd1.go +++ b/tsdb/engine/tsm1/tsm1.go @@ -1,4 +1,4 @@ -package pd1 +package tsm1 import ( "encoding/binary" diff --git a/tsdb/engine/pd1/pd1_test.go b/tsdb/engine/tsm1/tsm1_test.go similarity index 99% rename from tsdb/engine/pd1/pd1_test.go rename to tsdb/engine/tsm1/tsm1_test.go index 02f1a576631..e8c3a724672 100644 --- a/tsdb/engine/pd1/pd1_test.go +++ b/tsdb/engine/tsm1/tsm1_test.go @@ -1,4 +1,4 @@ -package pd1_test +package tsm1_test import ( "encoding/binary" @@ -13,7 +13,7 @@ import ( "github.com/influxdb/influxdb/influxql" "github.com/influxdb/influxdb/models" "github.com/influxdb/influxdb/tsdb" - "github.com/influxdb/influxdb/tsdb/engine/pd1" + "github.com/influxdb/influxdb/tsdb/engine/tsm1" ) func TestEngine_WriteAndReadFloats(t *testing.T) { @@ -1236,21 +1236,21 @@ func TestEngine_Deletes(t *testing.T) { }() } -// Engine represents a test wrapper for pd1.Engine. +// Engine represents a test wrapper for tsm1.Engine. type Engine struct { - *pd1.Engine + *tsm1.Engine } // NewEngine returns a new instance of Engine. func NewEngine(opt tsdb.EngineOptions) *Engine { - dir, err := ioutil.TempDir("", "pd1-test") + dir, err := ioutil.TempDir("", "tsm1-test") if err != nil { panic("couldn't get temp dir") } // Create test wrapper and attach mocks. e := &Engine{ - Engine: pd1.NewEngine(dir, dir, opt).(*pd1.Engine), + Engine: tsm1.NewEngine(dir, dir, opt).(*tsm1.Engine), } return e diff --git a/tsdb/engine/pd1/tx.go b/tsdb/engine/tsm1/tx.go similarity index 99% rename from tsdb/engine/pd1/tx.go rename to tsdb/engine/tsm1/tx.go index 89d62e9d88a..d5f31110d18 100644 --- a/tsdb/engine/pd1/tx.go +++ b/tsdb/engine/tsm1/tx.go @@ -1,4 +1,4 @@ -package pd1 +package tsm1 import ( "io" diff --git a/tsdb/engine/pd1/wal.go b/tsdb/engine/tsm1/wal.go similarity index 98% rename from tsdb/engine/pd1/wal.go rename to tsdb/engine/tsm1/wal.go index f253db33f51..9cc2a2fd8fc 100644 --- a/tsdb/engine/pd1/wal.go +++ b/tsdb/engine/tsm1/wal.go @@ -1,4 +1,4 @@ -package pd1 +package tsm1 import ( "encoding/json" @@ -131,7 +131,7 @@ func NewLog(path string) *Log { SegmentSize: DefaultSegmentSize, FlushMemorySizeThreshold: tsdb.DefaultFlushMemorySizeThreshold, MaxMemorySizeThreshold: tsdb.DefaultMaxMemorySizeThreshold, - logger: log.New(os.Stderr, "[pd1wal] ", log.LstdFlags), + logger: log.New(os.Stderr, "[tsm1wal] ", log.LstdFlags), } } @@ -139,8 +139,8 @@ func NewLog(path string) *Log { func (l *Log) Open() error { if l.LoggingEnabled { - l.logger.Printf("PD1 WAL starting with %d flush memory size threshold and %d max memory size threshold\n", l.FlushMemorySizeThreshold, l.MaxMemorySizeThreshold) - l.logger.Printf("PD1 WAL writing to %s\n", l.path) + l.logger.Printf("tsm1 WAL starting with %d flush memory size threshold and %d max memory size threshold\n", l.FlushMemorySizeThreshold, l.MaxMemorySizeThreshold) + l.logger.Printf("tsm1 WAL writing to %s\n", l.path) } if err := os.MkdirAll(l.path, 0777); err != nil { return err diff --git a/tsdb/engine/pd1/wal_test.go b/tsdb/engine/tsm1/wal_test.go similarity index 85% rename from tsdb/engine/pd1/wal_test.go rename to tsdb/engine/tsm1/wal_test.go index 4718f4cebd2..dfc5cda8cc8 100644 --- a/tsdb/engine/pd1/wal_test.go +++ b/tsdb/engine/tsm1/wal_test.go @@ -1,4 +1,4 @@ -package pd1_test +package tsm1_test import ( "io/ioutil" @@ -8,19 +8,19 @@ import ( "github.com/influxdb/influxdb/models" "github.com/influxdb/influxdb/tsdb" - "github.com/influxdb/influxdb/tsdb/engine/pd1" + "github.com/influxdb/influxdb/tsdb/engine/tsm1" ) func TestWAL_TestWriteQueryOpen(t *testing.T) { w := NewWAL() defer w.Cleanup() - var vals map[string]pd1.Values + var vals map[string]tsm1.Values var fields map[string]*tsdb.MeasurementFields var series []*tsdb.SeriesCreate w.Index = &MockIndexWriter{ - fn: func(valuesByKey map[string]pd1.Values, measurementFieldsToSave map[string]*tsdb.MeasurementFields, seriesToCreate []*tsdb.SeriesCreate) error { + fn: func(valuesByKey map[string]tsm1.Values, measurementFieldsToSave map[string]*tsdb.MeasurementFields, seriesToCreate []*tsdb.SeriesCreate) error { vals = valuesByKey fields = measurementFieldsToSave series = seriesToCreate @@ -141,18 +141,18 @@ func TestWAL_TestWriteQueryOpen(t *testing.T) { } type Log struct { - *pd1.Log + *tsm1.Log path string } func NewWAL() *Log { - dir, err := ioutil.TempDir("", "pd1-test") + dir, err := ioutil.TempDir("", "tsm1-test") if err != nil { panic("couldn't get temp dir") } l := &Log{ - Log: pd1.NewLog(dir), + Log: tsm1.NewLog(dir), path: dir, } l.LoggingEnabled = true @@ -166,10 +166,10 @@ func (l *Log) Cleanup() error { } type MockIndexWriter struct { - fn func(valuesByKey map[string]pd1.Values, measurementFieldsToSave map[string]*tsdb.MeasurementFields, seriesToCreate []*tsdb.SeriesCreate) error + fn func(valuesByKey map[string]tsm1.Values, measurementFieldsToSave map[string]*tsdb.MeasurementFields, seriesToCreate []*tsdb.SeriesCreate) error } -func (m *MockIndexWriter) Write(valuesByKey map[string]pd1.Values, measurementFieldsToSave map[string]*tsdb.MeasurementFields, seriesToCreate []*tsdb.SeriesCreate) error { +func (m *MockIndexWriter) Write(valuesByKey map[string]tsm1.Values, measurementFieldsToSave map[string]*tsdb.MeasurementFields, seriesToCreate []*tsdb.SeriesCreate) error { return m.fn(valuesByKey, measurementFieldsToSave, seriesToCreate) } diff --git a/tsdb/engine/pd1/write_lock.go b/tsdb/engine/tsm1/write_lock.go similarity index 99% rename from tsdb/engine/pd1/write_lock.go rename to tsdb/engine/tsm1/write_lock.go index a791b663abb..f4514e58aaa 100644 --- a/tsdb/engine/pd1/write_lock.go +++ b/tsdb/engine/tsm1/write_lock.go @@ -1,4 +1,4 @@ -package pd1 +package tsm1 import ( "reflect" diff --git a/tsdb/engine/pd1/write_lock_test.go b/tsdb/engine/tsm1/write_lock_test.go similarity index 91% rename from tsdb/engine/pd1/write_lock_test.go rename to tsdb/engine/tsm1/write_lock_test.go index 4cdc88e816e..7fa17c530c5 100644 --- a/tsdb/engine/pd1/write_lock_test.go +++ b/tsdb/engine/tsm1/write_lock_test.go @@ -1,15 +1,15 @@ -package pd1_test +package tsm1_test import ( // "sync" "testing" "time" - "github.com/influxdb/influxdb/tsdb/engine/pd1" + "github.com/influxdb/influxdb/tsdb/engine/tsm1" ) func TestWriteLock_FullCover(t *testing.T) { - w := &pd1.WriteLock{} + w := &tsm1.WriteLock{} w.LockRange(2, 10) lock := make(chan bool) @@ -27,7 +27,7 @@ func TestWriteLock_FullCover(t *testing.T) { } func TestWriteLock_RightIntersect(t *testing.T) { - w := &pd1.WriteLock{} + w := &tsm1.WriteLock{} w.LockRange(2, 10) lock := make(chan bool) @@ -45,7 +45,7 @@ func TestWriteLock_RightIntersect(t *testing.T) { } func TestWriteLock_LeftIntersect(t *testing.T) { - w := &pd1.WriteLock{} + w := &tsm1.WriteLock{} w.LockRange(1, 4) lock := make(chan bool) @@ -63,7 +63,7 @@ func TestWriteLock_LeftIntersect(t *testing.T) { } func TestWriteLock_Inside(t *testing.T) { - w := &pd1.WriteLock{} + w := &tsm1.WriteLock{} w.LockRange(4, 8) lock := make(chan bool) @@ -81,7 +81,7 @@ func TestWriteLock_Inside(t *testing.T) { } func TestWriteLock_Same(t *testing.T) { - w := &pd1.WriteLock{} + w := &tsm1.WriteLock{} w.LockRange(2, 10) lock := make(chan bool) @@ -99,7 +99,7 @@ func TestWriteLock_Same(t *testing.T) { } // func TestWriteLock_FreeRangeWithContentionElsewhere(t *testing.T) { -// w := &pd1.WriteLock{} +// w := &tsm1.WriteLock{} // w.LockRange(2, 10) // lock := make(chan bool) From 316f74ce75f177bcf0de486f73d02fa4d840d2c9 Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Thu, 1 Oct 2015 15:30:13 -0400 Subject: [PATCH 055/139] Cleanup after pd1 -> tsm1 name change. --- tsdb/config.go | 8 ++++---- tsdb/engine.go | 6 +++--- tsdb/engine/engine.go | 2 +- tsdb/engine/tsm1/tsm1.go | 6 +++--- tsdb/shard.go | 2 +- 5 files changed, 12 insertions(+), 12 deletions(-) diff --git a/tsdb/config.go b/tsdb/config.go index 1e7e29a0d86..a329c1268bb 100644 --- a/tsdb/config.go +++ b/tsdb/config.go @@ -8,7 +8,7 @@ import ( const ( // DefaultEngine is the default engine for new shards - DefaultEngine = "pd1" + DefaultEngine = "tsm1" // DefaultMaxWALSize is the default size of the WAL before it is flushed. DefaultMaxWALSize = 100 * 1024 * 1024 // 100MB @@ -44,7 +44,7 @@ const ( // size for the in-memory WAL cache. DefaultPartitionSizeThreshold = 50 * 1024 * 1024 // 50MB - // Default WAL settings for the PD1 WAL + // Default WAL settings for the TSM1 WAL DefaultFlushMemorySizeThreshold = 5 * 1024 * 1024 // 5MB DefaultMaxMemorySizeThreshold = 100 * 1024 * 1024 // 100MB DefaultIndexCompactionAge = time.Minute @@ -71,11 +71,11 @@ type Config struct { WALFlushColdInterval toml.Duration `toml:"wal-flush-cold-interval"` WALPartitionSizeThreshold uint64 `toml:"wal-partition-size-threshold"` - // WAL configuration options for pd1 introduced in 0.9.5 + // WAL configuration options for tsm1 introduced in 0.9.5 WALFlushMemorySizeThreshold int `toml:"wal-flush-memory-size-threshold"` WALMaxMemorySizeThreshold int `toml:"wal-max-memory-size-threshold"` - // compaction options for pd1 introduced in 0.9.5 + // compaction options for tsm1 introduced in 0.9.5 // IndexCompactionAge specifies the duration after the data file creation time // at which it is eligible to be compacted diff --git a/tsdb/engine.go b/tsdb/engine.go index 407801d842e..97ca51d9ddb 100644 --- a/tsdb/engine.go +++ b/tsdb/engine.go @@ -46,7 +46,7 @@ type EngineFormat int const ( B1Format EngineFormat = iota BZ1Format - PD1Format + TSM1Format ) // NewEngineFunc creates a new engine. @@ -74,7 +74,7 @@ func NewEngine(path string, walPath string, options EngineOptions) (Engine, erro // Only bolt-based backends are currently supported so open it and check the format. var format string if err := func() error { - // if it's a dir then it's a pd1 engine + // if it's a dir then it's a tsm1 engine f, err := os.Open(path) if err != nil { return err @@ -85,7 +85,7 @@ func NewEngine(path string, walPath string, options EngineOptions) (Engine, erro return err } if fi.Mode().IsDir() { - format = "pd1" + format = "tsm1" return nil } diff --git a/tsdb/engine/engine.go b/tsdb/engine/engine.go index 03022f9d0e7..6c8cb51e193 100644 --- a/tsdb/engine/engine.go +++ b/tsdb/engine/engine.go @@ -3,5 +3,5 @@ package engine import ( _ "github.com/influxdb/influxdb/tsdb/engine/b1" _ "github.com/influxdb/influxdb/tsdb/engine/bz1" - _ "github.com/influxdb/influxdb/tsdb/engine/pd1" + _ "github.com/influxdb/influxdb/tsdb/engine/tsm1" ) diff --git a/tsdb/engine/tsm1/tsm1.go b/tsdb/engine/tsm1/tsm1.go index d569d5a1528..b47413868d8 100644 --- a/tsdb/engine/tsm1/tsm1.go +++ b/tsdb/engine/tsm1/tsm1.go @@ -24,7 +24,7 @@ import ( const ( // Format is the file format name of this engine. - Format = "pd1" + Format = "tsm1" // FieldsFileExtension is the extension for the file that stores compressed field // encoding data for this db @@ -125,7 +125,7 @@ func NewEngine(path string, walPath string, opt tsdb.EngineOptions) tsdb.Engine e := &Engine{ path: path, writeLock: &WriteLock{}, - logger: log.New(os.Stderr, "[pd1] ", log.LstdFlags), + logger: log.New(os.Stderr, "[tsm1] ", log.LstdFlags), // TODO: this is the function where we can inject a check against the in memory collisions HashSeriesField: hashSeriesField, @@ -180,7 +180,7 @@ func (e *Engine) PerformMaintenance() { // Format returns the format type of this engine func (e *Engine) Format() tsdb.EngineFormat { - return tsdb.PD1Format + return tsdb.TSM1Format } // Open opens and initializes the engine. diff --git a/tsdb/shard.go b/tsdb/shard.go index 1f606b613b3..b2589443163 100644 --- a/tsdb/shard.go +++ b/tsdb/shard.go @@ -234,7 +234,7 @@ func (s *Shard) WritePoints(points []models.Point) error { // make sure all data is encoded before attempting to save to bolt // only required for the b1 and bz1 formats - if s.engine.Format() != PD1Format { + if s.engine.Format() != TSM1Format { for _, p := range points { // Ignore if raw data has already been marshaled. if p.Data() != nil { From c8d8ebcf41b5e01b6a17a19d886007bcd53ab986 Mon Sep 17 00:00:00 2001 From: Jason Wilder Date: Thu, 1 Oct 2015 13:59:27 -0600 Subject: [PATCH 056/139] Fix TestStoreOpenShardCreateDelete Shard path can be a directory. --- tsdb/store.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tsdb/store.go b/tsdb/store.go index 1fb8560167e..bee68c7fddf 100644 --- a/tsdb/store.go +++ b/tsdb/store.go @@ -131,7 +131,7 @@ func (s *Store) DeleteShard(shardID uint64) error { return err } - if err := os.Remove(sh.path); err != nil { + if err := os.RemoveAll(sh.path); err != nil { return err } From 5800bdec531ecb60196fe298945cec9c1c62ce74 Mon Sep 17 00:00:00 2001 From: Jason Wilder Date: Fri, 2 Oct 2015 09:38:39 -0600 Subject: [PATCH 057/139] Disable copier test Not implemented for tsm1 engine --- services/copier/service_test.go | 1 + 1 file changed, 1 insertion(+) diff --git a/services/copier/service_test.go b/services/copier/service_test.go index a5266087d7f..ce1151d3cf8 100644 --- a/services/copier/service_test.go +++ b/services/copier/service_test.go @@ -19,6 +19,7 @@ import ( // Ensure the service can return shard data. func TestService_handleConn(t *testing.T) { + t.Skip("not implemented for tsm1 engine") s := MustOpenService() defer s.Close() From 1a174de9e5b497ff4c23e539c650f70abd06f86a Mon Sep 17 00:00:00 2001 From: Jason Wilder Date: Fri, 2 Oct 2015 09:39:08 -0600 Subject: [PATCH 058/139] Fix go vet errors --- tsdb/engine/tsm1/tsm1_test.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tsdb/engine/tsm1/tsm1_test.go b/tsdb/engine/tsm1/tsm1_test.go index e8c3a724672..6a39e41ed8c 100644 --- a/tsdb/engine/tsm1/tsm1_test.go +++ b/tsdb/engine/tsm1/tsm1_test.go @@ -731,7 +731,7 @@ func TestEngine_CursorDescendingOrder(t *testing.T) { verify() if err := e.WAL.Flush(); err != nil { - t.Fatalf("error flushing WAL %s", err.Error) + t.Fatalf("error flushing WAL %s", err.Error()) } fmt.Println("verify 2") @@ -997,7 +997,7 @@ func TestEngine_WriteIntoCompactedFile(t *testing.T) { } if err := e.Compact(true); err != nil { - t.Fatalf("error compacting: %s", err.Error) + t.Fatalf("error compacting: %s", err.Error()) } if err := e.WritePoints([]models.Point{p4}, nil, nil); err != nil { @@ -1005,7 +1005,7 @@ func TestEngine_WriteIntoCompactedFile(t *testing.T) { } if err := e.Compact(true); err != nil { - t.Fatalf("error compacting: %s", err.Error) + t.Fatalf("error compacting: %s", err.Error()) } if err := e.WritePoints([]models.Point{p5}, nil, nil); err != nil { From c2e89225331a0cffe55b38f2f3ba595071082ec4 Mon Sep 17 00:00:00 2001 From: Jason Wilder Date: Fri, 2 Oct 2015 10:03:20 -0600 Subject: [PATCH 059/139] Move compression encoding constants to encoders Will make it less error-prone to add new encodings int the future since each encoder has it's set of constants. There are some placeholder contants for uncompressed encodings which are not in all encoder currently. --- tsdb/engine/tsm1/bool.go | 9 +++- tsdb/engine/tsm1/encoding.go | 15 ------- tsdb/engine/tsm1/float.go | 18 +++++--- tsdb/engine/tsm1/float_test.go | 22 ++++++++-- tsdb/engine/tsm1/int.go | 15 +++++-- tsdb/engine/tsm1/string.go | 9 +++- tsdb/engine/tsm1/string_test.go | 20 ++++----- tsdb/engine/tsm1/timestamp.go | 21 ++++++--- tsdb/engine/tsm1/timestamp_test.go | 68 +++++++++++++++--------------- 9 files changed, 114 insertions(+), 83 deletions(-) diff --git a/tsdb/engine/tsm1/bool.go b/tsdb/engine/tsm1/bool.go index 8d9653d1991..ac08555cfe5 100644 --- a/tsdb/engine/tsm1/bool.go +++ b/tsdb/engine/tsm1/bool.go @@ -7,6 +7,13 @@ package tsm1 import "encoding/binary" +const ( + // boolUncompressed is an uncompressed boolean format + boolUncompressed = 0 + // boolCompressedBitPacked is an bit packed format using 1 bit per boolean + boolCompressedBitPacked = 1 +) + type BoolEncoder interface { Write(b bool) Bytes() ([]byte, error) @@ -75,7 +82,7 @@ func (e *boolEncoder) Bytes() ([]byte, error) { b := make([]byte, 10+1) // Store the encoding type in the 4 high bits of the first byte - b[0] = byte(EncodingBitPacked) << 4 + b[0] = byte(boolCompressedBitPacked) << 4 i := 1 // Encode the number of bools written diff --git a/tsdb/engine/tsm1/encoding.go b/tsdb/engine/tsm1/encoding.go index 4b6a112d568..eef2b1d23e7 100644 --- a/tsdb/engine/tsm1/encoding.go +++ b/tsdb/engine/tsm1/encoding.go @@ -10,21 +10,6 @@ import ( ) const ( - // EncodingPackedSimple is a bit-packed format - EncodingPackedSimple = 0 - - // EncodingRLE is a run-length encoded format - EncodingRLE = 1 - - // EncodingUncompressed is a non-compressed format - EncodingUncompressed = 2 - - // EncodingBitPacked is a basic bit-packed format - EncodingBitPacked = 3 - - // EncodingSnappy is a snappy encoded format - EncodingSnappy = 4 - // BlockFloat64 designates a block encodes float64 values BlockFloat64 = 0 diff --git a/tsdb/engine/tsm1/float.go b/tsdb/engine/tsm1/float.go index ff6a61c505d..7ac1d93b492 100644 --- a/tsdb/engine/tsm1/float.go +++ b/tsdb/engine/tsm1/float.go @@ -17,6 +17,13 @@ import ( "github.com/dgryski/go-bitstream" ) +const ( + // floatUncompressed is an uncompressed format using 8 bytes per value + floatUncompressed = 0 + // floatCompressedGorilla is a compressed format using the gorilla paper encoding + floatCompressedGorilla = 1 +) + type FloatEncoder struct { val float64 @@ -43,7 +50,7 @@ func NewFloatEncoder() *FloatEncoder { } func (s *FloatEncoder) Bytes() []byte { - return s.buf.Bytes() + return append([]byte{floatCompressedGorilla << 4}, s.buf.Bytes()...) } func (s *FloatEncoder) Finish() { @@ -95,11 +102,6 @@ func (s *FloatEncoder) Push(v float64) { s.val = v } -func (s *FloatEncoder) FloatDecoder() *FloatDecoder { - iter, _ := NewFloatDecoder(s.buf.Bytes()) - return iter -} - type FloatDecoder struct { val float64 @@ -117,7 +119,9 @@ type FloatDecoder struct { } func NewFloatDecoder(b []byte) (*FloatDecoder, error) { - br := bitstream.NewReader(bytes.NewReader(b)) + // first byte is the compression type but we currently just have gorilla + // compression + br := bitstream.NewReader(bytes.NewReader(b[1:])) v, err := br.ReadBits(64) if err != nil { diff --git a/tsdb/engine/tsm1/float_test.go b/tsdb/engine/tsm1/float_test.go index 00b259bf95d..49e811a174e 100644 --- a/tsdb/engine/tsm1/float_test.go +++ b/tsdb/engine/tsm1/float_test.go @@ -28,7 +28,12 @@ func TestFloatEncoder_Simple(t *testing.T) { s.Finish() - it := s.FloatDecoder() + b := s.Bytes() + + it, err := tsm1.NewFloatDecoder(b) + if err != nil { + t.Fatalf("unexpected error creating float decoder: %v", err) + } want := []float64{ 12, @@ -100,7 +105,13 @@ func TestFloatEncoder_Roundtrip(t *testing.T) { } s.Finish() - it := s.FloatDecoder() + b := s.Bytes() + + it, err := tsm1.NewFloatDecoder(b) + if err != nil { + t.Fatalf("unexpected error creating float decoder: %v", err) + } + for _, w := range TwoHoursData { if !it.Next() { t.Fatalf("Next()=false, want true") @@ -137,11 +148,16 @@ func BenchmarkFloatDecoder(b *testing.B) { s.Push(tt.v) } s.Finish() + bytes := s.Bytes() b.ResetTimer() for i := 0; i < b.N; i++ { - it := s.FloatDecoder() + it, err := tsm1.NewFloatDecoder(bytes) + if err != nil { + b.Fatalf("unexpected error creating float decoder: %v", err) + } + for j := 0; j < len(TwoHoursData); it.Next() { j++ } diff --git a/tsdb/engine/tsm1/int.go b/tsdb/engine/tsm1/int.go index b178c503751..40e615657dd 100644 --- a/tsdb/engine/tsm1/int.go +++ b/tsdb/engine/tsm1/int.go @@ -27,6 +27,13 @@ import ( "github.com/jwilder/encoding/simple8b" ) +const ( + // intUncompressed is an uncompressed format using 8 bytes per point + intUncompressed = 0 + // intCompressedSimple is a bit-packed format using simple8b encoding + intCompressedSimple = 1 +) + type Int64Encoder interface { Write(v int64) Bytes() ([]byte, error) @@ -68,7 +75,7 @@ func (e *int64Encoder) encodePacked() ([]byte, error) { b := make([]byte, 1+len(encoded)*8) // 4 high bits of first byte store the encoding type for the block - b[0] = byte(EncodingPackedSimple) << 4 + b[0] = byte(intCompressedSimple) << 4 for i, v := range encoded { binary.BigEndian.PutUint64(b[1+i*8:1+i*8+8], v) @@ -79,7 +86,7 @@ func (e *int64Encoder) encodePacked() ([]byte, error) { func (e *int64Encoder) encodeUncompressed() ([]byte, error) { b := make([]byte, 1+len(e.values)*8) // 4 high bits of first byte store the encoding type for the block - b[0] = byte(EncodingUncompressed) << 4 + b[0] = byte(intUncompressed) << 4 for i, v := range e.values { binary.BigEndian.PutUint64(b[1+i*8:1+i*8+8], v) @@ -123,9 +130,9 @@ func (d *int64Decoder) Next() bool { if d.i >= d.n { switch d.encoding { - case EncodingUncompressed: + case intUncompressed: d.decodeUncompressed() - case EncodingPackedSimple: + case intCompressedSimple: d.decodePacked() default: panic(fmt.Sprintf("unknown encoding %v", d.encoding)) diff --git a/tsdb/engine/tsm1/string.go b/tsdb/engine/tsm1/string.go index 1b5dafac3f0..a2025809de9 100644 --- a/tsdb/engine/tsm1/string.go +++ b/tsdb/engine/tsm1/string.go @@ -12,6 +12,13 @@ import ( "github.com/golang/snappy" ) +const ( + // stringUncompressed is a an uncompressed format encoding strings as raw bytes + stringUncompressed = 0 + // stringCompressedSnappy is a compressed encoding using Snappy compression + stringCompressedSnappy = 1 +) + type StringEncoder interface { Write(s string) Bytes() ([]byte, error) @@ -45,7 +52,7 @@ func (e *stringEncoder) Bytes() ([]byte, error) { // Compress the currently appended bytes using snappy and prefix with // a 1 byte header for future extension data := snappy.Encode(nil, e.bytes) - return append([]byte{EncodingSnappy << 4}, data...), nil + return append([]byte{stringCompressedSnappy << 4}, data...), nil } type stringDecoder struct { diff --git a/tsdb/engine/tsm1/string_test.go b/tsdb/engine/tsm1/string_test.go index 8710a50b365..f1bf1738152 100644 --- a/tsdb/engine/tsm1/string_test.go +++ b/tsdb/engine/tsm1/string_test.go @@ -1,27 +1,25 @@ -package tsm1_test +package tsm1 import ( "fmt" "testing" - - "github.com/influxdb/influxdb/tsdb/engine/tsm1" ) func Test_StringEncoder_NoValues(t *testing.T) { - enc := tsm1.NewStringEncoder() + enc := NewStringEncoder() b, err := enc.Bytes() if err != nil { t.Fatalf("unexpected error: %v", err) } - dec := tsm1.NewStringDecoder(b) + dec := NewStringDecoder(b) if dec.Next() { t.Fatalf("unexpected next value: got true, exp false") } } func Test_StringEncoder_Single(t *testing.T) { - enc := tsm1.NewStringEncoder() + enc := NewStringEncoder() v1 := "v1" enc.Write(v1) b, err := enc.Bytes() @@ -29,7 +27,7 @@ func Test_StringEncoder_Single(t *testing.T) { t.Fatalf("unexpected error: %v", err) } - dec := tsm1.NewStringDecoder(b) + dec := NewStringDecoder(b) if !dec.Next() { t.Fatalf("unexpected next value: got false, exp true") } @@ -40,7 +38,7 @@ func Test_StringEncoder_Single(t *testing.T) { } func Test_StringEncoder_Multi_Compressed(t *testing.T) { - enc := tsm1.NewStringEncoder() + enc := NewStringEncoder() values := make([]string, 10) for i := range values { @@ -53,15 +51,15 @@ func Test_StringEncoder_Multi_Compressed(t *testing.T) { t.Fatalf("unexpected error: %v", err) } - if b[0]>>4 != tsm1.EncodingSnappy { - t.Fatalf("unexpected encoding: got %v, exp %v", b[0], tsm1.EncodingSnappy) + if b[0]>>4 != stringCompressedSnappy { + t.Fatalf("unexpected encoding: got %v, exp %v", b[0], stringCompressedSnappy) } if exp := 47; len(b) != exp { t.Fatalf("unexpected length: got %v, exp %v", len(b), exp) } - dec := tsm1.NewStringDecoder(b) + dec := NewStringDecoder(b) for i, v := range values { if !dec.Next() { diff --git a/tsdb/engine/tsm1/timestamp.go b/tsdb/engine/tsm1/timestamp.go index 59990f5cb41..8fcb0b4ce2b 100644 --- a/tsdb/engine/tsm1/timestamp.go +++ b/tsdb/engine/tsm1/timestamp.go @@ -41,6 +41,15 @@ import ( "github.com/jwilder/encoding/simple8b" ) +const ( + // timeUncompressed is a an uncompressed format using 8 bytes per timestamp + timeUncompressed = 0 + // timeCompressedPackedSimple is a bit-packed format using simple8b encoding + timeCompressedPackedSimple = 1 + // timeCompressedRLE is a run-length encoding format + timeCompressedRLE = 2 +) + // TimeEncoder encodes time.Time to byte slices. type TimeEncoder interface { Write(t time.Time) @@ -135,7 +144,7 @@ func (e *encoder) encodePacked(div uint64, dts []uint64) ([]byte, error) { b := make([]byte, 8+1) // 4 high bits used for the encoding type - b[0] = byte(EncodingPackedSimple) << 4 + b[0] = byte(timeCompressedPackedSimple) << 4 // 4 low bits are the log10 divisor b[0] |= byte(math.Log10(float64(div))) @@ -153,7 +162,7 @@ func (e *encoder) encodePacked(div uint64, dts []uint64) ([]byte, error) { func (e *encoder) encodeRaw() ([]byte, error) { b := make([]byte, 1+len(e.ts)*8) - b[0] = byte(EncodingUncompressed) << 4 + b[0] = byte(timeUncompressed) << 4 for i, v := range e.ts { binary.BigEndian.PutUint64(b[1+i*8:1+i*8+8], uint64(v)) } @@ -165,7 +174,7 @@ func (e *encoder) encodeRLE(first, delta, div uint64, n int) ([]byte, error) { b := make([]byte, 1+10*3) // 4 high bits used for the encoding type - b[0] = byte(EncodingRLE) << 4 + b[0] = byte(timeCompressedRLE) << 4 // 4 low bits are the log10 divisor b[0] |= byte(math.Log10(float64(div))) @@ -213,11 +222,11 @@ func (d *decoder) decode(b []byte) { // Encoding type is stored in the 4 high bits of the first byte encoding := b[0] >> 4 switch encoding { - case EncodingUncompressed: + case timeUncompressed: d.decodeRaw(b[1:]) - case EncodingRLE: + case timeCompressedRLE: d.decodeRLE(b) - case EncodingPackedSimple: + case timeCompressedPackedSimple: d.decodePacked(b) default: panic(fmt.Sprintf("unknown encoding: %v", encoding)) diff --git a/tsdb/engine/tsm1/timestamp_test.go b/tsdb/engine/tsm1/timestamp_test.go index dbb5a2341a6..402a6578a11 100644 --- a/tsdb/engine/tsm1/timestamp_test.go +++ b/tsdb/engine/tsm1/timestamp_test.go @@ -1,14 +1,12 @@ -package tsm1_test +package tsm1 import ( "testing" "time" - - "github.com/influxdb/influxdb/tsdb/engine/tsm1" ) func Test_TimeEncoder(t *testing.T) { - enc := tsm1.NewTimeEncoder() + enc := NewTimeEncoder() x := []time.Time{} now := time.Unix(0, 0) @@ -24,11 +22,11 @@ func Test_TimeEncoder(t *testing.T) { t.Fatalf("unexpected error: %v", err) } - if got := b[0] >> 4; got != tsm1.EncodingPackedSimple { + if got := b[0] >> 4; got != timeCompressedPackedSimple { t.Fatalf("Wrong encoding used: expected uncompressed, got %v", got) } - dec := tsm1.NewTimeDecoder(b) + dec := NewTimeDecoder(b) for i, v := range x { if !dec.Next() { t.Fatalf("Next == false, expected true") @@ -41,20 +39,20 @@ func Test_TimeEncoder(t *testing.T) { } func Test_TimeEncoder_NoValues(t *testing.T) { - enc := tsm1.NewTimeEncoder() + enc := NewTimeEncoder() b, err := enc.Bytes() if err != nil { t.Fatalf("unexpected error: %v", err) } - dec := tsm1.NewTimeDecoder(b) + dec := NewTimeDecoder(b) if dec.Next() { t.Fatalf("unexpected next value: got true, exp false") } } func Test_TimeEncoder_One(t *testing.T) { - enc := tsm1.NewTimeEncoder() + enc := NewTimeEncoder() tm := time.Unix(0, 0) enc.Write(tm) @@ -63,11 +61,11 @@ func Test_TimeEncoder_One(t *testing.T) { t.Fatalf("unexpected error: %v", err) } - if got := b[0] >> 4; got != tsm1.EncodingPackedSimple { + if got := b[0] >> 4; got != timeCompressedPackedSimple { t.Fatalf("Wrong encoding used: expected uncompressed, got %v", got) } - dec := tsm1.NewTimeDecoder(b) + dec := NewTimeDecoder(b) if !dec.Next() { t.Fatalf("unexpected next value: got true, exp false") } @@ -78,7 +76,7 @@ func Test_TimeEncoder_One(t *testing.T) { } func Test_TimeEncoder_Two(t *testing.T) { - enc := tsm1.NewTimeEncoder() + enc := NewTimeEncoder() t1 := time.Unix(0, 0) t2 := time.Unix(0, 1) enc.Write(t1) @@ -89,11 +87,11 @@ func Test_TimeEncoder_Two(t *testing.T) { t.Fatalf("unexpected error: %v", err) } - if got := b[0] >> 4; got != tsm1.EncodingPackedSimple { + if got := b[0] >> 4; got != timeCompressedPackedSimple { t.Fatalf("Wrong encoding used: expected uncompressed, got %v", got) } - dec := tsm1.NewTimeDecoder(b) + dec := NewTimeDecoder(b) if !dec.Next() { t.Fatalf("unexpected next value: got true, exp false") } @@ -112,7 +110,7 @@ func Test_TimeEncoder_Two(t *testing.T) { } func Test_TimeEncoder_Three(t *testing.T) { - enc := tsm1.NewTimeEncoder() + enc := NewTimeEncoder() t1 := time.Unix(0, 0) t2 := time.Unix(0, 1) t3 := time.Unix(0, 2) @@ -126,11 +124,11 @@ func Test_TimeEncoder_Three(t *testing.T) { t.Fatalf("unexpected error: %v", err) } - if got := b[0] >> 4; got != tsm1.EncodingPackedSimple { + if got := b[0] >> 4; got != timeCompressedPackedSimple { t.Fatalf("Wrong encoding used: expected uncompressed, got %v", got) } - dec := tsm1.NewTimeDecoder(b) + dec := NewTimeDecoder(b) if !dec.Next() { t.Fatalf("unexpected next value: got true, exp false") } @@ -157,7 +155,7 @@ func Test_TimeEncoder_Three(t *testing.T) { } func Test_TimeEncoder_Large_Range(t *testing.T) { - enc := tsm1.NewTimeEncoder() + enc := NewTimeEncoder() t1 := time.Unix(0, 1442369134000000000) t2 := time.Unix(0, 1442369135000000000) enc.Write(t1) @@ -167,11 +165,11 @@ func Test_TimeEncoder_Large_Range(t *testing.T) { t.Fatalf("unexpected error: %v", err) } - if got := b[0] >> 4; got != tsm1.EncodingPackedSimple { + if got := b[0] >> 4; got != timeCompressedPackedSimple { t.Fatalf("Wrong encoding used: expected uncompressed, got %v", got) } - dec := tsm1.NewTimeDecoder(b) + dec := NewTimeDecoder(b) if !dec.Next() { t.Fatalf("unexpected next value: got true, exp false") } @@ -190,7 +188,7 @@ func Test_TimeEncoder_Large_Range(t *testing.T) { } func Test_TimeEncoder_Uncompressed(t *testing.T) { - enc := tsm1.NewTimeEncoder() + enc := NewTimeEncoder() t1 := time.Unix(0, 0) t2 := time.Unix(1, 0) @@ -210,11 +208,11 @@ func Test_TimeEncoder_Uncompressed(t *testing.T) { t.Fatalf("length mismatch: got %v, exp %v", len(b), exp) } - if got := b[0] >> 4; got != tsm1.EncodingUncompressed { + if got := b[0] >> 4; got != timeUncompressed { t.Fatalf("Wrong encoding used: expected uncompressed, got %v", got) } - dec := tsm1.NewTimeDecoder(b) + dec := NewTimeDecoder(b) if !dec.Next() { t.Fatalf("unexpected next value: got true, exp false") } @@ -241,7 +239,7 @@ func Test_TimeEncoder_Uncompressed(t *testing.T) { } func Test_TimeEncoder_RLE(t *testing.T) { - enc := tsm1.NewTimeEncoder() + enc := NewTimeEncoder() var ts []time.Time for i := 0; i < 500; i++ { ts = append(ts, time.Unix(int64(i), 0)) @@ -256,7 +254,7 @@ func Test_TimeEncoder_RLE(t *testing.T) { t.Fatalf("length mismatch: got %v, exp %v", len(b), exp) } - if got := b[0] >> 4; got != tsm1.EncodingRLE { + if got := b[0] >> 4; got != timeCompressedRLE { t.Fatalf("Wrong encoding used: expected uncompressed, got %v", got) } @@ -264,7 +262,7 @@ func Test_TimeEncoder_RLE(t *testing.T) { t.Fatalf("unexpected error: %v", err) } - dec := tsm1.NewTimeDecoder(b) + dec := NewTimeDecoder(b) for i, v := range ts { if !dec.Next() { t.Fatalf("Next == false, expected true") @@ -281,7 +279,7 @@ func Test_TimeEncoder_RLE(t *testing.T) { } func Test_TimeEncoder_Reverse(t *testing.T) { - enc := tsm1.NewTimeEncoder() + enc := NewTimeEncoder() ts := []time.Time{ time.Unix(0, 3), time.Unix(0, 2), @@ -297,11 +295,11 @@ func Test_TimeEncoder_Reverse(t *testing.T) { t.Fatalf("unexpected error: %v", err) } - if got := b[0] >> 4; got != tsm1.EncodingUncompressed { + if got := b[0] >> 4; got != timeUncompressed { t.Fatalf("Wrong encoding used: expected uncompressed, got %v", got) } - dec := tsm1.NewTimeDecoder(b) + dec := NewTimeDecoder(b) i := 0 for dec.Next() { if ts[i] != dec.Read() { @@ -312,7 +310,7 @@ func Test_TimeEncoder_Reverse(t *testing.T) { } func Test_TimeEncoder_220SecondDelta(t *testing.T) { - enc := tsm1.NewTimeEncoder() + enc := NewTimeEncoder() var ts []time.Time now := time.Now() for i := 0; i < 220; i++ { @@ -333,11 +331,11 @@ func Test_TimeEncoder_220SecondDelta(t *testing.T) { t.Fatalf("unexpected length: got %v, exp %v", len(b), exp) } - if got := b[0] >> 4; got != tsm1.EncodingRLE { + if got := b[0] >> 4; got != timeCompressedRLE { t.Fatalf("Wrong encoding used: expected uncompressed, got %v", got) } - dec := tsm1.NewTimeDecoder(b) + dec := NewTimeDecoder(b) i := 0 for dec.Next() { if ts[i] != dec.Read() { @@ -356,7 +354,7 @@ func Test_TimeEncoder_220SecondDelta(t *testing.T) { } func BenchmarkTimeEncoder(b *testing.B) { - enc := tsm1.NewTimeEncoder() + enc := NewTimeEncoder() x := make([]time.Time, 1024) for i := 0; i < len(x); i++ { x[i] = time.Now() @@ -371,7 +369,7 @@ func BenchmarkTimeEncoder(b *testing.B) { func BenchmarkTimeDecoder(b *testing.B) { x := make([]time.Time, 1024) - enc := tsm1.NewTimeEncoder() + enc := NewTimeEncoder() for i := 0; i < len(x); i++ { x[i] = time.Now() enc.Write(x[i]) @@ -382,7 +380,7 @@ func BenchmarkTimeDecoder(b *testing.B) { for i := 0; i < b.N; i++ { b.StopTimer() - dec := tsm1.NewTimeDecoder(bytes) + dec := NewTimeDecoder(bytes) b.StartTimer() for dec.Next() { } From 33ac598fe0f675cd0618199605c0d4d0c47a2a62 Mon Sep 17 00:00:00 2001 From: Jason Wilder Date: Fri, 2 Oct 2015 10:46:58 -0600 Subject: [PATCH 060/139] Propogate all encoding errors to engine Avoid panicing in lower level code and allow the engine to decide what it should do. --- tsdb/engine/tsm1/bool.go | 12 +++-- tsdb/engine/tsm1/encoding.go | 79 +++++++++++++++++++++++-------- tsdb/engine/tsm1/encoding_test.go | 30 +++++++++--- tsdb/engine/tsm1/float.go | 2 +- tsdb/engine/tsm1/float_test.go | 8 ++-- tsdb/engine/tsm1/int.go | 15 +++++- tsdb/engine/tsm1/string.go | 19 +++++--- tsdb/engine/tsm1/string_test.go | 15 ++++-- tsdb/engine/tsm1/timestamp.go | 12 +++-- tsdb/engine/tsm1/tsm1.go | 37 ++++++++++++--- 10 files changed, 174 insertions(+), 55 deletions(-) diff --git a/tsdb/engine/tsm1/bool.go b/tsdb/engine/tsm1/bool.go index ac08555cfe5..52ec8d849b0 100644 --- a/tsdb/engine/tsm1/bool.go +++ b/tsdb/engine/tsm1/bool.go @@ -22,6 +22,7 @@ type BoolEncoder interface { type BoolDecoder interface { Next() bool Read() bool + Error() error } type boolEncoder struct { @@ -93,9 +94,10 @@ func (e *boolEncoder) Bytes() ([]byte, error) { } type boolDecoder struct { - b []byte - i int - n int + b []byte + i int + n int + err error } func NewBoolDecoder(b []byte) BoolDecoder { @@ -128,3 +130,7 @@ func (e *boolDecoder) Read() bool { // Returns true if the bit is set return v&mask == mask } + +func (e *boolDecoder) Error() error { + return e.err +} diff --git a/tsdb/engine/tsm1/encoding.go b/tsdb/engine/tsm1/encoding.go index eef2b1d23e7..0d95bf5ac2b 100644 --- a/tsdb/engine/tsm1/encoding.go +++ b/tsdb/engine/tsm1/encoding.go @@ -65,7 +65,7 @@ func (v Values) MaxTime() int64 { return v[len(v)-1].Time().UnixNano() } -func (v Values) Encode(buf []byte) []byte { +func (v Values) Encode(buf []byte) ([]byte, error) { switch v[0].(type) { case *FloatValue: a := make([]*FloatValue, len(v)) @@ -96,7 +96,7 @@ func (v Values) Encode(buf []byte) []byte { return encodeStringBlock(buf, a) } - return nil + return nil, fmt.Errorf("unsupported value type %T", v[0]) } func (v Values) DecodeSameTypeBlock(block []byte) Values { @@ -185,9 +185,9 @@ func (f *FloatValue) Size() int { return 16 } -func encodeFloatBlock(buf []byte, values []*FloatValue) []byte { +func encodeFloatBlock(buf []byte, values []*FloatValue) ([]byte, error) { if len(values) == 0 { - return nil + return nil, nil } // A float block is encoded using different compression strategies @@ -209,7 +209,7 @@ func encodeFloatBlock(buf []byte, values []*FloatValue) []byte { // Encoded timestamp values tb, err := tsenc.Bytes() if err != nil { - panic(err.Error()) + return nil, err } // Encoded float values vb := venc.Bytes() @@ -218,7 +218,7 @@ func encodeFloatBlock(buf []byte, values []*FloatValue) []byte { // in the next byte, followed by the block block := packBlockHeader(values[0].Time(), BlockFloat64) block = append(block, packBlock(tb, vb)...) - return block + return block, nil } func decodeFloatBlock(block []byte) ([]Value, error) { @@ -249,6 +249,15 @@ func decodeFloatBlock(block []byte) ([]Value, error) { a = append(a, &FloatValue{ts, v}) } + // Did timestamp decoding have an error? + if dec.Error() != nil { + return nil, dec.Error() + } + // Did float decoding have an error? + if iter.Error() != nil { + return nil, iter.Error() + } + return a, nil } @@ -273,9 +282,9 @@ func (b *BoolValue) Value() interface{} { return b.value } -func encodeBoolBlock(buf []byte, values []*BoolValue) []byte { +func encodeBoolBlock(buf []byte, values []*BoolValue) ([]byte, error) { if len(values) == 0 { - return nil + return nil, nil } // A bool block is encoded using different compression strategies @@ -295,19 +304,19 @@ func encodeBoolBlock(buf []byte, values []*BoolValue) []byte { // Encoded timestamp values tb, err := tsenc.Bytes() if err != nil { - panic(err.Error()) + return nil, err } // Encoded float values vb, err := venc.Bytes() if err != nil { - panic(err.Error()) + return nil, err } // Preprend the first timestamp of the block in the first 8 bytes and the block // in the next byte, followed by the block block := packBlockHeader(values[0].Time(), BlockBool) block = append(block, packBlock(tb, vb)...) - return block + return block, nil } func decodeBoolBlock(block []byte) ([]Value, error) { @@ -335,6 +344,15 @@ func decodeBoolBlock(block []byte) ([]Value, error) { a = append(a, &BoolValue{ts, v}) } + // Did timestamp decoding have an error? + if dec.Error() != nil { + return nil, dec.Error() + } + // Did bool decoding have an error? + if vdec.Error() != nil { + return nil, vdec.Error() + } + return a, nil } @@ -361,7 +379,7 @@ func (v *Int64Value) Size() int { func (v *Int64Value) String() string { return fmt.Sprintf("%v", v.value) } -func encodeInt64Block(buf []byte, values []*Int64Value) []byte { +func encodeInt64Block(buf []byte, values []*Int64Value) ([]byte, error) { tsEnc := NewTimeEncoder() vEnc := NewInt64Encoder() for _, v := range values { @@ -372,17 +390,17 @@ func encodeInt64Block(buf []byte, values []*Int64Value) []byte { // Encoded timestamp values tb, err := tsEnc.Bytes() if err != nil { - panic(err.Error()) + return nil, err } // Encoded int64 values vb, err := vEnc.Bytes() if err != nil { - panic(err.Error()) + return nil, err } // Preprend the first timestamp of the block in the first 8 bytes block := packBlockHeader(values[0].Time(), BlockInt64) - return append(block, packBlock(tb, vb)...) + return append(block, packBlock(tb, vb)...), nil } func decodeInt64Block(block []byte) ([]Value, error) { @@ -411,6 +429,15 @@ func decodeInt64Block(block []byte) ([]Value, error) { a = append(a, &Int64Value{ts, v}) } + // Did timestamp decoding have an error? + if tsDec.Error() != nil { + return nil, tsDec.Error() + } + // Did int64 decoding have an error? + if vDec.Error() != nil { + return nil, vDec.Error() + } + return a, nil } @@ -437,7 +464,7 @@ func (v *StringValue) Size() int { func (v *StringValue) String() string { return v.value } -func encodeStringBlock(buf []byte, values []*StringValue) []byte { +func encodeStringBlock(buf []byte, values []*StringValue) ([]byte, error) { tsEnc := NewTimeEncoder() vEnc := NewStringEncoder() for _, v := range values { @@ -448,17 +475,17 @@ func encodeStringBlock(buf []byte, values []*StringValue) []byte { // Encoded timestamp values tb, err := tsEnc.Bytes() if err != nil { - panic(err.Error()) + return nil, err } // Encoded int64 values vb, err := vEnc.Bytes() if err != nil { - panic(err.Error()) + return nil, err } // Preprend the first timestamp of the block in the first 8 bytes block := packBlockHeader(values[0].Time(), BlockString) - return append(block, packBlock(tb, vb)...) + return append(block, packBlock(tb, vb)...), nil } func decodeStringBlock(block []byte) ([]Value, error) { @@ -477,7 +504,10 @@ func decodeStringBlock(block []byte) ([]Value, error) { // Setup our timestamp and value decoders tsDec := NewTimeDecoder(tb) - vDec := NewStringDecoder(vb) + vDec, err := NewStringDecoder(vb) + if err != nil { + return nil, err + } // Decode both a timestamp and value var a []Value @@ -487,6 +517,15 @@ func decodeStringBlock(block []byte) ([]Value, error) { a = append(a, &StringValue{ts, v}) } + // Did timestamp decoding have an error? + if tsDec.Error() != nil { + return nil, tsDec.Error() + } + // Did string decoding have an error? + if vDec.Error() != nil { + return nil, vDec.Error() + } + return a, nil } diff --git a/tsdb/engine/tsm1/encoding_test.go b/tsdb/engine/tsm1/encoding_test.go index c4889f18e5d..309b947eb60 100644 --- a/tsdb/engine/tsm1/encoding_test.go +++ b/tsdb/engine/tsm1/encoding_test.go @@ -19,7 +19,10 @@ func TestEncoding_FloatBlock(t *testing.T) { values[i] = tsm1.NewValue(t, float64(i)) } - b := values.Encode(nil) + b, err := values.Encode(nil) + if err != nil { + t.Fatalf("unexpected error: %v", err) + } decodedValues := values.DecodeSameTypeBlock(b) @@ -34,7 +37,10 @@ func TestEncoding_FloatBlock_ZeroTime(t *testing.T) { values[i] = tsm1.NewValue(time.Unix(0, 0), float64(i)) } - b := values.Encode(nil) + b, err := values.Encode(nil) + if err != nil { + t.Fatalf("unexpected error: %v", err) + } decodedValues := values.DecodeSameTypeBlock(b) @@ -51,7 +57,10 @@ func TestEncoding_IntBlock_Basic(t *testing.T) { values[i] = tsm1.NewValue(t, int64(i)) } - b := values.Encode(nil) + b, err := values.Encode(nil) + if err != nil { + t.Fatalf("unexpected error: %v", err) + } decodedValues := values.DecodeSameTypeBlock(b) @@ -83,7 +92,10 @@ func TestEncoding_IntBlock_Negatives(t *testing.T) { values[i] = tsm1.NewValue(t, int64(v)) } - b := values.Encode(nil) + b, err := values.Encode(nil) + if err != nil { + t.Fatalf("unexpected error: %v", err) + } decodedValues := values.DecodeSameTypeBlock(b) @@ -104,7 +116,10 @@ func TestEncoding_BoolBlock_Basic(t *testing.T) { values[i] = tsm1.NewValue(t, v) } - b := values.Encode(nil) + b, err := values.Encode(nil) + if err != nil { + t.Fatalf("unexpected error: %v", err) + } decodedValues := values.DecodeSameTypeBlock(b) @@ -121,7 +136,10 @@ func TestEncoding_StringBlock_Basic(t *testing.T) { values[i] = tsm1.NewValue(t, fmt.Sprintf("value %d", i)) } - b := values.Encode(nil) + b, err := values.Encode(nil) + if err != nil { + t.Fatalf("unexpected error: %v", err) + } decodedValues := values.DecodeSameTypeBlock(b) diff --git a/tsdb/engine/tsm1/float.go b/tsdb/engine/tsm1/float.go index 7ac1d93b492..22703258a36 100644 --- a/tsdb/engine/tsm1/float.go +++ b/tsdb/engine/tsm1/float.go @@ -205,6 +205,6 @@ func (it *FloatDecoder) Values() float64 { return it.val } -func (it *FloatDecoder) Err() error { +func (it *FloatDecoder) Error() error { return it.err } diff --git a/tsdb/engine/tsm1/float_test.go b/tsdb/engine/tsm1/float_test.go index 49e811a174e..794d62e5b7b 100644 --- a/tsdb/engine/tsm1/float_test.go +++ b/tsdb/engine/tsm1/float_test.go @@ -62,8 +62,8 @@ func TestFloatEncoder_Simple(t *testing.T) { t.Fatalf("Next()=true, want false") } - if err := it.Err(); err != nil { - t.Errorf("it.Err()=%v, want nil", err) + if err := it.Error(); err != nil { + t.Errorf("it.Error()=%v, want nil", err) } } @@ -127,8 +127,8 @@ func TestFloatEncoder_Roundtrip(t *testing.T) { t.Fatalf("Next()=true, want false") } - if err := it.Err(); err != nil { - t.Errorf("it.Err()=%v, want nil", err) + if err := it.Error(); err != nil { + t.Errorf("it.Error()=%v, want nil", err) } } diff --git a/tsdb/engine/tsm1/int.go b/tsdb/engine/tsm1/int.go index 40e615657dd..0060e279947 100644 --- a/tsdb/engine/tsm1/int.go +++ b/tsdb/engine/tsm1/int.go @@ -42,6 +42,7 @@ type Int64Encoder interface { type Int64Decoder interface { Next() bool Read() int64 + Error() error } type int64Encoder struct { @@ -101,6 +102,7 @@ type int64Decoder struct { n int encoding byte + err error } func NewInt64Decoder(b []byte) Int64Decoder { @@ -135,12 +137,16 @@ func (d *int64Decoder) Next() bool { case intCompressedSimple: d.decodePacked() default: - panic(fmt.Sprintf("unknown encoding %v", d.encoding)) + d.err = fmt.Errorf("unknown encoding %v", d.encoding) } } return d.i < d.n } +func (d *int64Decoder) Error() error { + return d.err +} + func (d *int64Decoder) Read() int64 { return ZigZagDecode(d.values[d.i]) } @@ -151,7 +157,12 @@ func (d *int64Decoder) decodePacked() { } v := binary.BigEndian.Uint64(d.bytes[0:8]) - n, _ := simple8b.Decode(d.values, v) + n, err := simple8b.Decode(d.values, v) + if err != nil { + // Should never happen, only error that could be returned is if the the value to be decoded was not + // actually encoded by simple8b encoder. + d.err = fmt.Errorf("failed to decode value %v: %v", v, err) + } d.n = n d.i = 0 diff --git a/tsdb/engine/tsm1/string.go b/tsdb/engine/tsm1/string.go index a2025809de9..da06bc53599 100644 --- a/tsdb/engine/tsm1/string.go +++ b/tsdb/engine/tsm1/string.go @@ -27,6 +27,7 @@ type StringEncoder interface { type StringDecoder interface { Next() bool Read() string + Error() error } type stringEncoder struct { @@ -56,21 +57,21 @@ func (e *stringEncoder) Bytes() ([]byte, error) { } type stringDecoder struct { - b []byte - l int - i int + b []byte + l int + i int + err error } -func NewStringDecoder(b []byte) StringDecoder { +func NewStringDecoder(b []byte) (StringDecoder, error) { // First byte stores the encoding type, only have snappy format // currently so ignore for now. data, err := snappy.Decode(nil, b[1:]) if err != nil { - // TODO: Need to propogate errors up the call stack better - panic(fmt.Sprintf("failed to decode string block: %v", err.Error())) + return nil, fmt.Errorf("failed to decode string block: %v", err.Error()) } - return &stringDecoder{b: data} + return &stringDecoder{b: data}, nil } func (e *stringDecoder) Next() bool { @@ -87,3 +88,7 @@ func (e *stringDecoder) Read() string { return string(e.b[e.i+n : e.i+n+int(length)]) } + +func (e *stringDecoder) Error() error { + return e.err +} diff --git a/tsdb/engine/tsm1/string_test.go b/tsdb/engine/tsm1/string_test.go index f1bf1738152..f5143514ecc 100644 --- a/tsdb/engine/tsm1/string_test.go +++ b/tsdb/engine/tsm1/string_test.go @@ -12,7 +12,10 @@ func Test_StringEncoder_NoValues(t *testing.T) { t.Fatalf("unexpected error: %v", err) } - dec := NewStringDecoder(b) + dec, err := NewStringDecoder(b) + if err != nil { + t.Fatalf("unexpected erorr creating string decoder: %v", err) + } if dec.Next() { t.Fatalf("unexpected next value: got true, exp false") } @@ -27,7 +30,10 @@ func Test_StringEncoder_Single(t *testing.T) { t.Fatalf("unexpected error: %v", err) } - dec := NewStringDecoder(b) + dec, err := NewStringDecoder(b) + if err != nil { + t.Fatalf("unexpected erorr creating string decoder: %v", err) + } if !dec.Next() { t.Fatalf("unexpected next value: got false, exp true") } @@ -59,7 +65,10 @@ func Test_StringEncoder_Multi_Compressed(t *testing.T) { t.Fatalf("unexpected length: got %v, exp %v", len(b), exp) } - dec := NewStringDecoder(b) + dec, err := NewStringDecoder(b) + if err != nil { + t.Fatalf("unexpected erorr creating string decoder: %v", err) + } for i, v := range values { if !dec.Next() { diff --git a/tsdb/engine/tsm1/timestamp.go b/tsdb/engine/tsm1/timestamp.go index 8fcb0b4ce2b..8605988b292 100644 --- a/tsdb/engine/tsm1/timestamp.go +++ b/tsdb/engine/tsm1/timestamp.go @@ -60,6 +60,7 @@ type TimeEncoder interface { type TimeDecoder interface { Next() bool Read() time.Time + Error() error } type encoder struct { @@ -191,8 +192,9 @@ func (e *encoder) encodeRLE(first, delta, div uint64, n int) ([]byte, error) { } type decoder struct { - v time.Time - ts []uint64 + v time.Time + ts []uint64 + err error } func NewTimeDecoder(b []byte) TimeDecoder { @@ -214,6 +216,10 @@ func (d *decoder) Read() time.Time { return d.v } +func (d *decoder) Error() error { + return d.err +} + func (d *decoder) decode(b []byte) { if len(b) == 0 { return @@ -229,7 +235,7 @@ func (d *decoder) decode(b []byte) { case timeCompressedPackedSimple: d.decodePacked(b) default: - panic(fmt.Sprintf("unknown encoding: %v", encoding)) + d.err = fmt.Errorf("unknown encoding: %v", encoding) } } diff --git a/tsdb/engine/tsm1/tsm1.go b/tsdb/engine/tsm1/tsm1.go index b47413868d8..a2ca218f490 100644 --- a/tsdb/engine/tsm1/tsm1.go +++ b/tsdb/engine/tsm1/tsm1.go @@ -599,7 +599,11 @@ func (e *Engine) Compact(fullCompaction bool) error { // write the previous values and clear if we've hit the limit if len(previousValues) > e.MaxPointsPerBlock { - b := previousValues.Encode(buf) + b, err := previousValues.Encode(buf) + if err != nil { + panic(fmt.Sprintf("failure encoding block: %v", err)) + } + if err := e.writeBlock(f, id, b); err != nil { // fail hard. If we can't write a file someone needs to get woken up panic(fmt.Sprintf("failure writing block: %s", err.Error())) @@ -616,7 +620,10 @@ func (e *Engine) Compact(fullCompaction bool) error { if nextID != id { // flush remaining values if len(previousValues) > 0 { - b := previousValues.Encode(buf) + b, err := previousValues.Encode(buf) + if err != nil { + panic(fmt.Sprintf("failure encoding block: %v", err)) + } currentPosition += uint32(blockHeaderSize + len(b)) previousValues = nil if err := e.writeBlock(f, id, b); err != nil { @@ -634,7 +641,11 @@ func (e *Engine) Compact(fullCompaction bool) error { } if len(previousValues) > 0 { - b := previousValues.Encode(buf) + b, err := previousValues.Encode(buf) + if err != nil { + panic(fmt.Sprintf("failure encoding block: %v", err)) + } + if err := e.writeBlock(f, minID, b); err != nil { // fail hard. If we can't write a file someone needs to get woken up panic(fmt.Sprintf("failure writing block: %s", err.Error())) @@ -1041,7 +1052,12 @@ func (e *Engine) rewriteFile(oldDF *dataFile, valuesByID map[uint64]Values) erro fpos, ok := oldIDToPosition[id] if !ok { // TODO: ensure we encode only the amount in a block - block := newVals.Encode(buf) + block, err := newVals.Encode(buf) + if err != nil { + f.Close() + return err + } + if err := e.writeBlock(f, id, block); err != nil { f.Close() return err @@ -1087,7 +1103,12 @@ func (e *Engine) rewriteFile(oldDF *dataFile, valuesByID map[uint64]Values) erro // TODO: ensure we encode only the amount in a block, refactor this wil line 450 into func if len(newVals) > 0 { // TODO: ensure we encode only the amount in a block - block := newVals.Encode(buf) + block, err := newVals.Encode(buf) + if err != nil { + f.Close() + return err + } + if _, err := f.Write(append(u64tob(id), u32tob(uint32(len(block)))...)); err != nil { f.Close() return err @@ -1586,7 +1607,11 @@ func (e *Engine) DecodeAndCombine(newValues Values, block, buf []byte, nextTime values = values[:e.MaxPointsPerBlock] } - return remainingValues, values.Encode(buf), nil + encoded, err := values.Encode(buf) + if err != nil { + return nil, nil, err + } + return remainingValues, encoded, nil } type dataFile struct { From 6f299aa9b24fb10282adb42c09cb87f21be55f9f Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Fri, 2 Oct 2015 17:17:07 -0400 Subject: [PATCH 061/139] Add db crash recovery --- tsdb/engine/tsm1/tsm1.go | 166 +++++++++++++++++++++++++++++++-------- 1 file changed, 133 insertions(+), 33 deletions(-) diff --git a/tsdb/engine/tsm1/tsm1.go b/tsdb/engine/tsm1/tsm1.go index a2ca218f490..bc00eea8d84 100644 --- a/tsdb/engine/tsm1/tsm1.go +++ b/tsdb/engine/tsm1/tsm1.go @@ -26,6 +26,10 @@ const ( // Format is the file format name of this engine. Format = "tsm1" + //IDsFileExtension is the extension for the file that keeps the compressed map + // of keys to uint64 IDs. + IDsFileExtension = "ids" + // FieldsFileExtension is the extension for the file that stores compressed field // encoding data for this db FieldsFileExtension = "fields" @@ -34,7 +38,15 @@ const ( // series metadata for series in this db SeriesFileExtension = "series" + // CollisionsFileExtension is the extension for the file that keeps a map of which + // keys have hash collisions and what their actual IDs are CollisionsFileExtension = "collisions" + + //CheckpointExtension is the extension given to files that checkpoint. + // The checkpoint files are created when a new file is first created. They + // are removed after the file has been synced and is safe for use. If a file + // has an associated checkpoint file, it wasn't safely written and both should be removed + CheckpointExtension = "check" ) type TimePrecision uint8 @@ -189,17 +201,22 @@ func (e *Engine) Open() error { return err } - // TODO: clean up previous series write - // TODO: clean up previous fields write - // TODO: clean up previous names write - // TODO: clean up any data files that didn't get cleaned up - // TODO: clean up previous collisions write + // perform any cleanup on metafiles that were halfway written + e.cleanupMetafile(SeriesFileExtension) + e.cleanupMetafile(FieldsFileExtension) + e.cleanupMetafile(IDsFileExtension) + e.cleanupMetafile(CollisionsFileExtension) files, err := filepath.Glob(filepath.Join(e.path, fmt.Sprintf("*.%s", Format))) if err != nil { return err } for _, fn := range files { + // if the file has a checkpoint it's not valid, so remove it + if removed := e.removeFileIfCheckpointExists(fn); removed { + continue + } + id, err := idFromFileName(fn) if err != nil { return err @@ -533,16 +550,8 @@ func (e *Engine) Compact(fullCompaction bool) error { ids := make([]uint64, len(files)) // initilaize for writing - f, err := os.OpenFile(fileName, os.O_CREATE|os.O_RDWR, 0666) - if err != nil { - return err - } + f, err := e.openFileAndCheckpoint(fileName) - // write the magic number - if _, err := f.Write(u32tob(magicNumber)); err != nil { - f.Close() - return err - } for i, df := range files { ids[i] = btou64(df.mmap[4:12]) positions[i] = 4 @@ -734,6 +743,10 @@ func (e *Engine) writeIndexAndGetDataFile(f *os.File, minTime, maxTime int64, id return nil, err } + if err := e.removeCheckpoint(f.Name()); err != nil { + return nil, err + } + // now open it as a memory mapped data file newDF, err := NewDataFile(f) if err != nil { @@ -788,7 +801,7 @@ func (e *Engine) convertKeysAndWriteMetadata(pointsByKey map[string]Values, meas } // read in keys and assign any that aren't defined - b, err := e.readCompressedFile("ids") + b, err := e.readCompressedFile(IDsFileExtension) if err != nil { return err, 0, 0, nil } @@ -858,7 +871,7 @@ func (e *Engine) convertKeysAndWriteMetadata(pointsByKey map[string]Values, meas if err != nil { return err, 0, 0, nil } - if err := e.replaceCompressedFile("ids", b); err != nil { + if err := e.replaceCompressedFile(IDsFileExtension, b); err != nil { return err, 0, 0, nil } } @@ -992,8 +1005,7 @@ func (e *Engine) rewriteFile(oldDF *dataFile, valuesByID map[uint64]Values) erro // always write in order by ID sort.Sort(uint64slice(ids)) - // TODO: add checkpoint file that indicates if this completed or not - f, err := os.OpenFile(e.nextFileName(), os.O_CREATE|os.O_RDWR, 0666) + f, err := e.openFileAndCheckpoint(e.nextFileName()) if err != nil { return err } @@ -1004,12 +1016,6 @@ func (e *Engine) rewriteFile(oldDF *dataFile, valuesByID map[uint64]Values) erro e.logger.Printf("rewriting index file %s with %s", oldDF.f.Name(), f.Name()) } - // write the magic number - if _, err := f.Write(u32tob(magicNumber)); err != nil { - f.Close() - return err - } - // now combine the old file data with the new values, keeping track of // their positions currentPosition := uint32(fileHeaderSize) @@ -1185,14 +1191,9 @@ func (e *Engine) flushDeletes() error { } func (e *Engine) writeNewFileExcludeDeletes(oldDF *dataFile) *dataFile { - // TODO: add checkpoint file that indicates if this completed or not - f, err := os.OpenFile(e.nextFileName(), os.O_CREATE|os.O_RDWR, 0666) + f, err := e.openFileAndCheckpoint(e.nextFileName()) if err != nil { - panic(fmt.Sprintf("error opening new index file: %s", err.Error())) - } - // write the magic number - if _, err := f.Write(u32tob(magicNumber)); err != nil { - panic(fmt.Sprintf("error writing new index file: %s", err.Error())) + panic(fmt.Sprintf("error opening new data file: %s", err.Error())) } ids := make([]uint64, 0) @@ -1614,6 +1615,99 @@ func (e *Engine) DecodeAndCombine(newValues Values, block, buf []byte, nextTime return remainingValues, encoded, nil } +// removeFileIfCheckpointExists will remove the file if its associated checkpoint fil is there. +// It returns true if the file was removed. This is for recovery of data files on startup +func (e *Engine) removeFileIfCheckpointExists(fileName string) bool { + checkpointName := fmt.Sprintf("%s.%s", fileName, CheckpointExtension) + _, err := os.Stat(checkpointName) + + // if there's no checkpoint, move on + if err != nil { + return false + } + + // there's a checkpoint so we know this file isn't safe so we should remove it + err = os.Remove(fileName) + if err != nil { + panic(fmt.Sprintf("error removing file %s", err.Error())) + } + + err = os.Remove(checkpointName) + if err != nil { + panic(fmt.Sprintf("error removing file %s", err.Error())) + } + + return true +} + +// cleanupMetafile will remove the tmp file if the other file exists, or rename the +// tmp file to be a regular file if the normal file is missing. This is for recovery on +// startup. +func (e *Engine) cleanupMetafile(name string) { + fileName := filepath.Join(e.path, name) + tmpName := fileName + "tmp" + + _, err := os.Stat(tmpName) + + // if the tmp file isn't there, we can just exit + if err != nil { + return + } + + _, err = os.Stat(fileName) + + // the regular file is there so we should just remove the tmp file + if err == nil { + err = os.Remove(tmpName) + if err != nil { + panic(fmt.Sprintf("error removing meta file %s: %s", tmpName, err.Error())) + } + } + + // regular file isn't there so have the tmp file take its place + err = os.Rename(tmpName, fileName) + if err != nil { + panic(fmt.Sprintf("error renaming meta file %s: %s", tmpName, err.Error())) + } +} + +// openFileAndCehckpoint will create a checkpoint file, open a new file for +// writing a data index, write the header and return the file +func (e *Engine) openFileAndCheckpoint(fileName string) (*os.File, error) { + checkpointFile := fmt.Sprintf("%s.%s", fileName, CheckpointExtension) + cf, err := os.OpenFile(checkpointFile, os.O_CREATE, 0666) + if err != nil { + return nil, err + } + // _, err = cf.Write(u32tob(magicNumber)) + // if err != nil { + // panic(err) + // } + if err := cf.Close(); err != nil { + return nil, err + } + _, err = os.Stat(checkpointFile) + + f, err := os.OpenFile(fileName, os.O_CREATE|os.O_RDWR, 0666) + if err != nil { + return nil, err + } + + // write the header, which is just the magic number + if _, err := f.Write(u32tob(magicNumber)); err != nil { + f.Close() + return nil, err + } + + return f, nil +} + +// removeCheckpoint removes the checkpoint for a new data file that was getting written +func (e *Engine) removeCheckpoint(fileName string) error { + checkpointFile := fmt.Sprintf("%s.%s", fileName, CheckpointExtension) + return os.Remove(checkpointFile) +} + type dataFile struct { f *os.File mu sync.RWMutex @@ -1631,6 +1725,8 @@ const ( seriesIDSize = 8 seriesPositionSize = 4 seriesHeaderSize = seriesIDSize + seriesPositionSize + minTimeOffset = 20 + maxTimeOffset = 12 ) func NewDataFile(f *os.File) (*dataFile, error) { @@ -1685,11 +1781,15 @@ func (d *dataFile) close() error { } func (d *dataFile) MinTime() int64 { - return int64(btou64(d.mmap[d.size-20 : d.size-12])) + minTimePosition := d.size - minTimeOffset + timeBytes := d.mmap[minTimePosition : minTimePosition+timeSize] + return int64(btou64(timeBytes)) } func (d *dataFile) MaxTime() int64 { - return int64(btou64(d.mmap[d.size-12 : d.size-4])) + maxTimePosition := d.size - maxTimeOffset + timeBytes := d.mmap[maxTimePosition : maxTimePosition+timeSize] + return int64(btou64(timeBytes)) } func (d *dataFile) SeriesCount() uint32 { From 4d1ce61c9f75811d57b679c4c589863f40fd25b8 Mon Sep 17 00:00:00 2001 From: Paris Holley Date: Sun, 4 Oct 2015 11:40:14 -0700 Subject: [PATCH 062/139] do not include empty tags in hash --- models/points.go | 9 ++++++++- models/points_test.go | 12 ++++++++++++ 2 files changed, 20 insertions(+), 1 deletion(-) diff --git a/models/points.go b/models/points.go index e709bfffb71..7faa6966e8d 100644 --- a/models/points.go +++ b/models/points.go @@ -1017,6 +1017,10 @@ func (p *point) Tags() Tags { i, key = scanTo(p.key, i, '=') i, value = scanTagValue(p.key, i+1) + if len(value) == 0 { + continue + } + tags[string(unescapeTag(key))] = string(unescapeTag(value)) i += 1 @@ -1137,7 +1141,10 @@ func (t Tags) HashKey() []byte { for k, v := range t { ek := escapeTag([]byte(k)) ev := escapeTag([]byte(v)) - escaped[string(ek)] = string(ev) + + if len(string(ev)) > 0 { + escaped[string(ek)] = string(ev) + } } // Extract keys and determine final size. diff --git a/models/points_test.go b/models/points_test.go index b86dd875100..b95ccb9ea19 100644 --- a/models/points_test.go +++ b/models/points_test.go @@ -599,6 +599,18 @@ func TestParsePointUnescape(t *testing.T) { }, time.Unix(0, 0))) + // tag with no value + test(t, `cpu,regions=east value="1"`, + models.NewPoint("cpu", + models.Tags{ + "regions": "east", + "foobar": "", + }, + models.Fields{ + "value": "1", + }, + time.Unix(0, 0))) + // commas in field values test(t, `cpu,regions=east value="1,0"`, models.NewPoint("cpu", From 6504df6ff3f3a3a1633d920df59d7400019dfc09 Mon Sep 17 00:00:00 2001 From: Jason Wilder Date: Mon, 5 Oct 2015 12:32:29 -0600 Subject: [PATCH 063/139] Fix comment typos --- tsdb/engine/tsm1/bool.go | 1 - tsdb/engine/tsm1/encoding.go | 10 +++++----- tsdb/engine/tsm1/float.go | 4 ++-- tsdb/engine/tsm1/int.go | 8 +++++--- tsdb/engine/tsm1/timestamp.go | 16 ++++++++-------- 5 files changed, 20 insertions(+), 19 deletions(-) diff --git a/tsdb/engine/tsm1/bool.go b/tsdb/engine/tsm1/bool.go index 52ec8d849b0..83f570a2ef2 100644 --- a/tsdb/engine/tsm1/bool.go +++ b/tsdb/engine/tsm1/bool.go @@ -114,7 +114,6 @@ func (e *boolDecoder) Next() bool { } func (e *boolDecoder) Read() bool { - // Index into the byte slice idx := e.i / 8 diff --git a/tsdb/engine/tsm1/encoding.go b/tsdb/engine/tsm1/encoding.go index 0d95bf5ac2b..3fd6d2b4bb5 100644 --- a/tsdb/engine/tsm1/encoding.go +++ b/tsdb/engine/tsm1/encoding.go @@ -214,7 +214,7 @@ func encodeFloatBlock(buf []byte, values []*FloatValue) ([]byte, error) { // Encoded float values vb := venc.Bytes() - // Preprend the first timestamp of the block in the first 8 bytes and the block + // Prepend the first timestamp of the block in the first 8 bytes and the block // in the next byte, followed by the block block := packBlockHeader(values[0].Time(), BlockFloat64) block = append(block, packBlock(tb, vb)...) @@ -312,7 +312,7 @@ func encodeBoolBlock(buf []byte, values []*BoolValue) ([]byte, error) { return nil, err } - // Preprend the first timestamp of the block in the first 8 bytes and the block + // Prepend the first timestamp of the block in the first 8 bytes and the block // in the next byte, followed by the block block := packBlockHeader(values[0].Time(), BlockBool) block = append(block, packBlock(tb, vb)...) @@ -398,7 +398,7 @@ func encodeInt64Block(buf []byte, values []*Int64Value) ([]byte, error) { return nil, err } - // Preprend the first timestamp of the block in the first 8 bytes + // Prepend the first timestamp of the block in the first 8 bytes block := packBlockHeader(values[0].Time(), BlockInt64) return append(block, packBlock(tb, vb)...), nil } @@ -477,13 +477,13 @@ func encodeStringBlock(buf []byte, values []*StringValue) ([]byte, error) { if err != nil { return nil, err } - // Encoded int64 values + // Encoded string values vb, err := vEnc.Bytes() if err != nil { return nil, err } - // Preprend the first timestamp of the block in the first 8 bytes + // Prepend the first timestamp of the block in the first 8 bytes block := packBlockHeader(values[0].Time(), BlockString) return append(block, packBlock(tb, vb)...), nil } diff --git a/tsdb/engine/tsm1/float.go b/tsdb/engine/tsm1/float.go index 22703258a36..8961c70f4ff 100644 --- a/tsdb/engine/tsm1/float.go +++ b/tsdb/engine/tsm1/float.go @@ -24,6 +24,7 @@ const ( floatCompressedGorilla = 1 ) +// FloatEncoder encodes multiple float64s into a byte slice type FloatEncoder struct { val float64 @@ -54,7 +55,6 @@ func (s *FloatEncoder) Bytes() []byte { } func (s *FloatEncoder) Finish() { - if !s.finished { // // write an end-of-stream record s.Push(math.NaN()) @@ -64,7 +64,6 @@ func (s *FloatEncoder) Finish() { } func (s *FloatEncoder) Push(v float64) { - if s.first { // first point s.val = v @@ -102,6 +101,7 @@ func (s *FloatEncoder) Push(v float64) { s.val = v } +// FloatDecoder decodes a byte slice into multipe float64 values type FloatDecoder struct { val float64 diff --git a/tsdb/engine/tsm1/int.go b/tsdb/engine/tsm1/int.go index 0060e279947..486f91a98d7 100644 --- a/tsdb/engine/tsm1/int.go +++ b/tsdb/engine/tsm1/int.go @@ -15,10 +15,10 @@ package tsm1 // or 8 byte uncompressed integers. The 4 high bits of the first byte indicate the encoding type // for the remaining bytes. // -// There are currently two encoding types that can be used with room for 15 more. These additional -// encoding slots are reserved for future use. One improvement to to be made is to use a patched +// There are currently two encoding types that can be used with room for 16 total. These additional +// encoding slots are reserved for future use. One improvement to be made is to use a patched // encoding such as PFOR if only a small number of values exceed the max compressed value range. This -// should improve compression ratios with very integers near the ends of the int64 range. +// should improve compression ratios with very large integers near the ends of the int64 range. import ( "encoding/binary" @@ -34,11 +34,13 @@ const ( intCompressedSimple = 1 ) +// Int64Encoder encoders int64 into byte slices type Int64Encoder interface { Write(v int64) Bytes() ([]byte, error) } +// Int64Decoder decodes a byte slice into int64s type Int64Decoder interface { Next() bool Read() int64 diff --git a/tsdb/engine/tsm1/timestamp.go b/tsdb/engine/tsm1/timestamp.go index 8605988b292..ad7ed644196 100644 --- a/tsdb/engine/tsm1/timestamp.go +++ b/tsdb/engine/tsm1/timestamp.go @@ -1,6 +1,6 @@ package tsm1 -// Timestamp encoding is adapative and based on structure of the timestamps that are encoded. It +// Timestamp encoding is adaptive and based on structure of the timestamps that are encoded. It // uses a combination of delta encoding, scaling and compression using simple8b, run length encoding // as well as falling back to no compression if needed. // @@ -20,9 +20,9 @@ package tsm1 // any value exceeds the maximum values, the deltas are stored uncompressed using 8b each. // // Each compressed byte slice has a 1 byte header indicating the compression type. The 4 high bits -// indicated the encoding type. The 4 low bits are using by the encoding type. +// indicated the encoding type. The 4 low bits are used by the encoding type. // -// For run length encoding, the 4 low bits store the log10 of the scaling factor. The next 8 bytes are +// For run-length encoding, the 4 low bits store the log10 of the scaling factor. The next 8 bytes are // the starting timestamp, next 1-10 bytes is the delta value using variable-length encoding, finally the // next 1-10 bytes is the count of values. // @@ -86,13 +86,13 @@ func (e *encoder) reduce() (max, divisor uint64, rle bool, deltas []uint64) { // Indicates whether the the deltas can be run-length encoded rle = true - // Interate in reverse so we can apply deltas in place + // Iterate in reverse so we can apply deltas in place for i := len(deltas) - 1; i > 0; i-- { // First differential encode the values deltas[i] = deltas[i] - deltas[i-1] - // We're also need to keep track of the max value and largest common divisor + // We also need to keep track of the max value and largest common divisor v := deltas[i] if v > max { @@ -121,11 +121,11 @@ func (e *encoder) Bytes() ([]byte, error) { // Maximum and largest common divisor. rle is true if dts (the delta timestamps), // are all the same. - max, mod, rle, dts := e.reduce() + max, div, rle, dts := e.reduce() // The deltas are all the same, so we can run-length encode them if rle && len(e.ts) > 60 { - return e.encodeRLE(e.ts[0], e.ts[1], mod, len(e.ts)) + return e.encodeRLE(e.ts[0], e.ts[1], div, len(e.ts)) } // We can't compress this time-range, the deltas exceed 1 << 60 @@ -133,7 +133,7 @@ func (e *encoder) Bytes() ([]byte, error) { return e.encodeRaw() } - return e.encodePacked(mod, dts) + return e.encodePacked(div, dts) } func (e *encoder) encodePacked(div uint64, dts []uint64) ([]byte, error) { From 1448db0fae284683f3c35b3a3bb75927c87a94e2 Mon Sep 17 00:00:00 2001 From: Jason Wilder Date: Mon, 5 Oct 2015 12:43:08 -0600 Subject: [PATCH 064/139] Make DecodeBlock panic if block size is too small Should never get a block size 9 bytes since Encode always returns the min timestampe and a 1 byte header. If we get this, the engine is confused. --- tsdb/engine/tsm1/encoding.go | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/tsdb/engine/tsm1/encoding.go b/tsdb/engine/tsm1/encoding.go index 3fd6d2b4bb5..040a12de666 100644 --- a/tsdb/engine/tsm1/encoding.go +++ b/tsdb/engine/tsm1/encoding.go @@ -21,6 +21,10 @@ const ( // BlockString designates a block encodes string values BlockString = 3 + + // encodedBlockHeaderSize is the size of the header for an encoded block. The first 8 bytes + // are the minimum timestamp of the block. The next byte is a block encoding type indicator. + encodedBlockHeaderSize = 9 ) type Value interface { @@ -120,8 +124,8 @@ func (v Values) DecodeSameTypeBlock(block []byte) Values { // DecodeBlock takes a byte array and will decode into values of the appropriate type // based on the block func DecodeBlock(block []byte) (Values, error) { - if len(block) == 0 { - return Values{}, nil + if len(block) <= encodedBlockHeaderSize { + panic(fmt.Sprintf("decode of short block: got %v, exp %v", len(block), encodedBlockHeaderSize)) } blockType := block[8] @@ -135,10 +139,8 @@ func DecodeBlock(block []byte) (Values, error) { case BlockString: return decodeStringBlock(block) default: + panic(fmt.Sprintf("unknown block type: %d", blockType)) } - - // TODO: add support for other block types - return nil, fmt.Errorf("unknown block type: %d", blockType) } // Deduplicate returns a new Values slice with any values From e9285f360e21061947c76f136462afbc8853d6b1 Mon Sep 17 00:00:00 2001 From: Jason Wilder Date: Mon, 5 Oct 2015 12:52:19 -0600 Subject: [PATCH 065/139] Avoid duplicating values slice when encoding --- tsdb/engine/tsm1/encoding.go | 43 ++++++++++-------------------------- tsdb/engine/tsm1/int.go | 1 + 2 files changed, 13 insertions(+), 31 deletions(-) diff --git a/tsdb/engine/tsm1/encoding.go b/tsdb/engine/tsm1/encoding.go index 040a12de666..3de88586320 100644 --- a/tsdb/engine/tsm1/encoding.go +++ b/tsdb/engine/tsm1/encoding.go @@ -72,32 +72,13 @@ func (v Values) MaxTime() int64 { func (v Values) Encode(buf []byte) ([]byte, error) { switch v[0].(type) { case *FloatValue: - a := make([]*FloatValue, len(v)) - for i, vv := range v { - a[i] = vv.(*FloatValue) - } - return encodeFloatBlock(buf, a) - + return encodeFloatBlock(buf, v) case *Int64Value: - a := make([]*Int64Value, len(v)) - for i, vv := range v { - a[i] = vv.(*Int64Value) - } - return encodeInt64Block(buf, a) - + return encodeInt64Block(buf, v) case *BoolValue: - a := make([]*BoolValue, len(v)) - for i, vv := range v { - a[i] = vv.(*BoolValue) - } - return encodeBoolBlock(buf, a) - + return encodeBoolBlock(buf, v) case *StringValue: - a := make([]*StringValue, len(v)) - for i, vv := range v { - a[i] = vv.(*StringValue) - } - return encodeStringBlock(buf, a) + return encodeStringBlock(buf, v) } return nil, fmt.Errorf("unsupported value type %T", v[0]) @@ -187,7 +168,7 @@ func (f *FloatValue) Size() int { return 16 } -func encodeFloatBlock(buf []byte, values []*FloatValue) ([]byte, error) { +func encodeFloatBlock(buf []byte, values []Value) ([]byte, error) { if len(values) == 0 { return nil, nil } @@ -204,7 +185,7 @@ func encodeFloatBlock(buf []byte, values []*FloatValue) ([]byte, error) { for _, v := range values { tsenc.Write(v.Time()) - venc.Push(v.value) + venc.Push(v.(*FloatValue).value) } venc.Finish() @@ -284,7 +265,7 @@ func (b *BoolValue) Value() interface{} { return b.value } -func encodeBoolBlock(buf []byte, values []*BoolValue) ([]byte, error) { +func encodeBoolBlock(buf []byte, values []Value) ([]byte, error) { if len(values) == 0 { return nil, nil } @@ -300,7 +281,7 @@ func encodeBoolBlock(buf []byte, values []*BoolValue) ([]byte, error) { for _, v := range values { tsenc.Write(v.Time()) - venc.Write(v.value) + venc.Write(v.(*BoolValue).value) } // Encoded timestamp values @@ -381,12 +362,12 @@ func (v *Int64Value) Size() int { func (v *Int64Value) String() string { return fmt.Sprintf("%v", v.value) } -func encodeInt64Block(buf []byte, values []*Int64Value) ([]byte, error) { +func encodeInt64Block(buf []byte, values []Value) ([]byte, error) { tsEnc := NewTimeEncoder() vEnc := NewInt64Encoder() for _, v := range values { tsEnc.Write(v.Time()) - vEnc.Write(v.value) + vEnc.Write(v.(*Int64Value).value) } // Encoded timestamp values @@ -466,12 +447,12 @@ func (v *StringValue) Size() int { func (v *StringValue) String() string { return v.value } -func encodeStringBlock(buf []byte, values []*StringValue) ([]byte, error) { +func encodeStringBlock(buf []byte, values []Value) ([]byte, error) { tsEnc := NewTimeEncoder() vEnc := NewStringEncoder() for _, v := range values { tsEnc.Write(v.Time()) - vEnc.Write(v.value) + vEnc.Write(v.(*StringValue).value) } // Encoded timestamp values diff --git a/tsdb/engine/tsm1/int.go b/tsdb/engine/tsm1/int.go index 486f91a98d7..9ce18fe96e3 100644 --- a/tsdb/engine/tsm1/int.go +++ b/tsdb/engine/tsm1/int.go @@ -109,6 +109,7 @@ type int64Decoder struct { func NewInt64Decoder(b []byte) Int64Decoder { d := &int64Decoder{ + // 240 is the maximum number of values that can be encoded into a single uint64 using simple8b values: make([]uint64, 240), } From a0841c4508781efc32c2679777e85e25c7686456 Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Sun, 4 Oct 2015 15:43:18 -0400 Subject: [PATCH 066/139] Updates based on @otoolp's PR comments --- cmd/influxd/run/server.go | 2 -- services/httpd/handler.go | 2 +- tsdb/config.go | 40 +++++++++++++++++++-------------------- tsdb/engine/tsm1/tsm1.go | 38 ++++++++++++++++++------------------- 4 files changed, 40 insertions(+), 42 deletions(-) diff --git a/cmd/influxd/run/server.go b/cmd/influxd/run/server.go index 64253f0f57e..13aa35a97ce 100644 --- a/cmd/influxd/run/server.go +++ b/cmd/influxd/run/server.go @@ -83,8 +83,6 @@ func NewServer(c *Config, buildInfo *BuildInfo) (*Server, error) { tsdbStore := tsdb.NewStore(c.Data.Dir) tsdbStore.EngineOptions.Config = c.Data - runtime.GOMAXPROCS(runtime.NumCPU()) - s := &Server{ buildInfo: *buildInfo, err: make(chan error), diff --git a/services/httpd/handler.go b/services/httpd/handler.go index 2fb96e99df7..b3c136e794d 100644 --- a/services/httpd/handler.go +++ b/services/httpd/handler.go @@ -83,7 +83,7 @@ func NewHandler(requireAuthentication, loggingEnabled, writeTrace bool, statMap mux: pat.New(), requireAuthentication: requireAuthentication, Logger: log.New(os.Stderr, "[http] ", log.LstdFlags), - loggingEnabled: false, + loggingEnabled: loggingEnabled, WriteTrace: writeTrace, statMap: statMap, } diff --git a/tsdb/config.go b/tsdb/config.go index a329c1268bb..115f895f139 100644 --- a/tsdb/config.go +++ b/tsdb/config.go @@ -45,12 +45,12 @@ const ( DefaultPartitionSizeThreshold = 50 * 1024 * 1024 // 50MB // Default WAL settings for the TSM1 WAL - DefaultFlushMemorySizeThreshold = 5 * 1024 * 1024 // 5MB - DefaultMaxMemorySizeThreshold = 100 * 1024 * 1024 // 100MB - DefaultIndexCompactionAge = time.Minute - DefaultIndexMinimumCompactionInterval = time.Minute - DefaultIndexCompactionFileCount = 5 - DefaultIndexCompactionFullAge = time.Minute + DefaultFlushMemorySizeThreshold = 5 * 1024 * 1024 // 5MB + DefaultMaxMemorySizeThreshold = 100 * 1024 * 1024 // 100MB + DefaultIndexCompactionAge = time.Minute + DefaultIndexMinCompactionInterval = time.Minute + DefaultIndexMinCompactionFileCount = 5 + DefaultIndexCompactionFullAge = 5 * time.Minute ) type Config struct { @@ -83,11 +83,11 @@ type Config struct { // IndexMinimumCompactionInterval specifies the minimum amount of time that must // pass after a compaction before another compaction is run - IndexMinimumCompactionInterval time.Duration `toml:"index-minimum-compaction-interval"` + IndexMinCompactionInterval time.Duration `toml:"index-min-compaction-interval"` // IndexCompactionFileCount specifies the minimum number of data files that // must be eligible for compaction before actually running one - IndexCompactionFileCount int `toml:"index-compaction-file-count"` + IndexMinCompactionFileCount int `toml:"index-compaction-min-file-count"` // IndexCompactionFullAge specifies how long after the last write was received // in the WAL that a full compaction should be performed. @@ -104,18 +104,18 @@ func NewConfig() Config { WALFlushInterval: toml.Duration(DefaultWALFlushInterval), WALPartitionFlushDelay: toml.Duration(DefaultWALPartitionFlushDelay), - WALLoggingEnabled: true, - WALReadySeriesSize: DefaultReadySeriesSize, - WALCompactionThreshold: DefaultCompactionThreshold, - WALMaxSeriesSize: DefaultMaxSeriesSize, - WALFlushColdInterval: toml.Duration(DefaultFlushColdInterval), - WALPartitionSizeThreshold: DefaultPartitionSizeThreshold, - WALFlushMemorySizeThreshold: DefaultFlushMemorySizeThreshold, - WALMaxMemorySizeThreshold: DefaultMaxMemorySizeThreshold, - IndexCompactionAge: DefaultIndexCompactionAge, - IndexCompactionFileCount: DefaultIndexCompactionFileCount, - IndexCompactionFullAge: DefaultIndexCompactionFullAge, - IndexMinimumCompactionInterval: DefaultIndexMinimumCompactionInterval, + WALLoggingEnabled: true, + WALReadySeriesSize: DefaultReadySeriesSize, + WALCompactionThreshold: DefaultCompactionThreshold, + WALMaxSeriesSize: DefaultMaxSeriesSize, + WALFlushColdInterval: toml.Duration(DefaultFlushColdInterval), + WALPartitionSizeThreshold: DefaultPartitionSizeThreshold, + WALFlushMemorySizeThreshold: DefaultFlushMemorySizeThreshold, + WALMaxMemorySizeThreshold: DefaultMaxMemorySizeThreshold, + IndexCompactionAge: DefaultIndexCompactionAge, + IndexMinCompactionFileCount: DefaultIndexMinCompactionFileCount, + IndexCompactionFullAge: DefaultIndexCompactionFullAge, + IndexMinCompactionInterval: DefaultIndexMinCompactionInterval, QueryLogEnabled: true, } diff --git a/tsdb/engine/tsm1/tsm1.go b/tsdb/engine/tsm1/tsm1.go index bc00eea8d84..b75d894f653 100644 --- a/tsdb/engine/tsm1/tsm1.go +++ b/tsdb/engine/tsm1/tsm1.go @@ -98,14 +98,14 @@ type Engine struct { WAL *Log - RotateFileSize uint32 - SkipCompaction bool - CompactionAge time.Duration - CompactionFileCount int - IndexCompactionFullAge time.Duration - IndexMinimumCompactionInterval time.Duration - MaxPointsPerBlock int - RotateBlockSize int + RotateFileSize uint32 + SkipCompaction bool + CompactionAge time.Duration + MinCompactionFileCount int + IndexCompactionFullAge time.Duration + IndexMinCompactionInterval time.Duration + MaxPointsPerBlock int + RotateBlockSize int // filesLock is only for modifying and accessing the files slice filesLock sync.RWMutex @@ -140,15 +140,15 @@ func NewEngine(path string, walPath string, opt tsdb.EngineOptions) tsdb.Engine logger: log.New(os.Stderr, "[tsm1] ", log.LstdFlags), // TODO: this is the function where we can inject a check against the in memory collisions - HashSeriesField: hashSeriesField, - WAL: w, - RotateFileSize: DefaultRotateFileSize, - CompactionAge: opt.Config.IndexCompactionAge, - CompactionFileCount: opt.Config.IndexCompactionFileCount, - IndexCompactionFullAge: opt.Config.IndexCompactionFullAge, - IndexMinimumCompactionInterval: opt.Config.IndexMinimumCompactionInterval, - MaxPointsPerBlock: DefaultMaxPointsPerBlock, - RotateBlockSize: DefaultRotateBlockSize, + HashSeriesField: hashSeriesField, + WAL: w, + RotateFileSize: DefaultRotateFileSize, + CompactionAge: opt.Config.IndexCompactionAge, + MinCompactionFileCount: opt.Config.IndexMinCompactionFileCount, + IndexCompactionFullAge: opt.Config.IndexCompactionFullAge, + IndexMinCompactionInterval: opt.Config.IndexMinCompactionInterval, + MaxPointsPerBlock: DefaultMaxPointsPerBlock, + RotateBlockSize: DefaultRotateBlockSize, } e.WAL.Index = e @@ -762,10 +762,10 @@ func (e *Engine) shouldCompact() bool { since := time.Since(e.lastCompactionTime) deletesPending := len(e.deletes) > 0 e.filesLock.RUnlock() - if running || since < e.IndexMinimumCompactionInterval || deletesPending { + if running || since < e.IndexMinCompactionInterval || deletesPending { return false } - return len(e.filesToCompact()) >= e.CompactionFileCount + return len(e.filesToCompact()) >= e.MinCompactionFileCount } func (e *Engine) filesToCompact() dataFiles { From ae36c57110b451c15465ff132e9098c801c695a9 Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Mon, 5 Oct 2015 17:21:07 -0400 Subject: [PATCH 067/139] Fix deletes not kept if shutdown before flush on tsm1 --- tsdb/engine/tsm1/tsm1.go | 136 ++++++++++++++++++++++++++--------- tsdb/engine/tsm1/tx.go | 4 +- tsdb/engine/tsm1/wal.go | 78 ++++++++++++++++---- tsdb/engine/tsm1/wal_test.go | 6 +- 4 files changed, 170 insertions(+), 54 deletions(-) diff --git a/tsdb/engine/tsm1/tsm1.go b/tsdb/engine/tsm1/tsm1.go index b75d894f653..aac359c19d1 100644 --- a/tsdb/engine/tsm1/tsm1.go +++ b/tsdb/engine/tsm1/tsm1.go @@ -13,6 +13,7 @@ import ( "path/filepath" "reflect" "sort" + "strings" "sync" "syscall" "time" @@ -47,6 +48,10 @@ const ( // are removed after the file has been synced and is safe for use. If a file // has an associated checkpoint file, it wasn't safely written and both should be removed CheckpointExtension = "check" + + // keyFieldSeparator separates the series key from the field name in the composite key + // that identifies a specific field in series + keyFieldSeparator = "#!~#" ) type TimePrecision uint8 @@ -115,8 +120,12 @@ type Engine struct { lastCompactionTime time.Time // deletes is a map of keys that are deleted, but haven't yet been - // compacted and flushed - deletes map[uint64]bool + // compacted and flushed. They map the ID to the corresponding key + deletes map[uint64]string + + // deleteMeasurements is a map of the measurements that are deleted + // but haven't yet been compacted and flushed + deleteMeasurements map[string]bool collisionsLock sync.RWMutex collisions map[string]uint64 @@ -240,7 +249,8 @@ func (e *Engine) Open() error { return err } - e.deletes = make(map[uint64]bool) + e.deletes = make(map[uint64]string) + e.deleteMeasurements = make(map[string]bool) // mark the last compaction as now so it doesn't try to compact while // flushing the WAL on load @@ -278,6 +288,7 @@ func (e *Engine) Close() error { e.currentFileID = 0 e.collisions = nil e.deletes = nil + e.deleteMeasurements = nil return nil } @@ -440,10 +451,16 @@ func (e *Engine) MarkDeletes(keys []string) { e.filesLock.Lock() defer e.filesLock.Unlock() for _, k := range keys { - e.deletes[e.keyToID(k)] = true + e.deletes[e.keyToID(k)] = k } } +func (e *Engine) MarkMeasurementDelete(name string) { + e.filesLock.Lock() + defer e.filesLock.Unlock() + e.deleteMeasurements[name] = true +} + // filesAndLock returns the data files that match the given range and // ensures that the write lock will hold for the entire range func (e *Engine) filesAndLock(min, max int64) (a dataFiles, lockStart, lockEnd int64) { @@ -1166,17 +1183,66 @@ func (e *Engine) rewriteFile(oldDF *dataFile, valuesByID map[uint64]Values) erro func (e *Engine) flushDeletes() error { e.writeLock.LockRange(math.MinInt64, math.MaxInt64) defer e.writeLock.UnlockRange(math.MinInt64, math.MaxInt64) + e.metaLock.Lock() + defer e.metaLock.Unlock() + + measurements := make(map[string]bool) + deletes := make(map[uint64]string) + e.filesLock.RLock() + for name, _ := range e.deleteMeasurements { + measurements[name] = true + } + for id, key := range e.deletes { + deletes[id] = key + } + e.filesLock.RUnlock() + + // if we're deleting measurements, rewrite the field data + if len(measurements) > 0 { + fields, err := e.readFields() + if err != nil { + return err + } + for name, _ := range measurements { + delete(fields, name) + } + if err := e.writeFields(fields); err != nil { + return err + } + } + series, err := e.readSeries() + if err != nil { + return err + } + for _, key := range deletes { + seriesName, _ := seriesAndFieldFromCompositeKey(key) + delete(series, seriesName) + } + if err := e.writeSeries(series); err != nil { + return err + } + + // now remove the raw time series data from the data files files := e.copyFilesCollection() newFiles := make(dataFiles, 0, len(files)) for _, f := range files { newFiles = append(newFiles, e.writeNewFileExcludeDeletes(f)) } + // update the delete map and files e.filesLock.Lock() defer e.filesLock.Unlock() + e.files = newFiles - e.deletes = make(map[uint64]bool) + + // remove the things we've deleted from the map + for name, _ := range measurements { + delete(e.deleteMeasurements, name) + } + for id, _ := range deletes { + delete(e.deletes, id) + } e.deletesPending.Add(1) go func() { @@ -1288,7 +1354,7 @@ func (e *Engine) keysWithFields(fields map[string]*tsdb.MeasurementFields, keys mf := fields[measurement] if mf != nil { for _, f := range mf.Fields { - a = append(a, seriesFieldKey(k, f.Name)) + a = append(a, SeriesFieldKey(k, f.Name)) } } @@ -1296,7 +1362,7 @@ func (e *Engine) keysWithFields(fields map[string]*tsdb.MeasurementFields, keys mf = e.WAL.measurementFieldsCache[measurement] if mf != nil { for _, f := range mf.Fields { - a = append(a, seriesFieldKey(k, f.Name)) + a = append(a, SeriesFieldKey(k, f.Name)) } } } @@ -1305,30 +1371,23 @@ func (e *Engine) keysWithFields(fields map[string]*tsdb.MeasurementFields, keys } // DeleteSeries deletes the series from the engine. -func (e *Engine) DeleteSeries(keys []string) error { +func (e *Engine) DeleteSeries(seriesKeys []string) error { + e.metaLock.Lock() + defer e.metaLock.Unlock() + fields, err := e.readFields() if err != nil { return err } - keyFields := e.keysWithFields(fields, keys) - - return e.deleteKeyFields(keyFields) -} - -func (e *Engine) deleteKeyFields(keyFields []string) error { - err := e.WAL.DeleteSeries(keyFields) - if err != nil { - return err - } + keyFields := e.keysWithFields(fields, seriesKeys) e.filesLock.Lock() defer e.filesLock.Unlock() - - for _, k := range keyFields { - e.deletes[e.keyToID(k)] = true + for _, key := range keyFields { + e.deletes[e.keyToID(key)] = key } - return nil + return e.WAL.DeleteSeries(keyFields) } // DeleteMeasurement deletes a measurement and all related series. @@ -1336,24 +1395,23 @@ func (e *Engine) DeleteMeasurement(name string, seriesKeys []string) error { e.metaLock.Lock() defer e.metaLock.Unlock() - // remove the field data from the index fields, err := e.readFields() if err != nil { return err } + // mark the measurement, series keys and the fields for deletion on the next flush + // also serves as a tombstone for any queries that come in before the flush keyFields := e.keysWithFields(fields, seriesKeys) + e.filesLock.Lock() + defer e.filesLock.Unlock() - delete(fields, name) - - if err := e.writeFields(fields); err != nil { - return err + e.deleteMeasurements[name] = true + for _, k := range keyFields { + e.deletes[e.keyToID(k)] = k } - e.WAL.DropMeasurementFields(name) - - // now delete all the measurement's series - return e.deleteKeyFields(keyFields) + return e.WAL.DeleteMeasurement(name, seriesKeys) } // SeriesCount returns the number of series buckets on the shard. @@ -1416,7 +1474,7 @@ func (e *Engine) keyToID(key string) uint64 { } func (e *Engine) keyAndFieldToID(series, field string) uint64 { - key := seriesFieldKey(series, field) + key := SeriesFieldKey(series, field) return e.keyToID(key) } @@ -1892,9 +1950,17 @@ func hashSeriesField(key string) uint64 { return h.Sum64() } -// seriesFieldKey combine a series key and field name for a unique string to be hashed to a numeric ID -func seriesFieldKey(seriesKey, field string) string { - return seriesKey + "#" + field +// SeriesFieldKey combine a series key and field name for a unique string to be hashed to a numeric ID +func SeriesFieldKey(seriesKey, field string) string { + return seriesKey + keyFieldSeparator + field +} + +func seriesAndFieldFromCompositeKey(key string) (string, string) { + parts := strings.Split(key, keyFieldSeparator) + if len(parts) != 0 { + return parts[0], strings.Join(parts[1:], keyFieldSeparator) + } + return parts[0], parts[1] } type uint64slice []uint64 diff --git a/tsdb/engine/tsm1/tx.go b/tsdb/engine/tsm1/tx.go index d5f31110d18..54653c28728 100644 --- a/tsdb/engine/tsm1/tx.go +++ b/tsdb/engine/tsm1/tx.go @@ -19,7 +19,7 @@ func (t *tx) Cursor(series string, fields []string, dec *tsdb.FieldCodec, ascend // don't add the overhead of the multifield cursor if we only have one field if len(fields) == 1 { id := t.engine.keyAndFieldToID(series, fields[0]) - isDeleted := t.engine.deletes[id] + _, isDeleted := t.engine.deletes[id] var indexCursor tsdb.Cursor if isDeleted { @@ -37,7 +37,7 @@ func (t *tx) Cursor(series string, fields []string, dec *tsdb.FieldCodec, ascend cursorFields := make([]string, 0) for _, field := range fields { id := t.engine.keyAndFieldToID(series, field) - isDeleted := t.engine.deletes[id] + _, isDeleted := t.engine.deletes[id] var indexCursor tsdb.Cursor if isDeleted { diff --git a/tsdb/engine/tsm1/wal.go b/tsdb/engine/tsm1/wal.go index 9cc2a2fd8fc..4f6607d02ae 100644 --- a/tsdb/engine/tsm1/wal.go +++ b/tsdb/engine/tsm1/wal.go @@ -119,6 +119,7 @@ type Log struct { type IndexWriter interface { Write(valuesByKey map[string]Values, measurementFieldsToSave map[string]*tsdb.MeasurementFields, seriesToCreate []*tsdb.SeriesCreate) error MarkDeletes(keys []string) + MarkMeasurementDelete(name string) } func NewLog(path string) *Log { @@ -168,7 +169,7 @@ func (l *Log) Cursor(series string, fields []string, dec *tsdb.FieldCodec, ascen if len(fields) != 1 { panic("wal cursor should only ever be called with 1 field") } - ck := seriesFieldKey(series, fields[0]) + ck := SeriesFieldKey(series, fields[0]) values := l.cache[ck] // if we're in the middle of a flush, combine the previous cache @@ -268,7 +269,7 @@ func (l *Log) addToCache(points []models.Point, fields map[string]*tsdb.Measurem for _, p := range points { for name, value := range p.Fields() { - k := seriesFieldKey(string(p.Key()), name) + k := SeriesFieldKey(string(p.Key()), name) v := NewValue(p.Time(), value) cacheValues := l.cache[k] @@ -388,11 +389,16 @@ func (l *Log) readFileToCache(fileName string) error { } l.addToCache(nil, nil, series, false) case deleteEntry: - var keys []string - if err := json.Unmarshal(data, &keys); err != nil { + d := &deleteData{} + if err := json.Unmarshal(data, &d); err != nil { return err } - l.Index.MarkDeletes(keys) + l.Index.MarkDeletes(d.Keys) + l.Index.MarkMeasurementDelete(d.MeasurementName) + l.deleteKeysFromCache(d.Keys) + if d.MeasurementName != "" { + l.deleteMeasurementFromCache(d.MeasurementName) + } } } } @@ -431,27 +437,62 @@ func (l *Log) Flush() error { return l.flush(idleFlush) } -func (l *Log) DropMeasurementFields(measurement string) { +func (l *Log) DeleteMeasurement(measurement string, keys []string) error { + d := &deleteData{MeasurementName: measurement, Keys: keys} + err := l.writeDeleteEntry(d) + if err != nil { + return err + } + + l.deleteKeysFromCache(keys) + l.deleteMeasurementFromCache(measurement) + + return nil +} + +func (l *Log) deleteMeasurementFromCache(name string) { l.cacheLock.Lock() defer l.cacheLock.Unlock() - delete(l.measurementFieldsCache, measurement) + delete(l.measurementFieldsCache, name) +} + +func (l *Log) writeDeleteEntry(d *deleteData) error { + js, err := json.Marshal(d) + if err != nil { + return err + } + data := snappy.Encode(nil, js) + return l.writeToLog(deleteEntry, data) } func (l *Log) DeleteSeries(keys []string) error { + l.deleteKeysFromCache(keys) + + return l.writeDeleteEntry(&deleteData{Keys: keys}) +} + +func (l *Log) deleteKeysFromCache(keys []string) { + seriesKeys := make(map[string]bool) + for _, k := range keys { + series, _ := seriesAndFieldFromCompositeKey(k) + seriesKeys[series] = true + } + l.cacheLock.Lock() + defer l.cacheLock.Unlock() + for _, k := range keys { delete(l.cache, k) } - l.cacheLock.Unlock() - b, err := json.Marshal(keys) - if err != nil { - return err + // now remove any of these that are marked for creation + var seriesCreate []*tsdb.SeriesCreate + for _, sc := range l.seriesToCreateCache { + if _, ok := seriesKeys[sc.Series.Key]; !ok { + seriesCreate = append(seriesCreate, sc) + } } - - cb := snappy.Encode(nil, b) - - return l.writeToLog(deleteEntry, cb) + l.seriesToCreateCache = seriesCreate } // Close will finish any flush that is currently in process and close file handles @@ -731,6 +772,13 @@ func (c *walCursor) nextReverse() Value { return c.cache[c.position] } +// deleteData holds the information for a delete entry +type deleteData struct { + // MeasurementName will be empty for deletes that are only against series + MeasurementName string + Keys []string +} + // idFromFileName parses the segment file ID from its name func idFromFileName(name string) (int, error) { parts := strings.Split(filepath.Base(name), ".") diff --git a/tsdb/engine/tsm1/wal_test.go b/tsdb/engine/tsm1/wal_test.go index dfc5cda8cc8..9df191c7a7e 100644 --- a/tsdb/engine/tsm1/wal_test.go +++ b/tsdb/engine/tsm1/wal_test.go @@ -111,11 +111,11 @@ func TestWAL_TestWriteQueryOpen(t *testing.T) { t.Fatalf("failed to open: %s", err.Error()) } - if len(vals["cpu,host=A#value"]) != 2 { + if len(vals[tsm1.SeriesFieldKey("cpu,host=A", "value")]) != 2 { t.Fatal("expected host A values to flush to index on open") } - if len(vals["cpu,host=B#value"]) != 1 { + if len(vals[tsm1.SeriesFieldKey("cpu,host=B", "value")]) != 1 { t.Fatal("expected host B values to flush to index on open") } @@ -174,3 +174,5 @@ func (m *MockIndexWriter) Write(valuesByKey map[string]tsm1.Values, measurementF } func (m *MockIndexWriter) MarkDeletes(keys []string) {} + +func (m *MockIndexWriter) MarkMeasurementDelete(name string) {} From e3b30e3a95b859b498128e5e61c890a556834261 Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Mon, 5 Oct 2015 19:57:49 -0400 Subject: [PATCH 068/139] Updates based on PR feedback --- models/points.go | 2 +- tsdb/engine.go | 2 +- tsdb/engine/tsm1/cursor.go | 42 ++++++++++++++++++++++++++++++++------ tsdb/engine/tsm1/tsm1.go | 4 ++++ 4 files changed, 42 insertions(+), 8 deletions(-) diff --git a/models/points.go b/models/points.go index 7faa6966e8d..8ebc0909405 100644 --- a/models/points.go +++ b/models/points.go @@ -1142,7 +1142,7 @@ func (t Tags) HashKey() []byte { ek := escapeTag([]byte(k)) ev := escapeTag([]byte(v)) - if len(string(ev)) > 0 { + if len(ev) > 0 { escaped[string(ek)] = string(ev) } } diff --git a/tsdb/engine.go b/tsdb/engine.go index 97ca51d9ddb..fb1b2108c5c 100644 --- a/tsdb/engine.go +++ b/tsdb/engine.go @@ -71,7 +71,7 @@ func NewEngine(path string, walPath string, options EngineOptions) (Engine, erro return newEngineFuncs[options.EngineVersion](path, walPath, options), nil } - // Only bolt-based backends are currently supported so open it and check the format. + // Only bolt and tsm1 based storage engines are currently supported var format string if err := func() error { // if it's a dir then it's a tsm1 engine diff --git a/tsdb/engine/tsm1/cursor.go b/tsdb/engine/tsm1/cursor.go index 06fd0bbf8f8..01ac2e3c7a7 100644 --- a/tsdb/engine/tsm1/cursor.go +++ b/tsdb/engine/tsm1/cursor.go @@ -6,6 +6,10 @@ import ( "github.com/influxdb/influxdb/tsdb" ) +// combinedEngineCursor holds a cursor for the WAL and the index +// and will combine the two together. Any points in the WAL with +// identical timestamps from the index will be preferred over the +// index point type combinedEngineCursor struct { walCursor tsdb.Cursor engineCursor tsdb.Cursor @@ -24,20 +28,26 @@ func NewCombinedEngineCursor(wc, ec tsdb.Cursor, ascending bool) tsdb.Cursor { } } +// SeekTo will seek both the index and WAL cursor func (c *combinedEngineCursor) SeekTo(seek int64) (key int64, value interface{}) { c.walKeyBuf, c.walValueBuf = c.walCursor.SeekTo(seek) c.engineKeyBuf, c.engineValueBuf = c.engineCursor.SeekTo(seek) return c.read() } +// Next returns the next value in the cursor func (c *combinedEngineCursor) Next() (int64, interface{}) { return c.read() } +// Ascending returns true if the cursor is time ascending func (c *combinedEngineCursor) Ascending() bool { return c.ascending } +// read will return the buffer value that is next from either the +// WAL or index cursor and repopulate the buffer value with the +// appropriate cursor's next value func (c *combinedEngineCursor) read() (key int64, value interface{}) { if c.walKeyBuf == tsdb.EOF && c.engineKeyBuf == tsdb.EOF { return tsdb.EOF, nil @@ -84,6 +94,9 @@ func (c *combinedEngineCursor) read() (key int64, value interface{}) { return } +// multieFieldCursor wraps cursors for multiple fields on the same series +// key. Instead of returning a plain interface value in the call for Next(), +// it returns a map[string]interface{} for the field values type multiFieldCursor struct { fields []string cursors []tsdb.Cursor @@ -158,18 +171,31 @@ func (c *emptyCursor) Next() (int64, interface{}) { return tsdb.EOF, func (c *emptyCursor) SeekTo(key int64) (int64, interface{}) { return tsdb.EOF, nil } func (c *emptyCursor) Ascending() bool { return c.ascending } +// cursor is a cursor for the data in the index type cursor struct { - id uint64 - f *dataFile + // id for the series key and field + id uint64 + + // f is the current data file we're reading from + f *dataFile + + // filesPos is the position in the files index we're reading from filesPos int // the index in the files slice we're looking at - pos uint32 - vals Values + + // pos is the position in the current data file we're reading + pos uint32 + + // vals is the current decoded block of Values we're iterating from + vals Values ascending bool - blockPositions []uint32 // only used for descending queries + // blockPositions is used for descending queries to keep track + // of what positions in the current data file encoded blocks for + // the id exist at + blockPositions []uint32 - // time acending list of data files + // time acending slice of read only data files files []*dataFile } @@ -334,10 +360,13 @@ func (c *cursor) seekDescending(seek int64) (int64, interface{}) { return tsdb.EOF, nil } +// blockMinTime is the minimum time for the block func (c *cursor) blockMinTime(pos uint32) int64 { return int64(btou64(c.f.mmap[pos+12 : pos+20])) } +// setBlockPositions will read the positions of all +// blocks for the cursor id in the given data file func (c *cursor) setBlockPositions() { pos := c.pos @@ -431,6 +460,7 @@ func (c *cursor) blockLength(pos uint32) uint32 { return btou32(c.f.mmap[pos+8 : pos+12]) } +// decodeBlock will decod the block and set the vals func (c *cursor) decodeBlock(position uint32) { length := c.blockLength(position) block := c.f.mmap[position+blockHeaderSize : position+blockHeaderSize+length] diff --git a/tsdb/engine/tsm1/tsm1.go b/tsdb/engine/tsm1/tsm1.go index aac359c19d1..59a0f3fe051 100644 --- a/tsdb/engine/tsm1/tsm1.go +++ b/tsdb/engine/tsm1/tsm1.go @@ -80,6 +80,8 @@ const ( // MAP_POPULATE is for the mmap syscall. For some reason this isn't defined in golang's syscall MAP_POPULATE = 0x8000 + // magicNumber is written as the first 4 bytes of a data file to + // identify the file as a tsm1 formatted file magicNumber uint32 = 0x16D116D1 ) @@ -1298,6 +1300,8 @@ func (e *Engine) writeNewFileExcludeDeletes(oldDF *dataFile) *dataFile { } func (e *Engine) nextFileName() string { + e.filesLock.Lock() + defer e.filesLock.Unlock() e.currentFileID++ return filepath.Join(e.path, fmt.Sprintf("%07d.%s", e.currentFileID, Format)) } From fb2a1cb2f30cb9c27bd42da0ce6e2ef7fde87e3f Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Wed, 2 Sep 2015 18:05:07 -0400 Subject: [PATCH 069/139] WIP: skeleton for encoding for new engine --- tsdb/engine/pd1/encoding.go | 51 ++++++++++++++++++++++++++++++++ tsdb/engine/pd1/encoding_test.go | 38 ++++++++++++++++++++++++ 2 files changed, 89 insertions(+) create mode 100644 tsdb/engine/pd1/encoding.go create mode 100644 tsdb/engine/pd1/encoding_test.go diff --git a/tsdb/engine/pd1/encoding.go b/tsdb/engine/pd1/encoding.go new file mode 100644 index 00000000000..7739ce22341 --- /dev/null +++ b/tsdb/engine/pd1/encoding.go @@ -0,0 +1,51 @@ +package pd1 + +import () + +type FloatValue struct { + Time int64 + Value float64 +} + +func EncodeFloatBlock(buf []byte, values []FloatValue) []byte { + return nil +} + +func DecodeFloatBlock(block []byte) ([]FloatValue, error) { + return nil, nil +} + +type BoolValue struct { + Time int64 + Value bool +} + +func EncodeBoolBlock(buf []byte, values []BoolValue) []byte { + return nil +} + +func DecodeBoolBlock(block []byte) ([]BoolValue, error) { + return nil, nil +} + +type Int64Value struct { + Time int64 + Value int64 +} + +func EncodeInt64Block(buf []byte, values []Int64Value) []byte { + return nil +} + +func DecodeInt64Block(block []byte) ([]Int64Value, error) { + return nil, nil +} + +type StringValue struct { + Time int64 + Value string +} + +func EncodeStringBlock(values []StringValue) []byte { + return nil +} diff --git a/tsdb/engine/pd1/encoding_test.go b/tsdb/engine/pd1/encoding_test.go new file mode 100644 index 00000000000..077053449a0 --- /dev/null +++ b/tsdb/engine/pd1/encoding_test.go @@ -0,0 +1,38 @@ +package pd1_test + +import ( + "math/rand" + "reflect" + "testing" + "time" + + "github.com/influxdb/influxdb/tsdb/engine/pd1" +) + +func TestEncoding_FloatBlock(t *testing.T) { + valueCount := 100 + times := getTimes(valueCount, 60, time.Second) + values := make([]pd1.FloatValue, len(times)) + for i, t := range times { + values[i] = pd1.FloatValue{Time: t, Value: rand.Float64()} + } + + b := pd1.EncodeFloatBlock(nil, values) + + decodedValues, err := pd1.DecodeFloatBlock(b) + if err != nil { + t.Fatalf("error decoding: %s", err.Error) + } + if !reflect.DeepEqual(decodedValues, values) { + t.Fatalf("values not equal:\n\tgot: %s\n\texp: %s", values, decodedValues) + } +} + +func getTimes(n, step int, precision time.Duration) []int64 { + t := time.Now().Round(precision) + a := make([]int64, n) + for i := 0; i < n; i++ { + a[i] = t.Add(60 * precision).UnixNano() + } + return a +} From 12ea1cb26f46c70d2784c32aa9ea452f978d6d9d Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Wed, 2 Sep 2015 18:45:32 -0400 Subject: [PATCH 070/139] Add comment about encoding float --- tsdb/engine/pd1/encoding.go | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/tsdb/engine/pd1/encoding.go b/tsdb/engine/pd1/encoding.go index 7739ce22341..b53f4f25635 100644 --- a/tsdb/engine/pd1/encoding.go +++ b/tsdb/engine/pd1/encoding.go @@ -1,12 +1,16 @@ package pd1 -import () +import ( + "time" +) type FloatValue struct { Time int64 Value float64 } +// First 8 bytes should be the timestamp, second 8 bytes should be +// the first float value func EncodeFloatBlock(buf []byte, values []FloatValue) []byte { return nil } From 7555ccbd7012127d55b356a01adf0ed938b98ca2 Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Sun, 6 Sep 2015 15:49:15 -0700 Subject: [PATCH 071/139] WIP: engine work --- cmd/influxd/run/server.go | 2 + services/httpd/handler.go | 2 +- tsdb/engine.go | 17 +- tsdb/engine/b1/b1.go | 2 +- tsdb/engine/bz1/bz1.go | 2 +- tsdb/engine/bz1/bz1_test.go | 4 +- tsdb/engine/engine.go | 1 + tsdb/engine/pd1/encoding.go | 36 +- tsdb/engine/pd1/encoding_test.go | 9 +- tsdb/engine/pd1/pd1.go | 1255 ++++++++++++++++++++++++++++++ tsdb/engine/pd1/pd1_test.go | 196 +++++ tsdb/shard.go | 13 +- 12 files changed, 1517 insertions(+), 22 deletions(-) create mode 100644 tsdb/engine/pd1/pd1.go create mode 100644 tsdb/engine/pd1/pd1_test.go diff --git a/cmd/influxd/run/server.go b/cmd/influxd/run/server.go index 0d23c0f1aa8..3a92dc6acc6 100644 --- a/cmd/influxd/run/server.go +++ b/cmd/influxd/run/server.go @@ -84,6 +84,8 @@ func NewServer(c *Config, buildInfo *BuildInfo) (*Server, error) { tsdbStore := tsdb.NewStore(c.Data.Dir) tsdbStore.EngineOptions.Config = c.Data + runtime.GOMAXPROCS(runtime.NumCPU()) + s := &Server{ buildInfo: *buildInfo, err: make(chan error), diff --git a/services/httpd/handler.go b/services/httpd/handler.go index b3c136e794d..2fb96e99df7 100644 --- a/services/httpd/handler.go +++ b/services/httpd/handler.go @@ -83,7 +83,7 @@ func NewHandler(requireAuthentication, loggingEnabled, writeTrace bool, statMap mux: pat.New(), requireAuthentication: requireAuthentication, Logger: log.New(os.Stderr, "[http] ", log.LstdFlags), - loggingEnabled: loggingEnabled, + loggingEnabled: false, WriteTrace: writeTrace, statMap: statMap, } diff --git a/tsdb/engine.go b/tsdb/engine.go index c8d5946139f..d2862b54863 100644 --- a/tsdb/engine.go +++ b/tsdb/engine.go @@ -24,7 +24,7 @@ type Engine interface { Close() error SetLogOutput(io.Writer) - LoadMetadataIndex(index *DatabaseIndex, measurementFields map[string]*MeasurementFields) error + LoadMetadataIndex(shard *Shard, index *DatabaseIndex, measurementFields map[string]*MeasurementFields) error Begin(writable bool) (Tx, error) WritePoints(points []models.Point, measurementFieldsToSave map[string]*MeasurementFields, seriesToCreate []*SeriesCreate) error @@ -60,6 +60,21 @@ func NewEngine(path string, walPath string, options EngineOptions) (Engine, erro // Only bolt-based backends are currently supported so open it and check the format. var format string if err := func() error { + // if it's a dir then it's a pd1 engine + f, err := os.Open(path) + if err != nil { + return err + } + fi, err := f.Stat() + f.Close() + if err != nil { + return err + } + if fi.Mode().IsDir() { + format = "pd1" + return nil + } + db, err := bolt.Open(path, 0666, &bolt.Options{Timeout: 1 * time.Second}) if err != nil { return err diff --git a/tsdb/engine/b1/b1.go b/tsdb/engine/b1/b1.go index fdc337b2f7a..356b327fe95 100644 --- a/tsdb/engine/b1/b1.go +++ b/tsdb/engine/b1/b1.go @@ -174,7 +174,7 @@ func (e *Engine) close() error { func (e *Engine) SetLogOutput(w io.Writer) { e.LogOutput = w } // LoadMetadataIndex loads the shard metadata into memory. -func (e *Engine) LoadMetadataIndex(index *tsdb.DatabaseIndex, measurementFields map[string]*tsdb.MeasurementFields) error { +func (e *Engine) LoadMetadataIndex(shard *tsdb.Shard, index *tsdb.DatabaseIndex, measurementFields map[string]*tsdb.MeasurementFields) error { return e.db.View(func(tx *bolt.Tx) error { // load measurement metadata meta := tx.Bucket([]byte("fields")) diff --git a/tsdb/engine/bz1/bz1.go b/tsdb/engine/bz1/bz1.go index e4d5682ca47..aa38600b806 100644 --- a/tsdb/engine/bz1/bz1.go +++ b/tsdb/engine/bz1/bz1.go @@ -176,7 +176,7 @@ func (e *Engine) close() error { func (e *Engine) SetLogOutput(w io.Writer) {} // LoadMetadataIndex loads the shard metadata into memory. -func (e *Engine) LoadMetadataIndex(index *tsdb.DatabaseIndex, measurementFields map[string]*tsdb.MeasurementFields) error { +func (e *Engine) LoadMetadataIndex(shard *tsdb.Shard, index *tsdb.DatabaseIndex, measurementFields map[string]*tsdb.MeasurementFields) error { if err := e.db.View(func(tx *bolt.Tx) error { // Load measurement metadata fields, err := e.readFields(tx) diff --git a/tsdb/engine/bz1/bz1_test.go b/tsdb/engine/bz1/bz1_test.go index 97873afe3b2..0b0cb1e60cc 100644 --- a/tsdb/engine/bz1/bz1_test.go +++ b/tsdb/engine/bz1/bz1_test.go @@ -38,7 +38,7 @@ func TestEngine_LoadMetadataIndex_Series(t *testing.T) { // Load metadata index. index := tsdb.NewDatabaseIndex() - if err := e.LoadMetadataIndex(index, make(map[string]*tsdb.MeasurementFields)); err != nil { + if err := e.LoadMetadataIndex(nil, index, make(map[string]*tsdb.MeasurementFields)); err != nil { t.Fatal(err) } @@ -80,7 +80,7 @@ func TestEngine_LoadMetadataIndex_Fields(t *testing.T) { // Load metadata index. mfs := make(map[string]*tsdb.MeasurementFields) - if err := e.LoadMetadataIndex(tsdb.NewDatabaseIndex(), mfs); err != nil { + if err := e.LoadMetadataIndex(nil, tsdb.NewDatabaseIndex(), mfs); err != nil { t.Fatal(err) } diff --git a/tsdb/engine/engine.go b/tsdb/engine/engine.go index c5565ff06cf..03022f9d0e7 100644 --- a/tsdb/engine/engine.go +++ b/tsdb/engine/engine.go @@ -3,4 +3,5 @@ package engine import ( _ "github.com/influxdb/influxdb/tsdb/engine/b1" _ "github.com/influxdb/influxdb/tsdb/engine/bz1" + _ "github.com/influxdb/influxdb/tsdb/engine/pd1" ) diff --git a/tsdb/engine/pd1/encoding.go b/tsdb/engine/pd1/encoding.go index b53f4f25635..285c83f8c0b 100644 --- a/tsdb/engine/pd1/encoding.go +++ b/tsdb/engine/pd1/encoding.go @@ -2,25 +2,43 @@ package pd1 import ( "time" + + "github.com/dgryski/go-tsz" ) type FloatValue struct { - Time int64 + Time time.Time Value float64 } -// First 8 bytes should be the timestamp, second 8 bytes should be -// the first float value +type FloatValues []FloatValue + +func (a FloatValues) Len() int { return len(a) } +func (a FloatValues) Swap(i, j int) { a[i], a[j] = a[j], a[i] } +func (a FloatValues) Less(i, j int) bool { return a[i].Time.UnixNano() < a[j].Time.UnixNano() } + +// TODO: make this work with nanosecond timestamps func EncodeFloatBlock(buf []byte, values []FloatValue) []byte { - return nil + s := tsz.New(uint32(values[0].Time.Unix())) + for _, v := range values { + s.Push(uint32(v.Time.Unix()), v.Value) + } + s.Finish() + return s.Bytes() } func DecodeFloatBlock(block []byte) ([]FloatValue, error) { - return nil, nil + iter, _ := tsz.NewIterator(block) + a := make([]FloatValue, 0) + for iter.Next() { + t, f := iter.Values() + a = append(a, FloatValue{time.Unix(int64(t), 0), f}) + } + return a, nil } type BoolValue struct { - Time int64 + Time time.Time Value bool } @@ -33,7 +51,7 @@ func DecodeBoolBlock(block []byte) ([]BoolValue, error) { } type Int64Value struct { - Time int64 + Time time.Time Value int64 } @@ -46,10 +64,10 @@ func DecodeInt64Block(block []byte) ([]Int64Value, error) { } type StringValue struct { - Time int64 + Time time.Time Value string } -func EncodeStringBlock(values []StringValue) []byte { +func EncodeStringBlock(buf []byte, values []StringValue) []byte { return nil } diff --git a/tsdb/engine/pd1/encoding_test.go b/tsdb/engine/pd1/encoding_test.go index 077053449a0..26bb4c2e076 100644 --- a/tsdb/engine/pd1/encoding_test.go +++ b/tsdb/engine/pd1/encoding_test.go @@ -23,16 +23,17 @@ func TestEncoding_FloatBlock(t *testing.T) { if err != nil { t.Fatalf("error decoding: %s", err.Error) } + if !reflect.DeepEqual(decodedValues, values) { - t.Fatalf("values not equal:\n\tgot: %s\n\texp: %s", values, decodedValues) + t.Fatalf("unexpected results:\n\tgot: %v\n\texp: %v\n", decodedValues, values) } } -func getTimes(n, step int, precision time.Duration) []int64 { +func getTimes(n, step int, precision time.Duration) []time.Time { t := time.Now().Round(precision) - a := make([]int64, n) + a := make([]time.Time, n) for i := 0; i < n; i++ { - a[i] = t.Add(60 * precision).UnixNano() + a[i] = t.Add(60 * precision) } return a } diff --git a/tsdb/engine/pd1/pd1.go b/tsdb/engine/pd1/pd1.go new file mode 100644 index 00000000000..d94a7d7795c --- /dev/null +++ b/tsdb/engine/pd1/pd1.go @@ -0,0 +1,1255 @@ +package pd1 + +import ( + "bytes" + "encoding/binary" + "encoding/json" + "fmt" + "hash/fnv" + "io" + "io/ioutil" + "math" + "os" + "path/filepath" + "sort" + "strconv" + "strings" + "sync" + "syscall" + "time" + + "github.com/golang/snappy" + "github.com/influxdb/influxdb/influxql" + "github.com/influxdb/influxdb/tsdb" +) + +const ( + // Format is the file format name of this engine. + Format = "pd1" + + // FieldsFileExtension is the extension for the file that stores compressed field + // encoding data for this db + FieldsFileExtension = "fld" + + // SeriesFileExtension is the extension for the file that stores the compressed + // series metadata for series in this db + SeriesFileExtension = "srs" + + CollisionsFileExtension = "col" +) + +type TimePrecision uint8 + +const ( + Seconds TimePrecision = iota + Milliseconds + Microseconds + Nanoseconds +) + +func init() { + tsdb.RegisterEngine(Format, NewEngine) +} + +const ( + // DefaultBlockSize is the default size of uncompressed points blocks. + DefaultBlockSize = 512 * 1024 // 512KB + + DefaultMaxFileSize = 50 * 1024 * 1024 // 50MB + + DefaultMaxPointsPerBlock = 5000 + + // MAP_POPULATE is for the mmap syscall. For some reason this isn't defined in golang's syscall + MAP_POPULATE = 0x8000 +) + +// Ensure Engine implements the interface. +var _ tsdb.Engine = &Engine{} + +// Engine represents a storage engine with compressed blocks. +type Engine struct { + mu sync.Mutex + path string + + shard *tsdb.Shard + + // HashSeriesField is a function that takes a series key and a field name + // and returns a hash identifier. It's not guaranteed to be unique. + HashSeriesField func(key string) uint64 + + filesLock sync.RWMutex + files dataFiles + currentFileID int + queryLock sync.RWMutex +} + +// NewEngine returns a new instance of Engine. +func NewEngine(path string, walPath string, opt tsdb.EngineOptions) tsdb.Engine { + e := &Engine{ + path: path, + + HashSeriesField: hashSeriesField, + } + + return e +} + +// Path returns the path the engine was opened with. +func (e *Engine) Path() string { return e.path } + +// Open opens and initializes the engine. +func (e *Engine) Open() error { + if err := os.MkdirAll(e.path, 0777); err != nil { + return err + } + + // TODO: clean up previous series write + // TODO: clean up previous fields write + // TODO: clean up previous names write + // TODO: clean up any data files that didn't get cleaned up + + return nil +} + +// Close closes the engine. +func (e *Engine) Close() error { + return nil +} + +// SetLogOutput is a no-op. +func (e *Engine) SetLogOutput(w io.Writer) {} + +// LoadMetadataIndex loads the shard metadata into memory. +func (e *Engine) LoadMetadataIndex(shard *tsdb.Shard, index *tsdb.DatabaseIndex, measurementFields map[string]*tsdb.MeasurementFields) error { + e.shard = shard + // TODO: write the metadata from the WAL + + // Load measurement metadata + fields, err := e.readFields() + if err != nil { + return err + } + for k, mf := range fields { + m := index.CreateMeasurementIndexIfNotExists(string(k)) + for name, _ := range mf.Fields { + m.SetFieldName(name) + } + mf.Codec = tsdb.NewFieldCodec(mf.Fields) + measurementFields[m.Name] = mf + } + + // Load series metadata + series, err := e.readSeries() + if err != nil { + return err + } + + // Load the series into the in-memory index in sorted order to ensure + // it's always consistent for testing purposes + a := make([]string, 0, len(series)) + for k, _ := range series { + a = append(a, k) + } + sort.Strings(a) + for _, key := range a { + s := series[key] + s.InitializeShards() + index.CreateSeriesIndexIfNotExists(tsdb.MeasurementFromSeriesKey(string(key)), s) + } + + return nil +} + +// WritePoints writes metadata and point data into the engine. +// Returns an error if new points are added to an existing key. +func (e *Engine) WritePoints(points []tsdb.Point, measurementFieldsToSave map[string]*tsdb.MeasurementFields, seriesToCreate []*tsdb.SeriesCreate) error { + // TODO: Write points to the WAL + + return nil +} + +func (e *Engine) WriteAndCompact(points []tsdb.Point, measurementFieldsToSave map[string]*tsdb.MeasurementFields, seriesToCreate []*tsdb.SeriesCreate) error { + e.mu.Lock() + defer e.mu.Unlock() + + if err := e.writeNewFields(measurementFieldsToSave); err != nil { + return err + } + if err := e.writeNewSeries(seriesToCreate); err != nil { + return err + } + + if len(points) == 0 { + return nil + } + + b, err := e.readCompressedFile("names") + if err != nil { + return err + } + ids := make(map[uint64]string) + + var names []string + if b != nil { + if err := json.Unmarshal(b, &names); err != nil { + return err + } + + for _, n := range names { + ids[e.HashSeriesField(n)] = n + } + } + + fmt.Println("read names: ", len(names), len(ids)) + + // these are values that are newer than anything stored in the shard + valuesByID := make(map[uint64]*valueCollection) + // map the points to the data file they belong to if they overlap + dataFileToValues := make(map[*dataFile]map[uint64]*valueCollection) + + // we keep track of the newest data file and if it should be + // rewritten with new data. + var newestDataFile *dataFile + overwriteNewestFile := false + if len(e.files) > 0 { + newestDataFile = e.files[len(e.files)-1] + overwriteNewestFile = newestDataFile.size < DefaultMaxFileSize + } + + // compute ids of new keys and arrange for insertion + for _, p := range points { + for fn, val := range p.Fields() { + n := seriesFieldKey(string(p.Key()), fn) + id := e.HashSeriesField(n) + if series, ok := ids[id]; !ok { + names = append(names, n) + } else { // possible collision? + if n != series { + // TODO: implement collision detection + panic("name collision!") + } + } + + ids[id] = n + + vals := valuesByID[id] + if vals == nil { + // TODO: deal with situation where there are already files, + // but the user is inserting a bunch of data that predates + // any of them. It's ok to rewrite the first file, but + // only to max size. Then we should create a new one + + // points always come in time increasing order. This is + // the first point we've seen for this key. So it might + // need to get put into an older file instead of a new + // one. Check and set accordingly + var df *dataFile + for i := len(e.files) - 1; i >= 0; i-- { + if p.UnixNano() > e.files[i].MaxTime() { + break + } + df = e.files[i] + } + vals = &valueCollection{} + + if df == nil || (df == newestDataFile && overwriteNewestFile) { + // this point is newer than anything we have stored + // or it belongs in the most recent file, which should get + // rewritten + valuesByID[id] = vals + } else { + // it overlaps with another file so mark it and it can be compacted + dfm := dataFileToValues[df] + if dfm == nil { + dfm = make(map[uint64]*valueCollection) + dataFileToValues[df] = dfm + } + + if vc := dfm[id]; vc == nil { + dfm[id] = vals + } else { + vals = vc + } + } + } + + switch t := val.(type) { + case float64: + vals.floatValues = append(vals.floatValues, FloatValue{Time: p.Time(), Value: t}) + case int64: + vals.intValues = append(vals.intValues, Int64Value{Time: p.Time(), Value: t}) + case bool: + vals.boolValues = append(vals.boolValues, BoolValue{Time: p.Time(), Value: t}) + case string: + vals.stringValues = append(vals.stringValues, StringValue{Time: p.Time(), Value: t}) + default: + panic("unsupported type") + } + } + } + + fmt.Println("writing names:", len(names)) + b, err = json.Marshal(names) + if err != nil { + return err + } + + if err := e.replaceCompressedFile("names", b); err != nil { + return err + } + + // flush values by id to either a new file or rewrite the old one + if overwriteNewestFile { + if err := e.rewriteFile(newestDataFile, valuesByID); err != nil { + return err + } else if err := e.rewriteFile(nil, valuesByID); err != nil { + return err + } + } + + // flush each of the old ones + for df, vals := range dataFileToValues { + fmt.Println("writing vals to old file: ", df.f.Name()) + if err := e.rewriteFile(df, vals); err != nil { + return err + } + } + + return nil +} + +func (e *Engine) rewriteFile(oldDF *dataFile, values map[uint64]*valueCollection) error { + // we need the values in sorted order so that we can merge them into the + // new file as we read the old file + ids := make([]uint64, 0, len(values)) + for id, _ := range values { + ids = append(ids, id) + } + + minTime := int64(math.MaxInt64) + maxTime := int64(math.MinInt64) + + // read header of ids to starting positions and times + oldIDToPosition := make(map[uint64]uint32) + if oldDF != nil { + oldIDToPosition = oldDF.IDToPosition() + minTime = oldDF.MinTime() + maxTime = oldDF.MaxTime() + } + for _, v := range values { + if minTime > v.MinTime().UnixNano() { + minTime = v.MinTime().UnixNano() + } + if maxTime < v.MaxTime().UnixNano() { + maxTime = v.MaxTime().UnixNano() + } + } + + // add any ids that are in the file that aren't getting flushed here + for id, _ := range oldIDToPosition { + if _, ok := values[id]; !ok { + ids = append(ids, id) + } + } + + // always write in order by ID + sort.Sort(uint64slice(ids)) + + // TODO: add checkpoint file that indicates if this completed or not + f, err := os.OpenFile(e.nextFileName(), os.O_CREATE|os.O_RDWR, 0666) + if err != nil { + return err + } + + // write the header of the file and keep track of the current file position + currentPosition := uint32(4) + // series count + if _, err := f.Write(u32tob(uint32(len(ids)))); err != nil { + f.Close() + return err + } + // empty min time and max time + currentPosition += 16 + if _, err := f.Write([]byte{0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00}); err != nil { + f.Close() + return nil + } + // write the series ids and empty starting positions + for _, id := range ids { + if _, err := f.Write(append(u64tob(id), []byte{0x00, 0x00, 0x00, 0x00}...)); err != nil { + f.Close() + return err + } + currentPosition += 12 + } + + // now combine the old file data with the new values, keeping track of + // their positions + newPositions := make([]uint32, len(ids)) + buf := make([]byte, DefaultMaxPointsPerBlock*20) + for i, id := range ids { + // mark the position for this ID + newPositions[i] = currentPosition + + newVals := values[id] + + // if this id is only in the file and not in the new values, just copy over from old file + if newVals == nil { + fpos := oldIDToPosition[id] + + // write the blocks until we hit whatever the next id is + for { + fid := btou64(oldDF.mmap[fpos : fpos+8]) + if fid != id { + break + } + length := btou32(oldDF.mmap[fpos+8 : fpos+12]) + if _, err := f.Write(oldDF.mmap[fpos : fpos+12+length]); err != nil { + f.Close() + return err + } + fpos += (12 + length) + currentPosition += (12 + length) + + // make sure we're not at the end of the file + if fpos >= oldDF.size { + break + } + } + + continue + } + + // if the values are not in the file, just write the new ones + fpos, ok := oldIDToPosition[id] + if !ok { + block := newVals.Encode(buf) + if _, err := f.Write(append(u64tob(id), u32tob(uint32(len(block)))...)); err != nil { + f.Close() + return err + } + if _, err := f.Write(block); err != nil { + f.Close() + return err + } + currentPosition += uint32(12 + len(block)) + + continue + } + + // it's in the file and the new values, combine them and write out + for { + fid := btou64(oldDF.mmap[fpos : fpos+8]) + if fid != id { + break + } + length := btou32(oldDF.mmap[fpos+8 : fpos+12]) + block := oldDF.mmap[fpos : fpos+12+length] + fpos += (12 + length) + + // determine if there's a block after this with the same id and get its time + hasFutureBlock := false + nextTime := int64(0) + if fpos < oldDF.size { + nextID := btou64(oldDF.mmap[fpos : fpos+8]) + if nextID == id { + hasFutureBlock = true + nextTime = int64(btou64(oldDF.mmap[fpos+12 : fpos+20])) + } + } + + newBlock, err := newVals.DecodeAndCombine(block, buf[:0], nextTime, hasFutureBlock) + if err != nil { + return err + } + if _, err := f.Write(append(u64tob(id), u32tob(uint32(len(newBlock)))...)); err != nil { + f.Close() + return err + } + if _, err := f.Write(newBlock); err != nil { + f.Close() + return err + } + + currentPosition += uint32(12 + len(newBlock)) + + if fpos >= oldDF.size { + break + } + } + } + + // write out the times and positions + if _, err := f.Seek(4, 0); err != nil { + f.Close() + return err + } + if _, err := f.Write(append(u64tob(uint64(minTime)), u64tob(uint64(maxTime))...)); err != nil { + f.Close() + return err + } + for _, pos := range newPositions { + if _, err := f.Seek(8, 1); err != nil { + f.Close() + return err + } + + if _, err := f.Write(u32tob(pos)); err != nil { + return err + } + } + if err := f.Sync(); err != nil { + return err + } + if _, err := f.Seek(0, 0); err != nil { + f.Close() + return err + } + + // now open it as a memory mapped data file + newDF, err := NewDataFile(f) + if err != nil { + return err + } + + // update the engine to point at the new dataFiles + e.queryLock.Lock() + var files dataFiles + for _, df := range e.files { + if df != oldDF { + files = append(files, df) + } + } + files = append(files, newDF) + sort.Sort(files) + e.files = files + e.queryLock.Unlock() + + // remove the old data file. no need to block returning the write, + // but we need to let any running queries finish before deleting it + if oldDF != nil { + go func(df *dataFile) { + if err := oldDF.Delete(); err != nil { + // TODO: log this error + } + }(oldDF) + } + + return nil +} + +func (e *Engine) nextFileName() string { + e.currentFileID++ + return filepath.Join(e.path, fmt.Sprintf("%07d.%s", e.currentFileID, Format)) +} + +func (e *Engine) readCompressedFile(name string) ([]byte, error) { + f, err := os.OpenFile(filepath.Join(e.path, name), os.O_RDONLY, 0666) + if os.IsNotExist(err) { + return nil, nil + } else if err != nil { + return nil, err + } + b, err := ioutil.ReadAll(f) + if err != nil { + return nil, err + } + + data, err := snappy.Decode(nil, b) + if err != nil { + return nil, err + } + return data, nil +} + +func (e *Engine) replaceCompressedFile(name string, data []byte) error { + tmpName := filepath.Join(e.path, name+"tmp") + f, err := os.OpenFile(tmpName, os.O_CREATE|os.O_RDWR, 0666) + if err != nil { + return err + } + b := snappy.Encode(nil, data) + if _, err := f.Write(b); err != nil { + return err + } + fmt.Println("compressed: ", len(b)) + if err := f.Close(); err != nil { + return err + } + if err := os.Remove(name); err != nil && !os.IsNotExist(err) { + return err + } + return os.Rename(tmpName, filepath.Join(e.path, name)) +} + +// DeleteSeries deletes the series from the engine. +func (e *Engine) DeleteSeries(keys []string) error { + return nil +} + +// DeleteMeasurement deletes a measurement and all related series. +func (e *Engine) DeleteMeasurement(name string, seriesKeys []string) error { + return nil +} + +// SeriesCount returns the number of series buckets on the shard. +func (e *Engine) SeriesCount() (n int, err error) { + return 0, nil +} + +// Begin starts a new transaction on the engine. +func (e *Engine) Begin(writable bool) (tsdb.Tx, error) { + return e, nil +} + +// TODO: make the cursor take a field name +func (e *Engine) Cursor(series string, direction tsdb.Direction) tsdb.Cursor { + measurementName := tsdb.MeasurementFromSeriesKey(series) + codec := e.shard.FieldCodec(measurementName) + if codec == nil { + return &cursor{} + } + field := codec.FieldByName("value") + if field == nil { + panic("pd1 engine only supports one field with name of value") + } + + // TODO: ensure we map the collisions + id := hashSeriesField(seriesFieldKey(series, field.Name)) + return newCursor(id, field.Type, e.copyFilesCollection(), direction) +} + +func (e *Engine) copyFilesCollection() []*dataFile { + e.filesLock.RLock() + defer e.filesLock.RUnlock() + a := make([]*dataFile, len(e.files)) + copy(a, e.files) + return a +} + +func (e *Engine) Size() int64 { return 0 } +func (e *Engine) Commit() error { return nil } +func (e *Engine) Rollback() error { return nil } +func (e *Engine) WriteTo(w io.Writer) (n int64, err error) { return 0, nil } + +func (e *Engine) writeNewFields(measurementFieldsToSave map[string]*tsdb.MeasurementFields) error { + if len(measurementFieldsToSave) == 0 { + return nil + } + + // read in all the previously saved fields + fields, err := e.readFields() + if err != nil { + return err + } + + // add the new ones or overwrite old ones + for name, mf := range measurementFieldsToSave { + fields[name] = mf + } + + return e.writeFields(fields) +} + +func (e *Engine) writeFields(fields map[string]*tsdb.MeasurementFields) error { + // compress and save everything + data, err := json.Marshal(fields) + if err != nil { + return err + } + + fn := e.path + "." + FieldsFileExtension + "tmp" + ff, err := os.OpenFile(fn, os.O_CREATE|os.O_RDWR, 0666) + if err != nil { + return err + } + _, err = ff.Write(snappy.Encode(nil, data)) + if err != nil { + return err + } + if err := ff.Close(); err != nil { + return err + } + fieldsFileName := e.path + "." + FieldsFileExtension + + if _, err := os.Stat(fieldsFileName); !os.IsNotExist(err) { + if err := os.Remove(fieldsFileName); err != nil { + return err + } + } + + return os.Rename(fn, fieldsFileName) +} + +func (e *Engine) readFields() (map[string]*tsdb.MeasurementFields, error) { + fields := make(map[string]*tsdb.MeasurementFields) + + f, err := os.OpenFile(e.path+"."+FieldsFileExtension, os.O_RDONLY, 0666) + if os.IsNotExist(err) { + return fields, nil + } else if err != nil { + return nil, err + } + b, err := ioutil.ReadAll(f) + if err != nil { + return nil, err + } + + data, err := snappy.Decode(nil, b) + if err != nil { + return nil, err + } + + if err := json.Unmarshal(data, &fields); err != nil { + return nil, err + } + + return fields, nil +} + +func (e *Engine) writeNewSeries(seriesToCreate []*tsdb.SeriesCreate) error { + if len(seriesToCreate) == 0 { + return nil + } + + // read in previously saved series + series, err := e.readSeries() + if err != nil { + return err + } + + // add new ones, compress and save + for _, s := range seriesToCreate { + series[s.Series.Key] = s.Series + } + + return e.writeSeries(series) +} + +func (e *Engine) writeSeries(series map[string]*tsdb.Series) error { + data, err := json.Marshal(series) + if err != nil { + return err + } + + fn := e.path + "." + SeriesFileExtension + "tmp" + ff, err := os.OpenFile(fn, os.O_CREATE|os.O_RDWR, 0666) + if err != nil { + return err + } + _, err = ff.Write(snappy.Encode(nil, data)) + if err != nil { + return err + } + if err := ff.Close(); err != nil { + return err + } + seriesFileName := e.path + "." + SeriesFileExtension + + if _, err := os.Stat(seriesFileName); !os.IsNotExist(err) { + if err := os.Remove(seriesFileName); err != nil && err != os.ErrNotExist { + return err + } + } + + return os.Rename(fn, seriesFileName) +} + +func (e *Engine) readSeries() (map[string]*tsdb.Series, error) { + series := make(map[string]*tsdb.Series) + + f, err := os.OpenFile(e.path+"."+SeriesFileExtension, os.O_RDONLY, 0666) + if os.IsNotExist(err) { + return series, nil + } else if err != nil { + return nil, err + } + defer f.Close() + b, err := ioutil.ReadAll(f) + if err != nil { + return nil, err + } + + data, err := snappy.Decode(nil, b) + if err != nil { + return nil, err + } + + if err := json.Unmarshal(data, &series); err != nil { + return nil, err + } + + return series, nil +} + +type valueCollection struct { + floatValues []FloatValue + boolValues []BoolValue + intValues []Int64Value + stringValues []StringValue +} + +func (v *valueCollection) MinTime() time.Time { + if v.floatValues != nil { + return v.floatValues[0].Time + } else if v.boolValues != nil { + return v.boolValues[0].Time + } else if v.intValues != nil { + return v.intValues[0].Time + } else if v.stringValues != nil { + return v.stringValues[0].Time + } + + return time.Unix(0, 0) +} + +func (v *valueCollection) MaxTime() time.Time { + if v.floatValues != nil { + return v.floatValues[len(v.floatValues)-1].Time + } else if v.boolValues != nil { + return v.boolValues[len(v.boolValues)-1].Time + } else if v.intValues != nil { + return v.intValues[len(v.intValues)-1].Time + } else if v.stringValues != nil { + return v.stringValues[len(v.stringValues)-1].Time + } + + return time.Unix(0, 0) +} + +func (v *valueCollection) Encode(buf []byte) []byte { + if v.floatValues != nil { + return EncodeFloatBlock(buf, v.floatValues) + } else if v.boolValues != nil { + return EncodeBoolBlock(buf, v.boolValues) + } else if v.intValues != nil { + return EncodeInt64Block(buf, v.intValues) + } else if v.stringValues != nil { + return EncodeStringBlock(buf, v.stringValues) + } + + return nil +} + +// DecodeAndCombine take an encoded block from a file, decodes it and interleaves the file +// values with the values in this collection. nextTime and hasNext refer to if the file +// has future encoded blocks so that this method can know how much of its values can be +// combined and output in the resulting encoded block. +func (v *valueCollection) DecodeAndCombine(block, buf []byte, nextTime int64, hasFutureBlock bool) ([]byte, error) { + if v.floatValues != nil { + values, err := DecodeFloatBlock(block) + if err != nil { + return nil, err + } + + if hasFutureBlock { + for i, val := range v.floatValues { + if val.Time.UnixNano() > nextTime { + values = append(values, v.floatValues[:i]...) + v.floatValues = v.floatValues[i:] + } + } + } else { + values = append(values, v.floatValues...) + } + sort.Sort(FloatValues(values)) + // TODO: deduplicate values + + if len(values) > DefaultMaxPointsPerBlock { + v.floatValues = values[DefaultMaxPointsPerBlock:] + values = values[:DefaultMaxPointsPerBlock] + } + + return EncodeFloatBlock(buf, values), nil + } else if v.boolValues != nil { + // TODO: wire up the other value types + return nil, fmt.Errorf("not implemented") + } else if v.intValues != nil { + return nil, fmt.Errorf("not implemented") + } else if v.stringValues != nil { + return nil, fmt.Errorf("not implemented") + } + + return nil, nil +} + +type dataFile struct { + f *os.File + mu sync.RWMutex + size uint32 + mmap []byte +} + +// byte size constants for the data file +const ( + seriesCountSize = 4 + timeSize = 8 + fileHeaderSize = seriesCountSize + (2 * timeSize) + seriesIDSize = 8 + seriesPositionSize = 4 + seriesHeaderSize = seriesIDSize + seriesPositionSize +) + +func NewDataFile(f *os.File) (*dataFile, error) { + fInfo, err := f.Stat() + if err != nil { + return nil, err + } + mmap, err := syscall.Mmap(int(f.Fd()), 0, int(fInfo.Size()), syscall.PROT_READ, syscall.MAP_SHARED|MAP_POPULATE) + if err != nil { + f.Close() + return nil, err + } + + return &dataFile{ + f: f, + mmap: mmap, + size: uint32(fInfo.Size()), + }, nil +} + +func (d *dataFile) Close() error { + d.mu.Lock() + defer d.mu.Unlock() + return d.close() +} + +func (d *dataFile) Delete() error { + d.mu.Lock() + defer d.mu.Unlock() + if err := d.close(); err != nil { + return err + } + return os.Remove(d.f.Name()) +} + +func (d *dataFile) close() error { + if d.mmap == nil { + return nil + } + err := syscall.Munmap(d.mmap) + if err != nil { + return err + } + + d.mmap = nil + return d.f.Close() +} + +func (d *dataFile) MinTime() int64 { + return int64(btou64(d.mmap[4:12])) +} + +func (d *dataFile) MaxTime() int64 { + return int64(btou64(d.mmap[12:20])) +} + +func (d *dataFile) SeriesCount() uint32 { + return btou32(d.mmap[:4]) +} + +func (d *dataFile) IDToPosition() map[uint64]uint32 { + count := int(d.SeriesCount()) + m := make(map[uint64]uint32) + + for i := 0; i < count; i++ { + offset := 20 + (i * 12) + id := btou64(d.mmap[offset : offset+8]) + m[id] = btou32(d.mmap[offset+8 : offset+12]) + } + + return m +} + +// StartingPositionForID returns the position in the file of the +// first block for the given ID. If zero is returned the ID doesn't +// have any data in this file. +func (d *dataFile) StartingPositionForID(id uint64) uint32 { + seriesCount := d.SeriesCount() + + min := 0 + max := seriesCount + // // set the minimum position to the first after the file header + // posMin := fileHeaderSize + + // // set the maximum position to the end of the series header + // posMax := fileHeaderSize + (seriesCount * seriesHeaderSize) + + for min < max { + mid := (max-min)/2 + min + + offset := mid*seriesHeaderSize + fileHeaderSize + checkID := btou64(d.mmap[offset:8]) + + if checkID == id { + return btou32(d.mmap[offset+8 : offset+12]) + } else if checkID < id { + min = mid + 1 + } + max = mid + } + return uint32(0) +} + +type dataFiles []*dataFile + +func (a dataFiles) Len() int { return len(a) } +func (a dataFiles) Swap(i, j int) { a[i], a[j] = a[j], a[i] } +func (a dataFiles) Less(i, j int) bool { return a[i].MinTime() < a[j].MinTime() } + +type cursor struct { + id uint64 + dataType influxql.DataType + f *dataFile + dataFilePos int + pos uint32 + vals []FloatValues + + direction tsdb.Direction + + // time acending list of data files + files []*dataFile +} + +func newCursor(id uint64, dataType influxql.DataType, files []*dataFile, direction tsdb.Direction) *cursor { + return &cursor{ + ids: id, + types: dataType, + direction: direction, + files: files, + } +} + +func (c *cursor) Seek(seek []byte) (key, value []byte) { return nil, nil } + +func (c *cursor) Next() (key, value []byte) { + if vals == nil { + // loop until we find a file with some data + for dataFilePos < len(c.files) { + f = c.files[c.dataFilePos] + c.dataFilePos++ + + // startPosition := f + } + } + return nil, nil +} + +func (c *cursor) next(id uint64) (key, value []byte) + +func (c *cursor) Direction() tsdb.Direction { return c.direction } + +// u64tob converts a uint64 into an 8-byte slice. +func u64tob(v uint64) []byte { + b := make([]byte, 8) + binary.BigEndian.PutUint64(b, v) + return b +} + +func btou64(b []byte) uint64 { + return binary.BigEndian.Uint64(b) +} + +func u32tob(v uint32) []byte { + b := make([]byte, 4) + binary.BigEndian.PutUint32(b, v) + return b +} + +func btou32(b []byte) uint32 { + return uint32(binary.BigEndian.Uint32(b)) +} + +func hashSeriesField(key string) uint64 { + h := fnv.New64a() + h.Write([]byte(key)) + return h.Sum64() +} + +// seriesFieldKey combine a series key and field name for a unique string to be hashed to a numeric ID +func seriesFieldKey(seriesKey, field string) string { + return seriesKey + "#" + field +} + +type uint64slice []uint64 + +func (a uint64slice) Len() int { return len(a) } +func (a uint64slice) Swap(i, j int) { a[i], a[j] = a[j], a[i] } +func (a uint64slice) Less(i, j int) bool { return a[i] < a[j] } + +/* TODO: REMOVE THIS STUFF */ +func (e *Engine) pointsToBlocks(points [][]byte) []byte { + var b bytes.Buffer + block := make([]byte, 0) + for _, p := range points { + block = append(block, p[0:8]...) + block = append(block, u32tob(uint32(len(p)-8))...) + block = append(block, p[8:]...) + if len(block) > DefaultBlockSize { + e.writeBlockToBuffer(block, &b) + block = make([]byte, 0) + } + } + if len(block) > 0 { + e.writeBlockToBuffer(block, &b) + } + + return b.Bytes() +} + +func (e *Engine) writeBlockToBuffer(block []byte, b *bytes.Buffer) { + // write the min time + if _, err := b.Write(block[0:8]); err != nil { + panic(err) + } + + // write the length of the compressed data + data := snappy.Encode(nil, block) + if _, err := b.Write(u32tob(uint32(len(data)))); err != nil { + panic(err) + } + + // write the compressed data + if _, err := b.Write(data); err != nil { + panic(err) + } +} + +func (e *Engine) readPointsFromFile(f *os.File) (map[uint64][][]byte, error) { + buf := make([]byte, 8) + if _, err := io.ReadFull(f, buf); err != nil { + return nil, err + } + seriesCount := btou64(buf) + positions := make([]uint64, seriesCount, seriesCount) + ids := make([]uint64, seriesCount, seriesCount) + + // read the series index file header + position := uint64(8) + for i := 0; uint64(i) < seriesCount; i++ { + // read the id of the series + if _, err := io.ReadFull(f, buf); err != nil { + return nil, err + } + ids[i] = btou64(buf) + + // read the min time and ignore + if _, err := io.ReadFull(f, buf); err != nil { + return nil, err + } + if _, err := io.ReadFull(f, buf); err != nil { + return nil, err + } + + // read the starting position of this id + if _, err := io.ReadFull(f, buf); err != nil { + return nil, err + } + positions[i] = btou64(buf) + position += 32 + } + + if position != positions[0] { + panic("we aren't at the right place") + } + + // read the raw data + seriesData := make(map[uint64][][]byte) + compressedBuff := make([]byte, DefaultBlockSize) + seriesPosition := 0 + for { + // read the min time and ignore + if _, err := io.ReadFull(f, buf); err == io.EOF { + break + } else if err != nil { + return nil, err + } + + // read the length of the compressed block + if _, err := io.ReadFull(f, buf[:4]); err != nil { + return nil, err + } + length := btou32(buf) + + if length > uint32(len(compressedBuff)) { + compressedBuff = make([]byte, length) + } + if _, err := io.ReadFull(f, compressedBuff[:length]); err != nil { + return nil, err + } + + data, err := snappy.Decode(nil, compressedBuff[:length]) + if err != nil { + return nil, err + } + id := ids[seriesPosition] + seriesData[id] = append(seriesData[id], e.pointsFromDataBlock(data)...) + position += uint64(12 + length) + + if seriesPosition+1 >= len(positions) { + continue + } + if positions[seriesPosition+1] == position { + seriesPosition += 1 + } + } + + return seriesData, nil +} + +func (e *Engine) pointsFromDataBlock(data []byte) [][]byte { + a := make([][]byte, 0) + for { + length := entryDataSize(data) + p := append(data[:8], data[12:12+length]...) + a = append(a, p) + data = data[12+length:] + if len(data) == 0 { + break + } + } + return a +} + +func entryDataSize(v []byte) int { return int(binary.BigEndian.Uint32(v[8:12])) } + +func (e *Engine) lastFileAndNewFile() (*os.File, *os.File, error) { + files, err := filepath.Glob(filepath.Join(e.path, fmt.Sprintf("*.%s", Format))) + if err != nil { + return nil, nil, err + } + + if len(files) == 0 { + newFile, err := os.OpenFile(filepath.Join(e.path, fmt.Sprintf("%07d.%s", 1, Format)), os.O_CREATE|os.O_RDWR, 0666) + if err != nil { + return nil, nil, err + } + return nil, newFile, nil + } + + oldFile, err := os.OpenFile(files[len(files)-1], os.O_RDONLY, 0666) + if err != nil { + return nil, nil, err + } + + info, err := oldFile.Stat() + if err != nil { + _ = oldFile.Close() + return nil, nil, err + } + + num := strings.Split(filepath.Base(files[len(files)-1]), ".")[0] + n, err := strconv.ParseUint(num, 10, 32) + if err != nil { + return nil, nil, err + } + newFile, err := os.OpenFile(filepath.Join(e.path, fmt.Sprintf("%07d.%s", n+1, Format)), os.O_CREATE|os.O_RDWR, 0666) + if err != nil { + return nil, nil, err + } + if info.Size() >= DefaultMaxFileSize { + oldFile.Close() + return nil, newFile, nil + } + return oldFile, newFile, nil +} diff --git a/tsdb/engine/pd1/pd1_test.go b/tsdb/engine/pd1/pd1_test.go new file mode 100644 index 00000000000..48cdff79293 --- /dev/null +++ b/tsdb/engine/pd1/pd1_test.go @@ -0,0 +1,196 @@ +package pd1_test + +import ( + "encoding/binary" + "fmt" + "io/ioutil" + "os" + "reflect" + "testing" + "time" + + "github.com/influxdb/influxdb/influxql" + "github.com/influxdb/influxdb/tsdb" + "github.com/influxdb/influxdb/tsdb/engine/pd1" +) + +func TestEngine_WriteAndReadFloats(t *testing.T) { + e := OpenDefaultEngine() + defer e.Close() + + codec := tsdb.NewFieldCodec(map[string]*tsdb.Field{ + "value": { + ID: uint8(1), + Name: "value", + Type: influxql.Float, + }, + }) + + p1 := parsePoint("cpu,host=A value=1.1 1000000000", codec) + p2 := parsePoint("cpu,host=B value=1.2 1000000000", codec) + p3 := parsePoint("cpu,host=A value=2.1 2000000000", codec) + p4 := parsePoint("cpu,host=B value=2.2 2000000000", codec) + + if err := e.WriteAndCompact([]tsdb.Point{p1, p2, p3}, nil, nil); err != nil { + t.Fatalf("failed to write points: %s", err.Error()) + } + + verify := func() { + c := e.Cursor("cpu,host=A", tsdb.Forward) + k, v := c.Next() + if btou64(k) != uint64(p1.UnixNano()) { + t.Fatalf("p1 time wrong:\n\texp:%d\n\tgot:%d\n", p1.UnixNano(), btou64(k)) + } + if !reflect.DeepEqual(v, p1.Data()) { + t.Fatal("p1 data not equal") + } + k, v = c.Next() + if btou64(k) != uint64(p3.UnixNano()) { + t.Fatalf("p3 time wrong:\n\texp:%d\n\tgot:%d\n", p3.UnixNano(), btou64(k)) + } + if !reflect.DeepEqual(v, p3.Data()) { + t.Fatal("p3 data not equal") + } + k, v = c.Next() + if k != nil { + t.Fatal("expected nil") + } + + c = e.Cursor("cpu,host=B", tsdb.Forward) + k, v = c.Next() + if btou64(k) != uint64(p2.UnixNano()) { + t.Fatalf("p2 time wrong:\n\texp:%d\n\tgot:%d\n", p2.UnixNano(), btou64(k)) + } + if !reflect.DeepEqual(v, p2.Data()) { + t.Fatal("p2 data not equal") + } + k, v = c.Next() + if k != nil { + t.Fatal("expected nil") + } + } + verify() + + if err := e.WriteAndCompact([]tsdb.Point{p4}, nil, nil); err != nil { + t.Fatalf("failed to write points: %s", err.Error()) + } + verify() + + c := e.Cursor("cpu,host=B", tsdb.Forward) + k, v := c.Seek(u64tob(2000000000)) + if btou64(k) != uint64(p4.UnixNano()) { + t.Fatalf("p4 time wrong:\n\texp:%d\n\tgot:%d\n", p4.UnixNano(), btou64(k)) + } + if !reflect.DeepEqual(v, p4.Data()) { + t.Fatal("p4 data not equal") + } +} + +func TestEngine_WriteIndexWithCollision(t *testing.T) { +} + +func TestEngine_WriteIndexBenchmarkNames(t *testing.T) { + t.Skip("whatevs") + + e := OpenDefaultEngine() + defer e.Close() + + codec := tsdb.NewFieldCodec(map[string]*tsdb.Field{ + "value": { + ID: uint8(1), + Name: "value", + Type: influxql.Float, + }, + }) + + var points []tsdb.Point + for i := 0; i < 100000; i++ { + points = append(points, parsePoint(fmt.Sprintf("cpu%d value=22.1", i), codec)) + } + + st := time.Now() + if err := e.WriteAndCompact(points, nil, nil); err != nil { + t.Fatalf("failed to write points: %s", err.Error()) + } + fmt.Println("took: ", time.Since(st)) + + st = time.Now() + if err := e.WriteAndCompact(points, nil, nil); err != nil { + t.Fatalf("failed to write points: %s", err.Error()) + } + fmt.Println("took: ", time.Since(st)) +} + +// Engine represents a test wrapper for pd1.Engine. +type Engine struct { + *pd1.Engine +} + +// NewEngine returns a new instance of Engine. +func NewEngine(opt tsdb.EngineOptions) *Engine { + dir, err := ioutil.TempDir("", "pd1-test") + if err != nil { + panic("couldn't get temp dir") + } + + // Create test wrapper and attach mocks. + e := &Engine{ + Engine: pd1.NewEngine(dir, dir, opt).(*pd1.Engine), + } + + return e +} + +// OpenEngine returns an opened instance of Engine. Panic on error. +func OpenEngine(opt tsdb.EngineOptions) *Engine { + e := NewEngine(opt) + if err := e.Open(); err != nil { + panic(err) + } + return e +} + +// OpenDefaultEngine returns an open Engine with default options. +func OpenDefaultEngine() *Engine { return OpenEngine(tsdb.NewEngineOptions()) } + +// Close closes the engine and removes all data. +func (e *Engine) Close() error { + e.Engine.Close() + os.RemoveAll(e.Path()) + return nil +} + +func parsePoints(buf string, codec *tsdb.FieldCodec) []tsdb.Point { + points, err := tsdb.ParsePointsString(buf) + if err != nil { + panic(fmt.Sprintf("couldn't parse points: %s", err.Error())) + } + for _, p := range points { + b, err := codec.EncodeFields(p.Fields()) + if err != nil { + panic(fmt.Sprintf("couldn't encode fields: %s", err.Error())) + } + p.SetData(b) + } + return points +} + +func parsePoint(buf string, codec *tsdb.FieldCodec) tsdb.Point { + return parsePoints(buf, codec)[0] +} + +func inttob(v int) []byte { + b := make([]byte, 8) + binary.BigEndian.PutUint64(b, uint64(v)) + return b +} + +func btou64(b []byte) uint64 { + return binary.BigEndian.Uint64(b) +} + +func u64tob(v uint64) []byte { + b := make([]byte, 8) + binary.BigEndian.PutUint64(b, v) + return b +} diff --git a/tsdb/shard.go b/tsdb/shard.go index e355cc661de..a14822f367c 100644 --- a/tsdb/shard.go +++ b/tsdb/shard.go @@ -121,7 +121,7 @@ func (s *Shard) Open() error { } // Load metadata index. - if err := s.engine.LoadMetadataIndex(s.index, s.measurementFields); err != nil { + if err := s.engine.LoadMetadataIndex(s, s.index, s.measurementFields); err != nil { return fmt.Errorf("load metadata index: %s", err) } @@ -741,15 +741,22 @@ func (f *FieldCodec) DecodeByID(targetID uint8, b []byte) (interface{}, error) { // DecodeByName scans a byte slice for a field with the given name, converts it to its // expected type, and return that value. func (f *FieldCodec) DecodeByName(name string, b []byte) (interface{}, error) { - fi := f.fieldByName(name) + fi := f.FieldByName(name) if fi == nil { return 0, ErrFieldNotFound } return f.DecodeByID(fi.ID, b) } +func (f *FieldCodec) Fields() (a []*Field) { + for _, f := range f.fieldsByID { + a = append(a, f) + } + return +} + // FieldByName returns the field by its name. It will return a nil if not found -func (f *FieldCodec) fieldByName(name string) *Field { +func (f *FieldCodec) FieldByName(name string) *Field { return f.fieldsByName[name] } From 2ba032b7a8fff67e9efab4d1cbd98c3e94708b85 Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Mon, 7 Sep 2015 15:56:21 -0700 Subject: [PATCH 072/139] WIP: finish basics of PD1. IT WORKS! (kind of) --- cmd/influx_stress/influx_stress.go | 1 + tsdb/engine.go | 3 + tsdb/engine/pd1/encoding.go | 22 ++- tsdb/engine/pd1/pd1.go | 248 +++++++++++++++++++++++------ tsdb/engine/pd1/pd1_test.go | 118 +++++++++----- tsdb/shard.go | 58 +++---- 6 files changed, 325 insertions(+), 125 deletions(-) diff --git a/cmd/influx_stress/influx_stress.go b/cmd/influx_stress/influx_stress.go index 660007125a9..bd2f1b4a2ad 100644 --- a/cmd/influx_stress/influx_stress.go +++ b/cmd/influx_stress/influx_stress.go @@ -3,6 +3,7 @@ package main import ( "flag" "fmt" + "net/url" "runtime" "sort" "time" diff --git a/tsdb/engine.go b/tsdb/engine.go index d2862b54863..b0e9254d95a 100644 --- a/tsdb/engine.go +++ b/tsdb/engine.go @@ -18,6 +18,9 @@ var ( ErrFormatNotFound = errors.New("format not found") ) +// DefaultEngine is the default engine used by the shard when initializing. +const DefaultEngine = "pd1" + // Engine represents a swappable storage engine for the shard. type Engine interface { Open() error diff --git a/tsdb/engine/pd1/encoding.go b/tsdb/engine/pd1/encoding.go index 285c83f8c0b..60f72766d83 100644 --- a/tsdb/engine/pd1/encoding.go +++ b/tsdb/engine/pd1/encoding.go @@ -1,16 +1,34 @@ package pd1 import ( + "encoding/binary" + "math" "time" "github.com/dgryski/go-tsz" ) +type Value interface { + TimeBytes() []byte + ValueBytes() []byte + Time() time.Time +} + type FloatValue struct { Time time.Time Value float64 } +func (f *FloatValue) TimeBytes() []byte { + return u64tob(uint64(f.Time.UnixNano())) +} + +func (f *FloatValue) ValueBytes() []byte { + buf := make([]byte, 8) + binary.BigEndian.PutUint64(buf, math.Float64bits(f.Value)) + return buf +} + type FloatValues []FloatValue func (a FloatValues) Len() int { return len(a) } @@ -24,11 +42,11 @@ func EncodeFloatBlock(buf []byte, values []FloatValue) []byte { s.Push(uint32(v.Time.Unix()), v.Value) } s.Finish() - return s.Bytes() + return append(u64tob(uint64(values[0].Time.UnixNano())), s.Bytes()...) } func DecodeFloatBlock(block []byte) ([]FloatValue, error) { - iter, _ := tsz.NewIterator(block) + iter, _ := tsz.NewIterator(block[8:]) a := make([]FloatValue, 0) for iter.Next() { t, f := iter.Values() diff --git a/tsdb/engine/pd1/pd1.go b/tsdb/engine/pd1/pd1.go index d94a7d7795c..4d3c752ae2e 100644 --- a/tsdb/engine/pd1/pd1.go +++ b/tsdb/engine/pd1/pd1.go @@ -55,9 +55,9 @@ const ( // DefaultBlockSize is the default size of uncompressed points blocks. DefaultBlockSize = 512 * 1024 // 512KB - DefaultMaxFileSize = 50 * 1024 * 1024 // 50MB + DefaultMaxFileSize = 5 * 1024 * 1024 // 5MB - DefaultMaxPointsPerBlock = 5000 + DefaultMaxPointsPerBlock = 1000 // MAP_POPULATE is for the mmap syscall. For some reason this isn't defined in golang's syscall MAP_POPULATE = 0x8000 @@ -71,12 +71,15 @@ type Engine struct { mu sync.Mutex path string - shard *tsdb.Shard - // HashSeriesField is a function that takes a series key and a field name // and returns a hash identifier. It's not guaranteed to be unique. HashSeriesField func(key string) uint64 + // Shard is an interface that can pull back field type information based on measurement name + Shard interface { + FieldCodec(measurementName string) *tsdb.FieldCodec + } + filesLock sync.RWMutex files dataFiles currentFileID int @@ -108,11 +111,34 @@ func (e *Engine) Open() error { // TODO: clean up previous names write // TODO: clean up any data files that didn't get cleaned up + files, err := filepath.Glob(filepath.Join(e.path, fmt.Sprintf("*.%s", Format))) + if err != nil { + return err + } + for _, fn := range files { + f, err := os.OpenFile(fn, os.O_RDONLY, 0666) + if err != nil { + return fmt.Errorf("error opening file %s: %s", fn, err.Error()) + } + df, err := NewDataFile(f) + if err != nil { + return fmt.Errorf("error opening memory map for file %s: %s", fn, err.Error()) + } + e.files = append(e.files, df) + } + sort.Sort(e.files) + return nil } // Close closes the engine. func (e *Engine) Close() error { + e.queryLock.Lock() + defer e.queryLock.Unlock() + + for _, df := range e.files { + _ = df.Close() + } return nil } @@ -121,7 +147,7 @@ func (e *Engine) SetLogOutput(w io.Writer) {} // LoadMetadataIndex loads the shard metadata into memory. func (e *Engine) LoadMetadataIndex(shard *tsdb.Shard, index *tsdb.DatabaseIndex, measurementFields map[string]*tsdb.MeasurementFields) error { - e.shard = shard + e.Shard = shard // TODO: write the metadata from the WAL // Load measurement metadata @@ -165,7 +191,7 @@ func (e *Engine) LoadMetadataIndex(shard *tsdb.Shard, index *tsdb.DatabaseIndex, func (e *Engine) WritePoints(points []tsdb.Point, measurementFieldsToSave map[string]*tsdb.MeasurementFields, seriesToCreate []*tsdb.SeriesCreate) error { // TODO: Write points to the WAL - return nil + return e.WriteAndCompact(points, measurementFieldsToSave, seriesToCreate) } func (e *Engine) WriteAndCompact(points []tsdb.Point, measurementFieldsToSave map[string]*tsdb.MeasurementFields, seriesToCreate []*tsdb.SeriesCreate) error { @@ -200,8 +226,6 @@ func (e *Engine) WriteAndCompact(points []tsdb.Point, measurementFieldsToSave ma } } - fmt.Println("read names: ", len(names), len(ids)) - // these are values that are newer than anything stored in the shard valuesByID := make(map[uint64]*valueCollection) // map the points to the data file they belong to if they overlap @@ -288,7 +312,6 @@ func (e *Engine) WriteAndCompact(points []tsdb.Point, measurementFieldsToSave ma } } - fmt.Println("writing names:", len(names)) b, err = json.Marshal(names) if err != nil { return err @@ -302,14 +325,13 @@ func (e *Engine) WriteAndCompact(points []tsdb.Point, measurementFieldsToSave ma if overwriteNewestFile { if err := e.rewriteFile(newestDataFile, valuesByID); err != nil { return err - } else if err := e.rewriteFile(nil, valuesByID); err != nil { - return err } + } else if err := e.rewriteFile(nil, valuesByID); err != nil { + return err } // flush each of the old ones for df, vals := range dataFileToValues { - fmt.Println("writing vals to old file: ", df.f.Name()) if err := e.rewriteFile(df, vals); err != nil { return err } @@ -374,6 +396,7 @@ func (e *Engine) rewriteFile(oldDF *dataFile, values map[uint64]*valueCollection f.Close() return nil } + // write the series ids and empty starting positions for _, id := range ids { if _, err := f.Write(append(u64tob(id), []byte{0x00, 0x00, 0x00, 0x00}...)); err != nil { @@ -423,6 +446,7 @@ func (e *Engine) rewriteFile(oldDF *dataFile, values map[uint64]*valueCollection // if the values are not in the file, just write the new ones fpos, ok := oldIDToPosition[id] if !ok { + // TODO: ensure we encode only the amount in a block block := newVals.Encode(buf) if _, err := f.Write(append(u64tob(id), u32tob(uint32(len(block)))...)); err != nil { f.Close() @@ -444,7 +468,7 @@ func (e *Engine) rewriteFile(oldDF *dataFile, values map[uint64]*valueCollection break } length := btou32(oldDF.mmap[fpos+8 : fpos+12]) - block := oldDF.mmap[fpos : fpos+12+length] + block := oldDF.mmap[fpos+12 : fpos+12+length] fpos += (12 + length) // determine if there's a block after this with the same id and get its time @@ -477,6 +501,21 @@ func (e *Engine) rewriteFile(oldDF *dataFile, values map[uint64]*valueCollection break } } + + // TODO: ensure we encode only the amount in a block, refactor this wil line 450 into func + if len(newVals.floatValues) > 0 { + // TODO: ensure we encode only the amount in a block + block := newVals.Encode(buf) + if _, err := f.Write(append(u64tob(id), u32tob(uint32(len(block)))...)); err != nil { + f.Close() + return err + } + if _, err := f.Write(block); err != nil { + f.Close() + return err + } + currentPosition += uint32(12 + len(block)) + } } // write out the times and positions @@ -572,7 +611,6 @@ func (e *Engine) replaceCompressedFile(name string, data []byte) error { if _, err := f.Write(b); err != nil { return err } - fmt.Println("compressed: ", len(b)) if err := f.Close(); err != nil { return err } @@ -605,7 +643,7 @@ func (e *Engine) Begin(writable bool) (tsdb.Tx, error) { // TODO: make the cursor take a field name func (e *Engine) Cursor(series string, direction tsdb.Direction) tsdb.Cursor { measurementName := tsdb.MeasurementFromSeriesKey(series) - codec := e.shard.FieldCodec(measurementName) + codec := e.Shard.FieldCodec(measurementName) if codec == nil { return &cursor{} } @@ -658,7 +696,7 @@ func (e *Engine) writeFields(fields map[string]*tsdb.MeasurementFields) error { return err } - fn := e.path + "." + FieldsFileExtension + "tmp" + fn := filepath.Join(e.path, FieldsFileExtension+"tmp") ff, err := os.OpenFile(fn, os.O_CREATE|os.O_RDWR, 0666) if err != nil { return err @@ -670,7 +708,7 @@ func (e *Engine) writeFields(fields map[string]*tsdb.MeasurementFields) error { if err := ff.Close(); err != nil { return err } - fieldsFileName := e.path + "." + FieldsFileExtension + fieldsFileName := filepath.Join(e.path, FieldsFileExtension) if _, err := os.Stat(fieldsFileName); !os.IsNotExist(err) { if err := os.Remove(fieldsFileName); err != nil { @@ -684,7 +722,7 @@ func (e *Engine) writeFields(fields map[string]*tsdb.MeasurementFields) error { func (e *Engine) readFields() (map[string]*tsdb.MeasurementFields, error) { fields := make(map[string]*tsdb.MeasurementFields) - f, err := os.OpenFile(e.path+"."+FieldsFileExtension, os.O_RDONLY, 0666) + f, err := os.OpenFile(filepath.Join(e.path, FieldsFileExtension), os.O_RDONLY, 0666) if os.IsNotExist(err) { return fields, nil } else if err != nil { @@ -732,7 +770,7 @@ func (e *Engine) writeSeries(series map[string]*tsdb.Series) error { return err } - fn := e.path + "." + SeriesFileExtension + "tmp" + fn := filepath.Join(e.path, SeriesFileExtension+"tmp") ff, err := os.OpenFile(fn, os.O_CREATE|os.O_RDWR, 0666) if err != nil { return err @@ -744,7 +782,7 @@ func (e *Engine) writeSeries(series map[string]*tsdb.Series) error { if err := ff.Close(); err != nil { return err } - seriesFileName := e.path + "." + SeriesFileExtension + seriesFileName := filepath.Join(e.path, SeriesFileExtension) if _, err := os.Stat(seriesFileName); !os.IsNotExist(err) { if err := os.Remove(seriesFileName); err != nil && err != os.ErrNotExist { @@ -758,7 +796,7 @@ func (e *Engine) writeSeries(series map[string]*tsdb.Series) error { func (e *Engine) readSeries() (map[string]*tsdb.Series, error) { series := make(map[string]*tsdb.Series) - f, err := os.OpenFile(e.path+"."+SeriesFileExtension, os.O_RDONLY, 0666) + f, err := os.OpenFile(filepath.Join(e.path, SeriesFileExtension), os.O_RDONLY, 0666) if os.IsNotExist(err) { return series, nil } else if err != nil { @@ -843,14 +881,15 @@ func (v *valueCollection) DecodeAndCombine(block, buf []byte, nextTime int64, ha } if hasFutureBlock { - for i, val := range v.floatValues { - if val.Time.UnixNano() > nextTime { - values = append(values, v.floatValues[:i]...) - v.floatValues = v.floatValues[i:] - } - } + // take all values that have times less than the future block and update the vals array + pos := sort.Search(len(v.floatValues), func(i int) bool { + return v.floatValues[i].Time.UnixNano() >= nextTime + }) + values = append(values, v.floatValues[:pos]...) + v.floatValues = v.floatValues[pos:] } else { values = append(values, v.floatValues...) + v.floatValues = nil } sort.Sort(FloatValues(values)) // TODO: deduplicate values @@ -955,7 +994,8 @@ func (d *dataFile) IDToPosition() map[uint64]uint32 { for i := 0; i < count; i++ { offset := 20 + (i * 12) id := btou64(d.mmap[offset : offset+8]) - m[id] = btou32(d.mmap[offset+8 : offset+12]) + pos := btou32(d.mmap[offset+8 : offset+12]) + m[id] = pos } return m @@ -968,26 +1008,23 @@ func (d *dataFile) StartingPositionForID(id uint64) uint32 { seriesCount := d.SeriesCount() min := 0 - max := seriesCount - // // set the minimum position to the first after the file header - // posMin := fileHeaderSize - - // // set the maximum position to the end of the series header - // posMax := fileHeaderSize + (seriesCount * seriesHeaderSize) + max := int(seriesCount) for min < max { mid := (max-min)/2 + min offset := mid*seriesHeaderSize + fileHeaderSize - checkID := btou64(d.mmap[offset:8]) + checkID := btou64(d.mmap[offset : offset+8]) if checkID == id { return btou32(d.mmap[offset+8 : offset+12]) } else if checkID < id { min = mid + 1 + } else { + max = mid } - max = mid } + return uint32(0) } @@ -998,12 +1035,12 @@ func (a dataFiles) Swap(i, j int) { a[i], a[j] = a[j], a[i] } func (a dataFiles) Less(i, j int) bool { return a[i].MinTime() < a[j].MinTime() } type cursor struct { - id uint64 - dataType influxql.DataType - f *dataFile - dataFilePos int - pos uint32 - vals []FloatValues + id uint64 + dataType influxql.DataType + f *dataFile + filesPos int // the index in the files slice we're looking at + pos uint32 + vals FloatValues direction tsdb.Direction @@ -1013,29 +1050,136 @@ type cursor struct { func newCursor(id uint64, dataType influxql.DataType, files []*dataFile, direction tsdb.Direction) *cursor { return &cursor{ - ids: id, - types: dataType, + id: id, + dataType: dataType, direction: direction, files: files, } } -func (c *cursor) Seek(seek []byte) (key, value []byte) { return nil, nil } +func (c *cursor) Seek(seek []byte) (key, value []byte) { + t := int64(btou64(seek)) + + if t < c.files[0].MinTime() { + c.filesPos = 0 + c.f = c.files[0] + } else { + for i, f := range c.files { + if t >= f.MinTime() && t <= f.MaxTime() { + c.filesPos = i + c.f = f + break + } + } + } + + if c.f == nil { + return nil, nil + } + + // TODO: make this for the reverse direction cursor + + // now find the spot in the file we need to go + for { + pos := c.f.StartingPositionForID(c.id) + + // if this id isn't in this file, move to next one or return + if pos == 0 { + c.filesPos++ + if c.filesPos >= len(c.files) { + return nil, nil + } + c.f = c.files[c.filesPos] + continue + } + + // seek to the block and values we're looking for + for { + // if the time is between this block and the next, + // decode this block and go, otherwise seek to next block + length := btou32(c.f.mmap[pos+8 : pos+12]) + + // if the next block has a time less than what we're seeking to, + // skip decoding this block and continue on + nextBlockPos := pos + 12 + length + if nextBlockPos < c.f.size { + nextBlockID := btou64(c.f.mmap[nextBlockPos : nextBlockPos+8]) + if nextBlockID == c.id { + nextBlockTime := int64(btou64(c.f.mmap[nextBlockPos+12 : nextBlockPos+20])) + if nextBlockTime <= t { + pos = nextBlockPos + continue + } + } + } + + // it must be in this block or not at all + tb, vb := c.decodeBlockAndGetValues(pos) + if int64(btou64(tb)) >= t { + return tb, vb + } + + // wasn't in the first value popped out of the block, check the rest + for i, v := range c.vals { + if v.Time.UnixNano() >= t { + c.vals = c.vals[i+1:] + return v.TimeBytes(), v.ValueBytes() + } + } + + // not in this one, let the top loop look for it in the next file + break + } + } +} func (c *cursor) Next() (key, value []byte) { - if vals == nil { + if len(c.vals) == 0 { + // if we have a file set, see if the next block is for this ID + if c.f != nil && c.pos < c.f.size { + nextBlockID := btou64(c.f.mmap[c.pos : c.pos+8]) + if nextBlockID == c.id { + return c.decodeBlockAndGetValues(c.pos) + } + } + + // if the file is nil we hit the end of the previous file, advance the file cursor + if c.f != nil { + c.filesPos++ + } + // loop until we find a file with some data - for dataFilePos < len(c.files) { - f = c.files[c.dataFilePos] - c.dataFilePos++ + for c.filesPos < len(c.files) { + f := c.files[c.filesPos] - // startPosition := f + startingPos := f.StartingPositionForID(c.id) + if startingPos == 0 { + continue + } + c.f = f + return c.decodeBlockAndGetValues(startingPos) } + + // we didn't get to a file that had a next value + return nil, nil } - return nil, nil + + v := c.vals[0] + c.vals = c.vals[1:] + + return v.TimeBytes(), v.ValueBytes() } -func (c *cursor) next(id uint64) (key, value []byte) +func (c *cursor) decodeBlockAndGetValues(position uint32) ([]byte, []byte) { + length := btou32(c.f.mmap[position+8 : position+12]) + block := c.f.mmap[position+12 : position+12+length] + c.vals, _ = DecodeFloatBlock(block) + c.pos = position + 12 + length + + v := c.vals[0] + c.vals = c.vals[1:] + return v.TimeBytes(), v.ValueBytes() +} func (c *cursor) Direction() tsdb.Direction { return c.direction } diff --git a/tsdb/engine/pd1/pd1_test.go b/tsdb/engine/pd1/pd1_test.go index 48cdff79293..7d915aab69e 100644 --- a/tsdb/engine/pd1/pd1_test.go +++ b/tsdb/engine/pd1/pd1_test.go @@ -4,8 +4,8 @@ import ( "encoding/binary" "fmt" "io/ioutil" + "math" "os" - "reflect" "testing" "time" @@ -18,37 +18,31 @@ func TestEngine_WriteAndReadFloats(t *testing.T) { e := OpenDefaultEngine() defer e.Close() - codec := tsdb.NewFieldCodec(map[string]*tsdb.Field{ - "value": { - ID: uint8(1), - Name: "value", - Type: influxql.Float, - }, - }) + e.Shard = newFieldCodecMock(map[string]influxql.DataType{"value": influxql.Float}) - p1 := parsePoint("cpu,host=A value=1.1 1000000000", codec) - p2 := parsePoint("cpu,host=B value=1.2 1000000000", codec) - p3 := parsePoint("cpu,host=A value=2.1 2000000000", codec) - p4 := parsePoint("cpu,host=B value=2.2 2000000000", codec) + p1 := parsePoint("cpu,host=A value=1.1 1000000000") + p2 := parsePoint("cpu,host=B value=1.2 1000000000") + p3 := parsePoint("cpu,host=A value=2.1 2000000000") + p4 := parsePoint("cpu,host=B value=2.2 2000000000") if err := e.WriteAndCompact([]tsdb.Point{p1, p2, p3}, nil, nil); err != nil { t.Fatalf("failed to write points: %s", err.Error()) } - verify := func() { + verify := func(checkSingleBVal bool) { c := e.Cursor("cpu,host=A", tsdb.Forward) k, v := c.Next() if btou64(k) != uint64(p1.UnixNano()) { t.Fatalf("p1 time wrong:\n\texp:%d\n\tgot:%d\n", p1.UnixNano(), btou64(k)) } - if !reflect.DeepEqual(v, p1.Data()) { + if 1.1 != btof64(v) { t.Fatal("p1 data not equal") } k, v = c.Next() if btou64(k) != uint64(p3.UnixNano()) { t.Fatalf("p3 time wrong:\n\texp:%d\n\tgot:%d\n", p3.UnixNano(), btou64(k)) } - if !reflect.DeepEqual(v, p3.Data()) { + if 2.1 != btof64(v) { t.Fatal("p3 data not equal") } k, v = c.Next() @@ -61,28 +55,56 @@ func TestEngine_WriteAndReadFloats(t *testing.T) { if btou64(k) != uint64(p2.UnixNano()) { t.Fatalf("p2 time wrong:\n\texp:%d\n\tgot:%d\n", p2.UnixNano(), btou64(k)) } - if !reflect.DeepEqual(v, p2.Data()) { + if 1.2 != btof64(v) { t.Fatal("p2 data not equal") } - k, v = c.Next() - if k != nil { - t.Fatal("expected nil") + + if checkSingleBVal { + k, v = c.Next() + if k != nil { + t.Fatal("expected nil") + } } } - verify() + verify(true) if err := e.WriteAndCompact([]tsdb.Point{p4}, nil, nil); err != nil { t.Fatalf("failed to write points: %s", err.Error()) } - verify() + verify(false) c := e.Cursor("cpu,host=B", tsdb.Forward) - k, v := c.Seek(u64tob(2000000000)) + k, v := c.Next() + if btou64(k) != uint64(p2.UnixNano()) { + t.Fatalf("p2 time wrong:\n\texp:%d\n\tgot:%d\n", p2.UnixNano(), btou64(k)) + } + if 1.2 != btof64(v) { + t.Fatal("p2 data not equal") + } + k, v = c.Next() + if btou64(k) != uint64(p4.UnixNano()) { + t.Fatalf("p2 time wrong:\n\texp:%d\n\tgot:%d\n", p2.UnixNano(), btou64(k)) + } + if 2.2 != btof64(v) { + t.Fatal("p2 data not equal") + } + + // verify we can seek + k, v = c.Seek(u64tob(2000000000)) if btou64(k) != uint64(p4.UnixNano()) { - t.Fatalf("p4 time wrong:\n\texp:%d\n\tgot:%d\n", p4.UnixNano(), btou64(k)) + t.Fatalf("p2 time wrong:\n\texp:%d\n\tgot:%d\n", p2.UnixNano(), btou64(k)) } - if !reflect.DeepEqual(v, p4.Data()) { - t.Fatal("p4 data not equal") + if 2.2 != btof64(v) { + t.Fatal("p2 data not equal") + } + + c = e.Cursor("cpu,host=A", tsdb.Forward) + k, v = c.Seek(u64tob(0)) + if btou64(k) != uint64(p1.UnixNano()) { + t.Fatalf("p1 time wrong:\n\texp:%d\n\tgot:%d\n", p1.UnixNano(), btou64(k)) + } + if 1.1 != btof64(v) { + t.Fatal("p1 data not equal") } } @@ -95,17 +117,9 @@ func TestEngine_WriteIndexBenchmarkNames(t *testing.T) { e := OpenDefaultEngine() defer e.Close() - codec := tsdb.NewFieldCodec(map[string]*tsdb.Field{ - "value": { - ID: uint8(1), - Name: "value", - Type: influxql.Float, - }, - }) - var points []tsdb.Point for i := 0; i < 100000; i++ { - points = append(points, parsePoint(fmt.Sprintf("cpu%d value=22.1", i), codec)) + points = append(points, parsePoint(fmt.Sprintf("cpu%d value=22.1", i))) } st := time.Now() @@ -160,23 +174,35 @@ func (e *Engine) Close() error { return nil } -func parsePoints(buf string, codec *tsdb.FieldCodec) []tsdb.Point { +func newFieldCodecMock(fields map[string]influxql.DataType) *FieldCodeMock { + m := make(map[string]*tsdb.Field) + + for n, t := range fields { + m[n] = &tsdb.Field{Name: n, Type: t} + } + codec := tsdb.NewFieldCodec(m) + + return &FieldCodeMock{codec: codec} +} + +type FieldCodeMock struct { + codec *tsdb.FieldCodec +} + +func (f *FieldCodeMock) FieldCodec(m string) *tsdb.FieldCodec { + return f.codec +} + +func parsePoints(buf string) []tsdb.Point { points, err := tsdb.ParsePointsString(buf) if err != nil { panic(fmt.Sprintf("couldn't parse points: %s", err.Error())) } - for _, p := range points { - b, err := codec.EncodeFields(p.Fields()) - if err != nil { - panic(fmt.Sprintf("couldn't encode fields: %s", err.Error())) - } - p.SetData(b) - } return points } -func parsePoint(buf string, codec *tsdb.FieldCodec) tsdb.Point { - return parsePoints(buf, codec)[0] +func parsePoint(buf string) tsdb.Point { + return parsePoints(buf)[0] } func inttob(v int) []byte { @@ -194,3 +220,7 @@ func u64tob(v uint64) []byte { binary.BigEndian.PutUint64(b, v) return b } + +func btof64(b []byte) float64 { + return math.Float64frombits(binary.BigEndian.Uint64(b)) +} diff --git a/tsdb/shard.go b/tsdb/shard.go index a14822f367c..2e04735665f 100644 --- a/tsdb/shard.go +++ b/tsdb/shard.go @@ -229,28 +229,29 @@ func (s *Shard) WritePoints(points []models.Point) error { } // make sure all data is encoded before attempting to save to bolt - for _, p := range points { - // Ignore if raw data has already been marshaled. - if p.Data() != nil { - continue - } - - // This was populated earlier, don't need to validate that it's there. - s.mu.RLock() - mf := s.measurementFields[p.Name()] - s.mu.RUnlock() - - // If a measurement is dropped while writes for it are in progress, this could be nil - if mf == nil { - return ErrFieldNotFound - } - - data, err := mf.Codec.EncodeFields(p.Fields()) - if err != nil { - return err - } - p.SetData(data) - } + // TODO: make this only commented out for pd1 engine + // for _, p := range points { + // // Ignore if raw data has already been marshaled. + // if p.Data() != nil { + // continue + // } + + // // This was populated earlier, don't need to validate that it's there. + // s.mu.RLock() + // mf := s.measurementFields[p.Name()] + // s.mu.RUnlock() + + // // If a measurement is dropped while writes for it are in progress, this could be nil + // if mf == nil { + // return ErrFieldNotFound + // } + + // data, err := mf.Codec.EncodeFields(p.Fields()) + // if err != nil { + // return err + // } + // p.SetData(data) + // } // Write to the engine. if err := s.engine.WritePoints(points, measurementFieldsToSave, seriesToCreate); err != nil { @@ -741,11 +742,14 @@ func (f *FieldCodec) DecodeByID(targetID uint8, b []byte) (interface{}, error) { // DecodeByName scans a byte slice for a field with the given name, converts it to its // expected type, and return that value. func (f *FieldCodec) DecodeByName(name string, b []byte) (interface{}, error) { - fi := f.FieldByName(name) - if fi == nil { - return 0, ErrFieldNotFound - } - return f.DecodeByID(fi.ID, b) + // TODO: this is a hack for PD1 testing, please to remove + return math.Float64frombits(binary.BigEndian.Uint64(b)), nil + + // fi := f.FieldByName(name) + // if fi == nil { + // return 0, ErrFieldNotFound + // } + // return f.DecodeByID(fi.ID, b) } func (f *FieldCodec) Fields() (a []*Field) { From 82e1be752730f86573c3d95b44d4d79b846107f0 Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Wed, 9 Sep 2015 11:29:50 -0700 Subject: [PATCH 073/139] WIP: more WAL work --- cmd/influx_stress/influx_stress.go | 1 + tsdb/config.go | 2 +- tsdb/engine/pd1/encoding.go | 119 ++++-- tsdb/engine/pd1/encoding_test.go | 34 +- tsdb/engine/pd1/pd1.go | 502 ++++++------------------- tsdb/engine/pd1/pd1_test.go | 9 +- tsdb/engine/pd1/wal.go | 581 +++++++++++++++++++++++++++++ 7 files changed, 820 insertions(+), 428 deletions(-) create mode 100644 tsdb/engine/pd1/wal.go diff --git a/cmd/influx_stress/influx_stress.go b/cmd/influx_stress/influx_stress.go index bd2f1b4a2ad..9d5d0c22b5e 100644 --- a/cmd/influx_stress/influx_stress.go +++ b/cmd/influx_stress/influx_stress.go @@ -3,6 +3,7 @@ package main import ( "flag" "fmt" + "math/rand" "net/url" "runtime" "sort" diff --git a/tsdb/config.go b/tsdb/config.go index 9843541e296..dfd267d2c37 100644 --- a/tsdb/config.go +++ b/tsdb/config.go @@ -42,7 +42,7 @@ const ( // we'll need to create backpressure, otherwise we'll fill up the memory and die. // This number multiplied by the parition count is roughly the max possible memory // size for the in-memory WAL cache. - DefaultPartitionSizeThreshold = 20 * 1024 * 1024 // 20MB + DefaultPartitionSizeThreshold = 50 * 1024 * 1024 // 50MB ) type Config struct { diff --git a/tsdb/engine/pd1/encoding.go b/tsdb/engine/pd1/encoding.go index 60f72766d83..7262e8a6c36 100644 --- a/tsdb/engine/pd1/encoding.go +++ b/tsdb/engine/pd1/encoding.go @@ -12,52 +12,127 @@ type Value interface { TimeBytes() []byte ValueBytes() []byte Time() time.Time + Value() interface{} + Size() int } +func NewValue(t time.Time, value interface{}) Value { + switch v := value.(type) { + // case int64: + // return &Int64Value{time: t, value: v} + case float64: + return &FloatValue{time: t, value: v} + // case bool: + // return &BoolValue{time: t, value: v} + // case string: + // return &StringValue{time: t, value: v} + } + return &EmptyValue{} +} + +type EmptyValue struct { +} + +func (e *EmptyValue) TimeBytes() []byte { return nil } +func (e *EmptyValue) ValueBytes() []byte { return nil } +func (e *EmptyValue) Time() time.Time { return time.Unix(0, 0) } +func (e *EmptyValue) Value() interface{} { return nil } +func (e *EmptyValue) Size() int { return 0 } + +// Values represented a time ascending sorted collection of Value types. +// the underlying type should be the same across all values, but the interface +// makes the code cleaner. +type Values []Value + +func (v Values) MinTime() int64 { + return v[0].Time().UnixNano() +} + +func (v Values) MaxTime() int64 { + return v[len(v)-1].Time().UnixNano() +} + +func (v Values) Encode(buf []byte) []byte { + switch v[0].(type) { + case *FloatValue: + a := make([]*FloatValue, len(v)) + for i, vv := range v { + a[i] = vv.(*FloatValue) + } + return EncodeFloatBlock(buf, a) + + // TODO: add support for other types + } + + return nil +} + +func (v Values) DecodeSameTypeBlock(block []byte) Values { + switch v[0].(type) { + case *FloatValue: + a, _ := DecodeFloatBlock(block) + return a + + // TODO: add support for other types + } + return nil +} + +// Sort methods +func (a Values) Len() int { return len(a) } +func (a Values) Swap(i, j int) { a[i], a[j] = a[j], a[i] } +func (a Values) Less(i, j int) bool { return a[i].Time().UnixNano() < a[j].Time().UnixNano() } + type FloatValue struct { - Time time.Time - Value float64 + time time.Time + value float64 +} + +func (f *FloatValue) Time() time.Time { + return f.time +} + +func (f *FloatValue) Value() interface{} { + return f.value } func (f *FloatValue) TimeBytes() []byte { - return u64tob(uint64(f.Time.UnixNano())) + return u64tob(uint64(f.Time().UnixNano())) } func (f *FloatValue) ValueBytes() []byte { buf := make([]byte, 8) - binary.BigEndian.PutUint64(buf, math.Float64bits(f.Value)) + binary.BigEndian.PutUint64(buf, math.Float64bits(f.value)) return buf } -type FloatValues []FloatValue - -func (a FloatValues) Len() int { return len(a) } -func (a FloatValues) Swap(i, j int) { a[i], a[j] = a[j], a[i] } -func (a FloatValues) Less(i, j int) bool { return a[i].Time.UnixNano() < a[j].Time.UnixNano() } +func (f *FloatValue) Size() int { + return 16 +} // TODO: make this work with nanosecond timestamps -func EncodeFloatBlock(buf []byte, values []FloatValue) []byte { - s := tsz.New(uint32(values[0].Time.Unix())) +func EncodeFloatBlock(buf []byte, values []*FloatValue) []byte { + s := tsz.New(uint32(values[0].Time().Unix())) for _, v := range values { - s.Push(uint32(v.Time.Unix()), v.Value) + s.Push(uint32(v.Time().Unix()), v.value) } s.Finish() - return append(u64tob(uint64(values[0].Time.UnixNano())), s.Bytes()...) + return append(u64tob(uint64(values[0].Time().UnixNano())), s.Bytes()...) } -func DecodeFloatBlock(block []byte) ([]FloatValue, error) { +func DecodeFloatBlock(block []byte) ([]Value, error) { iter, _ := tsz.NewIterator(block[8:]) - a := make([]FloatValue, 0) + a := make([]Value, 0) for iter.Next() { t, f := iter.Values() - a = append(a, FloatValue{time.Unix(int64(t), 0), f}) + a = append(a, &FloatValue{time.Unix(int64(t), 0), f}) } return a, nil } type BoolValue struct { - Time time.Time - Value bool + time time.Time + value bool } func EncodeBoolBlock(buf []byte, values []BoolValue) []byte { @@ -69,8 +144,8 @@ func DecodeBoolBlock(block []byte) ([]BoolValue, error) { } type Int64Value struct { - Time time.Time - Value int64 + time time.Time + value int64 } func EncodeInt64Block(buf []byte, values []Int64Value) []byte { @@ -82,8 +157,8 @@ func DecodeInt64Block(block []byte) ([]Int64Value, error) { } type StringValue struct { - Time time.Time - Value string + time time.Time + value string } func EncodeStringBlock(buf []byte, values []StringValue) []byte { diff --git a/tsdb/engine/pd1/encoding_test.go b/tsdb/engine/pd1/encoding_test.go index 26bb4c2e076..aa5a4b15e73 100644 --- a/tsdb/engine/pd1/encoding_test.go +++ b/tsdb/engine/pd1/encoding_test.go @@ -1,32 +1,32 @@ package pd1_test import ( - "math/rand" - "reflect" + // "math/rand" + // "reflect" "testing" "time" - "github.com/influxdb/influxdb/tsdb/engine/pd1" + // "github.com/influxdb/influxdb/tsdb/engine/pd1" ) func TestEncoding_FloatBlock(t *testing.T) { - valueCount := 100 - times := getTimes(valueCount, 60, time.Second) - values := make([]pd1.FloatValue, len(times)) - for i, t := range times { - values[i] = pd1.FloatValue{Time: t, Value: rand.Float64()} - } + // valueCount := 100 + // times := getTimes(valueCount, 60, time.Second) + // values := make([]Value, len(times)) + // for i, t := range times { + // values[i] = pd1.NewValue(t, rand.Float64()) + // } - b := pd1.EncodeFloatBlock(nil, values) + // b := pd1.EncodeFloatBlock(nil, values) - decodedValues, err := pd1.DecodeFloatBlock(b) - if err != nil { - t.Fatalf("error decoding: %s", err.Error) - } + // decodedValues, err := pd1.DecodeFloatBlock(b) + // if err != nil { + // t.Fatalf("error decoding: %s", err.Error) + // } - if !reflect.DeepEqual(decodedValues, values) { - t.Fatalf("unexpected results:\n\tgot: %v\n\texp: %v\n", decodedValues, values) - } + // if !reflect.DeepEqual(decodedValues, values) { + // t.Fatalf("unexpected results:\n\tgot: %v\n\texp: %v\n", decodedValues, values) + // } } func getTimes(n, step int, precision time.Duration) []time.Time { diff --git a/tsdb/engine/pd1/pd1.go b/tsdb/engine/pd1/pd1.go index 4d3c752ae2e..ef91fb66e2d 100644 --- a/tsdb/engine/pd1/pd1.go +++ b/tsdb/engine/pd1/pd1.go @@ -1,7 +1,6 @@ package pd1 import ( - "bytes" "encoding/binary" "encoding/json" "fmt" @@ -12,8 +11,6 @@ import ( "os" "path/filepath" "sort" - "strconv" - "strings" "sync" "syscall" "time" @@ -29,13 +26,13 @@ const ( // FieldsFileExtension is the extension for the file that stores compressed field // encoding data for this db - FieldsFileExtension = "fld" + FieldsFileExtension = "fields" // SeriesFileExtension is the extension for the file that stores the compressed // series metadata for series in this db - SeriesFileExtension = "srs" + SeriesFileExtension = "series" - CollisionsFileExtension = "col" + CollisionsFileExtension = "collisions" ) type TimePrecision uint8 @@ -55,7 +52,7 @@ const ( // DefaultBlockSize is the default size of uncompressed points blocks. DefaultBlockSize = 512 * 1024 // 512KB - DefaultMaxFileSize = 5 * 1024 * 1024 // 5MB + DefaultMaxFileSize = 10 * 1024 * 1024 // 10MB DefaultMaxPointsPerBlock = 1000 @@ -80,6 +77,8 @@ type Engine struct { FieldCodec(measurementName string) *tsdb.FieldCodec } + WAL *Log + filesLock sync.RWMutex files dataFiles currentFileID int @@ -88,11 +87,19 @@ type Engine struct { // NewEngine returns a new instance of Engine. func NewEngine(path string, walPath string, opt tsdb.EngineOptions) tsdb.Engine { + w := NewLog(path) + w.FlushColdInterval = time.Duration(opt.Config.WALFlushColdInterval) + w.MemorySizeThreshold = int(opt.Config.WALPartitionSizeThreshold) + w.LoggingEnabled = opt.Config.WALLoggingEnabled + e := &Engine{ path: path, + // TODO: this is the function where we can inject a check against the in memory collisions HashSeriesField: hashSeriesField, + WAL: w, } + e.WAL.Index = e return e } @@ -116,6 +123,13 @@ func (e *Engine) Open() error { return err } for _, fn := range files { + id, err := idFromFileName(fn) + if err != nil { + return err + } + if id >= e.currentFileID { + e.currentFileID = id + 1 + } f, err := os.OpenFile(fn, os.O_RDONLY, 0666) if err != nil { return fmt.Errorf("error opening file %s: %s", fn, err.Error()) @@ -128,6 +142,10 @@ func (e *Engine) Open() error { } sort.Sort(e.files) + if err := e.WAL.Open(); err != nil { + return err + } + return nil } @@ -189,12 +207,10 @@ func (e *Engine) LoadMetadataIndex(shard *tsdb.Shard, index *tsdb.DatabaseIndex, // WritePoints writes metadata and point data into the engine. // Returns an error if new points are added to an existing key. func (e *Engine) WritePoints(points []tsdb.Point, measurementFieldsToSave map[string]*tsdb.MeasurementFields, seriesToCreate []*tsdb.SeriesCreate) error { - // TODO: Write points to the WAL - - return e.WriteAndCompact(points, measurementFieldsToSave, seriesToCreate) + return e.WAL.WritePoints(points, measurementFieldsToSave, seriesToCreate) } -func (e *Engine) WriteAndCompact(points []tsdb.Point, measurementFieldsToSave map[string]*tsdb.MeasurementFields, seriesToCreate []*tsdb.SeriesCreate) error { +func (e *Engine) WriteAndCompact(pointsByKey map[string]Values, measurementFieldsToSave map[string]*tsdb.MeasurementFields, seriesToCreate []*tsdb.SeriesCreate) error { e.mu.Lock() defer e.mu.Unlock() @@ -205,120 +221,78 @@ func (e *Engine) WriteAndCompact(points []tsdb.Point, measurementFieldsToSave ma return err } - if len(points) == 0 { + if len(pointsByKey) == 0 { return nil } - b, err := e.readCompressedFile("names") + // read in keys and assign any that aren't defined + b, err := e.readCompressedFile("ids") if err != nil { return err } - ids := make(map[uint64]string) - - var names []string + ids := make(map[string]uint64) if b != nil { - if err := json.Unmarshal(b, &names); err != nil { + if err := json.Unmarshal(b, &ids); err != nil { return err } - - for _, n := range names { - ids[e.HashSeriesField(n)] = n - } } // these are values that are newer than anything stored in the shard - valuesByID := make(map[uint64]*valueCollection) - // map the points to the data file they belong to if they overlap - dataFileToValues := make(map[*dataFile]map[uint64]*valueCollection) - - // we keep track of the newest data file and if it should be - // rewritten with new data. - var newestDataFile *dataFile - overwriteNewestFile := false - if len(e.files) > 0 { - newestDataFile = e.files[len(e.files)-1] - overwriteNewestFile = newestDataFile.size < DefaultMaxFileSize - } - - // compute ids of new keys and arrange for insertion - for _, p := range points { - for fn, val := range p.Fields() { - n := seriesFieldKey(string(p.Key()), fn) - id := e.HashSeriesField(n) - if series, ok := ids[id]; !ok { - names = append(names, n) - } else { // possible collision? - if n != series { - // TODO: implement collision detection - panic("name collision!") + valuesByID := make(map[uint64]Values) + + idToKey := make(map[uint64]string) // we only use this map if new ids are being created + newKeys := false + for k, values := range pointsByKey { + var id uint64 + var ok bool + if id, ok = ids[k]; !ok { + // populate the map if we haven't already + if len(idToKey) == 0 { + for n, id := range ids { + idToKey[id] = n } } - ids[id] = n - - vals := valuesByID[id] - if vals == nil { - // TODO: deal with situation where there are already files, - // but the user is inserting a bunch of data that predates - // any of them. It's ok to rewrite the first file, but - // only to max size. Then we should create a new one - - // points always come in time increasing order. This is - // the first point we've seen for this key. So it might - // need to get put into an older file instead of a new - // one. Check and set accordingly - var df *dataFile - for i := len(e.files) - 1; i >= 0; i-- { - if p.UnixNano() > e.files[i].MaxTime() { - break - } - df = e.files[i] - } - vals = &valueCollection{} - - if df == nil || (df == newestDataFile && overwriteNewestFile) { - // this point is newer than anything we have stored - // or it belongs in the most recent file, which should get - // rewritten - valuesByID[id] = vals - } else { - // it overlaps with another file so mark it and it can be compacted - dfm := dataFileToValues[df] - if dfm == nil { - dfm = make(map[uint64]*valueCollection) - dataFileToValues[df] = dfm - } - - if vc := dfm[id]; vc == nil { - dfm[id] = vals - } else { - vals = vc - } + // now see if the hash id collides with a different key + hashID := hashSeriesField(k) + existingKey, idInMap := idToKey[hashID] + if idInMap { + // we only care if the keys are different. if so, it's a hash collision we have to keep track of + if k != existingKey { + // we have a collision, give this new key a different id and move on + // TODO: handle collisions + panic("name collision, not implemented yet!") } - } - - switch t := val.(type) { - case float64: - vals.floatValues = append(vals.floatValues, FloatValue{Time: p.Time(), Value: t}) - case int64: - vals.intValues = append(vals.intValues, Int64Value{Time: p.Time(), Value: t}) - case bool: - vals.boolValues = append(vals.boolValues, BoolValue{Time: p.Time(), Value: t}) - case string: - vals.stringValues = append(vals.stringValues, StringValue{Time: p.Time(), Value: t}) - default: - panic("unsupported type") + } else { + newKeys = true + ids[k] = hashID + idToKey[id] = k + id = hashID } } + + valuesByID[id] = values } - b, err = json.Marshal(names) - if err != nil { - return err + if newKeys { + b, err := json.Marshal(ids) + if err != nil { + return err + } + if err := e.replaceCompressedFile("ids", b); err != nil { + return err + } } - if err := e.replaceCompressedFile("names", b); err != nil { - return err + // TODO: handle values written in the past that force an old data file to get rewritten + + // we keep track of the newest data file and if it should be + // rewritten with new data. + var newestDataFile *dataFile + overwriteNewestFile := false + if len(e.files) > 0 { + newestDataFile = e.files[len(e.files)-1] + overwriteNewestFile = newestDataFile.size < DefaultMaxFileSize } // flush values by id to either a new file or rewrite the old one @@ -330,21 +304,14 @@ func (e *Engine) WriteAndCompact(points []tsdb.Point, measurementFieldsToSave ma return err } - // flush each of the old ones - for df, vals := range dataFileToValues { - if err := e.rewriteFile(df, vals); err != nil { - return err - } - } - return nil } -func (e *Engine) rewriteFile(oldDF *dataFile, values map[uint64]*valueCollection) error { +func (e *Engine) rewriteFile(oldDF *dataFile, valuesByID map[uint64]Values) error { // we need the values in sorted order so that we can merge them into the // new file as we read the old file - ids := make([]uint64, 0, len(values)) - for id, _ := range values { + ids := make([]uint64, 0, len(valuesByID)) + for id, _ := range valuesByID { ids = append(ids, id) } @@ -358,18 +325,18 @@ func (e *Engine) rewriteFile(oldDF *dataFile, values map[uint64]*valueCollection minTime = oldDF.MinTime() maxTime = oldDF.MaxTime() } - for _, v := range values { - if minTime > v.MinTime().UnixNano() { - minTime = v.MinTime().UnixNano() + for _, v := range valuesByID { + if minTime > v.MinTime() { + minTime = v.MinTime() } - if maxTime < v.MaxTime().UnixNano() { - maxTime = v.MaxTime().UnixNano() + if maxTime < v.MaxTime() { + maxTime = v.MaxTime() } } // add any ids that are in the file that aren't getting flushed here for id, _ := range oldIDToPosition { - if _, ok := values[id]; !ok { + if _, ok := valuesByID[id]; !ok { ids = append(ids, id) } } @@ -414,10 +381,10 @@ func (e *Engine) rewriteFile(oldDF *dataFile, values map[uint64]*valueCollection // mark the position for this ID newPositions[i] = currentPosition - newVals := values[id] + newVals := valuesByID[id] // if this id is only in the file and not in the new values, just copy over from old file - if newVals == nil { + if len(newVals) == 0 { fpos := oldIDToPosition[id] // write the blocks until we hit whatever the next id is @@ -482,7 +449,8 @@ func (e *Engine) rewriteFile(oldDF *dataFile, values map[uint64]*valueCollection } } - newBlock, err := newVals.DecodeAndCombine(block, buf[:0], nextTime, hasFutureBlock) + nv, newBlock, err := e.DecodeAndCombine(newVals, block, buf[:0], nextTime, hasFutureBlock) + newVals = nv if err != nil { return err } @@ -503,7 +471,7 @@ func (e *Engine) rewriteFile(oldDF *dataFile, values map[uint64]*valueCollection } // TODO: ensure we encode only the amount in a block, refactor this wil line 450 into func - if len(newVals.floatValues) > 0 { + if len(newVals) > 0 { // TODO: ensure we encode only the amount in a block block := newVals.Encode(buf) if _, err := f.Write(append(u64tob(id), u32tob(uint32(len(block)))...)); err != nil { @@ -820,96 +788,39 @@ func (e *Engine) readSeries() (map[string]*tsdb.Series, error) { return series, nil } -type valueCollection struct { - floatValues []FloatValue - boolValues []BoolValue - intValues []Int64Value - stringValues []StringValue -} - -func (v *valueCollection) MinTime() time.Time { - if v.floatValues != nil { - return v.floatValues[0].Time - } else if v.boolValues != nil { - return v.boolValues[0].Time - } else if v.intValues != nil { - return v.intValues[0].Time - } else if v.stringValues != nil { - return v.stringValues[0].Time - } - - return time.Unix(0, 0) -} - -func (v *valueCollection) MaxTime() time.Time { - if v.floatValues != nil { - return v.floatValues[len(v.floatValues)-1].Time - } else if v.boolValues != nil { - return v.boolValues[len(v.boolValues)-1].Time - } else if v.intValues != nil { - return v.intValues[len(v.intValues)-1].Time - } else if v.stringValues != nil { - return v.stringValues[len(v.stringValues)-1].Time - } - - return time.Unix(0, 0) -} - -func (v *valueCollection) Encode(buf []byte) []byte { - if v.floatValues != nil { - return EncodeFloatBlock(buf, v.floatValues) - } else if v.boolValues != nil { - return EncodeBoolBlock(buf, v.boolValues) - } else if v.intValues != nil { - return EncodeInt64Block(buf, v.intValues) - } else if v.stringValues != nil { - return EncodeStringBlock(buf, v.stringValues) - } - - return nil -} - // DecodeAndCombine take an encoded block from a file, decodes it and interleaves the file -// values with the values in this collection. nextTime and hasNext refer to if the file +// values with the values passed in. nextTime and hasNext refer to if the file // has future encoded blocks so that this method can know how much of its values can be // combined and output in the resulting encoded block. -func (v *valueCollection) DecodeAndCombine(block, buf []byte, nextTime int64, hasFutureBlock bool) ([]byte, error) { - if v.floatValues != nil { - values, err := DecodeFloatBlock(block) - if err != nil { - return nil, err - } - - if hasFutureBlock { - // take all values that have times less than the future block and update the vals array - pos := sort.Search(len(v.floatValues), func(i int) bool { - return v.floatValues[i].Time.UnixNano() >= nextTime - }) - values = append(values, v.floatValues[:pos]...) - v.floatValues = v.floatValues[pos:] - } else { - values = append(values, v.floatValues...) - v.floatValues = nil +func (e *Engine) DecodeAndCombine(newValues Values, block, buf []byte, nextTime int64, hasFutureBlock bool) (Values, []byte, error) { + values := newValues.DecodeSameTypeBlock(block) + + var remainingValues Values + + if hasFutureBlock { + // take all values that have times less than the future block and update the vals array + pos := sort.Search(len(newValues), func(i int) bool { + return newValues[i].Time().UnixNano() >= nextTime + }) + values = append(values, newValues[:pos]...) + remainingValues = newValues[pos:] + sort.Sort(values) + } else { + requireSort := values.MaxTime() > newValues.MinTime() + values = append(values, newValues...) + if requireSort { + sort.Sort(values) } - sort.Sort(FloatValues(values)) - // TODO: deduplicate values + } - if len(values) > DefaultMaxPointsPerBlock { - v.floatValues = values[DefaultMaxPointsPerBlock:] - values = values[:DefaultMaxPointsPerBlock] - } + // TODO: deduplicate values - return EncodeFloatBlock(buf, values), nil - } else if v.boolValues != nil { - // TODO: wire up the other value types - return nil, fmt.Errorf("not implemented") - } else if v.intValues != nil { - return nil, fmt.Errorf("not implemented") - } else if v.stringValues != nil { - return nil, fmt.Errorf("not implemented") + if len(values) > DefaultMaxPointsPerBlock { + remainingValues = values[DefaultMaxPointsPerBlock:] + values = values[:DefaultMaxPointsPerBlock] } - return nil, nil + return remainingValues, values.Encode(buf), nil } type dataFile struct { @@ -1040,7 +951,7 @@ type cursor struct { f *dataFile filesPos int // the index in the files slice we're looking at pos uint32 - vals FloatValues + vals Values direction tsdb.Direction @@ -1121,7 +1032,7 @@ func (c *cursor) Seek(seek []byte) (key, value []byte) { // wasn't in the first value popped out of the block, check the rest for i, v := range c.vals { - if v.Time.UnixNano() >= t { + if v.Time().UnixNano() >= t { c.vals = c.vals[i+1:] return v.TimeBytes(), v.ValueBytes() } @@ -1220,180 +1131,3 @@ type uint64slice []uint64 func (a uint64slice) Len() int { return len(a) } func (a uint64slice) Swap(i, j int) { a[i], a[j] = a[j], a[i] } func (a uint64slice) Less(i, j int) bool { return a[i] < a[j] } - -/* TODO: REMOVE THIS STUFF */ -func (e *Engine) pointsToBlocks(points [][]byte) []byte { - var b bytes.Buffer - block := make([]byte, 0) - for _, p := range points { - block = append(block, p[0:8]...) - block = append(block, u32tob(uint32(len(p)-8))...) - block = append(block, p[8:]...) - if len(block) > DefaultBlockSize { - e.writeBlockToBuffer(block, &b) - block = make([]byte, 0) - } - } - if len(block) > 0 { - e.writeBlockToBuffer(block, &b) - } - - return b.Bytes() -} - -func (e *Engine) writeBlockToBuffer(block []byte, b *bytes.Buffer) { - // write the min time - if _, err := b.Write(block[0:8]); err != nil { - panic(err) - } - - // write the length of the compressed data - data := snappy.Encode(nil, block) - if _, err := b.Write(u32tob(uint32(len(data)))); err != nil { - panic(err) - } - - // write the compressed data - if _, err := b.Write(data); err != nil { - panic(err) - } -} - -func (e *Engine) readPointsFromFile(f *os.File) (map[uint64][][]byte, error) { - buf := make([]byte, 8) - if _, err := io.ReadFull(f, buf); err != nil { - return nil, err - } - seriesCount := btou64(buf) - positions := make([]uint64, seriesCount, seriesCount) - ids := make([]uint64, seriesCount, seriesCount) - - // read the series index file header - position := uint64(8) - for i := 0; uint64(i) < seriesCount; i++ { - // read the id of the series - if _, err := io.ReadFull(f, buf); err != nil { - return nil, err - } - ids[i] = btou64(buf) - - // read the min time and ignore - if _, err := io.ReadFull(f, buf); err != nil { - return nil, err - } - if _, err := io.ReadFull(f, buf); err != nil { - return nil, err - } - - // read the starting position of this id - if _, err := io.ReadFull(f, buf); err != nil { - return nil, err - } - positions[i] = btou64(buf) - position += 32 - } - - if position != positions[0] { - panic("we aren't at the right place") - } - - // read the raw data - seriesData := make(map[uint64][][]byte) - compressedBuff := make([]byte, DefaultBlockSize) - seriesPosition := 0 - for { - // read the min time and ignore - if _, err := io.ReadFull(f, buf); err == io.EOF { - break - } else if err != nil { - return nil, err - } - - // read the length of the compressed block - if _, err := io.ReadFull(f, buf[:4]); err != nil { - return nil, err - } - length := btou32(buf) - - if length > uint32(len(compressedBuff)) { - compressedBuff = make([]byte, length) - } - if _, err := io.ReadFull(f, compressedBuff[:length]); err != nil { - return nil, err - } - - data, err := snappy.Decode(nil, compressedBuff[:length]) - if err != nil { - return nil, err - } - id := ids[seriesPosition] - seriesData[id] = append(seriesData[id], e.pointsFromDataBlock(data)...) - position += uint64(12 + length) - - if seriesPosition+1 >= len(positions) { - continue - } - if positions[seriesPosition+1] == position { - seriesPosition += 1 - } - } - - return seriesData, nil -} - -func (e *Engine) pointsFromDataBlock(data []byte) [][]byte { - a := make([][]byte, 0) - for { - length := entryDataSize(data) - p := append(data[:8], data[12:12+length]...) - a = append(a, p) - data = data[12+length:] - if len(data) == 0 { - break - } - } - return a -} - -func entryDataSize(v []byte) int { return int(binary.BigEndian.Uint32(v[8:12])) } - -func (e *Engine) lastFileAndNewFile() (*os.File, *os.File, error) { - files, err := filepath.Glob(filepath.Join(e.path, fmt.Sprintf("*.%s", Format))) - if err != nil { - return nil, nil, err - } - - if len(files) == 0 { - newFile, err := os.OpenFile(filepath.Join(e.path, fmt.Sprintf("%07d.%s", 1, Format)), os.O_CREATE|os.O_RDWR, 0666) - if err != nil { - return nil, nil, err - } - return nil, newFile, nil - } - - oldFile, err := os.OpenFile(files[len(files)-1], os.O_RDONLY, 0666) - if err != nil { - return nil, nil, err - } - - info, err := oldFile.Stat() - if err != nil { - _ = oldFile.Close() - return nil, nil, err - } - - num := strings.Split(filepath.Base(files[len(files)-1]), ".")[0] - n, err := strconv.ParseUint(num, 10, 32) - if err != nil { - return nil, nil, err - } - newFile, err := os.OpenFile(filepath.Join(e.path, fmt.Sprintf("%07d.%s", n+1, Format)), os.O_CREATE|os.O_RDWR, 0666) - if err != nil { - return nil, nil, err - } - if info.Size() >= DefaultMaxFileSize { - oldFile.Close() - return nil, newFile, nil - } - return oldFile, newFile, nil -} diff --git a/tsdb/engine/pd1/pd1_test.go b/tsdb/engine/pd1/pd1_test.go index 7d915aab69e..79817eb531d 100644 --- a/tsdb/engine/pd1/pd1_test.go +++ b/tsdb/engine/pd1/pd1_test.go @@ -25,7 +25,7 @@ func TestEngine_WriteAndReadFloats(t *testing.T) { p3 := parsePoint("cpu,host=A value=2.1 2000000000") p4 := parsePoint("cpu,host=B value=2.2 2000000000") - if err := e.WriteAndCompact([]tsdb.Point{p1, p2, p3}, nil, nil); err != nil { + if err := e.WritePoints([]tsdb.Point{p1, p2, p3}, nil, nil); err != nil { t.Fatalf("failed to write points: %s", err.Error()) } @@ -68,7 +68,7 @@ func TestEngine_WriteAndReadFloats(t *testing.T) { } verify(true) - if err := e.WriteAndCompact([]tsdb.Point{p4}, nil, nil); err != nil { + if err := e.WritePoints([]tsdb.Point{p4}, nil, nil); err != nil { t.Fatalf("failed to write points: %s", err.Error()) } verify(false) @@ -123,13 +123,13 @@ func TestEngine_WriteIndexBenchmarkNames(t *testing.T) { } st := time.Now() - if err := e.WriteAndCompact(points, nil, nil); err != nil { + if err := e.WritePoints(points, nil, nil); err != nil { t.Fatalf("failed to write points: %s", err.Error()) } fmt.Println("took: ", time.Since(st)) st = time.Now() - if err := e.WriteAndCompact(points, nil, nil); err != nil { + if err := e.WritePoints(points, nil, nil); err != nil { t.Fatalf("failed to write points: %s", err.Error()) } fmt.Println("took: ", time.Since(st)) @@ -161,6 +161,7 @@ func OpenEngine(opt tsdb.EngineOptions) *Engine { if err := e.Open(); err != nil { panic(err) } + e.WAL.SkipCache = true return e } diff --git a/tsdb/engine/pd1/wal.go b/tsdb/engine/pd1/wal.go new file mode 100644 index 00000000000..26f2af48ffe --- /dev/null +++ b/tsdb/engine/pd1/wal.go @@ -0,0 +1,581 @@ +package pd1 + +import ( + "bytes" + "fmt" + "io" + "log" + "os" + "path/filepath" + "sort" + "strconv" + "strings" + "sync" + "time" + + "github.com/golang/snappy" + "github.com/influxdb/influxdb/tsdb" +) + +const ( + // DefaultSegmentSize of 2MB is the size at which segment files will be rolled over + DefaultSegmentSize = 2 * 1024 * 1024 + + // FileExtension is the file extension we expect for wal segments + WALFileExtension = "wal" + + WALFilePrefix = "_" + + // defaultFlushCheckInterval is how often flushes are triggered automatically by the flush criteria + defaultFlushCheckInterval = time.Second +) + +// flushType indiciates why a flush and compaction are being run so the partition can +// do the appropriate type of compaction +type flushType int + +const ( + // noFlush indicates that no flush or compaction are necesssary at this time + noFlush flushType = iota + // memoryFlush indicates that we should look for the series using the most + // memory to flush out and compact all others + memoryFlush + // idleFlush indicates that we should flush all series in the parition, + // delete all segment files and hold off on opening a new one + idleFlush + // deleteFlush indicates that we're flushing because series need to be removed from the WAL + deleteFlush + + writeBufLen = 32 << 10 // 32kb +) + +// walEntry is a byte written to a wal segment file that indicates what the following compressed block contains +type walEntryType byte + +const ( + pointsEntry walEntryType = 0x01 + fieldsEntry walEntryType = 0x02 + seriesEntry walEntryType = 0x03 +) + +type Log struct { + path string + + flushCheckTimer *time.Timer // check this often to see if a background flush should happen + flushCheckInterval time.Duration + + // write variables + writeLock sync.Mutex + currentSegmentID int + currentSegmentFile *os.File + currentSegmentSize int + lastWriteTime time.Time + flushRunning bool + + // cache variables + cacheLock sync.RWMutex + cache map[string]Values + cacheDirtySort map[string]bool // this map should be small, only for dirty vals + flushCache map[string]Values // temporary map while flushing + memorySize int + measurementFieldsCache map[string]*tsdb.MeasurementFields + seriesToCreateCache []*tsdb.SeriesCreate + + // These coordinate closing and waiting for running goroutines. + wg sync.WaitGroup + closing chan struct{} + + // LogOutput is the writer used by the logger. + LogOutput io.Writer + logger *log.Logger + + // FlushColdInterval is the period of time after which a partition will do a + // full flush and compaction if it has been cold for writes. + FlushColdInterval time.Duration + + // SegmentSize is the file size at which a segment file will be rotated + SegmentSize int + + // MemorySizeThreshold specifies when the log should be forced to be flushed. + MemorySizeThreshold int + + // Index is the database series will be flushed to + Index IndexWriter + + // LoggingEnabled specifies if detailed logs should be output + LoggingEnabled bool + + // SkipCache specifies if the wal should immediately write to the index instead of + // caching data in memory. False by default so we buffer in memory before flushing to index. + SkipCache bool + + // SkipDurability specifies if the wal should not write the wal entries to disk. + // False by default which means all writes are durable even when cached before flushing to index. + SkipDurability bool +} + +// IndexWriter is an interface for the indexed database the WAL flushes data to +type IndexWriter interface { + WriteAndCompact(valuesByKey map[string]Values, measurementFieldsToSave map[string]*tsdb.MeasurementFields, seriesToCreate []*tsdb.SeriesCreate) error +} + +func NewLog(path string) *Log { + return &Log{ + path: path, + + // these options should be overriden by any options in the config + LogOutput: os.Stderr, + FlushColdInterval: tsdb.DefaultFlushColdInterval, + SegmentSize: DefaultSegmentSize, + MemorySizeThreshold: tsdb.DefaultPartitionSizeThreshold, + flushCheckInterval: defaultFlushCheckInterval, + logger: log.New(os.Stderr, "[pwl] ", log.LstdFlags), + } +} + +// Open opens and initializes the Log. Will recover from previous unclosed shutdowns +func (l *Log) Open() error { + + if l.LoggingEnabled { + l.logger.Printf("PD1 WAL starting with %d memory size threshold\n", l.MemorySizeThreshold) + l.logger.Printf("WAL writing to %s\n", l.path) + } + if err := os.MkdirAll(l.path, 0777); err != nil { + return err + } + + l.cache = make(map[string]Values) + l.cacheDirtySort = make(map[string]bool) + l.measurementFieldsCache = make(map[string]*tsdb.MeasurementFields) + // TODO: read segment files and flush them all to disk + + l.flushCheckTimer = time.NewTimer(l.flushCheckInterval) + + // Start background goroutines. + l.wg.Add(1) + l.closing = make(chan struct{}) + go l.autoflusher(l.closing) + + return nil +} + +// Cursor will return a cursor object to Seek and iterate with Next for the WAL cache for the given +func (l *Log) Cursor(key string, direction tsdb.Direction) tsdb.Cursor { + l.cacheLock.RLock() + defer l.cacheLock.RUnlock() + + // TODO: make this work for other fields + ck := seriesFieldKey(key, "value") + values := l.cache[ck] + + // if we're in the middle of a flush, combine the previous cache + // with this one for the cursor + if l.flushCache != nil { + if fc, ok := l.flushCache[ck]; ok { + c := make([]Value, len(fc), len(fc)+len(values)) + copy(c, fc) + c = append(c, values...) + + return newWALCursor(c, direction) + } + } + + if l.cacheDirtySort[ck] { + sort.Sort(values) + delete(l.cacheDirtySort, ck) + } + + // build a copy so writes afterwards don't change the result set + a := make([]Value, len(values)) + copy(a, values) + return newWALCursor(a, direction) +} + +func (l *Log) WritePoints(points []tsdb.Point, fields map[string]*tsdb.MeasurementFields, series []*tsdb.SeriesCreate) error { + // make the write durable if specified + if !l.SkipDurability { + pointStrings := make([]string, len(points)) + for i, p := range points { + pointStrings[i] = p.String() + } + data := strings.Join(pointStrings, "\n") + compressed := snappy.Encode(nil, []byte(data)) + + if err := l.writeToLog(pointsEntry, compressed); err != nil { + return err + } + + // TODO: write the fields + + // TODO: write the series + } + + // convert to values that can be either cached in memory or flushed to the index + l.cacheLock.Lock() + for _, p := range points { + for name, value := range p.Fields() { + k := seriesFieldKey(string(p.Key()), name) + v := NewValue(p.Time(), value) + cacheValues := l.cache[k] + + // only mark it as dirty if it isn't already + if _, ok := l.cacheDirtySort[k]; !ok && len(cacheValues) > 0 { + dirty := cacheValues[len(cacheValues)-1].Time().UnixNano() > v.Time().UnixNano() + if dirty { + l.cacheDirtySort[k] = true + } + } + l.memorySize += v.Size() + l.cache[k] = append(cacheValues, v) + } + } + + for k, v := range fields { + l.measurementFieldsCache[k] = v + } + l.seriesToCreateCache = append(l.seriesToCreateCache, series...) + l.lastWriteTime = time.Now() + l.cacheLock.Unlock() + + // usually skipping the cache is only for testing purposes and this was the easiest + // way to represent the logic (to cache and then immediately flush) + if l.SkipCache { + l.flush(idleFlush) + } + + return nil +} + +func (l *Log) writeToLog(writeType walEntryType, data []byte) error { + l.writeLock.Lock() + defer l.writeLock.Unlock() + + if l.currentSegmentFile == nil { + l.newSegmentFile() + } + + if _, err := l.currentSegmentFile.Write([]byte{byte(writeType)}); err != nil { + panic(fmt.Sprintf("error writing type to wal: %s", err.Error())) + } + if _, err := l.currentSegmentFile.Write(u32tob(uint32(len(data)))); err != nil { + panic(fmt.Sprintf("error writing len to wal: %s", err.Error())) + } + if _, err := l.currentSegmentFile.Write(data); err != nil { + panic(fmt.Sprintf("error writing data to wal: %s", err.Error())) + } + + return l.currentSegmentFile.Sync() +} + +// Flush will force a flush of the WAL to the index +func (l *Log) Flush() error { + return l.flush(idleFlush) +} + +func (l *Log) DeleteSeries(keys []string) error { + panic("not implemented") +} + +// Close will finish any flush that is currently in process and close file handles +func (l *Log) Close() error { + // stop the autoflushing process so it doesn't try to kick another one off + l.writeLock.Lock() + l.cacheLock.Lock() + + if l.closing != nil { + close(l.closing) + l.closing = nil + } + l.writeLock.Unlock() + l.cacheLock.Unlock() + + // Allow goroutines to finish running. + l.wg.Wait() + + // Lock the remainder of the closing process. + l.writeLock.Lock() + l.cacheLock.Lock() + defer l.writeLock.Unlock() + defer l.cacheLock.Unlock() + + l.cache = nil + l.measurementFieldsCache = nil + l.seriesToCreateCache = nil + if l.currentSegmentFile == nil { + return nil + } + if err := l.currentSegmentFile.Close(); err != nil { + return err + } + l.currentSegmentFile = nil + + return nil +} + +// close all the open Log partitions and file handles +func (l *Log) close() error { + l.cache = nil + l.cacheDirtySort = nil + if l.currentSegmentFile == nil { + return nil + } + if err := l.currentSegmentFile.Close(); err != nil { + return err + } + l.currentSegmentFile = nil + + return nil +} + +// flush writes all wal data in memory to the index +func (l *Log) flush(flush flushType) error { + l.writeLock.Lock() + if l.flushRunning { + l.writeLock.Unlock() + return nil + } + + l.flushRunning = true + defer func() { + l.writeLock.Lock() + l.flushRunning = false + l.writeLock.Unlock() + }() + lastFileID := l.currentSegmentID + if err := l.newSegmentFile(); err != nil { + // there's no recovering from this, fail hard + panic(fmt.Sprintf("error creating new wal file: %s", err.Error())) + } + l.writeLock.Unlock() + + // copy the cache items to new maps so we can empty them out + l.cacheLock.Lock() + + // move over the flush cache and make a copy to write + l.flushCache = l.cache + l.cache = make(map[string]Values) + l.cacheDirtySort = make(map[string]bool) + valuesByKey := make(map[string]Values) + + valueCount := 0 + for key, v := range l.flushCache { + valuesByKey[key] = v + valueCount += len(v) + } + + if l.LoggingEnabled { + ftype := "idle" + if flush == memoryFlush { + ftype = "memory" + } + l.logger.Printf("%s flush of %d keys with %d values of %d bytes\n", ftype, len(valuesByKey), valueCount, l.memorySize) + } + + // reset the memory being used by the cache + l.memorySize = 0 + + // reset the measurements for flushing + mfc := l.measurementFieldsCache + l.measurementFieldsCache = make(map[string]*tsdb.MeasurementFields) + + // reset the series for flushing + scc := l.seriesToCreateCache + l.seriesToCreateCache = nil + + l.cacheLock.Unlock() + + startTime := time.Now() + if err := l.Index.WriteAndCompact(valuesByKey, mfc, scc); err != nil { + return err + } + if l.LoggingEnabled { + l.logger.Printf("flush to index took %s\n", time.Since(startTime)) + } + + l.cacheLock.Lock() + l.flushCache = nil + l.cacheLock.Unlock() + + // remove all the old segment files + fileNames, err := l.segmentFileNames() + if err != nil { + return err + } + for _, fn := range fileNames { + id, err := idFromFileName(fn) + if err != nil { + return err + } + if id <= lastFileID { + err := os.Remove(fn) + if err != nil { + return err + } + } + } + + return nil +} + +// triggerAutoFlush will flush and compact any partitions that have hit the thresholds for compaction +func (l *Log) triggerAutoFlush() { + if f := l.shouldFlush(); f != noFlush { + if err := l.flush(f); err != nil { + l.logger.Printf("error flushing wal: %s\n", err) + } + } +} + +// autoflusher waits for notification of a flush and kicks it off in the background. +// This method runs in a separate goroutine. +func (l *Log) autoflusher(closing chan struct{}) { + defer l.wg.Done() + + for { + // Wait for close or flush signal. + select { + case <-closing: + return + case <-l.flushCheckTimer.C: + l.triggerAutoFlush() + l.flushCheckTimer.Reset(l.flushCheckInterval) + } + } +} + +// segmentFileNames will return all files that are WAL segment files in sorted order by ascending ID +func (l *Log) segmentFileNames() ([]string, error) { + names, err := filepath.Glob(filepath.Join(l.path, 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 *Log) newSegmentFile() error { + l.currentSegmentID += 1 + if l.currentSegmentFile != nil { + if err := l.currentSegmentFile.Close(); err != nil { + return err + } + } + + fileName := filepath.Join(l.path, fmt.Sprintf("%s%05d.%s", WALFilePrefix, l.currentSegmentID, WALFileExtension)) + ff, err := os.OpenFile(fileName, os.O_CREATE|os.O_RDWR, 0666) + if err != nil { + return err + } + l.currentSegmentSize = 0 + l.currentSegmentFile = ff + + return nil +} + +// shouldFlush +func (l *Log) shouldFlush() flushType { + l.cacheLock.RLock() + defer l.cacheLock.RUnlock() + + if len(l.cache) == 0 { + return noFlush + } + + if l.memorySize > l.MemorySizeThreshold { + return memoryFlush + } + + if time.Since(l.lastWriteTime) > l.FlushColdInterval { + return idleFlush + } + + return noFlush +} + +// cursor is a unidirectional iterator for a given entry in the cache +type walCursor struct { + cache Values + position int + direction tsdb.Direction +} + +func newWALCursor(cache Values, direction tsdb.Direction) *walCursor { + // position is set such that a call to Next will successfully advance + // to the next postion and return the value. + c := &walCursor{cache: cache, direction: direction, position: -1} + if direction.Reverse() { + c.position = len(c.cache) + } + return c +} + +func (c *walCursor) Direction() tsdb.Direction { return c.direction } + +// Seek will point the cursor to the given time (or key) +func (c *walCursor) Seek(seek []byte) (key, value []byte) { + // Seek cache index + c.position = sort.Search(len(c.cache), func(i int) bool { + return bytes.Compare(c.cache[i].TimeBytes(), seek) != -1 + }) + + // If seek is not in the cache, return the last value in the cache + if c.direction.Reverse() && c.position >= len(c.cache) { + c.position = len(c.cache) + } + + // Make sure our position points to something in the cache + if c.position < 0 || c.position >= len(c.cache) { + return nil, nil + } + + v := c.cache[c.position] + + return v.TimeBytes(), v.ValueBytes() +} + +// Next moves the cursor to the next key/value. will return nil if at the end +func (c *walCursor) Next() (key, value []byte) { + var v Value + if c.direction.Forward() { + v = c.nextForward() + } else { + v = c.nextReverse() + } + + return v.TimeBytes(), v.ValueBytes() +} + +// nextForward advances the cursor forward returning the next value +func (c *walCursor) nextForward() Value { + c.position++ + + if c.position >= len(c.cache) { + return &EmptyValue{} + } + + return c.cache[c.position] +} + +// nextReverse advances the cursor backwards returning the next value +func (c *walCursor) nextReverse() Value { + c.position-- + + if c.position < 0 { + return &EmptyValue{} + } + + return c.cache[c.position] +} + +// 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 +} From 2100e66437f9bcad8c9dbb5804628ceb25a66eb3 Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Thu, 17 Sep 2015 11:23:27 -0400 Subject: [PATCH 074/139] Add full durability to WAL and flush on startup --- tsdb/engine/pd1/wal.go | 184 ++++++++++++++++++++++++++++++++---- tsdb/engine/pd1/wal_test.go | 170 +++++++++++++++++++++++++++++++++ 2 files changed, 333 insertions(+), 21 deletions(-) create mode 100644 tsdb/engine/pd1/wal_test.go diff --git a/tsdb/engine/pd1/wal.go b/tsdb/engine/pd1/wal.go index 26f2af48ffe..03548f062e7 100644 --- a/tsdb/engine/pd1/wal.go +++ b/tsdb/engine/pd1/wal.go @@ -2,6 +2,7 @@ package pd1 import ( "bytes" + "encoding/json" "fmt" "io" "log" @@ -28,6 +29,8 @@ const ( // defaultFlushCheckInterval is how often flushes are triggered automatically by the flush criteria defaultFlushCheckInterval = time.Second + + writeBufLen = 32 << 10 // 32kb ) // flushType indiciates why a flush and compaction are being run so the partition can @@ -45,8 +48,8 @@ const ( idleFlush // deleteFlush indicates that we're flushing because series need to be removed from the WAL deleteFlush - - writeBufLen = 32 << 10 // 32kb + // startupFlush indicates that we're flushing because the database is starting up + startupFlush ) // walEntry is a byte written to a wal segment file that indicates what the following compressed block contains @@ -129,7 +132,7 @@ func NewLog(path string) *Log { SegmentSize: DefaultSegmentSize, MemorySizeThreshold: tsdb.DefaultPartitionSizeThreshold, flushCheckInterval: defaultFlushCheckInterval, - logger: log.New(os.Stderr, "[pwl] ", log.LstdFlags), + logger: log.New(os.Stderr, "[pd1wal] ", log.LstdFlags), } } @@ -138,7 +141,7 @@ func (l *Log) Open() error { if l.LoggingEnabled { l.logger.Printf("PD1 WAL starting with %d memory size threshold\n", l.MemorySizeThreshold) - l.logger.Printf("WAL writing to %s\n", l.path) + l.logger.Printf("PD1 WAL writing to %s\n", l.path) } if err := os.MkdirAll(l.path, 0777); err != nil { return err @@ -147,7 +150,11 @@ func (l *Log) Open() error { l.cache = make(map[string]Values) l.cacheDirtySort = make(map[string]bool) l.measurementFieldsCache = make(map[string]*tsdb.MeasurementFields) - // TODO: read segment files and flush them all to disk + + // flush out any WAL entries that are there from before + if err := l.readAndFlushWAL(); err != nil { + return err + } l.flushCheckTimer = time.NewTimer(l.flushCheckInterval) @@ -194,6 +201,7 @@ func (l *Log) Cursor(key string, direction tsdb.Direction) tsdb.Cursor { func (l *Log) WritePoints(points []tsdb.Point, fields map[string]*tsdb.MeasurementFields, series []*tsdb.SeriesCreate) error { // make the write durable if specified if !l.SkipDurability { + // write the points pointStrings := make([]string, len(points)) for i, p := range points { pointStrings[i] = p.String() @@ -205,13 +213,47 @@ func (l *Log) WritePoints(points []tsdb.Point, fields map[string]*tsdb.Measureme return err } - // TODO: write the fields + // write the new fields + if len(fields) > 0 { + data, err := json.Marshal(fields) + if err != nil { + return err + } + compressed = snappy.Encode(compressed, data) + if err := l.writeToLog(fieldsEntry, compressed); err != nil { + return err + } + } - // TODO: write the series + // write the new series + if len(series) > 0 { + data, err := json.Marshal(series) + if err != nil { + return err + } + compressed = snappy.Encode(compressed, data) + if err := l.writeToLog(seriesEntry, compressed); err != nil { + return err + } + } + } + + // add everything to the cache + l.addToCache(points, fields, series) + + // usually skipping the cache is only for testing purposes and this was the easiest + // way to represent the logic (to cache and then immediately flush) + if l.SkipCache { + l.flush(idleFlush) } - // convert to values that can be either cached in memory or flushed to the index + return nil +} + +func (l *Log) addToCache(points []tsdb.Point, fields map[string]*tsdb.MeasurementFields, series []*tsdb.SeriesCreate) { l.cacheLock.Lock() + defer l.cacheLock.Unlock() + for _, p := range points { for name, value := range p.Fields() { k := seriesFieldKey(string(p.Key()), name) @@ -235,25 +277,114 @@ func (l *Log) WritePoints(points []tsdb.Point, fields map[string]*tsdb.Measureme } l.seriesToCreateCache = append(l.seriesToCreateCache, series...) l.lastWriteTime = time.Now() - l.cacheLock.Unlock() +} - // usually skipping the cache is only for testing purposes and this was the easiest - // way to represent the logic (to cache and then immediately flush) - if l.SkipCache { - l.flush(idleFlush) +// readAndFlushWAL is called on open and will read the segment files in, flushing whenever +// the memory gets over the limit. Once all files have been read it will flush and remove the files +func (l *Log) readAndFlushWAL() error { + files, err := l.segmentFileNames() + if err != nil { + return err + } + + // read all the segment files and cache them, flushing along the way if we + // hit memory limits + for _, fn := range files { + if err := l.readFileToCache(fn); err != nil { + return err + } + + if l.memorySize > l.MemorySizeThreshold { + if err := l.flush(memoryFlush); err != nil { + return err + } + } + } + + // now flush and remove all the old files + if err := l.flush(startupFlush); err != nil { + return err } return nil } +func (l *Log) readFileToCache(fileName string) error { + f, err := os.OpenFile(fileName, os.O_RDONLY, 0666) + if err != nil { + return err + } + defer f.Close() + + buf := make([]byte, writeBufLen) + data := make([]byte, writeBufLen) + for { + // read the type and the length of the entry + _, err := io.ReadFull(f, buf[0:5]) + if err == io.EOF { + return nil + } else if err != nil { + l.logger.Printf("error reading segment file %s: %s", fileName, err.Error()) + return err + } + entryType := buf[0] + length := btou32(buf[1:5]) + + // read the compressed block and decompress it + if int(length) > len(buf) { + buf = make([]byte, length) + } + _, err = io.ReadFull(f, buf[0:length]) + if err == io.EOF { + l.logger.Printf("hit end of file while reading compressed wal entry from %s", fileName) + return nil + } else if err != nil { + return err + } + data, err = snappy.Decode(data, buf[0:length]) + if err != nil { + l.logger.Printf("error decoding compressed entry from %s: %s", fileName, err.Error()) + return nil + } + + // and marshal it and send it to the cache + switch walEntryType(entryType) { + case pointsEntry: + points, err := tsdb.ParsePoints(data) + if err != nil { + return err + } + l.addToCache(points, nil, nil) + case fieldsEntry: + fields := make(map[string]*tsdb.MeasurementFields) + if err := json.Unmarshal(data, &fields); err != nil { + return err + } + l.addToCache(nil, fields, nil) + case seriesEntry: + series := make([]*tsdb.SeriesCreate, 0) + if err := json.Unmarshal(data, &series); err != nil { + return err + } + l.addToCache(nil, nil, series) + } + } +} + func (l *Log) writeToLog(writeType walEntryType, data []byte) error { l.writeLock.Lock() defer l.writeLock.Unlock() if l.currentSegmentFile == nil { - l.newSegmentFile() + if err := l.newSegmentFile(); err != nil { + // fail hard since we can't write data + panic(fmt.Sprintf("error opening new segment file for wal: %s", err.Error())) + } } + // The panics here are an intentional choice. Based on reports from users + // it's better to fail hard if the database can't take writes. Then they'll + // get alerted and fix whatever is broken. Remove these and face Paul's wrath. if _, err := l.currentSegmentFile.Write([]byte{byte(writeType)}); err != nil { panic(fmt.Sprintf("error writing type to wal: %s", err.Error())) } @@ -329,12 +460,14 @@ func (l *Log) close() error { // flush writes all wal data in memory to the index func (l *Log) flush(flush flushType) error { + // only flush if there isn't one already running l.writeLock.Lock() if l.flushRunning { l.writeLock.Unlock() return nil } + // only hold the lock while we rotate the segment file l.flushRunning = true defer func() { l.writeLock.Lock() @@ -363,13 +496,7 @@ func (l *Log) flush(flush flushType) error { valueCount += len(v) } - if l.LoggingEnabled { - ftype := "idle" - if flush == memoryFlush { - ftype = "memory" - } - l.logger.Printf("%s flush of %d keys with %d values of %d bytes\n", ftype, len(valuesByKey), valueCount, l.memorySize) - } + flushSize := l.memorySize // reset the memory being used by the cache l.memorySize = 0 @@ -384,6 +511,21 @@ func (l *Log) flush(flush flushType) error { l.cacheLock.Unlock() + // exit if there's nothing to flush to the index + if len(valuesByKey) == 0 && len(mfc) == 0 && len(scc) == 0 { + return nil + } + + if l.LoggingEnabled { + ftype := "idle" + if flush == memoryFlush { + ftype = "memory" + } else if flush == startupFlush { + ftype = "startup" + } + l.logger.Printf("%s flush of %d keys with %d values of %d bytes\n", ftype, len(valuesByKey), valueCount, flushSize) + } + startTime := time.Now() if err := l.Index.WriteAndCompact(valuesByKey, mfc, scc); err != nil { return err diff --git a/tsdb/engine/pd1/wal_test.go b/tsdb/engine/pd1/wal_test.go new file mode 100644 index 00000000000..c1ef614650c --- /dev/null +++ b/tsdb/engine/pd1/wal_test.go @@ -0,0 +1,170 @@ +package pd1_test + +import ( + "io/ioutil" + "os" + "reflect" + "testing" + + "github.com/influxdb/influxdb/tsdb" + "github.com/influxdb/influxdb/tsdb/engine/pd1" +) + +func TestWAL_TestWriteQueryOpen(t *testing.T) { + w := NewWAL() + defer w.Cleanup() + + var vals map[string]pd1.Values + var fields map[string]*tsdb.MeasurementFields + var series []*tsdb.SeriesCreate + + w.Index = &MockIndexWriter{ + fn: func(valuesByKey map[string]pd1.Values, measurementFieldsToSave map[string]*tsdb.MeasurementFields, seriesToCreate []*tsdb.SeriesCreate) error { + vals = valuesByKey + fields = measurementFieldsToSave + series = seriesToCreate + return nil + }, + } + + if err := w.Open(); err != nil { + t.Fatalf("error opening: %s", err.Error) + } + + p1 := parsePoint("cpu,host=A value=1.1 1000000000") + p2 := parsePoint("cpu,host=B value=1.2 1000000000") + p3 := parsePoint("cpu,host=A value=2.1 2000000000") + p4 := parsePoint("cpu,host=B value=2.2 2000000000") + fieldsToWrite := map[string]*tsdb.MeasurementFields{"foo": {Fields: map[string]*tsdb.Field{"bar": {Name: "value"}}}} + seriesToWrite := []*tsdb.SeriesCreate{{Measurement: "asdf"}} + + if err := w.WritePoints([]tsdb.Point{p1, p2}, fieldsToWrite, seriesToWrite); err != nil { + t.Fatalf("failed to write points: %s", err.Error()) + } + + c := w.Cursor("cpu,host=A", tsdb.Forward) + k, v := c.Next() + if btou64(k) != uint64(p1.UnixNano()) { + t.Fatalf("p1 time wrong:\n\texp:%d\n\tgot:%d\n", p1.UnixNano(), btou64(k)) + } + if 1.1 != btof64(v) { + t.Fatal("p1 data not equal") + } + c = w.Cursor("cpu,host=B", tsdb.Forward) + k, v = c.Next() + if btou64(k) != uint64(p2.UnixNano()) { + t.Fatalf("p2 time wrong:\n\texp:%d\n\tgot:%d\n", p2.UnixNano(), btou64(k)) + } + if 1.2 != btof64(v) { + t.Fatal("p2 data not equal") + } + + k, v = c.Next() + if k != nil { + t.Fatal("expected nil") + } + + // ensure we can do another write to the wal and get stuff + if err := w.WritePoints([]tsdb.Point{p3}, nil, nil); err != nil { + t.Fatalf("failed to write: %s", err.Error) + } + + c = w.Cursor("cpu,host=A", tsdb.Forward) + k, v = c.Next() + if btou64(k) != uint64(p1.UnixNano()) { + t.Fatalf("p1 time wrong:\n\texp:%d\n\tgot:%d\n", p1.UnixNano(), btou64(k)) + } + if 1.1 != btof64(v) { + t.Fatal("p1 data not equal") + } + k, v = c.Next() + if btou64(k) != uint64(p3.UnixNano()) { + t.Fatalf("p3 time wrong:\n\texp:%d\n\tgot:%d\n", p3.UnixNano(), btou64(k)) + } + if 2.1 != btof64(v) { + t.Fatal("p3 data not equal") + } + + // ensure we can seek + k, v = c.Seek(u64tob(2000000000)) + if btou64(k) != uint64(p3.UnixNano()) { + t.Fatalf("p3 time wrong:\n\texp:%d\n\tgot:%d\n", p3.UnixNano(), btou64(k)) + } + if 2.1 != btof64(v) { + t.Fatal("p3 data not equal") + } + k, v = c.Next() + if k != nil { + t.Fatal("expected nil") + } + + // ensure we close and after open it flushes to the index + if err := w.Close(); err != nil { + t.Fatalf("failed to close: %s", err.Error()) + } + + if err := w.Open(); err != nil { + t.Fatalf("failed to open: %s", err.Error()) + } + + if len(vals["cpu,host=A#value"]) != 2 { + t.Fatal("expected host A values to flush to index on open") + } + + if len(vals["cpu,host=B#value"]) != 1 { + t.Fatal("expected host B values to flush to index on open") + } + + if err := w.WritePoints([]tsdb.Point{p4}, nil, nil); err != nil { + t.Fatalf("failed to write: %s", err.Error) + } + c = w.Cursor("cpu,host=B", tsdb.Forward) + k, v = c.Next() + if btou64(k) != uint64(p4.UnixNano()) { + t.Fatalf("p4 time wrong:\n\texp:%d\n\tgot:%d\n", p4.UnixNano(), btou64(k)) + } + if 2.2 != btof64(v) { + t.Fatal("p4 data not equal") + } + + if !reflect.DeepEqual(fields, fieldsToWrite) { + t.Fatal("fields not flushed") + } + + if !reflect.DeepEqual(series, seriesToWrite) { + t.Fatal("series not flushed") + } +} + +type Log struct { + *pd1.Log + path string +} + +func NewWAL() *Log { + dir, err := ioutil.TempDir("", "pd1-test") + if err != nil { + panic("couldn't get temp dir") + } + + l := &Log{ + Log: pd1.NewLog(dir), + path: dir, + } + l.LoggingEnabled = true + return l +} + +func (l *Log) Cleanup() error { + l.Close() + os.RemoveAll(l.path) + return nil +} + +type MockIndexWriter struct { + fn func(valuesByKey map[string]pd1.Values, measurementFieldsToSave map[string]*tsdb.MeasurementFields, seriesToCreate []*tsdb.SeriesCreate) error +} + +func (m *MockIndexWriter) WriteAndCompact(valuesByKey map[string]pd1.Values, measurementFieldsToSave map[string]*tsdb.MeasurementFields, seriesToCreate []*tsdb.SeriesCreate) error { + return m.fn(valuesByKey, measurementFieldsToSave, seriesToCreate) +} From 5e59cb93932eeb6b3f725e721d34c596192f7e62 Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Thu, 17 Sep 2015 15:46:37 -0400 Subject: [PATCH 075/139] Update encoding test to work with new interface. --- tsdb/engine/pd1/encoding_test.go | 31 +++++++++++++++---------------- 1 file changed, 15 insertions(+), 16 deletions(-) diff --git a/tsdb/engine/pd1/encoding_test.go b/tsdb/engine/pd1/encoding_test.go index aa5a4b15e73..82968912b0a 100644 --- a/tsdb/engine/pd1/encoding_test.go +++ b/tsdb/engine/pd1/encoding_test.go @@ -2,31 +2,30 @@ package pd1_test import ( // "math/rand" - // "reflect" + "fmt" + "reflect" "testing" "time" - // "github.com/influxdb/influxdb/tsdb/engine/pd1" + "github.com/influxdb/influxdb/tsdb/engine/pd1" ) func TestEncoding_FloatBlock(t *testing.T) { - // valueCount := 100 - // times := getTimes(valueCount, 60, time.Second) - // values := make([]Value, len(times)) - // for i, t := range times { - // values[i] = pd1.NewValue(t, rand.Float64()) - // } + valueCount := 1000 + times := getTimes(valueCount, 60, time.Second) + values := make(pd1.Values, len(times)) + for i, t := range times { + values[i] = pd1.NewValue(t, float64(i)) + } - // b := pd1.EncodeFloatBlock(nil, values) + b := values.Encode(nil) + fmt.Println("**** ", len(b)) - // decodedValues, err := pd1.DecodeFloatBlock(b) - // if err != nil { - // t.Fatalf("error decoding: %s", err.Error) - // } + decodedValues := values.DecodeSameTypeBlock(b) - // if !reflect.DeepEqual(decodedValues, values) { - // t.Fatalf("unexpected results:\n\tgot: %v\n\texp: %v\n", decodedValues, values) - // } + if !reflect.DeepEqual(decodedValues, values) { + t.Fatalf("unexpected results:\n\tgot: %v\n\texp: %v\n", decodedValues, values) + } } func getTimes(n, step int, precision time.Duration) []time.Time { From fe1f9a51e52162b13a7d7478b033ca43b898a8f9 Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Thu, 17 Sep 2015 15:46:53 -0400 Subject: [PATCH 076/139] Add memory settings and WAL backpressure --- tsdb/config.go | 22 +++++++--- tsdb/engine/pd1/pd1.go | 3 +- tsdb/engine/pd1/wal.go | 92 ++++++++++++++++++++++++++---------------- 3 files changed, 76 insertions(+), 41 deletions(-) diff --git a/tsdb/config.go b/tsdb/config.go index dfd267d2c37..b7bc409b4a9 100644 --- a/tsdb/config.go +++ b/tsdb/config.go @@ -43,6 +43,10 @@ const ( // This number multiplied by the parition count is roughly the max possible memory // size for the in-memory WAL cache. DefaultPartitionSizeThreshold = 50 * 1024 * 1024 // 50MB + + // Default WAL settings for the PD1 WAL + DefaultFlushMemorySizeThreshold = 50 * 1024 * 1024 // 50MB + DefaultMaxMemorySizeThreshold = 200 * 1024 * 1024 // 200MB ) type Config struct { @@ -63,6 +67,10 @@ type Config struct { WALFlushColdInterval toml.Duration `toml:"wal-flush-cold-interval"` WALPartitionSizeThreshold uint64 `toml:"wal-partition-size-threshold"` + // WAL configuration options for pd1 introduced in 0.9.5 + WALFlushMemorySizeThreshold int `toml:"wal-flush-memory-size-threshold"` + WALMaxMemorySizeThreshold int `toml:"wal-max-memory-size-threshold"` + // Query logging QueryLogEnabled bool `toml:"query-log-enabled"` } @@ -74,12 +82,14 @@ func NewConfig() Config { WALFlushInterval: toml.Duration(DefaultWALFlushInterval), WALPartitionFlushDelay: toml.Duration(DefaultWALPartitionFlushDelay), - WALLoggingEnabled: true, - WALReadySeriesSize: DefaultReadySeriesSize, - WALCompactionThreshold: DefaultCompactionThreshold, - WALMaxSeriesSize: DefaultMaxSeriesSize, - WALFlushColdInterval: toml.Duration(DefaultFlushColdInterval), - WALPartitionSizeThreshold: DefaultPartitionSizeThreshold, + WALLoggingEnabled: true, + WALReadySeriesSize: DefaultReadySeriesSize, + WALCompactionThreshold: DefaultCompactionThreshold, + WALMaxSeriesSize: DefaultMaxSeriesSize, + WALFlushColdInterval: toml.Duration(DefaultFlushColdInterval), + WALPartitionSizeThreshold: DefaultPartitionSizeThreshold, + WALFlushMemorySizeThreshold: DefaultFlushMemorySizeThreshold, + WALMaxMemorySizeThreshold: DefaultMaxMemorySizeThreshold, QueryLogEnabled: true, } diff --git a/tsdb/engine/pd1/pd1.go b/tsdb/engine/pd1/pd1.go index ef91fb66e2d..02708d7b45f 100644 --- a/tsdb/engine/pd1/pd1.go +++ b/tsdb/engine/pd1/pd1.go @@ -89,7 +89,8 @@ type Engine struct { func NewEngine(path string, walPath string, opt tsdb.EngineOptions) tsdb.Engine { w := NewLog(path) w.FlushColdInterval = time.Duration(opt.Config.WALFlushColdInterval) - w.MemorySizeThreshold = int(opt.Config.WALPartitionSizeThreshold) + w.FlushMemorySizeThreshold = opt.Config.WALFlushMemorySizeThreshold + w.MaxMemorySizeThreshold = opt.Config.WALMaxMemorySizeThreshold w.LoggingEnabled = opt.Config.WALLoggingEnabled e := &Engine{ diff --git a/tsdb/engine/pd1/wal.go b/tsdb/engine/pd1/wal.go index 03548f062e7..e3157a94b03 100644 --- a/tsdb/engine/pd1/wal.go +++ b/tsdb/engine/pd1/wal.go @@ -72,11 +72,11 @@ type Log struct { currentSegmentID int currentSegmentFile *os.File currentSegmentSize int - lastWriteTime time.Time - flushRunning bool - // cache variables + // cache and flush variables cacheLock sync.RWMutex + lastWriteTime time.Time + flushRunning bool cache map[string]Values cacheDirtySort map[string]bool // this map should be small, only for dirty vals flushCache map[string]Values // temporary map while flushing @@ -99,8 +99,11 @@ type Log struct { // SegmentSize is the file size at which a segment file will be rotated SegmentSize int - // MemorySizeThreshold specifies when the log should be forced to be flushed. - MemorySizeThreshold int + // FlushMemorySizeThreshold specifies when the log should be forced to be flushed + FlushMemorySizeThreshold int + + // MaxMemorySizeThreshold specifies the limit at which writes to the WAL should be rejected + MaxMemorySizeThreshold int // Index is the database series will be flushed to Index IndexWriter @@ -127,12 +130,13 @@ func NewLog(path string) *Log { path: path, // these options should be overriden by any options in the config - LogOutput: os.Stderr, - FlushColdInterval: tsdb.DefaultFlushColdInterval, - SegmentSize: DefaultSegmentSize, - MemorySizeThreshold: tsdb.DefaultPartitionSizeThreshold, - flushCheckInterval: defaultFlushCheckInterval, - logger: log.New(os.Stderr, "[pd1wal] ", log.LstdFlags), + LogOutput: os.Stderr, + FlushColdInterval: tsdb.DefaultFlushColdInterval, + SegmentSize: DefaultSegmentSize, + FlushMemorySizeThreshold: tsdb.DefaultFlushMemorySizeThreshold, + MaxMemorySizeThreshold: tsdb.DefaultMaxMemorySizeThreshold, + flushCheckInterval: defaultFlushCheckInterval, + logger: log.New(os.Stderr, "[pd1wal] ", log.LstdFlags), } } @@ -140,7 +144,7 @@ func NewLog(path string) *Log { func (l *Log) Open() error { if l.LoggingEnabled { - l.logger.Printf("PD1 WAL starting with %d memory size threshold\n", l.MemorySizeThreshold) + l.logger.Printf("PD1 WAL starting with %d flush memory size threshold and %d max memory size threshold\n", l.FlushMemorySizeThreshold, l.MaxMemorySizeThreshold) l.logger.Printf("PD1 WAL writing to %s\n", l.path) } if err := os.MkdirAll(l.path, 0777); err != nil { @@ -199,6 +203,11 @@ func (l *Log) Cursor(key string, direction tsdb.Direction) tsdb.Cursor { } func (l *Log) WritePoints(points []tsdb.Point, fields map[string]*tsdb.MeasurementFields, series []*tsdb.SeriesCreate) error { + // add everything to the cache, or return an error if we've hit our max memory + if addedToCache := l.addToCache(points, fields, series, true); !addedToCache { + return fmt.Errorf("WAL backed up flushing to index, hit max memory") + } + // make the write durable if specified if !l.SkipDurability { // write the points @@ -238,9 +247,6 @@ func (l *Log) WritePoints(points []tsdb.Point, fields map[string]*tsdb.Measureme } } - // add everything to the cache - l.addToCache(points, fields, series) - // usually skipping the cache is only for testing purposes and this was the easiest // way to represent the logic (to cache and then immediately flush) if l.SkipCache { @@ -250,10 +256,23 @@ func (l *Log) WritePoints(points []tsdb.Point, fields map[string]*tsdb.Measureme return nil } -func (l *Log) addToCache(points []tsdb.Point, fields map[string]*tsdb.MeasurementFields, series []*tsdb.SeriesCreate) { +// addToCache will add the points, measurements, and fields to the cache and return true if successful. They will be queryable +// immediately after return and will be flushed at the next flush cycle. Before adding to the cache we check if we're over the +// max memory threshold. If we are we request a flush in a new goroutine and return false, indicating we didn't add the values +// to the cache and that writes should return a failure. +func (l *Log) addToCache(points []tsdb.Point, fields map[string]*tsdb.MeasurementFields, series []*tsdb.SeriesCreate, checkMemory bool) bool { l.cacheLock.Lock() defer l.cacheLock.Unlock() + // if we should check memory and we're over the threshold, mark a flush as running and kick one off in a goroutine + if checkMemory && l.memorySize > l.MaxMemorySizeThreshold { + if !l.flushRunning { + l.flushRunning = true + go l.flush(memoryFlush) + } + return false + } + for _, p := range points { for name, value := range p.Fields() { k := seriesFieldKey(string(p.Key()), name) @@ -277,6 +296,8 @@ func (l *Log) addToCache(points []tsdb.Point, fields map[string]*tsdb.Measuremen } l.seriesToCreateCache = append(l.seriesToCreateCache, series...) l.lastWriteTime = time.Now() + + return true } // readAndFlushWAL is called on open and will read the segment files in, flushing whenever @@ -294,7 +315,7 @@ func (l *Log) readAndFlushWAL() error { return err } - if l.memorySize > l.MemorySizeThreshold { + if l.memorySize > l.MaxMemorySizeThreshold { if err := l.flush(memoryFlush); err != nil { return err } @@ -354,19 +375,19 @@ func (l *Log) readFileToCache(fileName string) error { if err != nil { return err } - l.addToCache(points, nil, nil) + l.addToCache(points, nil, nil, false) case fieldsEntry: fields := make(map[string]*tsdb.MeasurementFields) if err := json.Unmarshal(data, &fields); err != nil { return err } - l.addToCache(nil, fields, nil) + l.addToCache(nil, fields, nil, false) case seriesEntry: series := make([]*tsdb.SeriesCreate, 0) if err := json.Unmarshal(data, &series); err != nil { return err } - l.addToCache(nil, nil, series) + l.addToCache(nil, nil, series, false) } } } @@ -460,20 +481,24 @@ func (l *Log) close() error { // flush writes all wal data in memory to the index func (l *Log) flush(flush flushType) error { - // only flush if there isn't one already running - l.writeLock.Lock() - if l.flushRunning { - l.writeLock.Unlock() + // only flush if there isn't one already running. Memory flushes are only triggered + // by writes, which will mark the flush as running, so we can ignore it. + l.cacheLock.Lock() + if l.flushRunning && flush != memoryFlush { + l.cacheLock.Unlock() return nil } - // only hold the lock while we rotate the segment file + // mark the flush as running and ensure that it gets marked as not running when we return l.flushRunning = true defer func() { - l.writeLock.Lock() + l.cacheLock.Lock() l.flushRunning = false - l.writeLock.Unlock() + l.cacheLock.Unlock() }() + + // only hold the lock while we rotate the segment file + l.writeLock.Lock() lastFileID := l.currentSegmentID if err := l.newSegmentFile(); err != nil { // there's no recovering from this, fail hard @@ -482,9 +507,6 @@ func (l *Log) flush(flush flushType) error { l.writeLock.Unlock() // copy the cache items to new maps so we can empty them out - l.cacheLock.Lock() - - // move over the flush cache and make a copy to write l.flushCache = l.cache l.cache = make(map[string]Values) l.cacheDirtySort = make(map[string]bool) @@ -561,6 +583,7 @@ func (l *Log) flush(flush flushType) error { // triggerAutoFlush will flush and compact any partitions that have hit the thresholds for compaction func (l *Log) triggerAutoFlush() { + // if f := l.shouldFlush(); f != noFlush { if err := l.flush(f); err != nil { l.logger.Printf("error flushing wal: %s\n", err) @@ -615,17 +638,18 @@ func (l *Log) newSegmentFile() error { return nil } -// shouldFlush +// shouldFlush will return the flushType specifying whether we should flush. memoryFlush +// is never returned from this function since those can only be triggered by writes func (l *Log) shouldFlush() flushType { l.cacheLock.RLock() defer l.cacheLock.RUnlock() - if len(l.cache) == 0 { + if l.flushRunning { return noFlush } - if l.memorySize > l.MemorySizeThreshold { - return memoryFlush + if len(l.cache) == 0 { + return noFlush } if time.Since(l.lastWriteTime) > l.FlushColdInterval { From c5f6c57d7fc39c11b3cf033cf220c153458114a4 Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Thu, 17 Sep 2015 18:25:55 -0400 Subject: [PATCH 077/139] Update engine to put index at the end of data files --- tsdb/engine/pd1/pd1.go | 76 ++++++++++++++++++++---------------------- 1 file changed, 37 insertions(+), 39 deletions(-) diff --git a/tsdb/engine/pd1/pd1.go b/tsdb/engine/pd1/pd1.go index 02708d7b45f..b16561776b6 100644 --- a/tsdb/engine/pd1/pd1.go +++ b/tsdb/engine/pd1/pd1.go @@ -58,6 +58,8 @@ const ( // MAP_POPULATE is for the mmap syscall. For some reason this isn't defined in golang's syscall MAP_POPULATE = 0x8000 + + magicNumber uint32 = 0x16D116D1 ) // Ensure Engine implements the interface. @@ -308,6 +310,8 @@ func (e *Engine) WriteAndCompact(pointsByKey map[string]Values, measurementField return nil } +// rewriteFile will read in the old data file, if provided and merge the values +// in the passed map into a new data file func (e *Engine) rewriteFile(oldDF *dataFile, valuesByID map[uint64]Values) error { // we need the values in sorted order so that we can merge them into the // new file as we read the old file @@ -351,31 +355,15 @@ func (e *Engine) rewriteFile(oldDF *dataFile, valuesByID map[uint64]Values) erro return err } - // write the header of the file and keep track of the current file position - currentPosition := uint32(4) - // series count - if _, err := f.Write(u32tob(uint32(len(ids)))); err != nil { + // write the magic number + if _, err := f.Write(u32tob(magicNumber)); err != nil { f.Close() return err } - // empty min time and max time - currentPosition += 16 - if _, err := f.Write([]byte{0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00}); err != nil { - f.Close() - return nil - } - - // write the series ids and empty starting positions - for _, id := range ids { - if _, err := f.Write(append(u64tob(id), []byte{0x00, 0x00, 0x00, 0x00}...)); err != nil { - f.Close() - return err - } - currentPosition += 12 - } // now combine the old file data with the new values, keeping track of // their positions + currentPosition := uint32(4) newPositions := make([]uint32, len(ids)) buf := make([]byte, DefaultMaxPointsPerBlock*20) for i, id := range ids { @@ -487,25 +475,31 @@ func (e *Engine) rewriteFile(oldDF *dataFile, valuesByID map[uint64]Values) erro } } - // write out the times and positions - if _, err := f.Seek(4, 0); err != nil { - f.Close() - return err + // write the file index, starting with the series ids and their positions + for i, id := range ids { + if _, err := f.Write(u64tob(id)); err != nil { + f.Close() + return err + } + if _, err := f.Write(u32tob(newPositions[i])); err != nil { + f.Close() + return err + } } + + // write the min time, max time if _, err := f.Write(append(u64tob(uint64(minTime)), u64tob(uint64(maxTime))...)); err != nil { f.Close() return err } - for _, pos := range newPositions { - if _, err := f.Seek(8, 1); err != nil { - f.Close() - return err - } - if _, err := f.Write(u32tob(pos)); err != nil { - return err - } + // series count + if _, err := f.Write(u32tob(uint32(len(ids)))); err != nil { + f.Close() + return err } + + // sync it and see4k back to the beginning to hand off to the mmap if err := f.Sync(); err != nil { return err } @@ -888,23 +882,24 @@ func (d *dataFile) close() error { } func (d *dataFile) MinTime() int64 { - return int64(btou64(d.mmap[4:12])) + return int64(btou64(d.mmap[d.size-20 : d.size-12])) } func (d *dataFile) MaxTime() int64 { - return int64(btou64(d.mmap[12:20])) + return int64(btou64(d.mmap[d.size-12 : d.size-4])) } func (d *dataFile) SeriesCount() uint32 { - return btou32(d.mmap[:4]) + return btou32(d.mmap[d.size-4:]) } func (d *dataFile) IDToPosition() map[uint64]uint32 { count := int(d.SeriesCount()) m := make(map[uint64]uint32) + indexStart := d.size - uint32(count*12+20) for i := 0; i < count; i++ { - offset := 20 + (i * 12) + offset := indexStart + uint32(i*12) id := btou64(d.mmap[offset : offset+8]) pos := btou32(d.mmap[offset+8 : offset+12]) m[id] = pos @@ -917,15 +912,17 @@ func (d *dataFile) IDToPosition() map[uint64]uint32 { // first block for the given ID. If zero is returned the ID doesn't // have any data in this file. func (d *dataFile) StartingPositionForID(id uint64) uint32 { + seriesCount := d.SeriesCount() + indexStart := d.size - uint32(seriesCount*12+20) - min := 0 - max := int(seriesCount) + min := uint32(0) + max := uint32(seriesCount) for min < max { mid := (max-min)/2 + min - offset := mid*seriesHeaderSize + fileHeaderSize + offset := mid*seriesHeaderSize + indexStart checkID := btou64(d.mmap[offset : offset+8]) if checkID == id { @@ -1066,6 +1063,7 @@ func (c *cursor) Next() (key, value []byte) { startingPos := f.StartingPositionForID(c.id) if startingPos == 0 { + c.filesPos++ continue } c.f = f @@ -1119,7 +1117,7 @@ func btou32(b []byte) uint32 { func hashSeriesField(key string) uint64 { h := fnv.New64a() h.Write([]byte(key)) - return h.Sum64() + return h.Sum64() % 100 } // seriesFieldKey combine a series key and field name for a unique string to be hashed to a numeric ID From 7c8ab4f1d8fd54ded4c3d0c89da2039dcbb1e1b5 Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Fri, 18 Sep 2015 11:08:03 -0400 Subject: [PATCH 078/139] Add test for close and restart of engine and fix errors. --- tsdb/engine/pd1/pd1.go | 14 ++++++++++++-- tsdb/engine/pd1/pd1_test.go | 19 +++++++++++++++---- 2 files changed, 27 insertions(+), 6 deletions(-) diff --git a/tsdb/engine/pd1/pd1.go b/tsdb/engine/pd1/pd1.go index b16561776b6..df930a6124f 100644 --- a/tsdb/engine/pd1/pd1.go +++ b/tsdb/engine/pd1/pd1.go @@ -70,6 +70,10 @@ type Engine struct { mu sync.Mutex path string + // deletesPending mark how many old data files are waiting to be deleted. This will + // keep a close from returning until all deletes finish + deletesPending sync.WaitGroup + // HashSeriesField is a function that takes a series key and a field name // and returns a hash identifier. It's not guaranteed to be unique. HashSeriesField func(key string) uint64 @@ -157,9 +161,13 @@ func (e *Engine) Close() error { e.queryLock.Lock() defer e.queryLock.Unlock() + e.deletesPending.Wait() + for _, df := range e.files { _ = df.Close() } + e.files = nil + e.currentFileID = 0 return nil } @@ -530,11 +538,13 @@ func (e *Engine) rewriteFile(oldDF *dataFile, valuesByID map[uint64]Values) erro // remove the old data file. no need to block returning the write, // but we need to let any running queries finish before deleting it if oldDF != nil { - go func(df *dataFile) { + e.deletesPending.Add(1) + go func() { if err := oldDF.Delete(); err != nil { // TODO: log this error } - }(oldDF) + e.deletesPending.Done() + }() } return nil diff --git a/tsdb/engine/pd1/pd1_test.go b/tsdb/engine/pd1/pd1_test.go index 79817eb531d..0ed7c368c5d 100644 --- a/tsdb/engine/pd1/pd1_test.go +++ b/tsdb/engine/pd1/pd1_test.go @@ -16,7 +16,7 @@ import ( func TestEngine_WriteAndReadFloats(t *testing.T) { e := OpenDefaultEngine() - defer e.Close() + defer e.Cleanup() e.Shard = newFieldCodecMock(map[string]influxql.DataType{"value": influxql.Float}) @@ -47,6 +47,7 @@ func TestEngine_WriteAndReadFloats(t *testing.T) { } k, v = c.Next() if k != nil { + fmt.Println(btou64(k), btof64(v)) t.Fatal("expected nil") } @@ -106,6 +107,16 @@ func TestEngine_WriteAndReadFloats(t *testing.T) { if 1.1 != btof64(v) { t.Fatal("p1 data not equal") } + + if err := e.Close(); err != nil { + t.Fatalf("error closing: %s", err.Error()) + } + + if err := e.Open(); err != nil { + t.Fatalf("error opening: %s", err.Error()) + } + + verify(false) } func TestEngine_WriteIndexWithCollision(t *testing.T) { @@ -115,7 +126,7 @@ func TestEngine_WriteIndexBenchmarkNames(t *testing.T) { t.Skip("whatevs") e := OpenDefaultEngine() - defer e.Close() + defer e.Cleanup() var points []tsdb.Point for i := 0; i < 100000; i++ { @@ -168,8 +179,8 @@ func OpenEngine(opt tsdb.EngineOptions) *Engine { // OpenDefaultEngine returns an open Engine with default options. func OpenDefaultEngine() *Engine { return OpenEngine(tsdb.NewEngineOptions()) } -// Close closes the engine and removes all data. -func (e *Engine) Close() error { +// Cleanup closes the engine and removes all data. +func (e *Engine) Cleanup() error { e.Engine.Close() os.RemoveAll(e.Path()) return nil From 365a631b53c507adbd648e90bcb9de1467b3cf86 Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Fri, 18 Sep 2015 15:18:05 -0400 Subject: [PATCH 079/139] Update wal to only open new segment file on flush if its not an idle flush --- tsdb/engine/pd1/wal.go | 17 ++++++++++++++--- 1 file changed, 14 insertions(+), 3 deletions(-) diff --git a/tsdb/engine/pd1/wal.go b/tsdb/engine/pd1/wal.go index e3157a94b03..610fe74d31c 100644 --- a/tsdb/engine/pd1/wal.go +++ b/tsdb/engine/pd1/wal.go @@ -500,9 +500,20 @@ func (l *Log) flush(flush flushType) error { // only hold the lock while we rotate the segment file l.writeLock.Lock() lastFileID := l.currentSegmentID - if err := l.newSegmentFile(); err != nil { - // there's no recovering from this, fail hard - panic(fmt.Sprintf("error creating new wal file: %s", err.Error())) + // if it's an idle flush, don't open a new segment file + if flush == idleFlush { + if l.currentSegmentFile != nil { + if err := l.currentSegmentFile.Close(); err != nil { + return err + } + l.currentSegmentFile = nil + l.currentSegmentSize = 0 + } + } else { + if err := l.newSegmentFile(); err != nil { + // there's no recovering from this, fail hard + panic(fmt.Sprintf("error creating new wal file: %s", err.Error())) + } } l.writeLock.Unlock() From 982c28b94733bd378fed01cf8aebfbc850d88645 Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Fri, 25 Sep 2015 10:49:26 -0400 Subject: [PATCH 080/139] Update to work with new cursor definitiono and Point in models --- tsdb/config.go | 2 +- tsdb/engine.go | 3 -- tsdb/engine/pd1/encoding.go | 3 +- tsdb/engine/pd1/pd1.go | 49 +++++++++++------------ tsdb/engine/pd1/pd1_test.go | 79 +++++++++++++++++++------------------ tsdb/engine/pd1/wal.go | 45 ++++++++++----------- tsdb/engine/pd1/wal_test.go | 64 ++++++++++++++++-------------- 7 files changed, 125 insertions(+), 120 deletions(-) diff --git a/tsdb/config.go b/tsdb/config.go index b7bc409b4a9..4aec4a14e52 100644 --- a/tsdb/config.go +++ b/tsdb/config.go @@ -8,7 +8,7 @@ import ( const ( // DefaultEngine is the default engine for new shards - DefaultEngine = "bz1" + DefaultEngine = "pd1" // DefaultMaxWALSize is the default size of the WAL before it is flushed. DefaultMaxWALSize = 100 * 1024 * 1024 // 100MB diff --git a/tsdb/engine.go b/tsdb/engine.go index b0e9254d95a..d2862b54863 100644 --- a/tsdb/engine.go +++ b/tsdb/engine.go @@ -18,9 +18,6 @@ var ( ErrFormatNotFound = errors.New("format not found") ) -// DefaultEngine is the default engine used by the shard when initializing. -const DefaultEngine = "pd1" - // Engine represents a swappable storage engine for the shard. type Engine interface { Open() error diff --git a/tsdb/engine/pd1/encoding.go b/tsdb/engine/pd1/encoding.go index 7262e8a6c36..cad991aa7a8 100644 --- a/tsdb/engine/pd1/encoding.go +++ b/tsdb/engine/pd1/encoding.go @@ -6,6 +6,7 @@ import ( "time" "github.com/dgryski/go-tsz" + "github.com/influxdb/influxdb/tsdb" ) type Value interface { @@ -35,7 +36,7 @@ type EmptyValue struct { func (e *EmptyValue) TimeBytes() []byte { return nil } func (e *EmptyValue) ValueBytes() []byte { return nil } -func (e *EmptyValue) Time() time.Time { return time.Unix(0, 0) } +func (e *EmptyValue) Time() time.Time { return time.Unix(0, tsdb.EOF) } func (e *EmptyValue) Value() interface{} { return nil } func (e *EmptyValue) Size() int { return 0 } diff --git a/tsdb/engine/pd1/pd1.go b/tsdb/engine/pd1/pd1.go index df930a6124f..1428f04b45d 100644 --- a/tsdb/engine/pd1/pd1.go +++ b/tsdb/engine/pd1/pd1.go @@ -17,6 +17,7 @@ import ( "github.com/golang/snappy" "github.com/influxdb/influxdb/influxql" + "github.com/influxdb/influxdb/models" "github.com/influxdb/influxdb/tsdb" ) @@ -217,7 +218,7 @@ func (e *Engine) LoadMetadataIndex(shard *tsdb.Shard, index *tsdb.DatabaseIndex, // WritePoints writes metadata and point data into the engine. // Returns an error if new points are added to an existing key. -func (e *Engine) WritePoints(points []tsdb.Point, measurementFieldsToSave map[string]*tsdb.MeasurementFields, seriesToCreate []*tsdb.SeriesCreate) error { +func (e *Engine) WritePoints(points []models.Point, measurementFieldsToSave map[string]*tsdb.MeasurementFields, seriesToCreate []*tsdb.SeriesCreate) error { return e.WAL.WritePoints(points, measurementFieldsToSave, seriesToCreate) } @@ -614,7 +615,7 @@ func (e *Engine) Begin(writable bool) (tsdb.Tx, error) { } // TODO: make the cursor take a field name -func (e *Engine) Cursor(series string, direction tsdb.Direction) tsdb.Cursor { +func (e *Engine) Cursor(series string, fields []string, dec *tsdb.FieldCodec, ascending bool) tsdb.Cursor { measurementName := tsdb.MeasurementFromSeriesKey(series) codec := e.Shard.FieldCodec(measurementName) if codec == nil { @@ -627,7 +628,7 @@ func (e *Engine) Cursor(series string, direction tsdb.Direction) tsdb.Cursor { // TODO: ensure we map the collisions id := hashSeriesField(seriesFieldKey(series, field.Name)) - return newCursor(id, field.Type, e.copyFilesCollection(), direction) + return newCursor(id, field.Type, e.copyFilesCollection(), ascending) } func (e *Engine) copyFilesCollection() []*dataFile { @@ -961,30 +962,28 @@ type cursor struct { pos uint32 vals Values - direction tsdb.Direction + ascending bool // time acending list of data files files []*dataFile } -func newCursor(id uint64, dataType influxql.DataType, files []*dataFile, direction tsdb.Direction) *cursor { +func newCursor(id uint64, dataType influxql.DataType, files []*dataFile, ascending bool) *cursor { return &cursor{ id: id, dataType: dataType, - direction: direction, + ascending: ascending, files: files, } } -func (c *cursor) Seek(seek []byte) (key, value []byte) { - t := int64(btou64(seek)) - - if t < c.files[0].MinTime() { +func (c *cursor) SeekTo(seek int64) (int64, interface{}) { + if seek < c.files[0].MinTime() { c.filesPos = 0 c.f = c.files[0] } else { for i, f := range c.files { - if t >= f.MinTime() && t <= f.MaxTime() { + if seek >= f.MinTime() && seek <= f.MaxTime() { c.filesPos = i c.f = f break @@ -993,7 +992,7 @@ func (c *cursor) Seek(seek []byte) (key, value []byte) { } if c.f == nil { - return nil, nil + return tsdb.EOF, nil } // TODO: make this for the reverse direction cursor @@ -1006,7 +1005,7 @@ func (c *cursor) Seek(seek []byte) (key, value []byte) { if pos == 0 { c.filesPos++ if c.filesPos >= len(c.files) { - return nil, nil + return tsdb.EOF, nil } c.f = c.files[c.filesPos] continue @@ -1025,7 +1024,7 @@ func (c *cursor) Seek(seek []byte) (key, value []byte) { nextBlockID := btou64(c.f.mmap[nextBlockPos : nextBlockPos+8]) if nextBlockID == c.id { nextBlockTime := int64(btou64(c.f.mmap[nextBlockPos+12 : nextBlockPos+20])) - if nextBlockTime <= t { + if nextBlockTime <= seek { pos = nextBlockPos continue } @@ -1033,16 +1032,16 @@ func (c *cursor) Seek(seek []byte) (key, value []byte) { } // it must be in this block or not at all - tb, vb := c.decodeBlockAndGetValues(pos) - if int64(btou64(tb)) >= t { - return tb, vb + t, v := c.decodeBlockAndGetValues(pos) + if t >= seek { + return t, v } // wasn't in the first value popped out of the block, check the rest for i, v := range c.vals { - if v.Time().UnixNano() >= t { + if v.Time().UnixNano() >= seek { c.vals = c.vals[i+1:] - return v.TimeBytes(), v.ValueBytes() + return v.Time().UnixNano(), v.Value() } } @@ -1052,7 +1051,7 @@ func (c *cursor) Seek(seek []byte) (key, value []byte) { } } -func (c *cursor) Next() (key, value []byte) { +func (c *cursor) Next() (int64, interface{}) { if len(c.vals) == 0 { // if we have a file set, see if the next block is for this ID if c.f != nil && c.pos < c.f.size { @@ -1081,16 +1080,16 @@ func (c *cursor) Next() (key, value []byte) { } // we didn't get to a file that had a next value - return nil, nil + return tsdb.EOF, nil } v := c.vals[0] c.vals = c.vals[1:] - return v.TimeBytes(), v.ValueBytes() + return v.Time().UnixNano(), v.Value() } -func (c *cursor) decodeBlockAndGetValues(position uint32) ([]byte, []byte) { +func (c *cursor) decodeBlockAndGetValues(position uint32) (int64, interface{}) { length := btou32(c.f.mmap[position+8 : position+12]) block := c.f.mmap[position+12 : position+12+length] c.vals, _ = DecodeFloatBlock(block) @@ -1098,10 +1097,10 @@ func (c *cursor) decodeBlockAndGetValues(position uint32) ([]byte, []byte) { v := c.vals[0] c.vals = c.vals[1:] - return v.TimeBytes(), v.ValueBytes() + return v.Time().UnixNano(), v.Value() } -func (c *cursor) Direction() tsdb.Direction { return c.direction } +func (c *cursor) Ascending() bool { return c.ascending } // u64tob converts a uint64 into an 8-byte slice. func u64tob(v uint64) []byte { diff --git a/tsdb/engine/pd1/pd1_test.go b/tsdb/engine/pd1/pd1_test.go index 0ed7c368c5d..1a8f8613fc3 100644 --- a/tsdb/engine/pd1/pd1_test.go +++ b/tsdb/engine/pd1/pd1_test.go @@ -10,6 +10,7 @@ import ( "time" "github.com/influxdb/influxdb/influxql" + "github.com/influxdb/influxdb/models" "github.com/influxdb/influxdb/tsdb" "github.com/influxdb/influxdb/tsdb/engine/pd1" ) @@ -25,86 +26,88 @@ func TestEngine_WriteAndReadFloats(t *testing.T) { p3 := parsePoint("cpu,host=A value=2.1 2000000000") p4 := parsePoint("cpu,host=B value=2.2 2000000000") - if err := e.WritePoints([]tsdb.Point{p1, p2, p3}, nil, nil); err != nil { + if err := e.WritePoints([]models.Point{p1, p2, p3}, nil, nil); err != nil { t.Fatalf("failed to write points: %s", err.Error()) } + fields := []string{"value"} + var codec *tsdb.FieldCodec + verify := func(checkSingleBVal bool) { - c := e.Cursor("cpu,host=A", tsdb.Forward) + c := e.Cursor("cpu,host=A", fields, codec, true) k, v := c.Next() - if btou64(k) != uint64(p1.UnixNano()) { - t.Fatalf("p1 time wrong:\n\texp:%d\n\tgot:%d\n", p1.UnixNano(), btou64(k)) + if k != p1.UnixNano() { + t.Fatalf("p1 time wrong:\n\texp:%d\n\tgot:%d\n", p1.UnixNano(), k) } - if 1.1 != btof64(v) { + if 1.1 != v { t.Fatal("p1 data not equal") } k, v = c.Next() - if btou64(k) != uint64(p3.UnixNano()) { - t.Fatalf("p3 time wrong:\n\texp:%d\n\tgot:%d\n", p3.UnixNano(), btou64(k)) + if k != p3.UnixNano() { + t.Fatalf("p3 time wrong:\n\texp:%d\n\tgot:%d\n", p3.UnixNano(), k) } - if 2.1 != btof64(v) { + if 2.1 != v { t.Fatal("p3 data not equal") } k, v = c.Next() - if k != nil { - fmt.Println(btou64(k), btof64(v)) - t.Fatal("expected nil") + if k != tsdb.EOF { + t.Fatal("expected EOF") } - c = e.Cursor("cpu,host=B", tsdb.Forward) + c = e.Cursor("cpu,host=B", fields, codec, true) k, v = c.Next() - if btou64(k) != uint64(p2.UnixNano()) { - t.Fatalf("p2 time wrong:\n\texp:%d\n\tgot:%d\n", p2.UnixNano(), btou64(k)) + if k != p2.UnixNano() { + t.Fatalf("p2 time wrong:\n\texp:%d\n\tgot:%d\n", p2.UnixNano(), k) } - if 1.2 != btof64(v) { + if 1.2 != v { t.Fatal("p2 data not equal") } if checkSingleBVal { k, v = c.Next() - if k != nil { - t.Fatal("expected nil") + if k != tsdb.EOF { + t.Fatal("expected EOF") } } } verify(true) - if err := e.WritePoints([]tsdb.Point{p4}, nil, nil); err != nil { + if err := e.WritePoints([]models.Point{p4}, nil, nil); err != nil { t.Fatalf("failed to write points: %s", err.Error()) } verify(false) - c := e.Cursor("cpu,host=B", tsdb.Forward) + c := e.Cursor("cpu,host=B", fields, codec, true) k, v := c.Next() - if btou64(k) != uint64(p2.UnixNano()) { - t.Fatalf("p2 time wrong:\n\texp:%d\n\tgot:%d\n", p2.UnixNano(), btou64(k)) + if k != p2.UnixNano() { + t.Fatalf("p2 time wrong:\n\texp:%d\n\tgot:%d\n", p2.UnixNano(), k) } - if 1.2 != btof64(v) { + if 1.2 != v { t.Fatal("p2 data not equal") } k, v = c.Next() - if btou64(k) != uint64(p4.UnixNano()) { - t.Fatalf("p2 time wrong:\n\texp:%d\n\tgot:%d\n", p2.UnixNano(), btou64(k)) + if k != p4.UnixNano() { + t.Fatalf("p2 time wrong:\n\texp:%d\n\tgot:%d\n", p2.UnixNano(), k) } - if 2.2 != btof64(v) { + if 2.2 != v { t.Fatal("p2 data not equal") } // verify we can seek - k, v = c.Seek(u64tob(2000000000)) - if btou64(k) != uint64(p4.UnixNano()) { - t.Fatalf("p2 time wrong:\n\texp:%d\n\tgot:%d\n", p2.UnixNano(), btou64(k)) + k, v = c.SeekTo(2000000000) + if k != p4.UnixNano() { + t.Fatalf("p2 time wrong:\n\texp:%d\n\tgot:%d\n", p2.UnixNano(), k) } - if 2.2 != btof64(v) { + if 2.2 != v { t.Fatal("p2 data not equal") } - c = e.Cursor("cpu,host=A", tsdb.Forward) - k, v = c.Seek(u64tob(0)) - if btou64(k) != uint64(p1.UnixNano()) { - t.Fatalf("p1 time wrong:\n\texp:%d\n\tgot:%d\n", p1.UnixNano(), btou64(k)) + c = e.Cursor("cpu,host=A", fields, codec, true) + k, v = c.SeekTo(0) + if k != p1.UnixNano() { + t.Fatalf("p1 time wrong:\n\texp:%d\n\tgot:%d\n", p1.UnixNano(), k) } - if 1.1 != btof64(v) { + if 1.1 != v { t.Fatal("p1 data not equal") } @@ -128,7 +131,7 @@ func TestEngine_WriteIndexBenchmarkNames(t *testing.T) { e := OpenDefaultEngine() defer e.Cleanup() - var points []tsdb.Point + var points []models.Point for i := 0; i < 100000; i++ { points = append(points, parsePoint(fmt.Sprintf("cpu%d value=22.1", i))) } @@ -205,15 +208,15 @@ func (f *FieldCodeMock) FieldCodec(m string) *tsdb.FieldCodec { return f.codec } -func parsePoints(buf string) []tsdb.Point { - points, err := tsdb.ParsePointsString(buf) +func parsePoints(buf string) []models.Point { + points, err := models.ParsePointsString(buf) if err != nil { panic(fmt.Sprintf("couldn't parse points: %s", err.Error())) } return points } -func parsePoint(buf string) tsdb.Point { +func parsePoint(buf string) models.Point { return parsePoints(buf)[0] } diff --git a/tsdb/engine/pd1/wal.go b/tsdb/engine/pd1/wal.go index 610fe74d31c..7b538fed3a0 100644 --- a/tsdb/engine/pd1/wal.go +++ b/tsdb/engine/pd1/wal.go @@ -1,7 +1,6 @@ package pd1 import ( - "bytes" "encoding/json" "fmt" "io" @@ -14,8 +13,10 @@ import ( "sync" "time" - "github.com/golang/snappy" + "github.com/influxdb/influxdb/models" "github.com/influxdb/influxdb/tsdb" + + "github.com/golang/snappy" ) const ( @@ -171,12 +172,12 @@ func (l *Log) Open() error { } // Cursor will return a cursor object to Seek and iterate with Next for the WAL cache for the given -func (l *Log) Cursor(key string, direction tsdb.Direction) tsdb.Cursor { +func (l *Log) Cursor(series string, fields []string, dec *tsdb.FieldCodec, ascending bool) tsdb.Cursor { l.cacheLock.RLock() defer l.cacheLock.RUnlock() // TODO: make this work for other fields - ck := seriesFieldKey(key, "value") + ck := seriesFieldKey(series, "value") values := l.cache[ck] // if we're in the middle of a flush, combine the previous cache @@ -187,7 +188,7 @@ func (l *Log) Cursor(key string, direction tsdb.Direction) tsdb.Cursor { copy(c, fc) c = append(c, values...) - return newWALCursor(c, direction) + return newWALCursor(c, ascending) } } @@ -199,10 +200,10 @@ func (l *Log) Cursor(key string, direction tsdb.Direction) tsdb.Cursor { // build a copy so writes afterwards don't change the result set a := make([]Value, len(values)) copy(a, values) - return newWALCursor(a, direction) + return newWALCursor(a, ascending) } -func (l *Log) WritePoints(points []tsdb.Point, fields map[string]*tsdb.MeasurementFields, series []*tsdb.SeriesCreate) error { +func (l *Log) WritePoints(points []models.Point, fields map[string]*tsdb.MeasurementFields, series []*tsdb.SeriesCreate) error { // add everything to the cache, or return an error if we've hit our max memory if addedToCache := l.addToCache(points, fields, series, true); !addedToCache { return fmt.Errorf("WAL backed up flushing to index, hit max memory") @@ -260,7 +261,7 @@ func (l *Log) WritePoints(points []tsdb.Point, fields map[string]*tsdb.Measureme // immediately after return and will be flushed at the next flush cycle. Before adding to the cache we check if we're over the // max memory threshold. If we are we request a flush in a new goroutine and return false, indicating we didn't add the values // to the cache and that writes should return a failure. -func (l *Log) addToCache(points []tsdb.Point, fields map[string]*tsdb.MeasurementFields, series []*tsdb.SeriesCreate, checkMemory bool) bool { +func (l *Log) addToCache(points []models.Point, fields map[string]*tsdb.MeasurementFields, series []*tsdb.SeriesCreate, checkMemory bool) bool { l.cacheLock.Lock() defer l.cacheLock.Unlock() @@ -371,7 +372,7 @@ func (l *Log) readFileToCache(fileName string) error { // and marshal it and send it to the cache switch walEntryType(entryType) { case pointsEntry: - points, err := tsdb.ParsePoints(data) + points, err := models.ParsePoints(data) if err != nil { return err } @@ -674,53 +675,53 @@ func (l *Log) shouldFlush() flushType { type walCursor struct { cache Values position int - direction tsdb.Direction + ascending bool } -func newWALCursor(cache Values, direction tsdb.Direction) *walCursor { +func newWALCursor(cache Values, ascending bool) *walCursor { // position is set such that a call to Next will successfully advance // to the next postion and return the value. - c := &walCursor{cache: cache, direction: direction, position: -1} - if direction.Reverse() { + c := &walCursor{cache: cache, ascending: ascending, position: -1} + if !ascending { c.position = len(c.cache) } return c } -func (c *walCursor) Direction() tsdb.Direction { return c.direction } +func (c *walCursor) Ascending() bool { return c.ascending } // Seek will point the cursor to the given time (or key) -func (c *walCursor) Seek(seek []byte) (key, value []byte) { +func (c *walCursor) SeekTo(seek int64) (int64, interface{}) { // Seek cache index c.position = sort.Search(len(c.cache), func(i int) bool { - return bytes.Compare(c.cache[i].TimeBytes(), seek) != -1 + return c.cache[i].Time().UnixNano() >= seek }) // If seek is not in the cache, return the last value in the cache - if c.direction.Reverse() && c.position >= len(c.cache) { + if !c.ascending && c.position >= len(c.cache) { c.position = len(c.cache) } // Make sure our position points to something in the cache if c.position < 0 || c.position >= len(c.cache) { - return nil, nil + return tsdb.EOF, nil } v := c.cache[c.position] - return v.TimeBytes(), v.ValueBytes() + return v.Time().UnixNano(), v.Value() } // Next moves the cursor to the next key/value. will return nil if at the end -func (c *walCursor) Next() (key, value []byte) { +func (c *walCursor) Next() (int64, interface{}) { var v Value - if c.direction.Forward() { + if c.ascending { v = c.nextForward() } else { v = c.nextReverse() } - return v.TimeBytes(), v.ValueBytes() + return v.Time().UnixNano(), v.Value() } // nextForward advances the cursor forward returning the next value diff --git a/tsdb/engine/pd1/wal_test.go b/tsdb/engine/pd1/wal_test.go index c1ef614650c..de03dfbd601 100644 --- a/tsdb/engine/pd1/wal_test.go +++ b/tsdb/engine/pd1/wal_test.go @@ -6,6 +6,7 @@ import ( "reflect" "testing" + "github.com/influxdb/influxdb/models" "github.com/influxdb/influxdb/tsdb" "github.com/influxdb/influxdb/tsdb/engine/pd1" ) @@ -38,64 +39,67 @@ func TestWAL_TestWriteQueryOpen(t *testing.T) { fieldsToWrite := map[string]*tsdb.MeasurementFields{"foo": {Fields: map[string]*tsdb.Field{"bar": {Name: "value"}}}} seriesToWrite := []*tsdb.SeriesCreate{{Measurement: "asdf"}} - if err := w.WritePoints([]tsdb.Point{p1, p2}, fieldsToWrite, seriesToWrite); err != nil { + if err := w.WritePoints([]models.Point{p1, p2}, fieldsToWrite, seriesToWrite); err != nil { t.Fatalf("failed to write points: %s", err.Error()) } - c := w.Cursor("cpu,host=A", tsdb.Forward) + fieldNames := []string{"value"} + var codec *tsdb.FieldCodec + + c := w.Cursor("cpu,host=A", fieldNames, codec, true) k, v := c.Next() - if btou64(k) != uint64(p1.UnixNano()) { - t.Fatalf("p1 time wrong:\n\texp:%d\n\tgot:%d\n", p1.UnixNano(), btou64(k)) + if k != p1.UnixNano() { + t.Fatalf("p1 time wrong:\n\texp:%d\n\tgot:%d\n", p1.UnixNano(), k) } - if 1.1 != btof64(v) { + if 1.1 != v { t.Fatal("p1 data not equal") } - c = w.Cursor("cpu,host=B", tsdb.Forward) + c = w.Cursor("cpu,host=B", fieldNames, codec, true) k, v = c.Next() - if btou64(k) != uint64(p2.UnixNano()) { - t.Fatalf("p2 time wrong:\n\texp:%d\n\tgot:%d\n", p2.UnixNano(), btou64(k)) + if k != p2.UnixNano() { + t.Fatalf("p2 time wrong:\n\texp:%d\n\tgot:%d\n", p2.UnixNano(), k) } - if 1.2 != btof64(v) { + if 1.2 != v { t.Fatal("p2 data not equal") } k, v = c.Next() - if k != nil { - t.Fatal("expected nil") + if k != tsdb.EOF { + t.Fatal("expected EOF", k, v) } // ensure we can do another write to the wal and get stuff - if err := w.WritePoints([]tsdb.Point{p3}, nil, nil); err != nil { + if err := w.WritePoints([]models.Point{p3}, nil, nil); err != nil { t.Fatalf("failed to write: %s", err.Error) } - c = w.Cursor("cpu,host=A", tsdb.Forward) + c = w.Cursor("cpu,host=A", fieldNames, codec, true) k, v = c.Next() - if btou64(k) != uint64(p1.UnixNano()) { - t.Fatalf("p1 time wrong:\n\texp:%d\n\tgot:%d\n", p1.UnixNano(), btou64(k)) + if k != p1.UnixNano() { + t.Fatalf("p1 time wrong:\n\texp:%d\n\tgot:%d\n", p1.UnixNano(), k) } - if 1.1 != btof64(v) { + if 1.1 != v { t.Fatal("p1 data not equal") } k, v = c.Next() - if btou64(k) != uint64(p3.UnixNano()) { - t.Fatalf("p3 time wrong:\n\texp:%d\n\tgot:%d\n", p3.UnixNano(), btou64(k)) + if k != p3.UnixNano() { + t.Fatalf("p3 time wrong:\n\texp:%d\n\tgot:%d\n", p3.UnixNano(), k) } - if 2.1 != btof64(v) { + if 2.1 != v { t.Fatal("p3 data not equal") } // ensure we can seek - k, v = c.Seek(u64tob(2000000000)) - if btou64(k) != uint64(p3.UnixNano()) { - t.Fatalf("p3 time wrong:\n\texp:%d\n\tgot:%d\n", p3.UnixNano(), btou64(k)) + k, v = c.SeekTo(2000000000) + if k != p3.UnixNano() { + t.Fatalf("p3 time wrong:\n\texp:%d\n\tgot:%d\n", p3.UnixNano(), k) } - if 2.1 != btof64(v) { + if 2.1 != v { t.Fatal("p3 data not equal") } k, v = c.Next() - if k != nil { - t.Fatal("expected nil") + if k != tsdb.EOF { + t.Fatal("expected EOF") } // ensure we close and after open it flushes to the index @@ -115,15 +119,15 @@ func TestWAL_TestWriteQueryOpen(t *testing.T) { t.Fatal("expected host B values to flush to index on open") } - if err := w.WritePoints([]tsdb.Point{p4}, nil, nil); err != nil { + if err := w.WritePoints([]models.Point{p4}, nil, nil); err != nil { t.Fatalf("failed to write: %s", err.Error) } - c = w.Cursor("cpu,host=B", tsdb.Forward) + c = w.Cursor("cpu,host=B", fieldNames, codec, true) k, v = c.Next() - if btou64(k) != uint64(p4.UnixNano()) { - t.Fatalf("p4 time wrong:\n\texp:%d\n\tgot:%d\n", p4.UnixNano(), btou64(k)) + if k != p4.UnixNano() { + t.Fatalf("p4 time wrong:\n\texp:%d\n\tgot:%d\n", p4.UnixNano(), k) } - if 2.2 != btof64(v) { + if 2.2 != v { t.Fatal("p4 data not equal") } From 0770ccc87da9780f05a84bb3e458a471a374cea2 Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Fri, 25 Sep 2015 17:11:27 -0400 Subject: [PATCH 081/139] Make writes to historical areas possible --- tsdb/engine/pd1/encoding.go | 22 +++------ tsdb/engine/pd1/pd1.go | 99 +++++++++++++++++++++++++++++++------ tsdb/engine/pd1/pd1_test.go | 59 ++++++++++++++++++++++ tsdb/engine/pd1/wal.go | 3 ++ 4 files changed, 151 insertions(+), 32 deletions(-) diff --git a/tsdb/engine/pd1/encoding.go b/tsdb/engine/pd1/encoding.go index cad991aa7a8..5d1bee9a8fe 100644 --- a/tsdb/engine/pd1/encoding.go +++ b/tsdb/engine/pd1/encoding.go @@ -1,8 +1,6 @@ package pd1 import ( - "encoding/binary" - "math" "time" "github.com/dgryski/go-tsz" @@ -10,9 +8,8 @@ import ( ) type Value interface { - TimeBytes() []byte - ValueBytes() []byte Time() time.Time + UnixNano() int64 Value() interface{} Size() int } @@ -34,8 +31,7 @@ func NewValue(t time.Time, value interface{}) Value { type EmptyValue struct { } -func (e *EmptyValue) TimeBytes() []byte { return nil } -func (e *EmptyValue) ValueBytes() []byte { return nil } +func (e *EmptyValue) UnixNano() int64 { return tsdb.EOF } func (e *EmptyValue) Time() time.Time { return time.Unix(0, tsdb.EOF) } func (e *EmptyValue) Value() interface{} { return nil } func (e *EmptyValue) Size() int { return 0 } @@ -93,18 +89,12 @@ func (f *FloatValue) Time() time.Time { return f.time } -func (f *FloatValue) Value() interface{} { - return f.value -} - -func (f *FloatValue) TimeBytes() []byte { - return u64tob(uint64(f.Time().UnixNano())) +func (f *FloatValue) UnixNano() int64 { + return f.time.UnixNano() } -func (f *FloatValue) ValueBytes() []byte { - buf := make([]byte, 8) - binary.BigEndian.PutUint64(buf, math.Float64bits(f.value)) - return buf +func (f *FloatValue) Value() interface{} { + return f.value } func (f *FloatValue) Size() int { diff --git a/tsdb/engine/pd1/pd1.go b/tsdb/engine/pd1/pd1.go index 1428f04b45d..40aa7f8050d 100644 --- a/tsdb/engine/pd1/pd1.go +++ b/tsdb/engine/pd1/pd1.go @@ -53,7 +53,7 @@ const ( // DefaultBlockSize is the default size of uncompressed points blocks. DefaultBlockSize = 512 * 1024 // 512KB - DefaultMaxFileSize = 10 * 1024 * 1024 // 10MB + DefaultRotateFileSize = 10 * 1024 * 1024 // 10MB DefaultMaxPointsPerBlock = 1000 @@ -86,6 +86,8 @@ type Engine struct { WAL *Log + RotateFileSize uint32 + filesLock sync.RWMutex files dataFiles currentFileID int @@ -106,6 +108,7 @@ func NewEngine(path string, walPath string, opt tsdb.EngineOptions) tsdb.Engine // TODO: this is the function where we can inject a check against the in memory collisions HashSeriesField: hashSeriesField, WAL: w, + RotateFileSize: DefaultRotateFileSize, } e.WAL.Index = e @@ -172,6 +175,13 @@ func (e *Engine) Close() error { return nil } +// DataFileCount returns the number of data files in the database +func (e *Engine) DataFileCount() int { + e.filesLock.RLock() + defer e.filesLock.RUnlock() + return len(e.files) +} + // SetLogOutput is a no-op. func (e *Engine) SetLogOutput(w io.Writer) {} @@ -296,32 +306,89 @@ func (e *Engine) WriteAndCompact(pointsByKey map[string]Values, measurementField } } - // TODO: handle values written in the past that force an old data file to get rewritten - - // we keep track of the newest data file and if it should be - // rewritten with new data. - var newestDataFile *dataFile - overwriteNewestFile := false - if len(e.files) > 0 { - newestDataFile = e.files[len(e.files)-1] - overwriteNewestFile = newestDataFile.size < DefaultMaxFileSize + if len(e.files) == 0 { + return e.rewriteFile(nil, valuesByID) } - // flush values by id to either a new file or rewrite the old one - if overwriteNewestFile { - if err := e.rewriteFile(newestDataFile, valuesByID); err != nil { + maxTime := int64(math.MaxInt64) + // reverse through the data files and write in the data + for i := len(e.files) - 1; i >= 0; i-- { + f := e.files[i] + // max times are exclusive, so add 1 to it + fileMax := f.MaxTime() + 1 + fileMin := f.MinTime() + // if the file is < rotate, write all data between fileMin and maxTime + if f.size < e.RotateFileSize { + if err := e.rewriteFile(f, e.filterDataBetweenTimes(valuesByID, fileMin, maxTime)); err != nil { + return err + } + continue + } + // if the file is > rotate: + // write all data between fileMax and maxTime into new file + // write all data between fileMin and fileMax into old file + if err := e.rewriteFile(nil, e.filterDataBetweenTimes(valuesByID, fileMax, maxTime)); err != nil { return err } - } else if err := e.rewriteFile(nil, valuesByID); err != nil { - return err + if err := e.rewriteFile(f, e.filterDataBetweenTimes(valuesByID, fileMin, fileMax)); err != nil { + return err + } + maxTime = fileMin } + // for any data leftover, write into a new file since it's all older + // than any file we currently have + return e.rewriteFile(nil, valuesByID) +} - return nil +// filterDataBetweenTimes will create a new map with data between +// the minTime (inclusive) and maxTime (exclusive) while removing that +// data from the passed in map. It is assume that the Values arrays +// are sorted in time ascending order +func (e *Engine) filterDataBetweenTimes(valuesByID map[uint64]Values, minTime, maxTime int64) map[uint64]Values { + filteredValues := make(map[uint64]Values) + for id, values := range valuesByID { + maxIndex := len(values) + minIndex := 0 + // find the index of the first value in the range + for i, v := range values { + t := v.UnixNano() + if t >= minTime && t < maxTime { + minIndex = i + break + } + } + // go backwards to find the index of the last value in the range + for i := len(values) - 1; i >= 0; i-- { + t := values[i].UnixNano() + if t < maxTime { + maxIndex = i + 1 + break + } + } + + // write into the result map and filter the passed in map + filteredValues[id] = values[minIndex:maxIndex] + + // if we grabbed all the values, remove them from the passed in map + if minIndex == len(values) || (minIndex == 0 && maxIndex == len(values)) { + delete(valuesByID, id) + continue + } + + valuesByID[id] = values[0:minIndex] + if maxIndex < len(values) { + valuesByID[id] = append(valuesByID[id], values[maxIndex:]...) + } + } + return filteredValues } // rewriteFile will read in the old data file, if provided and merge the values // in the passed map into a new data file func (e *Engine) rewriteFile(oldDF *dataFile, valuesByID map[uint64]Values) error { + if len(valuesByID) == 0 { + return nil + } // we need the values in sorted order so that we can merge them into the // new file as we read the old file ids := make([]uint64, 0, len(valuesByID)) diff --git a/tsdb/engine/pd1/pd1_test.go b/tsdb/engine/pd1/pd1_test.go index 1a8f8613fc3..a20cc665ced 100644 --- a/tsdb/engine/pd1/pd1_test.go +++ b/tsdb/engine/pd1/pd1_test.go @@ -125,6 +125,65 @@ func TestEngine_WriteAndReadFloats(t *testing.T) { func TestEngine_WriteIndexWithCollision(t *testing.T) { } +func TestEngine_WriteIndexQueryAcrossDataFiles(t *testing.T) { + e := OpenDefaultEngine() + defer e.Cleanup() + + e.Shard = newFieldCodecMock(map[string]influxql.DataType{"value": influxql.Float}) + e.RotateFileSize = 10 + + p1 := parsePoint("cpu,host=A value=1.1 1000000000") + p2 := parsePoint("cpu,host=B value=1.1 1000000000") + p3 := parsePoint("cpu,host=A value=2.4 4000000000") + p4 := parsePoint("cpu,host=B value=2.4 4000000000") + + if err := e.WritePoints([]models.Point{p1, p2, p3, p4}, nil, nil); err != nil { + t.Fatalf("failed to write points: %s", err.Error()) + } + + p5 := parsePoint("cpu,host=A value=1.5 5000000000") + p6 := parsePoint("cpu,host=B value=2.5 5000000000") + p7 := parsePoint("cpu,host=A value=1.3 3000000000") + p8 := parsePoint("cpu,host=B value=2.3 3000000000") + + if err := e.WritePoints([]models.Point{p5, p6, p7, p8}, nil, nil); err != nil { + t.Fatalf("failed to write points: %s", err.Error()) + } + + if count := e.DataFileCount(); count != 2 { + t.Fatalf("expected 2 data files to exist but got %d", count) + } + + fields := []string{"value"} + var codec *tsdb.FieldCodec + + verify := func(series string, points []models.Point, seek int64) { + c := e.Cursor(series, fields, codec, true) + + // we we want to seek, do it and verify the first point matches + if seek != 0 { + k, v := c.SeekTo(seek) + p := points[0] + val := p.Fields()["value"] + if p.UnixNano() != k || val != v { + t.Fatalf("expected to seek to first point\n\texp: %d %f\n\tgot: %d %f", p.UnixNano(), val, k, v) + } + points = points[1:] + } + + for _, p := range points { + k, v := c.Next() + val := p.Fields()["value"] + if p.UnixNano() != k || val != v { + t.Fatalf("expected to seek to first point\n\texp: %d %f\n\tgot: %d %f", p.UnixNano(), val, k, v.(float64)) + } + } + } + + verify("cpu,host=A", []models.Point{p1, p7, p3, p5}, 0) + verify("cpu,host=B", []models.Point{p2, p8, p4, p6}, 0) +} + func TestEngine_WriteIndexBenchmarkNames(t *testing.T) { t.Skip("whatevs") diff --git a/tsdb/engine/pd1/wal.go b/tsdb/engine/pd1/wal.go index 7b538fed3a0..e7fb7fb39a4 100644 --- a/tsdb/engine/pd1/wal.go +++ b/tsdb/engine/pd1/wal.go @@ -521,6 +521,9 @@ func (l *Log) flush(flush flushType) error { // copy the cache items to new maps so we can empty them out l.flushCache = l.cache l.cache = make(map[string]Values) + for k, _ := range l.cacheDirtySort { + sort.Sort(l.flushCache[k]) + } l.cacheDirtySort = make(map[string]bool) valuesByKey := make(map[string]Values) From 7baba84a2161587cf1533d50b9a1ea5dde94c667 Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Sat, 26 Sep 2015 09:02:14 -0400 Subject: [PATCH 082/139] Ensure we don't have duplicate values. Fix panic in compaction. --- tsdb/engine/pd1/encoding.go | 19 +++++++++++++++ tsdb/engine/pd1/pd1.go | 19 ++++++++------- tsdb/engine/pd1/pd1_test.go | 47 +++++++++++++++++++++++++++++++++++++ tsdb/engine/pd1/wal.go | 4 ++-- 4 files changed, 79 insertions(+), 10 deletions(-) diff --git a/tsdb/engine/pd1/encoding.go b/tsdb/engine/pd1/encoding.go index 5d1bee9a8fe..bf06fb98807 100644 --- a/tsdb/engine/pd1/encoding.go +++ b/tsdb/engine/pd1/encoding.go @@ -1,6 +1,7 @@ package pd1 import ( + "sort" "time" "github.com/dgryski/go-tsz" @@ -75,6 +76,24 @@ func (v Values) DecodeSameTypeBlock(block []byte) Values { return nil } +// Deduplicate returns a new Values slice with any values +// that have the same timestamp removed. The Value that appears +// last in the slice is the one that is kept. The returned slice is in ascending order +func (v Values) Deduplicate() Values { + m := make(map[int64]Value) + for _, val := range v { + m[val.UnixNano()] = val + } + + a := make([]Value, 0, len(m)) + for _, val := range m { + a = append(a, val) + } + sort.Sort(Values(a)) + + return a +} + // Sort methods func (a Values) Len() int { return len(a) } func (a Values) Swap(i, j int) { a[i], a[j] = a[j], a[i] } diff --git a/tsdb/engine/pd1/pd1.go b/tsdb/engine/pd1/pd1.go index 40aa7f8050d..5ae35e47980 100644 --- a/tsdb/engine/pd1/pd1.go +++ b/tsdb/engine/pd1/pd1.go @@ -389,6 +389,7 @@ func (e *Engine) rewriteFile(oldDF *dataFile, valuesByID map[uint64]Values) erro if len(valuesByID) == 0 { return nil } + // we need the values in sorted order so that we can merge them into the // new file as we read the old file ids := make([]uint64, 0, len(valuesByID)) @@ -506,7 +507,7 @@ func (e *Engine) rewriteFile(oldDF *dataFile, valuesByID map[uint64]Values) erro // determine if there's a block after this with the same id and get its time hasFutureBlock := false nextTime := int64(0) - if fpos < oldDF.size { + if fpos < oldDF.indexPosition() { nextID := btou64(oldDF.mmap[fpos : fpos+8]) if nextID == id { hasFutureBlock = true @@ -530,7 +531,7 @@ func (e *Engine) rewriteFile(oldDF *dataFile, valuesByID map[uint64]Values) erro currentPosition += uint32(12 + len(newBlock)) - if fpos >= oldDF.size { + if fpos >= oldDF.indexPosition() { break } } @@ -877,17 +878,15 @@ func (e *Engine) DecodeAndCombine(newValues Values, block, buf []byte, nextTime }) values = append(values, newValues[:pos]...) remainingValues = newValues[pos:] - sort.Sort(values) + values = values.Deduplicate() } else { - requireSort := values.MaxTime() > newValues.MinTime() + requireSort := values.MaxTime() >= newValues.MinTime() values = append(values, newValues...) if requireSort { - sort.Sort(values) + values = values.Deduplicate() } } - // TODO: deduplicate values - if len(values) > DefaultMaxPointsPerBlock { remainingValues = values[DefaultMaxPointsPerBlock:] values = values[:DefaultMaxPointsPerBlock] @@ -986,6 +985,10 @@ func (d *dataFile) IDToPosition() map[uint64]uint32 { return m } +func (d *dataFile) indexPosition() uint32 { + return d.size - uint32(d.SeriesCount()*12+20) +} + // StartingPositionForID returns the position in the file of the // first block for the given ID. If zero is returned the ID doesn't // have any data in this file. @@ -1123,7 +1126,7 @@ func (c *cursor) Next() (int64, interface{}) { // if we have a file set, see if the next block is for this ID if c.f != nil && c.pos < c.f.size { nextBlockID := btou64(c.f.mmap[c.pos : c.pos+8]) - if nextBlockID == c.id { + if nextBlockID == c.id && c.pos != c.f.indexPosition() { return c.decodeBlockAndGetValues(c.pos) } } diff --git a/tsdb/engine/pd1/pd1_test.go b/tsdb/engine/pd1/pd1_test.go index a20cc665ced..3be9bc63bba 100644 --- a/tsdb/engine/pd1/pd1_test.go +++ b/tsdb/engine/pd1/pd1_test.go @@ -184,6 +184,53 @@ func TestEngine_WriteIndexQueryAcrossDataFiles(t *testing.T) { verify("cpu,host=B", []models.Point{p2, p8, p4, p6}, 0) } +func TestEngine_WriteOverwritePreviousPoint(t *testing.T) { + e := OpenDefaultEngine() + defer e.Cleanup() + + e.Shard = newFieldCodecMock(map[string]influxql.DataType{"value": influxql.Float}) + fields := []string{"value"} + var codec *tsdb.FieldCodec + + p1 := parsePoint("cpu,host=A value=1.1 1000000000") + p2 := parsePoint("cpu,host=A value=1.2 1000000000") + p3 := parsePoint("cpu,host=A value=1.3 1000000000") + + if err := e.WritePoints([]models.Point{p1, p2}, nil, nil); err != nil { + t.Fatalf("failed to write points: %s", err.Error()) + } + + c := e.Cursor("cpu,host=A", fields, codec, true) + k, v := c.Next() + if k != p2.UnixNano() { + t.Fatalf("time wrong:\n\texp:%d\n\tgot:%d\n", p2.UnixNano(), k) + } + if 1.2 != v { + t.Fatalf("data wrong:\n\texp:%f\n\tgot:%f", 1.2, v.(float64)) + } + k, v = c.Next() + if k != tsdb.EOF { + t.Fatal("expected EOF") + } + + if err := e.WritePoints([]models.Point{p3}, nil, nil); err != nil { + t.Fatalf("failed to write points: %s", err.Error()) + } + + c = e.Cursor("cpu,host=A", fields, codec, true) + k, v = c.Next() + if k != p3.UnixNano() { + t.Fatalf("time wrong:\n\texp:%d\n\tgot:%d\n", p3.UnixNano(), k) + } + if 1.3 != v { + t.Fatalf("data wrong:\n\texp:%f\n\tgot:%f", 1.3, v.(float64)) + } + k, v = c.Next() + if k != tsdb.EOF { + t.Fatal("expected EOF") + } +} + func TestEngine_WriteIndexBenchmarkNames(t *testing.T) { t.Skip("whatevs") diff --git a/tsdb/engine/pd1/wal.go b/tsdb/engine/pd1/wal.go index e7fb7fb39a4..063e0d884b7 100644 --- a/tsdb/engine/pd1/wal.go +++ b/tsdb/engine/pd1/wal.go @@ -282,7 +282,7 @@ func (l *Log) addToCache(points []models.Point, fields map[string]*tsdb.Measurem // only mark it as dirty if it isn't already if _, ok := l.cacheDirtySort[k]; !ok && len(cacheValues) > 0 { - dirty := cacheValues[len(cacheValues)-1].Time().UnixNano() > v.Time().UnixNano() + dirty := cacheValues[len(cacheValues)-1].Time().UnixNano() >= v.Time().UnixNano() if dirty { l.cacheDirtySort[k] = true } @@ -522,7 +522,7 @@ func (l *Log) flush(flush flushType) error { l.flushCache = l.cache l.cache = make(map[string]Values) for k, _ := range l.cacheDirtySort { - sort.Sort(l.flushCache[k]) + l.flushCache[k] = l.flushCache[k].Deduplicate() } l.cacheDirtySort = make(map[string]bool) valuesByKey := make(map[string]Values) From 2eb2a647d626b570fd850192f81b6b50b790a222 Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Sat, 26 Sep 2015 10:47:36 -0400 Subject: [PATCH 083/139] Add multicursor to combine wal and index --- tsdb/engine/pd1/pd1.go | 24 +++----- tsdb/engine/pd1/pd1_test.go | 119 ++++++++++++++++++++++++------------ tsdb/engine/pd1/wal.go | 3 + 3 files changed, 89 insertions(+), 57 deletions(-) diff --git a/tsdb/engine/pd1/pd1.go b/tsdb/engine/pd1/pd1.go index 5ae35e47980..9ecb6fe5b03 100644 --- a/tsdb/engine/pd1/pd1.go +++ b/tsdb/engine/pd1/pd1.go @@ -79,11 +79,6 @@ type Engine struct { // and returns a hash identifier. It's not guaranteed to be unique. HashSeriesField func(key string) uint64 - // Shard is an interface that can pull back field type information based on measurement name - Shard interface { - FieldCodec(measurementName string) *tsdb.FieldCodec - } - WAL *Log RotateFileSize uint32 @@ -187,9 +182,6 @@ func (e *Engine) SetLogOutput(w io.Writer) {} // LoadMetadataIndex loads the shard metadata into memory. func (e *Engine) LoadMetadataIndex(shard *tsdb.Shard, index *tsdb.DatabaseIndex, measurementFields map[string]*tsdb.MeasurementFields) error { - e.Shard = shard - // TODO: write the metadata from the WAL - // Load measurement metadata fields, err := e.readFields() if err != nil { @@ -682,21 +674,18 @@ func (e *Engine) Begin(writable bool) (tsdb.Tx, error) { return e, nil } -// TODO: make the cursor take a field name +// TODO: handle multiple fields and descending func (e *Engine) Cursor(series string, fields []string, dec *tsdb.FieldCodec, ascending bool) tsdb.Cursor { - measurementName := tsdb.MeasurementFromSeriesKey(series) - codec := e.Shard.FieldCodec(measurementName) - if codec == nil { - return &cursor{} - } - field := codec.FieldByName("value") - if field == nil { + field := dec.FieldByName("value") + if field == nil || len(fields) > 1 { panic("pd1 engine only supports one field with name of value") } // TODO: ensure we map the collisions id := hashSeriesField(seriesFieldKey(series, field.Name)) - return newCursor(id, field.Type, e.copyFilesCollection(), ascending) + indexCursor := newCursor(id, field.Type, e.copyFilesCollection(), ascending) + wc := e.WAL.Cursor(series, fields, dec, ascending) + return tsdb.MultiCursor(wc, indexCursor) } func (e *Engine) copyFilesCollection() []*dataFile { @@ -1167,6 +1156,7 @@ func (c *cursor) decodeBlockAndGetValues(position uint32) (int64, interface{}) { v := c.vals[0] c.vals = c.vals[1:] + return v.Time().UnixNano(), v.Value() } diff --git a/tsdb/engine/pd1/pd1_test.go b/tsdb/engine/pd1/pd1_test.go index 3be9bc63bba..097c5155363 100644 --- a/tsdb/engine/pd1/pd1_test.go +++ b/tsdb/engine/pd1/pd1_test.go @@ -19,8 +19,6 @@ func TestEngine_WriteAndReadFloats(t *testing.T) { e := OpenDefaultEngine() defer e.Cleanup() - e.Shard = newFieldCodecMock(map[string]influxql.DataType{"value": influxql.Float}) - p1 := parsePoint("cpu,host=A value=1.1 1000000000") p2 := parsePoint("cpu,host=B value=1.2 1000000000") p3 := parsePoint("cpu,host=A value=2.1 2000000000") @@ -31,11 +29,17 @@ func TestEngine_WriteAndReadFloats(t *testing.T) { } fields := []string{"value"} - var codec *tsdb.FieldCodec + codec := tsdb.NewFieldCodec(map[string]*tsdb.Field{ + "value": { + ID: uint8(1), + Name: "value", + Type: influxql.Float, + }, + }) verify := func(checkSingleBVal bool) { c := e.Cursor("cpu,host=A", fields, codec, true) - k, v := c.Next() + k, v := c.SeekTo(0) if k != p1.UnixNano() { t.Fatalf("p1 time wrong:\n\texp:%d\n\tgot:%d\n", p1.UnixNano(), k) } @@ -55,7 +59,7 @@ func TestEngine_WriteAndReadFloats(t *testing.T) { } c = e.Cursor("cpu,host=B", fields, codec, true) - k, v = c.Next() + k, v = c.SeekTo(0) if k != p2.UnixNano() { t.Fatalf("p2 time wrong:\n\texp:%d\n\tgot:%d\n", p2.UnixNano(), k) } @@ -78,7 +82,7 @@ func TestEngine_WriteAndReadFloats(t *testing.T) { verify(false) c := e.Cursor("cpu,host=B", fields, codec, true) - k, v := c.Next() + k, v := c.SeekTo(0) if k != p2.UnixNano() { t.Fatalf("p2 time wrong:\n\texp:%d\n\tgot:%d\n", p2.UnixNano(), k) } @@ -129,7 +133,6 @@ func TestEngine_WriteIndexQueryAcrossDataFiles(t *testing.T) { e := OpenDefaultEngine() defer e.Cleanup() - e.Shard = newFieldCodecMock(map[string]influxql.DataType{"value": influxql.Float}) e.RotateFileSize = 10 p1 := parsePoint("cpu,host=A value=1.1 1000000000") @@ -155,21 +158,24 @@ func TestEngine_WriteIndexQueryAcrossDataFiles(t *testing.T) { } fields := []string{"value"} - var codec *tsdb.FieldCodec + codec := tsdb.NewFieldCodec(map[string]*tsdb.Field{ + "value": { + ID: uint8(1), + Name: "value", + Type: influxql.Float, + }, + }) verify := func(series string, points []models.Point, seek int64) { c := e.Cursor(series, fields, codec, true) - // we we want to seek, do it and verify the first point matches - if seek != 0 { - k, v := c.SeekTo(seek) - p := points[0] - val := p.Fields()["value"] - if p.UnixNano() != k || val != v { - t.Fatalf("expected to seek to first point\n\texp: %d %f\n\tgot: %d %f", p.UnixNano(), val, k, v) - } - points = points[1:] + k, v := c.SeekTo(seek) + p := points[0] + val := p.Fields()["value"] + if p.UnixNano() != k || val != v { + t.Fatalf("expected to seek to first point\n\texp: %d %f\n\tgot: %d %f", p.UnixNano(), val, k, v) } + points = points[1:] for _, p := range points { k, v := c.Next() @@ -182,15 +188,22 @@ func TestEngine_WriteIndexQueryAcrossDataFiles(t *testing.T) { verify("cpu,host=A", []models.Point{p1, p7, p3, p5}, 0) verify("cpu,host=B", []models.Point{p2, p8, p4, p6}, 0) + verify("cpu,host=A", []models.Point{p5}, 5000000000) + verify("cpu,host=B", []models.Point{p6}, 5000000000) } func TestEngine_WriteOverwritePreviousPoint(t *testing.T) { e := OpenDefaultEngine() defer e.Cleanup() - e.Shard = newFieldCodecMock(map[string]influxql.DataType{"value": influxql.Float}) fields := []string{"value"} - var codec *tsdb.FieldCodec + codec := tsdb.NewFieldCodec(map[string]*tsdb.Field{ + "value": { + ID: uint8(1), + Name: "value", + Type: influxql.Float, + }, + }) p1 := parsePoint("cpu,host=A value=1.1 1000000000") p2 := parsePoint("cpu,host=A value=1.2 1000000000") @@ -201,7 +214,7 @@ func TestEngine_WriteOverwritePreviousPoint(t *testing.T) { } c := e.Cursor("cpu,host=A", fields, codec, true) - k, v := c.Next() + k, v := c.SeekTo(0) if k != p2.UnixNano() { t.Fatalf("time wrong:\n\texp:%d\n\tgot:%d\n", p2.UnixNano(), k) } @@ -218,7 +231,7 @@ func TestEngine_WriteOverwritePreviousPoint(t *testing.T) { } c = e.Cursor("cpu,host=A", fields, codec, true) - k, v = c.Next() + k, v = c.SeekTo(0) if k != p3.UnixNano() { t.Fatalf("time wrong:\n\texp:%d\n\tgot:%d\n", p3.UnixNano(), k) } @@ -231,6 +244,51 @@ func TestEngine_WriteOverwritePreviousPoint(t *testing.T) { } } +func TestEngine_CursorCombinesWALAndIndex(t *testing.T) { + e := OpenDefaultEngine() + defer e.Cleanup() + + fields := []string{"value"} + codec := tsdb.NewFieldCodec(map[string]*tsdb.Field{ + "value": { + ID: uint8(1), + Name: "value", + Type: influxql.Float, + }, + }) + + p1 := parsePoint("cpu,host=A value=1.1 1000000000") + p2 := parsePoint("cpu,host=A value=1.2 2000000000") + + if err := e.WritePoints([]models.Point{p1}, nil, nil); err != nil { + t.Fatalf("failed to write points: %s", err.Error()) + } + e.WAL.SkipCache = false + if err := e.WritePoints([]models.Point{p2}, nil, nil); err != nil { + t.Fatalf("failed to write points: %s", err.Error()) + } + + c := e.Cursor("cpu,host=A", fields, codec, true) + k, v := c.SeekTo(0) + if k != p1.UnixNano() { + t.Fatalf("time wrong:\n\texp:%d\n\tgot:%d\n", p1.UnixNano(), k) + } + if 1.1 != v { + t.Fatalf("data wrong:\n\texp:%f\n\tgot:%f", 1.1, v.(float64)) + } + k, v = c.Next() + if k != p2.UnixNano() { + t.Fatalf("time wrong:\n\texp:%d\n\tgot:%d\n", p2.UnixNano(), k) + } + if 1.2 != v { + t.Fatalf("data wrong:\n\texp:%f\n\tgot:%f", 1.2, v.(float64)) + } + k, v = c.Next() + if k != tsdb.EOF { + t.Fatal("expected EOF") + } +} + func TestEngine_WriteIndexBenchmarkNames(t *testing.T) { t.Skip("whatevs") @@ -295,25 +353,6 @@ func (e *Engine) Cleanup() error { return nil } -func newFieldCodecMock(fields map[string]influxql.DataType) *FieldCodeMock { - m := make(map[string]*tsdb.Field) - - for n, t := range fields { - m[n] = &tsdb.Field{Name: n, Type: t} - } - codec := tsdb.NewFieldCodec(m) - - return &FieldCodeMock{codec: codec} -} - -type FieldCodeMock struct { - codec *tsdb.FieldCodec -} - -func (f *FieldCodeMock) FieldCodec(m string) *tsdb.FieldCodec { - return f.codec -} - func parsePoints(buf string) []models.Point { points, err := models.ParsePointsString(buf) if err != nil { diff --git a/tsdb/engine/pd1/wal.go b/tsdb/engine/pd1/wal.go index 063e0d884b7..be9dd48e71d 100644 --- a/tsdb/engine/pd1/wal.go +++ b/tsdb/engine/pd1/wal.go @@ -177,6 +177,9 @@ func (l *Log) Cursor(series string, fields []string, dec *tsdb.FieldCodec, ascen defer l.cacheLock.RUnlock() // TODO: make this work for other fields + if len(fields) != 1 || fields[0] != "value" { + panic("pd1 wal only supports 1 field with name value") + } ck := seriesFieldKey(series, "value") values := l.cache[ck] From 48069e782c6eba7f3a7210b5444b04926bb604ac Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Sun, 27 Sep 2015 14:45:55 -0400 Subject: [PATCH 084/139] Add compaction and time range based write locks. --- tsdb/config.go | 16 +- tsdb/engine/pd1/encoding.go | 8 + tsdb/engine/pd1/pd1.go | 512 +++++++++++++++++++++++++--------- tsdb/engine/pd1/pd1_test.go | 84 ++++++ tsdb/engine/pd1/wal.go | 4 +- tsdb/engine/pd1/wal_test.go | 2 +- tsdb/engine/pd1/write_lock.go | 86 ++++++ 7 files changed, 582 insertions(+), 130 deletions(-) create mode 100644 tsdb/engine/pd1/write_lock.go diff --git a/tsdb/config.go b/tsdb/config.go index 4aec4a14e52..8716a455375 100644 --- a/tsdb/config.go +++ b/tsdb/config.go @@ -45,8 +45,10 @@ const ( DefaultPartitionSizeThreshold = 50 * 1024 * 1024 // 50MB // Default WAL settings for the PD1 WAL - DefaultFlushMemorySizeThreshold = 50 * 1024 * 1024 // 50MB + DefaultFlushMemorySizeThreshold = 10 * 1024 * 1024 // 10MB DefaultMaxMemorySizeThreshold = 200 * 1024 * 1024 // 200MB + DefaultIndexCompactionAge = 10 * time.Minute + DefaultIndexCompactionFileCount = 5 ) type Config struct { @@ -71,6 +73,16 @@ type Config struct { WALFlushMemorySizeThreshold int `toml:"wal-flush-memory-size-threshold"` WALMaxMemorySizeThreshold int `toml:"wal-max-memory-size-threshold"` + // compaction options for pd1 introduced in 0.9.5 + + // IndexCompactionAge specifies the duration after the data file creation time + // at which it is eligible to be compacted + IndexCompactionAge time.Duration `toml:"index-compaction-age"` + + // IndexCompactionFileCount specifies the minimum number of data files that + // must be eligible for compaction before actually running one + IndexCompactionFileCount int `toml:"index-compaction-file-count"` + // Query logging QueryLogEnabled bool `toml:"query-log-enabled"` } @@ -90,6 +102,8 @@ func NewConfig() Config { WALPartitionSizeThreshold: DefaultPartitionSizeThreshold, WALFlushMemorySizeThreshold: DefaultFlushMemorySizeThreshold, WALMaxMemorySizeThreshold: DefaultMaxMemorySizeThreshold, + IndexCompactionAge: DefaultIndexCompactionAge, + IndexCompactionFileCount: DefaultIndexCompactionFileCount, QueryLogEnabled: true, } diff --git a/tsdb/engine/pd1/encoding.go b/tsdb/engine/pd1/encoding.go index bf06fb98807..9a6d9fea7a6 100644 --- a/tsdb/engine/pd1/encoding.go +++ b/tsdb/engine/pd1/encoding.go @@ -76,6 +76,14 @@ func (v Values) DecodeSameTypeBlock(block []byte) Values { return nil } +// DecodeBlock takes a byte array and will decode into values of the appropriate type +// based on the block +func DecodeBlock(block []byte) Values { + // TODO: add support for other block types + a, _ := DecodeFloatBlock(block) + return a +} + // Deduplicate returns a new Values slice with any values // that have the same timestamp removed. The Value that appears // last in the slice is the one that is kept. The returned slice is in ascending order diff --git a/tsdb/engine/pd1/pd1.go b/tsdb/engine/pd1/pd1.go index 9ecb6fe5b03..01087194573 100644 --- a/tsdb/engine/pd1/pd1.go +++ b/tsdb/engine/pd1/pd1.go @@ -10,6 +10,7 @@ import ( "math" "os" "path/filepath" + "reflect" "sort" "sync" "syscall" @@ -50,8 +51,10 @@ func init() { } const ( - // DefaultBlockSize is the default size of uncompressed points blocks. - DefaultBlockSize = 512 * 1024 // 512KB + MaxDataFileSize = 1024 * 1024 * 1024 // 1GB + + // DefaultRotateBlockSize is the default size to rotate to a new compressed block + DefaultRotateBlockSize = 512 * 1024 // 512KB DefaultRotateFileSize = 10 * 1024 * 1024 // 10MB @@ -68,8 +71,9 @@ var _ tsdb.Engine = &Engine{} // Engine represents a storage engine with compressed blocks. type Engine struct { - mu sync.Mutex - path string + writeLock *writeLock + metaLock sync.Mutex + path string // deletesPending mark how many old data files are waiting to be deleted. This will // keep a close from returning until all deletes finish @@ -81,12 +85,19 @@ type Engine struct { WAL *Log - RotateFileSize uint32 + RotateFileSize uint32 + SkipCompaction bool + CompactionAge time.Duration + CompactionFileCount int + // filesLock is only for modifying and accessing the files slice filesLock sync.RWMutex files dataFiles currentFileID int - queryLock sync.RWMutex + + // queryLock keeps data files from being deleted or the store from + // being closed while queries are running + queryLock sync.RWMutex } // NewEngine returns a new instance of Engine. @@ -98,12 +109,15 @@ func NewEngine(path string, walPath string, opt tsdb.EngineOptions) tsdb.Engine w.LoggingEnabled = opt.Config.WALLoggingEnabled e := &Engine{ - path: path, + path: path, + writeLock: &writeLock{}, // TODO: this is the function where we can inject a check against the in memory collisions - HashSeriesField: hashSeriesField, - WAL: w, - RotateFileSize: DefaultRotateFileSize, + HashSeriesField: hashSeriesField, + WAL: w, + RotateFileSize: DefaultRotateFileSize, + CompactionAge: opt.Config.IndexCompactionAge, + CompactionFileCount: opt.Config.IndexCompactionFileCount, } e.WAL.Index = e @@ -157,9 +171,18 @@ func (e *Engine) Open() error { // Close closes the engine. func (e *Engine) Close() error { + // get all the locks so queries, writes, and compactions stop before closing e.queryLock.Lock() defer e.queryLock.Unlock() - + e.metaLock.Lock() + defer e.metaLock.Unlock() + min, max := int64(math.MinInt64), int64(math.MaxInt64) + e.writeLock.LockRange(min, max) + defer e.writeLock.UnlockRange(min, max) + e.filesLock.Lock() + defer e.filesLock.Unlock() + + // ensure all deletes have been processed e.deletesPending.Wait() for _, df := range e.files { @@ -224,38 +247,328 @@ func (e *Engine) WritePoints(points []models.Point, measurementFieldsToSave map[ return e.WAL.WritePoints(points, measurementFieldsToSave, seriesToCreate) } -func (e *Engine) WriteAndCompact(pointsByKey map[string]Values, measurementFieldsToSave map[string]*tsdb.MeasurementFields, seriesToCreate []*tsdb.SeriesCreate) error { - e.mu.Lock() - defer e.mu.Unlock() +func (e *Engine) Write(pointsByKey map[string]Values, measurementFieldsToSave map[string]*tsdb.MeasurementFields, seriesToCreate []*tsdb.SeriesCreate) error { + err, startTime, endTime, valuesByID := e.convertKeysAndWriteMetadata(pointsByKey, measurementFieldsToSave, seriesToCreate) + if err != nil { + return err + } + if len(valuesByID) == 0 { + return nil + } + + e.writeLock.LockRange(startTime, endTime) + defer e.writeLock.UnlockRange(startTime, endTime) - if err := e.writeNewFields(measurementFieldsToSave); err != nil { + if len(e.files) == 0 { + return e.rewriteFile(nil, valuesByID) + } + + maxTime := int64(math.MaxInt64) + // reverse through the data files and write in the data + files := e.copyFilesCollection() + for i := len(files) - 1; i >= 0; i-- { + f := files[i] + // max times are exclusive, so add 1 to it + fileMax := f.MaxTime() + 1 + fileMin := f.MinTime() + // if the file is < rotate, write all data between fileMin and maxTime + if f.size < e.RotateFileSize { + if err := e.rewriteFile(f, e.filterDataBetweenTimes(valuesByID, fileMin, maxTime)); err != nil { + return err + } + continue + } + // if the file is > rotate: + // write all data between fileMax and maxTime into new file + // write all data between fileMin and fileMax into old file + if err := e.rewriteFile(nil, e.filterDataBetweenTimes(valuesByID, fileMax, maxTime)); err != nil { + return err + } + if err := e.rewriteFile(f, e.filterDataBetweenTimes(valuesByID, fileMin, fileMax)); err != nil { + return err + } + maxTime = fileMin + } + // for any data leftover, write into a new file since it's all older + // than any file we currently have + err = e.rewriteFile(nil, valuesByID) + + if !e.SkipCompaction && e.shouldCompact() { + go e.Compact() + } + + return err +} + +func (e *Engine) Compact() error { + // we're looping here to ensure that the files we've marked to compact are + // still there after we've obtained the write lock + var minTime, maxTime int64 + var files dataFiles + for { + files = e.filesToCompact() + if len(files) < 2 { + return nil + } + minTime = files[0].MinTime() + maxTime = files[len(files)-1].MaxTime() + + e.writeLock.LockRange(minTime, maxTime) + + // if the files are different after obtaining the write lock, one or more + // was rewritten. Release the lock and try again. This shouldn't happen really. + if !reflect.DeepEqual(files, e.filesToCompact()) { + e.writeLock.UnlockRange(minTime, maxTime) + continue + } + + // we've got the write lock and the files are all there + break + } + defer e.writeLock.UnlockRange(minTime, maxTime) + + positions := make([]uint32, len(files)) + ids := make([]uint64, len(files)) + + // initilaize for writing + f, err := os.OpenFile(e.nextFileName(), os.O_CREATE|os.O_RDWR, 0666) + if err != nil { return err } - if err := e.writeNewSeries(seriesToCreate); err != nil { + + // write the magic number + if _, err := f.Write(u32tob(magicNumber)); err != nil { + f.Close() return err } + for i, df := range files { + ids[i] = btou64(df.mmap[4:12]) + positions[i] = 4 + } + currentPosition := uint32(fileHeaderSize) + newPositions := make([]uint32, 0) + newIDs := make([]uint64, 0) + buf := make([]byte, DefaultRotateBlockSize) + for { + // find the min ID so we can write it to the file + minID := uint64(math.MaxUint64) + for _, id := range ids { + if minID > id { + minID = id + } + } + if minID == 0 { // we've emptied all the files + break + } + + newIDs = append(newIDs, minID) + newPositions = append(newPositions, currentPosition) + + // write the blocks in order from the files with this id. as we + // go merge blocks together from one file to another, if the right size + var previousValues Values + for i, id := range ids { + if id != minID { + continue + } + df := files[i] + pos := positions[i] + fid, _, block := df.block(pos) + if fid != id { + panic("not possible") + } + newPos := pos + uint32(blockHeaderSize+len(block)) + positions[i] = newPos + + // write the blocks out to file that are already at their size limit + for { + // if the next block is the same ID, we don't need to decod this one + // so we can just write it out to the file + nextID, _, nextBlock := df.block(newPos) + newPos = newPos + uint32(blockHeaderSize+len(block)) + + if len(previousValues) > 0 { + previousValues = append(previousValues, previousValues.DecodeSameTypeBlock(block)...) + } else if len(block) > DefaultRotateBlockSize { + if _, err := f.Write(df.mmap[pos:newPos]); err != nil { + return err + } + currentPosition += uint32(newPos - pos) + } else { + previousValues = DecodeBlock(block) + } + + // write the previous values and clear if we've hit the limit + if len(previousValues) > DefaultMaxPointsPerBlock { + b := previousValues.Encode(buf) + if err := e.writeBlock(f, id, b); err != nil { + // fail hard. If we can't write a file someone needs to get woken up + panic(fmt.Sprintf("failure writing block: %s", err.Error())) + } + currentPosition += uint32(blockHeaderSize + len(b)) + previousValues = nil + } + + // move to the next block in this file only if the id is the same + if nextID != id { + ids[i] = nextID + break + } + positions[i] = newPos + block = nextBlock + newPos = newPos + uint32(blockHeaderSize+len(block)) + } + } + + if len(previousValues) > 0 { + b := previousValues.Encode(buf) + if err := e.writeBlock(f, minID, b); err != nil { + // fail hard. If we can't write a file someone needs to get woken up + panic(fmt.Sprintf("failure writing block: %s", err.Error())) + } + currentPosition += uint32(blockHeaderSize + len(b)) + } + } + + err, newDF := e.writeIndexAndGetDataFile(f, minTime, maxTime, newIDs, newPositions) + if err != nil { + return err + } + + // update engine with new file pointers + e.filesLock.Lock() + var newFiles dataFiles + for _, df := range e.files { + // exclude any files that were compacted + include := true + for _, f := range files { + if f == df { + include = false + break + } + } + if include { + newFiles = append(newFiles, df) + } + } + newFiles = append(newFiles, newDF) + sort.Sort(newFiles) + e.files = newFiles + e.filesLock.Unlock() + + // delete the old files in a goroutine so running queries won't block the write + // from completing + e.deletesPending.Add(1) + go func() { + for _, f := range files { + if err := f.Delete(); err != nil { + // TODO: log this error + } + } + e.deletesPending.Done() + }() + + return nil +} + +func (e *Engine) writeBlock(f *os.File, id uint64, block []byte) error { + if _, err := f.Write(append(u64tob(id), u32tob(uint32(len(block)))...)); err != nil { + return err + } + _, err := f.Write(block) + return err +} + +func (e *Engine) writeIndexAndGetDataFile(f *os.File, minTime, maxTime int64, ids []uint64, newPositions []uint32) (error, *dataFile) { + // write the file index, starting with the series ids and their positions + for i, id := range ids { + if _, err := f.Write(u64tob(id)); err != nil { + return err, nil + } + if _, err := f.Write(u32tob(newPositions[i])); err != nil { + return err, nil + } + } + + // write the min time, max time + if _, err := f.Write(append(u64tob(uint64(minTime)), u64tob(uint64(maxTime))...)); err != nil { + return err, nil + } + + // series count + if _, err := f.Write(u32tob(uint32(len(ids)))); err != nil { + return err, nil + } + + // sync it and see4k back to the beginning to hand off to the mmap + if err := f.Sync(); err != nil { + return err, nil + } + if _, err := f.Seek(0, 0); err != nil { + return err, nil + } + + // now open it as a memory mapped data file + newDF, err := NewDataFile(f) + if err != nil { + return err, nil + } + + return nil, newDF +} + +func (e *Engine) shouldCompact() bool { + return len(e.filesToCompact()) >= e.CompactionFileCount +} + +func (e *Engine) filesToCompact() dataFiles { + e.filesLock.RLock() + defer e.filesLock.RUnlock() + + a := make([]*dataFile, 0) + for _, df := range e.files { + if time.Since(df.modTime) > e.CompactionAge && df.size < MaxDataFileSize { + a = append(a, df) + } + } + return a +} + +func (e *Engine) convertKeysAndWriteMetadata(pointsByKey map[string]Values, measurementFieldsToSave map[string]*tsdb.MeasurementFields, seriesToCreate []*tsdb.SeriesCreate) (err error, minTime, maxTime int64, valuesByID map[uint64]Values) { + e.metaLock.Lock() + defer e.metaLock.Unlock() + + if err := e.writeNewFields(measurementFieldsToSave); err != nil { + return err, 0, 0, nil + } + if err := e.writeNewSeries(seriesToCreate); err != nil { + return err, 0, 0, nil + } if len(pointsByKey) == 0 { - return nil + return nil, 0, 0, nil } // read in keys and assign any that aren't defined b, err := e.readCompressedFile("ids") if err != nil { - return err + return err, 0, 0, nil } ids := make(map[string]uint64) if b != nil { if err := json.Unmarshal(b, &ids); err != nil { - return err + return err, 0, 0, nil } } // these are values that are newer than anything stored in the shard - valuesByID := make(map[uint64]Values) + valuesByID = make(map[uint64]Values) idToKey := make(map[uint64]string) // we only use this map if new ids are being created newKeys := false + // track the min and max time of values being inserted so we can lock that time range + minTime = int64(math.MaxInt64) + maxTime = int64(math.MinInt64) for k, values := range pointsByKey { var id uint64 var ok bool @@ -285,51 +598,27 @@ func (e *Engine) WriteAndCompact(pointsByKey map[string]Values, measurementField } } + if minTime > values.MinTime() { + minTime = values.MinTime() + } + if maxTime < values.MaxTime() { + maxTime = values.MaxTime() + } + valuesByID[id] = values } if newKeys { b, err := json.Marshal(ids) if err != nil { - return err + return err, 0, 0, nil } if err := e.replaceCompressedFile("ids", b); err != nil { - return err + return err, 0, 0, nil } } - if len(e.files) == 0 { - return e.rewriteFile(nil, valuesByID) - } - - maxTime := int64(math.MaxInt64) - // reverse through the data files and write in the data - for i := len(e.files) - 1; i >= 0; i-- { - f := e.files[i] - // max times are exclusive, so add 1 to it - fileMax := f.MaxTime() + 1 - fileMin := f.MinTime() - // if the file is < rotate, write all data between fileMin and maxTime - if f.size < e.RotateFileSize { - if err := e.rewriteFile(f, e.filterDataBetweenTimes(valuesByID, fileMin, maxTime)); err != nil { - return err - } - continue - } - // if the file is > rotate: - // write all data between fileMax and maxTime into new file - // write all data between fileMin and fileMax into old file - if err := e.rewriteFile(nil, e.filterDataBetweenTimes(valuesByID, fileMax, maxTime)); err != nil { - return err - } - if err := e.rewriteFile(f, e.filterDataBetweenTimes(valuesByID, fileMin, fileMax)); err != nil { - return err - } - maxTime = fileMin - } - // for any data leftover, write into a new file since it's all older - // than any file we currently have - return e.rewriteFile(nil, valuesByID) + return } // filterDataBetweenTimes will create a new map with data between @@ -432,7 +721,7 @@ func (e *Engine) rewriteFile(oldDF *dataFile, valuesByID map[uint64]Values) erro // now combine the old file data with the new values, keeping track of // their positions - currentPosition := uint32(4) + currentPosition := uint32(fileHeaderSize) newPositions := make([]uint32, len(ids)) buf := make([]byte, DefaultMaxPointsPerBlock*20) for i, id := range ids { @@ -473,39 +762,26 @@ func (e *Engine) rewriteFile(oldDF *dataFile, valuesByID map[uint64]Values) erro if !ok { // TODO: ensure we encode only the amount in a block block := newVals.Encode(buf) - if _, err := f.Write(append(u64tob(id), u32tob(uint32(len(block)))...)); err != nil { - f.Close() - return err - } - if _, err := f.Write(block); err != nil { + if err := e.writeBlock(f, id, block); err != nil { f.Close() return err } - currentPosition += uint32(12 + len(block)) + currentPosition += uint32(blockHeaderSize + len(block)) continue } // it's in the file and the new values, combine them and write out for { - fid := btou64(oldDF.mmap[fpos : fpos+8]) + fid, _, block := oldDF.block(fpos) if fid != id { break } - length := btou32(oldDF.mmap[fpos+8 : fpos+12]) - block := oldDF.mmap[fpos+12 : fpos+12+length] - fpos += (12 + length) + fpos += uint32(blockHeaderSize + len(block)) // determine if there's a block after this with the same id and get its time - hasFutureBlock := false - nextTime := int64(0) - if fpos < oldDF.indexPosition() { - nextID := btou64(oldDF.mmap[fpos : fpos+8]) - if nextID == id { - hasFutureBlock = true - nextTime = int64(btou64(oldDF.mmap[fpos+12 : fpos+20])) - } - } + nextID, nextTime, _ := oldDF.block(fpos) + hasFutureBlock := nextID == id nv, newBlock, err := e.DecodeAndCombine(newVals, block, buf[:0], nextTime, hasFutureBlock) newVals = nv @@ -521,7 +797,7 @@ func (e *Engine) rewriteFile(oldDF *dataFile, valuesByID map[uint64]Values) erro return err } - currentPosition += uint32(12 + len(newBlock)) + currentPosition += uint32(blockHeaderSize + len(newBlock)) if fpos >= oldDF.indexPosition() { break @@ -540,51 +816,18 @@ func (e *Engine) rewriteFile(oldDF *dataFile, valuesByID map[uint64]Values) erro f.Close() return err } - currentPosition += uint32(12 + len(block)) + currentPosition += uint32(blockHeaderSize + len(block)) } } - // write the file index, starting with the series ids and their positions - for i, id := range ids { - if _, err := f.Write(u64tob(id)); err != nil { - f.Close() - return err - } - if _, err := f.Write(u32tob(newPositions[i])); err != nil { - f.Close() - return err - } - } - - // write the min time, max time - if _, err := f.Write(append(u64tob(uint64(minTime)), u64tob(uint64(maxTime))...)); err != nil { - f.Close() - return err - } - - // series count - if _, err := f.Write(u32tob(uint32(len(ids)))); err != nil { - f.Close() - return err - } - - // sync it and see4k back to the beginning to hand off to the mmap - if err := f.Sync(); err != nil { - return err - } - if _, err := f.Seek(0, 0); err != nil { - f.Close() - return err - } - - // now open it as a memory mapped data file - newDF, err := NewDataFile(f) + err, newDF := e.writeIndexAndGetDataFile(f, minTime, maxTime, ids, newPositions) if err != nil { + f.Close() return err } // update the engine to point at the new dataFiles - e.queryLock.Lock() + e.filesLock.Lock() var files dataFiles for _, df := range e.files { if df != oldDF { @@ -594,7 +837,7 @@ func (e *Engine) rewriteFile(oldDF *dataFile, valuesByID map[uint64]Values) erro files = append(files, newDF) sort.Sort(files) e.files = files - e.queryLock.Unlock() + e.filesLock.Unlock() // remove the old data file. no need to block returning the write, // but we need to let any running queries finish before deleting it @@ -671,6 +914,7 @@ func (e *Engine) SeriesCount() (n int, err error) { // Begin starts a new transaction on the engine. func (e *Engine) Begin(writable bool) (tsdb.Tx, error) { + e.queryLock.RLock() return e, nil } @@ -696,10 +940,14 @@ func (e *Engine) copyFilesCollection() []*dataFile { return a } -func (e *Engine) Size() int64 { return 0 } -func (e *Engine) Commit() error { return nil } -func (e *Engine) Rollback() error { return nil } -func (e *Engine) WriteTo(w io.Writer) (n int64, err error) { return 0, nil } +// TODO: refactor the Tx interface to not have Size, Commit, or WriteTo since they're not used +func (e *Engine) Size() int64 { panic("not implemented") } +func (e *Engine) Commit() error { panic("not implemented") } +func (e *Engine) WriteTo(w io.Writer) (n int64, err error) { panic("not implemented") } +func (e *Engine) Rollback() error { + e.queryLock.RUnlock() + return nil +} func (e *Engine) writeNewFields(measurementFieldsToSave map[string]*tsdb.MeasurementFields) error { if len(measurementFieldsToSave) == 0 { @@ -885,17 +1133,19 @@ func (e *Engine) DecodeAndCombine(newValues Values, block, buf []byte, nextTime } type dataFile struct { - f *os.File - mu sync.RWMutex - size uint32 - mmap []byte + f *os.File + mu sync.RWMutex + size uint32 + modTime time.Time + mmap []byte } // byte size constants for the data file const ( + fileHeaderSize = 4 seriesCountSize = 4 timeSize = 8 - fileHeaderSize = seriesCountSize + (2 * timeSize) + blockHeaderSize = 12 seriesIDSize = 8 seriesPositionSize = 4 seriesHeaderSize = seriesIDSize + seriesPositionSize @@ -908,14 +1158,14 @@ func NewDataFile(f *os.File) (*dataFile, error) { } mmap, err := syscall.Mmap(int(f.Fd()), 0, int(fInfo.Size()), syscall.PROT_READ, syscall.MAP_SHARED|MAP_POPULATE) if err != nil { - f.Close() return nil, err } return &dataFile{ - f: f, - mmap: mmap, - size: uint32(fInfo.Size()), + f: f, + mmap: mmap, + size: uint32(fInfo.Size()), + modTime: fInfo.ModTime(), }, nil } @@ -1007,6 +1257,16 @@ func (d *dataFile) StartingPositionForID(id uint64) uint32 { return uint32(0) } +func (d *dataFile) block(pos uint32) (id uint64, t int64, block []byte) { + if pos < d.indexPosition() { + id = btou64(d.mmap[pos : pos+8]) + length := btou32(d.mmap[pos+8 : pos+12]) + block = d.mmap[pos+blockHeaderSize : pos+blockHeaderSize+length] + t = int64(btou64(d.mmap[pos+blockHeaderSize : pos+blockHeaderSize+8])) + } + return +} + type dataFiles []*dataFile func (a dataFiles) Len() int { return len(a) } diff --git a/tsdb/engine/pd1/pd1_test.go b/tsdb/engine/pd1/pd1_test.go index 097c5155363..db867e93039 100644 --- a/tsdb/engine/pd1/pd1_test.go +++ b/tsdb/engine/pd1/pd1_test.go @@ -289,6 +289,89 @@ func TestEngine_CursorCombinesWALAndIndex(t *testing.T) { } } +func TestEngine_Compaction(t *testing.T) { + e := OpenDefaultEngine() + defer e.Cleanup() + + e.RotateFileSize = 10 + + p1 := parsePoint("cpu,host=A value=1.1 1000000000") + p2 := parsePoint("cpu,host=B value=1.1 1000000000") + if err := e.WritePoints([]models.Point{p1, p2}, nil, nil); err != nil { + t.Fatalf("failed to write points: %s", err.Error()) + } + + p3 := parsePoint("cpu,host=A value=2.4 4000000000") + p4 := parsePoint("cpu,host=B value=2.4 4000000000") + if err := e.WritePoints([]models.Point{p3, p4}, nil, nil); err != nil { + t.Fatalf("failed to write points: %s", err.Error()) + } + + p5 := parsePoint("cpu,host=A value=1.5 5000000000") + p6 := parsePoint("cpu,host=B value=2.5 5000000000") + if err := e.WritePoints([]models.Point{p5, p6}, nil, nil); err != nil { + t.Fatalf("failed to write points: %s", err.Error()) + } + + p7 := parsePoint("cpu,host=A value=1.5 6000000000") + p8 := parsePoint("cpu,host=B value=2.5 6000000000") + if err := e.WritePoints([]models.Point{p7, p8}, nil, nil); err != nil { + t.Fatalf("failed to write points: %s", err.Error()) + } + + if count := e.DataFileCount(); count != 4 { + t.Fatalf("expected 3 data files to exist but got %d", count) + } + + fields := []string{"value"} + codec := tsdb.NewFieldCodec(map[string]*tsdb.Field{ + "value": { + ID: uint8(1), + Name: "value", + Type: influxql.Float, + }, + }) + + e.CompactionAge = time.Duration(0) + + if err := e.Compact(); err != nil { + t.Fatalf("error compacting: %s", err.Error()) + } + + if count := e.DataFileCount(); count != 1 { + t.Fatalf("expected compaction to reduce data file count to 1 but got %d", count) + } + + verify := func(series string, points []models.Point, seek int64) { + c := e.Cursor(series, fields, codec, true) + + k, v := c.SeekTo(seek) + p := points[0] + val := p.Fields()["value"] + if p.UnixNano() != k || val != v { + t.Fatalf("expected to seek to first point\n\texp: %d %f\n\tgot: %d %f", p.UnixNano(), val, k, v) + } + points = points[1:] + + for _, p := range points { + k, v := c.Next() + val := p.Fields()["value"] + if p.UnixNano() != k || val != v { + t.Fatalf("expected to seek to first point\n\texp: %d %f\n\tgot: %d %f", p.UnixNano(), val, k, v.(float64)) + } + } + } + + if err := e.Close(); err != nil { + t.Fatalf("error closing: %s", err.Error()) + } + if err := e.Open(); err != nil { + t.Fatalf("error opening: %s", err.Error()) + } + verify("cpu,host=A", []models.Point{p1, p3, p5, p7}, 0) + verify("cpu,host=B", []models.Point{p2, p4, p6, p8}, 0) +} + func TestEngine_WriteIndexBenchmarkNames(t *testing.T) { t.Skip("whatevs") @@ -340,6 +423,7 @@ func OpenEngine(opt tsdb.EngineOptions) *Engine { panic(err) } e.WAL.SkipCache = true + e.SkipCompaction = true return e } diff --git a/tsdb/engine/pd1/wal.go b/tsdb/engine/pd1/wal.go index be9dd48e71d..1447cb5f06a 100644 --- a/tsdb/engine/pd1/wal.go +++ b/tsdb/engine/pd1/wal.go @@ -123,7 +123,7 @@ type Log struct { // IndexWriter is an interface for the indexed database the WAL flushes data to type IndexWriter interface { - WriteAndCompact(valuesByKey map[string]Values, measurementFieldsToSave map[string]*tsdb.MeasurementFields, seriesToCreate []*tsdb.SeriesCreate) error + Write(valuesByKey map[string]Values, measurementFieldsToSave map[string]*tsdb.MeasurementFields, seriesToCreate []*tsdb.SeriesCreate) error } func NewLog(path string) *Log { @@ -567,7 +567,7 @@ func (l *Log) flush(flush flushType) error { } startTime := time.Now() - if err := l.Index.WriteAndCompact(valuesByKey, mfc, scc); err != nil { + if err := l.Index.Write(valuesByKey, mfc, scc); err != nil { return err } if l.LoggingEnabled { diff --git a/tsdb/engine/pd1/wal_test.go b/tsdb/engine/pd1/wal_test.go index de03dfbd601..509ed6984ca 100644 --- a/tsdb/engine/pd1/wal_test.go +++ b/tsdb/engine/pd1/wal_test.go @@ -169,6 +169,6 @@ type MockIndexWriter struct { fn func(valuesByKey map[string]pd1.Values, measurementFieldsToSave map[string]*tsdb.MeasurementFields, seriesToCreate []*tsdb.SeriesCreate) error } -func (m *MockIndexWriter) WriteAndCompact(valuesByKey map[string]pd1.Values, measurementFieldsToSave map[string]*tsdb.MeasurementFields, seriesToCreate []*tsdb.SeriesCreate) error { +func (m *MockIndexWriter) Write(valuesByKey map[string]pd1.Values, measurementFieldsToSave map[string]*tsdb.MeasurementFields, seriesToCreate []*tsdb.SeriesCreate) error { return m.fn(valuesByKey, measurementFieldsToSave, seriesToCreate) } diff --git a/tsdb/engine/pd1/write_lock.go b/tsdb/engine/pd1/write_lock.go new file mode 100644 index 00000000000..5f48fb9311a --- /dev/null +++ b/tsdb/engine/pd1/write_lock.go @@ -0,0 +1,86 @@ +package pd1 + +import ( + "sync" +) + +// writeLock is a lock that enables locking of ranges between a +// min and max value. We use this so that flushes from the WAL +// can occur concurrently along with compactions. +type writeLock struct { + mu sync.Mutex + + rangesLock sync.Mutex + ranges []*rangeLock +} + +// LockRange will ensure an exclusive lock between the min and +// max values inclusive. Any subsequent calls that have an +// an overlapping range will have to wait until the previous +// lock is released. A corresponding call to UnlockRange should +// be deferred. +func (w *writeLock) LockRange(min, max int64) { + w.mu.Lock() + defer w.mu.Unlock() + + r := &rangeLock{min: min, max: max} + ranges := w.currentlyLockedRanges() + + // ensure there are no currently locked ranges that overlap + for _, rr := range ranges { + if rr.overlaps(r) { + // wait until it gets unlocked + rr.mu.Lock() + // release the lock so the object can get GC'd + rr.mu.Unlock() + } + } + + // and lock the range + r.mu.Lock() + + // now that we know the range is free, add it to the locks + w.rangesLock.Lock() + w.ranges = append(w.ranges, r) + w.rangesLock.Unlock() +} + +// UnlockRange will release a previously locked range. +func (w *writeLock) UnlockRange(min, max int64) { + w.rangesLock.Lock() + defer w.rangesLock.Unlock() + + // take the range out of the slice and unlock it + a := make([]*rangeLock, 0) + for _, r := range w.ranges { + if r.min == min && r.max == max { + r.mu.Unlock() + continue + } + a = append(a, r) + } + w.ranges = a +} + +func (w *writeLock) currentlyLockedRanges() []*rangeLock { + w.rangesLock.Lock() + defer w.rangesLock.Unlock() + a := make([]*rangeLock, len(w.ranges)) + copy(a, w.ranges) + return a +} + +type rangeLock struct { + mu sync.Mutex + min int64 + max int64 +} + +func (r *rangeLock) overlaps(l *rangeLock) bool { + if l.min >= r.min && l.min <= r.max { + return true + } else if l.max >= r.min && l.max <= r.max { + return true + } + return false +} From 667b3e6c0825822fed2762e1f8b865511c3d748f Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Sun, 27 Sep 2015 16:44:57 -0400 Subject: [PATCH 085/139] Handle hash collisions on keys --- tsdb/engine/pd1/pd1.go | 89 ++++++++++++++++++++++++++++++------- tsdb/engine/pd1/pd1_test.go | 87 ++++++++++++++++++++++++++++++++++++ 2 files changed, 161 insertions(+), 15 deletions(-) diff --git a/tsdb/engine/pd1/pd1.go b/tsdb/engine/pd1/pd1.go index 01087194573..3496268bf9d 100644 --- a/tsdb/engine/pd1/pd1.go +++ b/tsdb/engine/pd1/pd1.go @@ -95,6 +95,9 @@ type Engine struct { files dataFiles currentFileID int + collisionsLock sync.RWMutex + collisions map[string]uint64 + // queryLock keeps data files from being deleted or the store from // being closed while queries are running queryLock sync.RWMutex @@ -137,6 +140,7 @@ func (e *Engine) Open() error { // TODO: clean up previous fields write // TODO: clean up previous names write // TODO: clean up any data files that didn't get cleaned up + // TODO: clean up previous collisions write files, err := filepath.Glob(filepath.Join(e.path, fmt.Sprintf("*.%s", Format))) if err != nil { @@ -166,6 +170,10 @@ func (e *Engine) Open() error { return err } + if err := e.readCollisions(); err != nil { + return err + } + return nil } @@ -190,6 +198,7 @@ func (e *Engine) Close() error { } e.files = nil e.currentFileID = 0 + e.collisions = nil return nil } @@ -564,7 +573,8 @@ func (e *Engine) convertKeysAndWriteMetadata(pointsByKey map[string]Values, meas // these are values that are newer than anything stored in the shard valuesByID = make(map[uint64]Values) - idToKey := make(map[uint64]string) // we only use this map if new ids are being created + idToKey := make(map[uint64]string) // we only use this map if new ids are being created + collisions := make(map[string]uint64) // we only use this if a collision is encountered newKeys := false // track the min and max time of values being inserted so we can lock that time range minTime = int64(math.MaxInt64) @@ -574,6 +584,7 @@ func (e *Engine) convertKeysAndWriteMetadata(pointsByKey map[string]Values, meas var ok bool if id, ok = ids[k]; !ok { // populate the map if we haven't already + if len(idToKey) == 0 { for n, id := range ids { idToKey[id] = n @@ -581,21 +592,26 @@ func (e *Engine) convertKeysAndWriteMetadata(pointsByKey map[string]Values, meas } // now see if the hash id collides with a different key - hashID := hashSeriesField(k) + hashID := e.HashSeriesField(k) existingKey, idInMap := idToKey[hashID] - if idInMap { - // we only care if the keys are different. if so, it's a hash collision we have to keep track of - if k != existingKey { - // we have a collision, give this new key a different id and move on - // TODO: handle collisions - panic("name collision, not implemented yet!") + // we only care if the keys are different. if so, it's a hash collision we have to keep track of + if idInMap && k != existingKey { + // we have a collision, find this new key the next available id + hashID = 0 + for { + hashID++ + if _, ok := idToKey[hashID]; !ok { + // next ID is available, use it + break + } } - } else { - newKeys = true - ids[k] = hashID - idToKey[id] = k - id = hashID + collisions[k] = hashID } + + newKeys = true + ids[k] = hashID + idToKey[hashID] = k + id = hashID } if minTime > values.MinTime() { @@ -618,9 +634,44 @@ func (e *Engine) convertKeysAndWriteMetadata(pointsByKey map[string]Values, meas } } + if len(collisions) > 0 { + e.saveNewCollisions(collisions) + } + return } +func (e *Engine) saveNewCollisions(collisions map[string]uint64) error { + e.collisionsLock.Lock() + defer e.collisionsLock.Unlock() + + for k, v := range collisions { + e.collisions[k] = v + } + + data, err := json.Marshal(e.collisions) + + if err != nil { + return err + } + + return e.replaceCompressedFile(CollisionsFileExtension, data) +} + +func (e *Engine) readCollisions() error { + e.collisions = make(map[string]uint64) + data, err := e.readCompressedFile(CollisionsFileExtension) + if err != nil { + return err + } + + if len(data) == 0 { + return nil + } + + return json.Unmarshal(data, &e.collisions) +} + // filterDataBetweenTimes will create a new map with data between // the minTime (inclusive) and maxTime (exclusive) while removing that // data from the passed in map. It is assume that the Values arrays @@ -925,8 +976,16 @@ func (e *Engine) Cursor(series string, fields []string, dec *tsdb.FieldCodec, as panic("pd1 engine only supports one field with name of value") } - // TODO: ensure we map the collisions - id := hashSeriesField(seriesFieldKey(series, field.Name)) + // get the ID for the key and be sure to check if it had hash collision before + key := seriesFieldKey(series, field.Name) + e.collisionsLock.RLock() + id, ok := e.collisions[key] + e.collisionsLock.RUnlock() + + if !ok { + id = e.HashSeriesField(key) + } + indexCursor := newCursor(id, field.Type, e.copyFilesCollection(), ascending) wc := e.WAL.Cursor(series, fields, dec, ascending) return tsdb.MultiCursor(wc, indexCursor) diff --git a/tsdb/engine/pd1/pd1_test.go b/tsdb/engine/pd1/pd1_test.go index db867e93039..35385ddfb66 100644 --- a/tsdb/engine/pd1/pd1_test.go +++ b/tsdb/engine/pd1/pd1_test.go @@ -372,6 +372,93 @@ func TestEngine_Compaction(t *testing.T) { verify("cpu,host=B", []models.Point{p2, p4, p6, p8}, 0) } +// Ensure that if two keys have the same fnv64-a id, we handle it +func TestEngine_KeyCollisionsAreHandled(t *testing.T) { + e := OpenDefaultEngine() + defer e.Cleanup() + + fields := []string{"value"} + codec := tsdb.NewFieldCodec(map[string]*tsdb.Field{ + "value": { + ID: uint8(1), + Name: "value", + Type: influxql.Float, + }, + }) + + // make sure two of these keys collide + e.HashSeriesField = func(key string) uint64 { + return 1 + } + p1 := parsePoint("cpu,host=A value=1.1 1000000000") + p2 := parsePoint("cpu,host=B value=1.2 1000000000") + p3 := parsePoint("cpu,host=C value=1.3 1000000000") + + if err := e.WritePoints([]models.Point{p1, p2, p3}, nil, nil); err != nil { + t.Fatalf("failed to write points: %s", err.Error()) + } + + verify := func(series string, points []models.Point, seek int64) { + c := e.Cursor(series, fields, codec, true) + + k, v := c.SeekTo(seek) + p := points[0] + val := p.Fields()["value"] + if p.UnixNano() != k || val != v { + t.Fatalf("expected to seek to first point\n\texp: %d %f\n\tgot: %d %f", p.UnixNano(), val, k, v) + } + points = points[1:] + + for _, p := range points { + k, v := c.Next() + val := p.Fields()["value"] + if p.UnixNano() != k || val != v { + t.Fatalf("expected to seek to first point\n\texp: %d %f\n\tgot: %d %f", p.UnixNano(), val, k, v.(float64)) + } + } + } + + verify("cpu,host=A", []models.Point{p1}, 0) + verify("cpu,host=B", []models.Point{p2}, 0) + verify("cpu,host=C", []models.Point{p3}, 0) + + p4 := parsePoint("cpu,host=A value=2.1 2000000000") + p5 := parsePoint("cpu,host=B value=2.2 2000000000") + p6 := parsePoint("cpu,host=C value=2.3 2000000000") + + if err := e.WritePoints([]models.Point{p4, p5, p6}, nil, nil); err != nil { + t.Fatalf("failed to write points: %s", err.Error()) + } + + verify("cpu,host=A", []models.Point{p1, p4}, 0) + verify("cpu,host=B", []models.Point{p2, p5}, 0) + verify("cpu,host=C", []models.Point{p3, p6}, 0) + + // verify collisions are handled after closing and reopening + if err := e.Close(); err != nil { + t.Fatalf("error closing: %s", err.Error()) + } + if err := e.Open(); err != nil { + t.Fatalf("error opening: %s", err.Error()) + } + + verify("cpu,host=A", []models.Point{p1, p4}, 0) + verify("cpu,host=B", []models.Point{p2, p5}, 0) + verify("cpu,host=C", []models.Point{p3, p6}, 0) + + p7 := parsePoint("cpu,host=A value=3.1 3000000000") + p8 := parsePoint("cpu,host=B value=3.2 3000000000") + p9 := parsePoint("cpu,host=C value=3.3 3000000000") + + if err := e.WritePoints([]models.Point{p7, p8, p9}, nil, nil); err != nil { + t.Fatalf("failed to write points: %s", err.Error()) + } + + verify("cpu,host=A", []models.Point{p1, p4, p7}, 0) + verify("cpu,host=B", []models.Point{p2, p5, p8}, 0) + verify("cpu,host=C", []models.Point{p3, p6, p9}, 0) +} + func TestEngine_WriteIndexBenchmarkNames(t *testing.T) { t.Skip("whatevs") From 6c94e738a002711bbcbb53acf877524d7c666595 Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Sun, 27 Sep 2015 19:37:03 -0400 Subject: [PATCH 086/139] Add support for multiple fields --- tsdb/engine/pd1/cursor.go | 148 ++++++++++++++++++++++++++++++++++ tsdb/engine/pd1/encoding.go | 5 +- tsdb/engine/pd1/pd1.go | 45 +++++++---- tsdb/engine/pd1/pd1_test.go | 156 ++++++++++++++++++++++++++++++++++++ tsdb/engine/pd1/wal.go | 11 +-- 5 files changed, 343 insertions(+), 22 deletions(-) create mode 100644 tsdb/engine/pd1/cursor.go diff --git a/tsdb/engine/pd1/cursor.go b/tsdb/engine/pd1/cursor.go new file mode 100644 index 00000000000..d31252aa32a --- /dev/null +++ b/tsdb/engine/pd1/cursor.go @@ -0,0 +1,148 @@ +package pd1 + +import ( + "math" + + "github.com/influxdb/influxdb/tsdb" +) + +type combinedEngineCursor struct { + walCursor tsdb.Cursor + engineCursor tsdb.Cursor + walKeyBuf int64 + walValueBuf interface{} + engineKeyBuf int64 + engineValueBuf interface{} + ascending bool +} + +func NewCombinedEngineCursor(wc, ec tsdb.Cursor, ascending bool) tsdb.Cursor { + return &combinedEngineCursor{ + walCursor: wc, + engineCursor: ec, + ascending: ascending, + } +} + +func (c *combinedEngineCursor) SeekTo(seek int64) (key int64, value interface{}) { + c.walKeyBuf, c.walValueBuf = c.walCursor.SeekTo(seek) + c.engineKeyBuf, c.engineValueBuf = c.engineCursor.SeekTo(seek) + return c.read() +} + +func (c *combinedEngineCursor) Next() (int64, interface{}) { + return c.read() +} + +func (c *combinedEngineCursor) Ascending() bool { + return c.ascending +} + +func (c *combinedEngineCursor) read() (key int64, value interface{}) { + key = tsdb.EOF + + // handle the case where they have the same point + if c.walKeyBuf != tsdb.EOF && c.walKeyBuf == c.engineKeyBuf { + // keep the wal value since it will overwrite the engine value + key = c.walKeyBuf + value = c.walValueBuf + c.walKeyBuf, c.walValueBuf = c.walCursor.Next() + // drop the engine value + _, _ = c.engineCursor.Next() + return + } + + // ascending order + if c.ascending { + if c.engineKeyBuf == tsdb.EOF || (c.walKeyBuf != tsdb.EOF && c.walKeyBuf < c.engineKeyBuf) { + key = c.walKeyBuf + value = c.walValueBuf + c.walKeyBuf, c.walValueBuf = c.walCursor.Next() + } else { + key = c.engineKeyBuf + value = c.engineValueBuf + c.engineKeyBuf, c.engineValueBuf = c.engineCursor.Next() + } + + return + } + + // descending order + if c.engineKeyBuf == tsdb.EOF || (c.walKeyBuf != tsdb.EOF && c.walKeyBuf > c.engineKeyBuf) { + key = c.walKeyBuf + value = c.walValueBuf + c.walKeyBuf, c.walValueBuf = c.walCursor.Next() + return + } + key = c.engineKeyBuf + value = c.engineValueBuf + c.engineKeyBuf, c.engineValueBuf = c.engineCursor.Next() + + return +} + +type multiFieldCursor struct { + fields []string + cursors []tsdb.Cursor + ascending bool + keyBuffer []int64 + valueBuffer []interface{} +} + +func NewMultiFieldCursor(fields []string, cursors []tsdb.Cursor, ascending bool) tsdb.Cursor { + return &multiFieldCursor{ + fields: fields, + cursors: cursors, + ascending: ascending, + keyBuffer: make([]int64, len(cursors)), + valueBuffer: make([]interface{}, len(cursors)), + } +} + +func (m *multiFieldCursor) SeekTo(seek int64) (key int64, value interface{}) { + for i, c := range m.cursors { + m.keyBuffer[i], m.valueBuffer[i] = c.SeekTo(seek) + } + return m.read() +} + +func (m *multiFieldCursor) Next() (int64, interface{}) { + return m.read() +} + +func (m *multiFieldCursor) Ascending() bool { + return m.ascending +} + +func (m *multiFieldCursor) read() (int64, interface{}) { + t := int64(math.MaxInt64) + if !m.ascending { + t = int64(math.MinInt64) + } + + // find the time we need to combine all fields + for _, k := range m.keyBuffer { + if k == tsdb.EOF { + continue + } + if m.ascending && t > k { + t = k + } else if !m.ascending && t < k { + t = k + } + } + + // get the value and advance each of the cursors that have the matching time + if t == math.MinInt64 || t == math.MaxInt64 { + return tsdb.EOF, nil + } + + mm := make(map[string]interface{}) + for i, k := range m.keyBuffer { + if k == t { + mm[m.fields[i]] = m.valueBuffer[i] + m.keyBuffer[i], m.valueBuffer[i] = m.cursors[i].Next() + } + } + return t, mm +} diff --git a/tsdb/engine/pd1/encoding.go b/tsdb/engine/pd1/encoding.go index 9a6d9fea7a6..b21d394af5d 100644 --- a/tsdb/engine/pd1/encoding.go +++ b/tsdb/engine/pd1/encoding.go @@ -78,10 +78,9 @@ func (v Values) DecodeSameTypeBlock(block []byte) Values { // DecodeBlock takes a byte array and will decode into values of the appropriate type // based on the block -func DecodeBlock(block []byte) Values { +func DecodeBlock(block []byte) (Values, error) { // TODO: add support for other block types - a, _ := DecodeFloatBlock(block) - return a + return DecodeFloatBlock(block) } // Deduplicate returns a new Values slice with any values diff --git a/tsdb/engine/pd1/pd1.go b/tsdb/engine/pd1/pd1.go index 3496268bf9d..a2d8687db9d 100644 --- a/tsdb/engine/pd1/pd1.go +++ b/tsdb/engine/pd1/pd1.go @@ -17,7 +17,6 @@ import ( "time" "github.com/golang/snappy" - "github.com/influxdb/influxdb/influxql" "github.com/influxdb/influxdb/models" "github.com/influxdb/influxdb/tsdb" ) @@ -404,7 +403,8 @@ func (e *Engine) Compact() error { } currentPosition += uint32(newPos - pos) } else { - previousValues = DecodeBlock(block) + // TODO: handle decode error + previousValues, _ = DecodeBlock(block) } // write the previous values and clear if we've hit the limit @@ -971,13 +971,35 @@ func (e *Engine) Begin(writable bool) (tsdb.Tx, error) { // TODO: handle multiple fields and descending func (e *Engine) Cursor(series string, fields []string, dec *tsdb.FieldCodec, ascending bool) tsdb.Cursor { - field := dec.FieldByName("value") - if field == nil || len(fields) > 1 { - panic("pd1 engine only supports one field with name of value") + files := e.copyFilesCollection() + + // don't add the overhead of the multifield cursor if we only have one field + if len(fields) == 1 { + id := e.keyAndFieldToID(series, fields[0]) + indexCursor := newCursor(id, files, ascending) + wc := e.WAL.Cursor(series, fields, dec, ascending) + return NewCombinedEngineCursor(wc, indexCursor, ascending) + } + + // multiple fields. use just the MultiFieldCursor, which also handles time collisions + // so we don't need to use the combined cursor + cursors := make([]tsdb.Cursor, 0) + cursorFields := make([]string, 0) + for _, field := range fields { + id := e.keyAndFieldToID(series, field) + indexCursor := newCursor(id, files, ascending) + wc := e.WAL.Cursor(series, []string{field}, dec, ascending) + // double up the fields since there's one for the wal and one for the index + cursorFields = append(cursorFields, field, field) + cursors = append(cursors, indexCursor, wc) } + return NewMultiFieldCursor(cursorFields, cursors, ascending) +} + +func (e *Engine) keyAndFieldToID(series, field string) uint64 { // get the ID for the key and be sure to check if it had hash collision before - key := seriesFieldKey(series, field.Name) + key := seriesFieldKey(series, field) e.collisionsLock.RLock() id, ok := e.collisions[key] e.collisionsLock.RUnlock() @@ -985,10 +1007,7 @@ func (e *Engine) Cursor(series string, fields []string, dec *tsdb.FieldCodec, as if !ok { id = e.HashSeriesField(key) } - - indexCursor := newCursor(id, field.Type, e.copyFilesCollection(), ascending) - wc := e.WAL.Cursor(series, fields, dec, ascending) - return tsdb.MultiCursor(wc, indexCursor) + return id } func (e *Engine) copyFilesCollection() []*dataFile { @@ -1334,7 +1353,6 @@ func (a dataFiles) Less(i, j int) bool { return a[i].MinTime() < a[j].MinTime() type cursor struct { id uint64 - dataType influxql.DataType f *dataFile filesPos int // the index in the files slice we're looking at pos uint32 @@ -1346,10 +1364,9 @@ type cursor struct { files []*dataFile } -func newCursor(id uint64, dataType influxql.DataType, files []*dataFile, ascending bool) *cursor { +func newCursor(id uint64, files []*dataFile, ascending bool) *cursor { return &cursor{ id: id, - dataType: dataType, ascending: ascending, files: files, } @@ -1470,7 +1487,7 @@ func (c *cursor) Next() (int64, interface{}) { func (c *cursor) decodeBlockAndGetValues(position uint32) (int64, interface{}) { length := btou32(c.f.mmap[position+8 : position+12]) block := c.f.mmap[position+12 : position+12+length] - c.vals, _ = DecodeFloatBlock(block) + c.vals, _ = DecodeBlock(block) c.pos = position + 12 + length v := c.vals[0] diff --git a/tsdb/engine/pd1/pd1_test.go b/tsdb/engine/pd1/pd1_test.go index 35385ddfb66..52cb9717f37 100644 --- a/tsdb/engine/pd1/pd1_test.go +++ b/tsdb/engine/pd1/pd1_test.go @@ -6,6 +6,7 @@ import ( "io/ioutil" "math" "os" + "reflect" "testing" "time" @@ -459,6 +460,161 @@ func TestEngine_KeyCollisionsAreHandled(t *testing.T) { verify("cpu,host=C", []models.Point{p3, p6, p9}, 0) } +func TestEngine_SupportMultipleFields(t *testing.T) { + e := OpenDefaultEngine() + defer e.Cleanup() + + fields := []string{"value", "foo"} + + p1 := parsePoint("cpu,host=A value=1.1 1000000000") + p2 := parsePoint("cpu,host=A value=1.2,foo=2.2 2000000000") + + if err := e.WritePoints([]models.Point{p1, p2}, nil, nil); err != nil { + t.Fatalf("failed to write points: %s", err.Error()) + } + c := e.Cursor("cpu,host=A", fields, nil, true) + k, v := c.SeekTo(0) + if k != p1.UnixNano() { + t.Fatalf("time wrong:\n\texp: %d\n\tgot: %d", p1.UnixNano(), k) + } + if !reflect.DeepEqual(v, map[string]interface{}{"value": 1.1}) { + t.Fatalf("value wrong: %v", v) + } + k, v = c.Next() + if k != p2.UnixNano() { + t.Fatalf("time wrong:\n\texp: %d\n\tgot: %d", p2.UnixNano(), k) + } + if !reflect.DeepEqual(v, map[string]interface{}{"value": 1.2, "foo": 2.2}) { + t.Fatalf("value wrong: %v", v) + } + k, _ = c.Next() + if k != tsdb.EOF { + t.Fatal("expected EOF") + } + + // verify we can update a field and it's still all good + p11 := parsePoint("cpu,host=A foo=2.1 1000000000") + if err := e.WritePoints([]models.Point{p11}, nil, nil); err != nil { + t.Fatalf("failed to write points: %s", err.Error()) + } + + c = e.Cursor("cpu,host=A", fields, nil, true) + k, v = c.SeekTo(0) + if k != p1.UnixNano() { + t.Fatalf("time wrong:\n\texp: %d\n\tgot: %d", p1.UnixNano(), k) + } + if !reflect.DeepEqual(v, map[string]interface{}{"value": 1.1, "foo": 2.1}) { + t.Fatalf("value wrong: %v", v) + } + k, v = c.Next() + if k != p2.UnixNano() { + t.Fatalf("time wrong:\n\texp: %d\n\tgot: %d", p2.UnixNano(), k) + } + if !reflect.DeepEqual(v, map[string]interface{}{"value": 1.2, "foo": 2.2}) { + t.Fatalf("value wrong: %v", v) + } + k, _ = c.Next() + if k != tsdb.EOF { + t.Fatal("expected EOF") + } + + // verify it's all good with the wal in the picture + e.WAL.SkipCache = false + + p3 := parsePoint("cpu,host=A value=1.3 3000000000") + p4 := parsePoint("cpu,host=A value=1.4,foo=2.4 4000000000") + if err := e.WritePoints([]models.Point{p3, p4}, nil, nil); err != nil { + t.Fatalf("failed to write points: %s", err.Error()) + } + + c = e.Cursor("cpu,host=A", fields, nil, true) + k, v = c.SeekTo(0) + if k != p1.UnixNano() { + t.Fatalf("time wrong:\n\texp: %d\n\tgot: %d", p1.UnixNano(), k) + } + if !reflect.DeepEqual(v, map[string]interface{}{"value": 1.1, "foo": 2.1}) { + t.Fatalf("value wrong: %v", v) + } + k, v = c.Next() + if k != p2.UnixNano() { + t.Fatalf("time wrong:\n\texp: %d\n\tgot: %d", p2.UnixNano(), k) + } + if !reflect.DeepEqual(v, map[string]interface{}{"value": 1.2, "foo": 2.2}) { + t.Fatalf("value wrong: %v", v) + } + k, v = c.Next() + if k != p3.UnixNano() { + t.Fatalf("time wrong:\n\texp: %d\n\tgot: %d", p3.UnixNano(), k) + } + if !reflect.DeepEqual(v, map[string]interface{}{"value": 1.3}) { + t.Fatalf("value wrong: %v", v) + } + k, v = c.Next() + if k != p4.UnixNano() { + t.Fatalf("time wrong:\n\texp: %d\n\tgot: %d", p2.UnixNano(), k) + } + if !reflect.DeepEqual(v, map[string]interface{}{"value": 1.4, "foo": 2.4}) { + t.Fatalf("value wrong: %v", v) + } + k, _ = c.Next() + if k != tsdb.EOF { + t.Fatal("expected EOF") + } + + p33 := parsePoint("cpu,host=A foo=2.3 3000000000") + if err := e.WritePoints([]models.Point{p33}, nil, nil); err != nil { + t.Fatalf("failed to write points: %s", err.Error()) + } + + c = e.Cursor("cpu,host=A", fields, nil, true) + k, v = c.SeekTo(0) + if k != p1.UnixNano() { + t.Fatalf("time wrong:\n\texp: %d\n\tgot: %d", p1.UnixNano(), k) + } + if !reflect.DeepEqual(v, map[string]interface{}{"value": 1.1, "foo": 2.1}) { + t.Fatalf("value wrong: %v", v) + } + k, v = c.Next() + if k != p2.UnixNano() { + t.Fatalf("time wrong:\n\texp: %d\n\tgot: %d", p2.UnixNano(), k) + } + if !reflect.DeepEqual(v, map[string]interface{}{"value": 1.2, "foo": 2.2}) { + t.Fatalf("value wrong: %v", v) + } + k, v = c.Next() + if k != p3.UnixNano() { + t.Fatalf("time wrong:\n\texp: %d\n\tgot: %d", p3.UnixNano(), k) + } + if !reflect.DeepEqual(v, map[string]interface{}{"value": 1.3, "foo": 2.3}) { + t.Fatalf("value wrong: %v", v) + } + k, v = c.Next() + if k != p4.UnixNano() { + t.Fatalf("time wrong:\n\texp: %d\n\tgot: %d", p2.UnixNano(), k) + } + if !reflect.DeepEqual(v, map[string]interface{}{"value": 1.4, "foo": 2.4}) { + t.Fatalf("value wrong: %v", v) + } + k, _ = c.Next() + if k != tsdb.EOF { + t.Fatal("expected EOF") + } + + // and ensure we can grab one of the fields + c = e.Cursor("cpu,host=A", []string{"value"}, nil, true) + k, v = c.SeekTo(4000000000) + if k != p4.UnixNano() { + t.Fatalf("time wrong:\n\texp: %d\n\tgot: %d", p4.UnixNano(), k) + } + if v != 1.4 { + t.Fatalf("value wrong: %v", v) + } + k, _ = c.Next() + if k != tsdb.EOF { + t.Fatal("expected EOF") + } +} + func TestEngine_WriteIndexBenchmarkNames(t *testing.T) { t.Skip("whatevs") diff --git a/tsdb/engine/pd1/wal.go b/tsdb/engine/pd1/wal.go index 1447cb5f06a..df1cbfe1b23 100644 --- a/tsdb/engine/pd1/wal.go +++ b/tsdb/engine/pd1/wal.go @@ -171,16 +171,17 @@ func (l *Log) Open() error { return nil } -// Cursor will return a cursor object to Seek and iterate with Next for the WAL cache for the given +// Cursor will return a cursor object to Seek and iterate with Next for the WAL cache for the given. +// This should only ever be called by the engine cursor method, which will always give it +// exactly one field. func (l *Log) Cursor(series string, fields []string, dec *tsdb.FieldCodec, ascending bool) tsdb.Cursor { l.cacheLock.RLock() defer l.cacheLock.RUnlock() - // TODO: make this work for other fields - if len(fields) != 1 || fields[0] != "value" { - panic("pd1 wal only supports 1 field with name value") + if len(fields) != 1 { + panic("wal cursor should only ever be called with 1 field") } - ck := seriesFieldKey(series, "value") + ck := seriesFieldKey(series, fields[0]) values := l.cache[ck] // if we're in the middle of a flush, combine the previous cache From d694454f47a197ecaf5eca6fd7014c55ce379e77 Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Mon, 28 Sep 2015 21:04:02 -0400 Subject: [PATCH 087/139] Fix wal flushing, compacting, and write lock --- tsdb/config.go | 6 +- tsdb/engine/pd1/pd1.go | 157 ++++++++++++++++++++++++++++++------ tsdb/engine/pd1/pd1_test.go | 2 + tsdb/engine/pd1/wal.go | 26 +++--- 4 files changed, 153 insertions(+), 38 deletions(-) diff --git a/tsdb/config.go b/tsdb/config.go index 8716a455375..2039ff813ba 100644 --- a/tsdb/config.go +++ b/tsdb/config.go @@ -45,9 +45,9 @@ const ( DefaultPartitionSizeThreshold = 50 * 1024 * 1024 // 50MB // Default WAL settings for the PD1 WAL - DefaultFlushMemorySizeThreshold = 10 * 1024 * 1024 // 10MB - DefaultMaxMemorySizeThreshold = 200 * 1024 * 1024 // 200MB - DefaultIndexCompactionAge = 10 * time.Minute + DefaultFlushMemorySizeThreshold = 5 * 1024 * 1024 // 5MB + DefaultMaxMemorySizeThreshold = 100 * 1024 * 1024 // 100MB + DefaultIndexCompactionAge = time.Minute DefaultIndexCompactionFileCount = 5 ) diff --git a/tsdb/engine/pd1/pd1.go b/tsdb/engine/pd1/pd1.go index a2d8687db9d..54cb6d47505 100644 --- a/tsdb/engine/pd1/pd1.go +++ b/tsdb/engine/pd1/pd1.go @@ -55,7 +55,7 @@ const ( // DefaultRotateBlockSize is the default size to rotate to a new compressed block DefaultRotateBlockSize = 512 * 1024 // 512KB - DefaultRotateFileSize = 10 * 1024 * 1024 // 10MB + DefaultRotateFileSize = 5 * 1024 * 1024 // 5MB DefaultMaxPointsPerBlock = 1000 @@ -90,9 +90,10 @@ type Engine struct { CompactionFileCount int // filesLock is only for modifying and accessing the files slice - filesLock sync.RWMutex - files dataFiles - currentFileID int + filesLock sync.RWMutex + files dataFiles + currentFileID int + compactionRunning bool collisionsLock sync.RWMutex collisions map[string]uint64 @@ -264,16 +265,21 @@ func (e *Engine) Write(pointsByKey map[string]Values, measurementFieldsToSave ma return nil } - e.writeLock.LockRange(startTime, endTime) - defer e.writeLock.UnlockRange(startTime, endTime) + files, lockStart, lockEnd := e.filesAndLock(startTime, endTime) + defer e.writeLock.UnlockRange(lockStart, lockEnd) - if len(e.files) == 0 { + if len(files) == 0 { return e.rewriteFile(nil, valuesByID) } maxTime := int64(math.MaxInt64) + + // do the file rewrites in parallel + var mu sync.Mutex + var writes sync.WaitGroup + var errors []error + // reverse through the data files and write in the data - files := e.copyFilesCollection() for i := len(files) - 1; i >= 0; i-- { f := files[i] // max times are exclusive, so add 1 to it @@ -281,31 +287,105 @@ func (e *Engine) Write(pointsByKey map[string]Values, measurementFieldsToSave ma fileMin := f.MinTime() // if the file is < rotate, write all data between fileMin and maxTime if f.size < e.RotateFileSize { - if err := e.rewriteFile(f, e.filterDataBetweenTimes(valuesByID, fileMin, maxTime)); err != nil { - return err - } + writes.Add(1) + go func(df *dataFile, vals map[uint64]Values) { + if err := e.rewriteFile(df, vals); err != nil { + mu.Lock() + errors = append(errors, err) + mu.Unlock() + } + writes.Done() + }(f, e.filterDataBetweenTimes(valuesByID, fileMin, maxTime)) continue } // if the file is > rotate: // write all data between fileMax and maxTime into new file // write all data between fileMin and fileMax into old file - if err := e.rewriteFile(nil, e.filterDataBetweenTimes(valuesByID, fileMax, maxTime)); err != nil { - return err - } - if err := e.rewriteFile(f, e.filterDataBetweenTimes(valuesByID, fileMin, fileMax)); err != nil { - return err - } + writes.Add(1) + go func(vals map[uint64]Values) { + if err := e.rewriteFile(nil, vals); err != nil { + mu.Lock() + errors = append(errors, err) + mu.Unlock() + } + writes.Done() + }(e.filterDataBetweenTimes(valuesByID, fileMax, maxTime)) + writes.Add(1) + go func(df *dataFile, vals map[uint64]Values) { + if err := e.rewriteFile(df, vals); err != nil { + mu.Lock() + errors = append(errors, err) + mu.Unlock() + } + writes.Done() + }(f, e.filterDataBetweenTimes(valuesByID, fileMin, fileMax)) maxTime = fileMin } // for any data leftover, write into a new file since it's all older // than any file we currently have - err = e.rewriteFile(nil, valuesByID) + writes.Add(1) + go func() { + if err := e.rewriteFile(nil, valuesByID); err != nil { + mu.Lock() + errors = append(errors, err) + mu.Unlock() + } + writes.Done() + }() + + writes.Wait() + + if len(errors) > 0 { + // TODO: log errors + return errors[0] + } if !e.SkipCompaction && e.shouldCompact() { go e.Compact() } - return err + return nil +} + +// filesAndLock returns the data files that match the given range and +// ensures that the write lock will hold for the entire range +func (e *Engine) filesAndLock(min, max int64) (a dataFiles, lockStart, lockEnd int64) { + for { + a = make([]*dataFile, 0) + files := e.copyFilesCollection() + + for _, f := range e.files { + fmin, fmax := f.MinTime(), f.MaxTime() + if min < fmax && fmin >= fmin { + a = append(a, f) + } else if max >= fmin && max < fmax { + a = append(a, f) + } + } + + if len(a) > 0 { + lockStart = a[0].MinTime() + lockEnd = a[len(a)-1].MaxTime() + if max > lockEnd { + lockEnd = max + } + } else { + lockStart = min + lockEnd = max + } + + e.writeLock.LockRange(lockStart, lockEnd) + + // it's possible for compaction to change the files collection while we + // were waiting for a write lock on the range. Make sure the files are still the + // same after we got the lock, otherwise try again. This shouldn't happen often. + filesAfterLock := e.copyFilesCollection() + if reflect.DeepEqual(files, filesAfterLock) { + return + } + + e.writeLock.UnlockRange(lockStart, lockEnd) + } } func (e *Engine) Compact() error { @@ -333,7 +413,24 @@ func (e *Engine) Compact() error { // we've got the write lock and the files are all there break } - defer e.writeLock.UnlockRange(minTime, maxTime) + + // mark the compaction as running + e.filesLock.Lock() + e.compactionRunning = true + e.filesLock.Unlock() + defer func() { + //release the lock + e.writeLock.UnlockRange(minTime, maxTime) + + // see if we should run aonther compaction + if e.shouldCompact() { + go e.Compact() + } else { + e.filesLock.Lock() + e.compactionRunning = false + e.filesLock.Unlock() + } + }() positions := make([]uint32, len(files)) ids := make([]uint64, len(files)) @@ -472,6 +569,7 @@ func (e *Engine) Compact() error { for _, f := range files { if err := f.Delete(); err != nil { // TODO: log this error + fmt.Println("ERROR DELETING:", f.f.Name()) } } e.deletesPending.Done() @@ -527,6 +625,12 @@ func (e *Engine) writeIndexAndGetDataFile(f *os.File, minTime, maxTime int64, id } func (e *Engine) shouldCompact() bool { + e.filesLock.RLock() + running := e.compactionRunning + e.filesLock.RUnlock() + if running { + return false + } return len(e.filesToCompact()) >= e.CompactionFileCount } @@ -538,6 +642,10 @@ func (e *Engine) filesToCompact() dataFiles { for _, df := range e.files { if time.Since(df.modTime) > e.CompactionAge && df.size < MaxDataFileSize { a = append(a, df) + } else if len(a) > 0 { + // only compact contiguous ranges. If we hit the negative case and + // there are files to compact, stop here + break } } return a @@ -744,7 +852,8 @@ func (e *Engine) rewriteFile(oldDF *dataFile, valuesByID map[uint64]Values) erro minTime = v.MinTime() } if maxTime < v.MaxTime() { - maxTime = v.MaxTime() + // add 1 ns to the time since maxTime is exclusive + maxTime = v.MaxTime() + 1 } } @@ -896,7 +1005,7 @@ func (e *Engine) rewriteFile(oldDF *dataFile, valuesByID map[uint64]Values) erro e.deletesPending.Add(1) go func() { if err := oldDF.Delete(); err != nil { - // TODO: log this error + fmt.Println("ERROR DELETING FROM REWRITE:", oldDF.f.Name()) } e.deletesPending.Done() }() @@ -1312,7 +1421,7 @@ func (d *dataFile) indexPosition() uint32 { func (d *dataFile) StartingPositionForID(id uint64) uint32 { seriesCount := d.SeriesCount() - indexStart := d.size - uint32(seriesCount*12+20) + indexStart := d.indexPosition() min := uint32(0) max := uint32(seriesCount) @@ -1522,7 +1631,7 @@ func btou32(b []byte) uint32 { func hashSeriesField(key string) uint64 { h := fnv.New64a() h.Write([]byte(key)) - return h.Sum64() % 100 + return h.Sum64() } // seriesFieldKey combine a series key and field name for a unique string to be hashed to a numeric ID diff --git a/tsdb/engine/pd1/pd1_test.go b/tsdb/engine/pd1/pd1_test.go index 52cb9717f37..35ec6c720f7 100644 --- a/tsdb/engine/pd1/pd1_test.go +++ b/tsdb/engine/pd1/pd1_test.go @@ -363,6 +363,8 @@ func TestEngine_Compaction(t *testing.T) { } } + verify("cpu,host=A", []models.Point{p1, p3, p5, p7}, 0) + verify("cpu,host=B", []models.Point{p2, p4, p6, p8}, 0) if err := e.Close(); err != nil { t.Fatalf("error closing: %s", err.Error()) } diff --git a/tsdb/engine/pd1/wal.go b/tsdb/engine/pd1/wal.go index df1cbfe1b23..c4c6d23ebe8 100644 --- a/tsdb/engine/pd1/wal.go +++ b/tsdb/engine/pd1/wal.go @@ -270,12 +270,14 @@ func (l *Log) addToCache(points []models.Point, fields map[string]*tsdb.Measurem defer l.cacheLock.Unlock() // if we should check memory and we're over the threshold, mark a flush as running and kick one off in a goroutine - if checkMemory && l.memorySize > l.MaxMemorySizeThreshold { + if checkMemory && l.memorySize > l.FlushMemorySizeThreshold { if !l.flushRunning { l.flushRunning = true go l.flush(memoryFlush) } - return false + if l.memorySize > l.MaxMemorySizeThreshold { + return false + } } for _, p := range points { @@ -401,7 +403,7 @@ func (l *Log) writeToLog(writeType walEntryType, data []byte) error { l.writeLock.Lock() defer l.writeLock.Unlock() - if l.currentSegmentFile == nil { + if l.currentSegmentFile == nil || l.currentSegmentSize > DefaultSegmentSize { if err := l.newSegmentFile(); err != nil { // fail hard since we can't write data panic(fmt.Sprintf("error opening new segment file for wal: %s", err.Error())) @@ -421,6 +423,8 @@ func (l *Log) writeToLog(writeType walEntryType, data []byte) error { panic(fmt.Sprintf("error writing data to wal: %s", err.Error())) } + l.currentSegmentSize += 5 + len(data) + return l.currentSegmentFile.Sync() } @@ -489,6 +493,7 @@ func (l *Log) flush(flush flushType) error { // only flush if there isn't one already running. Memory flushes are only triggered // by writes, which will mark the flush as running, so we can ignore it. l.cacheLock.Lock() + if l.flushRunning && flush != memoryFlush { l.cacheLock.Unlock() return nil @@ -523,19 +528,18 @@ func (l *Log) flush(flush flushType) error { l.writeLock.Unlock() // copy the cache items to new maps so we can empty them out - l.flushCache = l.cache - l.cache = make(map[string]Values) + l.flushCache = make(map[string]Values) for k, _ := range l.cacheDirtySort { l.flushCache[k] = l.flushCache[k].Deduplicate() } l.cacheDirtySort = make(map[string]bool) - valuesByKey := make(map[string]Values) valueCount := 0 - for key, v := range l.flushCache { - valuesByKey[key] = v + for key, v := range l.cache { + l.flushCache[key] = v valueCount += len(v) } + l.cache = make(map[string]Values) flushSize := l.memorySize @@ -553,7 +557,7 @@ func (l *Log) flush(flush flushType) error { l.cacheLock.Unlock() // exit if there's nothing to flush to the index - if len(valuesByKey) == 0 && len(mfc) == 0 && len(scc) == 0 { + if len(l.flushCache) == 0 && len(mfc) == 0 && len(scc) == 0 { return nil } @@ -564,11 +568,11 @@ func (l *Log) flush(flush flushType) error { } else if flush == startupFlush { ftype = "startup" } - l.logger.Printf("%s flush of %d keys with %d values of %d bytes\n", ftype, len(valuesByKey), valueCount, flushSize) + l.logger.Printf("%s flush of %d keys with %d values of %d bytes\n", ftype, len(l.flushCache), valueCount, flushSize) } startTime := time.Now() - if err := l.Index.Write(valuesByKey, mfc, scc); err != nil { + if err := l.Index.Write(l.flushCache, mfc, scc); err != nil { return err } if l.LoggingEnabled { From 1c8eac15233053331886da789ec1f3d3aac34bd6 Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Mon, 28 Sep 2015 22:50:00 -0400 Subject: [PATCH 088/139] Add PerformMaintenance to store for flushes and compactions. Also fixed shard to work again with b1 and bz1 engines. --- tsdb/config.go | 40 +++++++++++------- tsdb/engine.go | 14 +++++++ tsdb/engine/b1/b1.go | 8 ++++ tsdb/engine/bz1/bz1.go | 8 ++++ tsdb/engine/pd1/encoding_test.go | 3 -- tsdb/engine/pd1/pd1.go | 48 +++++++++++++++------ tsdb/engine/pd1/pd1_test.go | 25 ++++------- tsdb/engine/pd1/wal.go | 72 +++++--------------------------- tsdb/shard.go | 67 +++++++++++++++-------------- tsdb/store.go | 42 ++++++++++++++++++- 10 files changed, 188 insertions(+), 139 deletions(-) diff --git a/tsdb/config.go b/tsdb/config.go index 2039ff813ba..1e7e29a0d86 100644 --- a/tsdb/config.go +++ b/tsdb/config.go @@ -45,10 +45,12 @@ const ( DefaultPartitionSizeThreshold = 50 * 1024 * 1024 // 50MB // Default WAL settings for the PD1 WAL - DefaultFlushMemorySizeThreshold = 5 * 1024 * 1024 // 5MB - DefaultMaxMemorySizeThreshold = 100 * 1024 * 1024 // 100MB - DefaultIndexCompactionAge = time.Minute - DefaultIndexCompactionFileCount = 5 + DefaultFlushMemorySizeThreshold = 5 * 1024 * 1024 // 5MB + DefaultMaxMemorySizeThreshold = 100 * 1024 * 1024 // 100MB + DefaultIndexCompactionAge = time.Minute + DefaultIndexMinimumCompactionInterval = time.Minute + DefaultIndexCompactionFileCount = 5 + DefaultIndexCompactionFullAge = time.Minute ) type Config struct { @@ -79,10 +81,18 @@ type Config struct { // at which it is eligible to be compacted IndexCompactionAge time.Duration `toml:"index-compaction-age"` + // IndexMinimumCompactionInterval specifies the minimum amount of time that must + // pass after a compaction before another compaction is run + IndexMinimumCompactionInterval time.Duration `toml:"index-minimum-compaction-interval"` + // IndexCompactionFileCount specifies the minimum number of data files that // must be eligible for compaction before actually running one IndexCompactionFileCount int `toml:"index-compaction-file-count"` + // IndexCompactionFullAge specifies how long after the last write was received + // in the WAL that a full compaction should be performed. + IndexCompactionFullAge time.Duration `toml:"index-compaction-full-age"` + // Query logging QueryLogEnabled bool `toml:"query-log-enabled"` } @@ -94,16 +104,18 @@ func NewConfig() Config { WALFlushInterval: toml.Duration(DefaultWALFlushInterval), WALPartitionFlushDelay: toml.Duration(DefaultWALPartitionFlushDelay), - WALLoggingEnabled: true, - WALReadySeriesSize: DefaultReadySeriesSize, - WALCompactionThreshold: DefaultCompactionThreshold, - WALMaxSeriesSize: DefaultMaxSeriesSize, - WALFlushColdInterval: toml.Duration(DefaultFlushColdInterval), - WALPartitionSizeThreshold: DefaultPartitionSizeThreshold, - WALFlushMemorySizeThreshold: DefaultFlushMemorySizeThreshold, - WALMaxMemorySizeThreshold: DefaultMaxMemorySizeThreshold, - IndexCompactionAge: DefaultIndexCompactionAge, - IndexCompactionFileCount: DefaultIndexCompactionFileCount, + WALLoggingEnabled: true, + WALReadySeriesSize: DefaultReadySeriesSize, + WALCompactionThreshold: DefaultCompactionThreshold, + WALMaxSeriesSize: DefaultMaxSeriesSize, + WALFlushColdInterval: toml.Duration(DefaultFlushColdInterval), + WALPartitionSizeThreshold: DefaultPartitionSizeThreshold, + WALFlushMemorySizeThreshold: DefaultFlushMemorySizeThreshold, + WALMaxMemorySizeThreshold: DefaultMaxMemorySizeThreshold, + IndexCompactionAge: DefaultIndexCompactionAge, + IndexCompactionFileCount: DefaultIndexCompactionFileCount, + IndexCompactionFullAge: DefaultIndexCompactionFullAge, + IndexMinimumCompactionInterval: DefaultIndexMinimumCompactionInterval, QueryLogEnabled: true, } diff --git a/tsdb/engine.go b/tsdb/engine.go index d2862b54863..407801d842e 100644 --- a/tsdb/engine.go +++ b/tsdb/engine.go @@ -32,9 +32,23 @@ type Engine interface { DeleteMeasurement(name string, seriesKeys []string) error SeriesCount() (n int, err error) + // PerformMaintenance will get called periodically by the store + PerformMaintenance() + + // Format will return the format for the engine + Format() EngineFormat + io.WriterTo } +type EngineFormat int + +const ( + B1Format EngineFormat = iota + BZ1Format + PD1Format +) + // NewEngineFunc creates a new engine. type NewEngineFunc func(path string, walPath string, options EngineOptions) Engine diff --git a/tsdb/engine/b1/b1.go b/tsdb/engine/b1/b1.go index 356b327fe95..a3f63602cdc 100644 --- a/tsdb/engine/b1/b1.go +++ b/tsdb/engine/b1/b1.go @@ -91,6 +91,14 @@ func NewEngine(path string, walPath string, opt tsdb.EngineOptions) tsdb.Engine // Path returns the path the engine was initialized with. func (e *Engine) Path() string { return e.path } +// PerformMaintenance is for periodic maintenance of the store. A no-op for b1 +func (e *Engine) PerformMaintenance() {} + +// Format returns the format type of this engine +func (e *Engine) Format() tsdb.EngineFormat { + return tsdb.B1Format +} + // Open opens and initializes the engine. func (e *Engine) Open() error { if err := func() error { diff --git a/tsdb/engine/bz1/bz1.go b/tsdb/engine/bz1/bz1.go index aa38600b806..881b82dc431 100644 --- a/tsdb/engine/bz1/bz1.go +++ b/tsdb/engine/bz1/bz1.go @@ -114,6 +114,14 @@ func NewEngine(path string, walPath string, opt tsdb.EngineOptions) tsdb.Engine // Path returns the path the engine was opened with. func (e *Engine) Path() string { return e.path } +// PerformMaintenance is for periodic maintenance of the store. A no-op for bz1 +func (e *Engine) PerformMaintenance() {} + +// Format returns the format type of this engine +func (e *Engine) Format() tsdb.EngineFormat { + return tsdb.BZ1Format +} + // Open opens and initializes the engine. func (e *Engine) Open() error { if err := func() error { diff --git a/tsdb/engine/pd1/encoding_test.go b/tsdb/engine/pd1/encoding_test.go index 82968912b0a..02598a764da 100644 --- a/tsdb/engine/pd1/encoding_test.go +++ b/tsdb/engine/pd1/encoding_test.go @@ -1,8 +1,6 @@ package pd1_test import ( - // "math/rand" - "fmt" "reflect" "testing" "time" @@ -19,7 +17,6 @@ func TestEncoding_FloatBlock(t *testing.T) { } b := values.Encode(nil) - fmt.Println("**** ", len(b)) decodedValues := values.DecodeSameTypeBlock(b) diff --git a/tsdb/engine/pd1/pd1.go b/tsdb/engine/pd1/pd1.go index 54cb6d47505..1f56c066539 100644 --- a/tsdb/engine/pd1/pd1.go +++ b/tsdb/engine/pd1/pd1.go @@ -84,10 +84,11 @@ type Engine struct { WAL *Log - RotateFileSize uint32 - SkipCompaction bool - CompactionAge time.Duration - CompactionFileCount int + RotateFileSize uint32 + SkipCompaction bool + CompactionAge time.Duration + CompactionFileCount int + IndexCompactionFullAge time.Duration // filesLock is only for modifying and accessing the files slice filesLock sync.RWMutex @@ -116,11 +117,12 @@ func NewEngine(path string, walPath string, opt tsdb.EngineOptions) tsdb.Engine writeLock: &writeLock{}, // TODO: this is the function where we can inject a check against the in memory collisions - HashSeriesField: hashSeriesField, - WAL: w, - RotateFileSize: DefaultRotateFileSize, - CompactionAge: opt.Config.IndexCompactionAge, - CompactionFileCount: opt.Config.IndexCompactionFileCount, + HashSeriesField: hashSeriesField, + WAL: w, + RotateFileSize: DefaultRotateFileSize, + CompactionAge: opt.Config.IndexCompactionAge, + CompactionFileCount: opt.Config.IndexCompactionFileCount, + IndexCompactionFullAge: opt.Config.IndexCompactionFullAge, } e.WAL.Index = e @@ -130,6 +132,28 @@ func NewEngine(path string, walPath string, opt tsdb.EngineOptions) tsdb.Engine // Path returns the path the engine was opened with. func (e *Engine) Path() string { return e.path } +// PerformMaintenance is for periodic maintenance of the store. A no-op for b1 +func (e *Engine) PerformMaintenance() { + if f := e.WAL.shouldFlush(); f != noFlush { + go func() { + fmt.Println("maintenance autoflush") + e.WAL.flush(f) + if time.Since(e.WAL.lastWriteTime) > e.IndexCompactionFullAge { + fmt.Println("mainenance compact autoflush") + e.Compact(true) + } + }() + } else if time.Since(e.WAL.lastWriteTime) > e.IndexCompactionFullAge { + fmt.Println("compact full, suckas") + go e.Compact(true) + } +} + +// Format returns the format type of this engine +func (e *Engine) Format() tsdb.EngineFormat { + return tsdb.PD1Format +} + // Open opens and initializes the engine. func (e *Engine) Open() error { if err := os.MkdirAll(e.path, 0777); err != nil { @@ -341,7 +365,7 @@ func (e *Engine) Write(pointsByKey map[string]Values, measurementFieldsToSave ma } if !e.SkipCompaction && e.shouldCompact() { - go e.Compact() + go e.Compact(false) } return nil @@ -388,7 +412,7 @@ func (e *Engine) filesAndLock(min, max int64) (a dataFiles, lockStart, lockEnd i } } -func (e *Engine) Compact() error { +func (e *Engine) Compact(fullCompaction bool) error { // we're looping here to ensure that the files we've marked to compact are // still there after we've obtained the write lock var minTime, maxTime int64 @@ -424,7 +448,7 @@ func (e *Engine) Compact() error { // see if we should run aonther compaction if e.shouldCompact() { - go e.Compact() + go e.Compact(false) } else { e.filesLock.Lock() e.compactionRunning = false diff --git a/tsdb/engine/pd1/pd1_test.go b/tsdb/engine/pd1/pd1_test.go index 35ec6c720f7..491c7cd552b 100644 --- a/tsdb/engine/pd1/pd1_test.go +++ b/tsdb/engine/pd1/pd1_test.go @@ -137,18 +137,18 @@ func TestEngine_WriteIndexQueryAcrossDataFiles(t *testing.T) { e.RotateFileSize = 10 p1 := parsePoint("cpu,host=A value=1.1 1000000000") - p2 := parsePoint("cpu,host=B value=1.1 1000000000") - p3 := parsePoint("cpu,host=A value=2.4 4000000000") - p4 := parsePoint("cpu,host=B value=2.4 4000000000") + p2 := parsePoint("cpu,host=B value=1.2 1000000000") + p3 := parsePoint("cpu,host=A value=2.1 4000000000") + p4 := parsePoint("cpu,host=B value=2.2 4000000000") if err := e.WritePoints([]models.Point{p1, p2, p3, p4}, nil, nil); err != nil { t.Fatalf("failed to write points: %s", err.Error()) } - p5 := parsePoint("cpu,host=A value=1.5 5000000000") - p6 := parsePoint("cpu,host=B value=2.5 5000000000") - p7 := parsePoint("cpu,host=A value=1.3 3000000000") - p8 := parsePoint("cpu,host=B value=2.3 3000000000") + p5 := parsePoint("cpu,host=A value=3.1 5000000000") + p6 := parsePoint("cpu,host=B value=3.2 5000000000") + p7 := parsePoint("cpu,host=A value=4.1 3000000000") + p8 := parsePoint("cpu,host=B value=4.2 3000000000") if err := e.WritePoints([]models.Point{p5, p6, p7, p8}, nil, nil); err != nil { t.Fatalf("failed to write points: %s", err.Error()) @@ -159,16 +159,9 @@ func TestEngine_WriteIndexQueryAcrossDataFiles(t *testing.T) { } fields := []string{"value"} - codec := tsdb.NewFieldCodec(map[string]*tsdb.Field{ - "value": { - ID: uint8(1), - Name: "value", - Type: influxql.Float, - }, - }) verify := func(series string, points []models.Point, seek int64) { - c := e.Cursor(series, fields, codec, true) + c := e.Cursor(series, fields, nil, true) k, v := c.SeekTo(seek) p := points[0] @@ -335,7 +328,7 @@ func TestEngine_Compaction(t *testing.T) { e.CompactionAge = time.Duration(0) - if err := e.Compact(); err != nil { + if err := e.Compact(true); err != nil { t.Fatalf("error compacting: %s", err.Error()) } diff --git a/tsdb/engine/pd1/wal.go b/tsdb/engine/pd1/wal.go index c4c6d23ebe8..49b72b4a608 100644 --- a/tsdb/engine/pd1/wal.go +++ b/tsdb/engine/pd1/wal.go @@ -28,9 +28,6 @@ const ( WALFilePrefix = "_" - // defaultFlushCheckInterval is how often flushes are triggered automatically by the flush criteria - defaultFlushCheckInterval = time.Second - writeBufLen = 32 << 10 // 32kb ) @@ -85,10 +82,6 @@ type Log struct { measurementFieldsCache map[string]*tsdb.MeasurementFields seriesToCreateCache []*tsdb.SeriesCreate - // These coordinate closing and waiting for running goroutines. - wg sync.WaitGroup - closing chan struct{} - // LogOutput is the writer used by the logger. LogOutput io.Writer logger *log.Logger @@ -136,7 +129,6 @@ func NewLog(path string) *Log { SegmentSize: DefaultSegmentSize, FlushMemorySizeThreshold: tsdb.DefaultFlushMemorySizeThreshold, MaxMemorySizeThreshold: tsdb.DefaultMaxMemorySizeThreshold, - flushCheckInterval: defaultFlushCheckInterval, logger: log.New(os.Stderr, "[pd1wal] ", log.LstdFlags), } } @@ -161,13 +153,6 @@ func (l *Log) Open() error { return err } - l.flushCheckTimer = time.NewTimer(l.flushCheckInterval) - - // Start background goroutines. - l.wg.Add(1) - l.closing = make(chan struct{}) - go l.autoflusher(l.closing) - return nil } @@ -307,6 +292,12 @@ func (l *Log) addToCache(points []models.Point, fields map[string]*tsdb.Measurem return true } +func (l *Log) LastWriteTime() time.Time { + l.cacheLock.RLock() + defer l.cacheLock.RUnlock() + return l.lastWriteTime +} + // readAndFlushWAL is called on open and will read the segment files in, flushing whenever // the memory gets over the limit. Once all files have been read it will flush and remove the files func (l *Log) readAndFlushWAL() error { @@ -439,21 +430,6 @@ func (l *Log) DeleteSeries(keys []string) error { // Close will finish any flush that is currently in process and close file handles func (l *Log) Close() error { - // stop the autoflushing process so it doesn't try to kick another one off - l.writeLock.Lock() - l.cacheLock.Lock() - - if l.closing != nil { - close(l.closing) - l.closing = nil - } - l.writeLock.Unlock() - l.cacheLock.Unlock() - - // Allow goroutines to finish running. - l.wg.Wait() - - // Lock the remainder of the closing process. l.writeLock.Lock() l.cacheLock.Lock() defer l.writeLock.Unlock() @@ -529,17 +505,16 @@ func (l *Log) flush(flush flushType) error { // copy the cache items to new maps so we can empty them out l.flushCache = make(map[string]Values) - for k, _ := range l.cacheDirtySort { - l.flushCache[k] = l.flushCache[k].Deduplicate() - } - l.cacheDirtySort = make(map[string]bool) - valueCount := 0 for key, v := range l.cache { l.flushCache[key] = v valueCount += len(v) } l.cache = make(map[string]Values) + for k, _ := range l.cacheDirtySort { + l.flushCache[k] = l.flushCache[k].Deduplicate() + } + l.cacheDirtySort = make(map[string]bool) flushSize := l.memorySize @@ -604,33 +579,6 @@ func (l *Log) flush(flush flushType) error { return nil } -// triggerAutoFlush will flush and compact any partitions that have hit the thresholds for compaction -func (l *Log) triggerAutoFlush() { - // - if f := l.shouldFlush(); f != noFlush { - if err := l.flush(f); err != nil { - l.logger.Printf("error flushing wal: %s\n", err) - } - } -} - -// autoflusher waits for notification of a flush and kicks it off in the background. -// This method runs in a separate goroutine. -func (l *Log) autoflusher(closing chan struct{}) { - defer l.wg.Done() - - for { - // Wait for close or flush signal. - select { - case <-closing: - return - case <-l.flushCheckTimer.C: - l.triggerAutoFlush() - l.flushCheckTimer.Reset(l.flushCheckInterval) - } - } -} - // segmentFileNames will return all files that are WAL segment files in sorted order by ascending ID func (l *Log) segmentFileNames() ([]string, error) { names, err := filepath.Glob(filepath.Join(l.path, fmt.Sprintf("%s*.%s", WALFilePrefix, WALFileExtension))) diff --git a/tsdb/shard.go b/tsdb/shard.go index 2e04735665f..b1dbac32fcd 100644 --- a/tsdb/shard.go +++ b/tsdb/shard.go @@ -91,6 +91,12 @@ func NewShard(id uint64, index *DatabaseIndex, path string, walPath string, opti // Path returns the path set on the shard when it was created. func (s *Shard) Path() string { return s.path } +// PerformMaintenance gets called periodically to have the engine perform +// any maintenance tasks like WAL flushing and compaction +func (s *Shard) PerformMaintenance() { + s.engine.PerformMaintenance() +} + // open initializes and opens the shard's store. func (s *Shard) Open() error { if err := func() error { @@ -229,29 +235,31 @@ func (s *Shard) WritePoints(points []models.Point) error { } // make sure all data is encoded before attempting to save to bolt - // TODO: make this only commented out for pd1 engine - // for _, p := range points { - // // Ignore if raw data has already been marshaled. - // if p.Data() != nil { - // continue - // } - - // // This was populated earlier, don't need to validate that it's there. - // s.mu.RLock() - // mf := s.measurementFields[p.Name()] - // s.mu.RUnlock() - - // // If a measurement is dropped while writes for it are in progress, this could be nil - // if mf == nil { - // return ErrFieldNotFound - // } - - // data, err := mf.Codec.EncodeFields(p.Fields()) - // if err != nil { - // return err - // } - // p.SetData(data) - // } + // only required for the b1 and bz1 formats + if s.engine.Format() != PD1Format { + for _, p := range points { + // Ignore if raw data has already been marshaled. + if p.Data() != nil { + continue + } + + // This was populated earlier, don't need to validate that it's there. + s.mu.RLock() + mf := s.measurementFields[p.Name()] + s.mu.RUnlock() + + // If a measurement is dropped while writes for it are in progress, this could be nil + if mf == nil { + return ErrFieldNotFound + } + + data, err := mf.Codec.EncodeFields(p.Fields()) + if err != nil { + return err + } + p.SetData(data) + } + } // Write to the engine. if err := s.engine.WritePoints(points, measurementFieldsToSave, seriesToCreate); err != nil { @@ -742,14 +750,11 @@ func (f *FieldCodec) DecodeByID(targetID uint8, b []byte) (interface{}, error) { // DecodeByName scans a byte slice for a field with the given name, converts it to its // expected type, and return that value. func (f *FieldCodec) DecodeByName(name string, b []byte) (interface{}, error) { - // TODO: this is a hack for PD1 testing, please to remove - return math.Float64frombits(binary.BigEndian.Uint64(b)), nil - - // fi := f.FieldByName(name) - // if fi == nil { - // return 0, ErrFieldNotFound - // } - // return f.DecodeByID(fi.ID, b) + fi := f.FieldByName(name) + if fi == nil { + return 0, ErrFieldNotFound + } + return f.DecodeByID(fi.ID, b) } func (f *FieldCodec) Fields() (a []*Field) { diff --git a/tsdb/store.go b/tsdb/store.go index 13235a16844..62dab631172 100644 --- a/tsdb/store.go +++ b/tsdb/store.go @@ -9,6 +9,7 @@ import ( "strconv" "strings" "sync" + "time" "github.com/influxdb/influxdb/influxql" "github.com/influxdb/influxdb/models" @@ -29,6 +30,10 @@ var ( ErrShardNotFound = fmt.Errorf("shard not found") ) +const ( + MaintenanceCheckInterval = time.Minute +) + type Store struct { mu sync.RWMutex path string @@ -38,7 +43,9 @@ type Store struct { EngineOptions EngineOptions Logger *log.Logger - closing chan struct{} + + closing chan struct{} + wg sync.WaitGroup } // Path returns the store's root path. @@ -301,6 +308,32 @@ func (s *Store) loadShards() error { } +// periodicMaintenance is the method called in a goroutine on the opening of the store +// to perform periodic maintenance of the shards. +func (s *Store) periodicMaintenance() { + t := time.NewTicker(MaintenanceCheckInterval) + for { + select { + case <-t.C: + s.performMaintenance() + case <-s.closing: + t.Stop() + return + } + } +} + +// performMaintenance will loop through the shars and tell them +// to perform any maintenance tasks. Those tasks should kick off +// their own goroutines if it's anything that could take time. +func (s *Store) performMaintenance() { + s.mu.Lock() + defer s.mu.Unlock() + for _, sh := range s.shards { + sh.PerformMaintenance() + } +} + func (s *Store) Open() error { s.mu.Lock() defer s.mu.Unlock() @@ -326,6 +359,8 @@ func (s *Store) Open() error { return err } + go s.periodicMaintenance() + return nil } @@ -366,6 +401,11 @@ func (s *Store) CreateMapper(shardID uint64, stmt influxql.Statement, chunkSize func (s *Store) Close() error { s.mu.Lock() defer s.mu.Unlock() + if s.closing != nil { + close(s.closing) + s.closing = nil + } + s.wg.Wait() for _, sh := range s.shards { if err := sh.Close(); err != nil { From b1bdb4f15a930f561943fd40c803099afdccdb62 Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Tue, 29 Sep 2015 08:35:05 -0400 Subject: [PATCH 089/139] Make compaction run at most at set duration. --- tsdb/engine/pd1/pd1.go | 77 ++++++++++++++++++++++++------------------ 1 file changed, 44 insertions(+), 33 deletions(-) diff --git a/tsdb/engine/pd1/pd1.go b/tsdb/engine/pd1/pd1.go index 1f56c066539..8be62d069d1 100644 --- a/tsdb/engine/pd1/pd1.go +++ b/tsdb/engine/pd1/pd1.go @@ -84,17 +84,19 @@ type Engine struct { WAL *Log - RotateFileSize uint32 - SkipCompaction bool - CompactionAge time.Duration - CompactionFileCount int - IndexCompactionFullAge time.Duration + RotateFileSize uint32 + SkipCompaction bool + CompactionAge time.Duration + CompactionFileCount int + IndexCompactionFullAge time.Duration + IndexMinimumCompactionInterval time.Duration // filesLock is only for modifying and accessing the files slice - filesLock sync.RWMutex - files dataFiles - currentFileID int - compactionRunning bool + filesLock sync.RWMutex + files dataFiles + currentFileID int + compactionRunning bool + lastCompactionTime time.Time collisionsLock sync.RWMutex collisions map[string]uint64 @@ -117,12 +119,13 @@ func NewEngine(path string, walPath string, opt tsdb.EngineOptions) tsdb.Engine writeLock: &writeLock{}, // TODO: this is the function where we can inject a check against the in memory collisions - HashSeriesField: hashSeriesField, - WAL: w, - RotateFileSize: DefaultRotateFileSize, - CompactionAge: opt.Config.IndexCompactionAge, - CompactionFileCount: opt.Config.IndexCompactionFileCount, - IndexCompactionFullAge: opt.Config.IndexCompactionFullAge, + HashSeriesField: hashSeriesField, + WAL: w, + RotateFileSize: DefaultRotateFileSize, + CompactionAge: opt.Config.IndexCompactionAge, + CompactionFileCount: opt.Config.IndexCompactionFileCount, + IndexCompactionFullAge: opt.Config.IndexCompactionFullAge, + IndexMinimumCompactionInterval: opt.Config.IndexMinimumCompactionInterval, } e.WAL.Index = e @@ -136,15 +139,12 @@ func (e *Engine) Path() string { return e.path } func (e *Engine) PerformMaintenance() { if f := e.WAL.shouldFlush(); f != noFlush { go func() { - fmt.Println("maintenance autoflush") e.WAL.flush(f) - if time.Since(e.WAL.lastWriteTime) > e.IndexCompactionFullAge { - fmt.Println("mainenance compact autoflush") + if e.shouldCompact() { e.Compact(true) } }() - } else if time.Since(e.WAL.lastWriteTime) > e.IndexCompactionFullAge { - fmt.Println("compact full, suckas") + } else if e.shouldCompact() { go e.Compact(true) } } @@ -418,7 +418,11 @@ func (e *Engine) Compact(fullCompaction bool) error { var minTime, maxTime int64 var files dataFiles for { - files = e.filesToCompact() + if fullCompaction { + files = e.copyFilesCollection() + } else { + files = e.filesToCompact() + } if len(files) < 2 { return nil } @@ -429,7 +433,13 @@ func (e *Engine) Compact(fullCompaction bool) error { // if the files are different after obtaining the write lock, one or more // was rewritten. Release the lock and try again. This shouldn't happen really. - if !reflect.DeepEqual(files, e.filesToCompact()) { + var filesAfterLock dataFiles + if fullCompaction { + filesAfterLock = e.copyFilesCollection() + } else { + filesAfterLock = e.filesToCompact() + } + if !reflect.DeepEqual(files, filesAfterLock) { e.writeLock.UnlockRange(minTime, maxTime) continue } @@ -438,6 +448,9 @@ func (e *Engine) Compact(fullCompaction bool) error { break } + fmt.Println("Starting compaction with files:", len(files)) + st := time.Now() + // mark the compaction as running e.filesLock.Lock() e.compactionRunning = true @@ -445,15 +458,10 @@ func (e *Engine) Compact(fullCompaction bool) error { defer func() { //release the lock e.writeLock.UnlockRange(minTime, maxTime) - - // see if we should run aonther compaction - if e.shouldCompact() { - go e.Compact(false) - } else { - e.filesLock.Lock() - e.compactionRunning = false - e.filesLock.Unlock() - } + e.filesLock.Lock() + e.lastCompactionTime = time.Now() + e.compactionRunning = false + e.filesLock.Unlock() }() positions := make([]uint32, len(files)) @@ -586,6 +594,8 @@ func (e *Engine) Compact(fullCompaction bool) error { e.files = newFiles e.filesLock.Unlock() + fmt.Println("Compaction took ", time.Since(st)) + // delete the old files in a goroutine so running queries won't block the write // from completing e.deletesPending.Add(1) @@ -651,8 +661,9 @@ func (e *Engine) writeIndexAndGetDataFile(f *os.File, minTime, maxTime int64, id func (e *Engine) shouldCompact() bool { e.filesLock.RLock() running := e.compactionRunning + since := time.Since(e.lastCompactionTime) e.filesLock.RUnlock() - if running { + if running || since < e.IndexMinimumCompactionInterval { return false } return len(e.filesToCompact()) >= e.CompactionFileCount @@ -662,7 +673,7 @@ func (e *Engine) filesToCompact() dataFiles { e.filesLock.RLock() defer e.filesLock.RUnlock() - a := make([]*dataFile, 0) + var a dataFiles for _, df := range e.files { if time.Since(df.modTime) > e.CompactionAge && df.size < MaxDataFileSize { a = append(a, df) From 0fd116d1f201424f52b7d03f89ee3d3c5e6a737d Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Tue, 29 Sep 2015 08:59:25 -0400 Subject: [PATCH 090/139] Ensure data files can't be deleted while query is running. Also ensure that queries don't try to use files that have been deleted. --- tsdb/engine/pd1/pd1.go | 72 ++++++++++++++++++++++-------------------- tsdb/engine/pd1/tx.go | 52 ++++++++++++++++++++++++++++++ 2 files changed, 90 insertions(+), 34 deletions(-) create mode 100644 tsdb/engine/pd1/tx.go diff --git a/tsdb/engine/pd1/pd1.go b/tsdb/engine/pd1/pd1.go index 8be62d069d1..32edc84c7c3 100644 --- a/tsdb/engine/pd1/pd1.go +++ b/tsdb/engine/pd1/pd1.go @@ -1110,37 +1110,45 @@ func (e *Engine) SeriesCount() (n int, err error) { // Begin starts a new transaction on the engine. func (e *Engine) Begin(writable bool) (tsdb.Tx, error) { e.queryLock.RLock() - return e, nil -} -// TODO: handle multiple fields and descending -func (e *Engine) Cursor(series string, fields []string, dec *tsdb.FieldCodec, ascending bool) tsdb.Cursor { - files := e.copyFilesCollection() + var files dataFiles - // don't add the overhead of the multifield cursor if we only have one field - if len(fields) == 1 { - id := e.keyAndFieldToID(series, fields[0]) - indexCursor := newCursor(id, files, ascending) - wc := e.WAL.Cursor(series, fields, dec, ascending) - return NewCombinedEngineCursor(wc, indexCursor, ascending) - } + // we do this to ensure that the data files haven't been deleted from a compaction + // while we were waiting to get the query lock + for { + files = e.copyFilesCollection() + + // get the query lock + for _, f := range files { + f.mu.RLock() + } - // multiple fields. use just the MultiFieldCursor, which also handles time collisions - // so we don't need to use the combined cursor - cursors := make([]tsdb.Cursor, 0) - cursorFields := make([]string, 0) - for _, field := range fields { - id := e.keyAndFieldToID(series, field) - indexCursor := newCursor(id, files, ascending) - wc := e.WAL.Cursor(series, []string{field}, dec, ascending) - // double up the fields since there's one for the wal and one for the index - cursorFields = append(cursorFields, field, field) - cursors = append(cursors, indexCursor, wc) + // ensure they're all still open + reset := false + for _, f := range files { + if f.f == nil { + reset = true + break + } + } + + // if not, release and try again + if reset { + for _, f := range files { + f.mu.RUnlock() + } + continue + } + + // we're good to go + break } - return NewMultiFieldCursor(cursorFields, cursors, ascending) + return &tx{files: files, engine: e}, nil } +func (e *Engine) WriteTo(w io.Writer) (n int64, err error) { panic("not implemented") } + func (e *Engine) keyAndFieldToID(series, field string) uint64 { // get the ID for the key and be sure to check if it had hash collision before key := seriesFieldKey(series, field) @@ -1162,15 +1170,6 @@ func (e *Engine) copyFilesCollection() []*dataFile { return a } -// TODO: refactor the Tx interface to not have Size, Commit, or WriteTo since they're not used -func (e *Engine) Size() int64 { panic("not implemented") } -func (e *Engine) Commit() error { panic("not implemented") } -func (e *Engine) WriteTo(w io.Writer) (n int64, err error) { panic("not implemented") } -func (e *Engine) Rollback() error { - e.queryLock.RUnlock() - return nil -} - func (e *Engine) writeNewFields(measurementFieldsToSave map[string]*tsdb.MeasurementFields) error { if len(measurementFieldsToSave) == 0 { return nil @@ -1403,7 +1402,12 @@ func (d *dataFile) Delete() error { if err := d.close(); err != nil { return err } - return os.Remove(d.f.Name()) + err := os.Remove(d.f.Name()) + if err != nil { + return err + } + d.f = nil + return nil } func (d *dataFile) close() error { diff --git a/tsdb/engine/pd1/tx.go b/tsdb/engine/pd1/tx.go new file mode 100644 index 00000000000..16aac8b5e43 --- /dev/null +++ b/tsdb/engine/pd1/tx.go @@ -0,0 +1,52 @@ +package pd1 + +import ( + "io" + + "github.com/influxdb/influxdb/tsdb" +) + +type tx struct { + files dataFiles + engine *Engine +} + +// TODO: handle multiple fields and descending +func (t *tx) Cursor(series string, fields []string, dec *tsdb.FieldCodec, ascending bool) tsdb.Cursor { + // don't add the overhead of the multifield cursor if we only have one field + if len(fields) == 1 { + id := t.engine.keyAndFieldToID(series, fields[0]) + indexCursor := newCursor(id, t.files, ascending) + wc := t.engine.WAL.Cursor(series, fields, dec, ascending) + return NewCombinedEngineCursor(wc, indexCursor, ascending) + } + + // multiple fields. use just the MultiFieldCursor, which also handles time collisions + // so we don't need to use the combined cursor + cursors := make([]tsdb.Cursor, 0) + cursorFields := make([]string, 0) + for _, field := range fields { + id := t.engine.keyAndFieldToID(series, field) + indexCursor := newCursor(id, t.files, ascending) + wc := t.engine.WAL.Cursor(series, []string{field}, dec, ascending) + // double up the fields since there's one for the wal and one for the index + cursorFields = append(cursorFields, field, field) + cursors = append(cursors, indexCursor, wc) + } + + return NewMultiFieldCursor(cursorFields, cursors, ascending) +} + +func (t *tx) Rollback() error { + t.engine.queryLock.RUnlock() + for _, f := range t.files { + f.mu.RUnlock() + } + + return nil +} + +// TODO: refactor the Tx interface to not have Size, Commit, or WriteTo since they're not used +func (t *tx) Size() int64 { panic("not implemented") } +func (t *tx) Commit() error { panic("not implemented") } +func (t *tx) WriteTo(w io.Writer) (n int64, err error) { panic("not implemented") } From 0b33a71bb7e188b53d66a7e196b05a5ed52ce8c9 Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Tue, 29 Sep 2015 10:56:11 -0400 Subject: [PATCH 091/139] Add recover to maintenance. Change snapshot writer to not use bolt on shard. --- tsdb/engine/pd1/pd1.go | 3 --- tsdb/shard.go | 2 -- tsdb/snapshot_writer.go | 5 ++--- tsdb/store.go | 11 ++++++++++- 4 files changed, 12 insertions(+), 9 deletions(-) diff --git a/tsdb/engine/pd1/pd1.go b/tsdb/engine/pd1/pd1.go index 32edc84c7c3..82df8c60e6d 100644 --- a/tsdb/engine/pd1/pd1.go +++ b/tsdb/engine/pd1/pd1.go @@ -140,9 +140,6 @@ func (e *Engine) PerformMaintenance() { if f := e.WAL.shouldFlush(); f != noFlush { go func() { e.WAL.flush(f) - if e.shouldCompact() { - e.Compact(true) - } }() } else if e.shouldCompact() { go e.Compact(true) diff --git a/tsdb/shard.go b/tsdb/shard.go index b1dbac32fcd..1f606b613b3 100644 --- a/tsdb/shard.go +++ b/tsdb/shard.go @@ -16,7 +16,6 @@ import ( "github.com/influxdb/influxdb/models" "github.com/influxdb/influxdb/tsdb/internal" - "github.com/boltdb/bolt" "github.com/gogo/protobuf/proto" ) @@ -49,7 +48,6 @@ var ( // Data can be split across many shards. The query engine in TSDB is responsible // for combining the output of many shards into a single query result. type Shard struct { - db *bolt.DB // underlying data store index *DatabaseIndex path string walPath string diff --git a/tsdb/snapshot_writer.go b/tsdb/snapshot_writer.go index 785ca13908c..4a0a2d3edef 100644 --- a/tsdb/snapshot_writer.go +++ b/tsdb/snapshot_writer.go @@ -8,7 +8,6 @@ import ( "path/filepath" "time" - "github.com/boltdb/bolt" "github.com/influxdb/influxdb/snapshot" ) @@ -83,7 +82,7 @@ func appendShardSnapshotFile(sw *snapshot.Writer, sh *Shard, name string) error } // Begin transaction. - tx, err := sh.db.Begin(false) + tx, err := sh.engine.Begin(false) if err != nil { return fmt.Errorf("begin: %s", err) } @@ -103,7 +102,7 @@ func appendShardSnapshotFile(sw *snapshot.Writer, sh *Shard, name string) error // boltTxCloser wraps a Bolt transaction to implement io.Closer. type boltTxCloser struct { - *bolt.Tx + Tx } // Close rolls back the transaction. diff --git a/tsdb/store.go b/tsdb/store.go index 62dab631172..1fb8560167e 100644 --- a/tsdb/store.go +++ b/tsdb/store.go @@ -330,10 +330,19 @@ func (s *Store) performMaintenance() { s.mu.Lock() defer s.mu.Unlock() for _, sh := range s.shards { - sh.PerformMaintenance() + s.performMaintenanceOnShard(sh) } } +func (s *Store) performMaintenanceOnShard(shard *Shard) { + defer func() { + if r := recover(); r != nil { + s.Logger.Printf("recovered eror in maintenance on shard %d", shard.id) + } + }() + shard.PerformMaintenance() +} + func (s *Store) Open() error { s.mu.Lock() defer s.mu.Unlock() From db4ad33f3c2110821b52cd328cc3079d4639a571 Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Tue, 29 Sep 2015 11:27:02 -0400 Subject: [PATCH 092/139] Update tests to use transactions. Add test for single series 10k points. --- tsdb/engine/pd1/pd1_test.go | 126 +++++++++++++++++++++--------------- 1 file changed, 75 insertions(+), 51 deletions(-) diff --git a/tsdb/engine/pd1/pd1_test.go b/tsdb/engine/pd1/pd1_test.go index 491c7cd552b..e1abe87360c 100644 --- a/tsdb/engine/pd1/pd1_test.go +++ b/tsdb/engine/pd1/pd1_test.go @@ -10,7 +10,6 @@ import ( "testing" "time" - "github.com/influxdb/influxdb/influxql" "github.com/influxdb/influxdb/models" "github.com/influxdb/influxdb/tsdb" "github.com/influxdb/influxdb/tsdb/engine/pd1" @@ -30,16 +29,11 @@ func TestEngine_WriteAndReadFloats(t *testing.T) { } fields := []string{"value"} - codec := tsdb.NewFieldCodec(map[string]*tsdb.Field{ - "value": { - ID: uint8(1), - Name: "value", - Type: influxql.Float, - }, - }) verify := func(checkSingleBVal bool) { - c := e.Cursor("cpu,host=A", fields, codec, true) + tx, _ := e.Begin(false) + defer tx.Rollback() + c := tx.Cursor("cpu,host=A", fields, nil, true) k, v := c.SeekTo(0) if k != p1.UnixNano() { t.Fatalf("p1 time wrong:\n\texp:%d\n\tgot:%d\n", p1.UnixNano(), k) @@ -59,7 +53,7 @@ func TestEngine_WriteAndReadFloats(t *testing.T) { t.Fatal("expected EOF") } - c = e.Cursor("cpu,host=B", fields, codec, true) + c = tx.Cursor("cpu,host=B", fields, nil, true) k, v = c.SeekTo(0) if k != p2.UnixNano() { t.Fatalf("p2 time wrong:\n\texp:%d\n\tgot:%d\n", p2.UnixNano(), k) @@ -82,7 +76,9 @@ func TestEngine_WriteAndReadFloats(t *testing.T) { } verify(false) - c := e.Cursor("cpu,host=B", fields, codec, true) + tx, _ := e.Begin(false) + defer tx.Rollback() + c := tx.Cursor("cpu,host=B", fields, nil, true) k, v := c.SeekTo(0) if k != p2.UnixNano() { t.Fatalf("p2 time wrong:\n\texp:%d\n\tgot:%d\n", p2.UnixNano(), k) @@ -107,7 +103,7 @@ func TestEngine_WriteAndReadFloats(t *testing.T) { t.Fatal("p2 data not equal") } - c = e.Cursor("cpu,host=A", fields, codec, true) + c = tx.Cursor("cpu,host=A", fields, nil, true) k, v = c.SeekTo(0) if k != p1.UnixNano() { t.Fatalf("p1 time wrong:\n\texp:%d\n\tgot:%d\n", p1.UnixNano(), k) @@ -161,7 +157,9 @@ func TestEngine_WriteIndexQueryAcrossDataFiles(t *testing.T) { fields := []string{"value"} verify := func(series string, points []models.Point, seek int64) { - c := e.Cursor(series, fields, nil, true) + tx, _ := e.Begin(false) + defer tx.Rollback() + c := tx.Cursor(series, fields, nil, true) k, v := c.SeekTo(seek) p := points[0] @@ -191,13 +189,6 @@ func TestEngine_WriteOverwritePreviousPoint(t *testing.T) { defer e.Cleanup() fields := []string{"value"} - codec := tsdb.NewFieldCodec(map[string]*tsdb.Field{ - "value": { - ID: uint8(1), - Name: "value", - Type: influxql.Float, - }, - }) p1 := parsePoint("cpu,host=A value=1.1 1000000000") p2 := parsePoint("cpu,host=A value=1.2 1000000000") @@ -207,7 +198,9 @@ func TestEngine_WriteOverwritePreviousPoint(t *testing.T) { t.Fatalf("failed to write points: %s", err.Error()) } - c := e.Cursor("cpu,host=A", fields, codec, true) + tx, _ := e.Begin(false) + defer tx.Rollback() + c := tx.Cursor("cpu,host=A", fields, nil, true) k, v := c.SeekTo(0) if k != p2.UnixNano() { t.Fatalf("time wrong:\n\texp:%d\n\tgot:%d\n", p2.UnixNano(), k) @@ -224,7 +217,9 @@ func TestEngine_WriteOverwritePreviousPoint(t *testing.T) { t.Fatalf("failed to write points: %s", err.Error()) } - c = e.Cursor("cpu,host=A", fields, codec, true) + tx2, _ := e.Begin(false) + defer tx2.Rollback() + c = tx2.Cursor("cpu,host=A", fields, nil, true) k, v = c.SeekTo(0) if k != p3.UnixNano() { t.Fatalf("time wrong:\n\texp:%d\n\tgot:%d\n", p3.UnixNano(), k) @@ -243,13 +238,6 @@ func TestEngine_CursorCombinesWALAndIndex(t *testing.T) { defer e.Cleanup() fields := []string{"value"} - codec := tsdb.NewFieldCodec(map[string]*tsdb.Field{ - "value": { - ID: uint8(1), - Name: "value", - Type: influxql.Float, - }, - }) p1 := parsePoint("cpu,host=A value=1.1 1000000000") p2 := parsePoint("cpu,host=A value=1.2 2000000000") @@ -262,7 +250,9 @@ func TestEngine_CursorCombinesWALAndIndex(t *testing.T) { t.Fatalf("failed to write points: %s", err.Error()) } - c := e.Cursor("cpu,host=A", fields, codec, true) + tx, _ := e.Begin(false) + defer tx.Rollback() + c := tx.Cursor("cpu,host=A", fields, nil, true) k, v := c.SeekTo(0) if k != p1.UnixNano() { t.Fatalf("time wrong:\n\texp:%d\n\tgot:%d\n", p1.UnixNano(), k) @@ -318,13 +308,6 @@ func TestEngine_Compaction(t *testing.T) { } fields := []string{"value"} - codec := tsdb.NewFieldCodec(map[string]*tsdb.Field{ - "value": { - ID: uint8(1), - Name: "value", - Type: influxql.Float, - }, - }) e.CompactionAge = time.Duration(0) @@ -337,7 +320,9 @@ func TestEngine_Compaction(t *testing.T) { } verify := func(series string, points []models.Point, seek int64) { - c := e.Cursor(series, fields, codec, true) + tx, _ := e.Begin(false) + defer tx.Rollback() + c := tx.Cursor(series, fields, nil, true) k, v := c.SeekTo(seek) p := points[0] @@ -374,13 +359,6 @@ func TestEngine_KeyCollisionsAreHandled(t *testing.T) { defer e.Cleanup() fields := []string{"value"} - codec := tsdb.NewFieldCodec(map[string]*tsdb.Field{ - "value": { - ID: uint8(1), - Name: "value", - Type: influxql.Float, - }, - }) // make sure two of these keys collide e.HashSeriesField = func(key string) uint64 { @@ -395,7 +373,9 @@ func TestEngine_KeyCollisionsAreHandled(t *testing.T) { } verify := func(series string, points []models.Point, seek int64) { - c := e.Cursor(series, fields, codec, true) + tx, _ := e.Begin(false) + defer tx.Rollback() + c := tx.Cursor(series, fields, nil, true) k, v := c.SeekTo(seek) p := points[0] @@ -467,7 +447,9 @@ func TestEngine_SupportMultipleFields(t *testing.T) { if err := e.WritePoints([]models.Point{p1, p2}, nil, nil); err != nil { t.Fatalf("failed to write points: %s", err.Error()) } - c := e.Cursor("cpu,host=A", fields, nil, true) + tx, _ := e.Begin(false) + defer tx.Rollback() + c := tx.Cursor("cpu,host=A", fields, nil, true) k, v := c.SeekTo(0) if k != p1.UnixNano() { t.Fatalf("time wrong:\n\texp: %d\n\tgot: %d", p1.UnixNano(), k) @@ -493,7 +475,9 @@ func TestEngine_SupportMultipleFields(t *testing.T) { t.Fatalf("failed to write points: %s", err.Error()) } - c = e.Cursor("cpu,host=A", fields, nil, true) + tx2, _ := e.Begin(false) + defer tx2.Rollback() + c = tx2.Cursor("cpu,host=A", fields, nil, true) k, v = c.SeekTo(0) if k != p1.UnixNano() { t.Fatalf("time wrong:\n\texp: %d\n\tgot: %d", p1.UnixNano(), k) @@ -522,7 +506,9 @@ func TestEngine_SupportMultipleFields(t *testing.T) { t.Fatalf("failed to write points: %s", err.Error()) } - c = e.Cursor("cpu,host=A", fields, nil, true) + tx3, _ := e.Begin(false) + defer tx3.Rollback() + c = tx3.Cursor("cpu,host=A", fields, nil, true) k, v = c.SeekTo(0) if k != p1.UnixNano() { t.Fatalf("time wrong:\n\texp: %d\n\tgot: %d", p1.UnixNano(), k) @@ -561,7 +547,9 @@ func TestEngine_SupportMultipleFields(t *testing.T) { t.Fatalf("failed to write points: %s", err.Error()) } - c = e.Cursor("cpu,host=A", fields, nil, true) + tx4, _ := e.Begin(false) + defer tx4.Rollback() + c = tx4.Cursor("cpu,host=A", fields, nil, true) k, v = c.SeekTo(0) if k != p1.UnixNano() { t.Fatalf("time wrong:\n\texp: %d\n\tgot: %d", p1.UnixNano(), k) @@ -596,7 +584,7 @@ func TestEngine_SupportMultipleFields(t *testing.T) { } // and ensure we can grab one of the fields - c = e.Cursor("cpu,host=A", []string{"value"}, nil, true) + c = tx4.Cursor("cpu,host=A", []string{"value"}, nil, true) k, v = c.SeekTo(4000000000) if k != p4.UnixNano() { t.Fatalf("time wrong:\n\texp: %d\n\tgot: %d", p4.UnixNano(), k) @@ -610,6 +598,42 @@ func TestEngine_SupportMultipleFields(t *testing.T) { } } +func TestEngine_WriteManyPointsToSingleSeries(t *testing.T) { + e := OpenDefaultEngine() + defer e.Cleanup() + + fields := []string{"value"} + + var points []models.Point + for i := 1; i <= 10000; i++ { + points = append(points, parsePoint(fmt.Sprintf("cpu,host=A value=%d %d000000000", i, i))) + if i%500 == 0 { + if err := e.WritePoints(points, nil, nil); err != nil { + t.Fatalf("failed to write points: %s", err.Error()) + } + points = nil + } + } + + tx, _ := e.Begin(false) + defer tx.Rollback() + c := tx.Cursor("cpu,host=A", fields, nil, true) + k, v := c.SeekTo(0) + for i := 2; i <= 10000; i++ { + k, v = c.Next() + if k != int64(i)*1000000000 { + t.Fatalf("time wrong:\n\texp: %d\n\tgot: %d", i*1000000000, k) + } + if v != float64(i) { + t.Fatalf("value wrong:\n\texp:%v\n\tgot:%v", float64(i), v) + } + } + k, _ = c.Next() + if k != tsdb.EOF { + t.Fatal("expected EOF") + } +} + func TestEngine_WriteIndexBenchmarkNames(t *testing.T) { t.Skip("whatevs") From 12e14f105b150c8ea12cf889c1e0b676c6edd43f Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Tue, 29 Sep 2015 11:37:28 -0400 Subject: [PATCH 093/139] Update stress to use second timestamps and less random floats. --- cmd/influx_stress/influx_stress.go | 2 -- 1 file changed, 2 deletions(-) diff --git a/cmd/influx_stress/influx_stress.go b/cmd/influx_stress/influx_stress.go index 9d5d0c22b5e..660007125a9 100644 --- a/cmd/influx_stress/influx_stress.go +++ b/cmd/influx_stress/influx_stress.go @@ -3,8 +3,6 @@ package main import ( "flag" "fmt" - "math/rand" - "net/url" "runtime" "sort" "time" From 88248f3f8164f4cffd4c9b02c44ccb9dc3b1b9de Mon Sep 17 00:00:00 2001 From: Jason Wilder Date: Mon, 21 Sep 2015 13:56:10 -0600 Subject: [PATCH 094/139] Ensure we have files when iterating in cursor Prevents index out of bounds panic --- tsdb/engine/pd1/pd1.go | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/tsdb/engine/pd1/pd1.go b/tsdb/engine/pd1/pd1.go index 82df8c60e6d..d707661fad7 100644 --- a/tsdb/engine/pd1/pd1.go +++ b/tsdb/engine/pd1/pd1.go @@ -1518,6 +1518,10 @@ func newCursor(id uint64, files []*dataFile, ascending bool) *cursor { } func (c *cursor) SeekTo(seek int64) (int64, interface{}) { + if len(c.files) == 0 { + return tsdb.EOF, nil + } + if seek < c.files[0].MinTime() { c.filesPos = 0 c.f = c.files[0] From 112a03f24c6f06c1762598dd5434989fc3f019ff Mon Sep 17 00:00:00 2001 From: Jason Wilder Date: Mon, 21 Sep 2015 13:57:03 -0600 Subject: [PATCH 095/139] Fix go vet errors --- tsdb/engine/pd1/wal_test.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tsdb/engine/pd1/wal_test.go b/tsdb/engine/pd1/wal_test.go index 509ed6984ca..034ad2dd3e5 100644 --- a/tsdb/engine/pd1/wal_test.go +++ b/tsdb/engine/pd1/wal_test.go @@ -29,7 +29,7 @@ func TestWAL_TestWriteQueryOpen(t *testing.T) { } if err := w.Open(); err != nil { - t.Fatalf("error opening: %s", err.Error) + t.Fatalf("error opening: %s", err.Error()) } p1 := parsePoint("cpu,host=A value=1.1 1000000000") @@ -70,7 +70,7 @@ func TestWAL_TestWriteQueryOpen(t *testing.T) { // ensure we can do another write to the wal and get stuff if err := w.WritePoints([]models.Point{p3}, nil, nil); err != nil { - t.Fatalf("failed to write: %s", err.Error) + t.Fatalf("failed to write: %s", err.Error()) } c = w.Cursor("cpu,host=A", fieldNames, codec, true) @@ -120,7 +120,7 @@ func TestWAL_TestWriteQueryOpen(t *testing.T) { } if err := w.WritePoints([]models.Point{p4}, nil, nil); err != nil { - t.Fatalf("failed to write: %s", err.Error) + t.Fatalf("failed to write: %s", err.Error()) } c = w.Cursor("cpu,host=B", fieldNames, codec, true) k, v = c.Next() From 42e1babe7f70708a10a95ade2b70df4e9e3c20b1 Mon Sep 17 00:00:00 2001 From: Jason Wilder Date: Mon, 21 Sep 2015 14:52:41 -0600 Subject: [PATCH 096/139] Add time and float compression Time compression uses an adaptive approach using delta-encoding, frame-of-reference, run length encoding as well as compressed integer encoding. Float compression uses an implementation of the Gorilla paper encoding for timestamps based on XOR deltas and leading and trailing null suppression. --- tsdb/engine/pd1/encoding.go | 83 ++++++- tsdb/engine/pd1/float.go | 206 +++++++++++++++++ tsdb/engine/pd1/float_test.go | 149 +++++++++++++ tsdb/engine/pd1/timestamp.go | 286 ++++++++++++++++++++++++ tsdb/engine/pd1/timestamp_test.go | 353 ++++++++++++++++++++++++++++++ 5 files changed, 1066 insertions(+), 11 deletions(-) create mode 100644 tsdb/engine/pd1/float.go create mode 100644 tsdb/engine/pd1/float_test.go create mode 100644 tsdb/engine/pd1/timestamp.go create mode 100644 tsdb/engine/pd1/timestamp_test.go diff --git a/tsdb/engine/pd1/encoding.go b/tsdb/engine/pd1/encoding.go index b21d394af5d..6b29913e0e0 100644 --- a/tsdb/engine/pd1/encoding.go +++ b/tsdb/engine/pd1/encoding.go @@ -4,7 +4,6 @@ import ( "sort" "time" - "github.com/dgryski/go-tsz" "github.com/influxdb/influxdb/tsdb" ) @@ -127,23 +126,59 @@ func (f *FloatValue) Size() int { return 16 } -// TODO: make this work with nanosecond timestamps func EncodeFloatBlock(buf []byte, values []*FloatValue) []byte { - s := tsz.New(uint32(values[0].Time().Unix())) + if len(values) == 0 { + return []byte{} + } + + // A float block is encoded using different compression strategies + // for timestamps and values. + + // Encode values using Gorilla float compression + venc := NewFloatEncoder() + + // Encode timestamps using an adaptive encoder that uses delta-encoding, + // frame-or-reference and run length encoding. + tsenc := NewTimeEncoder() + for _, v := range values { - s.Push(uint32(v.Time().Unix()), v.value) + tsenc.Write(v.Time()) + venc.Push(v.value) } - s.Finish() - return append(u64tob(uint64(values[0].Time().UnixNano())), s.Bytes()...) + venc.Finish() + + // Encoded timestamp values + tb, err := tsenc.Bytes() + if err != nil { + panic(err.Error()) + } + // Encoded float values + vb := venc.Bytes() + + // Preprend the first timestamp of the block in the first 8 bytes + return append(u64tob(uint64(values[0].Time().UnixNano())), + packBlock(tb, vb)...) } func DecodeFloatBlock(block []byte) ([]Value, error) { - iter, _ := tsz.NewIterator(block[8:]) - a := make([]Value, 0) - for iter.Next() { - t, f := iter.Values() - a = append(a, &FloatValue{time.Unix(int64(t), 0), f}) + // The first 8 bytes is the minimum timestamp of the block + tb, vb := unpackBlock(block[8:]) + + // Setup our timestamp and value decoders + dec := NewTimeDecoder(tb) + iter, err := NewFloatDecoder(vb) + if err != nil { + return nil, err + } + + // Decode both a timestamp and value + var a []Value + for dec.Next() && iter.Next() { + ts := dec.Read() + v := iter.Values() + a = append(a, &FloatValue{ts, v}) } + return a, nil } @@ -181,3 +216,29 @@ type StringValue struct { func EncodeStringBlock(buf []byte, values []StringValue) []byte { return nil } + +func packBlock(ts []byte, values []byte) []byte { + // We encode the length of the timestamp block using a variable byte encoding. + // This allows small byte slices to take up 1 byte while larger ones use 2 or more. + b := make([]byte, 10) + i := binary.PutUvarint(b, uint64(len(ts))) + + // block is , , + block := append(b[:i], ts...) + + // We don't encode the value length because we know it's the rest of the block after + // the timestamp block. + return append(block, values...) +} + +func unpackBlock(buf []byte) (ts, values []byte) { + // Unpack the timestamp block length + tsLen, i := binary.Uvarint(buf) + + // Unpack the timestamp bytes + ts = buf[int(i) : int(i)+int(tsLen)] + + // Unpack the value bytes + values = buf[int(i)+int(tsLen):] + return +} diff --git a/tsdb/engine/pd1/float.go b/tsdb/engine/pd1/float.go new file mode 100644 index 00000000000..dddb9f39b24 --- /dev/null +++ b/tsdb/engine/pd1/float.go @@ -0,0 +1,206 @@ +package pd1 + +/* +This code is originally from: https://github.com/dgryski/go-tsz and has been modified to remove +the timestamp compression fuctionality. + +It implements the float compression as presented in: http://www.vldb.org/pvldb/vol8/p1816-teller.pdf. +This implementation uses a sentinel value of NaN which means that float64 NaN cannot be stored using +this version. +*/ + +import ( + "bytes" + "math" + + "github.com/dgryski/go-bits" + "github.com/dgryski/go-bitstream" +) + +type FloatEncoder struct { + val float64 + + leading uint64 + trailing uint64 + + buf bytes.Buffer + bw *bitstream.BitWriter + + first bool + finished bool +} + +func NewFloatEncoder() *FloatEncoder { + s := FloatEncoder{ + first: true, + leading: ^uint64(0), + } + + s.bw = bitstream.NewWriter(&s.buf) + + return &s + +} + +func (s *FloatEncoder) Bytes() []byte { + return s.buf.Bytes() +} + +func (s *FloatEncoder) Finish() { + + if !s.finished { + // // write an end-of-stream record + s.Push(math.NaN()) + s.bw.Flush(bitstream.Zero) + s.finished = true + } +} + +func (s *FloatEncoder) Push(v float64) { + + if s.first { + // first point + s.val = v + s.first = false + s.bw.WriteBits(math.Float64bits(v), 64) + return + } + + vDelta := math.Float64bits(v) ^ math.Float64bits(s.val) + + if vDelta == 0 { + s.bw.WriteBit(bitstream.Zero) + } else { + s.bw.WriteBit(bitstream.One) + + leading := bits.Clz(vDelta) + trailing := bits.Ctz(vDelta) + + // TODO(dgryski): check if it's 'cheaper' to reset the leading/trailing bits instead + if s.leading != ^uint64(0) && leading >= s.leading && trailing >= s.trailing { + s.bw.WriteBit(bitstream.Zero) + s.bw.WriteBits(vDelta>>s.trailing, 64-int(s.leading)-int(s.trailing)) + } else { + s.leading, s.trailing = leading, trailing + + s.bw.WriteBit(bitstream.One) + s.bw.WriteBits(leading, 5) + + sigbits := 64 - leading - trailing + s.bw.WriteBits(sigbits, 6) + s.bw.WriteBits(vDelta>>trailing, int(sigbits)) + } + } + + s.val = v +} + +func (s *FloatEncoder) FloatDecoder() *FloatDecoder { + iter, _ := NewFloatDecoder(s.buf.Bytes()) + return iter +} + +type FloatDecoder struct { + val float64 + + leading uint64 + trailing uint64 + + br *bitstream.BitReader + + b []byte + + first bool + finished bool + + err error +} + +func NewFloatDecoder(b []byte) (*FloatDecoder, error) { + br := bitstream.NewReader(bytes.NewReader(b)) + + v, err := br.ReadBits(64) + if err != nil { + return nil, err + } + + return &FloatDecoder{ + val: math.Float64frombits(v), + first: true, + br: br, + b: b, + }, nil +} + +func (it *FloatDecoder) Next() bool { + if it.err != nil || it.finished { + return false + } + + if it.first { + it.first = false + return true + } + + // read compressed value + bit, err := it.br.ReadBit() + if err != nil { + it.err = err + return false + } + + if bit == bitstream.Zero { + // it.val = it.val + } else { + bit, err := it.br.ReadBit() + if err != nil { + it.err = err + return false + } + if bit == bitstream.Zero { + // reuse leading/trailing zero bits + // it.leading, it.trailing = it.leading, it.trailing + } else { + bits, err := it.br.ReadBits(5) + if err != nil { + it.err = err + return false + } + it.leading = bits + + bits, err = it.br.ReadBits(6) + if err != nil { + it.err = err + return false + } + mbits := bits + it.trailing = 64 - it.leading - mbits + } + + mbits := int(64 - it.leading - it.trailing) + bits, err := it.br.ReadBits(mbits) + if err != nil { + it.err = err + return false + } + vbits := math.Float64bits(it.val) + vbits ^= (bits << it.trailing) + + val := math.Float64frombits(vbits) + if math.IsNaN(val) { + it.finished = true + return false + } + it.val = val + } + + return true +} + +func (it *FloatDecoder) Values() float64 { + return it.val +} + +func (it *FloatDecoder) Err() error { + return it.err +} diff --git a/tsdb/engine/pd1/float_test.go b/tsdb/engine/pd1/float_test.go new file mode 100644 index 00000000000..5bfa377ac1b --- /dev/null +++ b/tsdb/engine/pd1/float_test.go @@ -0,0 +1,149 @@ +package pd1_test + +import ( + "testing" + + "github.com/influxdb/influxdb/tsdb/engine/pd1" +) + +func TestExampleEncoding(t *testing.T) { + + // Example from the paper + s := pd1.NewFloatEncoder() + + s.Push(12) + s.Push(12) + s.Push(24) + + // extra tests + + // floating point masking/shifting bug + s.Push(13) + s.Push(24) + + // delta-of-delta sizes + s.Push(24) + s.Push(24) + s.Push(24) + + s.Finish() + + it := s.FloatDecoder() + + want := []float64{ + 12, + 12, + 24, + + 13, + 24, + + 24, + 24, + 24, + } + + for _, w := range want { + if !it.Next() { + t.Fatalf("Next()=false, want true") + } + vv := it.Values() + if w != vv { + t.Errorf("Values()=(%v), want (%v)\n", vv, w) + } + } + + if it.Next() { + t.Fatalf("Next()=true, want false") + } + + if err := it.Err(); err != nil { + t.Errorf("it.Err()=%v, want nil", err) + } +} + +var TwoHoursData = []struct { + v float64 +}{ + // 2h of data + {761}, {727}, {763}, {706}, {700}, + {679}, {757}, {708}, {739}, {707}, + {699}, {740}, {729}, {766}, {730}, + {715}, {705}, {693}, {765}, {724}, + {799}, {761}, {737}, {766}, {756}, + {719}, {722}, {801}, {747}, {731}, + {742}, {744}, {791}, {750}, {759}, + {809}, {751}, {705}, {770}, {792}, + {727}, {762}, {772}, {721}, {748}, + {753}, {744}, {716}, {776}, {659}, + {789}, {766}, {758}, {690}, {795}, + {770}, {758}, {723}, {767}, {765}, + {693}, {706}, {681}, {727}, {724}, + {780}, {678}, {696}, {758}, {740}, + {735}, {700}, {742}, {747}, {752}, + {734}, {743}, {732}, {746}, {770}, + {780}, {710}, {731}, {712}, {712}, + {741}, {770}, {770}, {754}, {718}, + {670}, {775}, {749}, {795}, {756}, + {741}, {787}, {721}, {745}, {782}, + {765}, {780}, {811}, {790}, {836}, + {743}, {858}, {739}, {762}, {770}, + {752}, {763}, {795}, {792}, {746}, + {786}, {785}, {774}, {786}, {718}, +} + +func TestRoundtrip(t *testing.T) { + + s := pd1.NewFloatEncoder() + for _, p := range TwoHoursData { + s.Push(p.v) + } + s.Finish() + + it := s.FloatDecoder() + for _, w := range TwoHoursData { + if !it.Next() { + t.Fatalf("Next()=false, want true") + } + vv := it.Values() + // t.Logf("it.Values()=(%+v, %+v)\n", time.Unix(int64(tt), 0), vv) + if w.v != vv { + t.Errorf("Values()=(%v), want (%v)\n", vv, w.v) + } + } + + if it.Next() { + t.Fatalf("Next()=true, want false") + } + + if err := it.Err(); err != nil { + t.Errorf("it.Err()=%v, want nil", err) + } +} + +func BenchmarkFloatEncoder(b *testing.B) { + for i := 0; i < b.N; i++ { + s := pd1.NewFloatEncoder() + for _, tt := range TwoHoursData { + s.Push(tt.v) + } + s.Finish() + } +} + +func BenchmarkFloatDecoder(b *testing.B) { + s := pd1.NewFloatEncoder() + for _, tt := range TwoHoursData { + s.Push(tt.v) + } + s.Finish() + + b.ResetTimer() + + for i := 0; i < b.N; i++ { + it := s.FloatDecoder() + for j := 0; j < len(TwoHoursData); it.Next() { + j++ + } + } +} diff --git a/tsdb/engine/pd1/timestamp.go b/tsdb/engine/pd1/timestamp.go new file mode 100644 index 00000000000..b83199ef043 --- /dev/null +++ b/tsdb/engine/pd1/timestamp.go @@ -0,0 +1,286 @@ +// Package timestamp provides structs and functions for converting streams of timestamps +// to byte slices. +// +// The encoding is adapative based on structure of the timestamps that are encoded. By default, +// a bit-packed format that compresses multiple 64bit timestamps into a single 64bit word is used. +// If the values are too large to be compressed using the bit-packed format, it will fall back to +// a raw 8byte per timestamp format. If the the values can be run-length encoded, based on the +// differences between consectutive values, a shorter, variable sized RLE format is used. +package pd1 + +import ( + "encoding/binary" + "math" + "time" + + "github.com/jwilder/encoding/simple8b" +) + +const ( + // EncodingPacked is a bit-packed format + EncodingPacked = 0 + // EncodingRLE is a run-length encoded format + EncodingRLE = 1 + // EncodingRAW is a non-compressed format + EncodingRaw = 2 +) + +// TimeEncoder encodes time.Time to byte slices. +type TimeEncoder interface { + Write(t time.Time) + Bytes() ([]byte, error) +} + +// TimeEncoder decodes byte slices to time.Time values. +type TimeDecoder interface { + Next() bool + Read() time.Time +} + +type encoder struct { + ts []int64 +} + +// NewTimeEncoder returns a TimeEncoder +func NewTimeEncoder() TimeEncoder { + return &encoder{} +} + +// Write adds a time.Time to the compressed stream. +func (e *encoder) Write(t time.Time) { + e.ts = append(e.ts, t.UnixNano()) +} + +func (e *encoder) reduce() (min, max, divisor int64, rle bool, deltas []int64) { + // We make a copy of the timestamps so that if we end up using using RAW encoding, + // we still have the original values to encode. + deltas = make([]int64, len(e.ts)) + copy(deltas, e.ts) + + // Starting values for a min, max and divisor + min, max, divisor = e.ts[0], 0, 1e12 + + // First differential encode the values in place + for i := len(deltas) - 1; i > 0; i-- { + deltas[i] = deltas[i] - deltas[i-1] + + // We also want to keep track of the min, max and divisor so we don't + // have to loop again + v := deltas[i] + if v < min { + min = v + } + + if v > max { + max = v + } + + for { + // If our value is divisible by 10, break. Otherwise, try the next smallest divisor. + if v%divisor == 0 { + break + } + divisor /= 10 + } + } + + // Are the deltas able to be run-length encoded? + rle = true + for i := 1; i < len(deltas); i++ { + deltas[i] = (deltas[i] - min) / divisor + // Skip the first value || see if prev = curr. The deltas can be RLE if the are all equal. + rle = i == 1 || rle && (deltas[i-1] == deltas[i]) + } + + // No point RLE encoding 1 value + rle = rle && len(deltas) > 1 + return +} + +// Bytes returns the encoded bytes of all written times. +func (e *encoder) Bytes() ([]byte, error) { + if len(e.ts) == 0 { + return []byte{}, nil + } + + // Minimum, maxim and largest common divisor. rle is true if dts (the delta timestamps), + // are all the same. + min, max, div, rle, dts := e.reduce() + + // The deltas are all the same, so we can run-length encode them + if rle && len(e.ts) > 60 { + return e.encodeRLE(e.ts[0], e.ts[1]-e.ts[0], div, len(e.ts)) + } + + // We can't compress this time-range, the deltas exceed 1 << 60. That would mean that two + // adjacent timestamps are nanosecond resolution and ~36.5yr apart. + if max > simple8b.MaxValue { + return e.encodeRaw() + } + + // Otherwise, encode them in a compressed format + return e.encodePacked(min, div, dts) +} + +func (e *encoder) encodePacked(min, div int64, dts []int64) ([]byte, error) { + enc := simple8b.NewEncoder() + for _, v := range dts[1:] { + enc.Write(uint64(v)) + } + + b := make([]byte, 8*2+1) + + // 4 high bits used for the encoding type + b[0] = byte(EncodingPacked) << 4 + // 4 low bits are the log10 divisor + b[0] |= byte(math.Log10(float64(div))) + + // The minimum timestamp value + binary.BigEndian.PutUint64(b[1:9], uint64(min)) + + // The first delta value + binary.BigEndian.PutUint64(b[9:17], uint64(dts[0])) + + // The compressed deltas + deltas, err := enc.Bytes() + if err != nil { + return nil, err + } + + return append(b, deltas...), nil +} + +func (e *encoder) encodeRaw() ([]byte, error) { + b := make([]byte, 1+len(e.ts)*8) + b[0] = byte(EncodingRaw) << 4 + for i, v := range e.ts { + binary.BigEndian.PutUint64(b[1+i*8:1+i*8+8], uint64(v)) + } + return b, nil +} + +func (e *encoder) encodeRLE(first, delta, div int64, n int) ([]byte, error) { + // Large varints can take up to 10 bytes + b := make([]byte, 1+10*3) + + // 4 high bits used for the encoding type + b[0] = byte(EncodingRLE) << 4 + // 4 low bits are the log10 divisor + b[0] |= byte(math.Log10(float64(div))) + + i := 1 + // The first timestamp + binary.BigEndian.PutUint64(b[i:], uint64(first)) + i += 8 + // The first delta + i += binary.PutUvarint(b[i:], uint64(delta/div)) + // The number of times the delta is repeated + i += binary.PutUvarint(b[i:], uint64(n)) + + return b[:i], nil +} + +type decoder struct { + v time.Time + ts []int64 +} + +func NewTimeDecoder(b []byte) TimeDecoder { + d := &decoder{} + d.decode(b) + return d +} + +func (d *decoder) Next() bool { + if len(d.ts) == 0 { + return false + } + d.v = time.Unix(0, d.ts[0]) + d.ts = d.ts[1:] + return true +} + +func (d *decoder) Read() time.Time { + return d.v +} + +func (d *decoder) decode(b []byte) { + if len(b) == 0 { + return + } + + // Encoding type is stored in the 4 high bits of the first byte + encoding := b[0] >> 4 + switch encoding { + case EncodingRaw: + d.decodeRaw(b[1:]) + case EncodingRLE: + d.decodeRLE(b) + default: + d.decodePacked(b) + } +} + +func (d *decoder) decodePacked(b []byte) { + div := int64(math.Pow10(int(b[0] & 0xF))) + min := int64(binary.BigEndian.Uint64(b[1:9])) + first := int64(binary.BigEndian.Uint64(b[9:17])) + + enc := simple8b.NewDecoder(b[17:]) + + deltas := []int64{first} + for enc.Next() { + deltas = append(deltas, int64(enc.Read())) + } + + // Compute the prefix sum and scale the deltas back up + for i := 1; i < len(deltas); i++ { + deltas[i] = (deltas[i] * div) + min + deltas[i] = deltas[i-1] + deltas[i] + } + + d.ts = deltas +} + +func (d *decoder) decodeRLE(b []byte) { + var i, n int + + // Lower 4 bits hold the 10 based exponent so we can scale the values back up + div := int64(math.Pow10(int(b[i] & 0xF))) + i += 1 + + // Next 8 bytes is the starting timestamp + first := binary.BigEndian.Uint64(b[i : i+8]) + i += 8 + + // Next 1-10 bytes is our (scaled down by factor of 10) run length values + value, n := binary.Uvarint(b[i:]) + + // Scale the value back up + value *= uint64(div) + i += n + + // Last 1-10 bytes is how many times the value repeats + count, n := binary.Uvarint(b[i:]) + + // Rebuild construct the original values now + deltas := make([]int64, count) + for i := range deltas { + deltas[i] = int64(value) + } + + // Reverse the delta-encoding + deltas[0] = int64(first) + for i := 1; i < len(deltas); i++ { + deltas[i] = deltas[i-1] + deltas[i] + } + + d.ts = deltas +} + +func (d *decoder) decodeRaw(b []byte) { + d.ts = make([]int64, len(b)/8) + for i := range d.ts { + d.ts[i] = int64(binary.BigEndian.Uint64(b[i*8 : i*8+8])) + } +} diff --git a/tsdb/engine/pd1/timestamp_test.go b/tsdb/engine/pd1/timestamp_test.go new file mode 100644 index 00000000000..806fba7c5eb --- /dev/null +++ b/tsdb/engine/pd1/timestamp_test.go @@ -0,0 +1,353 @@ +package pd1_test + +import ( + "testing" + "time" + + "github.com/influxdb/influxdb/tsdb/engine/pd1" +) + +func Test_TimeEncoder(t *testing.T) { + enc := pd1.NewTimeEncoder() + + x := []time.Time{} + now := time.Unix(0, 0) + x = append(x, now) + enc.Write(now) + for i := 1; i < 4; i++ { + x = append(x, now.Add(time.Duration(i)*time.Second)) + enc.Write(x[i]) + } + + b, err := enc.Bytes() + if err != nil { + t.Fatalf("unexpected error: %v", err) + } + + dec := pd1.NewTimeDecoder(b) + for i, v := range x { + if !dec.Next() { + t.Fatalf("Next == false, expected true") + } + + if v != dec.Read() { + t.Fatalf("Item %d mismatch, got %v, exp %v", i, dec.Read(), v) + } + } +} + +func Test_TimeEncoder_NoValues(t *testing.T) { + enc := pd1.NewTimeEncoder() + b, err := enc.Bytes() + if err != nil { + t.Fatalf("unexpected error: %v", err) + } + + dec := pd1.NewTimeDecoder(b) + if dec.Next() { + t.Fatalf("unexpected next value: got true, exp false") + } +} + +func Test_TimeEncoder_One(t *testing.T) { + enc := pd1.NewTimeEncoder() + tm := time.Unix(0, 0) + + enc.Write(tm) + b, err := enc.Bytes() + if err != nil { + t.Fatalf("unexpected error: %v", err) + } + + dec := pd1.NewTimeDecoder(b) + if !dec.Next() { + t.Fatalf("unexpected next value: got true, exp false") + } + + if tm != dec.Read() { + t.Fatalf("read value mismatch: got %v, exp %v", dec.Read(), tm) + } +} + +func Test_TimeEncoder_Two(t *testing.T) { + enc := pd1.NewTimeEncoder() + t1 := time.Unix(0, 0) + t2 := time.Unix(0, 1) + enc.Write(t1) + enc.Write(t2) + + b, err := enc.Bytes() + if err != nil { + t.Fatalf("unexpected error: %v", err) + } + + dec := pd1.NewTimeDecoder(b) + if !dec.Next() { + t.Fatalf("unexpected next value: got true, exp false") + } + + if t1 != dec.Read() { + t.Fatalf("read value mismatch: got %v, exp %v", dec.Read(), t1) + } + + if !dec.Next() { + t.Fatalf("unexpected next value: got true, exp false") + } + + if t2 != dec.Read() { + t.Fatalf("read value mismatch: got %v, exp %v", dec.Read(), t2) + } +} + +func Test_TimeEncoder_Three(t *testing.T) { + enc := pd1.NewTimeEncoder() + t1 := time.Unix(0, 0) + t2 := time.Unix(0, 1) + t3 := time.Unix(0, 2) + + enc.Write(t1) + enc.Write(t2) + enc.Write(t3) + + b, err := enc.Bytes() + if err != nil { + t.Fatalf("unexpected error: %v", err) + } + + dec := pd1.NewTimeDecoder(b) + if !dec.Next() { + t.Fatalf("unexpected next value: got true, exp false") + } + + if t1 != dec.Read() { + t.Fatalf("read value mismatch: got %v, exp %v", dec.Read(), t1) + } + + if !dec.Next() { + t.Fatalf("unexpected next value: got true, exp false") + } + + if t2 != dec.Read() { + t.Fatalf("read value mismatch: got %v, exp %v", dec.Read(), t2) + } + + if !dec.Next() { + t.Fatalf("unexpected next value: got true, exp false") + } + + if t3 != dec.Read() { + t.Fatalf("read value mismatch: got %v, exp %v", dec.Read(), t3) + } +} + +func Test_TimeEncoder_Large_Range(t *testing.T) { + enc := pd1.NewTimeEncoder() + t1 := time.Unix(0, 1442369134000000000) + t2 := time.Unix(0, 1442369135000000000) + enc.Write(t1) + enc.Write(t2) + b, err := enc.Bytes() + if err != nil { + t.Fatalf("unexpected error: %v", err) + } + + dec := pd1.NewTimeDecoder(b) + if !dec.Next() { + t.Fatalf("unexpected next value: got true, exp false") + } + + if t1 != dec.Read() { + t.Fatalf("read value mismatch: got %v, exp %v", dec.Read(), t1) + } + + if !dec.Next() { + t.Fatalf("unexpected next value: got true, exp false") + } + + if t2 != dec.Read() { + t.Fatalf("read value mismatch: got %v, exp %v", dec.Read(), t2) + } +} + +func Test_TimeEncoder_Raw(t *testing.T) { + enc := pd1.NewTimeEncoder() + t1 := time.Unix(0, 0) + t2 := time.Unix(1, 0) + + // about 36.5yrs in NS resolution is max range for compressed format + // This should cause the encoding to fallback to raw points + t3 := time.Unix(2, (2 << 59)) + enc.Write(t1) + enc.Write(t2) + enc.Write(t3) + + b, err := enc.Bytes() + if err != nil { + t.Fatalf("expected error: %v", err) + } + + if exp := 25; len(b) != exp { + t.Fatalf("length mismatch: got %v, exp %v", len(b), exp) + } + + dec := pd1.NewTimeDecoder(b) + if !dec.Next() { + t.Fatalf("unexpected next value: got true, exp false") + } + + if t1 != dec.Read() { + t.Fatalf("read value mismatch: got %v, exp %v", dec.Read(), t1) + } + + if !dec.Next() { + t.Fatalf("unexpected next value: got true, exp false") + } + + if t2 != dec.Read() { + t.Fatalf("read value mismatch: got %v, exp %v", dec.Read(), t2) + } + + if !dec.Next() { + t.Fatalf("unexpected next value: got true, exp false") + } + + if t3 != dec.Read() { + t.Fatalf("read value mismatch: got %v, exp %v", dec.Read(), t3) + } +} + +func Test_TimeEncoder_RLE(t *testing.T) { + enc := pd1.NewTimeEncoder() + var ts []time.Time + for i := 0; i < 500; i++ { + ts = append(ts, time.Unix(int64(i), 0)) + } + + for _, v := range ts { + enc.Write(v) + } + + b, err := enc.Bytes() + if exp := 12; len(b) != exp { + t.Fatalf("length mismatch: got %v, exp %v", len(b), exp) + } + + if err != nil { + t.Fatalf("unexpected error: %v", err) + } + + dec := pd1.NewTimeDecoder(b) + for i, v := range ts { + if !dec.Next() { + t.Fatalf("Next == false, expected true") + } + + if v != dec.Read() { + t.Fatalf("Item %d mismatch, got %v, exp %v", i, dec.Read(), v) + } + } + + if dec.Next() { + t.Fatalf("unexpected extra values") + } +} + +func Test_TimeEncoder_Reverse(t *testing.T) { + enc := pd1.NewTimeEncoder() + ts := []time.Time{ + time.Unix(0, 3), + time.Unix(0, 2), + time.Unix(0, 1), + } + + for _, v := range ts { + enc.Write(v) + } + + b, err := enc.Bytes() + if err != nil { + t.Fatalf("unexpected error: %v", err) + } + + dec := pd1.NewTimeDecoder(b) + i := 0 + for dec.Next() { + if ts[i] != dec.Read() { + t.Fatalf("read value %d mismatch: got %v, exp %v", i, dec.Read(), ts[i]) + } + i += 1 + } +} + +func Test_TimeEncoder_220SecondDelta(t *testing.T) { + enc := pd1.NewTimeEncoder() + var ts []time.Time + for i := 0; i < 220; i++ { + ts = append(ts, time.Unix(int64(i), 0)) + } + + for _, v := range ts { + enc.Write(v) + } + + b, err := enc.Bytes() + if err != nil { + t.Fatalf("unexpected error: %v", err) + } + + // Using RLE, should get 12 bytes + if exp := 12; len(b) != exp { + t.Fatalf("unexpected length: got %v, exp %v", len(b), exp) + } + + dec := pd1.NewTimeDecoder(b) + i := 0 + for dec.Next() { + if ts[i] != dec.Read() { + t.Fatalf("read value %d mismatch: got %v, exp %v", i, dec.Read(), ts[i]) + } + i += 1 + } + + if i != len(ts) { + t.Fatalf("Read too few values: exp %d, got %d", len(ts), i) + } + + if dec.Next() { + t.Fatalf("expecte Next() = false, got true") + } +} + +func BenchmarkTimeEncoder(b *testing.B) { + enc := pd1.NewTimeEncoder() + x := make([]time.Time, 1024) + for i := 0; i < len(x); i++ { + x[i] = time.Now() + enc.Write(x[i]) + } + + b.ResetTimer() + for i := 0; i < b.N; i++ { + enc.Bytes() + } +} + +func BenchmarkTimeDecoder(b *testing.B) { + x := make([]time.Time, 1024) + enc := pd1.NewTimeEncoder() + for i := 0; i < len(x); i++ { + x[i] = time.Now() + enc.Write(x[i]) + } + bytes, _ := enc.Bytes() + + b.ResetTimer() + + for i := 0; i < b.N; i++ { + b.StopTimer() + dec := pd1.NewTimeDecoder(bytes) + b.StartTimer() + for dec.Next() { + } + } +} From 4a37ba868d6a126c67569d8c3c42a55ed6c1d1fd Mon Sep 17 00:00:00 2001 From: Jason Wilder Date: Thu, 24 Sep 2015 14:29:51 -0600 Subject: [PATCH 097/139] Add int64 compression This is using zig zag encoding to convert int64 to uint64s and then using simple8b to compress them, falling back to uncompressed if the value exceeds 1 << 60. A patched encoding scheme would likely be better in general but this provides decent compression for integers that are not at the ends of the int64 range. --- tsdb/engine/pd1/encoding.go | 86 ++++++++++- tsdb/engine/pd1/encoding_test.go | 40 +++++ tsdb/engine/pd1/int.go | 113 +++++++++++++++ tsdb/engine/pd1/int_test.go | 241 +++++++++++++++++++++++++++++++ tsdb/engine/pd1/timestamp.go | 18 +-- 5 files changed, 480 insertions(+), 18 deletions(-) create mode 100644 tsdb/engine/pd1/int.go create mode 100644 tsdb/engine/pd1/int_test.go diff --git a/tsdb/engine/pd1/encoding.go b/tsdb/engine/pd1/encoding.go index 6b29913e0e0..055048620ce 100644 --- a/tsdb/engine/pd1/encoding.go +++ b/tsdb/engine/pd1/encoding.go @@ -1,12 +1,23 @@ package pd1 import ( + "encoding/binary" + "fmt" "sort" "time" "github.com/influxdb/influxdb/tsdb" ) +const ( + // EncodingPacked is a bit-packed format + EncodingPacked = 0 + // EncodingRLE is a run-length encoded format + EncodingRLE = 1 + // EncodingUncompressed is a non-compressed format + EncodingUncompressed = 2 +) + type Value interface { Time() time.Time UnixNano() int64 @@ -16,8 +27,8 @@ type Value interface { func NewValue(t time.Time, value interface{}) Value { switch v := value.(type) { - // case int64: - // return &Int64Value{time: t, value: v} + case int64: + return &Int64Value{time: t, value: v} case float64: return &FloatValue{time: t, value: v} // case bool: @@ -58,6 +69,13 @@ func (v Values) Encode(buf []byte) []byte { } return EncodeFloatBlock(buf, a) + case *Int64Value: + a := make([]*Int64Value, len(v)) + for i, vv := range v { + a[i] = vv.(*Int64Value) + } + return EncodeInt64Block(buf, a) + // TODO: add support for other types } @@ -69,6 +87,9 @@ func (v Values) DecodeSameTypeBlock(block []byte) Values { case *FloatValue: a, _ := DecodeFloatBlock(block) return a + case *Int64Value: + a, _ := DecodeInt64Block(block) + return a // TODO: add support for other types } @@ -200,12 +221,65 @@ type Int64Value struct { value int64 } -func EncodeInt64Block(buf []byte, values []Int64Value) []byte { - return nil +func (v *Int64Value) Time() time.Time { + return v.time } -func DecodeInt64Block(block []byte) ([]Int64Value, error) { - return nil, nil +func (v *Int64Value) Value() interface{} { + return v.value +} + +func (f *Int64Value) UnixNano() int64 { + return f.time.UnixNano() +} + +func (v *Int64Value) Size() int { + return 16 +} + +func (v *Int64Value) String() string { return fmt.Sprintf("%v", v.value) } + +func EncodeInt64Block(buf []byte, values []*Int64Value) []byte { + tsEnc := NewTimeEncoder() + vEnc := NewInt64Encoder() + for _, v := range values { + tsEnc.Write(v.Time()) + vEnc.Write(v.value) + } + + // Encoded timestamp values + tb, err := tsEnc.Bytes() + if err != nil { + panic(err.Error()) + } + // Encoded int64 values + vb, err := vEnc.Bytes() + if err != nil { + panic(err.Error()) + } + + // Preprend the first timestamp of the block in the first 8 bytes + return append(u64tob(uint64(values[0].Time().UnixNano())), + packBlock(tb, vb)...) +} + +func DecodeInt64Block(block []byte) ([]Value, error) { + // The first 8 bytes is the minimum timestamp of the block + tb, vb := unpackBlock(block[8:]) + + // Setup our timestamp and value decoders + tsDec := NewTimeDecoder(tb) + vDec := NewInt64Decoder(vb) + + // Decode both a timestamp and value + var a []Value + for tsDec.Next() && vDec.Next() { + ts := tsDec.Read() + v := vDec.Read() + a = append(a, &Int64Value{ts, v}) + } + + return a, nil } type StringValue struct { diff --git a/tsdb/engine/pd1/encoding_test.go b/tsdb/engine/pd1/encoding_test.go index 02598a764da..49006085d7d 100644 --- a/tsdb/engine/pd1/encoding_test.go +++ b/tsdb/engine/pd1/encoding_test.go @@ -1,6 +1,8 @@ package pd1_test import ( + // "math/rand" + "reflect" "testing" "time" @@ -25,6 +27,44 @@ func TestEncoding_FloatBlock(t *testing.T) { } } +func TestEncoding_IntBlock(t *testing.T) { + valueCount := 1000 + times := getTimes(valueCount, 60, time.Second) + values := make(pd1.Values, len(times)) + for i, t := range times { + values[i] = pd1.NewValue(t, int64(i)) + } + + b := values.Encode(nil) + + decodedValues := values.DecodeSameTypeBlock(b) + + if !reflect.DeepEqual(decodedValues, values) { + t.Fatalf("unexpected results:\n\tgot: %v\n\texp: %v\n", decodedValues, values) + } +} + +func TestEncoding_IntBlock_Negatives(t *testing.T) { + valueCount := 1000 + times := getTimes(valueCount, 60, time.Second) + values := make(pd1.Values, len(times)) + for i, t := range times { + v := int64(i) + if i%2 == 0 { + v = -v + } + values[i] = pd1.NewValue(t, int64(v)) + } + + b := values.Encode(nil) + + decodedValues := values.DecodeSameTypeBlock(b) + + if !reflect.DeepEqual(decodedValues, values) { + t.Fatalf("unexpected results:\n\tgot: %v\n\texp: %v\n", decodedValues, values) + } +} + func getTimes(n, step int, precision time.Duration) []time.Time { t := time.Now().Round(precision) a := make([]time.Time, n) diff --git a/tsdb/engine/pd1/int.go b/tsdb/engine/pd1/int.go new file mode 100644 index 00000000000..04ddc1a0f4f --- /dev/null +++ b/tsdb/engine/pd1/int.go @@ -0,0 +1,113 @@ +package pd1 + +import ( + "encoding/binary" + "fmt" + + "github.com/jwilder/encoding/simple8b" +) + +type int64Encoder struct { + values []int64 +} + +func NewInt64Encoder() *int64Encoder { + return &int64Encoder{} +} + +func (e *int64Encoder) Write(v int64) { + e.values = append(e.values, v) +} + +func (e *int64Encoder) zigZagEncode(x int64) uint64 { + return uint64(uint64(x<<1) ^ uint64((int64(x) >> 63))) +} + +func (e *int64Encoder) Bytes() ([]byte, error) { + enc := simple8b.NewEncoder() + + for _, v := range e.values { + n := e.zigZagEncode(v) + // Value is too large to encode using packed format + if n > simple8b.MaxValue { + return e.encodeUncompressed() + } + enc.Write(n) + } + + b, err := enc.Bytes() + if err != nil { + return nil, err + } + + return append([]byte{EncodingPacked << 4}, b...), nil +} + +func (e *int64Encoder) encodeUncompressed() ([]byte, error) { + b := make([]byte, 1+len(e.values)*8) + // 4 high bits of first byte store the encoding type for the block + b[0] = byte(EncodingUncompressed) << 4 + for i, v := range e.values { + binary.BigEndian.PutUint64(b[1+i*8:1+i*8+8], uint64(v)) + } + return b, nil +} + +type int64Decoder struct { + values []int64 + v int64 +} + +func NewInt64Decoder(b []byte) *int64Decoder { + d := &int64Decoder{} + d.decode(b) + return d +} + +func (d *int64Decoder) zigZagDecode(v uint64) int64 { + return int64((v >> 1) ^ uint64((int64(v&1)<<63)>>63)) +} + +func (d *int64Decoder) Next() bool { + if len(d.values) == 0 { + return false + } + d.v = d.values[0] + d.values = d.values[1:] + return true +} + +func (d *int64Decoder) Read() int64 { + return d.v +} + +func (d *int64Decoder) decode(b []byte) { + if len(b) == 0 { + return + } + + // Encoding type is stored in the 4 high bits of the first byte + encoding := b[0] >> 4 + switch encoding { + case EncodingUncompressed: + d.decodeUncompressed(b[1:]) + case EncodingPacked: + d.decodePacked(b[1:]) + default: + panic(fmt.Sprintf("unknown encoding %v", encoding)) + } +} + +func (d *int64Decoder) decodePacked(b []byte) { + dec := simple8b.NewDecoder(b) + for dec.Next() { + d.values = append(d.values, d.zigZagDecode(dec.Read())) + } +} + +func (d *int64Decoder) decodeUncompressed(b []byte) { + d.values = make([]int64, len(b)/8) + for i := range d.values { + d.values[i] = int64(binary.BigEndian.Uint64(b[i*8 : i*8+8])) + } +} diff --git a/tsdb/engine/pd1/int_test.go b/tsdb/engine/pd1/int_test.go new file mode 100644 index 00000000000..82042f77bd4 --- /dev/null +++ b/tsdb/engine/pd1/int_test.go @@ -0,0 +1,241 @@ +package pd1_test + +import ( + "math" + "testing" + + "github.com/influxdb/influxdb/tsdb/engine/pd1" +) + +func Test_Int64Encoder_NoValues(t *testing.T) { + enc := pd1.NewInt64Encoder() + b, err := enc.Bytes() + if err != nil { + t.Fatalf("unexpected error: %v", err) + } + + dec := pd1.NewInt64Decoder(b) + if dec.Next() { + t.Fatalf("unexpected next value: got true, exp false") + } +} + +func Test_Int64Encoder_One(t *testing.T) { + enc := pd1.NewInt64Encoder() + v1 := int64(1) + + enc.Write(1) + b, err := enc.Bytes() + if err != nil { + t.Fatalf("unexpected error: %v", err) + } + + dec := pd1.NewInt64Decoder(b) + if !dec.Next() { + t.Fatalf("unexpected next value: got true, exp false") + } + + if v1 != dec.Read() { + t.Fatalf("read value mismatch: got %v, exp %v", dec.Read(), v1) + } +} + +func Test_Int64Encoder_Two(t *testing.T) { + enc := pd1.NewInt64Encoder() + var v1, v2 int64 = 1, 2 + + enc.Write(v1) + enc.Write(v2) + + b, err := enc.Bytes() + if err != nil { + t.Fatalf("unexpected error: %v", err) + } + + dec := pd1.NewInt64Decoder(b) + if !dec.Next() { + t.Fatalf("unexpected next value: got true, exp false") + } + + if v1 != dec.Read() { + t.Fatalf("read value mismatch: got %v, exp %v", dec.Read(), v1) + } + + if !dec.Next() { + t.Fatalf("unexpected next value: got true, exp false") + } + + if v2 != dec.Read() { + t.Fatalf("read value mismatch: got %v, exp %v", dec.Read(), v2) + } +} + +func Test_Int64Encoder_Negative(t *testing.T) { + enc := pd1.NewInt64Encoder() + var v1, v2, v3 int64 = -2, 0, 1 + + enc.Write(v1) + enc.Write(v2) + enc.Write(v3) + + b, err := enc.Bytes() + if err != nil { + t.Fatalf("unexpected error: %v", err) + } + + dec := pd1.NewInt64Decoder(b) + if !dec.Next() { + t.Fatalf("unexpected next value: got true, exp false") + } + + if v1 != dec.Read() { + t.Fatalf("read value mismatch: got %v, exp %v", dec.Read(), v1) + } + + if !dec.Next() { + t.Fatalf("unexpected next value: got true, exp false") + } + + if v2 != dec.Read() { + t.Fatalf("read value mismatch: got %v, exp %v", dec.Read(), v2) + } + + if !dec.Next() { + t.Fatalf("unexpected next value: got true, exp false") + } + + if v3 != dec.Read() { + t.Fatalf("read value mismatch: got %v, exp %v", dec.Read(), v3) + } +} + +func Test_Int64Encoder_Large_Range(t *testing.T) { + enc := pd1.NewInt64Encoder() + var v1, v2 int64 = math.MinInt64, math.MaxInt64 + enc.Write(v1) + enc.Write(v2) + b, err := enc.Bytes() + if err != nil { + t.Fatalf("unexpected error: %v", err) + } + + dec := pd1.NewInt64Decoder(b) + if !dec.Next() { + t.Fatalf("unexpected next value: got true, exp false") + } + + if v1 != dec.Read() { + t.Fatalf("read value mismatch: got %v, exp %v", dec.Read(), v1) + } + + if !dec.Next() { + t.Fatalf("unexpected next value: got true, exp false") + } + + if v2 != dec.Read() { + t.Fatalf("read value mismatch: got %v, exp %v", dec.Read(), v2) + } +} + +func Test_Int64Encoder_Uncompressed(t *testing.T) { + enc := pd1.NewInt64Encoder() + var v1, v2, v3 int64 = 0, 1, 1 << 60 + + enc.Write(v1) + enc.Write(v2) + enc.Write(v3) + + b, err := enc.Bytes() + if err != nil { + t.Fatalf("expected error: %v", err) + } + + // 1 byte header + 3 * 8 byte values + if exp := 25; len(b) != exp { + t.Fatalf("length mismatch: got %v, exp %v", len(b), exp) + } + + dec := pd1.NewInt64Decoder(b) + if !dec.Next() { + t.Fatalf("unexpected next value: got true, exp false") + } + + if v1 != dec.Read() { + t.Fatalf("read value mismatch: got %v, exp %v", dec.Read(), v1) + } + + if !dec.Next() { + t.Fatalf("unexpected next value: got true, exp false") + } + + if v2 != dec.Read() { + t.Fatalf("read value mismatch: got %v, exp %v", dec.Read(), v2) + } + + if !dec.Next() { + t.Fatalf("unexpected next value: got true, exp false") + } + + if v3 != dec.Read() { + t.Fatalf("read value mismatch: got %v, exp %v", dec.Read(), v3) + } +} + +func Test_Int64Encoder_AllNegative(t *testing.T) { + enc := pd1.NewInt64Encoder() + values := []int64{ + -10, -5, -1, + } + + for _, v := range values { + enc.Write(v) + } + + b, err := enc.Bytes() + if err != nil { + t.Fatalf("unexpected error: %v", err) + } + + dec := pd1.NewInt64Decoder(b) + i := 0 + for dec.Next() { + if values[i] != dec.Read() { + t.Fatalf("read value %d mismatch: got %v, exp %v", i, dec.Read(), values[i]) + } + i += 1 + } +} + +func BenchmarkInt64Encoder(b *testing.B) { + enc := pd1.NewInt64Encoder() + x := make([]int64, 1024) + for i := 0; i < len(x); i++ { + x[i] = int64(i) + enc.Write(x[i]) + } + + b.ResetTimer() + for i := 0; i < b.N; i++ { + enc.Bytes() + } +} + +func BenchmarkInt64Decoder(b *testing.B) { + x := make([]int64, 1024) + enc := pd1.NewInt64Encoder() + for i := 0; i < len(x); i++ { + x[i] = int64(i) + enc.Write(x[i]) + } + bytes, _ := enc.Bytes() + + b.ResetTimer() + + for i := 0; i < b.N; i++ { + b.StopTimer() + dec := pd1.NewInt64Decoder(bytes) + b.StartTimer() + for dec.Next() { + } + } +} diff --git a/tsdb/engine/pd1/timestamp.go b/tsdb/engine/pd1/timestamp.go index b83199ef043..64907bb7607 100644 --- a/tsdb/engine/pd1/timestamp.go +++ b/tsdb/engine/pd1/timestamp.go @@ -10,21 +10,13 @@ package pd1 import ( "encoding/binary" + "fmt" "math" "time" "github.com/jwilder/encoding/simple8b" ) -const ( - // EncodingPacked is a bit-packed format - EncodingPacked = 0 - // EncodingRLE is a run-length encoded format - EncodingRLE = 1 - // EncodingRAW is a non-compressed format - EncodingRaw = 2 -) - // TimeEncoder encodes time.Time to byte slices. type TimeEncoder interface { Write(t time.Time) @@ -152,7 +144,7 @@ func (e *encoder) encodePacked(min, div int64, dts []int64) ([]byte, error) { func (e *encoder) encodeRaw() ([]byte, error) { b := make([]byte, 1+len(e.ts)*8) - b[0] = byte(EncodingRaw) << 4 + b[0] = byte(EncodingUncompressed) << 4 for i, v := range e.ts { binary.BigEndian.PutUint64(b[1+i*8:1+i*8+8], uint64(v)) } @@ -212,12 +204,14 @@ func (d *decoder) decode(b []byte) { // Encoding type is stored in the 4 high bits of the first byte encoding := b[0] >> 4 switch encoding { - case EncodingRaw: + case EncodingUncompressed: d.decodeRaw(b[1:]) case EncodingRLE: d.decodeRLE(b) - default: + case EncodingPacked: d.decodePacked(b) + default: + panic(fmt.Sprintf("unknown encoding: %v", encoding)) } } From ce1d45ecdad4287077f559755b02aeca3a32c638 Mon Sep 17 00:00:00 2001 From: Jason Wilder Date: Thu, 24 Sep 2015 16:42:48 -0600 Subject: [PATCH 098/139] Use zigzag encoding for timestamp deltas Previously were using a frame of reference approach where we would transform the (possibly negative) deltas into positive values from the minimum. That required an extra pass over the values as well as a large slice allocation so we could encode the originals in uncompressed form if they were too large. This switches the encoding to use zigzag encoding for the deltas which removes the extra slice allocation as well as the extra loops. Improves encoding performane by ~4x. --- tsdb/engine/pd1/encoding.go | 11 +++ tsdb/engine/pd1/encoding_test.go | 19 ++++-- tsdb/engine/pd1/int.go | 12 +--- tsdb/engine/pd1/timestamp.go | 109 ++++++++++++++---------------- tsdb/engine/pd1/timestamp_test.go | 3 +- 5 files changed, 82 insertions(+), 72 deletions(-) diff --git a/tsdb/engine/pd1/encoding.go b/tsdb/engine/pd1/encoding.go index 055048620ce..a2e16699456 100644 --- a/tsdb/engine/pd1/encoding.go +++ b/tsdb/engine/pd1/encoding.go @@ -316,3 +316,14 @@ func unpackBlock(buf []byte) (ts, values []byte) { values = buf[int(i)+int(tsLen):] return } + +// ZigZagEncode converts a int64 to a uint64 by zig zagging negative and positive values +// across even and odd numbers. Eg. [0,-1,1,-2] becomes [0, 1, 2, 3] +func ZigZagEncode(x int64) uint64 { + return uint64(uint64(x<<1) ^ uint64((int64(x) >> 63))) +} + +// ZigZagDecode converts a previously zigzag encoded uint64 back to a int64 +func ZigZagDecode(v uint64) int64 { + return int64((v >> 1) ^ uint64((int64(v&1)<<63)>>63)) +} diff --git a/tsdb/engine/pd1/encoding_test.go b/tsdb/engine/pd1/encoding_test.go index 49006085d7d..c249f1aa8ce 100644 --- a/tsdb/engine/pd1/encoding_test.go +++ b/tsdb/engine/pd1/encoding_test.go @@ -27,7 +27,7 @@ func TestEncoding_FloatBlock(t *testing.T) { } } -func TestEncoding_IntBlock(t *testing.T) { +func TestEncoding_IntBlock_Basic(t *testing.T) { valueCount := 1000 times := getTimes(valueCount, 60, time.Second) values := make(pd1.Values, len(times)) @@ -39,8 +39,19 @@ func TestEncoding_IntBlock(t *testing.T) { decodedValues := values.DecodeSameTypeBlock(b) - if !reflect.DeepEqual(decodedValues, values) { - t.Fatalf("unexpected results:\n\tgot: %v\n\texp: %v\n", decodedValues, values) + if len(decodedValues) != len(values) { + t.Fatalf("unexpected results length:\n\tgot: %v\n\texp: %v\n", len(decodedValues), len(values)) + } + + for i := 0; i < len(decodedValues); i++ { + + if decodedValues[i].Time() != values[i].Time() { + t.Fatalf("unexpected results:\n\tgot: %v\n\texp: %v\n", decodedValues[i].Time(), values[i].Time()) + } + + if decodedValues[i].Value() != values[i].Value() { + t.Fatalf("unexpected results:\n\tgot: %v\n\texp: %v\n", decodedValues[i].Value(), values[i].Value()) + } } } @@ -69,7 +80,7 @@ func getTimes(n, step int, precision time.Duration) []time.Time { t := time.Now().Round(precision) a := make([]time.Time, n) for i := 0; i < n; i++ { - a[i] = t.Add(60 * precision) + a[i] = t.Add(time.Duration(i*60) * precision) } return a } diff --git a/tsdb/engine/pd1/int.go b/tsdb/engine/pd1/int.go index 04ddc1a0f4f..c4c40b8778d 100644 --- a/tsdb/engine/pd1/int.go +++ b/tsdb/engine/pd1/int.go @@ -19,15 +19,11 @@ func (e *int64Encoder) Write(v int64) { e.values = append(e.values, v) } -func (e *int64Encoder) zigZagEncode(x int64) uint64 { - return uint64(uint64(x<<1) ^ uint64((int64(x) >> 63))) -} - func (e *int64Encoder) Bytes() ([]byte, error) { enc := simple8b.NewEncoder() for _, v := range e.values { - n := e.zigZagEncode(v) + n := ZigZagEncode(v) // Value is too large to encode using packed format if n > simple8b.MaxValue { return e.encodeUncompressed() @@ -64,10 +60,6 @@ func NewInt64Decoder(b []byte) *int64Decoder { return d } -func (d *int64Decoder) zigZagDecode(v uint64) int64 { - return int64((v >> 1) ^ uint64((int64(v&1)<<63)>>63)) -} - func (d *int64Decoder) Next() bool { if len(d.values) == 0 { return false @@ -101,7 +93,7 @@ func (d *int64Decoder) decode(b []byte) { func (d *int64Decoder) decodePacked(b []byte) { dec := simple8b.NewDecoder(b) for dec.Next() { - d.values = append(d.values, d.zigZagDecode(dec.Read())) + d.values = append(d.values, ZigZagDecode(dec.Read())) } } diff --git a/tsdb/engine/pd1/timestamp.go b/tsdb/engine/pd1/timestamp.go index 64907bb7607..9119dbd5182 100644 --- a/tsdb/engine/pd1/timestamp.go +++ b/tsdb/engine/pd1/timestamp.go @@ -30,7 +30,7 @@ type TimeDecoder interface { } type encoder struct { - ts []int64 + ts []uint64 } // NewTimeEncoder returns a TimeEncoder @@ -40,28 +40,29 @@ func NewTimeEncoder() TimeEncoder { // Write adds a time.Time to the compressed stream. func (e *encoder) Write(t time.Time) { - e.ts = append(e.ts, t.UnixNano()) + e.ts = append(e.ts, uint64(t.UnixNano())) } -func (e *encoder) reduce() (min, max, divisor int64, rle bool, deltas []int64) { - // We make a copy of the timestamps so that if we end up using using RAW encoding, - // we still have the original values to encode. - deltas = make([]int64, len(e.ts)) - copy(deltas, e.ts) +func (e *encoder) reduce() (max, divisor uint64, rle bool, deltas []uint64) { + // Compute the deltas in place to avoid allocating another slice + deltas = e.ts + // Starting values for a max and divisor + max, divisor = 0, 1e12 - // Starting values for a min, max and divisor - min, max, divisor = e.ts[0], 0, 1e12 + // Indicates whether the the deltas can be run-length encoded + rle = true - // First differential encode the values in place + // Interate in reverse so we can apply deltas in place for i := len(deltas) - 1; i > 0; i-- { - deltas[i] = deltas[i] - deltas[i-1] - // We also want to keep track of the min, max and divisor so we don't - // have to loop again + // First differential encode the values + delta := int64(deltas[i] - deltas[i-1]) + + // The delta may be negative so zigzag encode it into a postive value + deltas[i] = ZigZagEncode(delta) + + // We're also need to keep track of the max value and largest common divisor v := deltas[i] - if v < min { - min = v - } if v > max { max = v @@ -74,18 +75,10 @@ func (e *encoder) reduce() (min, max, divisor int64, rle bool, deltas []int64) { } divisor /= 10 } - } - // Are the deltas able to be run-length encoded? - rle = true - for i := 1; i < len(deltas); i++ { - deltas[i] = (deltas[i] - min) / divisor // Skip the first value || see if prev = curr. The deltas can be RLE if the are all equal. - rle = i == 1 || rle && (deltas[i-1] == deltas[i]) + rle = i != 0 || rle && (deltas[i-1] == deltas[i]) } - - // No point RLE encoding 1 value - rle = rle && len(deltas) > 1 return } @@ -95,43 +88,38 @@ func (e *encoder) Bytes() ([]byte, error) { return []byte{}, nil } - // Minimum, maxim and largest common divisor. rle is true if dts (the delta timestamps), + // Maximum and largest common divisor. rle is true if dts (the delta timestamps), // are all the same. - min, max, div, rle, dts := e.reduce() + max, mod, rle, dts := e.reduce() // The deltas are all the same, so we can run-length encode them if rle && len(e.ts) > 60 { - return e.encodeRLE(e.ts[0], e.ts[1]-e.ts[0], div, len(e.ts)) + return e.encodeRLE(e.ts[0], e.ts[1], mod, len(e.ts)) } - // We can't compress this time-range, the deltas exceed 1 << 60. That would mean that two - // adjacent timestamps are nanosecond resolution and ~36.5yr apart. + // We can't compress this time-range, the deltas exceed 1 << 60 if max > simple8b.MaxValue { return e.encodeRaw() } - // Otherwise, encode them in a compressed format - return e.encodePacked(min, div, dts) + return e.encodePacked(mod, dts) } -func (e *encoder) encodePacked(min, div int64, dts []int64) ([]byte, error) { +func (e *encoder) encodePacked(div uint64, dts []uint64) ([]byte, error) { enc := simple8b.NewEncoder() for _, v := range dts[1:] { - enc.Write(uint64(v)) + enc.Write(uint64(v) / div) } - b := make([]byte, 8*2+1) + b := make([]byte, 8+1) // 4 high bits used for the encoding type b[0] = byte(EncodingPacked) << 4 // 4 low bits are the log10 divisor b[0] |= byte(math.Log10(float64(div))) - // The minimum timestamp value - binary.BigEndian.PutUint64(b[1:9], uint64(min)) - // The first delta value - binary.BigEndian.PutUint64(b[9:17], uint64(dts[0])) + binary.BigEndian.PutUint64(b[1:9], uint64(dts[0])) // The compressed deltas deltas, err := enc.Bytes() @@ -151,7 +139,7 @@ func (e *encoder) encodeRaw() ([]byte, error) { return b, nil } -func (e *encoder) encodeRLE(first, delta, div int64, n int) ([]byte, error) { +func (e *encoder) encodeRLE(first, delta, div uint64, n int) ([]byte, error) { // Large varints can take up to 10 bytes b := make([]byte, 1+10*3) @@ -174,7 +162,7 @@ func (e *encoder) encodeRLE(first, delta, div int64, n int) ([]byte, error) { type decoder struct { v time.Time - ts []int64 + ts []uint64 } func NewTimeDecoder(b []byte) TimeDecoder { @@ -187,7 +175,7 @@ func (d *decoder) Next() bool { if len(d.ts) == 0 { return false } - d.v = time.Unix(0, d.ts[0]) + d.v = time.Unix(0, int64(d.ts[0])) d.ts = d.ts[1:] return true } @@ -216,21 +204,20 @@ func (d *decoder) decode(b []byte) { } func (d *decoder) decodePacked(b []byte) { - div := int64(math.Pow10(int(b[0] & 0xF))) - min := int64(binary.BigEndian.Uint64(b[1:9])) - first := int64(binary.BigEndian.Uint64(b[9:17])) + div := uint64(math.Pow10(int(b[0] & 0xF))) + first := uint64(binary.BigEndian.Uint64(b[1:9])) - enc := simple8b.NewDecoder(b[17:]) + enc := simple8b.NewDecoder(b[9:]) - deltas := []int64{first} + deltas := []uint64{first} for enc.Next() { - deltas = append(deltas, int64(enc.Read())) + deltas = append(deltas, enc.Read()) } // Compute the prefix sum and scale the deltas back up for i := 1; i < len(deltas); i++ { - deltas[i] = (deltas[i] * div) + min - deltas[i] = deltas[i-1] + deltas[i] + dgap := ZigZagDecode(deltas[i] * div) + deltas[i] = uint64(int64(deltas[i-1]) + dgap) } d.ts = deltas @@ -240,7 +227,7 @@ func (d *decoder) decodeRLE(b []byte) { var i, n int // Lower 4 bits hold the 10 based exponent so we can scale the values back up - div := int64(math.Pow10(int(b[i] & 0xF))) + mod := int64(math.Pow10(int(b[i] & 0xF))) i += 1 // Next 8 bytes is the starting timestamp @@ -250,21 +237,23 @@ func (d *decoder) decodeRLE(b []byte) { // Next 1-10 bytes is our (scaled down by factor of 10) run length values value, n := binary.Uvarint(b[i:]) + value = uint64(ZigZagDecode(value)) + // Scale the value back up - value *= uint64(div) + value *= uint64(mod) i += n // Last 1-10 bytes is how many times the value repeats count, n := binary.Uvarint(b[i:]) // Rebuild construct the original values now - deltas := make([]int64, count) + deltas := make([]uint64, count) for i := range deltas { - deltas[i] = int64(value) + deltas[i] = value } // Reverse the delta-encoding - deltas[0] = int64(first) + deltas[0] = first for i := 1; i < len(deltas); i++ { deltas[i] = deltas[i-1] + deltas[i] } @@ -273,8 +262,14 @@ func (d *decoder) decodeRLE(b []byte) { } func (d *decoder) decodeRaw(b []byte) { - d.ts = make([]int64, len(b)/8) + d.ts = make([]uint64, len(b)/8) for i := range d.ts { - d.ts[i] = int64(binary.BigEndian.Uint64(b[i*8 : i*8+8])) + d.ts[i] = binary.BigEndian.Uint64(b[i*8 : i*8+8]) + + delta := ZigZagDecode(d.ts[i]) + // Compute the prefix sum and scale the deltas back up + if i > 0 { + d.ts[i] = uint64(int64(d.ts[i-1]) + delta) + } } } diff --git a/tsdb/engine/pd1/timestamp_test.go b/tsdb/engine/pd1/timestamp_test.go index 806fba7c5eb..da78b2b6e85 100644 --- a/tsdb/engine/pd1/timestamp_test.go +++ b/tsdb/engine/pd1/timestamp_test.go @@ -282,8 +282,9 @@ func Test_TimeEncoder_Reverse(t *testing.T) { func Test_TimeEncoder_220SecondDelta(t *testing.T) { enc := pd1.NewTimeEncoder() var ts []time.Time + now := time.Now() for i := 0; i < 220; i++ { - ts = append(ts, time.Unix(int64(i), 0)) + ts = append(ts, now.Add(time.Duration(i*60)*time.Second)) } for _, v := range ts { From 092689c131bf04c99c901452e43adc2189ff3e37 Mon Sep 17 00:00:00 2001 From: Jason Wilder Date: Fri, 25 Sep 2015 13:30:24 -0600 Subject: [PATCH 099/139] Reduce memory allocations Converting between different encoders is wasting a lot of memory allocating different typed slices. --- tsdb/engine/pd1/int.go | 82 ++++++++++++++++++++++++++++--------- tsdb/engine/pd1/int_test.go | 10 +++-- 2 files changed, 70 insertions(+), 22 deletions(-) diff --git a/tsdb/engine/pd1/int.go b/tsdb/engine/pd1/int.go index c4c40b8778d..706ca5fce8e 100644 --- a/tsdb/engine/pd1/int.go +++ b/tsdb/engine/pd1/int.go @@ -7,42 +7,62 @@ import ( "github.com/jwilder/encoding/simple8b" ) +type Int64Encoder interface { + Write(v int64) + Bytes() ([]byte, error) +} + +type Int64Decoder interface { + Next() bool + Read() int64 +} + type int64Encoder struct { - values []int64 + values []uint64 } -func NewInt64Encoder() *int64Encoder { +func NewInt64Encoder() Int64Encoder { return &int64Encoder{} } func (e *int64Encoder) Write(v int64) { - e.values = append(e.values, v) + e.values = append(e.values, ZigZagEncode(v)) } func (e *int64Encoder) Bytes() ([]byte, error) { - enc := simple8b.NewEncoder() - for _, v := range e.values { - n := ZigZagEncode(v) // Value is too large to encode using packed format - if n > simple8b.MaxValue { + if v > simple8b.MaxValue { return e.encodeUncompressed() } - enc.Write(n) } - b, err := enc.Bytes() + return e.encodePacked() +} + +func (e *int64Encoder) encodePacked() ([]byte, error) { + encoded, err := simple8b.Encode(e.values) if err != nil { return nil, err } - return append([]byte{EncodingPacked << 4}, b...), nil + b := make([]byte, 1+len(encoded)*8+4) + // 4 high bits of first byte store the encoding type for the block + b[0] = byte(EncodingPacked) << 4 + + binary.BigEndian.PutUint32(b[1:5], uint32(len(e.values))) + + for i, v := range encoded { + binary.BigEndian.PutUint64(b[5+i*8:5+i*8+8], v) + } + return b, nil } func (e *int64Encoder) encodeUncompressed() ([]byte, error) { b := make([]byte, 1+len(e.values)*8) // 4 high bits of first byte store the encoding type for the block b[0] = byte(EncodingUncompressed) << 4 + for i, v := range e.values { binary.BigEndian.PutUint64(b[1+i*8:1+i*8+8], uint64(v)) } @@ -50,21 +70,30 @@ func (e *int64Encoder) encodeUncompressed() ([]byte, error) { } type int64Decoder struct { - values []int64 + values []uint64 v int64 + buf []uint64 + vbuf []uint64 } -func NewInt64Decoder(b []byte) *int64Decoder { - d := &int64Decoder{} +func NewInt64Decoder(b []byte) Int64Decoder { + d := &int64Decoder{ + buf: make([]uint64, 240), + vbuf: make([]uint64, 1), + } d.decode(b) return d } +func (d *int64Decoder) SetBytes(b []byte) { + d.decode(b) +} + func (d *int64Decoder) Next() bool { if len(d.values) == 0 { return false } - d.v = d.values[0] + d.v = ZigZagDecode(d.values[0]) d.values = d.values[1:] return true } @@ -91,15 +120,30 @@ func (d *int64Decoder) decode(b []byte) { } func (d *int64Decoder) decodePacked(b []byte) { - dec := simple8b.NewDecoder(b) - for dec.Next() { - d.values = append(d.values, ZigZagDecode(dec.Read())) + if len(b) == 0 { + return + } + + count := binary.BigEndian.Uint32(b[:4]) + + if count == 0 { + return + } + + d.values = make([]uint64, count) + b = b[4:] + j := 0 + for i := 0; i < len(b); i += 8 { + d.vbuf[0] = binary.BigEndian.Uint64(b[i : i+8]) + n, _ := simple8b.Decode(d.buf, d.vbuf) + copy(d.values[j:], d.buf[:n]) + j += n } } func (d *int64Decoder) decodeUncompressed(b []byte) { - d.values = make([]int64, len(b)/8) + d.values = make([]uint64, len(b)/8) for i := range d.values { - d.values[i] = int64(binary.BigEndian.Uint64(b[i*8 : i*8+8])) + d.values[i] = binary.BigEndian.Uint64(b[i*8 : i*8+8]) } } diff --git a/tsdb/engine/pd1/int_test.go b/tsdb/engine/pd1/int_test.go index 82042f77bd4..063ae05c075 100644 --- a/tsdb/engine/pd1/int_test.go +++ b/tsdb/engine/pd1/int_test.go @@ -220,6 +220,10 @@ func BenchmarkInt64Encoder(b *testing.B) { } } +type byteSetter interface { + SetBytes(b []byte) +} + func BenchmarkInt64Decoder(b *testing.B) { x := make([]int64, 1024) enc := pd1.NewInt64Encoder() @@ -231,10 +235,10 @@ func BenchmarkInt64Decoder(b *testing.B) { b.ResetTimer() + dec := pd1.NewInt64Decoder(bytes) + for i := 0; i < b.N; i++ { - b.StopTimer() - dec := pd1.NewInt64Decoder(bytes) - b.StartTimer() + dec.(byteSetter).SetBytes(bytes) for dec.Next() { } } From e42d8660d07b37cfaf5e4e590a694f41f5e436b8 Mon Sep 17 00:00:00 2001 From: Jason Wilder Date: Fri, 25 Sep 2015 14:19:24 -0600 Subject: [PATCH 100/139] Fix run length encoding check Values were run length encoded even when they should not have been --- tsdb/engine/pd1/timestamp.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tsdb/engine/pd1/timestamp.go b/tsdb/engine/pd1/timestamp.go index 9119dbd5182..04de5b991fb 100644 --- a/tsdb/engine/pd1/timestamp.go +++ b/tsdb/engine/pd1/timestamp.go @@ -77,7 +77,7 @@ func (e *encoder) reduce() (max, divisor uint64, rle bool, deltas []uint64) { } // Skip the first value || see if prev = curr. The deltas can be RLE if the are all equal. - rle = i != 0 || rle && (deltas[i-1] == deltas[i]) + rle = i == len(deltas)-1 || rle && (deltas[i+1] == deltas[i]) } return } From 95046c1e37922a6c62a0ed6643a39c0d4988b786 Mon Sep 17 00:00:00 2001 From: Jason Wilder Date: Fri, 25 Sep 2015 14:25:42 -0600 Subject: [PATCH 101/139] Add test assertions for time encoding type --- tsdb/engine/pd1/timestamp_test.go | 38 ++++++++++++++++++++++++++++++- 1 file changed, 37 insertions(+), 1 deletion(-) diff --git a/tsdb/engine/pd1/timestamp_test.go b/tsdb/engine/pd1/timestamp_test.go index da78b2b6e85..9eb3e0fe2c3 100644 --- a/tsdb/engine/pd1/timestamp_test.go +++ b/tsdb/engine/pd1/timestamp_test.go @@ -24,6 +24,10 @@ func Test_TimeEncoder(t *testing.T) { t.Fatalf("unexpected error: %v", err) } + if got := b[0] >> 4; got != pd1.EncodingPacked { + t.Fatalf("Wrong encoding used: expected uncompressed, got %v", got) + } + dec := pd1.NewTimeDecoder(b) for i, v := range x { if !dec.Next() { @@ -59,6 +63,10 @@ func Test_TimeEncoder_One(t *testing.T) { t.Fatalf("unexpected error: %v", err) } + if got := b[0] >> 4; got != pd1.EncodingPacked { + t.Fatalf("Wrong encoding used: expected uncompressed, got %v", got) + } + dec := pd1.NewTimeDecoder(b) if !dec.Next() { t.Fatalf("unexpected next value: got true, exp false") @@ -81,6 +89,10 @@ func Test_TimeEncoder_Two(t *testing.T) { t.Fatalf("unexpected error: %v", err) } + if got := b[0] >> 4; got != pd1.EncodingPacked { + t.Fatalf("Wrong encoding used: expected uncompressed, got %v", got) + } + dec := pd1.NewTimeDecoder(b) if !dec.Next() { t.Fatalf("unexpected next value: got true, exp false") @@ -114,6 +126,10 @@ func Test_TimeEncoder_Three(t *testing.T) { t.Fatalf("unexpected error: %v", err) } + if got := b[0] >> 4; got != pd1.EncodingPacked { + t.Fatalf("Wrong encoding used: expected uncompressed, got %v", got) + } + dec := pd1.NewTimeDecoder(b) if !dec.Next() { t.Fatalf("unexpected next value: got true, exp false") @@ -151,6 +167,10 @@ func Test_TimeEncoder_Large_Range(t *testing.T) { t.Fatalf("unexpected error: %v", err) } + if got := b[0] >> 4; got != pd1.EncodingPacked { + t.Fatalf("Wrong encoding used: expected uncompressed, got %v", got) + } + dec := pd1.NewTimeDecoder(b) if !dec.Next() { t.Fatalf("unexpected next value: got true, exp false") @@ -169,7 +189,7 @@ func Test_TimeEncoder_Large_Range(t *testing.T) { } } -func Test_TimeEncoder_Raw(t *testing.T) { +func Test_TimeEncoder_Uncompressed(t *testing.T) { enc := pd1.NewTimeEncoder() t1 := time.Unix(0, 0) t2 := time.Unix(1, 0) @@ -190,6 +210,10 @@ func Test_TimeEncoder_Raw(t *testing.T) { t.Fatalf("length mismatch: got %v, exp %v", len(b), exp) } + if got := b[0] >> 4; got != pd1.EncodingUncompressed { + t.Fatalf("Wrong encoding used: expected uncompressed, got %v", got) + } + dec := pd1.NewTimeDecoder(b) if !dec.Next() { t.Fatalf("unexpected next value: got true, exp false") @@ -232,6 +256,10 @@ func Test_TimeEncoder_RLE(t *testing.T) { t.Fatalf("length mismatch: got %v, exp %v", len(b), exp) } + if got := b[0] >> 4; got != pd1.EncodingRLE { + t.Fatalf("Wrong encoding used: expected uncompressed, got %v", got) + } + if err != nil { t.Fatalf("unexpected error: %v", err) } @@ -269,6 +297,10 @@ func Test_TimeEncoder_Reverse(t *testing.T) { t.Fatalf("unexpected error: %v", err) } + if got := b[0] >> 4; got != pd1.EncodingPacked { + t.Fatalf("Wrong encoding used: expected uncompressed, got %v", got) + } + dec := pd1.NewTimeDecoder(b) i := 0 for dec.Next() { @@ -301,6 +333,10 @@ func Test_TimeEncoder_220SecondDelta(t *testing.T) { t.Fatalf("unexpected length: got %v, exp %v", len(b), exp) } + if got := b[0] >> 4; got != pd1.EncodingRLE { + t.Fatalf("Wrong encoding used: expected uncompressed, got %v", got) + } + dec := pd1.NewTimeDecoder(b) i := 0 for dec.Next() { From 731ae271235ee850f7896371633dd0ed4cc8ae3e Mon Sep 17 00:00:00 2001 From: Jason Wilder Date: Fri, 25 Sep 2015 15:27:43 -0600 Subject: [PATCH 102/139] Remove unnecessary allocations from int64 decoder The decoder was creating a large slice and decoding all values when instead, it could decode one packed value as needed. --- tsdb/engine/pd1/int.go | 98 +++++++++++++++++-------------------- tsdb/engine/pd1/int_test.go | 4 ++ 2 files changed, 48 insertions(+), 54 deletions(-) diff --git a/tsdb/engine/pd1/int.go b/tsdb/engine/pd1/int.go index 706ca5fce8e..3c6cdf9c6ee 100644 --- a/tsdb/engine/pd1/int.go +++ b/tsdb/engine/pd1/int.go @@ -46,14 +46,12 @@ func (e *int64Encoder) encodePacked() ([]byte, error) { return nil, err } - b := make([]byte, 1+len(encoded)*8+4) + b := make([]byte, 1+len(encoded)*8) // 4 high bits of first byte store the encoding type for the block b[0] = byte(EncodingPacked) << 4 - binary.BigEndian.PutUint32(b[1:5], uint32(len(e.values))) - for i, v := range encoded { - binary.BigEndian.PutUint64(b[5+i*8:5+i*8+8], v) + binary.BigEndian.PutUint64(b[1+i*8:1+i*8+8], v) } return b, nil } @@ -64,86 +62,78 @@ func (e *int64Encoder) encodeUncompressed() ([]byte, error) { b[0] = byte(EncodingUncompressed) << 4 for i, v := range e.values { - binary.BigEndian.PutUint64(b[1+i*8:1+i*8+8], uint64(v)) + binary.BigEndian.PutUint64(b[1+i*8:1+i*8+8], v) } return b, nil } type int64Decoder struct { values []uint64 - v int64 - buf []uint64 - vbuf []uint64 + bytes []byte + i int + n int + + encoding byte } func NewInt64Decoder(b []byte) Int64Decoder { d := &int64Decoder{ - buf: make([]uint64, 240), - vbuf: make([]uint64, 1), + values: make([]uint64, 240), } - d.decode(b) + + d.SetBytes(b) return d } func (d *int64Decoder) SetBytes(b []byte) { - d.decode(b) + if len(b) > 0 { + d.encoding = b[0] >> 4 + d.bytes = b[1:] + } + d.i = 0 + d.n = 0 } func (d *int64Decoder) Next() bool { - if len(d.values) == 0 { + if d.i >= d.n && len(d.bytes) == 0 { return false } - d.v = ZigZagDecode(d.values[0]) - d.values = d.values[1:] - return true -} -func (d *int64Decoder) Read() int64 { - return d.v -} + d.i += 1 -func (d *int64Decoder) decode(b []byte) { - if len(b) == 0 { - return + if d.i >= d.n { + switch d.encoding { + case EncodingUncompressed: + d.decodeUncompressed() + case EncodingPacked: + d.decodePacked() + default: + panic(fmt.Sprintf("unknown encoding %v", d.encoding)) + } } + return d.i < d.n +} - // Encoding type is stored in the 4 high bits of the first byte - encoding := b[0] >> 4 - switch encoding { - case EncodingUncompressed: - d.decodeUncompressed(b[1:]) - case EncodingPacked: - d.decodePacked(b[1:]) - default: - panic(fmt.Sprintf("unknown encoding %v", encoding)) - } +func (d *int64Decoder) Read() int64 { + return ZigZagDecode(d.values[d.i]) } -func (d *int64Decoder) decodePacked(b []byte) { - if len(b) == 0 { +func (d *int64Decoder) decodePacked() { + if len(d.bytes) == 0 { return } - count := binary.BigEndian.Uint32(b[:4]) - - if count == 0 { - return - } + v := binary.BigEndian.Uint64(d.bytes[0:8]) + n, _ := simple8b.DecodeSingle(d.values, v) - d.values = make([]uint64, count) - b = b[4:] - j := 0 - for i := 0; i < len(b); i += 8 { - d.vbuf[0] = binary.BigEndian.Uint64(b[i : i+8]) - n, _ := simple8b.Decode(d.buf, d.vbuf) - copy(d.values[j:], d.buf[:n]) - j += n - } + d.n = n + d.i = 0 + d.bytes = d.bytes[8:] } -func (d *int64Decoder) decodeUncompressed(b []byte) { - d.values = make([]uint64, len(b)/8) - for i := range d.values { - d.values[i] = binary.BigEndian.Uint64(b[i*8 : i*8+8]) - } +func (d *int64Decoder) decodeUncompressed() { + d.values[0] = binary.BigEndian.Uint64(d.bytes[0:8]) + d.i = 0 + d.n = 1 + d.bytes = d.bytes[8:] } diff --git a/tsdb/engine/pd1/int_test.go b/tsdb/engine/pd1/int_test.go index 063ae05c075..7f1ba2eff74 100644 --- a/tsdb/engine/pd1/int_test.go +++ b/tsdb/engine/pd1/int_test.go @@ -199,6 +199,10 @@ func Test_Int64Encoder_AllNegative(t *testing.T) { dec := pd1.NewInt64Decoder(b) i := 0 for dec.Next() { + if i > len(values) { + t.Fatalf("read too many values: got %v, exp %v", i, len(values)) + } + if values[i] != dec.Read() { t.Fatalf("read value %d mismatch: got %v, exp %v", i, dec.Read(), values[i]) } From 1196587dc4df7f201c37ce6473d8cf571711c26b Mon Sep 17 00:00:00 2001 From: Jason Wilder Date: Mon, 28 Sep 2015 10:25:58 -0600 Subject: [PATCH 103/139] Keep track of the type of the block encoded Allowes decode to decode an arbitrary block correctly. --- tsdb/engine/pd1/encoding.go | 94 +++++++++++++++++++++++++++++-------- 1 file changed, 74 insertions(+), 20 deletions(-) diff --git a/tsdb/engine/pd1/encoding.go b/tsdb/engine/pd1/encoding.go index a2e16699456..227fe776123 100644 --- a/tsdb/engine/pd1/encoding.go +++ b/tsdb/engine/pd1/encoding.go @@ -16,6 +16,18 @@ const ( EncodingRLE = 1 // EncodingUncompressed is a non-compressed format EncodingUncompressed = 2 + + // BlockFloat64 designates a block encodes float64 values + BlockFloat64 = 0 + + // BlockInt64 designates a block encodes int64 values + BlockInt64 = 1 + + // BlockBool designates a block encodes bool values + BlockBool = 2 + + // BlockString designates a block encodes string values + BlockString = 3 ) type Value interface { @@ -67,14 +79,14 @@ func (v Values) Encode(buf []byte) []byte { for i, vv := range v { a[i] = vv.(*FloatValue) } - return EncodeFloatBlock(buf, a) + return encodeFloatBlock(buf, a) case *Int64Value: a := make([]*Int64Value, len(v)) for i, vv := range v { a[i] = vv.(*Int64Value) } - return EncodeInt64Block(buf, a) + return encodeInt64Block(buf, a) // TODO: add support for other types } @@ -85,10 +97,10 @@ func (v Values) Encode(buf []byte) []byte { func (v Values) DecodeSameTypeBlock(block []byte) Values { switch v[0].(type) { case *FloatValue: - a, _ := DecodeFloatBlock(block) + a, _ := decodeFloatBlock(block) return a case *Int64Value: - a, _ := DecodeInt64Block(block) + a, _ := decodeInt64Block(block) return a // TODO: add support for other types @@ -99,8 +111,25 @@ func (v Values) DecodeSameTypeBlock(block []byte) Values { // DecodeBlock takes a byte array and will decode into values of the appropriate type // based on the block func DecodeBlock(block []byte) (Values, error) { + if len(block) == 0 { + return Values{}, nil + } + + blockType := block[8] + switch blockType { + case BlockFloat64: + return decodeFloatBlock(block) + case BlockInt64: + return decodeInt64Block(block) + case BlockBool: + // return decodeBoolBlock(block) + case BlockString: + // return decodeStringBlock(block) + default: + } + // TODO: add support for other block types - return DecodeFloatBlock(block) + return nil, fmt.Errorf("unknown block type: %d", blockType) } // Deduplicate returns a new Values slice with any values @@ -147,9 +176,9 @@ func (f *FloatValue) Size() int { return 16 } -func EncodeFloatBlock(buf []byte, values []*FloatValue) []byte { +func encodeFloatBlock(buf []byte, values []*FloatValue) []byte { if len(values) == 0 { - return []byte{} + return nil } // A float block is encoded using different compression strategies @@ -176,14 +205,25 @@ func EncodeFloatBlock(buf []byte, values []*FloatValue) []byte { // Encoded float values vb := venc.Bytes() - // Preprend the first timestamp of the block in the first 8 bytes - return append(u64tob(uint64(values[0].Time().UnixNano())), - packBlock(tb, vb)...) + // Preprend the first timestamp of the block in the first 8 bytes and the block + // in the next byte, followed by the block + block := packBlockHeader(values[0].Time(), BlockFloat64) + block = append(block, packBlock(tb, vb)...) + return block } -func DecodeFloatBlock(block []byte) ([]Value, error) { +func decodeFloatBlock(block []byte) ([]Value, error) { // The first 8 bytes is the minimum timestamp of the block - tb, vb := unpackBlock(block[8:]) + block = block[8:] + + // Block type is the next block, make sure we actually have a float block + blockType := block[0] + if blockType != BlockFloat64 { + return nil, fmt.Errorf("invalid block type: exp %d, got %d", BlockFloat64, blockType) + } + block = block[1:] + + tb, vb := unpackBlock(block) // Setup our timestamp and value decoders dec := NewTimeDecoder(tb) @@ -208,11 +248,11 @@ type BoolValue struct { value bool } -func EncodeBoolBlock(buf []byte, values []BoolValue) []byte { +func encodeBoolBlock(buf []byte, values []BoolValue) []byte { return nil } -func DecodeBoolBlock(block []byte) ([]BoolValue, error) { +func eecodeBoolBlock(block []byte) ([]BoolValue, error) { return nil, nil } @@ -239,7 +279,7 @@ func (v *Int64Value) Size() int { func (v *Int64Value) String() string { return fmt.Sprintf("%v", v.value) } -func EncodeInt64Block(buf []byte, values []*Int64Value) []byte { +func encodeInt64Block(buf []byte, values []*Int64Value) []byte { tsEnc := NewTimeEncoder() vEnc := NewInt64Encoder() for _, v := range values { @@ -259,13 +299,23 @@ func EncodeInt64Block(buf []byte, values []*Int64Value) []byte { } // Preprend the first timestamp of the block in the first 8 bytes - return append(u64tob(uint64(values[0].Time().UnixNano())), - packBlock(tb, vb)...) + block := packBlockHeader(values[0].Time(), BlockInt64) + return append(block, packBlock(tb, vb)...) } -func DecodeInt64Block(block []byte) ([]Value, error) { +func decodeInt64Block(block []byte) ([]Value, error) { + // slice off the first 8 bytes (min timestmap for the block) + block = block[8:] + + blockType := block[0] + if blockType != BlockInt64 { + return nil, fmt.Errorf("invalid block type: exp %d, got %d", BlockInt64, blockType) + } + + block = block[1:] + // The first 8 bytes is the minimum timestamp of the block - tb, vb := unpackBlock(block[8:]) + tb, vb := unpackBlock(block) // Setup our timestamp and value decoders tsDec := NewTimeDecoder(tb) @@ -287,10 +337,14 @@ type StringValue struct { value string } -func EncodeStringBlock(buf []byte, values []StringValue) []byte { +func encodeStringBlock(buf []byte, blockType byte, values []StringValue) []byte { return nil } +func packBlockHeader(firstTime time.Time, blockType byte) []byte { + return append(u64tob(uint64(firstTime.UnixNano())), blockType) +} + func packBlock(ts []byte, values []byte) []byte { // We encode the length of the timestamp block using a variable byte encoding. // This allows small byte slices to take up 1 byte while larger ones use 2 or more. From cb23f5ac5337720ea907337692fbaa0bdf313d60 Mon Sep 17 00:00:00 2001 From: Jason Wilder Date: Mon, 28 Sep 2015 12:53:27 -0600 Subject: [PATCH 104/139] Add a compressed boolean encoding Packs booleans into bytes using 1 bit per value. --- tsdb/engine/pd1/bool.go | 118 ++++++++++++++++++++++++++++++ tsdb/engine/pd1/bool_test.go | 73 ++++++++++++++++++ tsdb/engine/pd1/encoding.go | 108 ++++++++++++++++++++++++--- tsdb/engine/pd1/encoding_test.go | 21 ++++++ tsdb/engine/pd1/int.go | 4 +- tsdb/engine/pd1/timestamp.go | 4 +- tsdb/engine/pd1/timestamp_test.go | 12 +-- 7 files changed, 319 insertions(+), 21 deletions(-) create mode 100644 tsdb/engine/pd1/bool.go create mode 100644 tsdb/engine/pd1/bool_test.go diff --git a/tsdb/engine/pd1/bool.go b/tsdb/engine/pd1/bool.go new file mode 100644 index 00000000000..065bd61c7f5 --- /dev/null +++ b/tsdb/engine/pd1/bool.go @@ -0,0 +1,118 @@ +package pd1 + +import "encoding/binary" + +type BoolEncoder interface { + Write(b bool) + Bytes() ([]byte, error) +} + +type BoolDecoder interface { + Next() bool + Read() bool +} + +type boolEncoder struct { + // The encoded bytes + bytes []byte + + // The current byte being encoded + b byte + + // The number of bools packed into b + i int + + // The total number of bools written + n int +} + +func NewBoolEncoder() BoolEncoder { + return &boolEncoder{} +} + +func (e *boolEncoder) Write(b bool) { + // If we have filled the current byte, flush it + if e.i >= 8 { + e.flush() + } + + // Use 1 bit for each boolen value, shift the current byte + // by 1 and set the least signficant bit acordingly + e.b = e.b << 1 + if b { + e.b |= 1 + } + + // Increment the current bool count + e.i += 1 + // Increment the total bool count + e.n += 1 +} + +func (e *boolEncoder) flush() { + // Pad remaining byte w/ 0s + for e.i < 8 { + e.b = e.b << 1 + e.i += 1 + } + + // If we have bits set, append them to the byte slice + if e.i > 0 { + e.bytes = append(e.bytes, e.b) + e.b = 0 + e.i = 0 + } +} + +func (e *boolEncoder) Bytes() ([]byte, error) { + // Ensure the current byte is flushed + e.flush() + b := make([]byte, 10+1) + + // Store the encoding type in the 4 high bits of the first byte + b[0] = byte(EncodingBitPacked) << 4 + + i := 1 + // Encode the number of bools written + i += binary.PutUvarint(b[i:], uint64(e.n)) + + // Append the packed booleans + return append(b[:i], e.bytes...), nil +} + +type boolDecoder struct { + b []byte + i int + n int +} + +func NewBoolDecoder(b []byte) BoolDecoder { + // First byte stores the encoding type, only have 1 bit-packet format + // currently ignore for now. + b = b[1:] + count, n := binary.Uvarint(b) + return &boolDecoder{b: b[n:], i: -1, n: int(count)} +} + +func (e *boolDecoder) Next() bool { + e.i += 1 + return e.i < e.n +} + +func (e *boolDecoder) Read() bool { + + // Index into the byte slice + idx := e.i / 8 + + // Bit position + pos := (8 - e.i%8) - 1 + + // The mask to select the bit + mask := byte(1 << uint(pos)) + + // The packed byte + v := e.b[idx] + + // Returns true if the bit is set + return v&mask == mask +} diff --git a/tsdb/engine/pd1/bool_test.go b/tsdb/engine/pd1/bool_test.go new file mode 100644 index 00000000000..ed642cff86e --- /dev/null +++ b/tsdb/engine/pd1/bool_test.go @@ -0,0 +1,73 @@ +package pd1_test + +import ( + "testing" + + "github.com/influxdb/influxdb/tsdb/engine/pd1" +) + +func Test_BoolEncoder_NoValues(t *testing.T) { + enc := pd1.NewBoolEncoder() + b, err := enc.Bytes() + if err != nil { + t.Fatalf("unexpected error: %v", err) + } + + dec := pd1.NewBoolDecoder(b) + if dec.Next() { + t.Fatalf("unexpected next value: got true, exp false") + } +} + +func Test_BoolEncoder_Single(t *testing.T) { + enc := pd1.NewBoolEncoder() + v1 := true + enc.Write(v1) + b, err := enc.Bytes() + if err != nil { + t.Fatalf("unexpected error: %v", err) + } + + dec := pd1.NewBoolDecoder(b) + if !dec.Next() { + t.Fatalf("unexpected next value: got false, exp true") + } + + if v1 != dec.Read() { + t.Fatalf("unexpected value: got %v, exp %v", dec.Read(), v1) + } +} + +func Test_BoolEncoder_Multi_Compressed(t *testing.T) { + enc := pd1.NewBoolEncoder() + + values := make([]bool, 10) + for i := range values { + values[i] = i%2 == 0 + enc.Write(values[i]) + } + + b, err := enc.Bytes() + if err != nil { + t.Fatalf("unexpected error: %v", err) + } + + if exp := 4; len(b) != exp { + t.Fatalf("unexpected length: got %v, exp %v", len(b), exp) + } + + dec := pd1.NewBoolDecoder(b) + + for i, v := range values { + if !dec.Next() { + t.Fatalf("unexpected next value: got false, exp true") + } + if v != dec.Read() { + t.Fatalf("unexpected value at pos %d: got %v, exp %v", i, dec.Read(), v) + } + } + + if dec.Next() { + t.Fatalf("unexpected next value: got true, exp false") + } +} diff --git a/tsdb/engine/pd1/encoding.go b/tsdb/engine/pd1/encoding.go index 227fe776123..d7abb195b5b 100644 --- a/tsdb/engine/pd1/encoding.go +++ b/tsdb/engine/pd1/encoding.go @@ -10,13 +10,18 @@ import ( ) const ( - // EncodingPacked is a bit-packed format - EncodingPacked = 0 + // EncodingPackedSimple is a bit-packed format + EncodingPackedSimple = 0 + // EncodingRLE is a run-length encoded format EncodingRLE = 1 + // EncodingUncompressed is a non-compressed format EncodingUncompressed = 2 + // EncodingBitPacked is a basic bit-packed format + EncodingBitPacked = 3 + // BlockFloat64 designates a block encodes float64 values BlockFloat64 = 0 @@ -43,8 +48,8 @@ func NewValue(t time.Time, value interface{}) Value { return &Int64Value{time: t, value: v} case float64: return &FloatValue{time: t, value: v} - // case bool: - // return &BoolValue{time: t, value: v} + case bool: + return &BoolValue{time: t, value: v} // case string: // return &StringValue{time: t, value: v} } @@ -88,6 +93,13 @@ func (v Values) Encode(buf []byte) []byte { } return encodeInt64Block(buf, a) + case *BoolValue: + a := make([]*BoolValue, len(v)) + for i, vv := range v { + a[i] = vv.(*BoolValue) + } + return encodeBoolBlock(buf, a) + // TODO: add support for other types } @@ -102,8 +114,9 @@ func (v Values) DecodeSameTypeBlock(block []byte) Values { case *Int64Value: a, _ := decodeInt64Block(block) return a - - // TODO: add support for other types + case *BoolValue: + a, _ := decodeBoolBlock(block) + return a } return nil } @@ -122,7 +135,7 @@ func DecodeBlock(block []byte) (Values, error) { case BlockInt64: return decodeInt64Block(block) case BlockBool: - // return decodeBoolBlock(block) + return decodeBoolBlock(block) case BlockString: // return decodeStringBlock(block) default: @@ -248,12 +261,85 @@ type BoolValue struct { value bool } -func encodeBoolBlock(buf []byte, values []BoolValue) []byte { - return nil +func (b *BoolValue) Time() time.Time { + return b.time +} + +func (b *BoolValue) Size() int { + return 9 +} + +func (b *BoolValue) UnixNano() int64 { + return b.time.UnixNano() +} + +func (b *BoolValue) Value() interface{} { + return b.value } -func eecodeBoolBlock(block []byte) ([]BoolValue, error) { - return nil, nil +func encodeBoolBlock(buf []byte, values []*BoolValue) []byte { + if len(values) == 0 { + return nil + } + + // A bool block is encoded using different compression strategies + // for timestamps and values. + + // Encode values using Gorilla float compression + venc := NewBoolEncoder() + + // Encode timestamps using an adaptive encoder + tsenc := NewTimeEncoder() + + for _, v := range values { + tsenc.Write(v.Time()) + venc.Write(v.value) + } + + // Encoded timestamp values + tb, err := tsenc.Bytes() + if err != nil { + panic(err.Error()) + } + // Encoded float values + vb, err := venc.Bytes() + if err != nil { + panic(err.Error()) + } + + // Preprend the first timestamp of the block in the first 8 bytes and the block + // in the next byte, followed by the block + block := packBlockHeader(values[0].Time(), BlockBool) + block = append(block, packBlock(tb, vb)...) + return block +} + +func decodeBoolBlock(block []byte) ([]Value, error) { + // The first 8 bytes is the minimum timestamp of the block + block = block[8:] + + // Block type is the next block, make sure we actually have a float block + blockType := block[0] + if blockType != BlockBool { + return nil, fmt.Errorf("invalid block type: exp %d, got %d", BlockBool, blockType) + } + block = block[1:] + + tb, vb := unpackBlock(block) + + // Setup our timestamp and value decoders + dec := NewTimeDecoder(tb) + vdec := NewBoolDecoder(vb) + + // Decode both a timestamp and value + var a []Value + for dec.Next() && vdec.Next() { + ts := dec.Read() + v := vdec.Read() + a = append(a, &BoolValue{ts, v}) + } + + return a, nil } type Int64Value struct { diff --git a/tsdb/engine/pd1/encoding_test.go b/tsdb/engine/pd1/encoding_test.go index c249f1aa8ce..aa5b67d6409 100644 --- a/tsdb/engine/pd1/encoding_test.go +++ b/tsdb/engine/pd1/encoding_test.go @@ -76,6 +76,27 @@ func TestEncoding_IntBlock_Negatives(t *testing.T) { } } +func TestEncoding_BoolBlock_Basic(t *testing.T) { + valueCount := 1000 + times := getTimes(valueCount, 60, time.Second) + values := make(pd1.Values, len(times)) + for i, t := range times { + v := true + if i%2 == 0 { + v = false + } + values[i] = pd1.NewValue(t, v) + } + + b := values.Encode(nil) + + decodedValues := values.DecodeSameTypeBlock(b) + + if !reflect.DeepEqual(decodedValues, values) { + t.Fatalf("unexpected results:\n\tgot: %v\n\texp: %v\n", decodedValues, values) + } +} + func getTimes(n, step int, precision time.Duration) []time.Time { t := time.Now().Round(precision) a := make([]time.Time, n) diff --git a/tsdb/engine/pd1/int.go b/tsdb/engine/pd1/int.go index 3c6cdf9c6ee..a7258e6d651 100644 --- a/tsdb/engine/pd1/int.go +++ b/tsdb/engine/pd1/int.go @@ -48,7 +48,7 @@ func (e *int64Encoder) encodePacked() ([]byte, error) { b := make([]byte, 1+len(encoded)*8) // 4 high bits of first byte store the encoding type for the block - b[0] = byte(EncodingPacked) << 4 + b[0] = byte(EncodingPackedSimple) << 4 for i, v := range encoded { binary.BigEndian.PutUint64(b[1+i*8:1+i*8+8], v) @@ -105,7 +105,7 @@ func (d *int64Decoder) Next() bool { switch d.encoding { case EncodingUncompressed: d.decodeUncompressed() - case EncodingPacked: + case EncodingPackedSimple: d.decodePacked() default: panic(fmt.Sprintf("unknown encoding %v", d.encoding)) diff --git a/tsdb/engine/pd1/timestamp.go b/tsdb/engine/pd1/timestamp.go index 04de5b991fb..6863a51afed 100644 --- a/tsdb/engine/pd1/timestamp.go +++ b/tsdb/engine/pd1/timestamp.go @@ -114,7 +114,7 @@ func (e *encoder) encodePacked(div uint64, dts []uint64) ([]byte, error) { b := make([]byte, 8+1) // 4 high bits used for the encoding type - b[0] = byte(EncodingPacked) << 4 + b[0] = byte(EncodingPackedSimple) << 4 // 4 low bits are the log10 divisor b[0] |= byte(math.Log10(float64(div))) @@ -196,7 +196,7 @@ func (d *decoder) decode(b []byte) { d.decodeRaw(b[1:]) case EncodingRLE: d.decodeRLE(b) - case EncodingPacked: + case EncodingPackedSimple: d.decodePacked(b) default: panic(fmt.Sprintf("unknown encoding: %v", encoding)) diff --git a/tsdb/engine/pd1/timestamp_test.go b/tsdb/engine/pd1/timestamp_test.go index 9eb3e0fe2c3..e0dd235d62a 100644 --- a/tsdb/engine/pd1/timestamp_test.go +++ b/tsdb/engine/pd1/timestamp_test.go @@ -24,7 +24,7 @@ func Test_TimeEncoder(t *testing.T) { t.Fatalf("unexpected error: %v", err) } - if got := b[0] >> 4; got != pd1.EncodingPacked { + if got := b[0] >> 4; got != pd1.EncodingPackedSimple { t.Fatalf("Wrong encoding used: expected uncompressed, got %v", got) } @@ -63,7 +63,7 @@ func Test_TimeEncoder_One(t *testing.T) { t.Fatalf("unexpected error: %v", err) } - if got := b[0] >> 4; got != pd1.EncodingPacked { + if got := b[0] >> 4; got != pd1.EncodingPackedSimple { t.Fatalf("Wrong encoding used: expected uncompressed, got %v", got) } @@ -89,7 +89,7 @@ func Test_TimeEncoder_Two(t *testing.T) { t.Fatalf("unexpected error: %v", err) } - if got := b[0] >> 4; got != pd1.EncodingPacked { + if got := b[0] >> 4; got != pd1.EncodingPackedSimple { t.Fatalf("Wrong encoding used: expected uncompressed, got %v", got) } @@ -126,7 +126,7 @@ func Test_TimeEncoder_Three(t *testing.T) { t.Fatalf("unexpected error: %v", err) } - if got := b[0] >> 4; got != pd1.EncodingPacked { + if got := b[0] >> 4; got != pd1.EncodingPackedSimple { t.Fatalf("Wrong encoding used: expected uncompressed, got %v", got) } @@ -167,7 +167,7 @@ func Test_TimeEncoder_Large_Range(t *testing.T) { t.Fatalf("unexpected error: %v", err) } - if got := b[0] >> 4; got != pd1.EncodingPacked { + if got := b[0] >> 4; got != pd1.EncodingPackedSimple { t.Fatalf("Wrong encoding used: expected uncompressed, got %v", got) } @@ -297,7 +297,7 @@ func Test_TimeEncoder_Reverse(t *testing.T) { t.Fatalf("unexpected error: %v", err) } - if got := b[0] >> 4; got != pd1.EncodingPacked { + if got := b[0] >> 4; got != pd1.EncodingPackedSimple { t.Fatalf("Wrong encoding used: expected uncompressed, got %v", got) } From 7e0df18e1ae83fc1aad7cdf3b6ec5a399e6dd00c Mon Sep 17 00:00:00 2001 From: Jason Wilder Date: Mon, 28 Sep 2015 23:06:17 -0600 Subject: [PATCH 105/139] Update simple8b api usage --- tsdb/engine/pd1/int.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tsdb/engine/pd1/int.go b/tsdb/engine/pd1/int.go index a7258e6d651..c8f42341c99 100644 --- a/tsdb/engine/pd1/int.go +++ b/tsdb/engine/pd1/int.go @@ -41,7 +41,7 @@ func (e *int64Encoder) Bytes() ([]byte, error) { } func (e *int64Encoder) encodePacked() ([]byte, error) { - encoded, err := simple8b.Encode(e.values) + encoded, err := simple8b.EncodeAll(e.values) if err != nil { return nil, err } @@ -124,7 +124,7 @@ func (d *int64Decoder) decodePacked() { } v := binary.BigEndian.Uint64(d.bytes[0:8]) - n, _ := simple8b.DecodeSingle(d.values, v) + n, _ := simple8b.Decode(d.values, v) d.n = n d.i = 0 From 2da52ec4fe1e7d90ac8e26247fa761fac2dcfd4a Mon Sep 17 00:00:00 2001 From: Jason Wilder Date: Tue, 29 Sep 2015 10:44:24 -0600 Subject: [PATCH 106/139] Fix deadlock in pd1_test.go The defer tx.Rollback() tries to free the queryLock but the defer e.Cleanup() runs before it and tries to take a write lock on the query lock (which blocks) and prevents tx.Rollback() from acquring the read lock. --- tsdb/engine/pd1/pd1_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tsdb/engine/pd1/pd1_test.go b/tsdb/engine/pd1/pd1_test.go index e1abe87360c..2a59c9ac4e2 100644 --- a/tsdb/engine/pd1/pd1_test.go +++ b/tsdb/engine/pd1/pd1_test.go @@ -77,7 +77,6 @@ func TestEngine_WriteAndReadFloats(t *testing.T) { verify(false) tx, _ := e.Begin(false) - defer tx.Rollback() c := tx.Cursor("cpu,host=B", fields, nil, true) k, v := c.SeekTo(0) if k != p2.UnixNano() { @@ -111,6 +110,7 @@ func TestEngine_WriteAndReadFloats(t *testing.T) { if 1.1 != v { t.Fatal("p1 data not equal") } + tx.Rollback() if err := e.Close(); err != nil { t.Fatalf("error closing: %s", err.Error()) From a4d92162ef3fd3d907de278d12f21b790d5c7ba5 Mon Sep 17 00:00:00 2001 From: Jason Wilder Date: Tue, 29 Sep 2015 10:46:19 -0600 Subject: [PATCH 107/139] Add documentation about compression --- tsdb/engine/pd1/bool.go | 5 +++++ tsdb/engine/pd1/int.go | 20 +++++++++++++++++ tsdb/engine/pd1/timestamp.go | 43 +++++++++++++++++++++++++++++------- 3 files changed, 60 insertions(+), 8 deletions(-) diff --git a/tsdb/engine/pd1/bool.go b/tsdb/engine/pd1/bool.go index 065bd61c7f5..4fad26586bb 100644 --- a/tsdb/engine/pd1/bool.go +++ b/tsdb/engine/pd1/bool.go @@ -1,5 +1,10 @@ package pd1 +// bool encoding uses 1 bit per value. Each compressed byte slice contains a 1 byte header +// indicating the compression type, followed by a variable byte encoded length indicating +// how many booleans are packed in the slice. The remaining bytes contains 1 byte for every +// 8 boolean values encoded. + import "encoding/binary" type BoolEncoder interface { diff --git a/tsdb/engine/pd1/int.go b/tsdb/engine/pd1/int.go index c8f42341c99..98cf717f303 100644 --- a/tsdb/engine/pd1/int.go +++ b/tsdb/engine/pd1/int.go @@ -1,5 +1,25 @@ package pd1 +// Int64 encoding uses two different strategies depending on the range of values in +// the uncompressed data. Encoded values are first encoding used zig zag encoding. +// This interleaves postiive and negative integers across a range of positive integers. +// +// For example, [-2,-1,0,1] becomes [3,1,0,2]. See +// https://developers.google.com/protocol-buffers/docs/encoding?hl=en#signed-integers +// for more information. +// +// If all the zig zag encoded values less than 1 << 60 - 1, they are compressed using +// simple8b encoding. If any values is larger than 1 << 60 - 1, the values are stored uncompressed. +// +// Each encoded byte slice, contains a 1 byte header followed by multiple 8 byte packed integers +// or 8 byte uncompressed integers. The 4 high bits of the first byte indicate the encoding type +// for the remaining bytes. +// +// There are currently two encoding types that can be used with room for 15 more. These additional +// encoding slots are reserved for future use. One improvement to to be made is to use a patched +// encoding such as PFOR if only a small number of values exceed the max compressed value range. This +// should improve compression ratios with very integers near the ends of the int64 range. + import ( "encoding/binary" "fmt" diff --git a/tsdb/engine/pd1/timestamp.go b/tsdb/engine/pd1/timestamp.go index 6863a51afed..88a2c2517bd 100644 --- a/tsdb/engine/pd1/timestamp.go +++ b/tsdb/engine/pd1/timestamp.go @@ -1,13 +1,40 @@ -// Package timestamp provides structs and functions for converting streams of timestamps -// to byte slices. -// -// The encoding is adapative based on structure of the timestamps that are encoded. By default, -// a bit-packed format that compresses multiple 64bit timestamps into a single 64bit word is used. -// If the values are too large to be compressed using the bit-packed format, it will fall back to -// a raw 8byte per timestamp format. If the the values can be run-length encoded, based on the -// differences between consectutive values, a shorter, variable sized RLE format is used. package pd1 +// Timestamp encoding is adapative and based on structure of the timestamps that are encoded. It +// uses a combination of delta encoding, zig zag encoding, scaling and compression using simple8b, +// run length encoding as well as falling back to no compression if needed. +// +// Timestamp values to be encoded should be sorted before encoding. When encoded, the values are +// first delta-encoded. The first value is the starting timestamp, subsequent values are the difference. +// from the prior value. +// +// Delta encoding can produce negative values. After delta encoding, the values are zig zag encoded +// to convert them to positive values. +// +// Timestamp resolution can also be in the nanosecond. Many timestamps are monotonically increasing +// and fall on even boundaries of time such as every 10s. When the timestamps have this structure, +// they are scaled by the largest common divisor that is also a factor of 10. This has the effect +// of converting very large integer deltas into very small one that can be reversed by multiplying them +// by the scaling factor. +// +// Using these adjusted values, if all the deltas are the same, the time range is stored using run +// length encoding. If run length encoding is not possible and all values are less than 1 << 60 - 1 +// (~36.5 yrs in nanosecond resolution), then the timestamps are encoded using simple8b encoding. If +// any value exceeds the maximum values, the deltas are stored uncompressed using 8b each. +// +// Each compressed byte slice has a 1 byte header indicating the compression type. The 4 high bits +// indicated the encoding type. The 4 low bits are using by the encoding type. +// +// For run length encoding, the 4 low bits store the log10 of the scaling factor. The next 8 bytes are +// the starting timestamp, next 1-10 bytes is the delta value using variable-length encoding, finally the +// next 1-10 bytes is the count of values. +// +// For simple8b encoding, the 4 low bits store the log10 of the scaling factor. The next 8 bytes is the +// first delta value stored uncompressed, the remaining bytes are 64bit words containg compressed delta +// values. +// +// For uncompressed encoding, the delta values are stored using 8 bytes each. + import ( "encoding/binary" "fmt" From 9f9692acdfa0835fa4d644f9e3a67e0cc388d5fa Mon Sep 17 00:00:00 2001 From: Jason Wilder Date: Tue, 29 Sep 2015 10:47:35 -0600 Subject: [PATCH 108/139] Rename float encoding tests --- tsdb/engine/pd1/float_test.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tsdb/engine/pd1/float_test.go b/tsdb/engine/pd1/float_test.go index 5bfa377ac1b..60702938f5f 100644 --- a/tsdb/engine/pd1/float_test.go +++ b/tsdb/engine/pd1/float_test.go @@ -6,7 +6,7 @@ import ( "github.com/influxdb/influxdb/tsdb/engine/pd1" ) -func TestExampleEncoding(t *testing.T) { +func TestFloatEncoder_Simple(t *testing.T) { // Example from the paper s := pd1.NewFloatEncoder() @@ -92,7 +92,7 @@ var TwoHoursData = []struct { {786}, {785}, {774}, {786}, {718}, } -func TestRoundtrip(t *testing.T) { +func TestFloatEncoder_Roundtrip(t *testing.T) { s := pd1.NewFloatEncoder() for _, p := range TwoHoursData { From c1213ba367ac87878a23e0b5cc23bee382fd1665 Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Tue, 29 Sep 2015 16:17:58 -0400 Subject: [PATCH 109/139] Update WAL to deduplicate values on Cursor query. Added test and have failing section for single value encoding. --- tsdb/engine/pd1/pd1_test.go | 51 +++++++++++++++++++++++++++---------- tsdb/engine/pd1/wal.go | 5 ++-- 2 files changed, 39 insertions(+), 17 deletions(-) diff --git a/tsdb/engine/pd1/pd1_test.go b/tsdb/engine/pd1/pd1_test.go index 2a59c9ac4e2..e3e80aee3f6 100644 --- a/tsdb/engine/pd1/pd1_test.go +++ b/tsdb/engine/pd1/pd1_test.go @@ -634,28 +634,51 @@ func TestEngine_WriteManyPointsToSingleSeries(t *testing.T) { } } -func TestEngine_WriteIndexBenchmarkNames(t *testing.T) { - t.Skip("whatevs") - +func TestEngine_WritePointsInMultipleRequestsWithSameTime(t *testing.T) { e := OpenDefaultEngine() defer e.Cleanup() - var points []models.Point - for i := 0; i < 100000; i++ { - points = append(points, parsePoint(fmt.Sprintf("cpu%d value=22.1", i))) - } + fields := []string{"value"} - st := time.Now() - if err := e.WritePoints(points, nil, nil); err != nil { + e.WAL.SkipCache = false + + if err := e.WritePoints([]models.Point{parsePoint("foo value=1 0")}, nil, nil); err != nil { t.Fatalf("failed to write points: %s", err.Error()) } - fmt.Println("took: ", time.Since(st)) - - st = time.Now() - if err := e.WritePoints(points, nil, nil); err != nil { + if err := e.WritePoints([]models.Point{parsePoint("foo value=2 0")}, nil, nil); err != nil { t.Fatalf("failed to write points: %s", err.Error()) } - fmt.Println("took: ", time.Since(st)) + if err := e.WritePoints([]models.Point{parsePoint("foo value=3 0")}, nil, nil); err != nil { + t.Fatalf("failed to write points: %s", err.Error()) + } + + verify := func() { + tx, _ := e.Begin(false) + defer tx.Rollback() + c := tx.Cursor("foo", fields, nil, true) + k, v := c.SeekTo(0) + if k != 0 { + t.Fatalf("expected 0 time but got %d", k) + } + if v != float64(3) { + t.Fatalf("expected 3 for value but got %f", v.(float64)) + } + k, _ = c.Next() + if k != tsdb.EOF { + t.Fatal("expected EOF") + } + } + + verify() + + if err := e.Close(); err != nil { + t.Fatalf("error closing: %s", err.Error()) + } + if err := e.Open(); err != nil { + t.Fatalf("error opening: %s", err.Error()) + } + + verify() } // Engine represents a test wrapper for pd1.Engine. diff --git a/tsdb/engine/pd1/wal.go b/tsdb/engine/pd1/wal.go index 49b72b4a608..f44937ac674 100644 --- a/tsdb/engine/pd1/wal.go +++ b/tsdb/engine/pd1/wal.go @@ -177,13 +177,12 @@ func (l *Log) Cursor(series string, fields []string, dec *tsdb.FieldCodec, ascen copy(c, fc) c = append(c, values...) - return newWALCursor(c, ascending) + return newWALCursor(Values(c).Deduplicate(), ascending) } } if l.cacheDirtySort[ck] { - sort.Sort(values) - delete(l.cacheDirtySort, ck) + values = Values(values).Deduplicate() } // build a copy so writes afterwards don't change the result set From be011b8da9572b3bdc8894dff5733767c5be9a27 Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Tue, 29 Sep 2015 20:03:37 -0400 Subject: [PATCH 110/139] Add logging to pd1 --- tsdb/engine/pd1/pd1.go | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/tsdb/engine/pd1/pd1.go b/tsdb/engine/pd1/pd1.go index d707661fad7..3834aad2261 100644 --- a/tsdb/engine/pd1/pd1.go +++ b/tsdb/engine/pd1/pd1.go @@ -7,6 +7,7 @@ import ( "hash/fnv" "io" "io/ioutil" + "log" "math" "os" "path/filepath" @@ -73,6 +74,7 @@ type Engine struct { writeLock *writeLock metaLock sync.Mutex path string + logger *log.Logger // deletesPending mark how many old data files are waiting to be deleted. This will // keep a close from returning until all deletes finish @@ -117,6 +119,7 @@ func NewEngine(path string, walPath string, opt tsdb.EngineOptions) tsdb.Engine e := &Engine{ path: path, writeLock: &writeLock{}, + logger: log.New(os.Stderr, "[pd1] ", log.LstdFlags), // TODO: this is the function where we can inject a check against the in memory collisions HashSeriesField: hashSeriesField, @@ -142,6 +145,7 @@ func (e *Engine) PerformMaintenance() { e.WAL.flush(f) }() } else if e.shouldCompact() { + e.logger.Println("compacting for maintenance") go e.Compact(true) } } @@ -445,7 +449,7 @@ func (e *Engine) Compact(fullCompaction bool) error { break } - fmt.Println("Starting compaction with files:", len(files)) + e.logger.Printf("Starting compaction in partition %s of %d files", e.path, len(files)) st := time.Now() // mark the compaction as running @@ -591,7 +595,7 @@ func (e *Engine) Compact(fullCompaction bool) error { e.files = newFiles e.filesLock.Unlock() - fmt.Println("Compaction took ", time.Since(st)) + e.logger.Println("Compaction took ", time.Since(st)) // delete the old files in a goroutine so running queries won't block the write // from completing From 861a15b3e631e95610d939d790eefe74d76c2b4a Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Tue, 29 Sep 2015 21:51:16 -0400 Subject: [PATCH 111/139] Fix panic when data file has small index --- tsdb/engine/pd1/pd1.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tsdb/engine/pd1/pd1.go b/tsdb/engine/pd1/pd1.go index 3834aad2261..a6bd2113a16 100644 --- a/tsdb/engine/pd1/pd1.go +++ b/tsdb/engine/pd1/pd1.go @@ -520,7 +520,7 @@ func (e *Engine) Compact(fullCompaction bool) error { // write the blocks out to file that are already at their size limit for { - // if the next block is the same ID, we don't need to decod this one + // if the next block is the same ID, we don't need to decode this one // so we can just write it out to the file nextID, _, nextBlock := df.block(newPos) newPos = newPos + uint32(blockHeaderSize+len(block)) @@ -1568,7 +1568,7 @@ func (c *cursor) SeekTo(seek int64) (int64, interface{}) { // if the next block has a time less than what we're seeking to, // skip decoding this block and continue on nextBlockPos := pos + 12 + length - if nextBlockPos < c.f.size { + if nextBlockPos < c.f.indexPosition() { nextBlockID := btou64(c.f.mmap[nextBlockPos : nextBlockPos+8]) if nextBlockID == c.id { nextBlockTime := int64(btou64(c.f.mmap[nextBlockPos+12 : nextBlockPos+20])) From c47d14540dc685cc66354f8cd377d3f0c7a9c852 Mon Sep 17 00:00:00 2001 From: Jason Wilder Date: Tue, 29 Sep 2015 16:08:18 -0600 Subject: [PATCH 112/139] Add compressed string encoding Uses snappy to compress multiple strings into a block --- tsdb/engine/pd1/encoding.go | 90 +++++++++++++++++++++++++++++--- tsdb/engine/pd1/encoding_test.go | 18 +++++++ tsdb/engine/pd1/string.go | 82 +++++++++++++++++++++++++++++ tsdb/engine/pd1/string_test.go | 78 +++++++++++++++++++++++++++ 4 files changed, 262 insertions(+), 6 deletions(-) create mode 100644 tsdb/engine/pd1/string.go create mode 100644 tsdb/engine/pd1/string_test.go diff --git a/tsdb/engine/pd1/encoding.go b/tsdb/engine/pd1/encoding.go index d7abb195b5b..005692d2735 100644 --- a/tsdb/engine/pd1/encoding.go +++ b/tsdb/engine/pd1/encoding.go @@ -22,6 +22,9 @@ const ( // EncodingBitPacked is a basic bit-packed format EncodingBitPacked = 3 + // EncodingSnappy is a snappy encoded format + EncodingSnappy = 4 + // BlockFloat64 designates a block encodes float64 values BlockFloat64 = 0 @@ -50,8 +53,8 @@ func NewValue(t time.Time, value interface{}) Value { return &FloatValue{time: t, value: v} case bool: return &BoolValue{time: t, value: v} - // case string: - // return &StringValue{time: t, value: v} + case string: + return &StringValue{time: t, value: v} } return &EmptyValue{} } @@ -100,7 +103,12 @@ func (v Values) Encode(buf []byte) []byte { } return encodeBoolBlock(buf, a) - // TODO: add support for other types + case *StringValue: + a := make([]*StringValue, len(v)) + for i, vv := range v { + a[i] = vv.(*StringValue) + } + return encodeStringBlock(buf, a) } return nil @@ -117,6 +125,9 @@ func (v Values) DecodeSameTypeBlock(block []byte) Values { case *BoolValue: a, _ := decodeBoolBlock(block) return a + case *StringValue: + a, _ := decodeStringBlock(block) + return a } return nil } @@ -137,7 +148,7 @@ func DecodeBlock(block []byte) (Values, error) { case BlockBool: return decodeBoolBlock(block) case BlockString: - // return decodeStringBlock(block) + return decodeStringBlock(block) default: } @@ -423,8 +434,75 @@ type StringValue struct { value string } -func encodeStringBlock(buf []byte, blockType byte, values []StringValue) []byte { - return nil +func (v *StringValue) Time() time.Time { + return v.time +} + +func (v *StringValue) Value() interface{} { + return v.value +} + +func (v *StringValue) UnixNano() int64 { + return v.time.UnixNano() +} + +func (v *StringValue) Size() int { + return 8 + len(v.value) +} + +func (v *StringValue) String() string { return v.value } + +func encodeStringBlock(buf []byte, values []*StringValue) []byte { + tsEnc := NewTimeEncoder() + vEnc := NewStringEncoder() + for _, v := range values { + tsEnc.Write(v.Time()) + vEnc.Write(v.value) + } + + // Encoded timestamp values + tb, err := tsEnc.Bytes() + if err != nil { + panic(err.Error()) + } + // Encoded int64 values + vb, err := vEnc.Bytes() + if err != nil { + panic(err.Error()) + } + + // Preprend the first timestamp of the block in the first 8 bytes + block := packBlockHeader(values[0].Time(), BlockString) + return append(block, packBlock(tb, vb)...) +} + +func decodeStringBlock(block []byte) ([]Value, error) { + // slice off the first 8 bytes (min timestmap for the block) + block = block[8:] + + blockType := block[0] + if blockType != BlockString { + return nil, fmt.Errorf("invalid block type: exp %d, got %d", BlockString, blockType) + } + + block = block[1:] + + // The first 8 bytes is the minimum timestamp of the block + tb, vb := unpackBlock(block) + + // Setup our timestamp and value decoders + tsDec := NewTimeDecoder(tb) + vDec := NewStringDecoder(vb) + + // Decode both a timestamp and value + var a []Value + for tsDec.Next() && vDec.Next() { + ts := tsDec.Read() + v := vDec.Read() + a = append(a, &StringValue{ts, v}) + } + + return a, nil } func packBlockHeader(firstTime time.Time, blockType byte) []byte { diff --git a/tsdb/engine/pd1/encoding_test.go b/tsdb/engine/pd1/encoding_test.go index aa5b67d6409..306ad155767 100644 --- a/tsdb/engine/pd1/encoding_test.go +++ b/tsdb/engine/pd1/encoding_test.go @@ -3,6 +3,7 @@ package pd1_test import ( // "math/rand" + "fmt" "reflect" "testing" "time" @@ -97,6 +98,23 @@ func TestEncoding_BoolBlock_Basic(t *testing.T) { } } +func TestEncoding_StringBlock_Basic(t *testing.T) { + valueCount := 1000 + times := getTimes(valueCount, 60, time.Second) + values := make(pd1.Values, len(times)) + for i, t := range times { + values[i] = pd1.NewValue(t, fmt.Sprintf("value %d", i)) + } + + b := values.Encode(nil) + + decodedValues := values.DecodeSameTypeBlock(b) + + if !reflect.DeepEqual(decodedValues, values) { + t.Fatalf("unexpected results:\n\tgot: %v\n\texp: %v\n", decodedValues, values) + } +} + func getTimes(n, step int, precision time.Duration) []time.Time { t := time.Now().Round(precision) a := make([]time.Time, n) diff --git a/tsdb/engine/pd1/string.go b/tsdb/engine/pd1/string.go new file mode 100644 index 00000000000..e548b68c560 --- /dev/null +++ b/tsdb/engine/pd1/string.go @@ -0,0 +1,82 @@ +package pd1 + +// String encoding uses snappy compression to compress each string. Each string is +// appended to byte slice prefixed with a variable byte length followed by the string +// bytes. The bytes are compressed using snappy compressor and a 1 byte header is used +// to indicate the type of encoding. + +import ( + "encoding/binary" + "fmt" + + "github.com/golang/snappy" +) + +type StringEncoder interface { + Write(s string) + Bytes() ([]byte, error) +} + +type StringDecoder interface { + Next() bool + Read() string +} + +type stringEncoder struct { + // The encoded bytes + bytes []byte +} + +func NewStringEncoder() StringEncoder { + return &stringEncoder{} +} + +func (e *stringEncoder) Write(s string) { + b := make([]byte, 10) + // Append the length of the string using variable byte encoding + i := binary.PutUvarint(b, uint64(len(s))) + e.bytes = append(e.bytes, b[:i]...) + + // Append the string bytes + e.bytes = append(e.bytes, s...) +} + +func (e *stringEncoder) Bytes() ([]byte, error) { + // Compress the currently appended bytes using snappy and prefix with + // a 1 byte header for future extension + data := snappy.Encode(nil, e.bytes) + return append([]byte{EncodingSnappy << 4}, data...), nil +} + +type stringDecoder struct { + b []byte + l int + i int +} + +func NewStringDecoder(b []byte) StringDecoder { + // First byte stores the encoding type, only have snappy format + // currently so ignore for now. + data, err := snappy.Decode(nil, b[1:]) + if err != nil { + // TODO: Need to propogate errors up the call stack better + panic(fmt.Sprintf("failed to decode string block: %v", err.Error())) + } + + return &stringDecoder{b: data} +} + +func (e *stringDecoder) Next() bool { + e.i += e.l + return e.i < len(e.b) +} + +func (e *stringDecoder) Read() string { + // Read the length of the string + length, n := binary.Uvarint(e.b[e.i:]) + + // The length of this string plus the length of the variable byte encoded length + e.l = int(length) + n + + return string(e.b[e.i+n : e.i+n+int(length)]) +} diff --git a/tsdb/engine/pd1/string_test.go b/tsdb/engine/pd1/string_test.go new file mode 100644 index 00000000000..c457de8697d --- /dev/null +++ b/tsdb/engine/pd1/string_test.go @@ -0,0 +1,78 @@ +package pd1_test + +import ( + "fmt" + "testing" + + "github.com/influxdb/influxdb/tsdb/engine/pd1" +) + +func Test_StringEncoder_NoValues(t *testing.T) { + enc := pd1.NewStringEncoder() + b, err := enc.Bytes() + if err != nil { + t.Fatalf("unexpected error: %v", err) + } + + dec := pd1.NewStringDecoder(b) + if dec.Next() { + t.Fatalf("unexpected next value: got true, exp false") + } +} + +func Test_StringEncoder_Single(t *testing.T) { + enc := pd1.NewStringEncoder() + v1 := "v1" + enc.Write(v1) + b, err := enc.Bytes() + if err != nil { + t.Fatalf("unexpected error: %v", err) + } + + dec := pd1.NewStringDecoder(b) + if !dec.Next() { + t.Fatalf("unexpected next value: got false, exp true") + } + + if v1 != dec.Read() { + t.Fatalf("unexpected value: got %v, exp %v", dec.Read(), v1) + } +} + +func Test_StringEncoder_Multi_Compressed(t *testing.T) { + enc := pd1.NewStringEncoder() + + values := make([]string, 10) + for i := range values { + values[i] = fmt.Sprintf("value %d", i) + enc.Write(values[i]) + } + + b, err := enc.Bytes() + if err != nil { + t.Fatalf("unexpected error: %v", err) + } + + if b[0]>>4 != pd1.EncodingSnappy { + t.Fatalf("unexpected encoding: got %v, exp %v", b[0], pd1.EncodingSnappy) + } + + if exp := 47; len(b) != exp { + t.Fatalf("unexpected length: got %v, exp %v", len(b), exp) + } + + dec := pd1.NewStringDecoder(b) + + for i, v := range values { + if !dec.Next() { + t.Fatalf("unexpected next value: got false, exp true") + } + if v != dec.Read() { + t.Fatalf("unexpected value at pos %d: got %v, exp %v", i, dec.Read(), v) + } + } + + if dec.Next() { + t.Fatalf("unexpected next value: got true, exp false") + } +} From 5d938d0a8b61bbc695c124eb3c321bd240f43f65 Mon Sep 17 00:00:00 2001 From: Jason Wilder Date: Tue, 29 Sep 2015 20:01:43 -0600 Subject: [PATCH 113/139] Add test with duplicate timestamps Should not happen but makes sure that the same values are encoded and decoded correctly. --- tsdb/engine/pd1/encoding_test.go | 15 +++++++++++++++ 1 file changed, 15 insertions(+) diff --git a/tsdb/engine/pd1/encoding_test.go b/tsdb/engine/pd1/encoding_test.go index 306ad155767..7126eec3887 100644 --- a/tsdb/engine/pd1/encoding_test.go +++ b/tsdb/engine/pd1/encoding_test.go @@ -28,6 +28,21 @@ func TestEncoding_FloatBlock(t *testing.T) { } } +func TestEncoding_FloatBlock_ZeroTime(t *testing.T) { + values := make(pd1.Values, 3) + for i := 0; i < 3; i++ { + values[i] = pd1.NewValue(time.Unix(0, 0), float64(i)) + } + + b := values.Encode(nil) + + decodedValues := values.DecodeSameTypeBlock(b) + + if !reflect.DeepEqual(decodedValues, values) { + t.Fatalf("unexpected results:\n\tgot: %v\n\texp: %v\n", decodedValues, values) + } +} + func TestEncoding_IntBlock_Basic(t *testing.T) { valueCount := 1000 times := getTimes(valueCount, 60, time.Second) From 3332236527953004ba1f03f77239258bdf45ca8f Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Wed, 30 Sep 2015 11:48:26 -0400 Subject: [PATCH 114/139] Fix bugs with writing old data and compaction. --- tsdb/engine/pd1/pd1.go | 69 ++++++++---- tsdb/engine/pd1/pd1_test.go | 213 ++++++++++++++++++++++++++++++++++++ 2 files changed, 261 insertions(+), 21 deletions(-) diff --git a/tsdb/engine/pd1/pd1.go b/tsdb/engine/pd1/pd1.go index a6bd2113a16..37e16c6d498 100644 --- a/tsdb/engine/pd1/pd1.go +++ b/tsdb/engine/pd1/pd1.go @@ -92,6 +92,8 @@ type Engine struct { CompactionFileCount int IndexCompactionFullAge time.Duration IndexMinimumCompactionInterval time.Duration + MaxPointsPerBlock int + RotateBlockSize int // filesLock is only for modifying and accessing the files slice filesLock sync.RWMutex @@ -129,6 +131,8 @@ func NewEngine(path string, walPath string, opt tsdb.EngineOptions) tsdb.Engine CompactionFileCount: opt.Config.IndexCompactionFileCount, IndexCompactionFullAge: opt.Config.IndexCompactionFullAge, IndexMinimumCompactionInterval: opt.Config.IndexMinimumCompactionInterval, + MaxPointsPerBlock: DefaultMaxPointsPerBlock, + RotateBlockSize: DefaultRotateBlockSize, } e.WAL.Index = e @@ -144,6 +148,7 @@ func (e *Engine) PerformMaintenance() { go func() { e.WAL.flush(f) }() + return } else if e.shouldCompact() { e.logger.Println("compacting for maintenance") go e.Compact(true) @@ -449,7 +454,11 @@ func (e *Engine) Compact(fullCompaction bool) error { break } - e.logger.Printf("Starting compaction in partition %s of %d files", e.path, len(files)) + var s string + if fullCompaction { + s = "FULL " + } + e.logger.Printf("Starting %scompaction in partition %s of %d files", s, e.path, len(files)) st := time.Now() // mark the compaction as running @@ -486,16 +495,16 @@ func (e *Engine) Compact(fullCompaction bool) error { currentPosition := uint32(fileHeaderSize) newPositions := make([]uint32, 0) newIDs := make([]uint64, 0) - buf := make([]byte, DefaultRotateBlockSize) + buf := make([]byte, e.RotateBlockSize) for { // find the min ID so we can write it to the file minID := uint64(math.MaxUint64) for _, id := range ids { - if minID > id { + if minID > id && id != 0 { minID = id } } - if minID == 0 { // we've emptied all the files + if minID == math.MaxUint64 { // we've emptied all the files break } @@ -520,14 +529,10 @@ func (e *Engine) Compact(fullCompaction bool) error { // write the blocks out to file that are already at their size limit for { - // if the next block is the same ID, we don't need to decode this one - // so we can just write it out to the file - nextID, _, nextBlock := df.block(newPos) - newPos = newPos + uint32(blockHeaderSize+len(block)) - + // write the values, the block or combine with previous if len(previousValues) > 0 { previousValues = append(previousValues, previousValues.DecodeSameTypeBlock(block)...) - } else if len(block) > DefaultRotateBlockSize { + } else if len(block) > e.RotateBlockSize { if _, err := f.Write(df.mmap[pos:newPos]); err != nil { return err } @@ -538,7 +543,7 @@ func (e *Engine) Compact(fullCompaction bool) error { } // write the previous values and clear if we've hit the limit - if len(previousValues) > DefaultMaxPointsPerBlock { + if len(previousValues) > e.MaxPointsPerBlock { b := previousValues.Encode(buf) if err := e.writeBlock(f, id, b); err != nil { // fail hard. If we can't write a file someone needs to get woken up @@ -548,14 +553,28 @@ func (e *Engine) Compact(fullCompaction bool) error { previousValues = nil } + // if the next block is the same ID, we don't need to decode this one + // so we can just write it out to the file + nextID, _, nextBlock := df.block(newPos) + // move to the next block in this file only if the id is the same if nextID != id { + // flush remaining values + if len(previousValues) > 0 { + b := previousValues.Encode(buf) + currentPosition += uint32(blockHeaderSize + len(b)) + previousValues = nil + if err := e.writeBlock(f, id, b); err != nil { + panic(fmt.Sprintf("error writing file %s: %s", f.Name(), err.Error())) + } + } ids[i] = nextID break } + pos = newPos + newPos = pos + uint32(blockHeaderSize+len(nextBlock)) positions[i] = newPos block = nextBlock - newPos = newPos + uint32(blockHeaderSize+len(block)) } } @@ -595,7 +614,7 @@ func (e *Engine) Compact(fullCompaction bool) error { e.files = newFiles e.filesLock.Unlock() - e.logger.Println("Compaction took ", time.Since(st)) + e.logger.Printf("Compaction of %s took %s", e.path, time.Since(st)) // delete the old files in a goroutine so running queries won't block the write // from completing @@ -603,8 +622,7 @@ func (e *Engine) Compact(fullCompaction bool) error { go func() { for _, f := range files { if err := f.Delete(); err != nil { - // TODO: log this error - fmt.Println("ERROR DELETING:", f.f.Name()) + e.logger.Println("ERROR DELETING:", f.f.Name()) } } e.deletesPending.Done() @@ -824,7 +842,7 @@ func (e *Engine) filterDataBetweenTimes(valuesByID map[uint64]Values, minTime, m filteredValues := make(map[uint64]Values) for id, values := range valuesByID { maxIndex := len(values) - minIndex := 0 + minIndex := -1 // find the index of the first value in the range for i, v := range values { t := v.UnixNano() @@ -833,6 +851,9 @@ func (e *Engine) filterDataBetweenTimes(valuesByID map[uint64]Values, minTime, m break } } + if minIndex == -1 { + continue + } // go backwards to find the index of the last value in the range for i := len(values) - 1; i >= 0; i-- { t := values[i].UnixNano() @@ -883,6 +904,7 @@ func (e *Engine) rewriteFile(oldDF *dataFile, valuesByID map[uint64]Values) erro minTime = oldDF.MinTime() maxTime = oldDF.MaxTime() } + for _, v := range valuesByID { if minTime > v.MinTime() { minTime = v.MinTime() @@ -919,7 +941,7 @@ func (e *Engine) rewriteFile(oldDF *dataFile, valuesByID map[uint64]Values) erro // their positions currentPosition := uint32(fileHeaderSize) newPositions := make([]uint32, len(ids)) - buf := make([]byte, DefaultMaxPointsPerBlock*20) + buf := make([]byte, e.MaxPointsPerBlock*20) for i, id := range ids { // mark the position for this ID newPositions[i] = currentPosition @@ -1041,7 +1063,7 @@ func (e *Engine) rewriteFile(oldDF *dataFile, valuesByID map[uint64]Values) erro e.deletesPending.Add(1) go func() { if err := oldDF.Delete(); err != nil { - fmt.Println("ERROR DELETING FROM REWRITE:", oldDF.f.Name()) + e.logger.Println("ERROR DELETING FROM REWRITE:", oldDF.f.Name()) } e.deletesPending.Done() }() @@ -1346,9 +1368,9 @@ func (e *Engine) DecodeAndCombine(newValues Values, block, buf []byte, nextTime } } - if len(values) > DefaultMaxPointsPerBlock { - remainingValues = values[DefaultMaxPointsPerBlock:] - values = values[:DefaultMaxPointsPerBlock] + if len(values) > e.MaxPointsPerBlock { + remainingValues = values[e.MaxPointsPerBlock:] + values = values[:e.MaxPointsPerBlock] } return remainingValues, values.Encode(buf), nil @@ -1485,6 +1507,11 @@ func (d *dataFile) StartingPositionForID(id uint64) uint32 { } func (d *dataFile) block(pos uint32) (id uint64, t int64, block []byte) { + defer func() { + if r := recover(); r != nil { + fmt.Println("FUCK: ", d.f.Name(), pos, id, t) + } + }() if pos < d.indexPosition() { id = btou64(d.mmap[pos : pos+8]) length := btou32(d.mmap[pos+8 : pos+12]) diff --git a/tsdb/engine/pd1/pd1_test.go b/tsdb/engine/pd1/pd1_test.go index e3e80aee3f6..9eda01fdc46 100644 --- a/tsdb/engine/pd1/pd1_test.go +++ b/tsdb/engine/pd1/pd1_test.go @@ -681,6 +681,219 @@ func TestEngine_WritePointsInMultipleRequestsWithSameTime(t *testing.T) { verify() } +func TestEngine_CompactWithSeriesInOneFile(t *testing.T) { + e := OpenDefaultEngine() + defer e.Cleanup() + + fields := []string{"value"} + + e.RotateFileSize = 10 + e.MaxPointsPerBlock = 1 + + p1 := parsePoint("cpu,host=A value=1.1 1000000000") + p2 := parsePoint("cpu,host=B value=1.2 2000000000") + p3 := parsePoint("cpu,host=A value=1.3 3000000000") + + if err := e.WritePoints([]models.Point{p1}, nil, nil); err != nil { + t.Fatalf("failed to write points: %s", err.Error()) + } + if err := e.WritePoints([]models.Point{p2}, nil, nil); err != nil { + t.Fatalf("failed to write points: %s", err.Error()) + } + if err := e.WritePoints([]models.Point{p3}, nil, nil); err != nil { + t.Fatalf("failed to write points: %s", err.Error()) + } + + if count := e.DataFileCount(); count != 3 { + t.Fatalf("expected 3 data files but got %d", count) + } + + verify := func() { + tx, _ := e.Begin(false) + defer tx.Rollback() + c := tx.Cursor("cpu,host=A", fields, nil, false) + k, v := c.SeekTo(0) + if k != 1000000000 { + t.Fatalf("expected time 1000000000 but got %d", k) + } + if v != 1.1 { + t.Fatalf("expected value 1.1 but got %f", v.(float64)) + } + k, v = c.Next() + if k != 3000000000 { + t.Fatalf("expected time 3000000000 but got %d", k) + } + c = tx.Cursor("cpu,host=B", fields, nil, false) + k, v = c.SeekTo(0) + if k != 2000000000 { + t.Fatalf("expected time 2000000000 but got %d", k) + } + if v != 1.2 { + t.Fatalf("expected value 1.2 but got %f", v.(float64)) + } + } + + fmt.Println("verify 1") + verify() + + if err := e.Compact(true); err != nil { + t.Fatalf("error compacting: %s", err.Error) + } + fmt.Println("verify 2") + verify() + + p4 := parsePoint("cpu,host=A value=1.4 4000000000") + if err := e.WritePoints([]models.Point{p4}, nil, nil); err != nil { + t.Fatalf("failed to write points: %s", err.Error()) + } + + if err := e.Compact(true); err != nil { + t.Fatalf("error compacting: %s", err.Error) + } + tx1, _ := e.Begin(false) + defer tx1.Rollback() + c := tx1.Cursor("cpu,host=A", fields, nil, false) + k, v := c.SeekTo(0) + if k != 1000000000 { + t.Fatalf("expected time 1000000000 but got %d", k) + } + if v != 1.1 { + t.Fatalf("expected value 1.1 but got %f", v.(float64)) + } + k, v = c.Next() + if k != 3000000000 { + t.Fatalf("expected time 3000000000 but got %d", k) + } + k, v = c.Next() + if k != 4000000000 { + t.Fatalf("expected time 3000000000 but got %d", k) + } +} + +// Ensure that compactions that happen where blocks from old data files +// skip decoding and just get copied over to the new data file works. +func TestEngine_CompactionWithCopiedBlocks(t *testing.T) { + e := OpenDefaultEngine() + defer e.Cleanup() + + fields := []string{"value"} + + e.RotateFileSize = 10 + e.MaxPointsPerBlock = 1 + e.RotateBlockSize = 10 + + p1 := parsePoint("cpu,host=A value=1.1 1000000000") + p2 := parsePoint("cpu,host=A value=1.2 2000000000") + p3 := parsePoint("cpu,host=A value=1.3 3000000000") + + if err := e.WritePoints([]models.Point{p1, p2}, nil, nil); err != nil { + t.Fatalf("failed to write points: %s", err.Error()) + } + if err := e.WritePoints([]models.Point{p3}, nil, nil); err != nil { + t.Fatalf("failed to write points: %s", err.Error()) + } + + verify := func() { + tx, _ := e.Begin(false) + defer tx.Rollback() + c := tx.Cursor("cpu,host=A", fields, nil, false) + k, _ := c.SeekTo(0) + if k != 1000000000 { + t.Fatalf("expected time 1000000000 but got %d", k) + } + k, _ = c.Next() + if k != 2000000000 { + t.Fatalf("expected time 2000000000 but got %d", k) + } + k, _ = c.Next() + if k != 3000000000 { + t.Fatalf("expected time 3000000000 but got %d", k) + } + } + + verify() + if err := e.Compact(true); err != nil { + t.Fatalf("error compacting: %s", err.Error) + } + fmt.Println("verify 2") + verify() + + p4 := parsePoint("cpu,host=B value=1.4 4000000000") + if err := e.WritePoints([]models.Point{p4}, nil, nil); err != nil { + t.Fatalf("failed to write points: %s", err.Error()) + } + + if err := e.Compact(true); err != nil { + t.Fatalf("error compacting: %s", err.Error) + } + fmt.Println("verify 3") + verify() + + p5 := parsePoint("cpu,host=A value=1.5 5000000000") + p6 := parsePoint("cpu,host=A value=1.6 6000000000") + p7 := parsePoint("cpu,host=B value=2.1 7000000000") + if err := e.WritePoints([]models.Point{p5, p6, p7}, nil, nil); err != nil { + t.Fatalf("failed to write points: %s", err.Error()) + } + + p8 := parsePoint("cpu,host=A value=1.5 7000000000") + p9 := parsePoint("cpu,host=A value=1.6 8000000000") + p10 := parsePoint("cpu,host=B value=2.1 8000000000") + if err := e.WritePoints([]models.Point{p8, p9, p10}, nil, nil); err != nil { + t.Fatalf("failed to write points: %s", err.Error()) + } + + if err := e.Compact(true); err != nil { + t.Fatalf("error compacting: %s", err.Error) + } + verify() + +} + +func TestEngine_RewritingOldBlocks(t *testing.T) { + e := OpenDefaultEngine() + defer e.Cleanup() + + fields := []string{"value"} + + e.MaxPointsPerBlock = 2 + + p1 := parsePoint("cpu,host=A value=1.1 1000000000") + p2 := parsePoint("cpu,host=A value=1.2 2000000000") + p3 := parsePoint("cpu,host=A value=1.3 3000000000") + p4 := parsePoint("cpu,host=A value=1.5 1500000000") + + if err := e.WritePoints([]models.Point{p1, p2}, nil, nil); err != nil { + t.Fatalf("failed to write points: %s", err.Error()) + } + if err := e.WritePoints([]models.Point{p3}, nil, nil); err != nil { + t.Fatalf("failed to write points: %s", err.Error()) + } + if err := e.WritePoints([]models.Point{p4}, nil, nil); err != nil { + t.Fatalf("failed to write points: %s", err.Error()) + } + + tx, _ := e.Begin(false) + defer tx.Rollback() + c := tx.Cursor("cpu,host=A", fields, nil, false) + k, _ := c.SeekTo(0) + if k != 1000000000 { + t.Fatalf("expected time 1000000000 but got %d", k) + } + k, _ = c.Next() + if k != 1500000000 { + t.Fatalf("expected time 1500000000 but got %d", k) + } + k, _ = c.Next() + if k != 2000000000 { + t.Fatalf("expected time 2000000000 but got %d", k) + } + k, _ = c.Next() + if k != 3000000000 { + t.Fatalf("expected time 3000000000 but got %d", k) + } +} + // Engine represents a test wrapper for pd1.Engine. type Engine struct { *pd1.Engine From 2366baaf0be31448b43e104617b7f278e97023f3 Mon Sep 17 00:00:00 2001 From: Jason Wilder Date: Wed, 30 Sep 2015 09:48:20 -0600 Subject: [PATCH 115/139] Handle partial reads when loading WAL If reading into fixed sized buffer using io.ReadFull, the func can return io.ErrUnexpectedEOF if the read was short. This was slipping through the error handling causing the shard to fail to load. --- tsdb/engine/pd1/wal.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tsdb/engine/pd1/wal.go b/tsdb/engine/pd1/wal.go index f44937ac674..6e51980f2b2 100644 --- a/tsdb/engine/pd1/wal.go +++ b/tsdb/engine/pd1/wal.go @@ -353,7 +353,7 @@ func (l *Log) readFileToCache(fileName string) error { buf = make([]byte, length) } _, err = io.ReadFull(f, buf[0:length]) - if err == io.EOF { + if err == io.EOF || err == io.ErrUnexpectedEOF { l.logger.Printf("hit end of file while reading compressed wal entry from %s", fileName) return nil } else if err != nil { From a2b139e006cb02c31052df9f7427812e041712b4 Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Wed, 30 Sep 2015 16:38:55 -0400 Subject: [PATCH 116/139] Fix compaction and multi-write bugs. * Fix bug with locking when the interval completely covers or is totally inside another one. * Fix bug with full compactions running when the index is actively being written to. --- tsdb/engine/pd1/pd1.go | 59 ++++++++++--- tsdb/engine/pd1/pd1_test.go | 67 +++++++++++++++ tsdb/engine/pd1/wal.go | 4 +- tsdb/engine/pd1/write_lock.go | 58 +++++++------ tsdb/engine/pd1/write_lock_test.go | 131 +++++++++++++++++++++++++++++ 5 files changed, 279 insertions(+), 40 deletions(-) create mode 100644 tsdb/engine/pd1/write_lock_test.go diff --git a/tsdb/engine/pd1/pd1.go b/tsdb/engine/pd1/pd1.go index 37e16c6d498..5f64c6c2f5a 100644 --- a/tsdb/engine/pd1/pd1.go +++ b/tsdb/engine/pd1/pd1.go @@ -71,7 +71,7 @@ var _ tsdb.Engine = &Engine{} // Engine represents a storage engine with compressed blocks. type Engine struct { - writeLock *writeLock + writeLock *WriteLock metaLock sync.Mutex path string logger *log.Logger @@ -120,7 +120,7 @@ func NewEngine(path string, walPath string, opt tsdb.EngineOptions) tsdb.Engine e := &Engine{ path: path, - writeLock: &writeLock{}, + writeLock: &WriteLock{}, logger: log.New(os.Stderr, "[pd1] ", log.LstdFlags), // TODO: this is the function where we can inject a check against the in memory collisions @@ -149,10 +149,28 @@ func (e *Engine) PerformMaintenance() { e.WAL.flush(f) }() return - } else if e.shouldCompact() { - e.logger.Println("compacting for maintenance") - go e.Compact(true) } + + // don't do a full compaction if the WAL received writes in the time window + if time.Since(e.WAL.LastWriteTime()) < e.IndexCompactionFullAge { + return + } + + e.filesLock.RLock() + running := e.compactionRunning + e.filesLock.RUnlock() + if running { + return + } + + // do a full compaction if all the index files are older than the compaction time + for _, f := range e.copyFilesCollection() { + if time.Since(f.modTime) < e.IndexCompactionFullAge { + return + } + } + + go e.Compact(true) } // Format returns the format type of this engine @@ -204,6 +222,8 @@ func (e *Engine) Open() error { return err } + e.lastCompactionTime = time.Now() + return nil } @@ -454,15 +474,12 @@ func (e *Engine) Compact(fullCompaction bool) error { break } - var s string - if fullCompaction { - s = "FULL " - } - e.logger.Printf("Starting %scompaction in partition %s of %d files", s, e.path, len(files)) - st := time.Now() - // mark the compaction as running e.filesLock.Lock() + if e.compactionRunning { + e.filesLock.Unlock() + return nil + } e.compactionRunning = true e.filesLock.Unlock() defer func() { @@ -474,11 +491,19 @@ func (e *Engine) Compact(fullCompaction bool) error { e.filesLock.Unlock() }() + var s string + if fullCompaction { + s = "FULL " + } + fileName := e.nextFileName() + e.logger.Printf("Starting %scompaction in partition %s of %d files to new file %s", s, e.path, len(files), fileName) + st := time.Now() + positions := make([]uint32, len(files)) ids := make([]uint64, len(files)) // initilaize for writing - f, err := os.OpenFile(e.nextFileName(), os.O_CREATE|os.O_RDWR, 0666) + f, err := os.OpenFile(fileName, os.O_CREATE|os.O_RDWR, 0666) if err != nil { return err } @@ -931,6 +956,12 @@ func (e *Engine) rewriteFile(oldDF *dataFile, valuesByID map[uint64]Values) erro return err } + if oldDF == nil { + e.logger.Printf("writing new index file %s", f.Name()) + } else { + e.logger.Printf("rewriting index file %s with %s", oldDF.f.Name(), f.Name()) + } + // write the magic number if _, err := f.Write(u32tob(magicNumber)); err != nil { f.Close() @@ -1509,7 +1540,7 @@ func (d *dataFile) StartingPositionForID(id uint64) uint32 { func (d *dataFile) block(pos uint32) (id uint64, t int64, block []byte) { defer func() { if r := recover(); r != nil { - fmt.Println("FUCK: ", d.f.Name(), pos, id, t) + panic(fmt.Sprintf("panic decoding file: %s at position %d for id %d at time %d", d.f.Name(), pos, id, t)) } }() if pos < d.indexPosition() { diff --git a/tsdb/engine/pd1/pd1_test.go b/tsdb/engine/pd1/pd1_test.go index 9eda01fdc46..494c63524fe 100644 --- a/tsdb/engine/pd1/pd1_test.go +++ b/tsdb/engine/pd1/pd1_test.go @@ -894,6 +894,73 @@ func TestEngine_RewritingOldBlocks(t *testing.T) { } } +func TestEngine_WriteIntoCompactedFile(t *testing.T) { + e := OpenDefaultEngine() + defer e.Cleanup() + + fields := []string{"value"} + + e.MaxPointsPerBlock = 3 + e.RotateFileSize = 10 + + p1 := parsePoint("cpu,host=A value=1.1 1000000000") + p2 := parsePoint("cpu,host=A value=1.2 2000000000") + p3 := parsePoint("cpu,host=A value=1.3 3000000000") + p4 := parsePoint("cpu,host=A value=1.5 4000000000") + p5 := parsePoint("cpu,host=A value=1.6 2500000000") + + if err := e.WritePoints([]models.Point{p1, p2}, nil, nil); err != nil { + t.Fatalf("failed to write points: %s", err.Error()) + } + if err := e.WritePoints([]models.Point{p3}, nil, nil); err != nil { + t.Fatalf("failed to write points: %s", err.Error()) + } + + if err := e.Compact(true); err != nil { + t.Fatalf("error compacting: %s", err.Error) + } + + if err := e.WritePoints([]models.Point{p4}, nil, nil); err != nil { + t.Fatalf("failed to write points: %s", err.Error()) + } + + if err := e.Compact(true); err != nil { + t.Fatalf("error compacting: %s", err.Error) + } + + if err := e.WritePoints([]models.Point{p5}, nil, nil); err != nil { + t.Fatalf("failed to write points: %s", err.Error()) + } + + if count := e.DataFileCount(); count != 1 { + t.Fatalf("execpted 1 data file but got %d", count) + } + + tx, _ := e.Begin(false) + defer tx.Rollback() + c := tx.Cursor("cpu,host=A", fields, nil, true) + k, _ := c.SeekTo(0) + if k != 1000000000 { + t.Fatalf("wrong time: %d", k) + } + k, _ = c.Next() + if k != 2000000000 { + t.Fatalf("wrong time: %d", k) + } + k, _ = c.Next() + if k != 2500000000 { + t.Fatalf("wrong time: %d", k) + } + k, _ = c.Next() + if k != 3000000000 { + t.Fatalf("wrong time: %d", k) + } + k, _ = c.Next() + if k != 4000000000 { + t.Fatalf("wrong time: %d", k) + } +} + // Engine represents a test wrapper for pd1.Engine. type Engine struct { *pd1.Engine diff --git a/tsdb/engine/pd1/wal.go b/tsdb/engine/pd1/wal.go index 6e51980f2b2..ef4399912cd 100644 --- a/tsdb/engine/pd1/wal.go +++ b/tsdb/engine/pd1/wal.go @@ -542,7 +542,7 @@ func (l *Log) flush(flush flushType) error { } else if flush == startupFlush { ftype = "startup" } - l.logger.Printf("%s flush of %d keys with %d values of %d bytes\n", ftype, len(l.flushCache), valueCount, flushSize) + l.logger.Printf("%s flush of %s with %d keys and %d total values of %d bytes\n", ftype, l.path, len(l.flushCache), valueCount, flushSize) } startTime := time.Now() @@ -550,7 +550,7 @@ func (l *Log) flush(flush flushType) error { return err } if l.LoggingEnabled { - l.logger.Printf("flush to index took %s\n", time.Since(startTime)) + l.logger.Printf("%s flush to index took %s\n", l.path, time.Since(startTime)) } l.cacheLock.Lock() diff --git a/tsdb/engine/pd1/write_lock.go b/tsdb/engine/pd1/write_lock.go index 5f48fb9311a..a791b663abb 100644 --- a/tsdb/engine/pd1/write_lock.go +++ b/tsdb/engine/pd1/write_lock.go @@ -1,15 +1,14 @@ package pd1 import ( + "reflect" "sync" ) // writeLock is a lock that enables locking of ranges between a // min and max value. We use this so that flushes from the WAL // can occur concurrently along with compactions. -type writeLock struct { - mu sync.Mutex - +type WriteLock struct { rangesLock sync.Mutex ranges []*rangeLock } @@ -19,34 +18,41 @@ type writeLock struct { // an overlapping range will have to wait until the previous // lock is released. A corresponding call to UnlockRange should // be deferred. -func (w *writeLock) LockRange(min, max int64) { - w.mu.Lock() - defer w.mu.Unlock() - +func (w *WriteLock) LockRange(min, max int64) { r := &rangeLock{min: min, max: max} - ranges := w.currentlyLockedRanges() + for { + ranges := w.currentlyLockedRanges() - // ensure there are no currently locked ranges that overlap - for _, rr := range ranges { - if rr.overlaps(r) { - // wait until it gets unlocked - rr.mu.Lock() - // release the lock so the object can get GC'd - rr.mu.Unlock() + // ensure there are no currently locked ranges that overlap + for _, rr := range ranges { + if rr.overlaps(r) { + // wait until it gets unlocked + rr.mu.Lock() + // release the lock so the object can get GC'd + rr.mu.Unlock() + } } - } - // and lock the range - r.mu.Lock() + // ensure that no one else got a lock on the range while we + // were waiting + w.rangesLock.Lock() + if len(w.ranges) == 0 || reflect.DeepEqual(ranges, w.ranges) { + // and lock the range + r.mu.Lock() - // now that we know the range is free, add it to the locks - w.rangesLock.Lock() - w.ranges = append(w.ranges, r) - w.rangesLock.Unlock() + // now that we know the range is free, add it to the locks + w.ranges = append(w.ranges, r) + w.rangesLock.Unlock() + return + } + + // try again + w.rangesLock.Unlock() + } } // UnlockRange will release a previously locked range. -func (w *writeLock) UnlockRange(min, max int64) { +func (w *WriteLock) UnlockRange(min, max int64) { w.rangesLock.Lock() defer w.rangesLock.Unlock() @@ -62,7 +68,7 @@ func (w *writeLock) UnlockRange(min, max int64) { w.ranges = a } -func (w *writeLock) currentlyLockedRanges() []*rangeLock { +func (w *WriteLock) currentlyLockedRanges() []*rangeLock { w.rangesLock.Lock() defer w.rangesLock.Unlock() a := make([]*rangeLock, len(w.ranges)) @@ -81,6 +87,10 @@ func (r *rangeLock) overlaps(l *rangeLock) bool { return true } else if l.max >= r.min && l.max <= r.max { return true + } else if l.min <= r.min && l.max >= r.max { + return true + } else if l.min >= r.min && l.max <= r.max { + return true } return false } diff --git a/tsdb/engine/pd1/write_lock_test.go b/tsdb/engine/pd1/write_lock_test.go new file mode 100644 index 00000000000..4cdc88e816e --- /dev/null +++ b/tsdb/engine/pd1/write_lock_test.go @@ -0,0 +1,131 @@ +package pd1_test + +import ( + // "sync" + "testing" + "time" + + "github.com/influxdb/influxdb/tsdb/engine/pd1" +) + +func TestWriteLock_FullCover(t *testing.T) { + w := &pd1.WriteLock{} + w.LockRange(2, 10) + + lock := make(chan bool) + timeout := time.NewTimer(10 * time.Millisecond) + go func() { + w.LockRange(1, 11) + lock <- true + }() + select { + case <-lock: + t.Fatal("able to get lock when we shouldn't") + case <-timeout.C: + // we're all good + } +} + +func TestWriteLock_RightIntersect(t *testing.T) { + w := &pd1.WriteLock{} + w.LockRange(2, 10) + + lock := make(chan bool) + timeout := time.NewTimer(10 * time.Millisecond) + go func() { + w.LockRange(5, 15) + lock <- true + }() + select { + case <-lock: + t.Fatal("able to get lock when we shouldn't") + case <-timeout.C: + // we're all good + } +} + +func TestWriteLock_LeftIntersect(t *testing.T) { + w := &pd1.WriteLock{} + w.LockRange(1, 4) + + lock := make(chan bool) + timeout := time.NewTimer(10 * time.Millisecond) + go func() { + w.LockRange(1, 11) + lock <- true + }() + select { + case <-lock: + t.Fatal("able to get lock when we shouldn't") + case <-timeout.C: + // we're all good + } +} + +func TestWriteLock_Inside(t *testing.T) { + w := &pd1.WriteLock{} + w.LockRange(4, 8) + + lock := make(chan bool) + timeout := time.NewTimer(10 * time.Millisecond) + go func() { + w.LockRange(1, 11) + lock <- true + }() + select { + case <-lock: + t.Fatal("able to get lock when we shouldn't") + case <-timeout.C: + // we're all good + } +} + +func TestWriteLock_Same(t *testing.T) { + w := &pd1.WriteLock{} + w.LockRange(2, 10) + + lock := make(chan bool) + timeout := time.NewTimer(10 * time.Millisecond) + go func() { + w.LockRange(2, 10) + lock <- true + }() + select { + case <-lock: + t.Fatal("able to get lock when we shouldn't") + case <-timeout.C: + // we're all good + } +} + +// func TestWriteLock_FreeRangeWithContentionElsewhere(t *testing.T) { +// w := &pd1.WriteLock{} +// w.LockRange(2, 10) + +// lock := make(chan bool) +// freeRange := make(chan bool) +// timeout := time.NewTimer(10 * time.Millisecond) +// var wg sync.WaitGroup + +// wg.Add(1) +// go func() { +// wg.Done() +// w.LockRange(4, 12) +// lock <- true +// }() + +// // make sure the other go func has gotten to the point of requesting the lock +// wg.Wait() +// go func() { +// w.LockRange(15, 23) +// freeRange <- true +// }() +// select { +// case <-lock: +// t.Fatal("able to get lock when we shouldn't") +// case <-timeout.C: +// t.Fatal("unable to get lock of free range when contention exists elsewhere") +// case <-freeRange: +// // we're all good +// } +// } From d9499f0598544be2607f437d2a1b16c54fe833ca Mon Sep 17 00:00:00 2001 From: Jason Wilder Date: Wed, 30 Sep 2015 13:11:49 -0600 Subject: [PATCH 117/139] Remove zig zag encoding from timestamp encoder Not needed since all timestamps will be sorted in ascending order. Negatives are not possible. --- tsdb/engine/pd1/timestamp.go | 22 +++++++--------------- tsdb/engine/pd1/timestamp_test.go | 2 +- 2 files changed, 8 insertions(+), 16 deletions(-) diff --git a/tsdb/engine/pd1/timestamp.go b/tsdb/engine/pd1/timestamp.go index 88a2c2517bd..b6fc6fef411 100644 --- a/tsdb/engine/pd1/timestamp.go +++ b/tsdb/engine/pd1/timestamp.go @@ -1,16 +1,13 @@ package pd1 // Timestamp encoding is adapative and based on structure of the timestamps that are encoded. It -// uses a combination of delta encoding, zig zag encoding, scaling and compression using simple8b, -// run length encoding as well as falling back to no compression if needed. +// uses a combination of delta encoding, scaling and compression using simple8b, run length encoding +// as well as falling back to no compression if needed. // // Timestamp values to be encoded should be sorted before encoding. When encoded, the values are // first delta-encoded. The first value is the starting timestamp, subsequent values are the difference. // from the prior value. // -// Delta encoding can produce negative values. After delta encoding, the values are zig zag encoded -// to convert them to positive values. -// // Timestamp resolution can also be in the nanosecond. Many timestamps are monotonically increasing // and fall on even boundaries of time such as every 10s. When the timestamps have this structure, // they are scaled by the largest common divisor that is also a factor of 10. This has the effect @@ -83,10 +80,7 @@ func (e *encoder) reduce() (max, divisor uint64, rle bool, deltas []uint64) { for i := len(deltas) - 1; i > 0; i-- { // First differential encode the values - delta := int64(deltas[i] - deltas[i-1]) - - // The delta may be negative so zigzag encode it into a postive value - deltas[i] = ZigZagEncode(delta) + deltas[i] = deltas[i] - deltas[i-1] // We're also need to keep track of the max value and largest common divisor v := deltas[i] @@ -243,8 +237,8 @@ func (d *decoder) decodePacked(b []byte) { // Compute the prefix sum and scale the deltas back up for i := 1; i < len(deltas); i++ { - dgap := ZigZagDecode(deltas[i] * div) - deltas[i] = uint64(int64(deltas[i-1]) + dgap) + dgap := deltas[i] * div + deltas[i] = deltas[i-1] + dgap } d.ts = deltas @@ -264,8 +258,6 @@ func (d *decoder) decodeRLE(b []byte) { // Next 1-10 bytes is our (scaled down by factor of 10) run length values value, n := binary.Uvarint(b[i:]) - value = uint64(ZigZagDecode(value)) - // Scale the value back up value *= uint64(mod) i += n @@ -293,10 +285,10 @@ func (d *decoder) decodeRaw(b []byte) { for i := range d.ts { d.ts[i] = binary.BigEndian.Uint64(b[i*8 : i*8+8]) - delta := ZigZagDecode(d.ts[i]) + delta := d.ts[i] // Compute the prefix sum and scale the deltas back up if i > 0 { - d.ts[i] = uint64(int64(d.ts[i-1]) + delta) + d.ts[i] = d.ts[i-1] + delta } } } diff --git a/tsdb/engine/pd1/timestamp_test.go b/tsdb/engine/pd1/timestamp_test.go index e0dd235d62a..f7030db8a40 100644 --- a/tsdb/engine/pd1/timestamp_test.go +++ b/tsdb/engine/pd1/timestamp_test.go @@ -297,7 +297,7 @@ func Test_TimeEncoder_Reverse(t *testing.T) { t.Fatalf("unexpected error: %v", err) } - if got := b[0] >> 4; got != pd1.EncodingPackedSimple { + if got := b[0] >> 4; got != pd1.EncodingUncompressed { t.Fatalf("Wrong encoding used: expected uncompressed, got %v", got) } From dbf6228817a0d328050b5492a800c8b0b84c35e8 Mon Sep 17 00:00:00 2001 From: Jason Wilder Date: Wed, 30 Sep 2015 13:12:53 -0600 Subject: [PATCH 118/139] Fix go vet --- tsdb/engine/pd1/pd1_test.go | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/tsdb/engine/pd1/pd1_test.go b/tsdb/engine/pd1/pd1_test.go index 494c63524fe..e11a9baf929 100644 --- a/tsdb/engine/pd1/pd1_test.go +++ b/tsdb/engine/pd1/pd1_test.go @@ -737,7 +737,7 @@ func TestEngine_CompactWithSeriesInOneFile(t *testing.T) { verify() if err := e.Compact(true); err != nil { - t.Fatalf("error compacting: %s", err.Error) + t.Fatalf("error compacting: %s", err.Error()) } fmt.Println("verify 2") verify() @@ -748,7 +748,7 @@ func TestEngine_CompactWithSeriesInOneFile(t *testing.T) { } if err := e.Compact(true); err != nil { - t.Fatalf("error compacting: %s", err.Error) + t.Fatalf("error compacting: %s", err.Error()) } tx1, _ := e.Begin(false) defer tx1.Rollback() @@ -813,7 +813,7 @@ func TestEngine_CompactionWithCopiedBlocks(t *testing.T) { verify() if err := e.Compact(true); err != nil { - t.Fatalf("error compacting: %s", err.Error) + t.Fatalf("error compacting: %s", err.Error()) } fmt.Println("verify 2") verify() @@ -824,7 +824,7 @@ func TestEngine_CompactionWithCopiedBlocks(t *testing.T) { } if err := e.Compact(true); err != nil { - t.Fatalf("error compacting: %s", err.Error) + t.Fatalf("error compacting: %s", err.Error()) } fmt.Println("verify 3") verify() @@ -844,7 +844,7 @@ func TestEngine_CompactionWithCopiedBlocks(t *testing.T) { } if err := e.Compact(true); err != nil { - t.Fatalf("error compacting: %s", err.Error) + t.Fatalf("error compacting: %s", err.Error()) } verify() From 4beca1a2453d0231dd66360dc9659efe2c124206 Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Wed, 30 Sep 2015 20:41:50 -0400 Subject: [PATCH 119/139] Implement reverse cursor direction on pd1 --- tsdb/engine/pd1/cursor.go | 25 +-- tsdb/engine/pd1/pd1.go | 292 ++++++++++++++++++++++++++---------- tsdb/engine/pd1/pd1_test.go | 120 ++++++++++++++- tsdb/engine/pd1/wal.go | 2 +- 4 files changed, 344 insertions(+), 95 deletions(-) diff --git a/tsdb/engine/pd1/cursor.go b/tsdb/engine/pd1/cursor.go index d31252aa32a..8319992c25b 100644 --- a/tsdb/engine/pd1/cursor.go +++ b/tsdb/engine/pd1/cursor.go @@ -39,45 +39,48 @@ func (c *combinedEngineCursor) Ascending() bool { } func (c *combinedEngineCursor) read() (key int64, value interface{}) { - key = tsdb.EOF + if c.walKeyBuf == tsdb.EOF && c.engineKeyBuf == tsdb.EOF { + return tsdb.EOF, nil + } // handle the case where they have the same point - if c.walKeyBuf != tsdb.EOF && c.walKeyBuf == c.engineKeyBuf { + if c.walKeyBuf == c.engineKeyBuf { // keep the wal value since it will overwrite the engine value key = c.walKeyBuf value = c.walValueBuf c.walKeyBuf, c.walValueBuf = c.walCursor.Next() - // drop the engine value - _, _ = c.engineCursor.Next() + + // overwrite the buffered engine values + c.engineKeyBuf, c.engineValueBuf = c.engineCursor.Next() return } // ascending order if c.ascending { - if c.engineKeyBuf == tsdb.EOF || (c.walKeyBuf != tsdb.EOF && c.walKeyBuf < c.engineKeyBuf) { + if c.walKeyBuf != tsdb.EOF && (c.walKeyBuf < c.engineKeyBuf || c.engineKeyBuf == tsdb.EOF) { key = c.walKeyBuf value = c.walValueBuf c.walKeyBuf, c.walValueBuf = c.walCursor.Next() - } else { - key = c.engineKeyBuf - value = c.engineValueBuf - c.engineKeyBuf, c.engineValueBuf = c.engineCursor.Next() + return } + key = c.engineKeyBuf + value = c.engineValueBuf + c.engineKeyBuf, c.engineValueBuf = c.engineCursor.Next() return } // descending order - if c.engineKeyBuf == tsdb.EOF || (c.walKeyBuf != tsdb.EOF && c.walKeyBuf > c.engineKeyBuf) { + if c.walKeyBuf != tsdb.EOF && c.walKeyBuf > c.engineKeyBuf { key = c.walKeyBuf value = c.walValueBuf c.walKeyBuf, c.walValueBuf = c.walCursor.Next() return } + key = c.engineKeyBuf value = c.engineValueBuf c.engineKeyBuf, c.engineValueBuf = c.engineCursor.Next() - return } diff --git a/tsdb/engine/pd1/pd1.go b/tsdb/engine/pd1/pd1.go index 5f64c6c2f5a..44f8325277f 100644 --- a/tsdb/engine/pd1/pd1.go +++ b/tsdb/engine/pd1/pd1.go @@ -1567,6 +1567,8 @@ type cursor struct { ascending bool + blockPositions []uint32 // only used for descending queries + // time acending list of data files files []*dataFile } @@ -1584,15 +1586,32 @@ func (c *cursor) SeekTo(seek int64) (int64, interface{}) { return tsdb.EOF, nil } - if seek < c.files[0].MinTime() { - c.filesPos = 0 - c.f = c.files[0] + if c.ascending { + if seek <= c.files[0].MinTime() { + c.filesPos = 0 + c.f = c.files[0] + } else { + for i, f := range c.files { + if seek >= f.MinTime() && seek <= f.MaxTime() { + c.filesPos = i + c.f = f + break + } + } + } } else { - for i, f := range c.files { - if seek >= f.MinTime() && seek <= f.MaxTime() { - c.filesPos = i - c.f = f - break + if seek >= c.files[len(c.files)-1].MaxTime() { + c.filesPos = len(c.files) - 1 + c.f = c.files[c.filesPos] + } else if seek < c.files[0].MinTime() { + return tsdb.EOF, nil + } else { + for i, f := range c.files { + if seek >= f.MinTime() && seek <= f.MaxTime() { + c.filesPos = i + c.f = f + break + } } } } @@ -1601,110 +1620,227 @@ func (c *cursor) SeekTo(seek int64) (int64, interface{}) { return tsdb.EOF, nil } - // TODO: make this for the reverse direction cursor - - // now find the spot in the file we need to go + // find the first file we need to check in for { - pos := c.f.StartingPositionForID(c.id) + if c.filesPos < 0 || c.filesPos >= len(c.files) { + return tsdb.EOF, nil + } + c.f = c.files[c.filesPos] + + c.pos = c.f.StartingPositionForID(c.id) // if this id isn't in this file, move to next one or return - if pos == 0 { - c.filesPos++ - if c.filesPos >= len(c.files) { - return tsdb.EOF, nil + if c.pos == 0 { + if c.ascending { + c.filesPos++ + } else { + c.filesPos-- + c.blockPositions = nil } - c.f = c.files[c.filesPos] continue } - // seek to the block and values we're looking for - for { - // if the time is between this block and the next, - // decode this block and go, otherwise seek to next block - length := btou32(c.f.mmap[pos+8 : pos+12]) - - // if the next block has a time less than what we're seeking to, - // skip decoding this block and continue on - nextBlockPos := pos + 12 + length - if nextBlockPos < c.f.indexPosition() { - nextBlockID := btou64(c.f.mmap[nextBlockPos : nextBlockPos+8]) - if nextBlockID == c.id { - nextBlockTime := int64(btou64(c.f.mmap[nextBlockPos+12 : nextBlockPos+20])) - if nextBlockTime <= seek { - pos = nextBlockPos - continue - } + // handle seek for correct order + k := tsdb.EOF + var v interface{} + + if c.ascending { + k, v = c.seekAscending(seek) + } else { + k, v = c.seekDescending(seek) + } + + if k != tsdb.EOF { + return k, v + } + + if c.ascending { + c.filesPos++ + } else { + c.filesPos-- + c.blockPositions = nil + } + } +} + +func (c *cursor) seekAscending(seek int64) (int64, interface{}) { + // seek to the block and values we're looking for + for { + // if the time is between this block and the next, + // decode this block and go, otherwise seek to next block + length := c.blockLength(c.pos) + + // if the next block has a time less than what we're seeking to, + // skip decoding this block and continue on + nextBlockPos := c.pos + blockHeaderSize + length + if nextBlockPos < c.f.indexPosition() { + nextBlockID := btou64(c.f.mmap[nextBlockPos : nextBlockPos+8]) + if nextBlockID == c.id { + nextBlockTime := c.blockMinTime(nextBlockPos) + if nextBlockTime <= seek { + c.pos = nextBlockPos + continue } } + } - // it must be in this block or not at all - t, v := c.decodeBlockAndGetValues(pos) - if t >= seek { - return t, v + // it must be in this block or not at all + c.decodeBlock(c.pos) + + // see if we can find it in this block + for i, v := range c.vals { + if v.Time().UnixNano() >= seek { + c.vals = c.vals[i+1:] + return v.Time().UnixNano(), v.Value() } + } + } +} + +func (c *cursor) seekDescending(seek int64) (int64, interface{}) { + c.setBlockPositions() + if len(c.blockPositions) == 0 { + return tsdb.EOF, nil + } + + for i := len(c.blockPositions) - 1; i >= 0; i-- { + pos := c.blockPositions[i] + if c.blockMinTime(pos) > seek { + continue + } + + c.decodeBlock(pos) + c.blockPositions = c.blockPositions[:i] - // wasn't in the first value popped out of the block, check the rest - for i, v := range c.vals { - if v.Time().UnixNano() >= seek { - c.vals = c.vals[i+1:] - return v.Time().UnixNano(), v.Value() + for i := len(c.vals) - 1; i >= 0; i-- { + val := c.vals[i] + if seek >= val.UnixNano() { + c.vals = c.vals[:i] + return val.UnixNano(), val.Value() + } + if seek < val.UnixNano() { + // we need to move to the next block + if i == 0 { + break } + val := c.vals[i-1] + c.vals = c.vals[:i-1] + return val.UnixNano(), val.Value() } + } + c.blockPositions = c.blockPositions[:i] + } - // not in this one, let the top loop look for it in the next file - break + return tsdb.EOF, nil +} + +func (c *cursor) blockMinTime(pos uint32) int64 { + return int64(btou64(c.f.mmap[pos+12 : pos+20])) +} + +func (c *cursor) setBlockPositions() { + pos := c.pos + + for { + if pos >= c.f.indexPosition() { + return + } + + length := c.blockLength(pos) + id := btou64(c.f.mmap[pos : pos+8]) + + if id != c.id { + return } + + c.blockPositions = append(c.blockPositions, pos) + pos += blockHeaderSize + length } } func (c *cursor) Next() (int64, interface{}) { - if len(c.vals) == 0 { - // if we have a file set, see if the next block is for this ID - if c.f != nil && c.pos < c.f.size { - nextBlockID := btou64(c.f.mmap[c.pos : c.pos+8]) - if nextBlockID == c.id && c.pos != c.f.indexPosition() { - return c.decodeBlockAndGetValues(c.pos) - } + if c.ascending { + return c.nextAscending() + } + return c.nextDescending() +} + +func (c *cursor) nextAscending() (int64, interface{}) { + if len(c.vals) > 0 { + v := c.vals[0] + c.vals = c.vals[1:] + + return v.Time().UnixNano(), v.Value() + } + + // if we have a file set, see if the next block is for this ID + if c.f != nil && c.pos < c.f.indexPosition() { + nextBlockID := btou64(c.f.mmap[c.pos : c.pos+8]) + if nextBlockID == c.id { + c.decodeBlock(c.pos) + return c.nextAscending() } + } - // if the file is nil we hit the end of the previous file, advance the file cursor - if c.f != nil { - c.filesPos++ + // loop through the files until we hit the next one that has this id + for { + c.filesPos++ + if c.filesPos >= len(c.files) { + return tsdb.EOF, nil } + c.f = c.files[c.filesPos] - // loop until we find a file with some data - for c.filesPos < len(c.files) { - f := c.files[c.filesPos] + startingPos := c.f.StartingPositionForID(c.id) + if startingPos == 0 { + // move to next file because it isn't in this one + continue + } - startingPos := f.StartingPositionForID(c.id) - if startingPos == 0 { - c.filesPos++ - continue - } - c.f = f - return c.decodeBlockAndGetValues(startingPos) + // we have a block with this id, decode and return + c.decodeBlock(startingPos) + return c.nextAscending() + } +} + +func (c *cursor) nextDescending() (int64, interface{}) { + if len(c.vals) > 0 { + v := c.vals[len(c.vals)-1] + if len(c.vals) >= 1 { + c.vals = c.vals[:len(c.vals)-1] + } else { + c.vals = nil } + return v.UnixNano(), v.Value() + } - // we didn't get to a file that had a next value - return tsdb.EOF, nil + for i := len(c.blockPositions) - 1; i >= 0; i-- { + c.decodeBlock(c.blockPositions[i]) + c.blockPositions = c.blockPositions[:i] + if len(c.vals) == 0 { + continue + } + val := c.vals[len(c.vals)-1] + c.vals = c.vals[:len(c.vals)-1] + return val.UnixNano(), val.Value() } - v := c.vals[0] - c.vals = c.vals[1:] + return tsdb.EOF, nil +} - return v.Time().UnixNano(), v.Value() +func (c *cursor) blockLength(pos uint32) uint32 { + return btou32(c.f.mmap[pos+8 : pos+12]) } -func (c *cursor) decodeBlockAndGetValues(position uint32) (int64, interface{}) { - length := btou32(c.f.mmap[position+8 : position+12]) - block := c.f.mmap[position+12 : position+12+length] +func (c *cursor) decodeBlock(position uint32) { + length := c.blockLength(position) + block := c.f.mmap[position+blockHeaderSize : position+blockHeaderSize+length] c.vals, _ = DecodeBlock(block) - c.pos = position + 12 + length - v := c.vals[0] - c.vals = c.vals[1:] - - return v.Time().UnixNano(), v.Value() + // only adavance the position if we're asceending. + // Descending queries use the blockPositions + if c.ascending { + c.pos = position + blockHeaderSize + length + } } func (c *cursor) Ascending() bool { return c.ascending } diff --git a/tsdb/engine/pd1/pd1_test.go b/tsdb/engine/pd1/pd1_test.go index e11a9baf929..5c28572a846 100644 --- a/tsdb/engine/pd1/pd1_test.go +++ b/tsdb/engine/pd1/pd1_test.go @@ -170,6 +170,7 @@ func TestEngine_WriteIndexQueryAcrossDataFiles(t *testing.T) { points = points[1:] for _, p := range points { + fmt.Println("> ", p.Time()) k, v := c.Next() val := p.Fields()["value"] if p.UnixNano() != k || val != v { @@ -178,9 +179,13 @@ func TestEngine_WriteIndexQueryAcrossDataFiles(t *testing.T) { } } + fmt.Println("v1") verify("cpu,host=A", []models.Point{p1, p7, p3, p5}, 0) + fmt.Println("v2") verify("cpu,host=B", []models.Point{p2, p8, p4, p6}, 0) + fmt.Println("v3") verify("cpu,host=A", []models.Point{p5}, 5000000000) + fmt.Println("v4") verify("cpu,host=B", []models.Point{p6}, 5000000000) } @@ -681,6 +686,81 @@ func TestEngine_WritePointsInMultipleRequestsWithSameTime(t *testing.T) { verify() } +func TestEngine_CursorDescendingOrder(t *testing.T) { + e := OpenDefaultEngine() + defer e.Cleanup() + + fields := []string{"value"} + + p1 := parsePoint("foo value=1 1") + p2 := parsePoint("foo value=2 2") + + e.WAL.SkipCache = false + + if err := e.WritePoints([]models.Point{p1, p2}, nil, nil); err != nil { + t.Fatalf("failed to write points: %s", err.Error()) + } + + verify := func() { + tx, _ := e.Begin(false) + defer tx.Rollback() + c := tx.Cursor("foo", fields, nil, false) + fmt.Println("seek") + k, v := c.SeekTo(5000000) + if k != 2 { + t.Fatalf("expected 2 time but got %d", k) + } + if v != float64(2) { + t.Fatalf("expected 2 for value but got %f", v.(float64)) + } + fmt.Println("next1") + k, v = c.Next() + if k != 1 { + t.Fatalf("expected 1 time but got %d", k) + } + fmt.Println("next2") + if v != float64(1) { + t.Fatalf("expected 1 for value but got %f", v.(float64)) + } + k, _ = c.Next() + if k != tsdb.EOF { + t.Fatal("expected EOF", k) + } + } + fmt.Println("verify 1") + verify() + + if err := e.WAL.Flush(); err != nil { + t.Fatalf("error flushing WAL %s", err.Error) + } + + fmt.Println("verify 2") + verify() + + p3 := parsePoint("foo value=3 3") + + if err := e.WritePoints([]models.Point{p3}, nil, nil); err != nil { + t.Fatalf("failed to write points: %s", err.Error()) + } + + func() { + tx, _ := e.Begin(false) + defer tx.Rollback() + c := tx.Cursor("foo", fields, nil, false) + k, v := c.SeekTo(234232) + if k != 3 { + t.Fatalf("expected 3 time but got %d", k) + } + if v != float64(3) { + t.Fatalf("expected 3 for value but got %f", v.(float64)) + } + k, _ = c.Next() + if k != 2 { + t.Fatalf("expected 2 time but got %d", k) + } + }() +} + func TestEngine_CompactWithSeriesInOneFile(t *testing.T) { e := OpenDefaultEngine() defer e.Cleanup() @@ -711,7 +791,7 @@ func TestEngine_CompactWithSeriesInOneFile(t *testing.T) { verify := func() { tx, _ := e.Begin(false) defer tx.Rollback() - c := tx.Cursor("cpu,host=A", fields, nil, false) + c := tx.Cursor("cpu,host=A", fields, nil, true) k, v := c.SeekTo(0) if k != 1000000000 { t.Fatalf("expected time 1000000000 but got %d", k) @@ -723,7 +803,7 @@ func TestEngine_CompactWithSeriesInOneFile(t *testing.T) { if k != 3000000000 { t.Fatalf("expected time 3000000000 but got %d", k) } - c = tx.Cursor("cpu,host=B", fields, nil, false) + c = tx.Cursor("cpu,host=B", fields, nil, true) k, v = c.SeekTo(0) if k != 2000000000 { t.Fatalf("expected time 2000000000 but got %d", k) @@ -752,7 +832,7 @@ func TestEngine_CompactWithSeriesInOneFile(t *testing.T) { } tx1, _ := e.Begin(false) defer tx1.Rollback() - c := tx1.Cursor("cpu,host=A", fields, nil, false) + c := tx1.Cursor("cpu,host=A", fields, nil, true) k, v := c.SeekTo(0) if k != 1000000000 { t.Fatalf("expected time 1000000000 but got %d", k) @@ -796,7 +876,7 @@ func TestEngine_CompactionWithCopiedBlocks(t *testing.T) { verify := func() { tx, _ := e.Begin(false) defer tx.Rollback() - c := tx.Cursor("cpu,host=A", fields, nil, false) + c := tx.Cursor("cpu,host=A", fields, nil, true) k, _ := c.SeekTo(0) if k != 1000000000 { t.Fatalf("expected time 1000000000 but got %d", k) @@ -875,7 +955,7 @@ func TestEngine_RewritingOldBlocks(t *testing.T) { tx, _ := e.Begin(false) defer tx.Rollback() - c := tx.Cursor("cpu,host=A", fields, nil, false) + c := tx.Cursor("cpu,host=A", fields, nil, true) k, _ := c.SeekTo(0) if k != 1000000000 { t.Fatalf("expected time 1000000000 but got %d", k) @@ -961,6 +1041,36 @@ func TestEngine_WriteIntoCompactedFile(t *testing.T) { } } +func TestEngine_DuplicatePointsInWalAndIndex(t *testing.T) { + e := OpenDefaultEngine() + defer e.Cleanup() + + fields := []string{"value"} + p1 := parsePoint("cpu,host=A value=1.1 1000000000") + p2 := parsePoint("cpu,host=A value=1.2 1000000000") + if err := e.WritePoints([]models.Point{p1}, nil, nil); err != nil { + t.Fatalf("failed to write points: %s", err.Error()) + } + e.WAL.SkipCache = false + if err := e.WritePoints([]models.Point{p2}, nil, nil); err != nil { + t.Fatalf("failed to write points: %s", err.Error()) + } + tx, _ := e.Begin(false) + defer tx.Rollback() + c := tx.Cursor("cpu,host=A", fields, nil, true) + k, v := c.SeekTo(0) + if k != 1000000000 { + t.Fatalf("wrong time: %d", k) + } + if v != 1.2 { + t.Fatalf("wrong value: %f", v.(float64)) + } + k, _ = c.Next() + if k != tsdb.EOF { + t.Fatal("expected EOF", k) + } +} + // Engine represents a test wrapper for pd1.Engine. type Engine struct { *pd1.Engine diff --git a/tsdb/engine/pd1/wal.go b/tsdb/engine/pd1/wal.go index ef4399912cd..b6cc0cc2143 100644 --- a/tsdb/engine/pd1/wal.go +++ b/tsdb/engine/pd1/wal.go @@ -657,7 +657,7 @@ func (c *walCursor) SeekTo(seek int64) (int64, interface{}) { // If seek is not in the cache, return the last value in the cache if !c.ascending && c.position >= len(c.cache) { - c.position = len(c.cache) + c.position = len(c.cache) - 1 } // Make sure our position points to something in the cache From 0a11a2fdbcd7e16408981de33ffc8f0d8befa081 Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Thu, 1 Oct 2015 15:16:28 -0400 Subject: [PATCH 120/139] Add deletes to new storage engine --- tsdb/engine/pd1/cursor.go | 295 +++++++++++++++++++++ tsdb/engine/pd1/pd1.go | 505 ++++++++++++++---------------------- tsdb/engine/pd1/pd1_test.go | 167 +++++++++++- tsdb/engine/pd1/tx.go | 21 +- tsdb/engine/pd1/wal.go | 31 ++- tsdb/engine/pd1/wal_test.go | 2 + 6 files changed, 711 insertions(+), 310 deletions(-) diff --git a/tsdb/engine/pd1/cursor.go b/tsdb/engine/pd1/cursor.go index 8319992c25b..4b49972131e 100644 --- a/tsdb/engine/pd1/cursor.go +++ b/tsdb/engine/pd1/cursor.go @@ -149,3 +149,298 @@ func (m *multiFieldCursor) read() (int64, interface{}) { } return t, mm } + +type emptyCursor struct { + ascending bool +} + +func (c *emptyCursor) Next() (int64, interface{}) { return tsdb.EOF, nil } +func (c *emptyCursor) SeekTo(key int64) (int64, interface{}) { return tsdb.EOF, nil } +func (c *emptyCursor) Ascending() bool { return c.ascending } + +type cursor struct { + id uint64 + f *dataFile + filesPos int // the index in the files slice we're looking at + pos uint32 + vals Values + + ascending bool + + blockPositions []uint32 // only used for descending queries + + // time acending list of data files + files []*dataFile +} + +func newCursor(id uint64, files []*dataFile, ascending bool) *cursor { + return &cursor{ + id: id, + ascending: ascending, + files: files, + } +} + +func (c *cursor) SeekTo(seek int64) (int64, interface{}) { + if len(c.files) == 0 { + return tsdb.EOF, nil + } + + if c.ascending { + if seek <= c.files[0].MinTime() { + c.filesPos = 0 + c.f = c.files[0] + } else { + for i, f := range c.files { + if seek >= f.MinTime() && seek <= f.MaxTime() { + c.filesPos = i + c.f = f + break + } + } + } + } else { + if seek >= c.files[len(c.files)-1].MaxTime() { + c.filesPos = len(c.files) - 1 + c.f = c.files[c.filesPos] + } else if seek < c.files[0].MinTime() { + return tsdb.EOF, nil + } else { + for i, f := range c.files { + if seek >= f.MinTime() && seek <= f.MaxTime() { + c.filesPos = i + c.f = f + break + } + } + } + } + + if c.f == nil { + return tsdb.EOF, nil + } + + // find the first file we need to check in + for { + if c.filesPos < 0 || c.filesPos >= len(c.files) { + return tsdb.EOF, nil + } + c.f = c.files[c.filesPos] + + c.pos = c.f.StartingPositionForID(c.id) + + // if this id isn't in this file, move to next one or return + if c.pos == 0 { + if c.ascending { + c.filesPos++ + } else { + c.filesPos-- + c.blockPositions = nil + } + continue + } + + // handle seek for correct order + k := tsdb.EOF + var v interface{} + + if c.ascending { + k, v = c.seekAscending(seek) + } else { + k, v = c.seekDescending(seek) + } + + if k != tsdb.EOF { + return k, v + } + + if c.ascending { + c.filesPos++ + } else { + c.filesPos-- + c.blockPositions = nil + } + } +} + +func (c *cursor) seekAscending(seek int64) (int64, interface{}) { + // seek to the block and values we're looking for + for { + // if the time is between this block and the next, + // decode this block and go, otherwise seek to next block + length := c.blockLength(c.pos) + + // if the next block has a time less than what we're seeking to, + // skip decoding this block and continue on + nextBlockPos := c.pos + blockHeaderSize + length + if nextBlockPos < c.f.indexPosition() { + nextBlockID := btou64(c.f.mmap[nextBlockPos : nextBlockPos+8]) + if nextBlockID == c.id { + nextBlockTime := c.blockMinTime(nextBlockPos) + if nextBlockTime <= seek { + c.pos = nextBlockPos + continue + } + } + } + + // it must be in this block or not at all + c.decodeBlock(c.pos) + + // see if we can find it in this block + for i, v := range c.vals { + if v.Time().UnixNano() >= seek { + c.vals = c.vals[i+1:] + return v.Time().UnixNano(), v.Value() + } + } + } +} + +func (c *cursor) seekDescending(seek int64) (int64, interface{}) { + c.setBlockPositions() + if len(c.blockPositions) == 0 { + return tsdb.EOF, nil + } + + for i := len(c.blockPositions) - 1; i >= 0; i-- { + pos := c.blockPositions[i] + if c.blockMinTime(pos) > seek { + continue + } + + c.decodeBlock(pos) + c.blockPositions = c.blockPositions[:i] + + for i := len(c.vals) - 1; i >= 0; i-- { + val := c.vals[i] + if seek >= val.UnixNano() { + c.vals = c.vals[:i] + return val.UnixNano(), val.Value() + } + if seek < val.UnixNano() { + // we need to move to the next block + if i == 0 { + break + } + val := c.vals[i-1] + c.vals = c.vals[:i-1] + return val.UnixNano(), val.Value() + } + } + c.blockPositions = c.blockPositions[:i] + } + + return tsdb.EOF, nil +} + +func (c *cursor) blockMinTime(pos uint32) int64 { + return int64(btou64(c.f.mmap[pos+12 : pos+20])) +} + +func (c *cursor) setBlockPositions() { + pos := c.pos + + for { + if pos >= c.f.indexPosition() { + return + } + + length := c.blockLength(pos) + id := btou64(c.f.mmap[pos : pos+8]) + + if id != c.id { + return + } + + c.blockPositions = append(c.blockPositions, pos) + pos += blockHeaderSize + length + } +} + +func (c *cursor) Next() (int64, interface{}) { + if c.ascending { + return c.nextAscending() + } + return c.nextDescending() +} + +func (c *cursor) nextAscending() (int64, interface{}) { + if len(c.vals) > 0 { + v := c.vals[0] + c.vals = c.vals[1:] + + return v.Time().UnixNano(), v.Value() + } + + // if we have a file set, see if the next block is for this ID + if c.f != nil && c.pos < c.f.indexPosition() { + nextBlockID := btou64(c.f.mmap[c.pos : c.pos+8]) + if nextBlockID == c.id { + c.decodeBlock(c.pos) + return c.nextAscending() + } + } + + // loop through the files until we hit the next one that has this id + for { + c.filesPos++ + if c.filesPos >= len(c.files) { + return tsdb.EOF, nil + } + c.f = c.files[c.filesPos] + + startingPos := c.f.StartingPositionForID(c.id) + if startingPos == 0 { + // move to next file because it isn't in this one + continue + } + + // we have a block with this id, decode and return + c.decodeBlock(startingPos) + return c.nextAscending() + } +} + +func (c *cursor) nextDescending() (int64, interface{}) { + if len(c.vals) > 0 { + v := c.vals[len(c.vals)-1] + if len(c.vals) >= 1 { + c.vals = c.vals[:len(c.vals)-1] + } else { + c.vals = nil + } + return v.UnixNano(), v.Value() + } + + for i := len(c.blockPositions) - 1; i >= 0; i-- { + c.decodeBlock(c.blockPositions[i]) + c.blockPositions = c.blockPositions[:i] + if len(c.vals) == 0 { + continue + } + val := c.vals[len(c.vals)-1] + c.vals = c.vals[:len(c.vals)-1] + return val.UnixNano(), val.Value() + } + + return tsdb.EOF, nil +} + +func (c *cursor) blockLength(pos uint32) uint32 { + return btou32(c.f.mmap[pos+8 : pos+12]) +} + +func (c *cursor) decodeBlock(position uint32) { + length := c.blockLength(position) + block := c.f.mmap[position+blockHeaderSize : position+blockHeaderSize+length] + c.vals, _ = DecodeBlock(block) + + // only adavance the position if we're asceending. + // Descending queries use the blockPositions + if c.ascending { + c.pos = position + blockHeaderSize + length + } +} + +func (c *cursor) Ascending() bool { return c.ascending } diff --git a/tsdb/engine/pd1/pd1.go b/tsdb/engine/pd1/pd1.go index 44f8325277f..c6a8e7c3c5f 100644 --- a/tsdb/engine/pd1/pd1.go +++ b/tsdb/engine/pd1/pd1.go @@ -102,6 +102,10 @@ type Engine struct { compactionRunning bool lastCompactionTime time.Time + // deletes is a map of keys that are deleted, but haven't yet been + // compacted and flushed + deletes map[uint64]bool + collisionsLock sync.RWMutex collisions map[string]uint64 @@ -158,8 +162,9 @@ func (e *Engine) PerformMaintenance() { e.filesLock.RLock() running := e.compactionRunning + deletesPending := len(e.deletes) > 0 e.filesLock.RUnlock() - if running { + if running || deletesPending { return } @@ -214,11 +219,17 @@ func (e *Engine) Open() error { } sort.Sort(e.files) - if err := e.WAL.Open(); err != nil { + if err := e.readCollisions(); err != nil { return err } - if err := e.readCollisions(); err != nil { + e.deletes = make(map[uint64]bool) + + // mark the last compaction as now so it doesn't try to compact while + // flushing the WAL on load + e.lastCompactionTime = time.Now() + + if err := e.WAL.Open(); err != nil { return err } @@ -249,6 +260,7 @@ func (e *Engine) Close() error { e.files = nil e.currentFileID = 0 e.collisions = nil + e.deletes = nil return nil } @@ -307,6 +319,14 @@ func (e *Engine) WritePoints(points []models.Point, measurementFieldsToSave map[ } func (e *Engine) Write(pointsByKey map[string]Values, measurementFieldsToSave map[string]*tsdb.MeasurementFields, seriesToCreate []*tsdb.SeriesCreate) error { + // Flush any deletes before writing new data from the WAL + e.filesLock.RLock() + hasDeletes := len(e.deletes) > 0 + e.filesLock.RUnlock() + if hasDeletes { + e.flushDeletes() + } + err, startTime, endTime, valuesByID := e.convertKeysAndWriteMetadata(pointsByKey, measurementFieldsToSave, seriesToCreate) if err != nil { return err @@ -397,6 +417,16 @@ func (e *Engine) Write(pointsByKey map[string]Values, measurementFieldsToSave ma return nil } +// MarkDeletes will mark the given keys for deletion in memory. They will be deleted from data +// files on the next flush. This mainly for the WAL to use on startup +func (e *Engine) MarkDeletes(keys []string) { + e.filesLock.Lock() + defer e.filesLock.Unlock() + for _, k := range keys { + e.deletes[e.keyToID(k)] = true + } +} + // filesAndLock returns the data files that match the given range and // ensures that the write lock will hold for the entire range func (e *Engine) filesAndLock(min, max int64) (a dataFiles, lockStart, lockEnd int64) { @@ -613,7 +643,7 @@ func (e *Engine) Compact(fullCompaction bool) error { } } - err, newDF := e.writeIndexAndGetDataFile(f, minTime, maxTime, newIDs, newPositions) + newDF, err := e.writeIndexAndGetDataFile(f, minTime, maxTime, newIDs, newPositions) if err != nil { return err } @@ -664,50 +694,51 @@ func (e *Engine) writeBlock(f *os.File, id uint64, block []byte) error { return err } -func (e *Engine) writeIndexAndGetDataFile(f *os.File, minTime, maxTime int64, ids []uint64, newPositions []uint32) (error, *dataFile) { +func (e *Engine) writeIndexAndGetDataFile(f *os.File, minTime, maxTime int64, ids []uint64, newPositions []uint32) (*dataFile, error) { // write the file index, starting with the series ids and their positions for i, id := range ids { if _, err := f.Write(u64tob(id)); err != nil { - return err, nil + return nil, err } if _, err := f.Write(u32tob(newPositions[i])); err != nil { - return err, nil + return nil, err } } // write the min time, max time if _, err := f.Write(append(u64tob(uint64(minTime)), u64tob(uint64(maxTime))...)); err != nil { - return err, nil + return nil, err } // series count if _, err := f.Write(u32tob(uint32(len(ids)))); err != nil { - return err, nil + return nil, err } // sync it and see4k back to the beginning to hand off to the mmap if err := f.Sync(); err != nil { - return err, nil + return nil, err } if _, err := f.Seek(0, 0); err != nil { - return err, nil + return nil, err } // now open it as a memory mapped data file newDF, err := NewDataFile(f) if err != nil { - return err, nil + return nil, err } - return nil, newDF + return newDF, nil } func (e *Engine) shouldCompact() bool { e.filesLock.RLock() running := e.compactionRunning since := time.Since(e.lastCompactionTime) + deletesPending := len(e.deletes) > 0 e.filesLock.RUnlock() - if running || since < e.IndexMinimumCompactionInterval { + if running || since < e.IndexMinimumCompactionInterval || deletesPending { return false } return len(e.filesToCompact()) >= e.CompactionFileCount @@ -1069,7 +1100,7 @@ func (e *Engine) rewriteFile(oldDF *dataFile, valuesByID map[uint64]Values) erro } } - err, newDF := e.writeIndexAndGetDataFile(f, minTime, maxTime, ids, newPositions) + newDF, err := e.writeIndexAndGetDataFile(f, minTime, maxTime, ids, newPositions) if err != nil { f.Close() return err @@ -1103,6 +1134,81 @@ func (e *Engine) rewriteFile(oldDF *dataFile, valuesByID map[uint64]Values) erro return nil } +// flushDeletes will lock the entire shard and rewrite all index files so they no +// longer contain the flushed IDs +func (e *Engine) flushDeletes() error { + e.writeLock.LockRange(math.MinInt64, math.MaxInt64) + defer e.writeLock.UnlockRange(math.MinInt64, math.MaxInt64) + + files := e.copyFilesCollection() + newFiles := make(dataFiles, 0, len(files)) + for _, f := range files { + newFiles = append(newFiles, e.writeNewFileExcludeDeletes(f)) + } + + e.filesLock.Lock() + defer e.filesLock.Unlock() + e.files = newFiles + e.deletes = make(map[uint64]bool) + + e.deletesPending.Add(1) + go func() { + for _, oldDF := range files { + if err := oldDF.Delete(); err != nil { + e.logger.Println("ERROR DELETING FROM REWRITE:", oldDF.f.Name()) + } + } + e.deletesPending.Done() + }() + return nil +} + +func (e *Engine) writeNewFileExcludeDeletes(oldDF *dataFile) *dataFile { + // TODO: add checkpoint file that indicates if this completed or not + f, err := os.OpenFile(e.nextFileName(), os.O_CREATE|os.O_RDWR, 0666) + if err != nil { + panic(fmt.Sprintf("error opening new index file: %s", err.Error())) + } + // write the magic number + if _, err := f.Write(u32tob(magicNumber)); err != nil { + panic(fmt.Sprintf("error writing new index file: %s", err.Error())) + } + + ids := make([]uint64, 0) + positions := make([]uint32, 0) + + indexPosition := oldDF.indexPosition() + currentPosition := uint32(fileHeaderSize) + currentID := uint64(0) + for currentPosition < indexPosition { + id := btou64(oldDF.mmap[currentPosition : currentPosition+8]) + length := btou32(oldDF.mmap[currentPosition+8 : currentPosition+blockHeaderSize]) + newPosition := currentPosition + blockHeaderSize + length + + if _, ok := e.deletes[id]; ok { + currentPosition = newPosition + continue + } + + if _, err := f.Write(oldDF.mmap[currentPosition:newPosition]); err != nil { + panic(fmt.Sprintf("error writing new index file: %s", err.Error())) + } + if id != currentID { + currentID = id + ids = append(ids, id) + positions = append(positions, currentPosition) + } + currentPosition = newPosition + } + + df, err := e.writeIndexAndGetDataFile(f, oldDF.MinTime(), oldDF.MaxTime(), ids, positions) + if err != nil { + panic(fmt.Sprintf("error writing new index file: %s", err.Error())) + } + + return df +} + func (e *Engine) nextFileName() string { e.currentFileID++ return filepath.Join(e.path, fmt.Sprintf("%07d.%s", e.currentFileID, Format)) @@ -1146,14 +1252,86 @@ func (e *Engine) replaceCompressedFile(name string, data []byte) error { return os.Rename(tmpName, filepath.Join(e.path, name)) } +// keysWithFields takes the map of measurements to their fields and a set of series keys +// and returns the columnar keys for the keys and fields +func (e *Engine) keysWithFields(fields map[string]*tsdb.MeasurementFields, keys []string) []string { + e.WAL.cacheLock.RLock() + defer e.WAL.cacheLock.RUnlock() + + a := make([]string, 0) + for _, k := range keys { + measurement := tsdb.MeasurementFromSeriesKey(k) + + // add the fields from the index + mf := fields[measurement] + if mf != nil { + for _, f := range mf.Fields { + a = append(a, seriesFieldKey(k, f.Name)) + } + } + + // now add any fields from the WAL that haven't been flushed yet + mf = e.WAL.measurementFieldsCache[measurement] + if mf != nil { + for _, f := range mf.Fields { + a = append(a, seriesFieldKey(k, f.Name)) + } + } + } + + return a +} + // DeleteSeries deletes the series from the engine. func (e *Engine) DeleteSeries(keys []string) error { + fields, err := e.readFields() + if err != nil { + return err + } + + keyFields := e.keysWithFields(fields, keys) + + return e.deleteKeyFields(keyFields) +} + +func (e *Engine) deleteKeyFields(keyFields []string) error { + err := e.WAL.DeleteSeries(keyFields) + if err != nil { + return err + } + e.filesLock.Lock() + defer e.filesLock.Unlock() + + for _, k := range keyFields { + e.deletes[e.keyToID(k)] = true + } + return nil } // DeleteMeasurement deletes a measurement and all related series. func (e *Engine) DeleteMeasurement(name string, seriesKeys []string) error { - return nil + e.metaLock.Lock() + defer e.metaLock.Unlock() + + // remove the field data from the index + fields, err := e.readFields() + if err != nil { + return err + } + + keyFields := e.keysWithFields(fields, seriesKeys) + + delete(fields, name) + + if err := e.writeFields(fields); err != nil { + return err + } + + e.WAL.DropMeasurementFields(name) + + // now delete all the measurement's series + return e.deleteKeyFields(keyFields) } // SeriesCount returns the number of series buckets on the shard. @@ -1203,9 +1381,8 @@ func (e *Engine) Begin(writable bool) (tsdb.Tx, error) { func (e *Engine) WriteTo(w io.Writer) (n int64, err error) { panic("not implemented") } -func (e *Engine) keyAndFieldToID(series, field string) uint64 { +func (e *Engine) keyToID(key string) uint64 { // get the ID for the key and be sure to check if it had hash collision before - key := seriesFieldKey(series, field) e.collisionsLock.RLock() id, ok := e.collisions[key] e.collisionsLock.RUnlock() @@ -1216,6 +1393,11 @@ func (e *Engine) keyAndFieldToID(series, field string) uint64 { return id } +func (e *Engine) keyAndFieldToID(series, field string) uint64 { + key := seriesFieldKey(series, field) + return e.keyToID(key) +} + func (e *Engine) copyFilesCollection() []*dataFile { e.filesLock.RLock() defer e.filesLock.RUnlock() @@ -1558,293 +1740,6 @@ func (a dataFiles) Len() int { return len(a) } func (a dataFiles) Swap(i, j int) { a[i], a[j] = a[j], a[i] } func (a dataFiles) Less(i, j int) bool { return a[i].MinTime() < a[j].MinTime() } -type cursor struct { - id uint64 - f *dataFile - filesPos int // the index in the files slice we're looking at - pos uint32 - vals Values - - ascending bool - - blockPositions []uint32 // only used for descending queries - - // time acending list of data files - files []*dataFile -} - -func newCursor(id uint64, files []*dataFile, ascending bool) *cursor { - return &cursor{ - id: id, - ascending: ascending, - files: files, - } -} - -func (c *cursor) SeekTo(seek int64) (int64, interface{}) { - if len(c.files) == 0 { - return tsdb.EOF, nil - } - - if c.ascending { - if seek <= c.files[0].MinTime() { - c.filesPos = 0 - c.f = c.files[0] - } else { - for i, f := range c.files { - if seek >= f.MinTime() && seek <= f.MaxTime() { - c.filesPos = i - c.f = f - break - } - } - } - } else { - if seek >= c.files[len(c.files)-1].MaxTime() { - c.filesPos = len(c.files) - 1 - c.f = c.files[c.filesPos] - } else if seek < c.files[0].MinTime() { - return tsdb.EOF, nil - } else { - for i, f := range c.files { - if seek >= f.MinTime() && seek <= f.MaxTime() { - c.filesPos = i - c.f = f - break - } - } - } - } - - if c.f == nil { - return tsdb.EOF, nil - } - - // find the first file we need to check in - for { - if c.filesPos < 0 || c.filesPos >= len(c.files) { - return tsdb.EOF, nil - } - c.f = c.files[c.filesPos] - - c.pos = c.f.StartingPositionForID(c.id) - - // if this id isn't in this file, move to next one or return - if c.pos == 0 { - if c.ascending { - c.filesPos++ - } else { - c.filesPos-- - c.blockPositions = nil - } - continue - } - - // handle seek for correct order - k := tsdb.EOF - var v interface{} - - if c.ascending { - k, v = c.seekAscending(seek) - } else { - k, v = c.seekDescending(seek) - } - - if k != tsdb.EOF { - return k, v - } - - if c.ascending { - c.filesPos++ - } else { - c.filesPos-- - c.blockPositions = nil - } - } -} - -func (c *cursor) seekAscending(seek int64) (int64, interface{}) { - // seek to the block and values we're looking for - for { - // if the time is between this block and the next, - // decode this block and go, otherwise seek to next block - length := c.blockLength(c.pos) - - // if the next block has a time less than what we're seeking to, - // skip decoding this block and continue on - nextBlockPos := c.pos + blockHeaderSize + length - if nextBlockPos < c.f.indexPosition() { - nextBlockID := btou64(c.f.mmap[nextBlockPos : nextBlockPos+8]) - if nextBlockID == c.id { - nextBlockTime := c.blockMinTime(nextBlockPos) - if nextBlockTime <= seek { - c.pos = nextBlockPos - continue - } - } - } - - // it must be in this block or not at all - c.decodeBlock(c.pos) - - // see if we can find it in this block - for i, v := range c.vals { - if v.Time().UnixNano() >= seek { - c.vals = c.vals[i+1:] - return v.Time().UnixNano(), v.Value() - } - } - } -} - -func (c *cursor) seekDescending(seek int64) (int64, interface{}) { - c.setBlockPositions() - if len(c.blockPositions) == 0 { - return tsdb.EOF, nil - } - - for i := len(c.blockPositions) - 1; i >= 0; i-- { - pos := c.blockPositions[i] - if c.blockMinTime(pos) > seek { - continue - } - - c.decodeBlock(pos) - c.blockPositions = c.blockPositions[:i] - - for i := len(c.vals) - 1; i >= 0; i-- { - val := c.vals[i] - if seek >= val.UnixNano() { - c.vals = c.vals[:i] - return val.UnixNano(), val.Value() - } - if seek < val.UnixNano() { - // we need to move to the next block - if i == 0 { - break - } - val := c.vals[i-1] - c.vals = c.vals[:i-1] - return val.UnixNano(), val.Value() - } - } - c.blockPositions = c.blockPositions[:i] - } - - return tsdb.EOF, nil -} - -func (c *cursor) blockMinTime(pos uint32) int64 { - return int64(btou64(c.f.mmap[pos+12 : pos+20])) -} - -func (c *cursor) setBlockPositions() { - pos := c.pos - - for { - if pos >= c.f.indexPosition() { - return - } - - length := c.blockLength(pos) - id := btou64(c.f.mmap[pos : pos+8]) - - if id != c.id { - return - } - - c.blockPositions = append(c.blockPositions, pos) - pos += blockHeaderSize + length - } -} - -func (c *cursor) Next() (int64, interface{}) { - if c.ascending { - return c.nextAscending() - } - return c.nextDescending() -} - -func (c *cursor) nextAscending() (int64, interface{}) { - if len(c.vals) > 0 { - v := c.vals[0] - c.vals = c.vals[1:] - - return v.Time().UnixNano(), v.Value() - } - - // if we have a file set, see if the next block is for this ID - if c.f != nil && c.pos < c.f.indexPosition() { - nextBlockID := btou64(c.f.mmap[c.pos : c.pos+8]) - if nextBlockID == c.id { - c.decodeBlock(c.pos) - return c.nextAscending() - } - } - - // loop through the files until we hit the next one that has this id - for { - c.filesPos++ - if c.filesPos >= len(c.files) { - return tsdb.EOF, nil - } - c.f = c.files[c.filesPos] - - startingPos := c.f.StartingPositionForID(c.id) - if startingPos == 0 { - // move to next file because it isn't in this one - continue - } - - // we have a block with this id, decode and return - c.decodeBlock(startingPos) - return c.nextAscending() - } -} - -func (c *cursor) nextDescending() (int64, interface{}) { - if len(c.vals) > 0 { - v := c.vals[len(c.vals)-1] - if len(c.vals) >= 1 { - c.vals = c.vals[:len(c.vals)-1] - } else { - c.vals = nil - } - return v.UnixNano(), v.Value() - } - - for i := len(c.blockPositions) - 1; i >= 0; i-- { - c.decodeBlock(c.blockPositions[i]) - c.blockPositions = c.blockPositions[:i] - if len(c.vals) == 0 { - continue - } - val := c.vals[len(c.vals)-1] - c.vals = c.vals[:len(c.vals)-1] - return val.UnixNano(), val.Value() - } - - return tsdb.EOF, nil -} - -func (c *cursor) blockLength(pos uint32) uint32 { - return btou32(c.f.mmap[pos+8 : pos+12]) -} - -func (c *cursor) decodeBlock(position uint32) { - length := c.blockLength(position) - block := c.f.mmap[position+blockHeaderSize : position+blockHeaderSize+length] - c.vals, _ = DecodeBlock(block) - - // only adavance the position if we're asceending. - // Descending queries use the blockPositions - if c.ascending { - c.pos = position + blockHeaderSize + length - } -} - -func (c *cursor) Ascending() bool { return c.ascending } - // u64tob converts a uint64 into an 8-byte slice. func u64tob(v uint64) []byte { b := make([]byte, 8) diff --git a/tsdb/engine/pd1/pd1_test.go b/tsdb/engine/pd1/pd1_test.go index 5c28572a846..02f1a576631 100644 --- a/tsdb/engine/pd1/pd1_test.go +++ b/tsdb/engine/pd1/pd1_test.go @@ -10,6 +10,7 @@ import ( "testing" "time" + "github.com/influxdb/influxdb/influxql" "github.com/influxdb/influxdb/models" "github.com/influxdb/influxdb/tsdb" "github.com/influxdb/influxdb/tsdb/engine/pd1" @@ -170,7 +171,6 @@ func TestEngine_WriteIndexQueryAcrossDataFiles(t *testing.T) { points = points[1:] for _, p := range points { - fmt.Println("> ", p.Time()) k, v := c.Next() val := p.Fields()["value"] if p.UnixNano() != k || val != v { @@ -1071,6 +1071,171 @@ func TestEngine_DuplicatePointsInWalAndIndex(t *testing.T) { } } +func TestEngine_Deletes(t *testing.T) { + e := OpenDefaultEngine() + defer e.Cleanup() + + fields := []string{"value"} + // Create metadata. + mf := &tsdb.MeasurementFields{Fields: make(map[string]*tsdb.Field)} + mf.CreateFieldIfNotExists("value", influxql.Float) + atag := map[string]string{"host": "A"} + btag := map[string]string{"host": "B"} + seriesToCreate := []*tsdb.SeriesCreate{ + {Series: tsdb.NewSeries(string(models.MakeKey([]byte("cpu"), atag)), atag)}, + {Series: tsdb.NewSeries(string(models.MakeKey([]byte("cpu"), btag)), btag)}, + } + + p1 := parsePoint("cpu,host=A value=1.1 1000000001") + p2 := parsePoint("cpu,host=A value=1.2 2000000001") + p3 := parsePoint("cpu,host=B value=2.1 1000000000") + p4 := parsePoint("cpu,host=B value=2.1 2000000000") + + e.SkipCompaction = true + e.WAL.SkipCache = false + + if err := e.WritePoints([]models.Point{p1, p3}, map[string]*tsdb.MeasurementFields{"cpu": mf}, seriesToCreate); err != nil { + t.Fatalf("failed to write points: %s", err.Error()) + } + + func() { + tx, _ := e.Begin(false) + defer tx.Rollback() + c := tx.Cursor("cpu,host=A", fields, nil, true) + k, _ := c.SeekTo(0) + if k != p1.UnixNano() { + t.Fatalf("time wrong:\n\texp:%d\n\tgot:%d\n", p1.UnixNano(), k) + } + }() + + if err := e.DeleteSeries([]string{"cpu,host=A"}); err != nil { + t.Fatalf("failed to delete series: %s", err.Error()) + } + + func() { + tx, _ := e.Begin(false) + defer tx.Rollback() + c := tx.Cursor("cpu,host=B", fields, nil, true) + k, _ := c.SeekTo(0) + if k != p3.UnixNano() { + t.Fatalf("time wrong:\n\texp:%d\n\tgot:%d\n", p1.UnixNano(), k) + } + c = tx.Cursor("cpu,host=A", fields, nil, true) + k, _ = c.SeekTo(0) + if k != tsdb.EOF { + t.Fatal("expected EOF", k) + } + }() + + if err := e.WritePoints([]models.Point{p2, p4}, nil, nil); err != nil { + t.Fatalf("failed to write points: %s", err.Error()) + } + + if err := e.WAL.Flush(); err != nil { + t.Fatalf("error flushing wal: %s", err.Error()) + } + + func() { + tx, _ := e.Begin(false) + defer tx.Rollback() + c := tx.Cursor("cpu,host=A", fields, nil, true) + k, _ := c.SeekTo(0) + if k != p2.UnixNano() { + t.Fatalf("time wrong:\n\texp:%d\n\tgot:%d\n", p1.UnixNano(), k) + } + }() + + if err := e.DeleteSeries([]string{"cpu,host=A"}); err != nil { + t.Fatalf("failed to delete series: %s", err.Error()) + } + + // we already know the delete on the wal works. open and close so + // the wal flushes to the index. To verify that the delete gets + // persisted and will go all the way through the index + + if err := e.Close(); err != nil { + t.Fatalf("error closing: %s", err.Error()) + } + if err := e.Open(); err != nil { + t.Fatalf("error opening: %s", err.Error()) + } + + verify := func() { + tx, _ := e.Begin(false) + defer tx.Rollback() + c := tx.Cursor("cpu,host=B", fields, nil, true) + k, _ := c.SeekTo(0) + if k != p3.UnixNano() { + t.Fatalf("time wrong:\n\texp:%d\n\tgot:%d\n", p1.UnixNano(), k) + } + c = tx.Cursor("cpu,host=A", fields, nil, true) + k, _ = c.SeekTo(0) + if k != tsdb.EOF { + t.Fatal("expected EOF") + } + } + + fmt.Println("verify 1") + verify() + + // open and close to verify thd delete was persisted + if err := e.Close(); err != nil { + t.Fatalf("error closing: %s", err.Error()) + } + if err := e.Open(); err != nil { + t.Fatalf("error opening: %s", err.Error()) + } + + fmt.Println("verify 2") + verify() + + if err := e.DeleteSeries([]string{"cpu,host=B"}); err != nil { + t.Fatalf("failed to delete series: %s", err.Error()) + } + + func() { + tx, _ := e.Begin(false) + defer tx.Rollback() + c := tx.Cursor("cpu,host=B", fields, nil, true) + k, _ := c.SeekTo(0) + if k != tsdb.EOF { + t.Fatal("expected EOF") + } + }() + + if err := e.WAL.Flush(); err != nil { + t.Fatalf("error flushing: %s", err.Error()) + } + + func() { + tx, _ := e.Begin(false) + defer tx.Rollback() + c := tx.Cursor("cpu,host=B", fields, nil, true) + k, _ := c.SeekTo(0) + if k != tsdb.EOF { + t.Fatal("expected EOF") + } + }() + + // open and close to verify thd delete was persisted + if err := e.Close(); err != nil { + t.Fatalf("error closing: %s", err.Error()) + } + if err := e.Open(); err != nil { + t.Fatalf("error opening: %s", err.Error()) + } + + func() { + tx, _ := e.Begin(false) + defer tx.Rollback() + c := tx.Cursor("cpu,host=B", fields, nil, true) + k, _ := c.SeekTo(0) + if k != tsdb.EOF { + t.Fatal("expected EOF") + } + }() +} + // Engine represents a test wrapper for pd1.Engine. type Engine struct { *pd1.Engine diff --git a/tsdb/engine/pd1/tx.go b/tsdb/engine/pd1/tx.go index 16aac8b5e43..89d62e9d88a 100644 --- a/tsdb/engine/pd1/tx.go +++ b/tsdb/engine/pd1/tx.go @@ -13,10 +13,20 @@ type tx struct { // TODO: handle multiple fields and descending func (t *tx) Cursor(series string, fields []string, dec *tsdb.FieldCodec, ascending bool) tsdb.Cursor { + t.engine.filesLock.RLock() + defer t.engine.filesLock.RUnlock() + // don't add the overhead of the multifield cursor if we only have one field if len(fields) == 1 { id := t.engine.keyAndFieldToID(series, fields[0]) - indexCursor := newCursor(id, t.files, ascending) + isDeleted := t.engine.deletes[id] + + var indexCursor tsdb.Cursor + if isDeleted { + indexCursor = &emptyCursor{ascending: ascending} + } else { + indexCursor = newCursor(id, t.files, ascending) + } wc := t.engine.WAL.Cursor(series, fields, dec, ascending) return NewCombinedEngineCursor(wc, indexCursor, ascending) } @@ -27,7 +37,14 @@ func (t *tx) Cursor(series string, fields []string, dec *tsdb.FieldCodec, ascend cursorFields := make([]string, 0) for _, field := range fields { id := t.engine.keyAndFieldToID(series, field) - indexCursor := newCursor(id, t.files, ascending) + isDeleted := t.engine.deletes[id] + + var indexCursor tsdb.Cursor + if isDeleted { + indexCursor = &emptyCursor{ascending: ascending} + } else { + indexCursor = newCursor(id, t.files, ascending) + } wc := t.engine.WAL.Cursor(series, []string{field}, dec, ascending) // double up the fields since there's one for the wal and one for the index cursorFields = append(cursorFields, field, field) diff --git a/tsdb/engine/pd1/wal.go b/tsdb/engine/pd1/wal.go index b6cc0cc2143..f253db33f51 100644 --- a/tsdb/engine/pd1/wal.go +++ b/tsdb/engine/pd1/wal.go @@ -57,6 +57,7 @@ const ( pointsEntry walEntryType = 0x01 fieldsEntry walEntryType = 0x02 seriesEntry walEntryType = 0x03 + deleteEntry walEntryType = 0x04 ) type Log struct { @@ -117,6 +118,7 @@ type Log struct { // IndexWriter is an interface for the indexed database the WAL flushes data to type IndexWriter interface { Write(valuesByKey map[string]Values, measurementFieldsToSave map[string]*tsdb.MeasurementFields, seriesToCreate []*tsdb.SeriesCreate) error + MarkDeletes(keys []string) } func NewLog(path string) *Log { @@ -385,6 +387,12 @@ func (l *Log) readFileToCache(fileName string) error { return err } l.addToCache(nil, nil, series, false) + case deleteEntry: + var keys []string + if err := json.Unmarshal(data, &keys); err != nil { + return err + } + l.Index.MarkDeletes(keys) } } } @@ -423,8 +431,27 @@ func (l *Log) Flush() error { return l.flush(idleFlush) } +func (l *Log) DropMeasurementFields(measurement string) { + l.cacheLock.Lock() + defer l.cacheLock.Unlock() + delete(l.measurementFieldsCache, measurement) +} + func (l *Log) DeleteSeries(keys []string) error { - panic("not implemented") + l.cacheLock.Lock() + for _, k := range keys { + delete(l.cache, k) + } + l.cacheLock.Unlock() + + b, err := json.Marshal(keys) + if err != nil { + return err + } + + cb := snappy.Encode(nil, b) + + return l.writeToLog(deleteEntry, cb) } // Close will finish any flush that is currently in process and close file handles @@ -531,7 +558,7 @@ func (l *Log) flush(flush flushType) error { l.cacheLock.Unlock() // exit if there's nothing to flush to the index - if len(l.flushCache) == 0 && len(mfc) == 0 && len(scc) == 0 { + if len(l.flushCache) == 0 && len(mfc) == 0 && len(scc) == 0 && flush != startupFlush { return nil } diff --git a/tsdb/engine/pd1/wal_test.go b/tsdb/engine/pd1/wal_test.go index 034ad2dd3e5..4718f4cebd2 100644 --- a/tsdb/engine/pd1/wal_test.go +++ b/tsdb/engine/pd1/wal_test.go @@ -172,3 +172,5 @@ type MockIndexWriter struct { func (m *MockIndexWriter) Write(valuesByKey map[string]pd1.Values, measurementFieldsToSave map[string]*tsdb.MeasurementFields, seriesToCreate []*tsdb.SeriesCreate) error { return m.fn(valuesByKey, measurementFieldsToSave, seriesToCreate) } + +func (m *MockIndexWriter) MarkDeletes(keys []string) {} From 594253cbba73979aaa098bf3d5b0bc4b8b92bc7b Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Thu, 1 Oct 2015 15:23:38 -0400 Subject: [PATCH 121/139] Rename storage engine to tsm1, for Time Structured Merge Tree! --- tsdb/engine/{pd1 => tsm1}/bool.go | 2 +- tsdb/engine/{pd1 => tsm1}/bool_test.go | 16 ++--- tsdb/engine/{pd1 => tsm1}/cursor.go | 2 +- tsdb/engine/{pd1 => tsm1}/encoding.go | 2 +- tsdb/engine/{pd1 => tsm1}/encoding_test.go | 28 ++++---- tsdb/engine/{pd1 => tsm1}/float.go | 2 +- tsdb/engine/{pd1 => tsm1}/float_test.go | 12 ++-- tsdb/engine/{pd1 => tsm1}/int.go | 2 +- tsdb/engine/{pd1 => tsm1}/int_test.go | 38 +++++------ tsdb/engine/{pd1 => tsm1}/string.go | 2 +- tsdb/engine/{pd1 => tsm1}/string_test.go | 20 +++--- tsdb/engine/{pd1 => tsm1}/timestamp.go | 2 +- tsdb/engine/{pd1 => tsm1}/timestamp_test.go | 68 +++++++++---------- tsdb/engine/{pd1/pd1.go => tsm1/tsm1.go} | 2 +- .../{pd1/pd1_test.go => tsm1/tsm1_test.go} | 12 ++-- tsdb/engine/{pd1 => tsm1}/tx.go | 2 +- tsdb/engine/{pd1 => tsm1}/wal.go | 8 +-- tsdb/engine/{pd1 => tsm1}/wal_test.go | 18 ++--- tsdb/engine/{pd1 => tsm1}/write_lock.go | 2 +- tsdb/engine/{pd1 => tsm1}/write_lock_test.go | 16 ++--- 20 files changed, 128 insertions(+), 128 deletions(-) rename tsdb/engine/{pd1 => tsm1}/bool.go (99%) rename tsdb/engine/{pd1 => tsm1}/bool_test.go (82%) rename tsdb/engine/{pd1 => tsm1}/cursor.go (99%) rename tsdb/engine/{pd1 => tsm1}/encoding.go (99%) rename tsdb/engine/{pd1 => tsm1}/encoding_test.go (83%) rename tsdb/engine/{pd1 => tsm1}/float.go (99%) rename tsdb/engine/{pd1 => tsm1}/float_test.go (93%) rename tsdb/engine/{pd1 => tsm1}/int.go (99%) rename tsdb/engine/{pd1 => tsm1}/int_test.go (87%) rename tsdb/engine/{pd1 => tsm1}/string.go (99%) rename tsdb/engine/{pd1 => tsm1}/string_test.go (76%) rename tsdb/engine/{pd1 => tsm1}/timestamp.go (99%) rename tsdb/engine/{pd1 => tsm1}/timestamp_test.go (84%) rename tsdb/engine/{pd1/pd1.go => tsm1/tsm1.go} (99%) rename tsdb/engine/{pd1/pd1_test.go => tsm1/tsm1_test.go} (99%) rename tsdb/engine/{pd1 => tsm1}/tx.go (99%) rename tsdb/engine/{pd1 => tsm1}/wal.go (98%) rename tsdb/engine/{pd1 => tsm1}/wal_test.go (85%) rename tsdb/engine/{pd1 => tsm1}/write_lock.go (99%) rename tsdb/engine/{pd1 => tsm1}/write_lock_test.go (91%) diff --git a/tsdb/engine/pd1/bool.go b/tsdb/engine/tsm1/bool.go similarity index 99% rename from tsdb/engine/pd1/bool.go rename to tsdb/engine/tsm1/bool.go index 4fad26586bb..8d9653d1991 100644 --- a/tsdb/engine/pd1/bool.go +++ b/tsdb/engine/tsm1/bool.go @@ -1,4 +1,4 @@ -package pd1 +package tsm1 // bool encoding uses 1 bit per value. Each compressed byte slice contains a 1 byte header // indicating the compression type, followed by a variable byte encoded length indicating diff --git a/tsdb/engine/pd1/bool_test.go b/tsdb/engine/tsm1/bool_test.go similarity index 82% rename from tsdb/engine/pd1/bool_test.go rename to tsdb/engine/tsm1/bool_test.go index ed642cff86e..ed68987afd1 100644 --- a/tsdb/engine/pd1/bool_test.go +++ b/tsdb/engine/tsm1/bool_test.go @@ -1,26 +1,26 @@ -package pd1_test +package tsm1_test import ( "testing" - "github.com/influxdb/influxdb/tsdb/engine/pd1" + "github.com/influxdb/influxdb/tsdb/engine/tsm1" ) func Test_BoolEncoder_NoValues(t *testing.T) { - enc := pd1.NewBoolEncoder() + enc := tsm1.NewBoolEncoder() b, err := enc.Bytes() if err != nil { t.Fatalf("unexpected error: %v", err) } - dec := pd1.NewBoolDecoder(b) + dec := tsm1.NewBoolDecoder(b) if dec.Next() { t.Fatalf("unexpected next value: got true, exp false") } } func Test_BoolEncoder_Single(t *testing.T) { - enc := pd1.NewBoolEncoder() + enc := tsm1.NewBoolEncoder() v1 := true enc.Write(v1) b, err := enc.Bytes() @@ -28,7 +28,7 @@ func Test_BoolEncoder_Single(t *testing.T) { t.Fatalf("unexpected error: %v", err) } - dec := pd1.NewBoolDecoder(b) + dec := tsm1.NewBoolDecoder(b) if !dec.Next() { t.Fatalf("unexpected next value: got false, exp true") } @@ -39,7 +39,7 @@ func Test_BoolEncoder_Single(t *testing.T) { } func Test_BoolEncoder_Multi_Compressed(t *testing.T) { - enc := pd1.NewBoolEncoder() + enc := tsm1.NewBoolEncoder() values := make([]bool, 10) for i := range values { @@ -56,7 +56,7 @@ func Test_BoolEncoder_Multi_Compressed(t *testing.T) { t.Fatalf("unexpected length: got %v, exp %v", len(b), exp) } - dec := pd1.NewBoolDecoder(b) + dec := tsm1.NewBoolDecoder(b) for i, v := range values { if !dec.Next() { diff --git a/tsdb/engine/pd1/cursor.go b/tsdb/engine/tsm1/cursor.go similarity index 99% rename from tsdb/engine/pd1/cursor.go rename to tsdb/engine/tsm1/cursor.go index 4b49972131e..06fd0bbf8f8 100644 --- a/tsdb/engine/pd1/cursor.go +++ b/tsdb/engine/tsm1/cursor.go @@ -1,4 +1,4 @@ -package pd1 +package tsm1 import ( "math" diff --git a/tsdb/engine/pd1/encoding.go b/tsdb/engine/tsm1/encoding.go similarity index 99% rename from tsdb/engine/pd1/encoding.go rename to tsdb/engine/tsm1/encoding.go index 005692d2735..4b6a112d568 100644 --- a/tsdb/engine/pd1/encoding.go +++ b/tsdb/engine/tsm1/encoding.go @@ -1,4 +1,4 @@ -package pd1 +package tsm1 import ( "encoding/binary" diff --git a/tsdb/engine/pd1/encoding_test.go b/tsdb/engine/tsm1/encoding_test.go similarity index 83% rename from tsdb/engine/pd1/encoding_test.go rename to tsdb/engine/tsm1/encoding_test.go index 7126eec3887..c4889f18e5d 100644 --- a/tsdb/engine/pd1/encoding_test.go +++ b/tsdb/engine/tsm1/encoding_test.go @@ -1,4 +1,4 @@ -package pd1_test +package tsm1_test import ( // "math/rand" @@ -8,15 +8,15 @@ import ( "testing" "time" - "github.com/influxdb/influxdb/tsdb/engine/pd1" + "github.com/influxdb/influxdb/tsdb/engine/tsm1" ) func TestEncoding_FloatBlock(t *testing.T) { valueCount := 1000 times := getTimes(valueCount, 60, time.Second) - values := make(pd1.Values, len(times)) + values := make(tsm1.Values, len(times)) for i, t := range times { - values[i] = pd1.NewValue(t, float64(i)) + values[i] = tsm1.NewValue(t, float64(i)) } b := values.Encode(nil) @@ -29,9 +29,9 @@ func TestEncoding_FloatBlock(t *testing.T) { } func TestEncoding_FloatBlock_ZeroTime(t *testing.T) { - values := make(pd1.Values, 3) + values := make(tsm1.Values, 3) for i := 0; i < 3; i++ { - values[i] = pd1.NewValue(time.Unix(0, 0), float64(i)) + values[i] = tsm1.NewValue(time.Unix(0, 0), float64(i)) } b := values.Encode(nil) @@ -46,9 +46,9 @@ func TestEncoding_FloatBlock_ZeroTime(t *testing.T) { func TestEncoding_IntBlock_Basic(t *testing.T) { valueCount := 1000 times := getTimes(valueCount, 60, time.Second) - values := make(pd1.Values, len(times)) + values := make(tsm1.Values, len(times)) for i, t := range times { - values[i] = pd1.NewValue(t, int64(i)) + values[i] = tsm1.NewValue(t, int64(i)) } b := values.Encode(nil) @@ -74,13 +74,13 @@ func TestEncoding_IntBlock_Basic(t *testing.T) { func TestEncoding_IntBlock_Negatives(t *testing.T) { valueCount := 1000 times := getTimes(valueCount, 60, time.Second) - values := make(pd1.Values, len(times)) + values := make(tsm1.Values, len(times)) for i, t := range times { v := int64(i) if i%2 == 0 { v = -v } - values[i] = pd1.NewValue(t, int64(v)) + values[i] = tsm1.NewValue(t, int64(v)) } b := values.Encode(nil) @@ -95,13 +95,13 @@ func TestEncoding_IntBlock_Negatives(t *testing.T) { func TestEncoding_BoolBlock_Basic(t *testing.T) { valueCount := 1000 times := getTimes(valueCount, 60, time.Second) - values := make(pd1.Values, len(times)) + values := make(tsm1.Values, len(times)) for i, t := range times { v := true if i%2 == 0 { v = false } - values[i] = pd1.NewValue(t, v) + values[i] = tsm1.NewValue(t, v) } b := values.Encode(nil) @@ -116,9 +116,9 @@ func TestEncoding_BoolBlock_Basic(t *testing.T) { func TestEncoding_StringBlock_Basic(t *testing.T) { valueCount := 1000 times := getTimes(valueCount, 60, time.Second) - values := make(pd1.Values, len(times)) + values := make(tsm1.Values, len(times)) for i, t := range times { - values[i] = pd1.NewValue(t, fmt.Sprintf("value %d", i)) + values[i] = tsm1.NewValue(t, fmt.Sprintf("value %d", i)) } b := values.Encode(nil) diff --git a/tsdb/engine/pd1/float.go b/tsdb/engine/tsm1/float.go similarity index 99% rename from tsdb/engine/pd1/float.go rename to tsdb/engine/tsm1/float.go index dddb9f39b24..ff6a61c505d 100644 --- a/tsdb/engine/pd1/float.go +++ b/tsdb/engine/tsm1/float.go @@ -1,4 +1,4 @@ -package pd1 +package tsm1 /* This code is originally from: https://github.com/dgryski/go-tsz and has been modified to remove diff --git a/tsdb/engine/pd1/float_test.go b/tsdb/engine/tsm1/float_test.go similarity index 93% rename from tsdb/engine/pd1/float_test.go rename to tsdb/engine/tsm1/float_test.go index 60702938f5f..00b259bf95d 100644 --- a/tsdb/engine/pd1/float_test.go +++ b/tsdb/engine/tsm1/float_test.go @@ -1,15 +1,15 @@ -package pd1_test +package tsm1_test import ( "testing" - "github.com/influxdb/influxdb/tsdb/engine/pd1" + "github.com/influxdb/influxdb/tsdb/engine/tsm1" ) func TestFloatEncoder_Simple(t *testing.T) { // Example from the paper - s := pd1.NewFloatEncoder() + s := tsm1.NewFloatEncoder() s.Push(12) s.Push(12) @@ -94,7 +94,7 @@ var TwoHoursData = []struct { func TestFloatEncoder_Roundtrip(t *testing.T) { - s := pd1.NewFloatEncoder() + s := tsm1.NewFloatEncoder() for _, p := range TwoHoursData { s.Push(p.v) } @@ -123,7 +123,7 @@ func TestFloatEncoder_Roundtrip(t *testing.T) { func BenchmarkFloatEncoder(b *testing.B) { for i := 0; i < b.N; i++ { - s := pd1.NewFloatEncoder() + s := tsm1.NewFloatEncoder() for _, tt := range TwoHoursData { s.Push(tt.v) } @@ -132,7 +132,7 @@ func BenchmarkFloatEncoder(b *testing.B) { } func BenchmarkFloatDecoder(b *testing.B) { - s := pd1.NewFloatEncoder() + s := tsm1.NewFloatEncoder() for _, tt := range TwoHoursData { s.Push(tt.v) } diff --git a/tsdb/engine/pd1/int.go b/tsdb/engine/tsm1/int.go similarity index 99% rename from tsdb/engine/pd1/int.go rename to tsdb/engine/tsm1/int.go index 98cf717f303..b178c503751 100644 --- a/tsdb/engine/pd1/int.go +++ b/tsdb/engine/tsm1/int.go @@ -1,4 +1,4 @@ -package pd1 +package tsm1 // Int64 encoding uses two different strategies depending on the range of values in // the uncompressed data. Encoded values are first encoding used zig zag encoding. diff --git a/tsdb/engine/pd1/int_test.go b/tsdb/engine/tsm1/int_test.go similarity index 87% rename from tsdb/engine/pd1/int_test.go rename to tsdb/engine/tsm1/int_test.go index 7f1ba2eff74..279b55e49bf 100644 --- a/tsdb/engine/pd1/int_test.go +++ b/tsdb/engine/tsm1/int_test.go @@ -1,27 +1,27 @@ -package pd1_test +package tsm1_test import ( "math" "testing" - "github.com/influxdb/influxdb/tsdb/engine/pd1" + "github.com/influxdb/influxdb/tsdb/engine/tsm1" ) func Test_Int64Encoder_NoValues(t *testing.T) { - enc := pd1.NewInt64Encoder() + enc := tsm1.NewInt64Encoder() b, err := enc.Bytes() if err != nil { t.Fatalf("unexpected error: %v", err) } - dec := pd1.NewInt64Decoder(b) + dec := tsm1.NewInt64Decoder(b) if dec.Next() { t.Fatalf("unexpected next value: got true, exp false") } } func Test_Int64Encoder_One(t *testing.T) { - enc := pd1.NewInt64Encoder() + enc := tsm1.NewInt64Encoder() v1 := int64(1) enc.Write(1) @@ -30,7 +30,7 @@ func Test_Int64Encoder_One(t *testing.T) { t.Fatalf("unexpected error: %v", err) } - dec := pd1.NewInt64Decoder(b) + dec := tsm1.NewInt64Decoder(b) if !dec.Next() { t.Fatalf("unexpected next value: got true, exp false") } @@ -41,7 +41,7 @@ func Test_Int64Encoder_One(t *testing.T) { } func Test_Int64Encoder_Two(t *testing.T) { - enc := pd1.NewInt64Encoder() + enc := tsm1.NewInt64Encoder() var v1, v2 int64 = 1, 2 enc.Write(v1) @@ -52,7 +52,7 @@ func Test_Int64Encoder_Two(t *testing.T) { t.Fatalf("unexpected error: %v", err) } - dec := pd1.NewInt64Decoder(b) + dec := tsm1.NewInt64Decoder(b) if !dec.Next() { t.Fatalf("unexpected next value: got true, exp false") } @@ -71,7 +71,7 @@ func Test_Int64Encoder_Two(t *testing.T) { } func Test_Int64Encoder_Negative(t *testing.T) { - enc := pd1.NewInt64Encoder() + enc := tsm1.NewInt64Encoder() var v1, v2, v3 int64 = -2, 0, 1 enc.Write(v1) @@ -83,7 +83,7 @@ func Test_Int64Encoder_Negative(t *testing.T) { t.Fatalf("unexpected error: %v", err) } - dec := pd1.NewInt64Decoder(b) + dec := tsm1.NewInt64Decoder(b) if !dec.Next() { t.Fatalf("unexpected next value: got true, exp false") } @@ -110,7 +110,7 @@ func Test_Int64Encoder_Negative(t *testing.T) { } func Test_Int64Encoder_Large_Range(t *testing.T) { - enc := pd1.NewInt64Encoder() + enc := tsm1.NewInt64Encoder() var v1, v2 int64 = math.MinInt64, math.MaxInt64 enc.Write(v1) enc.Write(v2) @@ -119,7 +119,7 @@ func Test_Int64Encoder_Large_Range(t *testing.T) { t.Fatalf("unexpected error: %v", err) } - dec := pd1.NewInt64Decoder(b) + dec := tsm1.NewInt64Decoder(b) if !dec.Next() { t.Fatalf("unexpected next value: got true, exp false") } @@ -138,7 +138,7 @@ func Test_Int64Encoder_Large_Range(t *testing.T) { } func Test_Int64Encoder_Uncompressed(t *testing.T) { - enc := pd1.NewInt64Encoder() + enc := tsm1.NewInt64Encoder() var v1, v2, v3 int64 = 0, 1, 1 << 60 enc.Write(v1) @@ -155,7 +155,7 @@ func Test_Int64Encoder_Uncompressed(t *testing.T) { t.Fatalf("length mismatch: got %v, exp %v", len(b), exp) } - dec := pd1.NewInt64Decoder(b) + dec := tsm1.NewInt64Decoder(b) if !dec.Next() { t.Fatalf("unexpected next value: got true, exp false") } @@ -182,7 +182,7 @@ func Test_Int64Encoder_Uncompressed(t *testing.T) { } func Test_Int64Encoder_AllNegative(t *testing.T) { - enc := pd1.NewInt64Encoder() + enc := tsm1.NewInt64Encoder() values := []int64{ -10, -5, -1, } @@ -196,7 +196,7 @@ func Test_Int64Encoder_AllNegative(t *testing.T) { t.Fatalf("unexpected error: %v", err) } - dec := pd1.NewInt64Decoder(b) + dec := tsm1.NewInt64Decoder(b) i := 0 for dec.Next() { if i > len(values) { @@ -211,7 +211,7 @@ func Test_Int64Encoder_AllNegative(t *testing.T) { } func BenchmarkInt64Encoder(b *testing.B) { - enc := pd1.NewInt64Encoder() + enc := tsm1.NewInt64Encoder() x := make([]int64, 1024) for i := 0; i < len(x); i++ { x[i] = int64(i) @@ -230,7 +230,7 @@ type byteSetter interface { func BenchmarkInt64Decoder(b *testing.B) { x := make([]int64, 1024) - enc := pd1.NewInt64Encoder() + enc := tsm1.NewInt64Encoder() for i := 0; i < len(x); i++ { x[i] = int64(i) enc.Write(x[i]) @@ -239,7 +239,7 @@ func BenchmarkInt64Decoder(b *testing.B) { b.ResetTimer() - dec := pd1.NewInt64Decoder(bytes) + dec := tsm1.NewInt64Decoder(bytes) for i := 0; i < b.N; i++ { dec.(byteSetter).SetBytes(bytes) diff --git a/tsdb/engine/pd1/string.go b/tsdb/engine/tsm1/string.go similarity index 99% rename from tsdb/engine/pd1/string.go rename to tsdb/engine/tsm1/string.go index e548b68c560..1b5dafac3f0 100644 --- a/tsdb/engine/pd1/string.go +++ b/tsdb/engine/tsm1/string.go @@ -1,4 +1,4 @@ -package pd1 +package tsm1 // String encoding uses snappy compression to compress each string. Each string is // appended to byte slice prefixed with a variable byte length followed by the string diff --git a/tsdb/engine/pd1/string_test.go b/tsdb/engine/tsm1/string_test.go similarity index 76% rename from tsdb/engine/pd1/string_test.go rename to tsdb/engine/tsm1/string_test.go index c457de8697d..8710a50b365 100644 --- a/tsdb/engine/pd1/string_test.go +++ b/tsdb/engine/tsm1/string_test.go @@ -1,27 +1,27 @@ -package pd1_test +package tsm1_test import ( "fmt" "testing" - "github.com/influxdb/influxdb/tsdb/engine/pd1" + "github.com/influxdb/influxdb/tsdb/engine/tsm1" ) func Test_StringEncoder_NoValues(t *testing.T) { - enc := pd1.NewStringEncoder() + enc := tsm1.NewStringEncoder() b, err := enc.Bytes() if err != nil { t.Fatalf("unexpected error: %v", err) } - dec := pd1.NewStringDecoder(b) + dec := tsm1.NewStringDecoder(b) if dec.Next() { t.Fatalf("unexpected next value: got true, exp false") } } func Test_StringEncoder_Single(t *testing.T) { - enc := pd1.NewStringEncoder() + enc := tsm1.NewStringEncoder() v1 := "v1" enc.Write(v1) b, err := enc.Bytes() @@ -29,7 +29,7 @@ func Test_StringEncoder_Single(t *testing.T) { t.Fatalf("unexpected error: %v", err) } - dec := pd1.NewStringDecoder(b) + dec := tsm1.NewStringDecoder(b) if !dec.Next() { t.Fatalf("unexpected next value: got false, exp true") } @@ -40,7 +40,7 @@ func Test_StringEncoder_Single(t *testing.T) { } func Test_StringEncoder_Multi_Compressed(t *testing.T) { - enc := pd1.NewStringEncoder() + enc := tsm1.NewStringEncoder() values := make([]string, 10) for i := range values { @@ -53,15 +53,15 @@ func Test_StringEncoder_Multi_Compressed(t *testing.T) { t.Fatalf("unexpected error: %v", err) } - if b[0]>>4 != pd1.EncodingSnappy { - t.Fatalf("unexpected encoding: got %v, exp %v", b[0], pd1.EncodingSnappy) + if b[0]>>4 != tsm1.EncodingSnappy { + t.Fatalf("unexpected encoding: got %v, exp %v", b[0], tsm1.EncodingSnappy) } if exp := 47; len(b) != exp { t.Fatalf("unexpected length: got %v, exp %v", len(b), exp) } - dec := pd1.NewStringDecoder(b) + dec := tsm1.NewStringDecoder(b) for i, v := range values { if !dec.Next() { diff --git a/tsdb/engine/pd1/timestamp.go b/tsdb/engine/tsm1/timestamp.go similarity index 99% rename from tsdb/engine/pd1/timestamp.go rename to tsdb/engine/tsm1/timestamp.go index b6fc6fef411..59990f5cb41 100644 --- a/tsdb/engine/pd1/timestamp.go +++ b/tsdb/engine/tsm1/timestamp.go @@ -1,4 +1,4 @@ -package pd1 +package tsm1 // Timestamp encoding is adapative and based on structure of the timestamps that are encoded. It // uses a combination of delta encoding, scaling and compression using simple8b, run length encoding diff --git a/tsdb/engine/pd1/timestamp_test.go b/tsdb/engine/tsm1/timestamp_test.go similarity index 84% rename from tsdb/engine/pd1/timestamp_test.go rename to tsdb/engine/tsm1/timestamp_test.go index f7030db8a40..dbb5a2341a6 100644 --- a/tsdb/engine/pd1/timestamp_test.go +++ b/tsdb/engine/tsm1/timestamp_test.go @@ -1,14 +1,14 @@ -package pd1_test +package tsm1_test import ( "testing" "time" - "github.com/influxdb/influxdb/tsdb/engine/pd1" + "github.com/influxdb/influxdb/tsdb/engine/tsm1" ) func Test_TimeEncoder(t *testing.T) { - enc := pd1.NewTimeEncoder() + enc := tsm1.NewTimeEncoder() x := []time.Time{} now := time.Unix(0, 0) @@ -24,11 +24,11 @@ func Test_TimeEncoder(t *testing.T) { t.Fatalf("unexpected error: %v", err) } - if got := b[0] >> 4; got != pd1.EncodingPackedSimple { + if got := b[0] >> 4; got != tsm1.EncodingPackedSimple { t.Fatalf("Wrong encoding used: expected uncompressed, got %v", got) } - dec := pd1.NewTimeDecoder(b) + dec := tsm1.NewTimeDecoder(b) for i, v := range x { if !dec.Next() { t.Fatalf("Next == false, expected true") @@ -41,20 +41,20 @@ func Test_TimeEncoder(t *testing.T) { } func Test_TimeEncoder_NoValues(t *testing.T) { - enc := pd1.NewTimeEncoder() + enc := tsm1.NewTimeEncoder() b, err := enc.Bytes() if err != nil { t.Fatalf("unexpected error: %v", err) } - dec := pd1.NewTimeDecoder(b) + dec := tsm1.NewTimeDecoder(b) if dec.Next() { t.Fatalf("unexpected next value: got true, exp false") } } func Test_TimeEncoder_One(t *testing.T) { - enc := pd1.NewTimeEncoder() + enc := tsm1.NewTimeEncoder() tm := time.Unix(0, 0) enc.Write(tm) @@ -63,11 +63,11 @@ func Test_TimeEncoder_One(t *testing.T) { t.Fatalf("unexpected error: %v", err) } - if got := b[0] >> 4; got != pd1.EncodingPackedSimple { + if got := b[0] >> 4; got != tsm1.EncodingPackedSimple { t.Fatalf("Wrong encoding used: expected uncompressed, got %v", got) } - dec := pd1.NewTimeDecoder(b) + dec := tsm1.NewTimeDecoder(b) if !dec.Next() { t.Fatalf("unexpected next value: got true, exp false") } @@ -78,7 +78,7 @@ func Test_TimeEncoder_One(t *testing.T) { } func Test_TimeEncoder_Two(t *testing.T) { - enc := pd1.NewTimeEncoder() + enc := tsm1.NewTimeEncoder() t1 := time.Unix(0, 0) t2 := time.Unix(0, 1) enc.Write(t1) @@ -89,11 +89,11 @@ func Test_TimeEncoder_Two(t *testing.T) { t.Fatalf("unexpected error: %v", err) } - if got := b[0] >> 4; got != pd1.EncodingPackedSimple { + if got := b[0] >> 4; got != tsm1.EncodingPackedSimple { t.Fatalf("Wrong encoding used: expected uncompressed, got %v", got) } - dec := pd1.NewTimeDecoder(b) + dec := tsm1.NewTimeDecoder(b) if !dec.Next() { t.Fatalf("unexpected next value: got true, exp false") } @@ -112,7 +112,7 @@ func Test_TimeEncoder_Two(t *testing.T) { } func Test_TimeEncoder_Three(t *testing.T) { - enc := pd1.NewTimeEncoder() + enc := tsm1.NewTimeEncoder() t1 := time.Unix(0, 0) t2 := time.Unix(0, 1) t3 := time.Unix(0, 2) @@ -126,11 +126,11 @@ func Test_TimeEncoder_Three(t *testing.T) { t.Fatalf("unexpected error: %v", err) } - if got := b[0] >> 4; got != pd1.EncodingPackedSimple { + if got := b[0] >> 4; got != tsm1.EncodingPackedSimple { t.Fatalf("Wrong encoding used: expected uncompressed, got %v", got) } - dec := pd1.NewTimeDecoder(b) + dec := tsm1.NewTimeDecoder(b) if !dec.Next() { t.Fatalf("unexpected next value: got true, exp false") } @@ -157,7 +157,7 @@ func Test_TimeEncoder_Three(t *testing.T) { } func Test_TimeEncoder_Large_Range(t *testing.T) { - enc := pd1.NewTimeEncoder() + enc := tsm1.NewTimeEncoder() t1 := time.Unix(0, 1442369134000000000) t2 := time.Unix(0, 1442369135000000000) enc.Write(t1) @@ -167,11 +167,11 @@ func Test_TimeEncoder_Large_Range(t *testing.T) { t.Fatalf("unexpected error: %v", err) } - if got := b[0] >> 4; got != pd1.EncodingPackedSimple { + if got := b[0] >> 4; got != tsm1.EncodingPackedSimple { t.Fatalf("Wrong encoding used: expected uncompressed, got %v", got) } - dec := pd1.NewTimeDecoder(b) + dec := tsm1.NewTimeDecoder(b) if !dec.Next() { t.Fatalf("unexpected next value: got true, exp false") } @@ -190,7 +190,7 @@ func Test_TimeEncoder_Large_Range(t *testing.T) { } func Test_TimeEncoder_Uncompressed(t *testing.T) { - enc := pd1.NewTimeEncoder() + enc := tsm1.NewTimeEncoder() t1 := time.Unix(0, 0) t2 := time.Unix(1, 0) @@ -210,11 +210,11 @@ func Test_TimeEncoder_Uncompressed(t *testing.T) { t.Fatalf("length mismatch: got %v, exp %v", len(b), exp) } - if got := b[0] >> 4; got != pd1.EncodingUncompressed { + if got := b[0] >> 4; got != tsm1.EncodingUncompressed { t.Fatalf("Wrong encoding used: expected uncompressed, got %v", got) } - dec := pd1.NewTimeDecoder(b) + dec := tsm1.NewTimeDecoder(b) if !dec.Next() { t.Fatalf("unexpected next value: got true, exp false") } @@ -241,7 +241,7 @@ func Test_TimeEncoder_Uncompressed(t *testing.T) { } func Test_TimeEncoder_RLE(t *testing.T) { - enc := pd1.NewTimeEncoder() + enc := tsm1.NewTimeEncoder() var ts []time.Time for i := 0; i < 500; i++ { ts = append(ts, time.Unix(int64(i), 0)) @@ -256,7 +256,7 @@ func Test_TimeEncoder_RLE(t *testing.T) { t.Fatalf("length mismatch: got %v, exp %v", len(b), exp) } - if got := b[0] >> 4; got != pd1.EncodingRLE { + if got := b[0] >> 4; got != tsm1.EncodingRLE { t.Fatalf("Wrong encoding used: expected uncompressed, got %v", got) } @@ -264,7 +264,7 @@ func Test_TimeEncoder_RLE(t *testing.T) { t.Fatalf("unexpected error: %v", err) } - dec := pd1.NewTimeDecoder(b) + dec := tsm1.NewTimeDecoder(b) for i, v := range ts { if !dec.Next() { t.Fatalf("Next == false, expected true") @@ -281,7 +281,7 @@ func Test_TimeEncoder_RLE(t *testing.T) { } func Test_TimeEncoder_Reverse(t *testing.T) { - enc := pd1.NewTimeEncoder() + enc := tsm1.NewTimeEncoder() ts := []time.Time{ time.Unix(0, 3), time.Unix(0, 2), @@ -297,11 +297,11 @@ func Test_TimeEncoder_Reverse(t *testing.T) { t.Fatalf("unexpected error: %v", err) } - if got := b[0] >> 4; got != pd1.EncodingUncompressed { + if got := b[0] >> 4; got != tsm1.EncodingUncompressed { t.Fatalf("Wrong encoding used: expected uncompressed, got %v", got) } - dec := pd1.NewTimeDecoder(b) + dec := tsm1.NewTimeDecoder(b) i := 0 for dec.Next() { if ts[i] != dec.Read() { @@ -312,7 +312,7 @@ func Test_TimeEncoder_Reverse(t *testing.T) { } func Test_TimeEncoder_220SecondDelta(t *testing.T) { - enc := pd1.NewTimeEncoder() + enc := tsm1.NewTimeEncoder() var ts []time.Time now := time.Now() for i := 0; i < 220; i++ { @@ -333,11 +333,11 @@ func Test_TimeEncoder_220SecondDelta(t *testing.T) { t.Fatalf("unexpected length: got %v, exp %v", len(b), exp) } - if got := b[0] >> 4; got != pd1.EncodingRLE { + if got := b[0] >> 4; got != tsm1.EncodingRLE { t.Fatalf("Wrong encoding used: expected uncompressed, got %v", got) } - dec := pd1.NewTimeDecoder(b) + dec := tsm1.NewTimeDecoder(b) i := 0 for dec.Next() { if ts[i] != dec.Read() { @@ -356,7 +356,7 @@ func Test_TimeEncoder_220SecondDelta(t *testing.T) { } func BenchmarkTimeEncoder(b *testing.B) { - enc := pd1.NewTimeEncoder() + enc := tsm1.NewTimeEncoder() x := make([]time.Time, 1024) for i := 0; i < len(x); i++ { x[i] = time.Now() @@ -371,7 +371,7 @@ func BenchmarkTimeEncoder(b *testing.B) { func BenchmarkTimeDecoder(b *testing.B) { x := make([]time.Time, 1024) - enc := pd1.NewTimeEncoder() + enc := tsm1.NewTimeEncoder() for i := 0; i < len(x); i++ { x[i] = time.Now() enc.Write(x[i]) @@ -382,7 +382,7 @@ func BenchmarkTimeDecoder(b *testing.B) { for i := 0; i < b.N; i++ { b.StopTimer() - dec := pd1.NewTimeDecoder(bytes) + dec := tsm1.NewTimeDecoder(bytes) b.StartTimer() for dec.Next() { } diff --git a/tsdb/engine/pd1/pd1.go b/tsdb/engine/tsm1/tsm1.go similarity index 99% rename from tsdb/engine/pd1/pd1.go rename to tsdb/engine/tsm1/tsm1.go index c6a8e7c3c5f..d569d5a1528 100644 --- a/tsdb/engine/pd1/pd1.go +++ b/tsdb/engine/tsm1/tsm1.go @@ -1,4 +1,4 @@ -package pd1 +package tsm1 import ( "encoding/binary" diff --git a/tsdb/engine/pd1/pd1_test.go b/tsdb/engine/tsm1/tsm1_test.go similarity index 99% rename from tsdb/engine/pd1/pd1_test.go rename to tsdb/engine/tsm1/tsm1_test.go index 02f1a576631..e8c3a724672 100644 --- a/tsdb/engine/pd1/pd1_test.go +++ b/tsdb/engine/tsm1/tsm1_test.go @@ -1,4 +1,4 @@ -package pd1_test +package tsm1_test import ( "encoding/binary" @@ -13,7 +13,7 @@ import ( "github.com/influxdb/influxdb/influxql" "github.com/influxdb/influxdb/models" "github.com/influxdb/influxdb/tsdb" - "github.com/influxdb/influxdb/tsdb/engine/pd1" + "github.com/influxdb/influxdb/tsdb/engine/tsm1" ) func TestEngine_WriteAndReadFloats(t *testing.T) { @@ -1236,21 +1236,21 @@ func TestEngine_Deletes(t *testing.T) { }() } -// Engine represents a test wrapper for pd1.Engine. +// Engine represents a test wrapper for tsm1.Engine. type Engine struct { - *pd1.Engine + *tsm1.Engine } // NewEngine returns a new instance of Engine. func NewEngine(opt tsdb.EngineOptions) *Engine { - dir, err := ioutil.TempDir("", "pd1-test") + dir, err := ioutil.TempDir("", "tsm1-test") if err != nil { panic("couldn't get temp dir") } // Create test wrapper and attach mocks. e := &Engine{ - Engine: pd1.NewEngine(dir, dir, opt).(*pd1.Engine), + Engine: tsm1.NewEngine(dir, dir, opt).(*tsm1.Engine), } return e diff --git a/tsdb/engine/pd1/tx.go b/tsdb/engine/tsm1/tx.go similarity index 99% rename from tsdb/engine/pd1/tx.go rename to tsdb/engine/tsm1/tx.go index 89d62e9d88a..d5f31110d18 100644 --- a/tsdb/engine/pd1/tx.go +++ b/tsdb/engine/tsm1/tx.go @@ -1,4 +1,4 @@ -package pd1 +package tsm1 import ( "io" diff --git a/tsdb/engine/pd1/wal.go b/tsdb/engine/tsm1/wal.go similarity index 98% rename from tsdb/engine/pd1/wal.go rename to tsdb/engine/tsm1/wal.go index f253db33f51..9cc2a2fd8fc 100644 --- a/tsdb/engine/pd1/wal.go +++ b/tsdb/engine/tsm1/wal.go @@ -1,4 +1,4 @@ -package pd1 +package tsm1 import ( "encoding/json" @@ -131,7 +131,7 @@ func NewLog(path string) *Log { SegmentSize: DefaultSegmentSize, FlushMemorySizeThreshold: tsdb.DefaultFlushMemorySizeThreshold, MaxMemorySizeThreshold: tsdb.DefaultMaxMemorySizeThreshold, - logger: log.New(os.Stderr, "[pd1wal] ", log.LstdFlags), + logger: log.New(os.Stderr, "[tsm1wal] ", log.LstdFlags), } } @@ -139,8 +139,8 @@ func NewLog(path string) *Log { func (l *Log) Open() error { if l.LoggingEnabled { - l.logger.Printf("PD1 WAL starting with %d flush memory size threshold and %d max memory size threshold\n", l.FlushMemorySizeThreshold, l.MaxMemorySizeThreshold) - l.logger.Printf("PD1 WAL writing to %s\n", l.path) + l.logger.Printf("tsm1 WAL starting with %d flush memory size threshold and %d max memory size threshold\n", l.FlushMemorySizeThreshold, l.MaxMemorySizeThreshold) + l.logger.Printf("tsm1 WAL writing to %s\n", l.path) } if err := os.MkdirAll(l.path, 0777); err != nil { return err diff --git a/tsdb/engine/pd1/wal_test.go b/tsdb/engine/tsm1/wal_test.go similarity index 85% rename from tsdb/engine/pd1/wal_test.go rename to tsdb/engine/tsm1/wal_test.go index 4718f4cebd2..dfc5cda8cc8 100644 --- a/tsdb/engine/pd1/wal_test.go +++ b/tsdb/engine/tsm1/wal_test.go @@ -1,4 +1,4 @@ -package pd1_test +package tsm1_test import ( "io/ioutil" @@ -8,19 +8,19 @@ import ( "github.com/influxdb/influxdb/models" "github.com/influxdb/influxdb/tsdb" - "github.com/influxdb/influxdb/tsdb/engine/pd1" + "github.com/influxdb/influxdb/tsdb/engine/tsm1" ) func TestWAL_TestWriteQueryOpen(t *testing.T) { w := NewWAL() defer w.Cleanup() - var vals map[string]pd1.Values + var vals map[string]tsm1.Values var fields map[string]*tsdb.MeasurementFields var series []*tsdb.SeriesCreate w.Index = &MockIndexWriter{ - fn: func(valuesByKey map[string]pd1.Values, measurementFieldsToSave map[string]*tsdb.MeasurementFields, seriesToCreate []*tsdb.SeriesCreate) error { + fn: func(valuesByKey map[string]tsm1.Values, measurementFieldsToSave map[string]*tsdb.MeasurementFields, seriesToCreate []*tsdb.SeriesCreate) error { vals = valuesByKey fields = measurementFieldsToSave series = seriesToCreate @@ -141,18 +141,18 @@ func TestWAL_TestWriteQueryOpen(t *testing.T) { } type Log struct { - *pd1.Log + *tsm1.Log path string } func NewWAL() *Log { - dir, err := ioutil.TempDir("", "pd1-test") + dir, err := ioutil.TempDir("", "tsm1-test") if err != nil { panic("couldn't get temp dir") } l := &Log{ - Log: pd1.NewLog(dir), + Log: tsm1.NewLog(dir), path: dir, } l.LoggingEnabled = true @@ -166,10 +166,10 @@ func (l *Log) Cleanup() error { } type MockIndexWriter struct { - fn func(valuesByKey map[string]pd1.Values, measurementFieldsToSave map[string]*tsdb.MeasurementFields, seriesToCreate []*tsdb.SeriesCreate) error + fn func(valuesByKey map[string]tsm1.Values, measurementFieldsToSave map[string]*tsdb.MeasurementFields, seriesToCreate []*tsdb.SeriesCreate) error } -func (m *MockIndexWriter) Write(valuesByKey map[string]pd1.Values, measurementFieldsToSave map[string]*tsdb.MeasurementFields, seriesToCreate []*tsdb.SeriesCreate) error { +func (m *MockIndexWriter) Write(valuesByKey map[string]tsm1.Values, measurementFieldsToSave map[string]*tsdb.MeasurementFields, seriesToCreate []*tsdb.SeriesCreate) error { return m.fn(valuesByKey, measurementFieldsToSave, seriesToCreate) } diff --git a/tsdb/engine/pd1/write_lock.go b/tsdb/engine/tsm1/write_lock.go similarity index 99% rename from tsdb/engine/pd1/write_lock.go rename to tsdb/engine/tsm1/write_lock.go index a791b663abb..f4514e58aaa 100644 --- a/tsdb/engine/pd1/write_lock.go +++ b/tsdb/engine/tsm1/write_lock.go @@ -1,4 +1,4 @@ -package pd1 +package tsm1 import ( "reflect" diff --git a/tsdb/engine/pd1/write_lock_test.go b/tsdb/engine/tsm1/write_lock_test.go similarity index 91% rename from tsdb/engine/pd1/write_lock_test.go rename to tsdb/engine/tsm1/write_lock_test.go index 4cdc88e816e..7fa17c530c5 100644 --- a/tsdb/engine/pd1/write_lock_test.go +++ b/tsdb/engine/tsm1/write_lock_test.go @@ -1,15 +1,15 @@ -package pd1_test +package tsm1_test import ( // "sync" "testing" "time" - "github.com/influxdb/influxdb/tsdb/engine/pd1" + "github.com/influxdb/influxdb/tsdb/engine/tsm1" ) func TestWriteLock_FullCover(t *testing.T) { - w := &pd1.WriteLock{} + w := &tsm1.WriteLock{} w.LockRange(2, 10) lock := make(chan bool) @@ -27,7 +27,7 @@ func TestWriteLock_FullCover(t *testing.T) { } func TestWriteLock_RightIntersect(t *testing.T) { - w := &pd1.WriteLock{} + w := &tsm1.WriteLock{} w.LockRange(2, 10) lock := make(chan bool) @@ -45,7 +45,7 @@ func TestWriteLock_RightIntersect(t *testing.T) { } func TestWriteLock_LeftIntersect(t *testing.T) { - w := &pd1.WriteLock{} + w := &tsm1.WriteLock{} w.LockRange(1, 4) lock := make(chan bool) @@ -63,7 +63,7 @@ func TestWriteLock_LeftIntersect(t *testing.T) { } func TestWriteLock_Inside(t *testing.T) { - w := &pd1.WriteLock{} + w := &tsm1.WriteLock{} w.LockRange(4, 8) lock := make(chan bool) @@ -81,7 +81,7 @@ func TestWriteLock_Inside(t *testing.T) { } func TestWriteLock_Same(t *testing.T) { - w := &pd1.WriteLock{} + w := &tsm1.WriteLock{} w.LockRange(2, 10) lock := make(chan bool) @@ -99,7 +99,7 @@ func TestWriteLock_Same(t *testing.T) { } // func TestWriteLock_FreeRangeWithContentionElsewhere(t *testing.T) { -// w := &pd1.WriteLock{} +// w := &tsm1.WriteLock{} // w.LockRange(2, 10) // lock := make(chan bool) From d47ddb54541fc6b7cdc39d69757c4c6ce7cf3f41 Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Thu, 1 Oct 2015 15:30:13 -0400 Subject: [PATCH 122/139] Cleanup after pd1 -> tsm1 name change. --- tsdb/config.go | 8 ++++---- tsdb/engine.go | 6 +++--- tsdb/engine/engine.go | 2 +- tsdb/engine/tsm1/tsm1.go | 6 +++--- tsdb/shard.go | 2 +- 5 files changed, 12 insertions(+), 12 deletions(-) diff --git a/tsdb/config.go b/tsdb/config.go index 1e7e29a0d86..a329c1268bb 100644 --- a/tsdb/config.go +++ b/tsdb/config.go @@ -8,7 +8,7 @@ import ( const ( // DefaultEngine is the default engine for new shards - DefaultEngine = "pd1" + DefaultEngine = "tsm1" // DefaultMaxWALSize is the default size of the WAL before it is flushed. DefaultMaxWALSize = 100 * 1024 * 1024 // 100MB @@ -44,7 +44,7 @@ const ( // size for the in-memory WAL cache. DefaultPartitionSizeThreshold = 50 * 1024 * 1024 // 50MB - // Default WAL settings for the PD1 WAL + // Default WAL settings for the TSM1 WAL DefaultFlushMemorySizeThreshold = 5 * 1024 * 1024 // 5MB DefaultMaxMemorySizeThreshold = 100 * 1024 * 1024 // 100MB DefaultIndexCompactionAge = time.Minute @@ -71,11 +71,11 @@ type Config struct { WALFlushColdInterval toml.Duration `toml:"wal-flush-cold-interval"` WALPartitionSizeThreshold uint64 `toml:"wal-partition-size-threshold"` - // WAL configuration options for pd1 introduced in 0.9.5 + // WAL configuration options for tsm1 introduced in 0.9.5 WALFlushMemorySizeThreshold int `toml:"wal-flush-memory-size-threshold"` WALMaxMemorySizeThreshold int `toml:"wal-max-memory-size-threshold"` - // compaction options for pd1 introduced in 0.9.5 + // compaction options for tsm1 introduced in 0.9.5 // IndexCompactionAge specifies the duration after the data file creation time // at which it is eligible to be compacted diff --git a/tsdb/engine.go b/tsdb/engine.go index 407801d842e..97ca51d9ddb 100644 --- a/tsdb/engine.go +++ b/tsdb/engine.go @@ -46,7 +46,7 @@ type EngineFormat int const ( B1Format EngineFormat = iota BZ1Format - PD1Format + TSM1Format ) // NewEngineFunc creates a new engine. @@ -74,7 +74,7 @@ func NewEngine(path string, walPath string, options EngineOptions) (Engine, erro // Only bolt-based backends are currently supported so open it and check the format. var format string if err := func() error { - // if it's a dir then it's a pd1 engine + // if it's a dir then it's a tsm1 engine f, err := os.Open(path) if err != nil { return err @@ -85,7 +85,7 @@ func NewEngine(path string, walPath string, options EngineOptions) (Engine, erro return err } if fi.Mode().IsDir() { - format = "pd1" + format = "tsm1" return nil } diff --git a/tsdb/engine/engine.go b/tsdb/engine/engine.go index 03022f9d0e7..6c8cb51e193 100644 --- a/tsdb/engine/engine.go +++ b/tsdb/engine/engine.go @@ -3,5 +3,5 @@ package engine import ( _ "github.com/influxdb/influxdb/tsdb/engine/b1" _ "github.com/influxdb/influxdb/tsdb/engine/bz1" - _ "github.com/influxdb/influxdb/tsdb/engine/pd1" + _ "github.com/influxdb/influxdb/tsdb/engine/tsm1" ) diff --git a/tsdb/engine/tsm1/tsm1.go b/tsdb/engine/tsm1/tsm1.go index d569d5a1528..b47413868d8 100644 --- a/tsdb/engine/tsm1/tsm1.go +++ b/tsdb/engine/tsm1/tsm1.go @@ -24,7 +24,7 @@ import ( const ( // Format is the file format name of this engine. - Format = "pd1" + Format = "tsm1" // FieldsFileExtension is the extension for the file that stores compressed field // encoding data for this db @@ -125,7 +125,7 @@ func NewEngine(path string, walPath string, opt tsdb.EngineOptions) tsdb.Engine e := &Engine{ path: path, writeLock: &WriteLock{}, - logger: log.New(os.Stderr, "[pd1] ", log.LstdFlags), + logger: log.New(os.Stderr, "[tsm1] ", log.LstdFlags), // TODO: this is the function where we can inject a check against the in memory collisions HashSeriesField: hashSeriesField, @@ -180,7 +180,7 @@ func (e *Engine) PerformMaintenance() { // Format returns the format type of this engine func (e *Engine) Format() tsdb.EngineFormat { - return tsdb.PD1Format + return tsdb.TSM1Format } // Open opens and initializes the engine. diff --git a/tsdb/shard.go b/tsdb/shard.go index 1f606b613b3..b2589443163 100644 --- a/tsdb/shard.go +++ b/tsdb/shard.go @@ -234,7 +234,7 @@ func (s *Shard) WritePoints(points []models.Point) error { // make sure all data is encoded before attempting to save to bolt // only required for the b1 and bz1 formats - if s.engine.Format() != PD1Format { + if s.engine.Format() != TSM1Format { for _, p := range points { // Ignore if raw data has already been marshaled. if p.Data() != nil { From ab791ba9134bba5d084bb25baa629f165ce17ad3 Mon Sep 17 00:00:00 2001 From: Jason Wilder Date: Thu, 1 Oct 2015 13:59:27 -0600 Subject: [PATCH 123/139] Fix TestStoreOpenShardCreateDelete Shard path can be a directory. --- tsdb/store.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tsdb/store.go b/tsdb/store.go index 1fb8560167e..bee68c7fddf 100644 --- a/tsdb/store.go +++ b/tsdb/store.go @@ -131,7 +131,7 @@ func (s *Store) DeleteShard(shardID uint64) error { return err } - if err := os.Remove(sh.path); err != nil { + if err := os.RemoveAll(sh.path); err != nil { return err } From 5d9b89d6012710dc32e68d2cb9ea7f641daa4bb7 Mon Sep 17 00:00:00 2001 From: Jason Wilder Date: Fri, 2 Oct 2015 09:38:39 -0600 Subject: [PATCH 124/139] Disable copier test Not implemented for tsm1 engine --- services/copier/service_test.go | 1 + 1 file changed, 1 insertion(+) diff --git a/services/copier/service_test.go b/services/copier/service_test.go index a5266087d7f..ce1151d3cf8 100644 --- a/services/copier/service_test.go +++ b/services/copier/service_test.go @@ -19,6 +19,7 @@ import ( // Ensure the service can return shard data. func TestService_handleConn(t *testing.T) { + t.Skip("not implemented for tsm1 engine") s := MustOpenService() defer s.Close() From b1a57e1628a24c05067d2d128f36841e09ad4d39 Mon Sep 17 00:00:00 2001 From: Jason Wilder Date: Fri, 2 Oct 2015 09:39:08 -0600 Subject: [PATCH 125/139] Fix go vet errors --- tsdb/engine/tsm1/tsm1_test.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tsdb/engine/tsm1/tsm1_test.go b/tsdb/engine/tsm1/tsm1_test.go index e8c3a724672..6a39e41ed8c 100644 --- a/tsdb/engine/tsm1/tsm1_test.go +++ b/tsdb/engine/tsm1/tsm1_test.go @@ -731,7 +731,7 @@ func TestEngine_CursorDescendingOrder(t *testing.T) { verify() if err := e.WAL.Flush(); err != nil { - t.Fatalf("error flushing WAL %s", err.Error) + t.Fatalf("error flushing WAL %s", err.Error()) } fmt.Println("verify 2") @@ -997,7 +997,7 @@ func TestEngine_WriteIntoCompactedFile(t *testing.T) { } if err := e.Compact(true); err != nil { - t.Fatalf("error compacting: %s", err.Error) + t.Fatalf("error compacting: %s", err.Error()) } if err := e.WritePoints([]models.Point{p4}, nil, nil); err != nil { @@ -1005,7 +1005,7 @@ func TestEngine_WriteIntoCompactedFile(t *testing.T) { } if err := e.Compact(true); err != nil { - t.Fatalf("error compacting: %s", err.Error) + t.Fatalf("error compacting: %s", err.Error()) } if err := e.WritePoints([]models.Point{p5}, nil, nil); err != nil { From 4c54c78009ead9e715bf835c2b90f5f89d3d5acf Mon Sep 17 00:00:00 2001 From: Jason Wilder Date: Fri, 2 Oct 2015 10:03:20 -0600 Subject: [PATCH 126/139] Move compression encoding constants to encoders Will make it less error-prone to add new encodings int the future since each encoder has it's set of constants. There are some placeholder contants for uncompressed encodings which are not in all encoder currently. --- tsdb/engine/tsm1/bool.go | 9 +++- tsdb/engine/tsm1/encoding.go | 15 ------- tsdb/engine/tsm1/float.go | 18 +++++--- tsdb/engine/tsm1/float_test.go | 22 ++++++++-- tsdb/engine/tsm1/int.go | 15 +++++-- tsdb/engine/tsm1/string.go | 9 +++- tsdb/engine/tsm1/string_test.go | 20 ++++----- tsdb/engine/tsm1/timestamp.go | 21 ++++++--- tsdb/engine/tsm1/timestamp_test.go | 68 +++++++++++++++--------------- 9 files changed, 114 insertions(+), 83 deletions(-) diff --git a/tsdb/engine/tsm1/bool.go b/tsdb/engine/tsm1/bool.go index 8d9653d1991..ac08555cfe5 100644 --- a/tsdb/engine/tsm1/bool.go +++ b/tsdb/engine/tsm1/bool.go @@ -7,6 +7,13 @@ package tsm1 import "encoding/binary" +const ( + // boolUncompressed is an uncompressed boolean format + boolUncompressed = 0 + // boolCompressedBitPacked is an bit packed format using 1 bit per boolean + boolCompressedBitPacked = 1 +) + type BoolEncoder interface { Write(b bool) Bytes() ([]byte, error) @@ -75,7 +82,7 @@ func (e *boolEncoder) Bytes() ([]byte, error) { b := make([]byte, 10+1) // Store the encoding type in the 4 high bits of the first byte - b[0] = byte(EncodingBitPacked) << 4 + b[0] = byte(boolCompressedBitPacked) << 4 i := 1 // Encode the number of bools written diff --git a/tsdb/engine/tsm1/encoding.go b/tsdb/engine/tsm1/encoding.go index 4b6a112d568..eef2b1d23e7 100644 --- a/tsdb/engine/tsm1/encoding.go +++ b/tsdb/engine/tsm1/encoding.go @@ -10,21 +10,6 @@ import ( ) const ( - // EncodingPackedSimple is a bit-packed format - EncodingPackedSimple = 0 - - // EncodingRLE is a run-length encoded format - EncodingRLE = 1 - - // EncodingUncompressed is a non-compressed format - EncodingUncompressed = 2 - - // EncodingBitPacked is a basic bit-packed format - EncodingBitPacked = 3 - - // EncodingSnappy is a snappy encoded format - EncodingSnappy = 4 - // BlockFloat64 designates a block encodes float64 values BlockFloat64 = 0 diff --git a/tsdb/engine/tsm1/float.go b/tsdb/engine/tsm1/float.go index ff6a61c505d..7ac1d93b492 100644 --- a/tsdb/engine/tsm1/float.go +++ b/tsdb/engine/tsm1/float.go @@ -17,6 +17,13 @@ import ( "github.com/dgryski/go-bitstream" ) +const ( + // floatUncompressed is an uncompressed format using 8 bytes per value + floatUncompressed = 0 + // floatCompressedGorilla is a compressed format using the gorilla paper encoding + floatCompressedGorilla = 1 +) + type FloatEncoder struct { val float64 @@ -43,7 +50,7 @@ func NewFloatEncoder() *FloatEncoder { } func (s *FloatEncoder) Bytes() []byte { - return s.buf.Bytes() + return append([]byte{floatCompressedGorilla << 4}, s.buf.Bytes()...) } func (s *FloatEncoder) Finish() { @@ -95,11 +102,6 @@ func (s *FloatEncoder) Push(v float64) { s.val = v } -func (s *FloatEncoder) FloatDecoder() *FloatDecoder { - iter, _ := NewFloatDecoder(s.buf.Bytes()) - return iter -} - type FloatDecoder struct { val float64 @@ -117,7 +119,9 @@ type FloatDecoder struct { } func NewFloatDecoder(b []byte) (*FloatDecoder, error) { - br := bitstream.NewReader(bytes.NewReader(b)) + // first byte is the compression type but we currently just have gorilla + // compression + br := bitstream.NewReader(bytes.NewReader(b[1:])) v, err := br.ReadBits(64) if err != nil { diff --git a/tsdb/engine/tsm1/float_test.go b/tsdb/engine/tsm1/float_test.go index 00b259bf95d..49e811a174e 100644 --- a/tsdb/engine/tsm1/float_test.go +++ b/tsdb/engine/tsm1/float_test.go @@ -28,7 +28,12 @@ func TestFloatEncoder_Simple(t *testing.T) { s.Finish() - it := s.FloatDecoder() + b := s.Bytes() + + it, err := tsm1.NewFloatDecoder(b) + if err != nil { + t.Fatalf("unexpected error creating float decoder: %v", err) + } want := []float64{ 12, @@ -100,7 +105,13 @@ func TestFloatEncoder_Roundtrip(t *testing.T) { } s.Finish() - it := s.FloatDecoder() + b := s.Bytes() + + it, err := tsm1.NewFloatDecoder(b) + if err != nil { + t.Fatalf("unexpected error creating float decoder: %v", err) + } + for _, w := range TwoHoursData { if !it.Next() { t.Fatalf("Next()=false, want true") @@ -137,11 +148,16 @@ func BenchmarkFloatDecoder(b *testing.B) { s.Push(tt.v) } s.Finish() + bytes := s.Bytes() b.ResetTimer() for i := 0; i < b.N; i++ { - it := s.FloatDecoder() + it, err := tsm1.NewFloatDecoder(bytes) + if err != nil { + b.Fatalf("unexpected error creating float decoder: %v", err) + } + for j := 0; j < len(TwoHoursData); it.Next() { j++ } diff --git a/tsdb/engine/tsm1/int.go b/tsdb/engine/tsm1/int.go index b178c503751..40e615657dd 100644 --- a/tsdb/engine/tsm1/int.go +++ b/tsdb/engine/tsm1/int.go @@ -27,6 +27,13 @@ import ( "github.com/jwilder/encoding/simple8b" ) +const ( + // intUncompressed is an uncompressed format using 8 bytes per point + intUncompressed = 0 + // intCompressedSimple is a bit-packed format using simple8b encoding + intCompressedSimple = 1 +) + type Int64Encoder interface { Write(v int64) Bytes() ([]byte, error) @@ -68,7 +75,7 @@ func (e *int64Encoder) encodePacked() ([]byte, error) { b := make([]byte, 1+len(encoded)*8) // 4 high bits of first byte store the encoding type for the block - b[0] = byte(EncodingPackedSimple) << 4 + b[0] = byte(intCompressedSimple) << 4 for i, v := range encoded { binary.BigEndian.PutUint64(b[1+i*8:1+i*8+8], v) @@ -79,7 +86,7 @@ func (e *int64Encoder) encodePacked() ([]byte, error) { func (e *int64Encoder) encodeUncompressed() ([]byte, error) { b := make([]byte, 1+len(e.values)*8) // 4 high bits of first byte store the encoding type for the block - b[0] = byte(EncodingUncompressed) << 4 + b[0] = byte(intUncompressed) << 4 for i, v := range e.values { binary.BigEndian.PutUint64(b[1+i*8:1+i*8+8], v) @@ -123,9 +130,9 @@ func (d *int64Decoder) Next() bool { if d.i >= d.n { switch d.encoding { - case EncodingUncompressed: + case intUncompressed: d.decodeUncompressed() - case EncodingPackedSimple: + case intCompressedSimple: d.decodePacked() default: panic(fmt.Sprintf("unknown encoding %v", d.encoding)) diff --git a/tsdb/engine/tsm1/string.go b/tsdb/engine/tsm1/string.go index 1b5dafac3f0..a2025809de9 100644 --- a/tsdb/engine/tsm1/string.go +++ b/tsdb/engine/tsm1/string.go @@ -12,6 +12,13 @@ import ( "github.com/golang/snappy" ) +const ( + // stringUncompressed is a an uncompressed format encoding strings as raw bytes + stringUncompressed = 0 + // stringCompressedSnappy is a compressed encoding using Snappy compression + stringCompressedSnappy = 1 +) + type StringEncoder interface { Write(s string) Bytes() ([]byte, error) @@ -45,7 +52,7 @@ func (e *stringEncoder) Bytes() ([]byte, error) { // Compress the currently appended bytes using snappy and prefix with // a 1 byte header for future extension data := snappy.Encode(nil, e.bytes) - return append([]byte{EncodingSnappy << 4}, data...), nil + return append([]byte{stringCompressedSnappy << 4}, data...), nil } type stringDecoder struct { diff --git a/tsdb/engine/tsm1/string_test.go b/tsdb/engine/tsm1/string_test.go index 8710a50b365..f1bf1738152 100644 --- a/tsdb/engine/tsm1/string_test.go +++ b/tsdb/engine/tsm1/string_test.go @@ -1,27 +1,25 @@ -package tsm1_test +package tsm1 import ( "fmt" "testing" - - "github.com/influxdb/influxdb/tsdb/engine/tsm1" ) func Test_StringEncoder_NoValues(t *testing.T) { - enc := tsm1.NewStringEncoder() + enc := NewStringEncoder() b, err := enc.Bytes() if err != nil { t.Fatalf("unexpected error: %v", err) } - dec := tsm1.NewStringDecoder(b) + dec := NewStringDecoder(b) if dec.Next() { t.Fatalf("unexpected next value: got true, exp false") } } func Test_StringEncoder_Single(t *testing.T) { - enc := tsm1.NewStringEncoder() + enc := NewStringEncoder() v1 := "v1" enc.Write(v1) b, err := enc.Bytes() @@ -29,7 +27,7 @@ func Test_StringEncoder_Single(t *testing.T) { t.Fatalf("unexpected error: %v", err) } - dec := tsm1.NewStringDecoder(b) + dec := NewStringDecoder(b) if !dec.Next() { t.Fatalf("unexpected next value: got false, exp true") } @@ -40,7 +38,7 @@ func Test_StringEncoder_Single(t *testing.T) { } func Test_StringEncoder_Multi_Compressed(t *testing.T) { - enc := tsm1.NewStringEncoder() + enc := NewStringEncoder() values := make([]string, 10) for i := range values { @@ -53,15 +51,15 @@ func Test_StringEncoder_Multi_Compressed(t *testing.T) { t.Fatalf("unexpected error: %v", err) } - if b[0]>>4 != tsm1.EncodingSnappy { - t.Fatalf("unexpected encoding: got %v, exp %v", b[0], tsm1.EncodingSnappy) + if b[0]>>4 != stringCompressedSnappy { + t.Fatalf("unexpected encoding: got %v, exp %v", b[0], stringCompressedSnappy) } if exp := 47; len(b) != exp { t.Fatalf("unexpected length: got %v, exp %v", len(b), exp) } - dec := tsm1.NewStringDecoder(b) + dec := NewStringDecoder(b) for i, v := range values { if !dec.Next() { diff --git a/tsdb/engine/tsm1/timestamp.go b/tsdb/engine/tsm1/timestamp.go index 59990f5cb41..8fcb0b4ce2b 100644 --- a/tsdb/engine/tsm1/timestamp.go +++ b/tsdb/engine/tsm1/timestamp.go @@ -41,6 +41,15 @@ import ( "github.com/jwilder/encoding/simple8b" ) +const ( + // timeUncompressed is a an uncompressed format using 8 bytes per timestamp + timeUncompressed = 0 + // timeCompressedPackedSimple is a bit-packed format using simple8b encoding + timeCompressedPackedSimple = 1 + // timeCompressedRLE is a run-length encoding format + timeCompressedRLE = 2 +) + // TimeEncoder encodes time.Time to byte slices. type TimeEncoder interface { Write(t time.Time) @@ -135,7 +144,7 @@ func (e *encoder) encodePacked(div uint64, dts []uint64) ([]byte, error) { b := make([]byte, 8+1) // 4 high bits used for the encoding type - b[0] = byte(EncodingPackedSimple) << 4 + b[0] = byte(timeCompressedPackedSimple) << 4 // 4 low bits are the log10 divisor b[0] |= byte(math.Log10(float64(div))) @@ -153,7 +162,7 @@ func (e *encoder) encodePacked(div uint64, dts []uint64) ([]byte, error) { func (e *encoder) encodeRaw() ([]byte, error) { b := make([]byte, 1+len(e.ts)*8) - b[0] = byte(EncodingUncompressed) << 4 + b[0] = byte(timeUncompressed) << 4 for i, v := range e.ts { binary.BigEndian.PutUint64(b[1+i*8:1+i*8+8], uint64(v)) } @@ -165,7 +174,7 @@ func (e *encoder) encodeRLE(first, delta, div uint64, n int) ([]byte, error) { b := make([]byte, 1+10*3) // 4 high bits used for the encoding type - b[0] = byte(EncodingRLE) << 4 + b[0] = byte(timeCompressedRLE) << 4 // 4 low bits are the log10 divisor b[0] |= byte(math.Log10(float64(div))) @@ -213,11 +222,11 @@ func (d *decoder) decode(b []byte) { // Encoding type is stored in the 4 high bits of the first byte encoding := b[0] >> 4 switch encoding { - case EncodingUncompressed: + case timeUncompressed: d.decodeRaw(b[1:]) - case EncodingRLE: + case timeCompressedRLE: d.decodeRLE(b) - case EncodingPackedSimple: + case timeCompressedPackedSimple: d.decodePacked(b) default: panic(fmt.Sprintf("unknown encoding: %v", encoding)) diff --git a/tsdb/engine/tsm1/timestamp_test.go b/tsdb/engine/tsm1/timestamp_test.go index dbb5a2341a6..402a6578a11 100644 --- a/tsdb/engine/tsm1/timestamp_test.go +++ b/tsdb/engine/tsm1/timestamp_test.go @@ -1,14 +1,12 @@ -package tsm1_test +package tsm1 import ( "testing" "time" - - "github.com/influxdb/influxdb/tsdb/engine/tsm1" ) func Test_TimeEncoder(t *testing.T) { - enc := tsm1.NewTimeEncoder() + enc := NewTimeEncoder() x := []time.Time{} now := time.Unix(0, 0) @@ -24,11 +22,11 @@ func Test_TimeEncoder(t *testing.T) { t.Fatalf("unexpected error: %v", err) } - if got := b[0] >> 4; got != tsm1.EncodingPackedSimple { + if got := b[0] >> 4; got != timeCompressedPackedSimple { t.Fatalf("Wrong encoding used: expected uncompressed, got %v", got) } - dec := tsm1.NewTimeDecoder(b) + dec := NewTimeDecoder(b) for i, v := range x { if !dec.Next() { t.Fatalf("Next == false, expected true") @@ -41,20 +39,20 @@ func Test_TimeEncoder(t *testing.T) { } func Test_TimeEncoder_NoValues(t *testing.T) { - enc := tsm1.NewTimeEncoder() + enc := NewTimeEncoder() b, err := enc.Bytes() if err != nil { t.Fatalf("unexpected error: %v", err) } - dec := tsm1.NewTimeDecoder(b) + dec := NewTimeDecoder(b) if dec.Next() { t.Fatalf("unexpected next value: got true, exp false") } } func Test_TimeEncoder_One(t *testing.T) { - enc := tsm1.NewTimeEncoder() + enc := NewTimeEncoder() tm := time.Unix(0, 0) enc.Write(tm) @@ -63,11 +61,11 @@ func Test_TimeEncoder_One(t *testing.T) { t.Fatalf("unexpected error: %v", err) } - if got := b[0] >> 4; got != tsm1.EncodingPackedSimple { + if got := b[0] >> 4; got != timeCompressedPackedSimple { t.Fatalf("Wrong encoding used: expected uncompressed, got %v", got) } - dec := tsm1.NewTimeDecoder(b) + dec := NewTimeDecoder(b) if !dec.Next() { t.Fatalf("unexpected next value: got true, exp false") } @@ -78,7 +76,7 @@ func Test_TimeEncoder_One(t *testing.T) { } func Test_TimeEncoder_Two(t *testing.T) { - enc := tsm1.NewTimeEncoder() + enc := NewTimeEncoder() t1 := time.Unix(0, 0) t2 := time.Unix(0, 1) enc.Write(t1) @@ -89,11 +87,11 @@ func Test_TimeEncoder_Two(t *testing.T) { t.Fatalf("unexpected error: %v", err) } - if got := b[0] >> 4; got != tsm1.EncodingPackedSimple { + if got := b[0] >> 4; got != timeCompressedPackedSimple { t.Fatalf("Wrong encoding used: expected uncompressed, got %v", got) } - dec := tsm1.NewTimeDecoder(b) + dec := NewTimeDecoder(b) if !dec.Next() { t.Fatalf("unexpected next value: got true, exp false") } @@ -112,7 +110,7 @@ func Test_TimeEncoder_Two(t *testing.T) { } func Test_TimeEncoder_Three(t *testing.T) { - enc := tsm1.NewTimeEncoder() + enc := NewTimeEncoder() t1 := time.Unix(0, 0) t2 := time.Unix(0, 1) t3 := time.Unix(0, 2) @@ -126,11 +124,11 @@ func Test_TimeEncoder_Three(t *testing.T) { t.Fatalf("unexpected error: %v", err) } - if got := b[0] >> 4; got != tsm1.EncodingPackedSimple { + if got := b[0] >> 4; got != timeCompressedPackedSimple { t.Fatalf("Wrong encoding used: expected uncompressed, got %v", got) } - dec := tsm1.NewTimeDecoder(b) + dec := NewTimeDecoder(b) if !dec.Next() { t.Fatalf("unexpected next value: got true, exp false") } @@ -157,7 +155,7 @@ func Test_TimeEncoder_Three(t *testing.T) { } func Test_TimeEncoder_Large_Range(t *testing.T) { - enc := tsm1.NewTimeEncoder() + enc := NewTimeEncoder() t1 := time.Unix(0, 1442369134000000000) t2 := time.Unix(0, 1442369135000000000) enc.Write(t1) @@ -167,11 +165,11 @@ func Test_TimeEncoder_Large_Range(t *testing.T) { t.Fatalf("unexpected error: %v", err) } - if got := b[0] >> 4; got != tsm1.EncodingPackedSimple { + if got := b[0] >> 4; got != timeCompressedPackedSimple { t.Fatalf("Wrong encoding used: expected uncompressed, got %v", got) } - dec := tsm1.NewTimeDecoder(b) + dec := NewTimeDecoder(b) if !dec.Next() { t.Fatalf("unexpected next value: got true, exp false") } @@ -190,7 +188,7 @@ func Test_TimeEncoder_Large_Range(t *testing.T) { } func Test_TimeEncoder_Uncompressed(t *testing.T) { - enc := tsm1.NewTimeEncoder() + enc := NewTimeEncoder() t1 := time.Unix(0, 0) t2 := time.Unix(1, 0) @@ -210,11 +208,11 @@ func Test_TimeEncoder_Uncompressed(t *testing.T) { t.Fatalf("length mismatch: got %v, exp %v", len(b), exp) } - if got := b[0] >> 4; got != tsm1.EncodingUncompressed { + if got := b[0] >> 4; got != timeUncompressed { t.Fatalf("Wrong encoding used: expected uncompressed, got %v", got) } - dec := tsm1.NewTimeDecoder(b) + dec := NewTimeDecoder(b) if !dec.Next() { t.Fatalf("unexpected next value: got true, exp false") } @@ -241,7 +239,7 @@ func Test_TimeEncoder_Uncompressed(t *testing.T) { } func Test_TimeEncoder_RLE(t *testing.T) { - enc := tsm1.NewTimeEncoder() + enc := NewTimeEncoder() var ts []time.Time for i := 0; i < 500; i++ { ts = append(ts, time.Unix(int64(i), 0)) @@ -256,7 +254,7 @@ func Test_TimeEncoder_RLE(t *testing.T) { t.Fatalf("length mismatch: got %v, exp %v", len(b), exp) } - if got := b[0] >> 4; got != tsm1.EncodingRLE { + if got := b[0] >> 4; got != timeCompressedRLE { t.Fatalf("Wrong encoding used: expected uncompressed, got %v", got) } @@ -264,7 +262,7 @@ func Test_TimeEncoder_RLE(t *testing.T) { t.Fatalf("unexpected error: %v", err) } - dec := tsm1.NewTimeDecoder(b) + dec := NewTimeDecoder(b) for i, v := range ts { if !dec.Next() { t.Fatalf("Next == false, expected true") @@ -281,7 +279,7 @@ func Test_TimeEncoder_RLE(t *testing.T) { } func Test_TimeEncoder_Reverse(t *testing.T) { - enc := tsm1.NewTimeEncoder() + enc := NewTimeEncoder() ts := []time.Time{ time.Unix(0, 3), time.Unix(0, 2), @@ -297,11 +295,11 @@ func Test_TimeEncoder_Reverse(t *testing.T) { t.Fatalf("unexpected error: %v", err) } - if got := b[0] >> 4; got != tsm1.EncodingUncompressed { + if got := b[0] >> 4; got != timeUncompressed { t.Fatalf("Wrong encoding used: expected uncompressed, got %v", got) } - dec := tsm1.NewTimeDecoder(b) + dec := NewTimeDecoder(b) i := 0 for dec.Next() { if ts[i] != dec.Read() { @@ -312,7 +310,7 @@ func Test_TimeEncoder_Reverse(t *testing.T) { } func Test_TimeEncoder_220SecondDelta(t *testing.T) { - enc := tsm1.NewTimeEncoder() + enc := NewTimeEncoder() var ts []time.Time now := time.Now() for i := 0; i < 220; i++ { @@ -333,11 +331,11 @@ func Test_TimeEncoder_220SecondDelta(t *testing.T) { t.Fatalf("unexpected length: got %v, exp %v", len(b), exp) } - if got := b[0] >> 4; got != tsm1.EncodingRLE { + if got := b[0] >> 4; got != timeCompressedRLE { t.Fatalf("Wrong encoding used: expected uncompressed, got %v", got) } - dec := tsm1.NewTimeDecoder(b) + dec := NewTimeDecoder(b) i := 0 for dec.Next() { if ts[i] != dec.Read() { @@ -356,7 +354,7 @@ func Test_TimeEncoder_220SecondDelta(t *testing.T) { } func BenchmarkTimeEncoder(b *testing.B) { - enc := tsm1.NewTimeEncoder() + enc := NewTimeEncoder() x := make([]time.Time, 1024) for i := 0; i < len(x); i++ { x[i] = time.Now() @@ -371,7 +369,7 @@ func BenchmarkTimeEncoder(b *testing.B) { func BenchmarkTimeDecoder(b *testing.B) { x := make([]time.Time, 1024) - enc := tsm1.NewTimeEncoder() + enc := NewTimeEncoder() for i := 0; i < len(x); i++ { x[i] = time.Now() enc.Write(x[i]) @@ -382,7 +380,7 @@ func BenchmarkTimeDecoder(b *testing.B) { for i := 0; i < b.N; i++ { b.StopTimer() - dec := tsm1.NewTimeDecoder(bytes) + dec := NewTimeDecoder(bytes) b.StartTimer() for dec.Next() { } From 1d754db00bd06811b19c11e1dfa9e760cf1299f9 Mon Sep 17 00:00:00 2001 From: Jason Wilder Date: Fri, 2 Oct 2015 10:46:58 -0600 Subject: [PATCH 127/139] Propogate all encoding errors to engine Avoid panicing in lower level code and allow the engine to decide what it should do. --- tsdb/engine/tsm1/bool.go | 12 +++-- tsdb/engine/tsm1/encoding.go | 79 +++++++++++++++++++++++-------- tsdb/engine/tsm1/encoding_test.go | 30 +++++++++--- tsdb/engine/tsm1/float.go | 2 +- tsdb/engine/tsm1/float_test.go | 8 ++-- tsdb/engine/tsm1/int.go | 15 +++++- tsdb/engine/tsm1/string.go | 19 +++++--- tsdb/engine/tsm1/string_test.go | 15 ++++-- tsdb/engine/tsm1/timestamp.go | 12 +++-- tsdb/engine/tsm1/tsm1.go | 37 ++++++++++++--- 10 files changed, 174 insertions(+), 55 deletions(-) diff --git a/tsdb/engine/tsm1/bool.go b/tsdb/engine/tsm1/bool.go index ac08555cfe5..52ec8d849b0 100644 --- a/tsdb/engine/tsm1/bool.go +++ b/tsdb/engine/tsm1/bool.go @@ -22,6 +22,7 @@ type BoolEncoder interface { type BoolDecoder interface { Next() bool Read() bool + Error() error } type boolEncoder struct { @@ -93,9 +94,10 @@ func (e *boolEncoder) Bytes() ([]byte, error) { } type boolDecoder struct { - b []byte - i int - n int + b []byte + i int + n int + err error } func NewBoolDecoder(b []byte) BoolDecoder { @@ -128,3 +130,7 @@ func (e *boolDecoder) Read() bool { // Returns true if the bit is set return v&mask == mask } + +func (e *boolDecoder) Error() error { + return e.err +} diff --git a/tsdb/engine/tsm1/encoding.go b/tsdb/engine/tsm1/encoding.go index eef2b1d23e7..0d95bf5ac2b 100644 --- a/tsdb/engine/tsm1/encoding.go +++ b/tsdb/engine/tsm1/encoding.go @@ -65,7 +65,7 @@ func (v Values) MaxTime() int64 { return v[len(v)-1].Time().UnixNano() } -func (v Values) Encode(buf []byte) []byte { +func (v Values) Encode(buf []byte) ([]byte, error) { switch v[0].(type) { case *FloatValue: a := make([]*FloatValue, len(v)) @@ -96,7 +96,7 @@ func (v Values) Encode(buf []byte) []byte { return encodeStringBlock(buf, a) } - return nil + return nil, fmt.Errorf("unsupported value type %T", v[0]) } func (v Values) DecodeSameTypeBlock(block []byte) Values { @@ -185,9 +185,9 @@ func (f *FloatValue) Size() int { return 16 } -func encodeFloatBlock(buf []byte, values []*FloatValue) []byte { +func encodeFloatBlock(buf []byte, values []*FloatValue) ([]byte, error) { if len(values) == 0 { - return nil + return nil, nil } // A float block is encoded using different compression strategies @@ -209,7 +209,7 @@ func encodeFloatBlock(buf []byte, values []*FloatValue) []byte { // Encoded timestamp values tb, err := tsenc.Bytes() if err != nil { - panic(err.Error()) + return nil, err } // Encoded float values vb := venc.Bytes() @@ -218,7 +218,7 @@ func encodeFloatBlock(buf []byte, values []*FloatValue) []byte { // in the next byte, followed by the block block := packBlockHeader(values[0].Time(), BlockFloat64) block = append(block, packBlock(tb, vb)...) - return block + return block, nil } func decodeFloatBlock(block []byte) ([]Value, error) { @@ -249,6 +249,15 @@ func decodeFloatBlock(block []byte) ([]Value, error) { a = append(a, &FloatValue{ts, v}) } + // Did timestamp decoding have an error? + if dec.Error() != nil { + return nil, dec.Error() + } + // Did float decoding have an error? + if iter.Error() != nil { + return nil, iter.Error() + } + return a, nil } @@ -273,9 +282,9 @@ func (b *BoolValue) Value() interface{} { return b.value } -func encodeBoolBlock(buf []byte, values []*BoolValue) []byte { +func encodeBoolBlock(buf []byte, values []*BoolValue) ([]byte, error) { if len(values) == 0 { - return nil + return nil, nil } // A bool block is encoded using different compression strategies @@ -295,19 +304,19 @@ func encodeBoolBlock(buf []byte, values []*BoolValue) []byte { // Encoded timestamp values tb, err := tsenc.Bytes() if err != nil { - panic(err.Error()) + return nil, err } // Encoded float values vb, err := venc.Bytes() if err != nil { - panic(err.Error()) + return nil, err } // Preprend the first timestamp of the block in the first 8 bytes and the block // in the next byte, followed by the block block := packBlockHeader(values[0].Time(), BlockBool) block = append(block, packBlock(tb, vb)...) - return block + return block, nil } func decodeBoolBlock(block []byte) ([]Value, error) { @@ -335,6 +344,15 @@ func decodeBoolBlock(block []byte) ([]Value, error) { a = append(a, &BoolValue{ts, v}) } + // Did timestamp decoding have an error? + if dec.Error() != nil { + return nil, dec.Error() + } + // Did bool decoding have an error? + if vdec.Error() != nil { + return nil, vdec.Error() + } + return a, nil } @@ -361,7 +379,7 @@ func (v *Int64Value) Size() int { func (v *Int64Value) String() string { return fmt.Sprintf("%v", v.value) } -func encodeInt64Block(buf []byte, values []*Int64Value) []byte { +func encodeInt64Block(buf []byte, values []*Int64Value) ([]byte, error) { tsEnc := NewTimeEncoder() vEnc := NewInt64Encoder() for _, v := range values { @@ -372,17 +390,17 @@ func encodeInt64Block(buf []byte, values []*Int64Value) []byte { // Encoded timestamp values tb, err := tsEnc.Bytes() if err != nil { - panic(err.Error()) + return nil, err } // Encoded int64 values vb, err := vEnc.Bytes() if err != nil { - panic(err.Error()) + return nil, err } // Preprend the first timestamp of the block in the first 8 bytes block := packBlockHeader(values[0].Time(), BlockInt64) - return append(block, packBlock(tb, vb)...) + return append(block, packBlock(tb, vb)...), nil } func decodeInt64Block(block []byte) ([]Value, error) { @@ -411,6 +429,15 @@ func decodeInt64Block(block []byte) ([]Value, error) { a = append(a, &Int64Value{ts, v}) } + // Did timestamp decoding have an error? + if tsDec.Error() != nil { + return nil, tsDec.Error() + } + // Did int64 decoding have an error? + if vDec.Error() != nil { + return nil, vDec.Error() + } + return a, nil } @@ -437,7 +464,7 @@ func (v *StringValue) Size() int { func (v *StringValue) String() string { return v.value } -func encodeStringBlock(buf []byte, values []*StringValue) []byte { +func encodeStringBlock(buf []byte, values []*StringValue) ([]byte, error) { tsEnc := NewTimeEncoder() vEnc := NewStringEncoder() for _, v := range values { @@ -448,17 +475,17 @@ func encodeStringBlock(buf []byte, values []*StringValue) []byte { // Encoded timestamp values tb, err := tsEnc.Bytes() if err != nil { - panic(err.Error()) + return nil, err } // Encoded int64 values vb, err := vEnc.Bytes() if err != nil { - panic(err.Error()) + return nil, err } // Preprend the first timestamp of the block in the first 8 bytes block := packBlockHeader(values[0].Time(), BlockString) - return append(block, packBlock(tb, vb)...) + return append(block, packBlock(tb, vb)...), nil } func decodeStringBlock(block []byte) ([]Value, error) { @@ -477,7 +504,10 @@ func decodeStringBlock(block []byte) ([]Value, error) { // Setup our timestamp and value decoders tsDec := NewTimeDecoder(tb) - vDec := NewStringDecoder(vb) + vDec, err := NewStringDecoder(vb) + if err != nil { + return nil, err + } // Decode both a timestamp and value var a []Value @@ -487,6 +517,15 @@ func decodeStringBlock(block []byte) ([]Value, error) { a = append(a, &StringValue{ts, v}) } + // Did timestamp decoding have an error? + if tsDec.Error() != nil { + return nil, tsDec.Error() + } + // Did string decoding have an error? + if vDec.Error() != nil { + return nil, vDec.Error() + } + return a, nil } diff --git a/tsdb/engine/tsm1/encoding_test.go b/tsdb/engine/tsm1/encoding_test.go index c4889f18e5d..309b947eb60 100644 --- a/tsdb/engine/tsm1/encoding_test.go +++ b/tsdb/engine/tsm1/encoding_test.go @@ -19,7 +19,10 @@ func TestEncoding_FloatBlock(t *testing.T) { values[i] = tsm1.NewValue(t, float64(i)) } - b := values.Encode(nil) + b, err := values.Encode(nil) + if err != nil { + t.Fatalf("unexpected error: %v", err) + } decodedValues := values.DecodeSameTypeBlock(b) @@ -34,7 +37,10 @@ func TestEncoding_FloatBlock_ZeroTime(t *testing.T) { values[i] = tsm1.NewValue(time.Unix(0, 0), float64(i)) } - b := values.Encode(nil) + b, err := values.Encode(nil) + if err != nil { + t.Fatalf("unexpected error: %v", err) + } decodedValues := values.DecodeSameTypeBlock(b) @@ -51,7 +57,10 @@ func TestEncoding_IntBlock_Basic(t *testing.T) { values[i] = tsm1.NewValue(t, int64(i)) } - b := values.Encode(nil) + b, err := values.Encode(nil) + if err != nil { + t.Fatalf("unexpected error: %v", err) + } decodedValues := values.DecodeSameTypeBlock(b) @@ -83,7 +92,10 @@ func TestEncoding_IntBlock_Negatives(t *testing.T) { values[i] = tsm1.NewValue(t, int64(v)) } - b := values.Encode(nil) + b, err := values.Encode(nil) + if err != nil { + t.Fatalf("unexpected error: %v", err) + } decodedValues := values.DecodeSameTypeBlock(b) @@ -104,7 +116,10 @@ func TestEncoding_BoolBlock_Basic(t *testing.T) { values[i] = tsm1.NewValue(t, v) } - b := values.Encode(nil) + b, err := values.Encode(nil) + if err != nil { + t.Fatalf("unexpected error: %v", err) + } decodedValues := values.DecodeSameTypeBlock(b) @@ -121,7 +136,10 @@ func TestEncoding_StringBlock_Basic(t *testing.T) { values[i] = tsm1.NewValue(t, fmt.Sprintf("value %d", i)) } - b := values.Encode(nil) + b, err := values.Encode(nil) + if err != nil { + t.Fatalf("unexpected error: %v", err) + } decodedValues := values.DecodeSameTypeBlock(b) diff --git a/tsdb/engine/tsm1/float.go b/tsdb/engine/tsm1/float.go index 7ac1d93b492..22703258a36 100644 --- a/tsdb/engine/tsm1/float.go +++ b/tsdb/engine/tsm1/float.go @@ -205,6 +205,6 @@ func (it *FloatDecoder) Values() float64 { return it.val } -func (it *FloatDecoder) Err() error { +func (it *FloatDecoder) Error() error { return it.err } diff --git a/tsdb/engine/tsm1/float_test.go b/tsdb/engine/tsm1/float_test.go index 49e811a174e..794d62e5b7b 100644 --- a/tsdb/engine/tsm1/float_test.go +++ b/tsdb/engine/tsm1/float_test.go @@ -62,8 +62,8 @@ func TestFloatEncoder_Simple(t *testing.T) { t.Fatalf("Next()=true, want false") } - if err := it.Err(); err != nil { - t.Errorf("it.Err()=%v, want nil", err) + if err := it.Error(); err != nil { + t.Errorf("it.Error()=%v, want nil", err) } } @@ -127,8 +127,8 @@ func TestFloatEncoder_Roundtrip(t *testing.T) { t.Fatalf("Next()=true, want false") } - if err := it.Err(); err != nil { - t.Errorf("it.Err()=%v, want nil", err) + if err := it.Error(); err != nil { + t.Errorf("it.Error()=%v, want nil", err) } } diff --git a/tsdb/engine/tsm1/int.go b/tsdb/engine/tsm1/int.go index 40e615657dd..0060e279947 100644 --- a/tsdb/engine/tsm1/int.go +++ b/tsdb/engine/tsm1/int.go @@ -42,6 +42,7 @@ type Int64Encoder interface { type Int64Decoder interface { Next() bool Read() int64 + Error() error } type int64Encoder struct { @@ -101,6 +102,7 @@ type int64Decoder struct { n int encoding byte + err error } func NewInt64Decoder(b []byte) Int64Decoder { @@ -135,12 +137,16 @@ func (d *int64Decoder) Next() bool { case intCompressedSimple: d.decodePacked() default: - panic(fmt.Sprintf("unknown encoding %v", d.encoding)) + d.err = fmt.Errorf("unknown encoding %v", d.encoding) } } return d.i < d.n } +func (d *int64Decoder) Error() error { + return d.err +} + func (d *int64Decoder) Read() int64 { return ZigZagDecode(d.values[d.i]) } @@ -151,7 +157,12 @@ func (d *int64Decoder) decodePacked() { } v := binary.BigEndian.Uint64(d.bytes[0:8]) - n, _ := simple8b.Decode(d.values, v) + n, err := simple8b.Decode(d.values, v) + if err != nil { + // Should never happen, only error that could be returned is if the the value to be decoded was not + // actually encoded by simple8b encoder. + d.err = fmt.Errorf("failed to decode value %v: %v", v, err) + } d.n = n d.i = 0 diff --git a/tsdb/engine/tsm1/string.go b/tsdb/engine/tsm1/string.go index a2025809de9..da06bc53599 100644 --- a/tsdb/engine/tsm1/string.go +++ b/tsdb/engine/tsm1/string.go @@ -27,6 +27,7 @@ type StringEncoder interface { type StringDecoder interface { Next() bool Read() string + Error() error } type stringEncoder struct { @@ -56,21 +57,21 @@ func (e *stringEncoder) Bytes() ([]byte, error) { } type stringDecoder struct { - b []byte - l int - i int + b []byte + l int + i int + err error } -func NewStringDecoder(b []byte) StringDecoder { +func NewStringDecoder(b []byte) (StringDecoder, error) { // First byte stores the encoding type, only have snappy format // currently so ignore for now. data, err := snappy.Decode(nil, b[1:]) if err != nil { - // TODO: Need to propogate errors up the call stack better - panic(fmt.Sprintf("failed to decode string block: %v", err.Error())) + return nil, fmt.Errorf("failed to decode string block: %v", err.Error()) } - return &stringDecoder{b: data} + return &stringDecoder{b: data}, nil } func (e *stringDecoder) Next() bool { @@ -87,3 +88,7 @@ func (e *stringDecoder) Read() string { return string(e.b[e.i+n : e.i+n+int(length)]) } + +func (e *stringDecoder) Error() error { + return e.err +} diff --git a/tsdb/engine/tsm1/string_test.go b/tsdb/engine/tsm1/string_test.go index f1bf1738152..f5143514ecc 100644 --- a/tsdb/engine/tsm1/string_test.go +++ b/tsdb/engine/tsm1/string_test.go @@ -12,7 +12,10 @@ func Test_StringEncoder_NoValues(t *testing.T) { t.Fatalf("unexpected error: %v", err) } - dec := NewStringDecoder(b) + dec, err := NewStringDecoder(b) + if err != nil { + t.Fatalf("unexpected erorr creating string decoder: %v", err) + } if dec.Next() { t.Fatalf("unexpected next value: got true, exp false") } @@ -27,7 +30,10 @@ func Test_StringEncoder_Single(t *testing.T) { t.Fatalf("unexpected error: %v", err) } - dec := NewStringDecoder(b) + dec, err := NewStringDecoder(b) + if err != nil { + t.Fatalf("unexpected erorr creating string decoder: %v", err) + } if !dec.Next() { t.Fatalf("unexpected next value: got false, exp true") } @@ -59,7 +65,10 @@ func Test_StringEncoder_Multi_Compressed(t *testing.T) { t.Fatalf("unexpected length: got %v, exp %v", len(b), exp) } - dec := NewStringDecoder(b) + dec, err := NewStringDecoder(b) + if err != nil { + t.Fatalf("unexpected erorr creating string decoder: %v", err) + } for i, v := range values { if !dec.Next() { diff --git a/tsdb/engine/tsm1/timestamp.go b/tsdb/engine/tsm1/timestamp.go index 8fcb0b4ce2b..8605988b292 100644 --- a/tsdb/engine/tsm1/timestamp.go +++ b/tsdb/engine/tsm1/timestamp.go @@ -60,6 +60,7 @@ type TimeEncoder interface { type TimeDecoder interface { Next() bool Read() time.Time + Error() error } type encoder struct { @@ -191,8 +192,9 @@ func (e *encoder) encodeRLE(first, delta, div uint64, n int) ([]byte, error) { } type decoder struct { - v time.Time - ts []uint64 + v time.Time + ts []uint64 + err error } func NewTimeDecoder(b []byte) TimeDecoder { @@ -214,6 +216,10 @@ func (d *decoder) Read() time.Time { return d.v } +func (d *decoder) Error() error { + return d.err +} + func (d *decoder) decode(b []byte) { if len(b) == 0 { return @@ -229,7 +235,7 @@ func (d *decoder) decode(b []byte) { case timeCompressedPackedSimple: d.decodePacked(b) default: - panic(fmt.Sprintf("unknown encoding: %v", encoding)) + d.err = fmt.Errorf("unknown encoding: %v", encoding) } } diff --git a/tsdb/engine/tsm1/tsm1.go b/tsdb/engine/tsm1/tsm1.go index b47413868d8..a2ca218f490 100644 --- a/tsdb/engine/tsm1/tsm1.go +++ b/tsdb/engine/tsm1/tsm1.go @@ -599,7 +599,11 @@ func (e *Engine) Compact(fullCompaction bool) error { // write the previous values and clear if we've hit the limit if len(previousValues) > e.MaxPointsPerBlock { - b := previousValues.Encode(buf) + b, err := previousValues.Encode(buf) + if err != nil { + panic(fmt.Sprintf("failure encoding block: %v", err)) + } + if err := e.writeBlock(f, id, b); err != nil { // fail hard. If we can't write a file someone needs to get woken up panic(fmt.Sprintf("failure writing block: %s", err.Error())) @@ -616,7 +620,10 @@ func (e *Engine) Compact(fullCompaction bool) error { if nextID != id { // flush remaining values if len(previousValues) > 0 { - b := previousValues.Encode(buf) + b, err := previousValues.Encode(buf) + if err != nil { + panic(fmt.Sprintf("failure encoding block: %v", err)) + } currentPosition += uint32(blockHeaderSize + len(b)) previousValues = nil if err := e.writeBlock(f, id, b); err != nil { @@ -634,7 +641,11 @@ func (e *Engine) Compact(fullCompaction bool) error { } if len(previousValues) > 0 { - b := previousValues.Encode(buf) + b, err := previousValues.Encode(buf) + if err != nil { + panic(fmt.Sprintf("failure encoding block: %v", err)) + } + if err := e.writeBlock(f, minID, b); err != nil { // fail hard. If we can't write a file someone needs to get woken up panic(fmt.Sprintf("failure writing block: %s", err.Error())) @@ -1041,7 +1052,12 @@ func (e *Engine) rewriteFile(oldDF *dataFile, valuesByID map[uint64]Values) erro fpos, ok := oldIDToPosition[id] if !ok { // TODO: ensure we encode only the amount in a block - block := newVals.Encode(buf) + block, err := newVals.Encode(buf) + if err != nil { + f.Close() + return err + } + if err := e.writeBlock(f, id, block); err != nil { f.Close() return err @@ -1087,7 +1103,12 @@ func (e *Engine) rewriteFile(oldDF *dataFile, valuesByID map[uint64]Values) erro // TODO: ensure we encode only the amount in a block, refactor this wil line 450 into func if len(newVals) > 0 { // TODO: ensure we encode only the amount in a block - block := newVals.Encode(buf) + block, err := newVals.Encode(buf) + if err != nil { + f.Close() + return err + } + if _, err := f.Write(append(u64tob(id), u32tob(uint32(len(block)))...)); err != nil { f.Close() return err @@ -1586,7 +1607,11 @@ func (e *Engine) DecodeAndCombine(newValues Values, block, buf []byte, nextTime values = values[:e.MaxPointsPerBlock] } - return remainingValues, values.Encode(buf), nil + encoded, err := values.Encode(buf) + if err != nil { + return nil, nil, err + } + return remainingValues, encoded, nil } type dataFile struct { From d9f94bdeebc20e130e6079ed46cfec278f8507c5 Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Fri, 2 Oct 2015 17:17:07 -0400 Subject: [PATCH 128/139] Add db crash recovery --- tsdb/engine/tsm1/tsm1.go | 166 +++++++++++++++++++++++++++++++-------- 1 file changed, 133 insertions(+), 33 deletions(-) diff --git a/tsdb/engine/tsm1/tsm1.go b/tsdb/engine/tsm1/tsm1.go index a2ca218f490..bc00eea8d84 100644 --- a/tsdb/engine/tsm1/tsm1.go +++ b/tsdb/engine/tsm1/tsm1.go @@ -26,6 +26,10 @@ const ( // Format is the file format name of this engine. Format = "tsm1" + //IDsFileExtension is the extension for the file that keeps the compressed map + // of keys to uint64 IDs. + IDsFileExtension = "ids" + // FieldsFileExtension is the extension for the file that stores compressed field // encoding data for this db FieldsFileExtension = "fields" @@ -34,7 +38,15 @@ const ( // series metadata for series in this db SeriesFileExtension = "series" + // CollisionsFileExtension is the extension for the file that keeps a map of which + // keys have hash collisions and what their actual IDs are CollisionsFileExtension = "collisions" + + //CheckpointExtension is the extension given to files that checkpoint. + // The checkpoint files are created when a new file is first created. They + // are removed after the file has been synced and is safe for use. If a file + // has an associated checkpoint file, it wasn't safely written and both should be removed + CheckpointExtension = "check" ) type TimePrecision uint8 @@ -189,17 +201,22 @@ func (e *Engine) Open() error { return err } - // TODO: clean up previous series write - // TODO: clean up previous fields write - // TODO: clean up previous names write - // TODO: clean up any data files that didn't get cleaned up - // TODO: clean up previous collisions write + // perform any cleanup on metafiles that were halfway written + e.cleanupMetafile(SeriesFileExtension) + e.cleanupMetafile(FieldsFileExtension) + e.cleanupMetafile(IDsFileExtension) + e.cleanupMetafile(CollisionsFileExtension) files, err := filepath.Glob(filepath.Join(e.path, fmt.Sprintf("*.%s", Format))) if err != nil { return err } for _, fn := range files { + // if the file has a checkpoint it's not valid, so remove it + if removed := e.removeFileIfCheckpointExists(fn); removed { + continue + } + id, err := idFromFileName(fn) if err != nil { return err @@ -533,16 +550,8 @@ func (e *Engine) Compact(fullCompaction bool) error { ids := make([]uint64, len(files)) // initilaize for writing - f, err := os.OpenFile(fileName, os.O_CREATE|os.O_RDWR, 0666) - if err != nil { - return err - } + f, err := e.openFileAndCheckpoint(fileName) - // write the magic number - if _, err := f.Write(u32tob(magicNumber)); err != nil { - f.Close() - return err - } for i, df := range files { ids[i] = btou64(df.mmap[4:12]) positions[i] = 4 @@ -734,6 +743,10 @@ func (e *Engine) writeIndexAndGetDataFile(f *os.File, minTime, maxTime int64, id return nil, err } + if err := e.removeCheckpoint(f.Name()); err != nil { + return nil, err + } + // now open it as a memory mapped data file newDF, err := NewDataFile(f) if err != nil { @@ -788,7 +801,7 @@ func (e *Engine) convertKeysAndWriteMetadata(pointsByKey map[string]Values, meas } // read in keys and assign any that aren't defined - b, err := e.readCompressedFile("ids") + b, err := e.readCompressedFile(IDsFileExtension) if err != nil { return err, 0, 0, nil } @@ -858,7 +871,7 @@ func (e *Engine) convertKeysAndWriteMetadata(pointsByKey map[string]Values, meas if err != nil { return err, 0, 0, nil } - if err := e.replaceCompressedFile("ids", b); err != nil { + if err := e.replaceCompressedFile(IDsFileExtension, b); err != nil { return err, 0, 0, nil } } @@ -992,8 +1005,7 @@ func (e *Engine) rewriteFile(oldDF *dataFile, valuesByID map[uint64]Values) erro // always write in order by ID sort.Sort(uint64slice(ids)) - // TODO: add checkpoint file that indicates if this completed or not - f, err := os.OpenFile(e.nextFileName(), os.O_CREATE|os.O_RDWR, 0666) + f, err := e.openFileAndCheckpoint(e.nextFileName()) if err != nil { return err } @@ -1004,12 +1016,6 @@ func (e *Engine) rewriteFile(oldDF *dataFile, valuesByID map[uint64]Values) erro e.logger.Printf("rewriting index file %s with %s", oldDF.f.Name(), f.Name()) } - // write the magic number - if _, err := f.Write(u32tob(magicNumber)); err != nil { - f.Close() - return err - } - // now combine the old file data with the new values, keeping track of // their positions currentPosition := uint32(fileHeaderSize) @@ -1185,14 +1191,9 @@ func (e *Engine) flushDeletes() error { } func (e *Engine) writeNewFileExcludeDeletes(oldDF *dataFile) *dataFile { - // TODO: add checkpoint file that indicates if this completed or not - f, err := os.OpenFile(e.nextFileName(), os.O_CREATE|os.O_RDWR, 0666) + f, err := e.openFileAndCheckpoint(e.nextFileName()) if err != nil { - panic(fmt.Sprintf("error opening new index file: %s", err.Error())) - } - // write the magic number - if _, err := f.Write(u32tob(magicNumber)); err != nil { - panic(fmt.Sprintf("error writing new index file: %s", err.Error())) + panic(fmt.Sprintf("error opening new data file: %s", err.Error())) } ids := make([]uint64, 0) @@ -1614,6 +1615,99 @@ func (e *Engine) DecodeAndCombine(newValues Values, block, buf []byte, nextTime return remainingValues, encoded, nil } +// removeFileIfCheckpointExists will remove the file if its associated checkpoint fil is there. +// It returns true if the file was removed. This is for recovery of data files on startup +func (e *Engine) removeFileIfCheckpointExists(fileName string) bool { + checkpointName := fmt.Sprintf("%s.%s", fileName, CheckpointExtension) + _, err := os.Stat(checkpointName) + + // if there's no checkpoint, move on + if err != nil { + return false + } + + // there's a checkpoint so we know this file isn't safe so we should remove it + err = os.Remove(fileName) + if err != nil { + panic(fmt.Sprintf("error removing file %s", err.Error())) + } + + err = os.Remove(checkpointName) + if err != nil { + panic(fmt.Sprintf("error removing file %s", err.Error())) + } + + return true +} + +// cleanupMetafile will remove the tmp file if the other file exists, or rename the +// tmp file to be a regular file if the normal file is missing. This is for recovery on +// startup. +func (e *Engine) cleanupMetafile(name string) { + fileName := filepath.Join(e.path, name) + tmpName := fileName + "tmp" + + _, err := os.Stat(tmpName) + + // if the tmp file isn't there, we can just exit + if err != nil { + return + } + + _, err = os.Stat(fileName) + + // the regular file is there so we should just remove the tmp file + if err == nil { + err = os.Remove(tmpName) + if err != nil { + panic(fmt.Sprintf("error removing meta file %s: %s", tmpName, err.Error())) + } + } + + // regular file isn't there so have the tmp file take its place + err = os.Rename(tmpName, fileName) + if err != nil { + panic(fmt.Sprintf("error renaming meta file %s: %s", tmpName, err.Error())) + } +} + +// openFileAndCehckpoint will create a checkpoint file, open a new file for +// writing a data index, write the header and return the file +func (e *Engine) openFileAndCheckpoint(fileName string) (*os.File, error) { + checkpointFile := fmt.Sprintf("%s.%s", fileName, CheckpointExtension) + cf, err := os.OpenFile(checkpointFile, os.O_CREATE, 0666) + if err != nil { + return nil, err + } + // _, err = cf.Write(u32tob(magicNumber)) + // if err != nil { + // panic(err) + // } + if err := cf.Close(); err != nil { + return nil, err + } + _, err = os.Stat(checkpointFile) + + f, err := os.OpenFile(fileName, os.O_CREATE|os.O_RDWR, 0666) + if err != nil { + return nil, err + } + + // write the header, which is just the magic number + if _, err := f.Write(u32tob(magicNumber)); err != nil { + f.Close() + return nil, err + } + + return f, nil +} + +// removeCheckpoint removes the checkpoint for a new data file that was getting written +func (e *Engine) removeCheckpoint(fileName string) error { + checkpointFile := fmt.Sprintf("%s.%s", fileName, CheckpointExtension) + return os.Remove(checkpointFile) +} + type dataFile struct { f *os.File mu sync.RWMutex @@ -1631,6 +1725,8 @@ const ( seriesIDSize = 8 seriesPositionSize = 4 seriesHeaderSize = seriesIDSize + seriesPositionSize + minTimeOffset = 20 + maxTimeOffset = 12 ) func NewDataFile(f *os.File) (*dataFile, error) { @@ -1685,11 +1781,15 @@ func (d *dataFile) close() error { } func (d *dataFile) MinTime() int64 { - return int64(btou64(d.mmap[d.size-20 : d.size-12])) + minTimePosition := d.size - minTimeOffset + timeBytes := d.mmap[minTimePosition : minTimePosition+timeSize] + return int64(btou64(timeBytes)) } func (d *dataFile) MaxTime() int64 { - return int64(btou64(d.mmap[d.size-12 : d.size-4])) + maxTimePosition := d.size - maxTimeOffset + timeBytes := d.mmap[maxTimePosition : maxTimePosition+timeSize] + return int64(btou64(timeBytes)) } func (d *dataFile) SeriesCount() uint32 { From 36898f94518ed2d2fe9cbe8b7c580814e44f27e4 Mon Sep 17 00:00:00 2001 From: Paris Holley Date: Sun, 4 Oct 2015 11:40:14 -0700 Subject: [PATCH 129/139] do not include empty tags in hash --- models/points.go | 9 ++++++++- models/points_test.go | 12 ++++++++++++ 2 files changed, 20 insertions(+), 1 deletion(-) diff --git a/models/points.go b/models/points.go index 1ded7234d5b..d91ff13afbe 100644 --- a/models/points.go +++ b/models/points.go @@ -1021,6 +1021,10 @@ func (p *point) Tags() Tags { i, key = scanTo(p.key, i, '=') i, value = scanTagValue(p.key, i+1) + if len(value) == 0 { + continue + } + tags[string(unescapeTag(key))] = string(unescapeTag(value)) i += 1 @@ -1141,7 +1145,10 @@ func (t Tags) HashKey() []byte { for k, v := range t { ek := escapeTag([]byte(k)) ev := escapeTag([]byte(v)) - escaped[string(ek)] = string(ev) + + if len(string(ev)) > 0 { + escaped[string(ek)] = string(ev) + } } // Extract keys and determine final size. diff --git a/models/points_test.go b/models/points_test.go index 1d4d8bf866a..4186b89d970 100644 --- a/models/points_test.go +++ b/models/points_test.go @@ -605,6 +605,18 @@ func TestParsePointUnescape(t *testing.T) { }, time.Unix(0, 0))) + // tag with no value + test(t, `cpu,regions=east value="1"`, + models.NewPoint("cpu", + models.Tags{ + "regions": "east", + "foobar": "", + }, + models.Fields{ + "value": "1", + }, + time.Unix(0, 0))) + // commas in field values test(t, `cpu,regions=east value="1,0"`, models.NewPoint("cpu", From b0449702e586cc01632f47c8c791af2e9c6a0dc6 Mon Sep 17 00:00:00 2001 From: Jason Wilder Date: Mon, 5 Oct 2015 12:32:29 -0600 Subject: [PATCH 130/139] Fix comment typos --- tsdb/engine/tsm1/bool.go | 1 - tsdb/engine/tsm1/encoding.go | 10 +++++----- tsdb/engine/tsm1/float.go | 4 ++-- tsdb/engine/tsm1/int.go | 8 +++++--- tsdb/engine/tsm1/timestamp.go | 16 ++++++++-------- 5 files changed, 20 insertions(+), 19 deletions(-) diff --git a/tsdb/engine/tsm1/bool.go b/tsdb/engine/tsm1/bool.go index 52ec8d849b0..83f570a2ef2 100644 --- a/tsdb/engine/tsm1/bool.go +++ b/tsdb/engine/tsm1/bool.go @@ -114,7 +114,6 @@ func (e *boolDecoder) Next() bool { } func (e *boolDecoder) Read() bool { - // Index into the byte slice idx := e.i / 8 diff --git a/tsdb/engine/tsm1/encoding.go b/tsdb/engine/tsm1/encoding.go index 0d95bf5ac2b..3fd6d2b4bb5 100644 --- a/tsdb/engine/tsm1/encoding.go +++ b/tsdb/engine/tsm1/encoding.go @@ -214,7 +214,7 @@ func encodeFloatBlock(buf []byte, values []*FloatValue) ([]byte, error) { // Encoded float values vb := venc.Bytes() - // Preprend the first timestamp of the block in the first 8 bytes and the block + // Prepend the first timestamp of the block in the first 8 bytes and the block // in the next byte, followed by the block block := packBlockHeader(values[0].Time(), BlockFloat64) block = append(block, packBlock(tb, vb)...) @@ -312,7 +312,7 @@ func encodeBoolBlock(buf []byte, values []*BoolValue) ([]byte, error) { return nil, err } - // Preprend the first timestamp of the block in the first 8 bytes and the block + // Prepend the first timestamp of the block in the first 8 bytes and the block // in the next byte, followed by the block block := packBlockHeader(values[0].Time(), BlockBool) block = append(block, packBlock(tb, vb)...) @@ -398,7 +398,7 @@ func encodeInt64Block(buf []byte, values []*Int64Value) ([]byte, error) { return nil, err } - // Preprend the first timestamp of the block in the first 8 bytes + // Prepend the first timestamp of the block in the first 8 bytes block := packBlockHeader(values[0].Time(), BlockInt64) return append(block, packBlock(tb, vb)...), nil } @@ -477,13 +477,13 @@ func encodeStringBlock(buf []byte, values []*StringValue) ([]byte, error) { if err != nil { return nil, err } - // Encoded int64 values + // Encoded string values vb, err := vEnc.Bytes() if err != nil { return nil, err } - // Preprend the first timestamp of the block in the first 8 bytes + // Prepend the first timestamp of the block in the first 8 bytes block := packBlockHeader(values[0].Time(), BlockString) return append(block, packBlock(tb, vb)...), nil } diff --git a/tsdb/engine/tsm1/float.go b/tsdb/engine/tsm1/float.go index 22703258a36..8961c70f4ff 100644 --- a/tsdb/engine/tsm1/float.go +++ b/tsdb/engine/tsm1/float.go @@ -24,6 +24,7 @@ const ( floatCompressedGorilla = 1 ) +// FloatEncoder encodes multiple float64s into a byte slice type FloatEncoder struct { val float64 @@ -54,7 +55,6 @@ func (s *FloatEncoder) Bytes() []byte { } func (s *FloatEncoder) Finish() { - if !s.finished { // // write an end-of-stream record s.Push(math.NaN()) @@ -64,7 +64,6 @@ func (s *FloatEncoder) Finish() { } func (s *FloatEncoder) Push(v float64) { - if s.first { // first point s.val = v @@ -102,6 +101,7 @@ func (s *FloatEncoder) Push(v float64) { s.val = v } +// FloatDecoder decodes a byte slice into multipe float64 values type FloatDecoder struct { val float64 diff --git a/tsdb/engine/tsm1/int.go b/tsdb/engine/tsm1/int.go index 0060e279947..486f91a98d7 100644 --- a/tsdb/engine/tsm1/int.go +++ b/tsdb/engine/tsm1/int.go @@ -15,10 +15,10 @@ package tsm1 // or 8 byte uncompressed integers. The 4 high bits of the first byte indicate the encoding type // for the remaining bytes. // -// There are currently two encoding types that can be used with room for 15 more. These additional -// encoding slots are reserved for future use. One improvement to to be made is to use a patched +// There are currently two encoding types that can be used with room for 16 total. These additional +// encoding slots are reserved for future use. One improvement to be made is to use a patched // encoding such as PFOR if only a small number of values exceed the max compressed value range. This -// should improve compression ratios with very integers near the ends of the int64 range. +// should improve compression ratios with very large integers near the ends of the int64 range. import ( "encoding/binary" @@ -34,11 +34,13 @@ const ( intCompressedSimple = 1 ) +// Int64Encoder encoders int64 into byte slices type Int64Encoder interface { Write(v int64) Bytes() ([]byte, error) } +// Int64Decoder decodes a byte slice into int64s type Int64Decoder interface { Next() bool Read() int64 diff --git a/tsdb/engine/tsm1/timestamp.go b/tsdb/engine/tsm1/timestamp.go index 8605988b292..ad7ed644196 100644 --- a/tsdb/engine/tsm1/timestamp.go +++ b/tsdb/engine/tsm1/timestamp.go @@ -1,6 +1,6 @@ package tsm1 -// Timestamp encoding is adapative and based on structure of the timestamps that are encoded. It +// Timestamp encoding is adaptive and based on structure of the timestamps that are encoded. It // uses a combination of delta encoding, scaling and compression using simple8b, run length encoding // as well as falling back to no compression if needed. // @@ -20,9 +20,9 @@ package tsm1 // any value exceeds the maximum values, the deltas are stored uncompressed using 8b each. // // Each compressed byte slice has a 1 byte header indicating the compression type. The 4 high bits -// indicated the encoding type. The 4 low bits are using by the encoding type. +// indicated the encoding type. The 4 low bits are used by the encoding type. // -// For run length encoding, the 4 low bits store the log10 of the scaling factor. The next 8 bytes are +// For run-length encoding, the 4 low bits store the log10 of the scaling factor. The next 8 bytes are // the starting timestamp, next 1-10 bytes is the delta value using variable-length encoding, finally the // next 1-10 bytes is the count of values. // @@ -86,13 +86,13 @@ func (e *encoder) reduce() (max, divisor uint64, rle bool, deltas []uint64) { // Indicates whether the the deltas can be run-length encoded rle = true - // Interate in reverse so we can apply deltas in place + // Iterate in reverse so we can apply deltas in place for i := len(deltas) - 1; i > 0; i-- { // First differential encode the values deltas[i] = deltas[i] - deltas[i-1] - // We're also need to keep track of the max value and largest common divisor + // We also need to keep track of the max value and largest common divisor v := deltas[i] if v > max { @@ -121,11 +121,11 @@ func (e *encoder) Bytes() ([]byte, error) { // Maximum and largest common divisor. rle is true if dts (the delta timestamps), // are all the same. - max, mod, rle, dts := e.reduce() + max, div, rle, dts := e.reduce() // The deltas are all the same, so we can run-length encode them if rle && len(e.ts) > 60 { - return e.encodeRLE(e.ts[0], e.ts[1], mod, len(e.ts)) + return e.encodeRLE(e.ts[0], e.ts[1], div, len(e.ts)) } // We can't compress this time-range, the deltas exceed 1 << 60 @@ -133,7 +133,7 @@ func (e *encoder) Bytes() ([]byte, error) { return e.encodeRaw() } - return e.encodePacked(mod, dts) + return e.encodePacked(div, dts) } func (e *encoder) encodePacked(div uint64, dts []uint64) ([]byte, error) { From cb28dabf623fd3fd41bcc4dd8aa765f17bf2b74f Mon Sep 17 00:00:00 2001 From: Jason Wilder Date: Mon, 5 Oct 2015 12:43:08 -0600 Subject: [PATCH 131/139] Make DecodeBlock panic if block size is too small Should never get a block size 9 bytes since Encode always returns the min timestampe and a 1 byte header. If we get this, the engine is confused. --- tsdb/engine/tsm1/encoding.go | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/tsdb/engine/tsm1/encoding.go b/tsdb/engine/tsm1/encoding.go index 3fd6d2b4bb5..040a12de666 100644 --- a/tsdb/engine/tsm1/encoding.go +++ b/tsdb/engine/tsm1/encoding.go @@ -21,6 +21,10 @@ const ( // BlockString designates a block encodes string values BlockString = 3 + + // encodedBlockHeaderSize is the size of the header for an encoded block. The first 8 bytes + // are the minimum timestamp of the block. The next byte is a block encoding type indicator. + encodedBlockHeaderSize = 9 ) type Value interface { @@ -120,8 +124,8 @@ func (v Values) DecodeSameTypeBlock(block []byte) Values { // DecodeBlock takes a byte array and will decode into values of the appropriate type // based on the block func DecodeBlock(block []byte) (Values, error) { - if len(block) == 0 { - return Values{}, nil + if len(block) <= encodedBlockHeaderSize { + panic(fmt.Sprintf("decode of short block: got %v, exp %v", len(block), encodedBlockHeaderSize)) } blockType := block[8] @@ -135,10 +139,8 @@ func DecodeBlock(block []byte) (Values, error) { case BlockString: return decodeStringBlock(block) default: + panic(fmt.Sprintf("unknown block type: %d", blockType)) } - - // TODO: add support for other block types - return nil, fmt.Errorf("unknown block type: %d", blockType) } // Deduplicate returns a new Values slice with any values From c6f2f9cec29a102288e37241139029407967c568 Mon Sep 17 00:00:00 2001 From: Jason Wilder Date: Mon, 5 Oct 2015 12:52:19 -0600 Subject: [PATCH 132/139] Avoid duplicating values slice when encoding --- tsdb/engine/tsm1/encoding.go | 43 ++++++++++-------------------------- tsdb/engine/tsm1/int.go | 1 + 2 files changed, 13 insertions(+), 31 deletions(-) diff --git a/tsdb/engine/tsm1/encoding.go b/tsdb/engine/tsm1/encoding.go index 040a12de666..3de88586320 100644 --- a/tsdb/engine/tsm1/encoding.go +++ b/tsdb/engine/tsm1/encoding.go @@ -72,32 +72,13 @@ func (v Values) MaxTime() int64 { func (v Values) Encode(buf []byte) ([]byte, error) { switch v[0].(type) { case *FloatValue: - a := make([]*FloatValue, len(v)) - for i, vv := range v { - a[i] = vv.(*FloatValue) - } - return encodeFloatBlock(buf, a) - + return encodeFloatBlock(buf, v) case *Int64Value: - a := make([]*Int64Value, len(v)) - for i, vv := range v { - a[i] = vv.(*Int64Value) - } - return encodeInt64Block(buf, a) - + return encodeInt64Block(buf, v) case *BoolValue: - a := make([]*BoolValue, len(v)) - for i, vv := range v { - a[i] = vv.(*BoolValue) - } - return encodeBoolBlock(buf, a) - + return encodeBoolBlock(buf, v) case *StringValue: - a := make([]*StringValue, len(v)) - for i, vv := range v { - a[i] = vv.(*StringValue) - } - return encodeStringBlock(buf, a) + return encodeStringBlock(buf, v) } return nil, fmt.Errorf("unsupported value type %T", v[0]) @@ -187,7 +168,7 @@ func (f *FloatValue) Size() int { return 16 } -func encodeFloatBlock(buf []byte, values []*FloatValue) ([]byte, error) { +func encodeFloatBlock(buf []byte, values []Value) ([]byte, error) { if len(values) == 0 { return nil, nil } @@ -204,7 +185,7 @@ func encodeFloatBlock(buf []byte, values []*FloatValue) ([]byte, error) { for _, v := range values { tsenc.Write(v.Time()) - venc.Push(v.value) + venc.Push(v.(*FloatValue).value) } venc.Finish() @@ -284,7 +265,7 @@ func (b *BoolValue) Value() interface{} { return b.value } -func encodeBoolBlock(buf []byte, values []*BoolValue) ([]byte, error) { +func encodeBoolBlock(buf []byte, values []Value) ([]byte, error) { if len(values) == 0 { return nil, nil } @@ -300,7 +281,7 @@ func encodeBoolBlock(buf []byte, values []*BoolValue) ([]byte, error) { for _, v := range values { tsenc.Write(v.Time()) - venc.Write(v.value) + venc.Write(v.(*BoolValue).value) } // Encoded timestamp values @@ -381,12 +362,12 @@ func (v *Int64Value) Size() int { func (v *Int64Value) String() string { return fmt.Sprintf("%v", v.value) } -func encodeInt64Block(buf []byte, values []*Int64Value) ([]byte, error) { +func encodeInt64Block(buf []byte, values []Value) ([]byte, error) { tsEnc := NewTimeEncoder() vEnc := NewInt64Encoder() for _, v := range values { tsEnc.Write(v.Time()) - vEnc.Write(v.value) + vEnc.Write(v.(*Int64Value).value) } // Encoded timestamp values @@ -466,12 +447,12 @@ func (v *StringValue) Size() int { func (v *StringValue) String() string { return v.value } -func encodeStringBlock(buf []byte, values []*StringValue) ([]byte, error) { +func encodeStringBlock(buf []byte, values []Value) ([]byte, error) { tsEnc := NewTimeEncoder() vEnc := NewStringEncoder() for _, v := range values { tsEnc.Write(v.Time()) - vEnc.Write(v.value) + vEnc.Write(v.(*StringValue).value) } // Encoded timestamp values diff --git a/tsdb/engine/tsm1/int.go b/tsdb/engine/tsm1/int.go index 486f91a98d7..9ce18fe96e3 100644 --- a/tsdb/engine/tsm1/int.go +++ b/tsdb/engine/tsm1/int.go @@ -109,6 +109,7 @@ type int64Decoder struct { func NewInt64Decoder(b []byte) Int64Decoder { d := &int64Decoder{ + // 240 is the maximum number of values that can be encoded into a single uint64 using simple8b values: make([]uint64, 240), } From bb398daf759d1259cb024f3b33661f2e951e60d1 Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Sun, 4 Oct 2015 15:43:18 -0400 Subject: [PATCH 133/139] Updates based on @otoolp's PR comments --- cmd/influxd/run/server.go | 2 -- services/httpd/handler.go | 2 +- tsdb/config.go | 40 +++++++++++++++++++-------------------- tsdb/engine/tsm1/tsm1.go | 38 ++++++++++++++++++------------------- 4 files changed, 40 insertions(+), 42 deletions(-) diff --git a/cmd/influxd/run/server.go b/cmd/influxd/run/server.go index 3a92dc6acc6..0d23c0f1aa8 100644 --- a/cmd/influxd/run/server.go +++ b/cmd/influxd/run/server.go @@ -84,8 +84,6 @@ func NewServer(c *Config, buildInfo *BuildInfo) (*Server, error) { tsdbStore := tsdb.NewStore(c.Data.Dir) tsdbStore.EngineOptions.Config = c.Data - runtime.GOMAXPROCS(runtime.NumCPU()) - s := &Server{ buildInfo: *buildInfo, err: make(chan error), diff --git a/services/httpd/handler.go b/services/httpd/handler.go index 2fb96e99df7..b3c136e794d 100644 --- a/services/httpd/handler.go +++ b/services/httpd/handler.go @@ -83,7 +83,7 @@ func NewHandler(requireAuthentication, loggingEnabled, writeTrace bool, statMap mux: pat.New(), requireAuthentication: requireAuthentication, Logger: log.New(os.Stderr, "[http] ", log.LstdFlags), - loggingEnabled: false, + loggingEnabled: loggingEnabled, WriteTrace: writeTrace, statMap: statMap, } diff --git a/tsdb/config.go b/tsdb/config.go index a329c1268bb..115f895f139 100644 --- a/tsdb/config.go +++ b/tsdb/config.go @@ -45,12 +45,12 @@ const ( DefaultPartitionSizeThreshold = 50 * 1024 * 1024 // 50MB // Default WAL settings for the TSM1 WAL - DefaultFlushMemorySizeThreshold = 5 * 1024 * 1024 // 5MB - DefaultMaxMemorySizeThreshold = 100 * 1024 * 1024 // 100MB - DefaultIndexCompactionAge = time.Minute - DefaultIndexMinimumCompactionInterval = time.Minute - DefaultIndexCompactionFileCount = 5 - DefaultIndexCompactionFullAge = time.Minute + DefaultFlushMemorySizeThreshold = 5 * 1024 * 1024 // 5MB + DefaultMaxMemorySizeThreshold = 100 * 1024 * 1024 // 100MB + DefaultIndexCompactionAge = time.Minute + DefaultIndexMinCompactionInterval = time.Minute + DefaultIndexMinCompactionFileCount = 5 + DefaultIndexCompactionFullAge = 5 * time.Minute ) type Config struct { @@ -83,11 +83,11 @@ type Config struct { // IndexMinimumCompactionInterval specifies the minimum amount of time that must // pass after a compaction before another compaction is run - IndexMinimumCompactionInterval time.Duration `toml:"index-minimum-compaction-interval"` + IndexMinCompactionInterval time.Duration `toml:"index-min-compaction-interval"` // IndexCompactionFileCount specifies the minimum number of data files that // must be eligible for compaction before actually running one - IndexCompactionFileCount int `toml:"index-compaction-file-count"` + IndexMinCompactionFileCount int `toml:"index-compaction-min-file-count"` // IndexCompactionFullAge specifies how long after the last write was received // in the WAL that a full compaction should be performed. @@ -104,18 +104,18 @@ func NewConfig() Config { WALFlushInterval: toml.Duration(DefaultWALFlushInterval), WALPartitionFlushDelay: toml.Duration(DefaultWALPartitionFlushDelay), - WALLoggingEnabled: true, - WALReadySeriesSize: DefaultReadySeriesSize, - WALCompactionThreshold: DefaultCompactionThreshold, - WALMaxSeriesSize: DefaultMaxSeriesSize, - WALFlushColdInterval: toml.Duration(DefaultFlushColdInterval), - WALPartitionSizeThreshold: DefaultPartitionSizeThreshold, - WALFlushMemorySizeThreshold: DefaultFlushMemorySizeThreshold, - WALMaxMemorySizeThreshold: DefaultMaxMemorySizeThreshold, - IndexCompactionAge: DefaultIndexCompactionAge, - IndexCompactionFileCount: DefaultIndexCompactionFileCount, - IndexCompactionFullAge: DefaultIndexCompactionFullAge, - IndexMinimumCompactionInterval: DefaultIndexMinimumCompactionInterval, + WALLoggingEnabled: true, + WALReadySeriesSize: DefaultReadySeriesSize, + WALCompactionThreshold: DefaultCompactionThreshold, + WALMaxSeriesSize: DefaultMaxSeriesSize, + WALFlushColdInterval: toml.Duration(DefaultFlushColdInterval), + WALPartitionSizeThreshold: DefaultPartitionSizeThreshold, + WALFlushMemorySizeThreshold: DefaultFlushMemorySizeThreshold, + WALMaxMemorySizeThreshold: DefaultMaxMemorySizeThreshold, + IndexCompactionAge: DefaultIndexCompactionAge, + IndexMinCompactionFileCount: DefaultIndexMinCompactionFileCount, + IndexCompactionFullAge: DefaultIndexCompactionFullAge, + IndexMinCompactionInterval: DefaultIndexMinCompactionInterval, QueryLogEnabled: true, } diff --git a/tsdb/engine/tsm1/tsm1.go b/tsdb/engine/tsm1/tsm1.go index bc00eea8d84..b75d894f653 100644 --- a/tsdb/engine/tsm1/tsm1.go +++ b/tsdb/engine/tsm1/tsm1.go @@ -98,14 +98,14 @@ type Engine struct { WAL *Log - RotateFileSize uint32 - SkipCompaction bool - CompactionAge time.Duration - CompactionFileCount int - IndexCompactionFullAge time.Duration - IndexMinimumCompactionInterval time.Duration - MaxPointsPerBlock int - RotateBlockSize int + RotateFileSize uint32 + SkipCompaction bool + CompactionAge time.Duration + MinCompactionFileCount int + IndexCompactionFullAge time.Duration + IndexMinCompactionInterval time.Duration + MaxPointsPerBlock int + RotateBlockSize int // filesLock is only for modifying and accessing the files slice filesLock sync.RWMutex @@ -140,15 +140,15 @@ func NewEngine(path string, walPath string, opt tsdb.EngineOptions) tsdb.Engine logger: log.New(os.Stderr, "[tsm1] ", log.LstdFlags), // TODO: this is the function where we can inject a check against the in memory collisions - HashSeriesField: hashSeriesField, - WAL: w, - RotateFileSize: DefaultRotateFileSize, - CompactionAge: opt.Config.IndexCompactionAge, - CompactionFileCount: opt.Config.IndexCompactionFileCount, - IndexCompactionFullAge: opt.Config.IndexCompactionFullAge, - IndexMinimumCompactionInterval: opt.Config.IndexMinimumCompactionInterval, - MaxPointsPerBlock: DefaultMaxPointsPerBlock, - RotateBlockSize: DefaultRotateBlockSize, + HashSeriesField: hashSeriesField, + WAL: w, + RotateFileSize: DefaultRotateFileSize, + CompactionAge: opt.Config.IndexCompactionAge, + MinCompactionFileCount: opt.Config.IndexMinCompactionFileCount, + IndexCompactionFullAge: opt.Config.IndexCompactionFullAge, + IndexMinCompactionInterval: opt.Config.IndexMinCompactionInterval, + MaxPointsPerBlock: DefaultMaxPointsPerBlock, + RotateBlockSize: DefaultRotateBlockSize, } e.WAL.Index = e @@ -762,10 +762,10 @@ func (e *Engine) shouldCompact() bool { since := time.Since(e.lastCompactionTime) deletesPending := len(e.deletes) > 0 e.filesLock.RUnlock() - if running || since < e.IndexMinimumCompactionInterval || deletesPending { + if running || since < e.IndexMinCompactionInterval || deletesPending { return false } - return len(e.filesToCompact()) >= e.CompactionFileCount + return len(e.filesToCompact()) >= e.MinCompactionFileCount } func (e *Engine) filesToCompact() dataFiles { From 26a93ec23e550f12b5199c550ca607acdee0b9e4 Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Mon, 5 Oct 2015 17:21:07 -0400 Subject: [PATCH 134/139] Fix deletes not kept if shutdown before flush on tsm1 --- tsdb/engine/tsm1/tsm1.go | 136 ++++++++++++++++++++++++++--------- tsdb/engine/tsm1/tx.go | 4 +- tsdb/engine/tsm1/wal.go | 78 ++++++++++++++++---- tsdb/engine/tsm1/wal_test.go | 6 +- 4 files changed, 170 insertions(+), 54 deletions(-) diff --git a/tsdb/engine/tsm1/tsm1.go b/tsdb/engine/tsm1/tsm1.go index b75d894f653..aac359c19d1 100644 --- a/tsdb/engine/tsm1/tsm1.go +++ b/tsdb/engine/tsm1/tsm1.go @@ -13,6 +13,7 @@ import ( "path/filepath" "reflect" "sort" + "strings" "sync" "syscall" "time" @@ -47,6 +48,10 @@ const ( // are removed after the file has been synced and is safe for use. If a file // has an associated checkpoint file, it wasn't safely written and both should be removed CheckpointExtension = "check" + + // keyFieldSeparator separates the series key from the field name in the composite key + // that identifies a specific field in series + keyFieldSeparator = "#!~#" ) type TimePrecision uint8 @@ -115,8 +120,12 @@ type Engine struct { lastCompactionTime time.Time // deletes is a map of keys that are deleted, but haven't yet been - // compacted and flushed - deletes map[uint64]bool + // compacted and flushed. They map the ID to the corresponding key + deletes map[uint64]string + + // deleteMeasurements is a map of the measurements that are deleted + // but haven't yet been compacted and flushed + deleteMeasurements map[string]bool collisionsLock sync.RWMutex collisions map[string]uint64 @@ -240,7 +249,8 @@ func (e *Engine) Open() error { return err } - e.deletes = make(map[uint64]bool) + e.deletes = make(map[uint64]string) + e.deleteMeasurements = make(map[string]bool) // mark the last compaction as now so it doesn't try to compact while // flushing the WAL on load @@ -278,6 +288,7 @@ func (e *Engine) Close() error { e.currentFileID = 0 e.collisions = nil e.deletes = nil + e.deleteMeasurements = nil return nil } @@ -440,10 +451,16 @@ func (e *Engine) MarkDeletes(keys []string) { e.filesLock.Lock() defer e.filesLock.Unlock() for _, k := range keys { - e.deletes[e.keyToID(k)] = true + e.deletes[e.keyToID(k)] = k } } +func (e *Engine) MarkMeasurementDelete(name string) { + e.filesLock.Lock() + defer e.filesLock.Unlock() + e.deleteMeasurements[name] = true +} + // filesAndLock returns the data files that match the given range and // ensures that the write lock will hold for the entire range func (e *Engine) filesAndLock(min, max int64) (a dataFiles, lockStart, lockEnd int64) { @@ -1166,17 +1183,66 @@ func (e *Engine) rewriteFile(oldDF *dataFile, valuesByID map[uint64]Values) erro func (e *Engine) flushDeletes() error { e.writeLock.LockRange(math.MinInt64, math.MaxInt64) defer e.writeLock.UnlockRange(math.MinInt64, math.MaxInt64) + e.metaLock.Lock() + defer e.metaLock.Unlock() + + measurements := make(map[string]bool) + deletes := make(map[uint64]string) + e.filesLock.RLock() + for name, _ := range e.deleteMeasurements { + measurements[name] = true + } + for id, key := range e.deletes { + deletes[id] = key + } + e.filesLock.RUnlock() + + // if we're deleting measurements, rewrite the field data + if len(measurements) > 0 { + fields, err := e.readFields() + if err != nil { + return err + } + for name, _ := range measurements { + delete(fields, name) + } + if err := e.writeFields(fields); err != nil { + return err + } + } + series, err := e.readSeries() + if err != nil { + return err + } + for _, key := range deletes { + seriesName, _ := seriesAndFieldFromCompositeKey(key) + delete(series, seriesName) + } + if err := e.writeSeries(series); err != nil { + return err + } + + // now remove the raw time series data from the data files files := e.copyFilesCollection() newFiles := make(dataFiles, 0, len(files)) for _, f := range files { newFiles = append(newFiles, e.writeNewFileExcludeDeletes(f)) } + // update the delete map and files e.filesLock.Lock() defer e.filesLock.Unlock() + e.files = newFiles - e.deletes = make(map[uint64]bool) + + // remove the things we've deleted from the map + for name, _ := range measurements { + delete(e.deleteMeasurements, name) + } + for id, _ := range deletes { + delete(e.deletes, id) + } e.deletesPending.Add(1) go func() { @@ -1288,7 +1354,7 @@ func (e *Engine) keysWithFields(fields map[string]*tsdb.MeasurementFields, keys mf := fields[measurement] if mf != nil { for _, f := range mf.Fields { - a = append(a, seriesFieldKey(k, f.Name)) + a = append(a, SeriesFieldKey(k, f.Name)) } } @@ -1296,7 +1362,7 @@ func (e *Engine) keysWithFields(fields map[string]*tsdb.MeasurementFields, keys mf = e.WAL.measurementFieldsCache[measurement] if mf != nil { for _, f := range mf.Fields { - a = append(a, seriesFieldKey(k, f.Name)) + a = append(a, SeriesFieldKey(k, f.Name)) } } } @@ -1305,30 +1371,23 @@ func (e *Engine) keysWithFields(fields map[string]*tsdb.MeasurementFields, keys } // DeleteSeries deletes the series from the engine. -func (e *Engine) DeleteSeries(keys []string) error { +func (e *Engine) DeleteSeries(seriesKeys []string) error { + e.metaLock.Lock() + defer e.metaLock.Unlock() + fields, err := e.readFields() if err != nil { return err } - keyFields := e.keysWithFields(fields, keys) - - return e.deleteKeyFields(keyFields) -} - -func (e *Engine) deleteKeyFields(keyFields []string) error { - err := e.WAL.DeleteSeries(keyFields) - if err != nil { - return err - } + keyFields := e.keysWithFields(fields, seriesKeys) e.filesLock.Lock() defer e.filesLock.Unlock() - - for _, k := range keyFields { - e.deletes[e.keyToID(k)] = true + for _, key := range keyFields { + e.deletes[e.keyToID(key)] = key } - return nil + return e.WAL.DeleteSeries(keyFields) } // DeleteMeasurement deletes a measurement and all related series. @@ -1336,24 +1395,23 @@ func (e *Engine) DeleteMeasurement(name string, seriesKeys []string) error { e.metaLock.Lock() defer e.metaLock.Unlock() - // remove the field data from the index fields, err := e.readFields() if err != nil { return err } + // mark the measurement, series keys and the fields for deletion on the next flush + // also serves as a tombstone for any queries that come in before the flush keyFields := e.keysWithFields(fields, seriesKeys) + e.filesLock.Lock() + defer e.filesLock.Unlock() - delete(fields, name) - - if err := e.writeFields(fields); err != nil { - return err + e.deleteMeasurements[name] = true + for _, k := range keyFields { + e.deletes[e.keyToID(k)] = k } - e.WAL.DropMeasurementFields(name) - - // now delete all the measurement's series - return e.deleteKeyFields(keyFields) + return e.WAL.DeleteMeasurement(name, seriesKeys) } // SeriesCount returns the number of series buckets on the shard. @@ -1416,7 +1474,7 @@ func (e *Engine) keyToID(key string) uint64 { } func (e *Engine) keyAndFieldToID(series, field string) uint64 { - key := seriesFieldKey(series, field) + key := SeriesFieldKey(series, field) return e.keyToID(key) } @@ -1892,9 +1950,17 @@ func hashSeriesField(key string) uint64 { return h.Sum64() } -// seriesFieldKey combine a series key and field name for a unique string to be hashed to a numeric ID -func seriesFieldKey(seriesKey, field string) string { - return seriesKey + "#" + field +// SeriesFieldKey combine a series key and field name for a unique string to be hashed to a numeric ID +func SeriesFieldKey(seriesKey, field string) string { + return seriesKey + keyFieldSeparator + field +} + +func seriesAndFieldFromCompositeKey(key string) (string, string) { + parts := strings.Split(key, keyFieldSeparator) + if len(parts) != 0 { + return parts[0], strings.Join(parts[1:], keyFieldSeparator) + } + return parts[0], parts[1] } type uint64slice []uint64 diff --git a/tsdb/engine/tsm1/tx.go b/tsdb/engine/tsm1/tx.go index d5f31110d18..54653c28728 100644 --- a/tsdb/engine/tsm1/tx.go +++ b/tsdb/engine/tsm1/tx.go @@ -19,7 +19,7 @@ func (t *tx) Cursor(series string, fields []string, dec *tsdb.FieldCodec, ascend // don't add the overhead of the multifield cursor if we only have one field if len(fields) == 1 { id := t.engine.keyAndFieldToID(series, fields[0]) - isDeleted := t.engine.deletes[id] + _, isDeleted := t.engine.deletes[id] var indexCursor tsdb.Cursor if isDeleted { @@ -37,7 +37,7 @@ func (t *tx) Cursor(series string, fields []string, dec *tsdb.FieldCodec, ascend cursorFields := make([]string, 0) for _, field := range fields { id := t.engine.keyAndFieldToID(series, field) - isDeleted := t.engine.deletes[id] + _, isDeleted := t.engine.deletes[id] var indexCursor tsdb.Cursor if isDeleted { diff --git a/tsdb/engine/tsm1/wal.go b/tsdb/engine/tsm1/wal.go index 9cc2a2fd8fc..4f6607d02ae 100644 --- a/tsdb/engine/tsm1/wal.go +++ b/tsdb/engine/tsm1/wal.go @@ -119,6 +119,7 @@ type Log struct { type IndexWriter interface { Write(valuesByKey map[string]Values, measurementFieldsToSave map[string]*tsdb.MeasurementFields, seriesToCreate []*tsdb.SeriesCreate) error MarkDeletes(keys []string) + MarkMeasurementDelete(name string) } func NewLog(path string) *Log { @@ -168,7 +169,7 @@ func (l *Log) Cursor(series string, fields []string, dec *tsdb.FieldCodec, ascen if len(fields) != 1 { panic("wal cursor should only ever be called with 1 field") } - ck := seriesFieldKey(series, fields[0]) + ck := SeriesFieldKey(series, fields[0]) values := l.cache[ck] // if we're in the middle of a flush, combine the previous cache @@ -268,7 +269,7 @@ func (l *Log) addToCache(points []models.Point, fields map[string]*tsdb.Measurem for _, p := range points { for name, value := range p.Fields() { - k := seriesFieldKey(string(p.Key()), name) + k := SeriesFieldKey(string(p.Key()), name) v := NewValue(p.Time(), value) cacheValues := l.cache[k] @@ -388,11 +389,16 @@ func (l *Log) readFileToCache(fileName string) error { } l.addToCache(nil, nil, series, false) case deleteEntry: - var keys []string - if err := json.Unmarshal(data, &keys); err != nil { + d := &deleteData{} + if err := json.Unmarshal(data, &d); err != nil { return err } - l.Index.MarkDeletes(keys) + l.Index.MarkDeletes(d.Keys) + l.Index.MarkMeasurementDelete(d.MeasurementName) + l.deleteKeysFromCache(d.Keys) + if d.MeasurementName != "" { + l.deleteMeasurementFromCache(d.MeasurementName) + } } } } @@ -431,27 +437,62 @@ func (l *Log) Flush() error { return l.flush(idleFlush) } -func (l *Log) DropMeasurementFields(measurement string) { +func (l *Log) DeleteMeasurement(measurement string, keys []string) error { + d := &deleteData{MeasurementName: measurement, Keys: keys} + err := l.writeDeleteEntry(d) + if err != nil { + return err + } + + l.deleteKeysFromCache(keys) + l.deleteMeasurementFromCache(measurement) + + return nil +} + +func (l *Log) deleteMeasurementFromCache(name string) { l.cacheLock.Lock() defer l.cacheLock.Unlock() - delete(l.measurementFieldsCache, measurement) + delete(l.measurementFieldsCache, name) +} + +func (l *Log) writeDeleteEntry(d *deleteData) error { + js, err := json.Marshal(d) + if err != nil { + return err + } + data := snappy.Encode(nil, js) + return l.writeToLog(deleteEntry, data) } func (l *Log) DeleteSeries(keys []string) error { + l.deleteKeysFromCache(keys) + + return l.writeDeleteEntry(&deleteData{Keys: keys}) +} + +func (l *Log) deleteKeysFromCache(keys []string) { + seriesKeys := make(map[string]bool) + for _, k := range keys { + series, _ := seriesAndFieldFromCompositeKey(k) + seriesKeys[series] = true + } + l.cacheLock.Lock() + defer l.cacheLock.Unlock() + for _, k := range keys { delete(l.cache, k) } - l.cacheLock.Unlock() - b, err := json.Marshal(keys) - if err != nil { - return err + // now remove any of these that are marked for creation + var seriesCreate []*tsdb.SeriesCreate + for _, sc := range l.seriesToCreateCache { + if _, ok := seriesKeys[sc.Series.Key]; !ok { + seriesCreate = append(seriesCreate, sc) + } } - - cb := snappy.Encode(nil, b) - - return l.writeToLog(deleteEntry, cb) + l.seriesToCreateCache = seriesCreate } // Close will finish any flush that is currently in process and close file handles @@ -731,6 +772,13 @@ func (c *walCursor) nextReverse() Value { return c.cache[c.position] } +// deleteData holds the information for a delete entry +type deleteData struct { + // MeasurementName will be empty for deletes that are only against series + MeasurementName string + Keys []string +} + // idFromFileName parses the segment file ID from its name func idFromFileName(name string) (int, error) { parts := strings.Split(filepath.Base(name), ".") diff --git a/tsdb/engine/tsm1/wal_test.go b/tsdb/engine/tsm1/wal_test.go index dfc5cda8cc8..9df191c7a7e 100644 --- a/tsdb/engine/tsm1/wal_test.go +++ b/tsdb/engine/tsm1/wal_test.go @@ -111,11 +111,11 @@ func TestWAL_TestWriteQueryOpen(t *testing.T) { t.Fatalf("failed to open: %s", err.Error()) } - if len(vals["cpu,host=A#value"]) != 2 { + if len(vals[tsm1.SeriesFieldKey("cpu,host=A", "value")]) != 2 { t.Fatal("expected host A values to flush to index on open") } - if len(vals["cpu,host=B#value"]) != 1 { + if len(vals[tsm1.SeriesFieldKey("cpu,host=B", "value")]) != 1 { t.Fatal("expected host B values to flush to index on open") } @@ -174,3 +174,5 @@ func (m *MockIndexWriter) Write(valuesByKey map[string]tsm1.Values, measurementF } func (m *MockIndexWriter) MarkDeletes(keys []string) {} + +func (m *MockIndexWriter) MarkMeasurementDelete(name string) {} From 267f34b94e8d4ab6987f0aace2a6954e83133f14 Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Mon, 5 Oct 2015 19:57:49 -0400 Subject: [PATCH 135/139] Updates based on PR feedback --- models/points.go | 2 +- tsdb/engine.go | 2 +- tsdb/engine/tsm1/cursor.go | 42 ++++++++++++++++++++++++++++++++------ tsdb/engine/tsm1/tsm1.go | 4 ++++ 4 files changed, 42 insertions(+), 8 deletions(-) diff --git a/models/points.go b/models/points.go index d91ff13afbe..592780389d4 100644 --- a/models/points.go +++ b/models/points.go @@ -1146,7 +1146,7 @@ func (t Tags) HashKey() []byte { ek := escapeTag([]byte(k)) ev := escapeTag([]byte(v)) - if len(string(ev)) > 0 { + if len(ev) > 0 { escaped[string(ek)] = string(ev) } } diff --git a/tsdb/engine.go b/tsdb/engine.go index 97ca51d9ddb..fb1b2108c5c 100644 --- a/tsdb/engine.go +++ b/tsdb/engine.go @@ -71,7 +71,7 @@ func NewEngine(path string, walPath string, options EngineOptions) (Engine, erro return newEngineFuncs[options.EngineVersion](path, walPath, options), nil } - // Only bolt-based backends are currently supported so open it and check the format. + // Only bolt and tsm1 based storage engines are currently supported var format string if err := func() error { // if it's a dir then it's a tsm1 engine diff --git a/tsdb/engine/tsm1/cursor.go b/tsdb/engine/tsm1/cursor.go index 06fd0bbf8f8..01ac2e3c7a7 100644 --- a/tsdb/engine/tsm1/cursor.go +++ b/tsdb/engine/tsm1/cursor.go @@ -6,6 +6,10 @@ import ( "github.com/influxdb/influxdb/tsdb" ) +// combinedEngineCursor holds a cursor for the WAL and the index +// and will combine the two together. Any points in the WAL with +// identical timestamps from the index will be preferred over the +// index point type combinedEngineCursor struct { walCursor tsdb.Cursor engineCursor tsdb.Cursor @@ -24,20 +28,26 @@ func NewCombinedEngineCursor(wc, ec tsdb.Cursor, ascending bool) tsdb.Cursor { } } +// SeekTo will seek both the index and WAL cursor func (c *combinedEngineCursor) SeekTo(seek int64) (key int64, value interface{}) { c.walKeyBuf, c.walValueBuf = c.walCursor.SeekTo(seek) c.engineKeyBuf, c.engineValueBuf = c.engineCursor.SeekTo(seek) return c.read() } +// Next returns the next value in the cursor func (c *combinedEngineCursor) Next() (int64, interface{}) { return c.read() } +// Ascending returns true if the cursor is time ascending func (c *combinedEngineCursor) Ascending() bool { return c.ascending } +// read will return the buffer value that is next from either the +// WAL or index cursor and repopulate the buffer value with the +// appropriate cursor's next value func (c *combinedEngineCursor) read() (key int64, value interface{}) { if c.walKeyBuf == tsdb.EOF && c.engineKeyBuf == tsdb.EOF { return tsdb.EOF, nil @@ -84,6 +94,9 @@ func (c *combinedEngineCursor) read() (key int64, value interface{}) { return } +// multieFieldCursor wraps cursors for multiple fields on the same series +// key. Instead of returning a plain interface value in the call for Next(), +// it returns a map[string]interface{} for the field values type multiFieldCursor struct { fields []string cursors []tsdb.Cursor @@ -158,18 +171,31 @@ func (c *emptyCursor) Next() (int64, interface{}) { return tsdb.EOF, func (c *emptyCursor) SeekTo(key int64) (int64, interface{}) { return tsdb.EOF, nil } func (c *emptyCursor) Ascending() bool { return c.ascending } +// cursor is a cursor for the data in the index type cursor struct { - id uint64 - f *dataFile + // id for the series key and field + id uint64 + + // f is the current data file we're reading from + f *dataFile + + // filesPos is the position in the files index we're reading from filesPos int // the index in the files slice we're looking at - pos uint32 - vals Values + + // pos is the position in the current data file we're reading + pos uint32 + + // vals is the current decoded block of Values we're iterating from + vals Values ascending bool - blockPositions []uint32 // only used for descending queries + // blockPositions is used for descending queries to keep track + // of what positions in the current data file encoded blocks for + // the id exist at + blockPositions []uint32 - // time acending list of data files + // time acending slice of read only data files files []*dataFile } @@ -334,10 +360,13 @@ func (c *cursor) seekDescending(seek int64) (int64, interface{}) { return tsdb.EOF, nil } +// blockMinTime is the minimum time for the block func (c *cursor) blockMinTime(pos uint32) int64 { return int64(btou64(c.f.mmap[pos+12 : pos+20])) } +// setBlockPositions will read the positions of all +// blocks for the cursor id in the given data file func (c *cursor) setBlockPositions() { pos := c.pos @@ -431,6 +460,7 @@ func (c *cursor) blockLength(pos uint32) uint32 { return btou32(c.f.mmap[pos+8 : pos+12]) } +// decodeBlock will decod the block and set the vals func (c *cursor) decodeBlock(position uint32) { length := c.blockLength(position) block := c.f.mmap[position+blockHeaderSize : position+blockHeaderSize+length] diff --git a/tsdb/engine/tsm1/tsm1.go b/tsdb/engine/tsm1/tsm1.go index aac359c19d1..59a0f3fe051 100644 --- a/tsdb/engine/tsm1/tsm1.go +++ b/tsdb/engine/tsm1/tsm1.go @@ -80,6 +80,8 @@ const ( // MAP_POPULATE is for the mmap syscall. For some reason this isn't defined in golang's syscall MAP_POPULATE = 0x8000 + // magicNumber is written as the first 4 bytes of a data file to + // identify the file as a tsm1 formatted file magicNumber uint32 = 0x16D116D1 ) @@ -1298,6 +1300,8 @@ func (e *Engine) writeNewFileExcludeDeletes(oldDF *dataFile) *dataFile { } func (e *Engine) nextFileName() string { + e.filesLock.Lock() + defer e.filesLock.Unlock() e.currentFileID++ return filepath.Join(e.path, fmt.Sprintf("%07d.%s", e.currentFileID, Format)) } From be477b2aab99281f077f62977a1b99ce0fa4feac Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Tue, 6 Oct 2015 12:26:45 -0700 Subject: [PATCH 136/139] Fix cursor bug on index --- tsdb/engine/tsm1/cursor.go | 7 +++- tsdb/engine/tsm1/tsm1_test.go | 69 +++++++++++++++++++++++++++++++++++ 2 files changed, 75 insertions(+), 1 deletion(-) diff --git a/tsdb/engine/tsm1/cursor.go b/tsdb/engine/tsm1/cursor.go index 01ac2e3c7a7..0cee157d3a3 100644 --- a/tsdb/engine/tsm1/cursor.go +++ b/tsdb/engine/tsm1/cursor.go @@ -311,6 +311,10 @@ func (c *cursor) seekAscending(seek int64) (int64, interface{}) { } // it must be in this block or not at all + id := btou64((c.f.mmap[c.pos : c.pos+8])) + if id != c.id { + return tsdb.EOF, nil + } c.decodeBlock(c.pos) // see if we can find it in this block @@ -389,7 +393,8 @@ func (c *cursor) setBlockPositions() { func (c *cursor) Next() (int64, interface{}) { if c.ascending { - return c.nextAscending() + k, v := c.nextAscending() + return k, v } return c.nextDescending() } diff --git a/tsdb/engine/tsm1/tsm1_test.go b/tsdb/engine/tsm1/tsm1_test.go index 6a39e41ed8c..421bd3c13c8 100644 --- a/tsdb/engine/tsm1/tsm1_test.go +++ b/tsdb/engine/tsm1/tsm1_test.go @@ -1236,6 +1236,75 @@ func TestEngine_Deletes(t *testing.T) { }() } +func TestEngine_IndexGoodAfterFlush(t *testing.T) { + e := OpenDefaultEngine() + defer e.Cleanup() + + fields := []string{"value"} + + p1 := parsePoint("test,tag=a value=2.5 1443916800000000000") + p2 := parsePoint("test value=3.5 1443916810000000000") + p3 := parsePoint("test,tag=b value=6.5 1443916860000000000") + p4 := parsePoint("test value=8.5 1443916861000000000") + + e.SkipCompaction = true + e.WAL.SkipCache = false + + for _, p := range []models.Point{p1, p2, p3, p4} { + if err := e.WritePoints([]models.Point{p}, nil, nil); err != nil { + t.Fatalf("failed to write points: %s", err.Error()) + } + } + + verify := func() { + tx, _ := e.Begin(false) + defer tx.Rollback() + c1 := tx.Cursor("test", fields, nil, true) + c2 := tx.Cursor("test,tag=a", fields, nil, true) + c3 := tx.Cursor("test,tag=b", fields, nil, true) + k, v := c1.SeekTo(1443916800000000001) + if k != p2.UnixNano() { + t.Fatalf("time wrong: %d", k) + } + if v != 3.5 { + t.Fatalf("value wrong: %f", v.(float64)) + } + k, v = c1.Next() + if k != p4.UnixNano() { + t.Fatalf("time wrong: %d", k) + } + if v != 8.5 { + t.Fatalf("value wrong: %f", v.(float64)) + } + if k, _ := c1.Next(); k != tsdb.EOF { + t.Fatalf("expected EOF: %d", k) + } + k, _ = c2.SeekTo(1443916800000000001) + if k != tsdb.EOF { + t.Fatalf("time wrong: %d", k) + } + k, v = c3.SeekTo(1443916800000000001) + if k != p3.UnixNano() { + t.Fatalf("time wrong: %d", k) + } + if v != 6.5 { + t.Fatalf("value wrong: %f", v.(float64)) + } + if k, _ := c3.Next(); k != tsdb.EOF { + t.Fatalf("expected EOF: %d", k) + } + } + + fmt.Println("verify1") + verify() + fmt.Println("flush") + if err := e.WAL.Flush(); err != nil { + t.Fatalf("error flushing: %s", err.Error) + } + fmt.Println("verify2") + verify() +} + // Engine represents a test wrapper for tsm1.Engine. type Engine struct { *tsm1.Engine From 41e3294d4adf5869a39a1bdcb1186c28dc3de967 Mon Sep 17 00:00:00 2001 From: Jason Wilder Date: Tue, 6 Oct 2015 14:00:31 -0600 Subject: [PATCH 137/139] Fix panic: assignment to entry in nil map Closing the store did not properly return an error for in-flight writes because the closing channel was set to nil when closed. A nil channel is not selectable so writes continue on past the guard checks and trigger panics. --- tsdb/store.go | 21 ++++++++++++++------- 1 file changed, 14 insertions(+), 7 deletions(-) diff --git a/tsdb/store.go b/tsdb/store.go index bee68c7fddf..be7076d00e2 100644 --- a/tsdb/store.go +++ b/tsdb/store.go @@ -28,6 +28,7 @@ func NewStore(path string) *Store { var ( ErrShardNotFound = fmt.Errorf("shard not found") + ErrStoreClosed = fmt.Errorf("store is closed") ) const ( @@ -46,6 +47,7 @@ type Store struct { closing chan struct{} wg sync.WaitGroup + opened bool } // Path returns the store's root path. @@ -78,7 +80,7 @@ func (s *Store) CreateShard(database, retentionPolicy string, shardID uint64) er select { case <-s.closing: - return fmt.Errorf("closing") + return ErrStoreClosed default: } @@ -369,6 +371,7 @@ func (s *Store) Open() error { } go s.periodicMaintenance() + s.opened = true return nil } @@ -376,6 +379,13 @@ func (s *Store) Open() error { func (s *Store) WriteToShard(shardID uint64, points []models.Point) error { s.mu.RLock() defer s.mu.RUnlock() + + select { + case <-s.closing: + return ErrStoreClosed + default: + } + sh, ok := s.shards[shardID] if !ok { return ErrShardNotFound @@ -410,9 +420,9 @@ func (s *Store) CreateMapper(shardID uint64, stmt influxql.Statement, chunkSize func (s *Store) Close() error { s.mu.Lock() defer s.mu.Unlock() - if s.closing != nil { + + if s.opened { close(s.closing) - s.closing = nil } s.wg.Wait() @@ -421,10 +431,7 @@ func (s *Store) Close() error { return err } } - if s.closing != nil { - close(s.closing) - } - s.closing = nil + s.opened = false s.shards = nil s.databaseIndexes = nil From 40ff4f4a86f4512e3cf05a90734b17bb9c219018 Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Tue, 6 Oct 2015 15:30:11 -0700 Subject: [PATCH 138/139] Change default to bz1 --- etc/config.sample.toml | 4 +++- tsdb/config.go | 2 +- tsdb/engine/tsm1/tsm1_test.go | 2 +- 3 files changed, 5 insertions(+), 3 deletions(-) diff --git a/etc/config.sample.toml b/etc/config.sample.toml index 0846a0b2919..fdf5d50d84d 100644 --- a/etc/config.sample.toml +++ b/etc/config.sample.toml @@ -37,7 +37,9 @@ reporting-disabled = false [data] dir = "/var/opt/influxdb/data" - # Controls the engine type for new shards. + # Controls the engine type for new shards. Options are b1, bz1, or tsm1. + # b1 is the 0.9.2 storage engine, bz1 is the 0.9.3 and 0.9.4 engine. + # tsm1 is the 0.9.5 engine # engine ="bz1" # The following WAL settings are for the b1 storage engine used in 0.9.2. They won't diff --git a/tsdb/config.go b/tsdb/config.go index 115f895f139..52d182c1172 100644 --- a/tsdb/config.go +++ b/tsdb/config.go @@ -8,7 +8,7 @@ import ( const ( // DefaultEngine is the default engine for new shards - DefaultEngine = "tsm1" + DefaultEngine = "bz1" // DefaultMaxWALSize is the default size of the WAL before it is flushed. DefaultMaxWALSize = 100 * 1024 * 1024 // 100MB diff --git a/tsdb/engine/tsm1/tsm1_test.go b/tsdb/engine/tsm1/tsm1_test.go index 421bd3c13c8..40e9f862415 100644 --- a/tsdb/engine/tsm1/tsm1_test.go +++ b/tsdb/engine/tsm1/tsm1_test.go @@ -1299,7 +1299,7 @@ func TestEngine_IndexGoodAfterFlush(t *testing.T) { verify() fmt.Println("flush") if err := e.WAL.Flush(); err != nil { - t.Fatalf("error flushing: %s", err.Error) + t.Fatalf("error flushing: %s", err.Error()) } fmt.Println("verify2") verify() From b11308133a62484fc494425466f3d437274ce617 Mon Sep 17 00:00:00 2001 From: Paul Dix Date: Tue, 6 Oct 2015 15:49:37 -0700 Subject: [PATCH 139/139] Only limit field count for non-tsm engines --- tsdb/engine/b1/b1_test.go | 4 ++-- tsdb/engine/tsm1/tsm1_test.go | 2 +- tsdb/shard.go | 10 ++++++---- 3 files changed, 9 insertions(+), 7 deletions(-) diff --git a/tsdb/engine/b1/b1_test.go b/tsdb/engine/b1/b1_test.go index 5c3c19ee3bc..31b90344c36 100644 --- a/tsdb/engine/b1/b1_test.go +++ b/tsdb/engine/b1/b1_test.go @@ -21,7 +21,7 @@ func TestEngine_WritePoints(t *testing.T) { // Create metadata. mf := &tsdb.MeasurementFields{Fields: make(map[string]*tsdb.Field)} - mf.CreateFieldIfNotExists("value", influxql.Float) + mf.CreateFieldIfNotExists("value", influxql.Float, true) seriesToCreate := []*tsdb.SeriesCreate{ {Series: tsdb.NewSeries(string(models.MakeKey([]byte("temperature"), nil)), nil)}, } @@ -84,7 +84,7 @@ func TestEngine_WritePoints_Reverse(t *testing.T) { // Create metadata. mf := &tsdb.MeasurementFields{Fields: make(map[string]*tsdb.Field)} - mf.CreateFieldIfNotExists("value", influxql.Float) + mf.CreateFieldIfNotExists("value", influxql.Float, true) seriesToCreate := []*tsdb.SeriesCreate{ {Series: tsdb.NewSeries(string(models.MakeKey([]byte("temperature"), nil)), nil)}, } diff --git a/tsdb/engine/tsm1/tsm1_test.go b/tsdb/engine/tsm1/tsm1_test.go index 40e9f862415..dbd353d7ee0 100644 --- a/tsdb/engine/tsm1/tsm1_test.go +++ b/tsdb/engine/tsm1/tsm1_test.go @@ -1078,7 +1078,7 @@ func TestEngine_Deletes(t *testing.T) { fields := []string{"value"} // Create metadata. mf := &tsdb.MeasurementFields{Fields: make(map[string]*tsdb.Field)} - mf.CreateFieldIfNotExists("value", influxql.Float) + mf.CreateFieldIfNotExists("value", influxql.Float, false) atag := map[string]string{"host": "A"} btag := map[string]string{"host": "B"} seriesToCreate := []*tsdb.SeriesCreate{ diff --git a/tsdb/shard.go b/tsdb/shard.go index b2589443163..3a7215e0857 100644 --- a/tsdb/shard.go +++ b/tsdb/shard.go @@ -367,7 +367,9 @@ func (s *Shard) createFieldsAndMeasurements(fieldsToCreate []*FieldCreate) (map[ measurementsToSave[f.Measurement] = m // add the field to the in memory index - if err := m.CreateFieldIfNotExists(f.Field.Name, f.Field.Type); err != nil { + // only limit the field count for non-tsm eninges + limitFieldCount := s.engine.Format() == B1Format || s.engine.Format() == BZ1Format + if err := m.CreateFieldIfNotExists(f.Field.Name, f.Field.Type, limitFieldCount); err != nil { return nil, err } @@ -475,7 +477,7 @@ func (m *MeasurementFields) UnmarshalBinary(buf []byte) error { // CreateFieldIfNotExists creates a new field with an autoincrementing ID. // Returns an error if 255 fields have already been created on the measurement or // the fields already exists with a different type. -func (m *MeasurementFields) CreateFieldIfNotExists(name string, typ influxql.DataType) error { +func (m *MeasurementFields) CreateFieldIfNotExists(name string, typ influxql.DataType, limitCount bool) error { // Ignore if the field already exists. if f := m.Fields[name]; f != nil { if f.Type != typ { @@ -484,8 +486,8 @@ func (m *MeasurementFields) CreateFieldIfNotExists(name string, typ influxql.Dat return nil } - // Only 255 fields are allowed. If we go over that then return an error. - if len(m.Fields)+1 > math.MaxUint8 { + // If we're supposed to limit the number of fields, only 255 are allowed. If we go over that then return an error. + if len(m.Fields)+1 > math.MaxUint8 && limitCount { return ErrFieldOverflow }