diff --git a/CHANGELOG.md b/CHANGELOG.md index 50392cf19c..ef04642f7b 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,5 +1,14 @@ ## unreleased +### Breaking Changes + +#### inmem index option removed +This release fully removes the `inmem` indexing option, along with the associated config options: +* `max-series-per-database` +* `max-values-per-tag` + +Replacement `tsi1` indexes will be automatically generated on startup for shards that need it. + ### Bug Fixes 1. [20339](https://github.com/influxdata/influxdb/pull/20339): Include upgrade helper script in goreleaser manifest. @@ -10,7 +19,9 @@ 1. [20362](https://github.com/influxdata/influxdb/pull/20362): Don't overwrite stack name/description on `influx stack update`. 1. [20355](https://github.com/influxdata/influxdb/pull/20355): Fix timeout setup for `influxd` graceful shutdown. 1. [20387](https://github.com/influxdata/influxdb/pull/20387): Improve error message shown when `influx` CLI can't find an org by name. -1. [20380](https://github.com/influxdata/influxdb/pull/20380): Remove duplication from task error messages +1. [20380](https://github.com/influxdata/influxdb/pull/20380): Remove duplication from task error messages. +1. [20313](https://github.com/influxdata/influxdb/pull/20313): Automatically build `tsi1` indexes for shards that need it instead of falling back to `inmem`. +1. [20313](https://github.com/influxdata/influxdb/pull/20313): Fix logging initialization for storage engine. ## v2.0.3 [2020-12-14] diff --git a/cmd/influx_inspect/buildtsi/buildtsi.go b/cmd/influx_inspect/buildtsi/buildtsi.go deleted file mode 100644 index 64fb1080f7..0000000000 --- a/cmd/influx_inspect/buildtsi/buildtsi.go +++ /dev/null @@ -1,558 +0,0 @@ -// Package buildtsi reads an in-memory index and exports it as a TSI index. -package buildtsi - -import ( - "errors" - "flag" - "fmt" - "io" - "io/ioutil" - "os" - "os/user" - "path/filepath" - "runtime" - "strconv" - "strings" - "sync/atomic" - - "github.com/influxdata/influxdb/v2/logger" - "github.com/influxdata/influxdb/v2/models" - "github.com/influxdata/influxdb/v2/pkg/file" - "github.com/influxdata/influxdb/v2/tsdb" - "github.com/influxdata/influxdb/v2/tsdb/engine/tsm1" - "github.com/influxdata/influxdb/v2/tsdb/index/tsi1" - "go.uber.org/zap" - "golang.org/x/sync/errgroup" -) - -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)) - return nil - } - - log.Info("Opening shard") - - // Remove temporary index files if this is being re-run. - tmpPath := filepath.Join(dataDir, ".index") - log.Info("Cleaning up partial index from previous run, if any") - if err := os.RemoveAll(tmpPath); err != nil { - return err - } - - // Open TSI index in temporary path. - 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), - ) - - tsiIndex.WithLogger(log) - - log.Info("Opening tsi index in temporary location", zap.String("path", tmpPath)) - if err := tsiIndex.Open(); err != nil { - return err - } - defer tsiIndex.Close() - - // Write out tsm1 files. - // Find shard files. - tsmPaths, err := collectTSMFiles(dataDir) - if err != nil { - return err - } - - log.Info("Iterating over tsm files") - for _, path := range tsmPaths { - log.Info("Processing tsm file", zap.String("path", path)) - if err := IndexTSMFile(tsiIndex, path, batchSize, log, verboseLogging); err != nil { - return err - } - } - - // Write out wal files. - walPaths, err := collectWALFiles(walDir) - if err != nil { - if !os.IsNotExist(err) { - return err - } - - } else { - log.Info("Building cache from wal files") - cache := tsm1.NewCache(maxCacheSize) - loader := tsm1.NewCacheLoader(walPaths) - loader.WithLogger(log) - if err := loader.Load(cache); err != nil { - return err - } - - log.Info("Iterating over cache") - 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) - - if verboseLogging { - log.Info("Series", zap.String("name", string(name)), zap.String("tags", tags.String())) - } - - keysBatch = append(keysBatch, seriesKey) - namesBatch = append(namesBatch, name) - tagsBatch = append(tagsBatch, tags) - - // Flush batch? - if len(keysBatch) == batchSize { - if err := tsiIndex.CreateSeriesListIfNotExists(keysBatch, namesBatch, tagsBatch); err != nil { - return fmt.Errorf("problem creating series: (%s)", err) - } - keysBatch = keysBatch[:0] - namesBatch = namesBatch[:0] - tagsBatch = tagsBatch[:0] - } - } - - // Flush any remaining series in the batches - if len(keysBatch) > 0 { - if err := tsiIndex.CreateSeriesListIfNotExists(keysBatch, namesBatch, tagsBatch); err != nil { - return fmt.Errorf("problem creating series: (%s)", err) - } - keysBatch = nil - namesBatch = nil - tagsBatch = nil - } - } - - // Attempt to compact the index & wait for all compactions to complete. - log.Info("compacting index") - tsiIndex.Compact() - tsiIndex.Wait() - - // Close TSI index. - log.Info("Closing tsi index") - if err := tsiIndex.Close(); err != nil { - return err - } - - // Rename TSI to standard path. - log.Info("Moving tsi to permanent location") - return os.Rename(tmpPath, indexPath) -} - -func IndexTSMFile(index *tsi1.Index, path string, batchSize int, log *zap.Logger, verboseLogging bool) error { - f, err := os.Open(path) - if err != nil { - return err - } - defer f.Close() - - r, err := tsm1.NewTSMReader(f) - if err != nil { - log.Warn("Unable to read, skipping", zap.String("path", path), zap.Error(err)) - return nil - } - defer r.Close() - - keysBatch := make([][]byte, 0, batchSize) - namesBatch := make([][]byte, 0, batchSize) - tagsBatch := make([]models.Tags, batchSize) - var ti int - for i := 0; i < r.KeyCount(); i++ { - key, _ := r.KeyAt(i) - seriesKey, _ := tsm1.SeriesAndFieldFromCompositeKey(key) - var name []byte - name, tagsBatch[ti] = models.ParseKeyBytesWithTags(seriesKey, tagsBatch[ti]) - - if verboseLogging { - log.Info("Series", zap.String("name", string(name)), zap.String("tags", tagsBatch[ti].String())) - } - - keysBatch = append(keysBatch, seriesKey) - namesBatch = append(namesBatch, name) - ti++ - - // Flush batch? - if len(keysBatch) == batchSize { - if err := index.CreateSeriesListIfNotExists(keysBatch, namesBatch, tagsBatch[:ti]); err != nil { - return fmt.Errorf("problem creating series: (%s)", err) - } - keysBatch = keysBatch[:0] - namesBatch = namesBatch[:0] - ti = 0 // Reset tags. - } - } - - // Flush any remaining series in the batches - if len(keysBatch) > 0 { - if err := index.CreateSeriesListIfNotExists(keysBatch, namesBatch, tagsBatch[:ti]); err != nil { - return fmt.Errorf("problem creating series: (%s)", err) - } - } - return nil -} - -func collectTSMFiles(path string) ([]string, error) { - fis, err := ioutil.ReadDir(path) - if err != nil { - return nil, err - } - - var paths []string - for _, fi := range fis { - if filepath.Ext(fi.Name()) != "."+tsm1.TSMFileExtension { - continue - } - paths = append(paths, filepath.Join(path, fi.Name())) - } - return paths, nil -} - -func collectWALFiles(path string) ([]string, error) { - if path == "" { - return nil, os.ErrNotExist - } - if _, err := os.Stat(path); os.IsNotExist(err) { - return nil, err - } - fis, err := ioutil.ReadDir(path) - if err != nil { - return nil, err - } - - var paths []string - for _, fi := range fis { - if filepath.Ext(fi.Name()) != "."+tsm1.WALFileExtension { - continue - } - paths = append(paths, filepath.Join(path, fi.Name())) - } - return paths, nil -} - -func isRoot() bool { - user, _ := user.Current() - return user != nil && user.Username == "root" -} diff --git a/cmd/influxd/upgrade/config_test.go b/cmd/influxd/upgrade/config_test.go index c3d32c2243..3be6522ed1 100644 --- a/cmd/influxd/upgrade/config_test.go +++ b/cmd/influxd/upgrade/config_test.go @@ -214,7 +214,7 @@ bind-address = "127.0.0.1:8088" wal-dir = "/var/lib/influxdb/wal" wal-fsync-delay = "0s" validate-keys = false - index-version = "inmem" + index-version = "tsi1" query-log-enabled = true cache-max-memory-size = 1073741824 cache-snapshot-memory-size = 26214400 @@ -222,8 +222,6 @@ bind-address = "127.0.0.1:8088" compact-full-write-cold-duration = "4h0m0s" compact-throughput = 50331648 compact-throughput-burst = 50331648 - max-series-per-database = 1000000 - max-values-per-tag = 100000 max-concurrent-compactions = 0 max-index-log-file-size = 1048576 series-id-set-cache-size = 100 diff --git a/logger/style_guide.md b/logger/style_guide.md index 5c9ec01a4d..fc7417c4f8 100644 --- a/logger/style_guide.md +++ b/logger/style_guide.md @@ -112,11 +112,6 @@ should be emitted only when a support engineer can take some action to remedy the situation _and_ the system may not continue operating properly in the near future without remedying the situation.** -An example of what may qualify as a warning is the `max-values-per-tag` -setting. If the server starts to approach the maximum number of values, -the server may stop being able to function properly when it reaches the -maximum number. - An example of what does not qualify as a warning is the `log-queries-after` setting. While the message is "warning" that a query was running for a long period of time, it is not clearly actionable and diff --git a/query/benchmarks/flux/config.toml b/query/benchmarks/flux/config.toml index 59ed203c3e..4a5ce33a4e 100644 --- a/query/benchmarks/flux/config.toml +++ b/query/benchmarks/flux/config.toml @@ -16,8 +16,6 @@ bind-address = ":8188" cache-snapshot-memory-size = 26214400 cache-snapshot-write-cold-duration = "10m0s" compact-full-write-cold-duration = "4h0m0s" - max-series-per-database = 1000000 - max-values-per-tag = 100000 max-concurrent-compactions = 0 trace-logging-enabled = false diff --git a/storage/engine.go b/storage/engine.go index 44a28fe2da..4aa07bba98 100644 --- a/storage/engine.go +++ b/storage/engine.go @@ -14,7 +14,6 @@ import ( "github.com/influxdata/influxdb/v2/models" "github.com/influxdata/influxdb/v2/tsdb" _ "github.com/influxdata/influxdb/v2/tsdb/engine" - _ "github.com/influxdata/influxdb/v2/tsdb/index/inmem" _ "github.com/influxdata/influxdb/v2/tsdb/index/tsi1" "github.com/influxdata/influxdb/v2/v1/coordinator" "github.com/influxdata/influxdb/v2/v1/services/meta" @@ -138,13 +137,11 @@ func NewEngine(path string, c Config, options ...Option) *Engine { // WithLogger sets the logger on the Store. It must be called before Open. func (e *Engine) WithLogger(log *zap.Logger) { - fields := []zap.Field{} - fields = append(fields, zap.String("service", "storage-engine")) - e.logger = log.With(fields...) + e.logger = log.With(zap.String("service", "storage-engine")) - e.tsdbStore.Logger = e.logger + e.tsdbStore.WithLogger(e.logger) if pw, ok := e.pointsWriter.(*coordinator.PointsWriter); ok { - pw.Logger = e.logger + pw.WithLogger(e.logger) } if e.retentionService != nil { diff --git a/tsdb/config.go b/tsdb/config.go index c319968c03..2261575d4f 100644 --- a/tsdb/config.go +++ b/tsdb/config.go @@ -52,10 +52,6 @@ const ( // block in a TSM file DefaultMaxPointsPerBlock = 1000 - // DefaultMaxSeriesPerDatabase is the maximum number of series a node can hold per database. - // This limit only applies to the "inmem" index. - DefaultMaxSeriesPerDatabase = 1000000 - // DefaultMaxValuesPerTag is the maximum number of values a tag can have within a measurement. DefaultMaxValuesPerTag = 100000 @@ -106,16 +102,6 @@ type Config struct { // Limits - // MaxSeriesPerDatabase is the maximum number of series a node can hold per database. - // When this limit is exceeded, writes return a 'max series per database exceeded' error. - // A value of 0 disables the limit. This limit only applies when using the "inmem" index. - MaxSeriesPerDatabase int `toml:"max-series-per-database"` - - // MaxValuesPerTag is the maximum number of tag values a single tag key can have within - // a measurement. When the limit is exceeded, writes return an error. - // A value of 0 disables the limit. - MaxValuesPerTag int `toml:"max-values-per-tag"` - // MaxConcurrentCompactions is the maximum number of concurrent level and full compactions // that can be running at one time across all shards. Compactions scheduled to run when the // limit is reached are blocked until a running compaction completes. Snapshot compactions are @@ -164,8 +150,6 @@ func NewConfig() Config { CompactThroughput: toml.Size(DefaultCompactThroughput), CompactThroughputBurst: toml.Size(DefaultCompactThroughputBurst), - MaxSeriesPerDatabase: DefaultMaxSeriesPerDatabase, - MaxValuesPerTag: DefaultMaxValuesPerTag, MaxConcurrentCompactions: DefaultMaxConcurrentCompactions, MaxIndexLogFileSize: toml.Size(DefaultMaxIndexLogFileSize), @@ -233,8 +217,6 @@ func (c Config) Diagnostics() (*diagnostics.Diagnostics, error) { "cache-snapshot-memory-size": c.CacheSnapshotMemorySize, "cache-snapshot-write-cold-duration": c.CacheSnapshotWriteColdDuration, "compact-full-write-cold-duration": c.CompactFullWriteColdDuration, - "max-series-per-database": c.MaxSeriesPerDatabase, - "max-values-per-tag": c.MaxValuesPerTag, "max-concurrent-compactions": c.MaxConcurrentCompactions, "max-index-log-file-size": c.MaxIndexLogFileSize, "series-id-set-cache-size": c.SeriesIDSetCacheSize, diff --git a/tsdb/config_test.go b/tsdb/config_test.go index 956b9829af..1dc1e2ab4d 100644 --- a/tsdb/config_test.go +++ b/tsdb/config_test.go @@ -61,11 +61,6 @@ func TestConfig_Validate_Error(t *testing.T) { t.Errorf("unexpected error: %s", err) } - c.Index = tsdb.InmemIndexName - if err := c.Validate(); err != nil { - t.Error(err) - } - c.Index = tsdb.TSI1IndexName if err := c.Validate(); err != nil { t.Error(err) diff --git a/tsdb/engine.go b/tsdb/engine.go index 3926150a8f..12923d7554 100644 --- a/tsdb/engine.go +++ b/tsdb/engine.go @@ -81,6 +81,8 @@ type Engine interface { IsIdle() bool Free() error + Reindex() error + io.WriterTo } @@ -92,11 +94,6 @@ type SeriesIDSets interface { // EngineFormat represents the format for an engine. type EngineFormat int -const ( - // TSM1Format is the format used by the tsm1 engine. - TSM1Format EngineFormat = 2 -) - // NewEngineFunc creates a new engine. type NewEngineFunc func(id uint64, i Index, path string, walPath string, sfile *SeriesFile, options EngineOptions) Engine @@ -161,7 +158,6 @@ type EngineOptions struct { EngineVersion string IndexVersion string ShardID uint64 - InmemIndex interface{} // shared in-memory index // Limits the concurrent number of TSM files that can be loaded at once. OpenLimiter limiter.Fixed @@ -208,9 +204,6 @@ func NewEngineOptions() EngineOptions { } } -// NewInmemIndex returns a new "inmem" index type. -var NewInmemIndex func(name string, sfile *SeriesFile) (interface{}, error) - type CompactionPlannerCreator func(cfg Config) interface{} // FileStoreObserver is passed notifications before the file store adds or deletes files. In this way, it can diff --git a/tsdb/engine/tsm1/engine.go b/tsdb/engine/tsm1/engine.go index f710327540..5e9e12dec4 100644 --- a/tsdb/engine/tsm1/engine.go +++ b/tsdb/engine/tsm1/engine.go @@ -33,7 +33,6 @@ import ( "github.com/influxdata/influxdb/v2/pkg/tracing" "github.com/influxdata/influxdb/v2/tsdb" _ "github.com/influxdata/influxdb/v2/tsdb/index" - "github.com/influxdata/influxdb/v2/tsdb/index/inmem" "github.com/influxdata/influxdb/v2/tsdb/index/tsi1" "github.com/influxdata/influxql" "go.uber.org/zap" @@ -234,15 +233,14 @@ func NewEngine(id uint64, idx tsdb.Index, path string, walPath string, sfile *ts planner.SetFileStore(fs) } - logger := zap.NewNop() stats := &EngineStatistics{} e := &Engine{ id: id, path: path, index: idx, sfile: sfile, - logger: logger, - traceLogger: logger, + logger: zap.NewNop(), + traceLogger: zap.NewNop(), traceLogging: opt.Config.TraceLoggingEnabled, WAL: wal, @@ -570,10 +568,6 @@ func (e *Engine) ScheduleFullCompaction() error { // Path returns the path the engine was opened with. func (e *Engine) Path() string { return e.path } -func (e *Engine) SetFieldName(measurement []byte, name string) { - e.index.SetFieldName(measurement, name) -} - func (e *Engine) MeasurementExists(name []byte) (bool, error) { return e.index.MeasurementExists(name) } @@ -810,9 +804,8 @@ func (e *Engine) LoadMetadataIndex(shardID uint64, index tsdb.Index) error { // Save reference to index for iterator creation. e.index = index - // If we have the cached fields index on disk and we're using TSI, we - // can skip scanning all the TSM files. - if e.index.Type() != inmem.IndexName && !e.fieldset.IsEmpty() { + // If we have the cached fields index on disk, we can skip scanning all the TSM files. + if !e.fieldset.IsEmpty() { return nil } @@ -1255,18 +1248,7 @@ func (e *Engine) addToIndexFromKey(keys [][]byte, fieldTypes []influxql.DataType tags = append(tags, models.ParseTags(keys[i])) } - // Build in-memory index, if necessary. - if e.index.Type() == inmem.IndexName { - if err := e.index.InitializeSeries(keys, names, tags); err != nil { - return err - } - } else { - if err := e.index.CreateSeriesListIfNotExists(keys, names, tags); err != nil { - return err - } - } - - return nil + return e.index.CreateSeriesListIfNotExists(keys, names, tags) } // WritePoints writes metadata and point data into the engine. @@ -1489,7 +1471,6 @@ func (e *Engine) DeleteSeriesRangeWithPredicate(itr tsdb.SeriesIterator, predica } } - e.index.Rebuild() return nil } @@ -1750,20 +1731,10 @@ func (e *Engine) deleteSeriesRange(seriesKeys [][]byte, min, max int64) error { // in any shard. var err error ids.ForEach(func(id uint64) { - name, tags := e.sfile.Series(id) if err1 := e.sfile.DeleteSeriesID(id); err1 != nil { err = err1 return } - - // In the case of the inmem index the series can be removed across - // the global index (all shards). - if index, ok := e.index.(*inmem.ShardIndex); ok { - key := models.MakeKey(name, tags) - if e := index.Index.DropSeriesGlobal(key); e != nil { - err = e - } - } }) if err != nil { return err @@ -2425,10 +2396,6 @@ func (e *Engine) CreateIterator(ctx context.Context, measurement string, opt que return newMergeFinalizerIterator(ctx, itrs, opt, e.logger) } -type indexTagSets interface { - TagSets(name []byte, options query.IteratorOptions) ([]*query.TagSet, error) -} - func (e *Engine) createCallIterator(ctx context.Context, measurement string, call *influxql.Call, opt query.IteratorOptions) ([]query.Iterator, error) { ref, _ := call.Args[0].(*influxql.VarRef) @@ -2443,13 +2410,8 @@ func (e *Engine) createCallIterator(ctx context.Context, measurement string, cal tagSets []*query.TagSet err error ) - if e.index.Type() == tsdb.InmemIndexName { - ts := e.index.(indexTagSets) - tagSets, err = ts.TagSets([]byte(measurement), opt) - } else { - indexSet := tsdb.IndexSet{Indexes: []tsdb.Index{e.index}, SeriesFile: e.sfile} - tagSets, err = indexSet.TagSets(e.sfile, []byte(measurement), opt) - } + indexSet := tsdb.IndexSet{Indexes: []tsdb.Index{e.index}, SeriesFile: e.sfile} + tagSets, err = indexSet.TagSets(e.sfile, []byte(measurement), opt) if err != nil { return nil, err @@ -2523,13 +2485,8 @@ func (e *Engine) createVarRefIterator(ctx context.Context, measurement string, o tagSets []*query.TagSet err error ) - if e.index.Type() == tsdb.InmemIndexName { - ts := e.index.(indexTagSets) - tagSets, err = ts.TagSets([]byte(measurement), opt) - } else { - indexSet := tsdb.IndexSet{Indexes: []tsdb.Index{e.index}, SeriesFile: e.sfile} - tagSets, err = indexSet.TagSets(e.sfile, []byte(measurement), opt) - } + indexSet := tsdb.IndexSet{Indexes: []tsdb.Index{e.index}, SeriesFile: e.sfile} + tagSets, err = indexSet.TagSets(e.sfile, []byte(measurement), opt) if err != nil { return nil, err @@ -3140,3 +3097,77 @@ func varRefSliceRemove(a []influxql.VarRef, v string) []influxql.VarRef { } return other } + +const reindexBatchSize = 10000 + +func (e *Engine) Reindex() error { + keys := make([][]byte, reindexBatchSize) + seriesKeys := make([][]byte, reindexBatchSize) + names := make([][]byte, reindexBatchSize) + tags := make([]models.Tags, reindexBatchSize) + + n := 0 + + reindexBatch := func() error { + if n == 0 { + return nil + } + + for i, key := range keys[:n] { + seriesKeys[i], _ = SeriesAndFieldFromCompositeKey(key) + names[i], tags[i] = models.ParseKeyBytes(seriesKeys[i]) + e.traceLogger.Debug( + "Read series during reindexing", + logger.Shard(e.id), + zap.String("name", string(names[i])), + zap.String("tags", tags[i].String()), + ) + } + + e.logger.Debug("Reindexing data batch", logger.Shard(e.id), zap.Int("batch_size", n)) + if err := e.index.CreateSeriesListIfNotExists(seriesKeys[:n], names[:n], tags[:n]); err != nil { + return err + } + + n = 0 + return nil + } + reindexKey := func(key []byte) error { + keys[n] = key + n++ + + if n < reindexBatchSize { + return nil + } + return reindexBatch() + } + + // Index data stored in TSM files. + e.logger.Info("Reindexing TSM data", logger.Shard(e.id)) + if err := e.FileStore.WalkKeys(nil, func(key []byte, _ byte) error { + return reindexKey(key) + }); err != nil { + return err + } + + // Make sure all TSM data is indexed. + if err := reindexBatch(); err != nil { + return err + } + + if !e.WALEnabled { + // All done. + return nil + } + + // Reindex data stored in the WAL cache. + e.logger.Info("Reindexing WAL data", logger.Shard(e.id)) + for _, key := range e.Cache.Keys() { + if err := reindexKey(key); err != nil { + return err + } + } + + // Make sure all WAL data is indexed. + return reindexBatch() +} diff --git a/tsdb/engine/tsm1/engine_test.go b/tsdb/engine/tsm1/engine_test.go index 21eb3c0cd6..af5dcd19aa 100644 --- a/tsdb/engine/tsm1/engine_test.go +++ b/tsdb/engine/tsm1/engine_test.go @@ -26,7 +26,7 @@ import ( "github.com/influxdata/influxdb/v2/pkg/deep" "github.com/influxdata/influxdb/v2/tsdb" "github.com/influxdata/influxdb/v2/tsdb/engine/tsm1" - "github.com/influxdata/influxdb/v2/tsdb/index/inmem" + "github.com/influxdata/influxdb/v2/tsdb/index/tsi1" "github.com/influxdata/influxql" ) @@ -174,7 +174,7 @@ func seriesExist(e *Engine, m string, dims []string) (int, error) { // Ensure that the engine can write & read shard digest files. func TestEngine_Digest(t *testing.T) { - e := MustOpenEngine(inmem.IndexName) + e := MustOpenEngine(tsi1.IndexName) defer e.Close() if err := e.Open(); err != nil { @@ -322,7 +322,7 @@ type span struct { // Ensure engine handles concurrent calls to Digest(). func TestEngine_Digest_Concurrent(t *testing.T) { - e := MustOpenEngine(inmem.IndexName) + e := MustOpenEngine(tsi1.IndexName) defer e.Close() if err := e.Open(); err != nil { @@ -392,7 +392,6 @@ func TestEngine_Backup(t *testing.T) { // Write those points to the engine. db := path.Base(f.Name()) opt := tsdb.NewEngineOptions() - opt.InmemIndex = inmem.NewIndex(db, sfile.SeriesFile) idx := tsdb.MustOpenIndex(1, db, filepath.Join(f.Name(), "index"), tsdb.NewSeriesIDSet(), sfile.SeriesFile, opt) defer idx.Close() @@ -499,7 +498,6 @@ func TestEngine_Export(t *testing.T) { // Write those points to the engine. db := path.Base(f.Name()) opt := tsdb.NewEngineOptions() - opt.InmemIndex = inmem.NewIndex(db, sfile.SeriesFile) idx := tsdb.MustOpenIndex(1, db, filepath.Join(f.Name(), "index"), tsdb.NewSeriesIDSet(), sfile.SeriesFile, opt) defer idx.Close() @@ -1046,8 +1044,6 @@ func TestEngine_CreateIterator_Condition(t *testing.T) { e.MeasurementFields([]byte("cpu")).CreateFieldIfNotExists([]byte("X"), influxql.Float) e.MeasurementFields([]byte("cpu")).CreateFieldIfNotExists([]byte("Y"), influxql.Float) e.CreateSeriesIfNotExists([]byte("cpu,host=A"), []byte("cpu"), models.NewTags(map[string]string{"host": "A"})) - e.SetFieldName([]byte("cpu"), "X") - e.SetFieldName([]byte("cpu"), "Y") if err := e.WritePointsString( `cpu,host=A value=1.1 1000000000`, @@ -1842,7 +1838,6 @@ func TestEngine_SnapshotsDisabled(t *testing.T) { // Create a tsm1 engine. db := path.Base(dir) opt := tsdb.NewEngineOptions() - opt.InmemIndex = inmem.NewIndex(db, sfile.SeriesFile) idx := tsdb.MustOpenIndex(1, db, filepath.Join(dir, "index"), tsdb.NewSeriesIDSet(), sfile.SeriesFile, opt) defer idx.Close() @@ -1870,10 +1865,11 @@ func TestEngine_SnapshotsDisabled(t *testing.T) { func TestEngine_ShouldCompactCache(t *testing.T) { nowTime := time.Now() - e, err := NewEngine(inmem.IndexName) + e, err := NewEngine(tsi1.IndexName) if err != nil { t.Fatal(err) } + defer e.Close() // mock the planner so compactions don't run during the test e.CompactionPlan = &mockPlanner{} @@ -1881,7 +1877,6 @@ func TestEngine_ShouldCompactCache(t *testing.T) { if err := e.Open(); err != nil { t.Fatalf("failed to open tsm1 engine: %s", err.Error()) } - defer e.Close() e.CacheFlushMemorySizeThreshold = 1024 e.CacheFlushWriteColdDuration = time.Minute @@ -2543,9 +2538,6 @@ func NewEngine(index string) (*Engine, error) { opt := tsdb.NewEngineOptions() opt.IndexVersion = index - if index == tsdb.InmemIndexName { - opt.InmemIndex = inmem.NewIndex(db, sfile) - } // Initialise series id sets. Need to do this as it's normally done at the // store level. seriesIDs := tsdb.NewSeriesIDSet() @@ -2616,7 +2608,6 @@ func (e *Engine) Reopen() error { db := path.Base(e.root) opt := tsdb.NewEngineOptions() - opt.InmemIndex = inmem.NewIndex(db, e.sfile) // Re-initialise the series id set seriesIDSet := tsdb.NewSeriesIDSet() diff --git a/tsdb/index.go b/tsdb/index.go index 0d7740594f..f5c5f823b4 100644 --- a/tsdb/index.go +++ b/tsdb/index.go @@ -21,8 +21,7 @@ import ( // Available index types. const ( - InmemIndexName = "inmem" - TSI1IndexName = "tsi1" + TSI1IndexName = "tsi1" ) // ErrIndexClosing can be returned to from an Index method if the index is currently closing. @@ -39,15 +38,11 @@ type Index interface { DropMeasurement(name []byte) error ForEachMeasurementName(fn func(name []byte) error) error - InitializeSeries(keys, names [][]byte, tags []models.Tags) error CreateSeriesIfNotExists(key, name []byte, tags models.Tags) error CreateSeriesListIfNotExists(keys, names [][]byte, tags []models.Tags) error DropSeries(seriesID uint64, key []byte, cascade bool) error DropMeasurementIfSeriesNotExist(name []byte) (bool, error) - // Used to clean up series in inmem index that were dropped with a shard. - DropSeriesGlobal(key []byte) error - MeasurementsSketches() (estimator.Sketch, estimator.Sketch, error) SeriesN() int64 SeriesSketches() (estimator.Sketch, estimator.Sketch, error) @@ -78,15 +73,10 @@ type Index interface { // Bytes estimates the memory footprint of this Index, in bytes. Bytes() int - // To be removed w/ tsi1. - SetFieldName(measurement []byte, name string) - Type() string - // Returns a unique reference ID to the index instance. - // For inmem, returns a reference to the backing Index, not ShardIndex. - UniqueReferenceID() uintptr - Rebuild() + // Returns a unique reference ID to the index instance. + UniqueReferenceID() uintptr } // SeriesElem represents a generic series element. @@ -1273,16 +1263,6 @@ type IndexSet struct { fieldSets []*MeasurementFieldSet // field sets for _all_ indexes in this set's DB. } -// HasInmemIndex returns true if any in-memory index is in use. -func (is IndexSet) HasInmemIndex() bool { - for _, idx := range is.Indexes { - if idx.Type() == InmemIndexName { - return true - } - } - return false -} - // Database returns the database name of the first index. func (is IndexSet) Database() string { if len(is.Indexes) == 0 { @@ -1314,28 +1294,6 @@ func (is IndexSet) HasField(measurement []byte, field string) bool { return false } -// DedupeInmemIndexes returns an index set which removes duplicate indexes. -// Useful because inmem indexes are shared by shards per database. -func (is IndexSet) DedupeInmemIndexes() IndexSet { - other := IndexSet{ - Indexes: make([]Index, 0, len(is.Indexes)), - SeriesFile: is.SeriesFile, - fieldSets: make([]*MeasurementFieldSet, 0, len(is.Indexes)), - } - - uniqueIndexes := make(map[uintptr]Index) - for _, idx := range is.Indexes { - uniqueIndexes[idx.UniqueReferenceID()] = idx - } - - for _, idx := range uniqueIndexes { - other.Indexes = append(other.Indexes, idx) - other.fieldSets = append(other.fieldSets, idx.FieldSet()) - } - - return other -} - // MeasurementNamesByExpr returns a slice of measurement names matching the // provided condition. If no condition is provided then all names are returned. func (is IndexSet) MeasurementNamesByExpr(auth query.Authorizer, expr influxql.Expr) ([][]byte, error) { @@ -2056,7 +2014,7 @@ func (is IndexSet) tagValueIterator(name, key []byte) (TagValueIterator, error) // TagKeyHasAuthorizedSeries determines if there exists an authorized series for // the provided measurement name and tag key. func (is IndexSet) TagKeyHasAuthorizedSeries(auth query.Authorizer, name, tagKey []byte) (bool, error) { - if !is.HasInmemIndex() && query.AuthorizerIsOpen(auth) { + if query.AuthorizerIsOpen(auth) { return true, nil } @@ -3015,17 +2973,6 @@ func (is IndexSet) TagSets(sfile *SeriesFile, name []byte, opt query.IteratorOpt return sortedTagsSets, nil } -// IndexFormat represents the format for an index. -type IndexFormat int - -const ( - // InMemFormat is the format used by the original in-memory shared index. - InMemFormat IndexFormat = 1 - - // TSI1Format is the format used by the tsi1 index. - TSI1Format IndexFormat = 2 -) - // NewIndexFunc creates a new index. type NewIndexFunc func(id uint64, database, path string, seriesIDSet *SeriesIDSet, sfile *SeriesFile, options EngineOptions) Index diff --git a/tsdb/index/index.go b/tsdb/index/index.go index b2b3418dfa..b040461958 100644 --- a/tsdb/index/index.go +++ b/tsdb/index/index.go @@ -1,6 +1,5 @@ package index // import "github.com/influxdata/influxdb/v2/tsdb/index" import ( - _ "github.com/influxdata/influxdb/v2/tsdb/index/inmem" _ "github.com/influxdata/influxdb/v2/tsdb/index/tsi1" ) diff --git a/tsdb/index/inmem/inmem.go b/tsdb/index/inmem/inmem.go deleted file mode 100644 index cd41d7f518..0000000000 --- a/tsdb/index/inmem/inmem.go +++ /dev/null @@ -1,1345 +0,0 @@ -/* -Package inmem implements a shared, in-memory index for each database. - -The in-memory index is the original index implementation and provides fast -access to index data. However, it also forces high memory usage for large -datasets and can cause OOM errors. - -Index is the shared index structure that provides most of the functionality. -However, ShardIndex is a light per-shard wrapper that adapts this original -shared index format to the new per-shard format. -*/ -package inmem - -import ( - "errors" - "fmt" - "regexp" - "sort" - "sync" - "unsafe" - - "github.com/influxdata/influxdb/v2/influxql/query" - "github.com/influxdata/influxdb/v2/models" - "github.com/influxdata/influxdb/v2/pkg/bytesutil" - "github.com/influxdata/influxdb/v2/pkg/escape" - "github.com/influxdata/influxdb/v2/pkg/estimator" - "github.com/influxdata/influxdb/v2/pkg/estimator/hll" - "github.com/influxdata/influxdb/v2/tsdb" - "github.com/influxdata/influxql" - "go.uber.org/zap" -) - -// IndexName is the name of this index. -const IndexName = tsdb.InmemIndexName - -func init() { - tsdb.NewInmemIndex = func(name string, sfile *tsdb.SeriesFile) (interface{}, error) { return NewIndex(name, sfile), nil } - - tsdb.RegisterIndex(IndexName, func(id uint64, database, path string, seriesIDSet *tsdb.SeriesIDSet, sfile *tsdb.SeriesFile, opt tsdb.EngineOptions) tsdb.Index { - return NewShardIndex(id, seriesIDSet, opt) - }) -} - -// Index is the in memory index of a collection of measurements, time -// series, and their tags. Exported functions are goroutine safe while -// un-exported functions assume the caller will use the appropriate locks. -type Index struct { - mu sync.RWMutex - - database string - sfile *tsdb.SeriesFile - fieldset *tsdb.MeasurementFieldSet - - // In-memory metadata index, built on load and updated when new series come in - measurements map[string]*measurement // measurement name to object and index - series map[string]*series // map series key to the Series object - - seriesSketch, seriesTSSketch estimator.Sketch - measurementsSketch, measurementsTSSketch estimator.Sketch - - // Mutex to control rebuilds of the index - rebuildQueue sync.Mutex -} - -// NewIndex returns a new initialized Index. -func NewIndex(database string, sfile *tsdb.SeriesFile) *Index { - index := &Index{ - database: database, - sfile: sfile, - measurements: make(map[string]*measurement), - series: make(map[string]*series), - } - - index.seriesSketch = hll.NewDefaultPlus() - index.seriesTSSketch = hll.NewDefaultPlus() - index.measurementsSketch = hll.NewDefaultPlus() - index.measurementsTSSketch = hll.NewDefaultPlus() - - return index -} - -func (i *Index) UniqueReferenceID() uintptr { - return uintptr(unsafe.Pointer(i)) -} - -// Bytes estimates the memory footprint of this Index, in bytes. -func (i *Index) Bytes() int { - var b int - i.mu.RLock() - b += 24 // mu RWMutex is 24 bytes - b += int(unsafe.Sizeof(i.database)) + len(i.database) - // Do not count SeriesFile because it belongs to the code that constructed this Index. - if i.fieldset != nil { - b += int(unsafe.Sizeof(i.fieldset)) + i.fieldset.Bytes() - } - b += int(unsafe.Sizeof(i.fieldset)) - for k, v := range i.measurements { - b += int(unsafe.Sizeof(k)) + len(k) - b += int(unsafe.Sizeof(v)) + v.bytes() - } - b += int(unsafe.Sizeof(i.measurements)) - for k, v := range i.series { - b += int(unsafe.Sizeof(k)) + len(k) - b += int(unsafe.Sizeof(v)) + v.bytes() - } - b += int(unsafe.Sizeof(i.series)) - b += int(unsafe.Sizeof(i.seriesSketch)) + i.seriesSketch.Bytes() - b += int(unsafe.Sizeof(i.seriesTSSketch)) + i.seriesTSSketch.Bytes() - b += int(unsafe.Sizeof(i.measurementsSketch)) + i.measurementsSketch.Bytes() - b += int(unsafe.Sizeof(i.measurementsTSSketch)) + i.measurementsTSSketch.Bytes() - b += 8 // rebuildQueue Mutex is 8 bytes - i.mu.RUnlock() - return b -} - -func (i *Index) Type() string { return IndexName } -func (i *Index) Open() (err error) { return nil } -func (i *Index) Close() error { return nil } - -func (i *Index) WithLogger(*zap.Logger) {} - -// Database returns the name of the database the index was initialized with. -func (i *Index) Database() string { - return i.database -} - -// Series returns a series by key. -func (i *Index) Series(key []byte) (*series, error) { - i.mu.RLock() - s := i.series[string(key)] - i.mu.RUnlock() - return s, nil -} - -// SeriesSketches returns the sketches for the series. -func (i *Index) SeriesSketches() (estimator.Sketch, estimator.Sketch, error) { - i.mu.RLock() - defer i.mu.RUnlock() - return i.seriesSketch.Clone(), i.seriesTSSketch.Clone(), nil -} - -// Measurement returns the measurement object from the index by the name -func (i *Index) Measurement(name []byte) (*measurement, error) { - i.mu.RLock() - defer i.mu.RUnlock() - return i.measurements[string(name)], nil -} - -// MeasurementExists returns true if the measurement exists. -func (i *Index) MeasurementExists(name []byte) (bool, error) { - i.mu.RLock() - defer i.mu.RUnlock() - return i.measurements[string(name)] != nil, nil -} - -// MeasurementsSketches returns the sketches for the measurements. -func (i *Index) MeasurementsSketches() (estimator.Sketch, estimator.Sketch, error) { - i.mu.RLock() - defer i.mu.RUnlock() - return i.measurementsSketch.Clone(), i.measurementsTSSketch.Clone(), nil -} - -// MeasurementsByName returns a list of measurements. -func (i *Index) MeasurementsByName(names [][]byte) ([]*measurement, error) { - i.mu.RLock() - defer i.mu.RUnlock() - - a := make([]*measurement, 0, len(names)) - for _, name := range names { - if m := i.measurements[string(name)]; m != nil { - a = append(a, m) - } - } - return a, nil -} - -// MeasurementIterator returns an iterator over all measurements in the index. -// MeasurementIterator does not support authorization. -func (i *Index) MeasurementIterator() (tsdb.MeasurementIterator, error) { - names, err := i.MeasurementNamesByExpr(nil, nil) - if err != nil { - return nil, err - } - return tsdb.NewMeasurementSliceIterator(names), nil -} - -// CreateSeriesListIfNotExists adds the series for the given measurement to the -// index and sets its ID or returns the existing series object -func (i *Index) CreateSeriesListIfNotExists(seriesIDSet *tsdb.SeriesIDSet, measurements map[string]int, - keys, names [][]byte, tagsSlice []models.Tags, opt *tsdb.EngineOptions, ignoreLimits bool) error { - - // Verify that the series will not exceed limit. - if !ignoreLimits { - i.mu.RLock() - if max := opt.Config.MaxSeriesPerDatabase; max > 0 && len(i.series)+len(keys) > max { - i.mu.RUnlock() - return errMaxSeriesPerDatabaseExceeded{limit: opt.Config.MaxSeriesPerDatabase} - } - i.mu.RUnlock() - } - - seriesIDs, err := i.sfile.CreateSeriesListIfNotExists(names, tagsSlice) - if err != nil { - return err - } - - i.mu.RLock() - // If there is a series for this ID, it's already been added. - seriesList := make([]*series, len(seriesIDs)) - for j, key := range keys { - seriesList[j] = i.series[string(key)] - } - i.mu.RUnlock() - - var hasNewSeries bool - for _, ss := range seriesList { - if ss == nil { - hasNewSeries = true - continue - } - - // This series might need to be added to the local bitset, if the series - // was created on another shard. - seriesIDSet.Lock() - if !seriesIDSet.ContainsNoLock(ss.ID) { - seriesIDSet.AddNoLock(ss.ID) - measurements[ss.Measurement.Name]++ - } - seriesIDSet.Unlock() - } - if !hasNewSeries { - return nil - } - - // get or create the measurement index - mms := make([]*measurement, len(names)) - for j, name := range names { - mms[j] = i.CreateMeasurementIndexIfNotExists(name) - } - - i.mu.Lock() - defer i.mu.Unlock() - - // Check for the series again under a write lock - var newSeriesN int - for j, key := range keys { - if seriesList[j] != nil { - continue - } - - ss := i.series[string(key)] - if ss == nil { - newSeriesN++ - continue - } - seriesList[j] = ss - - // This series might need to be added to the local bitset, if the series - // was created on another shard. - seriesIDSet.Lock() - if !seriesIDSet.ContainsNoLock(ss.ID) { - seriesIDSet.AddNoLock(ss.ID) - measurements[ss.Measurement.Name]++ - } - seriesIDSet.Unlock() - } - if newSeriesN == 0 { - return nil - } - - for j, key := range keys { - // Note, keys may contain duplicates (e.g., because of points for the same series - // in the same batch). If the duplicate series are new, the index must - // be rechecked on each iteration. - if seriesList[j] != nil || i.series[string(key)] != nil { - continue - } - - // set the in memory ID for query processing on this shard - // The series key and tags are clone to prevent a memory leak - skey := string(key) - ss := newSeries(seriesIDs[j], mms[j], skey, tagsSlice[j].Clone()) - i.series[skey] = ss - - mms[j].AddSeries(ss) - - // Add the series to the series sketch. - i.seriesSketch.Add(key) - - // This series needs to be added to the bitset tracking undeleted series IDs. - seriesIDSet.Lock() - seriesIDSet.AddNoLock(seriesIDs[j]) - measurements[mms[j].Name]++ - seriesIDSet.Unlock() - } - - return nil -} - -// CreateMeasurementIndexIfNotExists creates or retrieves an in memory index -// object for the measurement -func (i *Index) CreateMeasurementIndexIfNotExists(name []byte) *measurement { - name = escape.Unescape(name) - - // See if the measurement exists using a read-lock - i.mu.RLock() - m := i.measurements[string(name)] - if m != nil { - i.mu.RUnlock() - return m - } - i.mu.RUnlock() - - // Doesn't exist, so lock the index to create it - i.mu.Lock() - defer i.mu.Unlock() - - // Make sure it was created in between the time we released our read-lock - // and acquire the write lock - m = i.measurements[string(name)] - if m == nil { - m = newMeasurement(i.database, string(name)) - i.measurements[string(name)] = m - - // Add the measurement to the measurements sketch. - i.measurementsSketch.Add([]byte(name)) - } - return m -} - -// HasTagKey returns true if tag key exists. -func (i *Index) HasTagKey(name, key []byte) (bool, error) { - i.mu.RLock() - mm := i.measurements[string(name)] - i.mu.RUnlock() - - if mm == nil { - return false, nil - } - return mm.HasTagKey(string(key)), nil -} - -// HasTagValue returns true if tag value exists. -func (i *Index) HasTagValue(name, key, value []byte) (bool, error) { - i.mu.RLock() - mm := i.measurements[string(name)] - i.mu.RUnlock() - - if mm == nil { - return false, nil - } - return mm.HasTagKeyValue(key, value), nil -} - -// TagValueN returns the cardinality of a tag value. -func (i *Index) TagValueN(name, key []byte) int { - i.mu.RLock() - mm := i.measurements[string(name)] - i.mu.RUnlock() - - if mm == nil { - return 0 - } - return mm.CardinalityBytes(key) -} - -// MeasurementTagKeysByExpr returns an ordered set of tag keys filtered by an expression. -func (i *Index) MeasurementTagKeysByExpr(name []byte, expr influxql.Expr) (map[string]struct{}, error) { - i.mu.RLock() - mm := i.measurements[string(name)] - i.mu.RUnlock() - - if mm == nil { - return nil, nil - } - return mm.TagKeysByExpr(expr) -} - -// TagKeyHasAuthorizedSeries determines if there exists an authorized series for -// the provided measurement name and tag key. -func (i *Index) TagKeyHasAuthorizedSeries(auth query.Authorizer, name []byte, key string) bool { - i.mu.RLock() - mm := i.measurements[string(name)] - i.mu.RUnlock() - - if mm == nil { - return false - } - - // TODO(edd): This looks like it's inefficient. Since a series can have multiple - // tag key/value pairs on it, it's possible that the same unauthorised series - // will be checked multiple times. It would be more efficient if it were - // possible to get the set of unique series IDs for a given measurement name - // and tag key. - var authorized bool - mm.SeriesByTagKeyValue(key).Range(func(_ string, sIDs seriesIDs) bool { - if query.AuthorizerIsOpen(auth) { - authorized = true - return false - } - - for _, id := range sIDs { - s := mm.SeriesByID(id) - if s == nil { - continue - } - - if auth.AuthorizeSeriesRead(i.database, mm.NameBytes, s.Tags) { - authorized = true - return false - } - } - - // This tag key/value combination doesn't have any authorised series, so - // keep checking other tag values. - return true - }) - return authorized -} - -// MeasurementTagKeyValuesByExpr returns a set of tag values filtered by an expression. -// -// See tsm1.Engine.MeasurementTagKeyValuesByExpr for a fuller description of this -// method. -func (i *Index) MeasurementTagKeyValuesByExpr(auth query.Authorizer, name []byte, keys []string, expr influxql.Expr, keysSorted bool) ([][]string, error) { - i.mu.RLock() - mm := i.measurements[string(name)] - i.mu.RUnlock() - - if mm == nil || len(keys) == 0 { - return nil, nil - } - - results := make([][]string, len(keys)) - - // If we haven't been provided sorted keys, then we need to sort them. - if !keysSorted { - sort.Strings(keys) - } - - ids, _, _ := mm.WalkWhereForSeriesIds(expr) - if ids.Len() == 0 && expr == nil { - for ki, key := range keys { - values := mm.TagValues(auth, key) - sort.Strings(values) - results[ki] = values - } - return results, nil - } - - // This is the case where we have filtered series by some WHERE condition. - // We only care about the tag values for the keys given the - // filtered set of series ids. - - keyIdxs := make(map[string]int, len(keys)) - for ki, key := range keys { - keyIdxs[key] = ki - } - - resultSet := make([]stringSet, len(keys)) - for i := 0; i < len(resultSet); i++ { - resultSet[i] = newStringSet() - } - - // Iterate all series to collect tag values. - for _, id := range ids { - s := mm.SeriesByID(id) - if s == nil { - continue - } - if auth != nil && !auth.AuthorizeSeriesRead(i.database, s.Measurement.NameBytes, s.Tags) { - continue - } - - // Iterate the tag keys we're interested in and collect values - // from this series, if they exist. - for _, t := range s.Tags { - if idx, ok := keyIdxs[string(t.Key)]; ok { - resultSet[idx].add(string(t.Value)) - } else if string(t.Key) > keys[len(keys)-1] { - // The tag key is > the largest key we're interested in. - break - } - } - } - for i, s := range resultSet { - results[i] = s.list() - } - return results, nil -} - -// ForEachMeasurementTagKey iterates over all tag keys for a measurement. -func (i *Index) ForEachMeasurementTagKey(name []byte, fn func(key []byte) error) error { - // Ensure we do not hold a lock on the index while fn executes in case fn tries - // to acquire a lock on the index again. If another goroutine has Lock, this will - // deadlock. - i.mu.RLock() - mm := i.measurements[string(name)] - i.mu.RUnlock() - - if mm == nil { - return nil - } - - for _, key := range mm.TagKeys() { - if err := fn([]byte(key)); err != nil { - return err - } - } - - return nil -} - -// TagKeyCardinality returns the number of values for a measurement/tag key. -func (i *Index) TagKeyCardinality(name, key []byte) int { - i.mu.RLock() - mm := i.measurements[string(name)] - i.mu.RUnlock() - - if mm == nil { - return 0 - } - return mm.CardinalityBytes(key) -} - -// TagsForSeries returns the tag map for the passed in series -func (i *Index) TagsForSeries(key string) (models.Tags, error) { - i.mu.RLock() - ss := i.series[key] - i.mu.RUnlock() - - if ss == nil { - return nil, nil - } - return ss.Tags, nil -} - -// MeasurementNamesByExpr takes an expression containing only tags and returns a -// list of matching measurement names. -// -// TODO(edd): Remove authorisation from these methods. There shouldn't need to -// be any auth passed down into the index. -func (i *Index) MeasurementNamesByExpr(auth query.Authorizer, expr influxql.Expr) ([][]byte, error) { - i.mu.RLock() - defer i.mu.RUnlock() - - // Return all measurement names if no expression is provided. - if expr == nil { - a := make([][]byte, 0, len(i.measurements)) - for _, m := range i.measurements { - if m.Authorized(auth) { - a = append(a, m.NameBytes) - } - } - bytesutil.Sort(a) - return a, nil - } - - return i.measurementNamesByExpr(auth, expr) -} - -func (i *Index) measurementNamesByExpr(auth query.Authorizer, expr influxql.Expr) ([][]byte, error) { - if expr == nil { - return nil, nil - } - - switch e := expr.(type) { - case *influxql.BinaryExpr: - switch e.Op { - case influxql.EQ, influxql.NEQ, influxql.EQREGEX, influxql.NEQREGEX: - tag, ok := e.LHS.(*influxql.VarRef) - if !ok { - return nil, fmt.Errorf("left side of '%s' must be a tag key", e.Op.String()) - } - - tf := &TagFilter{ - Op: e.Op, - Key: tag.Val, - } - - if influxql.IsRegexOp(e.Op) { - re, ok := e.RHS.(*influxql.RegexLiteral) - if !ok { - return nil, fmt.Errorf("right side of '%s' must be a regular expression", e.Op.String()) - } - tf.Regex = re.Val - } else { - s, ok := e.RHS.(*influxql.StringLiteral) - if !ok { - return nil, fmt.Errorf("right side of '%s' must be a tag value string", e.Op.String()) - } - tf.Value = s.Val - } - - // Match on name, if specified. - if tag.Val == "_name" { - return i.measurementNamesByNameFilter(auth, tf.Op, tf.Value, tf.Regex), nil - } else if influxql.IsSystemName(tag.Val) { - return nil, nil - } - - return i.measurementNamesByTagFilters(auth, tf), nil - case influxql.OR, influxql.AND: - lhs, err := i.measurementNamesByExpr(auth, e.LHS) - if err != nil { - return nil, err - } - - rhs, err := i.measurementNamesByExpr(auth, e.RHS) - if err != nil { - return nil, err - } - - if e.Op == influxql.OR { - return bytesutil.Union(lhs, rhs), nil - } - return bytesutil.Intersect(lhs, rhs), nil - default: - return nil, fmt.Errorf("invalid tag comparison operator") - } - case *influxql.ParenExpr: - return i.measurementNamesByExpr(auth, e.Expr) - } - return nil, fmt.Errorf("%#v", expr) -} - -// measurementNamesByNameFilter returns the sorted measurements matching a name. -func (i *Index) measurementNamesByNameFilter(auth query.Authorizer, op influxql.Token, val string, regex *regexp.Regexp) [][]byte { - var names [][]byte - for _, m := range i.measurements { - var matched bool - switch op { - case influxql.EQ: - matched = m.Name == val - case influxql.NEQ: - matched = m.Name != val - case influxql.EQREGEX: - matched = regex.MatchString(m.Name) - case influxql.NEQREGEX: - matched = !regex.MatchString(m.Name) - } - - if matched && m.Authorized(auth) { - names = append(names, m.NameBytes) - } - } - bytesutil.Sort(names) - return names -} - -// measurementNamesByTagFilters returns the sorted measurements matching the filters on tag values. -func (i *Index) measurementNamesByTagFilters(auth query.Authorizer, filter *TagFilter) [][]byte { - // Build a list of measurements matching the filters. - var names [][]byte - var tagMatch bool - var authorized bool - - valEqual := filter.Regex.MatchString - if filter.Op == influxql.EQ || filter.Op == influxql.NEQ { - valEqual = func(s string) bool { return filter.Value == s } - } - - // Iterate through all measurements in the database. - for _, m := range i.measurements { - tagVals := m.SeriesByTagKeyValue(filter.Key) - if tagVals == nil { - continue - } - - tagMatch = false - // Authorization must be explicitly granted when an authorizer is present. - authorized = query.AuthorizerIsOpen(auth) - - // Check the tag values belonging to the tag key for equivalence to the - // tag value being filtered on. - tagVals.Range(func(tv string, seriesIDs seriesIDs) bool { - if !valEqual(tv) { - return true // No match. Keep checking. - } - - tagMatch = true - if query.AuthorizerIsOpen(auth) { - return false // No need to continue checking series, there is a match. - } - - // Is there a series with this matching tag value that is - // authorized to be read? - for _, sid := range seriesIDs { - s := m.SeriesByID(sid) - - // If the series is deleted then it can't be used to authorise against. - if s != nil && s.Deleted() { - continue - } - - if s != nil && auth.AuthorizeSeriesRead(i.database, m.NameBytes, s.Tags) { - // The Range call can return early as a matching - // tag value with an authorized series has been found. - authorized = true - return false - } - } - - // The matching tag value doesn't have any authorized series. - // Check for other matching tag values if this is a regex check. - return filter.Op == influxql.EQREGEX - }) - - // For negation operators, to determine if the measurement is authorized, - // an authorized series belonging to the measurement must be located. - // Then, the measurement can be added iff !tagMatch && authorized. - if auth != nil && !tagMatch && (filter.Op == influxql.NEQREGEX || filter.Op == influxql.NEQ) { - authorized = m.Authorized(auth) - } - - // tags match | operation is EQ | measurement matches - // -------------------------------------------------- - // True | True | True - // True | False | False - // False | True | False - // False | False | True - if tagMatch == (filter.Op == influxql.EQ || filter.Op == influxql.EQREGEX) && authorized { - names = append(names, m.NameBytes) - } - } - - bytesutil.Sort(names) - return names -} - -// MeasurementNamesByRegex returns the measurements that match the regex. -func (i *Index) MeasurementNamesByRegex(re *regexp.Regexp) ([][]byte, error) { - i.mu.RLock() - defer i.mu.RUnlock() - - var matches [][]byte - for _, m := range i.measurements { - if re.MatchString(m.Name) { - matches = append(matches, m.NameBytes) - } - } - return matches, nil -} - -// DropMeasurement removes the measurement and all of its underlying -// series from the database index -func (i *Index) DropMeasurement(name []byte) error { - i.mu.Lock() - defer i.mu.Unlock() - return i.dropMeasurement(string(name)) -} - -func (i *Index) dropMeasurement(name string) error { - // Update the tombstone sketch. - i.measurementsTSSketch.Add([]byte(name)) - - m := i.measurements[name] - if m == nil { - return nil - } - - delete(i.measurements, name) - for _, s := range m.SeriesByIDMap() { - delete(i.series, s.Key) - i.seriesTSSketch.Add([]byte(s.Key)) - } - return nil -} - -// DropMeasurementIfSeriesNotExist drops a measurement only if there are no more -// series for the measurement. -func (i *Index) DropMeasurementIfSeriesNotExist(name []byte) (bool, error) { - i.mu.Lock() - defer i.mu.Unlock() - - m := i.measurements[string(name)] - if m == nil { - return false, nil - } - - if m.HasSeries() { - return false, nil - } - - return true, i.dropMeasurement(string(name)) -} - -// DropSeriesGlobal removes the series key and its tags from the index. -func (i *Index) DropSeriesGlobal(key []byte) error { - if key == nil { - return nil - } - - i.mu.Lock() - defer i.mu.Unlock() - - k := string(key) - series := i.series[k] - if series == nil { - return nil - } - - // Update the tombstone sketch. - i.seriesTSSketch.Add([]byte(k)) - - // Remove from the index. - delete(i.series, k) - - // Remove the measurement's reference. - series.Measurement.DropSeries(series) - // Mark the series as deleted. - series.Delete() - - // If the measurement no longer has any series, remove it as well. - if !series.Measurement.HasSeries() { - i.dropMeasurement(series.Measurement.Name) - } - - return nil -} - -// TagSets returns a list of tag sets. -func (i *Index) TagSets(shardSeriesIDs *tsdb.SeriesIDSet, name []byte, opt query.IteratorOptions) ([]*query.TagSet, error) { - i.mu.RLock() - defer i.mu.RUnlock() - - mm := i.measurements[string(name)] - if mm == nil { - return nil, nil - } - - tagSets, err := mm.TagSets(shardSeriesIDs, opt) - if err != nil { - return nil, err - } - - return tagSets, nil -} - -func (i *Index) SeriesKeys() []string { - i.mu.RLock() - s := make([]string, 0, len(i.series)) - for k := range i.series { - s = append(s, k) - } - i.mu.RUnlock() - return s - -} - -// SetFieldSet sets a shared field set from the engine. -func (i *Index) SetFieldSet(fieldset *tsdb.MeasurementFieldSet) { - i.mu.Lock() - defer i.mu.Unlock() - i.fieldset = fieldset -} - -// FieldSet returns the assigned fieldset. -func (i *Index) FieldSet() *tsdb.MeasurementFieldSet { - i.mu.RLock() - defer i.mu.RUnlock() - return i.fieldset -} - -// SetFieldName adds a field name to a measurement. -func (i *Index) SetFieldName(measurement []byte, name string) { - m := i.CreateMeasurementIndexIfNotExists(measurement) - m.SetFieldName(name) -} - -// ForEachMeasurementName iterates over each measurement name. -func (i *Index) ForEachMeasurementName(fn func(name []byte) error) error { - i.mu.RLock() - mms := make(measurements, 0, len(i.measurements)) - for _, m := range i.measurements { - mms = append(mms, m) - } - sort.Sort(mms) - i.mu.RUnlock() - - for _, m := range mms { - if err := fn(m.NameBytes); err != nil { - return err - } - } - return nil -} - -func (i *Index) MeasurementSeriesIDIterator(name []byte) (tsdb.SeriesIDIterator, error) { - return i.MeasurementSeriesKeysByExprIterator(name, nil) -} - -func (i *Index) TagKeySeriesIDIterator(name, key []byte) (tsdb.SeriesIDIterator, error) { - i.mu.RLock() - defer i.mu.RUnlock() - - m := i.measurements[string(name)] - if m == nil { - return nil, nil - } - return tsdb.NewSeriesIDSliceIterator([]uint64(m.SeriesIDsByTagKey(key))), nil -} - -func (i *Index) TagValueSeriesIDIterator(name, key, value []byte) (tsdb.SeriesIDIterator, error) { - i.mu.RLock() - defer i.mu.RUnlock() - - m := i.measurements[string(name)] - if m == nil { - return nil, nil - } - return tsdb.NewSeriesIDSliceIterator([]uint64(m.SeriesIDsByTagValue(key, value))), nil -} - -func (i *Index) TagKeyIterator(name []byte) (tsdb.TagKeyIterator, error) { - i.mu.RLock() - defer i.mu.RUnlock() - - m := i.measurements[string(name)] - if m == nil { - return nil, nil - } - keys := m.TagKeys() - sort.Strings(keys) - - a := make([][]byte, len(keys)) - for i := range a { - a[i] = []byte(keys[i]) - } - return tsdb.NewTagKeySliceIterator(a), nil -} - -// TagValueIterator provides an iterator over all the tag values belonging to -// series with the provided measurement name and tag key. -// -// TagValueIterator does not currently support authorization. -func (i *Index) TagValueIterator(name, key []byte) (tsdb.TagValueIterator, error) { - i.mu.RLock() - defer i.mu.RUnlock() - - m := i.measurements[string(name)] - if m == nil { - return nil, nil - } - values := m.TagValues(nil, string(key)) - sort.Strings(values) - - a := make([][]byte, len(values)) - for i := range a { - a[i] = []byte(values[i]) - } - return tsdb.NewTagValueSliceIterator(a), nil -} - -func (i *Index) MeasurementSeriesKeysByExprIterator(name []byte, condition influxql.Expr) (tsdb.SeriesIDIterator, error) { - i.mu.RLock() - defer i.mu.RUnlock() - - m := i.measurements[string(name)] - if m == nil { - return nil, nil - } - - // Return all series if no condition specified. - if condition == nil { - return tsdb.NewSeriesIDSliceIterator([]uint64(m.SeriesIDs())), nil - } - - // Get series IDs that match the WHERE clause. - ids, filters, err := m.WalkWhereForSeriesIds(condition) - if err != nil { - return nil, err - } - - // Delete boolean literal true filter expressions. - // These are returned for `WHERE tagKey = 'tagVal'` type expressions and are okay. - filters.DeleteBoolLiteralTrues() - - // Check for unsupported field filters. - // Any remaining filters means there were fields (e.g., `WHERE value = 1.2`). - if filters.Len() > 0 { - return nil, errors.New("fields not supported in WHERE clause during deletion") - } - - return tsdb.NewSeriesIDSliceIterator([]uint64(ids)), nil -} - -func (i *Index) MeasurementSeriesKeysByExpr(name []byte, condition influxql.Expr) ([][]byte, error) { - i.mu.RLock() - defer i.mu.RUnlock() - - m := i.measurements[string(name)] - if m == nil { - return nil, nil - } - - // Return all series if no condition specified. - if condition == nil { - return m.SeriesKeys(), nil - } - - // Get series IDs that match the WHERE clause. - ids, filters, err := m.WalkWhereForSeriesIds(condition) - if err != nil { - return nil, err - } - - // Delete boolean literal true filter expressions. - // These are returned for `WHERE tagKey = 'tagVal'` type expressions and are okay. - filters.DeleteBoolLiteralTrues() - - // Check for unsupported field filters. - // Any remaining filters means there were fields (e.g., `WHERE value = 1.2`). - if filters.Len() > 0 { - return nil, errors.New("fields not supported in WHERE clause during deletion") - } - - return m.SeriesKeysByID(ids), nil -} - -// SeriesIDIterator returns an influxql iterator over matching series ids. -func (i *Index) SeriesIDIterator(opt query.IteratorOptions) (tsdb.SeriesIDIterator, error) { - i.mu.RLock() - defer i.mu.RUnlock() - - // Read and sort all measurements. - mms := make(measurements, 0, len(i.measurements)) - for _, mm := range i.measurements { - mms = append(mms, mm) - } - sort.Sort(mms) - - return &seriesIDIterator{ - database: i.database, - mms: mms, - opt: opt, - }, nil -} - -// DiskSizeBytes always returns zero bytes, since this is an in-memory index. -func (i *Index) DiskSizeBytes() int64 { return 0 } - -// Rebuild recreates the measurement indexes to allow deleted series to be removed -// and garbage collected. -func (i *Index) Rebuild() { - // Only allow one rebuild at a time. This will cause all subsequent rebuilds - // to queue. The measurement rebuild is idempotent and will not be rebuilt if - // it does not need to be. - i.rebuildQueue.Lock() - defer i.rebuildQueue.Unlock() - - i.ForEachMeasurementName(func(name []byte) error { - // Measurement never returns an error - m, _ := i.Measurement(name) - if m == nil { - return nil - } - - i.mu.Lock() - nm := m.Rebuild() - - i.measurements[string(name)] = nm - i.mu.Unlock() - return nil - }) -} - -// assignExistingSeries assigns the existing series to shardID and returns the series, names and tags that -// do not exists yet. -func (i *Index) assignExistingSeries(shardID uint64, seriesIDSet *tsdb.SeriesIDSet, measurements map[string]int, - keys, names [][]byte, tagsSlice []models.Tags) ([][]byte, [][]byte, []models.Tags) { - - i.mu.RLock() - var n int - for j, key := range keys { - if ss := i.series[string(key)]; ss == nil { - keys[n] = keys[j] - names[n] = names[j] - tagsSlice[n] = tagsSlice[j] - n++ - } else { - // Add the existing series to this shard's bitset, since this may - // be the first time the series is added to this shard. - if !seriesIDSet.Contains(ss.ID) { - seriesIDSet.Lock() - if !seriesIDSet.ContainsNoLock(ss.ID) { - seriesIDSet.AddNoLock(ss.ID) - measurements[string(names[j])]++ - } - seriesIDSet.Unlock() - } - } - } - i.mu.RUnlock() - return keys[:n], names[:n], tagsSlice[:n] -} - -// Ensure index implements interface. -var _ tsdb.Index = &ShardIndex{} - -// ShardIndex represents a shim between the TSDB index interface and the shared -// in-memory index. This is required because per-shard in-memory indexes will -// grow the heap size too large. -type ShardIndex struct { - id uint64 // shard id - - *Index // Shared reference to global database-wide index. - - // Bitset storing all undeleted series IDs associated with this shard. - seriesIDSet *tsdb.SeriesIDSet - - // mapping of measurements to the count of series ids in the set. protected - // by the seriesIDSet lock. - measurements map[string]int - - opt tsdb.EngineOptions -} - -// DropSeries removes the provided series id from the local bitset that tracks -// series in this shard only. -func (idx *ShardIndex) DropSeries(seriesID uint64, key []byte, _ bool) error { - // Remove from shard-local bitset if it exists. - idx.seriesIDSet.Lock() - if idx.seriesIDSet.ContainsNoLock(seriesID) { - idx.seriesIDSet.RemoveNoLock(seriesID) - - name := models.ParseName(key) - if curr := idx.measurements[string(name)]; curr <= 1 { - delete(idx.measurements, string(name)) - } else { - idx.measurements[string(name)] = curr - 1 - } - } - idx.seriesIDSet.Unlock() - return nil -} - -// DropMeasurementIfSeriesNotExist drops a measurement only if there are no more -// series for the measurement. -func (idx *ShardIndex) DropMeasurementIfSeriesNotExist(name []byte) (bool, error) { - idx.seriesIDSet.Lock() - curr := idx.measurements[string(name)] - idx.seriesIDSet.Unlock() - if curr > 0 { - return false, nil - } - - // we always report the measurement was dropped if it does not exist in our - // measurements mapping. - _, err := idx.Index.DropMeasurementIfSeriesNotExist(name) - return err == nil, err -} - -// CreateSeriesListIfNotExists creates a list of series if they doesn't exist in bulk. -func (idx *ShardIndex) CreateSeriesListIfNotExists(keys, names [][]byte, tagsSlice []models.Tags) error { - keys, names, tagsSlice = idx.assignExistingSeries(idx.id, idx.seriesIDSet, idx.measurements, keys, names, tagsSlice) - if len(keys) == 0 { - return nil - } - - var ( - reason string - droppedKeys [][]byte - ) - - // Ensure that no tags go over the maximum cardinality. - if maxValuesPerTag := idx.opt.Config.MaxValuesPerTag; maxValuesPerTag > 0 { - var n int - - outer: - for i, name := range names { - tags := tagsSlice[i] - for _, tag := range tags { - // Skip if the tag value already exists. - if ok, _ := idx.HasTagValue(name, tag.Key, tag.Value); ok { - continue - } - - // Read cardinality. Skip if we're below the threshold. - n := idx.TagValueN(name, tag.Key) - if n < maxValuesPerTag { - continue - } - - if reason == "" { - reason = fmt.Sprintf("max-values-per-tag limit exceeded (%d/%d): measurement=%q tag=%q value=%q", - n, maxValuesPerTag, name, string(tag.Key), string(tag.Value)) - } - - droppedKeys = append(droppedKeys, keys[i]) - continue outer - } - - // Increment success count if all checks complete. - if n != i { - keys[n], names[n], tagsSlice[n] = keys[i], names[i], tagsSlice[i] - } - n++ - } - - // Slice to only include successful points. - keys, names, tagsSlice = keys[:n], names[:n], tagsSlice[:n] - } - - if err := idx.Index.CreateSeriesListIfNotExists(idx.seriesIDSet, idx.measurements, keys, names, tagsSlice, &idx.opt, idx.opt.Config.MaxSeriesPerDatabase == 0); err != nil { - reason = err.Error() - droppedKeys = append(droppedKeys, keys...) - } - - // Report partial writes back to shard. - if len(droppedKeys) > 0 { - dropped := len(droppedKeys) // number dropped before deduping - bytesutil.SortDedup(droppedKeys) - return tsdb.PartialWriteError{ - Reason: reason, - Dropped: dropped, - DroppedKeys: droppedKeys, - } - } - - return nil -} - -// SeriesN returns the number of unique non-tombstoned series local to this shard. -func (idx *ShardIndex) SeriesN() int64 { - idx.mu.RLock() - defer idx.mu.RUnlock() - return int64(idx.seriesIDSet.Cardinality()) -} - -// InitializeSeries is called during start-up. -// This works the same as CreateSeriesListIfNotExists except it ignore limit errors. -func (idx *ShardIndex) InitializeSeries(keys, names [][]byte, tags []models.Tags) error { - return idx.Index.CreateSeriesListIfNotExists(idx.seriesIDSet, idx.measurements, keys, names, tags, &idx.opt, true) -} - -// CreateSeriesIfNotExists creates the provided series on the index if it is not -// already present. -func (idx *ShardIndex) CreateSeriesIfNotExists(key, name []byte, tags models.Tags) error { - return idx.Index.CreateSeriesListIfNotExists(idx.seriesIDSet, idx.measurements, [][]byte{key}, [][]byte{name}, []models.Tags{tags}, &idx.opt, false) -} - -// TagSets returns a list of tag sets based on series filtering. -func (idx *ShardIndex) TagSets(name []byte, opt query.IteratorOptions) ([]*query.TagSet, error) { - return idx.Index.TagSets(idx.seriesIDSet, name, opt) -} - -// SeriesIDSet returns the bitset associated with the series ids. -func (idx *ShardIndex) SeriesIDSet() *tsdb.SeriesIDSet { - return idx.seriesIDSet -} - -// NewShardIndex returns a new index for a shard. -func NewShardIndex(id uint64, seriesIDSet *tsdb.SeriesIDSet, opt tsdb.EngineOptions) tsdb.Index { - return &ShardIndex{ - Index: opt.InmemIndex.(*Index), - id: id, - seriesIDSet: seriesIDSet, - measurements: make(map[string]int), - opt: opt, - } -} - -// seriesIDIterator emits series ids. -type seriesIDIterator struct { - database string - mms measurements - keys struct { - buf []*series - i int - } - opt query.IteratorOptions -} - -// Stats returns stats about the points processed. -func (itr *seriesIDIterator) Stats() query.IteratorStats { return query.IteratorStats{} } - -// Close closes the iterator. -func (itr *seriesIDIterator) Close() error { return nil } - -// Next emits the next point in the iterator. -func (itr *seriesIDIterator) Next() (tsdb.SeriesIDElem, error) { - for { - // Load next measurement's keys if there are no more remaining. - if itr.keys.i >= len(itr.keys.buf) { - if err := itr.nextKeys(); err != nil { - return tsdb.SeriesIDElem{}, err - } - if len(itr.keys.buf) == 0 { - return tsdb.SeriesIDElem{}, nil - } - } - - // Read the next key. - series := itr.keys.buf[itr.keys.i] - itr.keys.i++ - - if !itr.opt.Authorizer.AuthorizeSeriesRead(itr.database, series.Measurement.NameBytes, series.Tags) { - continue - } - - return tsdb.SeriesIDElem{SeriesID: series.ID}, nil - } -} - -// nextKeys reads all keys for the next measurement. -func (itr *seriesIDIterator) nextKeys() error { - for { - // Ensure previous keys are cleared out. - itr.keys.i, itr.keys.buf = 0, itr.keys.buf[:0] - - // Read next measurement. - if len(itr.mms) == 0 { - return nil - } - mm := itr.mms[0] - itr.mms = itr.mms[1:] - - // Read all series keys. - ids, err := mm.SeriesIDsAllOrByExpr(itr.opt.Condition) - if err != nil { - return err - } else if len(ids) == 0 { - continue - } - itr.keys.buf = mm.SeriesByIDSlice(ids) - - // Sort series by key - sort.Slice(itr.keys.buf, func(i, j int) bool { - return itr.keys.buf[i].Key < itr.keys.buf[j].Key - }) - - return nil - } -} - -// errMaxSeriesPerDatabaseExceeded is a marker error returned during series creation -// to indicate that a new series would exceed the limits of the database. -type errMaxSeriesPerDatabaseExceeded struct { - limit int -} - -func (e errMaxSeriesPerDatabaseExceeded) Error() string { - return fmt.Sprintf("max-series-per-database limit exceeded: (%d)", e.limit) -} diff --git a/tsdb/index/inmem/inmem_test.go b/tsdb/index/inmem/inmem_test.go deleted file mode 100644 index ba2b6f3493..0000000000 --- a/tsdb/index/inmem/inmem_test.go +++ /dev/null @@ -1,186 +0,0 @@ -package inmem_test - -import ( - "fmt" - "io/ioutil" - "os" - "testing" - - "github.com/influxdata/influxdb/v2/models" - "github.com/influxdata/influxdb/v2/tsdb" - "github.com/influxdata/influxdb/v2/tsdb/index/inmem" -) - -func createData(lo, hi int) (keys, names [][]byte, tags []models.Tags) { - for i := lo; i < hi; i++ { - keys = append(keys, []byte(fmt.Sprintf("m0,tag0=t%d", i))) - names = append(names, []byte("m0")) - var t models.Tags - t.Set([]byte("tag0"), []byte(fmt.Sprintf("%d", i))) - tags = append(tags, t) - } - return -} - -func BenchmarkShardIndex_CreateSeriesListIfNotExists_MaxValuesExceeded(b *testing.B) { - sfile := mustOpenSeriesFile() - defer sfile.Close() - opt := tsdb.EngineOptions{InmemIndex: inmem.NewIndex("foo", sfile.SeriesFile)} - opt.Config.MaxValuesPerTag = 10 - si := inmem.NewShardIndex(1, tsdb.NewSeriesIDSet(), opt) - si.Open() - keys, names, tags := createData(0, 10) - si.CreateSeriesListIfNotExists(keys, names, tags) - b.ReportAllocs() - b.ResetTimer() - - keys, names, tags = createData(9, 5010) - for i := 0; i < b.N; i++ { - si.CreateSeriesListIfNotExists(keys, names, tags) - } -} - -func BenchmarkShardIndex_CreateSeriesListIfNotExists_MaxValuesNotExceeded(b *testing.B) { - sfile := mustOpenSeriesFile() - defer sfile.Close() - opt := tsdb.EngineOptions{InmemIndex: inmem.NewIndex("foo", sfile.SeriesFile)} - opt.Config.MaxValuesPerTag = 100000 - si := inmem.NewShardIndex(1, tsdb.NewSeriesIDSet(), opt) - si.Open() - keys, names, tags := createData(0, 10) - si.CreateSeriesListIfNotExists(keys, names, tags) - b.ReportAllocs() - b.ResetTimer() - - keys, names, tags = createData(9, 5010) - for i := 0; i < b.N; i++ { - si.CreateSeriesListIfNotExists(keys, names, tags) - } -} - -func BenchmarkShardIndex_CreateSeriesListIfNotExists_NoMaxValues(b *testing.B) { - sfile := mustOpenSeriesFile() - defer sfile.Close() - opt := tsdb.EngineOptions{InmemIndex: inmem.NewIndex("foo", sfile.SeriesFile)} - si := inmem.NewShardIndex(1, tsdb.NewSeriesIDSet(), opt) - si.Open() - keys, names, tags := createData(0, 10) - si.CreateSeriesListIfNotExists(keys, names, tags) - b.ReportAllocs() - b.ResetTimer() - - keys, names, tags = createData(9, 5010) - for i := 0; i < b.N; i++ { - si.CreateSeriesListIfNotExists(keys, names, tags) - } -} - -func BenchmarkShardIndex_CreateSeriesListIfNotExists_MaxSeriesExceeded(b *testing.B) { - sfile := mustOpenSeriesFile() - defer sfile.Close() - opt := tsdb.EngineOptions{InmemIndex: inmem.NewIndex("foo", sfile.SeriesFile)} - opt.Config.MaxValuesPerTag = 0 - opt.Config.MaxSeriesPerDatabase = 10 - si := inmem.NewShardIndex(1, tsdb.NewSeriesIDSet(), opt) - si.Open() - keys, names, tags := createData(0, 10) - si.CreateSeriesListIfNotExists(keys, names, tags) - b.ReportAllocs() - b.ResetTimer() - - keys, names, tags = createData(9, 5010) - for i := 0; i < b.N; i++ { - si.CreateSeriesListIfNotExists(keys, names, tags) - } -} - -func TestIndex_Bytes(t *testing.T) { - sfile := mustOpenSeriesFile() - defer sfile.Close() - opt := tsdb.EngineOptions{InmemIndex: inmem.NewIndex("foo", sfile.SeriesFile)} - si := inmem.NewShardIndex(1, tsdb.NewSeriesIDSet(), opt).(*inmem.ShardIndex) - - indexBaseBytes := si.Bytes() - - name := []byte("name") - err := si.CreateSeriesIfNotExists(name, name, models.Tags{}) - if err != nil { - t.Error(err) - t.FailNow() - } - - indexNewBytes := si.Bytes() - if indexBaseBytes >= indexNewBytes { - t.Errorf("index Bytes(): want >%d, got %d", indexBaseBytes, indexNewBytes) - } -} - -func TestIndex_MeasurementTracking(t *testing.T) { - sfile := mustOpenSeriesFile() - defer sfile.Close() - opt := tsdb.EngineOptions{InmemIndex: inmem.NewIndex("foo", sfile.SeriesFile)} - s1 := inmem.NewShardIndex(1, tsdb.NewSeriesIDSet(), opt).(*inmem.ShardIndex) - s2 := inmem.NewShardIndex(2, tsdb.NewSeriesIDSet(), opt).(*inmem.ShardIndex) - b := func(s string) []byte { return []byte(s) } - mt := func(k, v string) models.Tag { return models.Tag{Key: b(k), Value: b(v)} } - - s1.CreateSeriesIfNotExists(b("m,t=t1"), b("m"), models.Tags{mt("t", "t1")}) - s1.CreateSeriesIfNotExists(b("m,t=t2"), b("m"), models.Tags{mt("t", "t2")}) - s2.CreateSeriesIfNotExists(b("m,t=t1"), b("m"), models.Tags{mt("t", "t1")}) - s2.CreateSeriesIfNotExists(b("m,t=t2"), b("m"), models.Tags{mt("t", "t2")}) - series1, _ := s1.Series(b("m,t=t1")) - series2, _ := s1.Series(b("m,t=t2")) - - if ok, err := s1.DropMeasurementIfSeriesNotExist(b("m")); err != nil || ok { - t.Fatal("invalid drop") - } - if ok, err := s2.DropMeasurementIfSeriesNotExist(b("m")); err != nil || ok { - t.Fatal("invalid drop") - } - - s1.DropSeries(series1.ID, b(series1.Key), false) - s1.DropSeries(series2.ID, b(series2.Key), false) - - if ok, err := s1.DropMeasurementIfSeriesNotExist(b("m")); err != nil || !ok { - t.Fatal("invalid drop") - } - if ok, err := s2.DropMeasurementIfSeriesNotExist(b("m")); err != nil || ok { - t.Fatal("invalid drop") - } - - s2.DropSeries(series1.ID, b(series1.Key), false) - s2.DropSeries(series2.ID, b(series2.Key), false) - - if ok, err := s2.DropMeasurementIfSeriesNotExist(b("m")); err != nil || !ok { - t.Fatal("invalid drop") - } -} - -// seriesFileWrapper is a test wrapper for tsdb.seriesFileWrapper. -type seriesFileWrapper struct { - *tsdb.SeriesFile -} - -// newSeriesFileWrapper returns a new instance of seriesFileWrapper with a temporary file path. -func newSeriesFileWrapper() *seriesFileWrapper { - dir, err := ioutil.TempDir("", "tsdb-series-file-") - if err != nil { - panic(err) - } - return &seriesFileWrapper{SeriesFile: tsdb.NewSeriesFile(dir)} -} - -// mustOpenSeriesFile returns a new, open instance of seriesFileWrapper. Panic on error. -func mustOpenSeriesFile() *seriesFileWrapper { - f := newSeriesFileWrapper() - if err := f.Open(); err != nil { - panic(err) - } - return f -} - -// Close closes the log file and removes it from disk. -func (f *seriesFileWrapper) Close() error { - defer os.RemoveAll(f.Path()) - return f.SeriesFile.Close() -} diff --git a/tsdb/index/inmem/meta.go b/tsdb/index/inmem/meta.go deleted file mode 100644 index 213c128b35..0000000000 --- a/tsdb/index/inmem/meta.go +++ /dev/null @@ -1,1529 +0,0 @@ -package inmem - -import ( - "bytes" - "fmt" - "regexp" - "sort" - "sync" - "unsafe" - - "github.com/influxdata/influxdb/v2/influxql/query" - "github.com/influxdata/influxdb/v2/models" - "github.com/influxdata/influxdb/v2/pkg/bytesutil" - "github.com/influxdata/influxdb/v2/pkg/radix" - "github.com/influxdata/influxdb/v2/tsdb" - "github.com/influxdata/influxql" -) - -// Measurement represents a collection of time series in a database. It also -// contains in memory structures for indexing tags. Exported functions are -// goroutine safe while un-exported functions assume the caller will use the -// appropriate locks. -type measurement struct { - Database string - Name string `json:"name,omitempty"` - NameBytes []byte // cached version as []byte - - mu sync.RWMutex - fieldNames map[string]struct{} - - // in-memory index fields - seriesByID map[uint64]*series // lookup table for series by their id - seriesByTagKeyValue map[string]*tagKeyValue // map from tag key to value to sorted set of series ids - - // lazily created sorted series IDs - sortedSeriesIDs seriesIDs // sorted list of series IDs in this measurement - - // Indicates whether the seriesByTagKeyValueMap needs to be rebuilt as it contains deleted series - // that waste memory. - dirty bool -} - -// newMeasurement allocates and initializes a new Measurement. -func newMeasurement(database, name string) *measurement { - return &measurement{ - Database: database, - Name: name, - NameBytes: []byte(name), - - fieldNames: make(map[string]struct{}), - seriesByID: make(map[uint64]*series), - seriesByTagKeyValue: make(map[string]*tagKeyValue), - } -} - -// bytes estimates the memory footprint of this measurement, in bytes. -func (m *measurement) bytes() int { - var b int - m.mu.RLock() - b += int(unsafe.Sizeof(m.Database)) + len(m.Database) - b += int(unsafe.Sizeof(m.Name)) + len(m.Name) - if m.NameBytes != nil { - b += int(unsafe.Sizeof(m.NameBytes)) + len(m.NameBytes) - } - b += 24 // 24 bytes for m.mu RWMutex - b += int(unsafe.Sizeof(m.fieldNames)) - for fieldName := range m.fieldNames { - b += int(unsafe.Sizeof(fieldName)) + len(fieldName) - } - b += int(unsafe.Sizeof(m.seriesByID)) - for k, v := range m.seriesByID { - b += int(unsafe.Sizeof(k)) - b += int(unsafe.Sizeof(v)) - // Do not count footprint of each series, to avoid double-counting in Index.bytes(). - } - b += int(unsafe.Sizeof(m.seriesByTagKeyValue)) - for k, v := range m.seriesByTagKeyValue { - b += int(unsafe.Sizeof(k)) + len(k) - b += int(unsafe.Sizeof(v)) + v.bytes() - } - b += int(unsafe.Sizeof(m.sortedSeriesIDs)) - for _, seriesID := range m.sortedSeriesIDs { - b += int(unsafe.Sizeof(seriesID)) - } - b += int(unsafe.Sizeof(m.dirty)) - m.mu.RUnlock() - return b -} - -// Authorized determines if this Measurement is authorized to be read, according -// to the provided Authorizer. A measurement is authorized to be read if at -// least one undeleted series from the measurement is authorized to be read. -func (m *measurement) Authorized(auth query.Authorizer) bool { - // Note(edd): the cost of this check scales linearly with the number of series - // belonging to a measurement, which means it may become expensive when there - // are large numbers of series on a measurement. - // - // In the future we might want to push the set of series down into the - // authorizer, but that will require an API change. - for _, s := range m.SeriesByIDMap() { - if s != nil && s.Deleted() { - continue - } - - if query.AuthorizerIsOpen(auth) || auth.AuthorizeSeriesRead(m.Database, m.NameBytes, s.Tags) { - return true - } - } - return false -} - -func (m *measurement) HasField(name string) bool { - m.mu.RLock() - _, hasField := m.fieldNames[name] - m.mu.RUnlock() - return hasField -} - -// SeriesByID returns a series by identifier. -func (m *measurement) SeriesByID(id uint64) *series { - m.mu.RLock() - defer m.mu.RUnlock() - return m.seriesByID[id] -} - -// SeriesByIDMap returns the internal seriesByID map. -func (m *measurement) SeriesByIDMap() map[uint64]*series { - m.mu.RLock() - defer m.mu.RUnlock() - return m.seriesByID -} - -// SeriesByIDSlice returns a list of series by identifiers. -func (m *measurement) SeriesByIDSlice(ids []uint64) []*series { - m.mu.RLock() - defer m.mu.RUnlock() - a := make([]*series, len(ids)) - for i, id := range ids { - a[i] = m.seriesByID[id] - } - return a -} - -// AppendSeriesKeysByID appends keys for a list of series ids to a buffer. -func (m *measurement) AppendSeriesKeysByID(dst []string, ids []uint64) []string { - m.mu.RLock() - defer m.mu.RUnlock() - for _, id := range ids { - if s := m.seriesByID[id]; s != nil && !s.Deleted() { - dst = append(dst, s.Key) - } - } - return dst -} - -// SeriesKeysByID returns the a list of keys for a set of ids. -func (m *measurement) SeriesKeysByID(ids seriesIDs) [][]byte { - m.mu.RLock() - defer m.mu.RUnlock() - keys := make([][]byte, 0, len(ids)) - for _, id := range ids { - s := m.seriesByID[id] - if s == nil || s.Deleted() { - continue - } - keys = append(keys, []byte(s.Key)) - } - - if !bytesutil.IsSorted(keys) { - bytesutil.Sort(keys) - } - - return keys -} - -// SeriesKeys returns the keys of every series in this measurement -func (m *measurement) SeriesKeys() [][]byte { - m.mu.RLock() - defer m.mu.RUnlock() - keys := make([][]byte, 0, len(m.seriesByID)) - for _, s := range m.seriesByID { - if s.Deleted() { - continue - } - keys = append(keys, []byte(s.Key)) - } - - if !bytesutil.IsSorted(keys) { - bytesutil.Sort(keys) - } - - return keys -} - -func (m *measurement) SeriesIDs() seriesIDs { - m.mu.RLock() - if len(m.sortedSeriesIDs) == len(m.seriesByID) { - s := m.sortedSeriesIDs - m.mu.RUnlock() - return s - } - m.mu.RUnlock() - - m.mu.Lock() - if len(m.sortedSeriesIDs) == len(m.seriesByID) { - s := m.sortedSeriesIDs - m.mu.Unlock() - return s - } - - m.sortedSeriesIDs = m.sortedSeriesIDs[:0] - if cap(m.sortedSeriesIDs) < len(m.seriesByID) { - m.sortedSeriesIDs = make(seriesIDs, 0, len(m.seriesByID)) - } - - for k, v := range m.seriesByID { - if v.Deleted() { - continue - } - m.sortedSeriesIDs = append(m.sortedSeriesIDs, k) - } - sort.Sort(m.sortedSeriesIDs) - s := m.sortedSeriesIDs - m.mu.Unlock() - return s -} - -// HasTagKey returns true if at least one series in this measurement has written a value for the passed in tag key -func (m *measurement) HasTagKey(k string) bool { - m.mu.RLock() - defer m.mu.RUnlock() - _, hasTag := m.seriesByTagKeyValue[k] - return hasTag -} - -func (m *measurement) HasTagKeyValue(k, v []byte) bool { - m.mu.RLock() - defer m.mu.RUnlock() - return m.seriesByTagKeyValue[string(k)].Contains(string(v)) -} - -// HasSeries returns true if there is at least 1 series under this measurement. -func (m *measurement) HasSeries() bool { - m.mu.RLock() - defer m.mu.RUnlock() - return len(m.seriesByID) > 0 -} - -// CardinalityBytes returns the number of values associated with the given tag key. -func (m *measurement) CardinalityBytes(key []byte) int { - m.mu.RLock() - defer m.mu.RUnlock() - return m.seriesByTagKeyValue[string(key)].Cardinality() -} - -// AddSeries adds a series to the measurement's index. -// It returns true if the series was added successfully or false if the series was already present. -func (m *measurement) AddSeries(s *series) bool { - if s == nil { - return false - } - - m.mu.RLock() - if m.seriesByID[s.ID] != nil { - m.mu.RUnlock() - return false - } - m.mu.RUnlock() - - m.mu.Lock() - defer m.mu.Unlock() - - if m.seriesByID[s.ID] != nil { - return false - } - - m.seriesByID[s.ID] = s - - if len(m.seriesByID) == 1 || (len(m.sortedSeriesIDs) == len(m.seriesByID)-1 && s.ID > m.sortedSeriesIDs[len(m.sortedSeriesIDs)-1]) { - m.sortedSeriesIDs = append(m.sortedSeriesIDs, s.ID) - } - - // add this series id to the tag index on the measurement - for _, t := range s.Tags { - valueMap := m.seriesByTagKeyValue[string(t.Key)] - if valueMap == nil { - valueMap = newTagKeyValue() - m.seriesByTagKeyValue[string(t.Key)] = valueMap - } - valueMap.InsertSeriesIDByte(t.Value, s.ID) - } - - return true -} - -// DropSeries removes a series from the measurement's index. -func (m *measurement) DropSeries(series *series) { - seriesID := series.ID - m.mu.Lock() - defer m.mu.Unlock() - - // Existence check before delete here to clean up the caching/indexing only when needed - if _, ok := m.seriesByID[seriesID]; !ok { - return - } - delete(m.seriesByID, seriesID) - - // clear our lazily sorted set of ids - m.sortedSeriesIDs = m.sortedSeriesIDs[:0] - - // Mark that this measurements tagValue map has stale entries that need to be rebuilt. - m.dirty = true -} - -func (m *measurement) Rebuild() *measurement { - m.mu.RLock() - - // Nothing needs to be rebuilt. - if !m.dirty { - m.mu.RUnlock() - return m - } - - // Create a new measurement from the state of the existing measurement - nm := newMeasurement(m.Database, string(m.NameBytes)) - nm.fieldNames = m.fieldNames - m.mu.RUnlock() - - // Re-add each series to allow the measurement indexes to get re-created. If there were - // deletes, the existing measurement may have references to deleted series that need to be - // expunged. Note: we're NOT using SeriesIDs which returns the series in sorted order because - // we need to do this under a write lock to prevent races. The series are added in sorted - // order to prevent resorting them again after they are all re-added. - m.mu.Lock() - defer m.mu.Unlock() - - for k, v := range m.seriesByID { - if v.Deleted() { - continue - } - m.sortedSeriesIDs = append(m.sortedSeriesIDs, k) - } - sort.Sort(m.sortedSeriesIDs) - - for _, id := range m.sortedSeriesIDs { - if s := m.seriesByID[id]; s != nil { - // Explicitly set the new measurement on the series. - s.Measurement = nm - nm.AddSeries(s) - } - } - return nm -} - -// filters walks the where clause of a select statement and returns a map with all series ids -// matching the where clause and any filter expression that should be applied to each -func (m *measurement) filters(condition influxql.Expr) ([]uint64, map[uint64]influxql.Expr, error) { - if condition == nil { - return m.SeriesIDs(), nil, nil - } - return m.WalkWhereForSeriesIds(condition) -} - -// TagSets returns the unique tag sets that exist for the given tag keys. This is used to determine -// what composite series will be created by a group by. i.e. "group by region" should return: -// {"region":"uswest"}, {"region":"useast"} -// or region, service returns -// {"region": "uswest", "service": "redis"}, {"region": "uswest", "service": "mysql"}, etc... -// This will also populate the TagSet objects with the series IDs that match each tagset and any -// influx filter expression that goes with the series -// TODO: this shouldn't be exported. However, until tx.go and the engine get refactored into tsdb, we need it. -func (m *measurement) TagSets(shardSeriesIDs *tsdb.SeriesIDSet, opt query.IteratorOptions) ([]*query.TagSet, error) { - // get the unique set of series ids and the filters that should be applied to each - ids, filters, err := m.filters(opt.Condition) - if err != nil { - return nil, err - } - - var dims []string - if len(opt.Dimensions) > 0 { - dims = make([]string, len(opt.Dimensions)) - copy(dims, opt.Dimensions) - sort.Strings(dims) - } - - m.mu.RLock() - // For every series, get the tag values for the requested tag keys i.e. dimensions. This is the - // TagSet for that series. Series with the same TagSet are then grouped together, because for the - // purpose of GROUP BY they are part of the same composite series. - tagSets := make(map[string]*query.TagSet, 64) - var seriesN int - for _, id := range ids { - // Abort if the query was killed - select { - case <-opt.InterruptCh: - m.mu.RUnlock() - return nil, query.ErrQueryInterrupted - default: - } - - if opt.MaxSeriesN > 0 && seriesN > opt.MaxSeriesN { - m.mu.RUnlock() - return nil, fmt.Errorf("max-select-series limit exceeded: (%d/%d)", seriesN, opt.MaxSeriesN) - } - - s := m.seriesByID[id] - if s == nil || s.Deleted() || !shardSeriesIDs.Contains(id) { - continue - } - - if opt.Authorizer != nil && !opt.Authorizer.AuthorizeSeriesRead(m.Database, m.NameBytes, s.Tags) { - continue - } - - var tagsAsKey []byte - if len(dims) > 0 { - tagsAsKey = tsdb.MakeTagsKey(dims, s.Tags) - } - - tagSet := tagSets[string(tagsAsKey)] - if tagSet == nil { - // This TagSet is new, create a new entry for it. - tagSet = &query.TagSet{ - Tags: nil, - Key: tagsAsKey, - } - tagSets[string(tagsAsKey)] = tagSet - } - // Associate the series and filter with the Tagset. - tagSet.AddFilter(s.Key, filters[id]) - seriesN++ - } - // Release the lock while we sort all the tags - m.mu.RUnlock() - - // Sort the series in each tag set. - for _, t := range tagSets { - // Abort if the query was killed - select { - case <-opt.InterruptCh: - return nil, query.ErrQueryInterrupted - default: - } - - sort.Sort(t) - } - - // The TagSets have been created, as a map of TagSets. Just send - // the values back as a slice, sorting for consistency. - sortedTagsSets := make([]*query.TagSet, 0, len(tagSets)) - for _, v := range tagSets { - sortedTagsSets = append(sortedTagsSets, v) - } - sort.Sort(byTagKey(sortedTagsSets)) - - return sortedTagsSets, nil -} - -// intersectSeriesFilters performs an intersection for two sets of ids and filter expressions. -func intersectSeriesFilters(lids, rids seriesIDs, lfilters, rfilters FilterExprs) (seriesIDs, FilterExprs) { - // We only want to allocate a slice and map of the smaller size. - var ids []uint64 - if len(lids) > len(rids) { - ids = make([]uint64, 0, len(rids)) - } else { - ids = make([]uint64, 0, len(lids)) - } - - var filters FilterExprs - if len(lfilters) > len(rfilters) { - filters = make(FilterExprs, len(rfilters)) - } else { - filters = make(FilterExprs, len(lfilters)) - } - - // They're in sorted order so advance the counter as needed. - // This is, don't run comparisons against lower values that we've already passed. - for len(lids) > 0 && len(rids) > 0 { - lid, rid := lids[0], rids[0] - if lid == rid { - ids = append(ids, lid) - - var expr influxql.Expr - lfilter := lfilters[lid] - rfilter := rfilters[rid] - - if lfilter != nil && rfilter != nil { - be := &influxql.BinaryExpr{ - Op: influxql.AND, - LHS: lfilter, - RHS: rfilter, - } - expr = influxql.Reduce(be, nil) - } else if lfilter != nil { - expr = lfilter - } else if rfilter != nil { - expr = rfilter - } - - if expr != nil { - filters[lid] = expr - } - lids, rids = lids[1:], rids[1:] - } else if lid < rid { - lids = lids[1:] - } else { - rids = rids[1:] - } - } - return ids, filters -} - -// unionSeriesFilters performs a union for two sets of ids and filter expressions. -func unionSeriesFilters(lids, rids seriesIDs, lfilters, rfilters FilterExprs) (seriesIDs, FilterExprs) { - ids := make([]uint64, 0, len(lids)+len(rids)) - - // Setup the filters with the smallest size since we will discard filters - // that do not have a match on the other side. - var filters FilterExprs - if len(lfilters) < len(rfilters) { - filters = make(FilterExprs, len(lfilters)) - } else { - filters = make(FilterExprs, len(rfilters)) - } - - for len(lids) > 0 && len(rids) > 0 { - lid, rid := lids[0], rids[0] - if lid == rid { - ids = append(ids, lid) - - // If one side does not have a filter, then the series has been - // included on one side of the OR with no condition. Eliminate the - // filter in this case. - var expr influxql.Expr - lfilter := lfilters[lid] - rfilter := rfilters[rid] - if lfilter != nil && rfilter != nil { - be := &influxql.BinaryExpr{ - Op: influxql.OR, - LHS: lfilter, - RHS: rfilter, - } - expr = influxql.Reduce(be, nil) - } - - if expr != nil { - filters[lid] = expr - } - lids, rids = lids[1:], rids[1:] - } else if lid < rid { - ids = append(ids, lid) - - filter := lfilters[lid] - if filter != nil { - filters[lid] = filter - } - lids = lids[1:] - } else { - ids = append(ids, rid) - - filter := rfilters[rid] - if filter != nil { - filters[rid] = filter - } - rids = rids[1:] - } - } - - // Now append the remainder. - if len(lids) > 0 { - for i := 0; i < len(lids); i++ { - ids = append(ids, lids[i]) - - filter := lfilters[lids[i]] - if filter != nil { - filters[lids[i]] = filter - } - } - } else if len(rids) > 0 { - for i := 0; i < len(rids); i++ { - ids = append(ids, rids[i]) - - filter := rfilters[rids[i]] - if filter != nil { - filters[rids[i]] = filter - } - } - } - return ids, filters -} - -// SeriesIDsByTagKey returns a list of all series for a tag key. -func (m *measurement) SeriesIDsByTagKey(key []byte) seriesIDs { - tagVals := m.seriesByTagKeyValue[string(key)] - if tagVals == nil { - return nil - } - - var ids seriesIDs - tagVals.RangeAll(func(_ string, a seriesIDs) { - ids = append(ids, a...) - }) - sort.Sort(ids) - return ids -} - -// SeriesIDsByTagValue returns a list of all series for a tag value. -func (m *measurement) SeriesIDsByTagValue(key, value []byte) seriesIDs { - tagVals := m.seriesByTagKeyValue[string(key)] - if tagVals == nil { - return nil - } - return tagVals.Load(string(value)) -} - -// IDsForExpr returns the series IDs that are candidates to match the given expression. -func (m *measurement) IDsForExpr(n *influxql.BinaryExpr) seriesIDs { - ids, _, _ := m.idsForExpr(n) - return ids -} - -// idsForExpr returns a collection of series ids and a filter expression that should -// be used to filter points from those series. -func (m *measurement) idsForExpr(n *influxql.BinaryExpr) (seriesIDs, influxql.Expr, error) { - // If this binary expression has another binary expression, then this - // is some expression math and we should just pass it to the underlying query. - if _, ok := n.LHS.(*influxql.BinaryExpr); ok { - return m.SeriesIDs(), n, nil - } else if _, ok := n.RHS.(*influxql.BinaryExpr); ok { - return m.SeriesIDs(), n, nil - } - - // Retrieve the variable reference from the correct side of the expression. - name, ok := n.LHS.(*influxql.VarRef) - value := n.RHS - if !ok { - name, ok = n.RHS.(*influxql.VarRef) - if !ok { - // This is an expression we do not know how to evaluate. Let the - // query engine take care of this. - return m.SeriesIDs(), n, nil - } - value = n.LHS - } - - // For fields, return all series IDs from this measurement and return - // the expression passed in, as the filter. - if name.Val != "_name" && ((name.Type == influxql.Unknown && m.HasField(name.Val)) || name.Type == influxql.AnyField || (name.Type != influxql.Tag && name.Type != influxql.Unknown)) { - return m.SeriesIDs(), n, nil - } else if value, ok := value.(*influxql.VarRef); ok { - // Check if the RHS is a variable and if it is a field. - if value.Val != "_name" && ((value.Type == influxql.Unknown && m.HasField(value.Val)) || name.Type == influxql.AnyField || (value.Type != influxql.Tag && value.Type != influxql.Unknown)) { - return m.SeriesIDs(), n, nil - } - } - - // Retrieve list of series with this tag key. - tagVals := m.seriesByTagKeyValue[name.Val] - - // if we're looking for series with a specific tag value - if str, ok := value.(*influxql.StringLiteral); ok { - var ids seriesIDs - - // Special handling for "_name" to match measurement name. - if name.Val == "_name" { - if (n.Op == influxql.EQ && str.Val == m.Name) || (n.Op == influxql.NEQ && str.Val != m.Name) { - return m.SeriesIDs(), nil, nil - } - return nil, nil, nil - } - - if n.Op == influxql.EQ { - if str.Val != "" { - // return series that have a tag of specific value. - ids = tagVals.Load(str.Val) - } else { - // Make a copy of all series ids and mark the ones we need to evict. - sIDs := newEvictSeriesIDs(m.SeriesIDs()) - - // Go through each slice and mark the values we find as zero so - // they can be removed later. - tagVals.RangeAll(func(_ string, a seriesIDs) { - sIDs.mark(a) - }) - - // Make a new slice with only the remaining ids. - ids = sIDs.evict() - } - } else if n.Op == influxql.NEQ { - if str.Val != "" { - ids = m.SeriesIDs() - if vals := tagVals.Load(str.Val); len(vals) > 0 { - ids = ids.Reject(vals) - } - } else { - tagVals.RangeAll(func(_ string, a seriesIDs) { - ids = append(ids, a...) - }) - sort.Sort(ids) - } - } - return ids, nil, nil - } - - // if we're looking for series with a tag value that matches a regex - if re, ok := value.(*influxql.RegexLiteral); ok { - var ids seriesIDs - - // Special handling for "_name" to match measurement name. - if name.Val == "_name" { - match := re.Val.MatchString(m.Name) - if (n.Op == influxql.EQREGEX && match) || (n.Op == influxql.NEQREGEX && !match) { - return m.SeriesIDs(), &influxql.BooleanLiteral{Val: true}, nil - } - return nil, nil, nil - } - - // Check if we match the empty string to see if we should include series - // that are missing the tag. - empty := re.Val.MatchString("") - - // Gather the series that match the regex. If we should include the empty string, - // start with the list of all series and reject series that don't match our condition. - // If we should not include the empty string, include series that match our condition. - if empty && n.Op == influxql.EQREGEX { - // See comments above for EQ with a StringLiteral. - sIDs := newEvictSeriesIDs(m.SeriesIDs()) - tagVals.RangeAll(func(k string, a seriesIDs) { - if !re.Val.MatchString(k) { - sIDs.mark(a) - } - }) - ids = sIDs.evict() - } else if empty && n.Op == influxql.NEQREGEX { - ids = make(seriesIDs, 0, len(m.SeriesIDs())) - tagVals.RangeAll(func(k string, a seriesIDs) { - if !re.Val.MatchString(k) { - ids = append(ids, a...) - } - }) - sort.Sort(ids) - } else if !empty && n.Op == influxql.EQREGEX { - ids = make(seriesIDs, 0, len(m.SeriesIDs())) - tagVals.RangeAll(func(k string, a seriesIDs) { - if re.Val.MatchString(k) { - ids = append(ids, a...) - } - }) - sort.Sort(ids) - } else if !empty && n.Op == influxql.NEQREGEX { - // See comments above for EQ with a StringLiteral. - sIDs := newEvictSeriesIDs(m.SeriesIDs()) - tagVals.RangeAll(func(k string, a seriesIDs) { - if re.Val.MatchString(k) { - sIDs.mark(a) - } - }) - ids = sIDs.evict() - } - return ids, nil, nil - } - - // compare tag values - if ref, ok := value.(*influxql.VarRef); ok { - var ids seriesIDs - - if n.Op == influxql.NEQ { - ids = m.SeriesIDs() - } - - rhsTagVals := m.seriesByTagKeyValue[ref.Val] - tagVals.RangeAll(func(k string, a seriesIDs) { - tags := a.Intersect(rhsTagVals.Load(k)) - if n.Op == influxql.EQ { - ids = ids.Union(tags) - } else if n.Op == influxql.NEQ { - ids = ids.Reject(tags) - } - }) - return ids, nil, nil - } - - // We do not know how to evaluate this expression so pass it - // on to the query engine. - return m.SeriesIDs(), n, nil -} - -// FilterExprs represents a map of series IDs to filter expressions. -type FilterExprs map[uint64]influxql.Expr - -// DeleteBoolLiteralTrues deletes all elements whose filter expression is a boolean literal true. -func (fe FilterExprs) DeleteBoolLiteralTrues() { - for id, expr := range fe { - if e, ok := expr.(*influxql.BooleanLiteral); ok && e.Val { - delete(fe, id) - } - } -} - -// Len returns the number of elements. -func (fe FilterExprs) Len() int { - if fe == nil { - return 0 - } - return len(fe) -} - -// WalkWhereForSeriesIds recursively walks the WHERE clause and returns an ordered set of series IDs and -// a map from those series IDs to filter expressions that should be used to limit points returned in -// the final query result. -func (m *measurement) WalkWhereForSeriesIds(expr influxql.Expr) (seriesIDs, FilterExprs, error) { - switch n := expr.(type) { - case *influxql.BinaryExpr: - switch n.Op { - case influxql.EQ, influxql.NEQ, influxql.LT, influxql.LTE, influxql.GT, influxql.GTE, influxql.EQREGEX, influxql.NEQREGEX: - // Get the series IDs and filter expression for the tag or field comparison. - ids, expr, err := m.idsForExpr(n) - if err != nil { - return nil, nil, err - } - - if len(ids) == 0 { - return ids, nil, nil - } - - // If the expression is a boolean literal that is true, ignore it. - if b, ok := expr.(*influxql.BooleanLiteral); ok && b.Val { - expr = nil - } - - var filters FilterExprs - if expr != nil { - filters = make(FilterExprs, len(ids)) - for _, id := range ids { - filters[id] = expr - } - } - - return ids, filters, nil - case influxql.AND, influxql.OR: - // Get the series IDs and filter expressions for the LHS. - lids, lfilters, err := m.WalkWhereForSeriesIds(n.LHS) - if err != nil { - return nil, nil, err - } - - // Get the series IDs and filter expressions for the RHS. - rids, rfilters, err := m.WalkWhereForSeriesIds(n.RHS) - if err != nil { - return nil, nil, err - } - - // Combine the series IDs from the LHS and RHS. - if n.Op == influxql.AND { - ids, filters := intersectSeriesFilters(lids, rids, lfilters, rfilters) - return ids, filters, nil - } else { - ids, filters := unionSeriesFilters(lids, rids, lfilters, rfilters) - return ids, filters, nil - } - } - - ids, _, err := m.idsForExpr(n) - return ids, nil, err - case *influxql.ParenExpr: - // walk down the tree - return m.WalkWhereForSeriesIds(n.Expr) - case *influxql.BooleanLiteral: - if n.Val { - return m.SeriesIDs(), nil, nil - } - return nil, nil, nil - default: - return nil, nil, nil - } -} - -// SeriesIDsAllOrByExpr walks an expressions for matching series IDs -// or, if no expressions is given, returns all series IDs for the measurement. -func (m *measurement) SeriesIDsAllOrByExpr(expr influxql.Expr) (seriesIDs, error) { - // If no expression given or the measurement has no series, - // we can take just return the ids or nil accordingly. - if expr == nil { - return m.SeriesIDs(), nil - } - - m.mu.RLock() - l := len(m.seriesByID) - m.mu.RUnlock() - if l == 0 { - return nil, nil - } - - // Get series IDs that match the WHERE clause. - ids, _, err := m.WalkWhereForSeriesIds(expr) - if err != nil { - return nil, err - } - - return ids, nil -} - -// tagKeysByExpr extracts the tag keys wanted by the expression. -func (m *measurement) TagKeysByExpr(expr influxql.Expr) (map[string]struct{}, error) { - if expr == nil { - set := make(map[string]struct{}) - for _, key := range m.TagKeys() { - set[key] = struct{}{} - } - return set, nil - } - - switch e := expr.(type) { - case *influxql.BinaryExpr: - switch e.Op { - case influxql.EQ, influxql.NEQ, influxql.EQREGEX, influxql.NEQREGEX: - tag, ok := e.LHS.(*influxql.VarRef) - if !ok { - return nil, fmt.Errorf("left side of '%s' must be a tag key", e.Op.String()) - } else if tag.Val != "_tagKey" { - return nil, nil - } - - if influxql.IsRegexOp(e.Op) { - re, ok := e.RHS.(*influxql.RegexLiteral) - if !ok { - return nil, fmt.Errorf("right side of '%s' must be a regular expression", e.Op.String()) - } - return m.tagKeysByFilter(e.Op, "", re.Val), nil - } - - s, ok := e.RHS.(*influxql.StringLiteral) - if !ok { - return nil, fmt.Errorf("right side of '%s' must be a tag value string", e.Op.String()) - } - return m.tagKeysByFilter(e.Op, s.Val, nil), nil - - case influxql.AND, influxql.OR: - lhs, err := m.TagKeysByExpr(e.LHS) - if err != nil { - return nil, err - } - - rhs, err := m.TagKeysByExpr(e.RHS) - if err != nil { - return nil, err - } - - if lhs != nil && rhs != nil { - if e.Op == influxql.OR { - return stringSet(lhs).union(rhs), nil - } - return stringSet(lhs).intersect(rhs), nil - } else if lhs != nil { - return lhs, nil - } else if rhs != nil { - return rhs, nil - } - return nil, nil - default: - return nil, fmt.Errorf("invalid operator") - } - - case *influxql.ParenExpr: - return m.TagKeysByExpr(e.Expr) - } - - return nil, fmt.Errorf("%#v", expr) -} - -// tagKeysByFilter will filter the tag keys for the measurement. -func (m *measurement) tagKeysByFilter(op influxql.Token, val string, regex *regexp.Regexp) stringSet { - ss := newStringSet() - for _, key := range m.TagKeys() { - var matched bool - switch op { - case influxql.EQ: - matched = key == val - case influxql.NEQ: - matched = key != val - case influxql.EQREGEX: - matched = regex.MatchString(key) - case influxql.NEQREGEX: - matched = !regex.MatchString(key) - } - - if !matched { - continue - } - ss.add(key) - } - return ss -} - -// Measurements represents a list of *Measurement. -type measurements []*measurement - -// Len implements sort.Interface. -func (a measurements) Len() int { return len(a) } - -// Less implements sort.Interface. -func (a measurements) Less(i, j int) bool { return a[i].Name < a[j].Name } - -// Swap implements sort.Interface. -func (a measurements) Swap(i, j int) { a[i], a[j] = a[j], a[i] } - -// series belong to a Measurement and represent unique time series in a database. -type series struct { - mu sync.RWMutex - deleted bool - - // immutable - ID uint64 - Measurement *measurement - Key string - Tags models.Tags -} - -// newSeries returns an initialized series struct -func newSeries(id uint64, m *measurement, key string, tags models.Tags) *series { - return &series{ - ID: id, - Measurement: m, - Key: key, - Tags: tags, - } -} - -// bytes estimates the memory footprint of this series, in bytes. -func (s *series) bytes() int { - var b int - s.mu.RLock() - b += 24 // RWMutex uses 24 bytes - b += int(unsafe.Sizeof(s.deleted) + unsafe.Sizeof(s.ID)) - // Do not count s.Measurement to prevent double-counting in Index.Bytes. - b += int(unsafe.Sizeof(s.Key)) + len(s.Key) - for _, tag := range s.Tags { - b += int(unsafe.Sizeof(tag)) + len(tag.Key) + len(tag.Value) - } - b += int(unsafe.Sizeof(s.Tags)) - s.mu.RUnlock() - return b -} - -// Delete marks this series as deleted. A deleted series should not be returned for queries. -func (s *series) Delete() { - s.mu.Lock() - s.deleted = true - s.mu.Unlock() -} - -// Deleted indicates if this was previously deleted. -func (s *series) Deleted() bool { - s.mu.RLock() - v := s.deleted - s.mu.RUnlock() - return v -} - -// TagKeyValue provides goroutine-safe concurrent access to the set of series -// ids mapping to a set of tag values. -type tagKeyValue struct { - mu sync.RWMutex - entries map[string]*tagKeyValueEntry -} - -// bytes estimates the memory footprint of this tagKeyValue, in bytes. -func (t *tagKeyValue) bytes() int { - var b int - t.mu.RLock() - b += 24 // RWMutex is 24 bytes - b += int(unsafe.Sizeof(t.entries)) - for k, v := range t.entries { - b += int(unsafe.Sizeof(k)) + len(k) - b += len(v.m) * 8 // uint64 - b += len(v.a) * 8 // uint64 - b += int(unsafe.Sizeof(v) + unsafe.Sizeof(*v)) - } - t.mu.RUnlock() - return b -} - -// NewTagKeyValue initialises a new TagKeyValue. -func newTagKeyValue() *tagKeyValue { - return &tagKeyValue{entries: make(map[string]*tagKeyValueEntry)} -} - -// Cardinality returns the number of values in the TagKeyValue. -func (t *tagKeyValue) Cardinality() int { - if t == nil { - return 0 - } - - t.mu.RLock() - defer t.mu.RUnlock() - return len(t.entries) -} - -// Contains returns true if the TagKeyValue contains value. -func (t *tagKeyValue) Contains(value string) bool { - if t == nil { - return false - } - - t.mu.RLock() - defer t.mu.RUnlock() - _, ok := t.entries[value] - return ok -} - -// InsertSeriesIDByte adds a series id to the tag key value. -func (t *tagKeyValue) InsertSeriesIDByte(value []byte, id uint64) { - t.mu.Lock() - entry := t.entries[string(value)] - if entry == nil { - entry = newTagKeyValueEntry() - t.entries[string(value)] = entry - } - entry.m[id] = struct{}{} - t.mu.Unlock() -} - -// Load returns the SeriesIDs for the provided tag value. -func (t *tagKeyValue) Load(value string) seriesIDs { - if t == nil { - return nil - } - - t.mu.RLock() - entry := t.entries[value] - ids, changed := entry.ids() - t.mu.RUnlock() - - if changed { - t.mu.Lock() - entry.setIDs(ids) - t.mu.Unlock() - } - return ids -} - -// TagKeyValue is a no-op. -// -// If f returns false then iteration over any remaining keys or values will cease. -func (t *tagKeyValue) Range(f func(tagValue string, a seriesIDs) bool) { - if t == nil { - return - } - - t.mu.RLock() - for tagValue, entry := range t.entries { - ids, changed := entry.ids() - if changed { - t.mu.RUnlock() - t.mu.Lock() - entry.setIDs(ids) - t.mu.Unlock() - t.mu.RLock() - } - - if !f(tagValue, ids) { - t.mu.RUnlock() - return - } - } - t.mu.RUnlock() -} - -// RangeAll calls f sequentially on each key and value. A call to RangeAll on a -// nil TagKeyValue is a no-op. -func (t *tagKeyValue) RangeAll(f func(k string, a seriesIDs)) { - t.Range(func(k string, a seriesIDs) bool { - f(k, a) - return true - }) -} - -type tagKeyValueEntry struct { - m map[uint64]struct{} // series id set - a seriesIDs // lazily sorted list of series. -} - -func newTagKeyValueEntry() *tagKeyValueEntry { - return &tagKeyValueEntry{m: make(map[uint64]struct{})} -} - -func (e *tagKeyValueEntry) ids() (_ seriesIDs, changed bool) { - if e == nil { - return nil, false - } else if len(e.a) == len(e.m) { - return e.a, false - } - - a := make(seriesIDs, 0, len(e.m)) - for id := range e.m { - a = append(a, id) - } - radix.SortUint64s(a) - - return a, true -} - -func (e *tagKeyValueEntry) setIDs(a seriesIDs) { - if e == nil { - return - } - e.a = a -} - -// SeriesIDs is a convenience type for sorting, checking equality, and doing -// union and intersection of collections of series ids. -type seriesIDs []uint64 - -// Len implements sort.Interface. -func (a seriesIDs) Len() int { return len(a) } - -// Less implements sort.Interface. -func (a seriesIDs) Less(i, j int) bool { return a[i] < a[j] } - -// Swap implements sort.Interface. -func (a seriesIDs) Swap(i, j int) { a[i], a[j] = a[j], a[i] } - -// Equals assumes that both are sorted. -func (a seriesIDs) Equals(other seriesIDs) bool { - if len(a) != len(other) { - return false - } - for i, s := range other { - if a[i] != s { - return false - } - } - return true -} - -// Intersect returns a new collection of series ids in sorted order that is the intersection of the two. -// The two collections must already be sorted. -func (a seriesIDs) Intersect(other seriesIDs) seriesIDs { - l := a - r := other - - // we want to iterate through the shortest one and stop - if len(other) < len(a) { - l = other - r = a - } - - // they're in sorted order so advance the counter as needed. - // That is, don't run comparisons against lower values that we've already passed - var i, j int - - ids := make([]uint64, 0, len(l)) - for i < len(l) && j < len(r) { - if l[i] == r[j] { - ids = append(ids, l[i]) - i++ - j++ - } else if l[i] < r[j] { - i++ - } else { - j++ - } - } - - return seriesIDs(ids) -} - -// Union returns a new collection of series ids in sorted order that is the union of the two. -// The two collections must already be sorted. -func (a seriesIDs) Union(other seriesIDs) seriesIDs { - l := a - r := other - ids := make([]uint64, 0, len(l)+len(r)) - var i, j int - for i < len(l) && j < len(r) { - if l[i] == r[j] { - ids = append(ids, l[i]) - i++ - j++ - } else if l[i] < r[j] { - ids = append(ids, l[i]) - i++ - } else { - ids = append(ids, r[j]) - j++ - } - } - - // now append the remainder - if i < len(l) { - ids = append(ids, l[i:]...) - } else if j < len(r) { - ids = append(ids, r[j:]...) - } - - return ids -} - -// Reject returns a new collection of series ids in sorted order with the passed in set removed from the original. -// This is useful for the NOT operator. The two collections must already be sorted. -func (a seriesIDs) Reject(other seriesIDs) seriesIDs { - l := a - r := other - var i, j int - - ids := make([]uint64, 0, len(l)) - for i < len(l) && j < len(r) { - if l[i] == r[j] { - i++ - j++ - } else if l[i] < r[j] { - ids = append(ids, l[i]) - i++ - } else { - j++ - } - } - - // Append the remainder - if i < len(l) { - ids = append(ids, l[i:]...) - } - - return seriesIDs(ids) -} - -// seriesID is a series id that may or may not have been evicted from the -// current id list. -type seriesID struct { - val uint64 - evict bool -} - -// evictSeriesIDs is a slice of SeriesIDs with an extra field to mark if the -// field should be evicted or not. -type evictSeriesIDs struct { - ids []seriesID - sz int -} - -// newEvictSeriesIDs copies the ids into a new slice that can be used for -// evicting series from the slice. -func newEvictSeriesIDs(ids []uint64) evictSeriesIDs { - a := make([]seriesID, len(ids)) - for i, id := range ids { - a[i].val = id - } - return evictSeriesIDs{ - ids: a, - sz: len(a), - } -} - -// mark marks all of the ids in the sorted slice to be evicted from the list of -// series ids. If an id to be evicted does not exist, it just gets ignored. -func (a *evictSeriesIDs) mark(ids []uint64) { - sIDs := a.ids - for _, id := range ids { - if len(sIDs) == 0 { - break - } - - // Perform a binary search of the remaining slice if - // the first element does not match the value we're - // looking for. - i := 0 - if sIDs[0].val < id { - i = sort.Search(len(sIDs), func(i int) bool { - return sIDs[i].val >= id - }) - } - - if i >= len(sIDs) { - break - } else if sIDs[i].val == id { - if !sIDs[i].evict { - sIDs[i].evict = true - a.sz-- - } - // Skip over this series since it has been evicted and won't be - // encountered again. - i++ - } - sIDs = sIDs[i:] - } -} - -// evict creates a new slice with only the series that have not been evicted. -func (a *evictSeriesIDs) evict() (ids seriesIDs) { - if a.sz == 0 { - return ids - } - - // Make a new slice with only the remaining ids. - ids = make([]uint64, 0, a.sz) - for _, id := range a.ids { - if id.evict { - continue - } - ids = append(ids, id.val) - } - return ids -} - -// TagFilter represents a tag filter when looking up other tags or measurements. -type TagFilter struct { - Op influxql.Token - Key string - Value string - Regex *regexp.Regexp -} - -// TagKeys returns a list of the measurement's tag names, in sorted order. -func (m *measurement) TagKeys() []string { - m.mu.RLock() - keys := make([]string, 0, len(m.seriesByTagKeyValue)) - for k := range m.seriesByTagKeyValue { - keys = append(keys, k) - } - m.mu.RUnlock() - sort.Strings(keys) - return keys -} - -// TagValues returns all the values for the given tag key, in an arbitrary order. -func (m *measurement) TagValues(auth query.Authorizer, key string) []string { - m.mu.RLock() - defer m.mu.RUnlock() - values := make([]string, 0, m.seriesByTagKeyValue[key].Cardinality()) - - m.seriesByTagKeyValue[key].RangeAll(func(k string, a seriesIDs) { - if query.AuthorizerIsOpen(auth) { - values = append(values, k) - } else { - for _, sid := range a { - s := m.seriesByID[sid] - if s == nil { - continue - } - if auth.AuthorizeSeriesRead(m.Database, m.NameBytes, s.Tags) { - values = append(values, k) - return - } - } - } - }) - return values -} - -// SetFieldName adds the field name to the measurement. -func (m *measurement) SetFieldName(name string) { - m.mu.RLock() - _, ok := m.fieldNames[name] - m.mu.RUnlock() - - if ok { - return - } - - m.mu.Lock() - m.fieldNames[name] = struct{}{} - m.mu.Unlock() -} - -// SeriesByTagKeyValue returns the TagKeyValue for the provided tag key. -func (m *measurement) SeriesByTagKeyValue(key string) *tagKeyValue { - m.mu.RLock() - defer m.mu.RUnlock() - return m.seriesByTagKeyValue[key] -} - -// stringSet represents a set of strings. -type stringSet map[string]struct{} - -// newStringSet returns an empty stringSet. -func newStringSet() stringSet { - return make(map[string]struct{}) -} - -// add adds strings to the set. -func (s stringSet) add(ss ...string) { - for _, n := range ss { - s[n] = struct{}{} - } -} - -// list returns the current elements in the set, in sorted order. -func (s stringSet) list() []string { - l := make([]string, 0, len(s)) - for k := range s { - l = append(l, k) - } - sort.Strings(l) - return l -} - -// union returns the union of this set and another. -func (s stringSet) union(o stringSet) stringSet { - ns := newStringSet() - for k := range s { - ns[k] = struct{}{} - } - for k := range o { - ns[k] = struct{}{} - } - return ns -} - -// intersect returns the intersection of this set and another. -func (s stringSet) intersect(o stringSet) stringSet { - shorter, longer := s, o - if len(longer) < len(shorter) { - shorter, longer = longer, shorter - } - - ns := newStringSet() - for k := range shorter { - if _, ok := longer[k]; ok { - ns[k] = struct{}{} - } - } - return ns -} - -type byTagKey []*query.TagSet - -func (t byTagKey) Len() int { return len(t) } -func (t byTagKey) Less(i, j int) bool { return bytes.Compare(t[i].Key, t[j].Key) < 0 } -func (t byTagKey) Swap(i, j int) { t[i], t[j] = t[j], t[i] } diff --git a/tsdb/index/inmem/meta_test.go b/tsdb/index/inmem/meta_test.go deleted file mode 100644 index be8f3d1ca3..0000000000 --- a/tsdb/index/inmem/meta_test.go +++ /dev/null @@ -1,292 +0,0 @@ -package inmem - -import ( - "fmt" - "math/rand" - "strings" - "sync" - "testing" - "time" - - "github.com/influxdata/influxdb/v2/influxql/query" - "github.com/influxdata/influxdb/v2/models" - "github.com/influxdata/influxdb/v2/tsdb" - "github.com/influxdata/influxql" -) - -// Test comparing SeriesIDs for equality. -func TestSeriesIDs_Equals(t *testing.T) { - ids1 := seriesIDs([]uint64{1, 2, 3}) - ids2 := seriesIDs([]uint64{1, 2, 3}) - ids3 := seriesIDs([]uint64{4, 5, 6}) - - if !ids1.Equals(ids2) { - t.Fatal("expected ids1 == ids2") - } else if ids1.Equals(ids3) { - t.Fatal("expected ids1 != ids3") - } -} - -// Test intersecting sets of SeriesIDs. -func TestSeriesIDs_Intersect(t *testing.T) { - // Test swapping l & r, all branches of if-else, and exit loop when 'j < len(r)' - ids1 := seriesIDs([]uint64{1, 3, 4, 5, 6}) - ids2 := seriesIDs([]uint64{1, 2, 3, 7}) - exp := seriesIDs([]uint64{1, 3}) - got := ids1.Intersect(ids2) - - if !exp.Equals(got) { - t.Fatalf("exp=%v, got=%v", exp, got) - } - - // Test exit for loop when 'i < len(l)' - ids1 = seriesIDs([]uint64{1}) - ids2 = seriesIDs([]uint64{1, 2}) - exp = seriesIDs([]uint64{1}) - got = ids1.Intersect(ids2) - - if !exp.Equals(got) { - t.Fatalf("exp=%v, got=%v", exp, got) - } -} - -// Test union sets of SeriesIDs. -func TestSeriesIDs_Union(t *testing.T) { - // Test all branches of if-else, exit loop because of 'j < len(r)', and append remainder from left. - ids1 := seriesIDs([]uint64{1, 2, 3, 7}) - ids2 := seriesIDs([]uint64{1, 3, 4, 5, 6}) - exp := seriesIDs([]uint64{1, 2, 3, 4, 5, 6, 7}) - got := ids1.Union(ids2) - - if !exp.Equals(got) { - t.Fatalf("exp=%v, got=%v", exp, got) - } - - // Test exit because of 'i < len(l)' and append remainder from right. - ids1 = seriesIDs([]uint64{1}) - ids2 = seriesIDs([]uint64{1, 2}) - exp = seriesIDs([]uint64{1, 2}) - got = ids1.Union(ids2) - - if !exp.Equals(got) { - t.Fatalf("exp=%v, got=%v", exp, got) - } -} - -// Test removing one set of SeriesIDs from another. -func TestSeriesIDs_Reject(t *testing.T) { - // Test all branches of if-else, exit loop because of 'j < len(r)', and append remainder from left. - ids1 := seriesIDs([]uint64{1, 2, 3, 7}) - ids2 := seriesIDs([]uint64{1, 3, 4, 5, 6}) - exp := seriesIDs([]uint64{2, 7}) - got := ids1.Reject(ids2) - - if !exp.Equals(got) { - t.Fatalf("exp=%v, got=%v", exp, got) - } - - // Test exit because of 'i < len(l)'. - ids1 = seriesIDs([]uint64{1}) - ids2 = seriesIDs([]uint64{1, 2}) - exp = seriesIDs{} - got = ids1.Reject(ids2) - - if !exp.Equals(got) { - t.Fatalf("exp=%v, got=%v", exp, got) - } -} - -func TestMeasurement_AddSeries_Nil(t *testing.T) { - m := newMeasurement("foo", "cpu") - if m.AddSeries(nil) { - t.Fatalf("AddSeries mismatch: exp false, got true") - } -} - -func TestMeasurement_AppendSeriesKeysByID_Missing(t *testing.T) { - m := newMeasurement("foo", "cpu") - var dst []string - dst = m.AppendSeriesKeysByID(dst, []uint64{1}) - if exp, got := 0, len(dst); exp != got { - t.Fatalf("series len mismatch: exp %v, got %v", exp, got) - } -} - -func TestMeasurement_AppendSeriesKeysByID_Exists(t *testing.T) { - m := newMeasurement("foo", "cpu") - s := newSeries(1, m, "cpu,host=foo", models.Tags{models.NewTag([]byte("host"), []byte("foo"))}) - m.AddSeries(s) - - var dst []string - dst = m.AppendSeriesKeysByID(dst, []uint64{1}) - if exp, got := 1, len(dst); exp != got { - t.Fatalf("series len mismatch: exp %v, got %v", exp, got) - } - - if exp, got := "cpu,host=foo", dst[0]; exp != got { - t.Fatalf("series mismatch: exp %v, got %v", exp, got) - } -} - -func TestMeasurement_TagsSet_Deadlock(t *testing.T) { - m := newMeasurement("foo", "cpu") - s1 := newSeries(1, m, "cpu,host=foo", models.Tags{models.NewTag([]byte("host"), []byte("foo"))}) - m.AddSeries(s1) - - s2 := newSeries(2, m, "cpu,host=bar", models.Tags{models.NewTag([]byte("host"), []byte("bar"))}) - m.AddSeries(s2) - - m.DropSeries(s1) - - // This was deadlocking - s := tsdb.NewSeriesIDSet() - s.Add(1) - m.TagSets(s, query.IteratorOptions{}) - if got, exp := len(m.SeriesIDs()), 1; got != exp { - t.Fatalf("series count mismatch: got %v, exp %v", got, exp) - } -} - -// Ensures the tagKeyValue API contains no deadlocks or sync issues. -func TestTagKeyValue_Concurrent(t *testing.T) { - var wg sync.WaitGroup - done := make(chan struct{}) - time.AfterFunc(2*time.Second, func() { close(done) }) - - v := newTagKeyValue() - for i := 0; i < 4; i++ { - wg.Add(1) - go func(i int) { - defer wg.Done() - - rand := rand.New(rand.NewSource(int64(i))) - for { - // Continue running until time limit. - select { - case <-done: - return - default: - } - - // Randomly choose next API. - switch rand.Intn(7) { - case 0: - v.bytes() - case 1: - v.Cardinality() - case 2: - v.Contains(fmt.Sprintf("%d", rand.Intn(52)+65)) - case 3: - v.InsertSeriesIDByte([]byte(fmt.Sprintf("%d", rand.Intn(52)+65)), rand.Uint64()%1000) - case 4: - v.Load(fmt.Sprintf("%d", rand.Intn(52)+65)) - case 5: - v.Range(func(tagValue string, a seriesIDs) bool { - return rand.Intn(10) == 0 - }) - case 6: - v.RangeAll(func(k string, a seriesIDs) {}) - } - } - }(i) - } - wg.Wait() -} - -func BenchmarkMeasurement_SeriesIDForExp_EQRegex(b *testing.B) { - m := newMeasurement("foo", "cpu") - for i := 0; i < 100000; i++ { - s := newSeries(uint64(i), m, "cpu", models.Tags{models.NewTag( - []byte("host"), - []byte(fmt.Sprintf("host%d", i)))}) - m.AddSeries(s) - } - - if exp, got := 100000, len(m.SeriesKeys()); exp != got { - b.Fatalf("series count mismatch: exp %v got %v", exp, got) - } - - stmt, err := influxql.NewParser(strings.NewReader(`SELECT * FROM cpu WHERE host =~ /host\d+/`)).ParseStatement() - if err != nil { - b.Fatalf("invalid statement: %s", err) - } - - selectStmt := stmt.(*influxql.SelectStatement) - - b.ResetTimer() - for i := 0; i < b.N; i++ { - ids := m.IDsForExpr(selectStmt.Condition.(*influxql.BinaryExpr)) - if exp, got := 100000, len(ids); exp != got { - b.Fatalf("series count mismatch: exp %v got %v", exp, got) - } - - } -} - -func BenchmarkMeasurement_SeriesIDForExp_NERegex(b *testing.B) { - m := newMeasurement("foo", "cpu") - for i := 0; i < 100000; i++ { - s := newSeries(uint64(i), m, "cpu", models.Tags{models.Tag{ - Key: []byte("host"), - Value: []byte(fmt.Sprintf("host%d", i))}}) - m.AddSeries(s) - } - - if exp, got := 100000, len(m.SeriesKeys()); exp != got { - b.Fatalf("series count mismatch: exp %v got %v", exp, got) - } - - stmt, err := influxql.NewParser(strings.NewReader(`SELECT * FROM cpu WHERE host !~ /foo\d+/`)).ParseStatement() - if err != nil { - b.Fatalf("invalid statement: %s", err) - } - - selectStmt := stmt.(*influxql.SelectStatement) - - b.ResetTimer() - for i := 0; i < b.N; i++ { - ids := m.IDsForExpr(selectStmt.Condition.(*influxql.BinaryExpr)) - if exp, got := 100000, len(ids); exp != got { - b.Fatalf("series count mismatch: exp %v got %v", exp, got) - } - - } - -} - -func benchmarkTagSets(b *testing.B, n int, opt query.IteratorOptions) { - m := newMeasurement("foo", "m") - ss := tsdb.NewSeriesIDSet() - - for i := 0; i < n; i++ { - tags := map[string]string{"tag1": "value1", "tag2": "value2"} - s := newSeries(uint64(i), m, "m,tag1=value1,tag2=value2", models.NewTags(tags)) - ss.Add(uint64(i)) - m.AddSeries(s) - } - - // warm caches - m.TagSets(ss, opt) - - b.ReportAllocs() - b.ResetTimer() - for i := 0; i < b.N; i++ { - m.TagSets(ss, opt) - } -} - -func BenchmarkMeasurement_TagSetsNoDimensions_1000(b *testing.B) { - benchmarkTagSets(b, 1000, query.IteratorOptions{}) -} - -func BenchmarkMeasurement_TagSetsDimensions_1000(b *testing.B) { - benchmarkTagSets(b, 1000, query.IteratorOptions{Dimensions: []string{"tag1", "tag2"}}) -} - -func BenchmarkMeasurement_TagSetsNoDimensions_100000(b *testing.B) { - benchmarkTagSets(b, 100000, query.IteratorOptions{}) -} - -func BenchmarkMeasurement_TagSetsDimensions_100000(b *testing.B) { - benchmarkTagSets(b, 100000, query.IteratorOptions{Dimensions: []string{"tag1", "tag2"}}) -} diff --git a/tsdb/index/tsi1/index.go b/tsdb/index/tsi1/index.go index da81cd0f89..3cdc2c5fd3 100644 --- a/tsdb/index/tsi1/index.go +++ b/tsdb/index/tsi1/index.go @@ -3,7 +3,6 @@ package tsi1 import ( "errors" "fmt" - "io/ioutil" "os" "path/filepath" "regexp" @@ -775,11 +774,6 @@ func (i *Index) CreateSeriesIfNotExists(key, name []byte, tags models.Tags) erro return nil } -// InitializeSeries is a no-op. This only applies to the in-memory index. -func (i *Index) InitializeSeries(keys, names [][]byte, tags []models.Tags) error { - return nil -} - // DropSeries drops the provided series from the index. If cascade is true // and this is the last series to the measurement, the measurement will also be dropped. func (i *Index) DropSeries(seriesID uint64, key []byte, cascade bool) error { @@ -824,9 +818,6 @@ func (i *Index) DropSeries(seriesID uint64, key []byte, cascade bool) error { return nil } -// DropSeriesGlobal is a no-op on the tsi1 index. -func (i *Index) DropSeriesGlobal(key []byte) error { return nil } - // DropMeasurementIfSeriesNotExist drops a measurement only if there are no more // series for the measurement. func (i *Index) DropMeasurementIfSeriesNotExist(name []byte) (bool, error) { @@ -1109,27 +1100,3 @@ func (i *Index) RetainFileSet() (*FileSet, error) { } return fs, nil } - -// SetFieldName is a no-op on this index. -func (i *Index) SetFieldName(measurement []byte, name string) {} - -// Rebuild rebuilds an index. It's a no-op for this index. -func (i *Index) Rebuild() {} - -// IsIndexDir returns true if directory contains at least one partition directory. -func IsIndexDir(path string) (bool, error) { - fis, err := ioutil.ReadDir(path) - if err != nil { - return false, err - } - for _, fi := range fis { - if !fi.IsDir() { - continue - } else if ok, err := IsPartitionDir(filepath.Join(path, fi.Name())); err != nil { - return false, err - } else if ok { - return true, nil - } - } - return false, nil -} diff --git a/tsdb/index_test.go b/tsdb/index_test.go index 10071b2bca..6f50e64bb3 100644 --- a/tsdb/index_test.go +++ b/tsdb/index_test.go @@ -16,7 +16,6 @@ import ( "github.com/influxdata/influxdb/v2/models" "github.com/influxdata/influxdb/v2/pkg/slices" "github.com/influxdata/influxdb/v2/tsdb" - "github.com/influxdata/influxdb/v2/tsdb/index/inmem" "github.com/influxdata/influxdb/v2/tsdb/index/tsi1" "github.com/influxdata/influxql" ) @@ -227,59 +226,6 @@ func TestIndexSet_MeasurementNamesByPredicate(t *testing.T) { } } -func TestIndexSet_DedupeInmemIndexes(t *testing.T) { - testCases := []struct { - tsiN int // Quantity of TSI indexes - inmem1N int // Quantity of ShardIndexes proxying the first inmem Index - inmem2N int // Quantity of ShardIndexes proxying the second inmem Index - uniqueN int // Quantity of total, deduplicated indexes - }{ - {tsiN: 1, inmem1N: 0, uniqueN: 1}, - {tsiN: 2, inmem1N: 0, uniqueN: 2}, - {tsiN: 0, inmem1N: 1, uniqueN: 1}, - {tsiN: 0, inmem1N: 2, uniqueN: 1}, - {tsiN: 0, inmem1N: 1, inmem2N: 1, uniqueN: 2}, - {tsiN: 0, inmem1N: 2, inmem2N: 2, uniqueN: 2}, - {tsiN: 2, inmem1N: 2, inmem2N: 2, uniqueN: 4}, - } - - for _, testCase := range testCases { - name := fmt.Sprintf("%d/%d/%d -> %d", testCase.tsiN, testCase.inmem1N, testCase.inmem2N, testCase.uniqueN) - t.Run(name, func(t *testing.T) { - - var indexes []tsdb.Index - for i := 0; i < testCase.tsiN; i++ { - indexes = append(indexes, MustOpenNewIndex(tsi1.IndexName)) - } - if testCase.inmem1N > 0 { - sfile := MustOpenSeriesFile() - opts := tsdb.NewEngineOptions() - opts.IndexVersion = inmem.IndexName - opts.InmemIndex = inmem.NewIndex("db", sfile.SeriesFile) - - for i := 0; i < testCase.inmem1N; i++ { - indexes = append(indexes, inmem.NewShardIndex(uint64(i), tsdb.NewSeriesIDSet(), opts)) - } - } - if testCase.inmem2N > 0 { - sfile := MustOpenSeriesFile() - opts := tsdb.NewEngineOptions() - opts.IndexVersion = inmem.IndexName - opts.InmemIndex = inmem.NewIndex("db", sfile.SeriesFile) - - for i := 0; i < testCase.inmem2N; i++ { - indexes = append(indexes, inmem.NewShardIndex(uint64(i), tsdb.NewSeriesIDSet(), opts)) - } - } - - is := tsdb.IndexSet{Indexes: indexes}.DedupeInmemIndexes() - if len(is.Indexes) != testCase.uniqueN { - t.Errorf("expected %d indexes, got %d", testCase.uniqueN, len(is.Indexes)) - } - }) - } -} - func TestIndex_Sketches(t *testing.T) { checkCardinalities := func(t *testing.T, index *Index, state string, series, tseries, measurements, tmeasurements int) { t.Helper() @@ -436,10 +382,6 @@ func MustNewIndex(index string, eopts ...EngineOption) *Index { panic(err) } - if index == inmem.IndexName { - opts.InmemIndex = inmem.NewIndex("db0", sfile) - } - i, err := tsdb.NewIndex(0, "db0", filepath.Join(rootPath, "index"), tsdb.NewSeriesIDSet(), sfile, opts) if err != nil { panic(err) @@ -500,9 +442,6 @@ func (i *Index) Reopen() error { opts := tsdb.NewEngineOptions() opts.IndexVersion = i.indexType - if i.indexType == inmem.IndexName { - opts.InmemIndex = inmem.NewIndex("db0", i.sfile) - } idx, err := tsdb.NewIndex(0, "db0", filepath.Join(i.rootPath, "index"), tsdb.NewSeriesIDSet(), i.sfile, opts) if err != nil { @@ -532,7 +471,6 @@ func (i *Index) Close() error { // // Typical results on an i7 laptop. // -// BenchmarkIndexSet_TagSets/1M_series/inmem-8 100 10430732 ns/op 3556728 B/op 51 allocs/op // BenchmarkIndexSet_TagSets/1M_series/tsi1-8 100 18995530 ns/op 5221180 B/op 20379 allocs/op func BenchmarkIndexSet_TagSets(b *testing.B) { // Read line-protocol and coerce into tsdb format. @@ -588,11 +526,6 @@ func BenchmarkIndexSet_TagSets(b *testing.B) { } } - // TODO(edd): refactor how we call into tag sets in the tsdb package. - type indexTagSets interface { - TagSets(name []byte, options query.IteratorOptions) ([]*query.TagSet, error) - } - var errResult error // This benchmark will merge eight bitsets each containing ~10,000 series IDs. @@ -609,18 +542,8 @@ func BenchmarkIndexSet_TagSets(b *testing.B) { Indexes: []tsdb.Index{idx.Index}, } // For TSI implementation - var ts func() ([]*query.TagSet, error) - // TODO(edd): this is somewhat awkward. We should unify this difference somewhere higher - // up than the engine. I don't want to open an engine do a benchmark on - // different index implementations. - if indexType == tsdb.InmemIndexName { - ts = func() ([]*query.TagSet, error) { - return idx.Index.(indexTagSets).TagSets(name, opt) - } - } else { - ts = func() ([]*query.TagSet, error) { - return indexSet.TagSets(idx.sfile, name, opt) - } + ts := func() ([]*query.TagSet, error) { + return indexSet.TagSets(idx.sfile, name, opt) } b.Run(indexType, func(b *testing.B) { @@ -645,8 +568,6 @@ func BenchmarkIndexSet_TagSets(b *testing.B) { // // Typical results for an i7 laptop // -// BenchmarkIndex_ConcurrentWriteQuery/inmem/queries_100000/cache-8 1 5963346204 ns/op 2499655768 B/op 23964183 allocs/op -// BenchmarkIndex_ConcurrentWriteQuery/inmem/queries_100000/no_cache-8 1 5314841090 ns/op 2499495280 B/op 23963322 allocs/op // BenchmarkIndex_ConcurrentWriteQuery/tsi1/queries_100000/cache-8 1 1645048376 ns/op 2215402840 B/op 23048978 allocs/op // BenchmarkIndex_ConcurrentWriteQuery/tsi1/queries_100000/no_cache-8 1 22242155616 ns/op 28277544136 B/op 79620463 allocs/op func BenchmarkIndex_ConcurrentWriteQuery(b *testing.B) { diff --git a/tsdb/shard.go b/tsdb/shard.go index ef6d74cf9c..984ca9cedc 100644 --- a/tsdb/shard.go +++ b/tsdb/shard.go @@ -315,9 +315,15 @@ func (s *Shard) Open() error { if err != nil { return err } - idx.WithLogger(s.baseLogger) + // Check if the index needs to be rebuilt before Open() initializes + // its file system layout. + var shouldReindex bool + if _, err := os.Stat(ipath); os.IsNotExist(err) { + shouldReindex = true + } + // Open index. if err := idx.Open(); err != nil { return err @@ -340,8 +346,12 @@ func (s *Shard) Open() error { if err := e.Open(); err != nil { return err } + if shouldReindex { + if err := e.Reindex(); err != nil { + return err + } + } - // Load metadata index for the inmem index only. if err := e.LoadMetadataIndex(s.id, s.index); err != nil { return err } @@ -704,8 +714,6 @@ func (s *Shard) createFieldsAndMeasurements(fieldsToCreate []*FieldCreate) error if err := mf.CreateFieldIfNotExists([]byte(f.Field.Name), f.Field.Type); err != nil { return err } - - s.index.SetFieldName(f.Measurement, f.Field.Name) } if len(fieldsToCreate) > 0 { diff --git a/tsdb/shard_internal_test.go b/tsdb/shard_internal_test.go index f8b4abd959..a1dac59fcd 100644 --- a/tsdb/shard_internal_test.go +++ b/tsdb/shard_internal_test.go @@ -4,7 +4,6 @@ import ( "fmt" "io/ioutil" "os" - "path" "path/filepath" "regexp" "sort" @@ -231,9 +230,6 @@ func NewTempShard(index string) *TempShard { opt := NewEngineOptions() opt.IndexVersion = index opt.Config.WALDir = filepath.Join(dir, "wal") - if index == InmemIndexName { - opt.InmemIndex, _ = NewInmemIndex(path.Base(dir), sfile) - } return &TempShard{ Shard: NewShard(0, diff --git a/tsdb/shard_test.go b/tsdb/shard_test.go index adc17e9b10..bc68b63b7f 100644 --- a/tsdb/shard_test.go +++ b/tsdb/shard_test.go @@ -28,7 +28,6 @@ import ( "github.com/influxdata/influxdb/v2/tsdb" _ "github.com/influxdata/influxdb/v2/tsdb/engine" _ "github.com/influxdata/influxdb/v2/tsdb/index" - "github.com/influxdata/influxdb/v2/tsdb/index/inmem" "github.com/influxdata/influxql" ) @@ -43,7 +42,6 @@ func TestShardWriteAndIndex(t *testing.T) { opts := tsdb.NewEngineOptions() opts.Config.WALDir = filepath.Join(tmpDir, "wal") - opts.InmemIndex = inmem.NewIndex(filepath.Base(tmpDir), sfile.SeriesFile) sh := tsdb.NewShard(1, tmpShard, tmpWal, sfile.SeriesFile, opts) @@ -102,6 +100,84 @@ func TestShardWriteAndIndex(t *testing.T) { } } +func TestShardRebuildIndex(t *testing.T) { + tmpDir, _ := ioutil.TempDir("", "shard_test") + defer os.RemoveAll(tmpDir) + tmpShard := filepath.Join(tmpDir, "shard") + tmpWal := filepath.Join(tmpDir, "wal") + + sfile := MustOpenSeriesFile() + defer sfile.Close() + + opts := tsdb.NewEngineOptions() + opts.Config.WALDir = filepath.Join(tmpDir, "wal") + + sh := tsdb.NewShard(1, tmpShard, tmpWal, sfile.SeriesFile, opts) + if err := sh.Open(); err != nil { + t.Fatalf("error opening shard: %s", err.Error()) + } + + pt := models.MustNewPoint( + "cpu", + models.Tags{{Key: []byte("host"), Value: []byte("server")}}, + map[string]interface{}{"value": 1.0}, + time.Unix(1, 2), + ) + + err := sh.WritePoints([]models.Point{pt}) + if err != nil { + t.Fatalf(err.Error()) + } + + pt.SetTime(time.Unix(2, 3)) + err = sh.WritePoints([]models.Point{pt}) + if err != nil { + t.Fatalf(err.Error()) + } + + indexPath := filepath.Join(tmpShard, "index") + validateIndex := func() { + cnt := sh.SeriesN() + if got, exp := cnt, int64(1); got != exp { + t.Fatalf("got %v series, exp %v series in index", got, exp) + } + fi, err := os.Stat(indexPath) + + // Make sure index data is being persisted to disk. + if os.IsNotExist(err) { + t.Fatalf("index path %q does not exist", indexPath) + } + if !fi.IsDir() { + t.Fatalf("index path %q is not a directory", indexPath) + } + } + + validateIndex() + + // ensure the index gets rebuilt after its directory is deleted and + // the shard is reopened. + if err := sh.Close(); err != nil { + t.Fatalf(err.Error()) + } + if err := os.RemoveAll(indexPath); err != nil { + t.Fatalf(err.Error()) + } + + sh = tsdb.NewShard(1, tmpShard, tmpWal, sfile.SeriesFile, opts) + if err := sh.Open(); err != nil { + t.Fatalf("error opening shard: %s", err.Error()) + } + + validateIndex() + + // and ensure that we can still write data + pt.SetTime(time.Unix(2, 6)) + err = sh.WritePoints([]models.Point{pt}) + if err != nil { + t.Fatalf(err.Error()) + } +} + func TestShard_Open_CorruptFieldsIndex(t *testing.T) { tmpDir, _ := ioutil.TempDir("", "shard_test") defer os.RemoveAll(tmpDir) @@ -113,7 +189,6 @@ func TestShard_Open_CorruptFieldsIndex(t *testing.T) { opts := tsdb.NewEngineOptions() opts.Config.WALDir = filepath.Join(tmpDir, "wal") - opts.InmemIndex = inmem.NewIndex(filepath.Base(tmpDir), sfile.SeriesFile) sh := tsdb.NewShard(1, tmpShard, tmpWal, sfile.SeriesFile, opts) @@ -153,120 +228,6 @@ func TestShard_Open_CorruptFieldsIndex(t *testing.T) { } } -func TestMaxSeriesLimit(t *testing.T) { - tmpDir, _ := ioutil.TempDir("", "shard_test") - defer os.RemoveAll(tmpDir) - tmpShard := filepath.Join(tmpDir, "db", "rp", "1") - tmpWal := filepath.Join(tmpDir, "wal") - - sfile := MustOpenSeriesFile() - defer sfile.Close() - - opts := tsdb.NewEngineOptions() - opts.Config.WALDir = filepath.Join(tmpDir, "wal") - opts.Config.MaxSeriesPerDatabase = 1000 - opts.InmemIndex = inmem.NewIndex(filepath.Base(tmpDir), sfile.SeriesFile) - opts.IndexVersion = tsdb.InmemIndexName - - sh := tsdb.NewShard(1, tmpShard, tmpWal, sfile.SeriesFile, opts) - - if err := sh.Open(); err != nil { - t.Fatalf("error opening shard: %s", err.Error()) - } - - // Writing 1K series should succeed. - points := []models.Point{} - - for i := 0; i < 1000; i++ { - pt := models.MustNewPoint( - "cpu", - models.Tags{{Key: []byte("host"), Value: []byte(fmt.Sprintf("server%d", i))}}, - map[string]interface{}{"value": 1.0}, - time.Unix(1, 2), - ) - points = append(points, pt) - } - - err := sh.WritePoints(points) - if err != nil { - t.Fatalf(err.Error()) - } - - // Writing one more series should exceed the series limit. - pt := models.MustNewPoint( - "cpu", - models.Tags{{Key: []byte("host"), Value: []byte("server9999")}}, - map[string]interface{}{"value": 1.0}, - time.Unix(1, 2), - ) - - err = sh.WritePoints([]models.Point{pt}) - if err == nil { - t.Fatal("expected error") - } else if exp, got := `partial write: max-series-per-database limit exceeded: (1000) dropped=1`, err.Error(); exp != got { - t.Fatalf("unexpected error message:\n\texp = %s\n\tgot = %s", exp, got) - } - - sh.Close() -} - -func TestShard_MaxTagValuesLimit(t *testing.T) { - tmpDir, _ := ioutil.TempDir("", "shard_test") - defer os.RemoveAll(tmpDir) - tmpShard := filepath.Join(tmpDir, "db", "rp", "1") - tmpWal := filepath.Join(tmpDir, "wal") - - sfile := MustOpenSeriesFile() - defer sfile.Close() - - opts := tsdb.NewEngineOptions() - opts.Config.WALDir = filepath.Join(tmpDir, "wal") - opts.Config.MaxValuesPerTag = 1000 - opts.InmemIndex = inmem.NewIndex(filepath.Base(tmpDir), sfile.SeriesFile) - opts.IndexVersion = tsdb.InmemIndexName - - sh := tsdb.NewShard(1, tmpShard, tmpWal, sfile.SeriesFile, opts) - - if err := sh.Open(); err != nil { - t.Fatalf("error opening shard: %s", err.Error()) - } - - // Writing 1K series should succeed. - points := []models.Point{} - - for i := 0; i < 1000; i++ { - pt := models.MustNewPoint( - "cpu", - models.Tags{{Key: []byte("host"), Value: []byte(fmt.Sprintf("server%d", i))}}, - map[string]interface{}{"value": 1.0}, - time.Unix(1, 2), - ) - points = append(points, pt) - } - - err := sh.WritePoints(points) - if err != nil { - t.Fatalf(err.Error()) - } - - // Writing one more series should exceed the series limit. - pt := models.MustNewPoint( - "cpu", - models.Tags{{Key: []byte("host"), Value: []byte("server9999")}}, - map[string]interface{}{"value": 1.0}, - time.Unix(1, 2), - ) - - err = sh.WritePoints([]models.Point{pt}) - if err == nil { - t.Fatal("expected error") - } else if exp, got := `partial write: max-values-per-tag limit exceeded (1000/1000): measurement="cpu" tag="host" value="server9999" dropped=1`, err.Error(); exp != got { - t.Fatalf("unexpected error message:\n\texp = %s\n\tgot = %s", exp, got) - } - - sh.Close() -} - func TestWriteTimeTag(t *testing.T) { tmpDir, _ := ioutil.TempDir("", "shard_test") defer os.RemoveAll(tmpDir) @@ -278,7 +239,6 @@ func TestWriteTimeTag(t *testing.T) { opts := tsdb.NewEngineOptions() opts.Config.WALDir = filepath.Join(tmpDir, "wal") - opts.InmemIndex = inmem.NewIndex(filepath.Base(tmpDir), sfile.SeriesFile) sh := tsdb.NewShard(1, tmpShard, tmpWal, sfile.SeriesFile, opts) if err := sh.Open(); err != nil { @@ -329,7 +289,6 @@ func TestWriteTimeField(t *testing.T) { opts := tsdb.NewEngineOptions() opts.Config.WALDir = filepath.Join(tmpDir, "wal") - opts.InmemIndex = inmem.NewIndex(filepath.Base(tmpDir), sfile.SeriesFile) sh := tsdb.NewShard(1, tmpShard, tmpWal, sfile.SeriesFile, opts) if err := sh.Open(); err != nil { @@ -365,7 +324,6 @@ func TestShardWriteAddNewField(t *testing.T) { opts := tsdb.NewEngineOptions() opts.Config.WALDir = filepath.Join(tmpDir, "wal") - opts.InmemIndex = inmem.NewIndex(filepath.Base(tmpDir), sfile.SeriesFile) sh := tsdb.NewShard(1, tmpShard, tmpWal, sfile.SeriesFile, opts) if err := sh.Open(); err != nil { @@ -418,7 +376,6 @@ func TestShard_WritePoints_FieldConflictConcurrent(t *testing.T) { opts := tsdb.NewEngineOptions() opts.Config.WALDir = filepath.Join(tmpDir, "wal") - opts.InmemIndex = inmem.NewIndex(filepath.Base(tmpDir), sfile.SeriesFile) opts.SeriesIDSets = seriesIDSets([]*tsdb.SeriesIDSet{}) sh := tsdb.NewShard(1, tmpShard, tmpWal, sfile.SeriesFile, opts) @@ -507,7 +464,6 @@ func TestShard_WritePoints_FieldConflictConcurrentQuery(t *testing.T) { opts := tsdb.NewEngineOptions() opts.Config.WALDir = filepath.Join(tmpDir, "wal") - opts.InmemIndex = inmem.NewIndex(filepath.Base(tmpDir), sfile.SeriesFile) opts.SeriesIDSets = seriesIDSets([]*tsdb.SeriesIDSet{}) sh := tsdb.NewShard(1, tmpShard, tmpWal, sfile.SeriesFile, opts) @@ -658,7 +614,6 @@ func TestShard_Close_RemoveIndex(t *testing.T) { opts := tsdb.NewEngineOptions() opts.Config.WALDir = filepath.Join(tmpDir, "wal") - opts.InmemIndex = inmem.NewIndex(filepath.Base(tmpDir), sfile.SeriesFile) sh := tsdb.NewShard(1, tmpShard, tmpWal, sfile.SeriesFile, opts) if err := sh.Open(); err != nil { @@ -2054,7 +2009,6 @@ func openShard(sfile *SeriesFile) (*tsdb.Shard, string, error) { tmpWal := filepath.Join(tmpDir, "wal") opts := tsdb.NewEngineOptions() opts.Config.WALDir = tmpWal - opts.InmemIndex = inmem.NewIndex(filepath.Base(tmpDir), sfile.SeriesFile) shard := tsdb.NewShard(1, tmpShard, tmpWal, sfile.SeriesFile, opts) err := shard.Open() return shard, tmpDir, err @@ -2202,9 +2156,6 @@ func NewShards(index string, n int) Shards { opt := tsdb.NewEngineOptions() opt.IndexVersion = index opt.Config.WALDir = filepath.Join(dir, "wal") - if index == tsdb.InmemIndexName { - opt.InmemIndex = inmem.NewIndex(filepath.Base(dir), sfile.SeriesFile) - } // Initialise series id sets. Need to do this as it's normally done at the // store level. diff --git a/tsdb/store.go b/tsdb/store.go index b8c1ca02d7..296176b416 100644 --- a/tsdb/store.go +++ b/tsdb/store.go @@ -38,7 +38,7 @@ var ( ErrShardDeletion = errors.New("shard is being deleted") // ErrMultipleIndexTypes is returned when trying to do deletes on a database with // multiple index types. - ErrMultipleIndexTypes = errors.New("cannot delete data. DB contains shards using both inmem and tsi1 indexes. Please convert all shards to use the same index type to delete data.") + ErrMultipleIndexTypes = errors.New("cannot delete data. DB contains shards using multiple indexes. Please convert all shards to use the same index type to delete data") ) // Statistics gathered by the store. @@ -84,9 +84,6 @@ type Store struct { SeriesFileMaxSize int64 // Determines size of series file mmap. Can be altered in tests. path string - // shared per-database indexes, only if using "inmem". - indexes map[string]interface{} - // Maintains a set of shards that are in the process of deletion. // This prevents new shards from being created while old ones are being deleted. pendingShardDeletes map[uint64]struct{} @@ -108,17 +105,15 @@ type Store struct { // NewStore returns a new store with the given path and a default configuration. // The returned store must be initialized by calling Open before using it. func NewStore(path string) *Store { - logger := zap.NewNop() return &Store{ databases: make(map[string]*databaseState), path: path, sfiles: make(map[string]*SeriesFile), - indexes: make(map[string]interface{}), pendingShardDeletes: make(map[uint64]struct{}), epochs: make(map[uint64]*epochTracker), EngineOptions: NewEngineOptions(), - Logger: logger, - baseLogger: logger, + Logger: zap.NewNop(), + baseLogger: zap.NewNop(), } } @@ -187,7 +182,6 @@ func (s *Store) IndexBytes() int { is.Indexes = append(is.Indexes, shard.index) } s.mu.RUnlock() - is = is.DedupeInmemIndexes() var b int for _, idx := range is.Indexes { @@ -320,12 +314,6 @@ func (s *Store) loadShards() error { return err } - // Retrieve database index. - idx, err := s.createIndexIfNotExists(db.Name()) - if err != nil { - return err - } - // Load each retention policy within the database directory. rpDirs, err := ioutil.ReadDir(dbPath) if err != nil { @@ -386,16 +374,10 @@ func (s *Store) loadShards() error { // Copy options and assign shared index. opt := s.EngineOptions - opt.InmemIndex = idx // Provide an implementation of the ShardIDSets opt.SeriesIDSets = shardSet{store: s, db: db} - // Existing shards should continue to use inmem index. - if _, err := os.Stat(filepath.Join(path, "index")); os.IsNotExist(err) { - opt.IndexVersion = InmemIndexName - } - // Open engine. shard := NewShard(shardID, path, walPath, sfile, opt) @@ -406,8 +388,8 @@ func (s *Store) loadShards() error { err = shard.Open() if err != nil { - log.Info("Failed to open shard", logger.Shard(shardID), zap.Error(err)) - resC <- &res{err: fmt.Errorf("Failed to open shard: %d: %s", shardID, err)} + log.Error("Failed to open shard", logger.Shard(shardID), zap.Error(err)) + resC <- &res{err: fmt.Errorf("failed to open shard: %d: %s", shardID, err)} return } @@ -488,7 +470,6 @@ func (s *Store) Close() error { s.databases = make(map[string]*databaseState) s.sfiles = map[string]*SeriesFile{} - s.indexes = make(map[string]interface{}) s.pendingShardDeletes = make(map[uint64]struct{}) s.shards = nil s.opened = false // Store may now be opened again. @@ -529,28 +510,6 @@ func (s *Store) seriesFile(database string) *SeriesFile { return s.sfiles[database] } -// createIndexIfNotExists returns a shared index for a database, if the inmem -// index is being used. If the TSI index is being used, then this method is -// basically a no-op. -func (s *Store) createIndexIfNotExists(name string) (interface{}, error) { - if idx := s.indexes[name]; idx != nil { - return idx, nil - } - - sfile, err := s.openSeriesFile(name) - if err != nil { - return nil, err - } - - idx, err := NewInmemIndex(name, sfile) - if err != nil { - return nil, err - } - - s.indexes[name] = idx - return idx, nil -} - // Shard returns a shard by id. func (s *Store) Shard(id uint64) *Shard { s.mu.RLock() @@ -638,15 +597,8 @@ func (s *Store) CreateShard(database, retentionPolicy string, shardID uint64, en return err } - // Retrieve shared index, if needed. - idx, err := s.createIndexIfNotExists(database) - if err != nil { - return err - } - // Copy index options and pass in shared index. opt := s.EngineOptions - opt.InmemIndex = idx opt.SeriesIDSets = shardSet{store: s, db: database} path := filepath.Join(s.path, database, retentionPolicy, strconv.FormatUint(shardID, 10)) @@ -767,32 +719,6 @@ func (s *Store) DeleteShard(shardID uint64) error { if ss.Cardinality() > 0 { sfile := s.seriesFile(db) if sfile != nil { - // If the inmem index is in use, then the series being removed from the - // series file will also need to be removed from the index. - if index.Type() == InmemIndexName { - var keyBuf []byte // Series key buffer. - var name []byte - var tagsBuf models.Tags // Buffer for tags container. - var err error - - ss.ForEach(func(id uint64) { - skey := sfile.SeriesKey(id) // Series File series key - if skey == nil { - return - } - - name, tagsBuf = ParseSeriesKeyInto(skey, tagsBuf) - keyBuf = models.AppendMakeKey(keyBuf, name, tagsBuf) - if err = index.DropSeriesGlobal(keyBuf); err != nil { - return - } - }) - - if err != nil { - return err - } - } - ss.ForEach(func(id uint64) { sfile.DeleteSeriesID(id) }) @@ -872,9 +798,6 @@ func (s *Store) DeleteDatabase(name string) error { // Remove database from store list of databases delete(s.databases, name) - // Remove shared index for database if using inmem index. - delete(s.indexes, name) - return nil } @@ -1524,7 +1447,6 @@ func (s *Store) MeasurementNames(auth query.Authorizer, database string, cond in } is.Indexes = append(is.Indexes, index) } - is = is.DedupeInmemIndexes() return is.MeasurementNamesByExpr(auth, cond) } @@ -1610,7 +1532,6 @@ func (s *Store) TagKeys(auth query.Authorizer, shardIDs []uint64, cond influxql. s.mu.RUnlock() // Determine list of measurements. - is = is.DedupeInmemIndexes() names, err := is.MeasurementNamesByExpr(nil, measurementExpr) if err != nil { return nil, err @@ -1775,7 +1696,6 @@ func (s *Store) TagValues(auth query.Authorizer, shardIDs []uint64, cond influxq is.Indexes = append(is.Indexes, index) } s.mu.RUnlock() - is = is.DedupeInmemIndexes() // Stores each list of TagValues for each measurement. var allResults []tagValues @@ -1996,8 +1916,6 @@ func mergeTagValues(valueIdxs [][2]int, tvs ...tagValues) TagValues { func (s *Store) monitorShards() { t := time.NewTicker(10 * time.Second) defer t.Stop() - t2 := time.NewTicker(time.Minute) - defer t2.Stop() for { select { case <-s.closing: @@ -2016,88 +1934,6 @@ func (s *Store) monitorShards() { } } s.mu.RUnlock() - case <-t2.C: - if s.EngineOptions.Config.MaxValuesPerTag == 0 { - continue - } - - s.mu.RLock() - shards := s.filterShards(func(sh *Shard) bool { - return sh.IndexType() == InmemIndexName - }) - s.mu.RUnlock() - - // No inmem shards... - if len(shards) == 0 { - continue - } - - var dbLock sync.Mutex - databases := make(map[string]struct{}, len(shards)) - - s.walkShards(shards, func(sh *Shard) error { - db := sh.database - - // Only process 1 shard from each database - dbLock.Lock() - if _, ok := databases[db]; ok { - dbLock.Unlock() - return nil - } - databases[db] = struct{}{} - dbLock.Unlock() - - sfile := s.seriesFile(sh.database) - if sfile == nil { - return nil - } - - firstShardIndex, err := sh.Index() - if err != nil { - return err - } - - index, err := sh.Index() - if err != nil { - return err - } - - // inmem shards share the same index instance so just use the first one to avoid - // allocating the same measurements repeatedly - indexSet := IndexSet{Indexes: []Index{firstShardIndex}, SeriesFile: sfile} - names, err := indexSet.MeasurementNamesByExpr(nil, nil) - if err != nil { - s.Logger.Warn("Cannot retrieve measurement names", - zap.Error(err), - logger.Shard(sh.ID()), - logger.Database(db)) - return nil - } - - indexSet.Indexes = []Index{index} - for _, name := range names { - indexSet.ForEachMeasurementTagKey(name, func(k []byte) error { - n := sh.TagKeyCardinality(name, k) - perc := int(float64(n) / float64(s.EngineOptions.Config.MaxValuesPerTag) * 100) - if perc > 100 { - perc = 100 - } - - // Log at 80, 85, 90-100% levels - if perc == 80 || perc == 85 || perc >= 90 { - s.Logger.Warn("max-values-per-tag limit may be exceeded soon", - zap.String("perc", fmt.Sprintf("%d%%", perc)), - zap.Int("n", n), - zap.Int("max", s.EngineOptions.Config.MaxValuesPerTag), - logger.Database(db), - zap.ByteString("measurement", name), - zap.ByteString("tag", k)) - } - return nil - }) - } - return nil - }) } } } diff --git a/tsdb/store_test.go b/tsdb/store_test.go index b0b77bc8ff..ac32b50d01 100644 --- a/tsdb/store_test.go +++ b/tsdb/store_test.go @@ -27,7 +27,6 @@ import ( "github.com/influxdata/influxdb/v2/pkg/deep" "github.com/influxdata/influxdb/v2/pkg/slices" "github.com/influxdata/influxdb/v2/tsdb" - "github.com/influxdata/influxdb/v2/tsdb/index/inmem" "github.com/influxdata/influxql" ) @@ -198,50 +197,6 @@ func TestStore_CreateMixedShards(t *testing.T) { } } -func TestStore_DropMeasurementMixedShards(t *testing.T) { - - test := func(index1 string, index2 string) { - s := MustOpenStore(index1) - defer s.Close() - - if err := s.CreateShard("db0", "rp0", 1, true); err != nil { - t.Fatal(err) - } - - s.MustWriteToShardString(1, "mem,server=a v=1 10") - - s.EngineOptions.IndexVersion = index2 - s.index = index2 - if err := s.Reopen(); err != nil { - t.Fatal(err) - } - - if err := s.CreateShard("db0", "rp0", 2, true); err != nil { - t.Fatal(err) - } - - s.MustWriteToShardString(2, "mem,server=b v=1 20") - - s.MustWriteToShardString(1, "cpu,server=a v=1 10") - s.MustWriteToShardString(2, "cpu,server=b v=1 20") - - err := s.DeleteMeasurement("db0", "cpu") - if err != tsdb.ErrMultipleIndexTypes { - t.Fatal(err) - } else if err == nil { - t.Fatal("expect failure deleting measurement on multiple index types") - } - } - - indexes := tsdb.RegisteredIndexes() - for i := range indexes { - j := (i + 1) % len(indexes) - index1 := indexes[i] - index2 := indexes[j] - t.Run(fmt.Sprintf("%s-%s", index1, index2), func(t *testing.T) { test(index1, index2) }) - } -} - func TestStore_DropConcurrentWriteMultipleShards(t *testing.T) { test := func(index string) { @@ -786,10 +741,6 @@ func TestStore_BackupRestoreShard(t *testing.T) { } for _, index := range tsdb.RegisteredIndexes() { - if index == tsdb.TSI1IndexName { - t.Skip("Skipping failing test for tsi1") - } - t.Run(index, func(t *testing.T) { test(index) }) @@ -998,7 +949,6 @@ func TestStore_Cardinality_Unique(t *testing.T) { test := func(index string) { store := NewStore(index) - store.EngineOptions.Config.MaxSeriesPerDatabase = 0 if err := store.Open(); err != nil { panic(err) } @@ -1079,7 +1029,6 @@ func TestStore_Cardinality_Duplicates(t *testing.T) { test := func(index string) { store := NewStore(index) - store.EngineOptions.Config.MaxSeriesPerDatabase = 0 if err := store.Open(); err != nil { panic(err) } @@ -1147,7 +1096,6 @@ func TestStore_Cardinality_Compactions(t *testing.T) { test := func(index string) error { store := NewStore(index) - store.EngineOptions.Config.MaxSeriesPerDatabase = 0 if err := store.Open(); err != nil { panic(err) } @@ -1164,65 +1112,6 @@ func TestStore_Cardinality_Compactions(t *testing.T) { } } -func TestStore_Cardinality_Limit_On_InMem_Index(t *testing.T) { - - if testing.Short() || os.Getenv("GORACE") != "" || os.Getenv("APPVEYOR") != "" || os.Getenv("CIRCLECI") != "" { - t.Skip("Skipping test in short, race, circleci and appveyor mode.") - } - - store := NewStore("inmem") - store.EngineOptions.Config.MaxSeriesPerDatabase = 100000 - if err := store.Open(); err != nil { - panic(err) - } - defer store.Close() - - // Generate 200,000 series to write. - series := genTestSeries(64, 5, 5) - - // Add 1 point to each series. - points := make([]models.Point, 0, len(series)) - for _, s := range series { - points = append(points, models.MustNewPoint(s.Measurement, s.Tags, map[string]interface{}{"value": 1.0}, time.Now())) - } - - // Create shards to write points into. - numShards := 10 - for shardID := 0; shardID < numShards; shardID++ { - if err := store.CreateShard("db", "rp", uint64(shardID), true); err != nil { - t.Fatalf("create shard: %s", err) - } - } - - // Write series / points to the shards. - pointsPerShard := len(points) / numShards - - for shardID := 0; shardID < numShards; shardID++ { - from := shardID * pointsPerShard - to := from + pointsPerShard - - if err := store.Store.WriteToShard(uint64(shardID), points[from:to]); err != nil { - if !strings.Contains(err.Error(), "partial write: max-series-per-database limit exceeded:") { - t.Fatal(err) - } - } - } - - // Get updated series cardinality from store after writing data. - cardinality, err := store.Store.SeriesCardinality("db") - if err != nil { - t.Fatal(err) - } - expCardinality := store.EngineOptions.Config.MaxSeriesPerDatabase - - // Estimated cardinality should be well within 1.5% of the actual cardinality. - got := math.Abs(float64(cardinality)-float64(expCardinality)) / float64(expCardinality) - exp := 0.015 - if got > exp { - t.Errorf("got epsilon of %v for series cardinality %d (expected %d), which is larger than expected %v", got, cardinality, expCardinality, exp) - } -} - func TestStore_Sketches(t *testing.T) { checkCardinalities := func(store *tsdb.Store, series, tseries, measurements, tmeasurements int) error { @@ -1318,11 +1207,8 @@ func TestStore_Sketches(t *testing.T) { } } - // Check cardinalities. In this case, the indexes behave differently. + // Check cardinalities. expS, expTS, expM, expTM := 160, 80, 10, 5 - if index == inmem.IndexName { - expS, expTS, expM, expTM = 160, 80, 10, 5 - } // Check cardinalities - tombstones should be in if err := checkCardinalities(store.Store, expS, expTS, expM, expTM); err != nil { @@ -1334,11 +1220,8 @@ func TestStore_Sketches(t *testing.T) { return err } - // Check cardinalities. In this case, the indexes behave differently. + // Check cardinalities. expS, expTS, expM, expTM = 80, 80, 5, 5 - if index == inmem.IndexName { - expS, expTS, expM, expTM = 80, 0, 5, 0 - } if err := checkCardinalities(store.Store, expS, expTS, expM, expTM); err != nil { return fmt.Errorf("[initial|re-open|delete|re-open] %v", err) diff --git a/ui/src/writeData/components/telegrafPlugins/dcos.md b/ui/src/writeData/components/telegrafPlugins/dcos.md index 790590aeaf..4c9d46a921 100644 --- a/ui/src/writeData/components/telegrafPlugins/dcos.md +++ b/ui/src/writeData/components/telegrafPlugins/dcos.md @@ -13,9 +13,6 @@ your database. options to exclude unneeded tags. - Write to a database with an appropriate [retention policy](https://docs.influxdata.com/influxdb/latest/guides/downsampling_and_retention/). -- Limit series cardinality in your database using the - [`max-series-per-database`](https://docs.influxdata.com/influxdb/latest/administration/config/#max-series-per-database-1000000) and - [`max-values-per-tag`](https://docs.influxdata.com/influxdb/latest/administration/config/#max-values-per-tag-100000) settings. - Consider using the [Time Series Index](https://docs.influxdata.com/influxdb/latest/concepts/time-series-index/). - Monitor your databases diff --git a/ui/src/writeData/components/telegrafPlugins/kube_inventory.md b/ui/src/writeData/components/telegrafPlugins/kube_inventory.md index dbed6d6f01..67c1ff4d1b 100644 --- a/ui/src/writeData/components/telegrafPlugins/kube_inventory.md +++ b/ui/src/writeData/components/telegrafPlugins/kube_inventory.md @@ -27,8 +27,6 @@ avoid cardinality issues: - Use [metric filtering][] options to exclude unneeded measurements and tags. - Write to a database with an appropriate [retention policy][]. -- Limit series cardinality in your database using the - [max-series-per-database][] and [max-values-per-tag][] settings. - Consider using the [Time Series Index][tsi]. - Monitor your databases [series cardinality][]. - Consult the [InfluxDB documentation][influx-docs] for the most up-to-date techniques. @@ -305,8 +303,6 @@ kubernetes_statefulset,namespace=default,selector_select1=s1,statefulset_name=et [metric filtering]: https://github.com/influxdata/telegraf/blob/master/docs/CONFIGURATION.md#metric-filtering [retention policy]: https://docs.influxdata.com/influxdb/latest/guides/downsampling_and_retention/ -[max-series-per-database]: https://docs.influxdata.com/influxdb/latest/administration/config/#max-series-per-database-1000000 -[max-values-per-tag]: https://docs.influxdata.com/influxdb/latest/administration/config/#max-values-per-tag-100000 [tsi]: https://docs.influxdata.com/influxdb/latest/concepts/time-series-index/ [series cardinality]: https://docs.influxdata.com/influxdb/latest/query_language/spec/#show-cardinality [influx-docs]: https://docs.influxdata.com/influxdb/latest/ diff --git a/ui/src/writeData/components/telegrafPlugins/kubernetes.md b/ui/src/writeData/components/telegrafPlugins/kubernetes.md index a574bed06f..8ef5ef7b1d 100644 --- a/ui/src/writeData/components/telegrafPlugins/kubernetes.md +++ b/ui/src/writeData/components/telegrafPlugins/kubernetes.md @@ -28,8 +28,6 @@ avoid cardinality issues: - Use [metric filtering][] options to exclude unneeded measurements and tags. - Write to a database with an appropriate [retention policy][]. -- Limit series cardinality in your database using the - [max-series-per-database][] and [max-values-per-tag][] settings. - Consider using the [Time Series Index][tsi]. - Monitor your databases [series cardinality][]. - Consult the [InfluxDB documentation][influx-docs] for the most up-to-date techniques. @@ -154,8 +152,6 @@ kubernetes_system_container [metric filtering]: https://github.com/influxdata/telegraf/blob/master/docs/CONFIGURATION.md#metric-filtering [retention policy]: https://docs.influxdata.com/influxdb/latest/guides/downsampling_and_retention/ -[max-series-per-database]: https://docs.influxdata.com/influxdb/latest/administration/config/#max-series-per-database-1000000 -[max-values-per-tag]: https://docs.influxdata.com/influxdb/latest/administration/config/#max-values-per-tag-100000 [tsi]: https://docs.influxdata.com/influxdb/latest/concepts/time-series-index/ [series cardinality]: https://docs.influxdata.com/influxdb/latest/query_language/spec/#show-cardinality [influx-docs]: https://docs.influxdata.com/influxdb/latest/ diff --git a/ui/src/writeData/components/telegrafPlugins/passenger.md b/ui/src/writeData/components/telegrafPlugins/passenger.md index 688f4e69ae..6821635103 100644 --- a/ui/src/writeData/components/telegrafPlugins/passenger.md +++ b/ui/src/writeData/components/telegrafPlugins/passenger.md @@ -15,9 +15,6 @@ manage your series cardinality: `tagexclude` to remove the `pid` and `process_group_id` tags. - Write to a database with an appropriate [retention policy](https://docs.influxdata.com/influxdb/latest/guides/downsampling_and_retention/). -- Limit series cardinality in your database using the - [`max-series-per-database`](https://docs.influxdata.com/influxdb/latest/administration/config/#max-series-per-database-1000000) and - [`max-values-per-tag`](https://docs.influxdata.com/influxdb/latest/administration/config/#max-values-per-tag-100000) settings. - Consider using the [Time Series Index](https://docs.influxdata.com/influxdb/latest/concepts/time-series-index/). - Monitor your databases diff --git a/ui/src/writeData/components/telegrafPlugins/sflow.md b/ui/src/writeData/components/telegrafPlugins/sflow.md index 66d556e17c..9e5366706e 100644 --- a/ui/src/writeData/components/telegrafPlugins/sflow.md +++ b/ui/src/writeData/components/telegrafPlugins/sflow.md @@ -14,8 +14,6 @@ avoid cardinality issues: - Use [metric filtering][] options to exclude unneeded measurements and tags. - Write to a database with an appropriate [retention policy][]. -- Limit series cardinality in your database using the - [max-series-per-database][] and [max-values-per-tag][] settings. - Consider using the [Time Series Index][tsi]. - Monitor your databases [series cardinality][]. - Consult the [InfluxDB documentation][influx-docs] for the most up-to-date techniques. @@ -113,8 +111,6 @@ This sflow implementation was built from the reference document [metric filtering]: https://github.com/influxdata/telegraf/blob/master/docs/CONFIGURATION.md#metric-filtering [retention policy]: https://docs.influxdata.com/influxdb/latest/guides/downsampling_and_retention/ -[max-series-per-database]: https://docs.influxdata.com/influxdb/latest/administration/config/#max-series-per-database-1000000 -[max-values-per-tag]: https://docs.influxdata.com/influxdb/latest/administration/config/#max-values-per-tag-100000 [tsi]: https://docs.influxdata.com/influxdb/latest/concepts/time-series-index/ [series cardinality]: https://docs.influxdata.com/influxdb/latest/query_language/spec/#show-cardinality [influx-docs]: https://docs.influxdata.com/influxdb/latest/