Redfine append interface, remove old Prometheus storage from bench

This commit is contained in:
Fabian Reinartz 2016-12-21 00:02:37 +01:00
parent cddc29fa17
commit ee217adc7e
3 changed files with 68 additions and 191 deletions

View File

@ -1,16 +1,9 @@
build: build:
@go build . @go build .
bench_default: build bench: build
@echo ">> running benchmark" @echo ">> running benchmark"
@./tsdb bench write --out=benchout/default --engine=default --metrics=$(NUM_METRICS) testdata.1m @./tsdb bench write --out=benchout/ --metrics=$(NUM_METRICS) testdata.100k
@go tool pprof -svg ./tsdb benchout/default/cpu.prof > benchout/default/cpuprof.svg @go tool pprof -svg ./tsdb benchout/cpu.prof > benchout/cpuprof.svg
@go tool pprof -svg ./tsdb benchout/default/mem.prof > benchout/default/memprof.svg @go tool pprof -svg ./tsdb benchout/mem.prof > benchout/memprof.svg
@go tool pprof -svg ./tsdb benchout/default/block.prof > benchout/default/blockprof.svg @go tool pprof -svg ./tsdb benchout/block.prof > benchout/blockprof.svg
bench_tsdb: build
@echo ">> running benchmark"
@./tsdb bench write --out=benchout/tsdb --engine=tsdb --metrics=$(NUM_METRICS) testdata.1m
@go tool pprof -svg ./tsdb benchout/tsdb/cpu.prof > benchout/tsdb/cpuprof.svg
@go tool pprof -svg ./tsdb benchout/tsdb/mem.prof > benchout/tsdb/memprof.svg
@go tool pprof -svg ./tsdb benchout/tsdb/block.prof > benchout/tsdb/blockprof.svg

View File

@ -17,7 +17,6 @@ import (
dto "github.com/prometheus/client_model/go" dto "github.com/prometheus/client_model/go"
"github.com/prometheus/common/expfmt" "github.com/prometheus/common/expfmt"
"github.com/prometheus/common/model" "github.com/prometheus/common/model"
"github.com/prometheus/prometheus/storage/local"
"github.com/spf13/cobra" "github.com/spf13/cobra"
) )
@ -49,10 +48,9 @@ func NewBenchCommand() *cobra.Command {
type writeBenchmark struct { type writeBenchmark struct {
outPath string outPath string
cleanup bool cleanup bool
engine string
numMetrics int numMetrics int
storage benchmarkStorage storage *tsdb.DB
cpuprof *os.File cpuprof *os.File
memprof *os.File memprof *os.File
@ -66,7 +64,6 @@ func NewBenchWriteCommand() *cobra.Command {
Short: "run a write performance benchmark", Short: "run a write performance benchmark",
Run: wb.run, Run: wb.run,
} }
c.PersistentFlags().StringVar(&wb.engine, "engine", "tsdb", "the storage engine to use")
c.PersistentFlags().StringVar(&wb.outPath, "out", "benchout/", "set the output path") c.PersistentFlags().StringVar(&wb.outPath, "out", "benchout/", "set the output path")
c.PersistentFlags().IntVar(&wb.numMetrics, "metrics", 10000, "number of metrics to read") c.PersistentFlags().IntVar(&wb.numMetrics, "metrics", 10000, "number of metrics to read")
return c return c
@ -93,22 +90,12 @@ func (b *writeBenchmark) run(cmd *cobra.Command, args []string) {
dir := filepath.Join(b.outPath, "storage") dir := filepath.Join(b.outPath, "storage")
switch b.engine { st, err := tsdb.Open(dir, nil, nil)
case "tsdb": if err != nil {
st, err := newTSDBStorage(dir) exitWithError(err)
if err != nil {
exitWithError(err)
}
b.storage = st
case "default":
st, err := newDefaultStorage(dir)
if err != nil {
exitWithError(err)
}
b.storage = st
default:
exitWithError(fmt.Errorf("unknown storage engine %q", b.engine))
} }
b.storage = st
var metrics []model.Metric var metrics []model.Metric
measureTime("readData", func() { measureTime("readData", func() {
@ -138,7 +125,7 @@ func (b *writeBenchmark) run(cmd *cobra.Command, args []string) {
} }
}) })
measureTime("stopStorage", func() { measureTime("stopStorage", func() {
if err := b.storage.stop(); err != nil { if err := b.storage.Close(); err != nil {
exitWithError(err) exitWithError(err)
} }
b.stopProfiling() b.stopProfiling()
@ -171,7 +158,7 @@ func (b *writeBenchmark) ingestScrapes(metrics []model.Metric, scrapeCount int)
} }
func (b *writeBenchmark) ingestScrapesShard(metrics []model.Metric, scrapeCount int) error { func (b *writeBenchmark) ingestScrapesShard(metrics []model.Metric, scrapeCount int) error {
var sc tsdb.Vector app := b.storage.Appender()
ts := int64(model.Now()) ts := int64(model.Now())
type sample struct { type sample struct {
@ -196,82 +183,18 @@ func (b *writeBenchmark) ingestScrapesShard(metrics []model.Metric, scrapeCount
for i := 0; i < scrapeCount; i++ { for i := 0; i < scrapeCount; i++ {
ts += int64(30000) ts += int64(30000)
sc.Reset()
for _, s := range scrape { for _, s := range scrape {
s.value += 1000 s.value += 1000
sc.Add(s.labels, float64(s.value)) app.Add(s.labels, ts, float64(s.value))
} }
if err := b.storage.ingestScrape(ts, &sc); err != nil { if err := app.Commit(); err != nil {
return err return err
} }
} }
return nil return nil
} }
type benchmarkStorage interface {
ingestScrape(int64, *tsdb.Vector) error
stop() error
}
type tsdbStorage struct {
c *tsdb.DB
}
func (c *tsdbStorage) stop() error {
return c.c.Close()
}
func (c *tsdbStorage) ingestScrape(ts int64, s *tsdb.Vector) error {
return c.c.AppendVector(ts, s)
}
func newTSDBStorage(path string) (*tsdbStorage, error) {
c, err := tsdb.Open(path, nil, nil)
if err != nil {
return nil, err
}
return &tsdbStorage{
c: c,
}, nil
}
type defaultStorage struct {
s local.Storage
}
func (s *defaultStorage) ingestScrape(ts int64, scrape *tsdb.Vector) error {
for _, samples := range scrape.Buckets {
for _, smpl := range samples {
met := make(model.Metric, len(smpl.Labels))
for _, l := range smpl.Labels {
met[model.LabelName(l.Name)] = model.LabelValue(l.Value)
}
if err := s.s.Append(&model.Sample{
Metric: met,
Timestamp: model.Time(ts),
Value: model.SampleValue(smpl.Value),
}); err != nil {
return err
}
}
}
return nil
}
func (s *defaultStorage) stop() error {
return s.s.Stop()
}
func newDefaultStorage(path string) (*defaultStorage, error) {
s := local.NewMemorySeriesStorage(&local.MemorySeriesStorageOptions{
PersistenceStoragePath: path,
SyncStrategy: local.Adaptive,
CheckpointInterval: 5 * time.Minute,
})
return &defaultStorage{s: s}, s.Start()
}
func (b *writeBenchmark) startProfiling() { func (b *writeBenchmark) startProfiling() {
var err error var err error

145
db.go
View File

@ -42,7 +42,7 @@ type DB struct {
// TODO(fabxc): make configurable // TODO(fabxc): make configurable
const ( const (
shardShift = 0 shardShift = 3
numShards = 1 << shardShift numShards = 1 << shardShift
maxChunkSize = 1024 maxChunkSize = 1024
) )
@ -103,99 +103,70 @@ func (db *DB) Close() error {
return g.Wait() return g.Wait()
} }
// Appender adds a batch of samples. // Appender allows committing batches of samples to a database.
// The data held by the appender is reset after Commit returns.
type Appender interface { type Appender interface {
// Add adds a sample pair to the appended batch. // AddSeries registers a new known series label set with the appender
Add(l Labels, t int64, v float64) // and returns a reference number used to add samples to it over the
// life time of the Appender.
// AddSeries(Labels) uint64
// Commit submits the collected samples. // Add adds a sample pair for the referenced series.
Add(lset Labels, t int64, v float64)
// Commit submits the collected samples and purges the batch.
Commit() error Commit() error
} }
// Vector is a set of LabelSet associated with one value each. // Appender returns a new appender against the database.
// Label sets and values must have equal length. func (db *DB) Appender() Appender {
type Vector struct { return &bucketAppender{
Buckets map[uint16][]Sample db: db,
reused int buckets: make([][]hashedSample, numShards),
}
type Sample struct {
Hash uint64
Labels Labels
Value float64
}
// Reset the vector but keep resources allocated.
func (v *Vector) Reset() {
// Do a full reset every n-th reusage to avoid memory leaks.
if v.Buckets == nil || v.reused > 100 {
v.Buckets = make(map[uint16][]Sample, 0)
return
} }
for x, bkt := range v.Buckets {
v.Buckets[x] = bkt[:0]
}
v.reused++
} }
// Add a sample to the vector. type bucketAppender struct {
func (v *Vector) Add(lset Labels, val float64) { db *DB
buckets [][]hashedSample
}
func (ba *bucketAppender) Add(lset Labels, t int64, v float64) {
h := lset.Hash() h := lset.Hash()
s := uint16(h >> (64 - shardShift)) s := h >> (64 - shardShift)
v.Buckets[s] = append(v.Buckets[s], Sample{ ba.buckets[s] = append(ba.buckets[s], hashedSample{
Hash: h, hash: h,
Labels: lset, labels: lset,
Value: val, t: t,
v: v,
}) })
} }
// func (db *DB) Appender() Appender { func (ba *bucketAppender) reset() {
// return &bucketAppender{ for i := range ba.buckets {
// samples: make([]Sample, 1024), ba.buckets[i] = ba.buckets[i][:0]
// }
// }
// type bucketAppender struct {
// db *DB
// // buckets []Sam
// }
// func (a *bucketAppender) Add(l Labels, t int64, v float64) {
// }
// func (a *bucketAppender) Commit() error {
// // f
// }
// AppendVector adds values for a list of label sets for the given timestamp
// in milliseconds.
func (db *DB) AppendVector(ts int64, v *Vector) error {
// Sequentially add samples to shards.
for s, bkt := range v.Buckets {
shard := db.shards[s]
if err := shard.appendBatch(ts, bkt); err != nil {
// TODO(fabxc): handle gracefully and collect multi-error.
return err
}
} }
return nil
} }
func (db *DB) AppendSingle(lset Labels, ts int64, v float64) error { func (ba *bucketAppender) Commit() error {
sort.Sort(lset) defer ba.reset()
h := lset.Hash()
s := uint16(h >> (64 - shardShift))
return db.shards[s].appendBatch(ts, []Sample{ var merr MultiError
{
Hash: h, // Spill buckets into shards.
Labels: lset, for s, b := range ba.buckets {
Value: v, merr.Add(ba.db.shards[s].appendBatch(b))
}, }
}) return merr.Err()
}
type hashedSample struct {
hash uint64
labels Labels
t int64
v float64
} }
const sep = '\xff' const sep = '\xff'
@ -253,22 +224,14 @@ func (s *Shard) Close() error {
return e.Err() return e.Err()
} }
func (s *Shard) appendBatch(ts int64, samples []Sample) error { func (s *Shard) appendBatch(samples []hashedSample) error {
// TODO(fabxc): make configurable.
const persistenceTimeThreshold = 1000 * 60 * 60 // 1 hour if timestamp in ms
s.mtx.Lock() s.mtx.Lock()
defer s.mtx.Unlock() defer s.mtx.Unlock()
for _, smpl := range samples { var merr MultiError
if err := s.head.append(smpl.Hash, smpl.Labels, ts, smpl.Value); err != nil {
// TODO(fabxc): handle gracefully and collect multi-error.
return err
}
}
if ts > s.head.stats.MaxTime { for _, sm := range samples {
s.head.stats.MaxTime = ts merr.Add(s.head.append(sm.hash, sm.labels, sm.t, sm.v))
} }
// TODO(fabxc): randomize over time // TODO(fabxc): randomize over time
@ -284,7 +247,7 @@ func (s *Shard) appendBatch(ts int64, samples []Sample) error {
} }
} }
return nil return merr.Err()
} }
func intervalOverlap(amin, amax, bmin, bmax int64) bool { func intervalOverlap(amin, amax, bmin, bmax int64) bool {
@ -320,8 +283,6 @@ func (s *Shard) blocksForInterval(mint, maxt int64) []block {
bs = append(bs, s.head) bs = append(bs, s.head)
} }
fmt.Println("blocks for interval", bs)
return bs return bs
} }