diff --git a/cmd/influx_inspect/buildtsi/buildtsi.go b/cmd/influx_inspect/buildtsi/buildtsi.go index 07b9aca4f8..7520707d18 100644 --- a/cmd/influx_inspect/buildtsi/buildtsi.go +++ b/cmd/influx_inspect/buildtsi/buildtsi.go @@ -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" } diff --git a/cmd/influxd/generate/command.go b/cmd/influxd/generate/command.go deleted file mode 100644 index eed3336069..0000000000 --- a/cmd/influxd/generate/command.go +++ /dev/null @@ -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 ", - 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 -} diff --git a/cmd/influxd/generate/command_helpschema.go b/cmd/influxd/generate/command_helpschema.go deleted file mode 100644 index 411536edbf..0000000000 --- a/cmd/influxd/generate/command_helpschema.go +++ /dev/null @@ -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 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" -# 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" -# 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", 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" }, -] -` diff --git a/cmd/influxd/generate/command_simple.go b/cmd/influxd/generate/command_simple.go deleted file mode 100644 index 25f4b36130..0000000000 --- a/cmd/influxd/generate/command_simple.go +++ /dev/null @@ -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 -} diff --git a/cmd/influxd/generate/generator.go b/cmd/influxd/generate/generator.go deleted file mode 100644 index d982c5092a..0000000000 --- a/cmd/influxd/generate/generator.go +++ /dev/null @@ -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 -} diff --git a/cmd/influxd/generate/internal/shard/writer.go b/cmd/influxd/generate/internal/shard/writer.go deleted file mode 100644 index d3af2c0b08..0000000000 --- a/cmd/influxd/generate/internal/shard/writer.go +++ /dev/null @@ -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 -} diff --git a/cmd/influxd/generate/plan.go b/cmd/influxd/generate/plan.go deleted file mode 100644 index 55e6acaac2..0000000000 --- a/cmd/influxd/generate/plan.go +++ /dev/null @@ -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() -} diff --git a/cmd/influxd/generate/spec.go b/cmd/influxd/generate/spec.go deleted file mode 100644 index 52b63ac286..0000000000 --- a/cmd/influxd/generate/spec.go +++ /dev/null @@ -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 -} diff --git a/cmd/influxd/inspect/build_tsi.go b/cmd/influxd/inspect/build_tsi.go deleted file mode 100644 index 32124d5b28..0000000000 --- a/cmd/influxd/inspect/build_tsi.go +++ /dev/null @@ -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/data - WALPath string // optional. Defaults to /engine/wal - SeriesFilePath string // optional. Defaults to /engine/_series - IndexPath string // optional. Defaults to /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" -} diff --git a/cmd/influxd/inspect/compact_series_file.go b/cmd/influxd/inspect/compact_series_file.go deleted file mode 100644 index 86fceb8d75..0000000000 --- a/cmd/influxd/inspect/compact_series_file.go +++ /dev/null @@ -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/_series - IndexPath string // optional. Defaults to /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 -} diff --git a/cmd/influxd/inspect/dump_tsi1.go b/cmd/influxd/inspect/dump_tsi1.go deleted file mode 100644 index e9b8062672..0000000000 --- a/cmd/influxd/inspect/dump_tsi1.go +++ /dev/null @@ -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() -} diff --git a/cmd/influxd/inspect/dump_wal.go b/cmd/influxd/inspect/dump_wal.go deleted file mode 100644 index 35fe9bf7cc..0000000000 --- a/cmd/influxd/inspect/dump_wal.go +++ /dev/null @@ -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 -} diff --git a/cmd/influxd/inspect/export_blocks.go b/cmd/influxd/inspect/export_blocks.go deleted file mode 100644 index d630804157..0000000000 --- a/cmd/influxd/inspect/export_blocks.go +++ /dev/null @@ -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 - }, - } -} diff --git a/cmd/influxd/inspect/export_index.go b/cmd/influxd/inspect/export_index.go deleted file mode 100644 index 38cc95802a..0000000000 --- a/cmd/influxd/inspect/export_index.go +++ /dev/null @@ -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 -} diff --git a/cmd/influxd/inspect/inspect.go b/cmd/influxd/inspect/inspect.go index 368b65696a..6d368335dc 100644 --- a/cmd/influxd/inspect/inspect.go +++ b/cmd/influxd/inspect/inspect.go @@ -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...) diff --git a/cmd/influxd/inspect/report_tsi1.go b/cmd/influxd/inspect/report_tsi1.go deleted file mode 100644 index 82bf6b0356..0000000000 --- a/cmd/influxd/inspect/report_tsi1.go +++ /dev/null @@ -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 -} diff --git a/cmd/influxd/inspect/report_tsm.go b/cmd/influxd/inspect/report_tsm.go deleted file mode 100644 index 4caeebbb8b..0000000000 --- a/cmd/influxd/inspect/report_tsm.go +++ /dev/null @@ -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 -} diff --git a/cmd/influxd/inspect/verify_series.go b/cmd/influxd/inspect/verify_series.go deleted file mode 100644 index b590476aef..0000000000 --- a/cmd/influxd/inspect/verify_series.go +++ /dev/null @@ -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 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 -} diff --git a/cmd/influxd/inspect/verify_tsm.go b/cmd/influxd/inspect/verify_tsm.go deleted file mode 100644 index 7ab647fbd4..0000000000 --- a/cmd/influxd/inspect/verify_tsm.go +++ /dev/null @@ -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 ...", - 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 - - ... - 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() -} diff --git a/cmd/influxd/inspect/verify_wal.go b/cmd/influxd/inspect/verify_wal.go deleted file mode 100644 index 8dffe43a80..0000000000 --- a/cmd/influxd/inspect/verify_wal.go +++ /dev/null @@ -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 -} diff --git a/cmd/influxd/launcher/engine.go b/cmd/influxd/launcher/engine.go index 1ba180c11b..c26ce4144b 100644 --- a/cmd/influxd/launcher/engine.go +++ b/cmd/influxd/launcher/engine.go @@ -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) diff --git a/cmd/influxd/launcher/launcher.go b/cmd/influxd/launcher/launcher.go index d933dd7f8c..ce93315de8 100644 --- a/cmd/influxd/launcher/launcher.go +++ b/cmd/influxd/launcher/launcher.go @@ -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" diff --git a/cmd/influxd/main.go b/cmd/influxd/main.go index a08a01c15e..4264971e55 100644 --- a/cmd/influxd/main.go +++ b/cmd/influxd/main.go @@ -9,12 +9,10 @@ import ( "github.com/influxdata/flux" "github.com/influxdata/influxdb/v2" - "github.com/influxdata/influxdb/v2/cmd/influxd/generate" "github.com/influxdata/influxdb/v2/cmd/influxd/launcher" - "github.com/influxdata/influxdb/v2/cmd/influxd/restore" _ "github.com/influxdata/influxdb/v2/query/builtin" - _ "github.com/influxdata/influxdb/v2/v1/tsdb/index/tsi1" _ "github.com/influxdata/influxdb/v2/v1/tsdb/engine/tsm1" + _ "github.com/influxdata/influxdb/v2/v1/tsdb/index/tsi1" "github.com/spf13/cobra" ) @@ -32,8 +30,9 @@ func main() { influxdb.SetBuildInfo(version, commit, date) rootCmd := launcher.NewInfluxdCommand(context.Background(), - generate.Command, - restore.Command, + // FIXME + //generate.Command, + //restore.Command, &cobra.Command{ Use: "version", Short: "Print the influxd server version", diff --git a/gather/metrics.go b/gather/metrics.go index 6ce4007290..3fcbfb8385 100644 --- a/gather/metrics.go +++ b/gather/metrics.go @@ -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. diff --git a/gather/recorder.go b/gather/recorder.go index 614db95002..948322fd02 100644 --- a/gather/recorder.go +++ b/gather/recorder.go @@ -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. diff --git a/kit/cli/idflag.go b/kit/cli/idflag.go index 8f4a9689b8..81c02c0943 100644 --- a/kit/cli/idflag.go +++ b/kit/cli/idflag.go @@ -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") } diff --git a/mock/reads_resultset.go b/mock/reads_resultset.go index 16afa54bad..b96669f85b 100644 --- a/mock/reads_resultset.go +++ b/mock/reads_resultset.go @@ -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" ) diff --git a/pkg/data/gen/arrays.gen.go b/pkg/data/gen/arrays.gen.go index 666832c58e..93171ebcee 100644 --- a/pkg/data/gen/arrays.gen.go +++ b/pkg/data/gen/arrays.gen.go @@ -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...) } diff --git a/pkg/data/gen/arrays.gen.go.tmpl b/pkg/data/gen/arrays.gen.go.tmpl index bd854e3400..41e1f3ac0f 100644 --- a/pkg/data/gen/arrays.gen.go.tmpl +++ b/pkg/data/gen/arrays.gen.go.tmpl @@ -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...) } diff --git a/pkg/data/gen/merged_series_generator.go b/pkg/data/gen/merged_series_generator.go index cad08685df..31899f7b30 100644 --- a/pkg/data/gen/merged_series_generator.go +++ b/pkg/data/gen/merged_series_generator.go @@ -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 { diff --git a/pkg/data/gen/merged_series_generator_test.go b/pkg/data/gen/merged_series_generator_test.go index 8dd3aca286..0d22907b4f 100644 --- a/pkg/data/gen/merged_series_generator_test.go +++ b/pkg/data/gen/merged_series_generator_test.go @@ -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() diff --git a/pkg/data/gen/series_generator.go b/pkg/data/gen/series_generator.go index 0368baec0f..790876e63b 100644 --- a/pkg/data/gen/series_generator.go +++ b/pkg/data/gen/series_generator.go @@ -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 { diff --git a/pkg/data/gen/specs.go b/pkg/data/gen/specs.go index ef3febff14..458b1ef32d 100644 --- a/pkg/data/gen/specs.go +++ b/pkg/data/gen/specs.go @@ -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))) diff --git a/pkg/data/gen/specs_test.go b/pkg/data/gen/specs_test.go index aef0149053..8ba9c8627f 100644 --- a/pkg/data/gen/specs_test.go +++ b/pkg/data/gen/specs_test.go @@ -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 { diff --git a/pkg/data/gen/tags_sequence.go b/pkg/data/gen/tags_sequence.go index 0bc69c2d66..3860c0db42 100644 --- a/pkg/data/gen/tags_sequence.go +++ b/pkg/data/gen/tags_sequence.go @@ -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 { diff --git a/pkg/data/gen/values_sequence.gen.go b/pkg/data/gen/values_sequence.gen.go index 73a58a0b99..0e76441028 100644 --- a/pkg/data/gen/values_sequence.gen.go +++ b/pkg/data/gen/values_sequence.gen.go @@ -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] diff --git a/pkg/data/gen/values_sequence.gen.go.tmpl b/pkg/data/gen/values_sequence.gen.go.tmpl index 0c890006ed..b39e95f10d 100644 --- a/pkg/data/gen/values_sequence.gen.go.tmpl +++ b/pkg/data/gen/values_sequence.gen.go.tmpl @@ -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] diff --git a/storage/config.go b/storage/config.go index f6d6169dd1..00dbd58a91 100644 --- a/storage/config.go +++ b/storage/config.go @@ -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) -} diff --git a/storage/engine.go b/storage/engine.go index 1450e6cc50..66e5be0503 100644 --- a/storage/engine.go +++ b/storage/engine.go @@ -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. diff --git a/storage/engine_measurement_schema.go b/storage/engine_measurement_schema.go deleted file mode 100644 index 255a2dc87e..0000000000 --- a/storage/engine_measurement_schema.go +++ /dev/null @@ -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 -} diff --git a/storage/reads/Makefile b/storage/reads/Makefile deleted file mode 100644 index 891a654b8a..0000000000 --- a/storage/reads/Makefile +++ /dev/null @@ -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) diff --git a/storage/reads/aggregate_resultset.go b/storage/reads/aggregate_resultset.go deleted file mode 100644 index 95f4ae7c5f..0000000000 --- a/storage/reads/aggregate_resultset.go +++ /dev/null @@ -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 -} diff --git a/storage/reads/aggregate_resultset_test.go b/storage/reads/aggregate_resultset_test.go deleted file mode 100644 index 4fbb4e3692..0000000000 --- a/storage/reads/aggregate_resultset_test.go +++ /dev/null @@ -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) - } -} diff --git a/storage/reads/array_cursor.gen.go b/storage/reads/array_cursor.gen.go index 434265878c..76d26499e0 100644 --- a/storage/reads/array_cursor.gen.go +++ b/storage/reads/array_cursor.gen.go @@ -8,8 +8,6 @@ package reads import ( "errors" - "fmt" - "math" "github.com/influxdata/influxdb/v2/v1/tsdb/cursors" ) @@ -21,172 +19,6 @@ const ( MaxPointsPerBlock = 1000 ) -func newLimitArrayCursor(cur cursors.Cursor) cursors.Cursor { - switch cur := cur.(type) { - - case cursors.FloatArrayCursor: - return newFloatLimitArrayCursor(cur) - - case cursors.IntegerArrayCursor: - return newIntegerLimitArrayCursor(cur) - - case cursors.UnsignedArrayCursor: - return newUnsignedLimitArrayCursor(cur) - - case cursors.StringArrayCursor: - return newStringLimitArrayCursor(cur) - - case cursors.BooleanArrayCursor: - return newBooleanLimitArrayCursor(cur) - - 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) { - - case cursors.FloatArrayCursor: - return newFloatWindowFirstArrayCursor(cur, every, offset) - - case cursors.IntegerArrayCursor: - return newIntegerWindowFirstArrayCursor(cur, every, offset) - - case cursors.UnsignedArrayCursor: - return newUnsignedWindowFirstArrayCursor(cur, every, offset) - - case cursors.StringArrayCursor: - return newStringWindowFirstArrayCursor(cur, every, offset) - - case cursors.BooleanArrayCursor: - return newBooleanWindowFirstArrayCursor(cur, every, offset) - - 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) { - - case cursors.FloatArrayCursor: - return newFloatWindowLastArrayCursor(cur, every, offset) - - case cursors.IntegerArrayCursor: - return newIntegerWindowLastArrayCursor(cur, every, offset) - - case cursors.UnsignedArrayCursor: - return newUnsignedWindowLastArrayCursor(cur, every, offset) - - case cursors.StringArrayCursor: - return newStringWindowLastArrayCursor(cur, every, offset) - - case cursors.BooleanArrayCursor: - return newBooleanWindowLastArrayCursor(cur, every, offset) - - default: - panic(fmt.Sprintf("unreachable: %T", cur)) - } -} - -func newWindowCountArrayCursor(cur cursors.Cursor, every, offset int64) cursors.Cursor { - switch cur := cur.(type) { - - case cursors.FloatArrayCursor: - return newFloatWindowCountArrayCursor(cur, every, offset) - - case cursors.IntegerArrayCursor: - return newIntegerWindowCountArrayCursor(cur, every, offset) - - case cursors.UnsignedArrayCursor: - return newUnsignedWindowCountArrayCursor(cur, every, offset) - - case cursors.StringArrayCursor: - return newStringWindowCountArrayCursor(cur, every, offset) - - case cursors.BooleanArrayCursor: - return newBooleanWindowCountArrayCursor(cur, every, offset) - - default: - panic(fmt.Sprintf("unreachable: %T", cur)) - } -} - -func newWindowSumArrayCursor(cur cursors.Cursor, every, offset int64) cursors.Cursor { - switch cur := cur.(type) { - - case cursors.FloatArrayCursor: - return newFloatWindowSumArrayCursor(cur, every, offset) - - case cursors.IntegerArrayCursor: - return newIntegerWindowSumArrayCursor(cur, every, offset) - - case cursors.UnsignedArrayCursor: - return newUnsignedWindowSumArrayCursor(cur, every, offset) - - default: - panic(fmt.Sprintf("unsupported for aggregate sum: %T", cur)) - } -} - -func newWindowMinArrayCursor(cur cursors.Cursor, every, offset int64) cursors.Cursor { - switch cur := cur.(type) { - - case cursors.FloatArrayCursor: - return newFloatWindowMinArrayCursor(cur, every, offset) - - case cursors.IntegerArrayCursor: - return newIntegerWindowMinArrayCursor(cur, every, offset) - - case cursors.UnsignedArrayCursor: - return newUnsignedWindowMinArrayCursor(cur, every, offset) - - default: - panic(fmt.Sprintf("unsupported for aggregate min: %T", cur)) - } -} - -func newWindowMaxArrayCursor(cur cursors.Cursor, every, offset int64) cursors.Cursor { - switch cur := cur.(type) { - - case cursors.FloatArrayCursor: - return newFloatWindowMaxArrayCursor(cur, every, offset) - - case cursors.IntegerArrayCursor: - return newIntegerWindowMaxArrayCursor(cur, every, offset) - - case cursors.UnsignedArrayCursor: - return newUnsignedWindowMaxArrayCursor(cur, every, offset) - - default: - panic(fmt.Sprintf("unsupported for aggregate max: %T", cur)) - } -} - -func newWindowMeanArrayCursor(cur cursors.Cursor, every, offset int64) cursors.Cursor { - switch cur := cur.(type) { - - case cursors.FloatArrayCursor: - return newFloatWindowMeanArrayCursor(cur, every, offset) - - case cursors.IntegerArrayCursor: - return newIntegerWindowMeanArrayCursor(cur, every, offset) - - case cursors.UnsignedArrayCursor: - return newUnsignedWindowMeanArrayCursor(cur, every, offset) - - default: - panic(fmt.Sprintf("unsupported for aggregate mean: %T", cur)) - } -} - // ******************** // Float Array Cursor @@ -223,6 +55,8 @@ func (c *floatArrayFilterCursor) Next() *cursors.FloatArray { if c.tmp.Len() > 0 { a = c.tmp + c.tmp.Timestamps = nil + c.tmp.Values = nil } else { a = c.FloatArrayCursor.Next() } @@ -242,12 +76,6 @@ 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.FloatArrayCursor.Next() } @@ -257,13 +85,13 @@ LOOP: return c.res } -type floatArrayCursor struct { +type floatMultiShardArrayCursor struct { cursors.FloatArrayCursor cursorContext filter *floatArrayFilterCursor } -func (c *floatArrayCursor) reset(cur cursors.FloatArrayCursor, cursorIterator cursors.CursorIterator, cond expression) { +func (c *floatMultiShardArrayCursor) reset(cur cursors.FloatArrayCursor, itrs cursors.CursorIterators, cond expression) { if cond != nil { if c.filter == nil { c.filter = newFloatFilterArrayCursor(cond) @@ -273,17 +101,18 @@ func (c *floatArrayCursor) reset(cur cursors.FloatArrayCursor, cursorIterator cu } c.FloatArrayCursor = cur - c.cursorIterator = cursorIterator + c.itrs = itrs c.err = nil + c.count = 0 } -func (c *floatArrayCursor) Err() error { return c.err } +func (c *floatMultiShardArrayCursor) Err() error { return c.err } -func (c *floatArrayCursor) Stats() cursors.CursorStats { +func (c *floatMultiShardArrayCursor) Stats() cursors.CursorStats { return c.FloatArrayCursor.Stats() } -func (c *floatArrayCursor) Next() *cursors.FloatArray { +func (c *floatMultiShardArrayCursor) Next() *cursors.FloatArray { for { a := c.FloatArrayCursor.Next() if a.Len() == 0 { @@ -291,19 +120,31 @@ func (c *floatArrayCursor) Next() *cursors.FloatArray { continue } } + c.count += int64(a.Len()) + if c.count > c.limit { + diff := c.count - c.limit + c.count -= diff + rem := int64(a.Len()) - diff + a.Timestamps = a.Timestamps[:rem] + a.Values = a.Values[:rem] + } return a } } -func (c *floatArrayCursor) nextArrayCursor() bool { - if c.cursorIterator == nil { +func (c *floatMultiShardArrayCursor) nextArrayCursor() bool { + if len(c.itrs) == 0 { return false } c.FloatArrayCursor.Close() - cur, _ := c.cursorIterator.Next(c.ctx, c.req) - c.cursorIterator = nil + var itr cursors.CursorIterator + var cur cursors.Cursor + for cur == nil && len(c.itrs) > 0 { + itr, c.itrs = c.itrs[0], c.itrs[1:] + cur, _ = itr.Next(c.ctx, c.req) + } var ok bool if cur != nil { @@ -312,7 +153,7 @@ func (c *floatArrayCursor) nextArrayCursor() bool { if !ok { cur.Close() next = FloatEmptyArrayCursor - c.cursorIterator = nil + c.itrs = nil c.err = errors.New("expected float cursor") } else { if c.filter != nil { @@ -328,738 +169,72 @@ func (c *floatArrayCursor) nextArrayCursor() bool { return ok } -type floatLimitArrayCursor struct { +type floatArraySumCursor struct { cursors.FloatArrayCursor - res *cursors.FloatArray - done bool + ts [1]int64 + vs [1]float64 + res *cursors.FloatArray } -func newFloatLimitArrayCursor(cur cursors.FloatArrayCursor) *floatLimitArrayCursor { - return &floatLimitArrayCursor{ +func newFloatArraySumCursor(cur cursors.FloatArrayCursor) *floatArraySumCursor { + return &floatArraySumCursor{ FloatArrayCursor: cur, - res: cursors.NewFloatArrayLen(1), + res: &cursors.FloatArray{}, } } -func (c *floatLimitArrayCursor) Stats() cursors.CursorStats { return c.FloatArrayCursor.Stats() } +func (c floatArraySumCursor) Stats() cursors.CursorStats { return c.FloatArrayCursor.Stats() } -func (c *floatLimitArrayCursor) Next() *cursors.FloatArray { - if c.done { - return &cursors.FloatArray{} - } +func (c floatArraySumCursor) Next() *cursors.FloatArray { a := c.FloatArrayCursor.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 floatWindowLastArrayCursor struct { - cursors.FloatArrayCursor - every, offset, windowEnd int64 - res *cursors.FloatArray - tmp *cursors.FloatArray -} + ts := a.Timestamps[0] + var acc float64 -// 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 newFloatWindowLastArrayCursor(cur cursors.FloatArrayCursor, every, offset int64) *floatWindowLastArrayCursor { - return &floatWindowLastArrayCursor{ - FloatArrayCursor: cur, - every: every, - offset: offset, - windowEnd: math.MinInt64, - res: cursors.NewFloatArrayLen(MaxPointsPerBlock), - tmp: &cursors.FloatArray{}, - } -} - -func (c *floatWindowLastArrayCursor) Stats() cursors.CursorStats { - return c.FloatArrayCursor.Stats() -} - -func (c *floatWindowLastArrayCursor) Next() *cursors.FloatArray { - cur := -1 - -NEXT: - var a *cursors.FloatArray - - if c.tmp.Len() > 0 { - a = c.tmp - } else { + for { + for _, v := range a.Values { + acc += v + } a = c.FloatArrayCursor.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 floatWindowFirstArrayCursor struct { - cursors.FloatArrayCursor - every, offset, windowEnd int64 - res *cursors.FloatArray - tmp *cursors.FloatArray -} - -// 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 newFloatWindowFirstArrayCursor(cur cursors.FloatArrayCursor, every, offset int64) *floatWindowFirstArrayCursor { - return &floatWindowFirstArrayCursor{ - FloatArrayCursor: cur, - every: every, - offset: offset, - windowEnd: math.MinInt64, - res: cursors.NewFloatArrayLen(MaxPointsPerBlock), - tmp: &cursors.FloatArray{}, - } -} - -func (c *floatWindowFirstArrayCursor) Stats() cursors.CursorStats { - return c.FloatArrayCursor.Stats() -} - -func (c *floatWindowFirstArrayCursor) Next() *cursors.FloatArray { - c.res.Timestamps = c.res.Timestamps[:0] - c.res.Values = c.res.Values[:0] - -NEXT: - var a *cursors.FloatArray - - if c.tmp.Len() > 0 { - a = c.tmp - } else { - a = c.FloatArrayCursor.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:] + if len(a.Timestamps) == 0 { + c.ts[0] = ts + c.vs[0] = acc + c.res.Timestamps = c.ts[:] + c.res.Values = c.vs[:] return c.res } } - - c.tmp.Timestamps = nil - c.tmp.Values = nil - - goto NEXT } -type floatWindowCountArrayCursor struct { +type integerFloatCountArrayCursor struct { cursors.FloatArrayCursor - every, offset int64 - res *cursors.IntegerArray - tmp *cursors.FloatArray } -func newFloatWindowCountArrayCursor(cur cursors.FloatArrayCursor, every, offset int64) *floatWindowCountArrayCursor { - resLen := MaxPointsPerBlock - if every == 0 { - resLen = 1 - } - return &floatWindowCountArrayCursor{ - FloatArrayCursor: cur, - every: every, - offset: offset, - res: cursors.NewIntegerArrayLen(resLen), - tmp: &cursors.FloatArray{}, - } -} - -func (c *floatWindowCountArrayCursor) Stats() cursors.CursorStats { +func (c *integerFloatCountArrayCursor) Stats() cursors.CursorStats { return c.FloatArrayCursor.Stats() } -func (c *floatWindowCountArrayCursor) Next() *cursors.IntegerArray { - 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.FloatArray - if c.tmp.Len() > 0 { - a = c.tmp - } else { - a = c.FloatArrayCursor.Next() - } - - if a.Len() == 0 { +func (c *integerFloatCountArrayCursor) Next() *cursors.IntegerArray { + a := c.FloatArrayCursor.Next() + if len(a.Timestamps) == 0 { return &cursors.IntegerArray{} } - rowIdx := 0 - var acc int64 = 0 - - 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: + ts := a.Timestamps[0] + var acc int64 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 { - c.res.Timestamps[pos] = windowEnd - c.res.Values[pos] = acc - 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 - acc = 0 - windowEnd = WindowStop(a.Timestamps[rowIdx], c.every, c.offset) - windowHasPoints = false - - continue WINDOWS - } else { - acc++ - 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 + acc += int64(len(a.Timestamps)) a = c.FloatArrayCursor.Next() - if a.Len() == 0 { - // write the final point - // do not generate a point for empty windows - if windowHasPoints { - c.res.Timestamps[pos] = windowEnd - c.res.Values[pos] = acc - pos++ - } - break WINDOWS + if len(a.Timestamps) == 0 { + res := cursors.NewIntegerArrayLen(1) + res.Timestamps[0] = ts + res.Values[0] = acc + return res } - rowIdx = 0 } - - c.res.Timestamps = c.res.Timestamps[:pos] - c.res.Values = c.res.Values[:pos] - - return c.res -} - -type floatWindowSumArrayCursor struct { - cursors.FloatArrayCursor - every, offset int64 - res *cursors.FloatArray - tmp *cursors.FloatArray -} - -func newFloatWindowSumArrayCursor(cur cursors.FloatArrayCursor, every, offset int64) *floatWindowSumArrayCursor { - resLen := MaxPointsPerBlock - if every == 0 { - resLen = 1 - } - return &floatWindowSumArrayCursor{ - FloatArrayCursor: cur, - every: every, - offset: offset, - res: cursors.NewFloatArrayLen(resLen), - tmp: &cursors.FloatArray{}, - } -} - -func (c *floatWindowSumArrayCursor) Stats() cursors.CursorStats { - return c.FloatArrayCursor.Stats() -} - -func (c *floatWindowSumArrayCursor) Next() *cursors.FloatArray { - 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.FloatArray - if c.tmp.Len() > 0 { - a = c.tmp - } else { - a = c.FloatArrayCursor.Next() - } - - if a.Len() == 0 { - return &cursors.FloatArray{} - } - - rowIdx := 0 - var acc float64 = 0 - - 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 { - c.res.Timestamps[pos] = windowEnd - c.res.Values[pos] = acc - 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 - acc = 0 - windowEnd = WindowStop(a.Timestamps[rowIdx], c.every, c.offset) - windowHasPoints = false - - continue WINDOWS - } else { - acc += a.Values[rowIdx] - 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.FloatArrayCursor.Next() - if a.Len() == 0 { - // write the final point - // do not generate a point for empty windows - if windowHasPoints { - c.res.Timestamps[pos] = windowEnd - c.res.Values[pos] = acc - pos++ - } - break WINDOWS - } - rowIdx = 0 - } - - c.res.Timestamps = c.res.Timestamps[:pos] - c.res.Values = c.res.Values[:pos] - - return c.res -} - -type floatWindowMinArrayCursor struct { - cursors.FloatArrayCursor - every, offset int64 - res *cursors.FloatArray - tmp *cursors.FloatArray -} - -func newFloatWindowMinArrayCursor(cur cursors.FloatArrayCursor, every, offset int64) *floatWindowMinArrayCursor { - resLen := MaxPointsPerBlock - if every == 0 { - resLen = 1 - } - return &floatWindowMinArrayCursor{ - FloatArrayCursor: cur, - every: every, - offset: offset, - res: cursors.NewFloatArrayLen(resLen), - tmp: &cursors.FloatArray{}, - } -} - -func (c *floatWindowMinArrayCursor) Stats() cursors.CursorStats { - return c.FloatArrayCursor.Stats() -} - -func (c *floatWindowMinArrayCursor) Next() *cursors.FloatArray { - 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.FloatArray - if c.tmp.Len() > 0 { - a = c.tmp - } else { - a = c.FloatArrayCursor.Next() - } - - if a.Len() == 0 { - return &cursors.FloatArray{} - } - - rowIdx := 0 - var acc float64 = math.MaxFloat64 - var tsAcc int64 - - 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 { - c.res.Timestamps[pos] = tsAcc - c.res.Values[pos] = acc - 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 - acc = math.MaxFloat64 - windowEnd = WindowStop(a.Timestamps[rowIdx], c.every, c.offset) - windowHasPoints = false - - continue WINDOWS - } else { - if !windowHasPoints || a.Values[rowIdx] < acc { - acc = a.Values[rowIdx] - tsAcc = a.Timestamps[rowIdx] - } - 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.FloatArrayCursor.Next() - if a.Len() == 0 { - // write the final point - // do not generate a point for empty windows - if windowHasPoints { - c.res.Timestamps[pos] = tsAcc - c.res.Values[pos] = acc - pos++ - } - break WINDOWS - } - rowIdx = 0 - } - - c.res.Timestamps = c.res.Timestamps[:pos] - c.res.Values = c.res.Values[:pos] - - return c.res -} - -type floatWindowMaxArrayCursor struct { - cursors.FloatArrayCursor - every, offset int64 - res *cursors.FloatArray - tmp *cursors.FloatArray -} - -func newFloatWindowMaxArrayCursor(cur cursors.FloatArrayCursor, every, offset int64) *floatWindowMaxArrayCursor { - resLen := MaxPointsPerBlock - if every == 0 { - resLen = 1 - } - return &floatWindowMaxArrayCursor{ - FloatArrayCursor: cur, - every: every, - offset: offset, - res: cursors.NewFloatArrayLen(resLen), - tmp: &cursors.FloatArray{}, - } -} - -func (c *floatWindowMaxArrayCursor) Stats() cursors.CursorStats { - return c.FloatArrayCursor.Stats() -} - -func (c *floatWindowMaxArrayCursor) Next() *cursors.FloatArray { - 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.FloatArray - if c.tmp.Len() > 0 { - a = c.tmp - } else { - a = c.FloatArrayCursor.Next() - } - - if a.Len() == 0 { - return &cursors.FloatArray{} - } - - rowIdx := 0 - var acc float64 = -math.MaxFloat64 - var tsAcc int64 - - 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 { - c.res.Timestamps[pos] = tsAcc - c.res.Values[pos] = acc - 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 - acc = -math.MaxFloat64 - windowEnd = WindowStop(a.Timestamps[rowIdx], c.every, c.offset) - windowHasPoints = false - - continue WINDOWS - } else { - if !windowHasPoints || a.Values[rowIdx] > acc { - acc = a.Values[rowIdx] - tsAcc = a.Timestamps[rowIdx] - } - 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.FloatArrayCursor.Next() - if a.Len() == 0 { - // write the final point - // do not generate a point for empty windows - if windowHasPoints { - c.res.Timestamps[pos] = tsAcc - c.res.Values[pos] = acc - pos++ - } - break WINDOWS - } - rowIdx = 0 - } - - c.res.Timestamps = c.res.Timestamps[:pos] - c.res.Values = c.res.Values[:pos] - - return c.res -} - -type floatWindowMeanArrayCursor struct { - cursors.FloatArrayCursor - every, offset int64 - res *cursors.FloatArray - tmp *cursors.FloatArray -} - -func newFloatWindowMeanArrayCursor(cur cursors.FloatArrayCursor, every, offset int64) *floatWindowMeanArrayCursor { - resLen := MaxPointsPerBlock - if every == 0 { - resLen = 1 - } - return &floatWindowMeanArrayCursor{ - FloatArrayCursor: cur, - every: every, - offset: offset, - res: cursors.NewFloatArrayLen(resLen), - tmp: &cursors.FloatArray{}, - } -} - -func (c *floatWindowMeanArrayCursor) Stats() cursors.CursorStats { - return c.FloatArrayCursor.Stats() -} - -func (c *floatWindowMeanArrayCursor) Next() *cursors.FloatArray { - 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.FloatArray - if c.tmp.Len() > 0 { - a = c.tmp - } else { - a = c.FloatArrayCursor.Next() - } - - if a.Len() == 0 { - return &cursors.FloatArray{} - } - - rowIdx := 0 - var sum float64 - var count int64 - - 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 { - c.res.Timestamps[pos] = windowEnd - c.res.Values[pos] = sum / float64(count) - 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 - sum = 0 - count = 0 - windowEnd = WindowStop(a.Timestamps[rowIdx], c.every, c.offset) - windowHasPoints = false - - continue WINDOWS - } else { - sum += a.Values[rowIdx] - count++ - 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.FloatArrayCursor.Next() - if a.Len() == 0 { - // write the final point - // do not generate a point for empty windows - if windowHasPoints { - c.res.Timestamps[pos] = windowEnd - c.res.Values[pos] = sum / float64(count) - pos++ - } - break WINDOWS - } - rowIdx = 0 - } - - c.res.Timestamps = c.res.Timestamps[:pos] - c.res.Values = c.res.Values[:pos] - - return c.res } type floatEmptyArrayCursor struct { @@ -1109,6 +284,8 @@ func (c *integerArrayFilterCursor) Next() *cursors.IntegerArray { if c.tmp.Len() > 0 { a = c.tmp + c.tmp.Timestamps = nil + c.tmp.Values = nil } else { a = c.IntegerArrayCursor.Next() } @@ -1128,12 +305,6 @@ 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.IntegerArrayCursor.Next() } @@ -1143,13 +314,13 @@ LOOP: return c.res } -type integerArrayCursor struct { +type integerMultiShardArrayCursor struct { cursors.IntegerArrayCursor cursorContext filter *integerArrayFilterCursor } -func (c *integerArrayCursor) reset(cur cursors.IntegerArrayCursor, cursorIterator cursors.CursorIterator, cond expression) { +func (c *integerMultiShardArrayCursor) reset(cur cursors.IntegerArrayCursor, itrs cursors.CursorIterators, cond expression) { if cond != nil { if c.filter == nil { c.filter = newIntegerFilterArrayCursor(cond) @@ -1159,17 +330,18 @@ func (c *integerArrayCursor) reset(cur cursors.IntegerArrayCursor, cursorIterato } c.IntegerArrayCursor = cur - c.cursorIterator = cursorIterator + c.itrs = itrs c.err = nil + c.count = 0 } -func (c *integerArrayCursor) Err() error { return c.err } +func (c *integerMultiShardArrayCursor) Err() error { return c.err } -func (c *integerArrayCursor) Stats() cursors.CursorStats { +func (c *integerMultiShardArrayCursor) Stats() cursors.CursorStats { return c.IntegerArrayCursor.Stats() } -func (c *integerArrayCursor) Next() *cursors.IntegerArray { +func (c *integerMultiShardArrayCursor) Next() *cursors.IntegerArray { for { a := c.IntegerArrayCursor.Next() if a.Len() == 0 { @@ -1177,19 +349,31 @@ func (c *integerArrayCursor) Next() *cursors.IntegerArray { continue } } + c.count += int64(a.Len()) + if c.count > c.limit { + diff := c.count - c.limit + c.count -= diff + rem := int64(a.Len()) - diff + a.Timestamps = a.Timestamps[:rem] + a.Values = a.Values[:rem] + } return a } } -func (c *integerArrayCursor) nextArrayCursor() bool { - if c.cursorIterator == nil { +func (c *integerMultiShardArrayCursor) nextArrayCursor() bool { + if len(c.itrs) == 0 { return false } c.IntegerArrayCursor.Close() - cur, _ := c.cursorIterator.Next(c.ctx, c.req) - c.cursorIterator = nil + var itr cursors.CursorIterator + var cur cursors.Cursor + for cur == nil && len(c.itrs) > 0 { + itr, c.itrs = c.itrs[0], c.itrs[1:] + cur, _ = itr.Next(c.ctx, c.req) + } var ok bool if cur != nil { @@ -1198,7 +382,7 @@ func (c *integerArrayCursor) nextArrayCursor() bool { if !ok { cur.Close() next = IntegerEmptyArrayCursor - c.cursorIterator = nil + c.itrs = nil c.err = errors.New("expected integer cursor") } else { if c.filter != nil { @@ -1214,738 +398,72 @@ func (c *integerArrayCursor) nextArrayCursor() bool { return ok } -type integerLimitArrayCursor struct { +type integerArraySumCursor struct { cursors.IntegerArrayCursor - res *cursors.IntegerArray - done bool + ts [1]int64 + vs [1]int64 + res *cursors.IntegerArray } -func newIntegerLimitArrayCursor(cur cursors.IntegerArrayCursor) *integerLimitArrayCursor { - return &integerLimitArrayCursor{ +func newIntegerArraySumCursor(cur cursors.IntegerArrayCursor) *integerArraySumCursor { + return &integerArraySumCursor{ IntegerArrayCursor: cur, - res: cursors.NewIntegerArrayLen(1), + res: &cursors.IntegerArray{}, } } -func (c *integerLimitArrayCursor) Stats() cursors.CursorStats { return c.IntegerArrayCursor.Stats() } +func (c integerArraySumCursor) Stats() cursors.CursorStats { return c.IntegerArrayCursor.Stats() } -func (c *integerLimitArrayCursor) Next() *cursors.IntegerArray { - if c.done { - return &cursors.IntegerArray{} - } +func (c integerArraySumCursor) Next() *cursors.IntegerArray { a := c.IntegerArrayCursor.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 integerWindowLastArrayCursor struct { - cursors.IntegerArrayCursor - every, offset, windowEnd int64 - res *cursors.IntegerArray - tmp *cursors.IntegerArray -} + ts := a.Timestamps[0] + var acc int64 -// 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 newIntegerWindowLastArrayCursor(cur cursors.IntegerArrayCursor, every, offset int64) *integerWindowLastArrayCursor { - return &integerWindowLastArrayCursor{ - IntegerArrayCursor: cur, - every: every, - offset: offset, - windowEnd: math.MinInt64, - res: cursors.NewIntegerArrayLen(MaxPointsPerBlock), - tmp: &cursors.IntegerArray{}, - } -} - -func (c *integerWindowLastArrayCursor) Stats() cursors.CursorStats { - return c.IntegerArrayCursor.Stats() -} - -func (c *integerWindowLastArrayCursor) Next() *cursors.IntegerArray { - cur := -1 - -NEXT: - var a *cursors.IntegerArray - - if c.tmp.Len() > 0 { - a = c.tmp - } else { + for { + for _, v := range a.Values { + acc += v + } a = c.IntegerArrayCursor.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 integerWindowFirstArrayCursor struct { - cursors.IntegerArrayCursor - every, offset, windowEnd int64 - res *cursors.IntegerArray - tmp *cursors.IntegerArray -} - -// 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 newIntegerWindowFirstArrayCursor(cur cursors.IntegerArrayCursor, every, offset int64) *integerWindowFirstArrayCursor { - return &integerWindowFirstArrayCursor{ - IntegerArrayCursor: cur, - every: every, - offset: offset, - windowEnd: math.MinInt64, - res: cursors.NewIntegerArrayLen(MaxPointsPerBlock), - tmp: &cursors.IntegerArray{}, - } -} - -func (c *integerWindowFirstArrayCursor) Stats() cursors.CursorStats { - return c.IntegerArrayCursor.Stats() -} - -func (c *integerWindowFirstArrayCursor) Next() *cursors.IntegerArray { - c.res.Timestamps = c.res.Timestamps[:0] - c.res.Values = c.res.Values[:0] - -NEXT: - var a *cursors.IntegerArray - - if c.tmp.Len() > 0 { - a = c.tmp - } else { - a = c.IntegerArrayCursor.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:] + if len(a.Timestamps) == 0 { + c.ts[0] = ts + c.vs[0] = acc + c.res.Timestamps = c.ts[:] + c.res.Values = c.vs[:] return c.res } } - - c.tmp.Timestamps = nil - c.tmp.Values = nil - - goto NEXT } -type integerWindowCountArrayCursor struct { +type integerIntegerCountArrayCursor struct { cursors.IntegerArrayCursor - every, offset int64 - res *cursors.IntegerArray - tmp *cursors.IntegerArray } -func newIntegerWindowCountArrayCursor(cur cursors.IntegerArrayCursor, every, offset int64) *integerWindowCountArrayCursor { - resLen := MaxPointsPerBlock - if every == 0 { - resLen = 1 - } - return &integerWindowCountArrayCursor{ - IntegerArrayCursor: cur, - every: every, - offset: offset, - res: cursors.NewIntegerArrayLen(resLen), - tmp: &cursors.IntegerArray{}, - } -} - -func (c *integerWindowCountArrayCursor) Stats() cursors.CursorStats { +func (c *integerIntegerCountArrayCursor) Stats() cursors.CursorStats { return c.IntegerArrayCursor.Stats() } -func (c *integerWindowCountArrayCursor) Next() *cursors.IntegerArray { - 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.IntegerArray - if c.tmp.Len() > 0 { - a = c.tmp - } else { - a = c.IntegerArrayCursor.Next() - } - - if a.Len() == 0 { +func (c *integerIntegerCountArrayCursor) Next() *cursors.IntegerArray { + a := c.IntegerArrayCursor.Next() + if len(a.Timestamps) == 0 { return &cursors.IntegerArray{} } - rowIdx := 0 - var acc int64 = 0 - - 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: + ts := a.Timestamps[0] + var acc int64 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 { - c.res.Timestamps[pos] = windowEnd - c.res.Values[pos] = acc - 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 - acc = 0 - windowEnd = WindowStop(a.Timestamps[rowIdx], c.every, c.offset) - windowHasPoints = false - - continue WINDOWS - } else { - acc++ - 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 + acc += int64(len(a.Timestamps)) a = c.IntegerArrayCursor.Next() - if a.Len() == 0 { - // write the final point - // do not generate a point for empty windows - if windowHasPoints { - c.res.Timestamps[pos] = windowEnd - c.res.Values[pos] = acc - pos++ - } - break WINDOWS + if len(a.Timestamps) == 0 { + res := cursors.NewIntegerArrayLen(1) + res.Timestamps[0] = ts + res.Values[0] = acc + return res } - rowIdx = 0 } - - c.res.Timestamps = c.res.Timestamps[:pos] - c.res.Values = c.res.Values[:pos] - - return c.res -} - -type integerWindowSumArrayCursor struct { - cursors.IntegerArrayCursor - every, offset int64 - res *cursors.IntegerArray - tmp *cursors.IntegerArray -} - -func newIntegerWindowSumArrayCursor(cur cursors.IntegerArrayCursor, every, offset int64) *integerWindowSumArrayCursor { - resLen := MaxPointsPerBlock - if every == 0 { - resLen = 1 - } - return &integerWindowSumArrayCursor{ - IntegerArrayCursor: cur, - every: every, - offset: offset, - res: cursors.NewIntegerArrayLen(resLen), - tmp: &cursors.IntegerArray{}, - } -} - -func (c *integerWindowSumArrayCursor) Stats() cursors.CursorStats { - return c.IntegerArrayCursor.Stats() -} - -func (c *integerWindowSumArrayCursor) Next() *cursors.IntegerArray { - 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.IntegerArray - if c.tmp.Len() > 0 { - a = c.tmp - } else { - a = c.IntegerArrayCursor.Next() - } - - if a.Len() == 0 { - return &cursors.IntegerArray{} - } - - rowIdx := 0 - var acc int64 = 0 - - 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 { - c.res.Timestamps[pos] = windowEnd - c.res.Values[pos] = acc - 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 - acc = 0 - windowEnd = WindowStop(a.Timestamps[rowIdx], c.every, c.offset) - windowHasPoints = false - - continue WINDOWS - } else { - acc += a.Values[rowIdx] - 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.IntegerArrayCursor.Next() - if a.Len() == 0 { - // write the final point - // do not generate a point for empty windows - if windowHasPoints { - c.res.Timestamps[pos] = windowEnd - c.res.Values[pos] = acc - pos++ - } - break WINDOWS - } - rowIdx = 0 - } - - c.res.Timestamps = c.res.Timestamps[:pos] - c.res.Values = c.res.Values[:pos] - - return c.res -} - -type integerWindowMinArrayCursor struct { - cursors.IntegerArrayCursor - every, offset int64 - res *cursors.IntegerArray - tmp *cursors.IntegerArray -} - -func newIntegerWindowMinArrayCursor(cur cursors.IntegerArrayCursor, every, offset int64) *integerWindowMinArrayCursor { - resLen := MaxPointsPerBlock - if every == 0 { - resLen = 1 - } - return &integerWindowMinArrayCursor{ - IntegerArrayCursor: cur, - every: every, - offset: offset, - res: cursors.NewIntegerArrayLen(resLen), - tmp: &cursors.IntegerArray{}, - } -} - -func (c *integerWindowMinArrayCursor) Stats() cursors.CursorStats { - return c.IntegerArrayCursor.Stats() -} - -func (c *integerWindowMinArrayCursor) Next() *cursors.IntegerArray { - 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.IntegerArray - if c.tmp.Len() > 0 { - a = c.tmp - } else { - a = c.IntegerArrayCursor.Next() - } - - if a.Len() == 0 { - return &cursors.IntegerArray{} - } - - rowIdx := 0 - var acc int64 = math.MaxInt64 - var tsAcc int64 - - 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 { - c.res.Timestamps[pos] = tsAcc - c.res.Values[pos] = acc - 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 - acc = math.MaxInt64 - windowEnd = WindowStop(a.Timestamps[rowIdx], c.every, c.offset) - windowHasPoints = false - - continue WINDOWS - } else { - if !windowHasPoints || a.Values[rowIdx] < acc { - acc = a.Values[rowIdx] - tsAcc = a.Timestamps[rowIdx] - } - 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.IntegerArrayCursor.Next() - if a.Len() == 0 { - // write the final point - // do not generate a point for empty windows - if windowHasPoints { - c.res.Timestamps[pos] = tsAcc - c.res.Values[pos] = acc - pos++ - } - break WINDOWS - } - rowIdx = 0 - } - - c.res.Timestamps = c.res.Timestamps[:pos] - c.res.Values = c.res.Values[:pos] - - return c.res -} - -type integerWindowMaxArrayCursor struct { - cursors.IntegerArrayCursor - every, offset int64 - res *cursors.IntegerArray - tmp *cursors.IntegerArray -} - -func newIntegerWindowMaxArrayCursor(cur cursors.IntegerArrayCursor, every, offset int64) *integerWindowMaxArrayCursor { - resLen := MaxPointsPerBlock - if every == 0 { - resLen = 1 - } - return &integerWindowMaxArrayCursor{ - IntegerArrayCursor: cur, - every: every, - offset: offset, - res: cursors.NewIntegerArrayLen(resLen), - tmp: &cursors.IntegerArray{}, - } -} - -func (c *integerWindowMaxArrayCursor) Stats() cursors.CursorStats { - return c.IntegerArrayCursor.Stats() -} - -func (c *integerWindowMaxArrayCursor) Next() *cursors.IntegerArray { - 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.IntegerArray - if c.tmp.Len() > 0 { - a = c.tmp - } else { - a = c.IntegerArrayCursor.Next() - } - - if a.Len() == 0 { - return &cursors.IntegerArray{} - } - - rowIdx := 0 - var acc int64 = math.MinInt64 - var tsAcc int64 - - 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 { - c.res.Timestamps[pos] = tsAcc - c.res.Values[pos] = acc - 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 - acc = math.MinInt64 - windowEnd = WindowStop(a.Timestamps[rowIdx], c.every, c.offset) - windowHasPoints = false - - continue WINDOWS - } else { - if !windowHasPoints || a.Values[rowIdx] > acc { - acc = a.Values[rowIdx] - tsAcc = a.Timestamps[rowIdx] - } - 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.IntegerArrayCursor.Next() - if a.Len() == 0 { - // write the final point - // do not generate a point for empty windows - if windowHasPoints { - c.res.Timestamps[pos] = tsAcc - c.res.Values[pos] = acc - pos++ - } - break WINDOWS - } - rowIdx = 0 - } - - c.res.Timestamps = c.res.Timestamps[:pos] - c.res.Values = c.res.Values[:pos] - - return c.res -} - -type integerWindowMeanArrayCursor struct { - cursors.IntegerArrayCursor - every, offset int64 - res *cursors.FloatArray - tmp *cursors.IntegerArray -} - -func newIntegerWindowMeanArrayCursor(cur cursors.IntegerArrayCursor, every, offset int64) *integerWindowMeanArrayCursor { - resLen := MaxPointsPerBlock - if every == 0 { - resLen = 1 - } - return &integerWindowMeanArrayCursor{ - IntegerArrayCursor: cur, - every: every, - offset: offset, - res: cursors.NewFloatArrayLen(resLen), - tmp: &cursors.IntegerArray{}, - } -} - -func (c *integerWindowMeanArrayCursor) Stats() cursors.CursorStats { - return c.IntegerArrayCursor.Stats() -} - -func (c *integerWindowMeanArrayCursor) Next() *cursors.FloatArray { - 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.IntegerArray - if c.tmp.Len() > 0 { - a = c.tmp - } else { - a = c.IntegerArrayCursor.Next() - } - - if a.Len() == 0 { - return &cursors.FloatArray{} - } - - rowIdx := 0 - var sum int64 - var count int64 - - 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 { - c.res.Timestamps[pos] = windowEnd - c.res.Values[pos] = float64(sum) / float64(count) - 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 - sum = 0 - count = 0 - windowEnd = WindowStop(a.Timestamps[rowIdx], c.every, c.offset) - windowHasPoints = false - - continue WINDOWS - } else { - sum += a.Values[rowIdx] - count++ - 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.IntegerArrayCursor.Next() - if a.Len() == 0 { - // write the final point - // do not generate a point for empty windows - if windowHasPoints { - c.res.Timestamps[pos] = windowEnd - c.res.Values[pos] = float64(sum) / float64(count) - pos++ - } - break WINDOWS - } - rowIdx = 0 - } - - c.res.Timestamps = c.res.Timestamps[:pos] - c.res.Values = c.res.Values[:pos] - - return c.res } type integerEmptyArrayCursor struct { @@ -1995,6 +513,8 @@ func (c *unsignedArrayFilterCursor) Next() *cursors.UnsignedArray { if c.tmp.Len() > 0 { a = c.tmp + c.tmp.Timestamps = nil + c.tmp.Values = nil } else { a = c.UnsignedArrayCursor.Next() } @@ -2014,12 +534,6 @@ 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.UnsignedArrayCursor.Next() } @@ -2029,13 +543,13 @@ LOOP: return c.res } -type unsignedArrayCursor struct { +type unsignedMultiShardArrayCursor struct { cursors.UnsignedArrayCursor cursorContext filter *unsignedArrayFilterCursor } -func (c *unsignedArrayCursor) reset(cur cursors.UnsignedArrayCursor, cursorIterator cursors.CursorIterator, cond expression) { +func (c *unsignedMultiShardArrayCursor) reset(cur cursors.UnsignedArrayCursor, itrs cursors.CursorIterators, cond expression) { if cond != nil { if c.filter == nil { c.filter = newUnsignedFilterArrayCursor(cond) @@ -2045,17 +559,18 @@ func (c *unsignedArrayCursor) reset(cur cursors.UnsignedArrayCursor, cursorItera } c.UnsignedArrayCursor = cur - c.cursorIterator = cursorIterator + c.itrs = itrs c.err = nil + c.count = 0 } -func (c *unsignedArrayCursor) Err() error { return c.err } +func (c *unsignedMultiShardArrayCursor) Err() error { return c.err } -func (c *unsignedArrayCursor) Stats() cursors.CursorStats { +func (c *unsignedMultiShardArrayCursor) Stats() cursors.CursorStats { return c.UnsignedArrayCursor.Stats() } -func (c *unsignedArrayCursor) Next() *cursors.UnsignedArray { +func (c *unsignedMultiShardArrayCursor) Next() *cursors.UnsignedArray { for { a := c.UnsignedArrayCursor.Next() if a.Len() == 0 { @@ -2063,19 +578,31 @@ func (c *unsignedArrayCursor) Next() *cursors.UnsignedArray { continue } } + c.count += int64(a.Len()) + if c.count > c.limit { + diff := c.count - c.limit + c.count -= diff + rem := int64(a.Len()) - diff + a.Timestamps = a.Timestamps[:rem] + a.Values = a.Values[:rem] + } return a } } -func (c *unsignedArrayCursor) nextArrayCursor() bool { - if c.cursorIterator == nil { +func (c *unsignedMultiShardArrayCursor) nextArrayCursor() bool { + if len(c.itrs) == 0 { return false } c.UnsignedArrayCursor.Close() - cur, _ := c.cursorIterator.Next(c.ctx, c.req) - c.cursorIterator = nil + var itr cursors.CursorIterator + var cur cursors.Cursor + for cur == nil && len(c.itrs) > 0 { + itr, c.itrs = c.itrs[0], c.itrs[1:] + cur, _ = itr.Next(c.ctx, c.req) + } var ok bool if cur != nil { @@ -2084,7 +611,7 @@ func (c *unsignedArrayCursor) nextArrayCursor() bool { if !ok { cur.Close() next = UnsignedEmptyArrayCursor - c.cursorIterator = nil + c.itrs = nil c.err = errors.New("expected unsigned cursor") } else { if c.filter != nil { @@ -2100,738 +627,72 @@ func (c *unsignedArrayCursor) nextArrayCursor() bool { return ok } -type unsignedLimitArrayCursor struct { +type unsignedArraySumCursor struct { cursors.UnsignedArrayCursor - res *cursors.UnsignedArray - done bool + ts [1]int64 + vs [1]uint64 + res *cursors.UnsignedArray } -func newUnsignedLimitArrayCursor(cur cursors.UnsignedArrayCursor) *unsignedLimitArrayCursor { - return &unsignedLimitArrayCursor{ +func newUnsignedArraySumCursor(cur cursors.UnsignedArrayCursor) *unsignedArraySumCursor { + return &unsignedArraySumCursor{ UnsignedArrayCursor: cur, - res: cursors.NewUnsignedArrayLen(1), + res: &cursors.UnsignedArray{}, } } -func (c *unsignedLimitArrayCursor) Stats() cursors.CursorStats { return c.UnsignedArrayCursor.Stats() } +func (c unsignedArraySumCursor) Stats() cursors.CursorStats { return c.UnsignedArrayCursor.Stats() } -func (c *unsignedLimitArrayCursor) Next() *cursors.UnsignedArray { - if c.done { - return &cursors.UnsignedArray{} - } +func (c unsignedArraySumCursor) Next() *cursors.UnsignedArray { a := c.UnsignedArrayCursor.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 unsignedWindowLastArrayCursor struct { - cursors.UnsignedArrayCursor - every, offset, windowEnd int64 - res *cursors.UnsignedArray - tmp *cursors.UnsignedArray -} + ts := a.Timestamps[0] + var acc uint64 -// 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 newUnsignedWindowLastArrayCursor(cur cursors.UnsignedArrayCursor, every, offset int64) *unsignedWindowLastArrayCursor { - return &unsignedWindowLastArrayCursor{ - UnsignedArrayCursor: cur, - every: every, - offset: offset, - windowEnd: math.MinInt64, - res: cursors.NewUnsignedArrayLen(MaxPointsPerBlock), - tmp: &cursors.UnsignedArray{}, - } -} - -func (c *unsignedWindowLastArrayCursor) Stats() cursors.CursorStats { - return c.UnsignedArrayCursor.Stats() -} - -func (c *unsignedWindowLastArrayCursor) Next() *cursors.UnsignedArray { - cur := -1 - -NEXT: - var a *cursors.UnsignedArray - - if c.tmp.Len() > 0 { - a = c.tmp - } else { + for { + for _, v := range a.Values { + acc += v + } a = c.UnsignedArrayCursor.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 unsignedWindowFirstArrayCursor struct { - cursors.UnsignedArrayCursor - every, offset, windowEnd int64 - res *cursors.UnsignedArray - tmp *cursors.UnsignedArray -} - -// 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 newUnsignedWindowFirstArrayCursor(cur cursors.UnsignedArrayCursor, every, offset int64) *unsignedWindowFirstArrayCursor { - return &unsignedWindowFirstArrayCursor{ - UnsignedArrayCursor: cur, - every: every, - offset: offset, - windowEnd: math.MinInt64, - res: cursors.NewUnsignedArrayLen(MaxPointsPerBlock), - tmp: &cursors.UnsignedArray{}, - } -} - -func (c *unsignedWindowFirstArrayCursor) Stats() cursors.CursorStats { - return c.UnsignedArrayCursor.Stats() -} - -func (c *unsignedWindowFirstArrayCursor) Next() *cursors.UnsignedArray { - c.res.Timestamps = c.res.Timestamps[:0] - c.res.Values = c.res.Values[:0] - -NEXT: - var a *cursors.UnsignedArray - - if c.tmp.Len() > 0 { - a = c.tmp - } else { - a = c.UnsignedArrayCursor.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:] + if len(a.Timestamps) == 0 { + c.ts[0] = ts + c.vs[0] = acc + c.res.Timestamps = c.ts[:] + c.res.Values = c.vs[:] return c.res } } - - c.tmp.Timestamps = nil - c.tmp.Values = nil - - goto NEXT } -type unsignedWindowCountArrayCursor struct { +type integerUnsignedCountArrayCursor struct { cursors.UnsignedArrayCursor - every, offset int64 - res *cursors.IntegerArray - tmp *cursors.UnsignedArray } -func newUnsignedWindowCountArrayCursor(cur cursors.UnsignedArrayCursor, every, offset int64) *unsignedWindowCountArrayCursor { - resLen := MaxPointsPerBlock - if every == 0 { - resLen = 1 - } - return &unsignedWindowCountArrayCursor{ - UnsignedArrayCursor: cur, - every: every, - offset: offset, - res: cursors.NewIntegerArrayLen(resLen), - tmp: &cursors.UnsignedArray{}, - } -} - -func (c *unsignedWindowCountArrayCursor) Stats() cursors.CursorStats { +func (c *integerUnsignedCountArrayCursor) Stats() cursors.CursorStats { return c.UnsignedArrayCursor.Stats() } -func (c *unsignedWindowCountArrayCursor) Next() *cursors.IntegerArray { - 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.UnsignedArray - if c.tmp.Len() > 0 { - a = c.tmp - } else { - a = c.UnsignedArrayCursor.Next() - } - - if a.Len() == 0 { +func (c *integerUnsignedCountArrayCursor) Next() *cursors.IntegerArray { + a := c.UnsignedArrayCursor.Next() + if len(a.Timestamps) == 0 { return &cursors.IntegerArray{} } - rowIdx := 0 - var acc int64 = 0 - - 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: + ts := a.Timestamps[0] + var acc int64 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 { - c.res.Timestamps[pos] = windowEnd - c.res.Values[pos] = acc - 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 - acc = 0 - windowEnd = WindowStop(a.Timestamps[rowIdx], c.every, c.offset) - windowHasPoints = false - - continue WINDOWS - } else { - acc++ - 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 + acc += int64(len(a.Timestamps)) a = c.UnsignedArrayCursor.Next() - if a.Len() == 0 { - // write the final point - // do not generate a point for empty windows - if windowHasPoints { - c.res.Timestamps[pos] = windowEnd - c.res.Values[pos] = acc - pos++ - } - break WINDOWS + if len(a.Timestamps) == 0 { + res := cursors.NewIntegerArrayLen(1) + res.Timestamps[0] = ts + res.Values[0] = acc + return res } - rowIdx = 0 } - - c.res.Timestamps = c.res.Timestamps[:pos] - c.res.Values = c.res.Values[:pos] - - return c.res -} - -type unsignedWindowSumArrayCursor struct { - cursors.UnsignedArrayCursor - every, offset int64 - res *cursors.UnsignedArray - tmp *cursors.UnsignedArray -} - -func newUnsignedWindowSumArrayCursor(cur cursors.UnsignedArrayCursor, every, offset int64) *unsignedWindowSumArrayCursor { - resLen := MaxPointsPerBlock - if every == 0 { - resLen = 1 - } - return &unsignedWindowSumArrayCursor{ - UnsignedArrayCursor: cur, - every: every, - offset: offset, - res: cursors.NewUnsignedArrayLen(resLen), - tmp: &cursors.UnsignedArray{}, - } -} - -func (c *unsignedWindowSumArrayCursor) Stats() cursors.CursorStats { - return c.UnsignedArrayCursor.Stats() -} - -func (c *unsignedWindowSumArrayCursor) Next() *cursors.UnsignedArray { - 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.UnsignedArray - if c.tmp.Len() > 0 { - a = c.tmp - } else { - a = c.UnsignedArrayCursor.Next() - } - - if a.Len() == 0 { - return &cursors.UnsignedArray{} - } - - rowIdx := 0 - var acc uint64 = 0 - - 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 { - c.res.Timestamps[pos] = windowEnd - c.res.Values[pos] = acc - 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 - acc = 0 - windowEnd = WindowStop(a.Timestamps[rowIdx], c.every, c.offset) - windowHasPoints = false - - continue WINDOWS - } else { - acc += a.Values[rowIdx] - 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.UnsignedArrayCursor.Next() - if a.Len() == 0 { - // write the final point - // do not generate a point for empty windows - if windowHasPoints { - c.res.Timestamps[pos] = windowEnd - c.res.Values[pos] = acc - pos++ - } - break WINDOWS - } - rowIdx = 0 - } - - c.res.Timestamps = c.res.Timestamps[:pos] - c.res.Values = c.res.Values[:pos] - - return c.res -} - -type unsignedWindowMinArrayCursor struct { - cursors.UnsignedArrayCursor - every, offset int64 - res *cursors.UnsignedArray - tmp *cursors.UnsignedArray -} - -func newUnsignedWindowMinArrayCursor(cur cursors.UnsignedArrayCursor, every, offset int64) *unsignedWindowMinArrayCursor { - resLen := MaxPointsPerBlock - if every == 0 { - resLen = 1 - } - return &unsignedWindowMinArrayCursor{ - UnsignedArrayCursor: cur, - every: every, - offset: offset, - res: cursors.NewUnsignedArrayLen(resLen), - tmp: &cursors.UnsignedArray{}, - } -} - -func (c *unsignedWindowMinArrayCursor) Stats() cursors.CursorStats { - return c.UnsignedArrayCursor.Stats() -} - -func (c *unsignedWindowMinArrayCursor) Next() *cursors.UnsignedArray { - 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.UnsignedArray - if c.tmp.Len() > 0 { - a = c.tmp - } else { - a = c.UnsignedArrayCursor.Next() - } - - if a.Len() == 0 { - return &cursors.UnsignedArray{} - } - - rowIdx := 0 - var acc uint64 = math.MaxUint64 - var tsAcc int64 - - 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 { - c.res.Timestamps[pos] = tsAcc - c.res.Values[pos] = acc - 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 - acc = math.MaxUint64 - windowEnd = WindowStop(a.Timestamps[rowIdx], c.every, c.offset) - windowHasPoints = false - - continue WINDOWS - } else { - if !windowHasPoints || a.Values[rowIdx] < acc { - acc = a.Values[rowIdx] - tsAcc = a.Timestamps[rowIdx] - } - 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.UnsignedArrayCursor.Next() - if a.Len() == 0 { - // write the final point - // do not generate a point for empty windows - if windowHasPoints { - c.res.Timestamps[pos] = tsAcc - c.res.Values[pos] = acc - pos++ - } - break WINDOWS - } - rowIdx = 0 - } - - c.res.Timestamps = c.res.Timestamps[:pos] - c.res.Values = c.res.Values[:pos] - - return c.res -} - -type unsignedWindowMaxArrayCursor struct { - cursors.UnsignedArrayCursor - every, offset int64 - res *cursors.UnsignedArray - tmp *cursors.UnsignedArray -} - -func newUnsignedWindowMaxArrayCursor(cur cursors.UnsignedArrayCursor, every, offset int64) *unsignedWindowMaxArrayCursor { - resLen := MaxPointsPerBlock - if every == 0 { - resLen = 1 - } - return &unsignedWindowMaxArrayCursor{ - UnsignedArrayCursor: cur, - every: every, - offset: offset, - res: cursors.NewUnsignedArrayLen(resLen), - tmp: &cursors.UnsignedArray{}, - } -} - -func (c *unsignedWindowMaxArrayCursor) Stats() cursors.CursorStats { - return c.UnsignedArrayCursor.Stats() -} - -func (c *unsignedWindowMaxArrayCursor) Next() *cursors.UnsignedArray { - 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.UnsignedArray - if c.tmp.Len() > 0 { - a = c.tmp - } else { - a = c.UnsignedArrayCursor.Next() - } - - if a.Len() == 0 { - return &cursors.UnsignedArray{} - } - - rowIdx := 0 - var acc uint64 = 0 - var tsAcc int64 - - 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 { - c.res.Timestamps[pos] = tsAcc - c.res.Values[pos] = acc - 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 - acc = 0 - windowEnd = WindowStop(a.Timestamps[rowIdx], c.every, c.offset) - windowHasPoints = false - - continue WINDOWS - } else { - if !windowHasPoints || a.Values[rowIdx] > acc { - acc = a.Values[rowIdx] - tsAcc = a.Timestamps[rowIdx] - } - 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.UnsignedArrayCursor.Next() - if a.Len() == 0 { - // write the final point - // do not generate a point for empty windows - if windowHasPoints { - c.res.Timestamps[pos] = tsAcc - c.res.Values[pos] = acc - pos++ - } - break WINDOWS - } - rowIdx = 0 - } - - c.res.Timestamps = c.res.Timestamps[:pos] - c.res.Values = c.res.Values[:pos] - - return c.res -} - -type unsignedWindowMeanArrayCursor struct { - cursors.UnsignedArrayCursor - every, offset int64 - res *cursors.FloatArray - tmp *cursors.UnsignedArray -} - -func newUnsignedWindowMeanArrayCursor(cur cursors.UnsignedArrayCursor, every, offset int64) *unsignedWindowMeanArrayCursor { - resLen := MaxPointsPerBlock - if every == 0 { - resLen = 1 - } - return &unsignedWindowMeanArrayCursor{ - UnsignedArrayCursor: cur, - every: every, - offset: offset, - res: cursors.NewFloatArrayLen(resLen), - tmp: &cursors.UnsignedArray{}, - } -} - -func (c *unsignedWindowMeanArrayCursor) Stats() cursors.CursorStats { - return c.UnsignedArrayCursor.Stats() -} - -func (c *unsignedWindowMeanArrayCursor) Next() *cursors.FloatArray { - 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.UnsignedArray - if c.tmp.Len() > 0 { - a = c.tmp - } else { - a = c.UnsignedArrayCursor.Next() - } - - if a.Len() == 0 { - return &cursors.FloatArray{} - } - - rowIdx := 0 - var sum uint64 - var count int64 - - 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 { - c.res.Timestamps[pos] = windowEnd - c.res.Values[pos] = float64(sum) / float64(count) - 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 - sum = 0 - count = 0 - windowEnd = WindowStop(a.Timestamps[rowIdx], c.every, c.offset) - windowHasPoints = false - - continue WINDOWS - } else { - sum += a.Values[rowIdx] - count++ - 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.UnsignedArrayCursor.Next() - if a.Len() == 0 { - // write the final point - // do not generate a point for empty windows - if windowHasPoints { - c.res.Timestamps[pos] = windowEnd - c.res.Values[pos] = float64(sum) / float64(count) - pos++ - } - break WINDOWS - } - rowIdx = 0 - } - - c.res.Timestamps = c.res.Timestamps[:pos] - c.res.Values = c.res.Values[:pos] - - return c.res } type unsignedEmptyArrayCursor struct { @@ -2881,6 +742,8 @@ func (c *stringArrayFilterCursor) Next() *cursors.StringArray { if c.tmp.Len() > 0 { a = c.tmp + c.tmp.Timestamps = nil + c.tmp.Values = nil } else { a = c.StringArrayCursor.Next() } @@ -2900,12 +763,6 @@ 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.StringArrayCursor.Next() } @@ -2915,13 +772,13 @@ LOOP: return c.res } -type stringArrayCursor struct { +type stringMultiShardArrayCursor struct { cursors.StringArrayCursor cursorContext filter *stringArrayFilterCursor } -func (c *stringArrayCursor) reset(cur cursors.StringArrayCursor, cursorIterator cursors.CursorIterator, cond expression) { +func (c *stringMultiShardArrayCursor) reset(cur cursors.StringArrayCursor, itrs cursors.CursorIterators, cond expression) { if cond != nil { if c.filter == nil { c.filter = newStringFilterArrayCursor(cond) @@ -2931,17 +788,18 @@ func (c *stringArrayCursor) reset(cur cursors.StringArrayCursor, cursorIterator } c.StringArrayCursor = cur - c.cursorIterator = cursorIterator + c.itrs = itrs c.err = nil + c.count = 0 } -func (c *stringArrayCursor) Err() error { return c.err } +func (c *stringMultiShardArrayCursor) Err() error { return c.err } -func (c *stringArrayCursor) Stats() cursors.CursorStats { +func (c *stringMultiShardArrayCursor) Stats() cursors.CursorStats { return c.StringArrayCursor.Stats() } -func (c *stringArrayCursor) Next() *cursors.StringArray { +func (c *stringMultiShardArrayCursor) Next() *cursors.StringArray { for { a := c.StringArrayCursor.Next() if a.Len() == 0 { @@ -2949,19 +807,31 @@ func (c *stringArrayCursor) Next() *cursors.StringArray { continue } } + c.count += int64(a.Len()) + if c.count > c.limit { + diff := c.count - c.limit + c.count -= diff + rem := int64(a.Len()) - diff + a.Timestamps = a.Timestamps[:rem] + a.Values = a.Values[:rem] + } return a } } -func (c *stringArrayCursor) nextArrayCursor() bool { - if c.cursorIterator == nil { +func (c *stringMultiShardArrayCursor) nextArrayCursor() bool { + if len(c.itrs) == 0 { return false } c.StringArrayCursor.Close() - cur, _ := c.cursorIterator.Next(c.ctx, c.req) - c.cursorIterator = nil + var itr cursors.CursorIterator + var cur cursors.Cursor + for cur == nil && len(c.itrs) > 0 { + itr, c.itrs = c.itrs[0], c.itrs[1:] + cur, _ = itr.Next(c.ctx, c.req) + } var ok bool if cur != nil { @@ -2970,7 +840,7 @@ func (c *stringArrayCursor) nextArrayCursor() bool { if !ok { cur.Close() next = StringEmptyArrayCursor - c.cursorIterator = nil + c.itrs = nil c.err = errors.New("expected string cursor") } else { if c.filter != nil { @@ -2986,275 +856,32 @@ func (c *stringArrayCursor) nextArrayCursor() bool { return ok } -type stringLimitArrayCursor struct { +type integerStringCountArrayCursor struct { cursors.StringArrayCursor - res *cursors.StringArray - done bool } -func newStringLimitArrayCursor(cur cursors.StringArrayCursor) *stringLimitArrayCursor { - return &stringLimitArrayCursor{ - StringArrayCursor: cur, - res: cursors.NewStringArrayLen(1), - } +func (c *integerStringCountArrayCursor) Stats() cursors.CursorStats { + return c.StringArrayCursor.Stats() } -func (c *stringLimitArrayCursor) Stats() cursors.CursorStats { return c.StringArrayCursor.Stats() } - -func (c *stringLimitArrayCursor) Next() *cursors.StringArray { - if c.done { - return &cursors.StringArray{} - } +func (c *integerStringCountArrayCursor) Next() *cursors.IntegerArray { a := c.StringArrayCursor.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 stringWindowLastArrayCursor struct { - cursors.StringArrayCursor - every, offset, windowEnd int64 - res *cursors.StringArray - tmp *cursors.StringArray -} - -// 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 newStringWindowLastArrayCursor(cur cursors.StringArrayCursor, every, offset int64) *stringWindowLastArrayCursor { - return &stringWindowLastArrayCursor{ - StringArrayCursor: cur, - every: every, - offset: offset, - windowEnd: math.MinInt64, - res: cursors.NewStringArrayLen(MaxPointsPerBlock), - tmp: &cursors.StringArray{}, - } -} - -func (c *stringWindowLastArrayCursor) Stats() cursors.CursorStats { - return c.StringArrayCursor.Stats() -} - -func (c *stringWindowLastArrayCursor) Next() *cursors.StringArray { - cur := -1 - -NEXT: - var a *cursors.StringArray - - if c.tmp.Len() > 0 { - a = c.tmp - } else { - a = c.StringArrayCursor.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 stringWindowFirstArrayCursor struct { - cursors.StringArrayCursor - every, offset, windowEnd int64 - res *cursors.StringArray - tmp *cursors.StringArray -} - -// 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 newStringWindowFirstArrayCursor(cur cursors.StringArrayCursor, every, offset int64) *stringWindowFirstArrayCursor { - return &stringWindowFirstArrayCursor{ - StringArrayCursor: cur, - every: every, - offset: offset, - windowEnd: math.MinInt64, - res: cursors.NewStringArrayLen(MaxPointsPerBlock), - tmp: &cursors.StringArray{}, - } -} - -func (c *stringWindowFirstArrayCursor) Stats() cursors.CursorStats { - return c.StringArrayCursor.Stats() -} - -func (c *stringWindowFirstArrayCursor) Next() *cursors.StringArray { - c.res.Timestamps = c.res.Timestamps[:0] - c.res.Values = c.res.Values[:0] - -NEXT: - var a *cursors.StringArray - - if c.tmp.Len() > 0 { - a = c.tmp - } else { - a = c.StringArrayCursor.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 -} - -type stringWindowCountArrayCursor struct { - cursors.StringArrayCursor - every, offset int64 - res *cursors.IntegerArray - tmp *cursors.StringArray -} - -func newStringWindowCountArrayCursor(cur cursors.StringArrayCursor, every, offset int64) *stringWindowCountArrayCursor { - resLen := MaxPointsPerBlock - if every == 0 { - resLen = 1 - } - return &stringWindowCountArrayCursor{ - StringArrayCursor: cur, - every: every, - offset: offset, - res: cursors.NewIntegerArrayLen(resLen), - tmp: &cursors.StringArray{}, - } -} - -func (c *stringWindowCountArrayCursor) Stats() cursors.CursorStats { - return c.StringArrayCursor.Stats() -} - -func (c *stringWindowCountArrayCursor) Next() *cursors.IntegerArray { - 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.StringArray - if c.tmp.Len() > 0 { - a = c.tmp - } else { - a = c.StringArrayCursor.Next() - } - - if a.Len() == 0 { return &cursors.IntegerArray{} } - rowIdx := 0 - var acc int64 = 0 - - 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: + ts := a.Timestamps[0] + var acc int64 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 { - c.res.Timestamps[pos] = windowEnd - c.res.Values[pos] = acc - 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 - acc = 0 - windowEnd = WindowStop(a.Timestamps[rowIdx], c.every, c.offset) - windowHasPoints = false - - continue WINDOWS - } else { - acc++ - 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 + acc += int64(len(a.Timestamps)) a = c.StringArrayCursor.Next() - if a.Len() == 0 { - // write the final point - // do not generate a point for empty windows - if windowHasPoints { - c.res.Timestamps[pos] = windowEnd - c.res.Values[pos] = acc - pos++ - } - break WINDOWS + if len(a.Timestamps) == 0 { + res := cursors.NewIntegerArrayLen(1) + res.Timestamps[0] = ts + res.Values[0] = acc + return res } - rowIdx = 0 } - - c.res.Timestamps = c.res.Timestamps[:pos] - c.res.Values = c.res.Values[:pos] - - return c.res } type stringEmptyArrayCursor struct { @@ -3304,6 +931,8 @@ func (c *booleanArrayFilterCursor) Next() *cursors.BooleanArray { if c.tmp.Len() > 0 { a = c.tmp + c.tmp.Timestamps = nil + c.tmp.Values = nil } else { a = c.BooleanArrayCursor.Next() } @@ -3323,12 +952,6 @@ 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.BooleanArrayCursor.Next() } @@ -3338,13 +961,13 @@ LOOP: return c.res } -type booleanArrayCursor struct { +type booleanMultiShardArrayCursor struct { cursors.BooleanArrayCursor cursorContext filter *booleanArrayFilterCursor } -func (c *booleanArrayCursor) reset(cur cursors.BooleanArrayCursor, cursorIterator cursors.CursorIterator, cond expression) { +func (c *booleanMultiShardArrayCursor) reset(cur cursors.BooleanArrayCursor, itrs cursors.CursorIterators, cond expression) { if cond != nil { if c.filter == nil { c.filter = newBooleanFilterArrayCursor(cond) @@ -3354,17 +977,18 @@ func (c *booleanArrayCursor) reset(cur cursors.BooleanArrayCursor, cursorIterato } c.BooleanArrayCursor = cur - c.cursorIterator = cursorIterator + c.itrs = itrs c.err = nil + c.count = 0 } -func (c *booleanArrayCursor) Err() error { return c.err } +func (c *booleanMultiShardArrayCursor) Err() error { return c.err } -func (c *booleanArrayCursor) Stats() cursors.CursorStats { +func (c *booleanMultiShardArrayCursor) Stats() cursors.CursorStats { return c.BooleanArrayCursor.Stats() } -func (c *booleanArrayCursor) Next() *cursors.BooleanArray { +func (c *booleanMultiShardArrayCursor) Next() *cursors.BooleanArray { for { a := c.BooleanArrayCursor.Next() if a.Len() == 0 { @@ -3372,19 +996,31 @@ func (c *booleanArrayCursor) Next() *cursors.BooleanArray { continue } } + c.count += int64(a.Len()) + if c.count > c.limit { + diff := c.count - c.limit + c.count -= diff + rem := int64(a.Len()) - diff + a.Timestamps = a.Timestamps[:rem] + a.Values = a.Values[:rem] + } return a } } -func (c *booleanArrayCursor) nextArrayCursor() bool { - if c.cursorIterator == nil { +func (c *booleanMultiShardArrayCursor) nextArrayCursor() bool { + if len(c.itrs) == 0 { return false } c.BooleanArrayCursor.Close() - cur, _ := c.cursorIterator.Next(c.ctx, c.req) - c.cursorIterator = nil + var itr cursors.CursorIterator + var cur cursors.Cursor + for cur == nil && len(c.itrs) > 0 { + itr, c.itrs = c.itrs[0], c.itrs[1:] + cur, _ = itr.Next(c.ctx, c.req) + } var ok bool if cur != nil { @@ -3393,7 +1029,7 @@ func (c *booleanArrayCursor) nextArrayCursor() bool { if !ok { cur.Close() next = BooleanEmptyArrayCursor - c.cursorIterator = nil + c.itrs = nil c.err = errors.New("expected boolean cursor") } else { if c.filter != nil { @@ -3409,275 +1045,32 @@ func (c *booleanArrayCursor) nextArrayCursor() bool { return ok } -type booleanLimitArrayCursor struct { +type integerBooleanCountArrayCursor struct { cursors.BooleanArrayCursor - res *cursors.BooleanArray - done bool } -func newBooleanLimitArrayCursor(cur cursors.BooleanArrayCursor) *booleanLimitArrayCursor { - return &booleanLimitArrayCursor{ - BooleanArrayCursor: cur, - res: cursors.NewBooleanArrayLen(1), - } +func (c *integerBooleanCountArrayCursor) Stats() cursors.CursorStats { + return c.BooleanArrayCursor.Stats() } -func (c *booleanLimitArrayCursor) Stats() cursors.CursorStats { return c.BooleanArrayCursor.Stats() } - -func (c *booleanLimitArrayCursor) Next() *cursors.BooleanArray { - if c.done { - return &cursors.BooleanArray{} - } +func (c *integerBooleanCountArrayCursor) Next() *cursors.IntegerArray { a := c.BooleanArrayCursor.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 booleanWindowLastArrayCursor struct { - cursors.BooleanArrayCursor - every, offset, windowEnd int64 - res *cursors.BooleanArray - tmp *cursors.BooleanArray -} - -// 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 newBooleanWindowLastArrayCursor(cur cursors.BooleanArrayCursor, every, offset int64) *booleanWindowLastArrayCursor { - return &booleanWindowLastArrayCursor{ - BooleanArrayCursor: cur, - every: every, - offset: offset, - windowEnd: math.MinInt64, - res: cursors.NewBooleanArrayLen(MaxPointsPerBlock), - tmp: &cursors.BooleanArray{}, - } -} - -func (c *booleanWindowLastArrayCursor) Stats() cursors.CursorStats { - return c.BooleanArrayCursor.Stats() -} - -func (c *booleanWindowLastArrayCursor) Next() *cursors.BooleanArray { - cur := -1 - -NEXT: - var a *cursors.BooleanArray - - if c.tmp.Len() > 0 { - a = c.tmp - } else { - a = c.BooleanArrayCursor.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 booleanWindowFirstArrayCursor struct { - cursors.BooleanArrayCursor - every, offset, windowEnd int64 - res *cursors.BooleanArray - tmp *cursors.BooleanArray -} - -// 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 newBooleanWindowFirstArrayCursor(cur cursors.BooleanArrayCursor, every, offset int64) *booleanWindowFirstArrayCursor { - return &booleanWindowFirstArrayCursor{ - BooleanArrayCursor: cur, - every: every, - offset: offset, - windowEnd: math.MinInt64, - res: cursors.NewBooleanArrayLen(MaxPointsPerBlock), - tmp: &cursors.BooleanArray{}, - } -} - -func (c *booleanWindowFirstArrayCursor) Stats() cursors.CursorStats { - return c.BooleanArrayCursor.Stats() -} - -func (c *booleanWindowFirstArrayCursor) Next() *cursors.BooleanArray { - c.res.Timestamps = c.res.Timestamps[:0] - c.res.Values = c.res.Values[:0] - -NEXT: - var a *cursors.BooleanArray - - if c.tmp.Len() > 0 { - a = c.tmp - } else { - a = c.BooleanArrayCursor.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 -} - -type booleanWindowCountArrayCursor struct { - cursors.BooleanArrayCursor - every, offset int64 - res *cursors.IntegerArray - tmp *cursors.BooleanArray -} - -func newBooleanWindowCountArrayCursor(cur cursors.BooleanArrayCursor, every, offset int64) *booleanWindowCountArrayCursor { - resLen := MaxPointsPerBlock - if every == 0 { - resLen = 1 - } - return &booleanWindowCountArrayCursor{ - BooleanArrayCursor: cur, - every: every, - offset: offset, - res: cursors.NewIntegerArrayLen(resLen), - tmp: &cursors.BooleanArray{}, - } -} - -func (c *booleanWindowCountArrayCursor) Stats() cursors.CursorStats { - return c.BooleanArrayCursor.Stats() -} - -func (c *booleanWindowCountArrayCursor) Next() *cursors.IntegerArray { - 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.BooleanArray - if c.tmp.Len() > 0 { - a = c.tmp - } else { - a = c.BooleanArrayCursor.Next() - } - - if a.Len() == 0 { return &cursors.IntegerArray{} } - rowIdx := 0 - var acc int64 = 0 - - 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: + ts := a.Timestamps[0] + var acc int64 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 { - c.res.Timestamps[pos] = windowEnd - c.res.Values[pos] = acc - 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 - acc = 0 - windowEnd = WindowStop(a.Timestamps[rowIdx], c.every, c.offset) - windowHasPoints = false - - continue WINDOWS - } else { - acc++ - 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 + acc += int64(len(a.Timestamps)) a = c.BooleanArrayCursor.Next() - if a.Len() == 0 { - // write the final point - // do not generate a point for empty windows - if windowHasPoints { - c.res.Timestamps[pos] = windowEnd - c.res.Values[pos] = acc - pos++ - } - break WINDOWS + if len(a.Timestamps) == 0 { + res := cursors.NewIntegerArrayLen(1) + res.Timestamps[0] = ts + res.Values[0] = acc + return res } - rowIdx = 0 } - - c.res.Timestamps = c.res.Timestamps[:pos] - c.res.Values = c.res.Values[:pos] - - return c.res } type booleanEmptyArrayCursor struct { diff --git a/storage/reads/array_cursor.gen.go.tmpl b/storage/reads/array_cursor.gen.go.tmpl deleted file mode 100644 index fabe354cf5..0000000000 --- a/storage/reads/array_cursor.gen.go.tmpl +++ /dev/null @@ -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 . */}} diff --git a/storage/reads/array_cursor.gen.go.tmpldata b/storage/reads/array_cursor.gen.go.tmpldata deleted file mode 100644 index 86a1ae8619..0000000000 --- a/storage/reads/array_cursor.gen.go.tmpldata +++ /dev/null @@ -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" - } - ] - } -] diff --git a/storage/reads/array_cursor.go b/storage/reads/array_cursor.go index b701d8108c..7d45c57987 100644 --- a/storage/reads/array_cursor.go +++ b/storage/reads/array_cursor.go @@ -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) +} diff --git a/storage/reads/array_cursor_gen_test.go b/storage/reads/array_cursor_gen_test.go deleted file mode 100644 index 5345bdd6c6..0000000000 --- a/storage/reads/array_cursor_gen_test.go +++ /dev/null @@ -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) - } - }) - -} diff --git a/storage/reads/array_cursor_test.gen.go.tmpl b/storage/reads/array_cursor_test.gen.go.tmpl deleted file mode 100644 index c9cf130f5b..0000000000 --- a/storage/reads/array_cursor_test.gen.go.tmpl +++ /dev/null @@ -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 */}} diff --git a/storage/reads/array_cursor_test.go b/storage/reads/array_cursor_test.go deleted file mode 100644 index 9b96e69109..0000000000 --- a/storage/reads/array_cursor_test.go +++ /dev/null @@ -1,2024 +0,0 @@ -package reads - -import ( - "math" - "testing" - "time" - - "github.com/google/go-cmp/cmp" - "github.com/influxdata/influxdb/v2/v1/tsdb/cursors" -) - -func TestIntegerFilterArrayCursor(t *testing.T) { - var i int - expr := MockExpression{ - EvalBoolFunc: func(v Valuer) bool { - i++ - return i%2 == 0 - }, - } - - var resultN int - ac := MockIntegerArrayCursor{ - CloseFunc: func() {}, - ErrFunc: func() error { return nil }, - StatsFunc: func() cursors.CursorStats { return cursors.CursorStats{} }, - NextFunc: func() *cursors.IntegerArray { - resultN++ - if resultN == 4 { - return cursors.NewIntegerArrayLen(0) - } - return cursors.NewIntegerArrayLen(900) - }, - } - - c := newIntegerFilterArrayCursor(&expr) - c.reset(&ac) - - if got, want := len(c.Next().Timestamps), 1000; got != want { - t.Fatalf("len(Next())=%d, want %d", got, want) - } else if got, want := len(c.Next().Timestamps), 350; got != want { - t.Fatalf("len(Next())=%d, want %d", got, want) - } -} - -func makeIntegerArray(n int, tsStart time.Time, tsStep time.Duration, valueFn func(i int64) int64) *cursors.IntegerArray { - ia := &cursors.IntegerArray{ - Timestamps: make([]int64, n), - Values: make([]int64, n), - } - - for i := 0; i < n; i++ { - ia.Timestamps[i] = tsStart.UnixNano() + int64(i)*int64(tsStep) - ia.Values[i] = valueFn(int64(i)) - } - - return ia -} - -func makeFloatArray(n int, tsStart time.Time, tsStep time.Duration, valueFn func(i int64) float64) *cursors.FloatArray { - fa := &cursors.FloatArray{ - Timestamps: make([]int64, n), - Values: make([]float64, n), - } - - for i := 0; i < n; i++ { - fa.Timestamps[i] = tsStart.UnixNano() + int64(i)*int64(tsStep) - fa.Values[i] = valueFn(int64(i)) - } - - return fa -} - -func mustParseTime(ts string) time.Time { - t, err := time.Parse(time.RFC3339, ts) - if err != nil { - panic(err) - } - return t -} - -func copyIntegerArray(src *cursors.IntegerArray) *cursors.IntegerArray { - dst := cursors.NewIntegerArrayLen(src.Len()) - copy(dst.Timestamps, src.Timestamps) - copy(dst.Values, src.Values) - return dst -} - -func copyFloatArray(src *cursors.FloatArray) *cursors.FloatArray { - dst := cursors.NewFloatArrayLen(src.Len()) - copy(dst.Timestamps, src.Timestamps) - copy(dst.Values, src.Values) - return dst -} - -type aggArrayCursorTest struct { - name string - createCursorFn func(cur cursors.IntegerArrayCursor, every, offset int64) cursors.Cursor - every time.Duration - offset time.Duration - inputArrays []*cursors.IntegerArray - wantIntegers []*cursors.IntegerArray - wantFloats []*cursors.FloatArray -} - -func (a *aggArrayCursorTest) run(t *testing.T) { - t.Helper() - t.Run(a.name, func(t *testing.T) { - var resultN int - mc := &MockIntegerArrayCursor{ - CloseFunc: func() {}, - ErrFunc: func() error { return nil }, - StatsFunc: func() cursors.CursorStats { return cursors.CursorStats{} }, - NextFunc: func() *cursors.IntegerArray { - if resultN < len(a.inputArrays) { - a := a.inputArrays[resultN] - resultN++ - return a - } - return &cursors.IntegerArray{} - }, - } - c := a.createCursorFn(mc, int64(a.every), int64(a.offset)) - switch cursor := c.(type) { - case cursors.IntegerArrayCursor: - got := make([]*cursors.IntegerArray, 0, len(a.wantIntegers)) - for a := cursor.Next(); a.Len() != 0; a = cursor.Next() { - got = append(got, copyIntegerArray(a)) - } - - if diff := cmp.Diff(got, a.wantIntegers); diff != "" { - t.Fatalf("did not get expected result from count array cursor; -got/+want:\n%v", diff) - } - case cursors.FloatArrayCursor: - got := make([]*cursors.FloatArray, 0, len(a.wantFloats)) - for a := cursor.Next(); a.Len() != 0; a = cursor.Next() { - got = append(got, copyFloatArray(a)) - } - - if diff := cmp.Diff(got, a.wantFloats); diff != "" { - t.Fatalf("did not get expected result from count array cursor; -got/+want:\n%v", diff) - } - default: - t.Fatalf("unsupported cursor type: %T", cursor) - } - }) -} - -func TestLimitArrayCursor(t *testing.T) { - arr := []*cursors.IntegerArray{ - makeIntegerArray( - 1000, - mustParseTime("1970-01-01T00:00:01Z"), time.Millisecond, - func(i int64) int64 { return 3 + i }, - ), - makeIntegerArray( - 1000, - mustParseTime("1970-01-01T00:00:02Z"), time.Millisecond, - func(i int64) int64 { return 1003 + i }, - ), - } - idx := -1 - cur := &MockIntegerArrayCursor{ - CloseFunc: func() {}, - ErrFunc: func() error { return nil }, - StatsFunc: func() cursors.CursorStats { return cursors.CursorStats{} }, - NextFunc: func() *cursors.IntegerArray { - if idx++; idx < len(arr) { - return arr[idx] - } - return &cursors.IntegerArray{} - }, - } - aggCursor := newIntegerLimitArrayCursor(cur) - want := []*cursors.IntegerArray{ - { - Timestamps: []int64{mustParseTime("1970-01-01T00:00:01Z").UnixNano()}, - Values: []int64{3}, - }, - } - got := []*cursors.IntegerArray{} - for a := aggCursor.Next(); a.Len() != 0; a = aggCursor.Next() { - got = append(got, a) - } - if !cmp.Equal(want, got) { - t.Fatalf("unexpected result; -want/+got:\n%v", cmp.Diff(want, got)) - } -} - -func TestWindowFirstArrayCursor(t *testing.T) { - testcases := []aggArrayCursorTest{ - { - name: "window", - every: 15 * time.Minute, - inputArrays: []*cursors.IntegerArray{ - makeIntegerArray( - 60, - mustParseTime("2010-01-01T00:00:00Z"), time.Minute, - func(i int64) int64 { return i }, - ), - }, - wantIntegers: []*cursors.IntegerArray{ - makeIntegerArray( - 4, - mustParseTime("2010-01-01T00:00:00Z"), 15*time.Minute, - func(i int64) int64 { return 15 * i }, - ), - }, - }, - { - name: "offset window", - every: 15 * time.Minute, - offset: time.Minute, - inputArrays: []*cursors.IntegerArray{ - makeIntegerArray( - 60, - mustParseTime("2010-01-01T00:00:00Z"), time.Minute, - func(i int64) int64 { return i }, - ), - }, - wantIntegers: []*cursors.IntegerArray{ - { - Timestamps: []int64{ - mustParseTime("2010-01-01T00:00:00Z").UnixNano(), - mustParseTime("2010-01-01T00:01:00Z").UnixNano(), - mustParseTime("2010-01-01T00:16:00Z").UnixNano(), - mustParseTime("2010-01-01T00:31:00Z").UnixNano(), - mustParseTime("2010-01-01T00:46:00Z").UnixNano(), - }, - Values: []int64{0, 1, 16, 31, 46}, - }, - }, - }, - { - name: "empty windows", - every: time.Minute, - inputArrays: []*cursors.IntegerArray{ - makeIntegerArray( - 4, - mustParseTime("2010-01-01T00:00:00Z"), 15*time.Minute, - func(i int64) int64 { return i }, - ), - }, - wantIntegers: []*cursors.IntegerArray{ - makeIntegerArray( - 4, - mustParseTime("2010-01-01T00:00:00Z"), 15*time.Minute, - func(i int64) int64 { return i }, - ), - }, - }, - { - name: "empty offset windows", - every: time.Minute, - offset: time.Minute, - inputArrays: []*cursors.IntegerArray{ - makeIntegerArray( - 4, - mustParseTime("2010-01-01T00:00:00Z"), 15*time.Minute, - func(i int64) int64 { return i }, - ), - }, - wantIntegers: []*cursors.IntegerArray{ - makeIntegerArray( - 4, - mustParseTime("2010-01-01T00:00:00Z"), 15*time.Minute, - func(i int64) int64 { return i }, - ), - }, - }, - { - name: "unaligned window", - every: 15 * time.Minute, - inputArrays: []*cursors.IntegerArray{ - makeIntegerArray( - 60, - mustParseTime("2010-01-01T00:00:30Z"), time.Minute, - func(i int64) int64 { return i }, - ), - }, - wantIntegers: []*cursors.IntegerArray{ - makeIntegerArray( - 4, - mustParseTime("2010-01-01T00:00:30Z"), 15*time.Minute, - func(i int64) int64 { return 15 * i }, - ), - }, - }, - { - name: "unaligned offset window", - every: 15 * time.Minute, - offset: 45 * time.Second, - inputArrays: []*cursors.IntegerArray{ - makeIntegerArray( - 60, - mustParseTime("2010-01-01T00:00:30Z"), time.Minute, - func(i int64) int64 { return i }, - ), - }, - wantIntegers: []*cursors.IntegerArray{ - { - Timestamps: []int64{ - mustParseTime("2010-01-01T00:00:30Z").UnixNano(), - mustParseTime("2010-01-01T00:01:30Z").UnixNano(), - mustParseTime("2010-01-01T00:16:30Z").UnixNano(), - mustParseTime("2010-01-01T00:31:30Z").UnixNano(), - mustParseTime("2010-01-01T00:46:30Z").UnixNano(), - }, - Values: []int64{0, 1, 16, 31, 46}, - }, - }, - }, - { - name: "more unaligned window", - every: 15 * time.Minute, - inputArrays: []*cursors.IntegerArray{ - makeIntegerArray( - 60, - mustParseTime("2010-01-01T00:01:30Z"), time.Minute, - func(i int64) int64 { return i }, - ), - }, - wantIntegers: []*cursors.IntegerArray{ - { - Timestamps: []int64{ - mustParseTime("2010-01-01T00:01:30Z").UnixNano(), - mustParseTime("2010-01-01T00:15:30Z").UnixNano(), - mustParseTime("2010-01-01T00:30:30Z").UnixNano(), - mustParseTime("2010-01-01T00:45:30Z").UnixNano(), - mustParseTime("2010-01-01T01:00:30Z").UnixNano(), - }, - Values: []int64{0, 14, 29, 44, 59}, - }, - }, - }, - { - name: "window two input arrays", - every: 15 * time.Minute, - inputArrays: []*cursors.IntegerArray{ - makeIntegerArray( - 60, - mustParseTime("2010-01-01T00:00:00Z"), time.Minute, - func(i int64) int64 { return i }, - ), - makeIntegerArray( - 60, - mustParseTime("2010-01-01T01:00:00Z"), time.Minute, - func(i int64) int64 { return 60 + i }, - ), - }, - wantIntegers: []*cursors.IntegerArray{ - makeIntegerArray( - 8, - mustParseTime("2010-01-01T00:00:00Z"), 15*time.Minute, - func(i int64) int64 { return 15 * i }, - ), - }, - }, - { - name: "offset window two input arrays", - every: 30 * time.Minute, - offset: 27 * time.Minute, - inputArrays: []*cursors.IntegerArray{ - makeIntegerArray( - 60, - mustParseTime("2010-01-01T00:00:00Z"), time.Minute, - func(i int64) int64 { return i }, - ), - makeIntegerArray( - 60, - mustParseTime("2010-01-01T01:00:00Z"), time.Minute, - func(i int64) int64 { return 60 + i }, - ), - }, - wantIntegers: []*cursors.IntegerArray{ - { - Timestamps: []int64{ - mustParseTime("2010-01-01T00:00:00Z").UnixNano(), - mustParseTime("2010-01-01T00:27:00Z").UnixNano(), - mustParseTime("2010-01-01T00:57:00Z").UnixNano(), - mustParseTime("2010-01-01T01:27:00Z").UnixNano(), - mustParseTime("2010-01-01T01:57:00Z").UnixNano(), - }, - Values: []int64{0, 27, 57, 87, 117}, - }, - }, - }, - { - name: "window spans input arrays", - every: 40 * time.Minute, - inputArrays: []*cursors.IntegerArray{ - makeIntegerArray( - 60, - mustParseTime("2010-01-01T00:00:00Z"), time.Minute, - func(i int64) int64 { return i }, - ), - makeIntegerArray( - 60, - mustParseTime("2010-01-01T01:00:00Z"), time.Minute, - func(i int64) int64 { return 60 + i }, - ), - }, - wantIntegers: []*cursors.IntegerArray{ - makeIntegerArray( - 3, - mustParseTime("2010-01-01T00:00:00Z"), 40*time.Minute, - func(i int64) int64 { return 40 * i }, - ), - }, - }, - { - name: "offset window spans input arrays", - every: 40 * time.Minute, - offset: 10 * time.Minute, - inputArrays: []*cursors.IntegerArray{ - makeIntegerArray( - 60, - mustParseTime("2010-01-01T00:00:00Z"), time.Minute, - func(i int64) int64 { return i }, - ), - makeIntegerArray( - 60, - mustParseTime("2010-01-01T01:00:00Z"), time.Minute, - func(i int64) int64 { return 60 + i }, - ), - }, - wantIntegers: []*cursors.IntegerArray{ - { - Timestamps: []int64{ - mustParseTime("2010-01-01T00:00:00Z").UnixNano(), - mustParseTime("2010-01-01T00:10:00Z").UnixNano(), - mustParseTime("2010-01-01T00:50:00Z").UnixNano(), - mustParseTime("2010-01-01T01:30:00Z").UnixNano(), - }, - Values: []int64{0, 10, 50, 90}, - }, - }, - }, - { - name: "more windows than MaxPointsPerBlock", - every: 2 * time.Millisecond, - inputArrays: []*cursors.IntegerArray{ - makeIntegerArray( // 1 second, one point per ms - 1000, - mustParseTime("2010-01-01T00:00:00Z"), time.Millisecond, - func(i int64) int64 { return i }, - ), - makeIntegerArray( // 1 second, one point per ms - 1000, - mustParseTime("2010-01-01T00:00:01Z"), time.Millisecond, - func(i int64) int64 { return 1000 + i }, - ), - makeIntegerArray( // 1 second, one point per ms - 1000, - mustParseTime("2010-01-01T00:00:02Z"), time.Millisecond, - func(i int64) int64 { return 2000 + i }, - ), - }, - wantIntegers: []*cursors.IntegerArray{ - makeIntegerArray( - 1000, - mustParseTime("2010-01-01T00:00:00.000Z"), 2*time.Millisecond, - func(i int64) int64 { return 2 * i }, - ), - makeIntegerArray( - 500, - mustParseTime("2010-01-01T00:00:02.000Z"), 2*time.Millisecond, - func(i int64) int64 { return 2000 + 2*i }, - ), - }, - }, - { - name: "more offset windows than MaxPointsPerBlock", - every: 2 * time.Millisecond, - offset: 1 * time.Millisecond, - inputArrays: []*cursors.IntegerArray{ - makeIntegerArray( // 1 second, one point per ms - 1000, - mustParseTime("2010-01-01T00:00:00Z"), time.Millisecond, - func(i int64) int64 { return i }, - ), - makeIntegerArray( // 1 second, one point per ms - 1000, - mustParseTime("2010-01-01T00:00:01Z"), time.Millisecond, - func(i int64) int64 { return 1000 + i }, - ), - makeIntegerArray( // 1 second, one point per ms - 1000, - mustParseTime("2010-01-01T00:00:02Z"), time.Millisecond, - func(i int64) int64 { return 2000 + i }, - ), - }, - wantIntegers: []*cursors.IntegerArray{ - func() *cursors.IntegerArray { - arr := makeIntegerArray( - 999, - mustParseTime("2010-01-01T00:00:00.001Z"), 2*time.Millisecond, - func(i int64) int64 { return 1 + 2*i }, - ) - return &cursors.IntegerArray{ - Timestamps: append([]int64{mustParseTime("2010-01-01T00:00:00.000Z").UnixNano()}, arr.Timestamps...), - Values: append([]int64{0}, arr.Values...), - } - }(), - makeIntegerArray( - 501, - mustParseTime("2010-01-01T00:00:01.999Z"), 2*time.Millisecond, - func(i int64) int64 { return 1999 + 2*i }, - ), - }, - }, - { - name: "whole series", - inputArrays: []*cursors.IntegerArray{ - makeIntegerArray( - 60, - mustParseTime("2010-01-01T00:00:00Z"), time.Minute, - func(i int64) int64 { return 100 + i }, - ), - }, - wantIntegers: []*cursors.IntegerArray{ - makeIntegerArray( - 1, - mustParseTime("2010-01-01T00:00:00Z"), time.Minute, - func(int64) int64 { return 100 }, - ), - }, - }, - { - name: "whole series no points", - inputArrays: []*cursors.IntegerArray{{}}, - wantIntegers: []*cursors.IntegerArray{}, - }, - { - name: "whole series two arrays", - inputArrays: []*cursors.IntegerArray{ - makeIntegerArray( - 60, - mustParseTime("2010-01-01T00:00:00Z"), time.Minute, - func(i int64) int64 { return 10 + i }, - ), - makeIntegerArray( - 60, - mustParseTime("2010-01-01T01:00:00Z"), time.Minute, - func(i int64) int64 { return 70 + i }, - ), - }, - wantIntegers: []*cursors.IntegerArray{ - makeIntegerArray( - 1, - mustParseTime("2010-01-01T00:00:00Z"), time.Minute, - func(int64) int64 { return 10 }, - ), - }, - }, - { - name: "whole series span epoch", - inputArrays: []*cursors.IntegerArray{ - makeIntegerArray( - 120, - mustParseTime("1969-12-31T23:00:00Z"), time.Minute, - func(i int64) int64 { return 100 + i }, - ), - }, - wantIntegers: []*cursors.IntegerArray{ - makeIntegerArray( - 1, - mustParseTime("1969-12-31T23:00:00Z"), time.Minute, - func(int64) int64 { return 100 }, - ), - }, - }, - { - name: "whole series span epoch two arrays", - inputArrays: []*cursors.IntegerArray{ - makeIntegerArray( - 60, - mustParseTime("1969-12-31T23:00:00Z"), time.Minute, - func(i int64) int64 { return 100 + i }, - ), - makeIntegerArray( - 60, - mustParseTime("1970-01-01T00:00:00Z"), time.Minute, - func(i int64) int64 { return 160 + i }, - ), - }, - wantIntegers: []*cursors.IntegerArray{ - makeIntegerArray( - 1, - mustParseTime("1969-12-31T23:00:00Z"), time.Minute, - func(int64) int64 { return 100 }, - ), - }, - }, - { - name: "whole series, with max int64 timestamp", - inputArrays: []*cursors.IntegerArray{ - { - Timestamps: []int64{math.MaxInt64}, - Values: []int64{12}, - }, - }, - wantIntegers: []*cursors.IntegerArray{ - { - Timestamps: []int64{math.MaxInt64}, - Values: []int64{12}, - }, - }, - }, - } - for _, tc := range testcases { - tc.createCursorFn = func(cur cursors.IntegerArrayCursor, every, offset int64) cursors.Cursor { - if every == 0 { - return newIntegerLimitArrayCursor(cur) - } - return newIntegerWindowFirstArrayCursor(cur, every, offset) - } - tc.run(t) - } -} - -func TestWindowLastArrayCursor(t *testing.T) { - testcases := []aggArrayCursorTest{ - { - name: "window", - every: 15 * time.Minute, - inputArrays: []*cursors.IntegerArray{ - makeIntegerArray( - 60, - mustParseTime("2010-01-01T00:00:00Z"), time.Minute, - func(i int64) int64 { return i }, - ), - }, - wantIntegers: []*cursors.IntegerArray{ - makeIntegerArray( - 4, - mustParseTime("2010-01-01T00:14:00Z"), 15*time.Minute, - func(i int64) int64 { return 14 + 15*i }, - ), - }, - }, - { - name: "empty windows", - every: time.Minute, - inputArrays: []*cursors.IntegerArray{ - makeIntegerArray( - 4, - mustParseTime("2010-01-01T00:00:00Z"), 15*time.Minute, - func(i int64) int64 { return i }, - ), - }, - wantIntegers: []*cursors.IntegerArray{ - makeIntegerArray( - 4, - mustParseTime("2010-01-01T00:00:00Z"), 15*time.Minute, - func(i int64) int64 { return i }, - ), - }, - }, - { - name: "unaligned window", - every: 15 * time.Minute, - inputArrays: []*cursors.IntegerArray{ - makeIntegerArray( - 60, - mustParseTime("2010-01-01T00:00:30Z"), time.Minute, - func(i int64) int64 { return i }, - ), - }, - wantIntegers: []*cursors.IntegerArray{ - makeIntegerArray( - 4, - mustParseTime("2010-01-01T00:14:30Z"), 15*time.Minute, - func(i int64) int64 { return 14 + 15*i }, - ), - }, - }, - { - name: "more unaligned window", - every: 15 * time.Minute, - inputArrays: []*cursors.IntegerArray{ - makeIntegerArray( - 60, - mustParseTime("2010-01-01T00:01:30Z"), time.Minute, - func(i int64) int64 { return i }, - ), - }, - wantIntegers: []*cursors.IntegerArray{ - { - Timestamps: []int64{ - mustParseTime("2010-01-01T00:14:30Z").UnixNano(), - mustParseTime("2010-01-01T00:29:30Z").UnixNano(), - mustParseTime("2010-01-01T00:44:30Z").UnixNano(), - mustParseTime("2010-01-01T00:59:30Z").UnixNano(), - mustParseTime("2010-01-01T01:00:30Z").UnixNano(), - }, - Values: []int64{13, 28, 43, 58, 59}, - }, - }, - }, - { - name: "window two input arrays", - every: 15 * time.Minute, - inputArrays: []*cursors.IntegerArray{ - makeIntegerArray( - 60, - mustParseTime("2010-01-01T00:00:00Z"), time.Minute, - func(i int64) int64 { return i }, - ), - makeIntegerArray( - 60, - mustParseTime("2010-01-01T01:00:00Z"), time.Minute, - func(i int64) int64 { return 60 + i }, - ), - }, - wantIntegers: []*cursors.IntegerArray{ - makeIntegerArray( - 8, - mustParseTime("2010-01-01T00:14:00Z"), 15*time.Minute, - func(i int64) int64 { return 14 + 15*i }, - ), - }, - }, - { - name: "window spans input arrays", - every: 40 * time.Minute, - inputArrays: []*cursors.IntegerArray{ - makeIntegerArray( - 60, - mustParseTime("2010-01-01T00:00:00Z"), time.Minute, - func(i int64) int64 { return i }, - ), - makeIntegerArray( - 60, - mustParseTime("2010-01-01T01:00:00Z"), time.Minute, - func(i int64) int64 { return 60 + i }, - ), - }, - wantIntegers: []*cursors.IntegerArray{ - makeIntegerArray( - 3, - mustParseTime("2010-01-01T00:39:00Z"), 40*time.Minute, - func(i int64) int64 { return 39 + 40*i }, - ), - }, - }, - { - name: "more windows than MaxPointsPerBlock", - every: 2 * time.Millisecond, - inputArrays: []*cursors.IntegerArray{ - makeIntegerArray( // 1 second, one point per ms - 1000, - mustParseTime("2010-01-01T00:00:00Z"), time.Millisecond, - func(i int64) int64 { return i }, - ), - makeIntegerArray( // 1 second, one point per ms - 1000, - mustParseTime("2010-01-01T00:00:01Z"), time.Millisecond, - func(i int64) int64 { return 1000 + i }, - ), - makeIntegerArray( // 1 second, one point per ms - 1000, - mustParseTime("2010-01-01T00:00:02Z"), time.Millisecond, - func(i int64) int64 { return 2000 + i }, - ), - }, - wantIntegers: []*cursors.IntegerArray{ - makeIntegerArray( - 1000, - mustParseTime("2010-01-01T00:00:00.001Z"), 2*time.Millisecond, - func(i int64) int64 { return 1 + 2*i }, - ), - makeIntegerArray( - 500, - mustParseTime("2010-01-01T00:00:02.001Z"), 2*time.Millisecond, - func(i int64) int64 { return 2001 + 2*i }, - ), - }, - }, - { - name: "MaxPointsPerBlock", - every: time.Millisecond, - inputArrays: []*cursors.IntegerArray{ - makeIntegerArray( // 1 second, one point per ms - 1000, - mustParseTime("2010-01-01T00:00:00Z"), time.Millisecond, - func(i int64) int64 { return i }, - ), - makeIntegerArray( // 1 second, one point per ms - 1000, - mustParseTime("2010-01-01T00:00:01Z"), time.Millisecond, - func(i int64) int64 { return 1000 + i }, - ), - }, - wantIntegers: []*cursors.IntegerArray{ - makeIntegerArray( - 1000, - mustParseTime("2010-01-01T00:00:00Z"), time.Millisecond, - func(i int64) int64 { return i }, - ), - makeIntegerArray( - 1000, - mustParseTime("2010-01-01T00:00:01Z"), time.Millisecond, - func(i int64) int64 { return 1000 + i }, - ), - }, - }, - } - for _, tc := range testcases { - tc.createCursorFn = func(cur cursors.IntegerArrayCursor, every, offset int64) cursors.Cursor { - return newIntegerWindowLastArrayCursor(cur, every, offset) - } - tc.run(t) - } -} - -func TestIntegerCountArrayCursor(t *testing.T) { - maxTimestamp := time.Unix(0, math.MaxInt64) - - testcases := []aggArrayCursorTest{ - { - name: "window", - every: 15 * time.Minute, - inputArrays: []*cursors.IntegerArray{ - makeIntegerArray( - 60, - mustParseTime("2010-01-01T00:00:00Z"), time.Minute, - func(i int64) int64 { return 100 + i }, - ), - }, - wantIntegers: []*cursors.IntegerArray{ - makeIntegerArray(4, mustParseTime("2010-01-01T00:15:00Z"), 15*time.Minute, func(int64) int64 { return 15 }), - }, - }, - { - name: "offset window", - every: 15 * time.Minute, - offset: time.Minute, - inputArrays: []*cursors.IntegerArray{ - makeIntegerArray( - 60, - mustParseTime("2010-01-01T00:00:00Z"), time.Minute, - func(i int64) int64 { return i }, - ), - }, - wantIntegers: []*cursors.IntegerArray{ - makeIntegerArray(5, mustParseTime("2010-01-01T00:01:00Z"), 15*time.Minute, func(i int64) int64 { - switch i { - case 0: - return 1 - case 4: - return 14 - default: - return 15 - } - }), - }, - }, - { - name: "empty windows", - every: time.Minute, - inputArrays: []*cursors.IntegerArray{ - makeIntegerArray( - 4, - mustParseTime("2010-01-01T00:00:00Z"), 15*time.Minute, - func(i int64) int64 { return 100 + i }, - ), - }, - wantIntegers: []*cursors.IntegerArray{ - makeIntegerArray( - 4, - mustParseTime("2010-01-01T00:01:00Z"), 15*time.Minute, - func(i int64) int64 { return 1 }, - ), - }, - }, - { - name: "empty offset windows", - every: time.Minute, - offset: time.Minute, - inputArrays: []*cursors.IntegerArray{ - makeIntegerArray( - 4, - mustParseTime("2010-01-01T00:00:00Z"), 15*time.Minute, - func(i int64) int64 { return 100 + i }, - ), - }, - wantIntegers: []*cursors.IntegerArray{ - makeIntegerArray( - 4, - mustParseTime("2010-01-01T00:01:00Z"), 15*time.Minute, - func(int64) int64 { return 1 }, - ), - }, - }, - { - name: "unaligned window", - every: 15 * time.Minute, - inputArrays: []*cursors.IntegerArray{ - makeIntegerArray( - 60, - mustParseTime("2010-01-01T00:00:30Z"), time.Minute, - func(i int64) int64 { return 100 + i }, - ), - }, - wantIntegers: []*cursors.IntegerArray{ - makeIntegerArray( - 4, - mustParseTime("2010-01-01T00:15:00Z"), 15*time.Minute, - func(i int64) int64 { - return 15 - }), - }, - }, - { - name: "unaligned offset window", - every: 15 * time.Minute, - offset: 45 * time.Second, - inputArrays: []*cursors.IntegerArray{ - makeIntegerArray( - 60, - mustParseTime("2010-01-01T00:00:30Z"), time.Minute, - func(i int64) int64 { return i }, - ), - }, - wantIntegers: []*cursors.IntegerArray{ - makeIntegerArray( - 5, - mustParseTime("2010-01-01T00:00:45Z"), 15*time.Minute, - func(i int64) int64 { - switch i { - case 0: - return 1 - case 4: - return 14 - default: - return 15 - } - }), - }, - }, - { - name: "more unaligned window", - every: 15 * time.Minute, - inputArrays: []*cursors.IntegerArray{ - makeIntegerArray( - 60, - mustParseTime("2010-01-01T00:01:30Z"), time.Minute, - func(i int64) int64 { return 100 + i }, - ), - }, - wantIntegers: []*cursors.IntegerArray{ - makeIntegerArray( - 5, - mustParseTime("2010-01-01T00:15:00Z"), 15*time.Minute, - func(i int64) int64 { - switch i { - case 0: - return 14 - case 4: - return 1 - default: - return 15 - } - }), - }, - }, - { - name: "window two input arrays", - every: 15 * time.Minute, - inputArrays: []*cursors.IntegerArray{ - makeIntegerArray( - 60, - mustParseTime("2010-01-01T00:00:00Z"), time.Minute, - func(i int64) int64 { return 100 + i }, - ), - makeIntegerArray( - 60, - mustParseTime("2010-01-01T01:00:00Z"), time.Minute, - func(i int64) int64 { return 200 + i }, - ), - }, - wantIntegers: []*cursors.IntegerArray{ - makeIntegerArray(8, mustParseTime("2010-01-01T00:15:00Z"), 15*time.Minute, func(int64) int64 { return 15 }), - }, - }, - { - name: "offset window two input arrays", - every: 30 * time.Minute, - offset: 27 * time.Minute, - inputArrays: []*cursors.IntegerArray{ - makeIntegerArray( - 60, - mustParseTime("2010-01-01T00:00:00Z"), time.Minute, - func(i int64) int64 { return i }, - ), - makeIntegerArray( - 60, - mustParseTime("2010-01-01T01:00:00Z"), time.Minute, - func(i int64) int64 { return 60 + i }, - ), - }, - wantIntegers: []*cursors.IntegerArray{ - makeIntegerArray(5, mustParseTime("2010-01-01T00:27:00Z"), 30*time.Minute, func(i int64) int64 { - switch i { - case 0: - return 27 - case 4: - return 3 - default: - return 30 - } - }), - }, - }, - { - name: "window spans input arrays", - every: 40 * time.Minute, - inputArrays: []*cursors.IntegerArray{ - makeIntegerArray( - 60, - mustParseTime("2010-01-01T00:00:00Z"), time.Minute, - func(i int64) int64 { return 100 + i }, - ), - makeIntegerArray( - 60, - mustParseTime("2010-01-01T01:00:00Z"), time.Minute, - func(i int64) int64 { return 200 + i }, - ), - }, - wantIntegers: []*cursors.IntegerArray{ - makeIntegerArray(3, mustParseTime("2010-01-01T00:40:00Z"), 40*time.Minute, func(int64) int64 { return 40 }), - }, - }, - { - name: "offset window spans input arrays", - every: 40 * time.Minute, - offset: 10 * time.Minute, - inputArrays: []*cursors.IntegerArray{ - makeIntegerArray( - 60, - mustParseTime("2010-01-01T00:00:00Z"), time.Minute, - func(i int64) int64 { return i }, - ), - makeIntegerArray( - 60, - mustParseTime("2010-01-01T01:00:00Z"), time.Minute, - func(i int64) int64 { return 60 + i }, - ), - }, - wantIntegers: []*cursors.IntegerArray{ - makeIntegerArray(4, mustParseTime("2010-01-01T00:10:00Z"), 40*time.Minute, func(i int64) int64 { - switch i { - case 0: - return 10 - case 3: - return 30 - default: - return 40 - } - }), - }, - }, - { - name: "more windows than MaxPointsPerBlock", - every: 2 * time.Millisecond, - inputArrays: []*cursors.IntegerArray{ - makeIntegerArray( // 1 second, one point per ms - 1000, - mustParseTime("2010-01-01T00:00:00Z"), time.Millisecond, - func(i int64) int64 { return i }, - ), - makeIntegerArray( // 1 second, one point per ms - 1000, - mustParseTime("2010-01-01T00:00:01Z"), time.Millisecond, - func(i int64) int64 { return i }, - ), - makeIntegerArray( // 1 second, one point per ms - 1000, - mustParseTime("2010-01-01T00:00:02Z"), time.Millisecond, - func(i int64) int64 { return i }, - ), - }, - wantIntegers: []*cursors.IntegerArray{ - makeIntegerArray( - 1000, - mustParseTime("2010-01-01T00:00:00.002Z"), 2*time.Millisecond, - func(i int64) int64 { return 2 }, - ), - makeIntegerArray( - 500, - mustParseTime("2010-01-01T00:00:02.002Z"), 2*time.Millisecond, - func(i int64) int64 { return 2 }, - ), - }, - }, - { - name: "more offset windows than MaxPointsPerBlock", - every: 2 * time.Millisecond, - offset: 1 * time.Millisecond, - inputArrays: []*cursors.IntegerArray{ - makeIntegerArray( // 1 second, one point per ms - 1000, - mustParseTime("2010-01-01T00:00:00Z"), time.Millisecond, - func(i int64) int64 { return i }, - ), - makeIntegerArray( // 1 second, one point per ms - 1000, - mustParseTime("2010-01-01T00:00:01Z"), time.Millisecond, - func(i int64) int64 { return 1000 + i }, - ), - makeIntegerArray( // 1 second, one point per ms - 1000, - mustParseTime("2010-01-01T00:00:02Z"), time.Millisecond, - func(i int64) int64 { return 2000 + i }, - ), - }, - wantIntegers: []*cursors.IntegerArray{ - makeIntegerArray( - 1000, - mustParseTime("2010-01-01T00:00:00.001Z"), 2*time.Millisecond, - func(i int64) int64 { - switch i { - case 0: - return 1 - default: - return 2 - } - }, - ), - makeIntegerArray( - 501, - mustParseTime("2010-01-01T00:00:02.001Z"), 2*time.Millisecond, - func(i int64) int64 { - switch i { - case 500: - return 1 - default: - return 2 - } - }, - ), - }, - }, - { - name: "whole series", - inputArrays: []*cursors.IntegerArray{ - makeIntegerArray( - 60, - mustParseTime("2010-01-01T00:00:00Z"), time.Minute, - func(i int64) int64 { return 100 + i }, - ), - }, - wantIntegers: []*cursors.IntegerArray{ - makeIntegerArray(1, maxTimestamp, 40*time.Minute, func(i int64) int64 { return 60 }), - }, - }, - { - name: "whole series no points", - inputArrays: []*cursors.IntegerArray{{}}, - wantIntegers: []*cursors.IntegerArray{}, - }, - { - name: "whole series two arrays", - inputArrays: []*cursors.IntegerArray{ - makeIntegerArray( - 60, - mustParseTime("2010-01-01T00:00:00Z"), time.Minute, - func(i int64) int64 { return 100 + i }, - ), - makeIntegerArray( - 60, - mustParseTime("2010-01-01T01:00:00Z"), time.Minute, - func(i int64) int64 { return 100 + i }, - ), - }, - wantIntegers: []*cursors.IntegerArray{ - makeIntegerArray(1, maxTimestamp, 40*time.Minute, func(int64) int64 { return 120 }), - }, - }, - { - name: "whole series span epoch", - inputArrays: []*cursors.IntegerArray{ - makeIntegerArray( - 120, - mustParseTime("1969-12-31T23:00:00Z"), time.Minute, - func(i int64) int64 { return 100 + i }, - ), - }, - wantIntegers: []*cursors.IntegerArray{ - makeIntegerArray(1, maxTimestamp, 40*time.Minute, func(int64) int64 { return 120 }), - }, - }, - { - name: "whole series span epoch two arrays", - inputArrays: []*cursors.IntegerArray{ - makeIntegerArray( - 60, - mustParseTime("1969-12-31T23:00:00Z"), time.Minute, - func(i int64) int64 { return 100 + i }, - ), - makeIntegerArray( - 60, - mustParseTime("1970-01-01T00:00:00Z"), time.Minute, - func(i int64) int64 { return 100 + i }, - ), - }, - wantIntegers: []*cursors.IntegerArray{ - makeIntegerArray(1, maxTimestamp, 40*time.Minute, func(int64) int64 { return 120 }), - }, - }, - { - name: "whole series, with max int64 timestamp", - inputArrays: []*cursors.IntegerArray{ - { - Timestamps: []int64{math.MaxInt64}, - Values: []int64{0}, - }, - }, - wantIntegers: []*cursors.IntegerArray{ - { - Timestamps: []int64{math.MaxInt64}, - Values: []int64{1}, - }, - }, - }, - } - for _, tc := range testcases { - tc.createCursorFn = func(cur cursors.IntegerArrayCursor, every, offset int64) cursors.Cursor { - return newIntegerWindowCountArrayCursor(cur, every, offset) - } - tc.run(t) - } -} - -func TestIntegerSumArrayCursor(t *testing.T) { - maxTimestamp := time.Unix(0, math.MaxInt64) - - testcases := []aggArrayCursorTest{ - { - name: "window", - every: 15 * time.Minute, - inputArrays: []*cursors.IntegerArray{ - makeIntegerArray( - 60, - mustParseTime("2010-01-01T00:00:00Z"), time.Minute, - func(i int64) int64 { return 2 }, - ), - }, - wantIntegers: []*cursors.IntegerArray{ - makeIntegerArray(4, mustParseTime("2010-01-01T00:15:00Z"), 15*time.Minute, func(int64) int64 { return 30 }), - }, - }, - { - name: "empty windows", - every: time.Minute, - inputArrays: []*cursors.IntegerArray{ - makeIntegerArray( - 4, - mustParseTime("2010-01-01T00:00:00Z"), 15*time.Minute, - func(i int64) int64 { return 100 + i }, - ), - }, - wantIntegers: []*cursors.IntegerArray{ - makeIntegerArray( - 4, - mustParseTime("2010-01-01T00:01:00Z"), 15*time.Minute, - func(i int64) int64 { return 100 + i }, - ), - }, - }, - { - name: "unaligned window", - every: 15 * time.Minute, - inputArrays: []*cursors.IntegerArray{ - makeIntegerArray( - 60, - mustParseTime("2010-01-01T00:00:30Z"), time.Minute, - func(i int64) int64 { return 2 }, - ), - }, - wantIntegers: []*cursors.IntegerArray{ - makeIntegerArray( - 4, - mustParseTime("2010-01-01T00:15:00Z"), 15*time.Minute, - func(i int64) int64 { - return 30 - }), - }, - }, - { - name: "more unaligned window", - every: 15 * time.Minute, - inputArrays: []*cursors.IntegerArray{ - makeIntegerArray( - 60, - mustParseTime("2010-01-01T00:01:30Z"), time.Minute, - func(i int64) int64 { return 2 }, - ), - }, - wantIntegers: []*cursors.IntegerArray{ - makeIntegerArray( - 5, - mustParseTime("2010-01-01T00:15:00Z"), 15*time.Minute, - func(i int64) int64 { - switch i { - case 0: - return 28 - case 4: - return 2 - default: - return 30 - } - }), - }, - }, - { - name: "window two input arrays", - every: 15 * time.Minute, - inputArrays: []*cursors.IntegerArray{ - makeIntegerArray( - 60, - mustParseTime("2010-01-01T00:00:00Z"), time.Minute, - func(i int64) int64 { return 2 }, - ), - makeIntegerArray( - 60, - mustParseTime("2010-01-01T01:00:00Z"), time.Minute, - func(i int64) int64 { return 3 }, - ), - }, - wantIntegers: []*cursors.IntegerArray{ - makeIntegerArray(8, mustParseTime("2010-01-01T00:15:00Z"), 15*time.Minute, - func(i int64) int64 { - if i < 4 { - return 30 - } else { - return 45 - } - }), - }, - }, - { - name: "window spans input arrays", - every: 40 * time.Minute, - inputArrays: []*cursors.IntegerArray{ - makeIntegerArray( - 60, - mustParseTime("2010-01-01T00:00:00Z"), time.Minute, - func(i int64) int64 { return 2 }, - ), - makeIntegerArray( - 60, - mustParseTime("2010-01-01T01:00:00Z"), time.Minute, - func(i int64) int64 { return 3 }, - ), - }, - wantIntegers: []*cursors.IntegerArray{ - makeIntegerArray(3, mustParseTime("2010-01-01T00:40:00Z"), 40*time.Minute, - func(i int64) int64 { - switch i { - case 0: - return 80 - case 1: - return 100 - case 2: - return 120 - } - return -1 - }), - }, - }, - { - name: "more windows than MaxPointsPerBlock", - every: 2 * time.Millisecond, - inputArrays: []*cursors.IntegerArray{ - makeIntegerArray( // 1 second, one point per ms - 1000, - mustParseTime("2010-01-01T00:00:00Z"), time.Millisecond, - func(i int64) int64 { return 2 }, - ), - makeIntegerArray( // 1 second, one point per ms - 1000, - mustParseTime("2010-01-01T00:00:01Z"), time.Millisecond, - func(i int64) int64 { return 3 }, - ), - makeIntegerArray( // 1 second, one point per ms - 1000, - mustParseTime("2010-01-01T00:00:02Z"), time.Millisecond, - func(i int64) int64 { return 4 }, - ), - }, - wantIntegers: []*cursors.IntegerArray{ - makeIntegerArray( - 1000, - mustParseTime("2010-01-01T00:00:00.002Z"), 2*time.Millisecond, - func(i int64) int64 { - if i < 500 { - return 4 - } else { - return 6 - } - }, - ), - makeIntegerArray( - 500, - mustParseTime("2010-01-01T00:00:02.002Z"), 2*time.Millisecond, - func(i int64) int64 { return 8 }, - ), - }, - }, - { - name: "whole series", - inputArrays: []*cursors.IntegerArray{ - makeIntegerArray( - 60, - mustParseTime("2010-01-01T00:00:00Z"), time.Minute, - func(i int64) int64 { return 2 }, - ), - }, - wantIntegers: []*cursors.IntegerArray{ - makeIntegerArray(1, maxTimestamp, 40*time.Minute, func(i int64) int64 { return 120 }), - }, - }, - { - name: "whole series no points", - inputArrays: []*cursors.IntegerArray{{}}, - wantIntegers: []*cursors.IntegerArray{}, - }, - { - name: "whole series two arrays", - inputArrays: []*cursors.IntegerArray{ - makeIntegerArray( - 60, - mustParseTime("2010-01-01T00:00:00Z"), time.Minute, - func(i int64) int64 { return 2 }, - ), - makeIntegerArray( - 60, - mustParseTime("2010-01-01T01:00:00Z"), time.Minute, - func(i int64) int64 { return 3 }, - ), - }, - wantIntegers: []*cursors.IntegerArray{ - makeIntegerArray(1, maxTimestamp, 40*time.Minute, - func(int64) int64 { - return 300 - }), - }, - }, - { - name: "whole series span epoch", - inputArrays: []*cursors.IntegerArray{ - makeIntegerArray( - 120, - mustParseTime("1969-12-31T23:00:00Z"), time.Minute, - func(i int64) int64 { return 2 }, - ), - }, - wantIntegers: []*cursors.IntegerArray{ - makeIntegerArray(1, maxTimestamp, 40*time.Minute, func(int64) int64 { return 240 }), - }, - }, - { - name: "whole series span epoch two arrays", - inputArrays: []*cursors.IntegerArray{ - makeIntegerArray( - 60, - mustParseTime("1969-12-31T23:00:00Z"), time.Minute, - func(i int64) int64 { return 2 }, - ), - makeIntegerArray( - 60, - mustParseTime("1970-01-01T00:00:00Z"), time.Minute, - func(i int64) int64 { return 3 }, - ), - }, - wantIntegers: []*cursors.IntegerArray{ - makeIntegerArray(1, maxTimestamp, 40*time.Minute, func(int64) int64 { return 300 }), - }, - }, - { - name: "whole series, with max int64 timestamp", - inputArrays: []*cursors.IntegerArray{ - { - Timestamps: []int64{math.MaxInt64}, - Values: []int64{100}, - }, - }, - wantIntegers: []*cursors.IntegerArray{ - { - Timestamps: []int64{math.MaxInt64}, - Values: []int64{100}, - }, - }, - }, - } - for _, tc := range testcases { - tc.createCursorFn = func(cur cursors.IntegerArrayCursor, every, offset int64) cursors.Cursor { - return newIntegerWindowSumArrayCursor(cur, every, offset) - } - tc.run(t) - } -} - -func TestWindowMinArrayCursor(t *testing.T) { - testcases := []aggArrayCursorTest{ - { - name: "no window", - every: 0, - inputArrays: []*cursors.IntegerArray{ - makeIntegerArray( - 60, - mustParseTime("2010-01-01T00:00:00Z"), time.Minute, - func(i int64) int64 { return 100 + i }, - ), - }, - wantIntegers: []*cursors.IntegerArray{ - makeIntegerArray(1, mustParseTime("2010-01-01T00:00:00Z"), 0, func(int64) int64 { return 100 }), - }, - }, - { - name: "no window min int", - every: 0, - inputArrays: []*cursors.IntegerArray{ - makeIntegerArray( - 60, - mustParseTime("2010-01-01T00:00:00Z"), time.Minute, - func(i int64) int64 { - if i%2 == 0 { - return math.MinInt64 - } - return 0 - }, - ), - }, - wantIntegers: []*cursors.IntegerArray{ - makeIntegerArray(1, mustParseTime("2010-01-01T00:00:00Z"), 0, func(int64) int64 { return math.MinInt64 }), - }, - }, - { - name: "no window max int", - every: 0, - inputArrays: []*cursors.IntegerArray{ - makeIntegerArray( - 60, - mustParseTime("2010-01-01T00:00:00Z"), time.Minute, - func(i int64) int64 { - if i%2 == 0 { - return math.MaxInt64 - } - return 0 - }, - ), - }, - wantIntegers: []*cursors.IntegerArray{ - makeIntegerArray(1, mustParseTime("2010-01-01T00:01:00Z"), 0, func(int64) int64 { return 0 }), - }, - }, - { - name: "window", - every: time.Hour, - inputArrays: []*cursors.IntegerArray{ - makeIntegerArray( - 16, - mustParseTime("2010-01-01T00:00:00Z"), 15*time.Minute, - func(i int64) int64 { - base := (i / 4) * 100 - m := (i % 4) * 15 - return base + m - }, - ), - }, - wantIntegers: []*cursors.IntegerArray{ - makeIntegerArray(4, mustParseTime("2010-01-01T00:00:00Z"), time.Hour, - func(i int64) int64 { return i * 100 }), - }, - }, - { - name: "window offset", - every: time.Hour, - offset: 30 * time.Minute, - inputArrays: []*cursors.IntegerArray{ - makeIntegerArray( - 16, - mustParseTime("2010-01-01T00:00:00Z"), 15*time.Minute, - func(i int64) int64 { - base := (i / 4) * 100 - m := (i % 4) * 15 - return base + m - }, - ), - }, - wantIntegers: []*cursors.IntegerArray{ - { - Timestamps: []int64{ - mustParseTime("2010-01-01T00:00:00Z").UnixNano(), - mustParseTime("2010-01-01T00:30:00Z").UnixNano(), - mustParseTime("2010-01-01T01:30:00Z").UnixNano(), - mustParseTime("2010-01-01T02:30:00Z").UnixNano(), - mustParseTime("2010-01-01T03:30:00Z").UnixNano(), - }, - Values: []int64{0, 30, 130, 230, 330}, - }, - }, - }, - { - name: "window desc values", - every: time.Hour, - inputArrays: []*cursors.IntegerArray{ - makeIntegerArray( - 16, - mustParseTime("2010-01-01T00:00:00Z"), 15*time.Minute, - func(i int64) int64 { - base := (i / 4) * 100 - m := 60 - (i%4)*15 - return base + m - }, - ), - }, - wantIntegers: []*cursors.IntegerArray{ - makeIntegerArray(4, mustParseTime("2010-01-01T00:45:00Z"), time.Hour, - func(i int64) int64 { return i*100 + 15 }), - }, - }, - { - name: "window offset desc values", - every: time.Hour, - offset: 30 * time.Minute, - inputArrays: []*cursors.IntegerArray{ - makeIntegerArray( - 16, - mustParseTime("2010-01-01T00:00:00Z"), 15*time.Minute, - func(i int64) int64 { - base := (i / 4) * 100 - m := 60 - (i%4)*15 - return base + m - }, - ), - }, - wantIntegers: []*cursors.IntegerArray{ - { - Timestamps: []int64{ - mustParseTime("2010-01-01T00:15:00Z").UnixNano(), - mustParseTime("2010-01-01T00:45:00Z").UnixNano(), - mustParseTime("2010-01-01T01:45:00Z").UnixNano(), - mustParseTime("2010-01-01T02:45:00Z").UnixNano(), - mustParseTime("2010-01-01T03:45:00Z").UnixNano(), - }, - Values: []int64{45, 15, 115, 215, 315}, - }, - }, - }, - { - name: "window min int", - every: time.Hour, - inputArrays: []*cursors.IntegerArray{ - makeIntegerArray( - 16, - mustParseTime("2010-01-01T00:00:00Z"), 15*time.Minute, - func(i int64) int64 { - return math.MinInt64 - }, - ), - }, - wantIntegers: []*cursors.IntegerArray{ - makeIntegerArray(4, mustParseTime("2010-01-01T00:00:00Z"), time.Hour, - func(i int64) int64 { return math.MinInt64 }), - }, - }, - { - name: "window max int", - every: time.Hour, - inputArrays: []*cursors.IntegerArray{ - makeIntegerArray( - 16, - mustParseTime("2010-01-01T00:00:00Z"), 15*time.Minute, - func(i int64) int64 { - return math.MaxInt64 - }, - ), - }, - wantIntegers: []*cursors.IntegerArray{ - makeIntegerArray(4, mustParseTime("2010-01-01T00:00:00Z"), time.Hour, - func(i int64) int64 { return math.MaxInt64 }), - }, - }, - { - name: "empty window", - every: 15 * time.Minute, - inputArrays: []*cursors.IntegerArray{ - makeIntegerArray( - 2, - mustParseTime("2010-01-01T00:05:00Z"), 30*time.Minute, - func(i int64) int64 { - return 100 + i - }, - ), - }, - wantIntegers: []*cursors.IntegerArray{ - makeIntegerArray(2, mustParseTime("2010-01-01T00:05:00Z"), 30*time.Minute, - func(i int64) int64 { return 100 + i }), - }, - }, - } - for _, tc := range testcases { - tc.createCursorFn = func(cur cursors.IntegerArrayCursor, every, offset int64) cursors.Cursor { - return newIntegerWindowMinArrayCursor(cur, every, offset) - } - tc.run(t) - } -} - -func TestWindowMaxArrayCursor(t *testing.T) { - testcases := []aggArrayCursorTest{ - { - name: "no window", - every: 0, - inputArrays: []*cursors.IntegerArray{ - makeIntegerArray( - 60, - mustParseTime("2010-01-01T00:00:00Z"), time.Minute, - func(i int64) int64 { return 100 + i }, - ), - }, - wantIntegers: []*cursors.IntegerArray{ - makeIntegerArray(1, mustParseTime("2010-01-01T00:59:00Z"), 0, func(int64) int64 { return 159 }), - }, - }, - { - name: "no window min int", - every: 0, - inputArrays: []*cursors.IntegerArray{ - makeIntegerArray( - 60, - mustParseTime("2010-01-01T00:00:00Z"), time.Minute, - func(i int64) int64 { - if i%2 == 0 { - return math.MinInt64 - } - return 0 - }, - ), - }, - wantIntegers: []*cursors.IntegerArray{ - makeIntegerArray(1, mustParseTime("2010-01-01T00:01:00Z"), 0, func(int64) int64 { return 0 }), - }, - }, - { - name: "no window max int", - every: 0, - inputArrays: []*cursors.IntegerArray{ - makeIntegerArray( - 60, - mustParseTime("2010-01-01T00:00:00Z"), time.Minute, - func(i int64) int64 { - if i%2 == 0 { - return math.MaxInt64 - } - return 0 - }, - ), - }, - wantIntegers: []*cursors.IntegerArray{ - makeIntegerArray(1, mustParseTime("2010-01-01T00:00:00Z"), 0, func(int64) int64 { return math.MaxInt64 }), - }, - }, - { - name: "window", - every: time.Hour, - inputArrays: []*cursors.IntegerArray{ - makeIntegerArray( - 16, - mustParseTime("2010-01-01T00:00:00Z"), 15*time.Minute, - func(i int64) int64 { - base := (i / 4) * 100 - m := (i % 4) * 15 - return base + m - }, - ), - }, - wantIntegers: []*cursors.IntegerArray{ - makeIntegerArray(4, mustParseTime("2010-01-01T00:45:00Z"), time.Hour, - func(i int64) int64 { return i*100 + 45 }), - }, - }, - { - name: "window offset", - every: time.Hour, - offset: 30 * time.Minute, - inputArrays: []*cursors.IntegerArray{ - makeIntegerArray( - 16, - mustParseTime("2010-01-01T00:00:00Z"), 15*time.Minute, - func(i int64) int64 { - base := (i / 4) * 100 - m := (i % 4) * 15 - return base + m - }, - ), - }, - wantIntegers: []*cursors.IntegerArray{ - { - Timestamps: []int64{ - mustParseTime("2010-01-01T00:15:00Z").UnixNano(), - mustParseTime("2010-01-01T01:15:00Z").UnixNano(), - mustParseTime("2010-01-01T02:15:00Z").UnixNano(), - mustParseTime("2010-01-01T03:15:00Z").UnixNano(), - mustParseTime("2010-01-01T03:45:00Z").UnixNano(), - }, - Values: []int64{15, 115, 215, 315, 345}, - }, - }, - }, - { - name: "window desc values", - every: time.Hour, - inputArrays: []*cursors.IntegerArray{ - makeIntegerArray( - 16, - mustParseTime("2010-01-01T00:00:00Z"), 15*time.Minute, - func(i int64) int64 { - base := (i / 4) * 100 - m := 60 - (i%4)*15 - return base + m - }, - ), - }, - wantIntegers: []*cursors.IntegerArray{ - makeIntegerArray(4, mustParseTime("2010-01-01T00:00:00Z"), time.Hour, - func(i int64) int64 { return i*100 + 60 }), - }, - }, - { - name: "window offset desc values", - every: time.Hour, - offset: 30 * time.Minute, - inputArrays: []*cursors.IntegerArray{ - makeIntegerArray( - 16, - mustParseTime("2010-01-01T00:00:00Z"), 15*time.Minute, - func(i int64) int64 { - base := (i / 4) * 100 - m := 60 - (i%4)*15 - return base + m - }, - ), - }, - wantIntegers: []*cursors.IntegerArray{ - { - Timestamps: []int64{ - mustParseTime("2010-01-01T00:00:00Z").UnixNano(), - mustParseTime("2010-01-01T01:00:00Z").UnixNano(), - mustParseTime("2010-01-01T02:00:00Z").UnixNano(), - mustParseTime("2010-01-01T03:00:00Z").UnixNano(), - mustParseTime("2010-01-01T03:30:00Z").UnixNano(), - }, - Values: []int64{60, 160, 260, 360, 330}, - }, - }, - }, - { - name: "window min int", - every: time.Hour, - inputArrays: []*cursors.IntegerArray{ - makeIntegerArray( - 16, - mustParseTime("2010-01-01T00:00:00Z"), 15*time.Minute, - func(i int64) int64 { - return math.MinInt64 - }, - ), - }, - wantIntegers: []*cursors.IntegerArray{ - makeIntegerArray(4, mustParseTime("2010-01-01T00:00:00Z"), time.Hour, - func(i int64) int64 { return math.MinInt64 }), - }, - }, - { - name: "window max int", - every: time.Hour, - inputArrays: []*cursors.IntegerArray{ - makeIntegerArray( - 16, - mustParseTime("2010-01-01T00:00:00Z"), 15*time.Minute, - func(i int64) int64 { - return math.MaxInt64 - }, - ), - }, - wantIntegers: []*cursors.IntegerArray{ - makeIntegerArray(4, mustParseTime("2010-01-01T00:00:00Z"), time.Hour, - func(i int64) int64 { return math.MaxInt64 }), - }, - }, - { - name: "empty window", - every: 15 * time.Minute, - inputArrays: []*cursors.IntegerArray{ - makeIntegerArray( - 2, - mustParseTime("2010-01-01T00:05:00Z"), 30*time.Minute, - func(i int64) int64 { - return 100 + i - }, - ), - }, - wantIntegers: []*cursors.IntegerArray{ - makeIntegerArray(2, mustParseTime("2010-01-01T00:05:00Z"), 30*time.Minute, - func(i int64) int64 { return 100 + i }), - }, - }, - } - for _, tc := range testcases { - tc.createCursorFn = func(cur cursors.IntegerArrayCursor, every, offset int64) cursors.Cursor { - return newIntegerWindowMaxArrayCursor(cur, every, offset) - } - tc.run(t) - } -} - -func TestWindowMeanArrayCursor(t *testing.T) { - maxTimestamp := time.Unix(0, math.MaxInt64) - - testcases := []aggArrayCursorTest{ - { - name: "no window", - every: 0, - inputArrays: []*cursors.IntegerArray{ - makeIntegerArray( - 5, - mustParseTime("2010-01-01T00:00:00Z"), time.Minute, - func(i int64) int64 { return i + 1 }, - ), - }, - wantFloats: []*cursors.FloatArray{ - makeFloatArray(1, maxTimestamp, 0, func(int64) float64 { return 3.0 }), - }, - }, - { - name: "no window fraction result", - every: 0, - inputArrays: []*cursors.IntegerArray{ - makeIntegerArray( - 6, - mustParseTime("2010-01-01T00:00:00Z"), time.Minute, - func(i int64) int64 { return i + 1 }, - ), - }, - wantFloats: []*cursors.FloatArray{ - makeFloatArray(1, maxTimestamp, 0, func(int64) float64 { return 3.5 }), - }, - }, - { - name: "no window empty", - every: 0, - inputArrays: []*cursors.IntegerArray{}, - wantFloats: []*cursors.FloatArray{}, - }, - { - name: "window", - every: 30 * time.Minute, - inputArrays: []*cursors.IntegerArray{ - makeIntegerArray( - 8, - mustParseTime("2010-01-01T00:00:00Z"), 15*time.Minute, - func(i int64) int64 { - return i - }, - ), - }, - wantFloats: []*cursors.FloatArray{ - makeFloatArray(4, mustParseTime("2010-01-01T00:30:00Z"), 30*time.Minute, - func(i int64) float64 { return 0.5 + float64(i)*2 }), - }, - }, - { - name: "window offset", - every: 30 * time.Minute, - offset: 5 * time.Minute, - inputArrays: []*cursors.IntegerArray{ - makeIntegerArray( - 8, - mustParseTime("2010-01-01T00:00:00Z"), 15*time.Minute, - func(i int64) int64 { - return i - }, - ), - }, - wantFloats: []*cursors.FloatArray{ - makeFloatArray(5, mustParseTime("2010-01-01T00:05:00Z"), 30*time.Minute, - func(i int64) float64 { return []float64{0, 1.5, 3.5, 5.5, 7}[i] }), - }, - }, - { - name: "empty window", - every: 15 * time.Minute, - inputArrays: []*cursors.IntegerArray{ - makeIntegerArray( - 2, - mustParseTime("2010-01-01T00:05:00Z"), 30*time.Minute, - func(i int64) int64 { - return 100 + i - }, - ), - }, - wantFloats: []*cursors.FloatArray{ - makeFloatArray(2, mustParseTime("2010-01-01T00:15:00Z"), 30*time.Minute, - func(i int64) float64 { return 100 + float64(i) }), - }, - }, - } - for _, tc := range testcases { - tc.createCursorFn = func(cur cursors.IntegerArrayCursor, every, offset int64) cursors.Cursor { - return newIntegerWindowMeanArrayCursor(cur, every, offset) - } - tc.run(t) - } -} - -type MockExpression struct { - EvalBoolFunc func(v Valuer) bool -} - -func (e *MockExpression) EvalBool(v Valuer) bool { return e.EvalBoolFunc(v) } diff --git a/v1/storage/reads/flux_predicate.go b/storage/reads/flux_predicate.go similarity index 100% rename from v1/storage/reads/flux_predicate.go rename to storage/reads/flux_predicate.go diff --git a/v1/storage/reads/flux_reader.go b/storage/reads/flux_reader.go similarity index 100% rename from v1/storage/reads/flux_reader.go rename to storage/reads/flux_reader.go diff --git a/v1/storage/reads/flux_table.gen.go b/storage/reads/flux_table.gen.go similarity index 100% rename from v1/storage/reads/flux_table.gen.go rename to storage/reads/flux_table.gen.go diff --git a/v1/storage/reads/flux_table.go b/storage/reads/flux_table.go similarity index 100% rename from v1/storage/reads/flux_table.go rename to storage/reads/flux_table.go diff --git a/v1/storage/reads/flux_table_test.go b/storage/reads/flux_table_test.go similarity index 100% rename from v1/storage/reads/flux_table_test.go rename to storage/reads/flux_table_test.go diff --git a/v1/storage/reads/flux_tags_cache.go b/storage/reads/flux_tags_cache.go similarity index 100% rename from v1/storage/reads/flux_tags_cache.go rename to storage/reads/flux_tags_cache.go diff --git a/storage/reads/group_resultset.go b/storage/reads/group_resultset.go index b27ebb21f0..c1040a800a 100644 --- a/storage/reads/group_resultset.go +++ b/storage/reads/group_resultset.go @@ -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 } diff --git a/storage/reads/group_resultset_test.go b/storage/reads/group_resultset_test.go index 022b91d83f..d6d81880ae 100644 --- a/storage/reads/group_resultset_test.go +++ b/storage/reads/group_resultset_test.go @@ -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) - } -} diff --git a/v1/storage/reads/helpers_test.go b/storage/reads/helpers_test.go similarity index 100% rename from v1/storage/reads/helpers_test.go rename to storage/reads/helpers_test.go diff --git a/storage/reads/influxql_predicate_test.go b/storage/reads/influxql_predicate_test.go index 718867e7d6..606b7f8e18 100644 --- a/storage/reads/influxql_predicate_test.go +++ b/storage/reads/influxql_predicate_test.go @@ -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) { diff --git a/storage/reads/keymerger.go b/storage/reads/keymerger.go index da47ee2c1b..2ce4f07cd5 100644 --- a/storage/reads/keymerger.go +++ b/storage/reads/keymerger.go @@ -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 diff --git a/storage/reads/keymerger_test.go b/storage/reads/keymerger_test.go index f779bf4b2a..00bc60269e 100644 --- a/storage/reads/keymerger_test.go +++ b/storage/reads/keymerger_test.go @@ -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) { diff --git a/storage/reads/predicate_test.go b/storage/reads/predicate_test.go index 1e8643c9f9..c40b8f2e4b 100644 --- a/storage/reads/predicate_test.go +++ b/storage/reads/predicate_test.go @@ -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) { diff --git a/storage/reads/resultset.go b/storage/reads/resultset.go index 2506299f1e..71e90f29cb 100644 --- a/storage/reads/resultset.go +++ b/storage/reads/resultset.go @@ -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() } diff --git a/storage/reads/resultset_lineprotocol.go b/storage/reads/resultset_lineprotocol.go index 2fb95b7509..c88adfe965 100644 --- a/storage/reads/resultset_lineprotocol.go +++ b/storage/reads/resultset_lineprotocol.go @@ -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" ) diff --git a/storage/reads/series_cursor.go b/storage/reads/series_cursor.go index bd6420e92b..1d691d5e7e 100644 --- a/storage/reads/series_cursor.go +++ b/storage/reads/series_cursor.go @@ -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() } diff --git a/storage/reads/series_cursor_test.go b/storage/reads/series_cursor_test.go index ea531baa99..8eee6fe0b5 100644 --- a/storage/reads/series_cursor_test.go +++ b/storage/reads/series_cursor_test.go @@ -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 -} diff --git a/storage/reads/store.go b/storage/reads/store.go index 7ea4ab98aa..19cbc88691 100644 --- a/storage/reads/store.go +++ b/storage/reads/store.go @@ -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 } diff --git a/storage/reads/store_test.go b/storage/reads/store_test.go index 03a2e3764a..0c03c15be4 100644 --- a/storage/reads/store_test.go +++ b/storage/reads/store_test.go @@ -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" ) diff --git a/storage/reads/tagsbuffer.go b/storage/reads/tagsbuffer.go index e0982b0b27..1195960ab2 100644 --- a/storage/reads/tagsbuffer.go +++ b/storage/reads/tagsbuffer.go @@ -1,7 +1,7 @@ package reads import ( - "github.com/influxdata/influxdb/v2/models" + "github.com/influxdata/influxdb/v2/v1/models" ) type tagsBuffer struct { diff --git a/storage/reads/types.tmpldata b/storage/reads/types.tmpldata deleted file mode 100644 index b15cb586d5..0000000000 --- a/storage/reads/types.tmpldata +++ /dev/null @@ -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" - } -] diff --git a/storage/reads/viewer.go b/storage/reads/viewer.go index cece52f9f9..8eee6fe0b5 100644 --- a/storage/reads/viewer.go +++ b/storage/reads/viewer.go @@ -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) -} diff --git a/storage/readservice/source.go b/storage/readservice/source.go deleted file mode 100644 index 8c9f4b0cb6..0000000000 --- a/storage/readservice/source.go +++ /dev/null @@ -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) -} diff --git a/storage/readservice/store.go b/storage/readservice/store.go deleted file mode 100644 index a9e2ee0b02..0000000000 --- a/storage/readservice/store.go +++ /dev/null @@ -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 } diff --git a/storage/retention.go b/storage/retention.go index bf7f989b59..440acaa8d3 100644 --- a/storage/retention.go +++ b/storage/retention.go @@ -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)) } diff --git a/storage/series_cursor.go b/storage/series_cursor.go deleted file mode 100644 index 2bcce8f349..0000000000 --- a/storage/series_cursor.go +++ /dev/null @@ -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 -} diff --git a/storage/series_cursor_test.go b/storage/series_cursor_test.go deleted file mode 100644 index e500af559c..0000000000 --- a/storage/series_cursor_test.go +++ /dev/null @@ -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())) - } -} diff --git a/storage/wal/dump.go b/storage/wal/dump.go deleted file mode 100644 index 9efb313d55..0000000000 --- a/storage/wal/dump.go +++ /dev/null @@ -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 -} diff --git a/storage/wal/dump_test.go b/storage/wal/dump_test.go deleted file mode 100644 index b72eac8b62..0000000000 --- a/storage/wal/dump_test.go +++ /dev/null @@ -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) - } - -} diff --git a/storage/wal/helpers_test.go b/storage/wal/helpers_test.go deleted file mode 100644 index e5a5278d1c..0000000000 --- a/storage/wal/helpers_test.go +++ /dev/null @@ -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) -} diff --git a/storage/wal/metrics.go b/storage/wal/metrics.go deleted file mode 100644 index 132a5f19a8..0000000000 --- a/storage/wal/metrics.go +++ /dev/null @@ -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, - } -} diff --git a/storage/wal/metrics_test.go b/storage/wal/metrics_test.go deleted file mode 100644 index cfb7a215d3..0000000000 --- a/storage/wal/metrics_test.go +++ /dev/null @@ -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) - } - } - } -} diff --git a/storage/wal/pools.go b/storage/wal/pools.go deleted file mode 100644 index 140102f528..0000000000 --- a/storage/wal/pools.go +++ /dev/null @@ -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) -} diff --git a/storage/wal/reader.go b/storage/wal/reader.go deleted file mode 100644 index 2ebf126b18..0000000000 --- a/storage/wal/reader.go +++ /dev/null @@ -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() -} diff --git a/storage/wal/verifier_test.go b/storage/wal/verifier_test.go deleted file mode 100644 index 54eda0c04e..0000000000 --- a/storage/wal/verifier_test.go +++ /dev/null @@ -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 -} diff --git a/storage/wal/verify.go b/storage/wal/verify.go deleted file mode 100644 index 10ba531cf1..0000000000 --- a/storage/wal/verify.go +++ /dev/null @@ -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 -} diff --git a/storage/wal/wal.go b/storage/wal/wal.go deleted file mode 100644 index 3fc742b023..0000000000 --- a/storage/wal/wal.go +++ /dev/null @@ -1,1266 +0,0 @@ -package wal - -import ( - "bufio" - "context" - "encoding/binary" - "fmt" - "io" - "math" - "os" - "path/filepath" - "runtime" - "sort" - "strconv" - "strings" - "sync" - "sync/atomic" - "time" - - "github.com/golang/snappy" - "github.com/prometheus/client_golang/prometheus" - "go.uber.org/zap" - - "github.com/influxdata/influxdb/v2" - "github.com/influxdata/influxdb/v2/kit/tracing" - "github.com/influxdata/influxdb/v2/pkg/limiter" - "github.com/influxdata/influxdb/v2/pkg/pool" - "github.com/influxdata/influxdb/v2/v1/tsdb/engine/tsm1" -) - -const ( - // DefaultSegmentSize of 10MB is the size at which segment files will be rolled over. - DefaultSegmentSize = 10 * 1024 * 1024 - - // WALFileExtension is the file extension we expect for wal segments. - WALFileExtension = "wal" - - // WALFilePrefix is the prefix on all wal segment files. - WALFilePrefix = "_" - - // walEncodeBufSize is the size of the wal entry encoding buffer - walEncodeBufSize = 4 * 1024 * 1024 - - float64EntryType = 1 - integerEntryType = 2 - booleanEntryType = 3 - stringEntryType = 4 - unsignedEntryType = 5 -) - -// WalEntryType is a byte written to a wal segment file that indicates what the following compressed block contains. -type WalEntryType byte - -const ( - // WriteWALEntryType indicates a write entry. - WriteWALEntryType WalEntryType = 0x01 - - // DeleteWALEntryType indicates a delete entry. Deprecated. - _ WalEntryType = 0x02 - - // DeleteRangeWALEntryType indicates a delete range entry. Deprecated. - _ WalEntryType = 0x03 - - // DeleteBucketRangeWALEntryType indicates a delete bucket range entry. - DeleteBucketRangeWALEntryType WalEntryType = 0x04 -) - -var ( - // ErrWALClosed is returned when attempting to write to a closed WAL file. - ErrWALClosed = fmt.Errorf("WAL closed") - - // ErrWALCorrupt is returned when reading a corrupt WAL entry. - ErrWALCorrupt = fmt.Errorf("corrupted WAL entry") - - defaultWaitingWALWrites = runtime.GOMAXPROCS(0) * 2 - - // bytePool is a shared bytes pool buffer re-cycle []byte slices to reduce allocations. - bytesPool = pool.NewLimitedBytes(256, walEncodeBufSize*2) -) - -// WAL represents the write-ahead log used for writing TSM files. -type WAL struct { - // goroutines waiting for the next fsync - syncCount uint64 - syncWaiters chan chan error - - mu sync.RWMutex - lastWriteTime time.Time - - path string - enabled bool - - // write variables - currentSegmentID int - currentSegmentWriter *WALSegmentWriter - - // cache and flush variables - once sync.Once - closing chan struct{} - - // syncDelay sets the duration to wait before fsyncing writes. A value of 0 (default) - // will cause every write to be fsync'd. This must be set before the WAL - // is opened if a non-default value is required. - syncDelay time.Duration - - // WALOutput is the writer used by the logger. - logger *zap.Logger // Logger to be used for important messages - - // SegmentSize is the file size at which a segment file will be rotated - SegmentSize int - - tracker *walTracker - defaultMetricLabels prometheus.Labels // N.B this must not be mutated after Open is called. - - limiter limiter.Fixed -} - -// NewWAL initializes a new WAL at the given directory. -func NewWAL(path string) *WAL { - logger := zap.NewNop() - return &WAL{ - path: path, - enabled: true, - - // these options should be overridden by any options in the config - SegmentSize: DefaultSegmentSize, - closing: make(chan struct{}), - syncWaiters: make(chan chan error, 1024), - limiter: limiter.NewFixed(defaultWaitingWALWrites), - logger: logger, - } -} - -// WithFsyncDelay sets the fsync delay and should be called before the WAL is opened. -func (l *WAL) WithFsyncDelay(delay time.Duration) { - l.syncDelay = delay -} - -// SetEnabled sets if the WAL is enabled and should be called before the WAL is opened. -func (l *WAL) SetEnabled(enabled bool) { - l.enabled = enabled -} - -// WithLogger sets the WAL's logger. -func (l *WAL) WithLogger(log *zap.Logger) { - l.logger = log.With(zap.String("service", "wal")) -} - -// SetDefaultMetricLabels sets the default labels for metrics on the engine. -// It must be called before the Engine is opened. -func (l *WAL) SetDefaultMetricLabels(labels prometheus.Labels) { - l.defaultMetricLabels = make(prometheus.Labels, len(labels)) - for k, v := range labels { - l.defaultMetricLabels[k] = v - } -} - -// Path returns the directory the log was initialized with. -func (l *WAL) Path() string { - l.mu.RLock() - defer l.mu.RUnlock() - return l.path -} - -// Open opens and initializes the Log. Open can recover from previous unclosed shutdowns. -func (l *WAL) Open(ctx context.Context) error { - l.mu.Lock() - defer l.mu.Unlock() - - if !l.enabled { - return nil - } - - span, _ := tracing.StartSpanFromContext(ctx) - defer span.Finish() - - span.LogKV("segment_size", l.SegmentSize, - "path", l.path) - - // Initialise metrics for trackers. - mmu.Lock() - if wms == nil { - wms = newWALMetrics(l.defaultMetricLabels) - } - mmu.Unlock() - - // Set the shared metrics for the tracker - l.tracker = newWALTracker(wms, l.defaultMetricLabels) - - if err := os.MkdirAll(l.path, 0777); err != nil { - return err - } - - segments, err := SegmentFileNames(l.path) - if err != nil { - return err - } - l.tracker.SetSegments(uint64(len(segments))) - - if len(segments) > 0 { - lastSegment := segments[len(segments)-1] - id, err := idFromFileName(lastSegment) - if err != nil { - return err - } - - l.currentSegmentID = id - stat, err := os.Stat(lastSegment) - if err != nil { - return err - } - - if stat.Size() == 0 { - os.Remove(lastSegment) - segments = segments[:len(segments)-1] - l.tracker.DecSegments() - } else { - fd, err := os.OpenFile(lastSegment, os.O_RDWR, 0666) - if err != nil { - return err - } - if _, err := fd.Seek(0, io.SeekEnd); err != nil { - return err - } - l.currentSegmentWriter = NewWALSegmentWriter(fd) - - // Reset the current segment size stat - l.tracker.SetCurrentSegmentSize(uint64(stat.Size())) - } - } - - var totalOldDiskSize int64 - for _, seg := range segments { - stat, err := os.Stat(seg) - if err != nil { - return err - } - - if stat.Size() > 0 { - totalOldDiskSize += stat.Size() - if stat.ModTime().After(l.lastWriteTime) { - l.lastWriteTime = stat.ModTime().UTC() - } - } - } - l.tracker.SetOldSegmentSize(uint64(totalOldDiskSize)) - - l.closing = make(chan struct{}) - - return nil -} - -// scheduleSync will schedule an fsync to the current wal segment and notify any -// waiting gorutines. If an fsync is already scheduled, subsequent calls will -// not schedule a new fsync and will be handle by the existing scheduled fsync. -func (l *WAL) scheduleSync() { - // If we're not the first to sync, then another goroutine is fsyncing the wal for us. - if !atomic.CompareAndSwapUint64(&l.syncCount, 0, 1) { - return - } - - // Fsync the wal and notify all pending waiters - go func() { - var timerCh <-chan time.Time - - // time.NewTicker requires a > 0 delay, since 0 indicates no delay, use a closed - // channel which will always be ready to read from. - if l.syncDelay == 0 { - // Create a RW chan and close it - timerChrw := make(chan time.Time) - close(timerChrw) - // Convert it to a read-only - timerCh = timerChrw - } else { - t := time.NewTicker(l.syncDelay) - defer t.Stop() - timerCh = t.C - } - for { - select { - case <-timerCh: - l.mu.Lock() - if len(l.syncWaiters) == 0 { - atomic.StoreUint64(&l.syncCount, 0) - l.mu.Unlock() - return - } - - l.sync() - l.mu.Unlock() - case <-l.closing: - atomic.StoreUint64(&l.syncCount, 0) - return - } - } - }() -} - -// sync fsyncs the current wal segments and notifies any waiters. Callers must ensure -// a write lock on the WAL is obtained before calling sync. -func (l *WAL) sync() { - err := l.currentSegmentWriter.sync() - for len(l.syncWaiters) > 0 { - errC := <-l.syncWaiters - errC <- err - } -} - -// WriteMulti writes the given values to the WAL. It returns the WAL segment ID to -// which the points were written. If an error is returned the segment ID should -// be ignored. If the WAL is disabled, -1 and nil is returned. -func (l *WAL) WriteMulti(ctx context.Context, values map[string][]value.Value) (int, error) { - span, _ := tracing.StartSpanFromContext(ctx) - defer span.Finish() - - if !l.enabled { - return -1, nil - } - - entry := &WriteWALEntry{ - Values: values, - } - - id, err := l.writeToLog(entry) - if err != nil { - l.tracker.IncWritesErr() - return -1, err - } - l.tracker.IncWritesOK() - - return id, nil -} - -// ClosedSegments returns a slice of the names of the closed segment files. -func (l *WAL) ClosedSegments() ([]string, error) { - if !l.enabled { - return nil, nil - } - - l.mu.RLock() - defer l.mu.RUnlock() - - // Not loading files from disk so nothing to do - if l.path == "" { - return nil, nil - } - - var currentFile string - if l.currentSegmentWriter != nil { - currentFile = l.currentSegmentWriter.path() - } - - files, err := SegmentFileNames(l.path) - if err != nil { - return nil, err - } - - var closedFiles []string - for _, fn := range files { - // Skip the current path - if fn == currentFile { - continue - } - - closedFiles = append(closedFiles, fn) - } - - return closedFiles, nil -} - -// Remove deletes the given segment file paths from disk and cleans up any associated objects. -func (l *WAL) Remove(ctx context.Context, files []string) error { - if !l.enabled { - return nil - } - - span, _ := tracing.StartSpanFromContext(ctx) - defer span.Finish() - - l.mu.Lock() - defer l.mu.Unlock() - - for i, fn := range files { - span.LogKV(fmt.Sprintf("path-%d", i), fn) - os.RemoveAll(fn) - } - - // Refresh the on-disk size stats - segments, err := SegmentFileNames(l.path) - if err != nil { - return err - } - l.tracker.SetSegments(uint64(len(segments))) - - var totalOldDiskSize int64 - for _, seg := range segments { - stat, err := os.Stat(seg) - if err != nil { - return err - } - - totalOldDiskSize += stat.Size() - } - l.tracker.SetOldSegmentSize(uint64(totalOldDiskSize)) - return nil -} - -// LastWriteTime is the last time anything was written to the WAL. -func (l *WAL) LastWriteTime() time.Time { - l.mu.RLock() - defer l.mu.RUnlock() - return l.lastWriteTime -} - -// DiskSizeBytes returns the on-disk size of the WAL. -func (l *WAL) DiskSizeBytes() int64 { - return int64(l.tracker.OldSegmentSize() + l.tracker.CurrentSegmentSize()) -} - -func (l *WAL) writeToLog(entry WALEntry) (int, error) { - // limit how many concurrent encodings can be in flight. Since we can only - // write one at a time to disk, a slow disk can cause the allocations below - // to increase quickly. If we're backed up, wait until others have completed. - bytes := bytesPool.Get(entry.MarshalSize()) - - b, err := entry.Encode(bytes) - if err != nil { - bytesPool.Put(bytes) - return -1, err - } - - encBuf := bytesPool.Get(snappy.MaxEncodedLen(len(b))) - - compressed := snappy.Encode(encBuf, b) - bytesPool.Put(bytes) - - syncErr := make(chan error) - - segID, err := func() (int, error) { - l.mu.Lock() - defer l.mu.Unlock() - - // Make sure the log has not been closed - select { - case <-l.closing: - return -1, ErrWALClosed - default: - } - - // roll the segment file if needed - if err := l.rollSegment(); err != nil { - return -1, fmt.Errorf("error rolling WAL segment: %v", err) - } - - // write and sync - if err := l.currentSegmentWriter.Write(entry.Type(), compressed); err != nil { - return -1, fmt.Errorf("error writing WAL entry: %v", err) - } - - select { - case l.syncWaiters <- syncErr: - default: - return -1, fmt.Errorf("error syncing wal") - } - l.scheduleSync() - - // Update stats for current segment size - l.tracker.SetCurrentSegmentSize(uint64(l.currentSegmentWriter.size)) - l.lastWriteTime = time.Now().UTC() - - return l.currentSegmentID, nil - - }() - - bytesPool.Put(encBuf) - - if err != nil { - return segID, err - } - - // schedule an fsync and wait for it to complete - return segID, <-syncErr -} - -// rollSegment checks if the current segment is due to roll over to a new segment; -// and if so, opens a new segment file for future writes. -func (l *WAL) rollSegment() error { - if l.currentSegmentWriter == nil || l.currentSegmentWriter.size > DefaultSegmentSize { - if err := l.newSegmentFile(); err != nil { - // A drop database or RP call could trigger this error if writes were in-flight - // when the drop statement executes. - return fmt.Errorf("error opening new segment file for wal (2): %v", err) - } - return nil - } - - return nil -} - -// CloseSegment closes the current segment if it is non-empty and opens a new one. -func (l *WAL) CloseSegment() error { - if !l.enabled { - return nil - } - - l.mu.Lock() - defer l.mu.Unlock() - - if l.currentSegmentWriter == nil || l.currentSegmentWriter.size > 0 { - if err := l.newSegmentFile(); err != nil { - // A drop database or RP call could trigger this error if writes were in-flight - // when the drop statement executes. - return fmt.Errorf("error opening new segment file for wal (1): %v", err) - } - return nil - } - return nil -} - -// DeleteBucketRange deletes the data inside of the bucket between the two times, returning -// the segment ID for the operation. -func (l *WAL) DeleteBucketRange(orgID, bucketID influxdb.ID, min, max int64, pred []byte) (int, error) { - if !l.enabled { - return -1, nil - } - - entry := &DeleteBucketRangeWALEntry{ - OrgID: orgID, - BucketID: bucketID, - Min: min, - Max: max, - Predicate: pred, - } - - id, err := l.writeToLog(entry) - if err != nil { - return -1, err - } - return id, nil -} - -// Close will finish any flush that is currently in progress and close file handles. -func (l *WAL) Close() error { - l.mu.Lock() - defer l.mu.Unlock() - - if !l.enabled { - return nil - } - - l.once.Do(func() { - span, _ := tracing.StartSpanFromContextWithOperationName(context.Background(), "WAL.Close once.Do") - defer span.Finish() - - span.LogKV("path", l.path) - - // Close, but don't set to nil so future goroutines can still be signaled - close(l.closing) - - if l.currentSegmentWriter != nil { - l.sync() - l.currentSegmentWriter.close() - l.currentSegmentWriter = nil - } - }) - - return nil -} - -// SegmentFileNames will return all files that are WAL segment files in sorted order by ascending ID. -func SegmentFileNames(dir string) ([]string, error) { - names, err := filepath.Glob(filepath.Join(dir, fmt.Sprintf("%s*.%s", WALFilePrefix, WALFileExtension))) - if err != nil { - return nil, err - } - sort.Strings(names) - return names, nil -} - -// newSegmentFile will close the current segment file and open a new one, updating bookkeeping info on the log. -func (l *WAL) newSegmentFile() error { - l.currentSegmentID++ - if l.currentSegmentWriter != nil { - l.sync() - - if err := l.currentSegmentWriter.close(); err != nil { - return err - } - l.tracker.SetOldSegmentSize(uint64(l.currentSegmentWriter.size)) - } - - fileName := filepath.Join(l.path, fmt.Sprintf("%s%05d.%s", WALFilePrefix, l.currentSegmentID, WALFileExtension)) - fd, err := os.OpenFile(fileName, os.O_CREATE|os.O_RDWR, 0666) - if err != nil { - return err - } - l.currentSegmentWriter = NewWALSegmentWriter(fd) - l.tracker.IncSegments() - - // Reset the current segment size stat - l.tracker.SetCurrentSegmentSize(0) - return nil -} - -// walTracker tracks writes to the WAL. -// -// As well as being responsible for providing atomic reads and writes to the -// statistics, walTracker also mirrors any changes to the external prometheus -// metrics, which the Engine exposes. -// -// *NOTE* - walTracker fields should not be directory modified. Doing so -// could result in the Engine exposing inaccurate metrics. -type walTracker struct { - metrics *walMetrics - labels prometheus.Labels - oldSegmentBytes uint64 -} - -func newWALTracker(metrics *walMetrics, defaultLabels prometheus.Labels) *walTracker { - return &walTracker{metrics: metrics, labels: defaultLabels} -} - -// Labels returns a copy of the default labels used by the tracker's metrics. -// The returned map is safe for modification. -func (t *walTracker) Labels() prometheus.Labels { - labels := make(prometheus.Labels, len(t.labels)) - for k, v := range t.labels { - labels[k] = v - } - return labels -} - -// IncWrites increments the number of writes to the cache, with a required status. -func (t *walTracker) IncWrites(status string) { - labels := t.Labels() - labels["status"] = status - t.metrics.Writes.With(labels).Inc() -} - -// IncWritesOK increments the number of successful writes. -func (t *walTracker) IncWritesOK() { t.IncWrites("ok") } - -// IncWritesError increments the number of writes that encountered an error. -func (t *walTracker) IncWritesErr() { t.IncWrites("error") } - -// SetOldSegmentSize sets the size of all old segments on disk. -func (t *walTracker) SetOldSegmentSize(sz uint64) { - atomic.StoreUint64(&t.oldSegmentBytes, sz) - - labels := t.labels - t.metrics.OldSegmentBytes.With(labels).Set(float64(sz)) -} - -// OldSegmentSize returns the on-disk size of all old segments. -func (t *walTracker) OldSegmentSize() uint64 { return atomic.LoadUint64(&t.oldSegmentBytes) } - -// SetCurrentSegmentSize sets the size of all old segments on disk. -func (t *walTracker) SetCurrentSegmentSize(sz uint64) { - atomic.StoreUint64(&t.oldSegmentBytes, sz) - - labels := t.labels - t.metrics.CurrentSegmentBytes.With(labels).Set(float64(sz)) -} - -// CurrentSegmentSize returns the on-disk size of all old segments. -func (t *walTracker) CurrentSegmentSize() uint64 { return atomic.LoadUint64(&t.oldSegmentBytes) } - -// SetSegments sets the number of segments files on disk. -func (t *walTracker) SetSegments(sz uint64) { - labels := t.labels - t.metrics.Segments.With(labels).Set(float64(sz)) -} - -// IncSegments increases the number of segments files by one. -func (t *walTracker) IncSegments() { - labels := t.labels - t.metrics.Segments.With(labels).Inc() -} - -// DecSegments decreases the number of segments files by one. -func (t *walTracker) DecSegments() { - labels := t.labels - t.metrics.Segments.With(labels).Dec() -} - -// WALEntry is record stored in each WAL segment. Each entry has a type -// and an opaque, type dependent byte slice data attribute. -type WALEntry interface { - Type() WalEntryType - Encode(dst []byte) ([]byte, error) - MarshalBinary() ([]byte, error) - UnmarshalBinary(b []byte) error - MarshalSize() int -} - -// WriteWALEntry represents a write of points. -type WriteWALEntry struct { - Values map[string][]value.Value - sz int -} - -// MarshalSize returns the number of bytes the entry takes when marshaled. -func (w *WriteWALEntry) MarshalSize() int { - if w.sz > 0 || len(w.Values) == 0 { - return w.sz - } - - encLen := 7 * len(w.Values) // Type (1), Key Length (2), and Count (4) for each key - - // determine required length - for k, v := range w.Values { - encLen += len(k) - if len(v) == 0 { - return 0 - } - - encLen += 8 * len(v) // timestamps (8) - - switch v[0].(type) { - case value.FloatValue, value.IntegerValue, value.UnsignedValue: - encLen += 8 * len(v) - case value.BooleanValue: - encLen += 1 * len(v) - case value.StringValue: - for _, vv := range v { - str, ok := vv.(value.StringValue) - if !ok { - return 0 - } - encLen += 4 + len(str.RawValue()) - } - default: - return 0 - } - } - - w.sz = encLen - - return w.sz -} - -// Encode converts the WriteWALEntry into a byte stream using dst if it -// is large enough. If dst is too small, the slice will be grown to fit the -// encoded entry. -func (w *WriteWALEntry) Encode(dst []byte) ([]byte, error) { - // The entries values are encode as follows: - // - // For each key and slice of values, first a 1 byte type for the []Values - // slice is written. Following the type, the length and key bytes are written. - // Following the key, a 4 byte count followed by each value as a 8 byte time - // and N byte value. The value is dependent on the type being encoded. float64, - // int64, use 8 bytes, boolean uses 1 byte, and string is similar to the key encoding, - // except that string values have a 4-byte length, and keys only use 2 bytes. - // - // This structure is then repeated for each key an value slices. - // - // ┌────────────────────────────────────────────────────────────────────┐ - // │ WriteWALEntry │ - // ├──────┬─────────┬────────┬───────┬─────────┬─────────┬───┬──────┬───┤ - // │ Type │ Key Len │ Key │ Count │ Time │ Value │...│ Type │...│ - // │1 byte│ 2 bytes │ N bytes│4 bytes│ 8 bytes │ N bytes │ │1 byte│ │ - // └──────┴─────────┴────────┴───────┴─────────┴─────────┴───┴──────┴───┘ - - encLen := w.MarshalSize() // Type (1), Key Length (2), and Count (4) for each key - - // allocate or re-slice to correct size - if len(dst) < encLen { - dst = make([]byte, encLen) - } else { - dst = dst[:encLen] - } - - // Finally, encode the entry - var n int - var curType byte - - for k, v := range w.Values { - switch v[0].(type) { - case value.FloatValue: - curType = float64EntryType - case value.IntegerValue: - curType = integerEntryType - case value.UnsignedValue: - curType = unsignedEntryType - case value.BooleanValue: - curType = booleanEntryType - case value.StringValue: - curType = stringEntryType - default: - return nil, fmt.Errorf("unsupported value type: %T", v[0]) - } - dst[n] = curType - n++ - - binary.BigEndian.PutUint16(dst[n:n+2], uint16(len(k))) - n += 2 - n += copy(dst[n:], k) - - binary.BigEndian.PutUint32(dst[n:n+4], uint32(len(v))) - n += 4 - - for _, vv := range v { - binary.BigEndian.PutUint64(dst[n:n+8], uint64(vv.UnixNano())) - n += 8 - - switch vv := vv.(type) { - case value.FloatValue: - if curType != float64EntryType { - return nil, fmt.Errorf("incorrect value found in %T slice: %T", v[0].Value(), vv) - } - binary.BigEndian.PutUint64(dst[n:n+8], math.Float64bits(vv.RawValue())) - n += 8 - case value.IntegerValue: - if curType != integerEntryType { - return nil, fmt.Errorf("incorrect value found in %T slice: %T", v[0].Value(), vv) - } - binary.BigEndian.PutUint64(dst[n:n+8], uint64(vv.RawValue())) - n += 8 - case value.UnsignedValue: - if curType != unsignedEntryType { - return nil, fmt.Errorf("incorrect value found in %T slice: %T", v[0].Value(), vv) - } - binary.BigEndian.PutUint64(dst[n:n+8], uint64(vv.RawValue())) - n += 8 - case value.BooleanValue: - if curType != booleanEntryType { - return nil, fmt.Errorf("incorrect value found in %T slice: %T", v[0].Value(), vv) - } - if vv.RawValue() { - dst[n] = 1 - } else { - dst[n] = 0 - } - n++ - case value.StringValue: - if curType != stringEntryType { - return nil, fmt.Errorf("incorrect value found in %T slice: %T", v[0].Value(), vv) - } - binary.BigEndian.PutUint32(dst[n:n+4], uint32(len(vv.RawValue()))) - n += 4 - n += copy(dst[n:], vv.RawValue()) - default: - return nil, fmt.Errorf("unsupported value found in %T slice: %T", v[0].Value(), vv) - } - } - } - - return dst[:n], nil -} - -// MarshalBinary returns a binary representation of the entry in a new byte slice. -func (w *WriteWALEntry) MarshalBinary() ([]byte, error) { - // Temp buffer to write marshaled points into - b := make([]byte, w.MarshalSize()) - return w.Encode(b) -} - -// UnmarshalBinary deserializes the byte slice into w. -func (w *WriteWALEntry) UnmarshalBinary(b []byte) error { - var i int - for i < len(b) { - typ := b[i] - i++ - - if i+2 > len(b) { - return ErrWALCorrupt - } - - length := int(binary.BigEndian.Uint16(b[i : i+2])) - i += 2 - - if i+length > len(b) { - return ErrWALCorrupt - } - - k := string(b[i : i+length]) - i += length - - if i+4 > len(b) { - return ErrWALCorrupt - } - - nvals := int(binary.BigEndian.Uint32(b[i : i+4])) - i += 4 - - if nvals <= 0 || nvals > len(b) { - return ErrWALCorrupt - } - - switch typ { - case float64EntryType: - if i+16*nvals > len(b) { - return ErrWALCorrupt - } - - values := make([]value.Value, 0, nvals) - for j := 0; j < nvals; j++ { - un := int64(binary.BigEndian.Uint64(b[i : i+8])) - i += 8 - v := math.Float64frombits((binary.BigEndian.Uint64(b[i : i+8]))) - i += 8 - values = append(values, value.NewFloatValue(un, v)) - } - w.Values[k] = values - case integerEntryType: - if i+16*nvals > len(b) { - return ErrWALCorrupt - } - - values := make([]value.Value, 0, nvals) - for j := 0; j < nvals; j++ { - un := int64(binary.BigEndian.Uint64(b[i : i+8])) - i += 8 - v := int64(binary.BigEndian.Uint64(b[i : i+8])) - i += 8 - values = append(values, value.NewIntegerValue(un, v)) - } - w.Values[k] = values - - case unsignedEntryType: - if i+16*nvals > len(b) { - return ErrWALCorrupt - } - - values := make([]value.Value, 0, nvals) - for j := 0; j < nvals; j++ { - un := int64(binary.BigEndian.Uint64(b[i : i+8])) - i += 8 - v := binary.BigEndian.Uint64(b[i : i+8]) - i += 8 - values = append(values, value.NewUnsignedValue(un, v)) - } - w.Values[k] = values - - case booleanEntryType: - if i+9*nvals > len(b) { - return ErrWALCorrupt - } - - values := make([]value.Value, 0, nvals) - for j := 0; j < nvals; j++ { - un := int64(binary.BigEndian.Uint64(b[i : i+8])) - i += 8 - - v := b[i] - i += 1 - if v == 1 { - values = append(values, value.NewBooleanValue(un, true)) - } else { - values = append(values, value.NewBooleanValue(un, false)) - } - } - w.Values[k] = values - - case stringEntryType: - values := make([]value.Value, 0, nvals) - for j := 0; j < nvals; j++ { - if i+12 > len(b) { - return ErrWALCorrupt - } - - un := int64(binary.BigEndian.Uint64(b[i : i+8])) - i += 8 - - length := int(binary.BigEndian.Uint32(b[i : i+4])) - if i+length > len(b) { - return ErrWALCorrupt - } - - i += 4 - - if i+length > len(b) { - return ErrWALCorrupt - } - - v := string(b[i : i+length]) - i += length - values = append(values, value.NewStringValue(un, v)) - } - w.Values[k] = values - - default: - return fmt.Errorf("unsupported value type: %#v", typ) - } - } - return nil -} - -// Type returns WriteWALEntryType. -func (w *WriteWALEntry) Type() WalEntryType { - return WriteWALEntryType -} - -// DeleteBucketRangeWALEntry represents the deletion of data in a bucket. -type DeleteBucketRangeWALEntry struct { - OrgID influxdb.ID - BucketID influxdb.ID - Min, Max int64 - Predicate []byte -} - -// MarshalBinary returns a binary representation of the entry in a new byte slice. -func (w *DeleteBucketRangeWALEntry) MarshalBinary() ([]byte, error) { - b := make([]byte, w.MarshalSize()) - return w.Encode(b) -} - -// UnmarshalBinary deserializes the byte slice into w. -func (w *DeleteBucketRangeWALEntry) UnmarshalBinary(b []byte) error { - if len(b) < 2*influxdb.IDLength+16 { - return ErrWALCorrupt - } - - if err := w.OrgID.Decode(b[0:influxdb.IDLength]); err != nil { - return err - } - if err := w.BucketID.Decode(b[influxdb.IDLength : 2*influxdb.IDLength]); err != nil { - return err - } - w.Min = int64(binary.BigEndian.Uint64(b[2*influxdb.IDLength : 2*influxdb.IDLength+8])) - w.Max = int64(binary.BigEndian.Uint64(b[2*influxdb.IDLength+8 : 2*influxdb.IDLength+16])) - w.Predicate = b[2*influxdb.IDLength+16:] - - // Maintain backwards compatability where no predicate bytes means nil - if len(w.Predicate) == 0 { - w.Predicate = nil - } - - return nil -} - -// MarshalSize returns the number of bytes the entry takes when marshaled. -func (w *DeleteBucketRangeWALEntry) MarshalSize() int { - return 2*influxdb.IDLength + 16 + len(w.Predicate) -} - -// Encode converts the entry into a byte stream using b if it is large enough. -// If b is too small, a newly allocated slice is returned. -func (w *DeleteBucketRangeWALEntry) Encode(b []byte) ([]byte, error) { - sz := w.MarshalSize() - if len(b) < sz { - b = make([]byte, sz) - } - - orgID, err := w.OrgID.Encode() - if err != nil { - return nil, err - } - bucketID, err := w.BucketID.Encode() - if err != nil { - return nil, err - } - - copy(b, orgID) - copy(b[influxdb.IDLength:], bucketID) - binary.BigEndian.PutUint64(b[2*influxdb.IDLength:], uint64(w.Min)) - binary.BigEndian.PutUint64(b[2*influxdb.IDLength+8:], uint64(w.Max)) - copy(b[2*influxdb.IDLength+16:], w.Predicate) - - return b[:sz], nil -} - -// Type returns DeleteBucketRangeWALEntryType. -func (w *DeleteBucketRangeWALEntry) Type() WalEntryType { - return DeleteBucketRangeWALEntryType -} - -// WALSegmentWriter writes WAL segments. -type WALSegmentWriter struct { - bw *bufio.Writer - w io.WriteCloser - size int -} - -// NewWALSegmentWriter returns a new WALSegmentWriter writing to w. -func NewWALSegmentWriter(w io.WriteCloser) *WALSegmentWriter { - return &WALSegmentWriter{ - bw: bufio.NewWriterSize(w, 16*1024), - w: w, - } -} - -func (w *WALSegmentWriter) path() string { - if f, ok := w.w.(*os.File); ok { - return f.Name() - } - return "" -} - -// Write writes entryType and the buffer containing compressed entry data. -func (w *WALSegmentWriter) Write(entryType WalEntryType, compressed []byte) error { - var buf [5]byte - buf[0] = byte(entryType) - binary.BigEndian.PutUint32(buf[1:5], uint32(len(compressed))) - - if _, err := w.bw.Write(buf[:]); err != nil { - return err - } - - if _, err := w.bw.Write(compressed); err != nil { - return err - } - - w.size += len(buf) + len(compressed) - - return nil -} - -// Sync flushes the file systems in-memory copy of recently written data to disk, -// if w is writing to an os.File. -func (w *WALSegmentWriter) sync() error { - if err := w.bw.Flush(); err != nil { - return err - } - - if f, ok := w.w.(*os.File); ok { - return f.Sync() - } - return nil -} - -func (w *WALSegmentWriter) Flush() error { - return w.bw.Flush() -} - -func (w *WALSegmentWriter) close() error { - if err := w.Flush(); err != nil { - return err - } - return w.w.Close() -} - -// WALSegmentReader reads WAL segments. -type WALSegmentReader struct { - rc io.ReadCloser - r *bufio.Reader - entry WALEntry - n int64 - err error -} - -// NewWALSegmentReader returns a new WALSegmentReader reading from r. -func NewWALSegmentReader(r io.ReadCloser) *WALSegmentReader { - return &WALSegmentReader{ - rc: r, - r: bufio.NewReader(r), - } -} - -func (r *WALSegmentReader) Reset(rc io.ReadCloser) { - r.rc = rc - r.r.Reset(rc) - r.entry = nil - r.n = 0 - r.err = nil -} - -// Next indicates if there is a value to read. -func (r *WALSegmentReader) Next() bool { - var nReadOK int - - // read the type and the length of the entry - var lv [5]byte - n, err := io.ReadFull(r.r, lv[:]) - if err == io.EOF { - return false - } - - if err != nil { - r.err = err - // We return true here because we want the client code to call read which - // will return the this error to be handled. - return true - } - nReadOK += n - - entryType := lv[0] - length := binary.BigEndian.Uint32(lv[1:5]) - - b := *(getBuf(int(length))) - defer putBuf(&b) - - // read the compressed block and decompress it - n, err = io.ReadFull(r.r, b[:length]) - if err != nil { - r.err = err - return true - } - nReadOK += n - - decLen, err := snappy.DecodedLen(b[:length]) - if err != nil { - r.err = err - return true - } - decBuf := *(getBuf(decLen)) - defer putBuf(&decBuf) - - data, err := snappy.Decode(decBuf, b[:length]) - if err != nil { - r.err = err - return true - } - - // and marshal it and send it to the cache - switch WalEntryType(entryType) { - case WriteWALEntryType: - r.entry = &WriteWALEntry{ - Values: make(map[string][]value.Value), - } - case DeleteBucketRangeWALEntryType: - r.entry = &DeleteBucketRangeWALEntry{} - default: - r.err = fmt.Errorf("unknown wal entry type: %v", entryType) - return true - } - r.err = r.entry.UnmarshalBinary(data) - if r.err == nil { - // Read and decode of this entry was successful. - r.n += int64(nReadOK) - } - - return true -} - -// Read returns the next entry in the reader. -func (r *WALSegmentReader) Read() (WALEntry, error) { - if r.err != nil { - return nil, r.err - } - return r.entry, nil -} - -// Count returns the total number of bytes read successfully from the segment, as -// of the last call to Read(). The segment is guaranteed to be valid up to and -// including this number of bytes. -func (r *WALSegmentReader) Count() int64 { - return r.n -} - -// Error returns the last error encountered by the reader. -func (r *WALSegmentReader) Error() error { - return r.err -} - -// Close closes the underlying io.Reader. -func (r *WALSegmentReader) Close() error { - if r.rc == nil { - return nil - } - err := r.rc.Close() - r.rc = nil - return err -} - -// idFromFileName parses the segment file ID from its name. -func idFromFileName(name string) (int, error) { - parts := strings.Split(filepath.Base(name), ".") - if len(parts) != 2 { - return 0, fmt.Errorf("file %s has wrong name format to have an id", name) - } - - id, err := strconv.ParseUint(parts[0][1:], 10, 32) - - return int(id), err -} diff --git a/storage/wal/wal_test.go b/storage/wal/wal_test.go deleted file mode 100644 index a16244176a..0000000000 --- a/storage/wal/wal_test.go +++ /dev/null @@ -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) -} diff --git a/task/backend/analytical_storage_test.go b/task/backend/analytical_storage_test.go index 3d500a1f05..4dd05b3835 100644 --- a/task/backend/analytical_storage_test.go +++ b/task/backend/analytical_storage_test.go @@ -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) } diff --git a/task/backend/run_recorder.go b/task/backend/run_recorder.go index f8f9587bec..c9cf7f48bc 100644 --- a/task/backend/run_recorder.go +++ b/task/backend/run_recorder.go @@ -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}) } diff --git a/v1/models/points_test.go b/v1/models/points_test.go index 16a4216802..7514ef2787 100644 --- a/v1/models/points_test.go +++ b/v1/models/points_test.go @@ -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() diff --git a/v1/pkg/data/gen/arrays.gen.go b/v1/pkg/data/gen/arrays.gen.go deleted file mode 100644 index 791b7ef41e..0000000000 --- a/v1/pkg/data/gen/arrays.gen.go +++ /dev/null @@ -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) -} diff --git a/v1/pkg/data/gen/arrays.gen.go.tmpl b/v1/pkg/data/gen/arrays.gen.go.tmpl deleted file mode 100644 index 2bf37dc601..0000000000 --- a/v1/pkg/data/gen/arrays.gen.go.tmpl +++ /dev/null @@ -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}} \ No newline at end of file diff --git a/v1/pkg/data/gen/gen.go b/v1/pkg/data/gen/gen.go deleted file mode 100644 index 7af8a05d80..0000000000 --- a/v1/pkg/data/gen/gen.go +++ /dev/null @@ -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 diff --git a/v1/pkg/data/gen/merged_series_generator.go b/v1/pkg/data/gen/merged_series_generator.go deleted file mode 100644 index b5c819359f..0000000000 --- a/v1/pkg/data/gen/merged_series_generator.go +++ /dev/null @@ -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 -} diff --git a/v1/pkg/data/gen/merged_series_generator_test.go b/v1/pkg/data/gen/merged_series_generator_test.go deleted file mode 100644 index 0d22907b4f..0000000000 --- a/v1/pkg/data/gen/merged_series_generator_test.go +++ /dev/null @@ -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)) - } - }) - } -} diff --git a/v1/pkg/data/gen/precision_string.go b/v1/pkg/data/gen/precision_string.go deleted file mode 100644 index 8c78f3fc97..0000000000 --- a/v1/pkg/data/gen/precision_string.go +++ /dev/null @@ -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]] -} diff --git a/v1/pkg/data/gen/schema.go b/v1/pkg/data/gen/schema.go deleted file mode 100644 index 6306876746..0000000000 --- a/v1/pkg/data/gen/schema.go +++ /dev/null @@ -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, 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, 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 -} diff --git a/v1/pkg/data/gen/sequence.go b/v1/pkg/data/gen/sequence.go deleted file mode 100644 index e600471bd9..0000000000 --- a/v1/pkg/data/gen/sequence.go +++ /dev/null @@ -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 } diff --git a/v1/pkg/data/gen/series.go b/v1/pkg/data/gen/series.go deleted file mode 100644 index 81d7703f3f..0000000000 --- a/v1/pkg/data/gen/series.go +++ /dev/null @@ -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) -} diff --git a/v1/pkg/data/gen/series_generator.go b/v1/pkg/data/gen/series_generator.go deleted file mode 100644 index f5648f3198..0000000000 --- a/v1/pkg/data/gen/series_generator.go +++ /dev/null @@ -1,173 +0,0 @@ -package gen - -import ( - "math" - "time" - - "github.com/influxdata/influxdb/v2/v1/models" -) - -type SeriesGenerator interface { - // Next advances the series generator to the next series key. - Next() bool - - // Key returns the series key. - // The returned value may be cached. - Key() []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. - Tags() models.Tags - - // Field returns the name of the field. - // The returned value may be modified by a subsequent call to Next. - Field() []byte - - // TimeValuesGenerator returns a values sequence for the current series. - TimeValuesGenerator() TimeValuesSequence -} - -type TimeSequenceSpec struct { - // Count specifies the maximum number of values to generate. - Count int - - // Start specifies the starting time for the values. - Start time.Time - - // Delta specifies the interval between time stamps. - Delta time.Duration - - // Precision specifies the precision of timestamp intervals - Precision time.Duration -} - -func (ts TimeSequenceSpec) ForTimeRange(tr TimeRange) TimeSequenceSpec { - // Truncate time range - if ts.Delta > 0 { - tr = tr.Truncate(ts.Delta) - } else { - tr = tr.Truncate(ts.Precision) - } - - ts.Start = tr.Start - - if ts.Delta > 0 { - intervals := int(tr.End.Sub(tr.Start) / ts.Delta) - if intervals > ts.Count { - // if the number of intervals in the specified time range exceeds - // the maximum count, move the start forward to limit the number of values - ts.Start = tr.End.Add(-time.Duration(ts.Count) * ts.Delta) - } else { - ts.Count = intervals - } - } else { - ts.Delta = tr.End.Sub(tr.Start) / time.Duration(ts.Count) - if ts.Delta < ts.Precision { - // count is too high for the range of time and precision - ts.Count = int(tr.End.Sub(tr.Start) / ts.Precision) - ts.Delta = ts.Precision - } else { - ts.Delta = ts.Delta.Round(ts.Precision) - } - ts.Precision = 0 - } - - return ts -} - -type TimeRange struct { - Start time.Time - End time.Time -} - -func (t TimeRange) Truncate(d time.Duration) TimeRange { - return TimeRange{ - Start: t.Start.Truncate(d), - End: t.End.Truncate(d), - } -} - -type TimeValuesSequence interface { - Reset() - Next() bool - Values() Values -} - -type Values interface { - MinTime() int64 - MaxTime() int64 - Encode([]byte) ([]byte, error) -} - -type cache struct { - key []byte - tags models.Tags -} - -type seriesGenerator struct { - name []byte - tags TagsSequence - field []byte - vg TimeValuesSequence - n int64 - - c cache -} - -func NewSeriesGenerator(name []byte, field []byte, vg TimeValuesSequence, tags TagsSequence) SeriesGenerator { - return NewSeriesGeneratorLimit(name, field, vg, tags, math.MaxInt64) -} - -func NewSeriesGeneratorLimit(name []byte, field []byte, vg TimeValuesSequence, tags TagsSequence, n int64) SeriesGenerator { - return &seriesGenerator{ - name: name, - field: field, - tags: tags, - vg: vg, - n: n, - } -} - -func (g *seriesGenerator) Next() bool { - if g.n > 0 { - g.n-- - if g.tags.Next() { - g.c = cache{} - g.vg.Reset() - return true - } - g.n = 0 - } - - return false -} - -func (g *seriesGenerator) Key() []byte { - if len(g.c.key) == 0 { - g.c.key = models.MakeKey(g.name, g.tags.Value()) - } - return g.c.key -} - -func (g *seriesGenerator) Name() []byte { - return g.name -} - -func (g *seriesGenerator) Tags() models.Tags { - if len(g.c.tags) == 0 { - g.c.tags = g.tags.Value().Clone() - } - return g.c.tags -} - -func (g *seriesGenerator) Field() []byte { - return g.field -} - -func (g *seriesGenerator) TimeValuesGenerator() TimeValuesSequence { - return g.vg -} diff --git a/v1/pkg/data/gen/series_test.go b/v1/pkg/data/gen/series_test.go deleted file mode 100644 index 0772cc75ca..0000000000 --- a/v1/pkg/data/gen/series_test.go +++ /dev/null @@ -1,74 +0,0 @@ -package gen - -import ( - "testing" - - "github.com/google/go-cmp/cmp" -) - -func TestCompareSeries(t *testing.T) { - mk := func(k, f string) seriesKeyField { - return &constSeries{key: []byte(k), field: []byte(f)} - } - - tests := []struct { - name string - a seriesKeyField - b seriesKeyField - exp int - }{ - { - name: "nil a,b", - exp: 0, - }, - { - name: "a(nil) < b", - a: nil, - b: mk("cpu,t0=v0", "f0"), - exp: -1, - }, - { - name: "a > b(nil)", - a: mk("cpu,t0=v0", "f0"), - b: nil, - exp: 1, - }, - { - name: "a = b", - a: mk("cpu,t0=v0", "f0"), - b: mk("cpu,t0=v0", "f0"), - exp: 0, - }, - { - name: "a(f0) < b(f1)", - a: mk("cpu,t0=v0", "f0"), - b: mk("cpu,t0=v0", "f1"), - exp: -1, - }, - { - name: "a(v0) < b(v1)", - a: mk("cpu,t0=v0", "f0"), - b: mk("cpu,t0=v1", "f0"), - exp: -1, - }, - { - name: "a(f1) > b(f0)", - a: mk("cpu,t0=v0", "f1"), - b: mk("cpu,t0=v0", "f0"), - exp: 1, - }, - { - name: "a(v1) > b(v0)", - a: mk("cpu,t0=v1", "f0"), - b: mk("cpu,t0=v0", "f0"), - exp: 1, - }, - } - for _, tt := range tests { - t.Run(tt.name, func(t *testing.T) { - if got := CompareSeries(tt.a, tt.b); got != tt.exp { - t.Errorf("unexpected value -got/+exp\n%s", cmp.Diff(got, tt.exp)) - } - }) - } -} diff --git a/v1/pkg/data/gen/specs.go b/v1/pkg/data/gen/specs.go deleted file mode 100644 index 458b1ef32d..0000000000 --- a/v1/pkg/data/gen/specs.go +++ /dev/null @@ -1,570 +0,0 @@ -package gen - -import ( - "bufio" - "fmt" - "math/rand" - "os" - "path" - "path/filepath" - "sort" - "unicode/utf8" - - "github.com/BurntSushi/toml" - "github.com/influxdata/influxdb/v2/v1/models" - "github.com/pkg/errors" -) - -type Spec struct { - SeriesLimit *int64 - Measurements []MeasurementSpec -} - -func NewSeriesGeneratorFromSpec(s *Spec, tr TimeRange) SeriesGenerator { - sg := make([]SeriesGenerator, len(s.Measurements)) - for i := range s.Measurements { - sg[i] = newSeriesGeneratorFromMeasurementSpec(&s.Measurements[i], tr) - } - if s.SeriesLimit == nil { - return NewMergedSeriesGenerator(sg) - } - return NewMergedSeriesGeneratorLimit(sg, *s.SeriesLimit) -} - -type MeasurementSpec struct { - Name string - SeriesLimit *SeriesLimit - TagsSpec *TagsSpec - FieldValuesSpec *FieldValuesSpec -} - -func newSeriesGeneratorFromMeasurementSpec(ms *MeasurementSpec, tr TimeRange) SeriesGenerator { - if ms.SeriesLimit == nil { - return NewSeriesGenerator( - []byte(ms.Name), - []byte(ms.FieldValuesSpec.Name), - newTimeValuesSequenceFromFieldValuesSpec(ms.FieldValuesSpec, tr), - newTagsSequenceFromTagsSpec(ms.TagsSpec)) - } - return NewSeriesGeneratorLimit( - []byte(ms.Name), - []byte(ms.FieldValuesSpec.Name), - newTimeValuesSequenceFromFieldValuesSpec(ms.FieldValuesSpec, tr), - newTagsSequenceFromTagsSpec(ms.TagsSpec), - int64(*ms.SeriesLimit)) -} - -// NewTimeValuesSequenceFn returns a TimeValuesSequence that will generate a -// sequence of values based on the spec. -type NewTimeValuesSequenceFn func(spec TimeSequenceSpec) TimeValuesSequence - -type NewTagsValuesSequenceFn func() TagsSequence - -type NewCountableSequenceFn func() CountableSequence - -type TagsSpec struct { - Tags []*TagValuesSpec - Sample *sample -} - -func newTagsSequenceFromTagsSpec(ts *TagsSpec) TagsSequence { - var keys []string - var vals []CountableSequence - for _, spec := range ts.Tags { - keys = append(keys, spec.TagKey) - vals = append(vals, spec.Values()) - } - - var opts []tagsValuesOption - if ts.Sample != nil && *ts.Sample != 1.0 { - opts = append(opts, TagValuesSampleOption(float64(*ts.Sample))) - } - - return NewTagsValuesSequenceKeysValues(keys, vals, opts...) -} - -type TagValuesSpec struct { - TagKey string - Values NewCountableSequenceFn -} - -type FieldValuesSpec struct { - TimeSequenceSpec - Name string - DataType models.FieldType - Values NewTimeValuesSequenceFn -} - -func newTimeValuesSequenceFromFieldValuesSpec(fs *FieldValuesSpec, tr TimeRange) TimeValuesSequence { - return fs.Values(fs.TimeSequenceSpec.ForTimeRange(tr)) -} - -func NewSpecFromToml(s string) (*Spec, error) { - var out Schema - if _, err := toml.Decode(s, &out); err != nil { - return nil, err - } - return NewSpecFromSchema(&out) -} - -func NewSpecFromPath(p string) (*Spec, error) { - var err error - p, err = filepath.Abs(p) - if err != nil { - return nil, err - } - - var out Schema - if _, err := toml.DecodeFile(p, &out); err != nil { - return nil, err - } - return newSpecFromSchema(&out, schemaDir(path.Dir(p))) -} - -func NewSchemaFromPath(path string) (*Schema, error) { - var out Schema - if _, err := toml.DecodeFile(path, &out); err != nil { - return nil, err - } - return &out, nil -} - -type schemaToSpecState int - -const ( - stateOk schemaToSpecState = iota - stateErr -) - -type schemaToSpec struct { - schemaDir string - stack []interface{} - state schemaToSpecState - spec *Spec - err error -} - -func (s *schemaToSpec) push(v interface{}) { - s.stack = append(s.stack, v) -} - -func (s *schemaToSpec) pop() interface{} { - tail := len(s.stack) - 1 - v := s.stack[tail] - s.stack[tail] = nil - s.stack = s.stack[:tail] - return v -} - -func (s *schemaToSpec) peek() interface{} { - if len(s.stack) == 0 { - return nil - } - return s.stack[len(s.stack)-1] -} - -func (s *schemaToSpec) Visit(node SchemaNode) (w Visitor) { - switch s.state { - case stateOk: - if s.visit(node) { - return s - } - s.state = stateErr - - case stateErr: - s.visitErr(node) - } - - return nil -} - -func (s *schemaToSpec) visit(node SchemaNode) bool { - switch n := node.(type) { - case *Schema: - s.spec.Measurements = s.pop().([]MeasurementSpec) - if n.SeriesLimit != nil { - sl := int64(*n.SeriesLimit) - s.spec.SeriesLimit = &sl - } - - case Measurements: - // flatten measurements - var mss []MeasurementSpec - for { - if specs, ok := s.peek().([]MeasurementSpec); ok { - s.pop() - mss = append(mss, specs...) - continue - } - break - } - sort.Slice(mss, func(i, j int) bool { - return mss[i].Name < mss[j].Name - }) - - // validate field types are homogeneous for a single measurement - mg := make(map[string]models.FieldType) - for i := range mss { - spec := &mss[i] - key := spec.Name + "." + spec.FieldValuesSpec.Name - ft := spec.FieldValuesSpec.DataType - if dt, ok := mg[key]; !ok { - mg[key] = ft - } else if dt != ft { - s.err = fmt.Errorf("field %q data-type conflict, found %s and %s", - key, - dt, - ft) - return false - } - } - - s.push(mss) - - case *Measurement: - if len(n.Name) == 0 { - s.err = errors.New("missing measurement name") - return false - } - - fields := s.pop().([]*FieldValuesSpec) - tagsSpec := s.pop().(*TagsSpec) - - tagsSpec.Sample = n.Sample - - // default: sample 50% - if n.Sample == nil { - s := sample(0.5) - tagsSpec.Sample = &s - } - - if *tagsSpec.Sample <= 0.0 || *tagsSpec.Sample > 1.0 { - s.err = errors.New("invalid sample, must be 0 < sample ≤ 1.0") - return false - } - - var ms []MeasurementSpec - for _, spec := range fields { - ms = append(ms, MeasurementSpec{ - Name: n.Name, - SeriesLimit: n.SeriesLimit, - TagsSpec: tagsSpec, - FieldValuesSpec: spec, - }) - } - - // NOTE: sort each measurement name + field name to ensure series are produced - // in correct order - sort.Slice(ms, func(i, j int) bool { - return ms[i].FieldValuesSpec.Name < ms[j].FieldValuesSpec.Name - }) - s.push(ms) - - case Tags: - var ts TagsSpec - for { - if spec, ok := s.peek().(*TagValuesSpec); ok { - s.pop() - ts.Tags = append(ts.Tags, spec) - continue - } - break - } - // Tag keys must be sorted to produce a valid series key sequence - sort.Slice(ts.Tags, func(i, j int) bool { - return ts.Tags[i].TagKey < ts.Tags[j].TagKey - }) - - for i := 1; i < len(ts.Tags); i++ { - if ts.Tags[i-1].TagKey == ts.Tags[i].TagKey { - s.err = fmt.Errorf("duplicate tag keys %q", ts.Tags[i].TagKey) - return false - } - } - - s.push(&ts) - - case Fields: - // combine fields - var fs []*FieldValuesSpec - for { - if spec, ok := s.peek().(*FieldValuesSpec); ok { - s.pop() - fs = append(fs, spec) - continue - } - break - } - - sort.Slice(fs, func(i, j int) bool { - return fs[i].Name < fs[j].Name - }) - - for i := 1; i < len(fs); i++ { - if fs[i-1].Name == fs[i].Name { - s.err = fmt.Errorf("duplicate field names %q", fs[i].Name) - return false - } - } - - s.push(fs) - - case *Field: - fs, ok := s.peek().(*FieldValuesSpec) - if !ok { - panic(fmt.Sprintf("unexpected type %T", fs)) - } - - fs.TimeSequenceSpec = n.TimeSequenceSpec() - fs.Name = n.Name - - case *FieldConstantValue: - var fs FieldValuesSpec - switch v := n.Value.(type) { - case float64: - fs.DataType = models.Float - fs.Values = func(spec TimeSequenceSpec) TimeValuesSequence { - return NewTimeFloatValuesSequence( - spec.Count, - NewTimestampSequenceFromSpec(spec), - NewFloatConstantValuesSequence(v), - ) - } - case int64: - fs.DataType = models.Integer - fs.Values = func(spec TimeSequenceSpec) TimeValuesSequence { - return NewTimeIntegerValuesSequence( - spec.Count, - NewTimestampSequenceFromSpec(spec), - NewIntegerConstantValuesSequence(v), - ) - } - case string: - fs.DataType = models.String - fs.Values = func(spec TimeSequenceSpec) TimeValuesSequence { - return NewTimeStringValuesSequence( - spec.Count, - NewTimestampSequenceFromSpec(spec), - NewStringConstantValuesSequence(v), - ) - } - case bool: - fs.DataType = models.Boolean - fs.Values = func(spec TimeSequenceSpec) TimeValuesSequence { - return NewTimeBooleanValuesSequence( - spec.Count, - NewTimestampSequenceFromSpec(spec), - NewBooleanConstantValuesSequence(v), - ) - } - default: - panic(fmt.Sprintf("unexpected type %T", v)) - } - - s.push(&fs) - - case *FieldArraySource: - var fs FieldValuesSpec - switch v := n.Value.(type) { - case []float64: - fs.DataType = models.Float - fs.Values = func(spec TimeSequenceSpec) TimeValuesSequence { - return NewTimeFloatValuesSequence( - spec.Count, - NewTimestampSequenceFromSpec(spec), - NewFloatArrayValuesSequence(v), - ) - } - case []int64: - fs.DataType = models.Integer - fs.Values = func(spec TimeSequenceSpec) TimeValuesSequence { - return NewTimeIntegerValuesSequence( - spec.Count, - NewTimestampSequenceFromSpec(spec), - NewIntegerArrayValuesSequence(v), - ) - } - case []string: - fs.DataType = models.String - fs.Values = func(spec TimeSequenceSpec) TimeValuesSequence { - return NewTimeStringValuesSequence( - spec.Count, - NewTimestampSequenceFromSpec(spec), - NewStringArrayValuesSequence(v), - ) - } - case []bool: - fs.DataType = models.Boolean - fs.Values = func(spec TimeSequenceSpec) TimeValuesSequence { - return NewTimeBooleanValuesSequence( - spec.Count, - NewTimestampSequenceFromSpec(spec), - NewBooleanArrayValuesSequence(v), - ) - } - default: - panic(fmt.Sprintf("unexpected type %T", v)) - } - - s.push(&fs) - - case *FieldFloatRandomSource: - var fs FieldValuesSpec - fs.DataType = models.Float - fs.Values = NewTimeValuesSequenceFn(func(spec TimeSequenceSpec) TimeValuesSequence { - return NewTimeFloatValuesSequence( - spec.Count, - NewTimestampSequenceFromSpec(spec), - NewFloatRandomValuesSequence(n.Min, n.Max, rand.New(rand.NewSource(n.Seed))), - ) - }) - s.push(&fs) - - case *FieldIntegerZipfSource: - var fs FieldValuesSpec - fs.DataType = models.Integer - fs.Values = NewTimeValuesSequenceFn(func(spec TimeSequenceSpec) TimeValuesSequence { - return NewTimeIntegerValuesSequence( - spec.Count, - NewTimestampSequenceFromSpec(spec), - NewIntegerZipfValuesSequence(n), - ) - }) - s.push(&fs) - - case *Tag: - s.push(&TagValuesSpec{ - TagKey: n.Name, - Values: s.pop().(NewCountableSequenceFn), - }) - - case *TagSequenceSource: - s.push(NewCountableSequenceFn(func() CountableSequence { - return NewCounterByteSequence(n.Format, int(n.Start), int(n.Start+n.Count)) - })) - - case *TagFileSource: - p, err := s.resolvePath(n.Path) - if err != nil { - s.err = err - return false - } - - lines, err := s.readLines(p) - if err != nil { - s.err = err - return false - } - - s.push(NewCountableSequenceFn(func() CountableSequence { - return NewStringArraySequence(lines) - })) - - case *TagArraySource: - s.push(NewCountableSequenceFn(func() CountableSequence { - return NewStringArraySequence(n.Values) - })) - - case nil: - - default: - panic(fmt.Sprintf("unexpected type %T", node)) - } - - return true -} - -func (s *schemaToSpec) visitErr(node SchemaNode) { - switch n := node.(type) { - case *Schema: - s.err = fmt.Errorf("error processing schema: %v", s.err) - case *Measurement: - s.err = fmt.Errorf("measurement %q: %v", n.Name, s.err) - case *Tag: - s.err = fmt.Errorf("tag %q: %v", n.Name, s.err) - case *Field: - s.err = fmt.Errorf("field %q: %v", n.Name, s.err) - } -} - -func (s *schemaToSpec) resolvePath(p string) (string, error) { - fullPath := os.ExpandEnv(p) - if !filepath.IsAbs(fullPath) { - fullPath = filepath.Join(s.schemaDir, fullPath) - } - - fi, err := os.Stat(fullPath) - if err != nil { - return "", fmt.Errorf("error resolving path %q: %v", p, err) - } - - if fi.IsDir() { - return "", fmt.Errorf("path %q is not a file: resolved to %s", p, fullPath) - } - - return fullPath, nil -} - -func (s *schemaToSpec) readLines(p string) ([]string, error) { - fp, err := s.resolvePath(p) - if err != nil { - return nil, err - } - - f, err := os.Open(fp) - if err != nil { - return nil, fmt.Errorf("path error: %v", err) - } - defer f.Close() - scan := bufio.NewScanner(f) - scan.Split(bufio.ScanLines) - - n := 0 - var lines []string - - for scan.Scan() { - if len(scan.Bytes()) == 0 { - // skip empty lines - continue - } - - if !utf8.Valid(scan.Bytes()) { - return nil, fmt.Errorf("path %q, invalid UTF-8 on line %d", p, n) - } - lines = append(lines, scan.Text()) - } - - if scan.Err() != nil { - return nil, scan.Err() - } - - return lines, nil -} - -type option func(s *schemaToSpec) - -func schemaDir(p string) option { - return func(s *schemaToSpec) { - s.schemaDir = p - } -} - -func NewSpecFromSchema(root *Schema) (*Spec, error) { - return newSpecFromSchema(root) -} - -func newSpecFromSchema(root *Schema, opts ...option) (*Spec, error) { - var spec Spec - - vis := &schemaToSpec{spec: &spec} - for _, o := range opts { - o(vis) - } - - WalkUp(vis, root) - if vis.err != nil { - return nil, vis.err - } - - return &spec, nil -} diff --git a/v1/pkg/data/gen/specs_test.go b/v1/pkg/data/gen/specs_test.go deleted file mode 100644 index 8ba9c8627f..0000000000 --- a/v1/pkg/data/gen/specs_test.go +++ /dev/null @@ -1,220 +0,0 @@ -package gen - -import ( - "testing" - "time" - - "github.com/BurntSushi/toml" - "github.com/google/go-cmp/cmp" - "github.com/influxdata/influxdb/v2/v1/models" -) - -func countableSequenceFnCmp(a, b NewCountableSequenceFn) bool { - // these aren't comparable - return true -} - -func timeValuesSequenceFnCmp(a, b NewTimeValuesSequenceFn) bool { - // these aren't comparable - return true -} - -func TestSpecFromSchema(t *testing.T) { - in := ` -title = "example schema" - -[[measurements]] -name = "m0" -tags = [ - { name = "tag0", source = [ "host1", "host2" ] }, - { name = "tag1", source = [ "process1", "process2" ] }, - { name = "tag2", source = { type = "sequence", format = "value%s", start = 0, count = 100 } } -] -fields = [ - { name = "f0", count = 5000, source = 0.5 }, - { name = "f1", count = 5000, source = 2 }, -] -[[measurements]] -name = "m1" - -tags = [ - { name = "tag0", source = [ "host1", "host2" ] }, -] -fields = [ - { name = "f0", count = 5000, source = 0.5 }, -] -` - var out Schema - if _, err := toml.Decode(in, &out); err != nil { - t.Fatalf("unxpected error: %v", err) - } - - got, err := NewSpecFromSchema(&out) - if err != nil { - t.Error(err) - } - - samples := []sample{0.5} - exp := &Spec{ - SeriesLimit: nil, - Measurements: []MeasurementSpec{ - { - Name: "m0", - SeriesLimit: nil, - TagsSpec: &TagsSpec{ - Tags: []*TagValuesSpec{ - {TagKey: "tag0"}, - {TagKey: "tag1"}, - {TagKey: "tag2"}, - }, - Sample: &samples[0], - }, - FieldValuesSpec: &FieldValuesSpec{ - TimeSequenceSpec: TimeSequenceSpec{ - Count: 5000, - Precision: time.Millisecond, - }, - Name: "f0", - DataType: models.Float, - }, - }, - { - Name: "m0", - SeriesLimit: nil, - TagsSpec: &TagsSpec{ - Tags: []*TagValuesSpec{ - {TagKey: "tag0"}, - {TagKey: "tag1"}, - {TagKey: "tag2"}, - }, - Sample: &samples[0], - }, - FieldValuesSpec: &FieldValuesSpec{ - TimeSequenceSpec: TimeSequenceSpec{ - Count: 5000, - Precision: time.Millisecond, - }, - Name: "f1", - DataType: models.Integer, - }, - }, - { - Name: "m1", - SeriesLimit: nil, - TagsSpec: &TagsSpec{ - Tags: []*TagValuesSpec{ - {TagKey: "tag0"}, - }, - Sample: &samples[0], - }, - FieldValuesSpec: &FieldValuesSpec{ - TimeSequenceSpec: TimeSequenceSpec{ - Count: 5000, - Precision: time.Millisecond, - }, - Name: "f0", - DataType: models.Float, - }, - }, - }, - } - - // TODO(sgc): use a Spec rather than closures for NewCountableSequenceFn and NewTimeValuesSequenceFn - if !cmp.Equal(got, exp, cmp.Comparer(countableSequenceFnCmp), cmp.Comparer(timeValuesSequenceFnCmp)) { - t.Errorf("unexpected spec; -got/+exp\n%s", cmp.Diff(got, exp, cmp.Comparer(countableSequenceFnCmp), cmp.Comparer(timeValuesSequenceFnCmp))) - } -} - -func TestSpecFromSchemaError(t *testing.T) { - tests := []struct { - name string - in string - - decodeErr string - specErr string - }{ - { - in: ` -[[measurements]] -tags = [ { name = "tag0", source = [ "host1", "host2" ] } ] -fields = [ { name = "f0", count = 5000, source = 0.5 } ] -`, - specErr: "error processing schema: missing measurement name", - }, - { - in: ` -[[measurements]] -sample = -0.1 -tags = [ { name = "tag0", source = [ "host1", "host2" ] } ] -fields = [ { name = "f0", count = 5000, source = 0.5 } ] -`, - decodeErr: "sample: must be 0 < sample ≤ 1.0", - }, - { - in: ` -[[measurements]] -name = "m0" -tags = [ { source = [ "host1", "host2" ] } ] -fields = [ { name = "f0", count = 5000, source = 0.5 } ] -`, - decodeErr: "tag: missing or invalid value for name", - }, - { - in: ` -[[measurements]] -name = "m0" -tags = [ { name = "tag0" } ] -fields = [ { name = "f0", count = 5000, source = 0.5 } ] -`, - decodeErr: `missing source for tag "tag0"`, - }, - { - in: ` -[[measurements]] -name = "m0" -tags = [ { name = "tag0", source = [ "host1", "host2" ] } ] -fields = [ { count = 5000, source = 0.5 } ] -`, - decodeErr: `field: missing or invalid value for name`, - }, - { - in: ` -[[measurements]] -name = "m0" -tags = [ { name = "tag0", source = [ "host1", "host2" ] } ] -fields = [ { name = "f0", count = 5000 } ] -`, - decodeErr: `missing source for field "f0"`, - }, - } - - checkErr := func(t *testing.T, err error, exp string) { - t.Helper() - if exp == "" { - if err == nil { - return - } - - t.Errorf("unexpected error, got %v", err) - } - - if err == nil { - t.Errorf("expected error, got nil") - } else if err.Error() != exp { - t.Errorf("unexpected error, -got/+exp\n%s", cmp.Diff(err.Error(), exp)) - } - } - - for _, test := range tests { - t.Run(test.name, func(t *testing.T) { - var out Schema - _, err := toml.Decode(test.in, &out) - checkErr(t, err, test.decodeErr) - - if test.decodeErr == "" { - _, err = NewSpecFromSchema(&out) - checkErr(t, err, test.specErr) - } - }) - } -} diff --git a/v1/pkg/data/gen/tags_sequence.go b/v1/pkg/data/gen/tags_sequence.go deleted file mode 100644 index 3860c0db42..0000000000 --- a/v1/pkg/data/gen/tags_sequence.go +++ /dev/null @@ -1,175 +0,0 @@ -package gen - -import ( - "fmt" - "math" - "math/rand" - "sort" - - "github.com/influxdata/influxdb/v2/v1/models" -) - -type TagsSequence interface { - Next() bool - Value() models.Tags - Count() int -} - -type tagsValuesSequence struct { - tags models.Tags - vals []CountableSequence - n int - count int - sample float64 - src rand.Source - nextFn func(*tagsValuesSequence) bool -} - -type tagsValuesOption func(s *tagsValuesSequence) - -func TagValuesLimitOption(n int) tagsValuesOption { - return func(s *tagsValuesSequence) { - if n >= s.count { - return - } - - s.src = rand.NewSource(20040409) - s.sample = float64(n) / float64(s.count) - } -} - -func TagValuesSampleOption(n float64) tagsValuesOption { - return func(s *tagsValuesSequence) { - if n <= 0.0 || n > 1.0 { - panic("expect: 0.0 < n ≤ 1.0") - } - - s.src = rand.NewSource(int64(float64(math.MaxInt64>>1) * n)) - s.sample = n - s.nextFn = (*tagsValuesSequence).nextSample - } -} - -func NewTagsValuesSequenceKeysValues(keys []string, vals []CountableSequence, opts ...tagsValuesOption) TagsSequence { - tm := make(map[string]string, len(keys)) - for _, k := range keys { - tm[k] = "" - } - - count := 1 - for i := range vals { - count *= vals[i].Count() - } - - // models.Tags are ordered, so ensure vals are ordered with respect to keys - sort.Sort(keyValues{keys, vals}) - - s := &tagsValuesSequence{ - tags: models.NewTags(tm), - vals: vals, - count: count, - nextFn: (*tagsValuesSequence).next, - } - - for _, o := range opts { - o(s) - } - - return s -} - -func NewTagsValuesSequenceValues(prefix string, vals []CountableSequence) TagsSequence { - keys := make([]string, len(vals)) - // max tag width - tw := int(math.Ceil(math.Log10(float64(len(vals))))) - tf := fmt.Sprintf("%s%%0%dd", prefix, tw) - for i := range vals { - keys[i] = fmt.Sprintf(tf, i) - } - return NewTagsValuesSequenceKeysValues(keys, vals) -} - -func NewTagsValuesSequenceCounts(prefix string, counts []int) TagsSequence { - tv := make([]CountableSequence, len(counts)) - for i := range counts { - tv[i] = NewCounterByteSequenceCount(counts[i]) - } - return NewTagsValuesSequenceValues(prefix, tv) -} - -func (s *tagsValuesSequence) next() bool { - if s.n >= s.count { - return false - } - - for i := range s.vals { - s.tags[i].Value = []byte(s.vals[i].Value()) - } - - s.n++ - i := s.n - for j := len(s.vals) - 1; j >= 0; j-- { - v := s.vals[j] - v.Next() - c := v.Count() - if r := i % c; r != 0 { - break - } - i /= c - } - - return true -} - -func (s *tagsValuesSequence) skip() bool { - return (float64(s.src.Int63()>>10))*(1.0/9007199254740992.0) > s.sample -} - -func (s *tagsValuesSequence) nextSample() bool { - if s.n >= s.count { - return false - } - - for i := range s.vals { - s.tags[i].Value = []byte(s.vals[i].Value()) - } - - for { - s.n++ - i := s.n - for j := len(s.vals) - 1; j >= 0; j-- { - v := s.vals[j] - v.Next() - c := v.Count() - if r := i % c; r != 0 { - break - } - i /= c - } - - if !s.skip() { - break - } - } - - return true -} - -func (s *tagsValuesSequence) Next() bool { - return s.nextFn(s) -} - -func (s *tagsValuesSequence) Value() models.Tags { return s.tags } -func (s *tagsValuesSequence) Count() int { return s.count } - -type keyValues struct { - keys []string - vals []CountableSequence -} - -func (k keyValues) Len() int { return len(k.keys) } -func (k keyValues) Less(i, j int) bool { return k.keys[i] < k.keys[j] } -func (k keyValues) Swap(i, j int) { - k.keys[i], k.keys[j] = k.keys[j], k.keys[i] - k.vals[i], k.vals[j] = k.vals[j], k.vals[i] -} diff --git a/v1/pkg/data/gen/timestamp_sequence.go b/v1/pkg/data/gen/timestamp_sequence.go deleted file mode 100644 index e2b7aee66d..0000000000 --- a/v1/pkg/data/gen/timestamp_sequence.go +++ /dev/null @@ -1,36 +0,0 @@ -package gen - -type TimestampSequence interface { - Reset() - Write(ts []int64) -} - -type timestampSequence struct { - t int64 - start int64 - delta int64 -} - -func NewTimestampSequenceFromSpec(spec TimeSequenceSpec) TimestampSequence { - return ×tampSequence{ - t: spec.Start.UnixNano(), - start: spec.Start.UnixNano(), - delta: int64(spec.Delta), - } -} - -func (g *timestampSequence) Reset() { - g.t = g.start -} - -func (g *timestampSequence) Write(ts []int64) { - var ( - t = g.t - d = g.delta - ) - for i := 0; i < len(ts); i++ { - ts[i] = t - t += d - } - g.t = t -} diff --git a/v1/pkg/data/gen/toml.go b/v1/pkg/data/gen/toml.go deleted file mode 100644 index 21137734d1..0000000000 --- a/v1/pkg/data/gen/toml.go +++ /dev/null @@ -1,460 +0,0 @@ -package gen - -import ( - "errors" - "fmt" - "strings" - "time" - - "github.com/spf13/cast" -) - -type SeriesLimit int64 - -func (s *SeriesLimit) UnmarshalTOML(data interface{}) error { - v, ok := data.(int64) - if !ok { - return errors.New("series-limit: invalid value") - } - - if v < 0 { - return errors.New("series-limit: must be ≥ 0") - } - - *s = SeriesLimit(v) - return nil -} - -type sample float64 - -func (s *sample) UnmarshalTOML(data interface{}) error { - v, ok := data.(float64) - if !ok { - return errors.New("sample: must be a float") - } - - if v <= 0 || v > 1.0 { - return errors.New("sample: must be 0 < sample ≤ 1.0") - } - - *s = sample(v) - - return nil -} - -type duration struct { - time.Duration -} - -func (d *duration) UnmarshalTOML(data interface{}) error { - text, ok := data.(string) - if !ok { - return fmt.Errorf("invalid duration, expect a Go duration as a string: %T", data) - } - - return d.UnmarshalText([]byte(text)) -} - -func (d *duration) UnmarshalText(text []byte) error { - s := string(text) - - var err error - d.Duration, err = time.ParseDuration(s) - if err != nil { - return err - } - - if d.Duration == 0 { - d.Duration, err = time.ParseDuration("1" + s) - if err != nil { - return err - } - } - - if d.Duration <= 0 { - return fmt.Errorf("invalid duration, must be > 0: %s", d.Duration) - } - return nil -} - -type precision byte - -const ( - precisionMillisecond precision = iota // default - precisionNanosecond - precisionMicrosecond - precisionSecond - precisionMinute - precisionHour -) - -var precisionToDuration = [...]time.Duration{ - time.Millisecond, - time.Nanosecond, - time.Microsecond, - time.Second, - time.Minute, - time.Minute * 60, - time.Nanosecond, - time.Nanosecond, -} - -func (p *precision) ToDuration() time.Duration { - return precisionToDuration[*p&0x7] -} - -func (p *precision) UnmarshalTOML(data interface{}) error { - d, ok := data.(string) - if !ok { - return fmt.Errorf("invalid precision, expect one of (ns, us, ms, s, m, h): %T", data) - } - - d = strings.ToLower(d) - - switch d { - case "ns", "nanosecond": - *p = precisionNanosecond - case "us", "microsecond", "µs": - *p = precisionMicrosecond - case "ms", "millisecond": - *p = precisionMillisecond - case "s", "second": - *p = precisionSecond - case "m", "minute": - *p = precisionMinute - case "h", "hour": - *p = precisionHour - default: - return fmt.Errorf("invalid precision, expect one of (ns, ms, s, m, h): %s", d) - } - return nil -} - -func (t *Tag) UnmarshalTOML(data interface{}) error { - d, ok := data.(map[string]interface{}) - if !ok { - return nil - } - - if n, ok := d["name"].(string); !ok || n == "" { - return errors.New("tag: missing or invalid value for name") - } else { - t.Name = n - } - - // infer source - - if _, ok := d["source"]; !ok { - return fmt.Errorf("missing source for tag %q", t.Name) - } - - switch v := d["source"].(type) { - case int64, string, float64, bool: - if src, err := decodeTagConstantSource(v); err != nil { - return err - } else { - t.Source = src - } - case []interface{}: - if src, err := decodeTagArraySource(v); err != nil { - return err - } else { - t.Source = src - } - case map[string]interface{}: - if src, err := decodeTagSource(v); err != nil { - return err - } else { - t.Source = src - } - default: - return fmt.Errorf("invalid source for tag %q: %T", t.Name, v) - } - - return nil -} - -func decodeTagConstantSource(data interface{}) (TagSource, error) { - switch data.(type) { - case int64, string, float64, bool: - if src, err := cast.ToStringE(data); err != nil { - return nil, err - } else { - return &TagArraySource{Values: []string{src}}, nil - } - } - - return nil, errors.New("invalid constant tag source") -} - -func decodeTagArraySource(data []interface{}) (TagSource, error) { - if len(data) == 0 { - return nil, errors.New("empty array source") - } - - if src, err := cast.ToStringSliceE(data); err != nil { - return nil, err - } else { - return &TagArraySource{Values: src}, nil - } -} - -func decodeTagSource(data map[string]interface{}) (TagSource, error) { - typ, ok := data["type"].(string) - if !ok { - return nil, errors.New("missing type field") - } - switch typ { - case "sequence": - return decodeTagSequenceSource(data) - case "file": - return decodeTagFileSource(data) - default: - return nil, fmt.Errorf("invalid type field %q", typ) - } -} - -func decodeTagFileSource(data map[string]interface{}) (TagSource, error) { - var s TagFileSource - - if v, ok := data["path"].(string); ok { - s.Path = v - } else { - return nil, errors.New("file: missing path") - } - - return &s, nil -} - -func decodeTagSequenceSource(data map[string]interface{}) (TagSource, error) { - var s TagSequenceSource - - if v, ok := data["format"].(string); ok { - // TODO(sgc): validate format string - s.Format = v - } else { - s.Format = "value%s" - } - - if v, ok := data["start"]; ok { - if v, err := cast.ToInt64E(v); err != nil { - return nil, fmt.Errorf("tag.sequence: invalid start, %v", err) - } else if v < 0 { - return nil, fmt.Errorf("tag.sequence: start must be ≥ 0") - } else { - s.Start = v - } - } - - if v, ok := data["count"]; ok { - if v, err := cast.ToInt64E(v); err != nil { - return nil, fmt.Errorf("tag.sequence: invalid count, %v", err) - } else if v < 0 { - return nil, fmt.Errorf("tag.sequence: count must be > 0") - } else { - s.Count = v - } - } else { - return nil, fmt.Errorf("tag.sequence: missing count") - } - - return &s, nil -} - -func (t *Field) UnmarshalTOML(data interface{}) error { - d, ok := data.(map[string]interface{}) - if !ok { - return nil - } - - if n, ok := d["name"].(string); !ok || n == "" { - return errors.New("field: missing or invalid value for name") - } else { - t.Name = n - } - - if n, ok := d["count"]; !ok { - return errors.New("field: missing value for count") - } else if count, err := cast.ToInt64E(n); err != nil { - return fmt.Errorf("field: invalid count, %v", err) - } else if count <= 0 { - return errors.New("field: count must be > 0") - } else { - t.Count = count - } - - if n, ok := d["time-precision"]; ok { - var tp precision - if err := tp.UnmarshalTOML(n); err != nil { - return err - } - t.TimePrecision = &tp - } - - if n, ok := d["time-interval"]; ok { - var ti duration - if err := ti.UnmarshalTOML(n); err != nil { - return err - } - t.TimeInterval = &ti - t.TimePrecision = nil - } - - if t.TimePrecision == nil && t.TimeInterval == nil { - var tp precision - t.TimePrecision = &tp - } - - // infer source - if _, ok := d["source"]; !ok { - return fmt.Errorf("missing source for field %q", t.Name) - } - - switch v := d["source"].(type) { - case int64, string, float64, bool: - t.Source = &FieldConstantValue{v} - case []interface{}: - if src, err := decodeFieldArraySource(v); err != nil { - return err - } else { - t.Source = src - } - case map[string]interface{}: - if src, err := decodeFieldSource(v); err != nil { - return err - } else { - t.Source = src - } - default: - // unknown - return fmt.Errorf("invalid source for tag %q: %T", t.Name, v) - } - - return nil -} - -func decodeFieldArraySource(data []interface{}) (FieldSource, error) { - if len(data) == 0 { - return nil, errors.New("empty array") - } - - var ( - src interface{} - err error - ) - - // use first value to determine slice type - switch data[0].(type) { - case int64: - src, err = toInt64SliceE(data) - case float64: - src, err = toFloat64SliceE(data) - case string: - src, err = cast.ToStringSliceE(data) - case bool: - src, err = cast.ToBoolSliceE(data) - default: - err = fmt.Errorf("unsupported field source data type: %T", data[0]) - } - - if err != nil { - return nil, err - } - - return &FieldArraySource{Value: src}, nil -} - -func decodeFieldSource(data map[string]interface{}) (FieldSource, error) { - typ, ok := data["type"].(string) - if !ok { - return nil, errors.New("missing type field") - } - switch typ { - case "rand": - return decodeFloatRandomSource(data) - case "zipf": - return decodeIntegerZipfSource(data) - default: - return nil, fmt.Errorf("invalid type field %q", typ) - } -} - -func decodeFloatRandomSource(data map[string]interface{}) (FieldSource, error) { - var s FieldFloatRandomSource - - if v, ok := data["seed"]; ok { - if v, err := cast.ToInt64E(v); err != nil { - return nil, fmt.Errorf("rand: invalid seed, %v", err) - } else { - s.Seed = v - } - } - - if v, ok := data["min"]; ok { - if v, err := cast.ToFloat64E(v); err != nil { - return nil, fmt.Errorf("rand: invalid min, %v", err) - } else { - s.Min = v - } - } - - if v, ok := data["max"]; ok { - if v, err := cast.ToFloat64E(v); err != nil { - return nil, fmt.Errorf("rand: invalid max, %v", err) - } else { - s.Max = v - } - } else { - s.Max = 1.0 - } - - if !(s.Min <= s.Max) { - return nil, errors.New("rand: min ≤ max") - } - - return &s, nil -} - -func decodeIntegerZipfSource(data map[string]interface{}) (FieldSource, error) { - var s FieldIntegerZipfSource - - if v, ok := data["seed"]; ok { - if v, err := cast.ToInt64E(v); err != nil { - return nil, fmt.Errorf("zipf: invalid seed, %v", err) - } else { - s.Seed = v - } - } - - if v, ok := data["s"]; ok { - if v, err := cast.ToFloat64E(v); err != nil || v <= 1.0 { - return nil, fmt.Errorf("zipf: invalid value for s (s > 1), %v", err) - } else { - s.S = v - } - } else { - return nil, fmt.Errorf("zipf: missing value for s") - } - - if v, ok := data["v"]; ok { - if v, err := cast.ToFloat64E(v); err != nil || v < 1.0 { - return nil, fmt.Errorf("zipf: invalid value for v (v ≥ 1), %v", err) - } else { - s.V = v - } - } else { - return nil, fmt.Errorf("zipf: missing value for v") - } - - if v, ok := data["imax"]; ok { - if v, err := cast.ToUint64E(v); err != nil { - return nil, fmt.Errorf("zipf: invalid value for imax, %v", err) - } else { - s.IMAX = v - } - } else { - return nil, fmt.Errorf("zipf: missing value for imax") - } - - return &s, nil -} diff --git a/v1/pkg/data/gen/toml_test.go b/v1/pkg/data/gen/toml_test.go deleted file mode 100644 index 7055ae7cd0..0000000000 --- a/v1/pkg/data/gen/toml_test.go +++ /dev/null @@ -1,169 +0,0 @@ -package gen - -import ( - "fmt" - "strings" - "testing" - - "github.com/BurntSushi/toml" - "github.com/google/go-cmp/cmp" -) - -func visit(root *Schema) string { - w := &strings.Builder{} - - walkFn := func(node SchemaNode) bool { - switch n := node.(type) { - case *Schema: - - case Measurements: - fmt.Fprintln(w, "Measurements: ") - - case *Measurement: - fmt.Fprintln(w) - fmt.Fprintf(w, " Name: %s\n", n.Name) - - case Tags: - fmt.Fprintln(w, " Tags:") - - case Fields: - fmt.Fprintln(w, " Fields:") - - case *Field: - if n.TimePrecision != nil { - fmt.Fprintf(w, " %s: %s, count=%d, time-precision=%s\n", n.Name, n.Source, n.Count, *n.TimePrecision) - } else { - fmt.Fprintf(w, " %s: %s, count=%d, time-interval=%s\n", n.Name, n.Source, n.Count, n.TimeInterval) - } - - case *Tag: - fmt.Fprintf(w, " %s: %s\n", n.Name, n.Source) - - } - - return true - } - - WalkDown(VisitorFn(walkFn), root) - - return w.String() -} - -func TestSchema(t *testing.T) { - in := ` -title = "example schema" -series-limit = 10 - -[[measurements]] - name = "constant" - series-limit = 5 - - [[measurements.tags]] - name = "tag0" - source = [ "host1", "host2" ] - - [[measurements.tags]] - name = "tag1" - source = { type = "file", path = "foo.txt" } - - [[measurements.fields]] - name = "floatC" - count = 5000 - source = 0.5 - time-precision = "us" - - [[measurements.fields]] - name = "integerC" - count = 5000 - source = 3 - time-precision = "hour" - - [[measurements.fields]] - name = "stringC" - count = 5000 - source = "hello" - time-interval = "60s" - - [[measurements.fields]] - name = "stringA" - count = 5000 - source = ["hello", "world"] - - [[measurements.fields]] - name = "boolf" - count = 5000 - source = false - -[[measurements]] -name = "random" - - [[measurements.tags]] - name = "tagSeq" - source = { type = "sequence", format = "value%s", start = 0, count = 100 } - - [[measurements.fields]] - name = "floatR" - count = 5000 - source = { type = "rand", min = 0.5, max = 50.1, seed = 10 } - time-precision = "us" - -[[measurements]] -name = "array" - - [[measurements.tags]] - name = "tagSeq" - source = { type = "sequence", format = "value%s", start = 0, count = 100 } - - [[measurements.tags]] - name = "tagFile" - source = { type = "file", path = "foo.txt" } - - [[measurements.fields]] - name = "stringA" - count = 1000 - source = ["this", "that"] - time-precision = "us" - - [[measurements.fields]] - name = "integerA" - count = 1000 - source = [5, 6, 7] - time-interval = "90s" -` - var out Schema - _, err := toml.Decode(in, &out) - if err != nil { - t.Fatalf("unxpected error: %v", err) - } - - exp := `Measurements: - - Name: constant - Tags: - tag0: array, source=[]string{"host1", "host2"} - tag1: file, path=foo.txt - Fields: - floatC: constant, source=0.5, count=5000, time-precision=Microsecond - integerC: constant, source=3, count=5000, time-precision=Hour - stringC: constant, source="hello", count=5000, time-interval=1m0s - stringA: array, source=[]string{"hello", "world"}, count=5000, time-precision=Millisecond - boolf: constant, source=false, count=5000, time-precision=Millisecond - - Name: random - Tags: - tagSeq: sequence, prefix="value%s", range=[0,100) - Fields: - floatR: rand, seed=10, min=50.100000, max=50.100000, count=5000, time-precision=Microsecond - - Name: array - Tags: - tagSeq: sequence, prefix="value%s", range=[0,100) - tagFile: file, path=foo.txt - Fields: - stringA: array, source=[]string{"this", "that"}, count=1000, time-precision=Microsecond - integerA: array, source=[]int64{5, 6, 7}, count=1000, time-interval=1m30s -` - if got := visit(&out); !cmp.Equal(got, exp) { - t.Errorf("unexpected value, -got/+exp\n%s", cmp.Diff(got, exp)) - } -} diff --git a/v1/pkg/data/gen/types.tmpldata b/v1/pkg/data/gen/types.tmpldata deleted file mode 100644 index 82651de253..0000000000 --- a/v1/pkg/data/gen/types.tmpldata +++ /dev/null @@ -1,30 +0,0 @@ -[ - { - "Name":"Float", - "name":"float", - "Type":"float64", - "Rand":"Float64" - }, - { - "Name":"Integer", - "name":"integer", - "Type":"int64", - "Rand":"Int64" - }, - { - "Name":"Unsigned", - "name":"unsigned", - "Type":"uint64", - "Rand":"Uint64" - }, - { - "Name":"String", - "name":"string", - "Type":"string" - }, - { - "Name":"Boolean", - "name":"boolean", - "Type":"bool" - } -] diff --git a/v1/pkg/data/gen/util.go b/v1/pkg/data/gen/util.go deleted file mode 100644 index bb16508f5f..0000000000 --- a/v1/pkg/data/gen/util.go +++ /dev/null @@ -1,87 +0,0 @@ -package gen - -import ( - "fmt" - "reflect" - "sort" - - "github.com/spf13/cast" -) - -func min(a, b int) int { - if a < b { - return a - } - return b -} - -func sortDedupStrings(in []string) []string { - sort.Strings(in) - j := 0 - for i := 1; i < len(in); i++ { - if in[j] == in[i] { - continue - } - j++ - in[j] = in[i] - } - return in[:j+1] -} - -// ToInt64SliceE casts an interface to a []int64 type. -func toInt64SliceE(i interface{}) ([]int64, error) { - if i == nil { - return []int64{}, fmt.Errorf("unable to cast %#v of type %T to []int64", i, i) - } - - switch v := i.(type) { - case []int64: - return v, nil - } - - kind := reflect.TypeOf(i).Kind() - switch kind { - case reflect.Slice, reflect.Array: - s := reflect.ValueOf(i) - a := make([]int64, s.Len()) - for j := 0; j < s.Len(); j++ { - val, err := cast.ToInt64E(s.Index(j).Interface()) - if err != nil { - return []int64{}, fmt.Errorf("unable to cast %#v of type %T to []int64", i, i) - } - a[j] = val - } - return a, nil - default: - return []int64{}, fmt.Errorf("unable to cast %#v of type %T to []int64", i, i) - } -} - -// ToFloat64SliceE casts an interface to a []float64 type. -func toFloat64SliceE(i interface{}) ([]float64, error) { - if i == nil { - return []float64{}, fmt.Errorf("unable to cast %#v of type %T to []float64", i, i) - } - - switch v := i.(type) { - case []float64: - return v, nil - } - - kind := reflect.TypeOf(i).Kind() - switch kind { - case reflect.Slice, reflect.Array: - s := reflect.ValueOf(i) - a := make([]float64, s.Len()) - for j := 0; j < s.Len(); j++ { - val, err := cast.ToFloat64E(s.Index(j).Interface()) - if err != nil { - return []float64{}, fmt.Errorf("unable to cast %#v of type %T to []float64", i, i) - } - a[j] = val - } - return a, nil - default: - return []float64{}, fmt.Errorf("unable to cast %#v of type %T to []float64", i, i) - } -} diff --git a/v1/pkg/data/gen/values.gen.go b/v1/pkg/data/gen/values.gen.go deleted file mode 100644 index 4cc16b7647..0000000000 --- a/v1/pkg/data/gen/values.gen.go +++ /dev/null @@ -1,252 +0,0 @@ -// Generated by tmpl -// https://github.com/benbjohnson/tmpl -// -// DO NOT EDIT! -// Source: values.gen.go.tmpl - -package gen - -type floatConstantValuesSequence struct { - v float64 -} - -func NewFloatConstantValuesSequence(v float64) FloatValuesSequence { - return &floatConstantValuesSequence{ - v: v, - } -} - -func (g *floatConstantValuesSequence) Reset() { -} - -func (g *floatConstantValuesSequence) Write(vs []float64) { - for i := 0; i < len(vs); i++ { - vs[i] = g.v - } -} - -type integerConstantValuesSequence struct { - v int64 -} - -func NewIntegerConstantValuesSequence(v int64) IntegerValuesSequence { - return &integerConstantValuesSequence{ - v: v, - } -} - -func (g *integerConstantValuesSequence) Reset() { -} - -func (g *integerConstantValuesSequence) Write(vs []int64) { - for i := 0; i < len(vs); i++ { - vs[i] = g.v - } -} - -type unsignedConstantValuesSequence struct { - v uint64 -} - -func NewUnsignedConstantValuesSequence(v uint64) UnsignedValuesSequence { - return &unsignedConstantValuesSequence{ - v: v, - } -} - -func (g *unsignedConstantValuesSequence) Reset() { -} - -func (g *unsignedConstantValuesSequence) Write(vs []uint64) { - for i := 0; i < len(vs); i++ { - vs[i] = g.v - } -} - -type stringConstantValuesSequence struct { - v string -} - -func NewStringConstantValuesSequence(v string) StringValuesSequence { - return &stringConstantValuesSequence{ - v: v, - } -} - -func (g *stringConstantValuesSequence) Reset() { -} - -func (g *stringConstantValuesSequence) Write(vs []string) { - for i := 0; i < len(vs); i++ { - vs[i] = g.v - } -} - -type booleanConstantValuesSequence struct { - v bool -} - -func NewBooleanConstantValuesSequence(v bool) BooleanValuesSequence { - return &booleanConstantValuesSequence{ - v: v, - } -} - -func (g *booleanConstantValuesSequence) Reset() { -} - -func (g *booleanConstantValuesSequence) Write(vs []bool) { - for i := 0; i < len(vs); i++ { - vs[i] = g.v - } -} - -type floatArrayValuesSequence struct { - v []float64 - vi int -} - -func NewFloatArrayValuesSequence(v []float64) FloatValuesSequence { - return &floatArrayValuesSequence{ - v: v, - } -} - -func (g *floatArrayValuesSequence) Reset() { - g.vi = 0 -} - -func (g *floatArrayValuesSequence) Write(vs []float64) { - var ( - v = g.v - vi = g.vi - ) - for i := 0; i < len(vs); i++ { - if vi >= len(v) { - vi = 0 - } - vs[i] = v[vi] - vi += 1 - } - g.vi = vi -} - -type integerArrayValuesSequence struct { - v []int64 - vi int -} - -func NewIntegerArrayValuesSequence(v []int64) IntegerValuesSequence { - return &integerArrayValuesSequence{ - v: v, - } -} - -func (g *integerArrayValuesSequence) Reset() { - g.vi = 0 -} - -func (g *integerArrayValuesSequence) Write(vs []int64) { - var ( - v = g.v - vi = g.vi - ) - for i := 0; i < len(vs); i++ { - if vi >= len(v) { - vi = 0 - } - vs[i] = v[vi] - vi += 1 - } - g.vi = vi -} - -type unsignedArrayValuesSequence struct { - v []uint64 - vi int -} - -func NewUnsignedArrayValuesSequence(v []uint64) UnsignedValuesSequence { - return &unsignedArrayValuesSequence{ - v: v, - } -} - -func (g *unsignedArrayValuesSequence) Reset() { - g.vi = 0 -} - -func (g *unsignedArrayValuesSequence) Write(vs []uint64) { - var ( - v = g.v - vi = g.vi - ) - for i := 0; i < len(vs); i++ { - if vi >= len(v) { - vi = 0 - } - vs[i] = v[vi] - vi += 1 - } - g.vi = vi -} - -type stringArrayValuesSequence struct { - v []string - vi int -} - -func NewStringArrayValuesSequence(v []string) StringValuesSequence { - return &stringArrayValuesSequence{ - v: v, - } -} - -func (g *stringArrayValuesSequence) Reset() { - g.vi = 0 -} - -func (g *stringArrayValuesSequence) Write(vs []string) { - var ( - v = g.v - vi = g.vi - ) - for i := 0; i < len(vs); i++ { - if vi >= len(v) { - vi = 0 - } - vs[i] = v[vi] - vi += 1 - } - g.vi = vi -} - -type booleanArrayValuesSequence struct { - v []bool - vi int -} - -func NewBooleanArrayValuesSequence(v []bool) BooleanValuesSequence { - return &booleanArrayValuesSequence{ - v: v, - } -} - -func (g *booleanArrayValuesSequence) Reset() { - g.vi = 0 -} - -func (g *booleanArrayValuesSequence) Write(vs []bool) { - var ( - v = g.v - vi = g.vi - ) - for i := 0; i < len(vs); i++ { - if vi >= len(v) { - vi = 0 - } - vs[i] = v[vi] - vi += 1 - } - g.vi = vi -} diff --git a/v1/pkg/data/gen/values.gen.go.tmpl b/v1/pkg/data/gen/values.gen.go.tmpl deleted file mode 100644 index a76e5a09fe..0000000000 --- a/v1/pkg/data/gen/values.gen.go.tmpl +++ /dev/null @@ -1,54 +0,0 @@ -package gen - -{{range .}} -type {{.name}}ConstantValuesSequence struct { - v {{.Type}} -} - -func New{{.Name}}ConstantValuesSequence(v {{.Type}}) {{.Name}}ValuesSequence { - return &{{.name}}ConstantValuesSequence{ - v: v, - } -} - -func (g *{{.name}}ConstantValuesSequence) Reset() { -} - -func (g *{{.name}}ConstantValuesSequence) Write(vs []{{.Type}}) { - for i := 0; i < len(vs); i++ { - vs[i] = g.v - } -} -{{end}} - -{{range .}} -type {{.name}}ArrayValuesSequence struct { - v []{{.Type}} - vi int -} - -func New{{.Name}}ArrayValuesSequence(v []{{.Type}}) {{.Name}}ValuesSequence { - return &{{.name}}ArrayValuesSequence{ - v: v, - } -} - -func (g *{{.name}}ArrayValuesSequence) Reset() { - g.vi = 0 -} - -func (g *{{.name}}ArrayValuesSequence) Write(vs []{{.Type}}) { - var ( - v = g.v - vi = g.vi - ) - for i := 0; i < len(vs); i++ { - if vi >= len(v) { - vi = 0 - } - vs[i] = v[vi] - vi += 1 - } - g.vi = vi -} -{{end}} diff --git a/v1/pkg/data/gen/values.go b/v1/pkg/data/gen/values.go deleted file mode 100644 index 9dfc1cb809..0000000000 --- a/v1/pkg/data/gen/values.go +++ /dev/null @@ -1,46 +0,0 @@ -package gen - -import ( - "math/rand" -) - -type floatRandomValuesSequence struct { - r *rand.Rand - a float64 - b float64 -} - -func NewFloatRandomValuesSequence(min, max float64, r *rand.Rand) FloatValuesSequence { - return &floatRandomValuesSequence{r: r, a: max - min, b: min} -} - -func (g *floatRandomValuesSequence) Reset() {} - -func (g *floatRandomValuesSequence) Write(vs []float64) { - var ( - a = g.a - b = g.b - ) - for i := 0; i < len(vs); i++ { - vs[i] = a*g.r.Float64() + b // ax + b - } -} - -type integerRandomValuesSequence struct { - r *rand.Zipf -} - -// NewIntegerZipfValuesSequence produces int64 values using a Zipfian distribution -// described by s. -func NewIntegerZipfValuesSequence(s *FieldIntegerZipfSource) IntegerValuesSequence { - r := rand.New(rand.NewSource(s.Seed)) - return &integerRandomValuesSequence{r: rand.NewZipf(r, s.S, s.V, s.IMAX)} -} - -func (g *integerRandomValuesSequence) Reset() {} - -func (g *integerRandomValuesSequence) Write(vs []int64) { - for i := 0; i < len(vs); i++ { - vs[i] = int64(g.r.Uint64()) - } -} diff --git a/v1/pkg/data/gen/values_sequence.gen.go b/v1/pkg/data/gen/values_sequence.gen.go deleted file mode 100644 index 102bf6524f..0000000000 --- a/v1/pkg/data/gen/values_sequence.gen.go +++ /dev/null @@ -1,251 +0,0 @@ -// Generated by tmpl -// https://github.com/benbjohnson/tmpl -// -// DO NOT EDIT! -// Source: values_sequence.gen.go.tmpl - -package gen - -import ( - "github.com/influxdata/influxdb/v2/v1/tsdb" -) - -type FloatValuesSequence interface { - Reset() - Write(v []float64) -} - -type timeFloatValuesSequence struct { - vals floatArray - ts TimestampSequence - vs FloatValuesSequence - count int - n int -} - -func NewTimeFloatValuesSequence(count int, ts TimestampSequence, vs FloatValuesSequence) TimeValuesSequence { - return &timeFloatValuesSequence{ - vals: *newFloatArrayLen(tsdb.DefaultMaxPointsPerBlock), - ts: ts, - vs: vs, - count: count, - n: count, - } -} - -func (s *timeFloatValuesSequence) Reset() { - s.ts.Reset() - s.vs.Reset() - s.n = s.count -} - -func (s *timeFloatValuesSequence) Next() bool { - if s.n > 0 { - c := min(s.n, tsdb.DefaultMaxPointsPerBlock) - s.n -= c - s.vals.Timestamps = s.vals.Timestamps[:c] - s.vals.Values = s.vals.Values[:c] - - s.ts.Write(s.vals.Timestamps) - s.vs.Write(s.vals.Values) - return true - } - - return false -} - -func (s *timeFloatValuesSequence) Values() Values { - return &s.vals -} - -type IntegerValuesSequence interface { - Reset() - Write(v []int64) -} - -type timeIntegerValuesSequence struct { - vals integerArray - ts TimestampSequence - vs IntegerValuesSequence - count int - n int -} - -func NewTimeIntegerValuesSequence(count int, ts TimestampSequence, vs IntegerValuesSequence) TimeValuesSequence { - return &timeIntegerValuesSequence{ - vals: *newIntegerArrayLen(tsdb.DefaultMaxPointsPerBlock), - ts: ts, - vs: vs, - count: count, - n: count, - } -} - -func (s *timeIntegerValuesSequence) Reset() { - s.ts.Reset() - s.vs.Reset() - s.n = s.count -} - -func (s *timeIntegerValuesSequence) Next() bool { - if s.n > 0 { - c := min(s.n, tsdb.DefaultMaxPointsPerBlock) - s.n -= c - s.vals.Timestamps = s.vals.Timestamps[:c] - s.vals.Values = s.vals.Values[:c] - - s.ts.Write(s.vals.Timestamps) - s.vs.Write(s.vals.Values) - return true - } - - return false -} - -func (s *timeIntegerValuesSequence) Values() Values { - return &s.vals -} - -type UnsignedValuesSequence interface { - Reset() - Write(v []uint64) -} - -type timeUnsignedValuesSequence struct { - vals unsignedArray - ts TimestampSequence - vs UnsignedValuesSequence - count int - n int -} - -func NewTimeUnsignedValuesSequence(count int, ts TimestampSequence, vs UnsignedValuesSequence) TimeValuesSequence { - return &timeUnsignedValuesSequence{ - vals: *newUnsignedArrayLen(tsdb.DefaultMaxPointsPerBlock), - ts: ts, - vs: vs, - count: count, - n: count, - } -} - -func (s *timeUnsignedValuesSequence) Reset() { - s.ts.Reset() - s.vs.Reset() - s.n = s.count -} - -func (s *timeUnsignedValuesSequence) Next() bool { - if s.n > 0 { - c := min(s.n, tsdb.DefaultMaxPointsPerBlock) - s.n -= c - s.vals.Timestamps = s.vals.Timestamps[:c] - s.vals.Values = s.vals.Values[:c] - - s.ts.Write(s.vals.Timestamps) - s.vs.Write(s.vals.Values) - return true - } - - return false -} - -func (s *timeUnsignedValuesSequence) Values() Values { - return &s.vals -} - -type StringValuesSequence interface { - Reset() - Write(v []string) -} - -type timeStringValuesSequence struct { - vals stringArray - ts TimestampSequence - vs StringValuesSequence - count int - n int -} - -func NewTimeStringValuesSequence(count int, ts TimestampSequence, vs StringValuesSequence) TimeValuesSequence { - return &timeStringValuesSequence{ - vals: *newStringArrayLen(tsdb.DefaultMaxPointsPerBlock), - ts: ts, - vs: vs, - count: count, - n: count, - } -} - -func (s *timeStringValuesSequence) Reset() { - s.ts.Reset() - s.vs.Reset() - s.n = s.count -} - -func (s *timeStringValuesSequence) Next() bool { - if s.n > 0 { - c := min(s.n, tsdb.DefaultMaxPointsPerBlock) - s.n -= c - s.vals.Timestamps = s.vals.Timestamps[:c] - s.vals.Values = s.vals.Values[:c] - - s.ts.Write(s.vals.Timestamps) - s.vs.Write(s.vals.Values) - return true - } - - return false -} - -func (s *timeStringValuesSequence) Values() Values { - return &s.vals -} - -type BooleanValuesSequence interface { - Reset() - Write(v []bool) -} - -type timeBooleanValuesSequence struct { - vals booleanArray - ts TimestampSequence - vs BooleanValuesSequence - count int - n int -} - -func NewTimeBooleanValuesSequence(count int, ts TimestampSequence, vs BooleanValuesSequence) TimeValuesSequence { - return &timeBooleanValuesSequence{ - vals: *newBooleanArrayLen(tsdb.DefaultMaxPointsPerBlock), - ts: ts, - vs: vs, - count: count, - n: count, - } -} - -func (s *timeBooleanValuesSequence) Reset() { - s.ts.Reset() - s.vs.Reset() - s.n = s.count -} - -func (s *timeBooleanValuesSequence) Next() bool { - if s.n > 0 { - c := min(s.n, tsdb.DefaultMaxPointsPerBlock) - s.n -= c - s.vals.Timestamps = s.vals.Timestamps[:c] - s.vals.Values = s.vals.Values[:c] - - s.ts.Write(s.vals.Timestamps) - s.vs.Write(s.vals.Values) - return true - } - - return false -} - -func (s *timeBooleanValuesSequence) Values() Values { - return &s.vals -} diff --git a/v1/pkg/data/gen/values_sequence.gen.go.tmpl b/v1/pkg/data/gen/values_sequence.gen.go.tmpl deleted file mode 100644 index 384df68e50..0000000000 --- a/v1/pkg/data/gen/values_sequence.gen.go.tmpl +++ /dev/null @@ -1,55 +0,0 @@ -package gen - -import ( - "github.com/influxdata/influxdb/v2/v1/tsdb" -) - -{{range .}} -type {{.Name}}ValuesSequence interface { - Reset() - Write(v []{{.Type}}) -} - -type time{{.Name}}ValuesSequence struct { - vals {{.name}}Array - ts TimestampSequence - vs {{.Name}}ValuesSequence - count int - n int -} - -func NewTime{{.Name}}ValuesSequence(count int, ts TimestampSequence, vs {{.Name}}ValuesSequence) TimeValuesSequence { - return &time{{.Name}}ValuesSequence{ - vals: *new{{.Name}}ArrayLen(tsdb.DefaultMaxPointsPerBlock), - ts: ts, - vs: vs, - count: count, - n: count, - } -} - -func (s *time{{.Name}}ValuesSequence) Reset() { - s.ts.Reset() - s.vs.Reset() - s.n = s.count -} - -func (s *time{{.Name}}ValuesSequence) Next() bool { - if s.n > 0 { - c := min(s.n, tsdb.DefaultMaxPointsPerBlock) - s.n -= c - s.vals.Timestamps = s.vals.Timestamps[:c] - s.vals.Values = s.vals.Values[:c] - - s.ts.Write(s.vals.Timestamps) - s.vs.Write(s.vals.Values) - return true - } - - return false -} - -func (s *time{{.Name}}ValuesSequence) Values() Values { - return &s.vals -} -{{end}} \ No newline at end of file diff --git a/v1/services/storage/predicate_test.go b/v1/services/storage/predicate_test.go index a4502002cc..77027fa623 100644 --- a/v1/services/storage/predicate_test.go +++ b/v1/services/storage/predicate_test.go @@ -6,7 +6,7 @@ import ( "github.com/influxdata/influxdb/v2/pkg/testing/assert" "github.com/influxdata/influxdb/v2/storage/reads/datatypes" "github.com/influxdata/influxdb/v2/v1/services/storage" - "github.com/influxdata/influxdb/v2/v1/storage/reads" + "github.com/influxdata/influxdb/v2/storage/reads" "github.com/influxdata/influxql" ) diff --git a/v1/services/storage/series_cursor.go b/v1/services/storage/series_cursor.go index 14825b64b7..ed0378f9b6 100644 --- a/v1/services/storage/series_cursor.go +++ b/v1/services/storage/series_cursor.go @@ -8,7 +8,7 @@ import ( "github.com/influxdata/influxdb/v2/influxql/query" "github.com/influxdata/influxdb/v2/storage/reads/datatypes" "github.com/influxdata/influxdb/v2/v1/models" - "github.com/influxdata/influxdb/v2/v1/storage/reads" + "github.com/influxdata/influxdb/v2/storage/reads" "github.com/influxdata/influxdb/v2/v1/tsdb" "github.com/influxdata/influxql" opentracing "github.com/opentracing/opentracing-go" diff --git a/v1/services/storage/store.go b/v1/services/storage/store.go index 82d9bb4aa9..477ae68329 100644 --- a/v1/services/storage/store.go +++ b/v1/services/storage/store.go @@ -12,7 +12,7 @@ import ( "github.com/influxdata/influxdb/v2/storage/reads/datatypes" "github.com/influxdata/influxdb/v2/v1/models" "github.com/influxdata/influxdb/v2/v1/services/meta" - "github.com/influxdata/influxdb/v2/v1/storage/reads" + "github.com/influxdata/influxdb/v2/storage/reads" "github.com/influxdata/influxdb/v2/v1/tsdb" "github.com/influxdata/influxdb/v2/v1/tsdb/cursors" "github.com/influxdata/influxql" diff --git a/v1/storage/reads/array_cursor.gen.go b/v1/storage/reads/array_cursor.gen.go deleted file mode 100644 index 76d26499e0..0000000000 --- a/v1/storage/reads/array_cursor.gen.go +++ /dev/null @@ -1,1085 +0,0 @@ -// Generated by tmpl -// https://github.com/benbjohnson/tmpl -// -// DO NOT EDIT! -// Source: array_cursor.gen.go.tmpl - -package reads - -import ( - "errors" - - "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 -) - -// ******************** -// Float Array Cursor - -type floatArrayFilterCursor struct { - cursors.FloatArrayCursor - cond expression - m *singleValue - res *cursors.FloatArray - tmp *cursors.FloatArray -} - -func newFloatFilterArrayCursor(cond expression) *floatArrayFilterCursor { - return &floatArrayFilterCursor{ - cond: cond, - m: &singleValue{}, - res: cursors.NewFloatArrayLen(MaxPointsPerBlock), - tmp: &cursors.FloatArray{}, - } -} - -func (c *floatArrayFilterCursor) reset(cur cursors.FloatArrayCursor) { - c.FloatArrayCursor = cur - c.tmp.Timestamps, c.tmp.Values = nil, nil -} - -func (c *floatArrayFilterCursor) Stats() cursors.CursorStats { return c.FloatArrayCursor.Stats() } - -func (c *floatArrayFilterCursor) Next() *cursors.FloatArray { - 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.FloatArray - - if c.tmp.Len() > 0 { - a = c.tmp - c.tmp.Timestamps = nil - c.tmp.Values = nil - } else { - a = c.FloatArrayCursor.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 - } - } - } - a = c.FloatArrayCursor.Next() - } - - c.res.Timestamps = c.res.Timestamps[:pos] - c.res.Values = c.res.Values[:pos] - - return c.res -} - -type floatMultiShardArrayCursor struct { - cursors.FloatArrayCursor - cursorContext - filter *floatArrayFilterCursor -} - -func (c *floatMultiShardArrayCursor) reset(cur cursors.FloatArrayCursor, itrs cursors.CursorIterators, cond expression) { - if cond != nil { - if c.filter == nil { - c.filter = newFloatFilterArrayCursor(cond) - } - c.filter.reset(cur) - cur = c.filter - } - - c.FloatArrayCursor = cur - c.itrs = itrs - c.err = nil - c.count = 0 -} - -func (c *floatMultiShardArrayCursor) Err() error { return c.err } - -func (c *floatMultiShardArrayCursor) Stats() cursors.CursorStats { - return c.FloatArrayCursor.Stats() -} - -func (c *floatMultiShardArrayCursor) Next() *cursors.FloatArray { - for { - a := c.FloatArrayCursor.Next() - if a.Len() == 0 { - if c.nextArrayCursor() { - continue - } - } - c.count += int64(a.Len()) - if c.count > c.limit { - diff := c.count - c.limit - c.count -= diff - rem := int64(a.Len()) - diff - a.Timestamps = a.Timestamps[:rem] - a.Values = a.Values[:rem] - } - return a - } -} - -func (c *floatMultiShardArrayCursor) nextArrayCursor() bool { - if len(c.itrs) == 0 { - return false - } - - c.FloatArrayCursor.Close() - - var itr cursors.CursorIterator - var cur cursors.Cursor - for cur == nil && len(c.itrs) > 0 { - itr, c.itrs = c.itrs[0], c.itrs[1:] - cur, _ = itr.Next(c.ctx, c.req) - } - - var ok bool - if cur != nil { - var next cursors.FloatArrayCursor - next, ok = cur.(cursors.FloatArrayCursor) - if !ok { - cur.Close() - next = FloatEmptyArrayCursor - c.itrs = nil - c.err = errors.New("expected float cursor") - } else { - if c.filter != nil { - c.filter.reset(next) - next = c.filter - } - } - c.FloatArrayCursor = next - } else { - c.FloatArrayCursor = FloatEmptyArrayCursor - } - - return ok -} - -type floatArraySumCursor struct { - cursors.FloatArrayCursor - ts [1]int64 - vs [1]float64 - res *cursors.FloatArray -} - -func newFloatArraySumCursor(cur cursors.FloatArrayCursor) *floatArraySumCursor { - return &floatArraySumCursor{ - FloatArrayCursor: cur, - res: &cursors.FloatArray{}, - } -} - -func (c floatArraySumCursor) Stats() cursors.CursorStats { return c.FloatArrayCursor.Stats() } - -func (c floatArraySumCursor) Next() *cursors.FloatArray { - a := c.FloatArrayCursor.Next() - if len(a.Timestamps) == 0 { - return a - } - - ts := a.Timestamps[0] - var acc float64 - - for { - for _, v := range a.Values { - acc += v - } - a = c.FloatArrayCursor.Next() - if len(a.Timestamps) == 0 { - c.ts[0] = ts - c.vs[0] = acc - c.res.Timestamps = c.ts[:] - c.res.Values = c.vs[:] - return c.res - } - } -} - -type integerFloatCountArrayCursor struct { - cursors.FloatArrayCursor -} - -func (c *integerFloatCountArrayCursor) Stats() cursors.CursorStats { - return c.FloatArrayCursor.Stats() -} - -func (c *integerFloatCountArrayCursor) Next() *cursors.IntegerArray { - a := c.FloatArrayCursor.Next() - if len(a.Timestamps) == 0 { - return &cursors.IntegerArray{} - } - - ts := a.Timestamps[0] - var acc int64 - for { - acc += int64(len(a.Timestamps)) - a = c.FloatArrayCursor.Next() - if len(a.Timestamps) == 0 { - res := cursors.NewIntegerArrayLen(1) - res.Timestamps[0] = ts - res.Values[0] = acc - return res - } - } -} - -type floatEmptyArrayCursor struct { - res cursors.FloatArray -} - -var FloatEmptyArrayCursor cursors.FloatArrayCursor = &floatEmptyArrayCursor{} - -func (c *floatEmptyArrayCursor) Err() error { return nil } -func (c *floatEmptyArrayCursor) Close() {} -func (c *floatEmptyArrayCursor) Stats() cursors.CursorStats { return cursors.CursorStats{} } -func (c *floatEmptyArrayCursor) Next() *cursors.FloatArray { return &c.res } - -// ******************** -// Integer Array Cursor - -type integerArrayFilterCursor struct { - cursors.IntegerArrayCursor - cond expression - m *singleValue - res *cursors.IntegerArray - tmp *cursors.IntegerArray -} - -func newIntegerFilterArrayCursor(cond expression) *integerArrayFilterCursor { - return &integerArrayFilterCursor{ - cond: cond, - m: &singleValue{}, - res: cursors.NewIntegerArrayLen(MaxPointsPerBlock), - tmp: &cursors.IntegerArray{}, - } -} - -func (c *integerArrayFilterCursor) reset(cur cursors.IntegerArrayCursor) { - c.IntegerArrayCursor = cur - c.tmp.Timestamps, c.tmp.Values = nil, nil -} - -func (c *integerArrayFilterCursor) Stats() cursors.CursorStats { return c.IntegerArrayCursor.Stats() } - -func (c *integerArrayFilterCursor) Next() *cursors.IntegerArray { - 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.IntegerArray - - if c.tmp.Len() > 0 { - a = c.tmp - c.tmp.Timestamps = nil - c.tmp.Values = nil - } else { - a = c.IntegerArrayCursor.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 - } - } - } - a = c.IntegerArrayCursor.Next() - } - - c.res.Timestamps = c.res.Timestamps[:pos] - c.res.Values = c.res.Values[:pos] - - return c.res -} - -type integerMultiShardArrayCursor struct { - cursors.IntegerArrayCursor - cursorContext - filter *integerArrayFilterCursor -} - -func (c *integerMultiShardArrayCursor) reset(cur cursors.IntegerArrayCursor, itrs cursors.CursorIterators, cond expression) { - if cond != nil { - if c.filter == nil { - c.filter = newIntegerFilterArrayCursor(cond) - } - c.filter.reset(cur) - cur = c.filter - } - - c.IntegerArrayCursor = cur - c.itrs = itrs - c.err = nil - c.count = 0 -} - -func (c *integerMultiShardArrayCursor) Err() error { return c.err } - -func (c *integerMultiShardArrayCursor) Stats() cursors.CursorStats { - return c.IntegerArrayCursor.Stats() -} - -func (c *integerMultiShardArrayCursor) Next() *cursors.IntegerArray { - for { - a := c.IntegerArrayCursor.Next() - if a.Len() == 0 { - if c.nextArrayCursor() { - continue - } - } - c.count += int64(a.Len()) - if c.count > c.limit { - diff := c.count - c.limit - c.count -= diff - rem := int64(a.Len()) - diff - a.Timestamps = a.Timestamps[:rem] - a.Values = a.Values[:rem] - } - return a - } -} - -func (c *integerMultiShardArrayCursor) nextArrayCursor() bool { - if len(c.itrs) == 0 { - return false - } - - c.IntegerArrayCursor.Close() - - var itr cursors.CursorIterator - var cur cursors.Cursor - for cur == nil && len(c.itrs) > 0 { - itr, c.itrs = c.itrs[0], c.itrs[1:] - cur, _ = itr.Next(c.ctx, c.req) - } - - var ok bool - if cur != nil { - var next cursors.IntegerArrayCursor - next, ok = cur.(cursors.IntegerArrayCursor) - if !ok { - cur.Close() - next = IntegerEmptyArrayCursor - c.itrs = nil - c.err = errors.New("expected integer cursor") - } else { - if c.filter != nil { - c.filter.reset(next) - next = c.filter - } - } - c.IntegerArrayCursor = next - } else { - c.IntegerArrayCursor = IntegerEmptyArrayCursor - } - - return ok -} - -type integerArraySumCursor struct { - cursors.IntegerArrayCursor - ts [1]int64 - vs [1]int64 - res *cursors.IntegerArray -} - -func newIntegerArraySumCursor(cur cursors.IntegerArrayCursor) *integerArraySumCursor { - return &integerArraySumCursor{ - IntegerArrayCursor: cur, - res: &cursors.IntegerArray{}, - } -} - -func (c integerArraySumCursor) Stats() cursors.CursorStats { return c.IntegerArrayCursor.Stats() } - -func (c integerArraySumCursor) Next() *cursors.IntegerArray { - a := c.IntegerArrayCursor.Next() - if len(a.Timestamps) == 0 { - return a - } - - ts := a.Timestamps[0] - var acc int64 - - for { - for _, v := range a.Values { - acc += v - } - a = c.IntegerArrayCursor.Next() - if len(a.Timestamps) == 0 { - c.ts[0] = ts - c.vs[0] = acc - c.res.Timestamps = c.ts[:] - c.res.Values = c.vs[:] - return c.res - } - } -} - -type integerIntegerCountArrayCursor struct { - cursors.IntegerArrayCursor -} - -func (c *integerIntegerCountArrayCursor) Stats() cursors.CursorStats { - return c.IntegerArrayCursor.Stats() -} - -func (c *integerIntegerCountArrayCursor) Next() *cursors.IntegerArray { - a := c.IntegerArrayCursor.Next() - if len(a.Timestamps) == 0 { - return &cursors.IntegerArray{} - } - - ts := a.Timestamps[0] - var acc int64 - for { - acc += int64(len(a.Timestamps)) - a = c.IntegerArrayCursor.Next() - if len(a.Timestamps) == 0 { - res := cursors.NewIntegerArrayLen(1) - res.Timestamps[0] = ts - res.Values[0] = acc - return res - } - } -} - -type integerEmptyArrayCursor struct { - res cursors.IntegerArray -} - -var IntegerEmptyArrayCursor cursors.IntegerArrayCursor = &integerEmptyArrayCursor{} - -func (c *integerEmptyArrayCursor) Err() error { return nil } -func (c *integerEmptyArrayCursor) Close() {} -func (c *integerEmptyArrayCursor) Stats() cursors.CursorStats { return cursors.CursorStats{} } -func (c *integerEmptyArrayCursor) Next() *cursors.IntegerArray { return &c.res } - -// ******************** -// Unsigned Array Cursor - -type unsignedArrayFilterCursor struct { - cursors.UnsignedArrayCursor - cond expression - m *singleValue - res *cursors.UnsignedArray - tmp *cursors.UnsignedArray -} - -func newUnsignedFilterArrayCursor(cond expression) *unsignedArrayFilterCursor { - return &unsignedArrayFilterCursor{ - cond: cond, - m: &singleValue{}, - res: cursors.NewUnsignedArrayLen(MaxPointsPerBlock), - tmp: &cursors.UnsignedArray{}, - } -} - -func (c *unsignedArrayFilterCursor) reset(cur cursors.UnsignedArrayCursor) { - c.UnsignedArrayCursor = cur - c.tmp.Timestamps, c.tmp.Values = nil, nil -} - -func (c *unsignedArrayFilterCursor) Stats() cursors.CursorStats { return c.UnsignedArrayCursor.Stats() } - -func (c *unsignedArrayFilterCursor) Next() *cursors.UnsignedArray { - 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.UnsignedArray - - if c.tmp.Len() > 0 { - a = c.tmp - c.tmp.Timestamps = nil - c.tmp.Values = nil - } else { - a = c.UnsignedArrayCursor.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 - } - } - } - a = c.UnsignedArrayCursor.Next() - } - - c.res.Timestamps = c.res.Timestamps[:pos] - c.res.Values = c.res.Values[:pos] - - return c.res -} - -type unsignedMultiShardArrayCursor struct { - cursors.UnsignedArrayCursor - cursorContext - filter *unsignedArrayFilterCursor -} - -func (c *unsignedMultiShardArrayCursor) reset(cur cursors.UnsignedArrayCursor, itrs cursors.CursorIterators, cond expression) { - if cond != nil { - if c.filter == nil { - c.filter = newUnsignedFilterArrayCursor(cond) - } - c.filter.reset(cur) - cur = c.filter - } - - c.UnsignedArrayCursor = cur - c.itrs = itrs - c.err = nil - c.count = 0 -} - -func (c *unsignedMultiShardArrayCursor) Err() error { return c.err } - -func (c *unsignedMultiShardArrayCursor) Stats() cursors.CursorStats { - return c.UnsignedArrayCursor.Stats() -} - -func (c *unsignedMultiShardArrayCursor) Next() *cursors.UnsignedArray { - for { - a := c.UnsignedArrayCursor.Next() - if a.Len() == 0 { - if c.nextArrayCursor() { - continue - } - } - c.count += int64(a.Len()) - if c.count > c.limit { - diff := c.count - c.limit - c.count -= diff - rem := int64(a.Len()) - diff - a.Timestamps = a.Timestamps[:rem] - a.Values = a.Values[:rem] - } - return a - } -} - -func (c *unsignedMultiShardArrayCursor) nextArrayCursor() bool { - if len(c.itrs) == 0 { - return false - } - - c.UnsignedArrayCursor.Close() - - var itr cursors.CursorIterator - var cur cursors.Cursor - for cur == nil && len(c.itrs) > 0 { - itr, c.itrs = c.itrs[0], c.itrs[1:] - cur, _ = itr.Next(c.ctx, c.req) - } - - var ok bool - if cur != nil { - var next cursors.UnsignedArrayCursor - next, ok = cur.(cursors.UnsignedArrayCursor) - if !ok { - cur.Close() - next = UnsignedEmptyArrayCursor - c.itrs = nil - c.err = errors.New("expected unsigned cursor") - } else { - if c.filter != nil { - c.filter.reset(next) - next = c.filter - } - } - c.UnsignedArrayCursor = next - } else { - c.UnsignedArrayCursor = UnsignedEmptyArrayCursor - } - - return ok -} - -type unsignedArraySumCursor struct { - cursors.UnsignedArrayCursor - ts [1]int64 - vs [1]uint64 - res *cursors.UnsignedArray -} - -func newUnsignedArraySumCursor(cur cursors.UnsignedArrayCursor) *unsignedArraySumCursor { - return &unsignedArraySumCursor{ - UnsignedArrayCursor: cur, - res: &cursors.UnsignedArray{}, - } -} - -func (c unsignedArraySumCursor) Stats() cursors.CursorStats { return c.UnsignedArrayCursor.Stats() } - -func (c unsignedArraySumCursor) Next() *cursors.UnsignedArray { - a := c.UnsignedArrayCursor.Next() - if len(a.Timestamps) == 0 { - return a - } - - ts := a.Timestamps[0] - var acc uint64 - - for { - for _, v := range a.Values { - acc += v - } - a = c.UnsignedArrayCursor.Next() - if len(a.Timestamps) == 0 { - c.ts[0] = ts - c.vs[0] = acc - c.res.Timestamps = c.ts[:] - c.res.Values = c.vs[:] - return c.res - } - } -} - -type integerUnsignedCountArrayCursor struct { - cursors.UnsignedArrayCursor -} - -func (c *integerUnsignedCountArrayCursor) Stats() cursors.CursorStats { - return c.UnsignedArrayCursor.Stats() -} - -func (c *integerUnsignedCountArrayCursor) Next() *cursors.IntegerArray { - a := c.UnsignedArrayCursor.Next() - if len(a.Timestamps) == 0 { - return &cursors.IntegerArray{} - } - - ts := a.Timestamps[0] - var acc int64 - for { - acc += int64(len(a.Timestamps)) - a = c.UnsignedArrayCursor.Next() - if len(a.Timestamps) == 0 { - res := cursors.NewIntegerArrayLen(1) - res.Timestamps[0] = ts - res.Values[0] = acc - return res - } - } -} - -type unsignedEmptyArrayCursor struct { - res cursors.UnsignedArray -} - -var UnsignedEmptyArrayCursor cursors.UnsignedArrayCursor = &unsignedEmptyArrayCursor{} - -func (c *unsignedEmptyArrayCursor) Err() error { return nil } -func (c *unsignedEmptyArrayCursor) Close() {} -func (c *unsignedEmptyArrayCursor) Stats() cursors.CursorStats { return cursors.CursorStats{} } -func (c *unsignedEmptyArrayCursor) Next() *cursors.UnsignedArray { return &c.res } - -// ******************** -// String Array Cursor - -type stringArrayFilterCursor struct { - cursors.StringArrayCursor - cond expression - m *singleValue - res *cursors.StringArray - tmp *cursors.StringArray -} - -func newStringFilterArrayCursor(cond expression) *stringArrayFilterCursor { - return &stringArrayFilterCursor{ - cond: cond, - m: &singleValue{}, - res: cursors.NewStringArrayLen(MaxPointsPerBlock), - tmp: &cursors.StringArray{}, - } -} - -func (c *stringArrayFilterCursor) reset(cur cursors.StringArrayCursor) { - c.StringArrayCursor = cur - c.tmp.Timestamps, c.tmp.Values = nil, nil -} - -func (c *stringArrayFilterCursor) Stats() cursors.CursorStats { return c.StringArrayCursor.Stats() } - -func (c *stringArrayFilterCursor) Next() *cursors.StringArray { - 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.StringArray - - if c.tmp.Len() > 0 { - a = c.tmp - c.tmp.Timestamps = nil - c.tmp.Values = nil - } else { - a = c.StringArrayCursor.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 - } - } - } - a = c.StringArrayCursor.Next() - } - - c.res.Timestamps = c.res.Timestamps[:pos] - c.res.Values = c.res.Values[:pos] - - return c.res -} - -type stringMultiShardArrayCursor struct { - cursors.StringArrayCursor - cursorContext - filter *stringArrayFilterCursor -} - -func (c *stringMultiShardArrayCursor) reset(cur cursors.StringArrayCursor, itrs cursors.CursorIterators, cond expression) { - if cond != nil { - if c.filter == nil { - c.filter = newStringFilterArrayCursor(cond) - } - c.filter.reset(cur) - cur = c.filter - } - - c.StringArrayCursor = cur - c.itrs = itrs - c.err = nil - c.count = 0 -} - -func (c *stringMultiShardArrayCursor) Err() error { return c.err } - -func (c *stringMultiShardArrayCursor) Stats() cursors.CursorStats { - return c.StringArrayCursor.Stats() -} - -func (c *stringMultiShardArrayCursor) Next() *cursors.StringArray { - for { - a := c.StringArrayCursor.Next() - if a.Len() == 0 { - if c.nextArrayCursor() { - continue - } - } - c.count += int64(a.Len()) - if c.count > c.limit { - diff := c.count - c.limit - c.count -= diff - rem := int64(a.Len()) - diff - a.Timestamps = a.Timestamps[:rem] - a.Values = a.Values[:rem] - } - return a - } -} - -func (c *stringMultiShardArrayCursor) nextArrayCursor() bool { - if len(c.itrs) == 0 { - return false - } - - c.StringArrayCursor.Close() - - var itr cursors.CursorIterator - var cur cursors.Cursor - for cur == nil && len(c.itrs) > 0 { - itr, c.itrs = c.itrs[0], c.itrs[1:] - cur, _ = itr.Next(c.ctx, c.req) - } - - var ok bool - if cur != nil { - var next cursors.StringArrayCursor - next, ok = cur.(cursors.StringArrayCursor) - if !ok { - cur.Close() - next = StringEmptyArrayCursor - c.itrs = nil - c.err = errors.New("expected string cursor") - } else { - if c.filter != nil { - c.filter.reset(next) - next = c.filter - } - } - c.StringArrayCursor = next - } else { - c.StringArrayCursor = StringEmptyArrayCursor - } - - return ok -} - -type integerStringCountArrayCursor struct { - cursors.StringArrayCursor -} - -func (c *integerStringCountArrayCursor) Stats() cursors.CursorStats { - return c.StringArrayCursor.Stats() -} - -func (c *integerStringCountArrayCursor) Next() *cursors.IntegerArray { - a := c.StringArrayCursor.Next() - if len(a.Timestamps) == 0 { - return &cursors.IntegerArray{} - } - - ts := a.Timestamps[0] - var acc int64 - for { - acc += int64(len(a.Timestamps)) - a = c.StringArrayCursor.Next() - if len(a.Timestamps) == 0 { - res := cursors.NewIntegerArrayLen(1) - res.Timestamps[0] = ts - res.Values[0] = acc - return res - } - } -} - -type stringEmptyArrayCursor struct { - res cursors.StringArray -} - -var StringEmptyArrayCursor cursors.StringArrayCursor = &stringEmptyArrayCursor{} - -func (c *stringEmptyArrayCursor) Err() error { return nil } -func (c *stringEmptyArrayCursor) Close() {} -func (c *stringEmptyArrayCursor) Stats() cursors.CursorStats { return cursors.CursorStats{} } -func (c *stringEmptyArrayCursor) Next() *cursors.StringArray { return &c.res } - -// ******************** -// Boolean Array Cursor - -type booleanArrayFilterCursor struct { - cursors.BooleanArrayCursor - cond expression - m *singleValue - res *cursors.BooleanArray - tmp *cursors.BooleanArray -} - -func newBooleanFilterArrayCursor(cond expression) *booleanArrayFilterCursor { - return &booleanArrayFilterCursor{ - cond: cond, - m: &singleValue{}, - res: cursors.NewBooleanArrayLen(MaxPointsPerBlock), - tmp: &cursors.BooleanArray{}, - } -} - -func (c *booleanArrayFilterCursor) reset(cur cursors.BooleanArrayCursor) { - c.BooleanArrayCursor = cur - c.tmp.Timestamps, c.tmp.Values = nil, nil -} - -func (c *booleanArrayFilterCursor) Stats() cursors.CursorStats { return c.BooleanArrayCursor.Stats() } - -func (c *booleanArrayFilterCursor) Next() *cursors.BooleanArray { - 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.BooleanArray - - if c.tmp.Len() > 0 { - a = c.tmp - c.tmp.Timestamps = nil - c.tmp.Values = nil - } else { - a = c.BooleanArrayCursor.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 - } - } - } - a = c.BooleanArrayCursor.Next() - } - - c.res.Timestamps = c.res.Timestamps[:pos] - c.res.Values = c.res.Values[:pos] - - return c.res -} - -type booleanMultiShardArrayCursor struct { - cursors.BooleanArrayCursor - cursorContext - filter *booleanArrayFilterCursor -} - -func (c *booleanMultiShardArrayCursor) reset(cur cursors.BooleanArrayCursor, itrs cursors.CursorIterators, cond expression) { - if cond != nil { - if c.filter == nil { - c.filter = newBooleanFilterArrayCursor(cond) - } - c.filter.reset(cur) - cur = c.filter - } - - c.BooleanArrayCursor = cur - c.itrs = itrs - c.err = nil - c.count = 0 -} - -func (c *booleanMultiShardArrayCursor) Err() error { return c.err } - -func (c *booleanMultiShardArrayCursor) Stats() cursors.CursorStats { - return c.BooleanArrayCursor.Stats() -} - -func (c *booleanMultiShardArrayCursor) Next() *cursors.BooleanArray { - for { - a := c.BooleanArrayCursor.Next() - if a.Len() == 0 { - if c.nextArrayCursor() { - continue - } - } - c.count += int64(a.Len()) - if c.count > c.limit { - diff := c.count - c.limit - c.count -= diff - rem := int64(a.Len()) - diff - a.Timestamps = a.Timestamps[:rem] - a.Values = a.Values[:rem] - } - return a - } -} - -func (c *booleanMultiShardArrayCursor) nextArrayCursor() bool { - if len(c.itrs) == 0 { - return false - } - - c.BooleanArrayCursor.Close() - - var itr cursors.CursorIterator - var cur cursors.Cursor - for cur == nil && len(c.itrs) > 0 { - itr, c.itrs = c.itrs[0], c.itrs[1:] - cur, _ = itr.Next(c.ctx, c.req) - } - - var ok bool - if cur != nil { - var next cursors.BooleanArrayCursor - next, ok = cur.(cursors.BooleanArrayCursor) - if !ok { - cur.Close() - next = BooleanEmptyArrayCursor - c.itrs = nil - c.err = errors.New("expected boolean cursor") - } else { - if c.filter != nil { - c.filter.reset(next) - next = c.filter - } - } - c.BooleanArrayCursor = next - } else { - c.BooleanArrayCursor = BooleanEmptyArrayCursor - } - - return ok -} - -type integerBooleanCountArrayCursor struct { - cursors.BooleanArrayCursor -} - -func (c *integerBooleanCountArrayCursor) Stats() cursors.CursorStats { - return c.BooleanArrayCursor.Stats() -} - -func (c *integerBooleanCountArrayCursor) Next() *cursors.IntegerArray { - a := c.BooleanArrayCursor.Next() - if len(a.Timestamps) == 0 { - return &cursors.IntegerArray{} - } - - ts := a.Timestamps[0] - var acc int64 - for { - acc += int64(len(a.Timestamps)) - a = c.BooleanArrayCursor.Next() - if len(a.Timestamps) == 0 { - res := cursors.NewIntegerArrayLen(1) - res.Timestamps[0] = ts - res.Values[0] = acc - return res - } - } -} - -type booleanEmptyArrayCursor struct { - res cursors.BooleanArray -} - -var BooleanEmptyArrayCursor cursors.BooleanArrayCursor = &booleanEmptyArrayCursor{} - -func (c *booleanEmptyArrayCursor) Err() error { return nil } -func (c *booleanEmptyArrayCursor) Close() {} -func (c *booleanEmptyArrayCursor) Stats() cursors.CursorStats { return cursors.CursorStats{} } -func (c *booleanEmptyArrayCursor) Next() *cursors.BooleanArray { return &c.res } diff --git a/v1/storage/reads/array_cursor.go b/v1/storage/reads/array_cursor.go deleted file mode 100644 index 7d45c57987..0000000000 --- a/v1/storage/reads/array_cursor.go +++ /dev/null @@ -1,163 +0,0 @@ -package reads - -import ( - "context" - "fmt" - - "github.com/influxdata/influxdb/v2/storage/reads/datatypes" - "github.com/influxdata/influxdb/v2/v1/tsdb/cursors" -) - -type singleValue struct { - v interface{} -} - -func (v *singleValue) Value(key string) (interface{}, bool) { - return v.v, true -} - -func newAggregateArrayCursor(ctx context.Context, agg *datatypes.Aggregate, cursor cursors.Cursor) cursors.Cursor { - if cursor == nil { - return nil - } - - switch agg.Type { - case datatypes.AggregateTypeSum: - return newSumArrayCursor(cursor) - case datatypes.AggregateTypeCount: - return newCountArrayCursor(cursor) - default: - // TODO(sgc): should be validated higher up - panic("invalid aggregate") - } -} - -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 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, - EndTime: end, - }, - } - - cc := cursorContext{ - ctx: ctx, - limit: limit, - req: &m.req, - } - - m.cursors.i.cursorContext = cc - m.cursors.f.cursorContext = cc - m.cursors.u.cursorContext = cc - m.cursors.b.cursorContext = cc - m.cursors.s.cursorContext = cc - - return m -} - -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 row.ValueCond != nil { - cond = &astExpr{row.ValueCond} - } - - 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) - } - - if cur == nil { - return nil - } - - switch c := cur.(type) { - case cursors.IntegerArrayCursor: - m.cursors.i.reset(c, row.Query, cond) - return &m.cursors.i - case cursors.FloatArrayCursor: - m.cursors.f.reset(c, row.Query, cond) - return &m.cursors.f - case cursors.UnsignedArrayCursor: - m.cursors.u.reset(c, row.Query, cond) - return &m.cursors.u - case cursors.StringArrayCursor: - m.cursors.s.reset(c, row.Query, cond) - return &m.cursors.s - case cursors.BooleanArrayCursor: - 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) -} diff --git a/v1/storage/reads/gen.go b/v1/storage/reads/gen.go deleted file mode 100644 index 8eee6fe0b5..0000000000 --- a/v1/storage/reads/gen.go +++ /dev/null @@ -1 +0,0 @@ -package reads diff --git a/v1/storage/reads/group_resultset.go b/v1/storage/reads/group_resultset.go deleted file mode 100644 index 7ad4f97671..0000000000 --- a/v1/storage/reads/group_resultset.go +++ /dev/null @@ -1,346 +0,0 @@ -package reads - -import ( - "bytes" - "context" - "fmt" - "math" - "sort" - - "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 - agg *datatypes.Aggregate - mb multiShardCursors - - i int - rows []*SeriesRow - keys [][]byte - nilSort []byte - rgc groupByCursor - km KeyMerger - - newCursorFn func() (SeriesCursor, error) - nextGroupFn func(c *groupResultSet) GroupCursor - sortFn func(c *groupResultSet) (int, error) - - eof bool -} - -type GroupOption func(g *groupResultSet) - -// GroupOptionNilSortLo configures nil values to be sorted lower than any -// other value -func GroupOptionNilSortLo() GroupOption { - return func(g *groupResultSet) { - g.nilSort = NilSortLo - } -} - -func NewGroupResultSet(ctx context.Context, req *datatypes.ReadGroupRequest, newCursorFn func() (SeriesCursor, error), opts ...GroupOption) GroupResultSet { - g := &groupResultSet{ - ctx: ctx, - req: req, - agg: req.Aggregate, - keys: make([][]byte, len(req.GroupKeys)), - nilSort: NilSortHi, - newCursorFn: newCursorFn, - } - - for _, o := range opts { - o(g) - } - - g.mb = newMultiShardArrayCursors(ctx, req.Range.Start, req.Range.End, true, math.MaxInt64) - - for i, k := range req.GroupKeys { - g.keys[i] = []byte(k) - } - - switch req.Group { - case datatypes.GroupBy: - g.sortFn = groupBySort - g.nextGroupFn = groupByNextGroup - g.rgc = groupByCursor{ - ctx: ctx, - mb: g.mb, - agg: req.Aggregate, - vals: make([][]byte, len(req.GroupKeys)), - } - - case datatypes.GroupNone: - g.sortFn = groupNoneSort - g.nextGroupFn = groupNoneNextGroup - - default: - panic("not implemented") - } - - n, err := g.sort() - if n == 0 || err != nil { - return nil - } - - return g -} - -// NilSort values determine the lexicographical order of nil values in the -// partition key -var ( - // nil sorts lowest - NilSortLo = []byte{0x00} - // nil sorts highest - NilSortHi = []byte{0xff} -) - -func (g *groupResultSet) Err() error { return nil } - -func (g *groupResultSet) Close() {} - -func (g *groupResultSet) Next() GroupCursor { - if g.eof { - return nil - } - - return g.nextGroupFn(g) -} - -func (g *groupResultSet) sort() (int, error) { - n, err := g.sortFn(g) - return n, err -} - -// seriesHasPoints reads the first block of TSM data to verify the series has points for -// the time range of the query. -func (g *groupResultSet) seriesHasPoints(row *SeriesRow) bool { - // TODO(sgc): this is expensive. Storage engine must provide efficient time range queries of series keys. - cur := g.mb.createCursor(*row) - var ts []int64 - switch c := cur.(type) { - case cursors.IntegerArrayCursor: - a := c.Next() - ts = a.Timestamps - case cursors.FloatArrayCursor: - a := c.Next() - ts = a.Timestamps - case cursors.UnsignedArrayCursor: - a := c.Next() - ts = a.Timestamps - case cursors.BooleanArrayCursor: - a := c.Next() - ts = a.Timestamps - case cursors.StringArrayCursor: - a := c.Next() - ts = a.Timestamps - case nil: - return false - default: - panic(fmt.Sprintf("unreachable: %T", c)) - } - cur.Close() - return len(ts) > 0 -} - -func groupNoneNextGroup(g *groupResultSet) GroupCursor { - cur, err := g.newCursorFn() - if err != nil { - // TODO(sgc): store error - return nil - } else if cur == nil { - return nil - } - - g.eof = true - return &groupNoneCursor{ - ctx: g.ctx, - mb: g.mb, - agg: g.agg, - cur: cur, - keys: g.km.Get(), - } -} - -func groupNoneSort(g *groupResultSet) (int, error) { - cur, err := g.newCursorFn() - if err != nil { - return 0, err - } else if cur == nil { - return 0, nil - } - - allTime := g.req.Hints.HintSchemaAllTime() - g.km.Clear() - n := 0 - row := cur.Next() - for row != nil { - if allTime || g.seriesHasPoints(row) { - n++ - g.km.MergeTagKeys(row.Tags) - } - row = cur.Next() - } - - cur.Close() - return n, nil -} - -func groupByNextGroup(g *groupResultSet) GroupCursor { - row := g.rows[g.i] - for i := range g.keys { - g.rgc.vals[i] = row.Tags.Get(g.keys[i]) - } - - g.km.Clear() - rowKey := row.SortKey - j := g.i - for j < len(g.rows) && bytes.Equal(rowKey, g.rows[j].SortKey) { - g.km.MergeTagKeys(g.rows[j].Tags) - j++ - } - - g.rgc.reset(g.rows[g.i:j]) - g.rgc.keys = g.km.Get() - - g.i = j - if j == len(g.rows) { - g.eof = true - } - - return &g.rgc -} - -func groupBySort(g *groupResultSet) (int, error) { - cur, err := g.newCursorFn() - if err != nil { - return 0, err - } else if cur == nil { - return 0, nil - } - - var rows []*SeriesRow - vals := make([][]byte, len(g.keys)) - tagsBuf := &tagsBuffer{sz: 4096} - allTime := g.req.Hints.HintSchemaAllTime() - - row := cur.Next() - for row != nil { - if allTime || g.seriesHasPoints(row) { - nr := *row - nr.SeriesTags = tagsBuf.copyTags(nr.SeriesTags) - nr.Tags = tagsBuf.copyTags(nr.Tags) - - l := len(g.keys) // for sort key separators - for i, k := range g.keys { - vals[i] = nr.Tags.Get(k) - if len(vals[i]) == 0 { - vals[i] = g.nilSort - } - l += len(vals[i]) - } - - nr.SortKey = make([]byte, 0, l) - for _, v := range vals { - nr.SortKey = append(nr.SortKey, v...) - // separate sort key values with ascii null character - nr.SortKey = append(nr.SortKey, '\000') - } - - rows = append(rows, &nr) - } - row = cur.Next() - } - - sort.Slice(rows, func(i, j int) bool { - return bytes.Compare(rows[i].SortKey, rows[j].SortKey) == -1 - }) - - g.rows = rows - - cur.Close() - return len(rows), nil -} - -type groupNoneCursor struct { - ctx context.Context - mb multiShardCursors - agg *datatypes.Aggregate - cur SeriesCursor - row SeriesRow - keys [][]byte -} - -func (c *groupNoneCursor) Err() error { return nil } -func (c *groupNoneCursor) Tags() models.Tags { return c.row.Tags } -func (c *groupNoneCursor) Keys() [][]byte { return c.keys } -func (c *groupNoneCursor) PartitionKeyVals() [][]byte { return nil } -func (c *groupNoneCursor) Close() { c.cur.Close() } -func (c *groupNoneCursor) Stats() cursors.CursorStats { return c.row.Query.Stats() } - -func (c *groupNoneCursor) Next() bool { - row := c.cur.Next() - if row == nil { - return false - } - - c.row = *row - - return true -} - -func (c *groupNoneCursor) Cursor() cursors.Cursor { - cur := c.mb.createCursor(c.row) - if c.agg != nil { - cur = c.mb.newAggregateCursor(c.ctx, c.agg, cur) - } - return cur -} - -type groupByCursor struct { - ctx context.Context - mb multiShardCursors - agg *datatypes.Aggregate - i int - rows []*SeriesRow - keys [][]byte - vals [][]byte -} - -func (c *groupByCursor) reset(rows []*SeriesRow) { - c.i = 0 - c.rows = rows -} - -func (c *groupByCursor) Err() error { return nil } -func (c *groupByCursor) Keys() [][]byte { return c.keys } -func (c *groupByCursor) PartitionKeyVals() [][]byte { return c.vals } -func (c *groupByCursor) Tags() models.Tags { return c.rows[c.i-1].Tags } -func (c *groupByCursor) Close() {} - -func (c *groupByCursor) Next() bool { - if c.i < len(c.rows) { - c.i++ - return true - } - return false -} - -func (c *groupByCursor) Cursor() cursors.Cursor { - cur := c.mb.createCursor(*c.rows[c.i-1]) - if c.agg != nil { - cur = c.mb.newAggregateCursor(c.ctx, c.agg, cur) - } - return cur -} - -func (c *groupByCursor) Stats() cursors.CursorStats { - var stats cursors.CursorStats - for _, row := range c.rows { - stats.Add(row.Query.Stats()) - } - return stats -} diff --git a/v1/storage/reads/group_resultset_test.go b/v1/storage/reads/group_resultset_test.go deleted file mode 100644 index dcb4fce830..0000000000 --- a/v1/storage/reads/group_resultset_test.go +++ /dev/null @@ -1,465 +0,0 @@ -package reads_test - -import ( - "context" - "strings" - "testing" - - "github.com/google/go-cmp/cmp" - - "github.com/influxdata/influxdb/v2/storage/reads/datatypes" - "github.com/influxdata/influxdb/v2/v1/models" - "github.com/influxdata/influxdb/v2/v1/pkg/data/gen" - "github.com/influxdata/influxdb/v2/v1/storage/reads" -) - -func TestNewGroupResultSet_Sorting(t *testing.T) { - tests := []struct { - name string - cur reads.SeriesCursor - group datatypes.ReadGroupRequest_Group - keys []string - exp string - }{ - { - name: "group by tag1 in all series", - cur: &sliceSeriesCursor{ - rows: newSeriesRows( - "cpu,tag0=val00,tag1=val10", - "cpu,tag0=val00,tag1=val11", - "cpu,tag0=val00,tag1=val12", - "cpu,tag0=val01,tag1=val10", - "cpu,tag0=val01,tag1=val11", - "cpu,tag0=val01,tag1=val12", - )}, - group: datatypes.GroupBy, - keys: []string{"tag1"}, - exp: `group: - tag key : _m,tag0,tag1 - partition key: val10 - series: _m=cpu,tag0=val00,tag1=val10 - series: _m=cpu,tag0=val01,tag1=val10 -group: - tag key : _m,tag0,tag1 - partition key: val11 - series: _m=cpu,tag0=val00,tag1=val11 - series: _m=cpu,tag0=val01,tag1=val11 -group: - tag key : _m,tag0,tag1 - partition key: val12 - series: _m=cpu,tag0=val00,tag1=val12 - series: _m=cpu,tag0=val01,tag1=val12 -`, - }, - { - name: "group by tags key collision", - cur: &sliceSeriesCursor{ - rows: newSeriesRows( - "cpu,tag0=000,tag1=111", - "cpu,tag0=00,tag1=0111", - "cpu,tag0=0,tag1=00111", - "cpu,tag0=0001,tag1=11", - "cpu,tag0=00011,tag1=1", - )}, - group: datatypes.GroupBy, - keys: []string{"tag0", "tag1"}, - exp: `group: - tag key : _m,tag0,tag1 - partition key: 0,00111 - series: _m=cpu,tag0=0,tag1=00111 -group: - tag key : _m,tag0,tag1 - partition key: 00,0111 - series: _m=cpu,tag0=00,tag1=0111 -group: - tag key : _m,tag0,tag1 - partition key: 000,111 - series: _m=cpu,tag0=000,tag1=111 -group: - tag key : _m,tag0,tag1 - partition key: 0001,11 - series: _m=cpu,tag0=0001,tag1=11 -group: - tag key : _m,tag0,tag1 - partition key: 00011,1 - series: _m=cpu,tag0=00011,tag1=1 -`, - }, - { - name: "group by tags key sort collision", - cur: &sliceSeriesCursor{ - rows: newSeriesRows( - "cpu,tag0=a,tag1=b", - "cpu,tag0=a*,tag1=b", - "cpu,tag0=a*", - )}, - group: datatypes.GroupBy, - keys: []string{"tag0", "tag1"}, - exp: `group: - tag key : _m,tag0,tag1 - partition key: a,b - series: _m=cpu,tag0=a,tag1=b -group: - tag key : _m,tag0,tag1 - partition key: a*,b - series: _m=cpu,tag0=a*,tag1=b -group: - tag key : _m,tag0 - partition key: a*, - series: _m=cpu,tag0=a* -`, - }, - { - name: "group by tags missing tag", - cur: &sliceSeriesCursor{ - rows: newSeriesRows( - "cpu,tag0=a,tag1=b", - "cpu,tag1=b", - )}, - group: datatypes.GroupBy, - keys: []string{"tag0", "tag1"}, - exp: `group: - tag key : _m,tag0,tag1 - partition key: a,b - series: _m=cpu,tag0=a,tag1=b -group: - tag key : _m,tag1 - partition key: ,b - series: _m=cpu,tag1=b -`, - }, - { - name: "group by tag1 in partial series", - cur: &sliceSeriesCursor{ - rows: newSeriesRows( - "aaa,tag0=val00", - "aaa,tag0=val01", - "cpu,tag0=val00,tag1=val10", - "cpu,tag0=val00,tag1=val11", - "cpu,tag0=val00,tag1=val12", - "cpu,tag0=val01,tag1=val10", - "cpu,tag0=val01,tag1=val11", - "cpu,tag0=val01,tag1=val12", - )}, - group: datatypes.GroupBy, - keys: []string{"tag1"}, - exp: `group: - tag key : _m,tag0,tag1 - partition key: val10 - series: _m=cpu,tag0=val00,tag1=val10 - series: _m=cpu,tag0=val01,tag1=val10 -group: - tag key : _m,tag0,tag1 - partition key: val11 - series: _m=cpu,tag0=val01,tag1=val11 - series: _m=cpu,tag0=val00,tag1=val11 -group: - tag key : _m,tag0,tag1 - partition key: val12 - series: _m=cpu,tag0=val01,tag1=val12 - series: _m=cpu,tag0=val00,tag1=val12 -group: - tag key : _m,tag0 - partition key: - series: _m=aaa,tag0=val00 - series: _m=aaa,tag0=val01 -`, - }, - { - name: "group by tag2,tag1 with partial series", - cur: &sliceSeriesCursor{ - rows: newSeriesRows( - "aaa,tag0=val00", - "aaa,tag0=val01", - "cpu,tag0=val00,tag1=val10", - "cpu,tag0=val00,tag1=val11", - "cpu,tag0=val00,tag1=val12", - "mem,tag1=val10,tag2=val20", - "mem,tag1=val11,tag2=val20", - "mem,tag1=val11,tag2=val21", - )}, - group: datatypes.GroupBy, - keys: []string{"tag2", "tag1"}, - exp: `group: - tag key : _m,tag1,tag2 - partition key: val20,val10 - series: _m=mem,tag1=val10,tag2=val20 -group: - tag key : _m,tag1,tag2 - partition key: val20,val11 - series: _m=mem,tag1=val11,tag2=val20 -group: - tag key : _m,tag1,tag2 - partition key: val21,val11 - series: _m=mem,tag1=val11,tag2=val21 -group: - tag key : _m,tag0,tag1 - partition key: ,val10 - series: _m=cpu,tag0=val00,tag1=val10 -group: - tag key : _m,tag0,tag1 - partition key: ,val11 - series: _m=cpu,tag0=val00,tag1=val11 -group: - tag key : _m,tag0,tag1 - partition key: ,val12 - series: _m=cpu,tag0=val00,tag1=val12 -group: - tag key : _m,tag0 - partition key: , - series: _m=aaa,tag0=val00 - series: _m=aaa,tag0=val01 -`, - }, - { - name: "group by tag0,tag2 with partial series", - cur: &sliceSeriesCursor{ - rows: newSeriesRows( - "aaa,tag0=val00", - "aaa,tag0=val01", - "cpu,tag0=val00,tag1=val10", - "cpu,tag0=val00,tag1=val11", - "cpu,tag0=val00,tag1=val12", - "mem,tag1=val10,tag2=val20", - "mem,tag1=val11,tag2=val20", - "mem,tag1=val11,tag2=val21", - )}, - group: datatypes.GroupBy, - keys: []string{"tag0", "tag2"}, - exp: `group: - tag key : _m,tag0,tag1 - partition key: val00, - series: _m=aaa,tag0=val00 - series: _m=cpu,tag0=val00,tag1=val10 - series: _m=cpu,tag0=val00,tag1=val11 - series: _m=cpu,tag0=val00,tag1=val12 -group: - tag key : _m,tag0 - partition key: val01, - series: _m=aaa,tag0=val01 -group: - tag key : _m,tag1,tag2 - partition key: ,val20 - series: _m=mem,tag1=val10,tag2=val20 - series: _m=mem,tag1=val11,tag2=val20 -group: - tag key : _m,tag1,tag2 - partition key: ,val21 - series: _m=mem,tag1=val11,tag2=val21 -`, - }, - } - - for _, tt := range tests { - t.Run(tt.name, func(t *testing.T) { - - newCursor := func() (reads.SeriesCursor, error) { - return tt.cur, nil - } - - var hints datatypes.HintFlags - hints.SetHintSchemaAllTime() - rs := reads.NewGroupResultSet(context.Background(), &datatypes.ReadGroupRequest{ - Group: tt.group, - GroupKeys: tt.keys, - // TODO(jlapacik): - // Hints is not used except for the tests in this file. - // Eventually this field should be removed entirely. - Hints: hints, - }, newCursor) - - sb := new(strings.Builder) - GroupResultSetToString(sb, rs, SkipNilCursor()) - - if got := sb.String(); !cmp.Equal(got, tt.exp) { - t.Errorf("unexpected value; -got/+exp\n%s", cmp.Diff(strings.Split(got, "\n"), strings.Split(tt.exp, "\n"))) - } - }) - } -} - -func TestNewGroupResultSet_GroupNone_NoDataReturnsNil(t *testing.T) { - newCursor := func() (reads.SeriesCursor, error) { - return &sliceSeriesCursor{ - rows: newSeriesRows( - "aaa,tag0=val00", - "aaa,tag0=val01", - )}, nil - } - - rs := reads.NewGroupResultSet(context.Background(), &datatypes.ReadGroupRequest{Group: datatypes.GroupNone}, newCursor) - if rs != nil { - t.Errorf("expected nil cursor") - } -} - -func TestNewGroupResultSet_GroupBy_NoDataReturnsNil(t *testing.T) { - newCursor := func() (reads.SeriesCursor, error) { - return &sliceSeriesCursor{ - rows: newSeriesRows( - "aaa,tag0=val00", - "aaa,tag0=val01", - )}, nil - } - - rs := reads.NewGroupResultSet(context.Background(), &datatypes.ReadGroupRequest{Group: datatypes.GroupBy, GroupKeys: []string{"tag0"}}, newCursor) - if rs != nil { - t.Errorf("expected nil cursor") - } -} - -func TestNewGroupResultSet_SortOrder(t *testing.T) { - tests := []struct { - name string - keys []string - opts []reads.GroupOption - exp string - }{ - { - name: "nil hi", - keys: []string{"tag0", "tag2"}, - exp: `group: - tag key : _m,tag0,tag1 - partition key: val00, - series: _m=aaa,tag0=val00 - series: _m=cpu,tag0=val00,tag1=val10 - series: _m=cpu,tag0=val00,tag1=val11 - series: _m=cpu,tag0=val00,tag1=val12 -group: - tag key : _m,tag0 - partition key: val01, - series: _m=aaa,tag0=val01 -group: - tag key : _m,tag1,tag2 - partition key: ,val20 - series: _m=mem,tag1=val10,tag2=val20 - series: _m=mem,tag1=val11,tag2=val20 -group: - tag key : _m,tag1,tag2 - partition key: ,val21 - series: _m=mem,tag1=val11,tag2=val21 -`, - }, - { - name: "nil lo", - keys: []string{"tag0", "tag2"}, - opts: []reads.GroupOption{reads.GroupOptionNilSortLo()}, - exp: `group: - tag key : _m,tag1,tag2 - partition key: ,val20 - series: _m=mem,tag1=val11,tag2=val20 - series: _m=mem,tag1=val10,tag2=val20 -group: - tag key : _m,tag1,tag2 - partition key: ,val21 - series: _m=mem,tag1=val11,tag2=val21 -group: - tag key : _m,tag0,tag1 - partition key: val00, - series: _m=cpu,tag0=val00,tag1=val10 - series: _m=cpu,tag0=val00,tag1=val11 - series: _m=cpu,tag0=val00,tag1=val12 - series: _m=aaa,tag0=val00 -group: - tag key : _m,tag0 - partition key: val01, - series: _m=aaa,tag0=val01 -`, - }, - } - for _, tt := range tests { - t.Run(tt.name, func(t *testing.T) { - newCursor := func() (reads.SeriesCursor, error) { - return &sliceSeriesCursor{ - rows: newSeriesRows( - "aaa,tag0=val00", - "aaa,tag0=val01", - "cpu,tag0=val00,tag1=val10", - "cpu,tag0=val00,tag1=val11", - "cpu,tag0=val00,tag1=val12", - "mem,tag1=val10,tag2=val20", - "mem,tag1=val11,tag2=val20", - "mem,tag1=val11,tag2=val21", - )}, nil - } - - var hints datatypes.HintFlags - hints.SetHintSchemaAllTime() - rs := reads.NewGroupResultSet(context.Background(), &datatypes.ReadGroupRequest{ - Group: datatypes.GroupBy, - GroupKeys: tt.keys, - // TODO(jlapacik): - // Hints is not used except for the tests in this file. - // Eventually this field should be removed entirely. - Hints: hints, - }, newCursor, tt.opts...) - - sb := new(strings.Builder) - GroupResultSetToString(sb, rs, SkipNilCursor()) - - if got := sb.String(); !cmp.Equal(got, tt.exp) { - t.Errorf("unexpected value; -got/+exp\n%s", cmp.Diff(strings.Split(got, "\n"), strings.Split(tt.exp, "\n"))) - } - }) - } -} - -type sliceSeriesCursor struct { - rows []reads.SeriesRow - i int -} - -func newSeriesRows(keys ...string) []reads.SeriesRow { - 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].Tags.Set([]byte("_m"), rows[i].Name) - } - return rows -} - -func (s *sliceSeriesCursor) Close() {} -func (s *sliceSeriesCursor) Err() error { return nil } - -func (s *sliceSeriesCursor) Next() *reads.SeriesRow { - if s.i < len(s.rows) { - s.i++ - return &s.rows[s.i-1] - } - return nil -} - -func BenchmarkNewGroupResultSet_GroupBy(b *testing.B) { - card := []int{10, 10, 10} - vals := make([]gen.CountableSequence, len(card)) - for i := range card { - vals[i] = gen.NewCounterByteSequenceCount(card[i]) - } - - tags := gen.NewTagsValuesSequenceValues("tag", vals) - rows := make([]reads.SeriesRow, tags.Count()) - for i := range rows { - tags.Next() - t := tags.Value().Clone() - rows[i].SeriesTags = t - rows[i].Tags = t - rows[i].Name = []byte("m0") - } - - cur := &sliceSeriesCursor{rows: rows} - newCursor := func() (reads.SeriesCursor, error) { - cur.i = 0 - return cur, nil - } - - b.ResetTimer() - b.ReportAllocs() - var hints datatypes.HintFlags - hints.SetHintSchemaAllTime() - - for i := 0; i < b.N; i++ { - rs := reads.NewGroupResultSet(context.Background(), &datatypes.ReadGroupRequest{Group: datatypes.GroupBy, GroupKeys: []string{"tag2"}, Hints: hints}, newCursor) - rs.Close() - } -} diff --git a/v1/storage/reads/influxql_eval.go b/v1/storage/reads/influxql_eval.go deleted file mode 100644 index 73679a4772..0000000000 --- a/v1/storage/reads/influxql_eval.go +++ /dev/null @@ -1,284 +0,0 @@ -package reads - -import ( - "math" - "regexp" - - "github.com/influxdata/influxql" -) - -// evalExpr evaluates expr against a map. -func evalExpr(expr influxql.Expr, m Valuer) interface{} { - if expr == nil { - return nil - } - - switch expr := expr.(type) { - case *influxql.BinaryExpr: - return evalBinaryExpr(expr, m) - case *influxql.BooleanLiteral: - return expr.Val - case *influxql.IntegerLiteral: - return expr.Val - case *influxql.UnsignedLiteral: - return expr.Val - case *influxql.NumberLiteral: - return expr.Val - case *influxql.ParenExpr: - return evalExpr(expr.Expr, m) - case *influxql.RegexLiteral: - return expr.Val - case *influxql.StringLiteral: - return expr.Val - case *influxql.VarRef: - v, _ := m.Value(expr.Val) - return v - default: - return nil - } -} - -func evalBinaryExpr(expr *influxql.BinaryExpr, m Valuer) interface{} { - lhs := evalExpr(expr.LHS, m) - rhs := evalExpr(expr.RHS, m) - if lhs == nil && rhs != nil { - // When the LHS is nil and the RHS is a boolean, implicitly cast the - // nil to false. - if _, ok := rhs.(bool); ok { - lhs = false - } - } else if lhs != nil && rhs == nil { - // Implicit cast of the RHS nil to false when the LHS is a boolean. - if _, ok := lhs.(bool); ok { - rhs = false - } - } - - // Evaluate if both sides are simple types. - switch lhs := lhs.(type) { - case bool: - rhs, ok := rhs.(bool) - switch expr.Op { - case influxql.AND: - return ok && (lhs && rhs) - case influxql.OR: - return ok && (lhs || rhs) - case influxql.BITWISE_AND: - return ok && (lhs && rhs) - case influxql.BITWISE_OR: - return ok && (lhs || rhs) - case influxql.BITWISE_XOR: - return ok && (lhs != rhs) - case influxql.EQ: - return ok && (lhs == rhs) - case influxql.NEQ: - return ok && (lhs != rhs) - } - case float64: - // Try the rhs as a float64 or int64 - rhsf, ok := rhs.(float64) - if !ok { - var rhsi int64 - if rhsi, ok = rhs.(int64); ok { - rhsf = float64(rhsi) - } - } - - rhs := rhsf - switch expr.Op { - case influxql.EQ: - return ok && (lhs == rhs) - case influxql.NEQ: - return ok && (lhs != rhs) - case influxql.LT: - return ok && (lhs < rhs) - case influxql.LTE: - return ok && (lhs <= rhs) - case influxql.GT: - return ok && (lhs > rhs) - case influxql.GTE: - return ok && (lhs >= rhs) - case influxql.ADD: - if !ok { - return nil - } - return lhs + rhs - case influxql.SUB: - if !ok { - return nil - } - return lhs - rhs - case influxql.MUL: - if !ok { - return nil - } - return lhs * rhs - case influxql.DIV: - if !ok { - return nil - } else if rhs == 0 { - return float64(0) - } - return lhs / rhs - case influxql.MOD: - if !ok { - return nil - } - return math.Mod(lhs, rhs) - } - case int64: - // Try as a float64 to see if a float cast is required. - rhsf, ok := rhs.(float64) - if ok { - lhs := float64(lhs) - rhs := rhsf - switch expr.Op { - case influxql.EQ: - return lhs == rhs - case influxql.NEQ: - return lhs != rhs - case influxql.LT: - return lhs < rhs - case influxql.LTE: - return lhs <= rhs - case influxql.GT: - return lhs > rhs - case influxql.GTE: - return lhs >= rhs - case influxql.ADD: - return lhs + rhs - case influxql.SUB: - return lhs - rhs - case influxql.MUL: - return lhs * rhs - case influxql.DIV: - if rhs == 0 { - return float64(0) - } - return lhs / rhs - case influxql.MOD: - return math.Mod(lhs, rhs) - } - } else { - rhs, ok := rhs.(int64) - switch expr.Op { - case influxql.EQ: - return ok && (lhs == rhs) - case influxql.NEQ: - return ok && (lhs != rhs) - case influxql.LT: - return ok && (lhs < rhs) - case influxql.LTE: - return ok && (lhs <= rhs) - case influxql.GT: - return ok && (lhs > rhs) - case influxql.GTE: - return ok && (lhs >= rhs) - case influxql.ADD: - if !ok { - return nil - } - return lhs + rhs - case influxql.SUB: - if !ok { - return nil - } - return lhs - rhs - case influxql.MUL: - if !ok { - return nil - } - return lhs * rhs - case influxql.DIV: - if !ok { - return nil - } else if rhs == 0 { - return float64(0) - } - return lhs / rhs - case influxql.MOD: - if !ok { - return nil - } else if rhs == 0 { - return int64(0) - } - return lhs % rhs - case influxql.BITWISE_AND: - if !ok { - return nil - } - return lhs & rhs - case influxql.BITWISE_OR: - if !ok { - return nil - } - return lhs | rhs - case influxql.BITWISE_XOR: - if !ok { - return nil - } - return lhs ^ rhs - } - } - case string: - switch expr.Op { - case influxql.EQ: - rhs, ok := rhs.(string) - if !ok { - return nil - } - return lhs == rhs - case influxql.NEQ: - rhs, ok := rhs.(string) - if !ok { - return nil - } - return lhs != rhs - case influxql.EQREGEX: - rhs, ok := rhs.(*regexp.Regexp) - if !ok { - return nil - } - return rhs.MatchString(lhs) - case influxql.NEQREGEX: - rhs, ok := rhs.(*regexp.Regexp) - if !ok { - return nil - } - return !rhs.MatchString(lhs) - } - case []byte: - switch expr.Op { - case influxql.EQ: - rhs, ok := rhs.(string) - if !ok { - return nil - } - return string(lhs) == rhs - case influxql.NEQ: - rhs, ok := rhs.(string) - if !ok { - return nil - } - return string(lhs) != rhs - case influxql.EQREGEX: - rhs, ok := rhs.(*regexp.Regexp) - if !ok { - return nil - } - return rhs.Match(lhs) - case influxql.NEQREGEX: - rhs, ok := rhs.(*regexp.Regexp) - if !ok { - return nil - } - return !rhs.Match(lhs) - } - } - return nil -} - -func EvalExprBool(expr influxql.Expr, m Valuer) bool { - v, _ := evalExpr(expr, m).(bool) - return v -} diff --git a/v1/storage/reads/influxql_expr.go b/v1/storage/reads/influxql_expr.go deleted file mode 100644 index 941d0e141c..0000000000 --- a/v1/storage/reads/influxql_expr.go +++ /dev/null @@ -1,25 +0,0 @@ -package reads - -import ( - "github.com/influxdata/influxql" -) - -// TODO(sgc): build expression evaluator that does not use influxql AST - -type expression interface { - EvalBool(v Valuer) bool -} - -type astExpr struct { - expr influxql.Expr -} - -func (e *astExpr) EvalBool(v Valuer) bool { - return EvalExprBool(e.expr, v) -} - -// Valuer is the interface that wraps the Value() method. -type Valuer interface { - // Value returns the value and existence flag for a given key. - Value(key string) (interface{}, bool) -} diff --git a/v1/storage/reads/influxql_predicate.go b/v1/storage/reads/influxql_predicate.go deleted file mode 100644 index f7f4e49bbf..0000000000 --- a/v1/storage/reads/influxql_predicate.go +++ /dev/null @@ -1,274 +0,0 @@ -package reads - -import ( - "regexp" - - "github.com/influxdata/influxdb/v2/storage/reads/datatypes" - "github.com/influxdata/influxql" - "github.com/pkg/errors" -) - -const ( - fieldRef = "$" -) - -// NodeToExpr transforms a predicate node to an influxql.Expr. -func NodeToExpr(node *datatypes.Node, remap map[string]string) (influxql.Expr, error) { - v := &nodeToExprVisitor{remap: remap} - WalkNode(v, node) - if err := v.Err(); err != nil { - return nil, err - } - - if len(v.exprs) > 1 { - return nil, errors.New("invalid expression") - } - - if len(v.exprs) == 0 { - return nil, nil - } - - // TODO(edd): It would be preferable if RewriteRegexConditions was a - // package level function in influxql. - stmt := &influxql.SelectStatement{ - Condition: v.exprs[0], - } - stmt.RewriteRegexConditions() - return stmt.Condition, nil -} - -type nodeToExprVisitor struct { - remap map[string]string - exprs []influxql.Expr - err error -} - -func (v *nodeToExprVisitor) Visit(n *datatypes.Node) NodeVisitor { - if v.err != nil { - return nil - } - - switch n.NodeType { - case datatypes.NodeTypeLogicalExpression: - if len(n.Children) > 1 { - op := influxql.AND - if n.GetLogical() == datatypes.LogicalOr { - op = influxql.OR - } - - WalkNode(v, n.Children[0]) - if v.err != nil { - return nil - } - - for i := 1; i < len(n.Children); i++ { - WalkNode(v, n.Children[i]) - if v.err != nil { - return nil - } - - if len(v.exprs) >= 2 { - lhs, rhs := v.pop2() - v.exprs = append(v.exprs, &influxql.BinaryExpr{LHS: lhs, Op: op, RHS: rhs}) - } - } - - return nil - } - - case datatypes.NodeTypeParenExpression: - if len(n.Children) != 1 { - v.err = errors.New("parenExpression expects one child") - return nil - } - - WalkNode(v, n.Children[0]) - if v.err != nil { - return nil - } - - if len(v.exprs) > 0 { - v.exprs = append(v.exprs, &influxql.ParenExpr{Expr: v.pop()}) - } - - return nil - - case datatypes.NodeTypeComparisonExpression: - WalkChildren(v, n) - - if len(v.exprs) < 2 { - v.err = errors.New("comparisonExpression expects two children") - return nil - } - - lhs, rhs := v.pop2() - - be := &influxql.BinaryExpr{LHS: lhs, RHS: rhs} - switch n.GetComparison() { - case datatypes.ComparisonEqual: - be.Op = influxql.EQ - case datatypes.ComparisonNotEqual: - be.Op = influxql.NEQ - case datatypes.ComparisonStartsWith: - // TODO(sgc): rewrite to anchored RE, as index does not support startsWith yet - v.err = errors.New("startsWith not implemented") - return nil - case datatypes.ComparisonRegex: - be.Op = influxql.EQREGEX - case datatypes.ComparisonNotRegex: - be.Op = influxql.NEQREGEX - case datatypes.ComparisonLess: - be.Op = influxql.LT - case datatypes.ComparisonLessEqual: - be.Op = influxql.LTE - case datatypes.ComparisonGreater: - be.Op = influxql.GT - case datatypes.ComparisonGreaterEqual: - be.Op = influxql.GTE - default: - v.err = errors.New("invalid comparison operator") - return nil - } - - v.exprs = append(v.exprs, be) - - return nil - - case datatypes.NodeTypeTagRef: - ref := n.GetTagRefValue() - if v.remap != nil { - if nk, ok := v.remap[ref]; ok { - ref = nk - } - } - - v.exprs = append(v.exprs, &influxql.VarRef{Val: ref, Type: influxql.Tag}) - return nil - - case datatypes.NodeTypeFieldRef: - v.exprs = append(v.exprs, &influxql.VarRef{Val: fieldRef}) - return nil - - case datatypes.NodeTypeLiteral: - switch val := n.Value.(type) { - case *datatypes.Node_StringValue: - v.exprs = append(v.exprs, &influxql.StringLiteral{Val: val.StringValue}) - - case *datatypes.Node_RegexValue: - // TODO(sgc): consider hashing the RegexValue and cache compiled version - re, err := regexp.Compile(val.RegexValue) - if err != nil { - v.err = err - } - v.exprs = append(v.exprs, &influxql.RegexLiteral{Val: re}) - return nil - - case *datatypes.Node_IntegerValue: - v.exprs = append(v.exprs, &influxql.IntegerLiteral{Val: val.IntegerValue}) - - case *datatypes.Node_UnsignedValue: - v.exprs = append(v.exprs, &influxql.UnsignedLiteral{Val: val.UnsignedValue}) - - case *datatypes.Node_FloatValue: - v.exprs = append(v.exprs, &influxql.NumberLiteral{Val: val.FloatValue}) - - case *datatypes.Node_BooleanValue: - v.exprs = append(v.exprs, &influxql.BooleanLiteral{Val: val.BooleanValue}) - - default: - v.err = errors.New("unexpected literal type") - return nil - } - - return nil - - default: - return v - } - return nil -} - -func (v *nodeToExprVisitor) Err() error { - return v.err -} - -func (v *nodeToExprVisitor) pop() influxql.Expr { - if len(v.exprs) == 0 { - panic("stack empty") - } - - var top influxql.Expr - top, v.exprs = v.exprs[len(v.exprs)-1], v.exprs[:len(v.exprs)-1] - return top -} - -func (v *nodeToExprVisitor) pop2() (influxql.Expr, influxql.Expr) { - if len(v.exprs) < 2 { - panic("stack empty") - } - - rhs := v.exprs[len(v.exprs)-1] - lhs := v.exprs[len(v.exprs)-2] - v.exprs = v.exprs[:len(v.exprs)-2] - return lhs, rhs -} - -func IsTrueBooleanLiteral(expr influxql.Expr) bool { - b, ok := expr.(*influxql.BooleanLiteral) - if ok { - return b.Val - } - return false -} - -func RewriteExprRemoveFieldValue(expr influxql.Expr) influxql.Expr { - return influxql.RewriteExpr(expr, func(expr influxql.Expr) influxql.Expr { - if be, ok := expr.(*influxql.BinaryExpr); ok { - if ref, ok := be.LHS.(*influxql.VarRef); ok { - if ref.Val == fieldRef { - return &influxql.BooleanLiteral{Val: true} - } - } - } - - return expr - }) -} - -type hasRefs struct { - refs []string - found []bool -} - -func (v *hasRefs) allFound() bool { - for _, val := range v.found { - if !val { - return false - } - } - return true -} - -func (v *hasRefs) Visit(node influxql.Node) influxql.Visitor { - if v.allFound() { - return nil - } - - if n, ok := node.(*influxql.VarRef); ok { - for i, r := range v.refs { - if !v.found[i] && r == n.Val { - v.found[i] = true - if v.allFound() { - return nil - } - } - } - } - return v -} - -func HasFieldValueKey(expr influxql.Expr) bool { - refs := hasRefs{refs: []string{fieldRef}, found: make([]bool, 1)} - influxql.Walk(&refs, expr) - return refs.found[0] -} diff --git a/v1/storage/reads/influxql_predicate_test.go b/v1/storage/reads/influxql_predicate_test.go deleted file mode 100644 index c034f3b958..0000000000 --- a/v1/storage/reads/influxql_predicate_test.go +++ /dev/null @@ -1,92 +0,0 @@ -package reads_test - -import ( - "testing" - - "github.com/influxdata/influxdb/v2/storage/reads/datatypes" - "github.com/influxdata/influxdb/v2/v1/storage/reads" -) - -func TestHasFieldValueKey(t *testing.T) { - predicates := []*datatypes.Node{ - { - NodeType: datatypes.NodeTypeComparisonExpression, - Value: &datatypes.Node_Comparison_{ - Comparison: datatypes.ComparisonLess, - }, - Children: []*datatypes.Node{ - { - NodeType: datatypes.NodeTypeFieldRef, - Value: &datatypes.Node_FieldRefValue{ - FieldRefValue: "_value", - }, - }, - { - NodeType: datatypes.NodeTypeLiteral, - Value: &datatypes.Node_IntegerValue{ - IntegerValue: 3000, - }, - }, - }, - }, - { - NodeType: datatypes.NodeTypeLogicalExpression, - Value: &datatypes.Node_Logical_{ - Logical: datatypes.LogicalAnd, - }, - Children: []*datatypes.Node{ - { - NodeType: datatypes.NodeTypeComparisonExpression, - Value: &datatypes.Node_Comparison_{ - Comparison: datatypes.ComparisonEqual, - }, - Children: []*datatypes.Node{ - { - NodeType: datatypes.NodeTypeTagRef, - Value: &datatypes.Node_TagRefValue{ - TagRefValue: "_measurement", - }, - }, - { - NodeType: datatypes.NodeTypeLiteral, - Value: &datatypes.Node_StringValue{ - StringValue: "cpu", - }, - }, - }, - }, - { - NodeType: datatypes.NodeTypeComparisonExpression, - Value: &datatypes.Node_Comparison_{ - Comparison: datatypes.ComparisonLess, - }, - Children: []*datatypes.Node{ - { - NodeType: datatypes.NodeTypeFieldRef, - Value: &datatypes.Node_FieldRefValue{ - FieldRefValue: "_value", - }, - }, - { - NodeType: datatypes.NodeTypeLiteral, - Value: &datatypes.Node_IntegerValue{ - IntegerValue: 3000, - }, - }, - }, - }, - }, - }, - } - for _, predicate := range predicates { - t.Run("", func(t *testing.T) { - expr, err := reads.NodeToExpr(predicate, nil) - if err != nil { - t.Fatalf("unexpected error converting predicate to InfluxQL expression: %v", err) - } - if !reads.HasFieldValueKey(expr) { - t.Fatalf("did not find a field reference in %v", expr) - } - }) - } -} diff --git a/v1/storage/reads/keymerger.go b/v1/storage/reads/keymerger.go deleted file mode 100644 index 2ce4f07cd5..0000000000 --- a/v1/storage/reads/keymerger.go +++ /dev/null @@ -1,109 +0,0 @@ -package reads - -import ( - "bytes" - "strings" - - "github.com/influxdata/influxdb/v2/v1/models" -) - -// tagsKeyMerger is responsible for determining a merged set of tag keys -type KeyMerger struct { - i int - tmp [][]byte - keys [2][][]byte -} - -func (km *KeyMerger) Clear() { - km.i = 0 - km.keys[0] = km.keys[0][:0] - if km.tmp != nil { - tmp := km.tmp[:cap(km.tmp)] - for i := range tmp { - tmp[i] = nil - } - } -} - -func (km *KeyMerger) Get() [][]byte { return km.keys[km.i&1] } - -func (km *KeyMerger) String() string { - var s []string - for _, k := range km.Get() { - s = append(s, string(k)) - } - return strings.Join(s, ",") -} - -func (km *KeyMerger) MergeTagKeys(tags models.Tags) { - if cap(km.tmp) < len(tags) { - km.tmp = make([][]byte, len(tags)) - } else { - km.tmp = km.tmp[:len(tags)] - } - - for i := range tags { - km.tmp[i] = tags[i].Key - } - - km.MergeKeys(km.tmp) -} - -func (km *KeyMerger) MergeKeys(in [][]byte) { - keys := km.keys[km.i&1] - i, j := 0, 0 - for i < len(keys) && j < len(in) && bytes.Equal(keys[i], in[j]) { - i++ - j++ - } - - if j == len(in) { - // no new tags - return - } - - km.i = (km.i + 1) & 1 - l := len(keys) + len(in) - if cap(km.keys[km.i]) < l { - km.keys[km.i] = make([][]byte, l) - } else { - km.keys[km.i] = km.keys[km.i][:l] - } - - keya := km.keys[km.i] - - // back up the pointers - if i > 0 { - i-- - j-- - } - - k := i - copy(keya[:k], keys[:k]) - - for i < len(keys) && j < len(in) { - cmp := bytes.Compare(keys[i], in[j]) - if cmp < 0 { - keya[k] = keys[i] - i++ - } else if cmp > 0 { - keya[k] = in[j] - j++ - } else { - keya[k] = keys[i] - i++ - j++ - } - k++ - } - - if i < len(keys) { - k += copy(keya[k:], keys[i:]) - } - - if j < len(in) { - k += copy(keya[k:], in[j:]) - } - - km.keys[km.i] = keya[:k] -} diff --git a/v1/storage/reads/keymerger_test.go b/v1/storage/reads/keymerger_test.go deleted file mode 100644 index 00bc60269e..0000000000 --- a/v1/storage/reads/keymerger_test.go +++ /dev/null @@ -1,204 +0,0 @@ -package reads - -import ( - "bytes" - "math/rand" - "strconv" - "testing" - - "github.com/google/go-cmp/cmp" - "github.com/influxdata/influxdb/v2/v1/models" -) - -func TestKeyMerger_MergeTagKeys(t *testing.T) { - tests := []struct { - name string - tags []models.Tags - exp string - }{ - { - name: "mixed", - tags: []models.Tags{ - models.ParseTags([]byte("foo,tag0=v0,tag1=v0,tag2=v0")), - models.ParseTags([]byte("foo,tag0=v0,tag1=v0,tag2=v1")), - models.ParseTags([]byte("foo,tag0=v0")), - models.ParseTags([]byte("foo,tag0=v0,tag3=v0")), - }, - exp: "tag0,tag1,tag2,tag3", - }, - { - name: "mixed 2", - tags: []models.Tags{ - models.ParseTags([]byte("foo,tag0=v0")), - models.ParseTags([]byte("foo,tag0=v0,tag3=v0")), - models.ParseTags([]byte("foo,tag0=v0,tag1=v0,tag2=v0")), - models.ParseTags([]byte("foo,tag0=v0,tag1=v0,tag2=v1")), - }, - exp: "tag0,tag1,tag2,tag3", - }, - { - name: "all different", - tags: []models.Tags{ - models.ParseTags([]byte("foo,tag0=v0")), - models.ParseTags([]byte("foo,tag1=v0")), - models.ParseTags([]byte("foo,tag2=v1")), - models.ParseTags([]byte("foo,tag3=v0")), - }, - exp: "tag0,tag1,tag2,tag3", - }, - { - name: "new tags,verify clear", - tags: []models.Tags{ - models.ParseTags([]byte("foo,tag9=v0")), - models.ParseTags([]byte("foo,tag8=v0")), - }, - exp: "tag8,tag9", - }, - } - - var km KeyMerger - for _, tt := range tests { - t.Run(tt.name, func(t *testing.T) { - km.Clear() - for _, tags := range tt.tags { - km.MergeTagKeys(tags) - } - - if got := km.String(); !cmp.Equal(got, tt.exp) { - t.Errorf("unexpected keys -got/+exp\n%s", cmp.Diff(got, tt.exp)) - } - }) - } -} - -var commaB = []byte(",") - -func TestKeyMerger_MergeKeys(t *testing.T) { - - tests := []struct { - name string - keys [][][]byte - exp string - }{ - { - name: "mixed", - keys: [][][]byte{ - bytes.Split([]byte("tag0,tag1,tag2"), commaB), - bytes.Split([]byte("tag0,tag1,tag2"), commaB), - bytes.Split([]byte("tag0"), commaB), - bytes.Split([]byte("tag0,tag3"), commaB), - }, - exp: "tag0,tag1,tag2,tag3", - }, - { - name: "mixed 2", - keys: [][][]byte{ - bytes.Split([]byte("tag0"), commaB), - bytes.Split([]byte("tag0,tag3"), commaB), - bytes.Split([]byte("tag0,tag1,tag2"), commaB), - bytes.Split([]byte("tag0,tag1,tag2"), commaB), - }, - exp: "tag0,tag1,tag2,tag3", - }, - { - name: "all different", - keys: [][][]byte{ - bytes.Split([]byte("tag0"), commaB), - bytes.Split([]byte("tag3"), commaB), - bytes.Split([]byte("tag1"), commaB), - bytes.Split([]byte("tag2"), commaB), - }, - exp: "tag0,tag1,tag2,tag3", - }, - { - name: "new tags,verify clear", - keys: [][][]byte{ - bytes.Split([]byte("tag9"), commaB), - bytes.Split([]byte("tag8"), commaB), - }, - exp: "tag8,tag9", - }, - } - - var km KeyMerger - for _, tt := range tests { - t.Run(tt.name, func(t *testing.T) { - km.Clear() - for _, keys := range tt.keys { - km.MergeKeys(keys) - } - - if got := km.String(); !cmp.Equal(got, tt.exp) { - t.Errorf("unexpected keys -got/+exp\n%s", cmp.Diff(got, tt.exp)) - } - }) - } -} - -func BenchmarkKeyMerger_MergeKeys(b *testing.B) { - keys := [][][]byte{ - bytes.Split([]byte("tag00,tag01,tag02"), commaB), - bytes.Split([]byte("tag00,tag01,tag02"), commaB), - bytes.Split([]byte("tag00,tag01,tag05,tag06,tag10,tag11,tag12,tag13,tag14,tag15"), commaB), - bytes.Split([]byte("tag00"), commaB), - bytes.Split([]byte("tag00,tag03"), commaB), - bytes.Split([]byte("tag01,tag03,tag13,tag14,tag15"), commaB), - bytes.Split([]byte("tag04,tag05"), commaB), - } - - rand.Seed(20040409) - - tests := []int{ - 10, - 1000, - 1000000, - } - - for _, n := range tests { - b.Run(strconv.Itoa(n), func(b *testing.B) { - b.ResetTimer() - - var km KeyMerger - for i := 0; i < b.N; i++ { - for j := 0; j < n; j++ { - km.MergeKeys(keys[rand.Int()%len(keys)]) - } - km.Clear() - } - }) - } -} - -func BenchmarkKeyMerger_MergeTagKeys(b *testing.B) { - tags := []models.Tags{ - models.ParseTags([]byte("foo,tag00=v0,tag01=v0,tag02=v0")), - models.ParseTags([]byte("foo,tag00=v0,tag01=v0,tag02=v0")), - models.ParseTags([]byte("foo,tag00=v0,tag01=v0,tag05=v0,tag06=v0,tag10=v0,tag11=v0,tag12=v0,tag13=v0,tag14=v0,tag15=v0")), - models.ParseTags([]byte("foo,tag00=v0")), - models.ParseTags([]byte("foo,tag00=v0,tag03=v0")), - models.ParseTags([]byte("foo,tag01=v0,tag03=v0,tag13=v0,tag14=v0,tag15=v0")), - models.ParseTags([]byte("foo,tag04=v0,tag05=v0")), - } - - rand.Seed(20040409) - - tests := []int{ - 10, - 1000, - 1000000, - } - - for _, n := range tests { - b.Run(strconv.Itoa(n), func(b *testing.B) { - b.ResetTimer() - - var km KeyMerger - for i := 0; i < b.N; i++ { - for j := 0; j < n; j++ { - km.MergeTagKeys(tags[rand.Int()%len(tags)]) - } - km.Clear() - } - }) - } -} diff --git a/v1/storage/reads/predicate.go b/v1/storage/reads/predicate.go deleted file mode 100644 index a94f4777ce..0000000000 --- a/v1/storage/reads/predicate.go +++ /dev/null @@ -1,142 +0,0 @@ -package reads - -import ( - "bytes" - "strconv" - - "github.com/influxdata/influxdb/v2/storage/reads/datatypes" -) - -// NodeVisitor can be called by Walk to traverse the Node hierarchy. -// The Visit() function is called once per node. -type NodeVisitor interface { - Visit(*datatypes.Node) NodeVisitor -} - -func WalkChildren(v NodeVisitor, node *datatypes.Node) { - for _, n := range node.Children { - WalkNode(v, n) - } -} - -func WalkNode(v NodeVisitor, node *datatypes.Node) { - if v = v.Visit(node); v == nil { - return - } - - WalkChildren(v, node) -} - -func PredicateToExprString(p *datatypes.Predicate) string { - if p == nil { - return "[none]" - } - - var v predicateExpressionPrinter - WalkNode(&v, p.Root) - return v.Buffer.String() -} - -type predicateExpressionPrinter struct { - bytes.Buffer -} - -func (v *predicateExpressionPrinter) Visit(n *datatypes.Node) NodeVisitor { - switch n.NodeType { - case datatypes.NodeTypeLogicalExpression: - if len(n.Children) > 0 { - var op string - if n.GetLogical() == datatypes.LogicalAnd { - op = " AND " - } else { - op = " OR " - } - WalkNode(v, n.Children[0]) - for _, e := range n.Children[1:] { - v.Buffer.WriteString(op) - WalkNode(v, e) - } - } - - return nil - - case datatypes.NodeTypeParenExpression: - if len(n.Children) == 1 { - v.Buffer.WriteString("( ") - WalkNode(v, n.Children[0]) - v.Buffer.WriteString(" )") - } - - return nil - - case datatypes.NodeTypeComparisonExpression: - WalkNode(v, n.Children[0]) - v.Buffer.WriteByte(' ') - switch n.GetComparison() { - case datatypes.ComparisonEqual: - v.Buffer.WriteByte('=') - case datatypes.ComparisonNotEqual: - v.Buffer.WriteString("!=") - case datatypes.ComparisonStartsWith: - v.Buffer.WriteString("startsWith") - case datatypes.ComparisonRegex: - v.Buffer.WriteString("=~") - case datatypes.ComparisonNotRegex: - v.Buffer.WriteString("!~") - case datatypes.ComparisonLess: - v.Buffer.WriteByte('<') - case datatypes.ComparisonLessEqual: - v.Buffer.WriteString("<=") - case datatypes.ComparisonGreater: - v.Buffer.WriteByte('>') - case datatypes.ComparisonGreaterEqual: - v.Buffer.WriteString(">=") - } - - v.Buffer.WriteByte(' ') - WalkNode(v, n.Children[1]) - return nil - - case datatypes.NodeTypeTagRef: - v.Buffer.WriteByte('\'') - v.Buffer.WriteString(n.GetTagRefValue()) - v.Buffer.WriteByte('\'') - return nil - - case datatypes.NodeTypeFieldRef: - v.Buffer.WriteByte('$') - return nil - - case datatypes.NodeTypeLiteral: - switch val := n.Value.(type) { - case *datatypes.Node_StringValue: - v.Buffer.WriteString(strconv.Quote(val.StringValue)) - - case *datatypes.Node_RegexValue: - v.Buffer.WriteByte('/') - v.Buffer.WriteString(val.RegexValue) - v.Buffer.WriteByte('/') - - case *datatypes.Node_IntegerValue: - v.Buffer.WriteString(strconv.FormatInt(val.IntegerValue, 10)) - - case *datatypes.Node_UnsignedValue: - v.Buffer.WriteString(strconv.FormatUint(val.UnsignedValue, 10)) - - case *datatypes.Node_FloatValue: - v.Buffer.WriteString(strconv.FormatFloat(val.FloatValue, 'f', 10, 64)) - - case *datatypes.Node_BooleanValue: - if val.BooleanValue { - v.Buffer.WriteString("true") - } else { - v.Buffer.WriteString("false") - } - } - - return nil - - default: - return v - } -} diff --git a/v1/storage/reads/predicate_test.go b/v1/storage/reads/predicate_test.go deleted file mode 100644 index 1e59f88a85..0000000000 --- a/v1/storage/reads/predicate_test.go +++ /dev/null @@ -1,58 +0,0 @@ -package reads_test - -import ( - "testing" - - "github.com/influxdata/influxdb/v2/storage/reads/datatypes" - "github.com/influxdata/influxdb/v2/v1/storage/reads" -) - -func TestPredicateToExprString(t *testing.T) { - cases := []struct { - n string - r *datatypes.Predicate - e string - }{ - { - n: "returns [none] for nil", - r: nil, - e: "[none]", - }, - { - n: "logical AND", - r: &datatypes.Predicate{ - Root: &datatypes.Node{ - NodeType: datatypes.NodeTypeLogicalExpression, - Value: &datatypes.Node_Logical_{Logical: datatypes.LogicalAnd}, - Children: []*datatypes.Node{ - { - NodeType: datatypes.NodeTypeComparisonExpression, - Value: &datatypes.Node_Comparison_{Comparison: datatypes.ComparisonEqual}, - Children: []*datatypes.Node{ - {NodeType: datatypes.NodeTypeTagRef, Value: &datatypes.Node_TagRefValue{TagRefValue: "host"}}, - {NodeType: datatypes.NodeTypeLiteral, Value: &datatypes.Node_StringValue{StringValue: "host1"}}, - }, - }, - { - NodeType: datatypes.NodeTypeComparisonExpression, - Value: &datatypes.Node_Comparison_{Comparison: datatypes.ComparisonRegex}, - Children: []*datatypes.Node{ - {NodeType: datatypes.NodeTypeTagRef, Value: &datatypes.Node_TagRefValue{TagRefValue: "region"}}, - {NodeType: datatypes.NodeTypeLiteral, Value: &datatypes.Node_RegexValue{RegexValue: "^us-west"}}, - }, - }, - }, - }, - }, - e: `'host' = "host1" AND 'region' =~ /^us-west/`, - }, - } - - for _, tc := range cases { - t.Run(tc.n, func(t *testing.T) { - if got, wanted := reads.PredicateToExprString(tc.r), tc.e; got != wanted { - t.Fatal("got:", got, "wanted:", wanted) - } - }) - } -} diff --git a/v1/storage/reads/resultset.go b/v1/storage/reads/resultset.go deleted file mode 100644 index 40be129698..0000000000 --- a/v1/storage/reads/resultset.go +++ /dev/null @@ -1,74 +0,0 @@ -package reads - -import ( - "context" - "math" - - "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 - cur SeriesCursor - row SeriesRow - mb multiShardCursors -} - -func NewFilteredResultSet(ctx context.Context, req *datatypes.ReadFilterRequest, cur SeriesCursor) ResultSet { - return &resultSet{ - ctx: ctx, - cur: cur, - mb: newMultiShardArrayCursors(ctx, req.Range.Start, req.Range.End, true, math.MaxInt64), - } -} - -func (r *resultSet) Err() error { return nil } - -// Close closes the result set. Close is idempotent. -func (r *resultSet) Close() { - if r == nil { - return // Nothing to do. - } - r.row.Query = nil - r.cur.Close() -} - -// Next returns true if there are more results available. -func (r *resultSet) Next() bool { - if r == nil { - return false - } - - row := r.cur.Next() - if row == nil { - return false - } - - r.row = *row - - return true -} - -func (r *resultSet) Cursor() cursors.Cursor { - cur := r.mb.createCursor(r.row) - if r.agg != nil { - cur = r.mb.newAggregateCursor(r.ctx, r.agg, cur) - } - return cur -} - -func (r *resultSet) Tags() models.Tags { - return r.row.Tags -} - -// Stats returns the stats for the underlying cursors. -// Available after resultset has been scanned. -func (r *resultSet) Stats() cursors.CursorStats { return r.row.Query.Stats() } diff --git a/v1/storage/reads/resultset_lineprotocol.go b/v1/storage/reads/resultset_lineprotocol.go deleted file mode 100644 index 97ccbc61a0..0000000000 --- a/v1/storage/reads/resultset_lineprotocol.go +++ /dev/null @@ -1,129 +0,0 @@ -package reads - -import ( - "errors" - "io" - "strconv" - - "github.com/influxdata/influxdb/v2/v1/models" - "github.com/influxdata/influxdb/v2/v1/tsdb/cursors" -) - -// ResultSetToLineProtocol transforms rs to line protocol and writes the -// output to wr. -func ResultSetToLineProtocol(wr io.Writer, rs ResultSet) (err error) { - defer rs.Close() - - line := make([]byte, 0, 4096) - for rs.Next() { - tags := rs.Tags() - name := tags.Get(models.MeasurementTagKeyBytes) - field := tags.Get(models.FieldKeyTagKeyBytes) - if len(name) == 0 || len(field) == 0 { - return errors.New("missing measurement / field") - } - - line = append(line[:0], name...) - if tags.Len() > 2 { - tags = tags[1 : len(tags)-1] // take first and last elements which are measurement and field keys - line = tags.AppendHashKey(line) - } - - line = append(line, ' ') - line = append(line, field...) - line = append(line, '=') - err = cursorToLineProtocol(wr, line, rs.Cursor()) - if err != nil { - return err - } - } - - return rs.Err() -} - -func cursorToLineProtocol(wr io.Writer, line []byte, cur cursors.Cursor) error { - var newLine = []byte{'\n'} - - switch ccur := cur.(type) { - case cursors.IntegerArrayCursor: - for { - a := ccur.Next() - if a.Len() > 0 { - for i := range a.Timestamps { - buf := strconv.AppendInt(line, a.Values[i], 10) - buf = append(buf, 'i', ' ') - buf = strconv.AppendInt(buf, a.Timestamps[i], 10) - wr.Write(buf) - wr.Write(newLine) - } - } else { - break - } - } - case cursors.FloatArrayCursor: - for { - a := ccur.Next() - if a.Len() > 0 { - for i := range a.Timestamps { - buf := strconv.AppendFloat(line, a.Values[i], 'f', -1, 64) - buf = append(buf, ' ') - buf = strconv.AppendInt(buf, a.Timestamps[i], 10) - wr.Write(buf) - wr.Write(newLine) - } - } else { - break - } - } - case cursors.UnsignedArrayCursor: - for { - a := ccur.Next() - if a.Len() > 0 { - for i := range a.Timestamps { - buf := strconv.AppendUint(line, a.Values[i], 10) - buf = append(buf, 'u', ' ') - buf = strconv.AppendInt(buf, a.Timestamps[i], 10) - wr.Write(buf) - wr.Write(newLine) - } - } else { - break - } - } - case cursors.BooleanArrayCursor: - for { - a := ccur.Next() - if a.Len() > 0 { - for i := range a.Timestamps { - buf := strconv.AppendBool(line, a.Values[i]) - buf = append(buf, ' ') - buf = strconv.AppendInt(buf, a.Timestamps[i], 10) - wr.Write(buf) - wr.Write(newLine) - } - } else { - break - } - } - case cursors.StringArrayCursor: - for { - a := ccur.Next() - if a.Len() > 0 { - for i := range a.Timestamps { - buf := strconv.AppendQuote(line, a.Values[i]) - buf = append(buf, 'i', ' ') - buf = strconv.AppendInt(buf, a.Timestamps[i], 10) - wr.Write(buf) - wr.Write(newLine) - } - } else { - break - } - } - default: - panic("unreachable") - } - - cur.Close() - return cur.Err() -} diff --git a/v1/storage/reads/series_cursor.go b/v1/storage/reads/series_cursor.go deleted file mode 100644 index 1d691d5e7e..0000000000 --- a/v1/storage/reads/series_cursor.go +++ /dev/null @@ -1,51 +0,0 @@ -package reads - -import ( - "context" - - "github.com/influxdata/influxdb/v2/v1/models" - "github.com/influxdata/influxdb/v2/v1/tsdb/cursors" - "github.com/influxdata/influxql" -) - -type SeriesCursor interface { - Close() - Next() *SeriesRow - Err() error -} - -type SeriesRow struct { - SortKey []byte - Name []byte // measurement name - SeriesTags models.Tags // unmodified series tags - Tags models.Tags - Field string - Query cursors.CursorIterators - ValueCond influxql.Expr -} - -type limitSeriesCursor struct { - SeriesCursor - n, o, c int64 -} - -func NewLimitSeriesCursor(ctx context.Context, cur SeriesCursor, n, o int64) SeriesCursor { - return &limitSeriesCursor{SeriesCursor: cur, o: o, n: n} -} - -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 - } - - if c.c >= c.n { - return nil - } - c.c++ - return c.SeriesCursor.Next() -} diff --git a/v1/storage/reads/series_cursor_test.go b/v1/storage/reads/series_cursor_test.go deleted file mode 100644 index 8eee6fe0b5..0000000000 --- a/v1/storage/reads/series_cursor_test.go +++ /dev/null @@ -1 +0,0 @@ -package reads diff --git a/v1/storage/reads/store.go b/v1/storage/reads/store.go deleted file mode 100644 index adf9650be8..0000000000 --- a/v1/storage/reads/store.go +++ /dev/null @@ -1,84 +0,0 @@ -package reads - -import ( - "context" - - "github.com/gogo/protobuf/proto" - "github.com/influxdata/influxdb/v2/storage/reads/datatypes" - "github.com/influxdata/influxdb/v2/v1/models" - "github.com/influxdata/influxdb/v2/v1/tsdb/cursors" -) - -type ResultSet interface { - // Next advances the ResultSet to the next cursor. It returns false - // when there are no more cursors. - Next() bool - - // Cursor returns the most recent cursor after a call to Next. - Cursor() cursors.Cursor - - // Tags returns the tags for the most recent cursor after a call to Next. - Tags() models.Tags - - // Close releases any resources allocated by the ResultSet. - Close() - - // Err returns the first error encountered by the ResultSet. - Err() error - - Stats() cursors.CursorStats -} - -type GroupResultSet interface { - // Next advances the GroupResultSet and returns the next GroupCursor. It - // returns nil if there are no more groups. - Next() GroupCursor - - // Close releases any resources allocated by the GroupResultSet. - Close() - - // Err returns the first error encountered by the GroupResultSet. - Err() error -} - -type GroupCursor interface { - // Next advances to the next cursor. Next will return false when there are no - // more cursors in the current group. - Next() bool - - // Cursor returns the most recent cursor after a call to Next. - Cursor() cursors.Cursor - - // Tags returns the tags for the most recent cursor after a call to Next. - Tags() models.Tags - - // Keys returns the union of all tag key names for all series produced by - // this GroupCursor. - Keys() [][]byte - - // PartitionKeyVals returns the values of all tags identified by the - // keys specified in ReadRequest#GroupKeys. The tag values values will - // appear in the same order as the GroupKeys. - // - // When the datatypes.GroupNone strategy is specified, PartitionKeyVals will - // be nil. - PartitionKeyVals() [][]byte - - // Close releases any resources allocated by the GroupCursor. - Close() - - // Err returns the first error encountered by the GroupCursor. - Err() error - - Stats() cursors.CursorStats -} - -type Store interface { - ReadFilter(ctx context.Context, req *datatypes.ReadFilterRequest) (ResultSet, error) - ReadGroup(ctx context.Context, req *datatypes.ReadGroupRequest) (GroupResultSet, error) - - TagKeys(ctx context.Context, req *datatypes.TagKeysRequest) (cursors.StringIterator, error) - TagValues(ctx context.Context, req *datatypes.TagValuesRequest) (cursors.StringIterator, error) - - GetSource(db, rp string) proto.Message -} diff --git a/v1/storage/reads/store_test.go b/v1/storage/reads/store_test.go deleted file mode 100644 index 8d639757ea..0000000000 --- a/v1/storage/reads/store_test.go +++ /dev/null @@ -1,219 +0,0 @@ -package reads_test - -import ( - "bytes" - "fmt" - "io" - "strings" - - "github.com/influxdata/influxdb/v2/v1/models" - "github.com/influxdata/influxdb/v2/v1/storage/reads" - "github.com/influxdata/influxdb/v2/v1/tsdb/cursors" -) - -func cursorToString(wr io.Writer, cur cursors.Cursor) { - switch ccur := cur.(type) { - case cursors.IntegerArrayCursor: - fmt.Fprintln(wr, "Integer") - for { - a := ccur.Next() - if a.Len() > 0 { - for i := range a.Timestamps { - fmt.Fprintf(wr, "%20d | %20d\n", a.Timestamps[i], a.Values[i]) - } - } else { - break - } - } - case cursors.FloatArrayCursor: - fmt.Fprintln(wr, "Float") - for { - a := ccur.Next() - if a.Len() > 0 { - for i := range a.Timestamps { - fmt.Fprintf(wr, "%20d | %18.2f\n", a.Timestamps[i], a.Values[i]) - } - } else { - break - } - } - case cursors.UnsignedArrayCursor: - fmt.Fprintln(wr, "Unsigned") - for { - a := ccur.Next() - if a.Len() > 0 { - for i := range a.Timestamps { - fmt.Fprintf(wr, "%20d | %20d\n", a.Timestamps[i], a.Values[i]) - } - } else { - break - } - } - case cursors.BooleanArrayCursor: - fmt.Fprintln(wr, "Boolean") - for { - a := ccur.Next() - if a.Len() > 0 { - for i := range a.Timestamps { - fmt.Fprintf(wr, "%20d | %t\n", a.Timestamps[i], a.Values[i]) - } - } else { - break - } - } - case cursors.StringArrayCursor: - fmt.Fprintln(wr, "String") - for { - a := ccur.Next() - if a.Len() > 0 { - for i := range a.Timestamps { - fmt.Fprintf(wr, "%20d | %20s\n", a.Timestamps[i], a.Values[i]) - } - } else { - break - } - } - default: - fmt.Fprintln(wr, "Invalid") - fmt.Fprintf(wr, "unreachable: %T\n", cur) - } - - if err := cur.Err(); err != nil && err != io.EOF { - fmt.Fprintf(wr, "cursor err: %s\n", cur.Err().Error()) - } - - cur.Close() -} - -const nilVal = "" - -func joinString(b [][]byte) string { - s := make([]string, len(b)) - for i := range b { - v := b[i] - if len(v) == 0 { - s[i] = nilVal - } else { - s[i] = string(v) - } - } - return strings.Join(s, ",") -} - -func tagsToString(wr io.Writer, tags models.Tags, opts ...optionFn) { - if k := tags.HashKey(); len(k) > 0 { - fmt.Fprintf(wr, "%s", string(k[1:])) - } - fmt.Fprintln(wr) -} - -func resultSetToString(wr io.Writer, rs reads.ResultSet, opts ...optionFn) { - var po PrintOptions - for _, o := range opts { - o(&po) - } - - iw := ensureIndentWriter(wr) - wr = iw - - for rs.Next() { - fmt.Fprint(wr, "series: ") - tagsToString(wr, rs.Tags()) - cur := rs.Cursor() - - if po.SkipNilCursor && cur == nil { - continue - } - - iw.Indent(2) - - fmt.Fprint(wr, "cursor:") - if cur == nil { - fmt.Fprintln(wr, nilVal) - goto LOOP - } - - cursorToString(wr, cur) - LOOP: - iw.Indent(-2) - } -} - -func GroupResultSetToString(wr io.Writer, rs reads.GroupResultSet, opts ...optionFn) { - iw := ensureIndentWriter(wr) - wr = iw - - gc := rs.Next() - for gc != nil { - fmt.Fprintln(wr, "group:") - iw.Indent(2) - fmt.Fprintf(wr, "tag key : %s\n", joinString(gc.Keys())) - fmt.Fprintf(wr, "partition key: %s\n", joinString(gc.PartitionKeyVals())) - iw.Indent(2) - resultSetToString(wr, gc, opts...) - iw.Indent(-4) - gc = rs.Next() - } -} - -type PrintOptions struct { - SkipNilCursor bool -} - -type optionFn func(o *PrintOptions) - -func SkipNilCursor() optionFn { - return func(o *PrintOptions) { - o.SkipNilCursor = true - } -} - -type indentWriter struct { - l int - p []byte - wr io.Writer - bol bool -} - -func ensureIndentWriter(wr io.Writer) *indentWriter { - if iw, ok := wr.(*indentWriter); ok { - return iw - } else { - return newIndentWriter(wr) - } -} - -func newIndentWriter(wr io.Writer) *indentWriter { - return &indentWriter{ - wr: wr, - bol: true, - } -} - -func (w *indentWriter) Indent(n int) { - w.l += n - if w.l < 0 { - panic("negative indent") - } - w.p = bytes.Repeat([]byte(" "), w.l) -} - -func (w *indentWriter) Write(p []byte) (n int, err error) { - for _, c := range p { - if w.bol { - _, err = w.wr.Write(w.p) - if err != nil { - break - } - w.bol = false - } - _, err = w.wr.Write([]byte{c}) - if err != nil { - break - } - n++ - w.bol = c == '\n' - } - - return n, err -} diff --git a/v1/storage/reads/tagsbuffer.go b/v1/storage/reads/tagsbuffer.go deleted file mode 100644 index 1195960ab2..0000000000 --- a/v1/storage/reads/tagsbuffer.go +++ /dev/null @@ -1,30 +0,0 @@ -package reads - -import ( - "github.com/influxdata/influxdb/v2/v1/models" -) - -type tagsBuffer struct { - sz int - i int - buf models.Tags -} - -func (tb *tagsBuffer) copyTags(src models.Tags) models.Tags { - var buf models.Tags - if len(src) > tb.sz { - buf = make(models.Tags, len(src)) - } else { - if tb.i+len(src) > len(tb.buf) { - tb.buf = make(models.Tags, tb.sz) - tb.i = 0 - } - - buf = tb.buf[tb.i : tb.i+len(src)] - tb.i += len(src) - } - - copy(buf, src) - - return buf -} diff --git a/v1/storage/reads/viewer.go b/v1/storage/reads/viewer.go deleted file mode 100644 index 8eee6fe0b5..0000000000 --- a/v1/storage/reads/viewer.go +++ /dev/null @@ -1 +0,0 @@ -package reads diff --git a/v1/tsdb/engine.go b/v1/tsdb/engine.go index a19561d240..42e44c136d 100644 --- a/v1/tsdb/engine.go +++ b/v1/tsdb/engine.go @@ -31,7 +31,7 @@ var ( // Engine represents a swappable storage engine for the shard. type Engine interface { - Open(context.Context) error + Open() error Close() error SetEnabled(enabled bool) SetCompactionsEnabled(enabled bool) diff --git a/v1/tsdb/engine/tsm1/cache.go b/v1/tsdb/engine/tsm1/cache.go index 972ebedfab..a3e56dde32 100644 --- a/v1/tsdb/engine/tsm1/cache.go +++ b/v1/tsdb/engine/tsm1/cache.go @@ -796,18 +796,29 @@ func (c *Cache) updateMemSize(b int64) { atomic.AddInt64(&c.stats.MemSizeBytes, b) } +const ( + valueTypeUndefined = 0 + valueTypeFloat64 = 1 + valueTypeInteger = 2 + valueTypeString = 3 + valueTypeBoolean = 4 + valueTypeUnsigned = 5 +) + func valueType(v Value) byte { switch v.(type) { case FloatValue: - return 1 + return valueTypeFloat64 case IntegerValue: - return 2 + return valueTypeInteger case StringValue: - return 3 + return valueTypeString case BooleanValue: - return 4 + return valueTypeBoolean + case UnsignedValue: + return valueTypeUnsigned default: - return 0 + return valueTypeUndefined } } diff --git a/v1/tsdb/engine/tsm1/encoding.go b/v1/tsdb/engine/tsm1/encoding.go index c5176ba9fe..ed42b222d1 100644 --- a/v1/tsdb/engine/tsm1/encoding.go +++ b/v1/tsdb/engine/tsm1/encoding.go @@ -344,6 +344,8 @@ func (v FloatValue) String() string { return fmt.Sprintf("%v %v", time.Unix(0, v.unixnano), v.value) } +func (v FloatValue) RawValue() float64 { return v.value } + func encodeFloatBlock(buf []byte, values []Value) ([]byte, error) { if len(values) == 0 { return nil, nil @@ -479,6 +481,8 @@ func (v BooleanValue) String() string { return fmt.Sprintf("%v %v", time.Unix(0, v.unixnano), v.Value()) } +func (v BooleanValue) RawValue() bool { return v.value } + func encodeBooleanBlock(buf []byte, values []Value) ([]byte, error) { if len(values) == 0 { return nil, nil @@ -606,6 +610,8 @@ func (v IntegerValue) String() string { return fmt.Sprintf("%v %v", time.Unix(0, v.unixnano), v.Value()) } +func (v IntegerValue) RawValue() int64 { return v.value } + func encodeIntegerBlock(buf []byte, values []Value) ([]byte, error) { tenc := getTimeEncoder(len(values)) venc := getIntegerEncoder(len(values)) @@ -725,6 +731,8 @@ func (v UnsignedValue) String() string { return fmt.Sprintf("%v %v", time.Unix(0, v.unixnano), v.Value()) } +func (v UnsignedValue) RawValue() uint64 { return v.value } + func encodeUnsignedBlock(buf []byte, values []Value) ([]byte, error) { tenc := getTimeEncoder(len(values)) venc := getUnsignedEncoder(len(values)) @@ -844,6 +852,8 @@ func (v StringValue) String() string { return fmt.Sprintf("%v %v", time.Unix(0, v.unixnano), v.Value()) } +func (v StringValue) RawValue() string { return v.value } + func encodeStringBlock(buf []byte, values []Value) ([]byte, error) { tenc := getTimeEncoder(len(values)) venc := getStringEncoder(len(values) * len(values[0].(StringValue).value)) diff --git a/v1/tsdb/engine/tsm1/engine.go b/v1/tsdb/engine/tsm1/engine.go index a64dc37af2..f3aee4e83a 100644 --- a/v1/tsdb/engine/tsm1/engine.go +++ b/v1/tsdb/engine/tsm1/engine.go @@ -718,7 +718,7 @@ func (e *Engine) DiskSize() int64 { // Open opens and initializes the engine. // TODO(edd): plumb context -func (e *Engine) Open(context.Context) error { +func (e *Engine) Open() error { if err := os.MkdirAll(e.path, 0777); err != nil { return err } diff --git a/v1/tsdb/engine/tsm1/predicate.go b/v1/tsdb/engine/tsm1/predicate.go new file mode 100644 index 0000000000..428ba77f97 --- /dev/null +++ b/v1/tsdb/engine/tsm1/predicate.go @@ -0,0 +1,691 @@ +package tsm1 + +import ( + "bytes" + "fmt" + "regexp" + + "github.com/influxdata/influxdb/v2" + "github.com/influxdata/influxdb/v2/storage/reads/datatypes" +) + +// Predicate is something that can match on a series key. +type Predicate interface { + Clone() influxdb.Predicate + Matches(key []byte) bool + Marshal() ([]byte, error) +} + +const ( // Enumeration of all predicate versions we support unmarshalling. + predicateVersionZero = '\x00' +) + +// UnmarshalPredicate takes stored predicate bytes from a Marshal call and returns a Predicate. +func UnmarshalPredicate(data []byte) (Predicate, error) { + if len(data) == 0 { + return nil, nil + } else if data[0] != predicateVersionZero { + return nil, fmt.Errorf("unknown tag byte: %x", data[0]) + } + + pred := new(datatypes.Predicate) + if err := pred.Unmarshal(data[1:]); err != nil { + return nil, err + } + return NewProtobufPredicate(pred) +} + +// +// Design +// + +// Predicates lazily evaluate with memoization so that we can walk a series key +// by the tags without parsing them into a structure and allocating. Each node +// in a predicate tree keeps a cache if it has enough information to have a +// definite value. The predicate state keeps track of all of the tag key/value +// pairs passed to it, and has a reset function to start over for a new series key. +// +// For example, imagine a query like +// +// ("tag1" == "val1" AND "tag2" == "val2") OR "tag3" == "val3" +// +// The state would have tag values set on it like +// +// state.Set("tag1", "val1") => NeedMore +// state.Set("tag2", "not-val2") => NeedMore +// state.Set("tag3", "val3") => True +// +// where after the first Set, the AND and OR clauses are both NeedMore, after +// the second Set, the AND clause is False and the OR clause is NeedMore, and +// after the last Set, the AND clause is still False, and the OR clause is True. +// +// Fast resetting is achieved by having each cache maintain a pointer to the state +// and both having a generation number. When the state resets, it bumps the generation +// number, and when the value is set in the cache, it is set with the current generation +// of the state. When querying the cache, it checks if the generation still matches. + +// +// Protobuf Implementation +// + +// NewProtobufPredicate returns a Predicate that matches based on the comparison structure +// described by the incoming protobuf. +func NewProtobufPredicate(pred *datatypes.Predicate) (Predicate, error) { + // Walk the predicate to collect the tag refs + locs := make(map[string]int) + walkPredicateNodes(pred.Root, func(node *datatypes.Node) { + if node.GetNodeType() == datatypes.NodeTypeTagRef { + switch value := node.GetValue().(type) { + case *datatypes.Node_TagRefValue: + // Only add to the matcher locations the first time we encounter + // the tag key reference. This prevents problems with redundant + // predicates like: + // + // foo = a AND foo = b + // foo = c AND foo = d + if _, ok := locs[value.TagRefValue]; !ok { + locs[value.TagRefValue] = len(locs) + } + } + } + }) + + // Construct the shared state and root predicate node. + state := newPredicateState(locs) + root, err := buildPredicateNode(state, pred.Root) + if err != nil { + return nil, err + } + + return &predicateMatcher{ + pred: pred, + state: state, + root: root, + }, nil +} + +// predicateMatcher implements Predicate for a protobuf. +type predicateMatcher struct { + pred *datatypes.Predicate + state *predicateState + root predicateNode +} + +// Clone returns a deep copy of p's state and root node. +// +// It is not safe to modify p.pred on the returned clone. +func (p *predicateMatcher) Clone() influxdb.Predicate { + state := p.state.Clone() + return &predicateMatcher{ + pred: p.pred, + state: state, + root: p.root.Clone(state), + } +} + +// Matches checks if the key matches the predicate by feeding individual tags into the +// state and returning as soon as the root node has a definite answer. +func (p *predicateMatcher) Matches(key []byte) bool { + p.state.Reset() + + // Extract the series from the composite key + key, _ = SeriesAndFieldFromCompositeKey(key) + + // Determine which popping algorithm to use. If there are no escape characters + // we can use the quicker method that only works in that case. + popTag := predicatePopTag + if bytes.IndexByte(key, '\\') != -1 { + popTag = predicatePopTagEscape + } + + // Feed tag pairs into the state and update until we have a definite response. + var tag, value []byte + for len(key) > 0 { + tag, value, key = popTag(key) + if tag == nil || !p.state.Set(tag, value) { + continue + } + resp := p.root.Update() + if resp == predicateResponse_true { + return true + } else if resp == predicateResponse_false { + return false + } + } + + // If it always needed more then it didn't match. For example, consider if + // the predicate matches `tag1=val1` but tag1 is not present in the key. + return false +} + +// Marshal returns a buffer representing the protobuf predicate. +func (p *predicateMatcher) Marshal() ([]byte, error) { + // Prefix it with the version byte so that we can change in the future if necessary + buf := make([]byte, 1+p.pred.Size()) + buf[0] = predicateVersionZero + _, err := p.pred.MarshalTo(buf[1:]) + return buf, err +} + +// walkPredicateNodes recursively calls the function for each node. +func walkPredicateNodes(node *datatypes.Node, fn func(node *datatypes.Node)) { + fn(node) + for _, ch := range node.Children { + walkPredicateNodes(ch, fn) + } +} + +// buildPredicateNode takes a protobuf node and converts it into a predicateNode. It is strict +// in what it accepts. +func buildPredicateNode(state *predicateState, node *datatypes.Node) (predicateNode, error) { + switch node.GetNodeType() { + case datatypes.NodeTypeComparisonExpression: + children := node.GetChildren() + if len(children) != 2 { + return nil, fmt.Errorf("invalid number of children for logical expression: %v", len(children)) + } + left, right := children[0], children[1] + + comp := &predicateNodeComparison{ + predicateCache: newPredicateCache(state), + comp: node.GetComparison(), + } + + // Fill in the left side of the comparison + switch left.GetNodeType() { + // Tag refs look up the location of the tag in the state + case datatypes.NodeTypeTagRef: + idx, ok := state.locs[left.GetTagRefValue()] + if !ok { + return nil, fmt.Errorf("invalid tag ref in comparison: %v", left.GetTagRefValue()) + } + comp.leftIndex = idx + + // Left literals are only allowed to be strings + case datatypes.NodeTypeLiteral: + lit, ok := left.GetValue().(*datatypes.Node_StringValue) + if !ok { + return nil, fmt.Errorf("invalid left literal in comparison: %v", left.GetValue()) + } + comp.leftLiteral = []byte(lit.StringValue) + + default: + return nil, fmt.Errorf("invalid left node in comparison: %v", left.GetNodeType()) + } + + // Fill in the right side of the comparison + switch right.GetNodeType() { + // Tag refs look up the location of the tag in the state + case datatypes.NodeTypeTagRef: + idx, ok := state.locs[right.GetTagRefValue()] + if !ok { + return nil, fmt.Errorf("invalid tag ref in comparison: %v", right.GetTagRefValue()) + } + comp.rightIndex = idx + + // Right literals are allowed to be regexes as well as strings + case datatypes.NodeTypeLiteral: + switch lit := right.GetValue().(type) { + case *datatypes.Node_StringValue: + comp.rightLiteral = []byte(lit.StringValue) + + case *datatypes.Node_RegexValue: + reg, err := regexp.Compile(lit.RegexValue) + if err != nil { + return nil, err + } + comp.rightReg = reg + + default: + return nil, fmt.Errorf("invalid right literal in comparison: %v", right.GetValue()) + } + + default: + return nil, fmt.Errorf("invalid right node in comparison: %v", right.GetNodeType()) + } + + // Ensure that a regex is set on the right if and only if the comparison is a regex + if comp.rightReg == nil { + if comp.comp == datatypes.ComparisonRegex || comp.comp == datatypes.ComparisonNotRegex { + return nil, fmt.Errorf("invalid comparison involving regex: %v", node) + } + } else { + if comp.comp != datatypes.ComparisonRegex && comp.comp != datatypes.ComparisonNotRegex { + return nil, fmt.Errorf("invalid comparison not against regex: %v", node) + } + } + + return comp, nil + + case datatypes.NodeTypeLogicalExpression: + children := node.GetChildren() + if len(children) != 2 { + return nil, fmt.Errorf("invalid number of children for logical expression: %v", len(children)) + } + + left, err := buildPredicateNode(state, children[0]) + if err != nil { + return nil, err + } + right, err := buildPredicateNode(state, children[1]) + if err != nil { + return nil, err + } + + switch node.GetLogical() { + case datatypes.LogicalAnd: + return &predicateNodeAnd{ + predicateCache: newPredicateCache(state), + left: left, + right: right, + }, nil + + case datatypes.LogicalOr: + return &predicateNodeOr{ + predicateCache: newPredicateCache(state), + left: left, + right: right, + }, nil + + default: + return nil, fmt.Errorf("unknown logical type: %v", node.GetLogical()) + } + + default: + return nil, fmt.Errorf("unsupported predicate type: %v", node.GetNodeType()) + } +} + +// +// Predicate Responses +// + +type predicateResponse uint8 + +const ( + predicateResponse_needMore predicateResponse = iota + predicateResponse_true + predicateResponse_false +) + +// +// Predicate State +// + +// predicateState keeps track of tag key=>value mappings with cheap methods +// to reset to a blank state. +type predicateState struct { + gen uint64 + locs map[string]int + values [][]byte +} + +// newPredicateState creates a predicateState given a map of keys to indexes into an +// an array. +func newPredicateState(locs map[string]int) *predicateState { + return &predicateState{ + gen: 1, // so that caches start out unfilled since they start at 0 + locs: locs, + values: make([][]byte, len(locs)), + } +} + +// Clone returns a deep copy of p. +func (p *predicateState) Clone() *predicateState { + q := &predicateState{ + gen: p.gen, + locs: make(map[string]int, len(p.locs)), + values: make([][]byte, len(p.values)), + } + + for k, v := range p.locs { + q.locs[k] = v + } + copy(q.values, p.values) + + return q +} + +// Reset clears any set values for the state. +func (p *predicateState) Reset() { + p.gen++ + + for i := range p.values { + p.values[i] = nil + } +} + +// Set sets the key to be the value and returns true if the key is part of the considered +// set of keys. +func (p *predicateState) Set(key, value []byte) bool { + i, ok := p.locs[string(key)] + if ok { + p.values[i] = value + } + return ok +} + +// +// Predicate Cache +// + +// predicateCache interacts with the predicateState to keep determined responses +// memoized until the state has been Reset to avoid recomputing nodes. +type predicateCache struct { + state *predicateState + gen uint64 + resp predicateResponse +} + +// newPredicateCache constructs a predicateCache for the provided state. +func newPredicateCache(state *predicateState) predicateCache { + return predicateCache{ + state: state, + gen: 0, + resp: predicateResponse_needMore, + } +} + +// Clone returns a deep copy of p. +func (p *predicateCache) Clone(state *predicateState) *predicateCache { + if state == nil { + state = p.state.Clone() + } + return &predicateCache{ + state: state, + gen: p.gen, + resp: p.resp, + } +} + +// Cached returns the cached response and a boolean indicating if it is valid. +func (p *predicateCache) Cached() (predicateResponse, bool) { + return p.resp, p.gen == p.state.gen +} + +// Store sets the cache to the provided response until the state is Reset. +func (p *predicateCache) Store(resp predicateResponse) { + p.gen = p.state.gen + p.resp = resp +} + +// +// Predicate Nodes +// + +// predicateNode is the interface that any parts of a predicate tree implement. +type predicateNode interface { + // Update informs the node that the state has been updated and asks it to return + // a response. + Update() predicateResponse + + // Clone returns a deep copy of the node. + Clone(state *predicateState) predicateNode +} + +// predicateNodeAnd combines two predicate nodes with an And. +type predicateNodeAnd struct { + predicateCache + left, right predicateNode +} + +// Clone returns a deep copy of p. +func (p *predicateNodeAnd) Clone(state *predicateState) predicateNode { + return &predicateNodeAnd{ + predicateCache: *p.predicateCache.Clone(state), + left: p.left.Clone(state), + right: p.right.Clone(state), + } +} + +// Update checks if both of the left and right nodes are true. If either is false +// then the node is definitely false. Otherwise, it needs more information. +func (p *predicateNodeAnd) Update() predicateResponse { + if resp, ok := p.Cached(); ok { + return resp + } + + left := p.left.Update() + if left == predicateResponse_false { + p.Store(predicateResponse_false) + return predicateResponse_false + } else if left == predicateResponse_needMore { + return predicateResponse_needMore + } + + right := p.right.Update() + if right == predicateResponse_false { + p.Store(predicateResponse_false) + return predicateResponse_false + } else if right == predicateResponse_needMore { + return predicateResponse_needMore + } + + return predicateResponse_true +} + +// predicateNodeOr combines two predicate nodes with an Or. +type predicateNodeOr struct { + predicateCache + left, right predicateNode +} + +// Clone returns a deep copy of p. +func (p *predicateNodeOr) Clone(state *predicateState) predicateNode { + return &predicateNodeOr{ + predicateCache: *p.predicateCache.Clone(state), + left: p.left.Clone(state), + right: p.right.Clone(state), + } +} + +// Update checks if either the left and right nodes are true. If both nodes +// are false, then the node is definitely fasle. Otherwise, it needs more information. +func (p *predicateNodeOr) Update() predicateResponse { + if resp, ok := p.Cached(); ok { + return resp + } + + left := p.left.Update() + if left == predicateResponse_true { + p.Store(predicateResponse_true) + return predicateResponse_true + } + + right := p.right.Update() + if right == predicateResponse_true { + p.Store(predicateResponse_true) + return predicateResponse_true + } + + if left == predicateResponse_false && right == predicateResponse_false { + p.Store(predicateResponse_false) + return predicateResponse_false + } + + return predicateResponse_needMore +} + +// predicateNodeComparison compares values of tags. +type predicateNodeComparison struct { + predicateCache + comp datatypes.Node_Comparison + rightReg *regexp.Regexp + leftLiteral []byte + rightLiteral []byte + leftIndex int + rightIndex int +} + +// Clone returns a deep copy of p. +func (p *predicateNodeComparison) Clone(state *predicateState) predicateNode { + q := &predicateNodeComparison{ + predicateCache: *p.predicateCache.Clone(state), + comp: p.comp, + rightReg: p.rightReg, + leftIndex: p.leftIndex, + rightIndex: p.rightIndex, + } + + if p.leftLiteral != nil { + q.leftLiteral = make([]byte, len(p.leftLiteral)) + copy(q.leftLiteral, p.leftLiteral) + } + if p.rightLiteral != nil { + q.rightLiteral = make([]byte, len(p.rightLiteral)) + copy(q.rightLiteral, p.rightLiteral) + } + return q +} + +// Update checks if both sides of the comparison are determined, and if so, evaluates +// the comparison to a determined truth value. +func (p *predicateNodeComparison) Update() predicateResponse { + if resp, ok := p.Cached(); ok { + return resp + } + + left := p.leftLiteral + if left == nil { + left = p.state.values[p.leftIndex] + if left == nil { + return predicateResponse_needMore + } + } + + right := p.rightLiteral + if right == nil && p.rightReg == nil { + right = p.state.values[p.rightIndex] + if right == nil { + return predicateResponse_needMore + } + } + + if predicateEval(p.comp, left, right, p.rightReg) { + p.Store(predicateResponse_true) + return predicateResponse_true + } else { + p.Store(predicateResponse_false) + return predicateResponse_false + } +} + +// predicateEval is a helper to do the appropriate comparison depending on which comparison +// enumeration value was passed. +func predicateEval(comp datatypes.Node_Comparison, left, right []byte, rightReg *regexp.Regexp) bool { + switch comp { + case datatypes.ComparisonEqual: + return string(left) == string(right) + case datatypes.ComparisonNotEqual: + return string(left) != string(right) + case datatypes.ComparisonStartsWith: + return bytes.HasPrefix(left, right) + case datatypes.ComparisonLess: + return string(left) < string(right) + case datatypes.ComparisonLessEqual: + return string(left) <= string(right) + case datatypes.ComparisonGreater: + return string(left) > string(right) + case datatypes.ComparisonGreaterEqual: + return string(left) >= string(right) + case datatypes.ComparisonRegex: + return rightReg.Match(left) + case datatypes.ComparisonNotRegex: + return !rightReg.Match(left) + } + return false +} + +// +// Popping Tags +// + +// The models package has some of this logic as well, but doesn't export ways to get +// at individual tags one at a time. In the common, no escape characters case, popping +// the first tag off of a series key takes around ~10ns. + +// predicatePopTag pops a tag=value pair from the front of series, returning the +// remainder in rest. it assumes there are no escaped characters in the series. +func predicatePopTag(series []byte) (tag, value []byte, rest []byte) { + // find the first ',' + i := bytes.IndexByte(series, ',') + if i >= 0 && i < len(series) { + series, rest = series[:i], series[i+1:] + } + + // find the first '=' + j := bytes.IndexByte(series, '=') + if j >= 0 && j < len(series) { + tag, value = series[:j], series[j+1:] + } + + return tag, value, rest +} + +// predicatePopTagEscape pops a tag=value pair from the front of series, returning the +// remainder in rest. it assumes there are possibly/likely escaped characters in the series. +func predicatePopTagEscape(series []byte) (tag, value []byte, rest []byte) { + // find the first unescaped ',' + for j := uint(0); j < uint(len(series)); { + i := bytes.IndexByte(series[j:], ',') + if i < 0 { + break // this is the last tag pair + } + + ui := uint(i) + j // make index relative to full series slice + if ui > 0 && series[ui-1] == '\\' { // the comma is escaped + j = ui + 1 + continue + } + + series, rest = series[:ui], series[ui+1:] + break + } + + // find the first unescaped '=' + for j := uint(0); j < uint(len(series)); { + i := bytes.IndexByte(series[j:], '=') + if i < 0 { + break // there is no tag value + } + ui := uint(i) + j // make index relative to full series slice + if ui > 0 && series[ui-1] == '\\' { // the equals is escaped + j = ui + 1 + continue + } + + tag, value = series[:ui], series[ui+1:] + break + } + + // sad time: it's possible this tag/value has escaped characters, so we have to + // find an unescape them. since the byte slice may refer to read-only memory, we + // can't do this in place, so we make copies. + if bytes.IndexByte(tag, '\\') != -1 { + unescapedTag := make([]byte, 0, len(tag)) + for i, c := range tag { + if c == '\\' && i+1 < len(tag) { + if c := tag[i+1]; c == ',' || c == ' ' || c == '=' { + continue + } + } + unescapedTag = append(unescapedTag, c) + } + tag = unescapedTag + } + + if bytes.IndexByte(value, '\\') != -1 { + unescapedValue := make([]byte, 0, len(value)) + for i, c := range value { + if c == '\\' && i+1 < len(value) { + if c := value[i+1]; c == ',' || c == ' ' || c == '=' { + continue + } + } + unescapedValue = append(unescapedValue, c) + } + value = unescapedValue + } + + return tag, value, rest +} \ No newline at end of file diff --git a/v1/tsdb/engine/tsm1/predicate_test.go b/v1/tsdb/engine/tsm1/predicate_test.go new file mode 100644 index 0000000000..9210fa3acf --- /dev/null +++ b/v1/tsdb/engine/tsm1/predicate_test.go @@ -0,0 +1,544 @@ +package tsm1 + +import ( + "fmt" + "reflect" + "testing" + + "github.com/influxdata/influxdb/v2/storage/reads/datatypes" +) + +func TestPredicatePopTagEscape(t *testing.T) { + cases := []struct { + Key string + Tag string + Value string + Rest string + }{ + {Key: "", Tag: "", Value: "", Rest: ""}, + {Key: "invalid", Tag: "", Value: "", Rest: ""}, + {Key: "region=west,server=b,foo=bar", Tag: "region", Value: "west", Rest: "server=b,foo=bar"}, + {Key: "region=west", Tag: "region", Value: "west", Rest: ""}, + {Key: `re\=gion=west,server=a`, Tag: `re=gion`, Value: "west", Rest: "server=a"}, + {Key: `region=w\,est,server=a`, Tag: `region`, Value: "w,est", Rest: "server=a"}, + {Key: `hi\ yo\ =w\,est,server=a`, Tag: `hi yo `, Value: "w,est", Rest: "server=a"}, + {Key: `\ e\ \=o=world,server=a`, Tag: ` e =o`, Value: "world", Rest: "server=a"}, + } + + for _, c := range cases { + tag, value, rest := predicatePopTagEscape([]byte(c.Key)) + if string(tag) != c.Tag { + t.Fatalf("got returned tag %q expected %q", tag, c.Tag) + } else if string(value) != c.Value { + t.Fatalf("got returned value %q expected %q", value, c.Value) + } else if string(rest) != c.Rest { + t.Fatalf("got returned remainder %q expected %q", rest, c.Rest) + } + } +} + +func TestPredicate_Matches(t *testing.T) { + cases := []struct { + Name string + Predicate *datatypes.Predicate + Key string + Matches bool + }{ + { + Name: "Basic Matching", + Predicate: predicate( + comparisonNode(datatypes.ComparisonEqual, tagNode("tag3"), stringNode("val3"))), + Key: "bucketorg,tag3=val3", + Matches: true, + }, + + { + Name: "Basic Unmatching", + Predicate: predicate( + comparisonNode(datatypes.ComparisonEqual, tagNode("tag3"), stringNode("val3"))), + Key: "bucketorg,tag3=val2", + Matches: false, + }, + + { + Name: "Compound Logical Matching", + Predicate: predicate( + orNode( + andNode( + comparisonNode(datatypes.ComparisonEqual, tagNode("foo"), stringNode("bar")), + comparisonNode(datatypes.ComparisonEqual, tagNode("baz"), stringNode("no"))), + comparisonNode(datatypes.ComparisonEqual, tagNode("tag3"), stringNode("val3")))), + Key: "bucketorg,foo=bar,baz=bif,tag3=val3", + Matches: true, + }, + + { + Name: "Compound Logical Unmatching", + Predicate: predicate( + orNode( + andNode( + comparisonNode(datatypes.ComparisonEqual, tagNode("foo"), stringNode("bar")), + comparisonNode(datatypes.ComparisonEqual, tagNode("baz"), stringNode("no"))), + comparisonNode(datatypes.ComparisonEqual, tagNode("tag3"), stringNode("val3")))), + Key: "bucketorg,foo=bar,baz=bif,tag3=val2", + Matches: false, + }, + + { + Name: "Logical Or Short Circuit", + Predicate: predicate( + orNode( + comparisonNode(datatypes.ComparisonEqual, tagNode("foo"), stringNode("bar")), + comparisonNode(datatypes.ComparisonEqual, tagNode("baz"), stringNode("no")))), + Key: "bucketorg,baz=bif,foo=bar,tag3=val3", + Matches: true, + }, + + { + Name: "Logical And Short Circuit", + Predicate: predicate( + andNode( + comparisonNode(datatypes.ComparisonEqual, tagNode("foo"), stringNode("no")), + comparisonNode(datatypes.ComparisonEqual, tagNode("baz"), stringNode("bif")))), + Key: "bucketorg,baz=bif,foo=bar,tag3=val3", + Matches: false, + }, + + { + Name: "Logical And Matching", + Predicate: predicate( + andNode( + comparisonNode(datatypes.ComparisonEqual, tagNode("foo"), stringNode("bar")), + comparisonNode(datatypes.ComparisonEqual, tagNode("baz"), stringNode("bif")))), + Key: "bucketorg,baz=bif,foo=bar,tag3=val3", + Matches: true, + }, + + { + Name: "Logical And Matching Reduce (Simplify)", + Predicate: predicate( + andNode( + comparisonNode(datatypes.ComparisonEqual, tagNode("foo"), stringNode("bar")), + comparisonNode(datatypes.ComparisonNotEqual, tagNode("foo"), stringNode("bif")))), + Key: "bucketorg,baz=bif,foo=bar,tag3=val3", + Matches: true, + }, + + { + Name: "Regex Matching", + Predicate: predicate( + comparisonNode(datatypes.ComparisonRegex, tagNode("tag3"), regexNode("...3"))), + Key: "bucketorg,tag3=val3", + Matches: true, + }, + + { + Name: "NotRegex Matching", + Predicate: predicate( + comparisonNode(datatypes.ComparisonNotRegex, tagNode("tag3"), regexNode("...4"))), + Key: "bucketorg,tag3=val3", + Matches: true, + }, + + { + Name: "Regex Unmatching", + Predicate: predicate( + comparisonNode(datatypes.ComparisonRegex, tagNode("tag3"), regexNode("...4"))), + Key: "bucketorg,tag3=val3", + Matches: false, + }, + + { + Name: "NotRegex Unmatching", + Predicate: predicate( + comparisonNode(datatypes.ComparisonNotRegex, tagNode("tag3"), regexNode("...3"))), + Key: "bucketorg,tag3=val3", + Matches: false, + }, + + { + Name: "Basic Matching Reversed", + Predicate: predicate( + comparisonNode(datatypes.ComparisonEqual, stringNode("val3"), tagNode("tag3"))), + Key: "bucketorg,tag2=val2,tag3=val3", + Matches: true, + }, + + { + Name: "Tag Matching Tag", + Predicate: predicate( + comparisonNode(datatypes.ComparisonEqual, tagNode("tag4"), tagNode("tag3"))), + Key: "bucketorg,tag3=val3,tag4=val3", + Matches: true, + }, + + { + Name: "No Tag", + Predicate: predicate( + comparisonNode(datatypes.ComparisonEqual, tagNode("tag4"), stringNode("val4"))), + Key: "bucketorg,tag3=val3", + Matches: false, + }, + + { + Name: "Not Equal", + Predicate: predicate( + comparisonNode(datatypes.ComparisonNotEqual, tagNode("tag3"), stringNode("val4"))), + Key: "bucketorg,tag3=val3", + Matches: true, + }, + + { + Name: "Starts With", + Predicate: predicate( + comparisonNode(datatypes.ComparisonStartsWith, tagNode("tag3"), stringNode("va"))), + Key: "bucketorg,tag3=val3", + Matches: true, + }, + + { + Name: "Less", + Predicate: predicate( + comparisonNode(datatypes.ComparisonLess, tagNode("tag3"), stringNode("val4"))), + Key: "bucketorg,tag3=val3", + Matches: true, + }, + + { + Name: "Less Equal", + Predicate: predicate( + comparisonNode(datatypes.ComparisonLessEqual, tagNode("tag3"), stringNode("val4"))), + Key: "bucketorg,tag3=val3", + Matches: true, + }, + + { + Name: "Greater", + Predicate: predicate( + comparisonNode(datatypes.ComparisonGreater, tagNode("tag3"), stringNode("u"))), + Key: "bucketorg,tag3=val3", + Matches: true, + }, + + { + Name: "Greater Equal;", + Predicate: predicate( + comparisonNode(datatypes.ComparisonGreaterEqual, tagNode("tag3"), stringNode("u"))), + Key: "bucketorg,tag3=val3", + Matches: true, + }, + + { + Name: "Escaping Matching", + Predicate: predicate( + comparisonNode(datatypes.ComparisonEqual, tagNode("tag3"), stringNode("val3"))), + Key: `bucketorg,tag1=\,foo,tag2=\ bar,tag2\=more=val2\,\ \=hello,tag3=val3`, + Matches: true, + }, + } + + for _, test := range cases { + t.Run(test.Name, func(t *testing.T) { + pred, err := NewProtobufPredicate(test.Predicate) + if err != nil { + t.Fatal("compile failure:", err) + } + + if got, exp := pred.Matches([]byte(test.Key)), test.Matches; got != exp { + t.Fatal("match failure:", "got", got, "!=", "exp", exp) + } + + // Clone and try again. + pred = pred.Clone() + if got, exp := pred.Matches([]byte(test.Key)), test.Matches; got != exp { + t.Fatal("cloned match failure:", "got", got, "!=", "exp", exp) + } + }) + } +} + +func TestPredicate_Unmarshal(t *testing.T) { + protoPred := predicate( + orNode( + andNode( + comparisonNode(datatypes.ComparisonEqual, tagNode("foo"), stringNode("bar")), + comparisonNode(datatypes.ComparisonEqual, tagNode("baz"), stringNode("no"))), + comparisonNode(datatypes.ComparisonEqual, tagNode("tag3"), stringNode("val3")))) + + pred1, err := NewProtobufPredicate(protoPred) + if err != nil { + t.Fatal(err) + } + + predData, err := pred1.Marshal() + if err != nil { + t.Fatal(err) + } + + pred2, err := UnmarshalPredicate(predData) + if err != nil { + t.Fatal(err) + } + + if !reflect.DeepEqual(pred1, pred2) { + t.Fatal("mismatch on unmarshal") + } +} + +func TestPredicate_Unmarshal_InvalidTag(t *testing.T) { + _, err := UnmarshalPredicate([]byte("\xff")) + if err == nil { + t.Fatal("expected error") + } +} + +func TestPredicate_Unmarshal_InvalidProtobuf(t *testing.T) { + _, err := UnmarshalPredicate([]byte("\x00\xff")) + if err == nil { + t.Fatal("expected error") + } +} + +func TestPredicate_Unmarshal_Empty(t *testing.T) { + pred, err := UnmarshalPredicate(nil) + if err != nil { + t.Fatal(err) + } else if pred != nil { + t.Fatal("expected no predicate") + } +} + +func TestPredicate_Invalid_Protobuf(t *testing.T) { + cases := []struct { + Name string + Predicate *datatypes.Predicate + }{ + { + Name: "Invalid Comparison Num Children", + Predicate: predicate(&datatypes.Node{ + NodeType: datatypes.NodeTypeComparisonExpression, + Value: &datatypes.Node_Comparison_{Comparison: datatypes.ComparisonEqual}, + Children: []*datatypes.Node{{}, {}, {}}, + }), + }, + + { + Name: "Mismatching Left Tag Type", + Predicate: predicate( + comparisonNode(datatypes.ComparisonEqual, &datatypes.Node{ + NodeType: datatypes.NodeTypeTagRef, + Value: &datatypes.Node_IntegerValue{IntegerValue: 2}, + }, tagNode("tag"))), + }, + + { + Name: "Mismatching Left Literal Type", + Predicate: predicate( + comparisonNode(datatypes.ComparisonEqual, &datatypes.Node{ + NodeType: datatypes.NodeTypeLiteral, + Value: &datatypes.Node_IntegerValue{IntegerValue: 2}, + }, tagNode("tag"))), + }, + + { + Name: "Invalid Left Node Type", + Predicate: predicate( + comparisonNode(datatypes.ComparisonEqual, &datatypes.Node{ + NodeType: datatypes.NodeTypeComparisonExpression, + Value: &datatypes.Node_Comparison_{Comparison: datatypes.ComparisonEqual}, + }, tagNode("tag"))), + }, + + { + Name: "Mismatching Right Tag Type", + Predicate: predicate( + comparisonNode(datatypes.ComparisonEqual, tagNode("tag"), &datatypes.Node{ + NodeType: datatypes.NodeTypeTagRef, + Value: &datatypes.Node_IntegerValue{IntegerValue: 2}, + })), + }, + + { + Name: "Invalid Regex", + Predicate: predicate( + comparisonNode(datatypes.ComparisonRegex, tagNode("tag3"), regexNode("("))), + }, + + { + Name: "Mismatching Right Literal Type", + Predicate: predicate( + comparisonNode(datatypes.ComparisonEqual, tagNode("tag"), &datatypes.Node{ + NodeType: datatypes.NodeTypeLiteral, + Value: &datatypes.Node_IntegerValue{IntegerValue: 2}, + })), + }, + + { + Name: "Invalid Right Node Type", + Predicate: predicate( + comparisonNode(datatypes.ComparisonEqual, tagNode("tag"), &datatypes.Node{ + NodeType: datatypes.NodeTypeComparisonExpression, + Value: &datatypes.Node_Comparison_{Comparison: datatypes.ComparisonEqual}, + })), + }, + + { + Name: "Invalid Comparison Without Regex", + Predicate: predicate( + comparisonNode(datatypes.ComparisonRegex, tagNode("tag3"), stringNode("val3"))), + }, + + { + Name: "Invalid Comparison With Regex", + Predicate: predicate( + comparisonNode(datatypes.ComparisonEqual, tagNode("tag3"), regexNode("."))), + }, + + { + Name: "Invalid Logical Operation Children", + Predicate: predicate(&datatypes.Node{ + NodeType: datatypes.NodeTypeLogicalExpression, + Value: &datatypes.Node_Logical_{Logical: datatypes.LogicalAnd}, + Children: []*datatypes.Node{{}, {}, {}}, + }), + }, + + { + Name: "Invalid Left Logical Expression", + Predicate: predicate( + andNode( + tagNode("tag"), + comparisonNode(datatypes.ComparisonEqual, tagNode("tag3"), stringNode("val3")), + )), + }, + + { + Name: "Invalid Right Logical Expression", + Predicate: predicate( + andNode( + comparisonNode(datatypes.ComparisonEqual, tagNode("tag3"), stringNode("val3")), + tagNode("tag"), + )), + }, + + { + Name: "Invalid Logical Value", + Predicate: predicate(&datatypes.Node{ + NodeType: datatypes.NodeTypeLogicalExpression, + Value: &datatypes.Node_Logical_{Logical: 9999}, + Children: []*datatypes.Node{ + comparisonNode(datatypes.ComparisonEqual, tagNode("tag3"), stringNode("val3")), + comparisonNode(datatypes.ComparisonEqual, tagNode("tag3"), stringNode("val3")), + }, + }), + }, + + { + Name: "Invalid Root Node", + Predicate: predicate(tagNode("tag3")), + }, + } + + for _, test := range cases { + t.Run(test.Name, func(t *testing.T) { + _, err := NewProtobufPredicate(test.Predicate) + if err == nil { + t.Fatal("expected compile failure") + } + }) + } +} + +func BenchmarkPredicate(b *testing.B) { + run := func(b *testing.B, predicate *datatypes.Predicate) { + pred, err := NewProtobufPredicate(predicate) + if err != nil { + b.Fatal(err) + } + + series := []byte("bucketorg,") + for i := 0; i < 10; i++ { + series = append(series, fmt.Sprintf("tag%d=val%d,", i, i)...) + } + series = series[:len(series)-1] + + b.SetBytes(int64(len(series))) + b.ReportAllocs() + b.ResetTimer() + + for i := 0; i < b.N; i++ { + pred.Matches(series) + } + } + + b.Run("Basic", func(b *testing.B) { + run(b, predicate( + comparisonNode(datatypes.ComparisonEqual, tagNode("tag5"), stringNode("val5")), + )) + }) + + b.Run("Compound", func(b *testing.B) { + run(b, predicate( + orNode( + andNode( + comparisonNode(datatypes.ComparisonEqual, tagNode("tag0"), stringNode("val0")), + comparisonNode(datatypes.ComparisonEqual, tagNode("tag6"), stringNode("val5")), + ), + comparisonNode(datatypes.ComparisonEqual, tagNode("tag5"), stringNode("val5")), + ), + )) + }) +} + +// +// Helpers to create predicate protobufs +// + +func tagNode(s string) *datatypes.Node { + return &datatypes.Node{ + NodeType: datatypes.NodeTypeTagRef, + Value: &datatypes.Node_TagRefValue{TagRefValue: s}, + } +} + +func stringNode(s string) *datatypes.Node { + return &datatypes.Node{ + NodeType: datatypes.NodeTypeLiteral, + Value: &datatypes.Node_StringValue{StringValue: s}, + } +} + +func regexNode(s string) *datatypes.Node { + return &datatypes.Node{ + NodeType: datatypes.NodeTypeLiteral, + Value: &datatypes.Node_RegexValue{RegexValue: s}, + } +} + +func comparisonNode(comp datatypes.Node_Comparison, left, right *datatypes.Node) *datatypes.Node { + return &datatypes.Node{ + NodeType: datatypes.NodeTypeComparisonExpression, + Value: &datatypes.Node_Comparison_{Comparison: comp}, + Children: []*datatypes.Node{left, right}, + } +} + +func andNode(left, right *datatypes.Node) *datatypes.Node { + return &datatypes.Node{ + NodeType: datatypes.NodeTypeLogicalExpression, + Value: &datatypes.Node_Logical_{Logical: datatypes.LogicalAnd}, + Children: []*datatypes.Node{left, right}, + } +} + +func orNode(left, right *datatypes.Node) *datatypes.Node { + return &datatypes.Node{ + NodeType: datatypes.NodeTypeLogicalExpression, + Value: &datatypes.Node_Logical_{Logical: datatypes.LogicalOr}, + Children: []*datatypes.Node{left, right}, + } +} + +func predicate(root *datatypes.Node) *datatypes.Predicate { + return &datatypes.Predicate{Root: root} +} \ No newline at end of file diff --git a/v1/tsdb/shard.go b/v1/tsdb/shard.go index ba9908ec1e..4408321292 100644 --- a/v1/tsdb/shard.go +++ b/v1/tsdb/shard.go @@ -337,7 +337,7 @@ func (s *Shard) Open() error { e.SetEnabled(false) // Open engine. - if err := e.Open(context.TODO()); err != nil { + if err := e.Open(); err != nil { return err }