chore: Converging on 1.x tsdb
parent
2b175291be
commit
26c18ac0a5
|
@ -2,29 +2,350 @@
|
|||
package buildtsi
|
||||
|
||||
import (
|
||||
"context"
|
||||
"errors"
|
||||
"flag"
|
||||
"fmt"
|
||||
"io"
|
||||
"io/ioutil"
|
||||
"os"
|
||||
"os/user"
|
||||
"path/filepath"
|
||||
"runtime"
|
||||
"strconv"
|
||||
"strings"
|
||||
"sync/atomic"
|
||||
|
||||
"github.com/influxdata/influxdb/v2/models"
|
||||
"github.com/influxdata/influxdb/v2/pkg/fs"
|
||||
"github.com/influxdata/influxdb/v2/storage/wal"
|
||||
"github.com/influxdata/influxdb/v2/toml"
|
||||
"github.com/influxdata/influxdb/v2/logger"
|
||||
"github.com/influxdata/influxdb/v2/pkg/file"
|
||||
"github.com/influxdata/influxdb/v2/v1/models"
|
||||
"github.com/influxdata/influxdb/v2/v1/tsdb"
|
||||
"github.com/influxdata/influxdb/v2/v1/tsdb/index/tsi1"
|
||||
"github.com/influxdata/influxdb/v2/v1/tsdb/engine/tsm1"
|
||||
"github.com/influxdata/influxdb/v2/v1/tsdb/index/tsi1"
|
||||
"go.uber.org/zap"
|
||||
"golang.org/x/sync/errgroup"
|
||||
)
|
||||
|
||||
func IndexShard(sfile *seriesfile.SeriesFile, indexPath, dataDir, walDir string, maxLogFileSize int64, maxCacheSize uint64, batchSize int, log *zap.Logger, verboseLogging bool) error {
|
||||
const defaultBatchSize = 10000
|
||||
|
||||
// Command represents the program execution for "influx_inspect buildtsi".
|
||||
type Command struct {
|
||||
Stderr io.Writer
|
||||
Stdout io.Writer
|
||||
Verbose bool
|
||||
Logger *zap.Logger
|
||||
|
||||
concurrency int // Number of goroutines to dedicate to shard index building.
|
||||
databaseFilter string
|
||||
retentionFilter string
|
||||
shardFilter string
|
||||
compactSeriesFile bool
|
||||
maxLogFileSize int64
|
||||
maxCacheSize uint64
|
||||
batchSize int
|
||||
}
|
||||
|
||||
// NewCommand returns a new instance of Command.
|
||||
func NewCommand() *Command {
|
||||
return &Command{
|
||||
Stderr: os.Stderr,
|
||||
Stdout: os.Stdout,
|
||||
Logger: zap.NewNop(),
|
||||
batchSize: defaultBatchSize,
|
||||
concurrency: runtime.GOMAXPROCS(0),
|
||||
}
|
||||
}
|
||||
|
||||
// Run executes the command.
|
||||
func (cmd *Command) Run(args ...string) error {
|
||||
fs := flag.NewFlagSet("buildtsi", flag.ExitOnError)
|
||||
dataDir := fs.String("datadir", "", "data directory")
|
||||
walDir := fs.String("waldir", "", "WAL directory")
|
||||
fs.IntVar(&cmd.concurrency, "concurrency", runtime.GOMAXPROCS(0), "Number of workers to dedicate to shard index building. Defaults to GOMAXPROCS")
|
||||
fs.StringVar(&cmd.databaseFilter, "database", "", "optional: database name")
|
||||
fs.StringVar(&cmd.retentionFilter, "retention", "", "optional: retention policy")
|
||||
fs.StringVar(&cmd.shardFilter, "shard", "", "optional: shard id")
|
||||
fs.BoolVar(&cmd.compactSeriesFile, "compact-series-file", false, "optional: compact existing series file. Do not rebuilt index.")
|
||||
fs.Int64Var(&cmd.maxLogFileSize, "max-log-file-size", tsdb.DefaultMaxIndexLogFileSize, "optional: maximum log file size")
|
||||
fs.Uint64Var(&cmd.maxCacheSize, "max-cache-size", tsdb.DefaultCacheMaxMemorySize, "optional: maximum cache size")
|
||||
fs.IntVar(&cmd.batchSize, "batch-size", defaultBatchSize, "optional: set the size of the batches we write to the index. Setting this can have adverse affects on performance and heap requirements")
|
||||
fs.BoolVar(&cmd.Verbose, "v", false, "verbose")
|
||||
fs.SetOutput(cmd.Stdout)
|
||||
if err := fs.Parse(args); err != nil {
|
||||
return err
|
||||
} else if fs.NArg() > 0 || *dataDir == "" || *walDir == "" {
|
||||
fs.Usage()
|
||||
return nil
|
||||
}
|
||||
cmd.Logger = logger.New(cmd.Stderr)
|
||||
|
||||
return cmd.run(*dataDir, *walDir)
|
||||
}
|
||||
|
||||
func (cmd *Command) run(dataDir, walDir string) error {
|
||||
// Verify the user actually wants to run as root.
|
||||
if isRoot() {
|
||||
fmt.Fprintln(cmd.Stdout, "You are currently running as root. This will build your")
|
||||
fmt.Fprintln(cmd.Stdout, "index files with root ownership and will be inaccessible")
|
||||
fmt.Fprintln(cmd.Stdout, "if you run influxd as a non-root user. You should run")
|
||||
fmt.Fprintln(cmd.Stdout, "buildtsi as the same user you are running influxd.")
|
||||
fmt.Fprint(cmd.Stdout, "Are you sure you want to continue? (y/N): ")
|
||||
var answer string
|
||||
if fmt.Scanln(&answer); !strings.HasPrefix(strings.TrimSpace(strings.ToLower(answer)), "y") {
|
||||
return fmt.Errorf("operation aborted")
|
||||
}
|
||||
}
|
||||
|
||||
if cmd.compactSeriesFile {
|
||||
if cmd.retentionFilter != "" {
|
||||
return errors.New("cannot specify retention policy when compacting series file")
|
||||
} else if cmd.shardFilter != "" {
|
||||
return errors.New("cannot specify shard ID when compacting series file")
|
||||
}
|
||||
}
|
||||
|
||||
fis, err := ioutil.ReadDir(dataDir)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
for _, fi := range fis {
|
||||
name := fi.Name()
|
||||
if !fi.IsDir() {
|
||||
continue
|
||||
} else if cmd.databaseFilter != "" && name != cmd.databaseFilter {
|
||||
continue
|
||||
}
|
||||
|
||||
if cmd.compactSeriesFile {
|
||||
if err := cmd.compactDatabaseSeriesFile(name, filepath.Join(dataDir, name)); err != nil {
|
||||
return err
|
||||
}
|
||||
continue
|
||||
}
|
||||
|
||||
if err := cmd.processDatabase(name, filepath.Join(dataDir, name), filepath.Join(walDir, name)); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// compactDatabaseSeriesFile compacts the series file segments associated with
|
||||
// the series file for the provided database.
|
||||
func (cmd *Command) compactDatabaseSeriesFile(dbName, path string) error {
|
||||
sfilePath := filepath.Join(path, tsdb.SeriesFileDirectory)
|
||||
paths, err := cmd.seriesFilePartitionPaths(sfilePath)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
// Build input channel.
|
||||
pathCh := make(chan string, len(paths))
|
||||
for _, path := range paths {
|
||||
pathCh <- path
|
||||
}
|
||||
close(pathCh)
|
||||
|
||||
// Concurrently process each partition in the series file
|
||||
var g errgroup.Group
|
||||
for i := 0; i < cmd.concurrency; i++ {
|
||||
g.Go(func() error {
|
||||
for path := range pathCh {
|
||||
if err := cmd.compactSeriesFilePartition(path); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
return nil
|
||||
})
|
||||
}
|
||||
if err := g.Wait(); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
// Build new series file indexes
|
||||
sfile := tsdb.NewSeriesFile(sfilePath)
|
||||
if err = sfile.Open(); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
compactor := tsdb.NewSeriesPartitionCompactor()
|
||||
for _, partition := range sfile.Partitions() {
|
||||
if err = compactor.Compact(partition); err != nil {
|
||||
return err
|
||||
}
|
||||
fmt.Fprintln(cmd.Stdout, "compacted ", partition.Path())
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (cmd *Command) compactSeriesFilePartition(path string) error {
|
||||
const tmpExt = ".tmp"
|
||||
|
||||
fmt.Fprintf(cmd.Stdout, "processing partition for %q\n", path)
|
||||
|
||||
// Open partition so index can recover from entries not in the snapshot.
|
||||
partitionID, err := strconv.Atoi(filepath.Base(path))
|
||||
if err != nil {
|
||||
return fmt.Errorf("cannot parse partition id from path: %s", path)
|
||||
}
|
||||
p := tsdb.NewSeriesPartition(partitionID, path, nil)
|
||||
if err := p.Open(); err != nil {
|
||||
return fmt.Errorf("cannot open partition: path=%s err=%s", path, err)
|
||||
}
|
||||
defer p.Close()
|
||||
|
||||
// Loop over segments and compact.
|
||||
indexPath := p.IndexPath()
|
||||
var segmentPaths []string
|
||||
for _, segment := range p.Segments() {
|
||||
fmt.Fprintf(cmd.Stdout, "processing segment %q %d\n", segment.Path(), segment.ID())
|
||||
|
||||
if err := segment.CompactToPath(segment.Path()+tmpExt, p.Index()); err != nil {
|
||||
return err
|
||||
}
|
||||
segmentPaths = append(segmentPaths, segment.Path())
|
||||
}
|
||||
|
||||
// Close partition.
|
||||
if err := p.Close(); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
// Remove the old segment files and replace with new ones.
|
||||
for _, dst := range segmentPaths {
|
||||
src := dst + tmpExt
|
||||
|
||||
fmt.Fprintf(cmd.Stdout, "renaming new segment %q to %q\n", src, dst)
|
||||
if err = file.RenameFile(src, dst); err != nil && !os.IsNotExist(err) {
|
||||
return fmt.Errorf("serious failure. Please rebuild index and series file: %v", err)
|
||||
}
|
||||
}
|
||||
|
||||
// Remove index file so it will be rebuilt when reopened.
|
||||
fmt.Fprintln(cmd.Stdout, "removing index file", indexPath)
|
||||
if err = os.Remove(indexPath); err != nil && !os.IsNotExist(err) { // index won't exist for low cardinality
|
||||
return err
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// seriesFilePartitionPaths returns the paths to each partition in the series file.
|
||||
func (cmd *Command) seriesFilePartitionPaths(path string) ([]string, error) {
|
||||
sfile := tsdb.NewSeriesFile(path)
|
||||
sfile.Logger = cmd.Logger
|
||||
if err := sfile.Open(); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
var paths []string
|
||||
for _, partition := range sfile.Partitions() {
|
||||
paths = append(paths, partition.Path())
|
||||
}
|
||||
if err := sfile.Close(); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return paths, nil
|
||||
}
|
||||
|
||||
func (cmd *Command) processDatabase(dbName, dataDir, walDir string) error {
|
||||
cmd.Logger.Info("Rebuilding database", zap.String("name", dbName))
|
||||
|
||||
sfile := tsdb.NewSeriesFile(filepath.Join(dataDir, tsdb.SeriesFileDirectory))
|
||||
sfile.Logger = cmd.Logger
|
||||
if err := sfile.Open(); err != nil {
|
||||
return err
|
||||
}
|
||||
defer sfile.Close()
|
||||
|
||||
fis, err := ioutil.ReadDir(dataDir)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
for _, fi := range fis {
|
||||
rpName := fi.Name()
|
||||
if !fi.IsDir() {
|
||||
continue
|
||||
} else if rpName == tsdb.SeriesFileDirectory {
|
||||
continue
|
||||
} else if cmd.retentionFilter != "" && rpName != cmd.retentionFilter {
|
||||
continue
|
||||
}
|
||||
|
||||
if err := cmd.processRetentionPolicy(sfile, dbName, rpName, filepath.Join(dataDir, rpName), filepath.Join(walDir, rpName)); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (cmd *Command) processRetentionPolicy(sfile *tsdb.SeriesFile, dbName, rpName, dataDir, walDir string) error {
|
||||
cmd.Logger.Info("Rebuilding retention policy", logger.Database(dbName), logger.RetentionPolicy(rpName))
|
||||
|
||||
fis, err := ioutil.ReadDir(dataDir)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
type shard struct {
|
||||
ID uint64
|
||||
Path string
|
||||
}
|
||||
|
||||
var shards []shard
|
||||
|
||||
for _, fi := range fis {
|
||||
if !fi.IsDir() {
|
||||
continue
|
||||
} else if cmd.shardFilter != "" && fi.Name() != cmd.shardFilter {
|
||||
continue
|
||||
}
|
||||
|
||||
shardID, err := strconv.ParseUint(fi.Name(), 10, 64)
|
||||
if err != nil {
|
||||
continue
|
||||
}
|
||||
|
||||
shards = append(shards, shard{shardID, fi.Name()})
|
||||
}
|
||||
|
||||
errC := make(chan error, len(shards))
|
||||
var maxi uint32 // index of maximum shard being worked on.
|
||||
for k := 0; k < cmd.concurrency; k++ {
|
||||
go func() {
|
||||
for {
|
||||
i := int(atomic.AddUint32(&maxi, 1) - 1) // Get next partition to work on.
|
||||
if i >= len(shards) {
|
||||
return // No more work.
|
||||
}
|
||||
|
||||
id, name := shards[i].ID, shards[i].Path
|
||||
log := cmd.Logger.With(logger.Database(dbName), logger.RetentionPolicy(rpName), logger.Shard(id))
|
||||
errC <- IndexShard(sfile, filepath.Join(dataDir, name), filepath.Join(walDir, name), cmd.maxLogFileSize, cmd.maxCacheSize, cmd.batchSize, log, cmd.Verbose)
|
||||
}
|
||||
}()
|
||||
}
|
||||
|
||||
// Check for error
|
||||
for i := 0; i < cap(errC); i++ {
|
||||
if err := <-errC; err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func IndexShard(sfile *tsdb.SeriesFile, dataDir, walDir string, maxLogFileSize int64, maxCacheSize uint64, batchSize int, log *zap.Logger, verboseLogging bool) error {
|
||||
log.Info("Rebuilding shard")
|
||||
|
||||
// Check if shard already has a TSI index.
|
||||
indexPath := filepath.Join(dataDir, "index")
|
||||
log.Info("Checking index path", zap.String("path", indexPath))
|
||||
if _, err := os.Stat(indexPath); !os.IsNotExist(err) {
|
||||
log.Info("TSI1 index already exists, skipping", zap.String("path", indexPath))
|
||||
log.Info("tsi1 index already exists, skipping", zap.String("path", indexPath))
|
||||
return nil
|
||||
}
|
||||
|
||||
|
@ -38,21 +359,19 @@ func IndexShard(sfile *seriesfile.SeriesFile, indexPath, dataDir, walDir string,
|
|||
}
|
||||
|
||||
// Open TSI index in temporary path.
|
||||
c := tsi1.NewConfig()
|
||||
c.MaxIndexLogFileSize = toml.Size(maxLogFileSize)
|
||||
|
||||
tsiIndex := tsi1.NewIndex(sfile, c,
|
||||
tsiIndex := tsi1.NewIndex(sfile, "",
|
||||
tsi1.WithPath(tmpPath),
|
||||
tsi1.WithMaximumLogFileSize(maxLogFileSize),
|
||||
tsi1.DisableFsync(),
|
||||
// Each new series entry in a log file is ~12 bytes so this should
|
||||
// roughly equate to one flush to the file for every batch.
|
||||
tsi1.WithLogFileBufferSize(12*batchSize),
|
||||
tsi1.DisableMetrics(), // Disable metrics when rebuilding an index
|
||||
)
|
||||
|
||||
tsiIndex.WithLogger(log)
|
||||
|
||||
log.Info("Opening tsi index in temporary location", zap.String("path", tmpPath))
|
||||
if err := tsiIndex.Open(context.Background()); err != nil {
|
||||
if err := tsiIndex.Open(); err != nil {
|
||||
return err
|
||||
}
|
||||
defer tsiIndex.Close()
|
||||
|
@ -81,7 +400,7 @@ func IndexShard(sfile *seriesfile.SeriesFile, indexPath, dataDir, walDir string,
|
|||
|
||||
} else {
|
||||
log.Info("Building cache from wal files")
|
||||
cache := tsm1.NewCache(uint64(tsm1.DefaultCacheMaxMemorySize))
|
||||
cache := tsm1.NewCache(maxCacheSize)
|
||||
loader := tsm1.NewCacheLoader(walPaths)
|
||||
loader.WithLogger(log)
|
||||
if err := loader.Load(cache); err != nil {
|
||||
|
@ -89,47 +408,46 @@ func IndexShard(sfile *seriesfile.SeriesFile, indexPath, dataDir, walDir string,
|
|||
}
|
||||
|
||||
log.Info("Iterating over cache")
|
||||
collection := &tsdb.SeriesCollection{
|
||||
Keys: make([][]byte, 0, batchSize),
|
||||
Names: make([][]byte, 0, batchSize),
|
||||
Tags: make([]models.Tags, 0, batchSize),
|
||||
Types: make([]models.FieldType, 0, batchSize),
|
||||
}
|
||||
keysBatch := make([][]byte, 0, batchSize)
|
||||
namesBatch := make([][]byte, 0, batchSize)
|
||||
tagsBatch := make([]models.Tags, 0, batchSize)
|
||||
|
||||
for _, key := range cache.Keys() {
|
||||
seriesKey, _ := tsm1.SeriesAndFieldFromCompositeKey(key)
|
||||
name, tags := models.ParseKeyBytes(seriesKey)
|
||||
typ, _ := cache.Type(key)
|
||||
|
||||
if verboseLogging {
|
||||
log.Info("Series", zap.String("name", string(name)), zap.String("tags", tags.String()))
|
||||
}
|
||||
|
||||
collection.Keys = append(collection.Keys, seriesKey)
|
||||
collection.Names = append(collection.Names, name)
|
||||
collection.Tags = append(collection.Tags, tags)
|
||||
collection.Types = append(collection.Types, typ)
|
||||
keysBatch = append(keysBatch, seriesKey)
|
||||
namesBatch = append(namesBatch, name)
|
||||
tagsBatch = append(tagsBatch, tags)
|
||||
|
||||
// Flush batch?
|
||||
if collection.Length() == batchSize {
|
||||
if err := tsiIndex.CreateSeriesListIfNotExists(collection); err != nil {
|
||||
if len(keysBatch) == batchSize {
|
||||
if err := tsiIndex.CreateSeriesListIfNotExists(keysBatch, namesBatch, tagsBatch); err != nil {
|
||||
return fmt.Errorf("problem creating series: (%s)", err)
|
||||
}
|
||||
collection.Truncate(0)
|
||||
keysBatch = keysBatch[:0]
|
||||
namesBatch = namesBatch[:0]
|
||||
tagsBatch = tagsBatch[:0]
|
||||
}
|
||||
}
|
||||
|
||||
// Flush any remaining series in the batches
|
||||
if collection.Length() > 0 {
|
||||
if err := tsiIndex.CreateSeriesListIfNotExists(collection); err != nil {
|
||||
if len(keysBatch) > 0 {
|
||||
if err := tsiIndex.CreateSeriesListIfNotExists(keysBatch, namesBatch, tagsBatch); err != nil {
|
||||
return fmt.Errorf("problem creating series: (%s)", err)
|
||||
}
|
||||
collection = nil
|
||||
keysBatch = nil
|
||||
namesBatch = nil
|
||||
tagsBatch = nil
|
||||
}
|
||||
}
|
||||
|
||||
// Attempt to compact the index & wait for all compactions to complete.
|
||||
log.Info("Compacting index")
|
||||
log.Info("compacting index")
|
||||
tsiIndex.Compact()
|
||||
tsiIndex.Wait()
|
||||
|
||||
|
@ -141,7 +459,7 @@ func IndexShard(sfile *seriesfile.SeriesFile, indexPath, dataDir, walDir string,
|
|||
|
||||
// Rename TSI to standard path.
|
||||
log.Info("Moving tsi to permanent location")
|
||||
return fs.RenameFile(tmpPath, indexPath)
|
||||
return os.Rename(tmpPath, indexPath)
|
||||
}
|
||||
|
||||
func IndexTSMFile(index *tsi1.Index, path string, batchSize int, log *zap.Logger, verboseLogging bool) error {
|
||||
|
@ -158,49 +476,38 @@ func IndexTSMFile(index *tsi1.Index, path string, batchSize int, log *zap.Logger
|
|||
}
|
||||
defer r.Close()
|
||||
|
||||
collection := &tsdb.SeriesCollection{
|
||||
Keys: make([][]byte, 0, batchSize),
|
||||
Names: make([][]byte, 0, batchSize),
|
||||
Tags: make([]models.Tags, batchSize),
|
||||
Types: make([]models.FieldType, 0, batchSize),
|
||||
}
|
||||
keysBatch := make([][]byte, 0, batchSize)
|
||||
namesBatch := make([][]byte, 0, batchSize)
|
||||
tagsBatch := make([]models.Tags, batchSize)
|
||||
var ti int
|
||||
iter := r.Iterator(nil)
|
||||
for iter.Next() {
|
||||
key := iter.Key()
|
||||
for i := 0; i < r.KeyCount(); i++ {
|
||||
key, _ := r.KeyAt(i)
|
||||
seriesKey, _ := tsm1.SeriesAndFieldFromCompositeKey(key)
|
||||
var name []byte
|
||||
name, collection.Tags[ti] = models.ParseKeyBytesWithTags(seriesKey, collection.Tags[ti])
|
||||
typ := iter.Type()
|
||||
name, tagsBatch[ti] = models.ParseKeyBytesWithTags(seriesKey, tagsBatch[ti])
|
||||
|
||||
if verboseLogging {
|
||||
log.Info("Series", zap.String("name", string(name)), zap.String("tags", collection.Tags[ti].String()))
|
||||
log.Info("Series", zap.String("name", string(name)), zap.String("tags", tagsBatch[ti].String()))
|
||||
}
|
||||
|
||||
collection.Keys = append(collection.Keys, seriesKey)
|
||||
collection.Names = append(collection.Names, name)
|
||||
collection.Types = append(collection.Types, modelsFieldType(typ))
|
||||
keysBatch = append(keysBatch, seriesKey)
|
||||
namesBatch = append(namesBatch, name)
|
||||
ti++
|
||||
|
||||
// Flush batch?
|
||||
if len(collection.Keys) == batchSize {
|
||||
collection.Truncate(ti)
|
||||
if err := index.CreateSeriesListIfNotExists(collection); err != nil {
|
||||
if len(keysBatch) == batchSize {
|
||||
if err := index.CreateSeriesListIfNotExists(keysBatch, namesBatch, tagsBatch[:ti]); err != nil {
|
||||
return fmt.Errorf("problem creating series: (%s)", err)
|
||||
}
|
||||
collection.Truncate(0)
|
||||
collection.Tags = collection.Tags[:batchSize]
|
||||
keysBatch = keysBatch[:0]
|
||||
namesBatch = namesBatch[:0]
|
||||
ti = 0 // Reset tags.
|
||||
}
|
||||
}
|
||||
if err := iter.Err(); err != nil {
|
||||
return fmt.Errorf("problem creating series: (%s)", err)
|
||||
}
|
||||
|
||||
// Flush any remaining series in the batches
|
||||
if len(collection.Keys) > 0 {
|
||||
collection.Truncate(ti)
|
||||
if err := index.CreateSeriesListIfNotExists(collection); err != nil {
|
||||
if len(keysBatch) > 0 {
|
||||
if err := index.CreateSeriesListIfNotExists(keysBatch, namesBatch, tagsBatch[:ti]); err != nil {
|
||||
return fmt.Errorf("problem creating series: (%s)", err)
|
||||
}
|
||||
}
|
||||
|
@ -237,7 +544,7 @@ func collectWALFiles(path string) ([]string, error) {
|
|||
|
||||
var paths []string
|
||||
for _, fi := range fis {
|
||||
if filepath.Ext(fi.Name()) != "."+wal.WALFileExtension {
|
||||
if filepath.Ext(fi.Name()) != "."+tsm1.WALFileExtension {
|
||||
continue
|
||||
}
|
||||
paths = append(paths, filepath.Join(path, fi.Name()))
|
||||
|
@ -245,19 +552,7 @@ func collectWALFiles(path string) ([]string, error) {
|
|||
return paths, nil
|
||||
}
|
||||
|
||||
func modelsFieldType(block byte) models.FieldType {
|
||||
switch block {
|
||||
case tsm1.BlockFloat64:
|
||||
return models.Float
|
||||
case tsm1.BlockInteger:
|
||||
return models.Integer
|
||||
case tsm1.BlockBoolean:
|
||||
return models.Boolean
|
||||
case tsm1.BlockString:
|
||||
return models.String
|
||||
case tsm1.BlockUnsigned:
|
||||
return models.Unsigned
|
||||
default:
|
||||
return models.Empty
|
||||
}
|
||||
func isRoot() bool {
|
||||
user, _ := user.Current()
|
||||
return user != nil && user.Username == "root"
|
||||
}
|
||||
|
|
|
@ -1,159 +0,0 @@
|
|||
package generate
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"os"
|
||||
"time"
|
||||
|
||||
"github.com/influxdata/influxdb/v2/bolt"
|
||||
"github.com/influxdata/influxdb/v2/cmd/influxd/internal/profile"
|
||||
"github.com/influxdata/influxdb/v2/internal/fs"
|
||||
"github.com/influxdata/influxdb/v2/kv"
|
||||
"github.com/influxdata/influxdb/v2/kv/migration"
|
||||
"github.com/influxdata/influxdb/v2/kv/migration/all"
|
||||
"github.com/influxdata/influxdb/v2/pkg/data/gen"
|
||||
"github.com/spf13/cobra"
|
||||
"go.uber.org/zap"
|
||||
)
|
||||
|
||||
var Command = &cobra.Command{
|
||||
Use: "generate <schema.toml>",
|
||||
Short: "Generate time series data sets using TOML schema",
|
||||
Long: `
|
||||
This command will generate time series data direct to disk using schema
|
||||
defined in a TOML file. Use the help-schema subcommand to produce a TOML
|
||||
file to STDOUT, which includes documentation describing the available options.
|
||||
|
||||
|
||||
|
||||
NOTES:
|
||||
|
||||
* The influxd server should not be running when using the generate tool
|
||||
as it modifies the index and TSM data.
|
||||
* This tool is intended for development and testing purposes only and
|
||||
SHOULD NOT be run on a production server.
|
||||
`,
|
||||
Args: cobra.ExactArgs(1),
|
||||
RunE: generateFE,
|
||||
}
|
||||
|
||||
var flags struct {
|
||||
printOnly bool
|
||||
storageSpec StorageSpec
|
||||
profile profile.Config
|
||||
}
|
||||
|
||||
func init() {
|
||||
Command.Flags().SortFlags = false
|
||||
|
||||
pfs := Command.PersistentFlags()
|
||||
pfs.SortFlags = false
|
||||
pfs.BoolVar(&flags.printOnly, "print", false, "Print data spec and exit")
|
||||
|
||||
flags.storageSpec.AddFlags(Command, pfs)
|
||||
|
||||
pfs.StringVar(&flags.profile.CPU, "cpuprofile", "", "Collect a CPU profile")
|
||||
pfs.StringVar(&flags.profile.Memory, "memprofile", "", "Collect a memory profile")
|
||||
}
|
||||
|
||||
func generateFE(_ *cobra.Command, args []string) error {
|
||||
storagePlan, err := flags.storageSpec.Plan()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
storagePlan.PrintPlan(os.Stdout)
|
||||
|
||||
spec, err := gen.NewSpecFromPath(args[0])
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
if err = assignOrgBucket(spec); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
if flags.printOnly {
|
||||
return nil
|
||||
}
|
||||
|
||||
return exec(storagePlan, spec)
|
||||
}
|
||||
|
||||
func assignOrgBucket(spec *gen.Spec) error {
|
||||
boltFile, err := fs.BoltFile()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
store := bolt.NewKVStore(zap.NewNop(), boltFile)
|
||||
if err = store.Open(context.Background()); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
s := kv.NewService(zap.NewNop(), store)
|
||||
|
||||
migrator, err := migration.NewMigrator(
|
||||
zap.NewNop(),
|
||||
store,
|
||||
all.Migrations[:]...,
|
||||
)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
// apply migrations to metadata store
|
||||
if err := migrator.Up(context.Background()); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
org, err := s.FindOrganizationByName(context.Background(), flags.storageSpec.Organization)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
bucket, err := s.FindBucketByName(context.Background(), org.ID, flags.storageSpec.Bucket)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
store.Close()
|
||||
|
||||
spec.OrgID = org.ID
|
||||
spec.BucketID = bucket.ID
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func exec(storagePlan *StoragePlan, spec *gen.Spec) error {
|
||||
tr := gen.TimeRange{
|
||||
Start: storagePlan.StartTime,
|
||||
End: storagePlan.EndTime,
|
||||
}
|
||||
sg := gen.NewSeriesGeneratorFromSpec(spec, tr)
|
||||
|
||||
stop := flags.profile.Start()
|
||||
defer stop()
|
||||
|
||||
var files []string
|
||||
start := time.Now().UTC()
|
||||
defer func() {
|
||||
elapsed := time.Since(start)
|
||||
fmt.Println()
|
||||
fmt.Println("Generated:")
|
||||
for _, f := range files {
|
||||
fmt.Println(f)
|
||||
}
|
||||
fmt.Println()
|
||||
fmt.Printf("Total time: %0.1f seconds\n", elapsed.Seconds())
|
||||
}()
|
||||
|
||||
path, err := fs.InfluxDir()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
g := &Generator{Clean: storagePlan.Clean}
|
||||
files, err = g.Run(context.Background(), path, sg)
|
||||
return err
|
||||
}
|
|
@ -1,187 +0,0 @@
|
|||
package generate
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
|
||||
"github.com/spf13/cobra"
|
||||
)
|
||||
|
||||
var helpSchemaCommand = &cobra.Command{
|
||||
Use: "help-schema",
|
||||
Short: "Print a documented TOML schema to STDOUT",
|
||||
Run: func(cmd *cobra.Command, args []string) {
|
||||
fmt.Print(documentedSchema)
|
||||
},
|
||||
}
|
||||
|
||||
func init() {
|
||||
Command.AddCommand(helpSchemaCommand)
|
||||
}
|
||||
|
||||
const documentedSchema = `title = "Documented schema"
|
||||
|
||||
# limit the maximum number of series generated across all measurements
|
||||
#
|
||||
# series-limit: integer, optional (default: unlimited)
|
||||
|
||||
[[measurements]]
|
||||
|
||||
# name of measurement
|
||||
#
|
||||
# NOTE:
|
||||
# Multiple definitions of the same measurement name are allowed and
|
||||
# will be merged together.
|
||||
name = "cpu"
|
||||
|
||||
# sample: float; where 0 < sample ≤ 1.0 (default: 0.5)
|
||||
# sample a subset of the tag set
|
||||
#
|
||||
# sample 25% of the tags
|
||||
#
|
||||
sample = 0.25
|
||||
|
||||
# Keys for defining a tag
|
||||
#
|
||||
# name: string, required
|
||||
# Name of field
|
||||
#
|
||||
# source: array<string> or object
|
||||
#
|
||||
# A literal array of string values defines the tag values.
|
||||
#
|
||||
# An object defines more complex generators. The type key determines the
|
||||
# type of generator.
|
||||
#
|
||||
# source types:
|
||||
#
|
||||
# type: "sequence"
|
||||
# generate a sequence of tag values
|
||||
#
|
||||
# format: string
|
||||
# a format string for the values (default: "value%s")
|
||||
# start: int (default: 0)
|
||||
# beginning value
|
||||
# count: int, required
|
||||
# ending value
|
||||
#
|
||||
# type: "file"
|
||||
# generate a sequence of tag values from a file source.
|
||||
# The data in the file is sorted, deduplicated and verified is valid UTF-8
|
||||
#
|
||||
# path: string
|
||||
# absolute path or relative path to current toml file
|
||||
tags = [
|
||||
# example sequence tag source. The range of values are automatically
|
||||
# prefixed with 0s
|
||||
# to ensure correct sort behavior.
|
||||
{ name = "host", source = { type = "sequence", format = "host-%s", start = 0, count = 5 } },
|
||||
|
||||
# tags can also be sourced from a file. The path is relative to the
|
||||
# schema.toml.
|
||||
# Each value must be on a new line. The file is also sorted, deduplicated
|
||||
# and UTF-8 validated.
|
||||
{ name = "rack", source = { type = "file", path = "files/racks.txt" } },
|
||||
|
||||
# Example string array source, which is also deduplicated and sorted
|
||||
{ name = "region", source = ["us-west-01","us-west-02","us-east"] },
|
||||
]
|
||||
|
||||
# Keys for defining a field
|
||||
#
|
||||
# name: string, required
|
||||
# Name of field
|
||||
#
|
||||
# count: int, required
|
||||
# The maximum number of values to generate. When multiple fields
|
||||
# have the same count and time-spec, they will share timestamps.
|
||||
#
|
||||
# A time-spec can be either time-precision or time-interval, which
|
||||
# determines how timestamps are generated and may also influence
|
||||
# the time range and number of values generated.
|
||||
#
|
||||
# time-precision: string [ns, us, ms, s, m, h] (default: ms)
|
||||
# Specifies the precision (rounding) for generated timestamps.
|
||||
#
|
||||
# If the precision results in fewer than "count" intervals for the
|
||||
# given time range the number of values will be reduced.
|
||||
#
|
||||
# Example:
|
||||
# count = 1000, start = 0s, end = 100s, time-precison = s
|
||||
# 100 values will be generated at [0s, 1s, 2s, ..., 99s]
|
||||
#
|
||||
# If the precision results in greater than "count" intervals for the
|
||||
# given time range, the interval will be rounded to the nearest multiple of
|
||||
# time-precision.
|
||||
#
|
||||
# Example:
|
||||
# count = 10, start = 0s, end = 100s, time-precison = s
|
||||
# 100 values will be generated at [0s, 10s, 20s, ..., 90s]
|
||||
#
|
||||
# time-interval: Go duration string (eg 90s, 1h30m)
|
||||
# Specifies the delta between generated timestamps.
|
||||
#
|
||||
# If the delta results in fewer than "count" intervals for the
|
||||
# given time range the number of values will be reduced.
|
||||
#
|
||||
# Example:
|
||||
# count = 100, start = 0s, end = 100s, time-interval = 10s
|
||||
# 10 values will be generated at [0s, 10s, 20s, ..., 90s]
|
||||
#
|
||||
# If the delta results in greater than "count" intervals for the
|
||||
# given time range, the start-time will be adjusted to ensure "count" values.
|
||||
#
|
||||
# Example:
|
||||
# count = 20, start = 0s, end = 1000s, time-interval = 10s
|
||||
# 20 values will be generated at [800s, 810s, ..., 900s, ..., 990s]
|
||||
#
|
||||
# source: int, float, boolean, string, array or object
|
||||
#
|
||||
# A literal int, float, boolean or string will produce
|
||||
# a constant value of the same data type.
|
||||
#
|
||||
# A literal array of homogeneous values will generate a repeating
|
||||
# sequence.
|
||||
#
|
||||
# An object defines more complex generators. The type key determines the
|
||||
# type of generator.
|
||||
#
|
||||
# source types:
|
||||
#
|
||||
# type: "rand<float>"
|
||||
# generate random float values
|
||||
# seed: seed to random number generator (default: 0)
|
||||
# min: minimum value (default: 0.0)
|
||||
# max: maximum value (default: 1.0)
|
||||
#
|
||||
# type: "zipf<integer>"
|
||||
# generate random integer values using a Zipf distribution
|
||||
# The generator generates values k ∈ [0, imax] such that P(k)
|
||||
# is proportional to (v + k) ** (-s). Requirements: s > 1 and v ≥ 1.
|
||||
# See https://golang.org/pkg/math/rand/#NewZipf for more information.
|
||||
#
|
||||
# seed: seed to random number generator (default: 0)
|
||||
# s: float > 1 (required)
|
||||
# v: float ≥ 1 (required)
|
||||
# imax: integer (required)
|
||||
#
|
||||
fields = [
|
||||
# Example constant float
|
||||
{ name = "system", count = 5000, source = 2.5 },
|
||||
|
||||
# Example random floats
|
||||
{ name = "user", count = 5000, source = { type = "rand<float>", seed = 10, min = 0.0, max = 1.0 } },
|
||||
]
|
||||
|
||||
# Multiple measurements may be defined.
|
||||
[[measurements]]
|
||||
name = "mem"
|
||||
tags = [
|
||||
{ name = "host", source = { type = "sequence", format = "host-%s", start = 0, count = 5 } },
|
||||
{ name = "region", source = ["us-west-01","us-west-02","us-east"] },
|
||||
]
|
||||
fields = [
|
||||
# An example of a sequence of integer values
|
||||
{ name = "free", count = 100, source = [10,15,20,25,30,35,30], time-precision = "ms" },
|
||||
{ name = "low_mem", count = 100, source = [false,true,true], time-precision = "ms" },
|
||||
]
|
||||
`
|
|
@ -1,85 +0,0 @@
|
|||
package generate
|
||||
|
||||
import (
|
||||
"os"
|
||||
"strings"
|
||||
"text/template"
|
||||
|
||||
"github.com/influxdata/influxdb/v2/pkg/data/gen"
|
||||
"github.com/spf13/cobra"
|
||||
)
|
||||
|
||||
var simpleCommand = &cobra.Command{
|
||||
Use: "simple",
|
||||
Short: "Generate simple data sets using only CLI flags",
|
||||
RunE: simpleGenerateFE,
|
||||
}
|
||||
|
||||
var simpleFlags struct {
|
||||
schemaSpec SchemaSpec
|
||||
}
|
||||
|
||||
func init() {
|
||||
simpleCommand.PersistentFlags().SortFlags = false
|
||||
simpleCommand.Flags().SortFlags = false
|
||||
simpleFlags.schemaSpec.AddFlags(simpleCommand, simpleCommand.Flags())
|
||||
|
||||
Command.AddCommand(simpleCommand)
|
||||
}
|
||||
|
||||
func simpleGenerateFE(_ *cobra.Command, _ []string) error {
|
||||
storagePlan, err := flags.storageSpec.Plan()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
storagePlan.PrintPlan(os.Stdout)
|
||||
|
||||
schemaPlan, err := simpleFlags.schemaSpec.Plan(storagePlan)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
schemaPlan.PrintPlan(os.Stdout)
|
||||
spec := planToSpec(schemaPlan)
|
||||
|
||||
if err = assignOrgBucket(spec); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
if flags.printOnly {
|
||||
return nil
|
||||
}
|
||||
|
||||
return exec(storagePlan, spec)
|
||||
}
|
||||
|
||||
var (
|
||||
tomlSchema = template.Must(template.New("schema").Parse(`
|
||||
title = "CLI schema"
|
||||
|
||||
[[measurements]]
|
||||
name = "m0"
|
||||
sample = 1.0
|
||||
tags = [
|
||||
{{- range $i, $e := .Tags }}
|
||||
{ name = "tag{{$i}}", source = { type = "sequence", format = "value%s", start = 0, count = {{$e}} } },{{ end }}
|
||||
]
|
||||
fields = [
|
||||
{ name = "v0", count = {{ .PointsPerSeries }}, source = 1.0 },
|
||||
]`))
|
||||
)
|
||||
|
||||
func planToSpec(p *SchemaPlan) *gen.Spec {
|
||||
var sb strings.Builder
|
||||
if err := tomlSchema.Execute(&sb, p); err != nil {
|
||||
panic(err)
|
||||
}
|
||||
|
||||
spec, err := gen.NewSpecFromToml(sb.String())
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
|
||||
return spec
|
||||
}
|
|
@ -1,161 +0,0 @@
|
|||
package generate
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"os"
|
||||
"path/filepath"
|
||||
"runtime"
|
||||
"sync"
|
||||
|
||||
"github.com/influxdata/influxdb/v2/cmd/influxd/generate/internal/shard"
|
||||
"github.com/influxdata/influxdb/v2/kit/errors"
|
||||
"github.com/influxdata/influxdb/v2/models"
|
||||
"github.com/influxdata/influxdb/v2/pkg/data/gen"
|
||||
"github.com/influxdata/influxdb/v2/pkg/limiter"
|
||||
"github.com/influxdata/influxdb/v2/storage"
|
||||
"github.com/influxdata/influxdb/v2/v1/tsdb"
|
||||
"github.com/influxdata/influxdb/v2/v1/tsdb/index/tsi1"
|
||||
"github.com/influxdata/influxdb/v2/v1/tsdb/engine/tsm1"
|
||||
)
|
||||
|
||||
type Generator struct {
|
||||
sfile *seriesfile.SeriesFile
|
||||
|
||||
// Clean specifies whether to clean any of the data related files
|
||||
Clean CleanLevel
|
||||
}
|
||||
|
||||
func (g *Generator) Run(ctx context.Context, path string, gen gen.SeriesGenerator) ([]string, error) {
|
||||
path = filepath.Join(path, "engine")
|
||||
config := storage.NewConfig()
|
||||
|
||||
switch g.Clean {
|
||||
case CleanLevelTSM:
|
||||
if err := os.RemoveAll(path); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
case CleanLevelAll:
|
||||
if err := os.RemoveAll(path); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
}
|
||||
|
||||
g.sfile = seriesfile.NewSeriesFile(config.GetSeriesFilePath(path))
|
||||
if err := g.sfile.Open(ctx); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
defer g.sfile.Close()
|
||||
g.sfile.DisableCompactions()
|
||||
|
||||
ti := tsi1.NewIndex(g.sfile, config.Index, tsi1.WithPath(config.GetIndexPath(path)))
|
||||
if err := ti.Open(ctx); err != nil {
|
||||
return nil, fmt.Errorf("error opening TSI1 index: %s", err.Error())
|
||||
}
|
||||
|
||||
files, err := g.writeShard(ti, gen, config.GetEnginePath(path))
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("error writing data: %s", err.Error())
|
||||
}
|
||||
|
||||
ti.Compact()
|
||||
ti.Wait()
|
||||
if err := ti.Close(); err != nil {
|
||||
return nil, fmt.Errorf("error compacting TSI1 index: %s", err.Error())
|
||||
}
|
||||
|
||||
var (
|
||||
wg sync.WaitGroup
|
||||
errs errors.List
|
||||
)
|
||||
|
||||
parts := g.sfile.Partitions()
|
||||
wg.Add(len(parts))
|
||||
ch := make(chan error, len(parts))
|
||||
limit := limiter.NewFixed(runtime.NumCPU())
|
||||
|
||||
for i := range parts {
|
||||
go func(n int) {
|
||||
limit.Take()
|
||||
defer func() {
|
||||
wg.Done()
|
||||
limit.Release()
|
||||
}()
|
||||
|
||||
p := parts[n]
|
||||
c := seriesfile.NewSeriesPartitionCompactor()
|
||||
if _, err := c.Compact(p); err != nil {
|
||||
ch <- fmt.Errorf("error compacting series partition %d: %s", n, err.Error())
|
||||
}
|
||||
}(i)
|
||||
}
|
||||
wg.Wait()
|
||||
|
||||
close(ch)
|
||||
for e := range ch {
|
||||
errs.Append(e)
|
||||
}
|
||||
|
||||
if err := errs.Err(); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return files, nil
|
||||
}
|
||||
|
||||
// seriesBatchSize specifies the number of series keys passed to the index.
|
||||
const seriesBatchSize = 1000
|
||||
|
||||
func (g *Generator) writeShard(idx *tsi1.Index, sg gen.SeriesGenerator, path string) ([]string, error) {
|
||||
if err := os.MkdirAll(path, 0777); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
sw, err := shard.NewWriter(path, shard.AutoNumber())
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
defer sw.Close()
|
||||
|
||||
coll := &tsdb.SeriesCollection{
|
||||
Keys: make([][]byte, 0, seriesBatchSize),
|
||||
Names: make([][]byte, 0, seriesBatchSize),
|
||||
Tags: make([]models.Tags, 0, seriesBatchSize),
|
||||
Types: make([]models.FieldType, 0, seriesBatchSize),
|
||||
}
|
||||
|
||||
for sg.Next() {
|
||||
seriesKey := sg.Key()
|
||||
coll.Keys = append(coll.Keys, seriesKey)
|
||||
coll.Names = append(coll.Names, sg.ID())
|
||||
coll.Tags = append(coll.Tags, sg.Tags())
|
||||
coll.Types = append(coll.Types, sg.FieldType())
|
||||
|
||||
if coll.Length() == seriesBatchSize {
|
||||
if err := idx.CreateSeriesListIfNotExists(coll); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
coll.Truncate(0)
|
||||
}
|
||||
|
||||
vg := sg.TimeValuesGenerator()
|
||||
|
||||
key := tsm1.SeriesFieldKeyBytes(string(seriesKey), string(sg.Field()))
|
||||
for vg.Next() {
|
||||
sw.WriteV(key, vg.Values())
|
||||
}
|
||||
|
||||
if err := sw.Err(); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
}
|
||||
|
||||
if coll.Length() > 0 {
|
||||
if err := idx.CreateSeriesListIfNotExists(coll); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
}
|
||||
|
||||
return sw.Files(), nil
|
||||
}
|
|
@ -1,191 +0,0 @@
|
|||
package shard
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"os"
|
||||
"path/filepath"
|
||||
|
||||
"github.com/influxdata/influxdb/v2/pkg/data/gen"
|
||||
"github.com/influxdata/influxdb/v2/v1/tsdb/engine/tsm1"
|
||||
)
|
||||
|
||||
const (
|
||||
maxTSMFileSize = uint32(2048 * 1024 * 1024) // 2GB
|
||||
)
|
||||
|
||||
type Writer struct {
|
||||
tw tsm1.TSMWriter
|
||||
path string
|
||||
ext string
|
||||
files []string
|
||||
gen, seq int
|
||||
err error
|
||||
buf []byte
|
||||
auto bool
|
||||
}
|
||||
|
||||
type option func(w *Writer)
|
||||
|
||||
// Generation specifies the generation number of the tsm files.
|
||||
func Generation(gen int) option {
|
||||
return func(w *Writer) {
|
||||
w.gen = gen
|
||||
}
|
||||
}
|
||||
|
||||
// Sequence specifies the starting sequence number of the tsm files.
|
||||
func Sequence(seq int) option {
|
||||
return func(w *Writer) {
|
||||
w.seq = seq
|
||||
}
|
||||
}
|
||||
|
||||
// Temporary configures the writer to create tsm.tmp files.
|
||||
func Temporary() option {
|
||||
return func(w *Writer) {
|
||||
w.ext = tsm1.TSMFileExtension + "." + tsm1.TmpTSMFileExtension
|
||||
}
|
||||
}
|
||||
|
||||
// AutoNumber will read the existing TSM file names and use generation + 1
|
||||
func AutoNumber() option {
|
||||
return func(w *Writer) {
|
||||
w.auto = true
|
||||
}
|
||||
}
|
||||
|
||||
func NewWriter(path string, opts ...option) (*Writer, error) {
|
||||
w := &Writer{path: path, gen: 1, seq: 1, ext: tsm1.TSMFileExtension}
|
||||
|
||||
for _, opt := range opts {
|
||||
opt(w)
|
||||
}
|
||||
|
||||
if w.auto {
|
||||
err := w.readExisting()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
}
|
||||
|
||||
w.nextTSM()
|
||||
if w.err != nil {
|
||||
return nil, w.err
|
||||
}
|
||||
|
||||
return w, nil
|
||||
}
|
||||
|
||||
func (w *Writer) Write(key []byte, values tsm1.Values) {
|
||||
if w.err != nil {
|
||||
return
|
||||
}
|
||||
|
||||
if w.tw.Size() > maxTSMFileSize {
|
||||
w.closeTSM()
|
||||
w.nextTSM()
|
||||
}
|
||||
|
||||
if err := w.tw.Write(key, values); err != nil {
|
||||
if err == tsm1.ErrMaxBlocksExceeded {
|
||||
w.closeTSM()
|
||||
w.nextTSM()
|
||||
} else {
|
||||
w.err = err
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (w *Writer) WriteV(key []byte, values gen.Values) {
|
||||
if w.err != nil {
|
||||
return
|
||||
}
|
||||
|
||||
if w.tw.Size() > maxTSMFileSize {
|
||||
w.closeTSM()
|
||||
w.nextTSM()
|
||||
}
|
||||
|
||||
minT, maxT := values.MinTime(), values.MaxTime()
|
||||
var err error
|
||||
if w.buf, err = values.Encode(w.buf); err != nil {
|
||||
w.err = err
|
||||
return
|
||||
}
|
||||
|
||||
if err := w.tw.WriteBlock(key, minT, maxT, w.buf); err != nil {
|
||||
if err == tsm1.ErrMaxBlocksExceeded {
|
||||
w.closeTSM()
|
||||
w.nextTSM()
|
||||
} else {
|
||||
w.err = err
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Close closes the writer.
|
||||
func (w *Writer) Close() {
|
||||
if w.tw != nil {
|
||||
w.closeTSM()
|
||||
}
|
||||
}
|
||||
|
||||
func (w *Writer) Err() error { return w.err }
|
||||
|
||||
// Files returns the full paths of all the files written by the Writer.
|
||||
func (w *Writer) Files() []string { return w.files }
|
||||
|
||||
func (w *Writer) nextTSM() {
|
||||
fileName := filepath.Join(w.path, fmt.Sprintf("%s.%s", tsm1.DefaultFormatFileName(w.gen, w.seq), w.ext))
|
||||
w.files = append(w.files, fileName)
|
||||
w.seq++
|
||||
|
||||
if fi, _ := os.Stat(fileName); fi != nil {
|
||||
w.err = fmt.Errorf("attempted to overwrite an existing TSM file: %q", fileName)
|
||||
return
|
||||
}
|
||||
|
||||
fd, err := os.OpenFile(fileName, os.O_CREATE|os.O_RDWR, 0666)
|
||||
if err != nil {
|
||||
w.err = err
|
||||
return
|
||||
}
|
||||
|
||||
// Create the writer for the new TSM file.
|
||||
w.tw, err = tsm1.NewTSMWriter(fd)
|
||||
if err != nil {
|
||||
w.err = err
|
||||
return
|
||||
}
|
||||
}
|
||||
|
||||
func (w *Writer) closeTSM() {
|
||||
if err := w.tw.WriteIndex(); err != nil && err != tsm1.ErrNoValues {
|
||||
w.err = err
|
||||
}
|
||||
|
||||
if err := w.tw.Close(); err != nil && w.err == nil {
|
||||
w.err = err
|
||||
}
|
||||
|
||||
w.tw = nil
|
||||
}
|
||||
|
||||
func (w *Writer) readExisting() error {
|
||||
files, err := filepath.Glob(filepath.Join(w.path, fmt.Sprintf("*.%s", tsm1.TSMFileExtension)))
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
for _, f := range files {
|
||||
generation, _, err := tsm1.DefaultParseFileName(f)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
if generation >= w.gen {
|
||||
w.gen = generation + 1
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
|
@ -1,60 +0,0 @@
|
|||
package generate
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"io"
|
||||
"strings"
|
||||
"text/tabwriter"
|
||||
"time"
|
||||
)
|
||||
|
||||
type StoragePlan struct {
|
||||
Organization string
|
||||
Bucket string
|
||||
StartTime time.Time
|
||||
EndTime time.Time
|
||||
Clean CleanLevel
|
||||
Path string
|
||||
}
|
||||
|
||||
func (p *StoragePlan) String() string {
|
||||
sb := new(strings.Builder)
|
||||
p.PrintPlan(sb)
|
||||
return sb.String()
|
||||
}
|
||||
|
||||
func (p *StoragePlan) PrintPlan(w io.Writer) {
|
||||
tw := tabwriter.NewWriter(w, 25, 4, 2, ' ', 0)
|
||||
fmt.Fprintf(tw, "Organization\t%s\n", p.Organization)
|
||||
fmt.Fprintf(tw, "Bucket\t%s\n", p.Bucket)
|
||||
fmt.Fprintf(tw, "Start time\t%s\n", p.StartTime)
|
||||
fmt.Fprintf(tw, "End time\t%s\n", p.EndTime)
|
||||
fmt.Fprintf(tw, "Clean data\t%s\n", p.Clean)
|
||||
tw.Flush()
|
||||
}
|
||||
|
||||
// TimeSpan returns the total duration for which the data set.
|
||||
func (p *StoragePlan) TimeSpan() time.Duration {
|
||||
return p.EndTime.Sub(p.StartTime)
|
||||
}
|
||||
|
||||
type SchemaPlan struct {
|
||||
StoragePlan *StoragePlan
|
||||
Tags TagCardinalities
|
||||
PointsPerSeries int
|
||||
}
|
||||
|
||||
func (p *SchemaPlan) String() string {
|
||||
sb := new(strings.Builder)
|
||||
p.PrintPlan(sb)
|
||||
return sb.String()
|
||||
}
|
||||
|
||||
func (p *SchemaPlan) PrintPlan(w io.Writer) {
|
||||
tw := tabwriter.NewWriter(w, 25, 4, 2, ' ', 0)
|
||||
fmt.Fprintf(tw, "Tag cardinalities\t%s\n", p.Tags)
|
||||
fmt.Fprintf(tw, "Points per series\t%d\n", p.PointsPerSeries)
|
||||
fmt.Fprintf(tw, "Total points\t%d\n", p.Tags.Cardinality()*p.PointsPerSeries)
|
||||
fmt.Fprintf(tw, "Total series\t%d\n", p.Tags.Cardinality())
|
||||
_ = tw.Flush()
|
||||
}
|
|
@ -1,153 +0,0 @@
|
|||
package generate
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"strconv"
|
||||
"strings"
|
||||
"time"
|
||||
|
||||
"github.com/spf13/cobra"
|
||||
"github.com/spf13/pflag"
|
||||
)
|
||||
|
||||
type CleanLevel int
|
||||
|
||||
const (
|
||||
// CleanLevelNone will not remove any data files.
|
||||
CleanLevelNone CleanLevel = iota
|
||||
|
||||
// CleanLevelTSM will only remove TSM data files.
|
||||
CleanLevelTSM
|
||||
|
||||
// CleanLevelAll will remove all TSM and index data files.
|
||||
CleanLevelAll
|
||||
)
|
||||
|
||||
func (i CleanLevel) String() string {
|
||||
switch i {
|
||||
case CleanLevelNone:
|
||||
return "none"
|
||||
case CleanLevelTSM:
|
||||
return "tsm"
|
||||
case CleanLevelAll:
|
||||
return "all"
|
||||
default:
|
||||
return strconv.FormatInt(int64(i), 10)
|
||||
}
|
||||
}
|
||||
|
||||
func (i *CleanLevel) Set(v string) error {
|
||||
switch v {
|
||||
case "none":
|
||||
*i = CleanLevelNone
|
||||
case "tsm":
|
||||
*i = CleanLevelTSM
|
||||
case "all":
|
||||
*i = CleanLevelAll
|
||||
default:
|
||||
return fmt.Errorf("invalid clean level %q, must be none, tsm or all", v)
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (i CleanLevel) Type() string {
|
||||
return "clean-level"
|
||||
}
|
||||
|
||||
type StorageSpec struct {
|
||||
Organization string
|
||||
Bucket string
|
||||
StartTime string
|
||||
EndTime string
|
||||
Clean CleanLevel
|
||||
}
|
||||
|
||||
func (a *StorageSpec) AddFlags(cmd *cobra.Command, fs *pflag.FlagSet) {
|
||||
fs.StringVar(&a.Organization, "org", "", "Name of organization")
|
||||
cmd.MarkFlagRequired("org")
|
||||
fs.StringVar(&a.Bucket, "bucket", "", "Name of bucket")
|
||||
cmd.MarkFlagRequired("bucket")
|
||||
start := time.Now().UTC().AddDate(0, 0, -7).Truncate(24 * time.Hour)
|
||||
fs.StringVar(&a.StartTime, "start-time", start.Format(time.RFC3339), "Start time")
|
||||
fs.StringVar(&a.EndTime, "end-time", start.AddDate(0, 0, 7).Format(time.RFC3339), "End time")
|
||||
fs.Var(&a.Clean, "clean", "Clean time series data files (none, tsm or all)")
|
||||
}
|
||||
|
||||
func (a *StorageSpec) Plan() (*StoragePlan, error) {
|
||||
plan := &StoragePlan{
|
||||
Organization: a.Organization,
|
||||
Bucket: a.Bucket,
|
||||
Clean: a.Clean,
|
||||
}
|
||||
|
||||
if a.StartTime != "" {
|
||||
if t, err := time.Parse(time.RFC3339, a.StartTime); err != nil {
|
||||
return nil, err
|
||||
} else {
|
||||
plan.StartTime = t.UTC()
|
||||
}
|
||||
}
|
||||
|
||||
if a.EndTime != "" {
|
||||
if t, err := time.Parse(time.RFC3339, a.EndTime); err != nil {
|
||||
return nil, err
|
||||
} else {
|
||||
plan.EndTime = t.UTC()
|
||||
}
|
||||
}
|
||||
|
||||
return plan, nil
|
||||
}
|
||||
|
||||
type TagCardinalities []int
|
||||
|
||||
func (t TagCardinalities) String() string {
|
||||
s := make([]string, 0, len(t))
|
||||
for i := 0; i < len(t); i++ {
|
||||
s = append(s, strconv.Itoa(t[i]))
|
||||
}
|
||||
return fmt.Sprintf("[%s]", strings.Join(s, ","))
|
||||
}
|
||||
|
||||
func (t TagCardinalities) Cardinality() int {
|
||||
n := 1
|
||||
for i := range t {
|
||||
n *= t[i]
|
||||
}
|
||||
return n
|
||||
}
|
||||
|
||||
func (t *TagCardinalities) Set(tags string) error {
|
||||
*t = (*t)[:0]
|
||||
for _, s := range strings.Split(tags, ",") {
|
||||
v, err := strconv.Atoi(s)
|
||||
if err != nil {
|
||||
return fmt.Errorf("cannot parse tag cardinality: %s", s)
|
||||
}
|
||||
*t = append(*t, v)
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (t *TagCardinalities) Type() string {
|
||||
return "tags"
|
||||
}
|
||||
|
||||
type SchemaSpec struct {
|
||||
Tags TagCardinalities
|
||||
PointsPerSeries int
|
||||
}
|
||||
|
||||
func (s *SchemaSpec) AddFlags(cmd *cobra.Command, fs *pflag.FlagSet) {
|
||||
s.Tags = []int{10, 10, 10}
|
||||
fs.Var(&s.Tags, "t", "Tag cardinality")
|
||||
fs.IntVar(&s.PointsPerSeries, "p", 100, "Points per series")
|
||||
}
|
||||
|
||||
func (s *SchemaSpec) Plan(sp *StoragePlan) (*SchemaPlan, error) {
|
||||
return &SchemaPlan{
|
||||
StoragePlan: sp,
|
||||
Tags: s.Tags,
|
||||
PointsPerSeries: s.PointsPerSeries,
|
||||
}, nil
|
||||
}
|
|
@ -1,135 +0,0 @@
|
|||
package inspect
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"io"
|
||||
"os"
|
||||
"os/user"
|
||||
"path/filepath"
|
||||
"runtime"
|
||||
"strings"
|
||||
|
||||
"github.com/influxdata/influxdb/v2/cmd/influx_inspect/buildtsi"
|
||||
"github.com/influxdata/influxdb/v2/logger"
|
||||
"github.com/influxdata/influxdb/v2/storage"
|
||||
"github.com/influxdata/influxdb/v2/v1/tsdb"
|
||||
"github.com/influxdata/influxdb/v2/v1/tsdb/index/tsi1"
|
||||
"github.com/influxdata/influxdb/v2/v1/tsdb/engine/tsm1"
|
||||
"github.com/spf13/cobra"
|
||||
)
|
||||
|
||||
const defaultBatchSize = 10000
|
||||
|
||||
var buildTSIFlags = struct {
|
||||
// Standard input/output, overridden for testing.
|
||||
Stderr io.Writer
|
||||
Stdout io.Writer
|
||||
|
||||
// Data path options
|
||||
DataPath string // optional. Defaults to <engine_path>/engine/data
|
||||
WALPath string // optional. Defaults to <engine_path>/engine/wal
|
||||
SeriesFilePath string // optional. Defaults to <engine_path>/engine/_series
|
||||
IndexPath string // optional. Defaults to <engine_path>/engine/index
|
||||
|
||||
BatchSize int // optional. Defaults to 10000
|
||||
MaxLogFileSize int64 // optional. Defaults to tsi1.DefaultMaxIndexLogFileSize
|
||||
MaxCacheSize uint64 // optional. Defaults to tsm1.DefaultCacheMaxMemorySize
|
||||
|
||||
Concurrency int // optional. Defaults to GOMAXPROCS(0)
|
||||
Verbose bool // optional. Defaults to false.
|
||||
}{
|
||||
Stderr: os.Stderr,
|
||||
Stdout: os.Stdout,
|
||||
}
|
||||
|
||||
// NewBuildTSICommand returns a new instance of Command with default setting applied.
|
||||
func NewBuildTSICommand() *cobra.Command {
|
||||
cmd := &cobra.Command{
|
||||
Use: "build-tsi",
|
||||
Short: "Rebuilds the TSI index and (where necessary) the Series File.",
|
||||
Long: `This command will rebuild the TSI index and if needed the Series
|
||||
File.
|
||||
|
||||
The index is built by reading all of the TSM indexes in the TSM data
|
||||
directory, and all of the WAL entries in the WAL data directory. If the
|
||||
Series File directory is missing, then the series file will be rebuilt.
|
||||
|
||||
If the TSI index directory already exists, then this tool will fail.
|
||||
|
||||
Performance of the tool can be tweaked by adjusting the max log file size,
|
||||
max cache file size and the batch size.
|
||||
|
||||
max-log-file-size determines how big in-memory parts of the index have to
|
||||
get before they're compacted into memory-mappable index files.
|
||||
Consider decreasing this from the default if you find the heap
|
||||
requirements of your TSI index are too much.
|
||||
|
||||
max-cache-size refers to the maximum cache size allowed. If there are WAL
|
||||
files to index, then they need to be replayed into a tsm1.Cache first
|
||||
by this tool. If the maximum cache size isn't large enough then there
|
||||
will be an error and this tool will fail. Increase max-cache-size to
|
||||
address this.
|
||||
|
||||
batch-size refers to the size of the batches written into the index.
|
||||
Increasing this can improve performance but can result in much more
|
||||
memory usage.
|
||||
`,
|
||||
RunE: RunBuildTSI,
|
||||
}
|
||||
|
||||
defaultPath := filepath.Join(os.Getenv("HOME"), "/.influxdbv2/engine/")
|
||||
defaultDataPath := filepath.Join(defaultPath, storage.DefaultEngineDirectoryName)
|
||||
defaultWALPath := filepath.Join(defaultPath, storage.DefaultWALDirectoryName)
|
||||
defaultSFilePath := filepath.Join(defaultPath, storage.DefaultSeriesFileDirectoryName)
|
||||
defaultIndexPath := filepath.Join(defaultPath, storage.DefaultIndexDirectoryName)
|
||||
|
||||
cmd.Flags().StringVar(&buildTSIFlags.DataPath, "tsm-path", defaultDataPath, "Path to the TSM data directory. Defaults to "+defaultDataPath)
|
||||
cmd.Flags().StringVar(&buildTSIFlags.WALPath, "wal-path", defaultWALPath, "Path to the WAL data directory. Defaults to "+defaultWALPath)
|
||||
cmd.Flags().StringVar(&buildTSIFlags.SeriesFilePath, "sfile-path", defaultSFilePath, "Path to the Series File directory. Defaults to "+defaultSFilePath)
|
||||
cmd.Flags().StringVar(&buildTSIFlags.IndexPath, "tsi-path", defaultIndexPath, "Path to the TSI index directory. Defaults to "+defaultIndexPath)
|
||||
|
||||
cmd.Flags().IntVar(&buildTSIFlags.Concurrency, "concurrency", runtime.GOMAXPROCS(0), "Number of workers to dedicate to shard index building. Defaults to GOMAXPROCS")
|
||||
cmd.Flags().Int64Var(&buildTSIFlags.MaxLogFileSize, "max-log-file-size", tsi1.DefaultMaxIndexLogFileSize, "optional: maximum log file size")
|
||||
cmd.Flags().Uint64Var(&buildTSIFlags.MaxCacheSize, "max-cache-size", uint64(tsm1.DefaultCacheMaxMemorySize), "optional: maximum cache size")
|
||||
cmd.Flags().IntVar(&buildTSIFlags.BatchSize, "batch-size", defaultBatchSize, "optional: set the size of the batches we write to the index. Setting this can have adverse affects on performance and heap requirements")
|
||||
cmd.Flags().BoolVar(&buildTSIFlags.Verbose, "v", false, "verbose")
|
||||
|
||||
cmd.SetOutput(buildTSIFlags.Stdout)
|
||||
|
||||
return cmd
|
||||
}
|
||||
|
||||
// RunBuildTSI executes the run command for BuildTSI.
|
||||
func RunBuildTSI(cmd *cobra.Command, args []string) error {
|
||||
// Verify the user actually wants to run as root.
|
||||
if isRoot() {
|
||||
fmt.Fprintln(buildTSIFlags.Stdout, "You are currently running as root. This will build your")
|
||||
fmt.Fprintln(buildTSIFlags.Stdout, "index files with root ownership and will be inaccessible")
|
||||
fmt.Fprintln(buildTSIFlags.Stdout, "if you run influxd as a non-root user. You should run")
|
||||
fmt.Fprintln(buildTSIFlags.Stdout, "influxd inspect buildtsi as the same user you are running influxd.")
|
||||
fmt.Fprint(buildTSIFlags.Stdout, "Are you sure you want to continue? (y/N): ")
|
||||
var answer string
|
||||
if fmt.Scanln(&answer); !strings.HasPrefix(strings.TrimSpace(strings.ToLower(answer)), "y") {
|
||||
return fmt.Errorf("operation aborted")
|
||||
}
|
||||
}
|
||||
|
||||
log := logger.New(buildTSIFlags.Stdout)
|
||||
|
||||
sfile := seriesfile.NewSeriesFile(buildTSIFlags.SeriesFilePath)
|
||||
sfile.Logger = log
|
||||
if err := sfile.Open(context.Background()); err != nil {
|
||||
return err
|
||||
}
|
||||
defer sfile.Close()
|
||||
|
||||
return buildtsi.IndexShard(sfile, buildTSIFlags.IndexPath, buildTSIFlags.DataPath, buildTSIFlags.WALPath,
|
||||
buildTSIFlags.MaxLogFileSize, buildTSIFlags.MaxCacheSize, buildTSIFlags.BatchSize,
|
||||
log, buildTSIFlags.Verbose)
|
||||
}
|
||||
|
||||
func isRoot() bool {
|
||||
user, _ := user.Current()
|
||||
return user != nil && user.Username == "root"
|
||||
}
|
|
@ -1,191 +0,0 @@
|
|||
package inspect
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"io"
|
||||
"os"
|
||||
"path/filepath"
|
||||
"runtime"
|
||||
"strconv"
|
||||
"strings"
|
||||
|
||||
"github.com/influxdata/influxdb/v2/internal/fs"
|
||||
"github.com/influxdata/influxdb/v2/storage"
|
||||
"github.com/spf13/cobra"
|
||||
"golang.org/x/sync/errgroup"
|
||||
)
|
||||
|
||||
var compactSeriesFileFlags = struct {
|
||||
// Standard input/output, overridden for testing.
|
||||
Stderr io.Writer
|
||||
Stdout io.Writer
|
||||
|
||||
// Data path options
|
||||
SeriesFilePath string // optional. Defaults to <engine_path>/engine/_series
|
||||
IndexPath string // optional. Defaults to <engine_path>/engine/index
|
||||
|
||||
Concurrency int // optional. Defaults to GOMAXPROCS(0)
|
||||
}{
|
||||
Stderr: os.Stderr,
|
||||
Stdout: os.Stdout,
|
||||
}
|
||||
|
||||
// NewCompactSeriesFileCommand returns a new instance of Command with default setting applied.
|
||||
func NewCompactSeriesFileCommand() *cobra.Command {
|
||||
cmd := &cobra.Command{
|
||||
Use: "compact-series-file",
|
||||
Short: "Compacts the series file to removed deleted series.",
|
||||
Long: `This command will compact the series file by removing deleted series.`,
|
||||
RunE: RunCompactSeriesFile,
|
||||
}
|
||||
|
||||
home, _ := fs.InfluxDir()
|
||||
defaultPath := filepath.Join(home, "engine")
|
||||
defaultSFilePath := filepath.Join(defaultPath, storage.DefaultSeriesFileDirectoryName)
|
||||
defaultIndexPath := filepath.Join(defaultPath, storage.DefaultIndexDirectoryName)
|
||||
|
||||
cmd.Flags().StringVar(&compactSeriesFileFlags.SeriesFilePath, "sfile-path", defaultSFilePath, "Path to the Series File directory. Defaults to "+defaultSFilePath)
|
||||
cmd.Flags().StringVar(&compactSeriesFileFlags.IndexPath, "tsi-path", defaultIndexPath, "Path to the TSI index directory. Defaults to "+defaultIndexPath)
|
||||
|
||||
cmd.Flags().IntVar(&compactSeriesFileFlags.Concurrency, "concurrency", runtime.GOMAXPROCS(0), "Number of workers to dedicate to compaction. Defaults to GOMAXPROCS. Max 8.")
|
||||
|
||||
cmd.SetOutput(compactSeriesFileFlags.Stdout)
|
||||
|
||||
return cmd
|
||||
}
|
||||
|
||||
// RunCompactSeriesFile executes the run command for CompactSeriesFile.
|
||||
func RunCompactSeriesFile(cmd *cobra.Command, args []string) error {
|
||||
// Verify the user actually wants to run as root.
|
||||
if isRoot() {
|
||||
fmt.Fprintln(compactSeriesFileFlags.Stdout, "You are currently running as root. This will compact your")
|
||||
fmt.Fprintln(compactSeriesFileFlags.Stdout, "series file with root ownership and will be inaccessible")
|
||||
fmt.Fprintln(compactSeriesFileFlags.Stdout, "if you run influxd as a non-root user. You should run")
|
||||
fmt.Fprintln(compactSeriesFileFlags.Stdout, "influxd inspect compact-series-file as the same user you are running influxd.")
|
||||
fmt.Fprint(compactSeriesFileFlags.Stdout, "Are you sure you want to continue? (y/N): ")
|
||||
var answer string
|
||||
if fmt.Scanln(&answer); !strings.HasPrefix(strings.TrimSpace(strings.ToLower(answer)), "y") {
|
||||
return fmt.Errorf("operation aborted")
|
||||
}
|
||||
}
|
||||
|
||||
paths, err := seriesFilePartitionPaths(compactSeriesFileFlags.SeriesFilePath)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
// Build input channel.
|
||||
pathCh := make(chan string, len(paths))
|
||||
for _, path := range paths {
|
||||
pathCh <- path
|
||||
}
|
||||
close(pathCh)
|
||||
|
||||
// Limit maximum concurrency to the total number of series file partitions.
|
||||
concurrency := compactSeriesFileFlags.Concurrency
|
||||
if concurrency > seriesfile.SeriesFilePartitionN {
|
||||
concurrency = seriesfile.SeriesFilePartitionN
|
||||
}
|
||||
|
||||
// Concurrently process each partition in the series file
|
||||
var g errgroup.Group
|
||||
for i := 0; i < concurrency; i++ {
|
||||
g.Go(func() error {
|
||||
for path := range pathCh {
|
||||
if err := compactSeriesFilePartition(path); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
return nil
|
||||
})
|
||||
}
|
||||
if err := g.Wait(); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
// Build new series file indexes
|
||||
sfile := seriesfile.NewSeriesFile(compactSeriesFileFlags.SeriesFilePath)
|
||||
if err = sfile.Open(context.Background()); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
compactor := seriesfile.NewSeriesPartitionCompactor()
|
||||
for _, partition := range sfile.Partitions() {
|
||||
duration, err := compactor.Compact(partition)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
fmt.Fprintf(compactSeriesFileFlags.Stdout, "compacted %s in %s\n", partition.Path(), duration)
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func compactSeriesFilePartition(path string) error {
|
||||
const tmpExt = ".tmp"
|
||||
|
||||
fmt.Fprintf(compactSeriesFileFlags.Stdout, "processing partition for %q\n", path)
|
||||
|
||||
// Open partition so index can recover from entries not in the snapshot.
|
||||
partitionID, err := strconv.Atoi(filepath.Base(path))
|
||||
if err != nil {
|
||||
return fmt.Errorf("cannot parse partition id from path: %s", path)
|
||||
}
|
||||
p := seriesfile.NewSeriesPartition(partitionID, path)
|
||||
if err := p.Open(); err != nil {
|
||||
return fmt.Errorf("cannot open partition: path=%s err=%s", path, err)
|
||||
}
|
||||
defer p.Close()
|
||||
|
||||
// Loop over segments and compact.
|
||||
indexPath := p.IndexPath()
|
||||
var segmentPaths []string
|
||||
for _, segment := range p.Segments() {
|
||||
fmt.Fprintf(compactSeriesFileFlags.Stdout, "processing segment %q %d\n", segment.Path(), segment.ID())
|
||||
|
||||
if err := segment.CompactToPath(segment.Path()+tmpExt, p.Index()); err != nil {
|
||||
return err
|
||||
}
|
||||
segmentPaths = append(segmentPaths, segment.Path())
|
||||
}
|
||||
|
||||
// Close partition.
|
||||
if err := p.Close(); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
// Remove the old segment files and replace with new ones.
|
||||
for _, dst := range segmentPaths {
|
||||
src := dst + tmpExt
|
||||
|
||||
fmt.Fprintf(compactSeriesFileFlags.Stdout, "renaming new segment %q to %q\n", src, dst)
|
||||
if err = os.Rename(src, dst); err != nil && !os.IsNotExist(err) {
|
||||
return fmt.Errorf("serious failure. Please rebuild index and series file: %v", err)
|
||||
}
|
||||
}
|
||||
|
||||
// Remove index file so it will be rebuilt when reopened.
|
||||
fmt.Fprintln(compactSeriesFileFlags.Stdout, "removing index file", indexPath)
|
||||
if err = os.Remove(indexPath); err != nil && !os.IsNotExist(err) { // index won't exist for low cardinality
|
||||
return err
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// seriesFilePartitionPaths returns the paths to each partition in the series file.
|
||||
func seriesFilePartitionPaths(path string) ([]string, error) {
|
||||
sfile := seriesfile.NewSeriesFile(path)
|
||||
if err := sfile.Open(context.Background()); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
var paths []string
|
||||
for _, partition := range sfile.Partitions() {
|
||||
paths = append(paths, partition.Path())
|
||||
}
|
||||
if err := sfile.Close(); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return paths, nil
|
||||
}
|
|
@ -1,141 +0,0 @@
|
|||
// inspects low-level details about tsi1 files.
|
||||
package inspect
|
||||
|
||||
import (
|
||||
"errors"
|
||||
"io"
|
||||
"path/filepath"
|
||||
"regexp"
|
||||
|
||||
"github.com/influxdata/influxdb/v2/internal/fs"
|
||||
"github.com/influxdata/influxdb/v2/v1/tsdb/index/tsi1"
|
||||
"github.com/spf13/cobra"
|
||||
"go.uber.org/zap"
|
||||
)
|
||||
|
||||
// Command represents the program execution for "influxd dumptsi".
|
||||
var measurementFilter, tagKeyFilter, tagValueFilter string
|
||||
var dumpTSIFlags = struct {
|
||||
// Standard input/output, overridden for testing.
|
||||
Stderr io.Writer
|
||||
Stdout io.Writer
|
||||
|
||||
seriesFilePath string
|
||||
dataPath string
|
||||
|
||||
showSeries bool
|
||||
showMeasurements bool
|
||||
showTagKeys bool
|
||||
showTagValues bool
|
||||
showTagValueSeries bool
|
||||
|
||||
measurementFilter *regexp.Regexp
|
||||
tagKeyFilter *regexp.Regexp
|
||||
tagValueFilter *regexp.Regexp
|
||||
}{}
|
||||
|
||||
// NewCommand returns a new instance of Command.
|
||||
func NewDumpTSICommand() *cobra.Command {
|
||||
cmd := &cobra.Command{
|
||||
Use: "dump-tsi",
|
||||
Short: "Dump low level tsi information",
|
||||
Long: `Dumps low-level details about tsi1 files.
|
||||
|
||||
Usage: influx_inspect dumptsi [flags] path...
|
||||
|
||||
-series
|
||||
Dump raw series data
|
||||
-measurements
|
||||
Dump raw measurement data
|
||||
-tag-keys
|
||||
Dump raw tag keys
|
||||
-tag-values
|
||||
Dump raw tag values
|
||||
-tag-value-series
|
||||
Dump raw series for each tag value
|
||||
-measurement-filter REGEXP
|
||||
Filters data by measurement regular expression
|
||||
-series-path PATH
|
||||
Path to the "_series" directory under the database data directory.
|
||||
-index-path PATH
|
||||
Path to the "index" directory under the database data directory.
|
||||
-tag-key-filter REGEXP
|
||||
Filters data by tag key regular expression
|
||||
-tag-value-filter REGEXP
|
||||
Filters data by tag value regular expression
|
||||
`,
|
||||
RunE: dumpTsi,
|
||||
}
|
||||
defaultDataDir, _ := fs.InfluxDir()
|
||||
defaultDataDir = filepath.Join(defaultDataDir, "engine")
|
||||
defaultIndexDir := filepath.Join(defaultDataDir, "index")
|
||||
defaultSeriesDir := filepath.Join(defaultDataDir, "_series")
|
||||
|
||||
cmd.Flags().StringVar(&dumpTSIFlags.seriesFilePath, "series-path", defaultSeriesDir, "Path to series file")
|
||||
cmd.Flags().StringVar(&dumpTSIFlags.dataPath, "index-path", defaultIndexDir, "Path to the index directory of the data engine")
|
||||
cmd.Flags().BoolVar(&dumpTSIFlags.showSeries, "series", false, "Show raw series data")
|
||||
cmd.Flags().BoolVar(&dumpTSIFlags.showMeasurements, "measurements", false, "Show raw measurement data")
|
||||
cmd.Flags().BoolVar(&dumpTSIFlags.showTagKeys, "tag-keys", false, "Show raw tag key data")
|
||||
cmd.Flags().BoolVar(&dumpTSIFlags.showTagValues, "tag-values", false, "Show raw tag value data")
|
||||
cmd.Flags().BoolVar(&dumpTSIFlags.showTagValueSeries, "tag-value-series", false, "Show raw series data for each value")
|
||||
cmd.Flags().StringVar(&measurementFilter, "measurement-filter", "", "Regex measurement filter")
|
||||
cmd.Flags().StringVar(&tagKeyFilter, "tag-key-filter", "", "Regex tag key filter")
|
||||
cmd.Flags().StringVar(&tagValueFilter, "tag-value-filter", "", "Regex tag value filter")
|
||||
|
||||
return cmd
|
||||
}
|
||||
|
||||
func dumpTsi(cmd *cobra.Command, args []string) error {
|
||||
logger := zap.NewNop()
|
||||
|
||||
// Parse filters.
|
||||
if measurementFilter != "" {
|
||||
re, err := regexp.Compile(measurementFilter)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
dumpTSIFlags.measurementFilter = re
|
||||
}
|
||||
if tagKeyFilter != "" {
|
||||
re, err := regexp.Compile(tagKeyFilter)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
dumpTSIFlags.tagKeyFilter = re
|
||||
}
|
||||
if tagValueFilter != "" {
|
||||
re, err := regexp.Compile(tagValueFilter)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
dumpTSIFlags.tagValueFilter = re
|
||||
}
|
||||
|
||||
if dumpTSIFlags.dataPath == "" {
|
||||
return errors.New("data path must be specified")
|
||||
}
|
||||
|
||||
// Some flags imply other flags.
|
||||
if dumpTSIFlags.showTagValueSeries {
|
||||
dumpTSIFlags.showTagValues = true
|
||||
}
|
||||
if dumpTSIFlags.showTagValues {
|
||||
dumpTSIFlags.showTagKeys = true
|
||||
}
|
||||
if dumpTSIFlags.showTagKeys {
|
||||
dumpTSIFlags.showMeasurements = true
|
||||
}
|
||||
|
||||
dump := tsi1.NewDumpTSI(logger)
|
||||
dump.SeriesFilePath = dumpTSIFlags.seriesFilePath
|
||||
dump.DataPath = dumpTSIFlags.dataPath
|
||||
dump.ShowSeries = dumpTSIFlags.showSeries
|
||||
dump.ShowMeasurements = dumpTSIFlags.showMeasurements
|
||||
dump.ShowTagKeys = dumpTSIFlags.showTagKeys
|
||||
dump.ShowTagValueSeries = dumpTSIFlags.showTagValueSeries
|
||||
dump.MeasurementFilter = dumpTSIFlags.measurementFilter
|
||||
dump.TagKeyFilter = dumpTSIFlags.tagKeyFilter
|
||||
dump.TagValueFilter = dumpTSIFlags.tagValueFilter
|
||||
|
||||
return dump.Run()
|
||||
}
|
|
@ -1,57 +0,0 @@
|
|||
package inspect
|
||||
|
||||
import (
|
||||
"os"
|
||||
|
||||
"github.com/influxdata/influxdb/v2/kit/errors"
|
||||
"github.com/influxdata/influxdb/v2/storage/wal"
|
||||
"github.com/spf13/cobra"
|
||||
)
|
||||
|
||||
var dumpWALFlags = struct {
|
||||
findDuplicates bool
|
||||
}{}
|
||||
|
||||
func NewDumpWALCommand() *cobra.Command {
|
||||
dumpTSMWALCommand := &cobra.Command{
|
||||
Use: "dumpwal",
|
||||
Short: "Dump TSM data from WAL files",
|
||||
Long: `
|
||||
This tool dumps data from WAL files for debugging purposes. Given a list of filepath globs
|
||||
(patterns which match to .wal file paths), the tool will parse and print out the entries in each file.
|
||||
It has two modes of operation, depending on the --find-duplicates flag.
|
||||
|
||||
--find-duplicates=false (default): for each file, the following is printed:
|
||||
* The file name
|
||||
* for each entry,
|
||||
* The type of the entry (either [write] or [delete-bucket-range]);
|
||||
* The formatted entry contents
|
||||
--find-duplicates=true: for each file, the following is printed:
|
||||
* The file name
|
||||
* A list of keys in the file that have out of order timestamps
|
||||
`,
|
||||
RunE: inspectDumpWAL,
|
||||
}
|
||||
|
||||
dumpTSMWALCommand.Flags().BoolVarP(
|
||||
&dumpWALFlags.findDuplicates,
|
||||
"find-duplicates", "", false, "ignore dumping entries; only report keys in the WAL that are out of order")
|
||||
|
||||
return dumpTSMWALCommand
|
||||
}
|
||||
|
||||
func inspectDumpWAL(cmd *cobra.Command, args []string) error {
|
||||
dumper := &wal.Dump{
|
||||
Stdout: os.Stdout,
|
||||
Stderr: os.Stderr,
|
||||
FileGlobs: args,
|
||||
FindDuplicates: dumpWALFlags.findDuplicates,
|
||||
}
|
||||
|
||||
if len(args) == 0 {
|
||||
return errors.New("no files provided. aborting")
|
||||
}
|
||||
|
||||
_, err := dumper.Run(true)
|
||||
return err
|
||||
}
|
|
@ -1,30 +0,0 @@
|
|||
package inspect
|
||||
|
||||
import (
|
||||
"os"
|
||||
|
||||
"github.com/influxdata/influxdb/v2/v1/tsdb/engine/tsm1"
|
||||
"github.com/spf13/cobra"
|
||||
)
|
||||
|
||||
func NewExportBlocksCommand() *cobra.Command {
|
||||
return &cobra.Command{
|
||||
Use: `export-blocks`,
|
||||
Short: "Exports block data",
|
||||
Long: `
|
||||
This command will export all blocks in one or more TSM1 files to
|
||||
another format for easier inspection and debugging.`,
|
||||
RunE: func(cmd *cobra.Command, args []string) error {
|
||||
e := tsm1.NewSQLBlockExporter(os.Stdout)
|
||||
for _, arg := range args {
|
||||
if err := e.ExportFile(arg); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
if err := e.Close(); err != nil {
|
||||
return err
|
||||
}
|
||||
return nil
|
||||
},
|
||||
}
|
||||
}
|
|
@ -1,62 +0,0 @@
|
|||
package inspect
|
||||
|
||||
import (
|
||||
"bufio"
|
||||
"context"
|
||||
"os"
|
||||
"path/filepath"
|
||||
|
||||
"github.com/influxdata/influxdb/v2/internal/fs"
|
||||
"github.com/influxdata/influxdb/v2/v1/tsdb"
|
||||
"github.com/influxdata/influxdb/v2/v1/tsdb/index/tsi1"
|
||||
"github.com/spf13/cobra"
|
||||
)
|
||||
|
||||
func NewExportIndexCommand() *cobra.Command {
|
||||
cmd := &cobra.Command{
|
||||
Use: `export-index`,
|
||||
Short: "Exports TSI index data",
|
||||
Long: `
|
||||
This command will export all series in a TSI index to
|
||||
SQL format for easier inspection and debugging.`,
|
||||
}
|
||||
|
||||
defaultDataDir, _ := fs.InfluxDir()
|
||||
defaultDataDir = filepath.Join(defaultDataDir, "engine")
|
||||
defaultIndexDir := filepath.Join(defaultDataDir, "index")
|
||||
defaultSeriesDir := filepath.Join(defaultDataDir, "_series")
|
||||
|
||||
var seriesFilePath, dataPath string
|
||||
cmd.Flags().StringVar(&seriesFilePath, "series-path", defaultSeriesDir, "Path to series file")
|
||||
cmd.Flags().StringVar(&dataPath, "index-path", defaultIndexDir, "Path to the index directory of the data engine")
|
||||
|
||||
cmd.RunE = func(cmd *cobra.Command, args []string) error {
|
||||
// Initialize series file.
|
||||
sfile := tsdb.NewSeriesFile(seriesFilePath)
|
||||
if err := sfile.Open(context.Background()); err != nil {
|
||||
return err
|
||||
}
|
||||
defer sfile.Close()
|
||||
|
||||
// Open index.
|
||||
idx := tsi1.NewIndex(sfile, tsi1.NewConfig(), tsi1.WithPath(dataPath), tsi1.DisableCompactions())
|
||||
if err := idx.Open(context.Background()); err != nil {
|
||||
return err
|
||||
}
|
||||
defer idx.Close()
|
||||
|
||||
// Dump out index data.
|
||||
w := bufio.NewWriter(os.Stdout)
|
||||
e := tsi1.NewSQLIndexExporter(w)
|
||||
if err := e.ExportIndex(idx); err != nil {
|
||||
return err
|
||||
} else if err := e.Close(); err != nil {
|
||||
return err
|
||||
} else if err := w.Flush(); err != nil {
|
||||
return err
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
return cmd
|
||||
}
|
|
@ -14,17 +14,17 @@ func NewCommand() *cobra.Command {
|
|||
// List of available sub-commands
|
||||
// If a new sub-command is created, it must be added here
|
||||
subCommands := []*cobra.Command{
|
||||
NewBuildTSICommand(),
|
||||
NewCompactSeriesFileCommand(),
|
||||
NewExportBlocksCommand(),
|
||||
NewExportIndexCommand(),
|
||||
NewReportTSMCommand(),
|
||||
NewVerifyTSMCommand(),
|
||||
NewVerifyWALCommand(),
|
||||
NewReportTSICommand(),
|
||||
NewVerifySeriesFileCommand(),
|
||||
NewDumpWALCommand(),
|
||||
NewDumpTSICommand(),
|
||||
//NewBuildTSICommand(),
|
||||
//NewCompactSeriesFileCommand(),
|
||||
//NewExportBlocksCommand(),
|
||||
//NewExportIndexCommand(),
|
||||
//NewReportTSMCommand(),
|
||||
//NewVerifyTSMCommand(),
|
||||
//NewVerifyWALCommand(),
|
||||
//NewReportTSICommand(),
|
||||
//NewVerifySeriesFileCommand(),
|
||||
//NewDumpWALCommand(),
|
||||
//NewDumpTSICommand(),
|
||||
}
|
||||
|
||||
base.AddCommand(subCommands...)
|
||||
|
|
|
@ -1,99 +0,0 @@
|
|||
package inspect
|
||||
|
||||
import (
|
||||
"errors"
|
||||
"io"
|
||||
"os"
|
||||
|
||||
"github.com/influxdata/influxdb/v2"
|
||||
"github.com/influxdata/influxdb/v2/v1/tsdb/index/tsi1"
|
||||
"github.com/spf13/cobra"
|
||||
)
|
||||
|
||||
// Command represents the program execution for "influxd inspect report-tsi".
|
||||
var reportTSIFlags = struct {
|
||||
// Standard input/output, overridden for testing.
|
||||
Stderr io.Writer
|
||||
Stdout io.Writer
|
||||
|
||||
// Data path options
|
||||
Path string // optional. Defaults to dbPath/engine/index
|
||||
SeriesFilePath string // optional. Defaults to dbPath/_series
|
||||
|
||||
// Tenant filtering options
|
||||
Org string
|
||||
Bucket string
|
||||
|
||||
// Reporting options
|
||||
TopN int
|
||||
ByMeasurement bool
|
||||
byTagKey bool // currently unused
|
||||
}{}
|
||||
|
||||
// NewReportTsiCommand returns a new instance of Command with default setting applied.
|
||||
func NewReportTSICommand() *cobra.Command {
|
||||
cmd := &cobra.Command{
|
||||
Use: "report-tsi",
|
||||
Short: "Reports the cardinality of TSI files",
|
||||
Long: `This command will analyze TSI files within a storage engine directory, reporting
|
||||
the cardinality of data within the files, divided into org and bucket cardinalities.
|
||||
|
||||
For each report, the following is output:
|
||||
|
||||
* All orgs and buckets in the index;
|
||||
* The series cardinality within each org and each bucket;
|
||||
* The time taken to read the index.
|
||||
|
||||
Depending on the --measurements flag, series cardinality is segmented
|
||||
in the following ways:
|
||||
|
||||
* Series cardinality for each organization;
|
||||
* Series cardinality for each bucket;
|
||||
* Series cardinality for each measurement;`,
|
||||
RunE: RunReportTSI,
|
||||
}
|
||||
|
||||
cmd.Flags().StringVar(&reportTSIFlags.Path, "path", os.Getenv("HOME")+"/.influxdbv2/engine/index", "Path to index. Defaults $HOME/.influxdbv2/engine/index")
|
||||
cmd.Flags().StringVar(&reportTSIFlags.SeriesFilePath, "series-file", os.Getenv("HOME")+"/.influxdbv2/engine/_series", "Optional path to series file. Defaults $HOME/.influxdbv2/engine/_series")
|
||||
cmd.Flags().BoolVarP(&reportTSIFlags.ByMeasurement, "measurements", "m", false, "Segment cardinality by measurements")
|
||||
cmd.Flags().IntVarP(&reportTSIFlags.TopN, "top", "t", 0, "Limit results to top n")
|
||||
cmd.Flags().StringVarP(&reportTSIFlags.Bucket, "bucket_id", "b", "", "If bucket is specified, org must be specified. A bucket id must be a base-16 string")
|
||||
cmd.Flags().StringVarP(&reportTSIFlags.Org, "org_id", "o", "", "Only specified org data will be reported. An org id must be a base-16 string")
|
||||
|
||||
cmd.SetOutput(reportTSIFlags.Stdout)
|
||||
|
||||
return cmd
|
||||
}
|
||||
|
||||
// RunReportTSI executes the run command for ReportTSI.
|
||||
func RunReportTSI(cmd *cobra.Command, args []string) error {
|
||||
report := tsi1.NewReportCommand()
|
||||
report.DataPath = reportTSIFlags.Path
|
||||
report.ByMeasurement = reportTSIFlags.ByMeasurement
|
||||
report.TopN = reportTSIFlags.TopN
|
||||
report.SeriesDirPath = reportTSIFlags.SeriesFilePath
|
||||
|
||||
report.Stdout = os.Stdout
|
||||
report.Stderr = os.Stderr
|
||||
|
||||
var err error
|
||||
if reportTSIFlags.Org != "" {
|
||||
if report.OrgID, err = influxdb.IDFromString(reportTSIFlags.Org); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
|
||||
if reportTSIFlags.Bucket != "" {
|
||||
if report.BucketID, err = influxdb.IDFromString(reportTSIFlags.Bucket); err != nil {
|
||||
return err
|
||||
} else if report.OrgID == nil {
|
||||
return errors.New("org must be provided if filtering by bucket")
|
||||
}
|
||||
}
|
||||
|
||||
// Run command with printing enabled
|
||||
if _, err = report.Run(true); err != nil {
|
||||
return err
|
||||
}
|
||||
return nil
|
||||
}
|
|
@ -1,109 +0,0 @@
|
|||
package inspect
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"github.com/influxdata/influxdb/v2"
|
||||
"github.com/influxdata/influxdb/v2/internal/fs"
|
||||
"github.com/influxdata/influxdb/v2/kit/errors"
|
||||
"github.com/influxdata/influxdb/v2/v1/tsdb/engine/tsm1"
|
||||
"github.com/spf13/cobra"
|
||||
"os"
|
||||
"path/filepath"
|
||||
)
|
||||
|
||||
// reportTSMFlags defines the `report-tsm` Command.
|
||||
var reportTSMFlags = struct {
|
||||
pattern string
|
||||
exact bool
|
||||
detailed bool
|
||||
|
||||
orgID, bucketID string
|
||||
dataDir string
|
||||
}{}
|
||||
|
||||
func NewReportTSMCommand() *cobra.Command {
|
||||
|
||||
reportTSMCommand := &cobra.Command{
|
||||
Use: "report-tsm",
|
||||
Short: "Run TSM report",
|
||||
Long: `
|
||||
This command will analyze TSM files within a storage engine directory, reporting
|
||||
the cardinality within the files as well as the time range that the point data
|
||||
covers.
|
||||
|
||||
This command only interrogates the index within each file, and does not read any
|
||||
block data. To reduce heap requirements, by default report-tsm estimates the
|
||||
overall cardinality in the file set by using the HLL++ algorithm. Exact
|
||||
cardinalities can be determined by using the --exact flag.
|
||||
|
||||
For each file, the following is output:
|
||||
|
||||
* The full filename;
|
||||
* The series cardinality within the file;
|
||||
* The number of series first encountered within the file;
|
||||
* The min and max timestamp associated with TSM data in the file; and
|
||||
* The time taken to load the TSM index and apply any tombstones.
|
||||
|
||||
The summary section then outputs the total time range and series cardinality for
|
||||
the fileset. Depending on the --detailed flag, series cardinality is segmented
|
||||
in the following ways:
|
||||
|
||||
* Series cardinality for each organization;
|
||||
* Series cardinality for each bucket;
|
||||
* Series cardinality for each measurement;
|
||||
* Number of field keys for each measurement; and
|
||||
* Number of tag values for each tag key.`,
|
||||
RunE: inspectReportTSMF,
|
||||
}
|
||||
|
||||
reportTSMCommand.Flags().StringVarP(&reportTSMFlags.pattern, "pattern", "", "", "only process TSM files containing pattern")
|
||||
reportTSMCommand.Flags().BoolVarP(&reportTSMFlags.exact, "exact", "", false, "calculate and exact cardinality count. Warning, may use significant memory...")
|
||||
reportTSMCommand.Flags().BoolVarP(&reportTSMFlags.detailed, "detailed", "", false, "emit series cardinality segmented by measurements, tag keys and fields. Warning, may take a while.")
|
||||
|
||||
reportTSMCommand.Flags().StringVarP(&reportTSMFlags.orgID, "org-id", "", "", "process only data belonging to organization ID.")
|
||||
reportTSMCommand.Flags().StringVarP(&reportTSMFlags.bucketID, "bucket-id", "", "", "process only data belonging to bucket ID. Requires org flag to be set.")
|
||||
|
||||
dir, err := fs.InfluxDir()
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
dir = filepath.Join(dir, "engine/data")
|
||||
reportTSMCommand.Flags().StringVarP(&reportTSMFlags.dataDir, "data-dir", "", dir, fmt.Sprintf("use provided data directory (defaults to %s).", dir))
|
||||
|
||||
return reportTSMCommand
|
||||
}
|
||||
|
||||
// inspectReportTSMF runs the report-tsm tool.
|
||||
func inspectReportTSMF(cmd *cobra.Command, args []string) error {
|
||||
report := &tsm1.Report{
|
||||
Stderr: os.Stderr,
|
||||
Stdout: os.Stdout,
|
||||
Dir: reportTSMFlags.dataDir,
|
||||
Pattern: reportTSMFlags.pattern,
|
||||
Detailed: reportTSMFlags.detailed,
|
||||
Exact: reportTSMFlags.exact,
|
||||
}
|
||||
|
||||
if reportTSMFlags.orgID == "" && reportTSMFlags.bucketID != "" {
|
||||
return errors.New("org-id must be set for non-empty bucket-id")
|
||||
}
|
||||
|
||||
if reportTSMFlags.orgID != "" {
|
||||
orgID, err := influxdb.IDFromString(reportTSMFlags.orgID)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
report.OrgID = orgID
|
||||
}
|
||||
|
||||
if reportTSMFlags.bucketID != "" {
|
||||
bucketID, err := influxdb.IDFromString(reportTSMFlags.bucketID)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
report.BucketID = bucketID
|
||||
}
|
||||
|
||||
_, err := report.Run(true)
|
||||
return err
|
||||
}
|
|
@ -1,69 +0,0 @@
|
|||
package inspect
|
||||
|
||||
import (
|
||||
"os"
|
||||
"runtime"
|
||||
|
||||
"github.com/influxdata/influxdb/v2/cmd/influx_inspect/verify/seriesfile"
|
||||
"github.com/influxdata/influxdb/v2/logger"
|
||||
"github.com/spf13/cobra"
|
||||
"go.uber.org/zap/zapcore"
|
||||
)
|
||||
|
||||
// NewVerifySeriesFileCommand returns a new instance of verifySeriesCommand
|
||||
// for execution of "influx_inspect verify-seriesfile".
|
||||
func NewVerifySeriesFileCommand() *cobra.Command {
|
||||
verifySeriesCommand := &cobra.Command{
|
||||
Use: "verify-seriesfile",
|
||||
Short: "Verifies the integrity of Series files",
|
||||
Long: `Verifies the integrity of Series files.
|
||||
Usage: influx_inspect verify-seriesfile [flags]
|
||||
--series-file <path>
|
||||
Path to a series file. This defaults to ` + os.Getenv("HOME") + `/.influxdbv2/engine/_series.
|
||||
--v
|
||||
Enable verbose logging.
|
||||
--c
|
||||
How many concurrent workers to run.
|
||||
Defaults to "` + string(runtime.GOMAXPROCS(0)) + `" on this machine.`,
|
||||
RunE: verifySeriesRun,
|
||||
}
|
||||
|
||||
verifySeriesCommand.Flags().StringVar(&VerifySeriesFlags.seriesFile, "series-file", os.Getenv("HOME")+"/.influxdbv2/engine/_series",
|
||||
"Path to a series file. This defaults to "+os.Getenv("HOME")+"/.influxdbv2/engine/_series")
|
||||
verifySeriesCommand.Flags().BoolVarP(&VerifySeriesFlags.verbose, "v", "v", false,
|
||||
"Verbose output.")
|
||||
verifySeriesCommand.Flags().IntVarP(&VerifySeriesFlags.concurrent, "c", "c", runtime.GOMAXPROCS(0),
|
||||
"How many concurrent workers to run.")
|
||||
|
||||
return verifySeriesCommand
|
||||
}
|
||||
|
||||
var VerifySeriesFlags = struct {
|
||||
seriesFile string
|
||||
verbose bool
|
||||
concurrent int
|
||||
}{}
|
||||
|
||||
// verifySeriesRun executes the command.
|
||||
func verifySeriesRun(cmd *cobra.Command, args []string) error {
|
||||
config := logger.NewConfig()
|
||||
config.Level = zapcore.WarnLevel
|
||||
if VerifySeriesFlags.verbose {
|
||||
config.Level = zapcore.InfoLevel
|
||||
}
|
||||
logger, err := config.New(os.Stderr)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
v := seriesfile.NewVerify()
|
||||
v.Logger = logger
|
||||
v.Concurrent = VerifySeriesFlags.concurrent
|
||||
|
||||
if VerifySeriesFlags.seriesFile != "" {
|
||||
_, err := v.VerifySeriesFile(VerifySeriesFlags.seriesFile)
|
||||
return err
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
|
@ -1,72 +0,0 @@
|
|||
package inspect
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"os"
|
||||
"path/filepath"
|
||||
|
||||
"github.com/influxdata/influxdb/v2/kit/cli"
|
||||
"github.com/influxdata/influxdb/v2/v1/tsdb/engine/tsm1"
|
||||
"github.com/spf13/cobra"
|
||||
)
|
||||
|
||||
// verifyTSMFlags defines the `verify-tsm` Command.
|
||||
var verifyTSMFlags = struct {
|
||||
cli.OrgBucket
|
||||
path string
|
||||
}{}
|
||||
|
||||
func NewVerifyTSMCommand() *cobra.Command {
|
||||
cmd := &cobra.Command{
|
||||
Use: "verify-tsm <pathspec>...",
|
||||
Short: "Checks the consistency of TSM files",
|
||||
Long: `
|
||||
This command will analyze a set of TSM files for inconsistencies between the
|
||||
TSM index and the blocks.
|
||||
|
||||
The checks performed by this command are:
|
||||
|
||||
* CRC-32 checksums match for each block
|
||||
* TSM index min and max timestamps match decoded data
|
||||
|
||||
OPTIONS
|
||||
|
||||
<pathspec>...
|
||||
A list of files or directories to search for TSM files.
|
||||
|
||||
An optional organization or organization and bucket may be specified to limit
|
||||
the analysis.
|
||||
`,
|
||||
RunE: verifyTSMF,
|
||||
}
|
||||
|
||||
verifyTSMFlags.AddFlags(cmd)
|
||||
|
||||
return cmd
|
||||
}
|
||||
|
||||
func verifyTSMF(cmd *cobra.Command, args []string) error {
|
||||
verify := tsm1.VerifyTSM{
|
||||
Stdout: os.Stdout,
|
||||
OrgID: verifyTSMFlags.Org,
|
||||
BucketID: verifyTSMFlags.Bucket,
|
||||
}
|
||||
|
||||
// resolve all pathspecs
|
||||
for _, arg := range args {
|
||||
fi, err := os.Stat(arg)
|
||||
if err != nil {
|
||||
fmt.Printf("Error processing path %q: %v", arg, err)
|
||||
continue
|
||||
}
|
||||
|
||||
if fi.IsDir() {
|
||||
files, _ := filepath.Glob(filepath.Join(arg, "*."+tsm1.TSMFileExtension))
|
||||
verify.Paths = append(verify.Paths, files...)
|
||||
} else {
|
||||
verify.Paths = append(verify.Paths, arg)
|
||||
}
|
||||
}
|
||||
|
||||
return verify.Run()
|
||||
}
|
|
@ -1,57 +0,0 @@
|
|||
package inspect
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"github.com/influxdata/influxdb/v2/internal/fs"
|
||||
"github.com/influxdata/influxdb/v2/storage/wal"
|
||||
"github.com/spf13/cobra"
|
||||
"os"
|
||||
"path/filepath"
|
||||
)
|
||||
|
||||
func NewVerifyWALCommand() *cobra.Command {
|
||||
verifyWALCommand := &cobra.Command{
|
||||
Use: `verify-wal`,
|
||||
Short: "Check for WAL corruption",
|
||||
Long: `
|
||||
This command will analyze the WAL (Write-Ahead Log) in a storage directory to
|
||||
check if there are any corrupt files. If any corrupt files are found, the names
|
||||
of said corrupt files will be reported. The tool will also count the total number
|
||||
of entries in the scanned WAL files, in case this is of interest.
|
||||
|
||||
For each file, the following is output:
|
||||
* The file name;
|
||||
* "clean" (if the file is clean) OR
|
||||
The first position of any corruption that is found
|
||||
In the summary section, the following is printed:
|
||||
* The number of WAL files scanned;
|
||||
* The number of WAL entries scanned;
|
||||
* A list of files found to be corrupt`,
|
||||
RunE: inspectVerifyWAL,
|
||||
}
|
||||
|
||||
dir, err := fs.InfluxDir()
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
dir = filepath.Join(dir, "engine/wal")
|
||||
verifyWALCommand.Flags().StringVarP(&verifyWALFlags.dataDir, "data-dir", "", dir, fmt.Sprintf("use provided data directory (defaults to %s).", dir))
|
||||
|
||||
return verifyWALCommand
|
||||
}
|
||||
|
||||
var verifyWALFlags = struct {
|
||||
dataDir string
|
||||
}{}
|
||||
|
||||
// inspectReportTSMF runs the report-tsm tool.
|
||||
func inspectVerifyWAL(cmd *cobra.Command, args []string) error {
|
||||
report := &wal.Verifier{
|
||||
Stderr: os.Stderr,
|
||||
Stdout: os.Stdout,
|
||||
Dir: verifyWALFlags.dataDir,
|
||||
}
|
||||
|
||||
_, err := report.Run(true)
|
||||
return err
|
||||
}
|
|
@ -12,7 +12,6 @@ import (
|
|||
"github.com/influxdata/influxdb/v2/kit/prom"
|
||||
"github.com/influxdata/influxdb/v2/models"
|
||||
"github.com/influxdata/influxdb/v2/storage"
|
||||
"github.com/influxdata/influxdb/v2/storage/reads"
|
||||
"github.com/influxdata/influxdb/v2/v1/tsdb/cursors"
|
||||
"github.com/influxdata/influxql"
|
||||
"github.com/prometheus/client_golang/prometheus"
|
||||
|
@ -25,7 +24,6 @@ import (
|
|||
// to facilitate testing.
|
||||
type Engine interface {
|
||||
influxdb.DeleteService
|
||||
reads.Viewer
|
||||
storage.PointsWriter
|
||||
storage.BucketDeleter
|
||||
prom.PrometheusCollector
|
||||
|
@ -138,16 +136,6 @@ func (t *TemporaryEngine) PrometheusCollectors() []prometheus.Collector {
|
|||
return t.engine.PrometheusCollectors()
|
||||
}
|
||||
|
||||
// CreateCursorIterator calls into the underlying engines CreateCurorIterator.
|
||||
func (t *TemporaryEngine) CreateCursorIterator(ctx context.Context) (cursors.CursorIterator, error) {
|
||||
return t.engine.CreateCursorIterator(ctx)
|
||||
}
|
||||
|
||||
// CreateSeriesCursor calls into the underlying engines CreateSeriesCursor.
|
||||
func (t *TemporaryEngine) CreateSeriesCursor(ctx context.Context, orgID, bucketID influxdb.ID, cond influxql.Expr) (storage.SeriesCursor, error) {
|
||||
return t.engine.CreateSeriesCursor(ctx, orgID, bucketID, cond)
|
||||
}
|
||||
|
||||
// TagKeys calls into the underlying engines TagKeys.
|
||||
func (t *TemporaryEngine) TagKeys(ctx context.Context, orgID, bucketID influxdb.ID, start, end int64, predicate influxql.Expr) (cursors.StringIterator, error) {
|
||||
return t.engine.TagKeys(ctx, orgID, bucketID, start, end, predicate)
|
||||
|
|
|
@ -54,6 +54,7 @@ import (
|
|||
"github.com/influxdata/influxdb/v2/snowflake"
|
||||
"github.com/influxdata/influxdb/v2/source"
|
||||
"github.com/influxdata/influxdb/v2/storage"
|
||||
"github.com/influxdata/influxdb/v2/storage/reads"
|
||||
"github.com/influxdata/influxdb/v2/storage/readservice"
|
||||
taskbackend "github.com/influxdata/influxdb/v2/task/backend"
|
||||
"github.com/influxdata/influxdb/v2/task/backend/coordinator"
|
||||
|
@ -63,9 +64,8 @@ import (
|
|||
"github.com/influxdata/influxdb/v2/telemetry"
|
||||
"github.com/influxdata/influxdb/v2/tenant"
|
||||
storage2 "github.com/influxdata/influxdb/v2/v1/services/storage"
|
||||
"github.com/influxdata/influxdb/v2/v1/storage/reads"
|
||||
_ "github.com/influxdata/influxdb/v2/v1/tsdb/index/tsi1" // needed for tsi1
|
||||
_ "github.com/influxdata/influxdb/v2/v1/tsdb/engine/tsm1" // needed for tsm1
|
||||
_ "github.com/influxdata/influxdb/v2/v1/tsdb/index/tsi1" // needed for tsi1
|
||||
"github.com/influxdata/influxdb/v2/vault"
|
||||
pzap "github.com/influxdata/influxdb/v2/zap"
|
||||
"github.com/opentracing/opentracing-go"
|
||||
|
|
|
@ -9,12 +9,10 @@ import (
|
|||
|
||||
"github.com/influxdata/flux"
|
||||
"github.com/influxdata/influxdb/v2"
|
||||
"github.com/influxdata/influxdb/v2/cmd/influxd/generate"
|
||||
"github.com/influxdata/influxdb/v2/cmd/influxd/launcher"
|
||||
"github.com/influxdata/influxdb/v2/cmd/influxd/restore"
|
||||
_ "github.com/influxdata/influxdb/v2/query/builtin"
|
||||
_ "github.com/influxdata/influxdb/v2/v1/tsdb/index/tsi1"
|
||||
_ "github.com/influxdata/influxdb/v2/v1/tsdb/engine/tsm1"
|
||||
_ "github.com/influxdata/influxdb/v2/v1/tsdb/index/tsi1"
|
||||
"github.com/spf13/cobra"
|
||||
)
|
||||
|
||||
|
@ -32,8 +30,9 @@ func main() {
|
|||
influxdb.SetBuildInfo(version, commit, date)
|
||||
|
||||
rootCmd := launcher.NewInfluxdCommand(context.Background(),
|
||||
generate.Command,
|
||||
restore.Command,
|
||||
// FIXME
|
||||
//generate.Command,
|
||||
//restore.Command,
|
||||
&cobra.Command{
|
||||
Use: "version",
|
||||
Short: "Print the influxd server version",
|
||||
|
|
|
@ -7,7 +7,7 @@ import (
|
|||
|
||||
"github.com/gogo/protobuf/proto"
|
||||
"github.com/influxdata/influxdb/v2"
|
||||
"github.com/influxdata/influxdb/v2/models"
|
||||
"github.com/influxdata/influxdb/v2/v1/models"
|
||||
)
|
||||
|
||||
// MetricsCollection is the struct including metrics and other requirements.
|
||||
|
|
|
@ -6,7 +6,6 @@ import (
|
|||
|
||||
"github.com/influxdata/influxdb/v2/nats"
|
||||
"github.com/influxdata/influxdb/v2/storage"
|
||||
"github.com/influxdata/influxdb/v2/v1/tsdb"
|
||||
"go.uber.org/zap"
|
||||
)
|
||||
|
||||
|
@ -21,12 +20,8 @@ func (s PointWriter) Record(collected MetricsCollection) error {
|
|||
if err != nil {
|
||||
return err
|
||||
}
|
||||
ps, err = tsdb.ExplodePoints(collected.OrgID, collected.BucketID, ps)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
return s.Writer.WritePoints(context.TODO(), 0, 0, nil)
|
||||
return s.Writer.WritePoints(context.TODO(), 0, 0, ps)
|
||||
}
|
||||
|
||||
// Recorder record the metrics of a time based.
|
||||
|
|
|
@ -2,7 +2,6 @@ package cli
|
|||
|
||||
import (
|
||||
"github.com/influxdata/influxdb/v2"
|
||||
"github.com/influxdata/influxdb/v2/v1/tsdb"
|
||||
"github.com/spf13/cobra"
|
||||
"github.com/spf13/pflag"
|
||||
)
|
||||
|
@ -56,5 +55,6 @@ func (o *OrgBucket) OrgBucketID() (orgID, bucketID influxdb.ID) {
|
|||
}
|
||||
|
||||
func (o *OrgBucket) Name() [influxdb.IDLength]byte {
|
||||
return tsdb.EncodeName(o.OrgBucketID())
|
||||
// TODO: FIX THIS
|
||||
panic("TODO: Fix")
|
||||
}
|
||||
|
|
|
@ -1,9 +1,9 @@
|
|||
package mock
|
||||
|
||||
import (
|
||||
"github.com/influxdata/influxdb/v2/models"
|
||||
"github.com/influxdata/influxdb/v2/pkg/data/gen"
|
||||
"github.com/influxdata/influxdb/v2/storage/reads"
|
||||
"github.com/influxdata/influxdb/v2/v1/models"
|
||||
"github.com/influxdata/influxdb/v2/v1/tsdb/cursors"
|
||||
)
|
||||
|
||||
|
|
|
@ -7,21 +7,21 @@
|
|||
package gen
|
||||
|
||||
import (
|
||||
"github.com/influxdata/influxdb/v2/v1/tsdb/cursors"
|
||||
"github.com/influxdata/influxdb/v2/v1/tsdb"
|
||||
"github.com/influxdata/influxdb/v2/v1/tsdb/engine/tsm1"
|
||||
)
|
||||
|
||||
type FloatValues interface {
|
||||
Copy(*cursors.FloatArray)
|
||||
Copy(*tsdb.FloatArray)
|
||||
}
|
||||
|
||||
type floatArray struct {
|
||||
cursors.FloatArray
|
||||
tsdb.FloatArray
|
||||
}
|
||||
|
||||
func newFloatArrayLen(sz int) *floatArray {
|
||||
return &floatArray{
|
||||
FloatArray: cursors.FloatArray{
|
||||
FloatArray: tsdb.FloatArray{
|
||||
Timestamps: make([]int64, sz),
|
||||
Values: make([]float64, sz),
|
||||
},
|
||||
|
@ -32,22 +32,22 @@ func (a *floatArray) Encode(b []byte) ([]byte, error) {
|
|||
return tsm1.EncodeFloatArrayBlock(&a.FloatArray, b)
|
||||
}
|
||||
|
||||
func (a *floatArray) Copy(dst *cursors.FloatArray) {
|
||||
func (a *floatArray) Copy(dst *tsdb.FloatArray) {
|
||||
dst.Timestamps = append(dst.Timestamps[:0], a.Timestamps...)
|
||||
dst.Values = append(dst.Values[:0], a.Values...)
|
||||
}
|
||||
|
||||
type IntegerValues interface {
|
||||
Copy(*cursors.IntegerArray)
|
||||
Copy(*tsdb.IntegerArray)
|
||||
}
|
||||
|
||||
type integerArray struct {
|
||||
cursors.IntegerArray
|
||||
tsdb.IntegerArray
|
||||
}
|
||||
|
||||
func newIntegerArrayLen(sz int) *integerArray {
|
||||
return &integerArray{
|
||||
IntegerArray: cursors.IntegerArray{
|
||||
IntegerArray: tsdb.IntegerArray{
|
||||
Timestamps: make([]int64, sz),
|
||||
Values: make([]int64, sz),
|
||||
},
|
||||
|
@ -58,22 +58,22 @@ func (a *integerArray) Encode(b []byte) ([]byte, error) {
|
|||
return tsm1.EncodeIntegerArrayBlock(&a.IntegerArray, b)
|
||||
}
|
||||
|
||||
func (a *integerArray) Copy(dst *cursors.IntegerArray) {
|
||||
func (a *integerArray) Copy(dst *tsdb.IntegerArray) {
|
||||
dst.Timestamps = append(dst.Timestamps[:0], a.Timestamps...)
|
||||
dst.Values = append(dst.Values[:0], a.Values...)
|
||||
}
|
||||
|
||||
type UnsignedValues interface {
|
||||
Copy(*cursors.UnsignedArray)
|
||||
Copy(*tsdb.UnsignedArray)
|
||||
}
|
||||
|
||||
type unsignedArray struct {
|
||||
cursors.UnsignedArray
|
||||
tsdb.UnsignedArray
|
||||
}
|
||||
|
||||
func newUnsignedArrayLen(sz int) *unsignedArray {
|
||||
return &unsignedArray{
|
||||
UnsignedArray: cursors.UnsignedArray{
|
||||
UnsignedArray: tsdb.UnsignedArray{
|
||||
Timestamps: make([]int64, sz),
|
||||
Values: make([]uint64, sz),
|
||||
},
|
||||
|
@ -84,22 +84,22 @@ func (a *unsignedArray) Encode(b []byte) ([]byte, error) {
|
|||
return tsm1.EncodeUnsignedArrayBlock(&a.UnsignedArray, b)
|
||||
}
|
||||
|
||||
func (a *unsignedArray) Copy(dst *cursors.UnsignedArray) {
|
||||
func (a *unsignedArray) Copy(dst *tsdb.UnsignedArray) {
|
||||
dst.Timestamps = append(dst.Timestamps[:0], a.Timestamps...)
|
||||
dst.Values = append(dst.Values[:0], a.Values...)
|
||||
}
|
||||
|
||||
type StringValues interface {
|
||||
Copy(*cursors.StringArray)
|
||||
Copy(*tsdb.StringArray)
|
||||
}
|
||||
|
||||
type stringArray struct {
|
||||
cursors.StringArray
|
||||
tsdb.StringArray
|
||||
}
|
||||
|
||||
func newStringArrayLen(sz int) *stringArray {
|
||||
return &stringArray{
|
||||
StringArray: cursors.StringArray{
|
||||
StringArray: tsdb.StringArray{
|
||||
Timestamps: make([]int64, sz),
|
||||
Values: make([]string, sz),
|
||||
},
|
||||
|
@ -110,22 +110,22 @@ func (a *stringArray) Encode(b []byte) ([]byte, error) {
|
|||
return tsm1.EncodeStringArrayBlock(&a.StringArray, b)
|
||||
}
|
||||
|
||||
func (a *stringArray) Copy(dst *cursors.StringArray) {
|
||||
func (a *stringArray) Copy(dst *tsdb.StringArray) {
|
||||
dst.Timestamps = append(dst.Timestamps[:0], a.Timestamps...)
|
||||
dst.Values = append(dst.Values[:0], a.Values...)
|
||||
}
|
||||
|
||||
type BooleanValues interface {
|
||||
Copy(*cursors.BooleanArray)
|
||||
Copy(*tsdb.BooleanArray)
|
||||
}
|
||||
|
||||
type booleanArray struct {
|
||||
cursors.BooleanArray
|
||||
tsdb.BooleanArray
|
||||
}
|
||||
|
||||
func newBooleanArrayLen(sz int) *booleanArray {
|
||||
return &booleanArray{
|
||||
BooleanArray: cursors.BooleanArray{
|
||||
BooleanArray: tsdb.BooleanArray{
|
||||
Timestamps: make([]int64, sz),
|
||||
Values: make([]bool, sz),
|
||||
},
|
||||
|
@ -136,7 +136,7 @@ func (a *booleanArray) Encode(b []byte) ([]byte, error) {
|
|||
return tsm1.EncodeBooleanArrayBlock(&a.BooleanArray, b)
|
||||
}
|
||||
|
||||
func (a *booleanArray) Copy(dst *cursors.BooleanArray) {
|
||||
func (a *booleanArray) Copy(dst *tsdb.BooleanArray) {
|
||||
dst.Timestamps = append(dst.Timestamps[:0], a.Timestamps...)
|
||||
dst.Values = append(dst.Values[:0], a.Values...)
|
||||
}
|
||||
|
|
|
@ -1,7 +1,7 @@
|
|||
package gen
|
||||
|
||||
import (
|
||||
"github.com/influxdata/influxdb/v2/v1/tsdb/cursors"
|
||||
"github.com/influxdata/influxdb/v2/v1/tsdb"
|
||||
"github.com/influxdata/influxdb/v2/v1/tsdb/engine/tsm1"
|
||||
)
|
||||
|
||||
|
@ -9,16 +9,16 @@ import (
|
|||
{{ $typename := print .name "Array" }}
|
||||
{{ $tsdbname := print .Name "Array" }}
|
||||
type {{.Name}}Values interface {
|
||||
Copy(*cursors.{{$tsdbname}})
|
||||
Copy(*tsdb.{{$tsdbname}})
|
||||
}
|
||||
|
||||
type {{$typename}} struct {
|
||||
cursors.{{$tsdbname}}
|
||||
tsdb.{{$tsdbname}}
|
||||
}
|
||||
|
||||
func new{{$tsdbname}}Len(sz int) *{{$typename}} {
|
||||
return &{{$typename}}{
|
||||
{{$tsdbname}}: cursors.{{$tsdbname}}{
|
||||
{{$tsdbname}}: tsdb.{{$tsdbname}}{
|
||||
Timestamps: make([]int64, sz),
|
||||
Values: make([]{{.Type}}, sz),
|
||||
},
|
||||
|
@ -29,7 +29,7 @@ func (a *{{$typename}}) Encode(b []byte) ([]byte, error) {
|
|||
return tsm1.Encode{{$tsdbname}}Block(&a.{{$tsdbname}}, b)
|
||||
}
|
||||
|
||||
func (a *{{$typename}}) Copy(dst *cursors.{{$tsdbname}}) {
|
||||
func (a *{{$typename}}) Copy(dst *tsdb.{{$tsdbname}}) {
|
||||
dst.Timestamps = append(dst.Timestamps[:0], a.Timestamps...)
|
||||
dst.Values = append(dst.Values[:0], a.Values...)
|
||||
}
|
||||
|
|
|
@ -4,7 +4,7 @@ import (
|
|||
"container/heap"
|
||||
"math"
|
||||
|
||||
"github.com/influxdata/influxdb/v2/models"
|
||||
"github.com/influxdata/influxdb/v2/v1/models"
|
||||
)
|
||||
|
||||
type mergedSeriesGenerator struct {
|
||||
|
@ -79,8 +79,8 @@ func (s *mergedSeriesGenerator) Key() []byte {
|
|||
return s.heap.items[0].Key()
|
||||
}
|
||||
|
||||
func (s *mergedSeriesGenerator) ID() []byte {
|
||||
return s.heap.items[0].ID()
|
||||
func (s *mergedSeriesGenerator) Name() []byte {
|
||||
return s.heap.items[0].Name()
|
||||
}
|
||||
|
||||
func (s *mergedSeriesGenerator) Tags() models.Tags {
|
||||
|
|
|
@ -8,48 +8,33 @@ import (
|
|||
"time"
|
||||
|
||||
"github.com/google/go-cmp/cmp"
|
||||
"github.com/influxdata/influxdb/v2"
|
||||
"github.com/influxdata/influxdb/v2/models"
|
||||
"github.com/influxdata/influxdb/v2/v1/tsdb"
|
||||
)
|
||||
|
||||
var (
|
||||
org = influxdb.ID(0xff00ff00)
|
||||
bucket = influxdb.ID(0xcc00cc00)
|
||||
orgBucketID = tsdb.EncodeName(org, bucket)
|
||||
)
|
||||
|
||||
func sg(m, prefix, field string, counts ...int) SeriesGenerator {
|
||||
func sg(name, prefix, field string, counts ...int) SeriesGenerator {
|
||||
spec := TimeSequenceSpec{Count: 1, Start: time.Unix(0, 0), Delta: time.Second}
|
||||
ts := NewTimestampSequenceFromSpec(spec)
|
||||
vs := NewFloatConstantValuesSequence(1)
|
||||
vg := NewTimeFloatValuesSequence(spec.Count, ts, vs)
|
||||
return NewSeriesGenerator(orgBucketID, []byte(field), vg, NewTagsValuesSequenceCounts(m, field, prefix, counts))
|
||||
return NewSeriesGenerator([]byte(name), []byte(field), vg, NewTagsValuesSequenceCounts(prefix, counts))
|
||||
}
|
||||
|
||||
func tags(sb *strings.Builder, m, prefix, f string, vals []int) {
|
||||
func tags(sb *strings.Builder, prefix string, vals []int) {
|
||||
sb.WriteByte(',')
|
||||
|
||||
// max tag width
|
||||
tw := int(math.Ceil(math.Log10(float64(len(vals)))))
|
||||
tf := fmt.Sprintf("%s%%0%dd=value%%d", prefix, tw)
|
||||
tvs := make([]string, 0, len(vals)+2)
|
||||
|
||||
tvs = append(tvs, fmt.Sprintf("%s=%s", models.MeasurementTagKey, m))
|
||||
|
||||
tvs := make([]string, len(vals))
|
||||
for i := range vals {
|
||||
tvs = append(tvs, fmt.Sprintf(tf, i, vals[i]))
|
||||
tvs[i] = fmt.Sprintf(tf, i, vals[i])
|
||||
}
|
||||
|
||||
tvs = append(tvs, fmt.Sprintf("%s=%s", models.FieldKeyTagKey, f))
|
||||
|
||||
sb.WriteString(strings.Join(tvs, ","))
|
||||
}
|
||||
|
||||
func line(name, prefix, field string, vals ...int) string {
|
||||
var sb strings.Builder
|
||||
sb.Write(orgBucketID[:])
|
||||
tags(&sb, name, prefix, field, vals)
|
||||
sb.WriteString(name)
|
||||
tags(&sb, prefix, vals)
|
||||
sb.WriteString("#!~#")
|
||||
sb.WriteString(field)
|
||||
return sb.String()
|
||||
|
|
|
@ -4,7 +4,7 @@ import (
|
|||
"math"
|
||||
"time"
|
||||
|
||||
"github.com/influxdata/influxdb/v2/models"
|
||||
"github.com/influxdata/influxdb/v2/v1/models"
|
||||
)
|
||||
|
||||
type SeriesGenerator interface {
|
||||
|
@ -15,8 +15,9 @@ type SeriesGenerator interface {
|
|||
// The returned value may be cached.
|
||||
Key() []byte
|
||||
|
||||
// ID returns the org and bucket identifier for the series.
|
||||
ID() []byte
|
||||
// Name returns the name of the measurement.
|
||||
// The returned value may be modified by a subsequent call to Next.
|
||||
Name() []byte
|
||||
|
||||
// Tags returns the tag set.
|
||||
// The returned value may be modified by a subsequent call to Next.
|
||||
|
@ -40,7 +41,7 @@ type TimeSequenceSpec struct {
|
|||
// Start specifies the starting time for the values.
|
||||
Start time.Time
|
||||
|
||||
// Delta specifies the interval between timestamps.
|
||||
// Delta specifies the interval between time stamps.
|
||||
Delta time.Duration
|
||||
|
||||
// Precision specifies the precision of timestamp intervals
|
||||
|
@ -112,7 +113,7 @@ type cache struct {
|
|||
}
|
||||
|
||||
type seriesGenerator struct {
|
||||
id idType
|
||||
name []byte
|
||||
tags TagsSequence
|
||||
field []byte
|
||||
vg TimeValuesSequence
|
||||
|
@ -121,13 +122,13 @@ type seriesGenerator struct {
|
|||
c cache
|
||||
}
|
||||
|
||||
func NewSeriesGenerator(id idType, field []byte, vg TimeValuesSequence, tags TagsSequence) SeriesGenerator {
|
||||
return NewSeriesGeneratorLimit(id, field, vg, tags, math.MaxInt64)
|
||||
func NewSeriesGenerator(name []byte, field []byte, vg TimeValuesSequence, tags TagsSequence) SeriesGenerator {
|
||||
return NewSeriesGeneratorLimit(name, field, vg, tags, math.MaxInt64)
|
||||
}
|
||||
|
||||
func NewSeriesGeneratorLimit(id idType, field []byte, vg TimeValuesSequence, tags TagsSequence, n int64) SeriesGenerator {
|
||||
func NewSeriesGeneratorLimit(name []byte, field []byte, vg TimeValuesSequence, tags TagsSequence, n int64) SeriesGenerator {
|
||||
return &seriesGenerator{
|
||||
id: id,
|
||||
name: name,
|
||||
field: field,
|
||||
tags: tags,
|
||||
vg: vg,
|
||||
|
@ -151,13 +152,13 @@ func (g *seriesGenerator) Next() bool {
|
|||
|
||||
func (g *seriesGenerator) Key() []byte {
|
||||
if len(g.c.key) == 0 {
|
||||
g.c.key = models.MakeKey(g.id[:], g.tags.Value())
|
||||
g.c.key = models.MakeKey(g.name, g.tags.Value())
|
||||
}
|
||||
return g.c.key
|
||||
}
|
||||
|
||||
func (g *seriesGenerator) ID() []byte {
|
||||
return g.id[:]
|
||||
func (g *seriesGenerator) Name() []byte {
|
||||
return g.name
|
||||
}
|
||||
|
||||
func (g *seriesGenerator) Tags() models.Tags {
|
||||
|
|
|
@ -11,26 +11,19 @@ import (
|
|||
"unicode/utf8"
|
||||
|
||||
"github.com/BurntSushi/toml"
|
||||
"github.com/influxdata/influxdb/v2"
|
||||
"github.com/influxdata/influxdb/v2/models"
|
||||
"github.com/influxdata/influxdb/v2/v1/tsdb"
|
||||
"github.com/influxdata/influxdb/v2/v1/models"
|
||||
"github.com/pkg/errors"
|
||||
)
|
||||
|
||||
type Spec struct {
|
||||
OrgID influxdb.ID
|
||||
BucketID influxdb.ID
|
||||
SeriesLimit *int64
|
||||
Measurements []MeasurementSpec
|
||||
}
|
||||
|
||||
type idType [influxdb.IDLength]byte
|
||||
|
||||
func NewSeriesGeneratorFromSpec(s *Spec, tr TimeRange) SeriesGenerator {
|
||||
id := tsdb.EncodeName(s.OrgID, s.BucketID)
|
||||
sg := make([]SeriesGenerator, len(s.Measurements))
|
||||
for i := range s.Measurements {
|
||||
sg[i] = newSeriesGeneratorFromMeasurementSpec(id, &s.Measurements[i], tr)
|
||||
sg[i] = newSeriesGeneratorFromMeasurementSpec(&s.Measurements[i], tr)
|
||||
}
|
||||
if s.SeriesLimit == nil {
|
||||
return NewMergedSeriesGenerator(sg)
|
||||
|
@ -45,19 +38,19 @@ type MeasurementSpec struct {
|
|||
FieldValuesSpec *FieldValuesSpec
|
||||
}
|
||||
|
||||
func newSeriesGeneratorFromMeasurementSpec(id idType, ms *MeasurementSpec, tr TimeRange) SeriesGenerator {
|
||||
func newSeriesGeneratorFromMeasurementSpec(ms *MeasurementSpec, tr TimeRange) SeriesGenerator {
|
||||
if ms.SeriesLimit == nil {
|
||||
return NewSeriesGenerator(
|
||||
id,
|
||||
[]byte(ms.Name),
|
||||
[]byte(ms.FieldValuesSpec.Name),
|
||||
newTimeValuesSequenceFromFieldValuesSpec(ms.FieldValuesSpec, tr),
|
||||
newTagsSequenceFromTagsSpec(ms.Name, ms.FieldValuesSpec.Name, ms.TagsSpec))
|
||||
newTagsSequenceFromTagsSpec(ms.TagsSpec))
|
||||
}
|
||||
return NewSeriesGeneratorLimit(
|
||||
id,
|
||||
[]byte(ms.Name),
|
||||
[]byte(ms.FieldValuesSpec.Name),
|
||||
newTimeValuesSequenceFromFieldValuesSpec(ms.FieldValuesSpec, tr),
|
||||
newTagsSequenceFromTagsSpec(ms.Name, ms.FieldValuesSpec.Name, ms.TagsSpec),
|
||||
newTagsSequenceFromTagsSpec(ms.TagsSpec),
|
||||
int64(*ms.SeriesLimit))
|
||||
}
|
||||
|
||||
|
@ -74,21 +67,14 @@ type TagsSpec struct {
|
|||
Sample *sample
|
||||
}
|
||||
|
||||
func newTagsSequenceFromTagsSpec(m, f string, ts *TagsSpec) TagsSequence {
|
||||
func newTagsSequenceFromTagsSpec(ts *TagsSpec) TagsSequence {
|
||||
var keys []string
|
||||
var vals []CountableSequence
|
||||
|
||||
keys = append(keys, models.MeasurementTagKey)
|
||||
vals = append(vals, NewStringConstantSequence(m))
|
||||
|
||||
for _, spec := range ts.Tags {
|
||||
keys = append(keys, spec.TagKey)
|
||||
vals = append(vals, spec.Values())
|
||||
}
|
||||
|
||||
keys = append(keys, models.FieldKeyTagKey)
|
||||
vals = append(vals, NewStringConstantSequence(f))
|
||||
|
||||
var opts []tagsValuesOption
|
||||
if ts.Sample != nil && *ts.Sample != 1.0 {
|
||||
opts = append(opts, TagValuesSampleOption(float64(*ts.Sample)))
|
||||
|
|
|
@ -6,7 +6,7 @@ import (
|
|||
|
||||
"github.com/BurntSushi/toml"
|
||||
"github.com/google/go-cmp/cmp"
|
||||
"github.com/influxdata/influxdb/v2/models"
|
||||
"github.com/influxdata/influxdb/v2/v1/models"
|
||||
)
|
||||
|
||||
func countableSequenceFnCmp(a, b NewCountableSequenceFn) bool {
|
||||
|
|
|
@ -6,7 +6,7 @@ import (
|
|||
"math/rand"
|
||||
"sort"
|
||||
|
||||
"github.com/influxdata/influxdb/v2/models"
|
||||
"github.com/influxdata/influxdb/v2/v1/models"
|
||||
)
|
||||
|
||||
type TagsSequence interface {
|
||||
|
@ -78,33 +78,23 @@ func NewTagsValuesSequenceKeysValues(keys []string, vals []CountableSequence, op
|
|||
return s
|
||||
}
|
||||
|
||||
func NewTagsValuesSequenceValues(m, f, prefix string, tv []CountableSequence) TagsSequence {
|
||||
keys := make([]string, 0, len(tv)+2)
|
||||
vals := make([]CountableSequence, 0, len(keys))
|
||||
|
||||
keys = append(keys, models.MeasurementTagKey)
|
||||
vals = append(vals, NewStringConstantSequence(m))
|
||||
|
||||
func NewTagsValuesSequenceValues(prefix string, vals []CountableSequence) TagsSequence {
|
||||
keys := make([]string, len(vals))
|
||||
// max tag width
|
||||
tw := int(math.Ceil(math.Log10(float64(len(tv)))))
|
||||
tw := int(math.Ceil(math.Log10(float64(len(vals)))))
|
||||
tf := fmt.Sprintf("%s%%0%dd", prefix, tw)
|
||||
for i := range tv {
|
||||
keys = append(keys, fmt.Sprintf(tf, i))
|
||||
vals = append(vals, tv[i])
|
||||
for i := range vals {
|
||||
keys[i] = fmt.Sprintf(tf, i)
|
||||
}
|
||||
|
||||
keys = append(keys, models.FieldKeyTagKey)
|
||||
vals = append(vals, NewStringConstantSequence(f))
|
||||
|
||||
return NewTagsValuesSequenceKeysValues(keys, vals)
|
||||
}
|
||||
|
||||
func NewTagsValuesSequenceCounts(m, f, prefix string, counts []int) TagsSequence {
|
||||
func NewTagsValuesSequenceCounts(prefix string, counts []int) TagsSequence {
|
||||
tv := make([]CountableSequence, len(counts))
|
||||
for i := range counts {
|
||||
tv[i] = NewCounterByteSequenceCount(counts[i])
|
||||
}
|
||||
return NewTagsValuesSequenceValues(m, f, prefix, tv)
|
||||
return NewTagsValuesSequenceValues(prefix, tv)
|
||||
}
|
||||
|
||||
func (s *tagsValuesSequence) next() bool {
|
||||
|
|
|
@ -7,8 +7,8 @@
|
|||
package gen
|
||||
|
||||
import (
|
||||
"github.com/influxdata/influxdb/v2/models"
|
||||
"github.com/influxdata/influxdb/v2/v1/tsdb/cursors"
|
||||
"github.com/influxdata/influxdb/v2/v1/models"
|
||||
"github.com/influxdata/influxdb/v2/v1/tsdb"
|
||||
)
|
||||
|
||||
type FloatValuesSequence interface {
|
||||
|
@ -26,7 +26,7 @@ type timeFloatValuesSequence struct {
|
|||
|
||||
func NewTimeFloatValuesSequence(count int, ts TimestampSequence, vs FloatValuesSequence) TimeValuesSequence {
|
||||
return &timeFloatValuesSequence{
|
||||
vals: *newFloatArrayLen(cursors.DefaultMaxPointsPerBlock),
|
||||
vals: *newFloatArrayLen(tsdb.DefaultMaxPointsPerBlock),
|
||||
ts: ts,
|
||||
vs: vs,
|
||||
count: count,
|
||||
|
@ -42,7 +42,7 @@ func (s *timeFloatValuesSequence) Reset() {
|
|||
|
||||
func (s *timeFloatValuesSequence) Next() bool {
|
||||
if s.n > 0 {
|
||||
c := min(s.n, cursors.DefaultMaxPointsPerBlock)
|
||||
c := min(s.n, tsdb.DefaultMaxPointsPerBlock)
|
||||
s.n -= c
|
||||
s.vals.Timestamps = s.vals.Timestamps[:c]
|
||||
s.vals.Values = s.vals.Values[:c]
|
||||
|
@ -78,7 +78,7 @@ type timeIntegerValuesSequence struct {
|
|||
|
||||
func NewTimeIntegerValuesSequence(count int, ts TimestampSequence, vs IntegerValuesSequence) TimeValuesSequence {
|
||||
return &timeIntegerValuesSequence{
|
||||
vals: *newIntegerArrayLen(cursors.DefaultMaxPointsPerBlock),
|
||||
vals: *newIntegerArrayLen(tsdb.DefaultMaxPointsPerBlock),
|
||||
ts: ts,
|
||||
vs: vs,
|
||||
count: count,
|
||||
|
@ -94,7 +94,7 @@ func (s *timeIntegerValuesSequence) Reset() {
|
|||
|
||||
func (s *timeIntegerValuesSequence) Next() bool {
|
||||
if s.n > 0 {
|
||||
c := min(s.n, cursors.DefaultMaxPointsPerBlock)
|
||||
c := min(s.n, tsdb.DefaultMaxPointsPerBlock)
|
||||
s.n -= c
|
||||
s.vals.Timestamps = s.vals.Timestamps[:c]
|
||||
s.vals.Values = s.vals.Values[:c]
|
||||
|
@ -130,7 +130,7 @@ type timeUnsignedValuesSequence struct {
|
|||
|
||||
func NewTimeUnsignedValuesSequence(count int, ts TimestampSequence, vs UnsignedValuesSequence) TimeValuesSequence {
|
||||
return &timeUnsignedValuesSequence{
|
||||
vals: *newUnsignedArrayLen(cursors.DefaultMaxPointsPerBlock),
|
||||
vals: *newUnsignedArrayLen(tsdb.DefaultMaxPointsPerBlock),
|
||||
ts: ts,
|
||||
vs: vs,
|
||||
count: count,
|
||||
|
@ -146,7 +146,7 @@ func (s *timeUnsignedValuesSequence) Reset() {
|
|||
|
||||
func (s *timeUnsignedValuesSequence) Next() bool {
|
||||
if s.n > 0 {
|
||||
c := min(s.n, cursors.DefaultMaxPointsPerBlock)
|
||||
c := min(s.n, tsdb.DefaultMaxPointsPerBlock)
|
||||
s.n -= c
|
||||
s.vals.Timestamps = s.vals.Timestamps[:c]
|
||||
s.vals.Values = s.vals.Values[:c]
|
||||
|
@ -182,7 +182,7 @@ type timeStringValuesSequence struct {
|
|||
|
||||
func NewTimeStringValuesSequence(count int, ts TimestampSequence, vs StringValuesSequence) TimeValuesSequence {
|
||||
return &timeStringValuesSequence{
|
||||
vals: *newStringArrayLen(cursors.DefaultMaxPointsPerBlock),
|
||||
vals: *newStringArrayLen(tsdb.DefaultMaxPointsPerBlock),
|
||||
ts: ts,
|
||||
vs: vs,
|
||||
count: count,
|
||||
|
@ -198,7 +198,7 @@ func (s *timeStringValuesSequence) Reset() {
|
|||
|
||||
func (s *timeStringValuesSequence) Next() bool {
|
||||
if s.n > 0 {
|
||||
c := min(s.n, cursors.DefaultMaxPointsPerBlock)
|
||||
c := min(s.n, tsdb.DefaultMaxPointsPerBlock)
|
||||
s.n -= c
|
||||
s.vals.Timestamps = s.vals.Timestamps[:c]
|
||||
s.vals.Values = s.vals.Values[:c]
|
||||
|
@ -234,7 +234,7 @@ type timeBooleanValuesSequence struct {
|
|||
|
||||
func NewTimeBooleanValuesSequence(count int, ts TimestampSequence, vs BooleanValuesSequence) TimeValuesSequence {
|
||||
return &timeBooleanValuesSequence{
|
||||
vals: *newBooleanArrayLen(cursors.DefaultMaxPointsPerBlock),
|
||||
vals: *newBooleanArrayLen(tsdb.DefaultMaxPointsPerBlock),
|
||||
ts: ts,
|
||||
vs: vs,
|
||||
count: count,
|
||||
|
@ -250,7 +250,7 @@ func (s *timeBooleanValuesSequence) Reset() {
|
|||
|
||||
func (s *timeBooleanValuesSequence) Next() bool {
|
||||
if s.n > 0 {
|
||||
c := min(s.n, cursors.DefaultMaxPointsPerBlock)
|
||||
c := min(s.n, tsdb.DefaultMaxPointsPerBlock)
|
||||
s.n -= c
|
||||
s.vals.Timestamps = s.vals.Timestamps[:c]
|
||||
s.vals.Values = s.vals.Values[:c]
|
||||
|
|
|
@ -1,8 +1,8 @@
|
|||
package gen
|
||||
|
||||
import (
|
||||
"github.com/influxdata/influxdb/v2/models"
|
||||
"github.com/influxdata/influxdb/v2/v1/tsdb/cursors"
|
||||
"github.com/influxdata/influxdb/v2/v1/models"
|
||||
"github.com/influxdata/influxdb/v2/v1/tsdb"
|
||||
)
|
||||
|
||||
{{range .}}
|
||||
|
@ -21,7 +21,7 @@ type time{{.Name}}ValuesSequence struct {
|
|||
|
||||
func NewTime{{.Name}}ValuesSequence(count int, ts TimestampSequence, vs {{.Name}}ValuesSequence) TimeValuesSequence {
|
||||
return &time{{.Name}}ValuesSequence{
|
||||
vals: *new{{.Name}}ArrayLen(cursors.DefaultMaxPointsPerBlock),
|
||||
vals: *new{{.Name}}ArrayLen(tsdb.DefaultMaxPointsPerBlock),
|
||||
ts: ts,
|
||||
vs: vs,
|
||||
count: count,
|
||||
|
@ -37,7 +37,7 @@ func (s *time{{.Name}}ValuesSequence) Reset() {
|
|||
|
||||
func (s *time{{.Name}}ValuesSequence) Next() bool {
|
||||
if s.n > 0 {
|
||||
c := min(s.n, cursors.DefaultMaxPointsPerBlock)
|
||||
c := min(s.n, tsdb.DefaultMaxPointsPerBlock)
|
||||
s.n -= c
|
||||
s.vals.Timestamps = s.vals.Timestamps[:c]
|
||||
s.vals.Values = s.vals.Values[:c]
|
||||
|
|
|
@ -1,12 +1,10 @@
|
|||
package storage
|
||||
|
||||
import (
|
||||
"path/filepath"
|
||||
"time"
|
||||
|
||||
"github.com/influxdata/influxdb/v2/toml"
|
||||
"github.com/influxdata/influxdb/v2/v1/tsdb/index/tsi1"
|
||||
"github.com/influxdata/influxdb/v2/v1/tsdb/engine/tsm1"
|
||||
"github.com/influxdata/influxdb/v2/v1/tsdb"
|
||||
)
|
||||
|
||||
// Default configuration values.
|
||||
|
@ -20,67 +18,16 @@ const (
|
|||
|
||||
// Config holds the configuration for an Engine.
|
||||
type Config struct {
|
||||
tsdb.Config
|
||||
|
||||
// Frequency of retention in seconds.
|
||||
RetentionInterval toml.Duration `toml:"retention-interval"`
|
||||
|
||||
// Series file config.
|
||||
SeriesFilePath string `toml:"series-file-path"` // Overrides the default path.
|
||||
|
||||
// Series file config.
|
||||
SeriesFile seriesfile.Config `toml:"tsdb"`
|
||||
|
||||
// WAL config.
|
||||
WAL tsm1.WALConfig `toml:"wal"`
|
||||
WALPath string `toml:"wal-path"` // Overrides the default path.
|
||||
|
||||
// Engine config.
|
||||
Engine tsm1.Config `toml:"engine"`
|
||||
EnginePath string `toml:"engine-path"` // Overrides the default path.
|
||||
|
||||
// Index config.
|
||||
Index tsi1.Config `toml:"index"`
|
||||
IndexPath string `toml:"index-path"` // Overrides the default path.
|
||||
}
|
||||
|
||||
// NewConfig initialises a new config for an Engine.
|
||||
func NewConfig() Config {
|
||||
return Config{
|
||||
Config: tsdb.NewConfig(),
|
||||
RetentionInterval: toml.Duration(DefaultRetentionInterval),
|
||||
SeriesFile: seriesfile.NewConfig(),
|
||||
WAL: tsm1.NewWALConfig(),
|
||||
Engine: tsm1.NewConfig(),
|
||||
Index: tsi1.NewConfig(),
|
||||
}
|
||||
}
|
||||
|
||||
// GetSeriesFilePath returns the path to the series file.
|
||||
func (c Config) GetSeriesFilePath(base string) string {
|
||||
if c.SeriesFilePath != "" {
|
||||
return c.SeriesFilePath
|
||||
}
|
||||
return filepath.Join(base, DefaultSeriesFileDirectoryName)
|
||||
}
|
||||
|
||||
// GetIndexPath returns the path to the index.
|
||||
func (c Config) GetIndexPath(base string) string {
|
||||
if c.IndexPath != "" {
|
||||
return c.IndexPath
|
||||
}
|
||||
return filepath.Join(base, DefaultIndexDirectoryName)
|
||||
}
|
||||
|
||||
// GetWALPath returns the path to the WAL.
|
||||
func (c Config) GetWALPath(base string) string {
|
||||
if c.WALPath != "" {
|
||||
return c.WALPath
|
||||
}
|
||||
return filepath.Join(base, DefaultWALDirectoryName)
|
||||
}
|
||||
|
||||
// GetEnginePath returns the path to the engine.
|
||||
func (c Config) GetEnginePath(base string) string {
|
||||
if c.EnginePath != "" {
|
||||
return c.EnginePath
|
||||
}
|
||||
return filepath.Join(base, DefaultEngineDirectoryName)
|
||||
}
|
||||
|
|
|
@ -18,8 +18,6 @@ import (
|
|||
"github.com/influxdata/influxdb/v2/v1/models"
|
||||
"github.com/influxdata/influxdb/v2/v1/services/meta"
|
||||
"github.com/influxdata/influxdb/v2/v1/tsdb"
|
||||
"github.com/influxdata/influxdb/v2/v1/tsdb/cursors"
|
||||
"github.com/influxdata/influxql"
|
||||
"github.com/pkg/errors"
|
||||
"github.com/prometheus/client_golang/prometheus"
|
||||
"go.uber.org/zap"
|
||||
|
@ -338,30 +336,6 @@ func (e *Engine) Close() error {
|
|||
return nil
|
||||
}
|
||||
|
||||
// CreateSeriesCursor creates a SeriesCursor for usage with the read service.
|
||||
func (e *Engine) CreateSeriesCursor(ctx context.Context, orgID, bucketID influxdb.ID, cond influxql.Expr) (SeriesCursor, error) {
|
||||
e.mu.RLock()
|
||||
defer e.mu.RUnlock()
|
||||
if e.closing == nil {
|
||||
return nil, ErrEngineClosed
|
||||
}
|
||||
|
||||
// return newSeriesCursor(orgID, bucketID, e.index, e.sfile, cond)
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
// CreateCursorIterator creates a CursorIterator for usage with the read service.
|
||||
func (e *Engine) CreateCursorIterator(ctx context.Context) (cursors.CursorIterator, error) {
|
||||
e.mu.RLock()
|
||||
defer e.mu.RUnlock()
|
||||
if e.closing == nil {
|
||||
return nil, ErrEngineClosed
|
||||
}
|
||||
|
||||
// return e.engine.CreateCursorIterator(ctx)
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
// WritePoints writes the provided points to the engine.
|
||||
//
|
||||
// The Engine expects all points to have been correctly validated by the caller.
|
||||
|
|
|
@ -1,84 +0,0 @@
|
|||
package storage
|
||||
|
||||
import (
|
||||
"context"
|
||||
|
||||
"github.com/influxdata/influxdb/v2"
|
||||
"github.com/influxdata/influxdb/v2/v1/tsdb/cursors"
|
||||
"github.com/influxdata/influxql"
|
||||
)
|
||||
|
||||
// MeasurementNames returns an iterator which enumerates the measurements for the given
|
||||
// bucket and limited to the time range [start, end].
|
||||
//
|
||||
// MeasurementNames will always return a StringIterator if there is no error.
|
||||
//
|
||||
// If the context is canceled before MeasurementNames has finished processing, a non-nil
|
||||
// error will be returned along with statistics for the already scanned data.
|
||||
func (e *Engine) MeasurementNames(ctx context.Context, orgID, bucketID influxdb.ID, start, end int64, predicate influxql.Expr) (cursors.StringIterator, error) {
|
||||
e.mu.RLock()
|
||||
defer e.mu.RUnlock()
|
||||
if e.closing == nil {
|
||||
return cursors.EmptyStringIterator, nil
|
||||
}
|
||||
|
||||
// TODO - hook up to store meta api
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
// MeasurementTagValues returns an iterator which enumerates the tag values for the given
|
||||
// bucket, measurement and tag key, filtered using the optional the predicate and limited to the
|
||||
// time range [start, end].
|
||||
//
|
||||
// MeasurementTagValues will always return a StringIterator if there is no error.
|
||||
//
|
||||
// If the context is canceled before TagValues has finished processing, a non-nil
|
||||
// error will be returned along with statistics for the already scanned data.
|
||||
func (e *Engine) MeasurementTagValues(ctx context.Context, orgID, bucketID influxdb.ID, measurement, tagKey string, start, end int64, predicate influxql.Expr) (cursors.StringIterator, error) {
|
||||
e.mu.RLock()
|
||||
defer e.mu.RUnlock()
|
||||
if e.closing == nil {
|
||||
return cursors.EmptyStringIterator, nil
|
||||
}
|
||||
|
||||
// TODO - hook up to store meta api
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
// MeasurementTagKeys returns an iterator which enumerates the tag keys for the given
|
||||
// bucket and measurement, filtered using the optional the predicate and limited to the
|
||||
// time range [start, end].
|
||||
//
|
||||
// MeasurementTagKeys will always return a StringIterator if there is no error.
|
||||
//
|
||||
// If the context is canceled before MeasurementTagKeys has finished processing, a non-nil
|
||||
// error will be returned along with statistics for the already scanned data.
|
||||
func (e *Engine) MeasurementTagKeys(ctx context.Context, orgID, bucketID influxdb.ID, measurement string, start, end int64, predicate influxql.Expr) (cursors.StringIterator, error) {
|
||||
e.mu.RLock()
|
||||
defer e.mu.RUnlock()
|
||||
if e.closing == nil {
|
||||
return cursors.EmptyStringIterator, nil
|
||||
}
|
||||
|
||||
// TODO - hook up to store meta api
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
// MeasurementFields returns an iterator which enumerates the field schema for the given
|
||||
// bucket and measurement, filtered using the optional the predicate and limited to the
|
||||
// time range [start, end].
|
||||
//
|
||||
// MeasurementFields will always return a MeasurementFieldsIterator if there is no error.
|
||||
//
|
||||
// If the context is canceled before MeasurementFields has finished processing, a non-nil
|
||||
// error will be returned along with statistics for the already scanned data.
|
||||
func (e *Engine) MeasurementFields(ctx context.Context, orgID, bucketID influxdb.ID, measurement string, start, end int64, predicate influxql.Expr) (cursors.MeasurementFieldsIterator, error) {
|
||||
e.mu.RLock()
|
||||
defer e.mu.RUnlock()
|
||||
if e.closing == nil {
|
||||
return cursors.EmptyMeasurementFieldsIterator, nil
|
||||
}
|
||||
|
||||
// TODO - hook up to store meta api
|
||||
return nil, nil
|
||||
}
|
|
@ -1,32 +0,0 @@
|
|||
# List any generated files here
|
||||
TARGETS = array_cursor.gen.go
|
||||
|
||||
# List any source files used to generate the targets here
|
||||
SOURCES = gen.go \
|
||||
array_cursor.gen.go.tmpl \
|
||||
array_cursor_test.gen.go.tmpl \
|
||||
array_cursor.gen.go.tmpldata \
|
||||
types.tmpldata
|
||||
|
||||
# List any directories that have their own Makefile here
|
||||
SUBDIRS = datatypes
|
||||
|
||||
# Default target
|
||||
all: $(SUBDIRS) $(TARGETS)
|
||||
|
||||
# Recurse into subdirs for same make goal
|
||||
$(SUBDIRS):
|
||||
$(MAKE) -C $@ $(MAKECMDGOALS)
|
||||
|
||||
# Clean all targets recursively
|
||||
clean: $(SUBDIRS)
|
||||
rm -f $(TARGETS)
|
||||
|
||||
# Define go generate if not already defined
|
||||
GO_GENERATE := go generate
|
||||
|
||||
# Run go generate for the targets
|
||||
$(TARGETS): $(SOURCES)
|
||||
$(GO_GENERATE) -x
|
||||
|
||||
.PHONY: all clean $(SUBDIRS)
|
|
@ -1,93 +0,0 @@
|
|||
package reads
|
||||
|
||||
import (
|
||||
"context"
|
||||
"math"
|
||||
|
||||
"github.com/influxdata/influxdb/v2/kit/errors"
|
||||
"github.com/influxdata/influxdb/v2/kit/tracing"
|
||||
"github.com/influxdata/influxdb/v2/models"
|
||||
"github.com/influxdata/influxdb/v2/storage/reads/datatypes"
|
||||
"github.com/influxdata/influxdb/v2/v1/tsdb/cursors"
|
||||
)
|
||||
|
||||
type windowAggregateResultSet struct {
|
||||
ctx context.Context
|
||||
req *datatypes.ReadWindowAggregateRequest
|
||||
cursor SeriesCursor
|
||||
seriesRow *SeriesRow
|
||||
arrayCursors *arrayCursors
|
||||
}
|
||||
|
||||
func NewWindowAggregateResultSet(ctx context.Context, req *datatypes.ReadWindowAggregateRequest, cursor SeriesCursor) (ResultSet, error) {
|
||||
span, _ := tracing.StartSpanFromContext(ctx)
|
||||
defer span.Finish()
|
||||
|
||||
span.LogKV("aggregate_window_every", req.WindowEvery)
|
||||
for _, aggregate := range req.Aggregate {
|
||||
span.LogKV("aggregate_type", aggregate.String())
|
||||
}
|
||||
|
||||
if nAggs := len(req.Aggregate); nAggs != 1 {
|
||||
return nil, errors.Errorf(errors.InternalError, "attempt to create a windowAggregateResultSet with %v aggregate functions", nAggs)
|
||||
}
|
||||
|
||||
ascending := true
|
||||
|
||||
// The following is an optimization where in the case of a single window,
|
||||
// the selector `last` is implemented as a descending array cursor followed
|
||||
// by a limit array cursor that selects only the first point, i.e the point
|
||||
// with the largest timestamp, from the descending array cursor.
|
||||
//
|
||||
if req.Aggregate[0].Type == datatypes.AggregateTypeLast && (req.WindowEvery == 0 || req.WindowEvery == math.MaxInt64) {
|
||||
ascending = false
|
||||
}
|
||||
|
||||
results := &windowAggregateResultSet{
|
||||
ctx: ctx,
|
||||
req: req,
|
||||
cursor: cursor,
|
||||
arrayCursors: newArrayCursors(ctx, req.Range.Start, req.Range.End, ascending),
|
||||
}
|
||||
return results, nil
|
||||
}
|
||||
|
||||
func (r *windowAggregateResultSet) Next() bool {
|
||||
if r == nil {
|
||||
return false
|
||||
}
|
||||
r.seriesRow = r.cursor.Next()
|
||||
return r.seriesRow != nil
|
||||
}
|
||||
|
||||
func (r *windowAggregateResultSet) Cursor() cursors.Cursor {
|
||||
agg := r.req.Aggregate[0]
|
||||
every := r.req.WindowEvery
|
||||
offset := r.req.Offset
|
||||
cursor := r.arrayCursors.createCursor(*r.seriesRow)
|
||||
|
||||
if every == math.MaxInt64 {
|
||||
// This means to aggregate over whole series for the query's time range
|
||||
return newAggregateArrayCursor(r.ctx, agg, cursor)
|
||||
} else {
|
||||
return newWindowAggregateArrayCursor(r.ctx, agg, every, offset, cursor)
|
||||
}
|
||||
}
|
||||
|
||||
func (r *windowAggregateResultSet) Close() {}
|
||||
|
||||
func (r *windowAggregateResultSet) Err() error { return nil }
|
||||
|
||||
func (r *windowAggregateResultSet) Stats() cursors.CursorStats {
|
||||
if r.seriesRow == nil || r.seriesRow.Query == nil {
|
||||
return cursors.CursorStats{}
|
||||
}
|
||||
return r.seriesRow.Query.Stats()
|
||||
}
|
||||
|
||||
func (r *windowAggregateResultSet) Tags() models.Tags {
|
||||
if r.seriesRow == nil {
|
||||
return models.Tags{}
|
||||
}
|
||||
return r.seriesRow.Tags
|
||||
}
|
|
@ -1,181 +0,0 @@
|
|||
package reads_test
|
||||
|
||||
import (
|
||||
"context"
|
||||
"reflect"
|
||||
"testing"
|
||||
|
||||
"github.com/influxdata/influxdb/v2/models"
|
||||
"github.com/influxdata/influxdb/v2/storage/reads"
|
||||
"github.com/influxdata/influxdb/v2/storage/reads/datatypes"
|
||||
"github.com/influxdata/influxdb/v2/v1/tsdb/cursors"
|
||||
)
|
||||
|
||||
func TestNewWindowAggregateResultSet_Tags(t *testing.T) {
|
||||
|
||||
newCursor := sliceSeriesCursor{
|
||||
rows: newSeriesRows(
|
||||
"clicks click=1 1",
|
||||
)}
|
||||
|
||||
request := datatypes.ReadWindowAggregateRequest{
|
||||
Aggregate: []*datatypes.Aggregate{
|
||||
{
|
||||
Type: datatypes.AggregateTypeCount,
|
||||
},
|
||||
},
|
||||
}
|
||||
resultSet, err := reads.NewWindowAggregateResultSet(context.Background(), &request, &newCursor)
|
||||
|
||||
if err != nil {
|
||||
t.Fatalf("error creating WindowAggregateResultSet: %s", err)
|
||||
}
|
||||
|
||||
// If .Next() was never called, seriesRow is nil and tags are empty.
|
||||
expectedTags := "[]"
|
||||
if resultSet.Tags().String() != expectedTags {
|
||||
t.Errorf("expected tags: %s got: %s", expectedTags, resultSet.Tags().String())
|
||||
}
|
||||
|
||||
resultSet.Next()
|
||||
expectedTags = "[{_m clicks}]"
|
||||
if resultSet.Tags().String() != expectedTags {
|
||||
t.Errorf("expected tags: %s got: %s", expectedTags, resultSet.Tags().String())
|
||||
}
|
||||
}
|
||||
|
||||
type mockIntegerArrayCursor struct {
|
||||
callCount int
|
||||
}
|
||||
|
||||
func (i *mockIntegerArrayCursor) Close() {}
|
||||
func (i *mockIntegerArrayCursor) Err() error { return nil }
|
||||
func (i *mockIntegerArrayCursor) Stats() cursors.CursorStats { return cursors.CursorStats{} }
|
||||
func (i *mockIntegerArrayCursor) Next() *cursors.IntegerArray {
|
||||
if i.callCount == 1 {
|
||||
return &cursors.IntegerArray{}
|
||||
}
|
||||
i.callCount++
|
||||
return &cursors.IntegerArray{
|
||||
Timestamps: []int64{
|
||||
1000000000,
|
||||
1000000005,
|
||||
1000000010,
|
||||
1000000011,
|
||||
1000000012,
|
||||
1000000013,
|
||||
1000000014,
|
||||
1000000020,
|
||||
},
|
||||
Values: []int64{100, 55, 256, 83, 99, 124, 1979, 4, 67, 49929},
|
||||
}
|
||||
}
|
||||
|
||||
type mockCursorIterator struct{}
|
||||
|
||||
func (i *mockCursorIterator) Next(ctx context.Context, req *cursors.CursorRequest) (cursors.Cursor, error) {
|
||||
return &mockIntegerArrayCursor{}, nil
|
||||
}
|
||||
func (i *mockCursorIterator) Stats() cursors.CursorStats {
|
||||
return cursors.CursorStats{ScannedBytes: 500, ScannedValues: 10}
|
||||
}
|
||||
|
||||
type mockReadCursor struct {
|
||||
rows []reads.SeriesRow
|
||||
index int64
|
||||
}
|
||||
|
||||
func newMockReadCursor(keys ...string) mockReadCursor {
|
||||
rows := make([]reads.SeriesRow, len(keys))
|
||||
for i := range keys {
|
||||
rows[i].Name, rows[i].SeriesTags = models.ParseKeyBytes([]byte(keys[i]))
|
||||
rows[i].Tags = rows[i].SeriesTags.Clone()
|
||||
rows[i].Query = &mockCursorIterator{}
|
||||
}
|
||||
|
||||
return mockReadCursor{rows: rows}
|
||||
}
|
||||
|
||||
func (c *mockReadCursor) Next() *reads.SeriesRow {
|
||||
if c.index == int64(len(c.rows)) {
|
||||
return nil
|
||||
}
|
||||
row := c.rows[c.index]
|
||||
c.index++
|
||||
return &row
|
||||
}
|
||||
func (c *mockReadCursor) Close() {}
|
||||
func (c *mockReadCursor) Err() error { return nil }
|
||||
|
||||
// The stats from a WindowAggregateResultSet are retrieved from the cursor.
|
||||
func TestNewWindowAggregateResultSet_Stats(t *testing.T) {
|
||||
|
||||
newCursor := newMockReadCursor(
|
||||
"clicks click=1 1",
|
||||
)
|
||||
|
||||
request := datatypes.ReadWindowAggregateRequest{
|
||||
Aggregate: []*datatypes.Aggregate{
|
||||
{
|
||||
Type: datatypes.AggregateTypeCount,
|
||||
},
|
||||
},
|
||||
}
|
||||
resultSet, err := reads.NewWindowAggregateResultSet(context.Background(), &request, &newCursor)
|
||||
|
||||
if err != nil {
|
||||
t.Fatalf("error creating WindowAggregateResultSet: %s", err)
|
||||
}
|
||||
|
||||
// If .Next() was never called, seriesRow is nil and stats are empty.
|
||||
stats := resultSet.Stats()
|
||||
if stats.ScannedBytes != 0 || stats.ScannedValues != 0 {
|
||||
t.Errorf("expected statistics to be empty")
|
||||
}
|
||||
|
||||
resultSet.Next()
|
||||
stats = resultSet.Stats()
|
||||
if stats.ScannedBytes != 500 {
|
||||
t.Errorf("Expected scanned bytes: %d got: %d", 500, stats.ScannedBytes)
|
||||
}
|
||||
if stats.ScannedValues != 10 {
|
||||
t.Errorf("Expected scanned values: %d got: %d", 10, stats.ScannedValues)
|
||||
}
|
||||
}
|
||||
|
||||
// A count window aggregate is supported
|
||||
func TestNewWindowAggregateResultSet_Count(t *testing.T) {
|
||||
|
||||
newCursor := newMockReadCursor(
|
||||
"clicks click=1 1",
|
||||
)
|
||||
|
||||
request := datatypes.ReadWindowAggregateRequest{
|
||||
Aggregate: []*datatypes.Aggregate{
|
||||
&datatypes.Aggregate{Type: datatypes.AggregateTypeCount},
|
||||
},
|
||||
WindowEvery: 10,
|
||||
}
|
||||
resultSet, err := reads.NewWindowAggregateResultSet(context.Background(), &request, &newCursor)
|
||||
|
||||
if err != nil {
|
||||
t.Fatalf("error creating WindowAggregateResultSet: %s", err)
|
||||
}
|
||||
|
||||
if !resultSet.Next() {
|
||||
t.Fatalf("unexpected: resultSet could not advance")
|
||||
}
|
||||
cursor := resultSet.Cursor()
|
||||
if cursor == nil {
|
||||
t.Fatalf("unexpected: cursor was nil")
|
||||
}
|
||||
integerArrayCursor := cursor.(cursors.IntegerArrayCursor)
|
||||
integerArray := integerArrayCursor.Next()
|
||||
|
||||
if !reflect.DeepEqual(integerArray.Timestamps, []int64{1000000010, 1000000020, 1000000030}) {
|
||||
t.Errorf("unexpected count values: %v", integerArray.Timestamps)
|
||||
}
|
||||
if !reflect.DeepEqual(integerArray.Values, []int64{2, 5, 1}) {
|
||||
t.Errorf("unexpected count values: %v", integerArray.Values)
|
||||
}
|
||||
}
|
File diff suppressed because it is too large
Load Diff
|
@ -1,566 +0,0 @@
|
|||
package reads
|
||||
|
||||
import (
|
||||
"errors"
|
||||
"fmt"
|
||||
"math"
|
||||
|
||||
"github.com/influxdata/influxdb/v2/v1/tsdb/cursors"
|
||||
)
|
||||
|
||||
const (
|
||||
// MaxPointsPerBlock is the maximum number of points in an encoded
|
||||
// block in a TSM file. It should match the value in the tsm1
|
||||
// package, but we don't want to import it.
|
||||
MaxPointsPerBlock = 1000
|
||||
)
|
||||
|
||||
func newLimitArrayCursor(cur cursors.Cursor) cursors.Cursor {
|
||||
switch cur := cur.(type) {
|
||||
{{range .}}{{/* every type supports limit */}}
|
||||
case cursors.{{.Name}}ArrayCursor:
|
||||
return new{{.Name}}LimitArrayCursor(cur)
|
||||
{{end}}
|
||||
default:
|
||||
panic(fmt.Sprintf("unreachable: %T", cur))
|
||||
}
|
||||
}
|
||||
|
||||
func newWindowFirstArrayCursor(cur cursors.Cursor, every, offset int64) cursors.Cursor {
|
||||
if every == 0 {
|
||||
return newLimitArrayCursor(cur)
|
||||
}
|
||||
switch cur := cur.(type) {
|
||||
{{range .}}{{/* every type supports first */}}
|
||||
case cursors.{{.Name}}ArrayCursor:
|
||||
return new{{.Name}}WindowFirstArrayCursor(cur, every, offset)
|
||||
{{end}}
|
||||
default:
|
||||
panic(fmt.Sprintf("unreachable: %T", cur))
|
||||
}
|
||||
}
|
||||
|
||||
func newWindowLastArrayCursor(cur cursors.Cursor, every, offset int64) cursors.Cursor {
|
||||
if every == 0 {
|
||||
return newLimitArrayCursor(cur)
|
||||
}
|
||||
switch cur := cur.(type) {
|
||||
{{range .}}{{/* every type supports last */}}
|
||||
case cursors.{{.Name}}ArrayCursor:
|
||||
return new{{.Name}}WindowLastArrayCursor(cur, every, offset)
|
||||
{{end}}
|
||||
default:
|
||||
panic(fmt.Sprintf("unreachable: %T", cur))
|
||||
}
|
||||
}
|
||||
|
||||
func newWindowCountArrayCursor(cur cursors.Cursor, every, offset int64) cursors.Cursor {
|
||||
switch cur := cur.(type) {
|
||||
{{range .}}{{/* every type supports count */}}
|
||||
case cursors.{{.Name}}ArrayCursor:
|
||||
return new{{.Name}}WindowCountArrayCursor(cur, every, offset)
|
||||
{{end}}
|
||||
default:
|
||||
panic(fmt.Sprintf("unreachable: %T", cur))
|
||||
}
|
||||
}
|
||||
|
||||
func newWindowSumArrayCursor(cur cursors.Cursor, every, offset int64) cursors.Cursor {
|
||||
switch cur := cur.(type) {
|
||||
{{range .}}
|
||||
{{$Type := .Name}}
|
||||
{{range .Aggs}}
|
||||
{{if eq .Name "Sum"}}
|
||||
case cursors.{{$Type}}ArrayCursor:
|
||||
return new{{$Type}}WindowSumArrayCursor(cur, every, offset)
|
||||
{{end}}
|
||||
{{end}}{{/* for each supported agg fn */}}
|
||||
{{end}}{{/* for each field type */}}
|
||||
default:
|
||||
panic(fmt.Sprintf("unsupported for aggregate sum: %T", cur))
|
||||
}
|
||||
}
|
||||
|
||||
func newWindowMinArrayCursor(cur cursors.Cursor, every, offset int64) cursors.Cursor {
|
||||
switch cur := cur.(type) {
|
||||
{{range .}}
|
||||
{{$Type := .Name}}
|
||||
{{range .Aggs}}
|
||||
{{if eq .Name "Min"}}
|
||||
case cursors.{{$Type}}ArrayCursor:
|
||||
return new{{$Type}}WindowMinArrayCursor(cur, every, offset)
|
||||
{{end}}
|
||||
{{end}}{{/* for each supported agg fn */}}
|
||||
{{end}}{{/* for each field type */}}
|
||||
default:
|
||||
panic(fmt.Sprintf("unsupported for aggregate min: %T", cur))
|
||||
}
|
||||
}
|
||||
|
||||
func newWindowMaxArrayCursor(cur cursors.Cursor, every, offset int64) cursors.Cursor {
|
||||
switch cur := cur.(type) {
|
||||
{{range .}}
|
||||
{{$Type := .Name}}
|
||||
{{range .Aggs}}
|
||||
{{if eq .Name "Max"}}
|
||||
case cursors.{{$Type}}ArrayCursor:
|
||||
return new{{$Type}}WindowMaxArrayCursor(cur, every, offset)
|
||||
{{end}}
|
||||
{{end}}{{/* for each supported agg fn */}}
|
||||
{{end}}{{/* for each field type */}}
|
||||
default:
|
||||
panic(fmt.Sprintf("unsupported for aggregate max: %T", cur))
|
||||
}
|
||||
}
|
||||
|
||||
func newWindowMeanArrayCursor(cur cursors.Cursor, every, offset int64) cursors.Cursor {
|
||||
switch cur := cur.(type) {
|
||||
{{range .}}
|
||||
{{$Type := .Name}}
|
||||
{{range .Aggs}}
|
||||
{{if eq .Name "Mean"}}
|
||||
case cursors.{{$Type}}ArrayCursor:
|
||||
return new{{$Type}}WindowMeanArrayCursor(cur, every, offset)
|
||||
{{end}}
|
||||
{{end}}{{/* for each supported agg fn */}}
|
||||
{{end}}{{/* for each field type */}}
|
||||
default:
|
||||
panic(fmt.Sprintf("unsupported for aggregate mean: %T", cur))
|
||||
}
|
||||
}
|
||||
{{range .}}
|
||||
{{$arrayType := print "*cursors." .Name "Array"}}
|
||||
{{$type := print .name "ArrayFilterCursor"}}
|
||||
{{$Type := print .Name "ArrayFilterCursor"}}
|
||||
|
||||
// ********************
|
||||
// {{.Name}} Array Cursor
|
||||
|
||||
type {{$type}} struct {
|
||||
cursors.{{.Name}}ArrayCursor
|
||||
cond expression
|
||||
m *singleValue
|
||||
res {{$arrayType}}
|
||||
tmp {{$arrayType}}
|
||||
}
|
||||
|
||||
func new{{.Name}}FilterArrayCursor(cond expression) *{{$type}} {
|
||||
return &{{$type}}{
|
||||
cond: cond,
|
||||
m: &singleValue{},
|
||||
res: cursors.New{{.Name}}ArrayLen(MaxPointsPerBlock),
|
||||
tmp: &cursors.{{.Name}}Array{},
|
||||
}
|
||||
}
|
||||
|
||||
func (c *{{$type}}) reset(cur cursors.{{.Name}}ArrayCursor) {
|
||||
c.{{.Name}}ArrayCursor = cur
|
||||
c.tmp.Timestamps, c.tmp.Values = nil, nil
|
||||
}
|
||||
|
||||
func (c *{{$type}}) Stats() cursors.CursorStats { return c.{{.Name}}ArrayCursor.Stats() }
|
||||
|
||||
func (c *{{$type}}) Next() {{$arrayType}} {
|
||||
pos := 0
|
||||
c.res.Timestamps = c.res.Timestamps[:cap(c.res.Timestamps)]
|
||||
c.res.Values = c.res.Values[:cap(c.res.Values)]
|
||||
|
||||
var a {{$arrayType}}
|
||||
|
||||
if c.tmp.Len() > 0 {
|
||||
a = c.tmp
|
||||
} else {
|
||||
a = c.{{.Name}}ArrayCursor.Next()
|
||||
}
|
||||
|
||||
LOOP:
|
||||
for len(a.Timestamps) > 0 {
|
||||
for i, v := range a.Values {
|
||||
c.m.v = v
|
||||
if c.cond.EvalBool(c.m) {
|
||||
c.res.Timestamps[pos] = a.Timestamps[i]
|
||||
c.res.Values[pos] = v
|
||||
pos++
|
||||
if pos >= MaxPointsPerBlock {
|
||||
c.tmp.Timestamps = a.Timestamps[i+1:]
|
||||
c.tmp.Values = a.Values[i+1:]
|
||||
break LOOP
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Clear bufferred timestamps & values if we make it through a cursor.
|
||||
// The break above will skip this if a cursor is partially read.
|
||||
c.tmp.Timestamps = nil
|
||||
c.tmp.Values = nil
|
||||
|
||||
a = c.{{.Name}}ArrayCursor.Next()
|
||||
}
|
||||
|
||||
c.res.Timestamps = c.res.Timestamps[:pos]
|
||||
c.res.Values = c.res.Values[:pos]
|
||||
|
||||
return c.res
|
||||
}
|
||||
|
||||
type {{.name}}ArrayCursor struct {
|
||||
cursors.{{.Name}}ArrayCursor
|
||||
cursorContext
|
||||
filter *{{$type}}
|
||||
}
|
||||
|
||||
func (c *{{.name}}ArrayCursor) reset(cur cursors.{{.Name}}ArrayCursor, cursorIterator cursors.CursorIterator, cond expression) {
|
||||
if cond != nil {
|
||||
if c.filter == nil {
|
||||
c.filter = new{{.Name}}FilterArrayCursor(cond)
|
||||
}
|
||||
c.filter.reset(cur)
|
||||
cur = c.filter
|
||||
}
|
||||
|
||||
c.{{.Name}}ArrayCursor = cur
|
||||
c.cursorIterator = cursorIterator
|
||||
c.err = nil
|
||||
}
|
||||
|
||||
|
||||
func (c *{{.name}}ArrayCursor) Err() error { return c.err }
|
||||
|
||||
func (c *{{.name}}ArrayCursor) Stats() cursors.CursorStats {
|
||||
return c.{{.Name}}ArrayCursor.Stats()
|
||||
}
|
||||
|
||||
func (c *{{.name}}ArrayCursor) Next() {{$arrayType}} {
|
||||
for {
|
||||
a := c.{{.Name}}ArrayCursor.Next()
|
||||
if a.Len() == 0 {
|
||||
if c.nextArrayCursor() {
|
||||
continue
|
||||
}
|
||||
}
|
||||
return a
|
||||
}
|
||||
}
|
||||
|
||||
func (c *{{.name}}ArrayCursor) nextArrayCursor() bool {
|
||||
if c.cursorIterator == nil {
|
||||
return false
|
||||
}
|
||||
|
||||
c.{{.Name}}ArrayCursor.Close()
|
||||
|
||||
cur, _ := c.cursorIterator.Next(c.ctx, c.req)
|
||||
c.cursorIterator = nil
|
||||
|
||||
var ok bool
|
||||
if cur != nil {
|
||||
var next cursors.{{.Name}}ArrayCursor
|
||||
next, ok = cur.(cursors.{{.Name}}ArrayCursor)
|
||||
if !ok {
|
||||
cur.Close()
|
||||
next = {{.Name}}EmptyArrayCursor
|
||||
c.cursorIterator = nil
|
||||
c.err = errors.New("expected {{.name}} cursor")
|
||||
} else {
|
||||
if c.filter != nil {
|
||||
c.filter.reset(next)
|
||||
next = c.filter
|
||||
}
|
||||
}
|
||||
c.{{.Name}}ArrayCursor = next
|
||||
} else {
|
||||
c.{{.Name}}ArrayCursor = {{.Name}}EmptyArrayCursor
|
||||
}
|
||||
|
||||
return ok
|
||||
}
|
||||
|
||||
type {{.name}}LimitArrayCursor struct {
|
||||
cursors.{{.Name}}ArrayCursor
|
||||
res {{$arrayType}}
|
||||
done bool
|
||||
}
|
||||
|
||||
func new{{.Name}}LimitArrayCursor(cur cursors.{{.Name}}ArrayCursor) *{{.name}}LimitArrayCursor {
|
||||
return &{{.name}}LimitArrayCursor{
|
||||
{{.Name}}ArrayCursor: cur,
|
||||
res: cursors.New{{.Name}}ArrayLen(1),
|
||||
}
|
||||
}
|
||||
|
||||
func (c *{{.name}}LimitArrayCursor) Stats() cursors.CursorStats { return c.{{.Name}}ArrayCursor.Stats() }
|
||||
|
||||
func (c *{{.name}}LimitArrayCursor) Next() {{$arrayType}} {
|
||||
if c.done {
|
||||
return &cursors.{{.Name}}Array{}
|
||||
}
|
||||
a := c.{{.Name}}ArrayCursor.Next()
|
||||
if len(a.Timestamps) == 0 {
|
||||
return a
|
||||
}
|
||||
c.done = true
|
||||
c.res.Timestamps[0] = a.Timestamps[0]
|
||||
c.res.Values[0] = a.Values[0]
|
||||
return c.res
|
||||
}
|
||||
|
||||
type {{.name}}WindowLastArrayCursor struct {
|
||||
cursors.{{.Name}}ArrayCursor
|
||||
every, offset, windowEnd int64
|
||||
res {{$arrayType}}
|
||||
tmp {{$arrayType}}
|
||||
}
|
||||
|
||||
// Window array cursors assume that every != 0 && every != MaxInt64.
|
||||
// Such a cursor will panic in the first case and possibly overflow in the second.
|
||||
func new{{.Name}}WindowLastArrayCursor(cur cursors.{{.Name}}ArrayCursor, every, offset int64) *{{.name}}WindowLastArrayCursor {
|
||||
return &{{.name}}WindowLastArrayCursor{
|
||||
{{.Name}}ArrayCursor: cur,
|
||||
every: every,
|
||||
offset: offset,
|
||||
windowEnd: math.MinInt64,
|
||||
res: cursors.New{{.Name}}ArrayLen(MaxPointsPerBlock),
|
||||
tmp: &cursors.{{.Name}}Array{},
|
||||
}
|
||||
}
|
||||
|
||||
func (c *{{.name}}WindowLastArrayCursor) Stats() cursors.CursorStats {
|
||||
return c.{{.Name}}ArrayCursor.Stats()
|
||||
}
|
||||
|
||||
func (c *{{.name}}WindowLastArrayCursor) Next() *cursors.{{.Name}}Array {
|
||||
cur := -1
|
||||
|
||||
NEXT:
|
||||
var a *cursors.{{.Name}}Array
|
||||
|
||||
if c.tmp.Len() > 0 {
|
||||
a = c.tmp
|
||||
} else {
|
||||
a = c.{{.Name}}ArrayCursor.Next()
|
||||
}
|
||||
|
||||
if a.Len() == 0 {
|
||||
c.res.Timestamps = c.res.Timestamps[:cur+1]
|
||||
c.res.Values = c.res.Values[:cur+1]
|
||||
return c.res
|
||||
}
|
||||
|
||||
for i, t := range a.Timestamps {
|
||||
if t >= c.windowEnd {
|
||||
cur++
|
||||
}
|
||||
|
||||
if cur == MaxPointsPerBlock {
|
||||
c.tmp.Timestamps = a.Timestamps[i:]
|
||||
c.tmp.Values = a.Values[i:]
|
||||
return c.res
|
||||
}
|
||||
|
||||
c.res.Timestamps[cur] = t
|
||||
c.res.Values[cur] = a.Values[i]
|
||||
|
||||
c.windowEnd = WindowStop(t, c.every, c.offset)
|
||||
}
|
||||
|
||||
c.tmp.Timestamps = nil
|
||||
c.tmp.Values = nil
|
||||
|
||||
goto NEXT
|
||||
}
|
||||
|
||||
type {{.name}}WindowFirstArrayCursor struct {
|
||||
cursors.{{.Name}}ArrayCursor
|
||||
every, offset, windowEnd int64
|
||||
res {{$arrayType}}
|
||||
tmp {{$arrayType}}
|
||||
}
|
||||
|
||||
// Window array cursors assume that every != 0 && every != MaxInt64.
|
||||
// Such a cursor will panic in the first case and possibly overflow in the second.
|
||||
func new{{.Name}}WindowFirstArrayCursor(cur cursors.{{.Name}}ArrayCursor, every, offset int64) *{{.name}}WindowFirstArrayCursor {
|
||||
return &{{.name}}WindowFirstArrayCursor{
|
||||
{{.Name}}ArrayCursor: cur,
|
||||
every: every,
|
||||
offset: offset,
|
||||
windowEnd: math.MinInt64,
|
||||
res: cursors.New{{.Name}}ArrayLen(MaxPointsPerBlock),
|
||||
tmp: &cursors.{{.Name}}Array{},
|
||||
}
|
||||
}
|
||||
|
||||
func (c *{{.name}}WindowFirstArrayCursor) Stats() cursors.CursorStats {
|
||||
return c.{{.Name}}ArrayCursor.Stats()
|
||||
}
|
||||
|
||||
func (c *{{.name}}WindowFirstArrayCursor) Next() *cursors.{{.Name}}Array {
|
||||
c.res.Timestamps = c.res.Timestamps[:0]
|
||||
c.res.Values = c.res.Values[:0]
|
||||
|
||||
NEXT:
|
||||
var a *cursors.{{.Name}}Array
|
||||
|
||||
if c.tmp.Len() > 0 {
|
||||
a = c.tmp
|
||||
} else {
|
||||
a = c.{{.Name}}ArrayCursor.Next()
|
||||
}
|
||||
|
||||
if a.Len() == 0 {
|
||||
return c.res
|
||||
}
|
||||
|
||||
for i, t := range a.Timestamps {
|
||||
if t < c.windowEnd {
|
||||
continue
|
||||
}
|
||||
|
||||
c.windowEnd = WindowStop(t, c.every, c.offset)
|
||||
|
||||
c.res.Timestamps = append(c.res.Timestamps, t)
|
||||
c.res.Values = append(c.res.Values, a.Values[i])
|
||||
|
||||
if c.res.Len() == MaxPointsPerBlock {
|
||||
c.tmp.Timestamps = a.Timestamps[i+1:]
|
||||
c.tmp.Values = a.Values[i+1:]
|
||||
return c.res
|
||||
}
|
||||
}
|
||||
|
||||
c.tmp.Timestamps = nil
|
||||
c.tmp.Values = nil
|
||||
|
||||
goto NEXT
|
||||
}
|
||||
|
||||
{{/* create an aggregate cursor for each aggregate function supported by the type */}}
|
||||
{{$Name := .Name}}
|
||||
{{$name := .name}}
|
||||
{{range .Aggs}}
|
||||
{{$aggName := .Name}}
|
||||
|
||||
type {{$name}}Window{{$aggName}}ArrayCursor struct {
|
||||
cursors.{{$Name}}ArrayCursor
|
||||
every, offset int64
|
||||
res *cursors.{{.OutputTypeName}}Array
|
||||
tmp {{$arrayType}}
|
||||
}
|
||||
|
||||
func new{{$Name}}Window{{$aggName}}ArrayCursor(cur cursors.{{$Name}}ArrayCursor, every, offset int64) *{{$name}}Window{{$aggName}}ArrayCursor {
|
||||
resLen := MaxPointsPerBlock
|
||||
if every == 0 {
|
||||
resLen = 1
|
||||
}
|
||||
return &{{$name}}Window{{$aggName}}ArrayCursor{
|
||||
{{$Name}}ArrayCursor: cur,
|
||||
every: every,
|
||||
offset: offset,
|
||||
res: cursors.New{{.OutputTypeName}}ArrayLen(resLen),
|
||||
tmp: &cursors.{{$Name}}Array{},
|
||||
}
|
||||
}
|
||||
|
||||
func (c *{{$name}}Window{{$aggName}}ArrayCursor) Stats() cursors.CursorStats {
|
||||
return c.{{$Name}}ArrayCursor.Stats()
|
||||
}
|
||||
|
||||
func (c *{{$name}}Window{{$aggName}}ArrayCursor) Next() *cursors.{{.OutputTypeName}}Array {
|
||||
pos := 0
|
||||
c.res.Timestamps = c.res.Timestamps[:cap(c.res.Timestamps)]
|
||||
c.res.Values = c.res.Values[:cap(c.res.Values)]
|
||||
|
||||
var a *cursors.{{$Name}}Array
|
||||
if c.tmp.Len() > 0 {
|
||||
a = c.tmp
|
||||
} else {
|
||||
a = c.{{$Name}}ArrayCursor.Next()
|
||||
}
|
||||
|
||||
if a.Len() == 0 {
|
||||
return &cursors.{{.OutputTypeName}}Array{}
|
||||
}
|
||||
|
||||
rowIdx := 0
|
||||
{{.AccDecls}}
|
||||
|
||||
var windowEnd int64
|
||||
if c.every != 0 {
|
||||
windowEnd = WindowStop(a.Timestamps[rowIdx], c.every, c.offset)
|
||||
} else {
|
||||
windowEnd = math.MaxInt64
|
||||
}
|
||||
|
||||
windowHasPoints := false
|
||||
|
||||
// enumerate windows
|
||||
WINDOWS:
|
||||
for {
|
||||
for ; rowIdx < a.Len(); rowIdx++ {
|
||||
ts := a.Timestamps[rowIdx]
|
||||
if c.every != 0 && ts >= windowEnd {
|
||||
// new window detected, close the current window
|
||||
// do not generate a point for empty windows
|
||||
if windowHasPoints {
|
||||
{{.AccEmit}}
|
||||
pos++
|
||||
if pos >= MaxPointsPerBlock {
|
||||
// the output array is full,
|
||||
// save the remaining points in the input array in tmp.
|
||||
// they will be processed in the next call to Next()
|
||||
c.tmp.Timestamps = a.Timestamps[rowIdx:]
|
||||
c.tmp.Values = a.Values[rowIdx:]
|
||||
break WINDOWS
|
||||
}
|
||||
}
|
||||
|
||||
// start the new window
|
||||
{{.AccReset}}
|
||||
windowEnd = WindowStop(a.Timestamps[rowIdx], c.every, c.offset)
|
||||
windowHasPoints = false
|
||||
|
||||
continue WINDOWS
|
||||
} else {
|
||||
{{.Accumulate}}
|
||||
windowHasPoints = true
|
||||
}
|
||||
}
|
||||
|
||||
// Clear buffered timestamps & values if we make it through a cursor.
|
||||
// The break above will skip this if a cursor is partially read.
|
||||
c.tmp.Timestamps = nil
|
||||
c.tmp.Values = nil
|
||||
|
||||
// get the next chunk
|
||||
a = c.{{$Name}}ArrayCursor.Next()
|
||||
if a.Len() == 0 {
|
||||
// write the final point
|
||||
// do not generate a point for empty windows
|
||||
if windowHasPoints {
|
||||
{{.AccEmit}}
|
||||
pos++
|
||||
}
|
||||
break WINDOWS
|
||||
}
|
||||
rowIdx = 0
|
||||
}
|
||||
|
||||
c.res.Timestamps = c.res.Timestamps[:pos]
|
||||
c.res.Values = c.res.Values[:pos]
|
||||
|
||||
return c.res
|
||||
}
|
||||
|
||||
{{end}}{{/* range .Aggs */}}
|
||||
|
||||
type {{.name}}EmptyArrayCursor struct {
|
||||
res cursors.{{.Name}}Array
|
||||
}
|
||||
|
||||
var {{.Name}}EmptyArrayCursor cursors.{{.Name}}ArrayCursor = &{{.name}}EmptyArrayCursor{}
|
||||
|
||||
func (c *{{.name}}EmptyArrayCursor) Err() error { return nil }
|
||||
func (c *{{.name}}EmptyArrayCursor) Close() {}
|
||||
func (c *{{.name}}EmptyArrayCursor) Stats() cursors.CursorStats { return cursors.CursorStats{} }
|
||||
func (c *{{.name}}EmptyArrayCursor) Next() {{$arrayType}} { return &c.res }
|
||||
|
||||
{{end}}{{/* range . */}}
|
|
@ -1,173 +0,0 @@
|
|||
[
|
||||
{
|
||||
"Name":"Float",
|
||||
"name":"float",
|
||||
"Type":"float64",
|
||||
"Aggs": [
|
||||
{
|
||||
"Name":"Count",
|
||||
"OutputTypeName":"Integer",
|
||||
"AccDecls":"var acc int64 = 0",
|
||||
"Accumulate":"acc++",
|
||||
"AccEmit": "c.res.Timestamps[pos] = windowEnd; c.res.Values[pos] = acc",
|
||||
"AccReset":"acc = 0"
|
||||
},
|
||||
{
|
||||
"Name":"Sum",
|
||||
"OutputTypeName":"Float",
|
||||
"AccDecls":"var acc float64 = 0",
|
||||
"Accumulate":"acc += a.Values[rowIdx]",
|
||||
"AccEmit":"c.res.Timestamps[pos] = windowEnd; c.res.Values[pos] = acc",
|
||||
"AccReset":"acc = 0"
|
||||
},
|
||||
{
|
||||
"Name":"Min",
|
||||
"OutputTypeName":"Float",
|
||||
"AccDecls":"var acc float64 = math.MaxFloat64; var tsAcc int64",
|
||||
"Accumulate":"if !windowHasPoints || a.Values[rowIdx] < acc { acc = a.Values[rowIdx]; tsAcc = a.Timestamps[rowIdx] }",
|
||||
"AccEmit":"c.res.Timestamps[pos] = tsAcc; c.res.Values[pos] = acc",
|
||||
"AccReset":"acc = math.MaxFloat64"
|
||||
},
|
||||
{
|
||||
"Name":"Max",
|
||||
"OutputTypeName":"Float",
|
||||
"AccDecls":"var acc float64 = -math.MaxFloat64; var tsAcc int64",
|
||||
"Accumulate":"if !windowHasPoints || a.Values[rowIdx] > acc { acc = a.Values[rowIdx]; tsAcc = a.Timestamps[rowIdx] }",
|
||||
"AccEmit":"c.res.Timestamps[pos] = tsAcc; c.res.Values[pos] = acc",
|
||||
"AccReset":"acc = -math.MaxFloat64"
|
||||
},
|
||||
{
|
||||
"Name":"Mean",
|
||||
"OutputTypeName":"Float",
|
||||
"AccDecls":"var sum float64; var count int64",
|
||||
"Accumulate":"sum += a.Values[rowIdx]; count++",
|
||||
"AccEmit":"c.res.Timestamps[pos] = windowEnd; c.res.Values[pos] = sum / float64(count)",
|
||||
"AccReset":"sum = 0; count = 0"
|
||||
}
|
||||
]
|
||||
},
|
||||
{
|
||||
"Name":"Integer",
|
||||
"name":"integer",
|
||||
"Type":"int64",
|
||||
"Aggs": [
|
||||
{
|
||||
"Name":"Count",
|
||||
"OutputTypeName":"Integer",
|
||||
"AccDecls":"var acc int64 = 0",
|
||||
"Accumulate":"acc++",
|
||||
"AccEmit": "c.res.Timestamps[pos] = windowEnd; c.res.Values[pos] = acc",
|
||||
"AccReset":"acc = 0"
|
||||
},
|
||||
{
|
||||
"Name":"Sum",
|
||||
"OutputTypeName":"Integer",
|
||||
"AccDecls":"var acc int64 = 0",
|
||||
"Accumulate":"acc += a.Values[rowIdx]",
|
||||
"AccEmit":"c.res.Timestamps[pos] = windowEnd; c.res.Values[pos] = acc",
|
||||
"AccReset":"acc = 0"
|
||||
},
|
||||
{
|
||||
"Name":"Min",
|
||||
"OutputTypeName":"Integer",
|
||||
"AccDecls":"var acc int64 = math.MaxInt64; var tsAcc int64",
|
||||
"Accumulate":"if !windowHasPoints || a.Values[rowIdx] < acc { acc = a.Values[rowIdx]; tsAcc = a.Timestamps[rowIdx] }",
|
||||
"AccEmit":"c.res.Timestamps[pos] = tsAcc; c.res.Values[pos] = acc",
|
||||
"AccReset":"acc = math.MaxInt64"
|
||||
},
|
||||
{
|
||||
"Name":"Max",
|
||||
"OutputTypeName":"Integer",
|
||||
"AccDecls":"var acc int64 = math.MinInt64; var tsAcc int64",
|
||||
"Accumulate":"if !windowHasPoints || a.Values[rowIdx] > acc { acc = a.Values[rowIdx]; tsAcc = a.Timestamps[rowIdx] }",
|
||||
"AccEmit":"c.res.Timestamps[pos] = tsAcc; c.res.Values[pos] = acc",
|
||||
"AccReset":"acc = math.MinInt64"
|
||||
},
|
||||
{
|
||||
"Name":"Mean",
|
||||
"OutputTypeName":"Float",
|
||||
"AccDecls":"var sum int64; var count int64",
|
||||
"Accumulate":"sum += a.Values[rowIdx]; count++",
|
||||
"AccEmit":"c.res.Timestamps[pos] = windowEnd; c.res.Values[pos] = float64(sum) / float64(count)",
|
||||
"AccReset":"sum = 0; count = 0"
|
||||
}
|
||||
]
|
||||
},
|
||||
{
|
||||
"Name":"Unsigned",
|
||||
"name":"unsigned",
|
||||
"Type":"uint64",
|
||||
"Aggs": [
|
||||
{
|
||||
"Name":"Count",
|
||||
"OutputTypeName":"Integer",
|
||||
"AccDecls":"var acc int64 = 0",
|
||||
"Accumulate":"acc++",
|
||||
"AccEmit": "c.res.Timestamps[pos] = windowEnd; c.res.Values[pos] = acc",
|
||||
"AccReset":"acc = 0"
|
||||
},
|
||||
{
|
||||
"Name":"Sum",
|
||||
"OutputTypeName":"Unsigned",
|
||||
"AccDecls":"var acc uint64 = 0",
|
||||
"Accumulate":"acc += a.Values[rowIdx]",
|
||||
"AccEmit":"c.res.Timestamps[pos] = windowEnd; c.res.Values[pos] = acc",
|
||||
"AccReset":"acc = 0"
|
||||
},
|
||||
{
|
||||
"Name":"Min",
|
||||
"OutputTypeName":"Unsigned",
|
||||
"AccDecls":"var acc uint64 = math.MaxUint64; var tsAcc int64",
|
||||
"Accumulate":"if !windowHasPoints || a.Values[rowIdx] < acc { acc = a.Values[rowIdx]; tsAcc = a.Timestamps[rowIdx] }",
|
||||
"AccEmit":"c.res.Timestamps[pos] = tsAcc; c.res.Values[pos] = acc",
|
||||
"AccReset":"acc = math.MaxUint64"
|
||||
},
|
||||
{
|
||||
"Name":"Max",
|
||||
"OutputTypeName":"Unsigned",
|
||||
"AccDecls":"var acc uint64 = 0; var tsAcc int64",
|
||||
"Accumulate":"if !windowHasPoints || a.Values[rowIdx] > acc { acc = a.Values[rowIdx]; tsAcc = a.Timestamps[rowIdx] }",
|
||||
"AccEmit":"c.res.Timestamps[pos] = tsAcc; c.res.Values[pos] = acc",
|
||||
"AccReset":"acc = 0"
|
||||
},
|
||||
{
|
||||
"Name":"Mean",
|
||||
"OutputTypeName":"Float",
|
||||
"AccDecls":"var sum uint64; var count int64",
|
||||
"Accumulate":"sum += a.Values[rowIdx]; count++",
|
||||
"AccEmit":"c.res.Timestamps[pos] = windowEnd; c.res.Values[pos] = float64(sum) / float64(count)",
|
||||
"AccReset":"sum = 0; count = 0"
|
||||
}
|
||||
]
|
||||
},
|
||||
{
|
||||
"Name":"String",
|
||||
"name":"string",
|
||||
"Type":"string",
|
||||
"Aggs": [
|
||||
{
|
||||
"Name":"Count",
|
||||
"OutputTypeName":"Integer",
|
||||
"AccDecls":"var acc int64 = 0",
|
||||
"Accumulate":"acc++",
|
||||
"AccEmit": "c.res.Timestamps[pos] = windowEnd; c.res.Values[pos] = acc",
|
||||
"AccReset":"acc = 0"
|
||||
}
|
||||
]
|
||||
},
|
||||
{
|
||||
"Name":"Boolean",
|
||||
"name":"boolean",
|
||||
"Type":"bool",
|
||||
"Aggs": [
|
||||
{
|
||||
"Name":"Count",
|
||||
"OutputTypeName":"Integer",
|
||||
"AccDecls":"var acc int64 = 0",
|
||||
"Accumulate":"acc++",
|
||||
"AccEmit": "c.res.Timestamps[pos] = windowEnd; c.res.Values[pos] = acc",
|
||||
"AccReset":"acc = 0"
|
||||
}
|
||||
]
|
||||
}
|
||||
]
|
|
@ -17,62 +17,83 @@ func (v *singleValue) Value(key string) (interface{}, bool) {
|
|||
}
|
||||
|
||||
func newAggregateArrayCursor(ctx context.Context, agg *datatypes.Aggregate, cursor cursors.Cursor) cursors.Cursor {
|
||||
switch agg.Type {
|
||||
case datatypes.AggregateTypeFirst, datatypes.AggregateTypeLast:
|
||||
return newLimitArrayCursor(cursor)
|
||||
}
|
||||
return newWindowAggregateArrayCursor(ctx, agg, 0, 0, cursor)
|
||||
}
|
||||
|
||||
func newWindowAggregateArrayCursor(ctx context.Context, agg *datatypes.Aggregate, every, offset int64, cursor cursors.Cursor) cursors.Cursor {
|
||||
if cursor == nil {
|
||||
return nil
|
||||
}
|
||||
|
||||
switch agg.Type {
|
||||
case datatypes.AggregateTypeCount:
|
||||
return newWindowCountArrayCursor(cursor, every, offset)
|
||||
case datatypes.AggregateTypeSum:
|
||||
return newWindowSumArrayCursor(cursor, every, offset)
|
||||
case datatypes.AggregateTypeFirst:
|
||||
return newWindowFirstArrayCursor(cursor, every, offset)
|
||||
case datatypes.AggregateTypeLast:
|
||||
return newWindowLastArrayCursor(cursor, every, offset)
|
||||
case datatypes.AggregateTypeMin:
|
||||
return newWindowMinArrayCursor(cursor, every, offset)
|
||||
case datatypes.AggregateTypeMax:
|
||||
return newWindowMaxArrayCursor(cursor, every, offset)
|
||||
case datatypes.AggregateTypeMean:
|
||||
return newWindowMeanArrayCursor(cursor, every, offset)
|
||||
return newSumArrayCursor(cursor)
|
||||
case datatypes.AggregateTypeCount:
|
||||
return newCountArrayCursor(cursor)
|
||||
default:
|
||||
// TODO(sgc): should be validated higher up
|
||||
panic("invalid aggregate")
|
||||
}
|
||||
}
|
||||
|
||||
type cursorContext struct {
|
||||
ctx context.Context
|
||||
req *cursors.CursorRequest
|
||||
cursorIterator cursors.CursorIterator
|
||||
err error
|
||||
}
|
||||
|
||||
type arrayCursors struct {
|
||||
ctx context.Context
|
||||
req cursors.CursorRequest
|
||||
|
||||
cursors struct {
|
||||
i integerArrayCursor
|
||||
f floatArrayCursor
|
||||
u unsignedArrayCursor
|
||||
b booleanArrayCursor
|
||||
s stringArrayCursor
|
||||
func newSumArrayCursor(cur cursors.Cursor) cursors.Cursor {
|
||||
switch cur := cur.(type) {
|
||||
case cursors.FloatArrayCursor:
|
||||
return newFloatArraySumCursor(cur)
|
||||
case cursors.IntegerArrayCursor:
|
||||
return newIntegerArraySumCursor(cur)
|
||||
case cursors.UnsignedArrayCursor:
|
||||
return newUnsignedArraySumCursor(cur)
|
||||
default:
|
||||
// TODO(sgc): propagate an error instead?
|
||||
return nil
|
||||
}
|
||||
}
|
||||
|
||||
func newArrayCursors(ctx context.Context, start, end int64, asc bool) *arrayCursors {
|
||||
m := &arrayCursors{
|
||||
ctx: ctx,
|
||||
func newCountArrayCursor(cur cursors.Cursor) cursors.Cursor {
|
||||
switch cur := cur.(type) {
|
||||
case cursors.FloatArrayCursor:
|
||||
return &integerFloatCountArrayCursor{FloatArrayCursor: cur}
|
||||
case cursors.IntegerArrayCursor:
|
||||
return &integerIntegerCountArrayCursor{IntegerArrayCursor: cur}
|
||||
case cursors.UnsignedArrayCursor:
|
||||
return &integerUnsignedCountArrayCursor{UnsignedArrayCursor: cur}
|
||||
case cursors.StringArrayCursor:
|
||||
return &integerStringCountArrayCursor{StringArrayCursor: cur}
|
||||
case cursors.BooleanArrayCursor:
|
||||
return &integerBooleanCountArrayCursor{BooleanArrayCursor: cur}
|
||||
default:
|
||||
panic(fmt.Sprintf("unreachable: %T", cur))
|
||||
}
|
||||
}
|
||||
|
||||
type cursorContext struct {
|
||||
ctx context.Context
|
||||
req *cursors.CursorRequest
|
||||
itrs cursors.CursorIterators
|
||||
limit int64
|
||||
count int64
|
||||
err error
|
||||
}
|
||||
|
||||
type multiShardArrayCursors struct {
|
||||
ctx context.Context
|
||||
limit int64
|
||||
req cursors.CursorRequest
|
||||
|
||||
cursors struct {
|
||||
i integerMultiShardArrayCursor
|
||||
f floatMultiShardArrayCursor
|
||||
u unsignedMultiShardArrayCursor
|
||||
b booleanMultiShardArrayCursor
|
||||
s stringMultiShardArrayCursor
|
||||
}
|
||||
}
|
||||
|
||||
func newMultiShardArrayCursors(ctx context.Context, start, end int64, asc bool, limit int64) *multiShardArrayCursors {
|
||||
if limit < 0 {
|
||||
limit = 1
|
||||
}
|
||||
|
||||
m := &multiShardArrayCursors{
|
||||
ctx: ctx,
|
||||
limit: limit,
|
||||
req: cursors.CursorRequest{
|
||||
Ascending: asc,
|
||||
StartTime: start,
|
||||
|
@ -81,8 +102,9 @@ func newArrayCursors(ctx context.Context, start, end int64, asc bool) *arrayCurs
|
|||
}
|
||||
|
||||
cc := cursorContext{
|
||||
ctx: ctx,
|
||||
req: &m.req,
|
||||
ctx: ctx,
|
||||
limit: limit,
|
||||
req: &m.req,
|
||||
}
|
||||
|
||||
m.cursors.i.cursorContext = cc
|
||||
|
@ -94,42 +116,48 @@ func newArrayCursors(ctx context.Context, start, end int64, asc bool) *arrayCurs
|
|||
return m
|
||||
}
|
||||
|
||||
func (m *arrayCursors) createCursor(seriesRow SeriesRow) cursors.Cursor {
|
||||
m.req.Name = seriesRow.Name
|
||||
m.req.Tags = seriesRow.SeriesTags
|
||||
m.req.Field = seriesRow.Field
|
||||
func (m *multiShardArrayCursors) createCursor(row SeriesRow) cursors.Cursor {
|
||||
m.req.Name = row.Name
|
||||
m.req.Tags = row.SeriesTags
|
||||
m.req.Field = row.Field
|
||||
|
||||
var cond expression
|
||||
if seriesRow.ValueCond != nil {
|
||||
cond = &astExpr{seriesRow.ValueCond}
|
||||
if row.ValueCond != nil {
|
||||
cond = &astExpr{row.ValueCond}
|
||||
}
|
||||
|
||||
if seriesRow.Query == nil {
|
||||
return nil
|
||||
var shard cursors.CursorIterator
|
||||
var cur cursors.Cursor
|
||||
for cur == nil && len(row.Query) > 0 {
|
||||
shard, row.Query = row.Query[0], row.Query[1:]
|
||||
cur, _ = shard.Next(m.ctx, &m.req)
|
||||
}
|
||||
cur, _ := seriesRow.Query.Next(m.ctx, &m.req)
|
||||
seriesRow.Query = nil
|
||||
|
||||
if cur == nil {
|
||||
return nil
|
||||
}
|
||||
|
||||
switch c := cur.(type) {
|
||||
case cursors.IntegerArrayCursor:
|
||||
m.cursors.i.reset(c, seriesRow.Query, cond)
|
||||
m.cursors.i.reset(c, row.Query, cond)
|
||||
return &m.cursors.i
|
||||
case cursors.FloatArrayCursor:
|
||||
m.cursors.f.reset(c, seriesRow.Query, cond)
|
||||
m.cursors.f.reset(c, row.Query, cond)
|
||||
return &m.cursors.f
|
||||
case cursors.UnsignedArrayCursor:
|
||||
m.cursors.u.reset(c, seriesRow.Query, cond)
|
||||
m.cursors.u.reset(c, row.Query, cond)
|
||||
return &m.cursors.u
|
||||
case cursors.StringArrayCursor:
|
||||
m.cursors.s.reset(c, seriesRow.Query, cond)
|
||||
m.cursors.s.reset(c, row.Query, cond)
|
||||
return &m.cursors.s
|
||||
case cursors.BooleanArrayCursor:
|
||||
m.cursors.b.reset(c, seriesRow.Query, cond)
|
||||
m.cursors.b.reset(c, row.Query, cond)
|
||||
return &m.cursors.b
|
||||
default:
|
||||
panic(fmt.Sprintf("unreachable: %T", cur))
|
||||
}
|
||||
}
|
||||
|
||||
func (m *multiShardArrayCursors) newAggregateCursor(ctx context.Context, agg *datatypes.Aggregate, cursor cursors.Cursor) cursors.Cursor {
|
||||
return newAggregateArrayCursor(ctx, agg, cursor)
|
||||
}
|
||||
|
|
|
@ -1,746 +0,0 @@
|
|||
// Generated by tmpl
|
||||
// https://github.com/benbjohnson/tmpl
|
||||
//
|
||||
// DO NOT EDIT!
|
||||
// Source: array_cursor_test.gen.go.tmpl
|
||||
|
||||
package reads
|
||||
|
||||
import (
|
||||
"context"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/google/go-cmp/cmp"
|
||||
"github.com/influxdata/influxdb/v2/storage/reads/datatypes"
|
||||
"github.com/influxdata/influxdb/v2/tsdb/cursors"
|
||||
)
|
||||
|
||||
type MockFloatArrayCursor struct {
|
||||
CloseFunc func()
|
||||
ErrFunc func() error
|
||||
StatsFunc func() cursors.CursorStats
|
||||
NextFunc func() *cursors.FloatArray
|
||||
}
|
||||
|
||||
func (c *MockFloatArrayCursor) Close() { c.CloseFunc() }
|
||||
func (c *MockFloatArrayCursor) Err() error { return c.ErrFunc() }
|
||||
func (c *MockFloatArrayCursor) Stats() cursors.CursorStats { return c.StatsFunc() }
|
||||
func (c *MockFloatArrayCursor) Next() *cursors.FloatArray { return c.NextFunc() }
|
||||
|
||||
func TestNewAggregateArrayCursor_Float(t *testing.T) {
|
||||
|
||||
t.Run("Count", func(t *testing.T) {
|
||||
want := &floatWindowCountArrayCursor{
|
||||
FloatArrayCursor: &MockFloatArrayCursor{},
|
||||
res: cursors.NewIntegerArrayLen(1),
|
||||
tmp: &cursors.FloatArray{},
|
||||
}
|
||||
|
||||
agg := &datatypes.Aggregate{
|
||||
Type: datatypes.AggregateTypeCount,
|
||||
}
|
||||
|
||||
got := newAggregateArrayCursor(context.Background(), agg, &MockFloatArrayCursor{})
|
||||
|
||||
if diff := cmp.Diff(got, want, cmp.AllowUnexported(floatWindowCountArrayCursor{})); diff != "" {
|
||||
t.Fatalf("did not get expected cursor; -got/+want:\n%v", diff)
|
||||
}
|
||||
})
|
||||
|
||||
t.Run("Sum", func(t *testing.T) {
|
||||
want := &floatWindowSumArrayCursor{
|
||||
FloatArrayCursor: &MockFloatArrayCursor{},
|
||||
res: cursors.NewFloatArrayLen(1),
|
||||
tmp: &cursors.FloatArray{},
|
||||
}
|
||||
|
||||
agg := &datatypes.Aggregate{
|
||||
Type: datatypes.AggregateTypeSum,
|
||||
}
|
||||
|
||||
got := newAggregateArrayCursor(context.Background(), agg, &MockFloatArrayCursor{})
|
||||
|
||||
if diff := cmp.Diff(got, want, cmp.AllowUnexported(floatWindowSumArrayCursor{})); diff != "" {
|
||||
t.Fatalf("did not get expected cursor; -got/+want:\n%v", diff)
|
||||
}
|
||||
})
|
||||
|
||||
t.Run("Min", func(t *testing.T) {
|
||||
want := &floatWindowMinArrayCursor{
|
||||
FloatArrayCursor: &MockFloatArrayCursor{},
|
||||
res: cursors.NewFloatArrayLen(1),
|
||||
tmp: &cursors.FloatArray{},
|
||||
}
|
||||
|
||||
agg := &datatypes.Aggregate{
|
||||
Type: datatypes.AggregateTypeMin,
|
||||
}
|
||||
|
||||
got := newAggregateArrayCursor(context.Background(), agg, &MockFloatArrayCursor{})
|
||||
|
||||
if diff := cmp.Diff(got, want, cmp.AllowUnexported(floatWindowMinArrayCursor{})); diff != "" {
|
||||
t.Fatalf("did not get expected cursor; -got/+want:\n%v", diff)
|
||||
}
|
||||
})
|
||||
|
||||
t.Run("Max", func(t *testing.T) {
|
||||
want := &floatWindowMaxArrayCursor{
|
||||
FloatArrayCursor: &MockFloatArrayCursor{},
|
||||
res: cursors.NewFloatArrayLen(1),
|
||||
tmp: &cursors.FloatArray{},
|
||||
}
|
||||
|
||||
agg := &datatypes.Aggregate{
|
||||
Type: datatypes.AggregateTypeMax,
|
||||
}
|
||||
|
||||
got := newAggregateArrayCursor(context.Background(), agg, &MockFloatArrayCursor{})
|
||||
|
||||
if diff := cmp.Diff(got, want, cmp.AllowUnexported(floatWindowMaxArrayCursor{})); diff != "" {
|
||||
t.Fatalf("did not get expected cursor; -got/+want:\n%v", diff)
|
||||
}
|
||||
})
|
||||
|
||||
t.Run("Mean", func(t *testing.T) {
|
||||
want := &floatWindowMeanArrayCursor{
|
||||
FloatArrayCursor: &MockFloatArrayCursor{},
|
||||
res: cursors.NewFloatArrayLen(1),
|
||||
tmp: &cursors.FloatArray{},
|
||||
}
|
||||
|
||||
agg := &datatypes.Aggregate{
|
||||
Type: datatypes.AggregateTypeMean,
|
||||
}
|
||||
|
||||
got := newAggregateArrayCursor(context.Background(), agg, &MockFloatArrayCursor{})
|
||||
|
||||
if diff := cmp.Diff(got, want, cmp.AllowUnexported(floatWindowMeanArrayCursor{})); diff != "" {
|
||||
t.Fatalf("did not get expected cursor; -got/+want:\n%v", diff)
|
||||
}
|
||||
})
|
||||
|
||||
}
|
||||
|
||||
func TestNewWindowAggregateArrayCursor_Float(t *testing.T) {
|
||||
|
||||
t.Run("Count", func(t *testing.T) {
|
||||
want := &floatWindowCountArrayCursor{
|
||||
FloatArrayCursor: &MockFloatArrayCursor{},
|
||||
every: int64(time.Hour),
|
||||
res: cursors.NewIntegerArrayLen(MaxPointsPerBlock),
|
||||
tmp: &cursors.FloatArray{},
|
||||
}
|
||||
|
||||
agg := &datatypes.Aggregate{
|
||||
Type: datatypes.AggregateTypeCount,
|
||||
}
|
||||
|
||||
got := newWindowAggregateArrayCursor(context.Background(), agg, int64(time.Hour), 0, &MockFloatArrayCursor{})
|
||||
|
||||
if diff := cmp.Diff(got, want, cmp.AllowUnexported(floatWindowCountArrayCursor{})); diff != "" {
|
||||
t.Fatalf("did not get expected cursor; -got/+want:\n%v", diff)
|
||||
}
|
||||
})
|
||||
|
||||
t.Run("Sum", func(t *testing.T) {
|
||||
want := &floatWindowSumArrayCursor{
|
||||
FloatArrayCursor: &MockFloatArrayCursor{},
|
||||
every: int64(time.Hour),
|
||||
res: cursors.NewFloatArrayLen(MaxPointsPerBlock),
|
||||
tmp: &cursors.FloatArray{},
|
||||
}
|
||||
|
||||
agg := &datatypes.Aggregate{
|
||||
Type: datatypes.AggregateTypeSum,
|
||||
}
|
||||
|
||||
got := newWindowAggregateArrayCursor(context.Background(), agg, int64(time.Hour), 0, &MockFloatArrayCursor{})
|
||||
|
||||
if diff := cmp.Diff(got, want, cmp.AllowUnexported(floatWindowSumArrayCursor{})); diff != "" {
|
||||
t.Fatalf("did not get expected cursor; -got/+want:\n%v", diff)
|
||||
}
|
||||
})
|
||||
|
||||
t.Run("Min", func(t *testing.T) {
|
||||
want := &floatWindowMinArrayCursor{
|
||||
FloatArrayCursor: &MockFloatArrayCursor{},
|
||||
every: int64(time.Hour),
|
||||
res: cursors.NewFloatArrayLen(MaxPointsPerBlock),
|
||||
tmp: &cursors.FloatArray{},
|
||||
}
|
||||
|
||||
agg := &datatypes.Aggregate{
|
||||
Type: datatypes.AggregateTypeMin,
|
||||
}
|
||||
|
||||
got := newWindowAggregateArrayCursor(context.Background(), agg, int64(time.Hour), 0, &MockFloatArrayCursor{})
|
||||
|
||||
if diff := cmp.Diff(got, want, cmp.AllowUnexported(floatWindowMinArrayCursor{})); diff != "" {
|
||||
t.Fatalf("did not get expected cursor; -got/+want:\n%v", diff)
|
||||
}
|
||||
})
|
||||
|
||||
t.Run("Max", func(t *testing.T) {
|
||||
want := &floatWindowMaxArrayCursor{
|
||||
FloatArrayCursor: &MockFloatArrayCursor{},
|
||||
every: int64(time.Hour),
|
||||
res: cursors.NewFloatArrayLen(MaxPointsPerBlock),
|
||||
tmp: &cursors.FloatArray{},
|
||||
}
|
||||
|
||||
agg := &datatypes.Aggregate{
|
||||
Type: datatypes.AggregateTypeMax,
|
||||
}
|
||||
|
||||
got := newWindowAggregateArrayCursor(context.Background(), agg, int64(time.Hour), 0, &MockFloatArrayCursor{})
|
||||
|
||||
if diff := cmp.Diff(got, want, cmp.AllowUnexported(floatWindowMaxArrayCursor{})); diff != "" {
|
||||
t.Fatalf("did not get expected cursor; -got/+want:\n%v", diff)
|
||||
}
|
||||
})
|
||||
|
||||
t.Run("Mean", func(t *testing.T) {
|
||||
want := &floatWindowMeanArrayCursor{
|
||||
FloatArrayCursor: &MockFloatArrayCursor{},
|
||||
every: int64(time.Hour),
|
||||
res: cursors.NewFloatArrayLen(MaxPointsPerBlock),
|
||||
tmp: &cursors.FloatArray{},
|
||||
}
|
||||
|
||||
agg := &datatypes.Aggregate{
|
||||
Type: datatypes.AggregateTypeMean,
|
||||
}
|
||||
|
||||
got := newWindowAggregateArrayCursor(context.Background(), agg, int64(time.Hour), 0, &MockFloatArrayCursor{})
|
||||
|
||||
if diff := cmp.Diff(got, want, cmp.AllowUnexported(floatWindowMeanArrayCursor{})); diff != "" {
|
||||
t.Fatalf("did not get expected cursor; -got/+want:\n%v", diff)
|
||||
}
|
||||
})
|
||||
|
||||
}
|
||||
|
||||
type MockIntegerArrayCursor struct {
|
||||
CloseFunc func()
|
||||
ErrFunc func() error
|
||||
StatsFunc func() cursors.CursorStats
|
||||
NextFunc func() *cursors.IntegerArray
|
||||
}
|
||||
|
||||
func (c *MockIntegerArrayCursor) Close() { c.CloseFunc() }
|
||||
func (c *MockIntegerArrayCursor) Err() error { return c.ErrFunc() }
|
||||
func (c *MockIntegerArrayCursor) Stats() cursors.CursorStats { return c.StatsFunc() }
|
||||
func (c *MockIntegerArrayCursor) Next() *cursors.IntegerArray { return c.NextFunc() }
|
||||
|
||||
func TestNewAggregateArrayCursor_Integer(t *testing.T) {
|
||||
|
||||
t.Run("Count", func(t *testing.T) {
|
||||
want := &integerWindowCountArrayCursor{
|
||||
IntegerArrayCursor: &MockIntegerArrayCursor{},
|
||||
res: cursors.NewIntegerArrayLen(1),
|
||||
tmp: &cursors.IntegerArray{},
|
||||
}
|
||||
|
||||
agg := &datatypes.Aggregate{
|
||||
Type: datatypes.AggregateTypeCount,
|
||||
}
|
||||
|
||||
got := newAggregateArrayCursor(context.Background(), agg, &MockIntegerArrayCursor{})
|
||||
|
||||
if diff := cmp.Diff(got, want, cmp.AllowUnexported(integerWindowCountArrayCursor{})); diff != "" {
|
||||
t.Fatalf("did not get expected cursor; -got/+want:\n%v", diff)
|
||||
}
|
||||
})
|
||||
|
||||
t.Run("Sum", func(t *testing.T) {
|
||||
want := &integerWindowSumArrayCursor{
|
||||
IntegerArrayCursor: &MockIntegerArrayCursor{},
|
||||
res: cursors.NewIntegerArrayLen(1),
|
||||
tmp: &cursors.IntegerArray{},
|
||||
}
|
||||
|
||||
agg := &datatypes.Aggregate{
|
||||
Type: datatypes.AggregateTypeSum,
|
||||
}
|
||||
|
||||
got := newAggregateArrayCursor(context.Background(), agg, &MockIntegerArrayCursor{})
|
||||
|
||||
if diff := cmp.Diff(got, want, cmp.AllowUnexported(integerWindowSumArrayCursor{})); diff != "" {
|
||||
t.Fatalf("did not get expected cursor; -got/+want:\n%v", diff)
|
||||
}
|
||||
})
|
||||
|
||||
t.Run("Min", func(t *testing.T) {
|
||||
want := &integerWindowMinArrayCursor{
|
||||
IntegerArrayCursor: &MockIntegerArrayCursor{},
|
||||
res: cursors.NewIntegerArrayLen(1),
|
||||
tmp: &cursors.IntegerArray{},
|
||||
}
|
||||
|
||||
agg := &datatypes.Aggregate{
|
||||
Type: datatypes.AggregateTypeMin,
|
||||
}
|
||||
|
||||
got := newAggregateArrayCursor(context.Background(), agg, &MockIntegerArrayCursor{})
|
||||
|
||||
if diff := cmp.Diff(got, want, cmp.AllowUnexported(integerWindowMinArrayCursor{})); diff != "" {
|
||||
t.Fatalf("did not get expected cursor; -got/+want:\n%v", diff)
|
||||
}
|
||||
})
|
||||
|
||||
t.Run("Max", func(t *testing.T) {
|
||||
want := &integerWindowMaxArrayCursor{
|
||||
IntegerArrayCursor: &MockIntegerArrayCursor{},
|
||||
res: cursors.NewIntegerArrayLen(1),
|
||||
tmp: &cursors.IntegerArray{},
|
||||
}
|
||||
|
||||
agg := &datatypes.Aggregate{
|
||||
Type: datatypes.AggregateTypeMax,
|
||||
}
|
||||
|
||||
got := newAggregateArrayCursor(context.Background(), agg, &MockIntegerArrayCursor{})
|
||||
|
||||
if diff := cmp.Diff(got, want, cmp.AllowUnexported(integerWindowMaxArrayCursor{})); diff != "" {
|
||||
t.Fatalf("did not get expected cursor; -got/+want:\n%v", diff)
|
||||
}
|
||||
})
|
||||
|
||||
t.Run("Mean", func(t *testing.T) {
|
||||
want := &integerWindowMeanArrayCursor{
|
||||
IntegerArrayCursor: &MockIntegerArrayCursor{},
|
||||
res: cursors.NewFloatArrayLen(1),
|
||||
tmp: &cursors.IntegerArray{},
|
||||
}
|
||||
|
||||
agg := &datatypes.Aggregate{
|
||||
Type: datatypes.AggregateTypeMean,
|
||||
}
|
||||
|
||||
got := newAggregateArrayCursor(context.Background(), agg, &MockIntegerArrayCursor{})
|
||||
|
||||
if diff := cmp.Diff(got, want, cmp.AllowUnexported(integerWindowMeanArrayCursor{})); diff != "" {
|
||||
t.Fatalf("did not get expected cursor; -got/+want:\n%v", diff)
|
||||
}
|
||||
})
|
||||
|
||||
}
|
||||
|
||||
func TestNewWindowAggregateArrayCursor_Integer(t *testing.T) {
|
||||
|
||||
t.Run("Count", func(t *testing.T) {
|
||||
want := &integerWindowCountArrayCursor{
|
||||
IntegerArrayCursor: &MockIntegerArrayCursor{},
|
||||
every: int64(time.Hour),
|
||||
res: cursors.NewIntegerArrayLen(MaxPointsPerBlock),
|
||||
tmp: &cursors.IntegerArray{},
|
||||
}
|
||||
|
||||
agg := &datatypes.Aggregate{
|
||||
Type: datatypes.AggregateTypeCount,
|
||||
}
|
||||
|
||||
got := newWindowAggregateArrayCursor(context.Background(), agg, int64(time.Hour), 0, &MockIntegerArrayCursor{})
|
||||
|
||||
if diff := cmp.Diff(got, want, cmp.AllowUnexported(integerWindowCountArrayCursor{})); diff != "" {
|
||||
t.Fatalf("did not get expected cursor; -got/+want:\n%v", diff)
|
||||
}
|
||||
})
|
||||
|
||||
t.Run("Sum", func(t *testing.T) {
|
||||
want := &integerWindowSumArrayCursor{
|
||||
IntegerArrayCursor: &MockIntegerArrayCursor{},
|
||||
every: int64(time.Hour),
|
||||
res: cursors.NewIntegerArrayLen(MaxPointsPerBlock),
|
||||
tmp: &cursors.IntegerArray{},
|
||||
}
|
||||
|
||||
agg := &datatypes.Aggregate{
|
||||
Type: datatypes.AggregateTypeSum,
|
||||
}
|
||||
|
||||
got := newWindowAggregateArrayCursor(context.Background(), agg, int64(time.Hour), 0, &MockIntegerArrayCursor{})
|
||||
|
||||
if diff := cmp.Diff(got, want, cmp.AllowUnexported(integerWindowSumArrayCursor{})); diff != "" {
|
||||
t.Fatalf("did not get expected cursor; -got/+want:\n%v", diff)
|
||||
}
|
||||
})
|
||||
|
||||
t.Run("Min", func(t *testing.T) {
|
||||
want := &integerWindowMinArrayCursor{
|
||||
IntegerArrayCursor: &MockIntegerArrayCursor{},
|
||||
every: int64(time.Hour),
|
||||
res: cursors.NewIntegerArrayLen(MaxPointsPerBlock),
|
||||
tmp: &cursors.IntegerArray{},
|
||||
}
|
||||
|
||||
agg := &datatypes.Aggregate{
|
||||
Type: datatypes.AggregateTypeMin,
|
||||
}
|
||||
|
||||
got := newWindowAggregateArrayCursor(context.Background(), agg, int64(time.Hour), 0, &MockIntegerArrayCursor{})
|
||||
|
||||
if diff := cmp.Diff(got, want, cmp.AllowUnexported(integerWindowMinArrayCursor{})); diff != "" {
|
||||
t.Fatalf("did not get expected cursor; -got/+want:\n%v", diff)
|
||||
}
|
||||
})
|
||||
|
||||
t.Run("Max", func(t *testing.T) {
|
||||
want := &integerWindowMaxArrayCursor{
|
||||
IntegerArrayCursor: &MockIntegerArrayCursor{},
|
||||
every: int64(time.Hour),
|
||||
res: cursors.NewIntegerArrayLen(MaxPointsPerBlock),
|
||||
tmp: &cursors.IntegerArray{},
|
||||
}
|
||||
|
||||
agg := &datatypes.Aggregate{
|
||||
Type: datatypes.AggregateTypeMax,
|
||||
}
|
||||
|
||||
got := newWindowAggregateArrayCursor(context.Background(), agg, int64(time.Hour), 0, &MockIntegerArrayCursor{})
|
||||
|
||||
if diff := cmp.Diff(got, want, cmp.AllowUnexported(integerWindowMaxArrayCursor{})); diff != "" {
|
||||
t.Fatalf("did not get expected cursor; -got/+want:\n%v", diff)
|
||||
}
|
||||
})
|
||||
|
||||
t.Run("Mean", func(t *testing.T) {
|
||||
want := &integerWindowMeanArrayCursor{
|
||||
IntegerArrayCursor: &MockIntegerArrayCursor{},
|
||||
every: int64(time.Hour),
|
||||
res: cursors.NewFloatArrayLen(MaxPointsPerBlock),
|
||||
tmp: &cursors.IntegerArray{},
|
||||
}
|
||||
|
||||
agg := &datatypes.Aggregate{
|
||||
Type: datatypes.AggregateTypeMean,
|
||||
}
|
||||
|
||||
got := newWindowAggregateArrayCursor(context.Background(), agg, int64(time.Hour), 0, &MockIntegerArrayCursor{})
|
||||
|
||||
if diff := cmp.Diff(got, want, cmp.AllowUnexported(integerWindowMeanArrayCursor{})); diff != "" {
|
||||
t.Fatalf("did not get expected cursor; -got/+want:\n%v", diff)
|
||||
}
|
||||
})
|
||||
|
||||
}
|
||||
|
||||
type MockUnsignedArrayCursor struct {
|
||||
CloseFunc func()
|
||||
ErrFunc func() error
|
||||
StatsFunc func() cursors.CursorStats
|
||||
NextFunc func() *cursors.UnsignedArray
|
||||
}
|
||||
|
||||
func (c *MockUnsignedArrayCursor) Close() { c.CloseFunc() }
|
||||
func (c *MockUnsignedArrayCursor) Err() error { return c.ErrFunc() }
|
||||
func (c *MockUnsignedArrayCursor) Stats() cursors.CursorStats { return c.StatsFunc() }
|
||||
func (c *MockUnsignedArrayCursor) Next() *cursors.UnsignedArray { return c.NextFunc() }
|
||||
|
||||
func TestNewAggregateArrayCursor_Unsigned(t *testing.T) {
|
||||
|
||||
t.Run("Count", func(t *testing.T) {
|
||||
want := &unsignedWindowCountArrayCursor{
|
||||
UnsignedArrayCursor: &MockUnsignedArrayCursor{},
|
||||
res: cursors.NewIntegerArrayLen(1),
|
||||
tmp: &cursors.UnsignedArray{},
|
||||
}
|
||||
|
||||
agg := &datatypes.Aggregate{
|
||||
Type: datatypes.AggregateTypeCount,
|
||||
}
|
||||
|
||||
got := newAggregateArrayCursor(context.Background(), agg, &MockUnsignedArrayCursor{})
|
||||
|
||||
if diff := cmp.Diff(got, want, cmp.AllowUnexported(unsignedWindowCountArrayCursor{})); diff != "" {
|
||||
t.Fatalf("did not get expected cursor; -got/+want:\n%v", diff)
|
||||
}
|
||||
})
|
||||
|
||||
t.Run("Sum", func(t *testing.T) {
|
||||
want := &unsignedWindowSumArrayCursor{
|
||||
UnsignedArrayCursor: &MockUnsignedArrayCursor{},
|
||||
res: cursors.NewUnsignedArrayLen(1),
|
||||
tmp: &cursors.UnsignedArray{},
|
||||
}
|
||||
|
||||
agg := &datatypes.Aggregate{
|
||||
Type: datatypes.AggregateTypeSum,
|
||||
}
|
||||
|
||||
got := newAggregateArrayCursor(context.Background(), agg, &MockUnsignedArrayCursor{})
|
||||
|
||||
if diff := cmp.Diff(got, want, cmp.AllowUnexported(unsignedWindowSumArrayCursor{})); diff != "" {
|
||||
t.Fatalf("did not get expected cursor; -got/+want:\n%v", diff)
|
||||
}
|
||||
})
|
||||
|
||||
t.Run("Min", func(t *testing.T) {
|
||||
want := &unsignedWindowMinArrayCursor{
|
||||
UnsignedArrayCursor: &MockUnsignedArrayCursor{},
|
||||
res: cursors.NewUnsignedArrayLen(1),
|
||||
tmp: &cursors.UnsignedArray{},
|
||||
}
|
||||
|
||||
agg := &datatypes.Aggregate{
|
||||
Type: datatypes.AggregateTypeMin,
|
||||
}
|
||||
|
||||
got := newAggregateArrayCursor(context.Background(), agg, &MockUnsignedArrayCursor{})
|
||||
|
||||
if diff := cmp.Diff(got, want, cmp.AllowUnexported(unsignedWindowMinArrayCursor{})); diff != "" {
|
||||
t.Fatalf("did not get expected cursor; -got/+want:\n%v", diff)
|
||||
}
|
||||
})
|
||||
|
||||
t.Run("Max", func(t *testing.T) {
|
||||
want := &unsignedWindowMaxArrayCursor{
|
||||
UnsignedArrayCursor: &MockUnsignedArrayCursor{},
|
||||
res: cursors.NewUnsignedArrayLen(1),
|
||||
tmp: &cursors.UnsignedArray{},
|
||||
}
|
||||
|
||||
agg := &datatypes.Aggregate{
|
||||
Type: datatypes.AggregateTypeMax,
|
||||
}
|
||||
|
||||
got := newAggregateArrayCursor(context.Background(), agg, &MockUnsignedArrayCursor{})
|
||||
|
||||
if diff := cmp.Diff(got, want, cmp.AllowUnexported(unsignedWindowMaxArrayCursor{})); diff != "" {
|
||||
t.Fatalf("did not get expected cursor; -got/+want:\n%v", diff)
|
||||
}
|
||||
})
|
||||
|
||||
t.Run("Mean", func(t *testing.T) {
|
||||
want := &unsignedWindowMeanArrayCursor{
|
||||
UnsignedArrayCursor: &MockUnsignedArrayCursor{},
|
||||
res: cursors.NewFloatArrayLen(1),
|
||||
tmp: &cursors.UnsignedArray{},
|
||||
}
|
||||
|
||||
agg := &datatypes.Aggregate{
|
||||
Type: datatypes.AggregateTypeMean,
|
||||
}
|
||||
|
||||
got := newAggregateArrayCursor(context.Background(), agg, &MockUnsignedArrayCursor{})
|
||||
|
||||
if diff := cmp.Diff(got, want, cmp.AllowUnexported(unsignedWindowMeanArrayCursor{})); diff != "" {
|
||||
t.Fatalf("did not get expected cursor; -got/+want:\n%v", diff)
|
||||
}
|
||||
})
|
||||
|
||||
}
|
||||
|
||||
func TestNewWindowAggregateArrayCursor_Unsigned(t *testing.T) {
|
||||
|
||||
t.Run("Count", func(t *testing.T) {
|
||||
want := &unsignedWindowCountArrayCursor{
|
||||
UnsignedArrayCursor: &MockUnsignedArrayCursor{},
|
||||
every: int64(time.Hour),
|
||||
res: cursors.NewIntegerArrayLen(MaxPointsPerBlock),
|
||||
tmp: &cursors.UnsignedArray{},
|
||||
}
|
||||
|
||||
agg := &datatypes.Aggregate{
|
||||
Type: datatypes.AggregateTypeCount,
|
||||
}
|
||||
|
||||
got := newWindowAggregateArrayCursor(context.Background(), agg, int64(time.Hour), 0, &MockUnsignedArrayCursor{})
|
||||
|
||||
if diff := cmp.Diff(got, want, cmp.AllowUnexported(unsignedWindowCountArrayCursor{})); diff != "" {
|
||||
t.Fatalf("did not get expected cursor; -got/+want:\n%v", diff)
|
||||
}
|
||||
})
|
||||
|
||||
t.Run("Sum", func(t *testing.T) {
|
||||
want := &unsignedWindowSumArrayCursor{
|
||||
UnsignedArrayCursor: &MockUnsignedArrayCursor{},
|
||||
every: int64(time.Hour),
|
||||
res: cursors.NewUnsignedArrayLen(MaxPointsPerBlock),
|
||||
tmp: &cursors.UnsignedArray{},
|
||||
}
|
||||
|
||||
agg := &datatypes.Aggregate{
|
||||
Type: datatypes.AggregateTypeSum,
|
||||
}
|
||||
|
||||
got := newWindowAggregateArrayCursor(context.Background(), agg, int64(time.Hour), 0, &MockUnsignedArrayCursor{})
|
||||
|
||||
if diff := cmp.Diff(got, want, cmp.AllowUnexported(unsignedWindowSumArrayCursor{})); diff != "" {
|
||||
t.Fatalf("did not get expected cursor; -got/+want:\n%v", diff)
|
||||
}
|
||||
})
|
||||
|
||||
t.Run("Min", func(t *testing.T) {
|
||||
want := &unsignedWindowMinArrayCursor{
|
||||
UnsignedArrayCursor: &MockUnsignedArrayCursor{},
|
||||
every: int64(time.Hour),
|
||||
res: cursors.NewUnsignedArrayLen(MaxPointsPerBlock),
|
||||
tmp: &cursors.UnsignedArray{},
|
||||
}
|
||||
|
||||
agg := &datatypes.Aggregate{
|
||||
Type: datatypes.AggregateTypeMin,
|
||||
}
|
||||
|
||||
got := newWindowAggregateArrayCursor(context.Background(), agg, int64(time.Hour), 0, &MockUnsignedArrayCursor{})
|
||||
|
||||
if diff := cmp.Diff(got, want, cmp.AllowUnexported(unsignedWindowMinArrayCursor{})); diff != "" {
|
||||
t.Fatalf("did not get expected cursor; -got/+want:\n%v", diff)
|
||||
}
|
||||
})
|
||||
|
||||
t.Run("Max", func(t *testing.T) {
|
||||
want := &unsignedWindowMaxArrayCursor{
|
||||
UnsignedArrayCursor: &MockUnsignedArrayCursor{},
|
||||
every: int64(time.Hour),
|
||||
res: cursors.NewUnsignedArrayLen(MaxPointsPerBlock),
|
||||
tmp: &cursors.UnsignedArray{},
|
||||
}
|
||||
|
||||
agg := &datatypes.Aggregate{
|
||||
Type: datatypes.AggregateTypeMax,
|
||||
}
|
||||
|
||||
got := newWindowAggregateArrayCursor(context.Background(), agg, int64(time.Hour), 0, &MockUnsignedArrayCursor{})
|
||||
|
||||
if diff := cmp.Diff(got, want, cmp.AllowUnexported(unsignedWindowMaxArrayCursor{})); diff != "" {
|
||||
t.Fatalf("did not get expected cursor; -got/+want:\n%v", diff)
|
||||
}
|
||||
})
|
||||
|
||||
t.Run("Mean", func(t *testing.T) {
|
||||
want := &unsignedWindowMeanArrayCursor{
|
||||
UnsignedArrayCursor: &MockUnsignedArrayCursor{},
|
||||
every: int64(time.Hour),
|
||||
res: cursors.NewFloatArrayLen(MaxPointsPerBlock),
|
||||
tmp: &cursors.UnsignedArray{},
|
||||
}
|
||||
|
||||
agg := &datatypes.Aggregate{
|
||||
Type: datatypes.AggregateTypeMean,
|
||||
}
|
||||
|
||||
got := newWindowAggregateArrayCursor(context.Background(), agg, int64(time.Hour), 0, &MockUnsignedArrayCursor{})
|
||||
|
||||
if diff := cmp.Diff(got, want, cmp.AllowUnexported(unsignedWindowMeanArrayCursor{})); diff != "" {
|
||||
t.Fatalf("did not get expected cursor; -got/+want:\n%v", diff)
|
||||
}
|
||||
})
|
||||
|
||||
}
|
||||
|
||||
type MockStringArrayCursor struct {
|
||||
CloseFunc func()
|
||||
ErrFunc func() error
|
||||
StatsFunc func() cursors.CursorStats
|
||||
NextFunc func() *cursors.StringArray
|
||||
}
|
||||
|
||||
func (c *MockStringArrayCursor) Close() { c.CloseFunc() }
|
||||
func (c *MockStringArrayCursor) Err() error { return c.ErrFunc() }
|
||||
func (c *MockStringArrayCursor) Stats() cursors.CursorStats { return c.StatsFunc() }
|
||||
func (c *MockStringArrayCursor) Next() *cursors.StringArray { return c.NextFunc() }
|
||||
|
||||
func TestNewAggregateArrayCursor_String(t *testing.T) {
|
||||
|
||||
t.Run("Count", func(t *testing.T) {
|
||||
want := &stringWindowCountArrayCursor{
|
||||
StringArrayCursor: &MockStringArrayCursor{},
|
||||
res: cursors.NewIntegerArrayLen(1),
|
||||
tmp: &cursors.StringArray{},
|
||||
}
|
||||
|
||||
agg := &datatypes.Aggregate{
|
||||
Type: datatypes.AggregateTypeCount,
|
||||
}
|
||||
|
||||
got := newAggregateArrayCursor(context.Background(), agg, &MockStringArrayCursor{})
|
||||
|
||||
if diff := cmp.Diff(got, want, cmp.AllowUnexported(stringWindowCountArrayCursor{})); diff != "" {
|
||||
t.Fatalf("did not get expected cursor; -got/+want:\n%v", diff)
|
||||
}
|
||||
})
|
||||
|
||||
}
|
||||
|
||||
func TestNewWindowAggregateArrayCursor_String(t *testing.T) {
|
||||
|
||||
t.Run("Count", func(t *testing.T) {
|
||||
want := &stringWindowCountArrayCursor{
|
||||
StringArrayCursor: &MockStringArrayCursor{},
|
||||
every: int64(time.Hour),
|
||||
res: cursors.NewIntegerArrayLen(MaxPointsPerBlock),
|
||||
tmp: &cursors.StringArray{},
|
||||
}
|
||||
|
||||
agg := &datatypes.Aggregate{
|
||||
Type: datatypes.AggregateTypeCount,
|
||||
}
|
||||
|
||||
got := newWindowAggregateArrayCursor(context.Background(), agg, int64(time.Hour), 0, &MockStringArrayCursor{})
|
||||
|
||||
if diff := cmp.Diff(got, want, cmp.AllowUnexported(stringWindowCountArrayCursor{})); diff != "" {
|
||||
t.Fatalf("did not get expected cursor; -got/+want:\n%v", diff)
|
||||
}
|
||||
})
|
||||
|
||||
}
|
||||
|
||||
type MockBooleanArrayCursor struct {
|
||||
CloseFunc func()
|
||||
ErrFunc func() error
|
||||
StatsFunc func() cursors.CursorStats
|
||||
NextFunc func() *cursors.BooleanArray
|
||||
}
|
||||
|
||||
func (c *MockBooleanArrayCursor) Close() { c.CloseFunc() }
|
||||
func (c *MockBooleanArrayCursor) Err() error { return c.ErrFunc() }
|
||||
func (c *MockBooleanArrayCursor) Stats() cursors.CursorStats { return c.StatsFunc() }
|
||||
func (c *MockBooleanArrayCursor) Next() *cursors.BooleanArray { return c.NextFunc() }
|
||||
|
||||
func TestNewAggregateArrayCursor_Boolean(t *testing.T) {
|
||||
|
||||
t.Run("Count", func(t *testing.T) {
|
||||
want := &booleanWindowCountArrayCursor{
|
||||
BooleanArrayCursor: &MockBooleanArrayCursor{},
|
||||
res: cursors.NewIntegerArrayLen(1),
|
||||
tmp: &cursors.BooleanArray{},
|
||||
}
|
||||
|
||||
agg := &datatypes.Aggregate{
|
||||
Type: datatypes.AggregateTypeCount,
|
||||
}
|
||||
|
||||
got := newAggregateArrayCursor(context.Background(), agg, &MockBooleanArrayCursor{})
|
||||
|
||||
if diff := cmp.Diff(got, want, cmp.AllowUnexported(booleanWindowCountArrayCursor{})); diff != "" {
|
||||
t.Fatalf("did not get expected cursor; -got/+want:\n%v", diff)
|
||||
}
|
||||
})
|
||||
|
||||
}
|
||||
|
||||
func TestNewWindowAggregateArrayCursor_Boolean(t *testing.T) {
|
||||
|
||||
t.Run("Count", func(t *testing.T) {
|
||||
want := &booleanWindowCountArrayCursor{
|
||||
BooleanArrayCursor: &MockBooleanArrayCursor{},
|
||||
every: int64(time.Hour),
|
||||
res: cursors.NewIntegerArrayLen(MaxPointsPerBlock),
|
||||
tmp: &cursors.BooleanArray{},
|
||||
}
|
||||
|
||||
agg := &datatypes.Aggregate{
|
||||
Type: datatypes.AggregateTypeCount,
|
||||
}
|
||||
|
||||
got := newWindowAggregateArrayCursor(context.Background(), agg, int64(time.Hour), 0, &MockBooleanArrayCursor{})
|
||||
|
||||
if diff := cmp.Diff(got, want, cmp.AllowUnexported(booleanWindowCountArrayCursor{})); diff != "" {
|
||||
t.Fatalf("did not get expected cursor; -got/+want:\n%v", diff)
|
||||
}
|
||||
})
|
||||
|
||||
}
|
|
@ -1,75 +0,0 @@
|
|||
package reads
|
||||
|
||||
import (
|
||||
"context"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/google/go-cmp/cmp"
|
||||
"github.com/influxdata/influxdb/v2/storage/reads/datatypes"
|
||||
"github.com/influxdata/influxdb/v2/tsdb/cursors"
|
||||
)
|
||||
|
||||
{{range .}}
|
||||
{{$ColType := .Name}}
|
||||
{{$colType := .name}}
|
||||
|
||||
type Mock{{$ColType}}ArrayCursor struct {
|
||||
CloseFunc func()
|
||||
ErrFunc func() error
|
||||
StatsFunc func() cursors.CursorStats
|
||||
NextFunc func() *cursors.{{$ColType}}Array
|
||||
}
|
||||
|
||||
func (c *Mock{{$ColType}}ArrayCursor) Close() { c.CloseFunc() }
|
||||
func (c *Mock{{$ColType}}ArrayCursor) Err() error { return c.ErrFunc() }
|
||||
func (c *Mock{{$ColType}}ArrayCursor) Stats() cursors.CursorStats { return c.StatsFunc() }
|
||||
func (c *Mock{{$ColType}}ArrayCursor) Next() *cursors.{{$ColType}}Array { return c.NextFunc() }
|
||||
|
||||
func TestNewAggregateArrayCursor_{{$ColType}}(t *testing.T) {
|
||||
{{range .Aggs}}
|
||||
{{$Agg := .Name}}
|
||||
t.Run("{{$Agg}}", func(t *testing.T) {
|
||||
want := &{{$colType}}Window{{$Agg}}ArrayCursor{
|
||||
{{$ColType}}ArrayCursor: &Mock{{$ColType}}ArrayCursor{},
|
||||
res: cursors.New{{.OutputTypeName}}ArrayLen(1),
|
||||
tmp: &cursors.{{$ColType}}Array{},
|
||||
}
|
||||
|
||||
agg := &datatypes.Aggregate{
|
||||
Type: datatypes.AggregateType{{$Agg}},
|
||||
}
|
||||
|
||||
got := newAggregateArrayCursor(context.Background(), agg, &Mock{{$ColType}}ArrayCursor{})
|
||||
|
||||
if diff := cmp.Diff(got, want, cmp.AllowUnexported({{$colType}}Window{{$Agg}}ArrayCursor{})); diff != "" {
|
||||
t.Fatalf("did not get expected cursor; -got/+want:\n%v", diff)
|
||||
}
|
||||
})
|
||||
{{end}}
|
||||
}
|
||||
|
||||
func TestNewWindowAggregateArrayCursor_{{$ColType}}(t *testing.T) {
|
||||
{{range .Aggs}}
|
||||
{{$Agg := .Name}}
|
||||
t.Run("{{$Agg}}", func(t *testing.T) {
|
||||
want := &{{$colType}}Window{{$Agg}}ArrayCursor{
|
||||
{{$ColType}}ArrayCursor: &Mock{{$ColType}}ArrayCursor{},
|
||||
every: int64(time.Hour),
|
||||
res: cursors.New{{.OutputTypeName}}ArrayLen(MaxPointsPerBlock),
|
||||
tmp: &cursors.{{$ColType}}Array{},
|
||||
}
|
||||
|
||||
agg := &datatypes.Aggregate{
|
||||
Type: datatypes.AggregateType{{$Agg}},
|
||||
}
|
||||
|
||||
got := newWindowAggregateArrayCursor(context.Background(), agg, int64(time.Hour), 0, &Mock{{$ColType}}ArrayCursor{})
|
||||
|
||||
if diff := cmp.Diff(got, want, cmp.AllowUnexported({{$colType}}Window{{$Agg}}ArrayCursor{})); diff != "" {
|
||||
t.Fatalf("did not get expected cursor; -got/+want:\n%v", diff)
|
||||
}
|
||||
})
|
||||
{{end}}
|
||||
}
|
||||
{{end}}{{/* range over each supported field type */}}
|
File diff suppressed because it is too large
Load Diff
|
@ -4,18 +4,19 @@ import (
|
|||
"bytes"
|
||||
"context"
|
||||
"fmt"
|
||||
"math"
|
||||
"sort"
|
||||
|
||||
"github.com/influxdata/influxdb/v2/kit/tracing"
|
||||
"github.com/influxdata/influxdb/v2/models"
|
||||
"github.com/influxdata/influxdb/v2/storage/reads/datatypes"
|
||||
"github.com/influxdata/influxdb/v2/v1/models"
|
||||
"github.com/influxdata/influxdb/v2/v1/tsdb/cursors"
|
||||
)
|
||||
|
||||
type groupResultSet struct {
|
||||
ctx context.Context
|
||||
req *datatypes.ReadGroupRequest
|
||||
arrayCursors *arrayCursors
|
||||
agg *datatypes.Aggregate
|
||||
arrayCursors multiShardCursors
|
||||
|
||||
i int
|
||||
seriesRows []*SeriesRow
|
||||
|
@ -41,13 +42,10 @@ func GroupOptionNilSortLo() GroupOption {
|
|||
}
|
||||
|
||||
func NewGroupResultSet(ctx context.Context, req *datatypes.ReadGroupRequest, newSeriesCursorFn func() (SeriesCursor, error), opts ...GroupOption) GroupResultSet {
|
||||
span, _ := tracing.StartSpanFromContext(ctx)
|
||||
defer span.Finish()
|
||||
span.LogKV("group_type", req.Group.String())
|
||||
|
||||
g := &groupResultSet{
|
||||
ctx: ctx,
|
||||
req: req,
|
||||
agg: req.Aggregate,
|
||||
keys: make([][]byte, len(req.GroupKeys)),
|
||||
nilSort: NilSortHi,
|
||||
newSeriesCursorFn: newSeriesCursorFn,
|
||||
|
@ -57,17 +55,7 @@ func NewGroupResultSet(ctx context.Context, req *datatypes.ReadGroupRequest, new
|
|||
o(g)
|
||||
}
|
||||
|
||||
g.arrayCursors = newArrayCursors(
|
||||
ctx,
|
||||
req.Range.Start,
|
||||
req.Range.End,
|
||||
// The following is an optimization where the selector `last`
|
||||
// is implemented as a descending array cursor followed by a
|
||||
// limit array cursor that selects only the first point, i.e
|
||||
// the point with the largest timestamp, from the descending
|
||||
// array cursor.
|
||||
req.Aggregate == nil || req.Aggregate.Type != datatypes.AggregateTypeLast,
|
||||
)
|
||||
g.arrayCursors = newMultiShardArrayCursors(ctx, req.Range.Start, req.Range.End, true, math.MaxInt64)
|
||||
|
||||
for i, k := range req.GroupKeys {
|
||||
g.keys[i] = []byte(k)
|
||||
|
@ -85,8 +73,6 @@ func NewGroupResultSet(ctx context.Context, req *datatypes.ReadGroupRequest, new
|
|||
|
||||
if n, err := g.groupBySort(); n == 0 || err != nil {
|
||||
return nil
|
||||
} else {
|
||||
span.LogKV("rows", n)
|
||||
}
|
||||
|
||||
case datatypes.GroupNone:
|
||||
|
@ -94,8 +80,6 @@ func NewGroupResultSet(ctx context.Context, req *datatypes.ReadGroupRequest, new
|
|||
|
||||
if n, err := g.groupNoneSort(); n == 0 || err != nil {
|
||||
return nil
|
||||
} else {
|
||||
span.LogKV("rows", n)
|
||||
}
|
||||
|
||||
default:
|
||||
|
@ -170,7 +154,7 @@ func groupNoneNextGroup(g *groupResultSet) GroupCursor {
|
|||
return &groupNoneCursor{
|
||||
ctx: g.ctx,
|
||||
arrayCursors: g.arrayCursors,
|
||||
agg: g.req.Aggregate,
|
||||
agg: g.agg,
|
||||
cur: seriesCursor,
|
||||
keys: g.km.Get(),
|
||||
}
|
||||
|
@ -278,7 +262,7 @@ func (g *groupResultSet) groupBySort() (int, error) {
|
|||
|
||||
type groupNoneCursor struct {
|
||||
ctx context.Context
|
||||
arrayCursors *arrayCursors
|
||||
arrayCursors multiShardCursors
|
||||
agg *datatypes.Aggregate
|
||||
cur SeriesCursor
|
||||
row SeriesRow
|
||||
|
@ -310,14 +294,14 @@ func (c *groupNoneCursor) Next() bool {
|
|||
func (c *groupNoneCursor) Cursor() cursors.Cursor {
|
||||
cur := c.arrayCursors.createCursor(c.row)
|
||||
if c.agg != nil {
|
||||
cur = newAggregateArrayCursor(c.ctx, c.agg, cur)
|
||||
cur = c.arrayCursors.newAggregateCursor(c.ctx, c.agg, cur)
|
||||
}
|
||||
return cur
|
||||
}
|
||||
|
||||
type groupByCursor struct {
|
||||
ctx context.Context
|
||||
arrayCursors *arrayCursors
|
||||
arrayCursors multiShardCursors
|
||||
agg *datatypes.Aggregate
|
||||
i int
|
||||
seriesRows []*SeriesRow
|
||||
|
@ -351,7 +335,7 @@ func (c *groupByCursor) Next() bool {
|
|||
func (c *groupByCursor) Cursor() cursors.Cursor {
|
||||
cur := c.arrayCursors.createCursor(*c.seriesRows[c.i-1])
|
||||
if c.agg != nil {
|
||||
cur = newAggregateArrayCursor(c.ctx, c.agg, cur)
|
||||
cur = c.arrayCursors.newAggregateCursor(c.ctx, c.agg, cur)
|
||||
}
|
||||
return cur
|
||||
}
|
||||
|
|
|
@ -2,17 +2,15 @@ package reads_test
|
|||
|
||||
import (
|
||||
"context"
|
||||
"reflect"
|
||||
"strings"
|
||||
"testing"
|
||||
|
||||
"github.com/google/go-cmp/cmp"
|
||||
|
||||
"github.com/influxdata/influxdb/v2/models"
|
||||
"github.com/influxdata/influxdb/v2/pkg/data/gen"
|
||||
"github.com/influxdata/influxdb/v2/storage/reads"
|
||||
"github.com/influxdata/influxdb/v2/storage/reads/datatypes"
|
||||
"github.com/influxdata/influxdb/v2/tsdb/cursors"
|
||||
"github.com/influxdata/influxdb/v2/v1/models"
|
||||
"github.com/influxdata/influxdb/v2/storage/reads"
|
||||
)
|
||||
|
||||
func TestNewGroupResultSet_Sorting(t *testing.T) {
|
||||
|
@ -439,7 +437,7 @@ func BenchmarkNewGroupResultSet_GroupBy(b *testing.B) {
|
|||
vals[i] = gen.NewCounterByteSequenceCount(card[i])
|
||||
}
|
||||
|
||||
tags := gen.NewTagsValuesSequenceValues("m0", "f0", "tag", vals)
|
||||
tags := gen.NewTagsValuesSequenceValues("tag", vals)
|
||||
rows := make([]reads.SeriesRow, tags.Count())
|
||||
for i := range rows {
|
||||
tags.Next()
|
||||
|
@ -465,242 +463,3 @@ func BenchmarkNewGroupResultSet_GroupBy(b *testing.B) {
|
|||
rs.Close()
|
||||
}
|
||||
}
|
||||
|
||||
type mockIntArrayCursor struct {
|
||||
callCount int
|
||||
}
|
||||
|
||||
func (i *mockIntArrayCursor) Close() {}
|
||||
func (i *mockIntArrayCursor) Err() error { return nil }
|
||||
func (i *mockIntArrayCursor) Stats() cursors.CursorStats { return cursors.CursorStats{} }
|
||||
func (i *mockIntArrayCursor) Next() *cursors.IntegerArray {
|
||||
if i.callCount == 1 {
|
||||
return &cursors.IntegerArray{}
|
||||
}
|
||||
i.callCount++
|
||||
return &cursors.IntegerArray{
|
||||
Timestamps: []int64{
|
||||
1,
|
||||
3,
|
||||
5,
|
||||
7,
|
||||
9,
|
||||
11,
|
||||
},
|
||||
Values: []int64{1, 2, 3, 4, 5, 6},
|
||||
}
|
||||
}
|
||||
|
||||
type mockGroupCursorIterator struct{}
|
||||
|
||||
func (i *mockGroupCursorIterator) Next(ctx context.Context, req *cursors.CursorRequest) (cursors.Cursor, error) {
|
||||
return &mockIntArrayCursor{}, nil
|
||||
}
|
||||
func (i *mockGroupCursorIterator) Stats() cursors.CursorStats {
|
||||
return cursors.CursorStats{ScannedBytes: 35, ScannedValues: 6}
|
||||
}
|
||||
|
||||
type mockReadGroupCursor struct {
|
||||
rows []reads.SeriesRow
|
||||
}
|
||||
|
||||
/* Interface adherence means that mockReadGroupCursor can't be
|
||||
written to. This global variable is icky, but accomplishes
|
||||
the same idea.
|
||||
*/
|
||||
var mockReadGroupCursorIndex = 0
|
||||
|
||||
func (c mockReadGroupCursor) Close() {}
|
||||
func (c mockReadGroupCursor) Err() error { return nil }
|
||||
func (c mockReadGroupCursor) Next() *reads.SeriesRow {
|
||||
if mockReadGroupCursorIndex == len(c.rows) {
|
||||
return nil
|
||||
}
|
||||
row := c.rows[mockReadGroupCursorIndex]
|
||||
mockReadGroupCursorIndex++
|
||||
return &row
|
||||
}
|
||||
|
||||
func newMockReadGroupCursor(keys ...string) mockReadGroupCursor {
|
||||
// Reset the cursor index
|
||||
mockReadGroupCursorIndex = 0
|
||||
rows := make([]reads.SeriesRow, len(keys))
|
||||
for i := range keys {
|
||||
rows[i].Name, rows[i].SeriesTags = models.ParseKeyBytes([]byte(keys[i]))
|
||||
rows[i].Tags = rows[i].SeriesTags.Clone()
|
||||
rows[i].Query = &mockGroupCursorIterator{}
|
||||
}
|
||||
return mockReadGroupCursor{rows: rows}
|
||||
}
|
||||
|
||||
func newSeriesCursorFn() (reads.SeriesCursor, error) {
|
||||
cursor := newMockReadGroupCursor(
|
||||
"clicks,host=foo,location=chicago click=1 1",
|
||||
"clicks,host=bar,location=dallas click=2 3",
|
||||
"clicks,host=foo,location=dallas click=3 5",
|
||||
"clicks,host=bar,location=dallas click=4 7",
|
||||
"clicks click=5 9",
|
||||
"clicks click=6 11",
|
||||
)
|
||||
return cursor, nil
|
||||
}
|
||||
|
||||
func TestNewGroupResultSet_GroupBy_Sum(t *testing.T) {
|
||||
request := datatypes.ReadGroupRequest{
|
||||
Group: datatypes.GroupBy,
|
||||
GroupKeys: []string{"host", "location"},
|
||||
Aggregate: &datatypes.Aggregate{
|
||||
Type: datatypes.AggregateTypeSum,
|
||||
},
|
||||
Range: datatypes.TimestampRange{
|
||||
Start: 0,
|
||||
End: 15,
|
||||
},
|
||||
}
|
||||
resultSet := reads.NewGroupResultSet(context.Background(), &request, newSeriesCursorFn)
|
||||
|
||||
if resultSet == nil {
|
||||
t.Fatalf("resultSet was nil")
|
||||
}
|
||||
|
||||
groupByCursor := resultSet.Next()
|
||||
if groupByCursor == nil {
|
||||
t.Fatal("unexpected: groupByCursor was nil")
|
||||
}
|
||||
if !groupByCursor.Next() {
|
||||
t.Fatal("unexpected: groupByCursor.Next failed")
|
||||
}
|
||||
cursor := groupByCursor.Cursor()
|
||||
if cursor == nil {
|
||||
t.Fatal("unexpected: cursor was nil")
|
||||
}
|
||||
integerArrayCursor := cursor.(cursors.IntegerArrayCursor)
|
||||
integerArray := integerArrayCursor.Next()
|
||||
|
||||
if integerArray == nil {
|
||||
t.Fatalf("unexpected: integerArray was nil")
|
||||
}
|
||||
if !reflect.DeepEqual(integerArray.Values, []int64{21}) {
|
||||
t.Errorf("unexpected sum values: %v", integerArray.Values)
|
||||
}
|
||||
}
|
||||
|
||||
func TestNewGroupResultSet_GroupBy_Count(t *testing.T) {
|
||||
request := datatypes.ReadGroupRequest{
|
||||
Group: datatypes.GroupBy,
|
||||
GroupKeys: []string{"host", "location"},
|
||||
Aggregate: &datatypes.Aggregate{
|
||||
Type: datatypes.AggregateTypeCount,
|
||||
},
|
||||
Range: datatypes.TimestampRange{
|
||||
Start: 0,
|
||||
End: 15,
|
||||
},
|
||||
}
|
||||
resultSet := reads.NewGroupResultSet(context.Background(), &request, newSeriesCursorFn)
|
||||
|
||||
if resultSet == nil {
|
||||
t.Fatalf("resultSet was nil")
|
||||
}
|
||||
|
||||
groupByCursor := resultSet.Next()
|
||||
if groupByCursor == nil {
|
||||
t.Fatal("unexpected: groupByCursor was nil")
|
||||
}
|
||||
if !groupByCursor.Next() {
|
||||
t.Fatal("unexpected: groupByCursor.Next failed")
|
||||
}
|
||||
cursor := groupByCursor.Cursor()
|
||||
if cursor == nil {
|
||||
t.Fatal("unexpected: cursor was nil")
|
||||
}
|
||||
integerArrayCursor := cursor.(cursors.IntegerArrayCursor)
|
||||
integerArray := integerArrayCursor.Next()
|
||||
|
||||
if integerArray == nil {
|
||||
t.Fatalf("unexpected: integerArray was nil")
|
||||
}
|
||||
if !reflect.DeepEqual(integerArray.Values, []int64{6}) {
|
||||
t.Errorf("unexpected count values: %v", integerArray.Values)
|
||||
}
|
||||
}
|
||||
|
||||
func TestNewGroupResultSet_GroupBy_First(t *testing.T) {
|
||||
request := datatypes.ReadGroupRequest{
|
||||
Group: datatypes.GroupBy,
|
||||
GroupKeys: []string{"host", "location"},
|
||||
Aggregate: &datatypes.Aggregate{
|
||||
Type: datatypes.AggregateTypeFirst,
|
||||
},
|
||||
Range: datatypes.TimestampRange{
|
||||
Start: 0,
|
||||
End: 15,
|
||||
},
|
||||
}
|
||||
resultSet := reads.NewGroupResultSet(context.Background(), &request, newSeriesCursorFn)
|
||||
|
||||
if resultSet == nil {
|
||||
t.Fatalf("resultSet was nil")
|
||||
}
|
||||
|
||||
groupByCursor := resultSet.Next()
|
||||
if groupByCursor == nil {
|
||||
t.Fatal("unexpected: groupByCursor was nil")
|
||||
}
|
||||
if !groupByCursor.Next() {
|
||||
t.Fatal("unexpected: groupByCursor.Next failed")
|
||||
}
|
||||
cursor := groupByCursor.Cursor()
|
||||
if cursor == nil {
|
||||
t.Fatal("unexpected: cursor was nil")
|
||||
}
|
||||
integerArrayCursor := cursor.(cursors.IntegerArrayCursor)
|
||||
integerArray := integerArrayCursor.Next()
|
||||
|
||||
if integerArray == nil {
|
||||
t.Fatalf("unexpected: integerArray was nil")
|
||||
}
|
||||
if !reflect.DeepEqual(integerArray.Values, []int64{1}) {
|
||||
t.Errorf("unexpected first values: %v", integerArray.Values)
|
||||
}
|
||||
}
|
||||
|
||||
func TestNewGroupResultSet_GroupBy_Last(t *testing.T) {
|
||||
request := datatypes.ReadGroupRequest{
|
||||
Group: datatypes.GroupBy,
|
||||
GroupKeys: []string{"host", "location"},
|
||||
Aggregate: &datatypes.Aggregate{
|
||||
Type: datatypes.AggregateTypeLast,
|
||||
},
|
||||
Range: datatypes.TimestampRange{
|
||||
Start: 0,
|
||||
End: 15,
|
||||
},
|
||||
}
|
||||
resultSet := reads.NewGroupResultSet(context.Background(), &request, newSeriesCursorFn)
|
||||
|
||||
if resultSet == nil {
|
||||
t.Fatalf("resultSet was nil")
|
||||
}
|
||||
|
||||
groupByCursor := resultSet.Next()
|
||||
if groupByCursor == nil {
|
||||
t.Fatal("unexpected: groupByCursor was nil")
|
||||
}
|
||||
if !groupByCursor.Next() {
|
||||
t.Fatal("unexpected: groupByCursor.Next failed")
|
||||
}
|
||||
cursor := groupByCursor.Cursor()
|
||||
if cursor == nil {
|
||||
t.Fatal("unexpected: cursor was nil")
|
||||
}
|
||||
integerArrayCursor := cursor.(cursors.IntegerArrayCursor)
|
||||
integerArray := integerArrayCursor.Next()
|
||||
|
||||
if integerArray == nil {
|
||||
t.Fatalf("unexpected: integerArray was nil")
|
||||
}
|
||||
if !reflect.DeepEqual(integerArray.Values, []int64{1}) {
|
||||
t.Errorf("unexpected last values: %v", integerArray.Values)
|
||||
}
|
||||
}
|
||||
|
|
|
@ -3,8 +3,8 @@ package reads_test
|
|||
import (
|
||||
"testing"
|
||||
|
||||
"github.com/influxdata/influxdb/v2/storage/reads"
|
||||
"github.com/influxdata/influxdb/v2/storage/reads/datatypes"
|
||||
"github.com/influxdata/influxdb/v2/storage/reads"
|
||||
)
|
||||
|
||||
func TestHasFieldValueKey(t *testing.T) {
|
||||
|
|
|
@ -4,7 +4,7 @@ import (
|
|||
"bytes"
|
||||
"strings"
|
||||
|
||||
"github.com/influxdata/influxdb/v2/models"
|
||||
"github.com/influxdata/influxdb/v2/v1/models"
|
||||
)
|
||||
|
||||
// tagsKeyMerger is responsible for determining a merged set of tag keys
|
||||
|
|
|
@ -7,7 +7,7 @@ import (
|
|||
"testing"
|
||||
|
||||
"github.com/google/go-cmp/cmp"
|
||||
"github.com/influxdata/influxdb/v2/models"
|
||||
"github.com/influxdata/influxdb/v2/v1/models"
|
||||
)
|
||||
|
||||
func TestKeyMerger_MergeTagKeys(t *testing.T) {
|
||||
|
|
|
@ -3,8 +3,8 @@ package reads_test
|
|||
import (
|
||||
"testing"
|
||||
|
||||
"github.com/influxdata/influxdb/v2/storage/reads"
|
||||
"github.com/influxdata/influxdb/v2/storage/reads/datatypes"
|
||||
"github.com/influxdata/influxdb/v2/storage/reads"
|
||||
)
|
||||
|
||||
func TestPredicateToExprString(t *testing.T) {
|
||||
|
|
|
@ -2,25 +2,31 @@ package reads
|
|||
|
||||
import (
|
||||
"context"
|
||||
"math"
|
||||
|
||||
"github.com/influxdata/influxdb/v2/models"
|
||||
"github.com/influxdata/influxdb/v2/storage/reads/datatypes"
|
||||
"github.com/influxdata/influxdb/v2/v1/models"
|
||||
"github.com/influxdata/influxdb/v2/v1/tsdb/cursors"
|
||||
)
|
||||
|
||||
type multiShardCursors interface {
|
||||
createCursor(row SeriesRow) cursors.Cursor
|
||||
newAggregateCursor(ctx context.Context, agg *datatypes.Aggregate, cursor cursors.Cursor) cursors.Cursor
|
||||
}
|
||||
|
||||
type resultSet struct {
|
||||
ctx context.Context
|
||||
agg *datatypes.Aggregate
|
||||
seriesCursor SeriesCursor
|
||||
seriesRow SeriesRow
|
||||
arrayCursors *arrayCursors
|
||||
arrayCursors multiShardCursors
|
||||
}
|
||||
|
||||
func NewFilteredResultSet(ctx context.Context, req *datatypes.ReadFilterRequest, seriesCursor SeriesCursor) ResultSet {
|
||||
return &resultSet{
|
||||
ctx: ctx,
|
||||
seriesCursor: seriesCursor,
|
||||
arrayCursors: newArrayCursors(ctx, req.Range.Start, req.Range.End, true),
|
||||
arrayCursors: newMultiShardArrayCursors(ctx, req.Range.Start, req.Range.End, true, math.MaxInt64),
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -54,7 +60,7 @@ func (r *resultSet) Next() bool {
|
|||
func (r *resultSet) Cursor() cursors.Cursor {
|
||||
cur := r.arrayCursors.createCursor(r.seriesRow)
|
||||
if r.agg != nil {
|
||||
cur = newAggregateArrayCursor(r.ctx, r.agg, cur)
|
||||
cur = r.arrayCursors.newAggregateCursor(r.ctx, r.agg, cur)
|
||||
}
|
||||
return cur
|
||||
}
|
||||
|
@ -65,9 +71,4 @@ func (r *resultSet) Tags() models.Tags {
|
|||
|
||||
// Stats returns the stats for the underlying cursors.
|
||||
// Available after resultset has been scanned.
|
||||
func (r *resultSet) Stats() cursors.CursorStats {
|
||||
if r.seriesRow.Query == nil {
|
||||
return cursors.CursorStats{}
|
||||
}
|
||||
return r.seriesRow.Query.Stats()
|
||||
}
|
||||
func (r *resultSet) Stats() cursors.CursorStats { return r.seriesRow.Query.Stats() }
|
||||
|
|
|
@ -5,7 +5,7 @@ import (
|
|||
"io"
|
||||
"strconv"
|
||||
|
||||
"github.com/influxdata/influxdb/v2/models"
|
||||
"github.com/influxdata/influxdb/v2/v1/models"
|
||||
"github.com/influxdata/influxdb/v2/v1/tsdb/cursors"
|
||||
)
|
||||
|
||||
|
|
|
@ -2,14 +2,8 @@ package reads
|
|||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
|
||||
"github.com/influxdata/influxdb/v2"
|
||||
"github.com/influxdata/influxdb/v2/kit/tracing"
|
||||
"github.com/influxdata/influxdb/v2/models"
|
||||
"github.com/influxdata/influxdb/v2/query"
|
||||
"github.com/influxdata/influxdb/v2/storage"
|
||||
"github.com/influxdata/influxdb/v2/storage/reads/datatypes"
|
||||
"github.com/influxdata/influxdb/v2/v1/models"
|
||||
"github.com/influxdata/influxdb/v2/v1/tsdb/cursors"
|
||||
"github.com/influxdata/influxql"
|
||||
)
|
||||
|
@ -24,147 +18,34 @@ type SeriesRow struct {
|
|||
SortKey []byte
|
||||
Name []byte // measurement name
|
||||
SeriesTags models.Tags // unmodified series tags
|
||||
Tags models.Tags // SeriesTags with field key renamed from \xff to _field and measurement key renamed from \x00 to _measurement
|
||||
Tags models.Tags
|
||||
Field string
|
||||
Query cursors.CursorIterator
|
||||
Query cursors.CursorIterators
|
||||
ValueCond influxql.Expr
|
||||
}
|
||||
|
||||
var (
|
||||
fieldKeyBytes = []byte(datatypes.FieldKey)
|
||||
measurementKeyBytes = []byte(datatypes.MeasurementKey)
|
||||
)
|
||||
|
||||
type indexSeriesCursor struct {
|
||||
sqry storage.SeriesCursor
|
||||
err error
|
||||
cond influxql.Expr
|
||||
seriesRow SeriesRow
|
||||
eof bool
|
||||
hasValueExpr bool
|
||||
type limitSeriesCursor struct {
|
||||
SeriesCursor
|
||||
n, o, c int64
|
||||
}
|
||||
|
||||
func NewIndexSeriesCursor(ctx context.Context, orgID, bucketID influxdb.ID, predicate *datatypes.Predicate, viewer Viewer) (SeriesCursor, error) {
|
||||
span, ctx := tracing.StartSpanFromContext(ctx)
|
||||
defer span.Finish()
|
||||
func NewLimitSeriesCursor(ctx context.Context, cur SeriesCursor, n, o int64) SeriesCursor {
|
||||
return &limitSeriesCursor{SeriesCursor: cur, o: o, n: n}
|
||||
}
|
||||
|
||||
cursorIterator, err := viewer.CreateCursorIterator(ctx)
|
||||
if err != nil {
|
||||
return nil, tracing.LogError(span, err)
|
||||
}
|
||||
|
||||
if cursorIterator == nil {
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
opt := query.IteratorOptions{
|
||||
Aux: []influxql.VarRef{{Val: "key"}},
|
||||
Authorizer: query.OpenAuthorizer,
|
||||
Ascending: true,
|
||||
Ordered: true,
|
||||
}
|
||||
p := &indexSeriesCursor{seriesRow: SeriesRow{Query: cursorIterator}}
|
||||
|
||||
if root := predicate.GetRoot(); root != nil {
|
||||
if p.cond, err = NodeToExpr(root, nil); err != nil {
|
||||
return nil, tracing.LogError(span, err)
|
||||
}
|
||||
|
||||
p.hasValueExpr = HasFieldValueKey(p.cond)
|
||||
if !p.hasValueExpr {
|
||||
opt.Condition = p.cond
|
||||
} else {
|
||||
opt.Condition = influxql.Reduce(RewriteExprRemoveFieldValue(influxql.CloneExpr(p.cond)), nil)
|
||||
if IsTrueBooleanLiteral(opt.Condition) {
|
||||
opt.Condition = nil
|
||||
func (c *limitSeriesCursor) Next() *SeriesRow {
|
||||
if c.o > 0 {
|
||||
for i := int64(0); i < c.o; i++ {
|
||||
if c.SeriesCursor.Next() == nil {
|
||||
break
|
||||
}
|
||||
}
|
||||
c.o = 0
|
||||
}
|
||||
|
||||
p.sqry, err = viewer.CreateSeriesCursor(ctx, orgID, bucketID, opt.Condition)
|
||||
if err != nil {
|
||||
p.Close()
|
||||
return nil, tracing.LogError(span, err)
|
||||
}
|
||||
return p, nil
|
||||
}
|
||||
|
||||
func (c *indexSeriesCursor) Close() {
|
||||
if !c.eof {
|
||||
c.eof = true
|
||||
if c.sqry != nil {
|
||||
c.sqry.Close()
|
||||
c.sqry = nil
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func copyTags(dst, src models.Tags) models.Tags {
|
||||
if cap(dst) < src.Len() {
|
||||
dst = make(models.Tags, src.Len())
|
||||
} else {
|
||||
dst = dst[:src.Len()]
|
||||
}
|
||||
copy(dst, src)
|
||||
return dst
|
||||
}
|
||||
|
||||
// Next emits a series row containing a series key and possible predicate on that series.
|
||||
func (c *indexSeriesCursor) Next() *SeriesRow {
|
||||
if c.eof {
|
||||
if c.c >= c.n {
|
||||
return nil
|
||||
}
|
||||
|
||||
// next series key
|
||||
sr, err := c.sqry.Next()
|
||||
if err != nil {
|
||||
c.err = err
|
||||
c.Close()
|
||||
return nil
|
||||
} else if sr == nil {
|
||||
c.Close()
|
||||
return nil
|
||||
}
|
||||
|
||||
if len(sr.Tags) < 2 {
|
||||
// Invariant broken.
|
||||
c.err = fmt.Errorf("attempted to emit key with only tags: %s", sr.Tags)
|
||||
return nil
|
||||
}
|
||||
|
||||
c.seriesRow.Name = sr.Name
|
||||
// TODO(edd): check this.
|
||||
c.seriesRow.SeriesTags = copyTags(c.seriesRow.SeriesTags, sr.Tags)
|
||||
c.seriesRow.Tags = copyTags(c.seriesRow.Tags, sr.Tags)
|
||||
|
||||
if c.cond != nil && c.hasValueExpr {
|
||||
// TODO(sgc): lazily evaluate valueCond
|
||||
c.seriesRow.ValueCond = influxql.Reduce(c.cond, c)
|
||||
if IsTrueBooleanLiteral(c.seriesRow.ValueCond) {
|
||||
// we've reduced the expression to "true"
|
||||
c.seriesRow.ValueCond = nil
|
||||
}
|
||||
}
|
||||
|
||||
// Normalise the special tag keys to the emitted format.
|
||||
mv := c.seriesRow.Tags.Get(models.MeasurementTagKeyBytes)
|
||||
c.seriesRow.Tags.Delete(models.MeasurementTagKeyBytes)
|
||||
c.seriesRow.Tags.Set(measurementKeyBytes, mv)
|
||||
|
||||
fv := c.seriesRow.Tags.Get(models.FieldKeyTagKeyBytes)
|
||||
c.seriesRow.Field = string(fv)
|
||||
c.seriesRow.Tags.Delete(models.FieldKeyTagKeyBytes)
|
||||
c.seriesRow.Tags.Set(fieldKeyBytes, fv)
|
||||
|
||||
return &c.seriesRow
|
||||
}
|
||||
|
||||
func (c *indexSeriesCursor) Value(key string) (interface{}, bool) {
|
||||
res := c.seriesRow.Tags.Get([]byte(key))
|
||||
// Return res as a string so it compares correctly with the string literals
|
||||
return string(res), res != nil
|
||||
}
|
||||
|
||||
func (c *indexSeriesCursor) Err() error {
|
||||
return c.err
|
||||
c.c++
|
||||
return c.SeriesCursor.Next()
|
||||
}
|
||||
|
|
|
@ -1,113 +1 @@
|
|||
package reads
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"strings"
|
||||
"testing"
|
||||
|
||||
"github.com/google/go-cmp/cmp"
|
||||
"github.com/influxdata/influxdb/v2/models"
|
||||
"github.com/influxdata/influxdb/v2/storage"
|
||||
"github.com/influxdata/influxdb/v2/storage/reads/datatypes"
|
||||
"github.com/influxdata/influxql"
|
||||
)
|
||||
|
||||
func TestPlannerCondition(t *testing.T) {
|
||||
sqry := &floatIterator{
|
||||
Points: []storage.SeriesCursorRow{
|
||||
{
|
||||
Name: []byte("org_bucket"), Tags: models.Tags{
|
||||
{Key: models.MeasurementTagKeyBytes, Value: []byte("cpu")},
|
||||
{Key: []byte("host"), Value: []byte("host1")},
|
||||
{Key: models.FieldKeyTagKeyBytes, Value: []byte("system")},
|
||||
},
|
||||
},
|
||||
{
|
||||
Name: []byte("org_bucket"), Tags: models.Tags{
|
||||
{Key: models.MeasurementTagKeyBytes, Value: []byte("mem")},
|
||||
{Key: []byte("host"), Value: []byte("host1")},
|
||||
{Key: models.FieldKeyTagKeyBytes, Value: []byte("user")},
|
||||
},
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
expr := fmt.Sprintf(`(%[1]s = 'cpu' AND (%[2]s = 'user' OR %[2]s = 'system')) OR (%[1]s = 'mem' AND "_value" = 0)`, datatypes.MeasurementKey, datatypes.FieldKey)
|
||||
cond, err := parseExpr(expr)
|
||||
if err != nil {
|
||||
t.Fatal("ParseExpr", err)
|
||||
}
|
||||
|
||||
p := &indexSeriesCursor{
|
||||
sqry: sqry,
|
||||
cond: cond,
|
||||
hasValueExpr: true,
|
||||
}
|
||||
|
||||
var keys []string
|
||||
// In first row, value cond should reduce to "true" and be nil.
|
||||
row := p.Next()
|
||||
if row.ValueCond != nil {
|
||||
t.Errorf("expected nil ValueCond, got %s", row.ValueCond)
|
||||
}
|
||||
keys = append(keys, string(models.MakeKey(row.Name, row.Tags)))
|
||||
|
||||
// In second row, the value condition applies.
|
||||
row = p.Next()
|
||||
if want, got := "_value = 0", row.ValueCond.String(); !cmp.Equal(want, got) {
|
||||
t.Errorf("unexpected, %s", cmp.Diff(want, got))
|
||||
}
|
||||
keys = append(keys, string(models.MakeKey(row.Name, row.Tags)))
|
||||
|
||||
expr = `org_bucket,%[2]s=system,%[1]s=cpu,host=host1
|
||||
org_bucket,%[2]s=user,%[1]s=mem,host=host1`
|
||||
|
||||
expr = fmt.Sprintf(expr, datatypes.MeasurementKey, datatypes.FieldKey)
|
||||
|
||||
exp := strings.Split(expr, "\n")
|
||||
if !cmp.Equal(exp, keys) {
|
||||
t.Errorf("unexpected, %s", cmp.Diff(exp, keys))
|
||||
}
|
||||
}
|
||||
|
||||
// parseExpr parses the given InfluxQL expression and rewrites
|
||||
// _measurement and _field vars as their storage tag key equivalents.
|
||||
func parseExpr(expr string) (influxql.Expr, error) {
|
||||
e, err := influxql.ParseExpr(expr)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
e = influxql.RewriteExpr(e, func(expr influxql.Expr) influxql.Expr {
|
||||
if vr, ok := expr.(*influxql.VarRef); ok {
|
||||
switch vr.Val {
|
||||
case datatypes.MeasurementKey:
|
||||
vr.Val = models.MeasurementTagKey
|
||||
case datatypes.FieldKey:
|
||||
vr.Val = models.FieldKeyTagKey
|
||||
}
|
||||
}
|
||||
return expr
|
||||
})
|
||||
|
||||
return e, nil
|
||||
}
|
||||
|
||||
// floatIterator is a represents an iterator that reads from a slice.
|
||||
type floatIterator struct {
|
||||
Points []storage.SeriesCursorRow
|
||||
}
|
||||
|
||||
// Close is a no-op closer for testing.
|
||||
func (itr *floatIterator) Close() {
|
||||
}
|
||||
|
||||
func (itr *floatIterator) Next() (*storage.SeriesCursorRow, error) {
|
||||
if len(itr.Points) == 0 {
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
v := &itr.Points[0]
|
||||
itr.Points = itr.Points[1:]
|
||||
return v, nil
|
||||
}
|
||||
|
|
|
@ -4,9 +4,8 @@ import (
|
|||
"context"
|
||||
|
||||
"github.com/gogo/protobuf/proto"
|
||||
"github.com/influxdata/influxdb/v2/models"
|
||||
"github.com/influxdata/influxdb/v2/query"
|
||||
"github.com/influxdata/influxdb/v2/storage/reads/datatypes"
|
||||
"github.com/influxdata/influxdb/v2/v1/models"
|
||||
"github.com/influxdata/influxdb/v2/v1/tsdb/cursors"
|
||||
)
|
||||
|
||||
|
@ -83,28 +82,5 @@ type Store interface {
|
|||
TagKeys(ctx context.Context, req *datatypes.TagKeysRequest) (cursors.StringIterator, error)
|
||||
TagValues(ctx context.Context, req *datatypes.TagValuesRequest) (cursors.StringIterator, error)
|
||||
|
||||
GetSource(orgID, bucketID uint64) proto.Message
|
||||
}
|
||||
|
||||
type GroupCapability interface {
|
||||
query.GroupCapability
|
||||
}
|
||||
|
||||
type GroupStore interface {
|
||||
GetGroupCapability(ctx context.Context) GroupCapability
|
||||
}
|
||||
|
||||
// WindowAggregateCapability describes what is supported by WindowAggregateStore.
|
||||
type WindowAggregateCapability interface {
|
||||
query.WindowAggregateCapability
|
||||
}
|
||||
|
||||
// WindowAggregateStore implements the WindowAggregate capability.
|
||||
type WindowAggregateStore interface {
|
||||
// GetWindowAggregateCapability will get a detailed list of what the RPC call supports
|
||||
// for window aggregate.
|
||||
GetWindowAggregateCapability(ctx context.Context) WindowAggregateCapability
|
||||
|
||||
// WindowAggregate will invoke a ReadWindowAggregateRequest against the Store.
|
||||
WindowAggregate(ctx context.Context, req *datatypes.ReadWindowAggregateRequest) (ResultSet, error)
|
||||
GetSource(db, rp string) proto.Message
|
||||
}
|
||||
|
|
|
@ -6,7 +6,7 @@ import (
|
|||
"io"
|
||||
"strings"
|
||||
|
||||
"github.com/influxdata/influxdb/v2/models"
|
||||
"github.com/influxdata/influxdb/v2/v1/models"
|
||||
"github.com/influxdata/influxdb/v2/storage/reads"
|
||||
"github.com/influxdata/influxdb/v2/v1/tsdb/cursors"
|
||||
)
|
||||
|
|
|
@ -1,7 +1,7 @@
|
|||
package reads
|
||||
|
||||
import (
|
||||
"github.com/influxdata/influxdb/v2/models"
|
||||
"github.com/influxdata/influxdb/v2/v1/models"
|
||||
)
|
||||
|
||||
type tagsBuffer struct {
|
||||
|
|
|
@ -1,27 +0,0 @@
|
|||
[
|
||||
{
|
||||
"Name":"Float",
|
||||
"name":"float",
|
||||
"Type":"float64"
|
||||
},
|
||||
{
|
||||
"Name":"Integer",
|
||||
"name":"integer",
|
||||
"Type":"int64"
|
||||
},
|
||||
{
|
||||
"Name":"Unsigned",
|
||||
"name":"unsigned",
|
||||
"Type":"uint64"
|
||||
},
|
||||
{
|
||||
"Name":"String",
|
||||
"name":"string",
|
||||
"Type":"string"
|
||||
},
|
||||
{
|
||||
"Name":"Boolean",
|
||||
"name":"boolean",
|
||||
"Type":"bool"
|
||||
}
|
||||
]
|
|
@ -1,18 +1 @@
|
|||
package reads
|
||||
|
||||
import (
|
||||
"context"
|
||||
|
||||
"github.com/influxdata/influxdb/v2"
|
||||
"github.com/influxdata/influxdb/v2/storage"
|
||||
"github.com/influxdata/influxdb/v2/v1/tsdb/cursors"
|
||||
"github.com/influxdata/influxql"
|
||||
)
|
||||
|
||||
// Viewer is used by the store to query data from time-series files.
|
||||
type Viewer interface {
|
||||
CreateCursorIterator(ctx context.Context) (cursors.CursorIterator, error)
|
||||
CreateSeriesCursor(ctx context.Context, orgID, bucketID influxdb.ID, cond influxql.Expr) (storage.SeriesCursor, error)
|
||||
TagKeys(ctx context.Context, orgID, bucketID influxdb.ID, start, end int64, predicate influxql.Expr) (cursors.StringIterator, error)
|
||||
TagValues(ctx context.Context, orgID, bucketID influxdb.ID, tagKey string, start, end int64, predicate influxql.Expr) (cursors.StringIterator, error)
|
||||
}
|
||||
|
|
|
@ -1,34 +0,0 @@
|
|||
package readservice
|
||||
|
||||
import (
|
||||
"github.com/gogo/protobuf/types"
|
||||
"github.com/influxdata/influxdb/v2"
|
||||
)
|
||||
|
||||
// this is easier than fooling around with .proto files.
|
||||
|
||||
type readSource struct {
|
||||
BucketID uint64 `protobuf:"varint,1,opt,name=bucket_id,proto3"`
|
||||
OrganizationID uint64 `protobuf:"varint,2,opt,name=organization_id,proto3"`
|
||||
}
|
||||
|
||||
func (r *readSource) XXX_MessageName() string { return "readSource" }
|
||||
func (r *readSource) Reset() { *r = readSource{} }
|
||||
func (r *readSource) String() string { return "readSource{}" }
|
||||
func (r *readSource) ProtoMessage() {}
|
||||
|
||||
func getReadSource(any types.Any) (readSource, error) {
|
||||
var source readSource
|
||||
if err := types.UnmarshalAny(&any, &source); err != nil {
|
||||
return source, err
|
||||
}
|
||||
return source, nil
|
||||
}
|
||||
|
||||
func (r *readSource) GetOrgID() influxdb.ID {
|
||||
return influxdb.ID(r.OrganizationID)
|
||||
}
|
||||
|
||||
func (r *readSource) GetBucketID() influxdb.ID {
|
||||
return influxdb.ID(r.BucketID)
|
||||
}
|
|
@ -1,245 +0,0 @@
|
|||
package readservice
|
||||
|
||||
import (
|
||||
"context"
|
||||
"errors"
|
||||
|
||||
"github.com/gogo/protobuf/proto"
|
||||
"github.com/influxdata/influxdb/v2/kit/tracing"
|
||||
"github.com/influxdata/influxdb/v2/models"
|
||||
"github.com/influxdata/influxdb/v2/storage/reads"
|
||||
"github.com/influxdata/influxdb/v2/storage/reads/datatypes"
|
||||
"github.com/influxdata/influxdb/v2/v1/tsdb/cursors"
|
||||
"github.com/influxdata/influxql"
|
||||
)
|
||||
|
||||
type store struct {
|
||||
viewer reads.Viewer
|
||||
groupCap GroupCapability
|
||||
windowCap WindowAggregateCapability
|
||||
}
|
||||
|
||||
// NewStore creates a store used to query time-series data.
|
||||
func NewStore(viewer reads.Viewer) reads.Store {
|
||||
return &store{
|
||||
viewer: viewer,
|
||||
groupCap: GroupCapability{
|
||||
Count: true,
|
||||
Sum: true,
|
||||
First: true,
|
||||
Last: true,
|
||||
Min: true,
|
||||
Max: true,
|
||||
},
|
||||
windowCap: WindowAggregateCapability{
|
||||
Count: true,
|
||||
Sum: true,
|
||||
First: true,
|
||||
Last: true,
|
||||
Min: true,
|
||||
Max: true,
|
||||
Mean: true,
|
||||
Offset: true,
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
func (s *store) ReadFilter(ctx context.Context, req *datatypes.ReadFilterRequest) (reads.ResultSet, error) {
|
||||
span, ctx := tracing.StartSpanFromContext(ctx)
|
||||
defer span.Finish()
|
||||
|
||||
if req.ReadSource == nil {
|
||||
return nil, tracing.LogError(span, errors.New("missing read source"))
|
||||
}
|
||||
|
||||
source, err := getReadSource(*req.ReadSource)
|
||||
if err != nil {
|
||||
return nil, tracing.LogError(span, err)
|
||||
}
|
||||
|
||||
var cur reads.SeriesCursor
|
||||
if cur, err = reads.NewIndexSeriesCursor(ctx, source.GetOrgID(), source.GetBucketID(), req.Predicate, s.viewer); err != nil {
|
||||
return nil, tracing.LogError(span, err)
|
||||
} else if cur == nil {
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
return reads.NewFilteredResultSet(ctx, req, cur), nil
|
||||
}
|
||||
|
||||
func (s *store) GetGroupCapability(ctx context.Context) reads.GroupCapability {
|
||||
return s.groupCap
|
||||
}
|
||||
|
||||
func (s *store) ReadGroup(ctx context.Context, req *datatypes.ReadGroupRequest) (reads.GroupResultSet, error) {
|
||||
span, ctx := tracing.StartSpanFromContext(ctx)
|
||||
defer span.Finish()
|
||||
|
||||
if req.ReadSource == nil {
|
||||
return nil, tracing.LogError(span, errors.New("missing read source"))
|
||||
}
|
||||
|
||||
source, err := getReadSource(*req.ReadSource)
|
||||
if err != nil {
|
||||
return nil, tracing.LogError(span, err)
|
||||
}
|
||||
|
||||
newCursor := func() (reads.SeriesCursor, error) {
|
||||
return reads.NewIndexSeriesCursor(ctx, source.GetOrgID(), source.GetBucketID(), req.Predicate, s.viewer)
|
||||
}
|
||||
|
||||
return reads.NewGroupResultSet(ctx, req, newCursor), nil
|
||||
}
|
||||
|
||||
func (s *store) TagKeys(ctx context.Context, req *datatypes.TagKeysRequest) (cursors.StringIterator, error) {
|
||||
span, ctx := tracing.StartSpanFromContext(ctx)
|
||||
defer span.Finish()
|
||||
|
||||
if req.TagsSource == nil {
|
||||
return nil, tracing.LogError(span, errors.New("missing tags source"))
|
||||
}
|
||||
|
||||
if req.Range.Start == 0 {
|
||||
req.Range.Start = models.MinNanoTime
|
||||
}
|
||||
if req.Range.End == 0 {
|
||||
req.Range.End = models.MaxNanoTime
|
||||
}
|
||||
|
||||
var expr influxql.Expr
|
||||
var err error
|
||||
if root := req.Predicate.GetRoot(); root != nil {
|
||||
expr, err = reads.NodeToExpr(root, nil)
|
||||
if err != nil {
|
||||
return nil, tracing.LogError(span, err)
|
||||
}
|
||||
|
||||
if found := reads.HasFieldValueKey(expr); found {
|
||||
return nil, tracing.LogError(span, errors.New("field values unsupported"))
|
||||
}
|
||||
expr = influxql.Reduce(influxql.CloneExpr(expr), nil)
|
||||
if reads.IsTrueBooleanLiteral(expr) {
|
||||
expr = nil
|
||||
}
|
||||
}
|
||||
|
||||
readSource, err := getReadSource(*req.TagsSource)
|
||||
if err != nil {
|
||||
return nil, tracing.LogError(span, err)
|
||||
}
|
||||
return s.viewer.TagKeys(ctx, readSource.GetOrgID(), readSource.GetBucketID(), req.Range.Start, req.Range.End, expr)
|
||||
}
|
||||
|
||||
func (s *store) TagValues(ctx context.Context, req *datatypes.TagValuesRequest) (cursors.StringIterator, error) {
|
||||
span, ctx := tracing.StartSpanFromContext(ctx)
|
||||
defer span.Finish()
|
||||
|
||||
if req.TagsSource == nil {
|
||||
return nil, tracing.LogError(span, errors.New("missing tags source"))
|
||||
}
|
||||
|
||||
if req.Range.Start == 0 {
|
||||
req.Range.Start = models.MinNanoTime
|
||||
}
|
||||
if req.Range.End == 0 {
|
||||
req.Range.End = models.MaxNanoTime
|
||||
}
|
||||
|
||||
if req.TagKey == "" {
|
||||
return nil, tracing.LogError(span, errors.New("missing tag key"))
|
||||
}
|
||||
|
||||
var expr influxql.Expr
|
||||
var err error
|
||||
if root := req.Predicate.GetRoot(); root != nil {
|
||||
expr, err = reads.NodeToExpr(root, nil)
|
||||
if err != nil {
|
||||
return nil, tracing.LogError(span, err)
|
||||
}
|
||||
|
||||
if found := reads.HasFieldValueKey(expr); found {
|
||||
return nil, tracing.LogError(span, errors.New("field values unsupported"))
|
||||
}
|
||||
expr = influxql.Reduce(influxql.CloneExpr(expr), nil)
|
||||
if reads.IsTrueBooleanLiteral(expr) {
|
||||
expr = nil
|
||||
}
|
||||
}
|
||||
|
||||
readSource, err := getReadSource(*req.TagsSource)
|
||||
if err != nil {
|
||||
return nil, tracing.LogError(span, err)
|
||||
}
|
||||
return s.viewer.TagValues(ctx, readSource.GetOrgID(), readSource.GetBucketID(), req.TagKey, req.Range.Start, req.Range.End, expr)
|
||||
}
|
||||
|
||||
func (s *store) GetSource(orgID, bucketID uint64) proto.Message {
|
||||
return &readSource{
|
||||
BucketID: bucketID,
|
||||
OrganizationID: orgID,
|
||||
}
|
||||
}
|
||||
|
||||
func (s *store) GetWindowAggregateCapability(ctx context.Context) reads.WindowAggregateCapability {
|
||||
return s.windowCap
|
||||
}
|
||||
|
||||
// WindowAggregate will invoke a ReadWindowAggregateRequest against the Store.
|
||||
func (s *store) WindowAggregate(ctx context.Context, req *datatypes.ReadWindowAggregateRequest) (reads.ResultSet, error) {
|
||||
span, ctx := tracing.StartSpanFromContext(ctx)
|
||||
defer span.Finish()
|
||||
|
||||
if req.ReadSource == nil {
|
||||
return nil, tracing.LogError(span, errors.New("missing read source"))
|
||||
}
|
||||
|
||||
source, err := getReadSource(*req.ReadSource)
|
||||
if err != nil {
|
||||
return nil, tracing.LogError(span, err)
|
||||
}
|
||||
|
||||
var cur reads.SeriesCursor
|
||||
if cur, err = reads.NewIndexSeriesCursor(ctx, source.GetOrgID(), source.GetBucketID(), req.Predicate, s.viewer); err != nil {
|
||||
return nil, tracing.LogError(span, err)
|
||||
} else if cur == nil {
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
return reads.NewWindowAggregateResultSet(ctx, req, cur)
|
||||
}
|
||||
|
||||
type GroupCapability struct {
|
||||
Count bool
|
||||
Sum bool
|
||||
First bool
|
||||
Last bool
|
||||
Min bool
|
||||
Max bool
|
||||
}
|
||||
|
||||
func (c GroupCapability) HaveCount() bool { return c.Count }
|
||||
func (c GroupCapability) HaveSum() bool { return c.Sum }
|
||||
func (c GroupCapability) HaveFirst() bool { return c.First }
|
||||
func (c GroupCapability) HaveLast() bool { return c.Last }
|
||||
func (c GroupCapability) HaveMin() bool { return c.Min }
|
||||
func (c GroupCapability) HaveMax() bool { return c.Max }
|
||||
|
||||
type WindowAggregateCapability struct {
|
||||
Min bool
|
||||
Max bool
|
||||
Mean bool
|
||||
Count bool
|
||||
Sum bool
|
||||
First bool
|
||||
Last bool
|
||||
Offset bool
|
||||
}
|
||||
|
||||
func (w WindowAggregateCapability) HaveMin() bool { return w.Min }
|
||||
func (w WindowAggregateCapability) HaveMax() bool { return w.Max }
|
||||
func (w WindowAggregateCapability) HaveMean() bool { return w.Mean }
|
||||
func (w WindowAggregateCapability) HaveCount() bool { return w.Count }
|
||||
func (w WindowAggregateCapability) HaveSum() bool { return w.Sum }
|
||||
func (w WindowAggregateCapability) HaveFirst() bool { return w.First }
|
||||
func (w WindowAggregateCapability) HaveLast() bool { return w.Last }
|
||||
func (w WindowAggregateCapability) HaveOffset() bool { return w.Offset }
|
|
@ -26,7 +26,7 @@ type Deleter interface {
|
|||
|
||||
// A Snapshotter implementation can take snapshots of the entire engine.
|
||||
type Snapshotter interface {
|
||||
WriteSnapshot(ctx context.Context, status tsm1.CacheStatus) error
|
||||
WriteSnapshot(ctx context.Context) error
|
||||
}
|
||||
|
||||
// A BucketFinder is responsible for providing access to buckets via a filter.
|
||||
|
@ -123,7 +123,7 @@ func (s *retentionEnforcer) expireData(ctx context.Context, buckets []*influxdb.
|
|||
defer logEnd()
|
||||
|
||||
// Snapshot to clear the cache to reduce write contention.
|
||||
if err := s.Snapshotter.WriteSnapshot(ctx, tsm1.CacheStatusRetention); err != nil && err != tsm1.ErrSnapshotInProgress {
|
||||
if err := s.Snapshotter.WriteSnapshot(ctx); err != nil && err != tsm1.ErrSnapshotInProgress {
|
||||
logger.Warn("Unable to snapshot cache before retention", zap.Error(err))
|
||||
}
|
||||
|
||||
|
|
|
@ -1,146 +0,0 @@
|
|||
package storage
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"errors"
|
||||
|
||||
"github.com/influxdata/influxdb/v2"
|
||||
"github.com/influxdata/influxdb/v2/models"
|
||||
"github.com/influxdata/influxdb/v2/pkg/lifecycle"
|
||||
"github.com/influxdata/influxdb/v2/v1/tsdb"
|
||||
"github.com/influxdata/influxdb/v2/v1/tsdb/index/tsi1"
|
||||
"github.com/influxdata/influxql"
|
||||
)
|
||||
|
||||
var (
|
||||
errUnexpectedOrg = errors.New("seriesCursor: unexpected org")
|
||||
errUnexpectedTagComparisonOperator = errors.New("seriesCursor: unexpected tag comparison operator")
|
||||
)
|
||||
|
||||
type SeriesCursor interface {
|
||||
Close()
|
||||
Next() (*SeriesCursorRow, error)
|
||||
}
|
||||
|
||||
// seriesCursor is an implementation of SeriesCursor over an tsi1.Index.
|
||||
type seriesCursor struct {
|
||||
index *tsi1.Index
|
||||
indexref *lifecycle.Reference
|
||||
sfile *seriesfile.SeriesFile
|
||||
sfileref *lifecycle.Reference
|
||||
orgID influxdb.ID
|
||||
encodedOrgID []byte
|
||||
bucketID influxdb.ID
|
||||
keys [][]byte
|
||||
ofs int
|
||||
row SeriesCursorRow
|
||||
cond influxql.Expr
|
||||
init bool
|
||||
}
|
||||
|
||||
type SeriesCursorRow struct {
|
||||
Name []byte
|
||||
Tags models.Tags
|
||||
}
|
||||
|
||||
// newSeriesCursor returns a new instance of SeriesCursor.
|
||||
func newSeriesCursor(orgID, bucketID influxdb.ID, index *tsi1.Index, sfile *seriesfile.SeriesFile, cond influxql.Expr) (SeriesCursor, error) {
|
||||
if cond != nil {
|
||||
var err error
|
||||
influxql.WalkFunc(cond, func(node influxql.Node) {
|
||||
switch n := node.(type) {
|
||||
case *influxql.BinaryExpr:
|
||||
switch n.Op {
|
||||
case influxql.EQ, influxql.NEQ, influxql.EQREGEX, influxql.NEQREGEX, influxql.OR, influxql.AND:
|
||||
default:
|
||||
err = errUnexpectedTagComparisonOperator
|
||||
}
|
||||
}
|
||||
})
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
}
|
||||
|
||||
indexref, err := index.Acquire()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
sfileref, err := sfile.Acquire()
|
||||
if err != nil {
|
||||
indexref.Release()
|
||||
return nil, err
|
||||
}
|
||||
|
||||
encodedOrgID := tsdb.EncodeOrgName(orgID)
|
||||
return &seriesCursor{
|
||||
index: index,
|
||||
indexref: indexref,
|
||||
sfile: sfile,
|
||||
sfileref: sfileref,
|
||||
orgID: orgID,
|
||||
encodedOrgID: encodedOrgID[:],
|
||||
bucketID: bucketID,
|
||||
cond: cond,
|
||||
}, nil
|
||||
}
|
||||
|
||||
// Close closes the iterator. Safe to call multiple times.
|
||||
func (cur *seriesCursor) Close() {
|
||||
cur.sfileref.Release()
|
||||
cur.indexref.Release()
|
||||
}
|
||||
|
||||
// Next emits the next point in the iterator.
|
||||
func (cur *seriesCursor) Next() (*SeriesCursorRow, error) {
|
||||
if !cur.init {
|
||||
if err := cur.readSeriesKeys(); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
// Release before Close(), to hold the resources for as little time as possible.
|
||||
cur.sfileref.Release()
|
||||
cur.indexref.Release()
|
||||
|
||||
cur.init = true
|
||||
}
|
||||
|
||||
if cur.ofs < len(cur.keys) {
|
||||
cur.row.Name, cur.row.Tags = seriesfile.ParseSeriesKeyInto(cur.keys[cur.ofs], cur.row.Tags)
|
||||
if !bytes.HasPrefix(cur.row.Name, cur.encodedOrgID) {
|
||||
return nil, errUnexpectedOrg
|
||||
}
|
||||
cur.ofs++
|
||||
return &cur.row, nil
|
||||
}
|
||||
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
func (cur *seriesCursor) readSeriesKeys() error {
|
||||
name := tsdb.EncodeName(cur.orgID, cur.bucketID)
|
||||
sitr, err := cur.index.MeasurementSeriesByExprIterator(name[:], cur.cond)
|
||||
if err != nil {
|
||||
return err
|
||||
} else if sitr == nil {
|
||||
return nil
|
||||
}
|
||||
defer sitr.Close()
|
||||
|
||||
for {
|
||||
elem, err := sitr.Next()
|
||||
if err != nil {
|
||||
return err
|
||||
} else if elem.SeriesID.IsZero() {
|
||||
break
|
||||
}
|
||||
|
||||
key := cur.sfile.SeriesKey(elem.SeriesID)
|
||||
if len(key) == 0 {
|
||||
continue
|
||||
}
|
||||
cur.keys = append(cur.keys, key)
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
|
@ -1,42 +0,0 @@
|
|||
package storage
|
||||
|
||||
import (
|
||||
"testing"
|
||||
|
||||
"github.com/google/go-cmp/cmp"
|
||||
"github.com/influxdata/influxdb/v2"
|
||||
"github.com/influxdata/influxdb/v2/v1/tsdb"
|
||||
)
|
||||
|
||||
func Test_NewSeriesCursor_UnexpectedOrg(t *testing.T) {
|
||||
makeKey := func(orgID, bucketID influxdb.ID) []byte {
|
||||
name := tsdb.EncodeName(orgID, bucketID)
|
||||
return seriesfile.AppendSeriesKey(nil, name[:], nil)
|
||||
}
|
||||
|
||||
orgID := influxdb.ID(0x0f0f)
|
||||
encodedOrgID := tsdb.EncodeOrgName(orgID)
|
||||
bucketID := influxdb.ID(0xb0b0)
|
||||
cur := &seriesCursor{
|
||||
keys: [][]byte{
|
||||
makeKey(orgID, bucketID),
|
||||
makeKey(influxdb.ID(0xffff), bucketID),
|
||||
},
|
||||
orgID: orgID,
|
||||
encodedOrgID: encodedOrgID[:],
|
||||
init: true,
|
||||
}
|
||||
_, err := cur.Next()
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error: %v", err)
|
||||
}
|
||||
|
||||
_, err = cur.Next()
|
||||
if err == nil {
|
||||
t.Fatal("expected error")
|
||||
}
|
||||
|
||||
if !cmp.Equal(err.Error(), errUnexpectedOrg.Error()) {
|
||||
t.Errorf("unexpected error -got/+exp\n%s", cmp.Diff(err.Error(), errUnexpectedOrg.Error()))
|
||||
}
|
||||
}
|
|
@ -1,255 +0,0 @@
|
|||
package wal
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"io"
|
||||
"io/ioutil"
|
||||
"os"
|
||||
"path/filepath"
|
||||
"sort"
|
||||
"text/tabwriter"
|
||||
|
||||
"github.com/influxdata/influxdb/v2/storage/reads/datatypes"
|
||||
"github.com/influxdata/influxdb/v2/v1/tsdb"
|
||||
"github.com/influxdata/influxdb/v2/v1/tsdb/engine/tsm1"
|
||||
)
|
||||
|
||||
// Command represents the program execution for "influxd inspect dumpmwal
|
||||
// This command will dump all entries from a given list WAL filepath globs
|
||||
|
||||
type Dump struct {
|
||||
// Standard input/output
|
||||
Stderr io.Writer
|
||||
Stdout io.Writer
|
||||
|
||||
// A list of files to dump
|
||||
FileGlobs []string
|
||||
|
||||
// Whether or not to check for duplicate/out of order entries
|
||||
FindDuplicates bool
|
||||
}
|
||||
|
||||
type DumpReport struct {
|
||||
// The file this report corresponds to
|
||||
File string
|
||||
// Any keys found to be duplicated/out of order
|
||||
DuplicateKeys []string
|
||||
// A list of all the write wal entries from this file
|
||||
Writes []*WriteWALEntry
|
||||
// A list of all the delete wal entries from this file
|
||||
Deletes []*DeleteBucketRangeWALEntry
|
||||
}
|
||||
|
||||
// Run executes the dumpwal command, generating a list of DumpReports
|
||||
// for each requested file. The `print` flag indicates whether or not
|
||||
// the command should log output during execution. If the command is run
|
||||
// as a cli, Run(true) should be used, and if the tool is run programmatically,
|
||||
// output should likely be suppressed with Run(false).
|
||||
func (w *Dump) Run(print bool) ([]*DumpReport, error) {
|
||||
if w.Stderr == nil {
|
||||
w.Stderr = os.Stderr
|
||||
}
|
||||
|
||||
if w.Stdout == nil {
|
||||
w.Stdout = os.Stdout
|
||||
}
|
||||
|
||||
if !print {
|
||||
w.Stdout, w.Stderr = ioutil.Discard, ioutil.Discard
|
||||
}
|
||||
|
||||
twOut := tabwriter.NewWriter(w.Stdout, 8, 2, 1, ' ', 0)
|
||||
twErr := tabwriter.NewWriter(w.Stderr, 8, 2, 1, ' ', 0)
|
||||
|
||||
// Process each WAL file.
|
||||
paths, err := globAndDedupe(w.FileGlobs)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
var reports []*DumpReport
|
||||
for _, path := range paths {
|
||||
r, err := w.process(path, twOut, twErr)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
reports = append(reports, r)
|
||||
}
|
||||
|
||||
return reports, nil
|
||||
}
|
||||
|
||||
func globAndDedupe(globs []string) ([]string, error) {
|
||||
files := make(map[string]struct{})
|
||||
for _, filePattern := range globs {
|
||||
matches, err := filepath.Glob(filePattern)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
for _, match := range matches {
|
||||
files[match] = struct{}{}
|
||||
}
|
||||
}
|
||||
|
||||
return sortKeys(files), nil
|
||||
}
|
||||
|
||||
func sortKeys(m map[string]struct{}) []string {
|
||||
s := make([]string, 0, len(m))
|
||||
for k := range m {
|
||||
s = append(s, k)
|
||||
}
|
||||
sort.Strings(s)
|
||||
|
||||
return s
|
||||
}
|
||||
|
||||
func (w *Dump) process(path string, stdout, stderr io.Writer) (*DumpReport, error) {
|
||||
if filepath.Ext(path) != "."+WALFileExtension {
|
||||
fmt.Fprintf(stderr, "invalid wal filename, skipping %s", path)
|
||||
return nil, fmt.Errorf("invalid wal filename: %s", path)
|
||||
}
|
||||
|
||||
report := &DumpReport{
|
||||
File: path,
|
||||
}
|
||||
|
||||
fmt.Fprintf(stdout, "File: %s\n", path)
|
||||
|
||||
// Track the earliest timestamp for each key and a set of keys with out-of-order points.
|
||||
minTimestampByKey := make(map[string]int64)
|
||||
duplicateKeys := make(map[string]struct{})
|
||||
|
||||
// Open WAL reader.
|
||||
f, err := os.Open(path)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
defer f.Close()
|
||||
r := NewWALSegmentReader(f)
|
||||
|
||||
// Iterate over the WAL entries
|
||||
for r.Next() {
|
||||
entry, err := r.Read()
|
||||
if err != nil {
|
||||
fmt.Fprintf(stdout, "Error: cannot read entry: %v ", err)
|
||||
return nil, fmt.Errorf("cannot read entry: %v", err)
|
||||
}
|
||||
|
||||
switch entry := entry.(type) {
|
||||
case *WriteWALEntry:
|
||||
// MarshalSize must always be called to make sure the size of the entry is set
|
||||
sz := entry.MarshalSize()
|
||||
if !w.FindDuplicates {
|
||||
fmt.Fprintf(stdout, "[write] sz=%d\n", sz)
|
||||
}
|
||||
report.Writes = append(report.Writes, entry)
|
||||
|
||||
keys := make([]string, 0, len(entry.Values))
|
||||
for k := range entry.Values {
|
||||
keys = append(keys, k)
|
||||
}
|
||||
sort.Strings(keys)
|
||||
|
||||
for _, k := range keys {
|
||||
fmtKey, err := formatKeyOrgBucket(k)
|
||||
// if key cannot be properly formatted with org and bucket, skip printing
|
||||
if err != nil {
|
||||
fmt.Fprintf(stderr, "Invalid key: %v\n", err)
|
||||
return nil, fmt.Errorf("invalid key: %v", err)
|
||||
}
|
||||
|
||||
for _, v := range entry.Values[k] {
|
||||
t := v.UnixNano()
|
||||
|
||||
// Skip printing if we are only showing duplicate keys.
|
||||
if w.FindDuplicates {
|
||||
// Check for duplicate/out of order keys.
|
||||
if min, ok := minTimestampByKey[k]; ok && t <= min {
|
||||
duplicateKeys[k] = struct{}{}
|
||||
}
|
||||
minTimestampByKey[k] = t
|
||||
continue
|
||||
}
|
||||
|
||||
switch v := v.(type) {
|
||||
case value.IntegerValue:
|
||||
fmt.Fprintf(stdout, "%s %vi %d\n", fmtKey, v.Value(), t)
|
||||
case value.UnsignedValue:
|
||||
fmt.Fprintf(stdout, "%s %vu %d\n", fmtKey, v.Value(), t)
|
||||
case value.FloatValue:
|
||||
fmt.Fprintf(stdout, "%s %v %d\n", fmtKey, v.Value(), t)
|
||||
case value.BooleanValue:
|
||||
fmt.Fprintf(stdout, "%s %v %d\n", fmtKey, v.Value(), t)
|
||||
case value.StringValue:
|
||||
fmt.Fprintf(stdout, "%s %q %d\n", fmtKey, v.Value(), t)
|
||||
default:
|
||||
fmt.Fprintf(stdout, "%s EMPTY\n", fmtKey)
|
||||
}
|
||||
}
|
||||
}
|
||||
case *DeleteBucketRangeWALEntry:
|
||||
bucketID := entry.BucketID.String()
|
||||
orgID := entry.OrgID.String()
|
||||
|
||||
// MarshalSize must always be called to make sure the size of the entry is set
|
||||
sz := entry.MarshalSize()
|
||||
if !w.FindDuplicates {
|
||||
pred := new(datatypes.Predicate)
|
||||
if len(entry.Predicate) > 0 {
|
||||
if err := pred.Unmarshal(entry.Predicate[1:]); err != nil {
|
||||
return nil, fmt.Errorf("invalid predicate on wal entry: %#v\nerr: %v", entry, err)
|
||||
}
|
||||
}
|
||||
fmt.Fprintf(stdout, "[delete-bucket-range] org=%s bucket=%s min=%d max=%d sz=%d pred=%s\n", orgID, bucketID, entry.Min, entry.Max, sz, pred.String())
|
||||
}
|
||||
report.Deletes = append(report.Deletes, entry)
|
||||
default:
|
||||
return nil, fmt.Errorf("invalid wal entry: %#v", entry)
|
||||
}
|
||||
}
|
||||
|
||||
// Print keys with duplicate or out-of-order points, if requested.
|
||||
if w.FindDuplicates {
|
||||
keys := make([]string, 0, len(duplicateKeys))
|
||||
for k := range duplicateKeys {
|
||||
keys = append(keys, k)
|
||||
}
|
||||
sort.Strings(keys)
|
||||
|
||||
fmt.Fprintln(stdout, "Duplicate/out of order keys:")
|
||||
for _, k := range keys {
|
||||
fmtKey, err := formatKeyOrgBucket(k)
|
||||
// don't print keys that cannot be formatted with org/bucket
|
||||
if err != nil {
|
||||
fmt.Fprintf(stderr, "Error: %v\n", err)
|
||||
continue
|
||||
}
|
||||
fmt.Fprintf(stdout, " %s\n", fmtKey)
|
||||
}
|
||||
report.DuplicateKeys = keys
|
||||
}
|
||||
|
||||
return report, nil
|
||||
}
|
||||
|
||||
// removes the first 16 bytes of the key, formats as org and bucket id (hex),
|
||||
// and re-appends to the key so that it can be pretty printed
|
||||
func formatKeyOrgBucket(key string) (string, error) {
|
||||
b := []byte(key)
|
||||
if len(b) < 16 {
|
||||
return "", fmt.Errorf("key too short to format with org and bucket")
|
||||
}
|
||||
|
||||
var a [16]byte
|
||||
copy(a[:], b[:16])
|
||||
|
||||
org, bucket := tsdb.DecodeName(a)
|
||||
|
||||
s := fmt.Sprintf("%s%s", org.String(), bucket.String())
|
||||
k := s + string(b[16:])
|
||||
|
||||
return k, nil
|
||||
}
|
|
@ -1,303 +0,0 @@
|
|||
package wal
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"encoding/binary"
|
||||
"fmt"
|
||||
"io/ioutil"
|
||||
"os"
|
||||
"sort"
|
||||
"testing"
|
||||
|
||||
"github.com/google/go-cmp/cmp"
|
||||
"github.com/google/go-cmp/cmp/cmpopts"
|
||||
"github.com/influxdata/influxdb/v2"
|
||||
"github.com/influxdata/influxdb/v2/v1/tsdb"
|
||||
"github.com/influxdata/influxdb/v2/v1/tsdb/engine/tsm1"
|
||||
)
|
||||
|
||||
func TestWalDump_RunWriteEntries(t *testing.T) {
|
||||
dir := MustTempDir()
|
||||
defer os.RemoveAll(dir)
|
||||
file := mustTempWalFile(t, dir)
|
||||
|
||||
w := NewWALSegmentWriter(file)
|
||||
|
||||
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))
|
||||
|
||||
org := influxdb.ID(1)
|
||||
orgBytes := make([]byte, 8)
|
||||
binary.BigEndian.PutUint64(orgBytes, uint64(org))
|
||||
bucket := influxdb.ID(2)
|
||||
bucketBytes := make([]byte, 8)
|
||||
binary.BigEndian.PutUint64(bucketBytes, uint64(bucket))
|
||||
prefix := string(orgBytes) + string(bucketBytes)
|
||||
|
||||
values := map[string][]value.Value{
|
||||
prefix + ",cpu,host=A#!~#float": {p1},
|
||||
prefix + ",cpu,host=A#!~#int": {p2},
|
||||
prefix + ",cpu,host=A#!~#bool": {p3},
|
||||
prefix + ",cpu,host=A#!~#string": {p4},
|
||||
prefix + ",cpu,host=A#!~#unsigned": {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)
|
||||
}
|
||||
|
||||
file.Close()
|
||||
|
||||
var testOut bytes.Buffer
|
||||
dump := &Dump{
|
||||
Stderr: &testOut,
|
||||
Stdout: &testOut,
|
||||
FileGlobs: []string{file.Name()},
|
||||
}
|
||||
|
||||
wantOut := fmt.Sprintf(`File: %s
|
||||
[write] sz=291
|
||||
00000000000000010000000000000002,cpu,host=A#!~#bool true 1
|
||||
00000000000000010000000000000002,cpu,host=A#!~#float 1.1 1
|
||||
00000000000000010000000000000002,cpu,host=A#!~#int 1i 1
|
||||
00000000000000010000000000000002,cpu,host=A#!~#string "string" 1
|
||||
00000000000000010000000000000002,cpu,host=A#!~#unsigned 18446744073709551615u 1
|
||||
`, file.Name())
|
||||
|
||||
report, err := dump.Run(true)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
gotOut := testOut.String()
|
||||
|
||||
if !cmp.Equal(gotOut, wantOut) {
|
||||
t.Fatalf("Error: unexpected output: %v", cmp.Diff(gotOut, wantOut))
|
||||
}
|
||||
|
||||
wantReport := []*DumpReport{
|
||||
{
|
||||
File: file.Name(),
|
||||
Writes: []*WriteWALEntry{
|
||||
entry,
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
unexported := []interface{}{
|
||||
value.NewBooleanValue(0, false), value.NewStringValue(0, ""), value.NewIntegerValue(0, 0),
|
||||
value.NewUnsignedValue(0, 0), value.NewFloatValue(0, 0.0), WriteWALEntry{},
|
||||
}
|
||||
|
||||
if diff := cmp.Diff(report, wantReport, cmp.AllowUnexported(unexported...)); diff != "" {
|
||||
t.Fatalf("Error: unexpected output: %v", diff)
|
||||
}
|
||||
}
|
||||
|
||||
func TestWalDumpRun_DeleteRangeEntries(t *testing.T) {
|
||||
dir := MustTempDir()
|
||||
defer os.RemoveAll(dir)
|
||||
|
||||
file := mustTempWalFile(t, dir)
|
||||
|
||||
w := NewWALSegmentWriter(file)
|
||||
entry := &DeleteBucketRangeWALEntry{
|
||||
OrgID: influxdb.ID(1),
|
||||
BucketID: influxdb.ID(2),
|
||||
Min: 3,
|
||||
Max: 4,
|
||||
Predicate: []byte(nil),
|
||||
}
|
||||
|
||||
if err := w.Write(mustMarshalEntry(entry)); err != nil {
|
||||
fatal(t, "write points", err)
|
||||
}
|
||||
|
||||
if err := w.Flush(); err != nil {
|
||||
fatal(t, "flush", err)
|
||||
}
|
||||
|
||||
var testOut bytes.Buffer
|
||||
|
||||
dump := &Dump{
|
||||
Stderr: &testOut,
|
||||
Stdout: &testOut,
|
||||
FileGlobs: []string{file.Name()},
|
||||
}
|
||||
|
||||
name := file.Name()
|
||||
file.Close()
|
||||
|
||||
report, err := dump.Run(true)
|
||||
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
want := fmt.Sprintf(`File: %s
|
||||
[delete-bucket-range] org=0000000000000001 bucket=0000000000000002 min=3 max=4 sz=48 pred=
|
||||
`, name)
|
||||
got := testOut.String()
|
||||
|
||||
if !cmp.Equal(got, want) {
|
||||
t.Fatalf("Unexpected output %v", cmp.Diff(got, want))
|
||||
}
|
||||
|
||||
wantReport := []*DumpReport{
|
||||
{
|
||||
File: file.Name(),
|
||||
Deletes: []*DeleteBucketRangeWALEntry{
|
||||
entry,
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
unexported := []interface{}{
|
||||
value.NewBooleanValue(0, false), value.NewStringValue(0, ""), value.NewIntegerValue(0, 0),
|
||||
value.NewUnsignedValue(0, 0), value.NewFloatValue(0, 0.0), WriteWALEntry{},
|
||||
}
|
||||
if diff := cmp.Diff(report, wantReport, cmp.AllowUnexported(unexported...)); diff != "" {
|
||||
t.Fatalf("Error: unexpected report: %v", diff)
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
func TestWalDumpRun_EntriesOutOfOrder(t *testing.T) {
|
||||
dir := MustTempDir()
|
||||
defer os.RemoveAll(dir)
|
||||
file := mustTempWalFile(t, dir)
|
||||
|
||||
w := NewWALSegmentWriter(file)
|
||||
|
||||
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))
|
||||
|
||||
prefix := tsdb.EncodeNameString(influxdb.ID(0xDEAD), influxdb.ID(0xBEEF))
|
||||
|
||||
// write duplicate points to the WAL...
|
||||
values := map[string][]value.Value{
|
||||
prefix + ",_m=cpu,host=A#!~#float": {p1},
|
||||
prefix + ",_m=cpu,host=A#!~#int": {p2},
|
||||
prefix + ",_m=cpu,host=A#!~#bool": {p3},
|
||||
prefix + ",_m=cpu,host=A#!~#string": {p4},
|
||||
prefix + ",_m=cpu,host=A#!~#unsigned": {p5},
|
||||
}
|
||||
|
||||
var entries []*WriteWALEntry
|
||||
|
||||
for i := 0; i < 2; i++ {
|
||||
entry := &WriteWALEntry{
|
||||
Values: values,
|
||||
}
|
||||
if err := w.Write(mustMarshalEntry(entry)); err != nil {
|
||||
t.Fatalf("error writing points: %v", err)
|
||||
}
|
||||
|
||||
if err := w.Flush(); err != nil {
|
||||
t.Fatalf("error flushing wal: %v", err)
|
||||
}
|
||||
entries = append(entries, entry)
|
||||
}
|
||||
|
||||
name := file.Name()
|
||||
file.Close()
|
||||
|
||||
var testOut bytes.Buffer
|
||||
dump := &Dump{
|
||||
Stderr: &testOut,
|
||||
Stdout: &testOut,
|
||||
FileGlobs: []string{name},
|
||||
FindDuplicates: true,
|
||||
}
|
||||
|
||||
report, err := dump.Run(true)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
want := []*DumpReport{
|
||||
{
|
||||
File: name,
|
||||
DuplicateKeys: []string{
|
||||
prefix + ",_m=cpu,host=A#!~#float",
|
||||
prefix + ",_m=cpu,host=A#!~#int",
|
||||
prefix + ",_m=cpu,host=A#!~#bool",
|
||||
prefix + ",_m=cpu,host=A#!~#string",
|
||||
prefix + ",_m=cpu,host=A#!~#unsigned",
|
||||
},
|
||||
Writes: entries,
|
||||
},
|
||||
}
|
||||
|
||||
wantOut := fmt.Sprintf(`File: %s
|
||||
Duplicate/out of order keys:
|
||||
000000000000dead000000000000beef,_m=cpu,host=A#!~#bool
|
||||
000000000000dead000000000000beef,_m=cpu,host=A#!~#float
|
||||
000000000000dead000000000000beef,_m=cpu,host=A#!~#int
|
||||
000000000000dead000000000000beef,_m=cpu,host=A#!~#string
|
||||
000000000000dead000000000000beef,_m=cpu,host=A#!~#unsigned
|
||||
`, name)
|
||||
|
||||
gotOut := testOut.String()
|
||||
|
||||
sortFunc := func(a, b string) bool { return a < b }
|
||||
|
||||
unexported := []interface{}{
|
||||
value.NewBooleanValue(0, false), value.NewStringValue(0, ""), value.NewIntegerValue(0, 0),
|
||||
value.NewUnsignedValue(0, 0), value.NewFloatValue(0, 0.0), WriteWALEntry{},
|
||||
}
|
||||
|
||||
if diff := cmp.Diff(report, want, cmpopts.SortSlices(sortFunc), cmp.AllowUnexported(unexported...)); diff != "" {
|
||||
t.Fatalf("Error: unexpected report: %v", diff)
|
||||
}
|
||||
|
||||
if diff := cmp.Diff(gotOut, wantOut); diff != "" {
|
||||
t.Fatalf("Unexpected output: %v", diff)
|
||||
}
|
||||
}
|
||||
|
||||
func MustTempFilePattern(dir string, pattern string) *os.File {
|
||||
f, err := ioutil.TempFile(dir, pattern)
|
||||
if err != nil {
|
||||
panic(fmt.Sprintf("failed to create temp file: %v", err))
|
||||
}
|
||||
return f
|
||||
}
|
||||
|
||||
func TestGlobAndDedupe(t *testing.T) {
|
||||
dir := MustTempDir()
|
||||
file := MustTempFilePattern(dir, "pattern")
|
||||
file2 := MustTempFilePattern(dir, "pattern")
|
||||
|
||||
fmt.Println(dir)
|
||||
globs := []string{dir + "/*"}
|
||||
paths, _ := globAndDedupe(globs)
|
||||
want := []string{file.Name(), file2.Name()}
|
||||
sort.Strings(want)
|
||||
|
||||
if diff := cmp.Diff(paths, want); diff != "" {
|
||||
t.Fatalf("Unexpected output: %v", diff)
|
||||
}
|
||||
|
||||
globs = append(globs, dir+"/pattern*")
|
||||
paths, _ = globAndDedupe(globs)
|
||||
|
||||
if diff := cmp.Diff(paths, want); diff != "" {
|
||||
t.Fatalf("Unexpected output: %v", diff)
|
||||
}
|
||||
|
||||
}
|
|
@ -1,28 +0,0 @@
|
|||
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)
|
||||
}
|
|
@ -1,91 +0,0 @@
|
|||
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,
|
||||
}
|
||||
}
|
|
@ -1,75 +0,0 @@
|
|||
package wal
|
||||
|
||||
import (
|
||||
"testing"
|
||||
|
||||
"github.com/influxdata/influxdb/v2/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,27 +0,0 @@
|
|||
package wal
|
||||
|
||||
import "sync"
|
||||
|
||||
var bufPool sync.Pool
|
||||
|
||||
// getBuf returns a buffer with length size from the buffer pool.
|
||||
func getBuf(size int) *[]byte {
|
||||
x := bufPool.Get()
|
||||
if x == nil {
|
||||
b := make([]byte, size)
|
||||
return &b
|
||||
}
|
||||
buf := x.(*[]byte)
|
||||
if cap(*buf) < size {
|
||||
bufPool.Put(x)
|
||||
b := make([]byte, size)
|
||||
return &b
|
||||
}
|
||||
*buf = (*buf)[:size]
|
||||
return buf
|
||||
}
|
||||
|
||||
// putBuf returns a buffer to the pool.
|
||||
func putBuf(buf *[]byte) {
|
||||
bufPool.Put(buf)
|
||||
}
|
|
@ -1,86 +0,0 @@
|
|||
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,166 +0,0 @@
|
|||
package wal
|
||||
|
||||
import (
|
||||
"context"
|
||||
"io/ioutil"
|
||||
"math/rand"
|
||||
"os"
|
||||
"testing"
|
||||
|
||||
"github.com/google/go-cmp/cmp"
|
||||
"github.com/google/go-cmp/cmp/cmpopts"
|
||||
"github.com/influxdata/influxdb/v2/kit/errors"
|
||||
"github.com/influxdata/influxdb/v2/v1/tsdb/engine/tsm1"
|
||||
)
|
||||
|
||||
type Test struct {
|
||||
dir string
|
||||
corruptFiles []string
|
||||
}
|
||||
|
||||
func TestVerifyWALL_CleanFile(t *testing.T) {
|
||||
numTestEntries := 100
|
||||
test := CreateTest(t, func() (string, []string, error) {
|
||||
dir := MustTempDir()
|
||||
|
||||
w := NewWAL(dir)
|
||||
if err := w.Open(context.Background()); err != nil {
|
||||
return "", nil, errors.Wrap(err, "error opening wal")
|
||||
}
|
||||
|
||||
for i := 0; i < numTestEntries; i++ {
|
||||
writeRandomEntry(w, t)
|
||||
}
|
||||
|
||||
if err := w.Close(); err != nil {
|
||||
return "", nil, errors.Wrap(err, "error closing wal")
|
||||
}
|
||||
|
||||
return dir, []string{}, nil
|
||||
})
|
||||
defer test.Close()
|
||||
|
||||
verifier := &Verifier{Dir: test.dir}
|
||||
summary, err := verifier.Run(false)
|
||||
if err != nil {
|
||||
t.Fatalf("Unexpected error: %v\n", err)
|
||||
}
|
||||
|
||||
expectedEntries := numTestEntries
|
||||
if summary.EntryCount != expectedEntries {
|
||||
t.Fatalf("Error: expected %d entries, checked %d entries", expectedEntries, summary.EntryCount)
|
||||
}
|
||||
|
||||
if summary.CorruptFiles != nil {
|
||||
t.Fatalf("Error: expected no corrupt files")
|
||||
}
|
||||
}
|
||||
|
||||
func CreateTest(t *testing.T, createFiles func() (string, []string, error)) *Test {
|
||||
t.Helper()
|
||||
|
||||
dir, corruptFiles, err := createFiles()
|
||||
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
return &Test{
|
||||
dir: dir,
|
||||
corruptFiles: corruptFiles,
|
||||
}
|
||||
}
|
||||
|
||||
func TestVerifyWALL_CorruptFile(t *testing.T) {
|
||||
test := CreateTest(t, func() (string, []string, error) {
|
||||
dir := MustTempDir()
|
||||
f := mustTempWalFile(t, dir)
|
||||
writeCorruptEntries(f, t, 1)
|
||||
|
||||
path := f.Name()
|
||||
return dir, []string{path}, nil
|
||||
})
|
||||
|
||||
defer test.Close()
|
||||
|
||||
verifier := &Verifier{Dir: test.dir}
|
||||
expectedEntries := 2 // 1 valid entry + 1 corrupt entry
|
||||
|
||||
summary, err := verifier.Run(false)
|
||||
if err != nil {
|
||||
t.Fatalf("Unexpected error when running wal verification: %v", err)
|
||||
}
|
||||
|
||||
if summary.EntryCount != expectedEntries {
|
||||
t.Fatalf("Error: expected %d entries, found %d entries", expectedEntries, summary.EntryCount)
|
||||
}
|
||||
|
||||
want := test.corruptFiles
|
||||
got := summary.CorruptFiles
|
||||
lessFunc := func(a, b string) bool { return a < b }
|
||||
|
||||
if !cmp.Equal(summary.CorruptFiles, want, cmpopts.SortSlices(lessFunc)) {
|
||||
t.Fatalf("Error: unexpected list of corrupt files %v", cmp.Diff(got, want))
|
||||
}
|
||||
}
|
||||
|
||||
func writeRandomEntry(w *WAL, t *testing.T) {
|
||||
if _, err := w.WriteMulti(context.Background(), map[string][]value.Value{
|
||||
"cpu,host=A#!~#value": {
|
||||
value.NewValue(rand.Int63(), rand.Float64()),
|
||||
},
|
||||
}); err != nil {
|
||||
t.Fatalf("error writing entry: %v", err)
|
||||
}
|
||||
}
|
||||
|
||||
func writeCorruptEntries(file *os.File, t *testing.T, n int) {
|
||||
w := NewWALSegmentWriter(file)
|
||||
|
||||
// random byte sequence
|
||||
corruption := []byte{1, 4, 0, 0, 0}
|
||||
|
||||
p1 := value.NewValue(1, 1.1)
|
||||
values := map[string][]value.Value{
|
||||
"cpu,host=A#!~#float": {p1},
|
||||
}
|
||||
|
||||
for i := 0; i < n; i++ {
|
||||
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 := file.Write(corruption); err != nil {
|
||||
fatal(t, "corrupt WAL segment", err)
|
||||
}
|
||||
|
||||
if err := file.Close(); err != nil {
|
||||
t.Fatalf("Error: failed to close file: %v\n", err)
|
||||
}
|
||||
}
|
||||
|
||||
func (t *Test) Close() {
|
||||
err := os.RemoveAll(t.dir)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
}
|
||||
|
||||
func mustTempWalFile(t *testing.T, dir string) *os.File {
|
||||
file, err := ioutil.TempFile(dir, "corrupt*.wal")
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
return file
|
||||
}
|
|
@ -1,105 +0,0 @@
|
|||
package wal
|
||||
|
||||
import (
|
||||
"errors"
|
||||
"fmt"
|
||||
"io"
|
||||
"io/ioutil"
|
||||
"os"
|
||||
"path"
|
||||
"path/filepath"
|
||||
"text/tabwriter"
|
||||
"time"
|
||||
)
|
||||
|
||||
type Verifier struct {
|
||||
Stderr io.Writer
|
||||
Stdout io.Writer
|
||||
Dir string
|
||||
}
|
||||
|
||||
type VerificationSummary struct {
|
||||
EntryCount int
|
||||
FileCount int
|
||||
CorruptFiles []string
|
||||
}
|
||||
|
||||
func (v *Verifier) Run(print bool) (*VerificationSummary, error) {
|
||||
if v.Stderr == nil {
|
||||
v.Stderr = os.Stderr
|
||||
}
|
||||
|
||||
if v.Stdout == nil {
|
||||
v.Stdout = os.Stdout
|
||||
}
|
||||
|
||||
if !print {
|
||||
v.Stderr, v.Stdout = ioutil.Discard, ioutil.Discard
|
||||
}
|
||||
|
||||
dir, err := os.Stat(v.Dir)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
} else if !dir.IsDir() {
|
||||
return nil, errors.New("invalid data directory")
|
||||
}
|
||||
|
||||
files, err := filepath.Glob(path.Join(v.Dir, "*.wal"))
|
||||
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
|
||||
start := time.Now()
|
||||
tw := tabwriter.NewWriter(v.Stdout, 8, 2, 1, ' ', 0)
|
||||
|
||||
var corruptFiles []string
|
||||
var entriesScanned int
|
||||
|
||||
for _, fpath := range files {
|
||||
f, err := os.OpenFile(fpath, os.O_RDONLY, 0600)
|
||||
if err != nil {
|
||||
fmt.Fprintf(v.Stderr, "error opening file %s: %v. Exiting", fpath, err)
|
||||
}
|
||||
|
||||
clean := true
|
||||
reader := NewWALSegmentReader(f)
|
||||
for reader.Next() {
|
||||
entriesScanned++
|
||||
_, err := reader.Read()
|
||||
if err != nil {
|
||||
clean = false
|
||||
fmt.Fprintf(tw, "%s: corrupt entry found at position %d\n", fpath, reader.Count())
|
||||
corruptFiles = append(corruptFiles, fpath)
|
||||
break
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
if clean {
|
||||
fmt.Fprintf(tw, "%s: clean\n", fpath)
|
||||
}
|
||||
}
|
||||
|
||||
fmt.Fprintf(tw, "Results:\n")
|
||||
fmt.Fprintf(tw, " Files checked: %d\n", len(files))
|
||||
fmt.Fprintf(tw, " Total entries checked: %d\n", entriesScanned)
|
||||
fmt.Fprintf(tw, " Corrupt files found: ")
|
||||
if len(corruptFiles) == 0 {
|
||||
fmt.Fprintf(tw, "None")
|
||||
} else {
|
||||
for _, name := range corruptFiles {
|
||||
fmt.Fprintf(tw, "\n %s", name)
|
||||
}
|
||||
}
|
||||
|
||||
fmt.Fprintf(tw, "\nCompleted in %v\n", time.Since(start))
|
||||
|
||||
summary := &VerificationSummary{
|
||||
EntryCount: entriesScanned,
|
||||
CorruptFiles: corruptFiles,
|
||||
FileCount: len(files),
|
||||
}
|
||||
|
||||
return summary, nil
|
||||
}
|
1266
storage/wal/wal.go
1266
storage/wal/wal.go
File diff suppressed because it is too large
Load Diff
|
@ -1,588 +0,0 @@
|
|||
package wal
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"io"
|
||||
"math/rand"
|
||||
"os"
|
||||
"reflect"
|
||||
"testing"
|
||||
|
||||
"github.com/golang/snappy"
|
||||
|
||||
"github.com/influxdata/influxdb/v2"
|
||||
"github.com/influxdata/influxdb/v2/v1/tsdb/engine/tsm1/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,
|
||||
Predicate: []byte("predicate"),
|
||||
}
|
||||
|
||||
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(context.Background()); 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(context.Background(), 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(context.Background()); 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(),
|
||||
Predicate: make([]byte, rand.Intn(100)),
|
||||
}
|
||||
if len(in.Predicate) == 0 {
|
||||
in.Predicate = nil
|
||||
}
|
||||
|
||||
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,
|
||||
Predicate: []byte("predicate"),
|
||||
}
|
||||
|
||||
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)
|
||||
}
|
|
@ -21,11 +21,11 @@ import (
|
|||
"github.com/influxdata/influxdb/v2/query/fluxlang"
|
||||
stdlib "github.com/influxdata/influxdb/v2/query/stdlib/influxdata/influxdb"
|
||||
"github.com/influxdata/influxdb/v2/storage"
|
||||
storageflux "github.com/influxdata/influxdb/v2/storage/flux"
|
||||
"github.com/influxdata/influxdb/v2/storage/readservice"
|
||||
"github.com/influxdata/influxdb/v2/storage/reads"
|
||||
"github.com/influxdata/influxdb/v2/task/backend"
|
||||
"github.com/influxdata/influxdb/v2/task/servicetest"
|
||||
"github.com/influxdata/influxdb/v2/tenant"
|
||||
storage2 "github.com/influxdata/influxdb/v2/v1/services/storage"
|
||||
"github.com/stretchr/testify/require"
|
||||
"go.uber.org/zap"
|
||||
"go.uber.org/zap/zaptest"
|
||||
|
@ -190,8 +190,10 @@ func newAnalyticalBackend(t *testing.T, orgSvc influxdb.OrganizationService, buc
|
|||
)
|
||||
|
||||
// TODO(adam): do we need a proper secret service here?
|
||||
reader := storageflux.NewReader(readservice.NewStore(engine))
|
||||
deps, err := stdlib.NewDependencies(reader, engine, bucketSvc, orgSvc, nil, nil)
|
||||
storageStore := storage2.NewStore(engine.TSDBStore, engine.MetaClient)
|
||||
readsReader := reads.NewReader(storageStore)
|
||||
|
||||
deps, err := stdlib.NewDependencies(readsReader, engine, bucketSvc, orgSvc, nil, nil)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
|
|
@ -7,9 +7,8 @@ import (
|
|||
"time"
|
||||
|
||||
"github.com/influxdata/influxdb/v2"
|
||||
"github.com/influxdata/influxdb/v2/models"
|
||||
"github.com/influxdata/influxdb/v2/storage"
|
||||
"github.com/influxdata/influxdb/v2/v1/tsdb"
|
||||
"github.com/influxdata/influxdb/v2/v1/models"
|
||||
"go.uber.org/zap"
|
||||
)
|
||||
|
||||
|
@ -66,13 +65,6 @@ func (s *StoragePointsWriterRecorder) Record(ctx context.Context, orgID influxdb
|
|||
return err
|
||||
}
|
||||
|
||||
// use the tsdb explode points to convert to the new style.
|
||||
// We could split this on our own but its quite possible this could change.
|
||||
_, err = tsdb.ExplodePoints(orgID, bucketID, models.Points{point})
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
// TODO - fix
|
||||
return s.pw.WritePoints(ctx, 0, 0, nil)
|
||||
return s.pw.WritePoints(ctx, orgID, bucketID, models.Points{point})
|
||||
}
|
||||
|
|
|
@ -2545,6 +2545,56 @@ func BenchmarkMakeKey(b *testing.B) {
|
|||
}
|
||||
}
|
||||
|
||||
func BenchmarkNewTagsKeyValues(b *testing.B) {
|
||||
b.Run("sorted", func(b *testing.B) {
|
||||
b.Run("no dupes", func(b *testing.B) {
|
||||
kv := [][]byte{[]byte("tag0"), []byte("v0"), []byte("tag1"), []byte("v1"), []byte("tag2"), []byte("v2")}
|
||||
|
||||
b.Run("preallocate", func(b *testing.B) {
|
||||
t := make(models.Tags, 3)
|
||||
b.ReportAllocs()
|
||||
for i := 0; i < b.N; i++ {
|
||||
_, _ = models.NewTagsKeyValues(t, kv...)
|
||||
}
|
||||
})
|
||||
|
||||
b.Run("allocate", func(b *testing.B) {
|
||||
b.ReportAllocs()
|
||||
for i := 0; i < b.N; i++ {
|
||||
_, _ = models.NewTagsKeyValues(nil, kv...)
|
||||
}
|
||||
})
|
||||
})
|
||||
|
||||
b.Run("dupes", func(b *testing.B) {
|
||||
kv := [][]byte{[]byte("tag0"), []byte("v0"), []byte("tag1"), []byte("v1"), []byte("tag1"), []byte("v1"), []byte("tag2"), []byte("v2"), []byte("tag2"), []byte("v2")}
|
||||
b.ReportAllocs()
|
||||
|
||||
for i := 0; i < b.N; i++ {
|
||||
_, _ = models.NewTagsKeyValues(nil, kv...)
|
||||
}
|
||||
})
|
||||
})
|
||||
b.Run("unsorted", func(b *testing.B) {
|
||||
b.Run("no dupes", func(b *testing.B) {
|
||||
kv := [][]byte{[]byte("tag1"), []byte("v1"), []byte("tag0"), []byte("v0"), []byte("tag2"), []byte("v2")}
|
||||
b.ReportAllocs()
|
||||
|
||||
for i := 0; i < b.N; i++ {
|
||||
_, _ = models.NewTagsKeyValues(nil, kv...)
|
||||
}
|
||||
})
|
||||
b.Run("dupes", func(b *testing.B) {
|
||||
kv := [][]byte{[]byte("tag1"), []byte("v1"), []byte("tag2"), []byte("v2"), []byte("tag0"), []byte("v0"), []byte("tag1"), []byte("v1"), []byte("tag2"), []byte("v2")}
|
||||
b.ReportAllocs()
|
||||
|
||||
for i := 0; i < b.N; i++ {
|
||||
_, _ = models.NewTagsKeyValues(nil, kv...)
|
||||
}
|
||||
})
|
||||
})
|
||||
}
|
||||
|
||||
func init() {
|
||||
// Force uint support to be enabled for testing.
|
||||
models.EnableUintSupport()
|
||||
|
|
|
@ -1,97 +0,0 @@
|
|||
// Generated by tmpl
|
||||
// https://github.com/benbjohnson/tmpl
|
||||
//
|
||||
// DO NOT EDIT!
|
||||
// Source: arrays.gen.go.tmpl
|
||||
|
||||
package gen
|
||||
|
||||
import (
|
||||
"github.com/influxdata/influxdb/v2/v1/tsdb"
|
||||
"github.com/influxdata/influxdb/v2/v1/tsdb/engine/tsm1"
|
||||
)
|
||||
|
||||
type floatArray struct {
|
||||
tsdb.FloatArray
|
||||
}
|
||||
|
||||
func newFloatArrayLen(sz int) *floatArray {
|
||||
return &floatArray{
|
||||
FloatArray: tsdb.FloatArray{
|
||||
Timestamps: make([]int64, sz),
|
||||
Values: make([]float64, sz),
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
func (a *floatArray) Encode(b []byte) ([]byte, error) {
|
||||
return tsm1.EncodeFloatArrayBlock(&a.FloatArray, b)
|
||||
}
|
||||
|
||||
type integerArray struct {
|
||||
tsdb.IntegerArray
|
||||
}
|
||||
|
||||
func newIntegerArrayLen(sz int) *integerArray {
|
||||
return &integerArray{
|
||||
IntegerArray: tsdb.IntegerArray{
|
||||
Timestamps: make([]int64, sz),
|
||||
Values: make([]int64, sz),
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
func (a *integerArray) Encode(b []byte) ([]byte, error) {
|
||||
return tsm1.EncodeIntegerArrayBlock(&a.IntegerArray, b)
|
||||
}
|
||||
|
||||
type unsignedArray struct {
|
||||
tsdb.UnsignedArray
|
||||
}
|
||||
|
||||
func newUnsignedArrayLen(sz int) *unsignedArray {
|
||||
return &unsignedArray{
|
||||
UnsignedArray: tsdb.UnsignedArray{
|
||||
Timestamps: make([]int64, sz),
|
||||
Values: make([]uint64, sz),
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
func (a *unsignedArray) Encode(b []byte) ([]byte, error) {
|
||||
return tsm1.EncodeUnsignedArrayBlock(&a.UnsignedArray, b)
|
||||
}
|
||||
|
||||
type stringArray struct {
|
||||
tsdb.StringArray
|
||||
}
|
||||
|
||||
func newStringArrayLen(sz int) *stringArray {
|
||||
return &stringArray{
|
||||
StringArray: tsdb.StringArray{
|
||||
Timestamps: make([]int64, sz),
|
||||
Values: make([]string, sz),
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
func (a *stringArray) Encode(b []byte) ([]byte, error) {
|
||||
return tsm1.EncodeStringArrayBlock(&a.StringArray, b)
|
||||
}
|
||||
|
||||
type booleanArray struct {
|
||||
tsdb.BooleanArray
|
||||
}
|
||||
|
||||
func newBooleanArrayLen(sz int) *booleanArray {
|
||||
return &booleanArray{
|
||||
BooleanArray: tsdb.BooleanArray{
|
||||
Timestamps: make([]int64, sz),
|
||||
Values: make([]bool, sz),
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
func (a *booleanArray) Encode(b []byte) ([]byte, error) {
|
||||
return tsm1.EncodeBooleanArrayBlock(&a.BooleanArray, b)
|
||||
}
|
|
@ -1,27 +0,0 @@
|
|||
package gen
|
||||
|
||||
import (
|
||||
"github.com/influxdata/influxdb/v2/v1/tsdb"
|
||||
"github.com/influxdata/influxdb/v2/v1/tsdb/engine/tsm1"
|
||||
)
|
||||
|
||||
{{range .}}
|
||||
{{ $typename := print .name "Array" }}
|
||||
{{ $tsdbname := print .Name "Array" }}
|
||||
type {{$typename}} struct {
|
||||
tsdb.{{$tsdbname}}
|
||||
}
|
||||
|
||||
func new{{$tsdbname}}Len(sz int) *{{$typename}} {
|
||||
return &{{$typename}}{
|
||||
{{$tsdbname}}: tsdb.{{$tsdbname}}{
|
||||
Timestamps: make([]int64, sz),
|
||||
Values: make([]{{.Type}}, sz),
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
func (a *{{$typename}}) Encode(b []byte) ([]byte, error) {
|
||||
return tsm1.Encode{{$tsdbname}}Block(&a.{{$tsdbname}}, b)
|
||||
}
|
||||
{{end}}
|
|
@ -1,4 +0,0 @@
|
|||
package gen
|
||||
|
||||
//go:generate tmpl -data=@types.tmpldata arrays.gen.go.tmpl values.gen.go.tmpl values_sequence.gen.go.tmpl
|
||||
//go:generate stringer -type=precision -trimprefix=precision
|
|
@ -1,140 +0,0 @@
|
|||
package gen
|
||||
|
||||
import (
|
||||
"container/heap"
|
||||
"math"
|
||||
|
||||
"github.com/influxdata/influxdb/v2/v1/models"
|
||||
)
|
||||
|
||||
type mergedSeriesGenerator struct {
|
||||
heap seriesGeneratorHeap
|
||||
last constSeries
|
||||
n int64
|
||||
first bool
|
||||
}
|
||||
|
||||
func NewMergedSeriesGenerator(s []SeriesGenerator) SeriesGenerator {
|
||||
if len(s) == 0 {
|
||||
return nil
|
||||
} else if len(s) == 1 {
|
||||
return s[0]
|
||||
}
|
||||
|
||||
msg := &mergedSeriesGenerator{first: true, n: math.MaxInt64}
|
||||
msg.heap.init(s)
|
||||
return msg
|
||||
}
|
||||
|
||||
func NewMergedSeriesGeneratorLimit(s []SeriesGenerator, n int64) SeriesGenerator {
|
||||
if len(s) == 0 {
|
||||
return nil
|
||||
}
|
||||
|
||||
msg := &mergedSeriesGenerator{first: true, n: n}
|
||||
msg.heap.init(s)
|
||||
return msg
|
||||
}
|
||||
|
||||
func (s *mergedSeriesGenerator) Next() bool {
|
||||
if len(s.heap.items) == 0 {
|
||||
return false
|
||||
}
|
||||
|
||||
if s.n > 0 {
|
||||
s.n--
|
||||
if !s.first {
|
||||
top := s.heap.items[0]
|
||||
s.last.CopyFrom(top) // capture last key for duplicate checking
|
||||
|
||||
for {
|
||||
if top.Next() {
|
||||
if len(s.heap.items) > 1 {
|
||||
heap.Fix(&s.heap, 0)
|
||||
}
|
||||
} else {
|
||||
heap.Pop(&s.heap)
|
||||
if len(s.heap.items) == 0 {
|
||||
return false
|
||||
}
|
||||
}
|
||||
|
||||
top = s.heap.items[0]
|
||||
if CompareSeries(&s.last, top) == 0 {
|
||||
// duplicate key, get next
|
||||
continue
|
||||
}
|
||||
return true
|
||||
}
|
||||
}
|
||||
|
||||
s.first = false
|
||||
return true
|
||||
}
|
||||
|
||||
return false
|
||||
}
|
||||
|
||||
func (s *mergedSeriesGenerator) Key() []byte {
|
||||
return s.heap.items[0].Key()
|
||||
}
|
||||
|
||||
func (s *mergedSeriesGenerator) Name() []byte {
|
||||
return s.heap.items[0].Name()
|
||||
}
|
||||
|
||||
func (s *mergedSeriesGenerator) Tags() models.Tags {
|
||||
return s.heap.items[0].Tags()
|
||||
}
|
||||
|
||||
func (s *mergedSeriesGenerator) Field() []byte {
|
||||
return s.heap.items[0].Field()
|
||||
}
|
||||
|
||||
func (s *mergedSeriesGenerator) TimeValuesGenerator() TimeValuesSequence {
|
||||
return s.heap.items[0].TimeValuesGenerator()
|
||||
}
|
||||
|
||||
type seriesGeneratorHeap struct {
|
||||
items []SeriesGenerator
|
||||
}
|
||||
|
||||
func (h *seriesGeneratorHeap) init(results []SeriesGenerator) {
|
||||
if cap(h.items) < len(results) {
|
||||
h.items = make([]SeriesGenerator, 0, len(results))
|
||||
} else {
|
||||
h.items = h.items[:0]
|
||||
}
|
||||
|
||||
for _, rs := range results {
|
||||
if rs.Next() {
|
||||
h.items = append(h.items, rs)
|
||||
}
|
||||
}
|
||||
heap.Init(h)
|
||||
}
|
||||
|
||||
func (h *seriesGeneratorHeap) Less(i, j int) bool {
|
||||
return CompareSeries(h.items[i], h.items[j]) == -1
|
||||
}
|
||||
|
||||
func (h *seriesGeneratorHeap) Len() int {
|
||||
return len(h.items)
|
||||
}
|
||||
|
||||
func (h *seriesGeneratorHeap) Swap(i, j int) {
|
||||
h.items[i], h.items[j] = h.items[j], h.items[i]
|
||||
}
|
||||
|
||||
func (h *seriesGeneratorHeap) Push(x interface{}) {
|
||||
panic("not implemented")
|
||||
}
|
||||
|
||||
func (h *seriesGeneratorHeap) Pop() interface{} {
|
||||
old := h.items
|
||||
n := len(old)
|
||||
item := old[n-1]
|
||||
old[n-1] = nil
|
||||
h.items = old[0 : n-1]
|
||||
return item
|
||||
}
|
|
@ -1,213 +0,0 @@
|
|||
package gen
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"math"
|
||||
"strings"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/google/go-cmp/cmp"
|
||||
)
|
||||
|
||||
func sg(name, prefix, field string, counts ...int) SeriesGenerator {
|
||||
spec := TimeSequenceSpec{Count: 1, Start: time.Unix(0, 0), Delta: time.Second}
|
||||
ts := NewTimestampSequenceFromSpec(spec)
|
||||
vs := NewFloatConstantValuesSequence(1)
|
||||
vg := NewTimeFloatValuesSequence(spec.Count, ts, vs)
|
||||
return NewSeriesGenerator([]byte(name), []byte(field), vg, NewTagsValuesSequenceCounts(prefix, counts))
|
||||
}
|
||||
|
||||
func tags(sb *strings.Builder, prefix string, vals []int) {
|
||||
sb.WriteByte(',')
|
||||
|
||||
// max tag width
|
||||
tw := int(math.Ceil(math.Log10(float64(len(vals)))))
|
||||
tf := fmt.Sprintf("%s%%0%dd=value%%d", prefix, tw)
|
||||
tvs := make([]string, len(vals))
|
||||
for i := range vals {
|
||||
tvs[i] = fmt.Sprintf(tf, i, vals[i])
|
||||
}
|
||||
sb.WriteString(strings.Join(tvs, ","))
|
||||
}
|
||||
|
||||
func line(name, prefix, field string, vals ...int) string {
|
||||
var sb strings.Builder
|
||||
sb.WriteString(name)
|
||||
tags(&sb, prefix, vals)
|
||||
sb.WriteString("#!~#")
|
||||
sb.WriteString(field)
|
||||
return sb.String()
|
||||
}
|
||||
|
||||
func seriesGeneratorString(sg SeriesGenerator) []string {
|
||||
var lines []string
|
||||
for sg.Next() {
|
||||
lines = append(lines, fmt.Sprintf("%s#!~#%s", string(sg.Key()), string(sg.Field())))
|
||||
}
|
||||
return lines
|
||||
}
|
||||
|
||||
func TestNewMergedSeriesGenerator(t *testing.T) {
|
||||
tests := []struct {
|
||||
n string
|
||||
s []SeriesGenerator
|
||||
exp []string
|
||||
}{
|
||||
{
|
||||
n: "single",
|
||||
s: []SeriesGenerator{
|
||||
sg("cpu", "t", "f0", 2, 1),
|
||||
},
|
||||
exp: []string{
|
||||
line("cpu", "t", "f0", 0, 0),
|
||||
line("cpu", "t", "f0", 1, 0),
|
||||
},
|
||||
},
|
||||
{
|
||||
n: "multiple,interleaved",
|
||||
s: []SeriesGenerator{
|
||||
sg("cpu", "t", "f0", 2, 1),
|
||||
sg("cpu", "t", "f1", 2, 1),
|
||||
},
|
||||
exp: []string{
|
||||
line("cpu", "t", "f0", 0, 0),
|
||||
line("cpu", "t", "f1", 0, 0),
|
||||
line("cpu", "t", "f0", 1, 0),
|
||||
line("cpu", "t", "f1", 1, 0),
|
||||
},
|
||||
},
|
||||
{
|
||||
n: "multiple,sequential",
|
||||
s: []SeriesGenerator{
|
||||
sg("cpu", "t", "f0", 2),
|
||||
sg("cpu", "u", "f0", 2, 1),
|
||||
},
|
||||
exp: []string{
|
||||
line("cpu", "t", "f0", 0),
|
||||
line("cpu", "t", "f0", 1),
|
||||
line("cpu", "u", "f0", 0, 0),
|
||||
line("cpu", "u", "f0", 1, 0),
|
||||
},
|
||||
},
|
||||
{
|
||||
n: "multiple,sequential",
|
||||
s: []SeriesGenerator{
|
||||
sg("m1", "t", "f0", 2, 1),
|
||||
sg("m0", "t", "f0", 2, 1),
|
||||
},
|
||||
exp: []string{
|
||||
line("m0", "t", "f0", 0, 0),
|
||||
line("m0", "t", "f0", 1, 0),
|
||||
line("m1", "t", "f0", 0, 0),
|
||||
line("m1", "t", "f0", 1, 0),
|
||||
},
|
||||
},
|
||||
{
|
||||
// ensure duplicates are removed
|
||||
n: "duplicates",
|
||||
s: []SeriesGenerator{
|
||||
sg("cpu", "t", "f0", 2, 1),
|
||||
sg("cpu", "t", "f0", 2, 1),
|
||||
},
|
||||
exp: []string{
|
||||
line("cpu", "t", "f0", 0, 0),
|
||||
line("cpu", "t", "f0", 1, 0),
|
||||
},
|
||||
},
|
||||
{
|
||||
// ensure duplicates are removed, but non-dupes from same SeriesGenerator
|
||||
// are still included
|
||||
n: "duplicates,multiple,interleaved",
|
||||
s: []SeriesGenerator{
|
||||
sg("cpu", "t", "f0", 2, 1),
|
||||
sg("cpu", "t", "f1", 2, 1),
|
||||
sg("cpu", "t", "f0", 2, 1),
|
||||
sg("cpu", "t", "f1", 3, 1),
|
||||
},
|
||||
exp: []string{
|
||||
line("cpu", "t", "f0", 0, 0),
|
||||
line("cpu", "t", "f1", 0, 0),
|
||||
line("cpu", "t", "f0", 1, 0),
|
||||
line("cpu", "t", "f1", 1, 0),
|
||||
line("cpu", "t", "f1", 2, 0),
|
||||
},
|
||||
},
|
||||
}
|
||||
for _, tt := range tests {
|
||||
t.Run(tt.n, func(t *testing.T) {
|
||||
sg := NewMergedSeriesGenerator(tt.s)
|
||||
if got := seriesGeneratorString(sg); !cmp.Equal(got, tt.exp) {
|
||||
t.Errorf("unpexected -got/+exp\n%s", cmp.Diff(got, tt.exp))
|
||||
}
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func TestNewMergedSeriesGeneratorLimit(t *testing.T) {
|
||||
tests := []struct {
|
||||
n string
|
||||
s []SeriesGenerator
|
||||
lim int64
|
||||
exp []string
|
||||
}{
|
||||
{
|
||||
n: "single",
|
||||
s: []SeriesGenerator{
|
||||
sg("cpu", "t", "f0", 4, 1),
|
||||
},
|
||||
lim: 2,
|
||||
exp: []string{
|
||||
line("cpu", "t", "f0", 0, 0),
|
||||
line("cpu", "t", "f0", 1, 0),
|
||||
},
|
||||
},
|
||||
{
|
||||
n: "multiple,interleaved",
|
||||
s: []SeriesGenerator{
|
||||
sg("cpu", "t", "f0", 2, 1),
|
||||
sg("cpu", "t", "f1", 2, 1),
|
||||
},
|
||||
lim: 3,
|
||||
exp: []string{
|
||||
line("cpu", "t", "f0", 0, 0),
|
||||
line("cpu", "t", "f1", 0, 0),
|
||||
line("cpu", "t", "f0", 1, 0),
|
||||
},
|
||||
},
|
||||
{
|
||||
n: "multiple,sequential",
|
||||
s: []SeriesGenerator{
|
||||
sg("cpu", "t", "f0", 2),
|
||||
sg("cpu", "u", "f0", 2, 1),
|
||||
},
|
||||
lim: 2,
|
||||
exp: []string{
|
||||
line("cpu", "t", "f0", 0),
|
||||
line("cpu", "t", "f0", 1),
|
||||
},
|
||||
},
|
||||
{
|
||||
n: "multiple,sequential",
|
||||
s: []SeriesGenerator{
|
||||
sg("m1", "t", "f0", 2, 1),
|
||||
sg("m0", "t", "f0", 2, 1),
|
||||
},
|
||||
lim: 4,
|
||||
exp: []string{
|
||||
line("m0", "t", "f0", 0, 0),
|
||||
line("m0", "t", "f0", 1, 0),
|
||||
line("m1", "t", "f0", 0, 0),
|
||||
line("m1", "t", "f0", 1, 0),
|
||||
},
|
||||
},
|
||||
}
|
||||
for _, tt := range tests {
|
||||
t.Run(tt.n, func(t *testing.T) {
|
||||
sg := NewMergedSeriesGeneratorLimit(tt.s, tt.lim)
|
||||
if got := seriesGeneratorString(sg); !cmp.Equal(got, tt.exp) {
|
||||
t.Errorf("unpexected -got/+exp\n%s", cmp.Diff(got, tt.exp))
|
||||
}
|
||||
})
|
||||
}
|
||||
}
|
|
@ -1,16 +0,0 @@
|
|||
// Code generated by "stringer -type=precision -trimprefix=precision"; DO NOT EDIT.
|
||||
|
||||
package gen
|
||||
|
||||
import "strconv"
|
||||
|
||||
const _precision_name = "MillisecondNanosecondMicrosecondSecondMinuteHour"
|
||||
|
||||
var _precision_index = [...]uint8{0, 11, 21, 32, 38, 44, 48}
|
||||
|
||||
func (i precision) String() string {
|
||||
if i >= precision(len(_precision_index)-1) {
|
||||
return "precision(" + strconv.FormatInt(int64(i), 10) + ")"
|
||||
}
|
||||
return _precision_name[_precision_index[i]:_precision_index[i+1]]
|
||||
}
|
|
@ -1,257 +0,0 @@
|
|||
package gen
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
)
|
||||
|
||||
type Visitor interface {
|
||||
Visit(node SchemaNode) (w Visitor)
|
||||
}
|
||||
|
||||
type SchemaNode interface {
|
||||
node()
|
||||
}
|
||||
|
||||
type Schema struct {
|
||||
Title string
|
||||
Version string
|
||||
SeriesLimit *SeriesLimit `toml:"series-limit"`
|
||||
Measurements Measurements
|
||||
}
|
||||
|
||||
func (*Schema) node() {}
|
||||
|
||||
type Measurements []Measurement
|
||||
|
||||
func (Measurements) node() {}
|
||||
|
||||
type Tags []Tag
|
||||
|
||||
func (Tags) node() {}
|
||||
|
||||
type Fields []Field
|
||||
|
||||
func (Fields) node() {}
|
||||
|
||||
type Measurement struct {
|
||||
Name string
|
||||
SeriesLimit *SeriesLimit `toml:"series-limit"`
|
||||
Sample *sample
|
||||
Tags Tags
|
||||
Fields Fields
|
||||
}
|
||||
|
||||
func (*Measurement) node() {}
|
||||
|
||||
type TagSource interface {
|
||||
fmt.Stringer
|
||||
SchemaNode
|
||||
tagsource()
|
||||
}
|
||||
|
||||
type Tag struct {
|
||||
Name string
|
||||
Source TagSource
|
||||
}
|
||||
|
||||
func (*Tag) node() {}
|
||||
|
||||
type TagArraySource struct {
|
||||
Values []string
|
||||
}
|
||||
|
||||
func (*TagArraySource) node() {}
|
||||
func (*TagArraySource) tagsource() {}
|
||||
|
||||
func (s *TagArraySource) String() string {
|
||||
return fmt.Sprintf("array, source=%#v", s.Values)
|
||||
}
|
||||
|
||||
type TagSequenceSource struct {
|
||||
Format string
|
||||
Start int64
|
||||
Count int64
|
||||
}
|
||||
|
||||
func (*TagSequenceSource) node() {}
|
||||
func (*TagSequenceSource) tagsource() {}
|
||||
|
||||
func (t *TagSequenceSource) String() string {
|
||||
return fmt.Sprintf("sequence, prefix=%q, range=[%d,%d)", t.Format, t.Start, t.Start+t.Count)
|
||||
}
|
||||
|
||||
type TagFileSource struct {
|
||||
Path string
|
||||
}
|
||||
|
||||
func (*TagFileSource) node() {}
|
||||
func (*TagFileSource) tagsource() {}
|
||||
|
||||
func (s *TagFileSource) String() string {
|
||||
return fmt.Sprintf("file, path=%s", s.Path)
|
||||
}
|
||||
|
||||
type FieldSource interface {
|
||||
fmt.Stringer
|
||||
SchemaNode
|
||||
fieldsource()
|
||||
}
|
||||
|
||||
type Field struct {
|
||||
Name string
|
||||
Count int64
|
||||
TimePrecision *precision `toml:"time-precision"` // TimePrecision determines the precision for generated timestamp values
|
||||
TimeInterval *duration `toml:"time-interval"` // TimeInterval determines the duration between timestamp values
|
||||
Source FieldSource
|
||||
}
|
||||
|
||||
func (t *Field) TimeSequenceSpec() TimeSequenceSpec {
|
||||
if t.TimeInterval != nil {
|
||||
return TimeSequenceSpec{
|
||||
Count: int(t.Count),
|
||||
Delta: t.TimeInterval.Duration,
|
||||
}
|
||||
}
|
||||
|
||||
if t.TimePrecision != nil {
|
||||
return TimeSequenceSpec{
|
||||
Count: int(t.Count),
|
||||
Precision: t.TimePrecision.ToDuration(),
|
||||
}
|
||||
}
|
||||
|
||||
panic("TimeInterval and TimePrecision are nil")
|
||||
}
|
||||
|
||||
func (*Field) node() {}
|
||||
|
||||
type FieldConstantValue struct {
|
||||
Value interface{}
|
||||
}
|
||||
|
||||
func (*FieldConstantValue) node() {}
|
||||
func (*FieldConstantValue) fieldsource() {}
|
||||
|
||||
func (f *FieldConstantValue) String() string {
|
||||
return fmt.Sprintf("constant, source=%#v", f.Value)
|
||||
}
|
||||
|
||||
type FieldArraySource struct {
|
||||
Value interface{}
|
||||
}
|
||||
|
||||
func (*FieldArraySource) node() {}
|
||||
func (*FieldArraySource) fieldsource() {}
|
||||
|
||||
func (f *FieldArraySource) String() string {
|
||||
return fmt.Sprintf("array, source=%#v", f.Value)
|
||||
}
|
||||
|
||||
type FieldFloatRandomSource struct {
|
||||
Seed int64
|
||||
Min, Max float64
|
||||
}
|
||||
|
||||
func (*FieldFloatRandomSource) node() {}
|
||||
func (*FieldFloatRandomSource) fieldsource() {}
|
||||
|
||||
func (f *FieldFloatRandomSource) String() string {
|
||||
return fmt.Sprintf("rand<float>, seed=%d, min=%f, max=%f", f.Seed, f.Max, f.Max)
|
||||
}
|
||||
|
||||
type FieldIntegerZipfSource struct {
|
||||
Seed int64
|
||||
S, V float64
|
||||
IMAX uint64
|
||||
}
|
||||
|
||||
func (*FieldIntegerZipfSource) node() {}
|
||||
func (*FieldIntegerZipfSource) fieldsource() {}
|
||||
|
||||
func (f *FieldIntegerZipfSource) String() string {
|
||||
return fmt.Sprintf("rand<float>, seed=%d, s=%f, v=%f, imax=%d", f.Seed, f.S, f.V, f.IMAX)
|
||||
}
|
||||
|
||||
type VisitorFn func(node SchemaNode) bool
|
||||
|
||||
func (fn VisitorFn) Visit(node SchemaNode) (w Visitor) {
|
||||
if fn(node) {
|
||||
return fn
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// WalkDown performs a pre-order, depth-first traversal of the graph, calling v for each node.
|
||||
// Pre-order starts by calling the visitor for the root and each child as it traverses down
|
||||
// the graph to the leaves.
|
||||
func WalkDown(v Visitor, node SchemaNode) {
|
||||
walk(v, node, false)
|
||||
}
|
||||
|
||||
// WalkUp performs a post-order, depth-first traversal of the graph, calling v for each node.
|
||||
// Post-order starts by calling the visitor for the leaves then each parent as it traverses up
|
||||
// the graph to the root.
|
||||
func WalkUp(v Visitor, node SchemaNode) {
|
||||
walk(v, node, true)
|
||||
}
|
||||
|
||||
func walk(v Visitor, node SchemaNode, up bool) Visitor {
|
||||
if v == nil {
|
||||
return nil
|
||||
}
|
||||
|
||||
if !up {
|
||||
if v = v.Visit(node); v == nil {
|
||||
return nil
|
||||
}
|
||||
}
|
||||
|
||||
switch n := node.(type) {
|
||||
case *Schema:
|
||||
walk(v, n.Measurements, up)
|
||||
|
||||
case Measurements:
|
||||
v := v
|
||||
for i := range n {
|
||||
v = walk(v, &n[i], up)
|
||||
}
|
||||
|
||||
case *Measurement:
|
||||
v := v
|
||||
v = walk(v, n.Tags, up)
|
||||
walk(v, n.Fields, up)
|
||||
|
||||
case Fields:
|
||||
v := v
|
||||
for i := 0; i < len(n); i++ {
|
||||
v = walk(v, &n[i], up)
|
||||
}
|
||||
|
||||
case Tags:
|
||||
v := v
|
||||
for i := 0; i < len(n); i++ {
|
||||
v = walk(v, &n[i], up)
|
||||
}
|
||||
|
||||
case *Tag:
|
||||
walk(v, n.Source, up)
|
||||
|
||||
case *TagArraySource, *TagSequenceSource, *TagFileSource:
|
||||
// nothing to do
|
||||
|
||||
case *Field:
|
||||
walk(v, n.Source, up)
|
||||
|
||||
case *FieldConstantValue, *FieldArraySource, *FieldFloatRandomSource, *FieldIntegerZipfSource:
|
||||
// nothing to do
|
||||
|
||||
default:
|
||||
panic(fmt.Sprintf("schema.Walk: unexpected node type %T", n))
|
||||
}
|
||||
|
||||
if up && v != nil {
|
||||
v = v.Visit(node)
|
||||
}
|
||||
|
||||
return v
|
||||
}
|
|
@ -1,96 +0,0 @@
|
|||
package gen
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"math"
|
||||
)
|
||||
|
||||
type Sequence interface {
|
||||
Next() bool
|
||||
Value() string
|
||||
}
|
||||
|
||||
type CountableSequence interface {
|
||||
Sequence
|
||||
Count() int
|
||||
}
|
||||
|
||||
type CounterByteSequence struct {
|
||||
format string
|
||||
nfmt string
|
||||
val string
|
||||
s int
|
||||
i int
|
||||
end int
|
||||
}
|
||||
|
||||
func NewCounterByteSequenceCount(n int) *CounterByteSequence {
|
||||
return NewCounterByteSequence("value%s", 0, n)
|
||||
}
|
||||
|
||||
func NewCounterByteSequence(format string, start, end int) *CounterByteSequence {
|
||||
s := &CounterByteSequence{
|
||||
format: format,
|
||||
nfmt: fmt.Sprintf("%%0%dd", int(math.Ceil(math.Log10(float64(end))))),
|
||||
s: start,
|
||||
i: start,
|
||||
end: end,
|
||||
}
|
||||
s.update()
|
||||
return s
|
||||
}
|
||||
|
||||
func (s *CounterByteSequence) Next() bool {
|
||||
s.i++
|
||||
if s.i >= s.end {
|
||||
s.i = s.s
|
||||
}
|
||||
s.update()
|
||||
return true
|
||||
}
|
||||
|
||||
func (s *CounterByteSequence) update() {
|
||||
s.val = fmt.Sprintf(s.format, fmt.Sprintf(s.nfmt, s.i))
|
||||
}
|
||||
|
||||
func (s *CounterByteSequence) Value() string { return s.val }
|
||||
func (s *CounterByteSequence) Count() int { return s.end - s.s }
|
||||
|
||||
type StringArraySequence struct {
|
||||
vals []string
|
||||
c int
|
||||
i int
|
||||
}
|
||||
|
||||
func NewStringArraySequence(vals []string) *StringArraySequence {
|
||||
return &StringArraySequence{vals: sortDedupStrings(vals)}
|
||||
}
|
||||
|
||||
func (s *StringArraySequence) Next() bool {
|
||||
s.i++
|
||||
if s.i == len(s.vals) {
|
||||
s.i = 0
|
||||
}
|
||||
s.c = s.i
|
||||
return true
|
||||
}
|
||||
|
||||
func (s *StringArraySequence) Value() string {
|
||||
return s.vals[s.c]
|
||||
}
|
||||
|
||||
func (s *StringArraySequence) Count() int {
|
||||
return len(s.vals)
|
||||
}
|
||||
|
||||
type StringConstantSequence struct {
|
||||
val string
|
||||
}
|
||||
|
||||
func NewStringConstantSequence(val string) *StringConstantSequence {
|
||||
return &StringConstantSequence{val: val}
|
||||
}
|
||||
|
||||
func (s *StringConstantSequence) Next() bool { return true }
|
||||
func (s *StringConstantSequence) Value() string { return s.val }
|
||||
func (s *StringConstantSequence) Count() int { return 1 }
|
|
@ -1,63 +0,0 @@
|
|||
package gen
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
)
|
||||
|
||||
type seriesKeyField interface {
|
||||
// Key returns the series key.
|
||||
// The returned value may be cached.
|
||||
Key() []byte
|
||||
|
||||
// Field returns the name of the field.
|
||||
// The returned value may be modified by a subsequent call to Next.
|
||||
Field() []byte
|
||||
}
|
||||
|
||||
type constSeries struct {
|
||||
key []byte
|
||||
field []byte
|
||||
}
|
||||
|
||||
func (s *constSeries) Key() []byte { return s.key }
|
||||
func (s *constSeries) Field() []byte { return s.field }
|
||||
|
||||
var nilSeries seriesKeyField = &constSeries{}
|
||||
|
||||
// Compare returns an integer comparing two SeriesGenerator instances
|
||||
// lexicographically.
|
||||
// The result will be 0 if a==b, -1 if a < b, and +1 if a > b.
|
||||
// A nil argument is equivalent to an empty SeriesGenerator.
|
||||
func CompareSeries(a, b seriesKeyField) int {
|
||||
if a == nil {
|
||||
a = nilSeries
|
||||
}
|
||||
if b == nil {
|
||||
b = nilSeries
|
||||
}
|
||||
|
||||
switch res := bytes.Compare(a.Key(), b.Key()); res {
|
||||
case 0:
|
||||
return bytes.Compare(a.Field(), b.Field())
|
||||
default:
|
||||
return res
|
||||
}
|
||||
}
|
||||
|
||||
func (s *constSeries) CopyFrom(a seriesKeyField) {
|
||||
key := a.Key()
|
||||
if cap(s.key) < len(key) {
|
||||
s.key = make([]byte, len(key))
|
||||
} else {
|
||||
s.key = s.key[:len(key)]
|
||||
}
|
||||
copy(s.key, key)
|
||||
|
||||
field := a.Field()
|
||||
if cap(s.field) < len(field) {
|
||||
s.field = make([]byte, len(field))
|
||||
} else {
|
||||
s.field = s.field[:len(field)]
|
||||
}
|
||||
copy(s.field, field)
|
||||
}
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue