Merge pull request #11364 from influxdata/jmw-hoist-wal

Move the WAL from the tsm1.Engine to the storage.Engine
pull/11678/head
Jeff Wendling 2019-02-05 08:47:21 -08:00 committed by GitHub
commit d728c8fa03
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
33 changed files with 1972 additions and 3229 deletions

View File

@ -17,6 +17,7 @@ import (
"github.com/influxdata/influxdb/logger"
"github.com/influxdata/influxdb/models"
"github.com/influxdata/influxdb/storage"
"github.com/influxdata/influxdb/storage/wal"
"github.com/influxdata/influxdb/toml"
"github.com/influxdata/influxdb/tsdb"
"github.com/influxdata/influxdb/tsdb/tsi1"
@ -422,7 +423,7 @@ func collectWALFiles(path string) ([]string, error) {
var paths []string
for _, fi := range fis {
if filepath.Ext(fi.Name()) != "."+tsm1.WALFileExtension {
if filepath.Ext(fi.Name()) != "."+wal.WALFileExtension {
continue
}
paths = append(paths, filepath.Join(path, fi.Name()))

View File

@ -1353,6 +1353,15 @@ func NewPoint(name string, tags Tags, fields Fields, t time.Time) (Point, error)
}, nil
}
// NewPointFromSeries returns a Point given the serialized key, some fields, and a time.
func NewPointFromSeries(key []byte, fields Fields, t time.Time) Point {
return &point{
key: key,
time: t,
fields: fields.MarshalBinary(),
}
}
// pointKey checks some basic requirements for valid points, and returns the
// key, along with an possible error.
func pointKey(measurement string, tags Tags, fields Fields, t time.Time) ([]byte, error) {

View File

@ -12,9 +12,11 @@ import (
platform "github.com/influxdata/influxdb"
"github.com/influxdata/influxdb/logger"
"github.com/influxdata/influxdb/models"
"github.com/influxdata/influxdb/storage/wal"
"github.com/influxdata/influxdb/tsdb"
"github.com/influxdata/influxdb/tsdb/tsi1"
"github.com/influxdata/influxdb/tsdb/tsm1"
"github.com/influxdata/influxdb/tsdb/value"
"github.com/influxdata/influxql"
"github.com/prometheus/client_golang/prometheus"
"go.uber.org/zap"
@ -38,7 +40,7 @@ type Engine struct {
index *tsi1.Index
sfile *tsdb.SeriesFile
engine *tsm1.Engine
wal *tsm1.WAL
wal *wal.WAL
retentionEnforcer *retentionEnforcer
defaultMetricLabels prometheus.Labels
@ -119,27 +121,28 @@ func NewEngine(path string, c Config, options ...Option) *Engine {
tsi1.WithPath(c.GetIndexPath(path)))
// Initialize WAL
var wal tsm1.Log = new(tsm1.NopWAL)
if c.WAL.Enabled {
e.wal = tsm1.NewWAL(c.GetWALPath(path))
e.wal.WithFsyncDelay(time.Duration(c.WAL.FsyncDelay))
e.wal.EnableTraceLogging(c.TraceLoggingEnabled)
wal = e.wal
}
e.wal = wal.NewWAL(c.GetWALPath(path))
e.wal.WithFsyncDelay(time.Duration(c.WAL.FsyncDelay))
e.wal.EnableTraceLogging(c.TraceLoggingEnabled)
e.wal.SetEnabled(c.WAL.Enabled)
// Initialise Engine
e.engine = tsm1.NewEngine(c.GetEnginePath(path), e.index, c.Engine,
tsm1.WithWAL(wal),
tsm1.WithTraceLogging(c.TraceLoggingEnabled))
tsm1.WithTraceLogging(c.TraceLoggingEnabled),
tsm1.WithSnapshotter(e))
// Apply options.
for _, option := range options {
option(e)
}
// Set default metrics labels.
e.engine.SetDefaultMetricLabels(e.defaultMetricLabels)
e.sfile.SetDefaultMetricLabels(e.defaultMetricLabels)
e.index.SetDefaultMetricLabels(e.defaultMetricLabels)
if e.wal != nil {
e.wal.SetDefaultMetricLabels(e.defaultMetricLabels)
}
return e
}
@ -160,6 +163,7 @@ func (e *Engine) WithLogger(log *zap.Logger) {
e.sfile.WithLogger(e.logger)
e.index.WithLogger(e.logger)
e.engine.WithLogger(e.logger)
e.wal.WithLogger(e.logger)
e.retentionEnforcer.WithLogger(e.logger)
}
@ -170,13 +174,14 @@ func (e *Engine) PrometheusCollectors() []prometheus.Collector {
metrics = append(metrics, tsdb.PrometheusCollectors()...)
metrics = append(metrics, tsi1.PrometheusCollectors()...)
metrics = append(metrics, tsm1.PrometheusCollectors()...)
metrics = append(metrics, wal.PrometheusCollectors()...)
metrics = append(metrics, e.retentionEnforcer.PrometheusCollectors()...)
return metrics
}
// Open opens the store and all underlying resources. It returns an error if
// any of the underlying systems fail to open.
func (e *Engine) Open() error {
func (e *Engine) Open() (err error) {
e.mu.Lock()
defer e.mu.Unlock()
@ -184,19 +189,20 @@ func (e *Engine) Open() error {
return nil // Already open
}
if err := e.sfile.Open(); err != nil {
// Open the services in order and clean up if any fail.
var oh openHelper
oh.Open(e.sfile)
oh.Open(e.index)
oh.Open(e.wal)
oh.Open(e.engine)
if err := oh.Done(); err != nil {
return err
}
if err := e.index.Open(); err != nil {
if err := e.replayWAL(); err != nil {
return err
}
if err := e.engine.Open(); err != nil {
return err
}
e.engine.SetCompactionsEnabled(true) // TODO(edd):is this needed?
e.closing = make(chan struct{})
// TODO(edd) background tasks will be run in priority order via a scheduler.
@ -207,6 +213,54 @@ func (e *Engine) Open() error {
return nil
}
// replayWAL reads the WAL segment files and replays them.
func (e *Engine) replayWAL() error {
if !e.config.WAL.Enabled {
return nil
}
now := time.Now()
walPaths, err := wal.SegmentFileNames(e.wal.Path())
if err != nil {
return err
}
// TODO(jeff): we should just do snapshots and wait for them so that we don't hit
// OOM situations when reloading huge WALs.
// Disable the max size during loading
limit := e.engine.Cache.MaxSize()
defer func() { e.engine.Cache.SetMaxSize(limit) }()
e.engine.Cache.SetMaxSize(0)
// Execute all the entries in the WAL again
reader := wal.NewWALReader(walPaths)
reader.WithLogger(e.logger)
err = reader.Read(func(entry wal.WALEntry) error {
switch en := entry.(type) {
case *wal.WriteWALEntry:
points := tsm1.ValuesToPoints(en.Values)
err := e.writePointsLocked(tsdb.NewSeriesCollection(points), en.Values)
if _, ok := err.(tsdb.PartialWriteError); ok {
err = nil
}
return err
case *wal.DeleteBucketRangeWALEntry:
return e.deleteBucketRangeLocked(en.OrgID, en.BucketID, en.Min, en.Max)
}
return nil
})
e.logger.Info("Reloaded WAL",
zap.String("path", e.wal.Path()),
zap.Duration("duration", time.Since(now)),
zap.Error(err))
return err
}
// runRetentionEnforcer runs the retention enforcer in a separate goroutine.
//
// Currently this just runs on an interval, but in the future we will add the
@ -267,17 +321,15 @@ func (e *Engine) Close() error {
defer e.mu.Unlock()
e.closing = nil
if err := e.sfile.Close(); err != nil {
return err
}
if err := e.index.Close(); err != nil {
return err
}
return e.engine.Close()
var ch closeHelper
ch.Close(e.engine)
ch.Close(e.wal)
ch.Close(e.index)
ch.Close(e.sfile)
return ch.Done()
}
// CreateSeriesCursor creates a SeriesCursor for usage with the read service.
func (e *Engine) CreateSeriesCursor(ctx context.Context, req SeriesCursorRequest, cond influxql.Expr) (SeriesCursor, error) {
e.mu.RLock()
defer e.mu.RUnlock()
@ -287,6 +339,7 @@ func (e *Engine) CreateSeriesCursor(ctx context.Context, req SeriesCursorRequest
return newSeriesCursor(req, e.index, cond)
}
// CreateCursorIterator creates a CursorIterator for usage with the read service.
func (e *Engine) CreateCursorIterator(ctx context.Context) (tsdb.CursorIterator, error) {
e.mu.RLock()
defer e.mu.RUnlock()
@ -302,9 +355,7 @@ func (e *Engine) CreateCursorIterator(ctx context.Context) (tsdb.CursorIterator,
// WritePoints will however determine if there are any field type conflicts, and
// return an appropriate error in that case.
func (e *Engine) WritePoints(points []models.Point) error {
collection := tsdb.NewSeriesCollection(points)
j := 0
collection, j := tsdb.NewSeriesCollection(points), 0
for iter := collection.Iterator(); iter.Next(); {
tags := iter.Tags()
@ -350,47 +401,110 @@ func (e *Engine) WritePoints(points []models.Point) error {
return ErrEngineClosed
}
// Add new series to the index and series file. Check for partial writes.
// Convert the points to values for adding to the WAL/Cache.
values, err := tsm1.PointsToValues(collection.Points)
if err != nil {
return err
}
// Add the write to the WAL to be replayed if there is a crash or shutdown.
if _, err := e.wal.WriteMulti(values); err != nil {
return err
}
return e.writePointsLocked(collection, values)
}
// writePointsLocked does the work of writing points and must be called under some sort of lock.
func (e *Engine) writePointsLocked(collection *tsdb.SeriesCollection, values map[string][]value.Value) error {
// TODO(jeff): keep track of the values in the collection so that partial write
// errors get tracked all the way. Right now, the engine doesn't drop any values
// but if it ever did, the errors could end up missing some data.
// Add new series to the index and series file.
if err := e.index.CreateSeriesListIfNotExists(collection); err != nil {
// ignore PartialWriteErrors. The collection captures it.
// TODO(edd/jeff): should we just remove PartialWriteError from the index then?
if _, ok := err.(tsdb.PartialWriteError); !ok {
return err
}
// If there was a PartialWriteError, that means the passed in values may contain
// more than the points so we need to recreate them.
if collection.PartialWriteError() != nil {
var err error
values, err = tsm1.PointsToValues(collection.Points)
if err != nil {
return err
}
}
// Write the points to the cache and WAL.
if err := e.engine.WritePoints(collection.Points); err != nil {
// Write the values to the engine.
if err := e.engine.WriteValues(values); err != nil {
return err
}
return collection.PartialWriteError()
}
// AcquireSegments closes the current WAL segment, gets the set of all the currently closed
// segments, and calls the callback. It does all of this under the lock on the engine.
func (e *Engine) AcquireSegments(fn func(segs []string) error) error {
e.mu.Lock()
defer e.mu.Unlock()
if err := e.wal.CloseSegment(); err != nil {
return err
}
segments, err := e.wal.ClosedSegments()
if err != nil {
return err
}
return fn(segments)
}
// CommitSegments calls the callback and if that does not return an error, removes the segment
// files from the WAL. It does all of this under the lock on the engine.
func (e *Engine) CommitSegments(segs []string, fn func() error) error {
e.mu.Lock()
defer e.mu.Unlock()
if err := fn(); err != nil {
return err
}
return e.wal.Remove(segs)
}
// DeleteBucket deletes an entire bucket from the storage engine.
func (e *Engine) DeleteBucket(orgID, bucketID platform.ID) error {
return e.DeleteBucketRange(orgID, bucketID, math.MinInt64, math.MaxInt64)
}
// DeleteBucketRange deletes an entire bucket from the storage engine.
func (e *Engine) DeleteBucketRange(orgID, bucketID platform.ID, min, max int64) error {
e.mu.RLock()
defer e.mu.RUnlock()
if e.closing == nil {
return ErrEngineClosed
}
// Add the delete to the WAL to be replayed if there is a crash or shutdown.
if _, err := e.wal.DeleteBucketRange(orgID, bucketID, min, max); err != nil {
return err
}
return e.deleteBucketRangeLocked(orgID, bucketID, min, max)
}
// deleteBucketRangeLocked does the work of deleting a bucket range and must be called under
// some sort of lock.
func (e *Engine) deleteBucketRangeLocked(orgID, bucketID platform.ID, min, max int64) error {
// 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[:])
return e.engine.DeleteBucket(name, math.MinInt64, math.MaxInt64)
}
// DeleteSeriesRangeWithPredicate deletes all series data iterated over if fn returns
// true for that series.
func (e *Engine) DeleteSeriesRangeWithPredicate(itr tsdb.SeriesIterator, fn func([]byte, models.Tags) (int64, int64, bool)) error {
e.mu.RLock()
defer e.mu.RUnlock()
if e.closing == nil {
return ErrEngineClosed
}
return e.engine.DeleteSeriesRangeWithPredicate(itr, fn)
return e.engine.DeleteBucketRange(name, min, max)
}
// SeriesCardinality returns the number of series in the engine.

View File

@ -16,8 +16,6 @@ type retentionMetrics struct {
labels prometheus.Labels
Checks *prometheus.CounterVec
CheckDuration *prometheus.HistogramVec
Unprocessable *prometheus.CounterVec
Series *prometheus.CounterVec
}
func newRetentionMetrics(labels prometheus.Labels) *retentionMetrics {
@ -25,17 +23,19 @@ func newRetentionMetrics(labels prometheus.Labels) *retentionMetrics {
for k := range labels {
names = append(names, k)
}
names = append(names, "status") // All metrics include status
sort.Strings(names)
checksNames := append(append([]string(nil), names...), "status", "org_id", "bucket_id")
sort.Strings(checksNames)
return &retentionMetrics{
labels: labels,
Checks: prometheus.NewCounterVec(prometheus.CounterOpts{
Namespace: namespace,
Subsystem: retentionSubsystem,
Name: "checks_total",
Help: "Number of retention check operations performed.",
}, names),
Help: "Number of retention check operations performed by org/bucket id.",
}, checksNames),
CheckDuration: prometheus.NewHistogramVec(prometheus.HistogramOpts{
Namespace: namespace,
@ -45,20 +45,6 @@ func newRetentionMetrics(labels prometheus.Labels) *retentionMetrics {
// 25 buckets spaced exponentially between 10s and ~2h
Buckets: prometheus.ExponentialBuckets(10, 1.32, 25),
}, names),
Unprocessable: prometheus.NewCounterVec(prometheus.CounterOpts{
Namespace: namespace,
Subsystem: retentionSubsystem,
Name: "unprocessable_buckets_total",
Help: "Number of buckets that could not be operated on.",
}, names),
Series: prometheus.NewCounterVec(prometheus.CounterOpts{
Namespace: namespace,
Subsystem: retentionSubsystem,
Name: "series_total",
Help: "Number of series that a delete was applied to.",
}, names),
}
}
@ -76,7 +62,5 @@ func (rm *retentionMetrics) PrometheusCollectors() []prometheus.Collector {
return []prometheus.Collector{
rm.Checks,
rm.CheckDuration,
rm.Unprocessable,
rm.Series,
}
}

61
storage/opener.go Normal file
View File

@ -0,0 +1,61 @@
package storage
import (
"io"
)
// opener is something that can be opened and closed.
type opener interface {
Open() error
io.Closer
}
// openHelper is a helper to abstract the pattern of opening multiple things,
// exiting early if any open fails, and closing any of the opened things
// in the case of failure.
type openHelper struct {
opened []io.Closer
err error
}
// Open attempts to open the opener. If an error has happened already
// then no calls are made to the opener.
func (o *openHelper) Open(op opener) {
if o.err != nil {
return
}
o.err = op.Open()
if o.err == nil {
o.opened = append(o.opened, op)
}
}
// Done returns the error of the first open and closes in reverse
// order any opens that have already happened if there was an error.
func (o *openHelper) Done() error {
if o.err == nil {
return nil
}
for i := len(o.opened) - 1; i >= 0; i-- {
o.opened[i].Close()
}
return o.err
}
// closeHelper is a helper to abstract the pattern of closing multiple
// things and keeping track of the first encountered error.
type closeHelper struct {
err error
}
// Close closes the closer and keeps track of the first error.
func (c *closeHelper) Close(cl io.Closer) {
if err := cl.Close(); c.err == nil {
c.err = err
}
}
// Done returns the first error.
func (c *closeHelper) Done() error {
return c.err
}

View File

@ -106,7 +106,7 @@ func (c *{{.name}}MultiShardArrayCursor) reset(cur cursors.{{.Name}}ArrayCursor,
}
func (c *{{.name}}MultiShardArrayCursor) Err() error { return c.err }
func (c *{{.name}}MultiShardArrayCursor) Err() error { return c.err }
func (c *{{.name}}MultiShardArrayCursor) Stats() cursors.CursorStats {
return c.{{.Name}}ArrayCursor.Stats()
@ -200,17 +200,17 @@ func (c {{$type}}) Next() {{$arrayType}} {
var acc {{.Type}}
for {
for _, v := range a.Values {
acc += v
}
for _, v := range a.Values {
acc += v
}
a = c.{{.Name}}ArrayCursor.Next()
if len(a.Timestamps) == 0 {
if len(a.Timestamps) == 0 {
c.ts[0] = ts
c.vs[0] = acc
c.res.Timestamps = c.ts[:]
c.res.Values = c.vs[:]
return c.res
}
}
}
}
@ -230,11 +230,11 @@ func (c *integer{{.Name}}CountArrayCursor) Next() *cursors.IntegerArray {
return &cursors.IntegerArray{}
}
ts := a.Timestamps[0]
var acc int64
for {
acc += int64(len(a.Timestamps))
a = c.{{.Name}}ArrayCursor.Next()
ts := a.Timestamps[0]
var acc int64
for {
acc += int64(len(a.Timestamps))
a = c.{{.Name}}ArrayCursor.Next()
if len(a.Timestamps) == 0 {
res := cursors.NewIntegerArrayLen(1)
res.Timestamps[0] = ts

View File

@ -4,28 +4,21 @@ import (
"context"
"errors"
"math"
"sync"
"sync/atomic"
"time"
platform "github.com/influxdata/influxdb"
"github.com/influxdata/influxdb/logger"
"github.com/influxdata/influxdb/models"
"github.com/influxdata/influxdb/tsdb"
"github.com/influxdata/influxql"
"github.com/prometheus/client_golang/prometheus"
"go.uber.org/zap"
)
const (
bucketAPITimeout = 10 * time.Second
engineAPITimeout = time.Minute
)
// A Deleter implementation is capable of deleting data from a storage engine.
type Deleter interface {
CreateSeriesCursor(context.Context, SeriesCursorRequest, influxql.Expr) (SeriesCursor, error)
DeleteSeriesRangeWithPredicate(tsdb.SeriesIterator, func([]byte, models.Tags) (int64, int64, bool)) error
DeleteBucketRange(orgID, bucketID platform.ID, min, max int64) error
}
// A BucketFinder is responsible for providing access to buckets via a filter.
@ -78,169 +71,59 @@ func (s *retentionEnforcer) run() {
log, logEnd := logger.NewOperation(s.logger, "Data retention check", "data_retention_check")
defer logEnd()
rpByBucketID, err := s.getRetentionPeriodPerBucket()
buckets, err := s.getBucketInformation()
if err != nil {
log.Error("Unable to determine bucket:RP mapping", zap.Error(err))
log.Error("Unable to determine bucket information", zap.Error(err))
return
}
now := time.Now().UTC()
labels := s.metrics.Labels()
labels["status"] = "ok"
if err := s.expireData(rpByBucketID, now); err != nil {
log.Error("Deletion not successful", zap.Error(err))
labels["status"] = "error"
}
s.metrics.CheckDuration.With(labels).Observe(time.Since(now).Seconds())
s.metrics.Checks.With(labels).Inc()
s.expireData(buckets, now)
s.metrics.CheckDuration.With(s.metrics.Labels()).Observe(time.Since(now).Seconds())
}
// expireData runs a delete operation on the storage engine.
//
// Any series data that (1) belongs to a bucket in the provided map and
// Any series data that (1) belongs to a bucket in the provided list and
// (2) falls outside the bucket's indicated retention period will be deleted.
func (s *retentionEnforcer) expireData(rpByBucketID map[platform.ID]time.Duration, now time.Time) error {
_, logEnd := logger.NewOperation(s.logger, "Data deletion", "data_deletion")
func (s *retentionEnforcer) expireData(buckets []*platform.Bucket, now time.Time) {
logger, logEnd := logger.NewOperation(s.logger, "Data deletion", "data_deletion")
defer logEnd()
ctx, cancel := context.WithTimeout(context.Background(), engineAPITimeout)
defer cancel()
cur, err := s.Engine.CreateSeriesCursor(ctx, SeriesCursorRequest{}, nil)
if err != nil {
return err
}
defer cur.Close()
var mu sync.Mutex
badMSketch := make(map[string]struct{}) // Badly formatted measurements.
missingBSketch := make(map[platform.ID]struct{}) // Missing buckets.
var seriesDeleted uint64 // Number of series where a delete is attempted.
var seriesSkipped uint64 // Number of series that were skipped from delete.
fn := func(name []byte, tags models.Tags) (int64, int64, bool) {
if len(name) != platform.IDLength {
mu.Lock()
badMSketch[string(name)] = struct{}{}
mu.Unlock()
atomic.AddUint64(&seriesSkipped, 1)
return 0, 0, false
labels := s.metrics.Labels()
for _, b := range buckets {
if b.RetentionPeriod == 0 {
continue
}
var n [16]byte
copy(n[:], name)
_, bucketID := tsdb.DecodeName(n)
retentionPeriod, ok := rpByBucketID[bucketID]
if !ok {
mu.Lock()
missingBSketch[bucketID] = struct{}{}
mu.Unlock()
atomic.AddUint64(&seriesSkipped, 1)
return 0, 0, false
}
if retentionPeriod == 0 {
return 0, 0, false
}
atomic.AddUint64(&seriesDeleted, 1)
to := now.Add(-retentionPeriod).UnixNano()
return math.MinInt64, to, true
}
defer func() {
if s.metrics == nil {
return
}
labels := s.metrics.Labels()
labels["status"] = "bad_measurement"
s.metrics.Unprocessable.With(labels).Add(float64(len(badMSketch)))
labels["status"] = "missing_bucket"
s.metrics.Unprocessable.With(labels).Add(float64(len(missingBSketch)))
labels["status"] = "ok"
s.metrics.Series.With(labels).Add(float64(atomic.LoadUint64(&seriesDeleted)))
labels["org_id"] = b.OrganizationID.String()
labels["bucket_id"] = b.ID.String()
labels["status"] = "skipped"
s.metrics.Series.With(labels).Add(float64(atomic.LoadUint64(&seriesSkipped)))
}()
max := now.Add(-b.RetentionPeriod).UnixNano()
err := s.Engine.DeleteBucketRange(b.OrganizationID, b.ID, math.MinInt64, max)
if err != nil {
labels["status"] = "error"
logger.Info("unable to delete bucket range",
zap.String("bucket id", b.ID.String()),
zap.String("org id", b.OrganizationID.String()),
zap.Error(err))
}
return s.Engine.DeleteSeriesRangeWithPredicate(newSeriesIteratorAdapter(cur), fn)
s.metrics.Checks.With(labels).Inc()
}
}
// getRetentionPeriodPerBucket returns a map of (bucket ID -> retention period)
// for all buckets.
func (s *retentionEnforcer) getRetentionPeriodPerBucket() (map[platform.ID]time.Duration, error) {
// getBucketInformation returns a slice of buckets to run retention on.
func (s *retentionEnforcer) getBucketInformation() ([]*platform.Bucket, error) {
ctx, cancel := context.WithTimeout(context.Background(), bucketAPITimeout)
defer cancel()
buckets, _, err := s.BucketService.FindBuckets(ctx, platform.BucketFilter{})
if err != nil {
return nil, err
}
rpByBucketID := make(map[platform.ID]time.Duration, len(buckets))
for _, bucket := range buckets {
rpByBucketID[bucket.ID] = bucket.RetentionPeriod
}
return rpByBucketID, nil
return buckets, err
}
// PrometheusCollectors satisfies the prom.PrometheusCollector interface.
func (s *retentionEnforcer) PrometheusCollectors() []prometheus.Collector {
return s.metrics.PrometheusCollectors()
}
type seriesIteratorAdapter struct {
itr SeriesCursor
ea seriesElemAdapter
elem tsdb.SeriesElem
}
func newSeriesIteratorAdapter(itr SeriesCursor) *seriesIteratorAdapter {
si := &seriesIteratorAdapter{itr: itr}
si.elem = &si.ea
return si
}
// Next returns the next tsdb.SeriesElem.
//
// The returned tsdb.SeriesElem is valid for use until Next is called again.
func (s *seriesIteratorAdapter) Next() (tsdb.SeriesElem, error) {
if s.itr == nil {
return nil, nil
}
row, err := s.itr.Next()
if err != nil {
return nil, err
}
if row == nil {
return nil, nil
}
s.ea.name = row.Name
s.ea.tags = row.Tags
return s.elem, nil
}
func (s *seriesIteratorAdapter) Close() error {
if s.itr != nil {
err := s.itr.Close()
s.itr = nil
return err
}
return nil
}
type seriesElemAdapter struct {
name []byte
tags models.Tags
}
func (e *seriesElemAdapter) Name() []byte { return e.name }
func (e *seriesElemAdapter) Tags() models.Tags { return e.tags }
func (e *seriesElemAdapter) Deleted() bool { return false }
func (e *seriesElemAdapter) Expr() influxql.Expr { return nil }

View File

@ -2,7 +2,6 @@ package storage
import (
"context"
"fmt"
"math"
"math/rand"
"reflect"
@ -10,103 +9,74 @@ import (
"time"
platform "github.com/influxdata/influxdb"
"github.com/influxdata/influxdb/models"
"github.com/influxdata/influxdb/tsdb"
"github.com/influxdata/influxql"
)
func TestService_expireData(t *testing.T) {
func TestRetentionService(t *testing.T) {
engine := NewTestEngine()
service := newRetentionEnforcer(engine, NewTestBucketFinder())
now := time.Date(2018, 4, 10, 23, 12, 33, 0, time.UTC)
t.Run("no rpByBucketID", func(t *testing.T) {
if err := service.expireData(nil, now); err != nil {
t.Error(err)
}
if err := service.expireData(map[platform.ID]time.Duration{}, now); err != nil {
t.Error(err)
}
t.Run("no buckets", func(t *testing.T) {
service.expireData(nil, now)
service.expireData([]*platform.Bucket{}, now)
})
// Generate some measurement names
var names [][]byte
rpByBucketID := map[platform.ID]time.Duration{}
expMatchedFrequencies := map[string]int{} // To be used for verifying test results.
expRejectedFrequencies := map[string]int{} // To be used for verifying test results.
// Generate some buckets to expire
buckets := []*platform.Bucket{}
expMatched := map[string]struct{}{} // To be used for verifying test results.
expRejected := map[string]struct{}{} // To be used for verifying test results.
for i := 0; i < 15; i++ {
repeat := rand.Intn(10) + 1 // [1, 10]
name := genMeasurementName()
for j := 0; j < repeat; j++ {
names = append(names, name)
}
var n [16]byte
copy(n[:], name)
_, bucketID := tsdb.DecodeName(n)
orgID, bucketID := tsdb.DecodeName(n)
// Put 1/3rd in the rpByBucketID into the set to delete and 1/3rd into the set
// to not delete because no rp, and 1/3rd into the set to not delete because 0 rp.
if i%3 == 0 {
rpByBucketID[bucketID] = 3 * time.Hour
expMatchedFrequencies[string(name)] = repeat
buckets = append(buckets, &platform.Bucket{
OrganizationID: orgID,
ID: bucketID,
RetentionPeriod: 3 * time.Hour,
})
expMatched[string(name)] = struct{}{}
} else if i%3 == 1 {
expRejectedFrequencies[string(name)] = repeat
expRejected[string(name)] = struct{}{}
} else if i%3 == 2 {
rpByBucketID[bucketID] = 0
expRejectedFrequencies[string(name)] = repeat
buckets = append(buckets, &platform.Bucket{
OrganizationID: orgID,
ID: bucketID,
RetentionPeriod: 0,
})
expRejected[string(name)] = struct{}{}
}
}
// Add a badly formatted measurement.
for i := 0; i < 5; i++ {
names = append(names, []byte("zyzwrong"))
}
expRejectedFrequencies["zyzwrong"] = 5
gotMatchedFrequencies := map[string]int{}
gotRejectedFrequencies := map[string]int{}
engine.DeleteSeriesRangeWithPredicateFn = func(_ tsdb.SeriesIterator, fn func([]byte, models.Tags) (int64, int64, bool)) error {
// Iterate over the generated names updating the frequencies by which
// the predicate function in expireData matches or rejects them.
for _, name := range names {
from, to, shouldDelete := fn(name, nil)
if shouldDelete {
gotMatchedFrequencies[string(name)]++
if from != math.MinInt64 {
return fmt.Errorf("got from %d, expected %d", from, math.MinInt64)
}
wantTo := now.Add(-3 * time.Hour).UnixNano()
if to != wantTo {
return fmt.Errorf("got to %d, expected %d", to, wantTo)
}
} else {
gotRejectedFrequencies[string(name)]++
}
gotMatched := map[string]struct{}{}
engine.DeleteBucketRangeFn = func(orgID, bucketID platform.ID, from, to int64) error {
if from != math.MinInt64 {
t.Fatalf("got from %d, expected %d", from, math.MinInt64)
}
wantTo := now.Add(-3 * time.Hour).UnixNano()
if to != wantTo {
t.Fatalf("got to %d, expected %d", to, wantTo)
}
name := tsdb.EncodeName(orgID, bucketID)
if _, ok := expRejected[string(name[:])]; ok {
t.Fatalf("got a delete for %x", name)
}
gotMatched[string(name[:])] = struct{}{}
return nil
}
t.Run("multiple bucket", func(t *testing.T) {
if err := service.expireData(rpByBucketID, now); err != nil {
t.Error(err)
t.Run("multiple buckets", func(t *testing.T) {
service.expireData(buckets, now)
if !reflect.DeepEqual(gotMatched, expMatched) {
t.Fatalf("got\n%#v\nexpected\n%#v", gotMatched, expMatched)
}
// Verify that the correct series were marked to be deleted.
t.Run("matched", func(t *testing.T) {
if !reflect.DeepEqual(gotMatchedFrequencies, expMatchedFrequencies) {
t.Fatalf("got\n%#v\nexpected\n%#v", gotMatchedFrequencies, expMatchedFrequencies)
}
})
t.Run("rejected", func(t *testing.T) {
// Verify that badly formatted measurements were rejected.
if !reflect.DeepEqual(gotRejectedFrequencies, expRejectedFrequencies) {
t.Fatalf("got\n%#v\nexpected\n%#v", gotRejectedFrequencies, expRejectedFrequencies)
}
})
})
}
@ -120,40 +90,18 @@ func genMeasurementName() []byte {
return b
}
type TestSeriesCursor struct {
CloseFn func() error
NextFn func() (*SeriesCursorRow, error)
}
func (f *TestSeriesCursor) Close() error { return f.CloseFn() }
func (f *TestSeriesCursor) Next() (*SeriesCursorRow, error) { return f.NextFn() }
type TestEngine struct {
CreateSeriesCursorFn func(context.Context, SeriesCursorRequest, influxql.Expr) (SeriesCursor, error)
DeleteSeriesRangeWithPredicateFn func(tsdb.SeriesIterator, func([]byte, models.Tags) (int64, int64, bool)) error
SeriesCursor *TestSeriesCursor
DeleteBucketRangeFn func(platform.ID, platform.ID, int64, int64) error
}
func NewTestEngine() *TestEngine {
cursor := &TestSeriesCursor{
CloseFn: func() error { return nil },
NextFn: func() (*SeriesCursorRow, error) { return nil, nil },
}
return &TestEngine{
SeriesCursor: cursor,
CreateSeriesCursorFn: func(context.Context, SeriesCursorRequest, influxql.Expr) (SeriesCursor, error) { return cursor, nil },
DeleteSeriesRangeWithPredicateFn: func(tsdb.SeriesIterator, func([]byte, models.Tags) (int64, int64, bool)) error { return nil },
DeleteBucketRangeFn: func(platform.ID, platform.ID, int64, int64) error { return nil },
}
}
func (e *TestEngine) CreateSeriesCursor(ctx context.Context, req SeriesCursorRequest, cond influxql.Expr) (SeriesCursor, error) {
return e.CreateSeriesCursorFn(ctx, req, cond)
}
func (e *TestEngine) DeleteSeriesRangeWithPredicate(itr tsdb.SeriesIterator, fn func([]byte, models.Tags) (int64, int64, bool)) error {
return e.DeleteSeriesRangeWithPredicateFn(itr, fn)
func (e *TestEngine) DeleteBucketRange(orgID, bucketID platform.ID, min, max int64) error {
return e.DeleteBucketRangeFn(orgID, bucketID, min, max)
}
type TestBucketFinder struct {

View File

@ -0,0 +1,28 @@
package wal
import (
"fmt"
"io/ioutil"
"os"
"testing"
)
func MustTempDir() string {
dir, err := ioutil.TempDir("", "tsm1-test")
if err != nil {
panic(fmt.Sprintf("failed to create temp dir: %v", err))
}
return dir
}
func MustTempFile(dir string) *os.File {
f, err := ioutil.TempFile(dir, "tsm1test")
if err != nil {
panic(fmt.Sprintf("failed to create temp file: %v", err))
}
return f
}
func fatal(t *testing.T, msg string, err error) {
t.Fatalf("unexpected error %v: %v", msg, err)
}

91
storage/wal/metrics.go Normal file
View File

@ -0,0 +1,91 @@
package wal
import (
"sort"
"sync"
"github.com/prometheus/client_golang/prometheus"
)
// The following package variables act as singletons, to be shared by all
// storage.Engine instantiations. This allows multiple WALs to be monitored
// within the same process.
var (
wms *walMetrics // main metrics
mmu sync.RWMutex
)
// PrometheusCollectors returns all the metrics associated with the tsdb package.
func PrometheusCollectors() []prometheus.Collector {
mmu.RLock()
defer mmu.RUnlock()
var collectors []prometheus.Collector
if wms != nil {
collectors = append(collectors, wms.PrometheusCollectors()...)
}
return collectors
}
// namespace is the leading part of all published metrics for the Storage service.
const namespace = "storage"
const walSubsystem = "wal" // sub-system associated with metrics for the WAL.
// walMetrics are a set of metrics concerned with tracking data about compactions.
type walMetrics struct {
OldSegmentBytes *prometheus.GaugeVec
CurrentSegmentBytes *prometheus.GaugeVec
Segments *prometheus.GaugeVec
Writes *prometheus.CounterVec
}
// newWALMetrics initialises the prometheus metrics for tracking the WAL.
func newWALMetrics(labels prometheus.Labels) *walMetrics {
var names []string
for k := range labels {
names = append(names, k)
}
sort.Strings(names)
writeNames := append(append([]string(nil), names...), "status")
sort.Strings(writeNames)
return &walMetrics{
OldSegmentBytes: prometheus.NewGaugeVec(prometheus.GaugeOpts{
Namespace: namespace,
Subsystem: walSubsystem,
Name: "old_segment_bytes",
Help: "Number of bytes old WAL segments using on disk.",
}, names),
CurrentSegmentBytes: prometheus.NewGaugeVec(prometheus.GaugeOpts{
Namespace: namespace,
Subsystem: walSubsystem,
Name: "current_segment_bytes",
Help: "Number of bytes TSM files using on disk.",
}, names),
Segments: prometheus.NewGaugeVec(prometheus.GaugeOpts{
Namespace: namespace,
Subsystem: walSubsystem,
Name: "segments_total",
Help: "Number of WAL segment files on disk.",
}, names),
Writes: prometheus.NewCounterVec(prometheus.CounterOpts{
Namespace: namespace,
Subsystem: walSubsystem,
Name: "writes_total",
Help: "Number of writes to the WAL.",
}, writeNames),
}
}
// PrometheusCollectors satisfies the prom.PrometheusCollector interface.
func (m *walMetrics) PrometheusCollectors() []prometheus.Collector {
return []prometheus.Collector{
m.OldSegmentBytes,
m.CurrentSegmentBytes,
m.Segments,
m.Writes,
}
}

View File

@ -0,0 +1,75 @@
package wal
import (
"testing"
"github.com/influxdata/influxdb/kit/prom/promtest"
"github.com/prometheus/client_golang/prometheus"
)
func TestMetrics_WAL(t *testing.T) {
// metrics to be shared by multiple file stores.
metrics := newWALMetrics(prometheus.Labels{"engine_id": "", "node_id": ""})
t1 := newWALTracker(metrics, prometheus.Labels{"engine_id": "0", "node_id": "0"})
t2 := newWALTracker(metrics, prometheus.Labels{"engine_id": "1", "node_id": "0"})
reg := prometheus.NewRegistry()
reg.MustRegister(metrics.PrometheusCollectors()...)
base := namespace + "_" + walSubsystem + "_"
// All the metric names
gauges := []string{
base + "old_segment_bytes",
base + "current_segment_bytes",
base + "segments_total",
}
counters := []string{
base + "writes_total",
}
// Generate some measurements.
for i, tracker := range []*walTracker{t1, t2} {
tracker.SetOldSegmentSize(uint64(i + len(gauges[0])))
tracker.SetCurrentSegmentSize(uint64(i + len(gauges[1])))
tracker.SetSegments(uint64(i + len(gauges[2])))
labels := tracker.Labels()
labels["status"] = "ok"
tracker.metrics.Writes.With(labels).Add(float64(i + len(counters[0])))
}
// Test that all the correct metrics are present.
mfs, err := reg.Gather()
if err != nil {
t.Fatal(err)
}
// The label variants for the two caches.
labelVariants := []prometheus.Labels{
prometheus.Labels{"engine_id": "0", "node_id": "0"},
prometheus.Labels{"engine_id": "1", "node_id": "0"},
}
for i, labels := range labelVariants {
for _, name := range gauges {
exp := float64(i + len(name))
metric := promtest.MustFindMetric(t, mfs, name, labels)
if got := metric.GetGauge().GetValue(); got != exp {
t.Errorf("[%s %d] got %v, expected %v", name, i, got, exp)
}
}
for _, name := range counters {
exp := float64(i + len(name))
labels["status"] = "ok"
metric := promtest.MustFindMetric(t, mfs, name, labels)
if got := metric.GetCounter().GetValue(); got != exp {
t.Errorf("[%s %d] got %v, expected %v", name, i, got, exp)
}
}
}
}

View File

@ -1,4 +1,4 @@
package tsm1
package wal
import "sync"

86
storage/wal/reader.go Normal file
View File

@ -0,0 +1,86 @@
package wal
import (
"os"
"sort"
"go.uber.org/zap"
)
// WALReader helps one read out the WAL into entries.
type WALReader struct {
files []string
logger *zap.Logger
r *WALSegmentReader
}
// NewWALReader constructs a WALReader over the given set of files.
func NewWALReader(files []string) *WALReader {
sort.Strings(files)
return &WALReader{
files: files,
logger: zap.NewNop(),
r: nil,
}
}
// WithLogger sets the logger for the WALReader.
func (r *WALReader) WithLogger(logger *zap.Logger) { r.logger = logger }
// Read calls the callback with every entry in the WAL 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 (r *WALReader) Read(cb func(WALEntry) error) error {
for _, file := range r.files {
if err := r.readFile(file, cb); err != nil {
return err
}
}
return nil
}
// readFile reads the file and calls the callback with each WAL entry.
// It uses the provided logger for information about progress and corruptions.
func (r *WALReader) readFile(file string, cb func(WALEntry) error) error {
f, err := os.OpenFile(file, os.O_CREATE|os.O_RDWR, 0666)
if err != nil {
return err
}
defer f.Close()
stat, err := f.Stat()
if err != nil {
return err
}
r.logger.Info("Reading file", zap.String("path", file), zap.Int64("size", stat.Size()))
if stat.Size() == 0 {
return nil
}
if r.r == nil {
r.r = NewWALSegmentReader(f)
} else {
r.r.Reset(f)
}
defer r.r.Close()
for r.r.Next() {
entry, err := r.r.Read()
if err != nil {
n := r.r.Count()
r.logger.Info("File corrupt", zap.Error(err), zap.String("path", file), zap.Int64("pos", n))
if err := f.Truncate(n); err != nil {
return err
}
break
}
if err := cb(entry); err != nil {
return err
}
}
return r.r.Close()
}

View File

@ -1,8 +1,7 @@
package tsm1
package wal
import (
"bufio"
"bytes"
"encoding/binary"
"fmt"
"io"
@ -18,29 +17,14 @@ import (
"time"
"github.com/golang/snappy"
"github.com/influxdata/influxdb"
"github.com/influxdata/influxdb/pkg/limiter"
"github.com/influxdata/influxdb/pkg/pool"
"github.com/influxdata/influxdb/tsdb/value"
"github.com/prometheus/client_golang/prometheus"
"go.uber.org/zap"
)
// Log describes an interface for a durable disk-based log.
type Log interface {
Open() error
Close() error
Path() string
LastWriteTime() time.Time
DiskSizeBytes() int64
WriteMulti(values map[string][]Value) (int, error)
DeleteRange(keys [][]byte, min, max int64) (int, error)
CloseSegment() error
ClosedSegments() ([]string, error)
Remove(files []string) error
}
const (
// DefaultSegmentSize of 10MB is the size at which segment files will be rolled over.
DefaultSegmentSize = 10 * 1024 * 1024
@ -68,11 +52,14 @@ const (
// WriteWALEntryType indicates a write entry.
WriteWALEntryType WalEntryType = 0x01
// DeleteWALEntryType indicates a delete entry.
DeleteWALEntryType WalEntryType = 0x02
// DeleteWALEntryType indicates a delete entry. Deprecated.
_ WalEntryType = 0x02
// DeleteRangeWALEntryType indicates a delete range entry.
DeleteRangeWALEntryType WalEntryType = 0x03
// DeleteRangeWALEntryType indicates a delete range entry. Deprecated.
_ WalEntryType = 0x03
// DeleteBucketRangeWALEntryType indicates a delete bucket range entry.
DeleteBucketRangeWALEntryType WalEntryType = 0x04
)
var (
@ -97,7 +84,8 @@ type WAL struct {
mu sync.RWMutex
lastWriteTime time.Time
path string
path string
enabled bool
// write variables
currentSegmentID int
@ -120,7 +108,9 @@ type WAL struct {
// SegmentSize is the file size at which a segment file will be rotated
SegmentSize int
tracker *walTracker
tracker *walTracker
defaultMetricLabels prometheus.Labels // N.B this must not be mutated after Open is called.
limiter limiter.Fixed
}
@ -128,7 +118,8 @@ type WAL struct {
func NewWAL(path string) *WAL {
logger := zap.NewNop()
return &WAL{
path: path,
path: path,
enabled: true,
// these options should be overriden by any options in the config
SegmentSize: DefaultSegmentSize,
@ -137,7 +128,6 @@ func NewWAL(path string) *WAL {
limiter: limiter.NewFixed(defaultWaitingWALWrites),
logger: logger,
traceLogger: logger,
tracker: newWALTracker(newWALMetrics(nil), nil),
}
}
@ -154,6 +144,11 @@ func (l *WAL) WithFsyncDelay(delay time.Duration) {
l.syncDelay = delay
}
// SetEnabled sets if the WAL is enabled and should be called before the WAL is opened.
func (l *WAL) SetEnabled(enabled bool) {
l.enabled = enabled
}
// WithLogger sets the WAL's logger.
func (l *WAL) WithLogger(log *zap.Logger) {
l.logger = log.With(zap.String("service", "wal"))
@ -163,6 +158,15 @@ func (l *WAL) WithLogger(log *zap.Logger) {
}
}
// SetDefaultMetricLabels sets the default labels for metrics on the engine.
// It must be called before the Engine is opened.
func (l *WAL) SetDefaultMetricLabels(labels prometheus.Labels) {
l.defaultMetricLabels = make(prometheus.Labels, len(labels))
for k, v := range labels {
l.defaultMetricLabels[k] = v
}
}
// Path returns the directory the log was initialized with.
func (l *WAL) Path() string {
l.mu.RLock()
@ -175,6 +179,20 @@ func (l *WAL) Open() error {
l.mu.Lock()
defer l.mu.Unlock()
if !l.enabled {
return nil
}
// Initialise metrics for trackers.
mmu.Lock()
if wms == nil {
wms = newWALMetrics(l.defaultMetricLabels)
}
mmu.Unlock()
// Set the shared metrics for the tracker
l.tracker = newWALTracker(wms, l.defaultMetricLabels)
l.traceLogger.Info("tsm1 WAL starting", zap.Int("segment_size", l.SegmentSize))
l.traceLogger.Info("tsm1 WAL writing", zap.String("path", l.path))
@ -182,7 +200,7 @@ func (l *WAL) Open() error {
return err
}
segments, err := segmentFileNames(l.path)
segments, err := SegmentFileNames(l.path)
if err != nil {
return err
}
@ -299,8 +317,12 @@ func (l *WAL) sync() {
// WriteMulti writes the given values to the WAL. It returns the WAL segment ID to
// which the points were written. If an error is returned the segment ID should
// be ignored.
func (l *WAL) WriteMulti(values map[string][]Value) (int, error) {
// be ignored. If the WAL is disabled, -1 and nil is returned.
func (l *WAL) WriteMulti(values map[string][]value.Value) (int, error) {
if !l.enabled {
return -1, nil
}
entry := &WriteWALEntry{
Values: values,
}
@ -317,8 +339,13 @@ func (l *WAL) WriteMulti(values map[string][]Value) (int, error) {
// ClosedSegments returns a slice of the names of the closed segment files.
func (l *WAL) ClosedSegments() ([]string, error) {
if !l.enabled {
return nil, nil
}
l.mu.RLock()
defer l.mu.RUnlock()
// Not loading files from disk so nothing to do
if l.path == "" {
return nil, nil
@ -329,7 +356,7 @@ func (l *WAL) ClosedSegments() ([]string, error) {
currentFile = l.currentSegmentWriter.path()
}
files, err := segmentFileNames(l.path)
files, err := SegmentFileNames(l.path)
if err != nil {
return nil, err
}
@ -349,15 +376,20 @@ func (l *WAL) ClosedSegments() ([]string, error) {
// Remove deletes the given segment file paths from disk and cleans up any associated objects.
func (l *WAL) Remove(files []string) error {
if !l.enabled {
return nil
}
l.mu.Lock()
defer l.mu.Unlock()
for _, fn := range files {
l.traceLogger.Info("Removing WAL file", zap.String("path", fn))
os.RemoveAll(fn)
}
// Refresh the on-disk size stats
segments, err := segmentFileNames(l.path)
segments, err := SegmentFileNames(l.path)
if err != nil {
return err
}
@ -470,8 +502,13 @@ func (l *WAL) rollSegment() error {
// CloseSegment closes the current segment if it is non-empty and opens a new one.
func (l *WAL) CloseSegment() error {
if !l.enabled {
return nil
}
l.mu.Lock()
defer l.mu.Unlock()
if l.currentSegmentWriter == nil || l.currentSegmentWriter.size > 0 {
if err := l.newSegmentFile(); err != nil {
// A drop database or RP call could trigger this error if writes were in-flight
@ -483,32 +520,18 @@ func (l *WAL) CloseSegment() error {
return nil
}
// Delete deletes the given keys, returning the segment ID for the operation.
func (l *WAL) Delete(keys [][]byte) (int, error) {
if len(keys) == 0 {
return 0, nil
}
entry := &DeleteWALEntry{
Keys: keys,
// DeleteBucketRange deletes the data inside of the bucket between the two times, returning
// the segment ID for the operation.
func (l *WAL) DeleteBucketRange(orgID, bucketID influxdb.ID, min, max int64) (int, error) {
if !l.enabled {
return -1, nil
}
id, err := l.writeToLog(entry)
if err != nil {
return -1, err
}
return id, nil
}
// DeleteRange deletes the given keys within the given time range,
// returning the segment ID for the operation.
func (l *WAL) DeleteRange(keys [][]byte, min, max int64) (int, error) {
if len(keys) == 0 {
return 0, nil
}
entry := &DeleteRangeWALEntry{
Keys: keys,
Min: min,
Max: max,
entry := &DeleteBucketRangeWALEntry{
OrgID: orgID,
BucketID: bucketID,
Min: min,
Max: max,
}
id, err := l.writeToLog(entry)
@ -523,6 +546,10 @@ func (l *WAL) Close() error {
l.mu.Lock()
defer l.mu.Unlock()
if !l.enabled {
return nil
}
l.once.Do(func() {
// Close, but don't set to nil so future goroutines can still be signaled
l.traceLogger.Info("Closing WAL file", zap.String("path", l.path))
@ -538,8 +565,8 @@ func (l *WAL) Close() error {
return nil
}
// segmentFileNames will return all files that are WAL segment files in sorted order by ascending ID.
func segmentFileNames(dir string) ([]string, error) {
// SegmentFileNames will return all files that are WAL segment files in sorted order by ascending ID.
func SegmentFileNames(dir string) ([]string, error) {
names, err := filepath.Glob(filepath.Join(dir, fmt.Sprintf("%s*.%s", WALFilePrefix, WALFileExtension)))
if err != nil {
return nil, err
@ -666,10 +693,11 @@ type WALEntry interface {
// WriteWALEntry represents a write of points.
type WriteWALEntry struct {
Values map[string][]Value
Values map[string][]value.Value
sz int
}
// MarshalSize returns the number of bytes the entry takes when marshaled.
func (w *WriteWALEntry) MarshalSize() int {
if w.sz > 0 || len(w.Values) == 0 {
return w.sz
@ -687,17 +715,17 @@ func (w *WriteWALEntry) MarshalSize() int {
encLen += 8 * len(v) // timestamps (8)
switch v[0].(type) {
case FloatValue, IntegerValue, UnsignedValue:
case value.FloatValue, value.IntegerValue, value.UnsignedValue:
encLen += 8 * len(v)
case BooleanValue:
case value.BooleanValue:
encLen += 1 * len(v)
case StringValue:
case value.StringValue:
for _, vv := range v {
str, ok := vv.(StringValue)
str, ok := vv.(value.StringValue)
if !ok {
return 0
}
encLen += 4 + len(str.value)
encLen += 4 + len(str.RawValue())
}
default:
return 0
@ -746,15 +774,15 @@ func (w *WriteWALEntry) Encode(dst []byte) ([]byte, error) {
for k, v := range w.Values {
switch v[0].(type) {
case FloatValue:
case value.FloatValue:
curType = float64EntryType
case IntegerValue:
case value.IntegerValue:
curType = integerEntryType
case UnsignedValue:
case value.UnsignedValue:
curType = unsignedEntryType
case BooleanValue:
case value.BooleanValue:
curType = booleanEntryType
case StringValue:
case value.StringValue:
curType = stringEntryType
default:
return nil, fmt.Errorf("unsupported value type: %T", v[0])
@ -774,41 +802,41 @@ func (w *WriteWALEntry) Encode(dst []byte) ([]byte, error) {
n += 8
switch vv := vv.(type) {
case FloatValue:
case value.FloatValue:
if curType != float64EntryType {
return nil, fmt.Errorf("incorrect value found in %T slice: %T", v[0].Value(), vv)
}
binary.BigEndian.PutUint64(dst[n:n+8], math.Float64bits(vv.value))
binary.BigEndian.PutUint64(dst[n:n+8], math.Float64bits(vv.RawValue()))
n += 8
case IntegerValue:
case value.IntegerValue:
if curType != integerEntryType {
return nil, fmt.Errorf("incorrect value found in %T slice: %T", v[0].Value(), vv)
}
binary.BigEndian.PutUint64(dst[n:n+8], uint64(vv.value))
binary.BigEndian.PutUint64(dst[n:n+8], uint64(vv.RawValue()))
n += 8
case UnsignedValue:
case value.UnsignedValue:
if curType != unsignedEntryType {
return nil, fmt.Errorf("incorrect value found in %T slice: %T", v[0].Value(), vv)
}
binary.BigEndian.PutUint64(dst[n:n+8], uint64(vv.value))
binary.BigEndian.PutUint64(dst[n:n+8], uint64(vv.RawValue()))
n += 8
case BooleanValue:
case value.BooleanValue:
if curType != booleanEntryType {
return nil, fmt.Errorf("incorrect value found in %T slice: %T", v[0].Value(), vv)
}
if vv.value {
if vv.RawValue() {
dst[n] = 1
} else {
dst[n] = 0
}
n++
case StringValue:
case value.StringValue:
if curType != stringEntryType {
return nil, fmt.Errorf("incorrect value found in %T slice: %T", v[0].Value(), vv)
}
binary.BigEndian.PutUint32(dst[n:n+4], uint32(len(vv.value)))
binary.BigEndian.PutUint32(dst[n:n+4], uint32(len(vv.RawValue())))
n += 4
n += copy(dst[n:], vv.value)
n += copy(dst[n:], vv.RawValue())
default:
return nil, fmt.Errorf("unsupported value found in %T slice: %T", v[0].Value(), vv)
}
@ -863,13 +891,13 @@ func (w *WriteWALEntry) UnmarshalBinary(b []byte) error {
return ErrWALCorrupt
}
values := make([]Value, 0, nvals)
values := make([]value.Value, 0, nvals)
for j := 0; j < nvals; j++ {
un := int64(binary.BigEndian.Uint64(b[i : i+8]))
i += 8
v := math.Float64frombits((binary.BigEndian.Uint64(b[i : i+8])))
i += 8
values = append(values, NewFloatValue(un, v))
values = append(values, value.NewFloatValue(un, v))
}
w.Values[k] = values
case integerEntryType:
@ -877,13 +905,13 @@ func (w *WriteWALEntry) UnmarshalBinary(b []byte) error {
return ErrWALCorrupt
}
values := make([]Value, 0, nvals)
values := make([]value.Value, 0, nvals)
for j := 0; j < nvals; j++ {
un := int64(binary.BigEndian.Uint64(b[i : i+8]))
i += 8
v := int64(binary.BigEndian.Uint64(b[i : i+8]))
i += 8
values = append(values, NewIntegerValue(un, v))
values = append(values, value.NewIntegerValue(un, v))
}
w.Values[k] = values
@ -892,13 +920,13 @@ func (w *WriteWALEntry) UnmarshalBinary(b []byte) error {
return ErrWALCorrupt
}
values := make([]Value, 0, nvals)
values := make([]value.Value, 0, nvals)
for j := 0; j < nvals; j++ {
un := int64(binary.BigEndian.Uint64(b[i : i+8]))
i += 8
v := binary.BigEndian.Uint64(b[i : i+8])
i += 8
values = append(values, NewUnsignedValue(un, v))
values = append(values, value.NewUnsignedValue(un, v))
}
w.Values[k] = values
@ -907,7 +935,7 @@ func (w *WriteWALEntry) UnmarshalBinary(b []byte) error {
return ErrWALCorrupt
}
values := make([]Value, 0, nvals)
values := make([]value.Value, 0, nvals)
for j := 0; j < nvals; j++ {
un := int64(binary.BigEndian.Uint64(b[i : i+8]))
i += 8
@ -915,15 +943,15 @@ func (w *WriteWALEntry) UnmarshalBinary(b []byte) error {
v := b[i]
i += 1
if v == 1 {
values = append(values, NewBooleanValue(un, true))
values = append(values, value.NewBooleanValue(un, true))
} else {
values = append(values, NewBooleanValue(un, false))
values = append(values, value.NewBooleanValue(un, false))
}
}
w.Values[k] = values
case stringEntryType:
values := make([]Value, 0, nvals)
values := make([]value.Value, 0, nvals)
for j := 0; j < nvals; j++ {
if i+12 > len(b) {
return ErrWALCorrupt
@ -945,7 +973,7 @@ func (w *WriteWALEntry) UnmarshalBinary(b []byte) error {
v := string(b[i : i+length])
i += length
values = append(values, NewStringValue(un, v))
values = append(values, value.NewStringValue(un, v))
}
w.Values[k] = values
@ -961,152 +989,70 @@ func (w *WriteWALEntry) Type() WalEntryType {
return WriteWALEntryType
}
// DeleteWALEntry represents the deletion of multiple series.
type DeleteWALEntry struct {
Keys [][]byte
sz int
}
// MarshalBinary returns a binary representation of the entry in a new byte slice.
func (w *DeleteWALEntry) MarshalBinary() ([]byte, error) {
b := make([]byte, w.MarshalSize())
return w.Encode(b)
}
// UnmarshalBinary deserializes the byte slice into w.
func (w *DeleteWALEntry) UnmarshalBinary(b []byte) error {
if len(b) == 0 {
return nil
}
// b originates from a pool. Copy what needs to be retained.
buf := make([]byte, len(b))
copy(buf, b)
w.Keys = bytes.Split(buf, []byte("\n"))
return nil
}
func (w *DeleteWALEntry) MarshalSize() int {
if w.sz > 0 || len(w.Keys) == 0 {
return w.sz
}
encLen := len(w.Keys) // newlines
for _, k := range w.Keys {
encLen += len(k)
}
w.sz = encLen
return encLen
}
// Encode converts the DeleteWALEntry into a byte slice, appending to dst.
func (w *DeleteWALEntry) Encode(dst []byte) ([]byte, error) {
sz := w.MarshalSize()
if len(dst) < sz {
dst = make([]byte, sz)
}
var n int
for _, k := range w.Keys {
n += copy(dst[n:], k)
n += copy(dst[n:], "\n")
}
// We return n-1 to strip off the last newline so that unmarshalling the value
// does not produce an empty string
return []byte(dst[:n-1]), nil
}
// Type returns DeleteWALEntryType.
func (w *DeleteWALEntry) Type() WalEntryType {
return DeleteWALEntryType
}
// DeleteRangeWALEntry represents the deletion of multiple series.
type DeleteRangeWALEntry struct {
Keys [][]byte
// DeleteBucketRangeWALEntry represents the deletion of data in a bucket.
type DeleteBucketRangeWALEntry struct {
OrgID influxdb.ID
BucketID influxdb.ID
Min, Max int64
sz int
}
// MarshalBinary returns a binary representation of the entry in a new byte slice.
func (w *DeleteRangeWALEntry) MarshalBinary() ([]byte, error) {
func (w *DeleteBucketRangeWALEntry) MarshalBinary() ([]byte, error) {
b := make([]byte, w.MarshalSize())
return w.Encode(b)
}
// UnmarshalBinary deserializes the byte slice into w.
func (w *DeleteRangeWALEntry) UnmarshalBinary(b []byte) error {
if len(b) < 16 {
func (w *DeleteBucketRangeWALEntry) UnmarshalBinary(b []byte) error {
if len(b) != 2*influxdb.IDLength+16 {
return ErrWALCorrupt
}
w.Min = int64(binary.BigEndian.Uint64(b[:8]))
w.Max = int64(binary.BigEndian.Uint64(b[8:16]))
i := 16
for i < len(b) {
if i+4 > len(b) {
return ErrWALCorrupt
}
sz := int(binary.BigEndian.Uint32(b[i : i+4]))
i += 4
if i+sz > len(b) {
return ErrWALCorrupt
}
// b originates from a pool. Copy what needs to be retained.
buf := make([]byte, sz)
copy(buf, b[i:i+sz])
w.Keys = append(w.Keys, buf)
i += sz
if err := w.OrgID.Decode(b[0:influxdb.IDLength]); err != nil {
return err
}
if err := w.BucketID.Decode(b[influxdb.IDLength : 2*influxdb.IDLength]); err != nil {
return err
}
w.Min = int64(binary.BigEndian.Uint64(b[2*influxdb.IDLength : 2*influxdb.IDLength+8]))
w.Max = int64(binary.BigEndian.Uint64(b[2*influxdb.IDLength+8 : 2*influxdb.IDLength+16]))
return nil
}
func (w *DeleteRangeWALEntry) MarshalSize() int {
if w.sz > 0 {
return w.sz
}
sz := 16 + len(w.Keys)*4
for _, k := range w.Keys {
sz += len(k)
}
w.sz = sz
return sz
// MarshalSize returns the number of bytes the entry takes when marshaled.
func (w *DeleteBucketRangeWALEntry) MarshalSize() int {
return 2*influxdb.IDLength + 16
}
// Encode converts the DeleteRangeWALEntry into a byte slice, appending to b.
func (w *DeleteRangeWALEntry) Encode(b []byte) ([]byte, error) {
// Encode converts the entry into a byte stream using b if it is large enough.
// If b is too small, a newly allocated slice is returned.
func (w *DeleteBucketRangeWALEntry) Encode(b []byte) ([]byte, error) {
sz := w.MarshalSize()
if len(b) < sz {
b = make([]byte, sz)
}
binary.BigEndian.PutUint64(b[:8], uint64(w.Min))
binary.BigEndian.PutUint64(b[8:16], uint64(w.Max))
i := 16
for _, k := range w.Keys {
binary.BigEndian.PutUint32(b[i:i+4], uint32(len(k)))
i += 4
i += copy(b[i:], k)
orgID, err := w.OrgID.Encode()
if err != nil {
return nil, err
}
bucketID, err := w.BucketID.Encode()
if err != nil {
return nil, err
}
return b[:i], nil
copy(b, orgID)
copy(b[influxdb.IDLength:], bucketID)
binary.BigEndian.PutUint64(b[2*influxdb.IDLength:], uint64(w.Min))
binary.BigEndian.PutUint64(b[2*influxdb.IDLength+8:], uint64(w.Max))
return b[:sz], nil
}
// Type returns DeleteRangeWALEntryType.
func (w *DeleteRangeWALEntry) Type() WalEntryType {
return DeleteRangeWALEntryType
// Type returns DeleteBucketRangeWALEntryType.
func (w *DeleteBucketRangeWALEntry) Type() WalEntryType {
return DeleteBucketRangeWALEntryType
}
// WALSegmentWriter writes WAL segments.
@ -1250,12 +1196,10 @@ func (r *WALSegmentReader) Next() bool {
switch WalEntryType(entryType) {
case WriteWALEntryType:
r.entry = &WriteWALEntry{
Values: make(map[string][]Value),
Values: make(map[string][]value.Value),
}
case DeleteWALEntryType:
r.entry = &DeleteWALEntry{}
case DeleteRangeWALEntryType:
r.entry = &DeleteRangeWALEntry{}
case DeleteBucketRangeWALEntryType:
r.entry = &DeleteBucketRangeWALEntry{}
default:
r.err = fmt.Errorf("unknown wal entry type: %v", entryType)
return true
@ -1310,20 +1254,3 @@ func idFromFileName(name string) (int, error) {
return int(id), err
}
// NopWAL implements the Log interface and provides a no-op WAL implementation.
type NopWAL struct{}
func (w NopWAL) Open() error { return nil }
func (w NopWAL) Close() error { return nil }
func (w NopWAL) Path() string { return "" }
func (w NopWAL) LastWriteTime() time.Time { return time.Time{} }
func (w NopWAL) DiskSizeBytes() int64 { return 0 }
func (w NopWAL) WriteMulti(values map[string][]Value) (int, error) { return 0, nil }
func (w NopWAL) DeleteRange(keys [][]byte, min, max int64) (int, error) { return 0, nil }
func (w NopWAL) CloseSegment() error { return nil }
func (w NopWAL) ClosedSegments() ([]string, error) { return nil, nil }
func (w NopWAL) Remove(files []string) error { return nil }

580
storage/wal/wal_test.go Normal file
View File

@ -0,0 +1,580 @@
package wal
import (
"fmt"
"io"
"math/rand"
"os"
"reflect"
"testing"
"github.com/golang/snappy"
"github.com/influxdata/influxdb"
"github.com/influxdata/influxdb/tsdb/value"
)
func TestWALWriter_WriteMulti_Single(t *testing.T) {
dir := MustTempDir()
defer os.RemoveAll(dir)
f := MustTempFile(dir)
w := NewWALSegmentWriter(f)
p1 := value.NewValue(1, 1.1)
p2 := value.NewValue(1, int64(1))
p3 := value.NewValue(1, true)
p4 := value.NewValue(1, "string")
p5 := value.NewValue(1, ^uint64(0))
values := map[string][]value.Value{
"cpu,host=A#!~#float": []value.Value{p1},
"cpu,host=A#!~#int": []value.Value{p2},
"cpu,host=A#!~#bool": []value.Value{p3},
"cpu,host=A#!~#string": []value.Value{p4},
"cpu,host=A#!~#unsigned": []value.Value{p5},
}
entry := &WriteWALEntry{
Values: values,
}
if err := w.Write(mustMarshalEntry(entry)); err != nil {
fatal(t, "write points", err)
}
if err := w.Flush(); err != nil {
fatal(t, "flush", err)
}
if _, err := f.Seek(0, io.SeekStart); err != nil {
fatal(t, "seek", err)
}
r := NewWALSegmentReader(f)
if !r.Next() {
t.Fatalf("expected next, got false")
}
we, err := r.Read()
if err != nil {
fatal(t, "read entry", err)
}
e, ok := we.(*WriteWALEntry)
if !ok {
t.Fatalf("expected WriteWALEntry: got %#v", e)
}
for k, v := range e.Values {
for i, vv := range v {
if got, exp := vv.String(), values[k][i].String(); got != exp {
t.Fatalf("points mismatch: got %v, exp %v", got, exp)
}
}
}
if n := r.Count(); n != MustReadFileSize(f) {
t.Fatalf("wrong count of bytes read, got %d, exp %d", n, MustReadFileSize(f))
}
}
func TestWALWriter_WriteMulti_LargeBatch(t *testing.T) {
dir := MustTempDir()
defer os.RemoveAll(dir)
f := MustTempFile(dir)
w := NewWALSegmentWriter(f)
var points []value.Value
for i := 0; i < 100000; i++ {
points = append(points, value.NewValue(int64(i), int64(1)))
}
values := map[string][]value.Value{
"cpu,host=A,server=01,foo=bar,tag=really-long#!~#float": points,
"mem,host=A,server=01,foo=bar,tag=really-long#!~#float": points,
}
entry := &WriteWALEntry{
Values: values,
}
if err := w.Write(mustMarshalEntry(entry)); err != nil {
fatal(t, "write points", err)
}
if err := w.Flush(); err != nil {
fatal(t, "flush", err)
}
if _, err := f.Seek(0, io.SeekStart); err != nil {
fatal(t, "seek", err)
}
r := NewWALSegmentReader(f)
if !r.Next() {
t.Fatalf("expected next, got false")
}
we, err := r.Read()
if err != nil {
fatal(t, "read entry", err)
}
e, ok := we.(*WriteWALEntry)
if !ok {
t.Fatalf("expected WriteWALEntry: got %#v", e)
}
for k, v := range e.Values {
for i, vv := range v {
if got, exp := vv.String(), values[k][i].String(); got != exp {
t.Fatalf("points mismatch: got %v, exp %v", got, exp)
}
}
}
if n := r.Count(); n != MustReadFileSize(f) {
t.Fatalf("wrong count of bytes read, got %d, exp %d", n, MustReadFileSize(f))
}
}
func TestWALWriter_WriteMulti_Multiple(t *testing.T) {
dir := MustTempDir()
defer os.RemoveAll(dir)
f := MustTempFile(dir)
w := NewWALSegmentWriter(f)
p1 := value.NewValue(1, int64(1))
p2 := value.NewValue(1, int64(2))
exp := []struct {
key string
values []value.Value
}{
{"cpu,host=A#!~#value", []value.Value{p1}},
{"cpu,host=B#!~#value", []value.Value{p2}},
}
for _, v := range exp {
entry := &WriteWALEntry{
Values: map[string][]value.Value{v.key: v.values},
}
if err := w.Write(mustMarshalEntry(entry)); err != nil {
fatal(t, "write points", err)
}
if err := w.Flush(); err != nil {
fatal(t, "flush", err)
}
}
// Seek back to the beinning of the file for reading
if _, err := f.Seek(0, io.SeekStart); err != nil {
fatal(t, "seek", err)
}
r := NewWALSegmentReader(f)
for _, ep := range exp {
if !r.Next() {
t.Fatalf("expected next, got false")
}
we, err := r.Read()
if err != nil {
fatal(t, "read entry", err)
}
e, ok := we.(*WriteWALEntry)
if !ok {
t.Fatalf("expected WriteWALEntry: got %#v", e)
}
for k, v := range e.Values {
if got, exp := k, ep.key; got != exp {
t.Fatalf("key mismatch. got %v, exp %v", got, exp)
}
if got, exp := len(v), len(ep.values); got != exp {
t.Fatalf("values length mismatch: got %v, exp %v", got, exp)
}
for i, vv := range v {
if got, exp := vv.String(), ep.values[i].String(); got != exp {
t.Fatalf("points mismatch: got %v, exp %v", got, exp)
}
}
}
}
if n := r.Count(); n != MustReadFileSize(f) {
t.Fatalf("wrong count of bytes read, got %d, exp %d", n, MustReadFileSize(f))
}
}
func TestWALWriter_DeleteBucketRange(t *testing.T) {
dir := MustTempDir()
defer os.RemoveAll(dir)
f := MustTempFile(dir)
w := NewWALSegmentWriter(f)
entry := &DeleteBucketRangeWALEntry{
OrgID: influxdb.ID(1),
BucketID: influxdb.ID(2),
Min: 3,
Max: 4,
}
if err := w.Write(mustMarshalEntry(entry)); err != nil {
fatal(t, "write points", err)
}
if err := w.Flush(); err != nil {
fatal(t, "flush", err)
}
if _, err := f.Seek(0, io.SeekStart); err != nil {
fatal(t, "seek", err)
}
r := NewWALSegmentReader(f)
if !r.Next() {
t.Fatalf("expected next, got false")
}
we, err := r.Read()
if err != nil {
fatal(t, "read entry", err)
}
e, ok := we.(*DeleteBucketRangeWALEntry)
if !ok {
t.Fatalf("expected WriteWALEntry: got %#v", e)
}
if !reflect.DeepEqual(entry, e) {
t.Fatalf("expected %+v but got %+v", entry, e)
}
}
func TestWAL_ClosedSegments(t *testing.T) {
dir := MustTempDir()
defer os.RemoveAll(dir)
w := NewWAL(dir)
if err := w.Open(); err != nil {
t.Fatalf("error opening WAL: %v", err)
}
files, err := w.ClosedSegments()
if err != nil {
t.Fatalf("error getting closed segments: %v", err)
}
if got, exp := len(files), 0; got != exp {
t.Fatalf("close segment length mismatch: got %v, exp %v", got, exp)
}
if _, err := w.WriteMulti(map[string][]value.Value{
"cpu,host=A#!~#value": []value.Value{
value.NewValue(1, 1.1),
},
}); err != nil {
t.Fatalf("error writing points: %v", err)
}
if err := w.Close(); err != nil {
t.Fatalf("error closing wal: %v", err)
}
// Re-open the WAL
w = NewWAL(dir)
defer w.Close()
if err := w.Open(); err != nil {
t.Fatalf("error opening WAL: %v", err)
}
files, err = w.ClosedSegments()
if err != nil {
t.Fatalf("error getting closed segments: %v", err)
}
if got, exp := len(files), 0; got != exp {
t.Fatalf("close segment length mismatch: got %v, exp %v", got, exp)
}
}
func TestWALWriter_Corrupt(t *testing.T) {
dir := MustTempDir()
defer os.RemoveAll(dir)
f := MustTempFile(dir)
w := NewWALSegmentWriter(f)
corruption := []byte{1, 4, 0, 0, 0}
p1 := value.NewValue(1, 1.1)
values := map[string][]value.Value{
"cpu,host=A#!~#float": []value.Value{p1},
}
entry := &WriteWALEntry{
Values: values,
}
if err := w.Write(mustMarshalEntry(entry)); err != nil {
fatal(t, "write points", err)
}
if err := w.Flush(); err != nil {
fatal(t, "flush", err)
}
// Write some random bytes to the file to simulate corruption.
if _, err := f.Write(corruption); err != nil {
fatal(t, "corrupt WAL segment", err)
}
// Create the WAL segment reader.
if _, err := f.Seek(0, io.SeekStart); err != nil {
fatal(t, "seek", err)
}
r := NewWALSegmentReader(f)
// Try to decode two entries.
if !r.Next() {
t.Fatalf("expected next, got false")
}
if _, err := r.Read(); err != nil {
fatal(t, "read entry", err)
}
if !r.Next() {
t.Fatalf("expected next, got false")
}
if _, err := r.Read(); err == nil {
fatal(t, "read entry did not return err", nil)
}
// Count should only return size of valid data.
expCount := MustReadFileSize(f) - int64(len(corruption))
if n := r.Count(); n != expCount {
t.Fatalf("wrong count of bytes read, got %d, exp %d", n, expCount)
}
}
// Reproduces a `panic: runtime error: makeslice: cap out of range` when run with
// GOARCH=386 go test -run TestWALSegmentReader_Corrupt -v ./tsdb/engine/tsm1/
func TestWALSegmentReader_Corrupt(t *testing.T) {
dir := MustTempDir()
defer os.RemoveAll(dir)
f := MustTempFile(dir)
w := NewWALSegmentWriter(f)
p4 := value.NewValue(1, "string")
values := map[string][]value.Value{
"cpu,host=A#!~#string": []value.Value{p4, p4},
}
entry := &WriteWALEntry{
Values: values,
}
typ, b := mustMarshalEntry(entry)
// This causes the nvals field to overflow on 32 bit systems which produces a
// negative count and a panic when reading the segment.
b[25] = 255
if err := w.Write(typ, b); err != nil {
fatal(t, "write points", err)
}
if err := w.Flush(); err != nil {
fatal(t, "flush", err)
}
// Create the WAL segment reader.
if _, err := f.Seek(0, io.SeekStart); err != nil {
fatal(t, "seek", err)
}
r := NewWALSegmentReader(f)
defer r.Close()
// Try to decode two entries.
for r.Next() {
r.Read()
}
}
func TestWriteWALSegment_UnmarshalBinary_WriteWALCorrupt(t *testing.T) {
p1 := value.NewValue(1, 1.1)
p2 := value.NewValue(1, int64(1))
p3 := value.NewValue(1, true)
p4 := value.NewValue(1, "string")
p5 := value.NewValue(1, uint64(1))
values := map[string][]value.Value{
"cpu,host=A#!~#float": []value.Value{p1, p1},
"cpu,host=A#!~#int": []value.Value{p2, p2},
"cpu,host=A#!~#bool": []value.Value{p3, p3},
"cpu,host=A#!~#string": []value.Value{p4, p4},
"cpu,host=A#!~#unsigned": []value.Value{p5, p5},
}
w := &WriteWALEntry{
Values: values,
}
b, err := w.MarshalBinary()
if err != nil {
t.Fatalf("unexpected error, got %v", err)
}
// Test every possible truncation of a write WAL entry
for i := 0; i < len(b); i++ {
// re-allocated to ensure capacity would be exceed if slicing
truncated := make([]byte, i)
copy(truncated, b[:i])
err := w.UnmarshalBinary(truncated)
if err != nil && err != ErrWALCorrupt {
t.Fatalf("unexpected error: %v", err)
}
}
}
func TestDeleteBucketRangeWALEntry_UnmarshalBinary(t *testing.T) {
for i := 0; i < 1000; i++ {
in := &DeleteBucketRangeWALEntry{
OrgID: influxdb.ID(rand.Int63()) + 1,
BucketID: influxdb.ID(rand.Int63()) + 1,
Min: rand.Int63(),
Max: rand.Int63(),
}
b, err := in.MarshalBinary()
if err != nil {
t.Fatalf("unexpected error, got %v", err)
}
out := &DeleteBucketRangeWALEntry{}
if err := out.UnmarshalBinary(b); err != nil {
t.Fatalf("%v", err)
}
if !reflect.DeepEqual(in, out) {
t.Errorf("got %+v, expected %+v", out, in)
}
}
}
func TestWriteWALSegment_UnmarshalBinary_DeleteBucketRangeWALCorrupt(t *testing.T) {
w := &DeleteBucketRangeWALEntry{
OrgID: influxdb.ID(1),
BucketID: influxdb.ID(2),
Min: 3,
Max: 4,
}
b, err := w.MarshalBinary()
if err != nil {
t.Fatalf("unexpected error, got %v", err)
}
// Test every possible truncation of a write WAL entry
for i := 0; i < len(b); i++ {
// re-allocated to ensure capacity would be exceed if slicing
truncated := make([]byte, i)
copy(truncated, b[:i])
err := w.UnmarshalBinary(truncated)
if err != nil && err != ErrWALCorrupt {
t.Fatalf("unexpected error: %v", err)
}
}
}
func BenchmarkWALSegmentWriter(b *testing.B) {
points := map[string][]value.Value{}
for i := 0; i < 5000; i++ {
k := "cpu,host=A#!~#value"
points[k] = append(points[k], value.NewValue(int64(i), 1.1))
}
dir := MustTempDir()
defer os.RemoveAll(dir)
f := MustTempFile(dir)
w := NewWALSegmentWriter(f)
write := &WriteWALEntry{
Values: points,
}
b.ResetTimer()
for i := 0; i < b.N; i++ {
if err := w.Write(mustMarshalEntry(write)); err != nil {
b.Fatalf("unexpected error writing entry: %v", err)
}
}
}
func BenchmarkWALSegmentReader(b *testing.B) {
points := map[string][]value.Value{}
for i := 0; i < 5000; i++ {
k := "cpu,host=A#!~#value"
points[k] = append(points[k], value.NewValue(int64(i), 1.1))
}
dir := MustTempDir()
defer os.RemoveAll(dir)
f := MustTempFile(dir)
w := NewWALSegmentWriter(f)
write := &WriteWALEntry{
Values: points,
}
for i := 0; i < 100; i++ {
if err := w.Write(mustMarshalEntry(write)); err != nil {
b.Fatalf("unexpected error writing entry: %v", err)
}
}
r := NewWALSegmentReader(f)
b.ResetTimer()
for i := 0; i < b.N; i++ {
b.StopTimer()
f.Seek(0, io.SeekStart)
b.StartTimer()
for r.Next() {
_, err := r.Read()
if err != nil {
b.Fatalf("unexpected error reading entry: %v", err)
}
}
}
}
// MustReadFileSize returns the size of the file, or panics.
func MustReadFileSize(f *os.File) int64 {
stat, err := os.Stat(f.Name())
if err != nil {
panic(fmt.Sprintf("failed to get size of file at %s: %s", f.Name(), err.Error()))
}
return stat.Size()
}
func mustMarshalEntry(entry WALEntry) (WalEntryType, []byte) {
bytes := make([]byte, 1024<<2)
b, err := entry.Encode(bytes)
if err != nil {
panic(fmt.Sprintf("error encoding: %v", err))
}
return entry.Type(), snappy.Encode(b, b)
}

View File

@ -83,12 +83,12 @@ func (c *floatArrayAscendingCursor) Next() *tsdb.FloatArray {
tkey := tvals.Timestamps[c.tsm.pos]
if ckey == tkey {
c.res.Timestamps[pos] = ckey
c.res.Values[pos] = cvals[c.cache.pos].(FloatValue).value
c.res.Values[pos] = cvals[c.cache.pos].(FloatValue).RawValue()
c.cache.pos++
c.tsm.pos++
} else if ckey < tkey {
c.res.Timestamps[pos] = ckey
c.res.Values[pos] = cvals[c.cache.pos].(FloatValue).value
c.res.Values[pos] = cvals[c.cache.pos].(FloatValue).RawValue()
c.cache.pos++
} else {
c.res.Timestamps[pos] = tkey
@ -126,7 +126,7 @@ func (c *floatArrayAscendingCursor) Next() *tsdb.FloatArray {
// TSM was exhausted
for pos < len(c.res.Timestamps) && c.cache.pos < len(cvals) {
c.res.Timestamps[pos] = cvals[c.cache.pos].UnixNano()
c.res.Values[pos] = cvals[c.cache.pos].(FloatValue).value
c.res.Values[pos] = cvals[c.cache.pos].(FloatValue).RawValue()
pos++
c.cache.pos++
}
@ -247,12 +247,12 @@ func (c *floatArrayDescendingCursor) Next() *tsdb.FloatArray {
tkey := tvals.Timestamps[c.tsm.pos]
if ckey == tkey {
c.res.Timestamps[pos] = ckey
c.res.Values[pos] = cvals[c.cache.pos].(FloatValue).value
c.res.Values[pos] = cvals[c.cache.pos].(FloatValue).RawValue()
c.cache.pos--
c.tsm.pos--
} else if ckey > tkey {
c.res.Timestamps[pos] = ckey
c.res.Values[pos] = cvals[c.cache.pos].(FloatValue).value
c.res.Values[pos] = cvals[c.cache.pos].(FloatValue).RawValue()
c.cache.pos--
} else {
c.res.Timestamps[pos] = tkey
@ -285,7 +285,7 @@ func (c *floatArrayDescendingCursor) Next() *tsdb.FloatArray {
// TSM was exhausted
for pos < len(c.res.Timestamps) && c.cache.pos >= 0 {
c.res.Timestamps[pos] = cvals[c.cache.pos].UnixNano()
c.res.Values[pos] = cvals[c.cache.pos].(FloatValue).value
c.res.Values[pos] = cvals[c.cache.pos].(FloatValue).RawValue()
pos++
c.cache.pos--
}
@ -391,12 +391,12 @@ func (c *integerArrayAscendingCursor) Next() *tsdb.IntegerArray {
tkey := tvals.Timestamps[c.tsm.pos]
if ckey == tkey {
c.res.Timestamps[pos] = ckey
c.res.Values[pos] = cvals[c.cache.pos].(IntegerValue).value
c.res.Values[pos] = cvals[c.cache.pos].(IntegerValue).RawValue()
c.cache.pos++
c.tsm.pos++
} else if ckey < tkey {
c.res.Timestamps[pos] = ckey
c.res.Values[pos] = cvals[c.cache.pos].(IntegerValue).value
c.res.Values[pos] = cvals[c.cache.pos].(IntegerValue).RawValue()
c.cache.pos++
} else {
c.res.Timestamps[pos] = tkey
@ -434,7 +434,7 @@ func (c *integerArrayAscendingCursor) Next() *tsdb.IntegerArray {
// TSM was exhausted
for pos < len(c.res.Timestamps) && c.cache.pos < len(cvals) {
c.res.Timestamps[pos] = cvals[c.cache.pos].UnixNano()
c.res.Values[pos] = cvals[c.cache.pos].(IntegerValue).value
c.res.Values[pos] = cvals[c.cache.pos].(IntegerValue).RawValue()
pos++
c.cache.pos++
}
@ -555,12 +555,12 @@ func (c *integerArrayDescendingCursor) Next() *tsdb.IntegerArray {
tkey := tvals.Timestamps[c.tsm.pos]
if ckey == tkey {
c.res.Timestamps[pos] = ckey
c.res.Values[pos] = cvals[c.cache.pos].(IntegerValue).value
c.res.Values[pos] = cvals[c.cache.pos].(IntegerValue).RawValue()
c.cache.pos--
c.tsm.pos--
} else if ckey > tkey {
c.res.Timestamps[pos] = ckey
c.res.Values[pos] = cvals[c.cache.pos].(IntegerValue).value
c.res.Values[pos] = cvals[c.cache.pos].(IntegerValue).RawValue()
c.cache.pos--
} else {
c.res.Timestamps[pos] = tkey
@ -593,7 +593,7 @@ func (c *integerArrayDescendingCursor) Next() *tsdb.IntegerArray {
// TSM was exhausted
for pos < len(c.res.Timestamps) && c.cache.pos >= 0 {
c.res.Timestamps[pos] = cvals[c.cache.pos].UnixNano()
c.res.Values[pos] = cvals[c.cache.pos].(IntegerValue).value
c.res.Values[pos] = cvals[c.cache.pos].(IntegerValue).RawValue()
pos++
c.cache.pos--
}
@ -699,12 +699,12 @@ func (c *unsignedArrayAscendingCursor) Next() *tsdb.UnsignedArray {
tkey := tvals.Timestamps[c.tsm.pos]
if ckey == tkey {
c.res.Timestamps[pos] = ckey
c.res.Values[pos] = cvals[c.cache.pos].(UnsignedValue).value
c.res.Values[pos] = cvals[c.cache.pos].(UnsignedValue).RawValue()
c.cache.pos++
c.tsm.pos++
} else if ckey < tkey {
c.res.Timestamps[pos] = ckey
c.res.Values[pos] = cvals[c.cache.pos].(UnsignedValue).value
c.res.Values[pos] = cvals[c.cache.pos].(UnsignedValue).RawValue()
c.cache.pos++
} else {
c.res.Timestamps[pos] = tkey
@ -742,7 +742,7 @@ func (c *unsignedArrayAscendingCursor) Next() *tsdb.UnsignedArray {
// TSM was exhausted
for pos < len(c.res.Timestamps) && c.cache.pos < len(cvals) {
c.res.Timestamps[pos] = cvals[c.cache.pos].UnixNano()
c.res.Values[pos] = cvals[c.cache.pos].(UnsignedValue).value
c.res.Values[pos] = cvals[c.cache.pos].(UnsignedValue).RawValue()
pos++
c.cache.pos++
}
@ -863,12 +863,12 @@ func (c *unsignedArrayDescendingCursor) Next() *tsdb.UnsignedArray {
tkey := tvals.Timestamps[c.tsm.pos]
if ckey == tkey {
c.res.Timestamps[pos] = ckey
c.res.Values[pos] = cvals[c.cache.pos].(UnsignedValue).value
c.res.Values[pos] = cvals[c.cache.pos].(UnsignedValue).RawValue()
c.cache.pos--
c.tsm.pos--
} else if ckey > tkey {
c.res.Timestamps[pos] = ckey
c.res.Values[pos] = cvals[c.cache.pos].(UnsignedValue).value
c.res.Values[pos] = cvals[c.cache.pos].(UnsignedValue).RawValue()
c.cache.pos--
} else {
c.res.Timestamps[pos] = tkey
@ -901,7 +901,7 @@ func (c *unsignedArrayDescendingCursor) Next() *tsdb.UnsignedArray {
// TSM was exhausted
for pos < len(c.res.Timestamps) && c.cache.pos >= 0 {
c.res.Timestamps[pos] = cvals[c.cache.pos].UnixNano()
c.res.Values[pos] = cvals[c.cache.pos].(UnsignedValue).value
c.res.Values[pos] = cvals[c.cache.pos].(UnsignedValue).RawValue()
pos++
c.cache.pos--
}
@ -1007,12 +1007,12 @@ func (c *stringArrayAscendingCursor) Next() *tsdb.StringArray {
tkey := tvals.Timestamps[c.tsm.pos]
if ckey == tkey {
c.res.Timestamps[pos] = ckey
c.res.Values[pos] = cvals[c.cache.pos].(StringValue).value
c.res.Values[pos] = cvals[c.cache.pos].(StringValue).RawValue()
c.cache.pos++
c.tsm.pos++
} else if ckey < tkey {
c.res.Timestamps[pos] = ckey
c.res.Values[pos] = cvals[c.cache.pos].(StringValue).value
c.res.Values[pos] = cvals[c.cache.pos].(StringValue).RawValue()
c.cache.pos++
} else {
c.res.Timestamps[pos] = tkey
@ -1050,7 +1050,7 @@ func (c *stringArrayAscendingCursor) Next() *tsdb.StringArray {
// TSM was exhausted
for pos < len(c.res.Timestamps) && c.cache.pos < len(cvals) {
c.res.Timestamps[pos] = cvals[c.cache.pos].UnixNano()
c.res.Values[pos] = cvals[c.cache.pos].(StringValue).value
c.res.Values[pos] = cvals[c.cache.pos].(StringValue).RawValue()
pos++
c.cache.pos++
}
@ -1173,12 +1173,12 @@ func (c *stringArrayDescendingCursor) Next() *tsdb.StringArray {
tkey := tvals.Timestamps[c.tsm.pos]
if ckey == tkey {
c.res.Timestamps[pos] = ckey
c.res.Values[pos] = cvals[c.cache.pos].(StringValue).value
c.res.Values[pos] = cvals[c.cache.pos].(StringValue).RawValue()
c.cache.pos--
c.tsm.pos--
} else if ckey > tkey {
c.res.Timestamps[pos] = ckey
c.res.Values[pos] = cvals[c.cache.pos].(StringValue).value
c.res.Values[pos] = cvals[c.cache.pos].(StringValue).RawValue()
c.cache.pos--
} else {
c.res.Timestamps[pos] = tkey
@ -1211,7 +1211,7 @@ func (c *stringArrayDescendingCursor) Next() *tsdb.StringArray {
// TSM was exhausted
for pos < len(c.res.Timestamps) && c.cache.pos >= 0 {
c.res.Timestamps[pos] = cvals[c.cache.pos].UnixNano()
c.res.Values[pos] = cvals[c.cache.pos].(StringValue).value
c.res.Values[pos] = cvals[c.cache.pos].(StringValue).RawValue()
pos++
c.cache.pos--
}
@ -1319,12 +1319,12 @@ func (c *booleanArrayAscendingCursor) Next() *tsdb.BooleanArray {
tkey := tvals.Timestamps[c.tsm.pos]
if ckey == tkey {
c.res.Timestamps[pos] = ckey
c.res.Values[pos] = cvals[c.cache.pos].(BooleanValue).value
c.res.Values[pos] = cvals[c.cache.pos].(BooleanValue).RawValue()
c.cache.pos++
c.tsm.pos++
} else if ckey < tkey {
c.res.Timestamps[pos] = ckey
c.res.Values[pos] = cvals[c.cache.pos].(BooleanValue).value
c.res.Values[pos] = cvals[c.cache.pos].(BooleanValue).RawValue()
c.cache.pos++
} else {
c.res.Timestamps[pos] = tkey
@ -1362,7 +1362,7 @@ func (c *booleanArrayAscendingCursor) Next() *tsdb.BooleanArray {
// TSM was exhausted
for pos < len(c.res.Timestamps) && c.cache.pos < len(cvals) {
c.res.Timestamps[pos] = cvals[c.cache.pos].UnixNano()
c.res.Values[pos] = cvals[c.cache.pos].(BooleanValue).value
c.res.Values[pos] = cvals[c.cache.pos].(BooleanValue).RawValue()
pos++
c.cache.pos++
}
@ -1483,12 +1483,12 @@ func (c *booleanArrayDescendingCursor) Next() *tsdb.BooleanArray {
tkey := tvals.Timestamps[c.tsm.pos]
if ckey == tkey {
c.res.Timestamps[pos] = ckey
c.res.Values[pos] = cvals[c.cache.pos].(BooleanValue).value
c.res.Values[pos] = cvals[c.cache.pos].(BooleanValue).RawValue()
c.cache.pos--
c.tsm.pos--
} else if ckey > tkey {
c.res.Timestamps[pos] = ckey
c.res.Values[pos] = cvals[c.cache.pos].(BooleanValue).value
c.res.Values[pos] = cvals[c.cache.pos].(BooleanValue).RawValue()
c.cache.pos--
} else {
c.res.Timestamps[pos] = tkey
@ -1521,7 +1521,7 @@ func (c *booleanArrayDescendingCursor) Next() *tsdb.BooleanArray {
// TSM was exhausted
for pos < len(c.res.Timestamps) && c.cache.pos >= 0 {
c.res.Timestamps[pos] = cvals[c.cache.pos].UnixNano()
c.res.Values[pos] = cvals[c.cache.pos].(BooleanValue).value
c.res.Values[pos] = cvals[c.cache.pos].(BooleanValue).RawValue()
pos++
c.cache.pos--
}

View File

@ -54,7 +54,7 @@ c.end = end
})
}
func (c *{{$type}}) Err() error { return nil }
func (c *{{$type}}) Err() error { return nil }
// close closes the cursor and any dependent cursors.
func (c *{{$type}}) Close() {
@ -82,12 +82,12 @@ func (c *{{$type}}) Next() {{$arrayType}} {
tkey := tvals.Timestamps[c.tsm.pos]
if ckey == tkey {
c.res.Timestamps[pos] = ckey
c.res.Values[pos] = cvals[c.cache.pos].({{.Name}}Value).value
c.res.Values[pos] = cvals[c.cache.pos].({{.Name}}Value).RawValue()
c.cache.pos++
c.tsm.pos++
} else if ckey < tkey {
c.res.Timestamps[pos] = ckey
c.res.Values[pos] = cvals[c.cache.pos].({{.Name}}Value).value
c.res.Values[pos] = cvals[c.cache.pos].({{.Name}}Value).RawValue()
c.cache.pos++
} else {
c.res.Timestamps[pos] = tkey
@ -125,7 +125,7 @@ func (c *{{$type}}) Next() {{$arrayType}} {
// TSM was exhausted
for pos < len(c.res.Timestamps) && c.cache.pos < len(cvals) {
c.res.Timestamps[pos] = cvals[c.cache.pos].UnixNano()
c.res.Values[pos] = cvals[c.cache.pos].({{.Name}}Value).value
c.res.Values[pos] = cvals[c.cache.pos].({{.Name}}Value).RawValue()
pos++
c.cache.pos++
}
@ -228,7 +228,7 @@ func (c *{{$type}}) reset(seek, end int64, cacheValues Values, tsmKeyCursor *Key
}
}
func (c *{{$type}}) Err() error { return nil }
func (c *{{$type}}) Err() error { return nil }
func (c *{{$type}}) Close() {
if c.tsm.keyCursor != nil {
@ -254,12 +254,12 @@ func (c *{{$type}}) Next() {{$arrayType}} {
tkey := tvals.Timestamps[c.tsm.pos]
if ckey == tkey {
c.res.Timestamps[pos] = ckey
c.res.Values[pos] = cvals[c.cache.pos].({{.Name}}Value).value
c.res.Values[pos] = cvals[c.cache.pos].({{.Name}}Value).RawValue()
c.cache.pos--
c.tsm.pos--
} else if ckey > tkey {
c.res.Timestamps[pos] = ckey
c.res.Values[pos] = cvals[c.cache.pos].({{.Name}}Value).value
c.res.Values[pos] = cvals[c.cache.pos].({{.Name}}Value).RawValue()
c.cache.pos--
} else {
c.res.Timestamps[pos] = tkey
@ -292,7 +292,7 @@ func (c *{{$type}}) Next() {{$arrayType}} {
// TSM was exhausted
for pos < len(c.res.Timestamps) && c.cache.pos >= 0 {
c.res.Timestamps[pos] = cvals[c.cache.pos].UnixNano()
c.res.Values[pos] = cvals[c.cache.pos].({{.Name}}Value).value
c.res.Values[pos] = cvals[c.cache.pos].({{.Name}}Value).RawValue()
pos++
c.cache.pos--
}

View File

@ -0,0 +1,42 @@
[
{
"Name":"Float",
"name":"float",
"Type":"float64",
"ValueType":"FloatValue",
"Nil":"0",
"Size":"8"
},
{
"Name":"Integer",
"name":"integer",
"Type":"int64",
"ValueType":"IntegerValue",
"Nil":"0",
"Size":"8"
},
{
"Name":"Unsigned",
"name":"unsigned",
"Type":"uint64",
"ValueType":"UnsignedValue",
"Nil":"0",
"Size":"8"
},
{
"Name":"String",
"name":"string",
"Type":"string",
"ValueType":"StringValue",
"Nil":"\"\"",
"Size":"0"
},
{
"Name":"Boolean",
"name":"boolean",
"Type":"bool",
"ValueType":"BooleanValue",
"Nil":"false",
"Size":"1"
}
]

View File

@ -1,14 +1,15 @@
package tsm1
import (
"bytes"
"fmt"
"math"
"os"
"sync"
"sync/atomic"
"time"
"github.com/influxdata/influxdb/models"
"github.com/influxdata/influxdb/storage/wal"
"github.com/influxdata/influxdb/tsdb"
"github.com/influxdata/influxql"
"github.com/prometheus/client_golang/prometheus"
@ -545,50 +546,50 @@ func (c *Cache) Values(key []byte) Values {
return values
}
// Delete removes all values for the given keys from the cache.
func (c *Cache) Delete(keys [][]byte) {
c.DeleteRange(keys, math.MinInt64, math.MaxInt64)
}
// DeleteRange removes the values for all keys containing points
// with timestamps between between min and max from the cache.
//
// TODO(edd): Lock usage could possibly be optimised if necessary.
func (c *Cache) DeleteRange(keys [][]byte, min, max int64) {
// DeleteBucketRange removes values for all keys containing points
// with timestamps between min and max contained in the bucket identified
// by name from the cache.
func (c *Cache) DeleteBucketRange(name []byte, min, max int64) {
c.init()
// TODO(edd/jeff): find a way to optimize lock usage
c.mu.Lock()
defer c.mu.Unlock()
var toDelete [][]byte
var total uint64
for _, k := range keys {
// Make sure key exist in the cache, skip if it does not
e := c.store.entry(k)
if e == nil {
continue
}
// applySerial only errors if the closure returns an error.
_ = c.store.applySerial(func(k []byte, e *entry) error {
if !bytes.HasPrefix(k, name) {
return nil
}
total += uint64(e.size())
// Everything is being deleted.
// if everything is being deleted, just stage it to be deleted and move on.
if min == math.MinInt64 && max == math.MaxInt64 {
total += uint64(len(k)) // all entries and the key.
c.store.remove(k)
continue
toDelete = append(toDelete, k)
return nil
}
// Filter what to delete by time range.
// filter the values and subtract out the remaining bytes from the reduction.
e.filter(min, max)
total -= uint64(e.size())
// if it has no entries left, flag it to be deleted.
if e.count() == 0 {
// Nothing left in cache for that key
total += uint64(len(k)) // all entries and the key.
c.store.remove(k)
continue
toDelete = append(toDelete, k)
}
// Just update what is being deleted by the size of the filtered entries.
total -= uint64(e.size())
return nil
})
for _, k := range toDelete {
total += uint64(len(k))
c.store.remove(k)
}
c.tracker.DecCacheSize(total) // Decrease the live cache size.
c.tracker.DecCacheSize(total)
c.tracker.SetMemBytes(uint64(c.Size()))
}
@ -624,92 +625,45 @@ func (c *Cache) ApplyEntryFn(f func(key []byte, entry *entry) error) error {
}
// 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.
// contained within those files.
type CacheLoader struct {
files []string
Logger *zap.Logger
reader *wal.WALReader
}
// NewCacheLoader returns a new instance of a CacheLoader.
func NewCacheLoader(files []string) *CacheLoader {
return &CacheLoader{
files: files,
Logger: zap.NewNop(),
reader: wal.NewWALReader(files),
}
}
// 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 {
return cl.reader.Read(func(entry wal.WALEntry) error {
switch en := entry.(type) {
case *wal.WriteWALEntry:
return cache.WriteMulti(en.Values)
var r *WALSegmentReader
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
}
defer f.Close()
case *wal.DeleteBucketRangeWALEntry:
// 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[:])
// Log some information about the segments.
stat, err := os.Stat(f.Name())
if err != nil {
return err
}
cl.Logger.Info("Reading file", zap.String("path", f.Name()), zap.Int64("size", stat.Size()))
// Nothing to read, skip it
if stat.Size() == 0 {
return nil
}
if r == nil {
r = NewWALSegmentReader(f)
defer r.Close()
} else {
r.Reset(f)
}
for r.Next() {
entry, err := r.Read()
if err != nil {
n := r.Count()
cl.Logger.Info("File corrupt", zap.Error(err), zap.String("path", f.Name()), zap.Int64("pos", 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 *DeleteRangeWALEntry:
cache.DeleteRange(t.Keys, t.Min, t.Max)
case *DeleteWALEntry:
cache.Delete(t.Keys)
}
}
return r.Close()
}(); err != nil {
return err
cache.DeleteBucketRange(name, en.Min, en.Max)
return nil
}
}
return nil
return nil
})
}
// WithLogger sets the logger on the CacheLoader.
func (cl *CacheLoader) WithLogger(log *zap.Logger) {
cl.Logger = log.With(zap.String("service", "cacheloader"))
func (cl *CacheLoader) WithLogger(logger *zap.Logger) {
cl.reader.WithLogger(logger.With(zap.String("service", "cacheloader")))
}
// LastWriteTime returns the time that the cache was last written to.
func (c *Cache) LastWriteTime() time.Time {
c.mu.RLock()
defer c.mu.RUnlock()

View File

@ -15,6 +15,8 @@ import (
"sync/atomic"
"testing"
"github.com/influxdata/influxdb/storage/wal"
"github.com/golang/snappy"
)
@ -167,7 +169,7 @@ func TestCache_CacheWriteMulti_TypeConflict(t *testing.T) {
}
}
func TestCache_Cache_DeleteRange(t *testing.T) {
func TestCache_Cache_DeleteBucketRange(t *testing.T) {
v0 := NewValue(1, 1.0)
v1 := NewValue(2, 2.0)
v2 := NewValue(3, 3.0)
@ -187,7 +189,7 @@ func TestCache_Cache_DeleteRange(t *testing.T) {
t.Fatalf("cache keys incorrect after 2 writes, exp %v, got %v", exp, keys)
}
c.DeleteRange([][]byte{[]byte("bar")}, 2, math.MaxInt64)
c.DeleteBucketRange([]byte("bar"), 2, math.MaxInt64)
if exp, keys := [][]byte{[]byte("bar"), []byte("foo")}, c.Keys(); !reflect.DeepEqual(keys, exp) {
t.Fatalf("cache keys incorrect after delete, exp %v, got %v", exp, keys)
@ -206,7 +208,7 @@ func TestCache_Cache_DeleteRange(t *testing.T) {
}
}
func TestCache_DeleteRange_NoValues(t *testing.T) {
func TestCache_DeleteBucketRange_NoValues(t *testing.T) {
v0 := NewValue(1, 1.0)
v1 := NewValue(2, 2.0)
v2 := NewValue(3, 3.0)
@ -226,7 +228,7 @@ func TestCache_DeleteRange_NoValues(t *testing.T) {
t.Fatalf("cache keys incorrect after 2 writes, exp %v, got %v", exp, keys)
}
c.DeleteRange([][]byte{[]byte("foo")}, math.MinInt64, math.MaxInt64)
c.DeleteBucketRange([]byte("foo"), math.MinInt64, math.MaxInt64)
if exp, keys := 0, len(c.Keys()); !reflect.DeepEqual(keys, exp) {
t.Fatalf("cache keys incorrect after 2 writes, exp %v, got %v", exp, keys)
@ -241,7 +243,7 @@ func TestCache_DeleteRange_NoValues(t *testing.T) {
}
}
func TestCache_DeleteRange_NotSorted(t *testing.T) {
func TestCache_DeleteBucketRange_NotSorted(t *testing.T) {
v0 := NewValue(1, 1.0)
v1 := NewValue(3, 3.0)
v2 := NewValue(2, 2.0)
@ -261,7 +263,7 @@ func TestCache_DeleteRange_NotSorted(t *testing.T) {
t.Fatalf("cache keys incorrect after 2 writes, exp %v, got %v", exp, keys)
}
c.DeleteRange([][]byte{[]byte("foo")}, 1, 3)
c.DeleteBucketRange([]byte("foo"), 1, 3)
if exp, keys := 0, len(c.Keys()); !reflect.DeepEqual(keys, exp) {
t.Fatalf("cache keys incorrect after delete, exp %v, got %v", exp, keys)
@ -276,49 +278,10 @@ func TestCache_DeleteRange_NotSorted(t *testing.T) {
}
}
func TestCache_Cache_Delete(t *testing.T) {
v0 := NewValue(1, 1.0)
v1 := NewValue(2, 2.0)
v2 := NewValue(3, 3.0)
values := Values{v0, v1, v2}
valuesSize := uint64(v0.Size() + v1.Size() + v2.Size())
c := NewCache(30 * valuesSize)
if err := c.WriteMulti(map[string][]Value{"foo": values, "bar": values}); err != nil {
t.Fatalf("failed to write key foo to cache: %s", err.Error())
}
if n := c.Size(); n != 2*valuesSize+6 {
t.Fatalf("cache size incorrect after 2 writes, exp %d, got %d", 2*valuesSize, n)
}
if exp, keys := [][]byte{[]byte("bar"), []byte("foo")}, c.Keys(); !reflect.DeepEqual(keys, exp) {
t.Fatalf("cache keys incorrect after 2 writes, exp %v, got %v", exp, keys)
}
c.Delete([][]byte{[]byte("bar")})
if exp, keys := [][]byte{[]byte("foo")}, c.Keys(); !reflect.DeepEqual(keys, exp) {
t.Fatalf("cache keys incorrect after 2 writes, exp %v, got %v", exp, keys)
}
if got, exp := c.Size(), valuesSize+3; exp != got {
t.Fatalf("cache size incorrect after 2 writes, exp %d, got %d", exp, got)
}
if got, exp := len(c.Values([]byte("bar"))), 0; got != exp {
t.Fatalf("cache values mismatch: got %v, exp %v", got, exp)
}
if got, exp := len(c.Values([]byte("foo"))), 3; got != exp {
t.Fatalf("cache values mismatch: got %v, exp %v", got, exp)
}
}
func TestCache_Cache_Delete_NonExistent(t *testing.T) {
func TestCache_DeleteBucketRange_NonExistent(t *testing.T) {
c := NewCache(1024)
c.Delete([][]byte{[]byte("bar")})
c.DeleteBucketRange([]byte("bar"), math.MinInt64, math.MaxInt64)
if got, exp := c.Size(), uint64(0); exp != got {
t.Fatalf("cache size incorrect exp %d, got %d", exp, got)
@ -604,7 +567,7 @@ func TestCacheLoader_LoadSingle(t *testing.T) {
dir := mustTempDir()
defer os.RemoveAll(dir)
f := mustTempFile(dir)
w := NewWALSegmentWriter(f)
w := wal.NewWALSegmentWriter(f)
p1 := NewValue(1, 1.1)
p2 := NewValue(1, int64(1))
@ -616,7 +579,7 @@ func TestCacheLoader_LoadSingle(t *testing.T) {
"baz": {p3},
}
entry := &WriteWALEntry{
entry := &wal.WriteWALEntry{
Values: values,
}
@ -676,7 +639,7 @@ func TestCacheLoader_LoadDouble(t *testing.T) {
dir := mustTempDir()
defer os.RemoveAll(dir)
f1, f2 := mustTempFile(dir), mustTempFile(dir)
w1, w2 := NewWALSegmentWriter(f1), NewWALSegmentWriter(f2)
w1, w2 := wal.NewWALSegmentWriter(f1), wal.NewWALSegmentWriter(f2)
p1 := NewValue(1, 1.1)
p2 := NewValue(1, int64(1))
@ -685,8 +648,8 @@ func TestCacheLoader_LoadDouble(t *testing.T) {
// Write first and second segment.
segmentWrite := func(w *WALSegmentWriter, values map[string][]Value) {
entry := &WriteWALEntry{
segmentWrite := func(w *wal.WALSegmentWriter, values map[string][]Value) {
entry := &wal.WriteWALEntry{
Values: values,
}
if err := w1.Write(mustMarshalEntry(entry)); err != nil {
@ -735,73 +698,6 @@ func TestCacheLoader_LoadDouble(t *testing.T) {
}
}
// Ensure the CacheLoader can load deleted series
func TestCacheLoader_LoadDeleted(t *testing.T) {
// Create a WAL segment.
dir := mustTempDir()
defer os.RemoveAll(dir)
f := mustTempFile(dir)
w := NewWALSegmentWriter(f)
p1 := NewValue(1, 1.0)
p2 := NewValue(2, 2.0)
p3 := NewValue(3, 3.0)
values := map[string][]Value{
"foo": {p1, p2, p3},
}
entry := &WriteWALEntry{
Values: values,
}
if err := w.Write(mustMarshalEntry(entry)); err != nil {
t.Fatal("write points", err)
}
if err := w.Flush(); err != nil {
t.Fatalf("flush error: %v", err)
}
dentry := &DeleteRangeWALEntry{
Keys: [][]byte{[]byte("foo")},
Min: 2,
Max: 3,
}
if err := w.Write(mustMarshalEntry(dentry)); err != nil {
t.Fatal("write points", err)
}
if err := w.Flush(); err != nil {
t.Fatalf("flush error: %v", err)
}
// Load the cache using the segment.
cache := NewCache(1024)
loader := NewCacheLoader([]string{f.Name()})
if err := loader.Load(cache); err != nil {
t.Fatalf("failed to load cache: %s", err.Error())
}
// Check the cache.
if values := cache.Values([]byte("foo")); !reflect.DeepEqual(values, Values{p1}) {
t.Fatalf("cache key foo not as expected, got %v, exp %v", values, Values{p1})
}
// Reload the cache using the segment.
cache = NewCache(1024)
loader = NewCacheLoader([]string{f.Name()})
if err := loader.Load(cache); err != nil {
t.Fatalf("failed to load cache: %s", err.Error())
}
// Check the cache.
if values := cache.Values([]byte("foo")); !reflect.DeepEqual(values, Values{p1}) {
t.Fatalf("cache key foo not as expected, got %v, exp %v", values, Values{p1})
}
}
func TestCache_Split(t *testing.T) {
v0 := NewValue(1, 1.0)
v1 := NewValue(2, 2.0)
@ -857,7 +753,7 @@ func mustTempFile(dir string) *os.File {
return f
}
func mustMarshalEntry(entry WALEntry) (WalEntryType, []byte) {
func mustMarshalEntry(entry wal.WALEntry) (wal.WalEntryType, []byte) {
bytes := make([]byte, 1024<<2)
b, err := entry.Encode(bytes)

View File

@ -212,8 +212,8 @@ type FloatValues []FloatValue
func NewFloatArrayFromValues(v FloatValues) *tsdb.FloatArray {
a := tsdb.NewFloatArrayLen(len(v))
for i, val := range v {
a.Timestamps[i] = val.unixnano
a.Values[i] = val.value
a.Timestamps[i] = val.UnixNano()
a.Values[i] = val.RawValue()
}
return a
}
@ -442,8 +442,8 @@ func encodeFloatValuesBlock(buf []byte, values []FloatValue) ([]byte, error) {
var b []byte
err := func() error {
for _, v := range values {
tsenc.Write(v.unixnano)
venc.Write(v.value)
tsenc.Write(v.UnixNano())
venc.Write(v.RawValue())
}
venc.Flush()
@ -482,8 +482,8 @@ type IntegerValues []IntegerValue
func NewIntegerArrayFromValues(v IntegerValues) *tsdb.IntegerArray {
a := tsdb.NewIntegerArrayLen(len(v))
for i, val := range v {
a.Timestamps[i] = val.unixnano
a.Values[i] = val.value
a.Timestamps[i] = val.UnixNano()
a.Values[i] = val.RawValue()
}
return a
}
@ -712,8 +712,8 @@ func encodeIntegerValuesBlock(buf []byte, values []IntegerValue) ([]byte, error)
var b []byte
err := func() error {
for _, v := range values {
tsenc.Write(v.unixnano)
venc.Write(v.value)
tsenc.Write(v.UnixNano())
venc.Write(v.RawValue())
}
venc.Flush()
@ -752,8 +752,8 @@ type UnsignedValues []UnsignedValue
func NewUnsignedArrayFromValues(v UnsignedValues) *tsdb.UnsignedArray {
a := tsdb.NewUnsignedArrayLen(len(v))
for i, val := range v {
a.Timestamps[i] = val.unixnano
a.Values[i] = val.value
a.Timestamps[i] = val.UnixNano()
a.Values[i] = val.RawValue()
}
return a
}
@ -982,8 +982,8 @@ func encodeUnsignedValuesBlock(buf []byte, values []UnsignedValue) ([]byte, erro
var b []byte
err := func() error {
for _, v := range values {
tsenc.Write(v.unixnano)
venc.Write(int64(v.value))
tsenc.Write(v.UnixNano())
venc.Write(int64(v.RawValue()))
}
venc.Flush()
@ -1022,8 +1022,8 @@ type StringValues []StringValue
func NewStringArrayFromValues(v StringValues) *tsdb.StringArray {
a := tsdb.NewStringArrayLen(len(v))
for i, val := range v {
a.Timestamps[i] = val.unixnano
a.Values[i] = val.value
a.Timestamps[i] = val.UnixNano()
a.Values[i] = val.RawValue()
}
return a
}
@ -1252,8 +1252,8 @@ func encodeStringValuesBlock(buf []byte, values []StringValue) ([]byte, error) {
var b []byte
err := func() error {
for _, v := range values {
tsenc.Write(v.unixnano)
venc.Write(v.value)
tsenc.Write(v.UnixNano())
venc.Write(v.RawValue())
}
venc.Flush()
@ -1292,8 +1292,8 @@ type BooleanValues []BooleanValue
func NewBooleanArrayFromValues(v BooleanValues) *tsdb.BooleanArray {
a := tsdb.NewBooleanArrayLen(len(v))
for i, val := range v {
a.Timestamps[i] = val.unixnano
a.Values[i] = val.value
a.Timestamps[i] = val.UnixNano()
a.Values[i] = val.RawValue()
}
return a
}
@ -1522,8 +1522,8 @@ func encodeBooleanValuesBlock(buf []byte, values []BooleanValue) ([]byte, error)
var b []byte
err := func() error {
for _, v := range values {
tsenc.Write(v.unixnano)
venc.Write(v.value)
tsenc.Write(v.UnixNano())
venc.Write(v.RawValue())
}
venc.Flush()

View File

@ -15,8 +15,8 @@ type {{.Name}}Values []{{.Name}}Value
func New{{.Name}}ArrayFromValues(v {{.Name}}Values) *tsdb.{{.Name}}Array {
a := tsdb.New{{.Name}}ArrayLen(len(v))
for i, val := range v {
a.Timestamps[i] = val.unixnano
a.Values[i] = val.value
a.Timestamps[i] = val.UnixNano()
a.Values[i] = val.RawValue()
}
return a
}
@ -247,8 +247,8 @@ func encode{{ .Name }}ValuesBlock(buf []byte, values []{{.Name}}Value) ([]byte,
var b []byte
err := func() error {
for _, v := range values {
tsenc.Write(v.unixnano)
venc.Write({{if .CastType}}{{.CastType}}(v.value){{else}}v.value{{end}})
tsenc.Write(v.UnixNano())
venc.Write({{if .CastType}}{{.CastType}}(v.RawValue()){{else}}v.RawValue(){{end}})
}
venc.Flush()

View File

@ -14,7 +14,7 @@ func makeIntegerValues(count int, min, max int64) IntegerValues {
inc := (max - min) / int64(count)
for i := 0; i < count; i++ {
vals[i].unixnano = ts
vals[i] = NewRawIntegerValue(ts, 0)
ts += inc
}
@ -24,7 +24,7 @@ func makeIntegerValues(count int, min, max int64) IntegerValues {
func makeIntegerValuesFromSlice(t []int64) IntegerValues {
iv := make(IntegerValues, len(t))
for i, v := range t {
iv[i].unixnano = v
iv[i] = NewRawIntegerValue(v, 0)
}
return iv
}
@ -91,7 +91,7 @@ func TestIntegerValues_Exclude(t *testing.T) {
vals = vals.Exclude(tc.min, tc.max)
var got []int64
for _, v := range vals {
got = append(got, v.unixnano)
got = append(got, v.UnixNano())
}
opt := cmp.AllowUnexported(IntegerValue{})
if !cmp.Equal(tc.exp, got, opt) {
@ -122,7 +122,7 @@ func TestIntegerValues_Include(t *testing.T) {
vals = vals.Include(tc.min, tc.max)
var got []int64
for _, v := range vals {
got = append(got, v.unixnano)
got = append(got, v.UnixNano())
}
opt := cmp.AllowUnexported(IntegerValue{})
if !cmp.Equal(tc.exp, got, opt) {

View File

@ -4,10 +4,8 @@ import (
"encoding/binary"
"fmt"
"runtime"
"time"
"github.com/influxdata/influxdb/pkg/pool"
"github.com/influxdata/influxdb/tsdb"
"github.com/influxdata/influxql"
)
@ -93,89 +91,6 @@ var (
})
)
// Value represents a TSM-encoded value.
type Value interface {
// UnixNano returns the timestamp of the value in nanoseconds since unix epoch.
UnixNano() int64
// Value returns the underlying value.
Value() interface{}
// Size returns the number of bytes necessary to represent the value and its timestamp.
Size() int
// String returns the string representation of the value and its timestamp.
String() string
// internalOnly is unexported to ensure implementations of Value
// can only originate in this package.
internalOnly()
}
// NewValue returns a new Value with the underlying type dependent on value.
func NewValue(t int64, value interface{}) Value {
switch v := value.(type) {
case int64:
return IntegerValue{unixnano: t, value: v}
case uint64:
return UnsignedValue{unixnano: t, value: v}
case float64:
return FloatValue{unixnano: t, value: v}
case bool:
return BooleanValue{unixnano: t, value: v}
case string:
return StringValue{unixnano: t, value: v}
}
return EmptyValue{}
}
// NewIntegerValue returns a new integer value.
func NewIntegerValue(t int64, v int64) Value {
return IntegerValue{unixnano: t, value: v}
}
// NewUnsignedValue returns a new unsigned integer value.
func NewUnsignedValue(t int64, v uint64) Value {
return UnsignedValue{unixnano: t, value: v}
}
// NewFloatValue returns a new float value.
func NewFloatValue(t int64, v float64) Value {
return FloatValue{unixnano: t, value: v}
}
// NewBooleanValue returns a new boolean value.
func NewBooleanValue(t int64, v bool) Value {
return BooleanValue{unixnano: t, value: v}
}
// NewStringValue returns a new string value.
func NewStringValue(t int64, v string) Value {
return StringValue{unixnano: t, value: v}
}
// EmptyValue is used when there is no appropriate other value.
type EmptyValue struct{}
// UnixNano returns tsdb.EOF.
func (e EmptyValue) UnixNano() int64 { return tsdb.EOF }
// Value returns nil.
func (e EmptyValue) Value() interface{} { return nil }
// Size returns 0.
func (e EmptyValue) Size() int { return 0 }
// String returns the empty string.
func (e EmptyValue) String() string { return "" }
func (EmptyValue) internalOnly() {}
func (StringValue) internalOnly() {}
func (IntegerValue) internalOnly() {}
func (UnsignedValue) internalOnly() {}
func (BooleanValue) internalOnly() {}
func (FloatValue) internalOnly() {}
// Encode converts the values to a byte slice. If there are no values,
// this function panics.
func (a Values) Encode(buf []byte) ([]byte, error) {
@ -318,32 +233,6 @@ func DecodeBlock(block []byte, vals []Value) ([]Value, error) {
}
}
// FloatValue represents a float64 value.
type FloatValue struct {
unixnano int64
value float64
}
// UnixNano returns the timestamp of the value.
func (v FloatValue) UnixNano() int64 {
return v.unixnano
}
// Value returns the underlying float64 value.
func (v FloatValue) Value() interface{} {
return v.value
}
// Size returns the number of bytes necessary to represent the value and its timestamp.
func (v FloatValue) Size() int {
return 16
}
// String returns the string representation of the value and its timestamp.
func (v FloatValue) String() string {
return fmt.Sprintf("%v %v", time.Unix(0, v.unixnano), v.value)
}
func encodeFloatBlock(buf []byte, values []Value) ([]byte, error) {
if len(values) == 0 {
return nil, nil
@ -373,8 +262,8 @@ func encodeFloatBlockUsing(buf []byte, values []Value, tsenc TimeEncoder, venc *
for _, v := range values {
vv := v.(FloatValue)
tsenc.Write(vv.unixnano)
venc.Write(vv.value)
tsenc.Write(vv.UnixNano())
venc.Write(vv.RawValue())
}
venc.Flush()
@ -432,7 +321,7 @@ func DecodeFloatBlock(block []byte, a *[]FloatValue) ([]FloatValue, error) {
// Decode both a timestamp and value
j := 0
for j < len(a) && tdec.Next() && vdec.Next() {
a[j] = FloatValue{unixnano: tdec.Read(), value: vdec.Values()}
a[j] = NewRawFloatValue(tdec.Read(), vdec.Values())
j++
}
i = j
@ -453,32 +342,6 @@ func DecodeFloatBlock(block []byte, a *[]FloatValue) ([]FloatValue, error) {
return (*a)[:i], err
}
// BooleanValue represents a boolean value.
type BooleanValue struct {
unixnano int64
value bool
}
// Size returns the number of bytes necessary to represent the value and its timestamp.
func (v BooleanValue) Size() int {
return 9
}
// UnixNano returns the timestamp of the value in nanoseconds since unix epoch.
func (v BooleanValue) UnixNano() int64 {
return v.unixnano
}
// Value returns the underlying boolean value.
func (v BooleanValue) Value() interface{} {
return v.value
}
// String returns the string representation of the value and its timestamp.
func (v BooleanValue) String() string {
return fmt.Sprintf("%v %v", time.Unix(0, v.unixnano), v.Value())
}
func encodeBooleanBlock(buf []byte, values []Value) ([]byte, error) {
if len(values) == 0 {
return nil, nil
@ -505,8 +368,8 @@ func encodeBooleanBlockUsing(buf []byte, values []Value, tenc TimeEncoder, venc
for _, v := range values {
vv := v.(BooleanValue)
tenc.Write(vv.unixnano)
venc.Write(vv.value)
tenc.Write(vv.UnixNano())
venc.Write(vv.RawValue())
}
// Encoded timestamp values
@ -560,7 +423,7 @@ func DecodeBooleanBlock(block []byte, a *[]BooleanValue) ([]BooleanValue, error)
// Decode both a timestamp and value
j := 0
for j < len(a) && tdec.Next() && vdec.Next() {
a[j] = BooleanValue{unixnano: tdec.Read(), value: vdec.Read()}
a[j] = NewRawBooleanValue(tdec.Read(), vdec.Read())
j++
}
i = j
@ -580,32 +443,6 @@ func DecodeBooleanBlock(block []byte, a *[]BooleanValue) ([]BooleanValue, error)
return (*a)[:i], err
}
// IntegerValue represents an int64 value.
type IntegerValue struct {
unixnano int64
value int64
}
// Value returns the underlying int64 value.
func (v IntegerValue) Value() interface{} {
return v.value
}
// UnixNano returns the timestamp of the value.
func (v IntegerValue) UnixNano() int64 {
return v.unixnano
}
// Size returns the number of bytes necessary to represent the value and its timestamp.
func (v IntegerValue) Size() int {
return 16
}
// String returns the string representation of the value and its timestamp.
func (v IntegerValue) String() string {
return fmt.Sprintf("%v %v", time.Unix(0, v.unixnano), v.Value())
}
func encodeIntegerBlock(buf []byte, values []Value) ([]byte, error) {
tenc := getTimeEncoder(len(values))
venc := getIntegerEncoder(len(values))
@ -624,8 +461,8 @@ func encodeIntegerBlockUsing(buf []byte, values []Value, tenc TimeEncoder, venc
for _, v := range values {
vv := v.(IntegerValue)
tenc.Write(vv.unixnano)
venc.Write(vv.value)
tenc.Write(vv.UnixNano())
venc.Write(vv.RawValue())
}
// Encoded timestamp values
@ -679,7 +516,7 @@ func DecodeIntegerBlock(block []byte, a *[]IntegerValue) ([]IntegerValue, error)
// Decode both a timestamp and value
j := 0
for j < len(a) && tdec.Next() && vdec.Next() {
a[j] = IntegerValue{unixnano: tdec.Read(), value: vdec.Read()}
a[j] = NewRawIntegerValue(tdec.Read(), vdec.Read())
j++
}
i = j
@ -699,32 +536,6 @@ func DecodeIntegerBlock(block []byte, a *[]IntegerValue) ([]IntegerValue, error)
return (*a)[:i], err
}
// UnsignedValue represents an int64 value.
type UnsignedValue struct {
unixnano int64
value uint64
}
// Value returns the underlying int64 value.
func (v UnsignedValue) Value() interface{} {
return v.value
}
// UnixNano returns the timestamp of the value.
func (v UnsignedValue) UnixNano() int64 {
return v.unixnano
}
// Size returns the number of bytes necessary to represent the value and its timestamp.
func (v UnsignedValue) Size() int {
return 16
}
// String returns the string representation of the value and its timestamp.
func (v UnsignedValue) String() string {
return fmt.Sprintf("%v %v", time.Unix(0, v.unixnano), v.Value())
}
func encodeUnsignedBlock(buf []byte, values []Value) ([]byte, error) {
tenc := getTimeEncoder(len(values))
venc := getUnsignedEncoder(len(values))
@ -743,8 +554,8 @@ func encodeUnsignedBlockUsing(buf []byte, values []Value, tenc TimeEncoder, venc
for _, v := range values {
vv := v.(UnsignedValue)
tenc.Write(vv.unixnano)
venc.Write(int64(vv.value))
tenc.Write(vv.UnixNano())
venc.Write(int64(vv.RawValue()))
}
// Encoded timestamp values
@ -798,7 +609,7 @@ func DecodeUnsignedBlock(block []byte, a *[]UnsignedValue) ([]UnsignedValue, err
// Decode both a timestamp and value
j := 0
for j < len(a) && tdec.Next() && vdec.Next() {
a[j] = UnsignedValue{unixnano: tdec.Read(), value: uint64(vdec.Read())}
a[j] = NewRawUnsignedValue(tdec.Read(), uint64(vdec.Read()))
j++
}
i = j
@ -818,35 +629,9 @@ func DecodeUnsignedBlock(block []byte, a *[]UnsignedValue) ([]UnsignedValue, err
return (*a)[:i], err
}
// StringValue represents a string value.
type StringValue struct {
unixnano int64
value string
}
// Value returns the underlying string value.
func (v StringValue) Value() interface{} {
return v.value
}
// UnixNano returns the timestamp of the value.
func (v StringValue) UnixNano() int64 {
return v.unixnano
}
// Size returns the number of bytes necessary to represent the value and its timestamp.
func (v StringValue) Size() int {
return 8 + len(v.value)
}
// String returns the string representation of the value and its timestamp.
func (v StringValue) String() string {
return fmt.Sprintf("%v %v", time.Unix(0, v.unixnano), v.Value())
}
func encodeStringBlock(buf []byte, values []Value) ([]byte, error) {
tenc := getTimeEncoder(len(values))
venc := getStringEncoder(len(values) * len(values[0].(StringValue).value))
venc := getStringEncoder(len(values) * len(values[0].(StringValue).RawValue()))
b, err := encodeStringBlockUsing(buf, values, tenc, venc)
@ -862,8 +647,8 @@ func encodeStringBlockUsing(buf []byte, values []Value, tenc TimeEncoder, venc S
for _, v := range values {
vv := v.(StringValue)
tenc.Write(vv.unixnano)
venc.Write(vv.value)
tenc.Write(vv.UnixNano())
venc.Write(vv.RawValue())
}
// Encoded timestamp values
@ -920,7 +705,7 @@ func DecodeStringBlock(block []byte, a *[]StringValue) ([]StringValue, error) {
// Decode both a timestamp and value
j := 0
for j < len(a) && tdec.Next() && vdec.Next() {
a[j] = StringValue{unixnano: tdec.Read(), value: vdec.Read()}
a[j] = NewRawStringValue(tdec.Read(), vdec.Read())
j++
}
i = j

View File

@ -4,11 +4,8 @@ package tsm1 // import "github.com/influxdata/influxdb/tsdb/tsm1"
import (
"bytes"
"context"
"errors"
"fmt"
"io"
"io/ioutil"
"math"
"os"
"path/filepath"
"regexp"
@ -20,7 +17,6 @@ import (
"github.com/influxdata/influxdb/logger"
"github.com/influxdata/influxdb/models"
"github.com/influxdata/influxdb/pkg/bytesutil"
"github.com/influxdata/influxdb/pkg/limiter"
"github.com/influxdata/influxdb/pkg/metrics"
"github.com/influxdata/influxdb/query"
@ -31,6 +27,7 @@ import (
"go.uber.org/zap"
)
//go:generate env GO111MODULE=on go run github.com/benbjohnson/tmpl -data=@array_cursor.gen.go.tmpldata array_cursor.gen.go.tmpl array_cursor_iterator.gen.go.tmpl
//go:generate env GO111MODULE=on go run github.com/influxdata/influxdb/tools/tmpl -i -data=file_store.gen.go.tmpldata file_store.gen.go.tmpl=file_store.gen.go
//go:generate env GO111MODULE=on go run github.com/influxdata/influxdb/tools/tmpl -i -d isArray=y -data=file_store.gen.go.tmpldata file_store.gen.go.tmpl=file_store_array.gen.go
//go:generate env GO111MODULE=on go run github.com/benbjohnson/tmpl -data=@encoding.gen.go.tmpldata encoding.gen.go.tmpl
@ -40,7 +37,6 @@ import (
var (
// Static objects to prevent small allocs.
keyFieldSeparatorBytes = []byte(keyFieldSeparator)
emptyBytes = []byte{}
)
var (
@ -66,9 +62,6 @@ const (
// that identifies a specific field in series
keyFieldSeparator = "#!~#"
// deleteFlushThreshold is the size in bytes of a batch of series keys to delete.
deleteFlushThreshold = 50 * 1024 * 1024
// MaxPointsPerBlock is the maximum number of points in an encoded block in a TSM file
MaxPointsPerBlock = 1000
)
@ -77,34 +70,41 @@ const (
// an Engine.
type EngineOption func(i *Engine)
// WithWAL sets the WAL for the Engine
var WithWAL = func(wal Log) EngineOption {
// be defensive: it's very easy to pass in a nil WAL here
// which will panic. Set any nil WALs to the NopWAL.
if pwal, _ := wal.(*WAL); pwal == nil {
wal = NopWAL{}
}
return func(e *Engine) {
e.WAL = wal
}
}
// WithTraceLogging sets if trace logging is enabled for the engine.
var WithTraceLogging = func(logging bool) EngineOption {
func WithTraceLogging(logging bool) EngineOption {
return func(e *Engine) {
e.FileStore.enableTraceLogging(logging)
}
}
// WithCompactionPlanner sets the compaction planner for the engine.
var WithCompactionPlanner = func(planner CompactionPlanner) EngineOption {
func WithCompactionPlanner(planner CompactionPlanner) EngineOption {
return func(e *Engine) {
planner.SetFileStore(e.FileStore)
e.CompactionPlan = planner
}
}
// Snapshotter allows upward signaling of the tsm1 engine to the storage engine. Hopefully
// it can be removed one day. The weird interface is due to the weird inversion of locking
// that has to happen.
type Snapshotter interface {
AcquireSegments(func(segments []string) error) error
CommitSegments(segments []string, fn func() error) error
}
type noSnapshotter struct{}
func (noSnapshotter) AcquireSegments(fn func([]string) error) error { return fn(nil) }
func (noSnapshotter) CommitSegments(_ []string, fn func() error) error { return fn() }
// WithSnapshotter sets the callbacks for the engine to use when creating snapshots.
func WithSnapshotter(snapshotter Snapshotter) EngineOption {
return func(e *Engine) {
e.snapshotter = snapshotter
}
}
// Engine represents a storage engine with compressed blocks.
type Engine struct {
mu sync.RWMutex
@ -132,7 +132,6 @@ type Engine struct {
traceLogger *zap.Logger // Logger to be used when trace-logging is on.
traceLogging bool
WAL Log
Cache *Cache
Compactor *Compactor
CompactionPlan CompactionPlanner
@ -161,7 +160,8 @@ type Engine struct {
// Limiter for concurrent compactions.
compactionLimiter limiter.Fixed
scheduler *scheduler
scheduler *scheduler
snapshotter Snapshotter
}
// NewEngine returns a new instance of Engine.
@ -207,7 +207,6 @@ func NewEngine(path string, idx *tsi1.Index, config Config, options ...EngineOpt
logger: logger,
traceLogger: logger,
WAL: NopWAL{},
Cache: cache,
FileStore: fs,
@ -221,6 +220,7 @@ func NewEngine(path string, idx *tsi1.Index, config Config, options ...EngineOpt
formatFileName: DefaultFormatFileName,
compactionLimiter: limiter.NewFixed(maxCompactions),
scheduler: newScheduler(maxCompactions),
snapshotter: new(noSnapshotter),
}
for _, option := range options {
@ -477,27 +477,11 @@ func (e *Engine) SeriesN() int64 {
return e.index.SeriesN()
}
// LastModified returns the time when this shard was last modified.
func (e *Engine) LastModified() time.Time {
fsTime := e.FileStore.LastModified()
if e.WAL.LastWriteTime().After(fsTime) {
return e.WAL.LastWriteTime()
}
return fsTime
}
// MeasurementStats returns the current measurement stats for the engine.
func (e *Engine) MeasurementStats() (MeasurementStats, error) {
return e.FileStore.MeasurementStats()
}
// DiskSize returns the total size in bytes of all TSM and WAL segments on disk.
func (e *Engine) DiskSize() int64 {
walDiskSizeBytes := e.WAL.DiskSizeBytes()
return e.FileStore.DiskSizeBytes() + walDiskSizeBytes
}
func (e *Engine) initTrackers() {
mmu.Lock()
defer mmu.Unlock()
@ -512,10 +496,6 @@ func (e *Engine) initTrackers() {
e.FileStore.tracker = newFileTracker(bms.fileMetrics, e.defaultMetricLabels)
e.Cache.tracker = newCacheTracker(bms.cacheMetrics, e.defaultMetricLabels)
// Set default metrics on WAL if enabled.
if wal, ok := e.WAL.(*WAL); ok {
wal.tracker = newWALTracker(bms.walMetrics, e.defaultMetricLabels)
}
e.scheduler.setCompactionTracker(e.compactionTracker)
}
@ -531,18 +511,10 @@ func (e *Engine) Open() error {
return err
}
if err := e.WAL.Open(); err != nil {
return err
}
if err := e.FileStore.Open(); err != nil {
return err
}
if err := e.reloadCache(); err != nil {
return err
}
e.Compactor.Open()
if e.enableCompactionsOnOpen {
@ -564,7 +536,8 @@ func (e *Engine) Close() error {
if err := e.FileStore.Close(); err != nil {
return err
}
return e.WAL.Close()
return nil
}
// WithLogger sets the logger for the engine.
@ -575,10 +548,6 @@ func (e *Engine) WithLogger(log *zap.Logger) {
e.traceLogger = e.logger
}
if wal, ok := e.WAL.(*WAL); ok {
wal.WithLogger(e.logger)
}
e.FileStore.WithLogger(e.logger)
}
@ -595,435 +564,33 @@ func (e *Engine) Free() error {
return e.FileStore.Free()
}
// WritePoints writes metadata and point data into the engine.
// It returns an error if new points are added to an existing key.
// WritePoints saves the set of points in the engine.
func (e *Engine) WritePoints(points []models.Point) error {
values := make(map[string][]Value, len(points))
var (
keyBuf []byte
baseLen int
)
for _, p := range points {
keyBuf = append(keyBuf[:0], p.Key()...)
keyBuf = append(keyBuf, keyFieldSeparator...)
baseLen = len(keyBuf)
iter := p.FieldIterator()
t := p.Time().UnixNano()
for iter.Next() {
keyBuf = append(keyBuf[:baseLen], iter.FieldKey()...)
var v Value
switch iter.Type() {
case models.Float:
fv, err := iter.FloatValue()
if err != nil {
return err
}
v = NewFloatValue(t, fv)
case models.Integer:
iv, err := iter.IntegerValue()
if err != nil {
return err
}
v = NewIntegerValue(t, iv)
case models.Unsigned:
iv, err := iter.UnsignedValue()
if err != nil {
return err
}
v = NewUnsignedValue(t, iv)
case models.String:
v = NewStringValue(t, iter.StringValue())
case models.Boolean:
bv, err := iter.BooleanValue()
if err != nil {
return err
}
v = NewBooleanValue(t, bv)
default:
return fmt.Errorf("unknown field type for %s: %s", string(iter.FieldKey()), p.String())
}
values[string(keyBuf)] = append(values[string(keyBuf)], v)
}
values, err := PointsToValues(points)
if err != nil {
return err
}
return e.WriteValues(values)
}
// WriteValues saves the set of values in the engine.
func (e *Engine) WriteValues(values map[string][]Value) error {
e.mu.RLock()
defer e.mu.RUnlock()
// first try to write to the cache
if err := e.Cache.WriteMulti(values); err != nil {
return err
}
// Then make the write durable in the cache.
if _, err := e.WAL.WriteMulti(values); err != nil {
return err
}
return nil
}
// DeleteSeriesRange removes the values between min and max (inclusive) from all series
func (e *Engine) DeleteSeriesRange(itr tsdb.SeriesIterator, min, max int64) error {
return e.DeleteSeriesRangeWithPredicate(itr, func(name []byte, tags models.Tags) (int64, int64, bool) {
return min, max, true
})
}
// DeleteSeriesRangeWithPredicate removes the values between min and max (inclusive) from all series
// for which predicate() returns true. If predicate() is nil, then all values in range are removed.
func (e *Engine) DeleteSeriesRangeWithPredicate(itr tsdb.SeriesIterator, predicate func(name []byte, tags models.Tags) (int64, int64, bool)) error {
var disableOnce bool
// Ensure that the index does not compact away the measurement or series we're
// going to delete before we're done with them.
e.index.DisableCompactions()
defer e.index.EnableCompactions()
e.index.Wait()
fs, err := e.index.RetainFileSet()
if err != nil {
return err
}
defer fs.Release()
var (
sz int
min, max int64 = math.MinInt64, math.MaxInt64
// Indicator that the min/max time for the current batch has changed and
// we need to flush the current batch before appending to it.
flushBatch bool
)
// These are reversed from min/max to ensure they are different the first time through.
newMin, newMax := int64(math.MaxInt64), int64(math.MinInt64)
// There is no predicate, so setup newMin/newMax to delete the full time range.
if predicate == nil {
newMin = min
newMax = max
}
batch := make([][]byte, 0, 10000)
for {
elem, err := itr.Next()
if err != nil {
return err
} else if elem == nil {
break
}
// See if the series should be deleted and if so, what range of time.
if predicate != nil {
var shouldDelete bool
newMin, newMax, shouldDelete = predicate(elem.Name(), elem.Tags())
if !shouldDelete {
continue
}
// If the min/max happens to change for the batch, we need to flush
// the current batch and start a new one.
flushBatch = (min != newMin || max != newMax) && len(batch) > 0
}
if elem.Expr() != nil {
if v, ok := elem.Expr().(*influxql.BooleanLiteral); !ok || !v.Val {
return errors.New("fields not supported in WHERE clause during deletion")
}
}
if !disableOnce {
// Disable and abort running compactions so that tombstones added existing tsm
// files don't get removed. This would cause deleted measurements/series to
// re-appear once the compaction completed. We only disable the level compactions
// so that snapshotting does not stop while writing out tombstones. If it is stopped,
// and writing tombstones takes a long time, writes can get rejected due to the cache
// filling up.
e.disableLevelCompactions(true)
defer e.enableLevelCompactions(true)
e.sfile.DisableCompactions()
defer e.sfile.EnableCompactions()
e.sfile.Wait()
disableOnce = true
}
if sz >= deleteFlushThreshold || flushBatch {
// Delete all matching batch.
if err := e.deleteSeriesRange(batch, min, max); err != nil {
return err
}
batch = batch[:0]
sz = 0
flushBatch = false
}
// Use the new min/max time for the next iteration
min = newMin
max = newMax
key := models.MakeKey(elem.Name(), elem.Tags())
sz += len(key)
batch = append(batch, key)
}
if len(batch) > 0 {
// Delete all matching batch.
if err := e.deleteSeriesRange(batch, min, max); err != nil {
return err
}
}
e.index.Rebuild()
return nil
}
// deleteSeriesRange removes the values between min and max (inclusive) from all series. This
// does not update the index or disable compactions. This should mainly be called by DeleteSeriesRange
// and not directly.
func (e *Engine) deleteSeriesRange(seriesKeys [][]byte, min, max int64) error {
if len(seriesKeys) == 0 {
return nil
}
// Ensure keys are sorted since lower layers require them to be.
if !bytesutil.IsSorted(seriesKeys) {
bytesutil.Sort(seriesKeys)
}
// Min and max time in the engine are slightly different from the query language values.
if min == influxql.MinTime {
min = math.MinInt64
}
if max == influxql.MaxTime {
max = math.MaxInt64
}
// Run the delete on each TSM file in parallel
if err := e.FileStore.Apply(func(r TSMFile) error {
// See if this TSM file contains the keys and time range
minKey, maxKey := seriesKeys[0], seriesKeys[len(seriesKeys)-1]
tsmMin, tsmMax := r.KeyRange()
tsmMin, _ = SeriesAndFieldFromCompositeKey(tsmMin)
tsmMax, _ = SeriesAndFieldFromCompositeKey(tsmMax)
overlaps := bytes.Compare(tsmMin, maxKey) <= 0 && bytes.Compare(tsmMax, minKey) >= 0
if !overlaps || !r.OverlapsTimeRange(min, max) {
return nil
}
// Delete each key we find in the file. We seek to the min key and walk from there.
batch := r.BatchDelete()
iter := r.Iterator(minKey)
var j int
for iter.Next() {
indexKey := iter.Key()
seriesKey, _ := SeriesAndFieldFromCompositeKey(indexKey)
for j < len(seriesKeys) && bytes.Compare(seriesKeys[j], seriesKey) < 0 {
j++
}
if j >= len(seriesKeys) {
break
}
if bytes.Equal(seriesKeys[j], seriesKey) {
if err := batch.DeleteRange([][]byte{indexKey}, min, max); err != nil {
batch.Rollback()
return err
}
}
}
if err := iter.Err(); err != nil {
batch.Rollback()
return err
}
return batch.Commit()
}); err != nil {
return err
}
// find the keys in the cache and remove them
deleteKeys := make([][]byte, 0, len(seriesKeys))
// ApplySerialEntryFn cannot return an error in this invocation.
_ = e.Cache.ApplyEntryFn(func(k []byte, _ *entry) error {
seriesKey, _ := SeriesAndFieldFromCompositeKey([]byte(k))
// Cache does not walk keys in sorted order, so search the sorted
// series we need to delete to see if any of the cache keys match.
i := bytesutil.SearchBytes(seriesKeys, seriesKey)
if i < len(seriesKeys) && bytes.Equal(seriesKey, seriesKeys[i]) {
// k is the measurement + tags + sep + field
deleteKeys = append(deleteKeys, k)
}
return nil
})
// Sort the series keys because ApplyEntryFn iterates over the keys randomly.
bytesutil.Sort(deleteKeys)
e.Cache.DeleteRange(deleteKeys, min, max)
// delete from the WAL
if _, err := e.WAL.DeleteRange(deleteKeys, min, max); err != nil {
return err
}
// The series are deleted on disk, but the index may still say they exist.
// Depending on the the min,max time passed in, the series may or not actually
// exists now. To reconcile the index, we walk the series keys that still exists
// on disk and cross out any keys that match the passed in series. Any series
// left in the slice at the end do not exist and can be deleted from the index.
// Note: this is inherently racy if writes are occurring to the same measurement/series are
// being removed. A write could occur and exist in the cache at this point, but we
// would delete it from the index.
minKey := seriesKeys[0]
// Apply runs this func concurrently. The seriesKeys slice is mutated concurrently
// by different goroutines setting positions to nil.
if err := e.FileStore.Apply(func(r TSMFile) error {
var j int
// Start from the min deleted key that exists in this file.
iter := r.Iterator(minKey)
for iter.Next() {
if j >= len(seriesKeys) {
return nil
}
indexKey := iter.Key()
seriesKey, _ := SeriesAndFieldFromCompositeKey(indexKey)
// Skip over any deleted keys that are less than our tsm key
cmp := bytes.Compare(seriesKeys[j], seriesKey)
for j < len(seriesKeys) && cmp < 0 {
j++
if j >= len(seriesKeys) {
return nil
}
cmp = bytes.Compare(seriesKeys[j], seriesKey)
}
// We've found a matching key, cross it out so we do not remove it from the index.
if j < len(seriesKeys) && cmp == 0 {
seriesKeys[j] = emptyBytes
j++
}
}
return iter.Err()
}); err != nil {
return err
}
// Have we deleted all values for the series? If so, we need to remove
// the series from the index.
if len(seriesKeys) > 0 {
buf := make([]byte, 1024) // For use when accessing series file.
ids := tsdb.NewSeriesIDSet()
measurements := make(map[string]struct{}, 1)
for _, k := range seriesKeys {
if len(k) == 0 {
continue // This key was wiped because it shouldn't be removed from index.
}
name, tags := models.ParseKeyBytes(k)
sid := e.sfile.SeriesID(name, tags, buf)
if sid.IsZero() {
continue
}
// See if this series was found in the cache earlier
i := bytesutil.SearchBytes(deleteKeys, k)
var hasCacheValues bool
// If there are multiple fields, they will have the same prefix. If any field
// has values, then we can't delete it from the index.
for i < len(deleteKeys) && bytes.HasPrefix(deleteKeys[i], k) {
if e.Cache.Values(deleteKeys[i]).Len() > 0 {
hasCacheValues = true
break
}
i++
}
if hasCacheValues {
continue
}
measurements[string(name)] = struct{}{}
// Remove the series from the local index.
if err := e.index.DropSeries(sid, k, false); err != nil {
return err
}
// Add the id to the set of delete ids.
ids.Add(sid)
}
for k := range measurements {
if err := e.index.DropMeasurementIfSeriesNotExist([]byte(k)); err != nil {
return err
}
}
// Remove the remaining ids from the series file as they no longer exist
// in any shard.
var err error
ids.ForEach(func(id tsdb.SeriesID) {
if err1 := e.sfile.DeleteSeriesID(id); err1 != nil {
err = err1
}
})
if err != nil {
return err
}
}
return nil
}
// DeleteMeasurement deletes a measurement and all related series.
func (e *Engine) DeleteMeasurement(name []byte) error {
// Delete the bulk of data outside of the fields lock.
if err := e.deleteMeasurement(name); err != nil {
return err
}
return nil
}
// DeleteMeasurement deletes a measurement and all related series.
func (e *Engine) deleteMeasurement(name []byte) error {
// Attempt to find the series keys.
itr, err := e.index.MeasurementSeriesIDIterator(name)
if err != nil {
return err
} else if itr == nil {
return nil
}
defer itr.Close()
return e.DeleteSeriesRange(tsdb.NewSeriesIteratorAdapter(e.sfile, itr), math.MinInt64, math.MaxInt64)
}
// ForEachMeasurementName iterates over each measurement name in the engine.
func (e *Engine) ForEachMeasurementName(fn func(name []byte) error) error {
return e.index.ForEachMeasurementName(fn)
}
func (e *Engine) CreateSeriesListIfNotExists(collection *tsdb.SeriesCollection) error {
return e.index.CreateSeriesListIfNotExists(collection)
}
// WriteTo is not implemented.
func (e *Engine) WriteTo(w io.Writer) (n int64, err error) { panic("not implemented") }
// compactionLevel describes a snapshot or levelled compaction.
type compactionLevel int
@ -1194,24 +761,18 @@ func (e *Engine) WriteSnapshot() error {
logEnd()
}()
closedFiles, snapshot, err := func() (segments []string, snapshot *Cache, err error) {
var (
snapshot *Cache
segments []string
)
if err := e.snapshotter.AcquireSegments(func(segs []string) (err error) {
segments = segs
e.mu.Lock()
defer e.mu.Unlock()
if err = e.WAL.CloseSegment(); err != nil {
return nil, nil, err
}
segments, err = e.WAL.ClosedSegments()
if err != nil {
return nil, nil, err
}
snapshot, err = e.Cache.Snapshot()
return segments, snapshot, err
}()
if err != nil {
e.mu.Unlock()
return err
}); err != nil {
return err
}
@ -1229,11 +790,11 @@ func (e *Engine) WriteSnapshot() error {
zap.String("path", e.path),
zap.Duration("duration", time.Since(dedup)))
return e.writeSnapshotAndCommit(log, closedFiles, snapshot)
return e.writeSnapshotAndCommit(log, snapshot, segments)
}
// writeSnapshotAndCommit will write the passed cache to a new TSM file and remove the closed WAL segments.
func (e *Engine) writeSnapshotAndCommit(log *zap.Logger, closedFiles []string, snapshot *Cache) (err error) {
func (e *Engine) writeSnapshotAndCommit(log *zap.Logger, snapshot *Cache, segments []string) (err error) {
defer func() {
if err != nil {
e.Cache.ClearSnapshot(false)
@ -1247,23 +808,20 @@ func (e *Engine) writeSnapshotAndCommit(log *zap.Logger, closedFiles []string, s
return err
}
e.mu.RLock()
defer e.mu.RUnlock()
return e.snapshotter.CommitSegments(segments, func() error {
e.mu.RLock()
defer e.mu.RUnlock()
// update the file store with these new files
if err := e.FileStore.Replace(nil, newFiles); err != nil {
log.Info("Error adding new TSM files from snapshot", zap.Error(err))
return err
}
// update the file store with these new files
if err := e.FileStore.Replace(nil, newFiles); err != nil {
log.Info("Error adding new TSM files from snapshot", zap.Error(err))
return err
}
// clear the snapshot from the in-memory cache, then the old WAL files
e.Cache.ClearSnapshot(true)
if err := e.WAL.Remove(closedFiles); err != nil {
log.Info("Error removing closed WAL segments", zap.Error(err))
}
return nil
// clear the snapshot from the in-memory cache
e.Cache.ClearSnapshot(true)
return nil
})
}
// compactCache continually checks if the WAL cache should be written to disk.
@ -1568,32 +1126,6 @@ func (e *Engine) fullCompactionStrategy(group CompactionGroup, optimize bool) *c
return s
}
// reloadCache reads the WAL segment files and loads them into the cache.
func (e *Engine) reloadCache() error {
now := time.Now()
files, err := segmentFileNames(e.WAL.Path())
if err != nil {
return err
}
limit := e.Cache.MaxSize()
defer func() {
e.Cache.SetMaxSize(limit)
}()
// Disable the max size during loading
e.Cache.SetMaxSize(0)
loader := NewCacheLoader(files)
loader.WithLogger(e.logger)
if err := loader.Load(e.Cache); err != nil {
return err
}
e.traceLogger.Info("Reloaded WAL cache", zap.String("path", e.WAL.Path()), zap.Duration("duration", time.Since(now)))
return nil
}
// cleanup removes all temp files and dirs that exist on disk. This is should only be run at startup to avoid
// removing tmp files that are still in use.
func (e *Engine) cleanup() error {

View File

@ -11,10 +11,10 @@ import (
"github.com/influxdata/influxql"
)
// DeleteBucket removes all TSM data belonging to a bucket, and removes all index
// DeleteBucketRange removes all TSM data belonging to a bucket, and removes all index
// and series file data associated with the bucket. The provided time range ensures
// that only bucket data for that range is removed.
func (e *Engine) DeleteBucket(name []byte, min, max int64) error {
func (e *Engine) DeleteBucketRange(name []byte, min, max int64) error {
// TODO(jeff): we need to block writes to this prefix while deletes are in progress
// otherwise we can end up in a situation where we have staged data in the cache or
// WAL that was deleted from the index, or worse. This needs to happen at a higher
@ -76,9 +76,6 @@ func (e *Engine) DeleteBucket(name []byte, min, max int64) error {
return err
}
// TODO(jeff): add a DeletePrefix to the Cache and WAL.
// TODO(jeff): add a Tombstone entry into the WAL for deletes.
var deleteKeys [][]byte
// ApplySerialEntryFn cannot return an error in this invocation.
@ -99,11 +96,8 @@ func (e *Engine) DeleteBucket(name []byte, min, max int64) error {
// Sort the series keys because ApplyEntryFn iterates over the keys randomly.
bytesutil.Sort(deleteKeys)
// Delete from the cache and WAL.
e.Cache.DeleteRange(deleteKeys, min, max)
if _, err := e.WAL.DeleteRange(deleteKeys, min, max); err != nil {
return err
}
// Delete from the cache.
e.Cache.DeleteBucketRange(name, min, max)
// Now that all of the data is purged, we need to find if some keys are fully deleted
// and if so, remove them from the index.
@ -138,6 +132,14 @@ func (e *Engine) DeleteBucket(name []byte, min, max int64) error {
return err
}
// ApplySerialEntryFn cannot return an error in this invocation.
_ = e.Cache.ApplyEntryFn(func(k []byte, _ *entry) error {
if bytes.HasPrefix(k, name) {
delete(possiblyDead.keys, string(k))
}
return nil
})
if len(possiblyDead.keys) > 0 {
buf := make([]byte, 1024)

View File

@ -8,7 +8,7 @@ import (
"github.com/influxdata/influxdb/models"
)
func TestEngine_DeletePrefix(t *testing.T) {
func TestEngine_DeleteBucket(t *testing.T) {
// Create a few points.
p1 := MustParsePointString("cpu,host=0 value=1.1 6")
p2 := MustParsePointString("cpu,host=A value=1.2 2")
@ -44,7 +44,7 @@ func TestEngine_DeletePrefix(t *testing.T) {
t.Fatalf("series count mismatch: exp %v, got %v", exp, got)
}
if err := e.DeleteBucket([]byte("cpu"), 0, 3); err != nil {
if err := e.DeleteBucketRange([]byte("cpu"), 0, 3); err != nil {
t.Fatalf("failed to delete series: %v", err)
}
@ -90,7 +90,7 @@ func TestEngine_DeletePrefix(t *testing.T) {
iter.Close()
// Deleting remaining series should remove them from the series.
if err := e.DeleteBucket([]byte("cpu"), 0, 9); err != nil {
if err := e.DeleteBucketRange([]byte("cpu"), 0, 9); err != nil {
t.Fatalf("failed to delete series: %v", err)
}

View File

@ -1,7 +1,6 @@
package tsm1_test
import (
"bytes"
"fmt"
"io/ioutil"
"math"
@ -59,8 +58,9 @@ func TestIndex_SeriesIDSet(t *testing.T) {
}
// Drop all the series for the gpu measurement and they should no longer
// be in the series ID set.
if err := engine.DeleteMeasurement([]byte("gpu")); err != nil {
// be in the series ID set. This relies on the fact that DeleteBucketRange is really
// operating on prefixes.
if err := engine.DeleteBucketRange([]byte("gpu"), math.MinInt64, math.MaxInt64); err != nil {
t.Fatal(err)
}
@ -72,17 +72,6 @@ func TestIndex_SeriesIDSet(t *testing.T) {
delete(seriesIDMap, "gpu")
delete(seriesIDMap, "gpu,host=b")
// Drop the specific mem series
ditr := &seriesIterator{keys: [][]byte{[]byte("mem,host=z")}}
if err := engine.DeleteSeriesRange(ditr, math.MinInt64, math.MaxInt64); err != nil {
t.Fatal(err)
}
if engine.SeriesIDSet().Contains(seriesIDMap["mem,host=z"]) {
t.Fatalf("bitmap does not contain ID: %d for key %s, but should", seriesIDMap["mem,host=z"], "mem,host=z")
}
delete(seriesIDMap, "mem,host=z")
// The rest of the keys should still be in the set.
for key, id := range seriesIDMap {
if !engine.SeriesIDSet().Contains(id) {
@ -106,589 +95,6 @@ func TestIndex_SeriesIDSet(t *testing.T) {
}
}
// Ensures that deleting series from TSM files with multiple fields removes all the
/// series
func TestEngine_DeleteSeries(t *testing.T) {
// Create a few points.
p1 := MustParsePointString("cpu,host=A value=1.1 1000000000")
p2 := MustParsePointString("cpu,host=B value=1.2 2000000000")
p3 := MustParsePointString("cpu,host=A sum=1.3 3000000000")
e, err := NewEngine()
if err != nil {
t.Fatal(err)
}
// mock the planner so compactions don't run during the test
e.CompactionPlan = &mockPlanner{}
if err := e.Open(); err != nil {
t.Fatal(err)
}
defer e.Close()
if err := e.writePoints(p1, p2, p3); err != nil {
t.Fatalf("failed to write points: %s", err.Error())
}
if err := e.WriteSnapshot(); err != nil {
t.Fatalf("failed to snapshot: %s", err.Error())
}
keys := e.FileStore.Keys()
if exp, got := 3, len(keys); exp != got {
t.Fatalf("series count mismatch: exp %v, got %v", exp, got)
}
itr := &seriesIterator{keys: [][]byte{[]byte("cpu,host=A")}}
if err := e.DeleteSeriesRange(itr, math.MinInt64, math.MaxInt64); err != nil {
t.Fatalf("failed to delete series: %v", err)
}
keys = e.FileStore.Keys()
if exp, got := 1, len(keys); exp != got {
t.Fatalf("series count mismatch: exp %v, got %v", exp, got)
}
exp := "cpu,host=B#!~#value"
if _, ok := keys[exp]; !ok {
t.Fatalf("wrong series deleted: exp %v, got %v", exp, keys)
}
}
func TestEngine_DeleteSeriesRange(t *testing.T) {
// Create a few points.
p1 := MustParsePointString("cpu,host=0 value=1.1 6000000000") // Should not be deleted
p2 := MustParsePointString("cpu,host=A value=1.2 2000000000")
p3 := MustParsePointString("cpu,host=A value=1.3 3000000000")
p4 := MustParsePointString("cpu,host=B value=1.3 4000000000") // Should not be deleted
p5 := MustParsePointString("cpu,host=B value=1.3 5000000000") // Should not be deleted
p6 := MustParsePointString("cpu,host=C value=1.3 1000000000")
p7 := MustParsePointString("mem,host=C value=1.3 1000000000") // Should not be deleted
p8 := MustParsePointString("disk,host=C value=1.3 1000000000") // Should not be deleted
e, err := NewEngine()
if err != nil {
t.Fatal(err)
}
// mock the planner so compactions don't run during the test
e.CompactionPlan = &mockPlanner{}
if err := e.Open(); err != nil {
t.Fatal(err)
}
defer e.Close()
if err := e.writePoints(p1, p2, p3, p4, p5, p6, p7, p8); err != nil {
t.Fatalf("failed to write points: %s", err.Error())
}
if err := e.WriteSnapshot(); err != nil {
t.Fatalf("failed to snapshot: %s", err.Error())
}
keys := e.FileStore.Keys()
if exp, got := 6, len(keys); exp != got {
t.Fatalf("series count mismatch: exp %v, got %v", exp, got)
}
itr := &seriesIterator{keys: [][]byte{[]byte("cpu,host=0"), []byte("cpu,host=A"), []byte("cpu,host=B"), []byte("cpu,host=C")}}
if err := e.DeleteSeriesRange(itr, 0, 3000000000); err != nil {
t.Fatalf("failed to delete series: %v", err)
}
keys = e.FileStore.Keys()
if exp, got := 4, len(keys); exp != got {
t.Fatalf("series count mismatch: exp %v, got %v", exp, got)
}
exp := "cpu,host=B#!~#value"
if _, ok := keys[exp]; !ok {
t.Fatalf("wrong series deleted: exp %v, got %v", exp, keys)
}
// Check that the series still exists in the index
iter, err := e.index.MeasurementSeriesIDIterator([]byte("cpu"))
if err != nil {
t.Fatalf("iterator error: %v", err)
}
defer iter.Close()
elem, err := iter.Next()
if err != nil {
t.Fatal(err)
}
if elem.SeriesID.IsZero() {
t.Fatalf("series index mismatch: EOF, exp 2 series")
}
// Lookup series.
name, tags := e.sfile.Series(elem.SeriesID)
if got, exp := name, []byte("cpu"); !bytes.Equal(got, exp) {
t.Fatalf("series mismatch: got %s, exp %s", got, exp)
}
if !tags.Equal(models.NewTags(map[string]string{"host": "0"})) && !tags.Equal(models.NewTags(map[string]string{"host": "B"})) {
t.Fatalf(`series mismatch: got %s, exp either "host=0" or "host=B"`, tags)
}
iter.Close()
// Deleting remaining series should remove them from the series.
itr = &seriesIterator{keys: [][]byte{[]byte("cpu,host=0"), []byte("cpu,host=B")}}
if err := e.DeleteSeriesRange(itr, 0, 9000000000); err != nil {
t.Fatalf("failed to delete series: %v", err)
}
if iter, err = e.index.MeasurementSeriesIDIterator([]byte("cpu")); err != nil {
t.Fatalf("iterator error: %v", err)
}
if iter == nil {
return
}
defer iter.Close()
if elem, err = iter.Next(); err != nil {
t.Fatal(err)
}
if !elem.SeriesID.IsZero() {
t.Fatalf("got an undeleted series id, but series should be dropped from index")
}
}
func TestEngine_DeleteSeriesRangeWithPredicate(t *testing.T) {
// Create a few points.
p1 := MustParsePointString("cpu,host=A value=1.1 6000000000") // Should not be deleted
p2 := MustParsePointString("cpu,host=A value=1.2 2000000000") // Should not be deleted
p3 := MustParsePointString("cpu,host=B value=1.3 3000000000")
p4 := MustParsePointString("cpu,host=B value=1.3 4000000000")
p5 := MustParsePointString("cpu,host=C value=1.3 5000000000") // Should not be deleted
p6 := MustParsePointString("mem,host=B value=1.3 1000000000")
p7 := MustParsePointString("mem,host=C value=1.3 1000000000")
p8 := MustParsePointString("disk,host=C value=1.3 1000000000") // Should not be deleted
e, err := NewEngine()
if err != nil {
t.Fatal(err)
}
// mock the planner so compactions don't run during the test
e.CompactionPlan = &mockPlanner{}
if err := e.Open(); err != nil {
t.Fatal(err)
}
defer e.Close()
if err := e.writePoints(p1, p2, p3, p4, p5, p6, p7, p8); err != nil {
t.Fatalf("failed to write points: %s", err.Error())
}
if err := e.WriteSnapshot(); err != nil {
t.Fatalf("failed to snapshot: %s", err.Error())
}
keys := e.FileStore.Keys()
if exp, got := 6, len(keys); exp != got {
t.Fatalf("series count mismatch: exp %v, got %v", exp, got)
}
itr := &seriesIterator{keys: [][]byte{[]byte("cpu,host=A"), []byte("cpu,host=B"), []byte("cpu,host=C"), []byte("mem,host=B"), []byte("mem,host=C")}}
predicate := func(name []byte, tags models.Tags) (int64, int64, bool) {
if bytes.Equal(name, []byte("mem")) {
return math.MinInt64, math.MaxInt64, true
}
if bytes.Equal(name, []byte("cpu")) {
for _, tag := range tags {
if bytes.Equal(tag.Key, []byte("host")) && bytes.Equal(tag.Value, []byte("B")) {
return math.MinInt64, math.MaxInt64, true
}
}
}
return math.MinInt64, math.MaxInt64, false
}
if err := e.DeleteSeriesRangeWithPredicate(itr, predicate); err != nil {
t.Fatalf("failed to delete series: %v", err)
}
keys = e.FileStore.Keys()
if exp, got := 3, len(keys); exp != got {
t.Fatalf("series count mismatch: exp %v, got %v", exp, got)
}
exps := []string{"cpu,host=A#!~#value", "cpu,host=C#!~#value", "disk,host=C#!~#value"}
for _, exp := range exps {
if _, ok := keys[exp]; !ok {
t.Fatalf("wrong series deleted: exp %v, got %v", exps, keys)
}
}
// Check that the series still exists in the index
iter, err := e.index.MeasurementSeriesIDIterator([]byte("cpu"))
if err != nil {
t.Fatalf("iterator error: %v", err)
}
defer iter.Close()
elem, err := iter.Next()
if err != nil {
t.Fatal(err)
}
if elem.SeriesID.IsZero() {
t.Fatalf("series index mismatch: EOF, exp 2 series")
}
// Lookup series.
name, tags := e.sfile.Series(elem.SeriesID)
if got, exp := name, []byte("cpu"); !bytes.Equal(got, exp) {
t.Fatalf("series mismatch: got %s, exp %s", got, exp)
}
if !tags.Equal(models.NewTags(map[string]string{"host": "A"})) && !tags.Equal(models.NewTags(map[string]string{"host": "C"})) {
t.Fatalf(`series mismatch: got %s, exp either "host=A" or "host=C"`, tags)
}
iter.Close()
// Deleting remaining series should remove them from the series.
itr = &seriesIterator{keys: [][]byte{[]byte("cpu,host=A"), []byte("cpu,host=C")}}
if err := e.DeleteSeriesRange(itr, 0, 9000000000); err != nil {
t.Fatalf("failed to delete series: %v", err)
}
if iter, err = e.index.MeasurementSeriesIDIterator([]byte("cpu")); err != nil {
t.Fatalf("iterator error: %v", err)
}
if iter == nil {
return
}
defer iter.Close()
if elem, err = iter.Next(); err != nil {
t.Fatal(err)
}
if !elem.SeriesID.IsZero() {
t.Fatalf("got an undeleted series id, but series should be dropped from index")
}
}
// Tests that a nil predicate deletes all values returned from the series iterator.
func TestEngine_DeleteSeriesRangeWithPredicate_Nil(t *testing.T) {
// Create a few points.
p1 := MustParsePointString("cpu,host=A value=1.1 6000000000") // Should not be deleted
p2 := MustParsePointString("cpu,host=A value=1.2 2000000000") // Should not be deleted
p3 := MustParsePointString("cpu,host=B value=1.3 3000000000")
p4 := MustParsePointString("cpu,host=B value=1.3 4000000000")
p5 := MustParsePointString("cpu,host=C value=1.3 5000000000") // Should not be deleted
p6 := MustParsePointString("mem,host=B value=1.3 1000000000")
p7 := MustParsePointString("mem,host=C value=1.3 1000000000")
p8 := MustParsePointString("disk,host=C value=1.3 1000000000") // Should not be deleted
e, err := NewEngine()
if err != nil {
t.Fatal(err)
}
// mock the planner so compactions don't run during the test
e.CompactionPlan = &mockPlanner{}
if err := e.Open(); err != nil {
t.Fatal(err)
}
defer e.Close()
if err := e.writePoints(p1, p2, p3, p4, p5, p6, p7, p8); err != nil {
t.Fatalf("failed to write points: %s", err.Error())
}
if err := e.WriteSnapshot(); err != nil {
t.Fatalf("failed to snapshot: %s", err.Error())
}
keys := e.FileStore.Keys()
if exp, got := 6, len(keys); exp != got {
t.Fatalf("series count mismatch: exp %v, got %v", exp, got)
}
itr := &seriesIterator{keys: [][]byte{[]byte("cpu,host=A"), []byte("cpu,host=B"), []byte("cpu,host=C"), []byte("mem,host=B"), []byte("mem,host=C")}}
if err := e.DeleteSeriesRangeWithPredicate(itr, nil); err != nil {
t.Fatalf("failed to delete series: %v", err)
}
keys = e.FileStore.Keys()
if exp, got := 1, len(keys); exp != got {
t.Fatalf("series count mismatch: exp %v, got %v", exp, got)
}
// Check that the series still exists in the index
iter, err := e.index.MeasurementSeriesIDIterator([]byte("cpu"))
if err != nil {
t.Fatalf("iterator error: %v", err)
} else if iter == nil {
return
}
defer iter.Close()
if elem, err := iter.Next(); err != nil {
t.Fatal(err)
} else if !elem.SeriesID.IsZero() {
t.Fatalf("got an undeleted series id, but series should be dropped from index")
}
// Check that disk series still exists
iter, err = e.index.MeasurementSeriesIDIterator([]byte("disk"))
if err != nil {
t.Fatalf("iterator error: %v", err)
} else if iter == nil {
return
}
defer iter.Close()
if elem, err := iter.Next(); err != nil {
t.Fatal(err)
} else if elem.SeriesID.IsZero() {
t.Fatalf("got an undeleted series id, but series should be dropped from index")
}
}
func TestEngine_DeleteSeriesRangeWithPredicate_FlushBatch(t *testing.T) {
// Create a few points.
p1 := MustParsePointString("cpu,host=A value=1.1 6000000000") // Should not be deleted
p2 := MustParsePointString("cpu,host=A value=1.2 2000000000") // Should not be deleted
p3 := MustParsePointString("cpu,host=B value=1.3 3000000000")
p4 := MustParsePointString("cpu,host=B value=1.3 4000000000")
p5 := MustParsePointString("cpu,host=C value=1.3 5000000000") // Should not be deleted
p6 := MustParsePointString("mem,host=B value=1.3 1000000000")
p7 := MustParsePointString("mem,host=C value=1.3 1000000000")
p8 := MustParsePointString("disk,host=C value=1.3 1000000000") // Should not be deleted
e, err := NewEngine()
if err != nil {
t.Fatal(err)
}
// mock the planner so compactions don't run during the test
e.CompactionPlan = &mockPlanner{}
if err := e.Open(); err != nil {
t.Fatal(err)
}
defer e.Close()
if err := e.writePoints(p1, p2, p3, p4, p5, p6, p7, p8); err != nil {
t.Fatalf("failed to write points: %s", err.Error())
}
if err := e.WriteSnapshot(); err != nil {
t.Fatalf("failed to snapshot: %s", err.Error())
}
keys := e.FileStore.Keys()
if exp, got := 6, len(keys); exp != got {
t.Fatalf("series count mismatch: exp %v, got %v", exp, got)
}
itr := &seriesIterator{keys: [][]byte{[]byte("cpu,host=A"), []byte("cpu,host=B"), []byte("cpu,host=C"), []byte("mem,host=B"), []byte("mem,host=C")}}
predicate := func(name []byte, tags models.Tags) (int64, int64, bool) {
if bytes.Equal(name, []byte("mem")) {
return 1000000000, 1000000000, true
}
if bytes.Equal(name, []byte("cpu")) {
for _, tag := range tags {
if bytes.Equal(tag.Key, []byte("host")) && bytes.Equal(tag.Value, []byte("B")) {
return 3000000000, 4000000000, true
}
}
}
return math.MinInt64, math.MaxInt64, false
}
if err := e.DeleteSeriesRangeWithPredicate(itr, predicate); err != nil {
t.Fatalf("failed to delete series: %v", err)
}
keys = e.FileStore.Keys()
if exp, got := 3, len(keys); exp != got {
t.Fatalf("series count mismatch: exp %v, got %v", exp, got)
}
exps := []string{"cpu,host=A#!~#value", "cpu,host=C#!~#value", "disk,host=C#!~#value"}
for _, exp := range exps {
if _, ok := keys[exp]; !ok {
t.Fatalf("wrong series deleted: exp %v, got %v", exps, keys)
}
}
// Check that the series still exists in the index
iter, err := e.index.MeasurementSeriesIDIterator([]byte("cpu"))
if err != nil {
t.Fatalf("iterator error: %v", err)
}
defer iter.Close()
elem, err := iter.Next()
if err != nil {
t.Fatal(err)
}
if elem.SeriesID.IsZero() {
t.Fatalf("series index mismatch: EOF, exp 2 series")
}
// Lookup series.
name, tags := e.sfile.Series(elem.SeriesID)
if got, exp := name, []byte("cpu"); !bytes.Equal(got, exp) {
t.Fatalf("series mismatch: got %s, exp %s", got, exp)
}
if !tags.Equal(models.NewTags(map[string]string{"host": "A"})) && !tags.Equal(models.NewTags(map[string]string{"host": "C"})) {
t.Fatalf(`series mismatch: got %s, exp either "host=A" or "host=C"`, tags)
}
iter.Close()
// Deleting remaining series should remove them from the series.
itr = &seriesIterator{keys: [][]byte{[]byte("cpu,host=A"), []byte("cpu,host=C")}}
if err := e.DeleteSeriesRange(itr, 0, 9000000000); err != nil {
t.Fatalf("failed to delete series: %v", err)
}
if iter, err = e.index.MeasurementSeriesIDIterator([]byte("cpu")); err != nil {
t.Fatalf("iterator error: %v", err)
}
if iter == nil {
return
}
defer iter.Close()
if elem, err = iter.Next(); err != nil {
t.Fatal(err)
}
if !elem.SeriesID.IsZero() {
t.Fatalf("got an undeleted series id, but series should be dropped from index")
}
}
func TestEngine_DeleteSeriesRange_OutsideTime(t *testing.T) {
// Create a few points.
p1 := MustParsePointString("cpu,host=A value=1.1 1000000000") // Should not be deleted
e, err := NewEngine()
if err != nil {
t.Fatal(err)
}
// mock the planner so compactions don't run during the test
e.CompactionPlan = &mockPlanner{}
if err := e.Open(); err != nil {
t.Fatal(err)
}
defer e.Close()
if err := e.writePoints(p1); err != nil {
t.Fatalf("failed to write points: %s", err.Error())
}
if err := e.WriteSnapshot(); err != nil {
t.Fatalf("failed to snapshot: %s", err.Error())
}
keys := e.FileStore.Keys()
if exp, got := 1, len(keys); exp != got {
t.Fatalf("series count mismatch: exp %v, got %v", exp, got)
}
itr := &seriesIterator{keys: [][]byte{[]byte("cpu,host=A")}}
if err := e.DeleteSeriesRange(itr, 0, 0); err != nil {
t.Fatalf("failed to delete series: %v", err)
}
keys = e.FileStore.Keys()
if exp, got := 1, len(keys); exp != got {
t.Fatalf("series count mismatch: exp %v, got %v", exp, got)
}
exp := "cpu,host=A#!~#value"
if _, ok := keys[exp]; !ok {
t.Fatalf("wrong series deleted: exp %v, got %v", exp, keys)
}
// Check that the series still exists in the index
iter, err := e.index.MeasurementSeriesIDIterator([]byte("cpu"))
if err != nil {
t.Fatalf("iterator error: %v", err)
}
defer iter.Close()
elem, err := iter.Next()
if err != nil {
t.Fatal(err)
}
if elem.SeriesID.IsZero() {
t.Fatalf("series index mismatch: EOF, exp 1 series")
}
// Lookup series.
name, tags := e.sfile.Series(elem.SeriesID)
if got, exp := name, []byte("cpu"); !bytes.Equal(got, exp) {
t.Fatalf("series mismatch: got %s, exp %s", got, exp)
}
if got, exp := tags, models.NewTags(map[string]string{"host": "A"}); !got.Equal(exp) {
t.Fatalf("series mismatch: got %s, exp %s", got, exp)
}
}
func TestEngine_LastModified(t *testing.T) {
if testing.Short() {
t.Skip("skipping test in short mode")
}
// Create a few points.
p1 := MustParsePointString("cpu,host=A value=1.1 1000000000")
p2 := MustParsePointString("cpu,host=B value=1.2 2000000000")
p3 := MustParsePointString("cpu,host=A sum=1.3 3000000000")
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)
if err := e.Open(); err != nil {
t.Fatal(err)
}
defer e.Close()
if err := e.writePoints(p1, p2, p3); err != nil {
t.Fatalf("failed to write points: %s", err.Error())
}
lm := e.LastModified()
if lm.IsZero() {
t.Fatalf("expected non-zero time, got %v", lm.UTC())
}
e.SetEnabled(true)
// Artificial sleep added due to filesystems caching the mod time
// of files. This prevents the WAL last modified time from being
// returned and newer than the filestore's mod time.
time.Sleep(2 * time.Second) // Covers most filesystems.
if err := e.WriteSnapshot(); err != nil {
t.Fatalf("failed to snapshot: %s", err.Error())
}
lm2 := e.LastModified()
if got, exp := lm.Equal(lm2), false; exp != got {
t.Fatalf("expected time change, got %v, exp %v: %s == %s", got, exp, lm.String(), lm2.String())
}
itr := &seriesIterator{keys: [][]byte{[]byte("cpu,host=A")}}
if err := e.DeleteSeriesRange(itr, math.MinInt64, math.MaxInt64); err != nil {
t.Fatalf("failed to delete series: %v", err)
}
lm3 := e.LastModified()
if got, exp := lm2.Equal(lm3), false; exp != got {
t.Fatalf("expected time change, got %v, exp %v", got, exp)
}
}
func TestEngine_SnapshotsDisabled(t *testing.T) {
sfile := MustOpenSeriesFile()
defer sfile.Close()
@ -1027,7 +433,7 @@ func (e *Engine) WritePointsString(ptstr ...string) error {
func (e *Engine) writePoints(points ...models.Point) error {
// Write into the index.
collection := tsdb.NewSeriesCollection(points)
if err := e.CreateSeriesListIfNotExists(collection); err != nil {
if err := e.index.CreateSeriesListIfNotExists(collection); err != nil {
return err
}
// Write the points into the cache/wal.
@ -1112,30 +518,3 @@ 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) {}
type seriesIterator struct {
keys [][]byte
}
type series struct {
name []byte
tags models.Tags
deleted bool
}
func (s series) Name() []byte { return s.name }
func (s series) Tags() models.Tags { return s.tags }
func (s series) Deleted() bool { return s.deleted }
func (s series) Expr() influxql.Expr { return nil }
func (itr *seriesIterator) Close() error { return nil }
func (itr *seriesIterator) Next() (tsdb.SeriesElem, error) {
if len(itr.keys) == 0 {
return nil, nil
}
name, tags := models.ParseKeyBytes(itr.keys[0])
s := series{name: name, tags: tags}
itr.keys = itr.keys[1:]
return s, nil
}

View File

@ -25,7 +25,6 @@ func PrometheusCollectors() []prometheus.Collector {
collectors = append(collectors, bms.compactionMetrics.PrometheusCollectors()...)
collectors = append(collectors, bms.fileMetrics.PrometheusCollectors()...)
collectors = append(collectors, bms.cacheMetrics.PrometheusCollectors()...)
collectors = append(collectors, bms.walMetrics.PrometheusCollectors()...)
}
return collectors
}
@ -36,7 +35,6 @@ const namespace = "storage"
const compactionSubsystem = "compactions" // sub-system associated with metrics for compactions.
const fileStoreSubsystem = "tsm_files" // sub-system associated with metrics for TSM files.
const cacheSubsystem = "cache" // sub-system associated with metrics for the cache.
const walSubsystem = "wal" // sub-system associated with metrics for the WAL.
// blockMetrics are a set of metrics concerned with tracking data about block storage.
type blockMetrics struct {
@ -44,7 +42,6 @@ type blockMetrics struct {
*compactionMetrics
*fileMetrics
*cacheMetrics
*walMetrics
}
// newBlockMetrics initialises the prometheus metrics for the block subsystem.
@ -54,7 +51,6 @@ func newBlockMetrics(labels prometheus.Labels) *blockMetrics {
compactionMetrics: newCompactionMetrics(labels),
fileMetrics: newFileMetrics(labels),
cacheMetrics: newCacheMetrics(labels),
walMetrics: newWALMetrics(labels),
}
}
@ -64,7 +60,6 @@ func (m *blockMetrics) PrometheusCollectors() []prometheus.Collector {
metrics = append(metrics, m.compactionMetrics.PrometheusCollectors()...)
metrics = append(metrics, m.fileMetrics.PrometheusCollectors()...)
metrics = append(metrics, m.cacheMetrics.PrometheusCollectors()...)
metrics = append(metrics, m.walMetrics.PrometheusCollectors()...)
return metrics
}
@ -249,60 +244,3 @@ func (m *cacheMetrics) PrometheusCollectors() []prometheus.Collector {
m.Writes,
}
}
// walMetrics are a set of metrics concerned with tracking data about compactions.
type walMetrics struct {
OldSegmentBytes *prometheus.GaugeVec
CurrentSegmentBytes *prometheus.GaugeVec
Segments *prometheus.GaugeVec
Writes *prometheus.CounterVec
}
// newWALMetrics initialises the prometheus metrics for tracking the WAL.
func newWALMetrics(labels prometheus.Labels) *walMetrics {
var names []string
for k := range labels {
names = append(names, k)
}
sort.Strings(names)
writeNames := append(append([]string(nil), names...), "status")
sort.Strings(writeNames)
return &walMetrics{
OldSegmentBytes: prometheus.NewGaugeVec(prometheus.GaugeOpts{
Namespace: namespace,
Subsystem: walSubsystem,
Name: "old_segment_bytes",
Help: "Number of bytes old WAL segments using on disk.",
}, names),
CurrentSegmentBytes: prometheus.NewGaugeVec(prometheus.GaugeOpts{
Namespace: namespace,
Subsystem: walSubsystem,
Name: "current_segment_bytes",
Help: "Number of bytes TSM files using on disk.",
}, names),
Segments: prometheus.NewGaugeVec(prometheus.GaugeOpts{
Namespace: namespace,
Subsystem: walSubsystem,
Name: "segments_total",
Help: "Number of WAL segment files on disk.",
}, names),
Writes: prometheus.NewCounterVec(prometheus.CounterOpts{
Namespace: namespace,
Subsystem: walSubsystem,
Name: "writes_total",
Help: "Number of writes to the WAL.",
}, writeNames),
}
}
// PrometheusCollectors satisfies the prom.PrometheusCollector interface.
func (m *walMetrics) PrometheusCollectors() []prometheus.Collector {
return []prometheus.Collector{
m.OldSegmentBytes,
m.CurrentSegmentBytes,
m.Segments,
m.Writes,
}
}

View File

@ -130,73 +130,6 @@ func TestMetrics_Cache(t *testing.T) {
}
}
func TestMetrics_WAL(t *testing.T) {
// metrics to be shared by multiple file stores.
metrics := newWALMetrics(prometheus.Labels{"engine_id": "", "node_id": ""})
t1 := newWALTracker(metrics, prometheus.Labels{"engine_id": "0", "node_id": "0"})
t2 := newWALTracker(metrics, prometheus.Labels{"engine_id": "1", "node_id": "0"})
reg := prometheus.NewRegistry()
reg.MustRegister(metrics.PrometheusCollectors()...)
base := namespace + "_" + walSubsystem + "_"
// All the metric names
gauges := []string{
base + "old_segment_bytes",
base + "current_segment_bytes",
base + "segments_total",
}
counters := []string{
base + "writes_total",
}
// Generate some measurements.
for i, tracker := range []*walTracker{t1, t2} {
tracker.SetOldSegmentSize(uint64(i + len(gauges[0])))
tracker.SetCurrentSegmentSize(uint64(i + len(gauges[1])))
tracker.SetSegments(uint64(i + len(gauges[2])))
labels := tracker.Labels()
labels["status"] = "ok"
tracker.metrics.Writes.With(labels).Add(float64(i + len(counters[0])))
}
// Test that all the correct metrics are present.
mfs, err := reg.Gather()
if err != nil {
t.Fatal(err)
}
// The label variants for the two caches.
labelVariants := []prometheus.Labels{
prometheus.Labels{"engine_id": "0", "node_id": "0"},
prometheus.Labels{"engine_id": "1", "node_id": "0"},
}
for i, labels := range labelVariants {
for _, name := range gauges {
exp := float64(i + len(name))
metric := promtest.MustFindMetric(t, mfs, name, labels)
if got := metric.GetGauge().GetValue(); got != exp {
t.Errorf("[%s %d] got %v, expected %v", name, i, got, exp)
}
}
for _, name := range counters {
exp := float64(i + len(name))
labels["status"] = "ok"
metric := promtest.MustFindMetric(t, mfs, name, labels)
if got := metric.GetCounter().GetValue(); got != exp {
t.Errorf("[%s %d] got %v, expected %v", name, i, got, exp)
}
}
}
}
func TestMetrics_Compactions(t *testing.T) {
// metrics to be shared by multiple file stores.
metrics := newCompactionMetrics(prometheus.Labels{"engine_id": "", "node_id": ""})

123
tsdb/tsm1/value.go Normal file
View File

@ -0,0 +1,123 @@
package tsm1
import (
"fmt"
"time"
"github.com/influxdata/influxdb/models"
"github.com/influxdata/influxdb/tsdb/value"
)
type (
Value = value.Value
IntegerValue = value.IntegerValue
UnsignedValue = value.UnsignedValue
FloatValue = value.FloatValue
BooleanValue = value.BooleanValue
StringValue = value.StringValue
)
// NewValue returns a new Value with the underlying type dependent on value.
func NewValue(t int64, v interface{}) Value { return value.NewValue(t, v) }
// NewRawIntegerValue returns a new integer value.
func NewRawIntegerValue(t int64, v int64) IntegerValue { return value.NewRawIntegerValue(t, v) }
// NewRawUnsignedValue returns a new unsigned integer value.
func NewRawUnsignedValue(t int64, v uint64) UnsignedValue { return value.NewRawUnsignedValue(t, v) }
// NewRawFloatValue returns a new float value.
func NewRawFloatValue(t int64, v float64) FloatValue { return value.NewRawFloatValue(t, v) }
// NewRawBooleanValue returns a new boolean value.
func NewRawBooleanValue(t int64, v bool) BooleanValue { return value.NewRawBooleanValue(t, v) }
// NewRawStringValue returns a new string value.
func NewRawStringValue(t int64, v string) StringValue { return value.NewRawStringValue(t, v) }
// NewIntegerValue returns a new integer value.
func NewIntegerValue(t int64, v int64) Value { return value.NewIntegerValue(t, v) }
// NewUnsignedValue returns a new unsigned integer value.
func NewUnsignedValue(t int64, v uint64) Value { return value.NewUnsignedValue(t, v) }
// NewFloatValue returns a new float value.
func NewFloatValue(t int64, v float64) Value { return value.NewFloatValue(t, v) }
// NewBooleanValue returns a new boolean value.
func NewBooleanValue(t int64, v bool) Value { return value.NewBooleanValue(t, v) }
// NewStringValue returns a new string value.
func NewStringValue(t int64, v string) Value { return value.NewStringValue(t, v) }
// PointsToValues takes in a slice of points and returns it as a map of series key to
// values. It returns an error if any of the points could not be converted.
func PointsToValues(points []models.Point) (map[string][]Value, error) {
values := make(map[string][]Value, len(points))
var (
keyBuf []byte
baseLen int
)
for _, p := range points {
keyBuf = append(keyBuf[:0], p.Key()...)
keyBuf = append(keyBuf, keyFieldSeparator...)
baseLen = len(keyBuf)
iter := p.FieldIterator()
t := p.Time().UnixNano()
for iter.Next() {
keyBuf = append(keyBuf[:baseLen], iter.FieldKey()...)
var v Value
switch iter.Type() {
case models.Float:
fv, err := iter.FloatValue()
if err != nil {
return nil, err
}
v = NewFloatValue(t, fv)
case models.Integer:
iv, err := iter.IntegerValue()
if err != nil {
return nil, err
}
v = NewIntegerValue(t, iv)
case models.Unsigned:
iv, err := iter.UnsignedValue()
if err != nil {
return nil, err
}
v = NewUnsignedValue(t, iv)
case models.String:
v = NewStringValue(t, iter.StringValue())
case models.Boolean:
bv, err := iter.BooleanValue()
if err != nil {
return nil, err
}
v = NewBooleanValue(t, bv)
default:
return nil, fmt.Errorf("unknown field type for %s: %s",
string(iter.FieldKey()), p.String())
}
values[string(keyBuf)] = append(values[string(keyBuf)], v)
}
}
return values, nil
}
// ValuesToPoints takes in a map of values and returns a slice of models.Point.
func ValuesToPoints(values map[string][]Value) []models.Point {
points := make([]models.Point, 0, len(values))
for composite, vals := range values {
series, field := SeriesAndFieldFromCompositeKey([]byte(composite))
strField := string(field)
for _, val := range vals {
t := time.Unix(0, val.UnixNano())
fields := models.Fields{strField: val.Value()}
points = append(points, models.NewPointFromSeries(series, fields, t))
}
}
return points
}

View File

@ -1,864 +0,0 @@
package tsm1_test
import (
"fmt"
"io"
"os"
"reflect"
"testing"
"github.com/golang/snappy"
"github.com/influxdata/influxdb/pkg/slices"
"github.com/influxdata/influxdb/tsdb/tsm1"
)
func TestWALWriter_WriteMulti_Single(t *testing.T) {
dir := MustTempDir()
defer os.RemoveAll(dir)
f := MustTempFile(dir)
w := tsm1.NewWALSegmentWriter(f)
p1 := tsm1.NewValue(1, 1.1)
p2 := tsm1.NewValue(1, int64(1))
p3 := tsm1.NewValue(1, true)
p4 := tsm1.NewValue(1, "string")
p5 := tsm1.NewValue(1, ^uint64(0))
values := map[string][]tsm1.Value{
"cpu,host=A#!~#float": []tsm1.Value{p1},
"cpu,host=A#!~#int": []tsm1.Value{p2},
"cpu,host=A#!~#bool": []tsm1.Value{p3},
"cpu,host=A#!~#string": []tsm1.Value{p4},
"cpu,host=A#!~#unsigned": []tsm1.Value{p5},
}
entry := &tsm1.WriteWALEntry{
Values: values,
}
if err := w.Write(mustMarshalEntry(entry)); err != nil {
fatal(t, "write points", err)
}
if err := w.Flush(); err != nil {
fatal(t, "flush", err)
}
if _, err := f.Seek(0, io.SeekStart); err != nil {
fatal(t, "seek", err)
}
r := tsm1.NewWALSegmentReader(f)
if !r.Next() {
t.Fatalf("expected next, got false")
}
we, err := r.Read()
if err != nil {
fatal(t, "read entry", err)
}
e, ok := we.(*tsm1.WriteWALEntry)
if !ok {
t.Fatalf("expected WriteWALEntry: got %#v", e)
}
for k, v := range e.Values {
for i, vv := range v {
if got, exp := vv.String(), values[k][i].String(); got != exp {
t.Fatalf("points mismatch: got %v, exp %v", got, exp)
}
}
}
if n := r.Count(); n != MustReadFileSize(f) {
t.Fatalf("wrong count of bytes read, got %d, exp %d", n, MustReadFileSize(f))
}
}
func TestWALWriter_WriteMulti_LargeBatch(t *testing.T) {
dir := MustTempDir()
defer os.RemoveAll(dir)
f := MustTempFile(dir)
w := tsm1.NewWALSegmentWriter(f)
var points []tsm1.Value
for i := 0; i < 100000; i++ {
points = append(points, tsm1.NewValue(int64(i), int64(1)))
}
values := map[string][]tsm1.Value{
"cpu,host=A,server=01,foo=bar,tag=really-long#!~#float": points,
"mem,host=A,server=01,foo=bar,tag=really-long#!~#float": points,
}
entry := &tsm1.WriteWALEntry{
Values: values,
}
if err := w.Write(mustMarshalEntry(entry)); err != nil {
fatal(t, "write points", err)
}
if err := w.Flush(); err != nil {
fatal(t, "flush", err)
}
if _, err := f.Seek(0, io.SeekStart); err != nil {
fatal(t, "seek", err)
}
r := tsm1.NewWALSegmentReader(f)
if !r.Next() {
t.Fatalf("expected next, got false")
}
we, err := r.Read()
if err != nil {
fatal(t, "read entry", err)
}
e, ok := we.(*tsm1.WriteWALEntry)
if !ok {
t.Fatalf("expected WriteWALEntry: got %#v", e)
}
for k, v := range e.Values {
for i, vv := range v {
if got, exp := vv.String(), values[k][i].String(); got != exp {
t.Fatalf("points mismatch: got %v, exp %v", got, exp)
}
}
}
if n := r.Count(); n != MustReadFileSize(f) {
t.Fatalf("wrong count of bytes read, got %d, exp %d", n, MustReadFileSize(f))
}
}
func TestWALWriter_WriteMulti_Multiple(t *testing.T) {
dir := MustTempDir()
defer os.RemoveAll(dir)
f := MustTempFile(dir)
w := tsm1.NewWALSegmentWriter(f)
p1 := tsm1.NewValue(1, int64(1))
p2 := tsm1.NewValue(1, int64(2))
exp := []struct {
key string
values []tsm1.Value
}{
{"cpu,host=A#!~#value", []tsm1.Value{p1}},
{"cpu,host=B#!~#value", []tsm1.Value{p2}},
}
for _, v := range exp {
entry := &tsm1.WriteWALEntry{
Values: map[string][]tsm1.Value{v.key: v.values},
}
if err := w.Write(mustMarshalEntry(entry)); err != nil {
fatal(t, "write points", err)
}
if err := w.Flush(); err != nil {
fatal(t, "flush", err)
}
}
// Seek back to the beinning of the file for reading
if _, err := f.Seek(0, io.SeekStart); err != nil {
fatal(t, "seek", err)
}
r := tsm1.NewWALSegmentReader(f)
for _, ep := range exp {
if !r.Next() {
t.Fatalf("expected next, got false")
}
we, err := r.Read()
if err != nil {
fatal(t, "read entry", err)
}
e, ok := we.(*tsm1.WriteWALEntry)
if !ok {
t.Fatalf("expected WriteWALEntry: got %#v", e)
}
for k, v := range e.Values {
if got, exp := k, ep.key; got != exp {
t.Fatalf("key mismatch. got %v, exp %v", got, exp)
}
if got, exp := len(v), len(ep.values); got != exp {
t.Fatalf("values length mismatch: got %v, exp %v", got, exp)
}
for i, vv := range v {
if got, exp := vv.String(), ep.values[i].String(); got != exp {
t.Fatalf("points mismatch: got %v, exp %v", got, exp)
}
}
}
}
if n := r.Count(); n != MustReadFileSize(f) {
t.Fatalf("wrong count of bytes read, got %d, exp %d", n, MustReadFileSize(f))
}
}
func TestWALWriter_WriteDelete_Single(t *testing.T) {
dir := MustTempDir()
defer os.RemoveAll(dir)
f := MustTempFile(dir)
w := tsm1.NewWALSegmentWriter(f)
entry := &tsm1.DeleteWALEntry{
Keys: [][]byte{[]byte("cpu")},
}
if err := w.Write(mustMarshalEntry(entry)); err != nil {
fatal(t, "write points", err)
}
if err := w.Flush(); err != nil {
fatal(t, "flush", err)
}
if _, err := f.Seek(0, io.SeekStart); err != nil {
fatal(t, "seek", err)
}
r := tsm1.NewWALSegmentReader(f)
if !r.Next() {
t.Fatalf("expected next, got false")
}
we, err := r.Read()
if err != nil {
fatal(t, "read entry", err)
}
e, ok := we.(*tsm1.DeleteWALEntry)
if !ok {
t.Fatalf("expected WriteWALEntry: got %#v", e)
}
if got, exp := len(e.Keys), len(entry.Keys); got != exp {
t.Fatalf("key length mismatch: got %v, exp %v", got, exp)
}
if got, exp := string(e.Keys[0]), string(entry.Keys[0]); got != exp {
t.Fatalf("key mismatch: got %v, exp %v", got, exp)
}
}
func TestWALWriter_WriteMultiDelete_Multiple(t *testing.T) {
dir := MustTempDir()
defer os.RemoveAll(dir)
f := MustTempFile(dir)
w := tsm1.NewWALSegmentWriter(f)
p1 := tsm1.NewValue(1, true)
values := map[string][]tsm1.Value{
"cpu,host=A#!~#value": []tsm1.Value{p1},
}
writeEntry := &tsm1.WriteWALEntry{
Values: values,
}
if err := w.Write(mustMarshalEntry(writeEntry)); err != nil {
fatal(t, "write points", err)
}
if err := w.Flush(); err != nil {
fatal(t, "flush", err)
}
// Write the delete entry
deleteEntry := &tsm1.DeleteWALEntry{
Keys: [][]byte{[]byte("cpu,host=A#!~value")},
}
if err := w.Write(mustMarshalEntry(deleteEntry)); err != nil {
fatal(t, "write points", err)
}
if err := w.Flush(); err != nil {
fatal(t, "flush", err)
}
// Seek back to the beinning of the file for reading
if _, err := f.Seek(0, io.SeekStart); err != nil {
fatal(t, "seek", err)
}
r := tsm1.NewWALSegmentReader(f)
// Read the write points first
if !r.Next() {
t.Fatalf("expected next, got false")
}
we, err := r.Read()
if err != nil {
fatal(t, "read entry", err)
}
e, ok := we.(*tsm1.WriteWALEntry)
if !ok {
t.Fatalf("expected WriteWALEntry: got %#v", e)
}
for k, v := range e.Values {
if got, exp := len(v), len(values[k]); got != exp {
t.Fatalf("values length mismatch: got %v, exp %v", got, exp)
}
for i, vv := range v {
if got, exp := vv.String(), values[k][i].String(); got != exp {
t.Fatalf("points mismatch: got %v, exp %v", got, exp)
}
}
}
// Read the delete second
if !r.Next() {
t.Fatalf("expected next, got false")
}
we, err = r.Read()
if err != nil {
fatal(t, "read entry", err)
}
de, ok := we.(*tsm1.DeleteWALEntry)
if !ok {
t.Fatalf("expected DeleteWALEntry: got %#v", e)
}
if got, exp := len(de.Keys), len(deleteEntry.Keys); got != exp {
t.Fatalf("key length mismatch: got %v, exp %v", got, exp)
}
if got, exp := string(de.Keys[0]), string(deleteEntry.Keys[0]); got != exp {
t.Fatalf("key mismatch: got %v, exp %v", got, exp)
}
}
func TestWALWriter_WriteMultiDeleteRange_Multiple(t *testing.T) {
dir := MustTempDir()
defer os.RemoveAll(dir)
f := MustTempFile(dir)
w := tsm1.NewWALSegmentWriter(f)
p1 := tsm1.NewValue(1, 1.0)
p2 := tsm1.NewValue(2, 2.0)
p3 := tsm1.NewValue(3, 3.0)
values := map[string][]tsm1.Value{
"cpu,host=A#!~#value": []tsm1.Value{p1, p2, p3},
}
writeEntry := &tsm1.WriteWALEntry{
Values: values,
}
if err := w.Write(mustMarshalEntry(writeEntry)); err != nil {
fatal(t, "write points", err)
}
if err := w.Flush(); err != nil {
fatal(t, "flush", err)
}
// Write the delete entry
deleteEntry := &tsm1.DeleteRangeWALEntry{
Keys: [][]byte{[]byte("cpu,host=A#!~value")},
Min: 2,
Max: 3,
}
if err := w.Write(mustMarshalEntry(deleteEntry)); err != nil {
fatal(t, "write points", err)
}
if err := w.Flush(); err != nil {
fatal(t, "flush", err)
}
// Seek back to the beinning of the file for reading
if _, err := f.Seek(0, io.SeekStart); err != nil {
fatal(t, "seek", err)
}
r := tsm1.NewWALSegmentReader(f)
// Read the write points first
if !r.Next() {
t.Fatalf("expected next, got false")
}
we, err := r.Read()
if err != nil {
fatal(t, "read entry", err)
}
e, ok := we.(*tsm1.WriteWALEntry)
if !ok {
t.Fatalf("expected WriteWALEntry: got %#v", e)
}
for k, v := range e.Values {
if got, exp := len(v), len(values[k]); got != exp {
t.Fatalf("values length mismatch: got %v, exp %v", got, exp)
}
for i, vv := range v {
if got, exp := vv.String(), values[k][i].String(); got != exp {
t.Fatalf("points mismatch: got %v, exp %v", got, exp)
}
}
}
// Read the delete second
if !r.Next() {
t.Fatalf("expected next, got false")
}
we, err = r.Read()
if err != nil {
fatal(t, "read entry", err)
}
de, ok := we.(*tsm1.DeleteRangeWALEntry)
if !ok {
t.Fatalf("expected DeleteWALEntry: got %#v", e)
}
if got, exp := len(de.Keys), len(deleteEntry.Keys); got != exp {
t.Fatalf("key length mismatch: got %v, exp %v", got, exp)
}
if got, exp := string(de.Keys[0]), string(deleteEntry.Keys[0]); got != exp {
t.Fatalf("key mismatch: got %v, exp %v", got, exp)
}
if got, exp := de.Min, int64(2); got != exp {
t.Fatalf("min time mismatch: got %v, exp %v", got, exp)
}
if got, exp := de.Max, int64(3); got != exp {
t.Fatalf("min time mismatch: got %v, exp %v", got, exp)
}
}
func TestWAL_ClosedSegments(t *testing.T) {
dir := MustTempDir()
defer os.RemoveAll(dir)
w := tsm1.NewWAL(dir)
if err := w.Open(); err != nil {
t.Fatalf("error opening WAL: %v", err)
}
files, err := w.ClosedSegments()
if err != nil {
t.Fatalf("error getting closed segments: %v", err)
}
if got, exp := len(files), 0; got != exp {
t.Fatalf("close segment length mismatch: got %v, exp %v", got, exp)
}
if _, err := w.WriteMulti(map[string][]tsm1.Value{
"cpu,host=A#!~#value": []tsm1.Value{
tsm1.NewValue(1, 1.1),
},
}); err != nil {
t.Fatalf("error writing points: %v", err)
}
if err := w.Close(); err != nil {
t.Fatalf("error closing wal: %v", err)
}
// Re-open the WAL
w = tsm1.NewWAL(dir)
defer w.Close()
if err := w.Open(); err != nil {
t.Fatalf("error opening WAL: %v", err)
}
files, err = w.ClosedSegments()
if err != nil {
t.Fatalf("error getting closed segments: %v", err)
}
if got, exp := len(files), 0; got != exp {
t.Fatalf("close segment length mismatch: got %v, exp %v", got, exp)
}
}
func TestWAL_Delete(t *testing.T) {
dir := MustTempDir()
defer os.RemoveAll(dir)
w := tsm1.NewWAL(dir)
if err := w.Open(); err != nil {
t.Fatalf("error opening WAL: %v", err)
}
files, err := w.ClosedSegments()
if err != nil {
t.Fatalf("error getting closed segments: %v", err)
}
if got, exp := len(files), 0; got != exp {
t.Fatalf("close segment length mismatch: got %v, exp %v", got, exp)
}
if _, err := w.Delete([][]byte{[]byte("cpu")}); err != nil {
t.Fatalf("error writing points: %v", err)
}
if err := w.Close(); err != nil {
t.Fatalf("error closing wal: %v", err)
}
// Re-open the WAL
w = tsm1.NewWAL(dir)
defer w.Close()
if err := w.Open(); err != nil {
t.Fatalf("error opening WAL: %v", err)
}
files, err = w.ClosedSegments()
if err != nil {
t.Fatalf("error getting closed segments: %v", err)
}
if got, exp := len(files), 0; got != exp {
t.Fatalf("close segment length mismatch: got %v, exp %v", got, exp)
}
}
func TestWALWriter_Corrupt(t *testing.T) {
dir := MustTempDir()
defer os.RemoveAll(dir)
f := MustTempFile(dir)
w := tsm1.NewWALSegmentWriter(f)
corruption := []byte{1, 4, 0, 0, 0}
p1 := tsm1.NewValue(1, 1.1)
values := map[string][]tsm1.Value{
"cpu,host=A#!~#float": []tsm1.Value{p1},
}
entry := &tsm1.WriteWALEntry{
Values: values,
}
if err := w.Write(mustMarshalEntry(entry)); err != nil {
fatal(t, "write points", err)
}
if err := w.Flush(); err != nil {
fatal(t, "flush", err)
}
// Write some random bytes to the file to simulate corruption.
if _, err := f.Write(corruption); err != nil {
fatal(t, "corrupt WAL segment", err)
}
// Create the WAL segment reader.
if _, err := f.Seek(0, io.SeekStart); err != nil {
fatal(t, "seek", err)
}
r := tsm1.NewWALSegmentReader(f)
// Try to decode two entries.
if !r.Next() {
t.Fatalf("expected next, got false")
}
if _, err := r.Read(); err != nil {
fatal(t, "read entry", err)
}
if !r.Next() {
t.Fatalf("expected next, got false")
}
if _, err := r.Read(); err == nil {
fatal(t, "read entry did not return err", nil)
}
// Count should only return size of valid data.
expCount := MustReadFileSize(f) - int64(len(corruption))
if n := r.Count(); n != expCount {
t.Fatalf("wrong count of bytes read, got %d, exp %d", n, expCount)
}
}
// Reproduces a `panic: runtime error: makeslice: cap out of range` when run with
// GOARCH=386 go test -run TestWALSegmentReader_Corrupt -v ./tsdb/engine/tsm1/
func TestWALSegmentReader_Corrupt(t *testing.T) {
dir := MustTempDir()
defer os.RemoveAll(dir)
f := MustTempFile(dir)
w := tsm1.NewWALSegmentWriter(f)
p4 := tsm1.NewValue(1, "string")
values := map[string][]tsm1.Value{
"cpu,host=A#!~#string": []tsm1.Value{p4, p4},
}
entry := &tsm1.WriteWALEntry{
Values: values,
}
typ, b := mustMarshalEntry(entry)
// This causes the nvals field to overflow on 32 bit systems which produces a
// negative count and a panic when reading the segment.
b[25] = 255
if err := w.Write(typ, b); err != nil {
fatal(t, "write points", err)
}
if err := w.Flush(); err != nil {
fatal(t, "flush", err)
}
// Create the WAL segment reader.
if _, err := f.Seek(0, io.SeekStart); err != nil {
fatal(t, "seek", err)
}
r := tsm1.NewWALSegmentReader(f)
defer r.Close()
// Try to decode two entries.
for r.Next() {
r.Read()
}
}
func TestWriteWALSegment_UnmarshalBinary_WriteWALCorrupt(t *testing.T) {
p1 := tsm1.NewValue(1, 1.1)
p2 := tsm1.NewValue(1, int64(1))
p3 := tsm1.NewValue(1, true)
p4 := tsm1.NewValue(1, "string")
p5 := tsm1.NewValue(1, uint64(1))
values := map[string][]tsm1.Value{
"cpu,host=A#!~#float": []tsm1.Value{p1, p1},
"cpu,host=A#!~#int": []tsm1.Value{p2, p2},
"cpu,host=A#!~#bool": []tsm1.Value{p3, p3},
"cpu,host=A#!~#string": []tsm1.Value{p4, p4},
"cpu,host=A#!~#unsigned": []tsm1.Value{p5, p5},
}
w := &tsm1.WriteWALEntry{
Values: values,
}
b, err := w.MarshalBinary()
if err != nil {
t.Fatalf("unexpected error, got %v", err)
}
// Test every possible truncation of a write WAL entry
for i := 0; i < len(b); i++ {
// re-allocated to ensure capacity would be exceed if slicing
truncated := make([]byte, i)
copy(truncated, b[:i])
err := w.UnmarshalBinary(truncated)
if err != nil && err != tsm1.ErrWALCorrupt {
t.Fatalf("unexpected error: %v", err)
}
}
}
func TestDeleteWALEntry_UnmarshalBinary(t *testing.T) {
examples := []struct {
In []string
Out [][]byte
}{
{
In: []string{""},
Out: nil,
},
{
In: []string{"foo"},
Out: [][]byte{[]byte("foo")},
},
{
In: []string{"foo", "bar"},
Out: [][]byte{[]byte("foo"), []byte("bar")},
},
{
In: []string{"foo", "bar", "z", "abc"},
Out: [][]byte{[]byte("foo"), []byte("bar"), []byte("z"), []byte("abc")},
},
{
In: []string{"foo", "bar", "z", "a"},
Out: [][]byte{[]byte("foo"), []byte("bar"), []byte("z"), []byte("a")},
},
}
for i, example := range examples {
w := &tsm1.DeleteWALEntry{Keys: slices.StringsToBytes(example.In...)}
b, err := w.MarshalBinary()
if err != nil {
t.Fatalf("[example %d] unexpected error, got %v", i, err)
}
out := &tsm1.DeleteWALEntry{}
if err := out.UnmarshalBinary(b); err != nil {
t.Fatalf("[example %d] %v", i, err)
}
if !reflect.DeepEqual(example.Out, out.Keys) {
t.Errorf("[example %d] got %v, expected %v", i, out.Keys, example.Out)
}
}
}
func TestWriteWALSegment_UnmarshalBinary_DeleteWALCorrupt(t *testing.T) {
w := &tsm1.DeleteWALEntry{
Keys: [][]byte{[]byte("foo"), []byte("bar")},
}
b, err := w.MarshalBinary()
if err != nil {
t.Fatalf("unexpected error, got %v", err)
}
// Test every possible truncation of a write WAL entry
for i := 0; i < len(b); i++ {
// re-allocated to ensure capacity would be exceed if slicing
truncated := make([]byte, i)
copy(truncated, b[:i])
err := w.UnmarshalBinary(truncated)
if err != nil && err != tsm1.ErrWALCorrupt {
t.Fatalf("unexpected error: %v", err)
}
}
}
func TestWriteWALSegment_UnmarshalBinary_DeleteRangeWALCorrupt(t *testing.T) {
w := &tsm1.DeleteRangeWALEntry{
Keys: [][]byte{[]byte("foo"), []byte("bar")},
Min: 1,
Max: 2,
}
b, err := w.MarshalBinary()
if err != nil {
t.Fatalf("unexpected error, got %v", err)
}
// Test every possible truncation of a write WAL entry
for i := 0; i < len(b); i++ {
// re-allocated to ensure capacity would be exceed if slicing
truncated := make([]byte, i)
copy(truncated, b[:i])
err := w.UnmarshalBinary(truncated)
if err != nil && err != tsm1.ErrWALCorrupt {
t.Fatalf("unexpected error: %v", err)
}
}
}
func BenchmarkWALSegmentWriter(b *testing.B) {
points := map[string][]tsm1.Value{}
for i := 0; i < 5000; i++ {
k := "cpu,host=A#!~#value"
points[k] = append(points[k], tsm1.NewValue(int64(i), 1.1))
}
dir := MustTempDir()
defer os.RemoveAll(dir)
f := MustTempFile(dir)
w := tsm1.NewWALSegmentWriter(f)
write := &tsm1.WriteWALEntry{
Values: points,
}
b.ResetTimer()
for i := 0; i < b.N; i++ {
if err := w.Write(mustMarshalEntry(write)); err != nil {
b.Fatalf("unexpected error writing entry: %v", err)
}
}
}
func BenchmarkWALSegmentReader(b *testing.B) {
points := map[string][]tsm1.Value{}
for i := 0; i < 5000; i++ {
k := "cpu,host=A#!~#value"
points[k] = append(points[k], tsm1.NewValue(int64(i), 1.1))
}
dir := MustTempDir()
defer os.RemoveAll(dir)
f := MustTempFile(dir)
w := tsm1.NewWALSegmentWriter(f)
write := &tsm1.WriteWALEntry{
Values: points,
}
for i := 0; i < 100; i++ {
if err := w.Write(mustMarshalEntry(write)); err != nil {
b.Fatalf("unexpected error writing entry: %v", err)
}
}
r := tsm1.NewWALSegmentReader(f)
b.ResetTimer()
for i := 0; i < b.N; i++ {
b.StopTimer()
f.Seek(0, io.SeekStart)
b.StartTimer()
for r.Next() {
_, err := r.Read()
if err != nil {
b.Fatalf("unexpected error reading entry: %v", err)
}
}
}
}
// MustReadFileSize returns the size of the file, or panics.
func MustReadFileSize(f *os.File) int64 {
stat, err := os.Stat(f.Name())
if err != nil {
panic(fmt.Sprintf("failed to get size of file at %s: %s", f.Name(), err.Error()))
}
return stat.Size()
}
func mustMarshalEntry(entry tsm1.WALEntry) (tsm1.WalEntryType, []byte) {
bytes := make([]byte, 1024<<2)
b, err := entry.Encode(bytes)
if err != nil {
panic(fmt.Sprintf("error encoding: %v", err))
}
return entry.Type(), snappy.Encode(b, b)
}

236
tsdb/value/value.go Normal file
View File

@ -0,0 +1,236 @@
package value
import (
"fmt"
"time"
"github.com/influxdata/influxdb/tsdb"
)
// Value represents a TSM-encoded value.
type Value interface {
// UnixNano returns the timestamp of the value in nanoseconds since unix epoch.
UnixNano() int64
// Value returns the underlying value.
Value() interface{}
// Size returns the number of bytes necessary to represent the value and its timestamp.
Size() int
// String returns the string representation of the value and its timestamp.
String() string
// internalOnly is unexported to ensure implementations of Value
// can only originate in this package.
internalOnly()
}
// NewValue returns a new Value with the underlying type dependent on value.
func NewValue(t int64, value interface{}) Value {
switch v := value.(type) {
case int64:
return IntegerValue{unixnano: t, value: v}
case uint64:
return UnsignedValue{unixnano: t, value: v}
case float64:
return FloatValue{unixnano: t, value: v}
case bool:
return BooleanValue{unixnano: t, value: v}
case string:
return StringValue{unixnano: t, value: v}
}
return EmptyValue{}
}
// NewRawIntegerValue returns a new integer value.
func NewRawIntegerValue(t int64, v int64) IntegerValue { return IntegerValue{unixnano: t, value: v} }
// NewRawUnsignedValue returns a new unsigned integer value.
func NewRawUnsignedValue(t int64, v uint64) UnsignedValue { return UnsignedValue{unixnano: t, value: v} }
// NewRawFloatValue returns a new float value.
func NewRawFloatValue(t int64, v float64) FloatValue { return FloatValue{unixnano: t, value: v} }
// NewRawBooleanValue returns a new boolean value.
func NewRawBooleanValue(t int64, v bool) BooleanValue { return BooleanValue{unixnano: t, value: v} }
// NewRawStringValue returns a new string value.
func NewRawStringValue(t int64, v string) StringValue { return StringValue{unixnano: t, value: v} }
// NewIntegerValue returns a new integer value.
func NewIntegerValue(t int64, v int64) Value { return NewRawIntegerValue(t, v) }
// NewUnsignedValue returns a new unsigned integer value.
func NewUnsignedValue(t int64, v uint64) Value { return NewRawUnsignedValue(t, v) }
// NewFloatValue returns a new float value.
func NewFloatValue(t int64, v float64) Value { return NewRawFloatValue(t, v) }
// NewBooleanValue returns a new boolean value.
func NewBooleanValue(t int64, v bool) Value { return NewRawBooleanValue(t, v) }
// NewStringValue returns a new string value.
func NewStringValue(t int64, v string) Value { return NewRawStringValue(t, v) }
// EmptyValue is used when there is no appropriate other value.
type EmptyValue struct{}
// UnixNano returns tsdb.EOF.
func (e EmptyValue) UnixNano() int64 { return tsdb.EOF }
// Value returns nil.
func (e EmptyValue) Value() interface{} { return nil }
// Size returns 0.
func (e EmptyValue) Size() int { return 0 }
// String returns the empty string.
func (e EmptyValue) String() string { return "" }
func (EmptyValue) internalOnly() {}
func (StringValue) internalOnly() {}
func (IntegerValue) internalOnly() {}
func (UnsignedValue) internalOnly() {}
func (BooleanValue) internalOnly() {}
func (FloatValue) internalOnly() {}
// IntegerValue represents an int64 value.
type IntegerValue struct {
unixnano int64
value int64
}
// Value returns the underlying int64 value.
func (v IntegerValue) Value() interface{} {
return v.value
}
// UnixNano returns the timestamp of the value.
func (v IntegerValue) UnixNano() int64 {
return v.unixnano
}
// Size returns the number of bytes necessary to represent the value and its timestamp.
func (v IntegerValue) Size() int {
return 16
}
// String returns the string representation of the value and its timestamp.
func (v IntegerValue) String() string {
return fmt.Sprintf("%v %v", time.Unix(0, v.unixnano), v.Value())
}
func (v IntegerValue) RawValue() int64 { return v.value }
// UnsignedValue represents an int64 value.
type UnsignedValue struct {
unixnano int64
value uint64
}
// Value returns the underlying int64 value.
func (v UnsignedValue) Value() interface{} {
return v.value
}
// UnixNano returns the timestamp of the value.
func (v UnsignedValue) UnixNano() int64 {
return v.unixnano
}
// Size returns the number of bytes necessary to represent the value and its timestamp.
func (v UnsignedValue) Size() int {
return 16
}
// String returns the string representation of the value and its timestamp.
func (v UnsignedValue) String() string {
return fmt.Sprintf("%v %v", time.Unix(0, v.unixnano), v.Value())
}
func (v UnsignedValue) RawValue() uint64 { return v.value }
// FloatValue represents a float64 value.
type FloatValue struct {
unixnano int64
value float64
}
// UnixNano returns the timestamp of the value.
func (v FloatValue) UnixNano() int64 {
return v.unixnano
}
// Value returns the underlying float64 value.
func (v FloatValue) Value() interface{} {
return v.value
}
// Size returns the number of bytes necessary to represent the value and its timestamp.
func (v FloatValue) Size() int {
return 16
}
// String returns the string representation of the value and its timestamp.
func (v FloatValue) String() string {
return fmt.Sprintf("%v %v", time.Unix(0, v.unixnano), v.value)
}
func (v FloatValue) RawValue() float64 { return v.value }
// BooleanValue represents a boolean value.
type BooleanValue struct {
unixnano int64
value bool
}
// Size returns the number of bytes necessary to represent the value and its timestamp.
func (v BooleanValue) Size() int {
return 9
}
// UnixNano returns the timestamp of the value in nanoseconds since unix epoch.
func (v BooleanValue) UnixNano() int64 {
return v.unixnano
}
// Value returns the underlying boolean value.
func (v BooleanValue) Value() interface{} {
return v.value
}
// String returns the string representation of the value and its timestamp.
func (v BooleanValue) String() string {
return fmt.Sprintf("%v %v", time.Unix(0, v.unixnano), v.Value())
}
func (v BooleanValue) RawValue() bool { return v.value }
// StringValue represents a string value.
type StringValue struct {
unixnano int64
value string
}
// Value returns the underlying string value.
func (v StringValue) Value() interface{} {
return v.value
}
// UnixNano returns the timestamp of the value.
func (v StringValue) UnixNano() int64 {
return v.unixnano
}
// Size returns the number of bytes necessary to represent the value and its timestamp.
func (v StringValue) Size() int {
return 8 + len(v.value)
}
// String returns the string representation of the value and its timestamp.
func (v StringValue) String() string {
return fmt.Sprintf("%v %v", time.Unix(0, v.unixnano), v.Value())
}
func (v StringValue) RawValue() string { return v.value }