2018-09-26 17:39:21 +00:00
|
|
|
package tsm1_test
|
|
|
|
|
|
|
|
import (
|
2019-03-04 19:48:11 +00:00
|
|
|
"context"
|
2018-09-26 17:39:21 +00:00
|
|
|
"fmt"
|
|
|
|
"io/ioutil"
|
|
|
|
"math"
|
|
|
|
"os"
|
|
|
|
"path/filepath"
|
|
|
|
"runtime"
|
|
|
|
"strings"
|
|
|
|
"sync"
|
|
|
|
"testing"
|
|
|
|
"time"
|
|
|
|
|
2019-04-18 00:02:38 +00:00
|
|
|
"github.com/influxdata/influxdb"
|
2019-01-08 00:37:16 +00:00
|
|
|
"github.com/influxdata/influxdb/logger"
|
|
|
|
"github.com/influxdata/influxdb/models"
|
|
|
|
"github.com/influxdata/influxdb/tsdb"
|
|
|
|
"github.com/influxdata/influxdb/tsdb/tsi1"
|
|
|
|
"github.com/influxdata/influxdb/tsdb/tsm1"
|
2018-09-26 17:39:21 +00:00
|
|
|
"github.com/influxdata/influxql"
|
|
|
|
)
|
|
|
|
|
|
|
|
// Test that series id set gets updated and returned appropriately.
|
|
|
|
func TestIndex_SeriesIDSet(t *testing.T) {
|
2018-11-04 13:58:03 +00:00
|
|
|
engine := MustOpenEngine()
|
|
|
|
defer engine.Close()
|
|
|
|
|
|
|
|
// Add some series.
|
|
|
|
engine.MustAddSeries("cpu", map[string]string{"host": "a", "region": "west"})
|
|
|
|
engine.MustAddSeries("cpu", map[string]string{"host": "b", "region": "west"})
|
|
|
|
engine.MustAddSeries("cpu", map[string]string{"host": "b"})
|
|
|
|
engine.MustAddSeries("gpu", nil)
|
|
|
|
engine.MustAddSeries("gpu", map[string]string{"host": "b"})
|
|
|
|
engine.MustAddSeries("mem", map[string]string{"host": "z"})
|
|
|
|
|
|
|
|
// Collect series IDs.
|
|
|
|
seriesIDMap := map[string]tsdb.SeriesID{}
|
|
|
|
var e tsdb.SeriesIDElem
|
|
|
|
var err error
|
|
|
|
|
|
|
|
itr := engine.sfile.SeriesIDIterator()
|
|
|
|
for e, err = itr.Next(); ; e, err = itr.Next() {
|
|
|
|
if err != nil {
|
|
|
|
t.Fatal(err)
|
|
|
|
} else if e.SeriesID.IsZero() {
|
|
|
|
break
|
2018-09-26 17:39:21 +00:00
|
|
|
}
|
|
|
|
|
2018-11-04 13:58:03 +00:00
|
|
|
name, tags := tsdb.ParseSeriesKey(engine.sfile.SeriesKey(e.SeriesID))
|
|
|
|
key := fmt.Sprintf("%s%s", name, tags.HashKey())
|
|
|
|
seriesIDMap[key] = e.SeriesID
|
|
|
|
}
|
2018-09-26 17:39:21 +00:00
|
|
|
|
2018-11-04 13:58:03 +00:00
|
|
|
for _, id := range seriesIDMap {
|
|
|
|
if !engine.SeriesIDSet().Contains(id) {
|
|
|
|
t.Fatalf("bitmap does not contain ID: %d", id)
|
2018-09-26 17:39:21 +00:00
|
|
|
}
|
2018-11-04 13:58:03 +00:00
|
|
|
}
|
2018-09-26 17:39:21 +00:00
|
|
|
|
2018-11-04 13:58:03 +00:00
|
|
|
// Drop all the series for the gpu measurement and they should no longer
|
2019-04-11 04:53:06 +00:00
|
|
|
// be in the series ID set.
|
|
|
|
if err := engine.DeletePrefixRange([]byte("gpu"), math.MinInt64, math.MaxInt64, nil); err != nil {
|
2018-11-04 13:58:03 +00:00
|
|
|
t.Fatal(err)
|
|
|
|
}
|
2018-09-26 17:39:21 +00:00
|
|
|
|
2018-11-04 13:58:03 +00:00
|
|
|
if engine.SeriesIDSet().Contains(seriesIDMap["gpu"]) {
|
|
|
|
t.Fatalf("bitmap does not contain ID: %d for key %s, but should", seriesIDMap["gpu"], "gpu")
|
|
|
|
} else if engine.SeriesIDSet().Contains(seriesIDMap["gpu,host=b"]) {
|
|
|
|
t.Fatalf("bitmap does not contain ID: %d for key %s, but should", seriesIDMap["gpu,host=b"], "gpu,host=b")
|
|
|
|
}
|
|
|
|
delete(seriesIDMap, "gpu")
|
|
|
|
delete(seriesIDMap, "gpu,host=b")
|
2018-09-26 17:39:21 +00:00
|
|
|
|
2018-11-04 13:58:03 +00:00
|
|
|
// The rest of the keys should still be in the set.
|
|
|
|
for key, id := range seriesIDMap {
|
|
|
|
if !engine.SeriesIDSet().Contains(id) {
|
|
|
|
t.Fatalf("bitmap does not contain ID: %d for key %s, but should", id, key)
|
2018-09-26 17:39:21 +00:00
|
|
|
}
|
2018-11-04 13:58:03 +00:00
|
|
|
}
|
2018-09-26 17:39:21 +00:00
|
|
|
|
2018-11-04 13:58:03 +00:00
|
|
|
// Reopen the engine, and the series should be re-added to the bitmap.
|
|
|
|
if err := engine.Reopen(); err != nil {
|
|
|
|
t.Fatal(err)
|
|
|
|
}
|
2018-09-26 17:39:21 +00:00
|
|
|
|
2018-11-04 13:58:03 +00:00
|
|
|
// Check bitset is expected.
|
|
|
|
expected := tsdb.NewSeriesIDSet()
|
|
|
|
for _, id := range seriesIDMap {
|
|
|
|
expected.Add(id)
|
2018-09-26 17:39:21 +00:00
|
|
|
}
|
|
|
|
|
2018-11-04 13:58:03 +00:00
|
|
|
if !engine.SeriesIDSet().Equals(expected) {
|
|
|
|
t.Fatalf("got bitset %s, expected %s", engine.SeriesIDSet().String(), expected.String())
|
2018-09-26 17:39:21 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
func TestEngine_SnapshotsDisabled(t *testing.T) {
|
|
|
|
sfile := MustOpenSeriesFile()
|
|
|
|
defer sfile.Close()
|
|
|
|
|
|
|
|
// Generate temporary file.
|
|
|
|
dir, _ := ioutil.TempDir("", "tsm")
|
|
|
|
defer os.RemoveAll(dir)
|
|
|
|
|
|
|
|
// Create a tsm1 engine.
|
2018-10-31 19:19:54 +00:00
|
|
|
idx := MustOpenIndex(filepath.Join(dir, "index"), tsdb.NewSeriesIDSet(), sfile.SeriesFile)
|
2018-09-26 17:39:21 +00:00
|
|
|
defer idx.Close()
|
|
|
|
|
2018-10-31 19:19:54 +00:00
|
|
|
config := tsm1.NewConfig()
|
2018-11-08 17:19:27 +00:00
|
|
|
e := tsm1.NewEngine(filepath.Join(dir, "data"), idx, config,
|
|
|
|
tsm1.WithCompactionPlanner(newMockPlanner()))
|
2018-09-26 17:39:21 +00:00
|
|
|
|
|
|
|
e.SetEnabled(false)
|
2019-03-04 19:48:11 +00:00
|
|
|
if err := e.Open(context.Background()); err != nil {
|
2018-09-26 17:39:21 +00:00
|
|
|
t.Fatalf("failed to open tsm1 engine: %s", err.Error())
|
|
|
|
}
|
storage: fix problems with keeping resources alive
This commit adds the pkg/lifecycle.Resource to help manage opening,
closing, and leasing out references to some resource. A resource
cannot be closed until all acquired references have been released.
If the debug_ref tag is enabled, all resource acquisitions keep
track of the stack trace that created them and have a finalizer
associated with them to print on stderr if they are leaked. It also
registers a handler on SIGUSR2 to dump all of the currently live
resources.
Having resources tracked in a uniform way with a data type allows us
to do more sophisticated tracking with the debug_ref tag, as well.
For example, we could panic the process if a resource cannot be
closed within a certain time frame, or attempt to figure out the
DAG of resource ownership dynamically.
This commit also fixes many issues around resources, correctness
during error scenarios, reporting of errors, idempotency of
close, tracking of memory for some data structures, resource leaks
in tests, and out of order dependency closes in tests.
2019-02-25 23:51:08 +00:00
|
|
|
defer e.Close()
|
2018-09-26 17:39:21 +00:00
|
|
|
|
|
|
|
// Make sure Snapshots are disabled.
|
|
|
|
e.SetCompactionsEnabled(false)
|
|
|
|
e.Compactor.DisableSnapshots()
|
|
|
|
|
|
|
|
// Writing a snapshot should not fail when the snapshot is empty
|
|
|
|
// even if snapshots are disabled.
|
2019-07-23 18:40:05 +00:00
|
|
|
if err := e.WriteSnapshot(context.Background(), tsm1.CacheStatusColdNoWrites); err != nil {
|
2018-09-26 17:39:21 +00:00
|
|
|
t.Fatalf("failed to snapshot: %s", err.Error())
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
func TestEngine_ShouldCompactCache(t *testing.T) {
|
|
|
|
nowTime := time.Now()
|
|
|
|
|
|
|
|
e, err := NewEngine()
|
|
|
|
if err != nil {
|
|
|
|
t.Fatal(err)
|
|
|
|
}
|
|
|
|
|
|
|
|
// mock the planner so compactions don't run during the test
|
|
|
|
e.CompactionPlan = &mockPlanner{}
|
|
|
|
e.SetEnabled(false)
|
2019-03-04 19:48:11 +00:00
|
|
|
if err := e.Open(context.Background()); err != nil {
|
2018-09-26 17:39:21 +00:00
|
|
|
t.Fatalf("failed to open tsm1 engine: %s", err.Error())
|
|
|
|
}
|
|
|
|
defer e.Close()
|
|
|
|
|
2019-03-22 12:51:35 +00:00
|
|
|
if got, exp := e.ShouldCompactCache(nowTime), tsm1.CacheStatusOkay; got != exp {
|
|
|
|
t.Fatalf("got status %v, exp status %v - nothing written to cache, so should not compact", got, exp)
|
2018-09-26 17:39:21 +00:00
|
|
|
}
|
|
|
|
|
2019-04-24 16:12:15 +00:00
|
|
|
if err := e.WritePointsString("mm", "m,k=v f=3i"); err != nil {
|
2018-09-26 17:39:21 +00:00
|
|
|
t.Fatal(err)
|
|
|
|
}
|
|
|
|
|
2019-03-22 12:51:35 +00:00
|
|
|
if got, exp := e.ShouldCompactCache(nowTime), tsm1.CacheStatusOkay; got != exp {
|
|
|
|
t.Fatalf("got status %v, exp status %v - cache size < flush threshold and nothing written to FileStore, so should not compact", got, exp)
|
2018-09-26 17:39:21 +00:00
|
|
|
}
|
|
|
|
|
2019-03-22 12:51:35 +00:00
|
|
|
if got, exp := e.ShouldCompactCache(nowTime.Add(time.Hour)), tsm1.CacheStatusColdNoWrites; got != exp {
|
|
|
|
t.Fatalf("got status %v, exp status %v - last compaction was longer than flush write cold threshold, so should compact", got, exp)
|
2018-09-26 17:39:21 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
e.CacheFlushMemorySizeThreshold = 1
|
2019-03-22 12:51:35 +00:00
|
|
|
if got, exp := e.ShouldCompactCache(nowTime), tsm1.CacheStatusSizeExceeded; got != exp {
|
|
|
|
t.Fatalf("got status %v, exp status %v - cache size > flush threshold, so should compact", got, exp)
|
2018-09-26 17:39:21 +00:00
|
|
|
}
|
2019-03-22 13:45:47 +00:00
|
|
|
|
|
|
|
e.CacheFlushMemorySizeThreshold = 1024 // Reset.
|
|
|
|
if got, exp := e.ShouldCompactCache(nowTime), tsm1.CacheStatusOkay; got != exp {
|
|
|
|
t.Fatalf("got status %v, exp status %v - nothing written to cache, so should not compact", got, exp)
|
|
|
|
}
|
|
|
|
|
|
|
|
e.CacheFlushAgeDurationThreshold = 100 * time.Millisecond
|
|
|
|
time.Sleep(250 * time.Millisecond)
|
|
|
|
if got, exp := e.ShouldCompactCache(nowTime), tsm1.CacheStatusAgeExceeded; got != exp {
|
|
|
|
t.Fatalf("got status %v, exp status %v - cache age > max age threshold, so should compact", got, exp)
|
|
|
|
}
|
2018-09-26 17:39:21 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
func makeBlockTypeSlice(n int) []byte {
|
|
|
|
r := make([]byte, n)
|
|
|
|
b := tsm1.BlockFloat64
|
|
|
|
m := tsm1.BlockUnsigned + 1
|
|
|
|
for i := 0; i < len(r); i++ {
|
|
|
|
r[i] = b % m
|
|
|
|
}
|
|
|
|
return r
|
|
|
|
}
|
|
|
|
|
|
|
|
var blockType = influxql.Unknown
|
|
|
|
|
|
|
|
func BenchmarkBlockTypeToInfluxQLDataType(b *testing.B) {
|
|
|
|
t := makeBlockTypeSlice(1000)
|
|
|
|
for i := 0; i < b.N; i++ {
|
|
|
|
for j := 0; j < len(t); j++ {
|
|
|
|
blockType = tsm1.BlockTypeToInfluxQLDataType(t[j])
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
// This test ensures that "sync: WaitGroup is reused before previous Wait has returned" is
|
|
|
|
// is not raised.
|
|
|
|
func TestEngine_DisableEnableCompactions_Concurrent(t *testing.T) {
|
2018-11-04 13:58:03 +00:00
|
|
|
e := MustOpenEngine()
|
|
|
|
defer e.Close()
|
2018-09-26 17:39:21 +00:00
|
|
|
|
2018-11-04 13:58:03 +00:00
|
|
|
var wg sync.WaitGroup
|
|
|
|
wg.Add(2)
|
2018-09-26 17:39:21 +00:00
|
|
|
|
2018-11-04 13:58:03 +00:00
|
|
|
go func() {
|
|
|
|
defer wg.Done()
|
|
|
|
for i := 0; i < 1000; i++ {
|
|
|
|
e.SetCompactionsEnabled(true)
|
|
|
|
e.SetCompactionsEnabled(false)
|
|
|
|
}
|
|
|
|
}()
|
2018-09-26 17:39:21 +00:00
|
|
|
|
2018-11-04 13:58:03 +00:00
|
|
|
go func() {
|
|
|
|
defer wg.Done()
|
|
|
|
for i := 0; i < 1000; i++ {
|
|
|
|
e.SetCompactionsEnabled(false)
|
|
|
|
e.SetCompactionsEnabled(true)
|
|
|
|
}
|
|
|
|
}()
|
|
|
|
|
|
|
|
done := make(chan struct{})
|
|
|
|
go func() {
|
|
|
|
wg.Wait()
|
|
|
|
close(done)
|
|
|
|
}()
|
|
|
|
|
|
|
|
// Wait for waitgroup or fail if it takes too long.
|
|
|
|
select {
|
|
|
|
case <-time.NewTimer(30 * time.Second).C:
|
|
|
|
t.Fatalf("timed out after 30 seconds waiting for waitgroup")
|
|
|
|
case <-done:
|
2018-09-26 17:39:21 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
func BenchmarkEngine_WritePoints(b *testing.B) {
|
|
|
|
batchSizes := []int{10, 100, 1000, 5000, 10000}
|
|
|
|
for _, sz := range batchSizes {
|
2018-11-04 13:58:03 +00:00
|
|
|
e := MustOpenEngine()
|
|
|
|
pp := make([]models.Point, 0, sz)
|
|
|
|
for i := 0; i < sz; i++ {
|
2019-04-24 16:12:15 +00:00
|
|
|
p := MustParsePointString(fmt.Sprintf("cpu,host=%d value=1.2", i), "mm")
|
2018-11-04 13:58:03 +00:00
|
|
|
pp = append(pp, p)
|
|
|
|
}
|
2018-09-26 17:39:21 +00:00
|
|
|
|
2018-11-04 13:58:03 +00:00
|
|
|
b.Run(fmt.Sprintf("%d", sz), func(b *testing.B) {
|
|
|
|
b.ReportAllocs()
|
|
|
|
for i := 0; i < b.N; i++ {
|
|
|
|
err := e.WritePoints(pp)
|
|
|
|
if err != nil {
|
|
|
|
b.Fatal(err)
|
2018-09-26 17:39:21 +00:00
|
|
|
}
|
2018-11-04 13:58:03 +00:00
|
|
|
}
|
|
|
|
})
|
|
|
|
e.Close()
|
2018-09-26 17:39:21 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
func BenchmarkEngine_WritePoints_Parallel(b *testing.B) {
|
|
|
|
batchSizes := []int{1000, 5000, 10000, 25000, 50000, 75000, 100000, 200000}
|
|
|
|
for _, sz := range batchSizes {
|
2018-11-04 13:58:03 +00:00
|
|
|
e := MustOpenEngine()
|
2018-09-26 17:39:21 +00:00
|
|
|
|
2018-11-04 13:58:03 +00:00
|
|
|
cpus := runtime.GOMAXPROCS(0)
|
|
|
|
pp := make([]models.Point, 0, sz*cpus)
|
|
|
|
for i := 0; i < sz*cpus; i++ {
|
2019-04-24 16:12:15 +00:00
|
|
|
p := MustParsePointString(fmt.Sprintf("cpu,host=%d value=1.2,other=%di", i, i), "mm")
|
2018-11-04 13:58:03 +00:00
|
|
|
pp = append(pp, p)
|
|
|
|
}
|
2018-09-26 17:39:21 +00:00
|
|
|
|
2018-11-04 13:58:03 +00:00
|
|
|
b.Run(fmt.Sprintf("%d", sz), func(b *testing.B) {
|
|
|
|
b.ReportAllocs()
|
|
|
|
for i := 0; i < b.N; i++ {
|
|
|
|
var wg sync.WaitGroup
|
|
|
|
errC := make(chan error)
|
|
|
|
for i := 0; i < cpus; i++ {
|
|
|
|
wg.Add(1)
|
|
|
|
go func(i int) {
|
|
|
|
defer wg.Done()
|
|
|
|
from, to := i*sz, (i+1)*sz
|
|
|
|
err := e.WritePoints(pp[from:to])
|
2018-09-26 17:39:21 +00:00
|
|
|
if err != nil {
|
2018-11-04 13:58:03 +00:00
|
|
|
errC <- err
|
|
|
|
return
|
2018-09-26 17:39:21 +00:00
|
|
|
}
|
2018-11-04 13:58:03 +00:00
|
|
|
}(i)
|
|
|
|
}
|
|
|
|
|
|
|
|
go func() {
|
|
|
|
wg.Wait()
|
|
|
|
close(errC)
|
|
|
|
}()
|
|
|
|
|
|
|
|
for err := range errC {
|
|
|
|
if err != nil {
|
|
|
|
b.Error(err)
|
2018-09-26 17:39:21 +00:00
|
|
|
}
|
|
|
|
}
|
2018-11-04 13:58:03 +00:00
|
|
|
}
|
|
|
|
})
|
|
|
|
e.Close()
|
2018-09-26 17:39:21 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
// Engine is a test wrapper for tsm1.Engine.
|
|
|
|
type Engine struct {
|
|
|
|
*tsm1.Engine
|
|
|
|
root string
|
|
|
|
indexPath string
|
2018-10-23 09:49:15 +00:00
|
|
|
index *tsi1.Index
|
2018-09-26 17:39:21 +00:00
|
|
|
sfile *tsdb.SeriesFile
|
|
|
|
}
|
|
|
|
|
|
|
|
// NewEngine returns a new instance of Engine at a temporary location.
|
|
|
|
func NewEngine() (*Engine, error) {
|
|
|
|
root, err := ioutil.TempDir("", "tsm1-")
|
|
|
|
if err != nil {
|
|
|
|
panic(err)
|
|
|
|
}
|
|
|
|
|
|
|
|
// Setup series file.
|
2018-11-07 18:37:53 +00:00
|
|
|
sfile := tsdb.NewSeriesFile(filepath.Join(root, "_series"))
|
2018-09-26 17:39:21 +00:00
|
|
|
sfile.Logger = logger.New(os.Stdout)
|
2019-03-04 19:48:11 +00:00
|
|
|
if err = sfile.Open(context.Background()); err != nil {
|
2018-09-26 17:39:21 +00:00
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
|
2018-10-31 19:19:54 +00:00
|
|
|
idxPath := filepath.Join(root, "index")
|
|
|
|
idx := MustOpenIndex(idxPath, tsdb.NewSeriesIDSet(), sfile)
|
2018-09-26 17:39:21 +00:00
|
|
|
|
2018-10-31 19:19:54 +00:00
|
|
|
config := tsm1.NewConfig()
|
2018-11-08 17:19:27 +00:00
|
|
|
tsm1Engine := tsm1.NewEngine(filepath.Join(root, "data"), idx, config,
|
|
|
|
tsm1.WithCompactionPlanner(newMockPlanner()))
|
2018-09-26 17:39:21 +00:00
|
|
|
|
|
|
|
return &Engine{
|
|
|
|
Engine: tsm1Engine,
|
|
|
|
root: root,
|
|
|
|
indexPath: idxPath,
|
|
|
|
index: idx,
|
|
|
|
sfile: sfile,
|
|
|
|
}, nil
|
|
|
|
}
|
|
|
|
|
|
|
|
// MustOpenEngine returns a new, open instance of Engine.
|
|
|
|
func MustOpenEngine() *Engine {
|
|
|
|
e, err := NewEngine()
|
|
|
|
if err != nil {
|
|
|
|
panic(err)
|
|
|
|
}
|
|
|
|
|
2019-03-04 19:48:11 +00:00
|
|
|
if err := e.Open(context.Background()); err != nil {
|
2018-09-26 17:39:21 +00:00
|
|
|
panic(err)
|
|
|
|
}
|
|
|
|
return e
|
|
|
|
}
|
|
|
|
|
|
|
|
// Close closes the engine and removes all underlying data.
|
|
|
|
func (e *Engine) Close() error {
|
|
|
|
return e.close(true)
|
|
|
|
}
|
|
|
|
|
|
|
|
func (e *Engine) close(cleanup bool) error {
|
storage: fix problems with keeping resources alive
This commit adds the pkg/lifecycle.Resource to help manage opening,
closing, and leasing out references to some resource. A resource
cannot be closed until all acquired references have been released.
If the debug_ref tag is enabled, all resource acquisitions keep
track of the stack trace that created them and have a finalizer
associated with them to print on stderr if they are leaked. It also
registers a handler on SIGUSR2 to dump all of the currently live
resources.
Having resources tracked in a uniform way with a data type allows us
to do more sophisticated tracking with the debug_ref tag, as well.
For example, we could panic the process if a resource cannot be
closed within a certain time frame, or attempt to figure out the
DAG of resource ownership dynamically.
This commit also fixes many issues around resources, correctness
during error scenarios, reporting of errors, idempotency of
close, tracking of memory for some data structures, resource leaks
in tests, and out of order dependency closes in tests.
2019-02-25 23:51:08 +00:00
|
|
|
err := e.Engine.Close()
|
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
|
2018-09-26 17:39:21 +00:00
|
|
|
if e.index != nil {
|
|
|
|
e.index.Close()
|
|
|
|
}
|
|
|
|
|
|
|
|
if e.sfile != nil {
|
|
|
|
e.sfile.Close()
|
|
|
|
}
|
|
|
|
|
storage: fix problems with keeping resources alive
This commit adds the pkg/lifecycle.Resource to help manage opening,
closing, and leasing out references to some resource. A resource
cannot be closed until all acquired references have been released.
If the debug_ref tag is enabled, all resource acquisitions keep
track of the stack trace that created them and have a finalizer
associated with them to print on stderr if they are leaked. It also
registers a handler on SIGUSR2 to dump all of the currently live
resources.
Having resources tracked in a uniform way with a data type allows us
to do more sophisticated tracking with the debug_ref tag, as well.
For example, we could panic the process if a resource cannot be
closed within a certain time frame, or attempt to figure out the
DAG of resource ownership dynamically.
This commit also fixes many issues around resources, correctness
during error scenarios, reporting of errors, idempotency of
close, tracking of memory for some data structures, resource leaks
in tests, and out of order dependency closes in tests.
2019-02-25 23:51:08 +00:00
|
|
|
if cleanup {
|
|
|
|
os.RemoveAll(e.root)
|
|
|
|
}
|
|
|
|
|
|
|
|
return nil
|
2018-09-26 17:39:21 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
// Reopen closes and reopens the engine.
|
|
|
|
func (e *Engine) Reopen() error {
|
|
|
|
// Close engine without removing underlying engine data.
|
|
|
|
if err := e.close(false); err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
|
|
|
|
// Re-open series file. Must create a new series file using the same data.
|
|
|
|
e.sfile = tsdb.NewSeriesFile(e.sfile.Path())
|
2019-03-04 19:48:11 +00:00
|
|
|
if err := e.sfile.Open(context.Background()); err != nil {
|
2018-09-26 17:39:21 +00:00
|
|
|
return err
|
|
|
|
}
|
|
|
|
|
|
|
|
// Re-open index.
|
2018-10-31 19:19:54 +00:00
|
|
|
e.index = MustOpenIndex(e.indexPath, tsdb.NewSeriesIDSet(), e.sfile)
|
2018-09-26 17:39:21 +00:00
|
|
|
|
|
|
|
// Re-initialize engine.
|
2018-10-31 19:19:54 +00:00
|
|
|
config := tsm1.NewConfig()
|
2018-11-08 17:19:27 +00:00
|
|
|
e.Engine = tsm1.NewEngine(filepath.Join(e.root, "data"), e.index, config,
|
|
|
|
tsm1.WithCompactionPlanner(newMockPlanner()))
|
2018-09-26 17:39:21 +00:00
|
|
|
|
|
|
|
// Reopen engine
|
2019-03-04 19:48:11 +00:00
|
|
|
if err := e.Engine.Open(context.Background()); err != nil {
|
2018-09-26 17:39:21 +00:00
|
|
|
return err
|
|
|
|
}
|
|
|
|
|
|
|
|
// Reload series data into index (no-op on TSI).
|
2018-10-08 22:22:59 +00:00
|
|
|
return nil
|
2018-09-26 17:39:21 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
// SeriesIDSet provides access to the underlying series id bitset in the engine's
|
|
|
|
// index. It will panic if the underlying index does not have a SeriesIDSet
|
|
|
|
// method.
|
|
|
|
func (e *Engine) SeriesIDSet() *tsdb.SeriesIDSet {
|
|
|
|
return e.index.SeriesIDSet()
|
|
|
|
}
|
|
|
|
|
|
|
|
// AddSeries adds the provided series data to the index and writes a point to
|
|
|
|
// the engine with default values for a field and a time of now.
|
|
|
|
func (e *Engine) AddSeries(name string, tags map[string]string) error {
|
|
|
|
point, err := models.NewPoint(name, models.NewTags(tags), models.Fields{"v": 1.0}, time.Now())
|
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
return e.writePoints(point)
|
|
|
|
}
|
|
|
|
|
|
|
|
// WritePointsString calls WritePointsString on the underlying engine, but also
|
|
|
|
// adds the associated series to the index.
|
2019-04-24 16:12:15 +00:00
|
|
|
func (e *Engine) WritePointsString(mm string, ptstr ...string) error {
|
|
|
|
points, err := models.ParsePointsString(strings.Join(ptstr, "\n"), mm)
|
2018-09-26 17:39:21 +00:00
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
return e.writePoints(points...)
|
|
|
|
}
|
|
|
|
|
|
|
|
// writePoints adds the series for the provided points to the index, and writes
|
|
|
|
// the point data to the engine.
|
|
|
|
func (e *Engine) writePoints(points ...models.Point) error {
|
2018-11-09 04:00:43 +00:00
|
|
|
// Write into the index.
|
|
|
|
collection := tsdb.NewSeriesCollection(points)
|
2019-01-21 20:09:15 +00:00
|
|
|
if err := e.index.CreateSeriesListIfNotExists(collection); err != nil {
|
2018-11-09 04:00:43 +00:00
|
|
|
return err
|
2018-09-26 17:39:21 +00:00
|
|
|
}
|
|
|
|
// Write the points into the cache/wal.
|
|
|
|
return e.WritePoints(points)
|
|
|
|
}
|
|
|
|
|
|
|
|
// MustAddSeries calls AddSeries, panicking if there is an error.
|
|
|
|
func (e *Engine) MustAddSeries(name string, tags map[string]string) {
|
|
|
|
if err := e.AddSeries(name, tags); err != nil {
|
|
|
|
panic(err)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
// MustWriteSnapshot forces a snapshot of the engine. Panic on error.
|
|
|
|
func (e *Engine) MustWriteSnapshot() {
|
2019-07-23 18:40:05 +00:00
|
|
|
if err := e.WriteSnapshot(context.Background(), tsm1.CacheStatusColdNoWrites); err != nil {
|
2018-09-26 17:39:21 +00:00
|
|
|
panic(err)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2019-04-18 00:02:38 +00:00
|
|
|
// MustWritePointsString parses and writes the specified points to the
|
|
|
|
// provided org and bucket. Panic on error.
|
|
|
|
func (e *Engine) MustWritePointsString(org, bucket influxdb.ID, buf string) {
|
|
|
|
err := e.writePoints(MustParseExplodePoints(org, bucket, buf)...)
|
|
|
|
if err != nil {
|
|
|
|
panic(err)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2019-04-23 04:43:42 +00:00
|
|
|
// MustDeleteBucketRange calls DeletePrefixRange using the org and bucket for
|
|
|
|
// the prefix. Panic on error.
|
2019-04-18 00:02:38 +00:00
|
|
|
func (e *Engine) MustDeleteBucketRange(orgID, bucketID influxdb.ID, min, max int64) {
|
|
|
|
// TODO(edd): we need to clean up how we're encoding the prefix so that we
|
|
|
|
// don't have to remember to get it right everywhere we need to touch TSM data.
|
|
|
|
encoded := tsdb.EncodeName(orgID, bucketID)
|
|
|
|
name := models.EscapeMeasurement(encoded[:])
|
|
|
|
|
2019-04-11 04:53:06 +00:00
|
|
|
err := e.DeletePrefixRange(name, min, max, nil)
|
2019-04-18 00:02:38 +00:00
|
|
|
if err != nil {
|
|
|
|
panic(err)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2018-10-31 19:19:54 +00:00
|
|
|
func MustOpenIndex(path string, seriesIDSet *tsdb.SeriesIDSet, sfile *tsdb.SeriesFile) *tsi1.Index {
|
|
|
|
idx := tsi1.NewIndex(sfile, tsi1.NewConfig(), tsi1.WithPath(path))
|
2019-03-04 19:48:11 +00:00
|
|
|
if err := idx.Open(context.Background()); err != nil {
|
2018-10-02 15:06:37 +00:00
|
|
|
panic(err)
|
|
|
|
}
|
|
|
|
return idx
|
|
|
|
}
|
|
|
|
|
2018-09-26 17:39:21 +00:00
|
|
|
// SeriesFile is a test wrapper for tsdb.SeriesFile.
|
|
|
|
type SeriesFile struct {
|
|
|
|
*tsdb.SeriesFile
|
|
|
|
}
|
|
|
|
|
|
|
|
// NewSeriesFile returns a new instance of SeriesFile with a temporary file path.
|
|
|
|
func NewSeriesFile() *SeriesFile {
|
|
|
|
dir, err := ioutil.TempDir("", "tsdb-series-file-")
|
|
|
|
if err != nil {
|
|
|
|
panic(err)
|
|
|
|
}
|
|
|
|
return &SeriesFile{SeriesFile: tsdb.NewSeriesFile(dir)}
|
|
|
|
}
|
|
|
|
|
|
|
|
// MustOpenSeriesFile returns a new, open instance of SeriesFile. Panic on error.
|
|
|
|
func MustOpenSeriesFile() *SeriesFile {
|
|
|
|
f := NewSeriesFile()
|
2019-03-04 19:48:11 +00:00
|
|
|
if err := f.Open(context.Background()); err != nil {
|
2018-09-26 17:39:21 +00:00
|
|
|
panic(err)
|
|
|
|
}
|
|
|
|
return f
|
|
|
|
}
|
|
|
|
|
|
|
|
// Close closes the log file and removes it from disk.
|
|
|
|
func (f *SeriesFile) Close() {
|
|
|
|
defer os.RemoveAll(f.Path())
|
|
|
|
if err := f.SeriesFile.Close(); err != nil {
|
|
|
|
panic(err)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
// MustParsePointsString parses points from a string. Panic on error.
|
2019-04-24 16:12:15 +00:00
|
|
|
func MustParsePointsString(buf, mm string) []models.Point {
|
|
|
|
a, err := models.ParsePointsString(buf, mm)
|
2018-09-26 17:39:21 +00:00
|
|
|
if err != nil {
|
|
|
|
panic(err)
|
|
|
|
}
|
|
|
|
return a
|
|
|
|
}
|
|
|
|
|
2019-04-18 00:02:38 +00:00
|
|
|
// MustParseExplodePoints parses points from a string and transforms using
|
|
|
|
// ExplodePoints using the provided org and bucket. Panic on error.
|
|
|
|
func MustParseExplodePoints(org, bucket influxdb.ID, buf string) []models.Point {
|
2019-04-24 16:12:15 +00:00
|
|
|
encoded := tsdb.EncodeName(org, bucket)
|
|
|
|
name := models.EscapeMeasurement(encoded[:])
|
|
|
|
return MustParsePointsString(buf, string(name))
|
2019-04-18 00:02:38 +00:00
|
|
|
}
|
|
|
|
|
2018-09-26 17:39:21 +00:00
|
|
|
// MustParsePointString parses the first point from a string. Panic on error.
|
2019-04-24 16:12:15 +00:00
|
|
|
func MustParsePointString(buf, mm string) models.Point { return MustParsePointsString(buf, mm)[0] }
|
2018-09-26 17:39:21 +00:00
|
|
|
|
|
|
|
type mockPlanner struct{}
|
|
|
|
|
2018-11-08 17:19:27 +00:00
|
|
|
func newMockPlanner() tsm1.CompactionPlanner {
|
2018-10-31 19:19:54 +00:00
|
|
|
return &mockPlanner{}
|
|
|
|
}
|
|
|
|
|
2018-09-26 17:39:21 +00:00
|
|
|
func (m *mockPlanner) Plan(lastWrite time.Time) []tsm1.CompactionGroup { return nil }
|
|
|
|
func (m *mockPlanner) PlanLevel(level int) []tsm1.CompactionGroup { return nil }
|
|
|
|
func (m *mockPlanner) PlanOptimize() []tsm1.CompactionGroup { return nil }
|
|
|
|
func (m *mockPlanner) Release(groups []tsm1.CompactionGroup) {}
|
|
|
|
func (m *mockPlanner) FullyCompacted() bool { return false }
|
|
|
|
func (m *mockPlanner) ForceFull() {}
|
|
|
|
func (m *mockPlanner) SetFileStore(fs *tsm1.FileStore) {}
|