influxdb/tsdb/tsm1/cache.go

718 lines
20 KiB
Go
Raw Normal View History

2018-09-26 17:39:21 +00:00
package tsm1
import (
"context"
2018-09-26 17:39:21 +00:00
"fmt"
"math"
perf(storage): reduce allocations when deleting from cache When deleting from the cache, each cache key must be checked to determine if it matches the prefix we're deleting. Since the keys are stored as strings in the cache (map keys) there were a lot of allocations happening because `applySerial` expects `[]byte` keys. It's beneficial to reduce allocations by refacting `applySerial` to work on strings. Whilst some allocations now have to happen the other way (string -> []byte), they only happen if we actually need to delete the key from the cache. Most of the keys don't get deleted so it's better doing it this way. Performance on the benchmark from the previous commit improved by ~40-50%. name old time/op new time/op delta Engine_DeletePrefixRange_Cache/exists-24 102ms ±11% 59ms ± 3% -41.95% (p=0.000 n=10+8) Engine_DeletePrefixRange_Cache/not_exists-24 97.1ms ± 4% 45.0ms ± 1% -53.66% (p=0.000 n=10+10) name old alloc/op new alloc/op delta Engine_DeletePrefixRange_Cache/exists-24 25.5MB ± 1% 3.1MB ± 2% -87.83% (p=0.000 n=10+10) Engine_DeletePrefixRange_Cache/not_exists-24 23.9MB ± 1% 0.1MB ±86% -99.65% (p=0.000 n=10+10) name old allocs/op new allocs/op delta Engine_DeletePrefixRange_Cache/exists-24 305k ± 1% 28k ± 1% -90.77% (p=0.000 n=10+10) Engine_DeletePrefixRange_Cache/not_exists-24 299k ± 1% 1k ±63% -99.74% (p=0.000 n=9+10) Raw benchmarks on a 24T/32GB/NVME machine are as follows: goos: linux goarch: amd64 pkg: github.com/influxdata/influxdb/tsdb/tsm1 BenchmarkEngine_DeletePrefixRange_Cache/exists-24 300 50379720 ns/op 3054106 B/op 27859 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/exists-24 300 57326032 ns/op 3124764 B/op 28217 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/exists-24 300 58943855 ns/op 3162146 B/op 28527 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/exists-24 300 60565115 ns/op 3138811 B/op 28176 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/exists-24 200 59775969 ns/op 3087910 B/op 27921 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/exists-24 300 59530451 ns/op 3120986 B/op 28207 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/exists-24 300 59185532 ns/op 3113066 B/op 28302 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/exists-24 300 59295867 ns/op 3100832 B/op 28108 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/exists-24 300 59599776 ns/op 3100686 B/op 28113 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/exists-24 200 62065907 ns/op 3048527 B/op 27879 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/not_exists-24 300 44979062 ns/op 123026 B/op 1244 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/not_exists-24 300 44733344 ns/op 52650 B/op 479 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/not_exists-24 300 44534180 ns/op 35119 B/op 398 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/not_exists-24 300 45179881 ns/op 105256 B/op 706 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/not_exists-24 300 44918964 ns/op 47426 B/op 621 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/not_exists-24 300 45000465 ns/op 63164 B/op 564 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/not_exists-24 300 45332999 ns/op 117008 B/op 1146 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/not_exists-24 300 45652342 ns/op 66221 B/op 616 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/not_exists-24 300 45083957 ns/op 154354 B/op 1143 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/not_exists-24 300 44560228 ns/op 65024 B/op 724 allocs/op PASS ok github.com/influxdata/influxdb/tsdb/tsm1 1690.583s
2019-08-30 15:41:47 +00:00
"strings"
2018-09-26 17:39:21 +00:00
"sync"
"sync/atomic"
"time"
"github.com/influxdata/influxdb/kit/tracing"
"github.com/influxdata/influxdb/models"
"github.com/influxdata/influxdb/storage/wal"
"github.com/influxdata/influxdb/tsdb"
2018-09-26 17:39:21 +00:00
"github.com/influxdata/influxql"
"github.com/prometheus/client_golang/prometheus"
2018-09-26 17:39:21 +00:00
"go.uber.org/zap"
)
var (
// ErrSnapshotInProgress is returned if a snapshot is attempted while one is already running.
ErrSnapshotInProgress = fmt.Errorf("snapshot in progress")
)
// CacheMemorySizeLimitExceededError is the type of error returned from the cache when
// a write would place it over its size limit.
type CacheMemorySizeLimitExceededError struct {
Size uint64
Limit uint64
}
func (c CacheMemorySizeLimitExceededError) Error() string {
return fmt.Sprintf("cache-max-memory-size exceeded: (%d/%d)", c.Size, c.Limit)
}
2018-09-26 17:39:21 +00:00
// ErrCacheMemorySizeLimitExceeded returns an error indicating an operation
// could not be completed due to exceeding the cache-max-memory-size setting.
func ErrCacheMemorySizeLimitExceeded(n, limit uint64) error {
return CacheMemorySizeLimitExceededError{Size: n, Limit: limit}
2018-09-26 17:39:21 +00:00
}
// Cache maintains an in-memory store of Values for a set of keys.
type Cache struct {
mu sync.RWMutex
store *ring
2018-09-26 17:39:21 +00:00
maxSize uint64
// snapshots are the cache objects that are currently being written to tsm files
// they're kept in memory while flushing so they can be queried along with the cache.
// they are read only and should never be modified
snapshot *Cache
snapshotting bool
2018-11-07 16:28:34 +00:00
tracker *cacheTracker
2018-09-26 17:39:21 +00:00
lastSnapshot time.Time
lastWriteTime time.Time
}
// NewCache returns an instance of a cache which will use a maximum of maxSize bytes of memory.
// Only used for engine caches, never for snapshots.
func NewCache(maxSize uint64) *Cache {
return &Cache{
2018-09-26 17:39:21 +00:00
maxSize: maxSize,
store: newRing(),
2018-09-26 17:39:21 +00:00
lastSnapshot: time.Now(),
tracker: newCacheTracker(newCacheMetrics(nil), nil),
2018-09-26 17:39:21 +00:00
}
}
// Write writes the set of values for the key to the cache. This function is goroutine-safe.
// It returns an error if the cache will exceed its max size by adding the new values.
func (c *Cache) Write(key []byte, values []Value) error {
addedSize := uint64(Values(values).Size())
// Enough room in the cache?
limit := c.maxSize
n := c.Size() + addedSize
if limit > 0 && n > limit {
2018-11-07 16:28:34 +00:00
c.tracker.IncWritesErr()
c.tracker.AddWrittenBytesDrop(uint64(addedSize))
2018-09-26 17:39:21 +00:00
return ErrCacheMemorySizeLimitExceeded(n, limit)
}
newKey, err := c.store.write(key, values)
if err != nil {
2018-11-07 16:28:34 +00:00
c.tracker.IncWritesErr()
c.tracker.AddWrittenBytesErr(uint64(addedSize))
2018-09-26 17:39:21 +00:00
return err
}
if newKey {
addedSize += uint64(len(key))
}
// Update the cache size and the memory size stat.
2018-11-07 16:28:34 +00:00
c.tracker.IncCacheSize(addedSize)
c.tracker.AddMemBytes(addedSize)
c.tracker.AddWrittenBytesOK(uint64(addedSize))
c.tracker.IncWritesOK()
2018-09-26 17:39:21 +00:00
return nil
}
// WriteMulti writes the map of keys and associated values to the cache. This
// function is goroutine-safe. It returns an error if the cache will exceeded
// its max size by adding the new values. The write attempts to write as many
// values as possible. If one key fails, the others can still succeed and an
// error will be returned.
func (c *Cache) WriteMulti(values map[string][]Value) error {
var addedSize uint64
for _, v := range values {
addedSize += uint64(Values(v).Size())
}
// Enough room in the cache?
limit := c.maxSize // maxSize is safe for reading without a lock.
n := c.Size() + addedSize
if limit > 0 && n > limit {
2018-11-07 16:28:34 +00:00
c.tracker.IncWritesErr()
c.tracker.AddWrittenBytesDrop(uint64(addedSize))
2018-09-26 17:39:21 +00:00
return ErrCacheMemorySizeLimitExceeded(n, limit)
}
var werr error
c.mu.RLock()
store := c.store
c.mu.RUnlock()
2018-11-01 18:58:56 +00:00
var bytesWrittenErr uint64
// We'll optimistically set size here, and then decrement it for write errors.
2018-09-26 17:39:21 +00:00
for k, v := range values {
newKey, err := store.write([]byte(k), v)
if err != nil {
// The write failed, hold onto the error and adjust the size delta.
werr = err
addedSize -= uint64(Values(v).Size())
2018-11-01 18:58:56 +00:00
bytesWrittenErr += uint64(Values(v).Size())
2018-09-26 17:39:21 +00:00
}
2018-11-01 18:58:56 +00:00
2018-09-26 17:39:21 +00:00
if newKey {
addedSize += uint64(len(k))
}
}
// Some points in the batch were dropped. An error is returned so
// error stat is incremented as well.
if werr != nil {
2018-11-07 16:28:34 +00:00
c.tracker.IncWritesErr()
c.tracker.IncWritesDrop()
c.tracker.AddWrittenBytesErr(bytesWrittenErr)
2018-09-26 17:39:21 +00:00
}
// Update the memory size stat
2018-11-07 16:28:34 +00:00
c.tracker.IncCacheSize(addedSize)
c.tracker.AddMemBytes(addedSize)
c.tracker.IncWritesOK()
c.tracker.AddWrittenBytesOK(addedSize)
2018-09-26 17:39:21 +00:00
c.mu.Lock()
c.lastWriteTime = time.Now()
c.mu.Unlock()
return werr
}
// Snapshot takes a snapshot of the current cache, adds it to the slice of caches that
// are being flushed, and resets the current cache with new values.
func (c *Cache) Snapshot() (*Cache, error) {
c.mu.Lock()
defer c.mu.Unlock()
if c.snapshotting {
return nil, ErrSnapshotInProgress
}
c.snapshotting = true
2018-11-07 16:28:34 +00:00
c.tracker.IncSnapshotsActive() // increment the number of times we tried to do this
2018-09-26 17:39:21 +00:00
// If no snapshot exists, create a new one, otherwise update the existing snapshot
if c.snapshot == nil {
c.snapshot = &Cache{
store: newRing(),
tracker: newCacheTracker(c.tracker.metrics, c.tracker.labels),
2018-09-26 17:39:21 +00:00
}
}
// Did a prior snapshot exist that failed? If so, return the existing
// snapshot to retry.
if c.snapshot.Size() > 0 {
return c.snapshot, nil
}
c.snapshot.store, c.store = c.store, c.snapshot.store
snapshotSize := c.Size()
2018-11-07 16:28:34 +00:00
c.snapshot.tracker.SetSnapshotSize(snapshotSize) // Save the size of the snapshot on the snapshot cache
c.tracker.SetSnapshotSize(snapshotSize) // Save the size of the snapshot on the live cache
2018-09-26 17:39:21 +00:00
// Reset the cache's store.
c.store.reset()
2018-11-07 16:28:34 +00:00
c.tracker.SetCacheSize(0)
2018-09-26 17:39:21 +00:00
c.lastSnapshot = time.Now()
2018-11-07 16:28:34 +00:00
c.tracker.AddSnapshottedBytes(snapshotSize) // increment the number of bytes added to the snapshot
c.tracker.SetDiskBytes(0)
c.tracker.SetSnapshotsActive(0)
2018-09-26 17:39:21 +00:00
return c.snapshot, nil
}
// Deduplicate sorts the snapshot before returning it. The compactor and any queries
// coming in while it writes will need the values sorted.
func (c *Cache) Deduplicate() {
c.mu.RLock()
store := c.store
c.mu.RUnlock()
// Apply a function that simply calls deduplicate on each entry in the ring.
// apply cannot return an error in this invocation.
_ = store.apply(func(_ []byte, e *entry) error { e.deduplicate(); return nil })
}
// ClearSnapshot removes the snapshot cache from the list of flushing caches and
// adjusts the size.
func (c *Cache) ClearSnapshot(success bool) {
c.mu.RLock()
snapStore := c.snapshot.store
c.mu.RUnlock()
// reset the snapshot store outside of the write lock
if success {
snapStore.reset()
}
c.mu.Lock()
defer c.mu.Unlock()
c.snapshotting = false
if success {
2018-11-07 16:28:34 +00:00
snapshotSize := c.tracker.SnapshotSize()
c.tracker.SetSnapshotsActive(0)
c.tracker.SubMemBytes(snapshotSize) // decrement the number of bytes in cache
2018-09-26 17:39:21 +00:00
// Reset the snapshot to a fresh Cache.
c.snapshot = &Cache{
2018-11-07 16:28:34 +00:00
store: c.snapshot.store,
tracker: newCacheTracker(c.tracker.metrics, c.tracker.labels),
2018-09-26 17:39:21 +00:00
}
2018-11-07 16:28:34 +00:00
c.tracker.SetSnapshotSize(0)
c.tracker.SetDiskBytes(0)
c.tracker.SetSnapshotsActive(0)
2018-09-26 17:39:21 +00:00
}
}
// Size returns the number of point-calcuated bytes the cache currently uses.
func (c *Cache) Size() uint64 {
2018-11-07 16:28:34 +00:00
return c.tracker.CacheSize() + c.tracker.SnapshotSize()
2018-09-26 17:39:21 +00:00
}
// MaxSize returns the maximum number of bytes the cache may consume.
func (c *Cache) MaxSize() uint64 {
return c.maxSize
}
func (c *Cache) Count() int {
c.mu.RLock()
n := c.store.count()
c.mu.RUnlock()
return n
}
// Keys returns a sorted slice of all keys under management by the cache.
func (c *Cache) Keys() [][]byte {
c.mu.RLock()
store := c.store
c.mu.RUnlock()
return store.keys(true)
}
func (c *Cache) Split(n int) []*Cache {
if n == 1 {
return []*Cache{c}
}
caches := make([]*Cache, n)
storers := c.store.split(n)
for i := 0; i < n; i++ {
caches[i] = &Cache{
store: storers[i],
}
}
return caches
}
2018-10-11 16:45:47 +00:00
// Type returns the series type for a key.
func (c *Cache) Type(key []byte) (models.FieldType, error) {
c.mu.RLock()
e := c.store.entry(key)
if e == nil && c.snapshot != nil {
e = c.snapshot.store.entry(key)
}
c.mu.RUnlock()
if e != nil {
typ, err := e.InfluxQLType()
if err != nil {
return models.Empty, tsdb.ErrUnknownFieldType
}
switch typ {
case influxql.Float:
return models.Float, nil
case influxql.Integer:
return models.Integer, nil
case influxql.Unsigned:
return models.Unsigned, nil
case influxql.Boolean:
return models.Boolean, nil
case influxql.String:
return models.String, nil
}
}
return models.Empty, tsdb.ErrUnknownFieldType
}
2018-09-26 17:39:21 +00:00
// Values returns a copy of all values, deduped and sorted, for the given key.
func (c *Cache) Values(key []byte) Values {
var snapshotEntries *entry
c.mu.RLock()
e := c.store.entry(key)
if c.snapshot != nil {
snapshotEntries = c.snapshot.store.entry(key)
}
c.mu.RUnlock()
if e == nil {
if snapshotEntries == nil {
// No values in hot cache or snapshots.
return nil
}
} else {
e.deduplicate()
}
// Build the sequence of entries that will be returned, in the correct order.
// Calculate the required size of the destination buffer.
var entries []*entry
sz := 0
if snapshotEntries != nil {
snapshotEntries.deduplicate() // guarantee we are deduplicated
entries = append(entries, snapshotEntries)
sz += snapshotEntries.count()
}
if e != nil {
entries = append(entries, e)
sz += e.count()
}
// Any entries? If not, return.
if sz == 0 {
return nil
}
// Create the buffer, and copy all hot values and snapshots. Individual
// entries are sorted at this point, so now the code has to check if the
// resultant buffer will be sorted from start to finish.
values := make(Values, sz)
n := 0
for _, e := range entries {
e.mu.RLock()
n += copy(values[n:], e.values)
e.mu.RUnlock()
}
values = values[:n]
values = values.Deduplicate()
return values
}
// DeleteBucketRange removes values for all keys containing points
// with timestamps between min and max contained in the bucket identified
// by name from the cache.
perf(storage): reduce allocations when deleting from cache When deleting from the cache, each cache key must be checked to determine if it matches the prefix we're deleting. Since the keys are stored as strings in the cache (map keys) there were a lot of allocations happening because `applySerial` expects `[]byte` keys. It's beneficial to reduce allocations by refacting `applySerial` to work on strings. Whilst some allocations now have to happen the other way (string -> []byte), they only happen if we actually need to delete the key from the cache. Most of the keys don't get deleted so it's better doing it this way. Performance on the benchmark from the previous commit improved by ~40-50%. name old time/op new time/op delta Engine_DeletePrefixRange_Cache/exists-24 102ms ±11% 59ms ± 3% -41.95% (p=0.000 n=10+8) Engine_DeletePrefixRange_Cache/not_exists-24 97.1ms ± 4% 45.0ms ± 1% -53.66% (p=0.000 n=10+10) name old alloc/op new alloc/op delta Engine_DeletePrefixRange_Cache/exists-24 25.5MB ± 1% 3.1MB ± 2% -87.83% (p=0.000 n=10+10) Engine_DeletePrefixRange_Cache/not_exists-24 23.9MB ± 1% 0.1MB ±86% -99.65% (p=0.000 n=10+10) name old allocs/op new allocs/op delta Engine_DeletePrefixRange_Cache/exists-24 305k ± 1% 28k ± 1% -90.77% (p=0.000 n=10+10) Engine_DeletePrefixRange_Cache/not_exists-24 299k ± 1% 1k ±63% -99.74% (p=0.000 n=9+10) Raw benchmarks on a 24T/32GB/NVME machine are as follows: goos: linux goarch: amd64 pkg: github.com/influxdata/influxdb/tsdb/tsm1 BenchmarkEngine_DeletePrefixRange_Cache/exists-24 300 50379720 ns/op 3054106 B/op 27859 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/exists-24 300 57326032 ns/op 3124764 B/op 28217 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/exists-24 300 58943855 ns/op 3162146 B/op 28527 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/exists-24 300 60565115 ns/op 3138811 B/op 28176 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/exists-24 200 59775969 ns/op 3087910 B/op 27921 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/exists-24 300 59530451 ns/op 3120986 B/op 28207 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/exists-24 300 59185532 ns/op 3113066 B/op 28302 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/exists-24 300 59295867 ns/op 3100832 B/op 28108 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/exists-24 300 59599776 ns/op 3100686 B/op 28113 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/exists-24 200 62065907 ns/op 3048527 B/op 27879 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/not_exists-24 300 44979062 ns/op 123026 B/op 1244 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/not_exists-24 300 44733344 ns/op 52650 B/op 479 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/not_exists-24 300 44534180 ns/op 35119 B/op 398 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/not_exists-24 300 45179881 ns/op 105256 B/op 706 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/not_exists-24 300 44918964 ns/op 47426 B/op 621 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/not_exists-24 300 45000465 ns/op 63164 B/op 564 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/not_exists-24 300 45332999 ns/op 117008 B/op 1146 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/not_exists-24 300 45652342 ns/op 66221 B/op 616 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/not_exists-24 300 45083957 ns/op 154354 B/op 1143 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/not_exists-24 300 44560228 ns/op 65024 B/op 724 allocs/op PASS ok github.com/influxdata/influxdb/tsdb/tsm1 1690.583s
2019-08-30 15:41:47 +00:00
func (c *Cache) DeleteBucketRange(ctx context.Context, name string, min, max int64, pred Predicate) {
span, _ := tracing.StartSpanFromContext(ctx)
defer span.Finish()
2019-02-04 19:26:21 +00:00
// TODO(edd/jeff): find a way to optimize lock usage
2018-09-26 17:39:21 +00:00
c.mu.Lock()
defer c.mu.Unlock()
perf(storage): reduce allocations when deleting from cache When deleting from the cache, each cache key must be checked to determine if it matches the prefix we're deleting. Since the keys are stored as strings in the cache (map keys) there were a lot of allocations happening because `applySerial` expects `[]byte` keys. It's beneficial to reduce allocations by refacting `applySerial` to work on strings. Whilst some allocations now have to happen the other way (string -> []byte), they only happen if we actually need to delete the key from the cache. Most of the keys don't get deleted so it's better doing it this way. Performance on the benchmark from the previous commit improved by ~40-50%. name old time/op new time/op delta Engine_DeletePrefixRange_Cache/exists-24 102ms ±11% 59ms ± 3% -41.95% (p=0.000 n=10+8) Engine_DeletePrefixRange_Cache/not_exists-24 97.1ms ± 4% 45.0ms ± 1% -53.66% (p=0.000 n=10+10) name old alloc/op new alloc/op delta Engine_DeletePrefixRange_Cache/exists-24 25.5MB ± 1% 3.1MB ± 2% -87.83% (p=0.000 n=10+10) Engine_DeletePrefixRange_Cache/not_exists-24 23.9MB ± 1% 0.1MB ±86% -99.65% (p=0.000 n=10+10) name old allocs/op new allocs/op delta Engine_DeletePrefixRange_Cache/exists-24 305k ± 1% 28k ± 1% -90.77% (p=0.000 n=10+10) Engine_DeletePrefixRange_Cache/not_exists-24 299k ± 1% 1k ±63% -99.74% (p=0.000 n=9+10) Raw benchmarks on a 24T/32GB/NVME machine are as follows: goos: linux goarch: amd64 pkg: github.com/influxdata/influxdb/tsdb/tsm1 BenchmarkEngine_DeletePrefixRange_Cache/exists-24 300 50379720 ns/op 3054106 B/op 27859 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/exists-24 300 57326032 ns/op 3124764 B/op 28217 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/exists-24 300 58943855 ns/op 3162146 B/op 28527 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/exists-24 300 60565115 ns/op 3138811 B/op 28176 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/exists-24 200 59775969 ns/op 3087910 B/op 27921 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/exists-24 300 59530451 ns/op 3120986 B/op 28207 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/exists-24 300 59185532 ns/op 3113066 B/op 28302 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/exists-24 300 59295867 ns/op 3100832 B/op 28108 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/exists-24 300 59599776 ns/op 3100686 B/op 28113 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/exists-24 200 62065907 ns/op 3048527 B/op 27879 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/not_exists-24 300 44979062 ns/op 123026 B/op 1244 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/not_exists-24 300 44733344 ns/op 52650 B/op 479 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/not_exists-24 300 44534180 ns/op 35119 B/op 398 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/not_exists-24 300 45179881 ns/op 105256 B/op 706 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/not_exists-24 300 44918964 ns/op 47426 B/op 621 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/not_exists-24 300 45000465 ns/op 63164 B/op 564 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/not_exists-24 300 45332999 ns/op 117008 B/op 1146 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/not_exists-24 300 45652342 ns/op 66221 B/op 616 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/not_exists-24 300 45083957 ns/op 154354 B/op 1143 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/not_exists-24 300 44560228 ns/op 65024 B/op 724 allocs/op PASS ok github.com/influxdata/influxdb/tsdb/tsm1 1690.583s
2019-08-30 15:41:47 +00:00
var toDelete []string
2018-11-01 18:58:56 +00:00
var total uint64
2018-09-26 17:39:21 +00:00
// applySerial only errors if the closure returns an error.
perf(storage): reduce allocations when deleting from cache When deleting from the cache, each cache key must be checked to determine if it matches the prefix we're deleting. Since the keys are stored as strings in the cache (map keys) there were a lot of allocations happening because `applySerial` expects `[]byte` keys. It's beneficial to reduce allocations by refacting `applySerial` to work on strings. Whilst some allocations now have to happen the other way (string -> []byte), they only happen if we actually need to delete the key from the cache. Most of the keys don't get deleted so it's better doing it this way. Performance on the benchmark from the previous commit improved by ~40-50%. name old time/op new time/op delta Engine_DeletePrefixRange_Cache/exists-24 102ms ±11% 59ms ± 3% -41.95% (p=0.000 n=10+8) Engine_DeletePrefixRange_Cache/not_exists-24 97.1ms ± 4% 45.0ms ± 1% -53.66% (p=0.000 n=10+10) name old alloc/op new alloc/op delta Engine_DeletePrefixRange_Cache/exists-24 25.5MB ± 1% 3.1MB ± 2% -87.83% (p=0.000 n=10+10) Engine_DeletePrefixRange_Cache/not_exists-24 23.9MB ± 1% 0.1MB ±86% -99.65% (p=0.000 n=10+10) name old allocs/op new allocs/op delta Engine_DeletePrefixRange_Cache/exists-24 305k ± 1% 28k ± 1% -90.77% (p=0.000 n=10+10) Engine_DeletePrefixRange_Cache/not_exists-24 299k ± 1% 1k ±63% -99.74% (p=0.000 n=9+10) Raw benchmarks on a 24T/32GB/NVME machine are as follows: goos: linux goarch: amd64 pkg: github.com/influxdata/influxdb/tsdb/tsm1 BenchmarkEngine_DeletePrefixRange_Cache/exists-24 300 50379720 ns/op 3054106 B/op 27859 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/exists-24 300 57326032 ns/op 3124764 B/op 28217 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/exists-24 300 58943855 ns/op 3162146 B/op 28527 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/exists-24 300 60565115 ns/op 3138811 B/op 28176 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/exists-24 200 59775969 ns/op 3087910 B/op 27921 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/exists-24 300 59530451 ns/op 3120986 B/op 28207 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/exists-24 300 59185532 ns/op 3113066 B/op 28302 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/exists-24 300 59295867 ns/op 3100832 B/op 28108 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/exists-24 300 59599776 ns/op 3100686 B/op 28113 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/exists-24 200 62065907 ns/op 3048527 B/op 27879 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/not_exists-24 300 44979062 ns/op 123026 B/op 1244 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/not_exists-24 300 44733344 ns/op 52650 B/op 479 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/not_exists-24 300 44534180 ns/op 35119 B/op 398 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/not_exists-24 300 45179881 ns/op 105256 B/op 706 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/not_exists-24 300 44918964 ns/op 47426 B/op 621 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/not_exists-24 300 45000465 ns/op 63164 B/op 564 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/not_exists-24 300 45332999 ns/op 117008 B/op 1146 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/not_exists-24 300 45652342 ns/op 66221 B/op 616 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/not_exists-24 300 45083957 ns/op 154354 B/op 1143 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/not_exists-24 300 44560228 ns/op 65024 B/op 724 allocs/op PASS ok github.com/influxdata/influxdb/tsdb/tsm1 1690.583s
2019-08-30 15:41:47 +00:00
_ = c.store.applySerial(func(k string, e *entry) error {
if !strings.HasPrefix(k, name) {
return nil
}
perf(storage): reduce allocations when deleting from cache When deleting from the cache, each cache key must be checked to determine if it matches the prefix we're deleting. Since the keys are stored as strings in the cache (map keys) there were a lot of allocations happening because `applySerial` expects `[]byte` keys. It's beneficial to reduce allocations by refacting `applySerial` to work on strings. Whilst some allocations now have to happen the other way (string -> []byte), they only happen if we actually need to delete the key from the cache. Most of the keys don't get deleted so it's better doing it this way. Performance on the benchmark from the previous commit improved by ~40-50%. name old time/op new time/op delta Engine_DeletePrefixRange_Cache/exists-24 102ms ±11% 59ms ± 3% -41.95% (p=0.000 n=10+8) Engine_DeletePrefixRange_Cache/not_exists-24 97.1ms ± 4% 45.0ms ± 1% -53.66% (p=0.000 n=10+10) name old alloc/op new alloc/op delta Engine_DeletePrefixRange_Cache/exists-24 25.5MB ± 1% 3.1MB ± 2% -87.83% (p=0.000 n=10+10) Engine_DeletePrefixRange_Cache/not_exists-24 23.9MB ± 1% 0.1MB ±86% -99.65% (p=0.000 n=10+10) name old allocs/op new allocs/op delta Engine_DeletePrefixRange_Cache/exists-24 305k ± 1% 28k ± 1% -90.77% (p=0.000 n=10+10) Engine_DeletePrefixRange_Cache/not_exists-24 299k ± 1% 1k ±63% -99.74% (p=0.000 n=9+10) Raw benchmarks on a 24T/32GB/NVME machine are as follows: goos: linux goarch: amd64 pkg: github.com/influxdata/influxdb/tsdb/tsm1 BenchmarkEngine_DeletePrefixRange_Cache/exists-24 300 50379720 ns/op 3054106 B/op 27859 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/exists-24 300 57326032 ns/op 3124764 B/op 28217 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/exists-24 300 58943855 ns/op 3162146 B/op 28527 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/exists-24 300 60565115 ns/op 3138811 B/op 28176 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/exists-24 200 59775969 ns/op 3087910 B/op 27921 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/exists-24 300 59530451 ns/op 3120986 B/op 28207 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/exists-24 300 59185532 ns/op 3113066 B/op 28302 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/exists-24 300 59295867 ns/op 3100832 B/op 28108 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/exists-24 300 59599776 ns/op 3100686 B/op 28113 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/exists-24 200 62065907 ns/op 3048527 B/op 27879 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/not_exists-24 300 44979062 ns/op 123026 B/op 1244 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/not_exists-24 300 44733344 ns/op 52650 B/op 479 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/not_exists-24 300 44534180 ns/op 35119 B/op 398 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/not_exists-24 300 45179881 ns/op 105256 B/op 706 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/not_exists-24 300 44918964 ns/op 47426 B/op 621 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/not_exists-24 300 45000465 ns/op 63164 B/op 564 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/not_exists-24 300 45332999 ns/op 117008 B/op 1146 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/not_exists-24 300 45652342 ns/op 66221 B/op 616 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/not_exists-24 300 45083957 ns/op 154354 B/op 1143 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/not_exists-24 300 44560228 ns/op 65024 B/op 724 allocs/op PASS ok github.com/influxdata/influxdb/tsdb/tsm1 1690.583s
2019-08-30 15:41:47 +00:00
// TODO(edd): either use an unsafe conversion to []byte, or add a MatchesString
// method to tsm1.Predicate.
if pred != nil && !pred.Matches([]byte(k)) {
2019-04-29 16:28:34 +00:00
return nil
}
2018-11-01 18:58:56 +00:00
total += uint64(e.size())
// if everything is being deleted, just stage it to be deleted and move on.
2018-09-26 17:39:21 +00:00
if min == math.MinInt64 && max == math.MaxInt64 {
toDelete = append(toDelete, k)
return nil
2018-09-26 17:39:21 +00:00
}
// filter the values and subtract out the remaining bytes from the reduction.
2018-09-26 17:39:21 +00:00
e.filter(min, max)
total -= uint64(e.size())
// if it has no entries left, flag it to be deleted.
2018-09-26 17:39:21 +00:00
if e.count() == 0 {
toDelete = append(toDelete, k)
2018-09-26 17:39:21 +00:00
}
return nil
})
for _, k := range toDelete {
total += uint64(len(k))
2019-08-30 20:07:32 +00:00
// TODO(edd): either use unsafe conversion to []byte or add a removeString method.
perf(storage): reduce allocations when deleting from cache When deleting from the cache, each cache key must be checked to determine if it matches the prefix we're deleting. Since the keys are stored as strings in the cache (map keys) there were a lot of allocations happening because `applySerial` expects `[]byte` keys. It's beneficial to reduce allocations by refacting `applySerial` to work on strings. Whilst some allocations now have to happen the other way (string -> []byte), they only happen if we actually need to delete the key from the cache. Most of the keys don't get deleted so it's better doing it this way. Performance on the benchmark from the previous commit improved by ~40-50%. name old time/op new time/op delta Engine_DeletePrefixRange_Cache/exists-24 102ms ±11% 59ms ± 3% -41.95% (p=0.000 n=10+8) Engine_DeletePrefixRange_Cache/not_exists-24 97.1ms ± 4% 45.0ms ± 1% -53.66% (p=0.000 n=10+10) name old alloc/op new alloc/op delta Engine_DeletePrefixRange_Cache/exists-24 25.5MB ± 1% 3.1MB ± 2% -87.83% (p=0.000 n=10+10) Engine_DeletePrefixRange_Cache/not_exists-24 23.9MB ± 1% 0.1MB ±86% -99.65% (p=0.000 n=10+10) name old allocs/op new allocs/op delta Engine_DeletePrefixRange_Cache/exists-24 305k ± 1% 28k ± 1% -90.77% (p=0.000 n=10+10) Engine_DeletePrefixRange_Cache/not_exists-24 299k ± 1% 1k ±63% -99.74% (p=0.000 n=9+10) Raw benchmarks on a 24T/32GB/NVME machine are as follows: goos: linux goarch: amd64 pkg: github.com/influxdata/influxdb/tsdb/tsm1 BenchmarkEngine_DeletePrefixRange_Cache/exists-24 300 50379720 ns/op 3054106 B/op 27859 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/exists-24 300 57326032 ns/op 3124764 B/op 28217 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/exists-24 300 58943855 ns/op 3162146 B/op 28527 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/exists-24 300 60565115 ns/op 3138811 B/op 28176 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/exists-24 200 59775969 ns/op 3087910 B/op 27921 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/exists-24 300 59530451 ns/op 3120986 B/op 28207 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/exists-24 300 59185532 ns/op 3113066 B/op 28302 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/exists-24 300 59295867 ns/op 3100832 B/op 28108 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/exists-24 300 59599776 ns/op 3100686 B/op 28113 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/exists-24 200 62065907 ns/op 3048527 B/op 27879 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/not_exists-24 300 44979062 ns/op 123026 B/op 1244 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/not_exists-24 300 44733344 ns/op 52650 B/op 479 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/not_exists-24 300 44534180 ns/op 35119 B/op 398 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/not_exists-24 300 45179881 ns/op 105256 B/op 706 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/not_exists-24 300 44918964 ns/op 47426 B/op 621 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/not_exists-24 300 45000465 ns/op 63164 B/op 564 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/not_exists-24 300 45332999 ns/op 117008 B/op 1146 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/not_exists-24 300 45652342 ns/op 66221 B/op 616 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/not_exists-24 300 45083957 ns/op 154354 B/op 1143 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/not_exists-24 300 44560228 ns/op 65024 B/op 724 allocs/op PASS ok github.com/influxdata/influxdb/tsdb/tsm1 1690.583s
2019-08-30 15:41:47 +00:00
c.store.remove([]byte(k))
2018-09-26 17:39:21 +00:00
}
c.tracker.DecCacheSize(total)
2018-11-07 16:28:34 +00:00
c.tracker.SetMemBytes(uint64(c.Size()))
2018-09-26 17:39:21 +00:00
}
// SetMaxSize updates the memory limit of the cache.
func (c *Cache) SetMaxSize(size uint64) {
c.mu.Lock()
c.maxSize = size
c.mu.Unlock()
}
// values returns the values for the key. It assumes the data is already sorted.
// It doesn't lock the cache but it does read-lock the entry if there is one for the key.
// values should only be used in compact.go in the CacheKeyIterator.
func (c *Cache) values(key []byte) Values {
e := c.store.entry(key)
if e == nil {
return nil
}
e.mu.RLock()
v := e.values
e.mu.RUnlock()
return v
}
// ApplyEntryFn applies the function f to each entry in the Cache.
// ApplyEntryFn calls f on each entry in turn, within the same goroutine.
// It is safe for use by multiple goroutines.
perf(storage): reduce allocations when deleting from cache When deleting from the cache, each cache key must be checked to determine if it matches the prefix we're deleting. Since the keys are stored as strings in the cache (map keys) there were a lot of allocations happening because `applySerial` expects `[]byte` keys. It's beneficial to reduce allocations by refacting `applySerial` to work on strings. Whilst some allocations now have to happen the other way (string -> []byte), they only happen if we actually need to delete the key from the cache. Most of the keys don't get deleted so it's better doing it this way. Performance on the benchmark from the previous commit improved by ~40-50%. name old time/op new time/op delta Engine_DeletePrefixRange_Cache/exists-24 102ms ±11% 59ms ± 3% -41.95% (p=0.000 n=10+8) Engine_DeletePrefixRange_Cache/not_exists-24 97.1ms ± 4% 45.0ms ± 1% -53.66% (p=0.000 n=10+10) name old alloc/op new alloc/op delta Engine_DeletePrefixRange_Cache/exists-24 25.5MB ± 1% 3.1MB ± 2% -87.83% (p=0.000 n=10+10) Engine_DeletePrefixRange_Cache/not_exists-24 23.9MB ± 1% 0.1MB ±86% -99.65% (p=0.000 n=10+10) name old allocs/op new allocs/op delta Engine_DeletePrefixRange_Cache/exists-24 305k ± 1% 28k ± 1% -90.77% (p=0.000 n=10+10) Engine_DeletePrefixRange_Cache/not_exists-24 299k ± 1% 1k ±63% -99.74% (p=0.000 n=9+10) Raw benchmarks on a 24T/32GB/NVME machine are as follows: goos: linux goarch: amd64 pkg: github.com/influxdata/influxdb/tsdb/tsm1 BenchmarkEngine_DeletePrefixRange_Cache/exists-24 300 50379720 ns/op 3054106 B/op 27859 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/exists-24 300 57326032 ns/op 3124764 B/op 28217 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/exists-24 300 58943855 ns/op 3162146 B/op 28527 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/exists-24 300 60565115 ns/op 3138811 B/op 28176 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/exists-24 200 59775969 ns/op 3087910 B/op 27921 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/exists-24 300 59530451 ns/op 3120986 B/op 28207 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/exists-24 300 59185532 ns/op 3113066 B/op 28302 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/exists-24 300 59295867 ns/op 3100832 B/op 28108 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/exists-24 300 59599776 ns/op 3100686 B/op 28113 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/exists-24 200 62065907 ns/op 3048527 B/op 27879 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/not_exists-24 300 44979062 ns/op 123026 B/op 1244 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/not_exists-24 300 44733344 ns/op 52650 B/op 479 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/not_exists-24 300 44534180 ns/op 35119 B/op 398 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/not_exists-24 300 45179881 ns/op 105256 B/op 706 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/not_exists-24 300 44918964 ns/op 47426 B/op 621 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/not_exists-24 300 45000465 ns/op 63164 B/op 564 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/not_exists-24 300 45332999 ns/op 117008 B/op 1146 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/not_exists-24 300 45652342 ns/op 66221 B/op 616 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/not_exists-24 300 45083957 ns/op 154354 B/op 1143 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/not_exists-24 300 44560228 ns/op 65024 B/op 724 allocs/op PASS ok github.com/influxdata/influxdb/tsdb/tsm1 1690.583s
2019-08-30 15:41:47 +00:00
func (c *Cache) ApplyEntryFn(f func(key string, entry *entry) error) error {
2018-09-26 17:39:21 +00:00
c.mu.RLock()
store := c.store
c.mu.RUnlock()
return store.applySerial(f)
}
// CacheLoader processes a set of WAL segment files, and loads a cache with the data
// contained within those files.
2018-09-26 17:39:21 +00:00
type CacheLoader struct {
reader *wal.WALReader
2018-09-26 17:39:21 +00:00
}
// NewCacheLoader returns a new instance of a CacheLoader.
func NewCacheLoader(files []string) *CacheLoader {
return &CacheLoader{
reader: wal.NewWALReader(files),
2018-09-26 17:39:21 +00:00
}
}
// Load returns a cache loaded with the data contained within the segment files.
func (cl *CacheLoader) Load(cache *Cache) error {
return cl.reader.Read(func(entry wal.WALEntry) error {
switch en := entry.(type) {
case *wal.WriteWALEntry:
return cache.WriteMulti(en.Values)
2018-09-26 17:39:21 +00:00
case *wal.DeleteBucketRangeWALEntry:
2019-04-29 16:28:34 +00:00
var pred Predicate
if len(en.Predicate) > 0 {
var err error
pred, err = UnmarshalPredicate(en.Predicate)
if err != nil {
return err
}
}
// 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(en.OrgID, en.BucketID)
name := models.EscapeMeasurement(encoded[:])
perf(storage): reduce allocations when deleting from cache When deleting from the cache, each cache key must be checked to determine if it matches the prefix we're deleting. Since the keys are stored as strings in the cache (map keys) there were a lot of allocations happening because `applySerial` expects `[]byte` keys. It's beneficial to reduce allocations by refacting `applySerial` to work on strings. Whilst some allocations now have to happen the other way (string -> []byte), they only happen if we actually need to delete the key from the cache. Most of the keys don't get deleted so it's better doing it this way. Performance on the benchmark from the previous commit improved by ~40-50%. name old time/op new time/op delta Engine_DeletePrefixRange_Cache/exists-24 102ms ±11% 59ms ± 3% -41.95% (p=0.000 n=10+8) Engine_DeletePrefixRange_Cache/not_exists-24 97.1ms ± 4% 45.0ms ± 1% -53.66% (p=0.000 n=10+10) name old alloc/op new alloc/op delta Engine_DeletePrefixRange_Cache/exists-24 25.5MB ± 1% 3.1MB ± 2% -87.83% (p=0.000 n=10+10) Engine_DeletePrefixRange_Cache/not_exists-24 23.9MB ± 1% 0.1MB ±86% -99.65% (p=0.000 n=10+10) name old allocs/op new allocs/op delta Engine_DeletePrefixRange_Cache/exists-24 305k ± 1% 28k ± 1% -90.77% (p=0.000 n=10+10) Engine_DeletePrefixRange_Cache/not_exists-24 299k ± 1% 1k ±63% -99.74% (p=0.000 n=9+10) Raw benchmarks on a 24T/32GB/NVME machine are as follows: goos: linux goarch: amd64 pkg: github.com/influxdata/influxdb/tsdb/tsm1 BenchmarkEngine_DeletePrefixRange_Cache/exists-24 300 50379720 ns/op 3054106 B/op 27859 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/exists-24 300 57326032 ns/op 3124764 B/op 28217 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/exists-24 300 58943855 ns/op 3162146 B/op 28527 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/exists-24 300 60565115 ns/op 3138811 B/op 28176 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/exists-24 200 59775969 ns/op 3087910 B/op 27921 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/exists-24 300 59530451 ns/op 3120986 B/op 28207 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/exists-24 300 59185532 ns/op 3113066 B/op 28302 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/exists-24 300 59295867 ns/op 3100832 B/op 28108 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/exists-24 300 59599776 ns/op 3100686 B/op 28113 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/exists-24 200 62065907 ns/op 3048527 B/op 27879 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/not_exists-24 300 44979062 ns/op 123026 B/op 1244 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/not_exists-24 300 44733344 ns/op 52650 B/op 479 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/not_exists-24 300 44534180 ns/op 35119 B/op 398 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/not_exists-24 300 45179881 ns/op 105256 B/op 706 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/not_exists-24 300 44918964 ns/op 47426 B/op 621 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/not_exists-24 300 45000465 ns/op 63164 B/op 564 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/not_exists-24 300 45332999 ns/op 117008 B/op 1146 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/not_exists-24 300 45652342 ns/op 66221 B/op 616 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/not_exists-24 300 45083957 ns/op 154354 B/op 1143 allocs/op BenchmarkEngine_DeletePrefixRange_Cache/not_exists-24 300 44560228 ns/op 65024 B/op 724 allocs/op PASS ok github.com/influxdata/influxdb/tsdb/tsm1 1690.583s
2019-08-30 15:41:47 +00:00
cache.DeleteBucketRange(context.Background(), string(name), en.Min, en.Max, pred)
return nil
2018-09-26 17:39:21 +00:00
}
return nil
})
2018-09-26 17:39:21 +00:00
}
// WithLogger sets the logger on the CacheLoader.
func (cl *CacheLoader) WithLogger(logger *zap.Logger) {
cl.reader.WithLogger(logger.With(zap.String("service", "cacheloader")))
2018-09-26 17:39:21 +00:00
}
// LastWriteTime returns the time that the cache was last written to.
2018-09-26 17:39:21 +00:00
func (c *Cache) LastWriteTime() time.Time {
c.mu.RLock()
defer c.mu.RUnlock()
return c.lastWriteTime
}
// Age returns the age of the cache, which is the duration since it was last
// snapshotted.
func (c *Cache) Age() time.Duration {
c.mu.RLock()
defer c.mu.RUnlock()
return time.Since(c.lastSnapshot)
}
2018-09-26 17:39:21 +00:00
// UpdateAge updates the age statistic based on the current time.
func (c *Cache) UpdateAge() {
c.mu.RLock()
defer c.mu.RUnlock()
2018-11-07 16:28:34 +00:00
c.tracker.SetAge(time.Since(c.lastSnapshot))
2018-09-26 17:39:21 +00:00
}
2018-11-01 18:58:56 +00:00
// cacheTracker tracks writes to the cache and snapshots.
//
// As well as being responsible for providing atomic reads and writes to the
// statistics, cacheTracker also mirrors any changes to the external prometheus
// metrics, which the Engine exposes.
//
// *NOTE* - cacheTracker fields should not be directory modified. Doing so
// could result in the Engine exposing inaccurate metrics.
type cacheTracker struct {
metrics *cacheMetrics
labels prometheus.Labels
2018-11-01 18:58:56 +00:00
snapshotsActive uint64
snapshotSize uint64
cacheSize uint64
// Used in testing.
memSizeBytes uint64
snapshottedBytes uint64
writesDropped uint64
writesErr uint64
}
func newCacheTracker(metrics *cacheMetrics, defaultLabels prometheus.Labels) *cacheTracker {
return &cacheTracker{metrics: metrics, labels: defaultLabels}
}
// Labels returns a copy of the default labels used by the tracker's metrics.
// The returned map is safe for modification.
func (t *cacheTracker) Labels() prometheus.Labels {
labels := make(prometheus.Labels, len(t.labels))
for k, v := range t.labels {
labels[k] = v
}
return labels
2018-11-01 18:58:56 +00:00
}
// AddMemBytes increases the number of in-memory cache bytes.
func (t *cacheTracker) AddMemBytes(bytes uint64) {
atomic.AddUint64(&t.memSizeBytes, bytes)
labels := t.labels
2018-11-01 18:58:56 +00:00
t.metrics.MemSize.With(labels).Add(float64(bytes))
2018-09-26 17:39:21 +00:00
}
2018-11-01 18:58:56 +00:00
// SubMemBytes decreases the number of in-memory cache bytes.
func (t *cacheTracker) SubMemBytes(bytes uint64) {
atomic.AddUint64(&t.memSizeBytes, ^(bytes - 1))
labels := t.labels
2018-11-01 18:58:56 +00:00
t.metrics.MemSize.With(labels).Sub(float64(bytes))
}
// SetMemBytes sets the number of in-memory cache bytes.
func (t *cacheTracker) SetMemBytes(bytes uint64) {
atomic.StoreUint64(&t.memSizeBytes, bytes)
labels := t.labels
2018-11-01 18:58:56 +00:00
t.metrics.MemSize.With(labels).Set(float64(bytes))
2018-09-26 17:39:21 +00:00
}
2018-11-01 18:58:56 +00:00
// AddBytesWritten increases the number of bytes written to the cache.
func (t *cacheTracker) AddBytesWritten(bytes uint64) {
labels := t.labels
2018-11-01 18:58:56 +00:00
t.metrics.MemSize.With(labels).Add(float64(bytes))
}
// AddSnapshottedBytes increases the number of bytes snapshotted.
func (t *cacheTracker) AddSnapshottedBytes(bytes uint64) {
atomic.AddUint64(&t.snapshottedBytes, bytes)
labels := t.labels
2018-11-01 18:58:56 +00:00
t.metrics.SnapshottedBytes.With(labels).Add(float64(bytes))
}
// SetDiskBytes sets the number of bytes on disk used by snapshot data.
func (t *cacheTracker) SetDiskBytes(bytes uint64) {
labels := t.labels
2018-11-01 18:58:56 +00:00
t.metrics.DiskSize.With(labels).Set(float64(bytes))
}
// IncSnapshotsActive increases the number of active snapshots.
func (t *cacheTracker) IncSnapshotsActive() {
atomic.AddUint64(&t.snapshotsActive, 1)
labels := t.labels
2018-11-01 18:58:56 +00:00
t.metrics.SnapshotsActive.With(labels).Inc()
}
// SetSnapshotsActive sets the number of bytes on disk used by snapshot data.
func (t *cacheTracker) SetSnapshotsActive(n uint64) {
atomic.StoreUint64(&t.snapshotsActive, n)
labels := t.labels
2018-11-01 18:58:56 +00:00
t.metrics.SnapshotsActive.With(labels).Set(float64(n))
}
// AddWrittenBytes increases the number of bytes written to the cache, with a required status.
func (t *cacheTracker) AddWrittenBytes(status string, bytes uint64) {
labels := t.Labels()
2018-11-01 18:58:56 +00:00
labels["status"] = status
t.metrics.WrittenBytes.With(labels).Add(float64(bytes))
}
// AddWrittenBytesOK increments the number of successful writes.
func (t *cacheTracker) AddWrittenBytesOK(bytes uint64) { t.AddWrittenBytes("ok", bytes) }
// AddWrittenBytesError increments the number of writes that encountered an error.
func (t *cacheTracker) AddWrittenBytesErr(bytes uint64) { t.AddWrittenBytes("error", bytes) }
// AddWrittenBytesDrop increments the number of writes that were dropped.
func (t *cacheTracker) AddWrittenBytesDrop(bytes uint64) { t.AddWrittenBytes("dropped", bytes) }
// IncWrites increments the number of writes to the cache, with a required status.
func (t *cacheTracker) IncWrites(status string) {
labels := t.Labels()
2018-11-01 18:58:56 +00:00
labels["status"] = status
t.metrics.Writes.With(labels).Inc()
}
// IncWritesOK increments the number of successful writes.
func (t *cacheTracker) IncWritesOK() { t.IncWrites("ok") }
// IncWritesError increments the number of writes that encountered an error.
func (t *cacheTracker) IncWritesErr() {
atomic.AddUint64(&t.writesErr, 1)
t.IncWrites("error")
}
// IncWritesDrop increments the number of writes that were dropped.
func (t *cacheTracker) IncWritesDrop() {
atomic.AddUint64(&t.writesDropped, 1)
t.IncWrites("dropped")
}
// CacheSize returns the live cache size.
func (t *cacheTracker) CacheSize() uint64 { return atomic.LoadUint64(&t.cacheSize) }
// IncCacheSize increases the live cache size by sz bytes.
func (t *cacheTracker) IncCacheSize(sz uint64) { atomic.AddUint64(&t.cacheSize, sz) }
// DecCacheSize decreases the live cache size by sz bytes.
func (t *cacheTracker) DecCacheSize(sz uint64) { atomic.AddUint64(&t.cacheSize, ^(sz - 1)) }
// SetCacheSize sets the live cache size to sz.
func (t *cacheTracker) SetCacheSize(sz uint64) { atomic.StoreUint64(&t.cacheSize, sz) }
// SetSnapshotSize sets the last successful snapshot size.
func (t *cacheTracker) SetSnapshotSize(sz uint64) { atomic.StoreUint64(&t.snapshotSize, sz) }
// SnapshotSize returns the last successful snapshot size.
func (t *cacheTracker) SnapshotSize() uint64 { return atomic.LoadUint64(&t.snapshotSize) }
// SetAge sets the time since the last successful snapshot
func (t *cacheTracker) SetAge(d time.Duration) {
labels := t.Labels()
2018-11-01 18:58:56 +00:00
t.metrics.Age.With(labels).Set(d.Seconds())
2018-09-26 17:39:21 +00:00
}
func valueType(v Value) byte {
switch v.(type) {
case FloatValue:
return 1
case IntegerValue:
return 2
case StringValue:
return 3
case BooleanValue:
return 4
default:
return 0
}
}