Merge pull request #11364 from influxdata/jmw-hoist-wal
Move the WAL from the tsm1.Engine to the storage.Enginepull/11678/head
commit
d728c8fa03
|
@ -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()))
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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,
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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
|
||||
}
|
|
@ -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
|
||||
|
|
|
@ -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 }
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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)
|
||||
}
|
|
@ -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,
|
||||
}
|
||||
}
|
|
@ -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)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -1,4 +1,4 @@
|
|||
package tsm1
|
||||
package wal
|
||||
|
||||
import "sync"
|
||||
|
|
@ -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()
|
||||
}
|
|
@ -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 }
|
|
@ -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)
|
||||
}
|
|
@ -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--
|
||||
}
|
||||
|
|
|
@ -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--
|
||||
}
|
||||
|
|
|
@ -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"
|
||||
}
|
||||
]
|
|
@ -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()
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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()
|
||||
|
||||
|
|
|
@ -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()
|
||||
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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)
|
||||
|
|
@ -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)
|
||||
}
|
||||
|
|
@ -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
|
||||
}
|
||||
|
|
|
@ -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,
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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": ""})
|
||||
|
|
|
@ -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
|
||||
}
|
|
@ -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)
|
||||
}
|
|
@ -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 }
|
Loading…
Reference in New Issue