From 3446c6024d17c51ceae92c1477ce7eab361bca7e Mon Sep 17 00:00:00 2001
From: Paul Dix <paul@pauldix.net>
Date: Wed, 2 Sep 2015 18:05:07 -0400
Subject: [PATCH 01/68] 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 0000000000..7739ce2234
--- /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 0000000000..077053449a
--- /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 <paul@pauldix.net>
Date: Wed, 2 Sep 2015 18:45:32 -0400
Subject: [PATCH 02/68] 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 7739ce2234..b53f4f2563 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 <paul@pauldix.net>
Date: Sun, 6 Sep 2015 15:49:15 -0700
Subject: [PATCH 03/68] 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 13aa35a97c..64253f0f57 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 b3c136e794..2fb96e99df 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 c8d5946139..d2862b5486 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 fdc337b2f7..356b327fe9 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 e4d5682ca4..aa38600b80 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 97873afe3b..0b0cb1e60c 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 c5565ff06c..03022f9d0e 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 b53f4f2563..285c83f8c0 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 077053449a..26bb4c2e07 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 0000000000..d94a7d7795
--- /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 0000000000..48cdff7929
--- /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 e355cc661d..a14822f367 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 <paul@pauldix.net>
Date: Mon, 7 Sep 2015 15:56:21 -0700
Subject: [PATCH 04/68] 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             | 254 ++++++++++++++++++++++-------
 tsdb/engine/pd1/pd1_test.go        | 120 +++++++++-----
 tsdb/shard.go                      |  52 +++---
 6 files changed, 326 insertions(+), 126 deletions(-)

diff --git a/cmd/influx_stress/influx_stress.go b/cmd/influx_stress/influx_stress.go
index 9292fb02d9..9fe9e2af4b 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 d2862b5486..b0e9254d95 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 285c83f8c0..60f72766d8 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 d94a7d7795..4d3c752ae2 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))
 
-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
+	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
 		}
 	}
-	return nil, nil
 }
 
-func (c *cursor) next(id uint64) (key, value []byte)
+func (c *cursor) Next() (key, value []byte) {
+	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 c.filesPos < len(c.files) {
+			f := c.files[c.filesPos]
+
+			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
+	}
+
+	v := c.vals[0]
+	c.vals = c.vals[1:]
+
+	return v.TimeBytes(), v.ValueBytes()
+}
+
+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 48cdff7929..7d915aab69 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))
-	if btou64(k) != uint64(p4.UnixNano()) {
-		t.Fatalf("p4 time wrong:\n\texp:%d\n\tgot:%d\n", p4.UnixNano(), btou64(k))
+	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, p4.Data()) {
-		t.Fatal("p4 data not equal")
+	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("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")
+	}
+
+	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 a14822f367..2e04735665 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
-		}
+	// 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()
+	// 	// 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
-		}
+	// 	// 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)
-	}
+	// 	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 <paul@pauldix.net>
Date: Wed, 9 Sep 2015 11:29:50 -0700
Subject: [PATCH 05/68] WIP: more WAL work

---
 cmd/influx_stress/influx_stress.go |   1 +
 tsdb/config.go                     |   2 +-
 tsdb/engine/pd1/encoding.go        | 129 +++++--
 tsdb/engine/pd1/encoding_test.go   |  34 +-
 tsdb/engine/pd1/pd1.go             | 506 ++++++-------------------
 tsdb/engine/pd1/pd1_test.go        |   9 +-
 tsdb/engine/pd1/wal.go             | 581 +++++++++++++++++++++++++++++
 7 files changed, 827 insertions(+), 435 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 9fe9e2af4b..2247a5329b 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 9843541e29..dfd267d2c3 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 60f72766d8..7262e8a6c3 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() }
-
-// TODO: make this work with nanosecond timestamps
-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.Finish()
-	return append(u64tob(uint64(values[0].Time.UnixNano())), s.Bytes()...)
+func (f *FloatValue) Size() int {
+	return 16
 }
 
-func DecodeFloatBlock(block []byte) ([]FloatValue, error) {
+// TODO: make this work with nanosecond timestamps
+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.Finish()
+	return append(u64tob(uint64(values[0].Time().UnixNano())), s.Bytes()...)
+}
+
+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 26bb4c2e07..aa5a4b15e7 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 4d3c752ae2..ef91fb66e2 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,31 +221,70 @@ 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)
+	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
+				}
+			}
+
+			// 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!")
+				}
+			} else {
+				newKeys = true
+				ids[k] = hashID
+				idToKey[id] = k
+				id = hashID
+			}
+		}
+
+		valuesByID[id] = values
+	}
+
+	if newKeys {
+		b, err := json.Marshal(ids)
+		if err != nil {
+			return err
+		}
+		if err := e.replaceCompressedFile("ids", 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.
@@ -240,87 +295,6 @@ func (e *Engine) WriteAndCompact(points []tsdb.Point, measurementFieldsToSave ma
 		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")
-			}
-		}
-	}
-
-	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 {
@@ -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
-		}
+func (e *Engine) DecodeAndCombine(newValues Values, block, buf []byte, nextTime int64, hasFutureBlock bool) (Values, []byte, error) {
+	values := newValues.DecodeSameTypeBlock(block)
 
-		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
-		}
-		sort.Sort(FloatValues(values))
-		// TODO: deduplicate values
+	var remainingValues Values
 
-		if len(values) > DefaultMaxPointsPerBlock {
-			v.floatValues = values[DefaultMaxPointsPerBlock:]
-			values = values[:DefaultMaxPointsPerBlock]
+	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)
 		}
-
-		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
+	// TODO: deduplicate values
+
+	if len(values) > DefaultMaxPointsPerBlock {
+		remainingValues = values[DefaultMaxPointsPerBlock:]
+		values = values[:DefaultMaxPointsPerBlock]
+	}
+
+	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 7d915aab69..79817eb531 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 0000000000..26f2af48ff
--- /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 <paul@pauldix.net>
Date: Thu, 17 Sep 2015 11:23:27 -0400
Subject: [PATCH 06/68] 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 26f2af48ff..03548f062e 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
+			}
+		}
 	}
 
-	// convert to values that can be either cached in memory or flushed to the index
+	// 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)
+	}
+
+	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 0000000000..c1ef614650
--- /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 <paul@pauldix.net>
Date: Thu, 17 Sep 2015 15:46:37 -0400
Subject: [PATCH 07/68] 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 aa5a4b15e7..82968912b0 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 <paul@pauldix.net>
Date: Thu, 17 Sep 2015 15:46:53 -0400
Subject: [PATCH 08/68] 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 dfd267d2c3..b7bc409b4a 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 ef91fb66e2..02708d7b45 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 03548f062e..e3157a94b0 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 <paul@pauldix.net>
Date: Thu, 17 Sep 2015 18:25:55 -0400
Subject: [PATCH 09/68] Update engine to put index at the end of data files

---
 tsdb/engine/pd1/pd1.go | 78 ++++++++++++++++++++----------------------
 1 file changed, 38 insertions(+), 40 deletions(-)

diff --git a/tsdb/engine/pd1/pd1.go b/tsdb/engine/pd1/pd1.go
index 02708d7b45..b16561776b 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()
 
-	min := 0
-	max := int(seriesCount)
+	seriesCount := d.SeriesCount()
+	indexStart := d.size - uint32(seriesCount*12+20)
+
+	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 <paul@pauldix.net>
Date: Fri, 18 Sep 2015 11:08:03 -0400
Subject: [PATCH 10/68] 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 b16561776b..df930a6124 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 79817eb531..0ed7c368c5 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 <paul@pauldix.net>
Date: Fri, 18 Sep 2015 15:18:05 -0400
Subject: [PATCH 11/68] 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 e3157a94b0..610fe74d31 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 <paul@pauldix.net>
Date: Fri, 25 Sep 2015 10:49:26 -0400
Subject: [PATCH 12/68] 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 b7bc409b4a..4aec4a14e5 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 b0e9254d95..d2862b5486 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 7262e8a6c3..cad991aa7a 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 df930a6124..1428f04b45 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 0ed7c368c5..1a8f8613fc 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 610fe74d31..7b538fed3a 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 c1ef614650..de03dfbd60 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 <paul@pauldix.net>
Date: Fri, 25 Sep 2015 17:11:27 -0400
Subject: [PATCH 13/68] 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 cad991aa7a..5d1bee9a8f 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,20 +89,14 @@ func (f *FloatValue) Time() time.Time {
 	return f.time
 }
 
+func (f *FloatValue) UnixNano() int64 {
+	return f.time.UnixNano()
+}
+
 func (f *FloatValue) Value() interface{} {
 	return f.value
 }
 
-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
-}
-
 func (f *FloatValue) Size() int {
 	return 16
 }
diff --git a/tsdb/engine/pd1/pd1.go b/tsdb/engine/pd1/pd1.go
index 1428f04b45..40aa7f8050 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 1a8f8613fc..a20cc665ce 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 7b538fed3a..e7fb7fb39a 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 <paul@pauldix.net>
Date: Sat, 26 Sep 2015 09:02:14 -0400
Subject: [PATCH 14/68] 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 5d1bee9a8f..bf06fb9880 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 40aa7f8050..5ae35e4798 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 a20cc665ce..3be9bc63bb 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 e7fb7fb39a..063e0d884b 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 <paul@pauldix.net>
Date: Sat, 26 Sep 2015 10:47:36 -0400
Subject: [PATCH 15/68] 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 5ae35e4798..9ecb6fe5b0 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 3be9bc63bb..097c515536 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 063e0d884b..be9dd48e71 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 <paul@pauldix.net>
Date: Sun, 27 Sep 2015 14:45:55 -0400
Subject: [PATCH 16/68] 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 4aec4a14e5..8716a45537 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 bf06fb9880..9a6d9fea7a 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 9ecb6fe5b0..0108719457 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()
-
-	if err := e.writeNewFields(measurementFieldsToSave); err != nil {
+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 err := e.writeNewSeries(seriesToCreate); err != nil {
+	if len(valuesByID) == 0 {
+		return nil
+	}
+
+	e.writeLock.LockRange(startTime, endTime)
+	defer e.writeLock.UnlockRange(startTime, endTime)
+
+	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
 	}
 
+	// 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 {
+			if err := e.writeBlock(f, id, block); err != nil {
 				f.Close()
 				return err
 			}
-			if _, err := f.Write(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 097c515536..db867e9303 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 be9dd48e71..1447cb5f06 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 de03dfbd60..509ed6984c 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 0000000000..5f48fb9311
--- /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 <paul@pauldix.net>
Date: Sun, 27 Sep 2015 16:44:57 -0400
Subject: [PATCH 17/68] 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 0108719457..3496268bf9 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 db867e9303..35385ddfb6 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 <paul@pauldix.net>
Date: Sun, 27 Sep 2015 19:37:03 -0400
Subject: [PATCH 18/68] 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 0000000000..d31252aa32
--- /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 9a6d9fea7a..b21d394af5 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 3496268bf9..a2d8687db9 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 35385ddfb6..52cb9717f3 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 1447cb5f06..df1cbfe1b2 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 <paul@pauldix.net>
Date: Mon, 28 Sep 2015 21:04:02 -0400
Subject: [PATCH 19/68] 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 8716a45537..2039ff813b 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 a2d8687db9..54cb6d4750 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 52cb9717f3..35ec6c720f 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 df1cbfe1b2..c4c6d23ebe 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 <paul@pauldix.net>
Date: Mon, 28 Sep 2015 22:50:00 -0400
Subject: [PATCH 20/68] 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                    | 61 ++++++++++++++-------------
 tsdb/store.go                    | 42 ++++++++++++++++++-
 10 files changed, 185 insertions(+), 136 deletions(-)

diff --git a/tsdb/config.go b/tsdb/config.go
index 2039ff813b..1e7e29a0d8 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 d2862b5486..407801d842 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 356b327fe9..a3f63602cd 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 aa38600b80..881b82dc43 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 82968912b0..02598a764d 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 54cb6d4750..1f56c06653 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 35ec6c720f..491c7cd552 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 c4c6d23ebe..49b72b4a60 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 2e04735665..b1dbac32fc 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
-	// 	}
+	// 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()
+			// 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
-	// 	}
+			// 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)
-	// }
+			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 13235a1684..62dab63117 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 <paul@pauldix.net>
Date: Tue, 29 Sep 2015 08:35:05 -0400
Subject: [PATCH 21/68] 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 1f56c06653..8be62d069d 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 <paul@pauldix.net>
Date: Tue, 29 Sep 2015 08:59:25 -0400
Subject: [PATCH 22/68] 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 | 74 ++++++++++++++++++++++--------------------
 tsdb/engine/pd1/tx.go  | 52 +++++++++++++++++++++++++++++
 2 files changed, 91 insertions(+), 35 deletions(-)
 create mode 100644 tsdb/engine/pd1/tx.go

diff --git a/tsdb/engine/pd1/pd1.go b/tsdb/engine/pd1/pd1.go
index 8be62d069d..32edc84c7c 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()
+		}
+
+		// 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
 	}
 
-	// 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)
+	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 0000000000..16aac8b5e4
--- /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 <paul@pauldix.net>
Date: Tue, 29 Sep 2015 10:56:11 -0400
Subject: [PATCH 23/68] 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 32edc84c7c..82df8c60e6 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 b1dbac32fc..1f606b613b 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 785ca13908..4a0a2d3ede 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 62dab63117..1fb8560167 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 <paul@pauldix.net>
Date: Tue, 29 Sep 2015 11:27:02 -0400
Subject: [PATCH 24/68] 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 491c7cd552..e1abe87360 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 <paul@pauldix.net>
Date: Tue, 29 Sep 2015 11:37:28 -0400
Subject: [PATCH 25/68] 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 2247a5329b..9292fb02d9 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 f098c1bdc8..d968784b02 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 <jason@influxdb.com>
Date: Mon, 21 Sep 2015 13:56:10 -0600
Subject: [PATCH 26/68] 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 82df8c60e6..d707661fad 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 <jason@influxdb.com>
Date: Mon, 21 Sep 2015 13:57:03 -0600
Subject: [PATCH 27/68] 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 509ed6984c..034ad2dd3e 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 <jason@influxdb.com>
Date: Mon, 21 Sep 2015 13:58:33 -0600
Subject: [PATCH 28/68] 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 9292fb02d9..fee98f776c 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 <jason@influxdb.com>
Date: Mon, 21 Sep 2015 14:52:41 -0600
Subject: [PATCH 29/68] 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       |  85 ++++++-
 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, 1067 insertions(+), 12 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 b21d394af5..6b29913e0e 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()))
-	for _, v := range values {
-		s.Push(uint32(v.Time().Unix()), v.value)
+	if len(values) == 0 {
+		return []byte{}
 	}
-	s.Finish()
-	return append(u64tob(uint64(values[0].Time().UnixNano())), s.Bytes()...)
+
+	// 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 {
+		tsenc.Write(v.Time())
+		venc.Push(v.value)
+	}
+	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 <len timestamp bytes>, <ts bytes>, <value bytes>
+	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 0000000000..dddb9f39b2
--- /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 0000000000..5bfa377ac1
--- /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 0000000000..b83199ef04
--- /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 0000000000..806fba7c5e
--- /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 <mail@jasonwilder.com>
Date: Thu, 24 Sep 2015 14:29:51 -0600
Subject: [PATCH 30/68] 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 6b29913e0e..055048620c 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 02598a764d..49006085d7 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 0000000000..04ddc1a0f4
--- /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 0000000000..82042f77bd
--- /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 b83199ef04..64907bb760 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 <mail@jasonwilder.com>
Date: Thu, 24 Sep 2015 16:42:48 -0600
Subject: [PATCH 31/68] 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 055048620c..a2e1669945 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 49006085d7..c249f1aa8c 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 04ddc1a0f4..c4c40b8778 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 64907bb760..9119dbd518 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 806fba7c5e..da78b2b6e8 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 <mail@jasonwilder.com>
Date: Fri, 25 Sep 2015 13:30:24 -0600
Subject: [PATCH 32/68] Reduce memory allocations

Converting between different encoders is wasting a lot of memory allocating different
typed slices.
---
 tsdb/engine/pd1/int.go      | 84 ++++++++++++++++++++++++++++---------
 tsdb/engine/pd1/int_test.go | 10 +++--
 2 files changed, 71 insertions(+), 23 deletions(-)

diff --git a/tsdb/engine/pd1/int.go b/tsdb/engine/pd1/int.go
index c4c40b8778..706ca5fce8 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 struct {
-	values []int64
+type Int64Encoder interface {
+	Write(v int64)
+	Bytes() ([]byte, error)
 }
 
-func NewInt64Encoder() *int64Encoder {
+type Int64Decoder interface {
+	Next() bool
+	Read() int64
+}
+
+type int64Encoder struct {
+	values []uint64
+}
+
+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 82042f77bd..063ae05c07 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 <mail@jasonwilder.com>
Date: Fri, 25 Sep 2015 14:19:24 -0600
Subject: [PATCH 33/68] 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 9119dbd518..04de5b991f 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 <mail@jasonwilder.com>
Date: Fri, 25 Sep 2015 14:25:42 -0600
Subject: [PATCH 34/68] 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 da78b2b6e8..9eb3e0fe2c 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 <mail@jasonwilder.com>
Date: Fri, 25 Sep 2015 15:27:43 -0600
Subject: [PATCH 35/68] 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      | 102 ++++++++++++++++--------------------
 tsdb/engine/pd1/int_test.go |   4 ++
 2 files changed, 50 insertions(+), 56 deletions(-)

diff --git a/tsdb/engine/pd1/int.go b/tsdb/engine/pd1/int.go
index 706ca5fce8..3c6cdf9c6e 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
+
+	d.i += 1
+
+	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
 }
 
 func (d *int64Decoder) Read() int64 {
-	return d.v
+	return ZigZagDecode(d.values[d.i])
 }
 
-func (d *int64Decoder) decode(b []byte) {
-	if len(b) == 0 {
+func (d *int64Decoder) decodePacked() {
+	if len(d.bytes) == 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))
-	}
+	v := binary.BigEndian.Uint64(d.bytes[0:8])
+	n, _ := simple8b.DecodeSingle(d.values, v)
+
+	d.n = n
+	d.i = 0
+	d.bytes = d.bytes[8:]
 }
 
-func (d *int64Decoder) decodePacked(b []byte) {
-	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([]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 063ae05c07..7f1ba2eff7 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 <mail@jasonwilder.com>
Date: Mon, 28 Sep 2015 10:25:58 -0600
Subject: [PATCH 36/68] 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 a2e1669945..227fe77612 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 <mail@jasonwilder.com>
Date: Mon, 28 Sep 2015 12:53:27 -0600
Subject: [PATCH 37/68] 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 0000000000..065bd61c7f
--- /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 0000000000..ed642cff86
--- /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 227fe77612..d7abb195b5 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 eecodeBoolBlock(block []byte) ([]BoolValue, error) {
-	return nil, nil
+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 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 c249f1aa8c..aa5b67d640 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 3c6cdf9c6e..a7258e6d65 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 04de5b991f..6863a51afe 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 9eb3e0fe2c..e0dd235d62 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 <mail@jasonwilder.com>
Date: Mon, 28 Sep 2015 23:06:17 -0600
Subject: [PATCH 38/68] 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 a7258e6d65..c8f42341c9 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 <mail@jasonwilder.com>
Date: Tue, 29 Sep 2015 10:44:24 -0600
Subject: [PATCH 39/68] 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 e1abe87360..2a59c9ac4e 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 <mail@jasonwilder.com>
Date: Tue, 29 Sep 2015 10:46:19 -0600
Subject: [PATCH 40/68] 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 065bd61c7f..4fad26586b 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 c8f42341c9..98cf717f30 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 6863a51afe..88a2c2517b 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 <mail@jasonwilder.com>
Date: Tue, 29 Sep 2015 10:47:35 -0600
Subject: [PATCH 41/68] 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 5bfa377ac1..60702938f5 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 <paul@pauldix.net>
Date: Tue, 29 Sep 2015 16:17:58 -0400
Subject: [PATCH 42/68] 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 2a59c9ac4e..e3e80aee3f 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 49b72b4a60..f44937ac67 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 <paul@pauldix.net>
Date: Tue, 29 Sep 2015 20:03:37 -0400
Subject: [PATCH 43/68] 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 d707661fad..3834aad226 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 <paul@pauldix.net>
Date: Tue, 29 Sep 2015 21:51:16 -0400
Subject: [PATCH 44/68] 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 3834aad226..a6bd2113a1 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 <mail@jasonwilder.com>
Date: Tue, 29 Sep 2015 16:08:18 -0600
Subject: [PATCH 45/68] 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 d7abb195b5..005692d273 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 aa5b67d640..306ad15576 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 0000000000..e548b68c56
--- /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 0000000000..c457de8697
--- /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 <mail@jasonwilder.com>
Date: Tue, 29 Sep 2015 20:01:43 -0600
Subject: [PATCH 46/68] 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 306ad15576..7126eec388 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 <paul@pauldix.net>
Date: Wed, 30 Sep 2015 11:48:26 -0400
Subject: [PATCH 47/68] 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 a6bd2113a1..37e16c6d49 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 e3e80aee3f..9eda01fdc4 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 <mail@jasonwilder.com>
Date: Wed, 30 Sep 2015 09:48:20 -0600
Subject: [PATCH 48/68] 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 f44937ac67..6e51980f2b 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 <paul@pauldix.net>
Date: Wed, 30 Sep 2015 16:38:55 -0400
Subject: [PATCH 49/68] 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      |  60 +++++++------
 tsdb/engine/pd1/write_lock_test.go | 131 +++++++++++++++++++++++++++++
 5 files changed, 280 insertions(+), 41 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 37e16c6d49..5f64c6c2f5 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 9eda01fdc4..494c63524f 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 6e51980f2b..ef4399912c 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 5f48fb9311..a791b663ab 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()
+			}
 		}
+
+		// 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.ranges = append(w.ranges, r)
+			w.rangesLock.Unlock()
+			return
+		}
+
+		// try again
+		w.rangesLock.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) {
+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 0000000000..4cdc88e816
--- /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 <mail@jasonwilder.com>
Date: Wed, 30 Sep 2015 13:11:49 -0600
Subject: [PATCH 50/68] 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 88a2c2517b..b6fc6fef41 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 e0dd235d62..f7030db8a4 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 <mail@jasonwilder.com>
Date: Wed, 30 Sep 2015 13:12:53 -0600
Subject: [PATCH 51/68] 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 494c63524f..e11a9baf92 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 <paul@pauldix.net>
Date: Wed, 30 Sep 2015 20:41:50 -0400
Subject: [PATCH 52/68] Implement reverse cursor direction on pd1

---
 tsdb/engine/pd1/cursor.go   |  25 +--
 tsdb/engine/pd1/pd1.go      | 324 +++++++++++++++++++++++++-----------
 tsdb/engine/pd1/pd1_test.go | 120 ++++++++++++-
 tsdb/engine/pd1/wal.go      |   2 +-
 4 files changed, 360 insertions(+), 111 deletions(-)

diff --git a/tsdb/engine/pd1/cursor.go b/tsdb/engine/pd1/cursor.go
index d31252aa32..8319992c25 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 5f64c6c2f5..44f8325277 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])
+		// handle seek for correct order
+		k := tsdb.EOF
+		var v 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.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
-					}
-				}
-			}
-
-			// it must be in this block or not at all
-			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() >= seek {
-					c.vals = c.vals[i+1:]
-					return v.Time().UnixNano(), v.Value()
-				}
-			}
-
-			// not in this one, let the top loop look for it in the next file
-			break
+		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 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 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 c.filesPos < len(c.files) {
-			f := c.files[c.filesPos]
-
-			startingPos := f.StartingPositionForID(c.id)
-			if startingPos == 0 {
-				c.filesPos++
-				continue
-			}
-			c.f = f
-			return c.decodeBlockAndGetValues(startingPos)
-		}
-
-		// we didn't get to a file that had a next value
-		return tsdb.EOF, nil
+	if c.ascending {
+		return c.nextAscending()
 	}
-
-	v := c.vals[0]
-	c.vals = c.vals[1:]
-
-	return v.Time().UnixNano(), v.Value()
+	return c.nextDescending()
 }
 
-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) 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)
-	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 e11a9baf92..5c28572a84 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 ef4399912c..b6cc0cc214 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 <paul@pauldix.net>
Date: Thu, 1 Oct 2015 15:16:28 -0400
Subject: [PATCH 53/68] 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 8319992c25..4b49972131 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 44f8325277..c6a8e7c3c5 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 5c28572a84..02f1a57663 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 16aac8b5e4..89d62e9d88 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 b6cc0cc214..f253db33f5 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 034ad2dd3e..4718f4cebd 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 <paul@pauldix.net>
Date: Thu, 1 Oct 2015 15:23:38 -0400
Subject: [PATCH 54/68] 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 4fad26586b..8d9653d199 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 ed642cff86..ed68987afd 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 4b49972131..06fd0bbf8f 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 005692d273..4b6a112d56 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 7126eec388..c4889f18e5 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 dddb9f39b2..ff6a61c505 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 60702938f5..00b259bf95 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 98cf717f30..b178c50375 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 7f1ba2eff7..279b55e49b 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 e548b68c56..1b5dafac3f 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 c457de8697..8710a50b36 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 b6fc6fef41..59990f5cb4 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 f7030db8a4..dbb5a2341a 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 c6a8e7c3c5..d569d5a152 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 02f1a57663..e8c3a72467 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 89d62e9d88..d5f31110d1 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 f253db33f5..9cc2a2fd8f 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 4718f4cebd..dfc5cda8cc 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 a791b663ab..f4514e58aa 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 4cdc88e816..7fa17c530c 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 <paul@pauldix.net>
Date: Thu, 1 Oct 2015 15:30:13 -0400
Subject: [PATCH 55/68] 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 1e7e29a0d8..a329c1268b 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 407801d842..97ca51d9dd 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 03022f9d0e..6c8cb51e19 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 d569d5a152..b47413868d 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 1f606b613b..b258944316 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 <mail@jasonwilder.com>
Date: Thu, 1 Oct 2015 13:59:27 -0600
Subject: [PATCH 56/68] 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 1fb8560167..bee68c7fdd 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 <mail@jasonwilder.com>
Date: Fri, 2 Oct 2015 09:38:39 -0600
Subject: [PATCH 57/68] 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 a5266087d7..ce1151d3cf 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 <mail@jasonwilder.com>
Date: Fri, 2 Oct 2015 09:39:08 -0600
Subject: [PATCH 58/68] 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 e8c3a72467..6a39e41ed8 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 <mail@jasonwilder.com>
Date: Fri, 2 Oct 2015 10:03:20 -0600
Subject: [PATCH 59/68] 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 8d9653d199..ac08555cfe 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 4b6a112d56..eef2b1d23e 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 ff6a61c505..7ac1d93b49 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 00b259bf95..49e811a174 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 b178c50375..40e615657d 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 1b5dafac3f..a2025809de 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 8710a50b36..f1bf173815 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 59990f5cb4..8fcb0b4ce2 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 dbb5a2341a..402a6578a1 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 <mail@jasonwilder.com>
Date: Fri, 2 Oct 2015 10:46:58 -0600
Subject: [PATCH 60/68] 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 ac08555cfe..52ec8d849b 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 eef2b1d23e..0d95bf5ac2 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 c4889f18e5..309b947eb6 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 7ac1d93b49..22703258a3 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 49e811a174..794d62e5b7 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 40e615657d..0060e27994 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 a2025809de..da06bc5359 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 f1bf173815..f5143514ec 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 8fcb0b4ce2..8605988b29 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 b47413868d..a2ca218f49 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 <paul@pauldix.net>
Date: Fri, 2 Oct 2015 17:17:07 -0400
Subject: [PATCH 61/68] 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 a2ca218f49..bc00eea8d8 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 <mail@parisholley.com>
Date: Sun, 4 Oct 2015 11:40:14 -0700
Subject: [PATCH 62/68] 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 e709bfffb7..7faa6966e8 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 b86dd87510..b95ccb9ea1 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 <mail@jasonwilder.com>
Date: Mon, 5 Oct 2015 12:32:29 -0600
Subject: [PATCH 63/68] 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 52ec8d849b..83f570a2ef 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 0d95bf5ac2..3fd6d2b4bb 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 22703258a3..8961c70f4f 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 0060e27994..486f91a98d 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 8605988b29..ad7ed64419 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 <mail@jasonwilder.com>
Date: Mon, 5 Oct 2015 12:43:08 -0600
Subject: [PATCH 64/68] 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 3fd6d2b4bb..040a12de66 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 <mail@jasonwilder.com>
Date: Mon, 5 Oct 2015 12:52:19 -0600
Subject: [PATCH 65/68] 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 040a12de66..3de8858632 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 486f91a98d..9ce18fe96e 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 <paul@pauldix.net>
Date: Sun, 4 Oct 2015 15:43:18 -0400
Subject: [PATCH 66/68] 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 64253f0f57..13aa35a97c 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 2fb96e99df..b3c136e794 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 a329c1268b..115f895f13 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 bc00eea8d8..b75d894f65 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 <paul@pauldix.net>
Date: Mon, 5 Oct 2015 17:21:07 -0400
Subject: [PATCH 67/68] 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      |  90 +++++++++++++++++------
 tsdb/engine/tsm1/wal_test.go |   6 +-
 4 files changed, 176 insertions(+), 60 deletions(-)

diff --git a/tsdb/engine/tsm1/tsm1.go b/tsdb/engine/tsm1/tsm1.go
index b75d894f65..aac359c19d 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 d5f31110d1..54653c2872 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 9cc2a2fd8f..4f6607d02a 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) {
-	l.cacheLock.Lock()
-	defer l.cacheLock.Unlock()
-	delete(l.measurementFieldsCache, measurement)
-}
-
-func (l *Log) DeleteSeries(keys []string) error {
-	l.cacheLock.Lock()
-	for _, k := range keys {
-		delete(l.cache, k)
-	}
-	l.cacheLock.Unlock()
-
-	b, err := json.Marshal(keys)
+func (l *Log) DeleteMeasurement(measurement string, keys []string) error {
+	d := &deleteData{MeasurementName: measurement, Keys: keys}
+	err := l.writeDeleteEntry(d)
 	if err != nil {
 		return err
 	}
 
-	cb := snappy.Encode(nil, b)
+	l.deleteKeysFromCache(keys)
+	l.deleteMeasurementFromCache(measurement)
 
-	return l.writeToLog(deleteEntry, cb)
+	return nil
+}
+
+func (l *Log) deleteMeasurementFromCache(name string) {
+	l.cacheLock.Lock()
+	defer l.cacheLock.Unlock()
+	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)
+	}
+
+	// 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)
+		}
+	}
+	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 dfc5cda8cc..9df191c7a7 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 <paul@pauldix.net>
Date: Mon, 5 Oct 2015 19:57:49 -0400
Subject: [PATCH 68/68] 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 7faa6966e8..8ebc090940 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 97ca51d9dd..fb1b2108c5 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 06fd0bbf8f..01ac2e3c7a 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 aac359c19d..59a0f3fe05 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))
 }