chore: Converging on 1.x tsdb

pull/19446/head
Stuart Carnie 2020-04-24 09:47:22 -07:00
parent 2b175291be
commit 26c18ac0a5
No known key found for this signature in database
GPG Key ID: 848D9C9718D78B4F
146 changed files with 2197 additions and 20722 deletions

View File

@ -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"
}

View File

@ -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
}

View File

@ -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" },
]
`

View File

@ -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
}

View File

@ -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
}

View File

@ -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
}

View File

@ -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()
}

View File

@ -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
}

View File

@ -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"
}

View File

@ -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
}

View File

@ -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()
}

View File

@ -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
}

View File

@ -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
},
}
}

View File

@ -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
}

View File

@ -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...)

View File

@ -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
}

View File

@ -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
}

View File

@ -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
}

View File

@ -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()
}

View File

@ -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
}

View File

@ -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)

View File

@ -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"

View File

@ -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",

View File

@ -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.

View File

@ -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.

View File

@ -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")
}

View File

@ -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"
)

View File

@ -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...)
}

View File

@ -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...)
}

View File

@ -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 {

View File

@ -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()

View File

@ -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 {

View File

@ -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)))

View File

@ -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 {

View File

@ -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 {

View File

@ -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]

View File

@ -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]

View File

@ -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)
}

View File

@ -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.

View File

@ -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
}

View File

@ -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)

View File

@ -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
}

View File

@ -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

View File

@ -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 . */}}

View File

@ -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"
}
]
}
]

View File

@ -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)
}

View File

@ -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)
}
})
}

View File

@ -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

View File

@ -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
}

View File

@ -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)
}
}

View File

@ -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) {

View File

@ -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

View File

@ -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) {

View File

@ -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) {

View File

@ -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() }

View File

@ -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"
)

View File

@ -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()
}

View File

@ -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
}

View File

@ -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
}

View File

@ -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"
)

View File

@ -1,7 +1,7 @@
package reads
import (
"github.com/influxdata/influxdb/v2/models"
"github.com/influxdata/influxdb/v2/v1/models"
)
type tagsBuffer struct {

View File

@ -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"
}
]

View File

@ -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)
}

View File

@ -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)
}

View File

@ -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 }

View File

@ -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))
}

View File

@ -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
}

View File

@ -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()))
}
}

View File

@ -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
}

View File

@ -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)
}
}

View File

@ -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)
}

View File

@ -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,
}
}

View File

@ -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)
}
}
}
}

View File

@ -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)
}

View File

@ -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()
}

View File

@ -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
}

View 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
}

File diff suppressed because it is too large Load Diff

View File

@ -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)
}

View File

@ -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)
}

View File

@ -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})
}

View File

@ -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()

View File

@ -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)
}

View File

@ -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}}

View File

@ -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

View File

@ -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
}

View File

@ -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))
}
})
}
}

View File

@ -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]]
}

View File

@ -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
}

View File

@ -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 }

View File

@ -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