2015-11-11 20:06:02 +00:00
|
|
|
package tsm1
|
|
|
|
|
|
|
|
import (
|
2016-02-19 17:42:20 +00:00
|
|
|
"expvar"
|
2015-11-11 20:06:02 +00:00
|
|
|
"fmt"
|
2015-12-06 03:37:26 +00:00
|
|
|
"log"
|
|
|
|
"os"
|
2015-11-11 20:06:02 +00:00
|
|
|
"sort"
|
|
|
|
"sync"
|
2016-02-20 11:18:57 +00:00
|
|
|
"time"
|
2016-02-19 17:42:20 +00:00
|
|
|
|
|
|
|
"github.com/influxdata/influxdb"
|
2016-02-26 19:41:54 +00:00
|
|
|
"github.com/influxdata/influxdb/tsdb"
|
2015-11-11 20:06:02 +00:00
|
|
|
)
|
|
|
|
|
2016-03-14 17:50:44 +00:00
|
|
|
var (
|
|
|
|
ErrCacheMemoryExceeded = fmt.Errorf("cache maximum memory size exceeded")
|
|
|
|
ErrCacheInvalidCheckpoint = fmt.Errorf("invalid checkpoint")
|
|
|
|
ErrSnapshotInProgress = fmt.Errorf("snapshot in progress")
|
|
|
|
)
|
2015-11-11 20:06:02 +00:00
|
|
|
|
2015-11-21 03:28:16 +00:00
|
|
|
// entry is a set of values and some metadata.
|
2015-11-11 20:06:02 +00:00
|
|
|
type entry struct {
|
2016-02-24 23:12:10 +00:00
|
|
|
mu sync.RWMutex
|
2015-12-03 13:11:50 +00:00
|
|
|
values Values // All stored values.
|
|
|
|
needSort bool // true if the values are out of order and require deduping.
|
2015-11-11 20:06:02 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
// newEntry returns a new instance of entry.
|
|
|
|
func newEntry() *entry {
|
|
|
|
return &entry{}
|
|
|
|
}
|
|
|
|
|
|
|
|
// add adds the given values to the entry.
|
|
|
|
func (e *entry) add(values []Value) {
|
2016-02-24 23:12:10 +00:00
|
|
|
e.mu.Lock()
|
|
|
|
defer e.mu.Unlock()
|
2016-02-19 16:59:33 +00:00
|
|
|
// See if the new values are sorted or contain duplicate timestamps
|
|
|
|
var prevTime int64
|
|
|
|
for _, v := range values {
|
|
|
|
if v.UnixNano() <= prevTime {
|
|
|
|
e.needSort = true
|
|
|
|
break
|
|
|
|
}
|
|
|
|
prevTime = v.UnixNano()
|
|
|
|
}
|
|
|
|
|
2015-12-03 13:11:50 +00:00
|
|
|
// if there are existing values make sure they're all less than the first of
|
|
|
|
// the new values being added
|
2016-01-27 21:26:42 +00:00
|
|
|
if len(e.values) == 0 {
|
|
|
|
e.values = values
|
|
|
|
} else {
|
|
|
|
l := len(e.values)
|
2015-12-03 13:11:50 +00:00
|
|
|
lastValTime := e.values[l-1].UnixNano()
|
|
|
|
if lastValTime >= values[0].UnixNano() {
|
|
|
|
e.needSort = true
|
2015-12-01 15:39:12 +00:00
|
|
|
}
|
2016-01-27 21:26:42 +00:00
|
|
|
e.values = append(e.values, values...)
|
2015-12-01 15:39:12 +00:00
|
|
|
}
|
2015-11-11 20:06:02 +00:00
|
|
|
}
|
|
|
|
|
2015-12-04 23:46:57 +00:00
|
|
|
// deduplicate sorts and orders the entry's values. If values are already deduped and
|
|
|
|
// and sorted, the function does no work and simply returns.
|
|
|
|
func (e *entry) deduplicate() {
|
2016-02-24 23:12:10 +00:00
|
|
|
e.mu.Lock()
|
|
|
|
defer e.mu.Unlock()
|
|
|
|
|
2015-12-05 00:37:33 +00:00
|
|
|
if !e.needSort || len(e.values) == 0 {
|
2015-12-04 23:46:57 +00:00
|
|
|
return
|
|
|
|
}
|
|
|
|
e.values = e.values.Deduplicate()
|
|
|
|
e.needSort = false
|
|
|
|
}
|
|
|
|
|
2016-02-24 23:12:10 +00:00
|
|
|
func (e *entry) count() int {
|
|
|
|
e.mu.RLock()
|
|
|
|
defer e.mu.RUnlock()
|
|
|
|
return len(e.values)
|
|
|
|
}
|
|
|
|
|
2016-02-19 17:42:20 +00:00
|
|
|
// Statistics gathered by the Cache.
|
|
|
|
const (
|
2016-02-20 11:18:57 +00:00
|
|
|
// levels - point in time measures
|
|
|
|
|
|
|
|
statCacheMemoryBytes = "memBytes" // level: Size of in-memory cache in bytes
|
|
|
|
statCacheDiskBytes = "diskBytes" // level: Size of on-disk snapshots in bytes
|
|
|
|
statSnapshots = "snapshotCount" // level: Number of active snapshots.
|
|
|
|
statCacheAgeMs = "cacheAgeMs" // level: Number of milliseconds since cache was last snapshoted at sample time
|
|
|
|
|
|
|
|
// counters - accumulative measures
|
|
|
|
|
|
|
|
statCachedBytes = "cachedBytes" // counter: Total number of bytes written into snapshots.
|
|
|
|
statWALCompactionTimeMs = "WALCompactionTimeMs" // counter: Total number of milliseconds spent compacting snapshots
|
2016-02-19 17:42:20 +00:00
|
|
|
)
|
|
|
|
|
2015-12-03 13:11:50 +00:00
|
|
|
// Cache maintains an in-memory store of Values for a set of keys.
|
2015-11-11 20:06:02 +00:00
|
|
|
type Cache struct {
|
2016-02-24 23:12:10 +00:00
|
|
|
commit sync.Mutex
|
2015-12-03 13:11:50 +00:00
|
|
|
mu sync.RWMutex
|
|
|
|
store map[string]*entry
|
|
|
|
size uint64
|
|
|
|
maxSize uint64
|
|
|
|
|
2015-12-04 15:29:58 +00:00
|
|
|
// snapshots are the cache objects that are currently being written to tsm files
|
2015-12-03 13:11:50 +00:00
|
|
|
// 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
|
2016-02-23 05:59:14 +00:00
|
|
|
snapshot *Cache
|
|
|
|
snapshotSize uint64
|
2016-03-14 17:50:44 +00:00
|
|
|
snapshotting bool
|
2016-02-19 17:42:20 +00:00
|
|
|
|
2016-02-23 19:05:20 +00:00
|
|
|
// This number is the number of pending or failed WriteSnaphot attempts since the last successful one.
|
|
|
|
snapshotAttempts int
|
|
|
|
|
2016-02-22 04:13:57 +00:00
|
|
|
statMap *expvar.Map // nil for snapshots.
|
2016-02-20 11:18:57 +00:00
|
|
|
lastSnapshot time.Time
|
2015-11-11 20:06:02 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
// NewCache returns an instance of a cache which will use a maximum of maxSize bytes of memory.
|
2016-02-22 04:13:57 +00:00
|
|
|
// Only used for engine caches, never for snapshots
|
2016-02-19 17:42:20 +00:00
|
|
|
func NewCache(maxSize uint64, path string) *Cache {
|
2016-02-26 19:41:54 +00:00
|
|
|
db, rp := tsdb.DecodeStorePath(path)
|
2016-02-21 04:33:33 +00:00
|
|
|
c := &Cache{
|
2016-02-26 19:41:54 +00:00
|
|
|
maxSize: maxSize,
|
|
|
|
store: make(map[string]*entry),
|
|
|
|
statMap: influxdb.NewStatistics(
|
|
|
|
"tsm1_cache:"+path,
|
|
|
|
"tsm1_cache",
|
|
|
|
map[string]string{"path": path, "database": db, "retentionPolicy": rp},
|
|
|
|
),
|
2016-02-20 11:18:57 +00:00
|
|
|
lastSnapshot: time.Now(),
|
2015-11-11 20:06:02 +00:00
|
|
|
}
|
2016-02-21 04:33:33 +00:00
|
|
|
c.UpdateAge()
|
|
|
|
c.UpdateCompactTime(0)
|
|
|
|
c.updateCachedBytes(0)
|
|
|
|
c.updateMemSize(0)
|
2016-02-23 19:05:20 +00:00
|
|
|
c.updateSnapshots()
|
2016-02-21 04:33:33 +00:00
|
|
|
return c
|
2015-11-11 20:06:02 +00:00
|
|
|
}
|
|
|
|
|
2015-12-03 13:11:50 +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 has exceeded its max size.
|
|
|
|
func (c *Cache) Write(key string, values []Value) error {
|
2015-11-11 20:06:02 +00:00
|
|
|
c.mu.Lock()
|
|
|
|
|
2015-11-21 03:28:16 +00:00
|
|
|
// Enough room in the cache?
|
2016-02-21 21:26:35 +00:00
|
|
|
addedSize := Values(values).Size()
|
|
|
|
newSize := c.size + uint64(addedSize)
|
2016-02-23 05:59:14 +00:00
|
|
|
if c.maxSize > 0 && newSize+c.snapshotSize > c.maxSize {
|
2016-02-21 21:26:35 +00:00
|
|
|
c.mu.Unlock()
|
2015-11-11 20:06:02 +00:00
|
|
|
return ErrCacheMemoryExceeded
|
|
|
|
}
|
2015-12-03 13:11:50 +00:00
|
|
|
|
|
|
|
c.write(key, values)
|
|
|
|
c.size = newSize
|
2016-02-21 21:26:35 +00:00
|
|
|
c.mu.Unlock()
|
2015-12-03 13:11:50 +00:00
|
|
|
|
2016-02-19 17:42:20 +00:00
|
|
|
// Update the memory size stat
|
2016-02-21 21:26:35 +00:00
|
|
|
c.updateMemSize(int64(addedSize))
|
2015-12-03 13:11:50 +00:00
|
|
|
|
|
|
|
return nil
|
2015-11-26 14:00:25 +00:00
|
|
|
}
|
2015-11-11 20:06:02 +00:00
|
|
|
|
2015-12-03 13:11:50 +00:00
|
|
|
// WriteMulti writes the map of keys and associated values to the cache. This function is goroutine-safe.
|
|
|
|
// It returns an error if the cache has exceeded its max size.
|
|
|
|
func (c *Cache) WriteMulti(values map[string][]Value) error {
|
2015-11-26 14:00:25 +00:00
|
|
|
totalSz := 0
|
|
|
|
for _, v := range values {
|
|
|
|
totalSz += Values(v).Size()
|
|
|
|
}
|
|
|
|
|
|
|
|
// Enough room in the cache?
|
2016-01-25 23:46:14 +00:00
|
|
|
c.mu.RLock()
|
2015-12-03 13:11:50 +00:00
|
|
|
newSize := c.size + uint64(totalSz)
|
2016-02-23 05:59:14 +00:00
|
|
|
if c.maxSize > 0 && newSize+c.snapshotSize > c.maxSize {
|
2016-01-25 23:46:14 +00:00
|
|
|
c.mu.RUnlock()
|
2015-11-26 14:00:25 +00:00
|
|
|
return ErrCacheMemoryExceeded
|
2015-11-11 20:06:02 +00:00
|
|
|
}
|
2016-01-25 23:46:14 +00:00
|
|
|
c.mu.RUnlock()
|
2015-11-11 20:06:02 +00:00
|
|
|
|
2016-01-25 23:46:14 +00:00
|
|
|
c.mu.Lock()
|
2015-11-26 14:00:25 +00:00
|
|
|
for k, v := range values {
|
2015-12-03 13:11:50 +00:00
|
|
|
c.write(k, v)
|
2015-11-26 14:00:25 +00:00
|
|
|
}
|
2015-12-03 13:11:50 +00:00
|
|
|
c.size = newSize
|
2016-01-25 23:46:14 +00:00
|
|
|
c.mu.Unlock()
|
2015-12-03 13:11:50 +00:00
|
|
|
|
2016-02-19 17:42:20 +00:00
|
|
|
// Update the memory size stat
|
2016-02-21 21:26:35 +00:00
|
|
|
c.updateMemSize(int64(totalSz))
|
2016-02-19 17:42:20 +00:00
|
|
|
|
2015-11-11 20:06:02 +00:00
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2015-12-03 13:11:50 +00:00
|
|
|
// Snapshot will take a snapshot of the current cache, add it to the slice of caches that
|
|
|
|
// are being flushed, and reset the current cache with new values
|
2016-03-14 17:50:44 +00:00
|
|
|
func (c *Cache) Snapshot() (*Cache, error) {
|
2015-11-11 20:06:02 +00:00
|
|
|
c.mu.Lock()
|
|
|
|
defer c.mu.Unlock()
|
2015-12-03 13:11:50 +00:00
|
|
|
|
2016-03-14 17:50:44 +00:00
|
|
|
if c.snapshotting {
|
|
|
|
return nil, ErrSnapshotInProgress
|
|
|
|
}
|
|
|
|
|
|
|
|
c.snapshotting = true
|
2016-02-23 19:05:20 +00:00
|
|
|
c.snapshotAttempts++ // increment the number of times we tried to do this
|
|
|
|
|
2016-02-23 05:59:14 +00:00
|
|
|
// If no snapshot exists, create a new one, otherwise update the existing snapshot
|
|
|
|
if c.snapshot == nil {
|
|
|
|
c.snapshot = &Cache{
|
|
|
|
store: make(map[string]*entry),
|
|
|
|
}
|
2016-02-22 04:13:57 +00:00
|
|
|
}
|
2015-12-03 13:11:50 +00:00
|
|
|
|
2016-02-23 05:59:14 +00:00
|
|
|
// Append the current cache values to the snapshot
|
|
|
|
for k, e := range c.store {
|
2016-02-24 23:12:10 +00:00
|
|
|
e.mu.RLock()
|
2016-02-23 05:59:14 +00:00
|
|
|
if _, ok := c.snapshot.store[k]; ok {
|
|
|
|
c.snapshot.store[k].add(e.values)
|
|
|
|
} else {
|
|
|
|
c.snapshot.store[k] = e
|
|
|
|
}
|
|
|
|
c.snapshotSize += uint64(Values(e.values).Size())
|
2016-02-24 23:12:10 +00:00
|
|
|
if e.needSort {
|
|
|
|
c.snapshot.store[k].needSort = true
|
|
|
|
}
|
|
|
|
e.mu.RUnlock()
|
2016-02-23 05:59:14 +00:00
|
|
|
}
|
|
|
|
|
2016-02-23 19:16:46 +00:00
|
|
|
snapshotSize := c.size // record the number of bytes written into a snapshot
|
|
|
|
|
2016-02-23 05:59:14 +00:00
|
|
|
// Reset the cache
|
2015-12-03 13:11:50 +00:00
|
|
|
c.store = make(map[string]*entry)
|
|
|
|
c.size = 0
|
2016-02-20 11:18:57 +00:00
|
|
|
c.lastSnapshot = time.Now()
|
2015-12-03 13:11:50 +00:00
|
|
|
|
2016-02-23 19:16:46 +00:00
|
|
|
c.updateMemSize(-int64(snapshotSize)) // decrement the number of bytes in cache
|
|
|
|
c.updateCachedBytes(snapshotSize) // increment the number of bytes added to the snapshot
|
2016-02-23 19:05:20 +00:00
|
|
|
c.updateSnapshots()
|
2016-02-20 11:18:57 +00:00
|
|
|
|
2016-03-14 17:50:44 +00:00
|
|
|
return c.snapshot, nil
|
2016-01-25 22:37:34 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
// 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() {
|
|
|
|
for _, e := range c.store {
|
2015-12-04 23:46:57 +00:00
|
|
|
e.deduplicate()
|
2015-12-03 13:11:50 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
// ClearSnapshot will remove the snapshot cache from the list of flushing caches and
|
|
|
|
// adjust the size
|
2016-02-24 23:08:51 +00:00
|
|
|
func (c *Cache) ClearSnapshot(success bool) {
|
2016-03-14 17:50:44 +00:00
|
|
|
c.mu.Lock()
|
|
|
|
defer c.mu.Unlock()
|
2015-12-03 13:11:50 +00:00
|
|
|
|
2016-03-14 17:50:44 +00:00
|
|
|
c.snapshotting = false
|
2016-02-19 17:42:20 +00:00
|
|
|
|
2016-03-14 17:50:44 +00:00
|
|
|
if success {
|
2016-02-24 23:08:51 +00:00
|
|
|
c.snapshotAttempts = 0
|
|
|
|
c.snapshotSize = 0
|
|
|
|
c.snapshot = nil
|
|
|
|
|
|
|
|
c.updateSnapshots()
|
|
|
|
}
|
2015-11-11 20:06:02 +00:00
|
|
|
}
|
|
|
|
|
2015-11-21 03:28:16 +00:00
|
|
|
// Size returns the number of point-calcuated bytes the cache currently uses.
|
2015-11-11 20:06:02 +00:00
|
|
|
func (c *Cache) Size() uint64 {
|
|
|
|
c.mu.RLock()
|
|
|
|
defer c.mu.RUnlock()
|
|
|
|
return c.size
|
|
|
|
}
|
|
|
|
|
|
|
|
// MaxSize returns the maximum number of bytes the cache may consume.
|
|
|
|
func (c *Cache) MaxSize() uint64 {
|
|
|
|
return c.maxSize
|
|
|
|
}
|
|
|
|
|
|
|
|
// Keys returns a sorted slice of all keys under management by the cache.
|
|
|
|
func (c *Cache) Keys() []string {
|
2016-02-24 23:12:10 +00:00
|
|
|
c.mu.RLock()
|
|
|
|
defer c.mu.RUnlock()
|
|
|
|
|
2015-11-11 20:06:02 +00:00
|
|
|
var a []string
|
|
|
|
for k, _ := range c.store {
|
|
|
|
a = append(a, k)
|
|
|
|
}
|
|
|
|
sort.Strings(a)
|
|
|
|
return a
|
|
|
|
}
|
|
|
|
|
2015-12-03 13:11:50 +00:00
|
|
|
// Values returns a copy of all values, deduped and sorted, for the given key.
|
|
|
|
func (c *Cache) Values(key string) Values {
|
2015-12-05 15:57:55 +00:00
|
|
|
c.mu.RLock()
|
2016-02-24 23:17:53 +00:00
|
|
|
defer c.mu.RUnlock()
|
|
|
|
return c.merged(key)
|
2015-12-05 15:57:55 +00:00
|
|
|
}
|
2015-12-01 15:39:12 +00:00
|
|
|
|
2015-12-07 19:35:37 +00:00
|
|
|
// Delete will remove the keys from the cache
|
|
|
|
func (c *Cache) Delete(keys []string) {
|
|
|
|
c.mu.Lock()
|
|
|
|
defer c.mu.Unlock()
|
|
|
|
|
|
|
|
for _, k := range keys {
|
|
|
|
delete(c.store, k)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2015-12-05 15:57:55 +00:00
|
|
|
// merged returns a copy of hot and snapshot values. The copy will be merged, deduped, and
|
|
|
|
// sorted. It assumes all necessary locks have been taken. If the caller knows that the
|
|
|
|
// the hot source data for the key will not be changed, it is safe to call this function
|
|
|
|
// with a read-lock taken. Otherwise it must be called with a write-lock taken.
|
|
|
|
func (c *Cache) merged(key string) Values {
|
2015-12-04 23:37:45 +00:00
|
|
|
e := c.store[key]
|
|
|
|
if e == nil {
|
2016-02-23 05:59:14 +00:00
|
|
|
if c.snapshot == nil {
|
2015-12-05 00:37:33 +00:00
|
|
|
// No values in hot cache or snapshots.
|
|
|
|
return nil
|
2015-12-02 01:34:48 +00:00
|
|
|
}
|
2015-12-05 00:37:33 +00:00
|
|
|
} else {
|
|
|
|
e.deduplicate()
|
|
|
|
}
|
2015-12-02 01:34:48 +00:00
|
|
|
|
2015-12-05 00:37:33 +00:00
|
|
|
// 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
|
2016-02-23 05:59:14 +00:00
|
|
|
|
|
|
|
if c.snapshot != nil {
|
|
|
|
snapshotEntries := c.snapshot.store[key]
|
|
|
|
if snapshotEntries != nil {
|
2016-02-25 20:56:25 +00:00
|
|
|
snapshotEntries.deduplicate() // guarantee we are deduplicated
|
2016-02-23 05:59:14 +00:00
|
|
|
entries = append(entries, snapshotEntries)
|
2016-02-24 23:12:10 +00:00
|
|
|
sz += snapshotEntries.count()
|
2015-12-05 00:37:33 +00:00
|
|
|
}
|
|
|
|
}
|
2016-02-23 05:59:14 +00:00
|
|
|
|
2015-12-05 00:37:33 +00:00
|
|
|
if e != nil {
|
|
|
|
entries = append(entries, e)
|
2016-02-24 23:12:10 +00:00
|
|
|
sz += e.count()
|
2015-12-05 00:37:33 +00:00
|
|
|
}
|
2015-11-11 20:06:02 +00:00
|
|
|
|
2015-12-05 00:37:33 +00:00
|
|
|
// Any entries? If not, return.
|
|
|
|
if sz == 0 {
|
2015-12-04 23:37:45 +00:00
|
|
|
return nil
|
2015-12-01 15:39:12 +00:00
|
|
|
}
|
2015-11-11 20:06:02 +00:00
|
|
|
|
2015-12-05 00:37:33 +00:00
|
|
|
// 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.
|
|
|
|
var needSort bool
|
|
|
|
values := make(Values, sz)
|
|
|
|
n := 0
|
|
|
|
for _, e := range entries {
|
2016-02-24 23:12:10 +00:00
|
|
|
e.mu.RLock()
|
2015-12-05 00:37:33 +00:00
|
|
|
if !needSort && n > 0 {
|
2016-02-19 16:59:33 +00:00
|
|
|
needSort = values[n-1].UnixNano() >= e.values[0].UnixNano()
|
2015-12-05 00:37:33 +00:00
|
|
|
}
|
|
|
|
n += copy(values[n:], e.values)
|
2016-02-24 23:12:10 +00:00
|
|
|
e.mu.RUnlock()
|
2015-12-05 00:37:33 +00:00
|
|
|
}
|
2015-12-01 15:39:12 +00:00
|
|
|
|
2015-12-05 00:37:33 +00:00
|
|
|
if needSort {
|
|
|
|
values = values.Deduplicate()
|
2015-12-01 15:39:12 +00:00
|
|
|
}
|
|
|
|
|
2015-12-03 13:11:50 +00:00
|
|
|
return values
|
2015-11-11 20:06:02 +00:00
|
|
|
}
|
2015-11-26 14:00:25 +00:00
|
|
|
|
2015-12-04 21:40:01 +00:00
|
|
|
// Store returns the underlying cache store. This is not goroutine safe!
|
|
|
|
// Protect access by using the Lock and Unlock functions on Cache.
|
|
|
|
func (c *Cache) Store() map[string]*entry {
|
|
|
|
return c.store
|
|
|
|
}
|
|
|
|
|
|
|
|
func (c *Cache) Lock() {
|
|
|
|
c.mu.Lock()
|
|
|
|
}
|
|
|
|
|
|
|
|
func (c *Cache) Unlock() {
|
|
|
|
c.mu.Unlock()
|
|
|
|
}
|
|
|
|
|
2015-12-03 13:11:50 +00:00
|
|
|
// values returns the values for the key. It doesn't lock and assumes the data is
|
|
|
|
// already sorted. Should only be used in compact.go in the CacheKeyIterator
|
|
|
|
func (c *Cache) values(key string) Values {
|
|
|
|
e := c.store[key]
|
|
|
|
if e == nil {
|
|
|
|
return nil
|
2015-11-26 14:00:25 +00:00
|
|
|
}
|
2015-12-03 13:11:50 +00:00
|
|
|
return e.values
|
|
|
|
}
|
2015-11-26 14:00:25 +00:00
|
|
|
|
2015-12-03 13:11:50 +00:00
|
|
|
// write writes the set of values for the key to the cache. This function assumes
|
|
|
|
// the lock has been taken and does not enforce the cache size limits.
|
|
|
|
func (c *Cache) write(key string, values []Value) {
|
2015-11-26 14:00:25 +00:00
|
|
|
e, ok := c.store[key]
|
|
|
|
if !ok {
|
2015-12-03 13:11:50 +00:00
|
|
|
e = newEntry()
|
2015-11-26 14:00:25 +00:00
|
|
|
c.store[key] = e
|
|
|
|
}
|
2015-12-03 13:11:50 +00:00
|
|
|
e.add(values)
|
2015-11-26 14:00:25 +00:00
|
|
|
}
|
2015-12-06 03:37:26 +00:00
|
|
|
|
|
|
|
// CacheLoader processes a set of WAL segment files, and loads a cache with the data
|
|
|
|
// contained within those files. Processing of the supplied files take place in the
|
|
|
|
// order they exist in the files slice.
|
|
|
|
type CacheLoader struct {
|
|
|
|
files []string
|
|
|
|
|
|
|
|
Logger *log.Logger
|
|
|
|
}
|
|
|
|
|
|
|
|
// NewCacheLoader returns a new instance of a CacheLoader.
|
|
|
|
func NewCacheLoader(files []string) *CacheLoader {
|
|
|
|
return &CacheLoader{
|
|
|
|
files: files,
|
|
|
|
Logger: log.New(os.Stderr, "[cacheloader] ", log.LstdFlags),
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
// Load returns a cache loaded with the data contained within the segment files.
|
|
|
|
// If, during reading of a segment file, corruption is encountered, that segment
|
|
|
|
// file is truncated up to and including the last valid byte, and processing
|
|
|
|
// continues with the next segment file.
|
|
|
|
func (cl *CacheLoader) Load(cache *Cache) error {
|
|
|
|
for _, fn := range cl.files {
|
|
|
|
if err := func() error {
|
|
|
|
f, err := os.OpenFile(fn, os.O_CREATE|os.O_RDWR, 0666)
|
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
|
|
|
|
// Log some information about the segments.
|
|
|
|
stat, err := os.Stat(f.Name())
|
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
cl.Logger.Printf("reading file %s, size %d", f.Name(), stat.Size())
|
|
|
|
|
|
|
|
r := NewWALSegmentReader(f)
|
|
|
|
defer r.Close()
|
|
|
|
|
|
|
|
for r.Next() {
|
|
|
|
entry, err := r.Read()
|
|
|
|
if err != nil {
|
|
|
|
n := r.Count()
|
|
|
|
cl.Logger.Printf("file %s corrupt at position %d, truncating", f.Name(), n)
|
|
|
|
if err := f.Truncate(n); err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
break
|
|
|
|
}
|
|
|
|
|
|
|
|
switch t := entry.(type) {
|
|
|
|
case *WriteWALEntry:
|
|
|
|
if err := cache.WriteMulti(t.Values); err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
case *DeleteWALEntry:
|
2015-12-07 19:35:37 +00:00
|
|
|
cache.Delete(t.Keys)
|
2015-12-06 03:37:26 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
return nil
|
|
|
|
}(); err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
}
|
|
|
|
return nil
|
|
|
|
}
|
2016-02-20 11:18:57 +00:00
|
|
|
|
|
|
|
// Updates the age statistic
|
|
|
|
func (c *Cache) UpdateAge() {
|
|
|
|
c.mu.RLock()
|
|
|
|
defer c.mu.RUnlock()
|
|
|
|
ageStat := new(expvar.Int)
|
|
|
|
ageStat.Set(int64(time.Now().Sub(c.lastSnapshot) / time.Millisecond))
|
|
|
|
c.statMap.Set(statCacheAgeMs, ageStat)
|
|
|
|
}
|
|
|
|
|
|
|
|
// Updates WAL compaction time statistic
|
|
|
|
func (c *Cache) UpdateCompactTime(d time.Duration) {
|
|
|
|
c.statMap.Add(statWALCompactionTimeMs, int64(d/time.Millisecond))
|
|
|
|
}
|
2016-02-21 04:33:33 +00:00
|
|
|
|
|
|
|
// Update the cachedBytes counter
|
|
|
|
func (c *Cache) updateCachedBytes(b uint64) {
|
|
|
|
c.statMap.Add(statCachedBytes, int64(b))
|
|
|
|
}
|
|
|
|
|
|
|
|
// Update the memSize level
|
2016-02-21 21:26:35 +00:00
|
|
|
func (c *Cache) updateMemSize(b int64) {
|
|
|
|
c.statMap.Add(statCacheMemoryBytes, b)
|
2016-02-21 04:33:33 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
// Update the snapshotsCount and the diskSize levels
|
2016-02-23 19:05:20 +00:00
|
|
|
func (c *Cache) updateSnapshots() {
|
2016-02-21 04:33:33 +00:00
|
|
|
// Update disk stats
|
|
|
|
diskSizeStat := new(expvar.Int)
|
2016-02-23 05:59:14 +00:00
|
|
|
diskSizeStat.Set(int64(c.snapshotSize))
|
2016-02-21 04:33:33 +00:00
|
|
|
c.statMap.Set(statCacheDiskBytes, diskSizeStat)
|
|
|
|
|
|
|
|
snapshotsStat := new(expvar.Int)
|
2016-02-23 19:05:20 +00:00
|
|
|
snapshotsStat.Set(int64(c.snapshotAttempts))
|
2016-02-21 04:33:33 +00:00
|
|
|
c.statMap.Set(statSnapshots, snapshotsStat)
|
|
|
|
}
|