Merge pull request #1321 from influxdata/jmw-config-cleanup

Storage engine config cleanup
pull/10616/head
Jeff Wendling 2018-11-08 16:54:53 -07:00 committed by GitHub
commit 6b57c7ded0
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
31 changed files with 524 additions and 615 deletions

View File

@ -16,6 +16,7 @@ import (
"github.com/influxdata/platform/logger"
"github.com/influxdata/platform/models"
"github.com/influxdata/platform/storage"
"github.com/influxdata/platform/toml"
"github.com/influxdata/platform/tsdb"
"github.com/influxdata/platform/tsdb/tsi1"
@ -62,7 +63,7 @@ func (cmd *Command) Run(args ...string) error {
fs.StringVar(&cmd.retentionFilter, "retention", "", "optional: retention policy")
fs.StringVar(&cmd.shardFilter, "shard", "", "optional: shard id")
fs.Int64Var(&cmd.maxLogFileSize, "max-log-file-size", tsi1.DefaultMaxIndexLogFileSize, "optional: maximum log file size")
fs.Uint64Var(&cmd.maxCacheSize, "max-cache-size", tsdb.DefaultCacheMaxMemorySize, "optional: maximum cache size")
fs.Uint64Var(&cmd.maxCacheSize, "max-cache-size", tsm1.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)
@ -115,7 +116,7 @@ func (cmd *Command) run(dataDir, walDir string) error {
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.DefaultSeriesFileDirectory))
sfile := tsdb.NewSeriesFile(filepath.Join(dataDir, storage.DefaultSeriesFileDirectoryName))
sfile.Logger = cmd.Logger
if err := sfile.Open(); err != nil {
return err
@ -131,7 +132,7 @@ func (cmd *Command) processDatabase(dbName, dataDir, walDir string) error {
rpName := fi.Name()
if !fi.IsDir() {
continue
} else if rpName == tsdb.DefaultSeriesFileDirectory {
} else if rpName == storage.DefaultSeriesFileDirectoryName {
continue
} else if cmd.retentionFilter != "" && rpName != cmd.retentionFilter {
continue
@ -225,7 +226,7 @@ func IndexShard(sfile *tsdb.SeriesFile, dataDir, walDir string, maxLogFileSize i
c := tsi1.NewConfig()
c.MaxIndexLogFileSize = toml.Size(maxLogFileSize)
tsiIndex := tsi1.NewIndex(sfile, "", c,
tsiIndex := tsi1.NewIndex(sfile, c,
tsi1.WithPath(tmpPath),
tsi1.DisableFsync(),
// Each new series entry in a log file is ~12 bytes so this should
@ -264,7 +265,7 @@ func IndexShard(sfile *tsdb.SeriesFile, dataDir, walDir string, maxLogFileSize i
} else {
log.Info("Building cache from wal files")
cache := tsm1.NewCache(tsdb.DefaultCacheMaxMemorySize)
cache := tsm1.NewCache(tsm1.DefaultCacheMaxMemorySize)
loader := tsm1.NewCacheLoader(walPaths)
loader.WithLogger(log)
if err := loader.Load(cache); err != nil {

View File

@ -188,8 +188,6 @@ func run() error {
var pointsWriter storage.PointsWriter
{
config := storage.NewConfig()
config.EngineOptions.WALEnabled = true // Enable a disk-based WAL.
config.EngineOptions.Config = config.Config
engine := storage.NewEngine(enginePath, config, storage.WithRetentionEnforcer(bucketSvc))
engine.WithLogger(logger)

68
storage/compat/compat.go Normal file
View File

@ -0,0 +1,68 @@
// package compat helps with migrating toml files from influxdb.
//
// Platform has a different config layout that doesn't match the
// layout in the old toml files. Since we may want the ability to
// load those old files and still have them apply their relevant
// settings, this package allows one to convert from an old config
// struct to a new one.
package compat
import (
"github.com/influxdata/platform/storage"
"github.com/influxdata/platform/toml"
"github.com/influxdata/platform/tsdb/tsm1"
)
// Config matches the old toml layout from the influxdb repo, so that we can read
// in those files and convert them to the new config layout.
type Config struct {
Dir string `toml:"dir"`
WALDir string `toml:"wal-dir"`
WALFsyncDelay toml.Duration `toml:"wal-fsync-delay"`
ValidateKeys bool `toml:"validate-keys"`
CacheMaxMemorySize toml.Size `toml:"cache-max-memory-size"`
CacheSnapshotMemorySize toml.Size `toml:"cache-snapshot-memory-size"`
CacheSnapshotWriteColdDuration toml.Duration `toml:"cache-snapshot-write-cold-duration"`
CompactFullWriteColdDuration toml.Duration `toml:"compact-full-write-cold-duration"`
CompactThroughput toml.Size `toml:"compact-throughput"`
CompactThroughputBurst toml.Size `toml:"compact-throughput-burst"`
MaxConcurrentCompactions int `toml:"max-concurrent-compactions"`
TraceLoggingEnabled bool `toml:"trace-logging-enabled"`
TSMWillNeed bool `toml:"tsm-use-madv-willneed"`
}
// NewConfig constructs an old Config struct with appropriate defaults for a new Config.
func NewConfig() Config {
return Config{
WALFsyncDelay: toml.Duration(tsm1.DefaultWALFsyncDelay),
ValidateKeys: storage.DefaultValidateKeys,
CacheMaxMemorySize: toml.Size(tsm1.DefaultCacheMaxMemorySize),
CacheSnapshotMemorySize: toml.Size(tsm1.DefaultCacheSnapshotMemorySize),
CacheSnapshotWriteColdDuration: toml.Duration(tsm1.DefaultCacheSnapshotWriteColdDuration),
CompactFullWriteColdDuration: toml.Duration(tsm1.DefaultCompactFullWriteColdDuration),
CompactThroughput: toml.Size(tsm1.DefaultCompactThroughput),
CompactThroughputBurst: toml.Size(tsm1.DefaultCompactThroughputBurst),
MaxConcurrentCompactions: tsm1.DefaultCompactMaxConcurrent,
TraceLoggingEnabled: storage.DefaultTraceLoggingEnabled,
TSMWillNeed: tsm1.DefaultMADVWillNeed,
}
}
// Convert takes an old Config and converts it into a new Config. It also returns the value
// of the Dir key so that it can be passed through appropriately to the storage engine constructor.
func Convert(oldConfig Config) (string, storage.Config) {
newConfig := storage.NewConfig()
newConfig.TraceLoggingEnabled = oldConfig.TraceLoggingEnabled
newConfig.ValidateKeys = oldConfig.ValidateKeys
newConfig.Engine.MADVWillNeed = oldConfig.TSMWillNeed
newConfig.Engine.Cache.MaxMemorySize = oldConfig.CacheMaxMemorySize
newConfig.Engine.Cache.SnapshotMemorySize = oldConfig.CacheSnapshotMemorySize
newConfig.Engine.Cache.SnapshotWriteColdDuration = oldConfig.CacheSnapshotWriteColdDuration
newConfig.Engine.Compaction.FullWriteColdDuration = oldConfig.CompactFullWriteColdDuration
newConfig.Engine.Compaction.Throughput = oldConfig.CompactThroughput
newConfig.Engine.Compaction.ThroughputBurst = oldConfig.CompactThroughputBurst
newConfig.Engine.Compaction.MaxConcurrent = oldConfig.MaxConcurrentCompactions
newConfig.WALPath = oldConfig.WALDir
newConfig.WAL.FsyncDelay = oldConfig.WALFsyncDelay
return oldConfig.Dir, newConfig
}

View File

@ -1,30 +1,93 @@
package storage
import (
"github.com/influxdata/platform/tsdb"
"path/filepath"
"time"
"github.com/influxdata/platform/toml"
"github.com/influxdata/platform/tsdb/tsi1"
"github.com/influxdata/platform/tsdb/tsm1"
)
// Config defaults
const (
DefaultRetentionInterval = 3600 // 1 hour.
DefaultRetentionInterval = 1 * time.Hour
DefaultValidateKeys = false
DefaultTraceLoggingEnabled = false
DefaultSeriesFileDirectoryName = "_series"
DefaultIndexDirectoryName = "index"
DefaultWALDirectoryName = "wal"
DefaultEngineDirectoryName = "data"
)
// Config holds the configuration for an Engine.
type Config struct {
RetentionInterval int64 `toml:"retention_interval"` // Frequency of retention in seconds.
// Frequency of retention in seconds.
RetentionInterval toml.Duration `toml:"retention-interval"`
EngineOptions tsdb.EngineOptions `toml:"-"`
Index tsi1.Config `toml:"index"`
tsdb.Config
// Enables unicode validation on series keys on write.
ValidateKeys bool `toml:"validate-keys"`
// Enables trace logging for the engine.
TraceLoggingEnabled bool `toml:"trace-logging-enabled"`
// Series file config.
SeriesFilePath string `toml:"series-file-path"` // Overrides the default path.
// WAL config.
WAL tsm1.WALConfig `toml:"wal"`
WALPath string `toml:"wal-path"` // Overrides the default path.
// Engine config.
Engine tsm1.Config `toml:"engine"`
EnginePath string `toml:"engine-path"` // Overrides the default path.
// Index config.
Index tsi1.Config `toml:"index"`
IndexPath string `toml:"index-path"` // Overrides the default path.
}
// NewConfig initialises a new config for an Engine.
func NewConfig() Config {
return Config{
RetentionInterval: DefaultRetentionInterval,
EngineOptions: tsdb.NewEngineOptions(),
Index: tsi1.NewConfig(),
Config: tsdb.NewConfig(),
RetentionInterval: toml.Duration(DefaultRetentionInterval),
ValidateKeys: DefaultValidateKeys,
TraceLoggingEnabled: DefaultTraceLoggingEnabled,
WAL: tsm1.NewWALConfig(),
Engine: tsm1.NewConfig(),
Index: tsi1.NewConfig(),
}
}
// GetSeriesFilePath returns the path to the series file.
func (c Config) GetSeriesFilePath(base string) string {
if c.SeriesFilePath != "" {
return c.SeriesFilePath
}
return filepath.Join(base, DefaultSeriesFileDirectoryName)
}
// GetIndexPath returns the path to the index.
func (c Config) GetIndexPath(base string) string {
if c.IndexPath != "" {
return c.IndexPath
}
return filepath.Join(base, DefaultIndexDirectoryName)
}
// GetWALPath returns the path to the WAL.
func (c Config) GetWALPath(base string) string {
if c.WALPath != "" {
return c.WALPath
}
return filepath.Join(base, DefaultWALDirectoryName)
}
// GetEnginePath returns the path to the engine.
func (c Config) GetEnginePath(base string) string {
if c.EnginePath != "" {
return c.EnginePath
}
return filepath.Join(base, DefaultEngineDirectoryName)
}

View File

@ -5,7 +5,6 @@ import (
"context"
"errors"
"fmt"
"path/filepath"
"sync"
"time"
@ -37,6 +36,7 @@ type Engine struct {
index *tsi1.Index
sfile *tsdb.SeriesFile
engine *tsm1.Engine
wal *tsm1.WAL
retentionEnforcer *retentionEnforcer
// Tracks all goroutines started by the Engine.
@ -50,7 +50,7 @@ type Option func(*Engine)
// WithTSMFilenameFormatter sets a function on the underlying tsm1.Engine to specify
// how TSM files are named.
var WithTSMFilenameFormatter = func(fn tsm1.FormatFileNameFunc) Option {
func WithTSMFilenameFormatter(fn tsm1.FormatFileNameFunc) Option {
return func(e *Engine) {
e.engine.WithFormatFileNameFunc(fn)
}
@ -58,7 +58,7 @@ var WithTSMFilenameFormatter = func(fn tsm1.FormatFileNameFunc) Option {
// WithEngineID sets an engine id, which can be useful for logging when multiple
// engines are in use.
var WithEngineID = func(id int) Option {
func WithEngineID(id int) Option {
return func(e *Engine) {
e.engineID = &id
}
@ -66,7 +66,7 @@ var WithEngineID = func(id int) Option {
// WithNodeID sets a node id on the engine, which can be useful for logging
// when a system has engines running on multiple nodes.
var WithNodeID = func(id int) Option {
func WithNodeID(id int) Option {
return func(e *Engine) {
e.nodeID = &id
}
@ -75,7 +75,7 @@ var WithNodeID = func(id int) Option {
// WithRetentionEnforcer initialises a retention enforcer on the engine.
// WithRetentionEnforcer must be called after other options to ensure that all
// metrics are labelled correctly.
var WithRetentionEnforcer = func(finder BucketFinder) Option {
func WithRetentionEnforcer(finder BucketFinder) Option {
return func(e *Engine) {
e.retentionEnforcer = newRetentionEnforcer(e, finder)
@ -92,28 +92,47 @@ var WithRetentionEnforcer = func(finder BucketFinder) Option {
}
}
// WithFileStoreObserver makes the engine have the provided file store observer.
func WithFileStoreObserver(obs tsm1.FileStoreObserver) Option {
return func(e *Engine) {
e.engine.WithFileStoreObserver(obs)
}
}
// WithCompactionPlanner makes the engine have the provided compaction planner.
func WithCompactionPlanner(planner tsm1.CompactionPlanner) Option {
return func(e *Engine) {
e.engine.WithCompactionPlanner(planner)
}
}
// NewEngine initialises a new storage engine, including a series file, index and
// TSM engine.
func NewEngine(path string, c Config, options ...Option) *Engine {
e := &Engine{
config: c,
path: path,
sfile: tsdb.NewSeriesFile(filepath.Join(path, tsdb.DefaultSeriesFileDirectory)),
logger: zap.NewNop(),
}
// Initialize series file.
e.sfile = tsdb.NewSeriesFile(c.GetSeriesFilePath(path))
// Initialise index.
index := tsi1.NewIndex(e.sfile, "remove me", c.Index,
tsi1.WithPath(filepath.Join(path, tsi1.DefaultIndexDirectoryName)),
)
e.index = index
e.index = tsi1.NewIndex(e.sfile, c.Index,
tsi1.WithPath(c.GetIndexPath(path)))
// Initialize WAL
if c.WAL.Enabled {
e.wal = tsm1.NewWAL(c.GetWALPath(path))
e.wal.WithFsyncDelay(time.Duration(c.WAL.FsyncDelay))
e.wal.EnableTraceLogging(c.TraceLoggingEnabled)
}
// Initialise Engine
// TODO(edd): should just be able to use the config values for data/wal.
engine := tsm1.NewEngine(0, e.index, filepath.Join(path, "data"), filepath.Join(path, "wal"), e.sfile, c.EngineOptions)
// TODO(edd): Once the tsdb.Engine abstraction is gone, this won't be needed.
e.engine = engine.(*tsm1.Engine)
e.engine = tsm1.NewEngine(c.GetEnginePath(path), e.index, c.Engine,
tsm1.WithWAL(e.wal),
tsm1.WithTraceLogging(c.TraceLoggingEnabled))
// Apply options.
for _, option := range options {
@ -190,17 +209,16 @@ func (e *Engine) Open() error {
// ability to reschedule the retention enforcement if there are not enough
// resources available.
func (e *Engine) runRetentionEnforcer() {
if e.config.RetentionInterval == 0 {
interval := time.Duration(e.config.RetentionInterval)
if interval == 0 {
e.logger.Info("Retention enforcer disabled")
return // Enforcer disabled.
}
if e.config.RetentionInterval < 0 {
e.logger.Error("Negative retention interval", zap.Int64("interval", e.config.RetentionInterval))
} else if interval < 0 {
e.logger.Error("Negative retention interval", logger.DurationLiteral("check_interval", interval))
return
}
interval := time.Duration(e.config.RetentionInterval) * time.Second
l := e.logger.With(zap.String("component", "retention_enforcer"), logger.DurationLiteral("check_interval", interval))
l.Info("Starting")

View File

@ -190,9 +190,6 @@ type Engine struct {
func NewEngine(c storage.Config) *Engine {
path, _ := ioutil.TempDir("", "storage_engine_test")
// TODO(edd) clean this up...
c.EngineOptions.Config = c.Config
engine := storage.NewEngine(path, c)
return &Engine{
path: path,

View File

@ -10,7 +10,13 @@ import (
"errors"
"github.com/influxdata/platform/tsdb/cursors"
"github.com/influxdata/platform/tsdb/defaults"
)
const (
// MaxPointsPerBlock is the maximum number of points in an encoded
// block in a TSM file. It should match the value in the tsm1
// package, but we don't want to import it.
MaxPointsPerBlock = 1000
)
// ********************
@ -28,7 +34,7 @@ func newFloatFilterArrayCursor(cond expression) *floatArrayFilterCursor {
return &floatArrayFilterCursor{
cond: cond,
m: &singleValue{},
res: cursors.NewFloatArrayLen(defaults.DefaultMaxPointsPerBlock),
res: cursors.NewFloatArrayLen(MaxPointsPerBlock),
tmp: &cursors.FloatArray{},
}
}
@ -61,7 +67,7 @@ LOOP:
c.res.Timestamps[pos] = a.Timestamps[i]
c.res.Values[pos] = v
pos++
if pos >= defaults.DefaultMaxPointsPerBlock {
if pos >= MaxPointsPerBlock {
c.tmp.Timestamps = a.Timestamps[i+1:]
c.tmp.Values = a.Values[i+1:]
break LOOP
@ -244,7 +250,7 @@ func newIntegerFilterArrayCursor(cond expression) *integerArrayFilterCursor {
return &integerArrayFilterCursor{
cond: cond,
m: &singleValue{},
res: cursors.NewIntegerArrayLen(defaults.DefaultMaxPointsPerBlock),
res: cursors.NewIntegerArrayLen(MaxPointsPerBlock),
tmp: &cursors.IntegerArray{},
}
}
@ -277,7 +283,7 @@ LOOP:
c.res.Timestamps[pos] = a.Timestamps[i]
c.res.Values[pos] = v
pos++
if pos >= defaults.DefaultMaxPointsPerBlock {
if pos >= MaxPointsPerBlock {
c.tmp.Timestamps = a.Timestamps[i+1:]
c.tmp.Values = a.Values[i+1:]
break LOOP
@ -460,7 +466,7 @@ func newUnsignedFilterArrayCursor(cond expression) *unsignedArrayFilterCursor {
return &unsignedArrayFilterCursor{
cond: cond,
m: &singleValue{},
res: cursors.NewUnsignedArrayLen(defaults.DefaultMaxPointsPerBlock),
res: cursors.NewUnsignedArrayLen(MaxPointsPerBlock),
tmp: &cursors.UnsignedArray{},
}
}
@ -493,7 +499,7 @@ LOOP:
c.res.Timestamps[pos] = a.Timestamps[i]
c.res.Values[pos] = v
pos++
if pos >= defaults.DefaultMaxPointsPerBlock {
if pos >= MaxPointsPerBlock {
c.tmp.Timestamps = a.Timestamps[i+1:]
c.tmp.Values = a.Values[i+1:]
break LOOP
@ -676,7 +682,7 @@ func newStringFilterArrayCursor(cond expression) *stringArrayFilterCursor {
return &stringArrayFilterCursor{
cond: cond,
m: &singleValue{},
res: cursors.NewStringArrayLen(defaults.DefaultMaxPointsPerBlock),
res: cursors.NewStringArrayLen(MaxPointsPerBlock),
tmp: &cursors.StringArray{},
}
}
@ -709,7 +715,7 @@ LOOP:
c.res.Timestamps[pos] = a.Timestamps[i]
c.res.Values[pos] = v
pos++
if pos >= defaults.DefaultMaxPointsPerBlock {
if pos >= MaxPointsPerBlock {
c.tmp.Timestamps = a.Timestamps[i+1:]
c.tmp.Values = a.Values[i+1:]
break LOOP
@ -854,7 +860,7 @@ func newBooleanFilterArrayCursor(cond expression) *booleanArrayFilterCursor {
return &booleanArrayFilterCursor{
cond: cond,
m: &singleValue{},
res: cursors.NewBooleanArrayLen(defaults.DefaultMaxPointsPerBlock),
res: cursors.NewBooleanArrayLen(MaxPointsPerBlock),
tmp: &cursors.BooleanArray{},
}
}
@ -887,7 +893,7 @@ LOOP:
c.res.Timestamps[pos] = a.Timestamps[i]
c.res.Values[pos] = v
pos++
if pos >= defaults.DefaultMaxPointsPerBlock {
if pos >= MaxPointsPerBlock {
c.tmp.Timestamps = a.Timestamps[i+1:]
c.tmp.Values = a.Values[i+1:]
break LOOP

View File

@ -4,7 +4,13 @@ import (
"errors"
"github.com/influxdata/platform/tsdb/cursors"
"github.com/influxdata/platform/tsdb/defaults"
)
const (
// MaxPointsPerBlock is the maximum number of points in an encoded
// block in a TSM file. It should match the value in the tsm1
// package, but we don't want to import it.
MaxPointsPerBlock = 1000
)
{{range .}}
@ -27,7 +33,7 @@ func new{{.Name}}FilterArrayCursor(cond expression) *{{$type}} {
return &{{$type}}{
cond: cond,
m: &singleValue{},
res: cursors.New{{.Name}}ArrayLen(defaults.DefaultMaxPointsPerBlock),
res: cursors.New{{.Name}}ArrayLen(MaxPointsPerBlock),
tmp: &cursors.{{.Name}}Array{},
}
}
@ -60,7 +66,7 @@ LOOP:
c.res.Timestamps[pos] = a.Timestamps[i]
c.res.Values[pos] = v
pos++
if pos >= defaults.DefaultMaxPointsPerBlock {
if pos >= MaxPointsPerBlock {
c.tmp.Timestamps = a.Timestamps[i+1:]
c.tmp.Values = a.Values[i+1:]
break LOOP

View File

@ -1,120 +1,8 @@
package tsdb
import (
"errors"
"github.com/influxdata/influxdb/monitor/diagnostics"
"github.com/influxdata/influxdb/query"
"github.com/influxdata/platform/toml"
"github.com/influxdata/platform/tsdb/defaults"
)
// TODO(jeff): port things to use the defaults package
// EOF represents a "not found" key returned by a Cursor.
const EOF = query.ZeroTime
const ( // See the defaults package for explanations of what these mean
DefaultEngine = defaults.DefaultEngine
DefaultIndex = defaults.DefaultIndex
DefaultCacheMaxMemorySize = defaults.DefaultCacheMaxMemorySize
DefaultCacheSnapshotMemorySize = defaults.DefaultCacheSnapshotMemorySize
DefaultCacheSnapshotWriteColdDuration = defaults.DefaultCacheSnapshotWriteColdDuration
DefaultCompactFullWriteColdDuration = defaults.DefaultCompactFullWriteColdDuration
DefaultCompactThroughput = defaults.DefaultCompactThroughput
DefaultCompactThroughputBurst = defaults.DefaultCompactThroughputBurst
DefaultMaxPointsPerBlock = defaults.DefaultMaxPointsPerBlock
DefaultMaxConcurrentCompactions = defaults.DefaultMaxConcurrentCompactions
DefaultSeriesFileDirectory = defaults.DefaultSeriesFileDirectory
)
// Config holds the configuration for the tsbd package.
type Config struct {
Dir string `toml:"dir"`
Engine string `toml:"-"`
Index string `toml:"index-version"`
// General WAL configuration options
WALDir string `toml:"wal-dir"`
// WALFsyncDelay is the amount of time that a write will wait before fsyncing. A duration
// greater than 0 can be used to batch up multiple fsync calls. This is useful for slower
// disks or when WAL write contention is seen. A value of 0 fsyncs every write to the WAL.
WALFsyncDelay toml.Duration `toml:"wal-fsync-delay"`
// Enables unicode validation on series keys on write.
ValidateKeys bool `toml:"validate-keys"`
// Query logging
QueryLogEnabled bool `toml:"query-log-enabled"`
// Compaction options for tsm1 (descriptions above with defaults)
CacheMaxMemorySize toml.Size `toml:"cache-max-memory-size"`
CacheSnapshotMemorySize toml.Size `toml:"cache-snapshot-memory-size"`
CacheSnapshotWriteColdDuration toml.Duration `toml:"cache-snapshot-write-cold-duration"`
CompactFullWriteColdDuration toml.Duration `toml:"compact-full-write-cold-duration"`
CompactThroughput toml.Size `toml:"compact-throughput"`
CompactThroughputBurst toml.Size `toml:"compact-throughput-burst"`
// Limits
// 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
// not affected by this limit. A value of 0 limits compactions to runtime.GOMAXPROCS(0).
MaxConcurrentCompactions int `toml:"max-concurrent-compactions"`
TraceLoggingEnabled bool `toml:"trace-logging-enabled"`
// TSMWillNeed controls whether we hint to the kernel that we intend to
// page in mmap'd sections of TSM files. This setting defaults to off, as it has
// been found to be problematic in some cases. It may help users who have
// slow disks.
TSMWillNeed bool `toml:"tsm-use-madv-willneed"`
}
// NewConfig returns the default configuration for tsdb.
func NewConfig() Config {
return Config{
Engine: DefaultEngine,
Index: DefaultIndex,
QueryLogEnabled: true,
CacheMaxMemorySize: toml.Size(DefaultCacheMaxMemorySize),
CacheSnapshotMemorySize: toml.Size(DefaultCacheSnapshotMemorySize),
CacheSnapshotWriteColdDuration: toml.Duration(DefaultCacheSnapshotWriteColdDuration),
CompactFullWriteColdDuration: toml.Duration(DefaultCompactFullWriteColdDuration),
CompactThroughput: toml.Size(DefaultCompactThroughput),
CompactThroughputBurst: toml.Size(DefaultCompactThroughputBurst),
MaxConcurrentCompactions: DefaultMaxConcurrentCompactions,
TraceLoggingEnabled: false,
TSMWillNeed: false,
}
}
// Validate validates the configuration hold by c.
func (c *Config) Validate() error {
if c.Dir == "" {
return errors.New("Data.Dir must be specified")
}
if c.MaxConcurrentCompactions < 0 {
return errors.New("max-concurrent-compactions must be greater than 0")
}
return nil
}
// Diagnostics returns a diagnostics representation of a subset of the Config.
func (c Config) Diagnostics() (*diagnostics.Diagnostics, error) {
return diagnostics.RowFromMap(map[string]interface{}{
"dir": c.Dir,
"wal-dir": c.WALDir,
"wal-fsync-delay": c.WALFsyncDelay,
"cache-max-memory-size": c.CacheMaxMemorySize,
"cache-snapshot-memory-size": c.CacheSnapshotMemorySize,
"cache-snapshot-write-cold-duration": c.CacheSnapshotWriteColdDuration,
"compact-full-write-cold-duration": c.CompactFullWriteColdDuration,
"max-concurrent-compactions": c.MaxConcurrentCompactions,
}), nil
}

View File

@ -1,55 +0,0 @@
package defaults
import "time"
const (
// DefaultEngine is the default engine for new shards
DefaultEngine = "tsm1"
// DefaultIndex is the default index for new shards
DefaultIndex = "tsi1"
// tsdb/engine/wal configuration options
// Default settings for TSM
// DefaultCacheMaxMemorySize is the maximum size a shard's cache can
// reach before it starts rejecting writes.
DefaultCacheMaxMemorySize = 1024 * 1024 * 1024 // 1GB
// DefaultCacheSnapshotMemorySize is the size at which the engine will
// snapshot the cache and write it to a TSM file, freeing up memory
DefaultCacheSnapshotMemorySize = 25 * 1024 * 1024 // 25MB
// DefaultCacheSnapshotWriteColdDuration is the length of time at which
// the engine will snapshot the cache and write it to a new TSM file if
// the shard hasn't received writes or deletes
DefaultCacheSnapshotWriteColdDuration = time.Duration(10 * time.Minute)
// DefaultCompactFullWriteColdDuration is the duration at which the engine
// will compact all TSM files in a shard if it hasn't received a write or delete
DefaultCompactFullWriteColdDuration = time.Duration(4 * time.Hour)
// DefaultCompactThroughput is the rate limit in bytes per second that we
// will allow TSM compactions to write to disk. Not that short bursts are allowed
// to happen at a possibly larger value, set by DefaultCompactThroughputBurst.
// A value of 0 here will disable compaction rate limiting
DefaultCompactThroughput = 48 * 1024 * 1024
// DefaultCompactThroughputBurst is the rate limit in bytes per second that we
// will allow TSM compactions to write to disk. If this is not set, the burst value
// will be set to equal the normal throughput
DefaultCompactThroughputBurst = 48 * 1024 * 1024
// DefaultMaxPointsPerBlock is the maximum number of points in an encoded
// block in a TSM file
DefaultMaxPointsPerBlock = 1000
// DefaultMaxConcurrentCompactions is the maximum number of concurrent full and level compactions
// that can run at one time. A value of 0 results in 50% of runtime.GOMAXPROCS(0) used at runtime.
DefaultMaxConcurrentCompactions = 0
// DefaultSeriesFileDirectory is the name of the directory containing series files for
// a database.
DefaultSeriesFileDirectory = "_series"
)

View File

@ -1,128 +0,0 @@
package tsdb
import (
"context"
"errors"
"regexp"
"runtime"
"time"
"github.com/influxdata/influxql"
"github.com/influxdata/platform/models"
"github.com/influxdata/platform/pkg/limiter"
"go.uber.org/zap"
)
var (
// ErrUnknownEngineFormat is returned when the engine format is
// unknown. ErrUnknownEngineFormat is currently returned if a format
// other than tsm1 is encountered.
ErrUnknownEngineFormat = errors.New("unknown engine format")
)
// Engine represents a swappable storage engine for the shard.
type Engine interface {
Open() error
Close() error
SetEnabled(enabled bool)
SetCompactionsEnabled(enabled bool)
ScheduleFullCompaction() error
WithLogger(*zap.Logger)
CreateCursorIterator(ctx context.Context) (CursorIterator, error)
WritePoints(points []models.Point) error
CreateSeriesIfNotExists(key, name []byte, tags models.Tags, typ models.FieldType) error
CreateSeriesListIfNotExists(collection *SeriesCollection) error
DeleteSeriesRange(itr SeriesIterator, min, max int64) error
DeleteSeriesRangeWithPredicate(itr SeriesIterator, predicate func(name []byte, tags models.Tags) (int64, int64, bool)) error
SeriesN() int64
MeasurementExists(name []byte) (bool, error)
MeasurementNamesByRegex(re *regexp.Regexp) ([][]byte, error)
ForEachMeasurementName(fn func(name []byte) error) error
DeleteMeasurement(name []byte) error
HasTagKey(name, key []byte) (bool, error)
MeasurementTagKeysByExpr(name []byte, expr influxql.Expr) (map[string]struct{}, error)
TagKeyCardinality(name, key []byte) int
// Statistics will return statistics relevant to this engine.
Statistics(tags map[string]string) []models.Statistic
LastModified() time.Time
DiskSize() int64
IsIdle() bool
Free() error
}
// SeriesIDSets provides access to the total set of series IDs
type SeriesIDSets interface {
ForEach(f func(ids *SeriesIDSet)) error
}
// EngineOptions represents the options used to initialize the engine.
type EngineOptions struct {
EngineVersion string
ShardID uint64
// Limits the concurrent number of TSM files that can be loaded at once.
OpenLimiter limiter.Fixed
// CompactionDisabled specifies shards should not schedule compactions.
// This option is intended for offline tooling.
CompactionDisabled bool
CompactionPlannerCreator CompactionPlannerCreator
CompactionLimiter limiter.Fixed
CompactionThroughputLimiter limiter.Rate
WALEnabled bool
MonitorDisabled bool
// DatabaseFilter is a predicate controlling which databases may be opened.
// If no function is set, all databases will be opened.
DatabaseFilter func(database string) bool
// RetentionPolicyFilter is a predicate controlling which combination of database and retention policy may be opened.
// nil will allow all combinations to pass.
RetentionPolicyFilter func(database, rp string) bool
// ShardFilter is a predicate controlling which combination of database, retention policy and shard group may be opened.
// nil will allow all combinations to pass.
ShardFilter func(database, rp string, id uint64) bool
Config Config
SeriesIDSets SeriesIDSets
OnNewEngine func(Engine)
FileStoreObserver FileStoreObserver
}
// NewEngineOptions constructs an EngineOptions object with safe default values.
// This should only be used in tests; production environments should read from a config file.
func NewEngineOptions() EngineOptions {
return EngineOptions{
EngineVersion: DefaultEngine,
Config: NewConfig(),
OpenLimiter: limiter.NewFixed(runtime.GOMAXPROCS(0)),
CompactionLimiter: limiter.NewFixed(1),
WALEnabled: false,
}
}
// 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
// be sure to observe every file that is added or removed even in the presence of process death.
type FileStoreObserver interface {
// FileFinishing is called before a file is renamed to it's final name.
FileFinishing(path string) error
// FileUnlinking is called before a file is unlinked.
FileUnlinking(path string) error
}

View File

@ -10,9 +10,6 @@ import (
"github.com/influxdata/platform/models"
)
// Available index types.
const TSI1IndexName = "tsi1"
// SeriesElem represents a generic series element.
type SeriesElem interface {
Name() []byte

View File

@ -80,7 +80,7 @@ func MustNewIndex(c tsi1.Config) *Index {
panic(err)
}
seriesPath, err := ioutil.TempDir(rootPath, tsdb.DefaultSeriesFileDirectory)
seriesPath, err := ioutil.TempDir(rootPath, "_series")
if err != nil {
panic(err)
}
@ -90,7 +90,7 @@ func MustNewIndex(c tsi1.Config) *Index {
panic(err)
}
i := tsi1.NewIndex(sfile, "remove-me", c, tsi1.WithPath(filepath.Join(rootPath, "index")))
i := tsi1.NewIndex(sfile, c, tsi1.WithPath(filepath.Join(rootPath, "index")))
if testing.Verbose() {
i.WithLogger(logger.New(os.Stderr))
@ -141,7 +141,8 @@ func (i *Index) Reopen() error {
return err
}
i.Index = tsi1.NewIndex(i.SeriesFile(), "remove-me", i.config, tsi1.WithPath(filepath.Join(i.rootPath, "index")))
i.Index = tsi1.NewIndex(i.SeriesFile(), i.config,
tsi1.WithPath(filepath.Join(i.rootPath, "index")))
return i.Index.Open()
}
@ -212,9 +213,9 @@ func BenchmarkIndex_TagSets(b *testing.B) {
// This benchmark will merge eight bitsets each containing ~10,000 series IDs.
b.Run("1M series", func(b *testing.B) {
b.ReportAllocs()
idx := MustOpenNewIndex(tsi1.NewConfig())
setup(idx)
defer idx.Close()
name := []byte("m4")
opt := query.IteratorOptions{Condition: influxql.MustParseExpr(`"tag5"::tag = 'value0'`)}
@ -223,6 +224,9 @@ func BenchmarkIndex_TagSets(b *testing.B) {
return idx.Index.TagSets(name, opt)
}
b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ {
// Will call TagSets on the appropriate implementation.
_, errResult = ts()
@ -342,7 +346,6 @@ func BenchmarkIndex_ConcurrentWriteQuery(b *testing.B) {
}
queries := []int{1e5}
for _, queryN := range queries {
b.Run(fmt.Sprintf("queries %d", queryN), func(b *testing.B) {
b.Run("cache", func(b *testing.B) {

View File

@ -6,10 +6,6 @@ import "github.com/influxdata/platform/toml"
// write-ahead log file will compact into an index file.
const DefaultMaxIndexLogFileSize = 1 * 1024 * 1024 // 1MB
// DefaultIndexDirectoryName is the default name of the directory holding the
// index data.
const DefaultIndexDirectoryName = "index"
// Config holds configurable Index options.
type Config struct {
// MaxIndexLogFileSize is the threshold, in bytes, when an index write-ahead log file will

View File

@ -25,9 +25,6 @@ import (
"go.uber.org/zap"
)
// IndexName is the name of the index.
const IndexName = tsdb.TSI1IndexName
// DefaultSeriesIDSetCacheSize is the default number of series ID sets to cache.
const DefaultSeriesIDSetCacheSize = 100
@ -123,8 +120,7 @@ type Index struct {
logger *zap.Logger // Index's logger.
// The following must be set when initializing an Index.
sfile *tsdb.SeriesFile // series lookup file
database string // Name of database.
sfile *tsdb.SeriesFile // series lookup file
// Index's version.
version int
@ -138,14 +134,13 @@ func (i *Index) UniqueReferenceID() uintptr {
}
// NewIndex returns a new instance of Index.
func NewIndex(sfile *tsdb.SeriesFile, database string, c Config, options ...IndexOption) *Index {
func NewIndex(sfile *tsdb.SeriesFile, c Config, options ...IndexOption) *Index {
idx := &Index{
tagValueCache: NewTagValueSeriesIDCache(DefaultSeriesIDSetCacheSize),
maxLogFileSize: int64(c.MaxIndexLogFileSize),
logger: zap.NewNop(),
version: Version,
sfile: sfile,
database: database,
PartitionN: DefaultPartitionN,
}
@ -172,18 +167,12 @@ func (i *Index) Bytes() int {
b += int(unsafe.Sizeof(i.logger))
b += int(unsafe.Sizeof(i.sfile))
// Do not count SeriesFile because it belongs to the code that constructed this Index.
b += int(unsafe.Sizeof(i.database)) + len(i.database)
b += int(unsafe.Sizeof(i.version))
b += int(unsafe.Sizeof(i.PartitionN))
i.mu.RUnlock()
return b
}
// Database returns the name of the database the index was initialized with.
func (i *Index) Database() string {
return i.database
}
// WithLogger sets the logger on the index after it's been created.
//
// It's not safe to call WithLogger after the index has been opened, or before
@ -192,9 +181,6 @@ func (i *Index) WithLogger(l *zap.Logger) {
i.logger = l.With(zap.String("index", "tsi"))
}
// Type returns the type of Index this is.
func (i *Index) Type() string { return IndexName }
// SeriesFile returns the series file attached to the index.
func (i *Index) SeriesFile() *tsdb.SeriesFile { return i.sfile }

View File

@ -407,7 +407,7 @@ func NewIndex(partitionN uint64, c tsi1.Config) *Index {
Config: tsi1.NewConfig(),
SeriesFile: NewSeriesFile(),
}
idx.Index = tsi1.NewIndex(idx.SeriesFile.SeriesFile, "db0", idx.Config, tsi1.WithPath(MustTempDir()))
idx.Index = tsi1.NewIndex(idx.SeriesFile.SeriesFile, idx.Config, tsi1.WithPath(MustTempDir()))
idx.Index.PartitionN = partitionN
return idx
}
@ -451,7 +451,7 @@ func (idx *Index) Reopen() error {
}
partitionN := idx.Index.PartitionN // Remember how many partitions to use.
idx.Index = tsi1.NewIndex(idx.SeriesFile.SeriesFile, "db0", idx.Config, tsi1.WithPath(idx.Index.Path()))
idx.Index = tsi1.NewIndex(idx.SeriesFile.SeriesFile, idx.Config, tsi1.WithPath(idx.Index.Path()))
idx.Index.PartitionN = partitionN
return idx.Open()
}

View File

@ -21,7 +21,7 @@ func TestLegacyOpen(t *testing.T) {
}
defer sfile.Close()
index := NewIndex(sfile, "db", NewConfig(), WithPath("testdata/index-file-index"))
index := NewIndex(sfile, NewConfig(), WithPath("testdata/index-file-index"))
if err := index.Open(); err != nil {
t.Fatal(err)
}

View File

@ -33,9 +33,9 @@ type floatArrayAscendingCursor struct {
func newFloatArrayAscendingCursor() *floatArrayAscendingCursor {
c := &floatArrayAscendingCursor{
res: tsdb.NewFloatArrayLen(tsdb.DefaultMaxPointsPerBlock),
res: tsdb.NewFloatArrayLen(MaxPointsPerBlock),
}
c.tsm.buf = tsdb.NewFloatArrayLen(tsdb.DefaultMaxPointsPerBlock)
c.tsm.buf = tsdb.NewFloatArrayLen(MaxPointsPerBlock)
return c
}
@ -169,9 +169,9 @@ type floatArrayDescendingCursor struct {
func newFloatArrayDescendingCursor() *floatArrayDescendingCursor {
c := &floatArrayDescendingCursor{
res: tsdb.NewFloatArrayLen(tsdb.DefaultMaxPointsPerBlock),
res: tsdb.NewFloatArrayLen(MaxPointsPerBlock),
}
c.tsm.buf = tsdb.NewFloatArrayLen(tsdb.DefaultMaxPointsPerBlock)
c.tsm.buf = tsdb.NewFloatArrayLen(MaxPointsPerBlock)
return c
}
@ -316,9 +316,9 @@ type integerArrayAscendingCursor struct {
func newIntegerArrayAscendingCursor() *integerArrayAscendingCursor {
c := &integerArrayAscendingCursor{
res: tsdb.NewIntegerArrayLen(tsdb.DefaultMaxPointsPerBlock),
res: tsdb.NewIntegerArrayLen(MaxPointsPerBlock),
}
c.tsm.buf = tsdb.NewIntegerArrayLen(tsdb.DefaultMaxPointsPerBlock)
c.tsm.buf = tsdb.NewIntegerArrayLen(MaxPointsPerBlock)
return c
}
@ -452,9 +452,9 @@ type integerArrayDescendingCursor struct {
func newIntegerArrayDescendingCursor() *integerArrayDescendingCursor {
c := &integerArrayDescendingCursor{
res: tsdb.NewIntegerArrayLen(tsdb.DefaultMaxPointsPerBlock),
res: tsdb.NewIntegerArrayLen(MaxPointsPerBlock),
}
c.tsm.buf = tsdb.NewIntegerArrayLen(tsdb.DefaultMaxPointsPerBlock)
c.tsm.buf = tsdb.NewIntegerArrayLen(MaxPointsPerBlock)
return c
}
@ -599,9 +599,9 @@ type unsignedArrayAscendingCursor struct {
func newUnsignedArrayAscendingCursor() *unsignedArrayAscendingCursor {
c := &unsignedArrayAscendingCursor{
res: tsdb.NewUnsignedArrayLen(tsdb.DefaultMaxPointsPerBlock),
res: tsdb.NewUnsignedArrayLen(MaxPointsPerBlock),
}
c.tsm.buf = tsdb.NewUnsignedArrayLen(tsdb.DefaultMaxPointsPerBlock)
c.tsm.buf = tsdb.NewUnsignedArrayLen(MaxPointsPerBlock)
return c
}
@ -735,9 +735,9 @@ type unsignedArrayDescendingCursor struct {
func newUnsignedArrayDescendingCursor() *unsignedArrayDescendingCursor {
c := &unsignedArrayDescendingCursor{
res: tsdb.NewUnsignedArrayLen(tsdb.DefaultMaxPointsPerBlock),
res: tsdb.NewUnsignedArrayLen(MaxPointsPerBlock),
}
c.tsm.buf = tsdb.NewUnsignedArrayLen(tsdb.DefaultMaxPointsPerBlock)
c.tsm.buf = tsdb.NewUnsignedArrayLen(MaxPointsPerBlock)
return c
}
@ -882,9 +882,9 @@ type stringArrayAscendingCursor struct {
func newStringArrayAscendingCursor() *stringArrayAscendingCursor {
c := &stringArrayAscendingCursor{
res: tsdb.NewStringArrayLen(tsdb.DefaultMaxPointsPerBlock),
res: tsdb.NewStringArrayLen(MaxPointsPerBlock),
}
c.tsm.buf = tsdb.NewStringArrayLen(tsdb.DefaultMaxPointsPerBlock)
c.tsm.buf = tsdb.NewStringArrayLen(MaxPointsPerBlock)
return c
}
@ -1018,9 +1018,9 @@ type stringArrayDescendingCursor struct {
func newStringArrayDescendingCursor() *stringArrayDescendingCursor {
c := &stringArrayDescendingCursor{
res: tsdb.NewStringArrayLen(tsdb.DefaultMaxPointsPerBlock),
res: tsdb.NewStringArrayLen(MaxPointsPerBlock),
}
c.tsm.buf = tsdb.NewStringArrayLen(tsdb.DefaultMaxPointsPerBlock)
c.tsm.buf = tsdb.NewStringArrayLen(MaxPointsPerBlock)
return c
}
@ -1165,9 +1165,9 @@ type booleanArrayAscendingCursor struct {
func newBooleanArrayAscendingCursor() *booleanArrayAscendingCursor {
c := &booleanArrayAscendingCursor{
res: tsdb.NewBooleanArrayLen(tsdb.DefaultMaxPointsPerBlock),
res: tsdb.NewBooleanArrayLen(MaxPointsPerBlock),
}
c.tsm.buf = tsdb.NewBooleanArrayLen(tsdb.DefaultMaxPointsPerBlock)
c.tsm.buf = tsdb.NewBooleanArrayLen(MaxPointsPerBlock)
return c
}
@ -1301,9 +1301,9 @@ type booleanArrayDescendingCursor struct {
func newBooleanArrayDescendingCursor() *booleanArrayDescendingCursor {
c := &booleanArrayDescendingCursor{
res: tsdb.NewBooleanArrayLen(tsdb.DefaultMaxPointsPerBlock),
res: tsdb.NewBooleanArrayLen(MaxPointsPerBlock),
}
c.tsm.buf = tsdb.NewBooleanArrayLen(tsdb.DefaultMaxPointsPerBlock)
c.tsm.buf = tsdb.NewBooleanArrayLen(MaxPointsPerBlock)
return c
}

View File

@ -32,9 +32,9 @@ type {{$type}} struct {
func new{{$Type}}() *{{$type}} {
c := &{{$type}}{
res: tsdb.New{{.Name}}ArrayLen(tsdb.DefaultMaxPointsPerBlock),
res: tsdb.New{{.Name}}ArrayLen(MaxPointsPerBlock),
}
c.tsm.buf = tsdb.New{{.Name}}ArrayLen(tsdb.DefaultMaxPointsPerBlock)
c.tsm.buf = tsdb.New{{.Name}}ArrayLen(MaxPointsPerBlock)
return c
}
@ -171,9 +171,9 @@ type {{$type}} struct {
func new{{$Type}}() *{{$type}} {
c := &{{$type}}{
res: tsdb.New{{.Name}}ArrayLen(tsdb.DefaultMaxPointsPerBlock),
res: tsdb.New{{.Name}}ArrayLen(MaxPointsPerBlock),
}
c.tsm.buf = tsdb.New{{.Name}}ArrayLen(tsdb.DefaultMaxPointsPerBlock)
c.tsm.buf = tsdb.New{{.Name}}ArrayLen(MaxPointsPerBlock)
return c
}
@ -299,4 +299,4 @@ func (c *{{$type}}) nextTSM() {{$arrayType}} {
return c.tsm.values
}
{{end}}
{{end}}

View File

@ -341,7 +341,7 @@ func (c *DefaultPlanner) PlanOptimize() []CompactionGroup {
cur := generations[i]
// Skip the file if it's over the max size and contains a full block and it does not have any tombstones
if cur.count() > 2 && cur.size() > uint64(maxTSMFileSize) && c.FileStore.BlockCount(cur.files[0].Path, 1) == tsdb.DefaultMaxPointsPerBlock && !cur.hasTombstones() {
if cur.count() > 2 && cur.size() > uint64(maxTSMFileSize) && c.FileStore.BlockCount(cur.files[0].Path, 1) == MaxPointsPerBlock && !cur.hasTombstones() {
continue
}
@ -426,7 +426,7 @@ func (c *DefaultPlanner) Plan(lastWrite time.Time) []CompactionGroup {
var skip bool
// Skip the file if it's over the max size and contains a full block and it does not have any tombstones
if len(generations) > 2 && group.size() > uint64(maxTSMFileSize) && c.FileStore.BlockCount(group.files[0].Path, 1) == tsdb.DefaultMaxPointsPerBlock && !group.hasTombstones() {
if len(generations) > 2 && group.size() > uint64(maxTSMFileSize) && c.FileStore.BlockCount(group.files[0].Path, 1) == MaxPointsPerBlock && !group.hasTombstones() {
skip = true
}
@ -502,7 +502,7 @@ func (c *DefaultPlanner) Plan(lastWrite time.Time) []CompactionGroup {
// Skip the file if it's over the max size and contains a full block or the generation is split
// over multiple files. In the latter case, that would mean the data in the file spilled over
// the 2GB limit.
if g.size() > uint64(maxTSMFileSize) && c.FileStore.BlockCount(g.files[0].Path, 1) == tsdb.DefaultMaxPointsPerBlock {
if g.size() > uint64(maxTSMFileSize) && c.FileStore.BlockCount(g.files[0].Path, 1) == MaxPointsPerBlock {
start = i + 1
}
@ -546,7 +546,7 @@ func (c *DefaultPlanner) Plan(lastWrite time.Time) []CompactionGroup {
}
// Skip the file if it's over the max size and it contains a full block
if gen.size() >= uint64(maxTSMFileSize) && c.FileStore.BlockCount(gen.files[0].Path, 1) == tsdb.DefaultMaxPointsPerBlock && !gen.hasTombstones() {
if gen.size() >= uint64(maxTSMFileSize) && c.FileStore.BlockCount(gen.files[0].Path, 1) == MaxPointsPerBlock && !gen.hasTombstones() {
startIndex++
continue
}
@ -846,7 +846,7 @@ func (c *Compactor) WriteSnapshot(cache *Cache) ([]string, error) {
resC := make(chan res, concurrency)
for i := 0; i < concurrency; i++ {
go func(sp *Cache) {
iter := NewCacheKeyIterator(sp, tsdb.DefaultMaxPointsPerBlock, intC)
iter := NewCacheKeyIterator(sp, MaxPointsPerBlock, intC)
files, err := c.writeNewFiles(c.FileStore.NextGeneration(), 0, nil, iter, throttle)
resC <- res{files: files, err: err}
@ -884,7 +884,7 @@ func (c *Compactor) WriteSnapshot(cache *Cache) ([]string, error) {
func (c *Compactor) compact(fast bool, tsmFiles []string) ([]string, error) {
size := c.Size
if size <= 0 {
size = tsdb.DefaultMaxPointsPerBlock
size = MaxPointsPerBlock
}
c.mu.RLock()
@ -1921,12 +1921,12 @@ func (c *cacheKeyIterator) encode() {
for i := 0; i < concurrency; i++ {
// Run one goroutine per CPU and encode a section of the key space concurrently
go func() {
tenc := getTimeEncoder(tsdb.DefaultMaxPointsPerBlock)
fenc := getFloatEncoder(tsdb.DefaultMaxPointsPerBlock)
benc := getBooleanEncoder(tsdb.DefaultMaxPointsPerBlock)
uenc := getUnsignedEncoder(tsdb.DefaultMaxPointsPerBlock)
senc := getStringEncoder(tsdb.DefaultMaxPointsPerBlock)
ienc := getIntegerEncoder(tsdb.DefaultMaxPointsPerBlock)
tenc := getTimeEncoder(MaxPointsPerBlock)
fenc := getFloatEncoder(MaxPointsPerBlock)
benc := getBooleanEncoder(MaxPointsPerBlock)
uenc := getUnsignedEncoder(MaxPointsPerBlock)
senc := getStringEncoder(MaxPointsPerBlock)
ienc := getIntegerEncoder(MaxPointsPerBlock)
defer putTimeEncoder(tenc)
defer putFloatEncoder(fenc)

View File

@ -11,7 +11,6 @@ import (
"time"
"github.com/google/go-cmp/cmp"
"github.com/influxdata/platform/tsdb"
"github.com/influxdata/platform/tsdb/tsm1"
)
@ -1479,7 +1478,7 @@ func TestDefaultPlanner_Plan_Min(t *testing.T) {
},
}
},
}, tsdb.DefaultCompactFullWriteColdDuration,
}, tsm1.DefaultCompactFullWriteColdDuration,
)
tsm := cp.Plan(time.Now())
@ -1527,7 +1526,7 @@ func TestDefaultPlanner_Plan_CombineSequence(t *testing.T) {
PathsFn: func() []tsm1.FileStat {
return data
},
}, tsdb.DefaultCompactFullWriteColdDuration,
}, tsm1.DefaultCompactFullWriteColdDuration,
)
expFiles := []tsm1.FileStat{data[0], data[1], data[2], data[3]}
@ -1588,7 +1587,7 @@ func TestDefaultPlanner_Plan_MultipleGroups(t *testing.T) {
PathsFn: func() []tsm1.FileStat {
return data
},
}, tsdb.DefaultCompactFullWriteColdDuration)
}, tsm1.DefaultCompactFullWriteColdDuration)
expFiles := []tsm1.FileStat{data[0], data[1], data[2], data[3],
data[4], data[5], data[6], data[7]}
@ -1678,7 +1677,7 @@ func TestDefaultPlanner_PlanLevel_SmallestCompactionStep(t *testing.T) {
PathsFn: func() []tsm1.FileStat {
return data
},
}, tsdb.DefaultCompactFullWriteColdDuration,
}, tsm1.DefaultCompactFullWriteColdDuration,
)
expFiles := []tsm1.FileStat{data[4], data[5], data[6], data[7], data[8], data[9], data[10], data[11]}
@ -1731,7 +1730,7 @@ func TestDefaultPlanner_PlanLevel_SplitFile(t *testing.T) {
PathsFn: func() []tsm1.FileStat {
return data
},
}, tsdb.DefaultCompactFullWriteColdDuration,
}, tsm1.DefaultCompactFullWriteColdDuration,
)
expFiles := []tsm1.FileStat{data[0], data[1], data[2], data[3], data[4]}
@ -1784,7 +1783,7 @@ func TestDefaultPlanner_PlanLevel_IsolatedHighLevel(t *testing.T) {
PathsFn: func() []tsm1.FileStat {
return data
},
}, tsdb.DefaultCompactFullWriteColdDuration,
}, tsm1.DefaultCompactFullWriteColdDuration,
)
expFiles := []tsm1.FileStat{}
@ -1827,7 +1826,7 @@ func TestDefaultPlanner_PlanLevel3_MinFiles(t *testing.T) {
PathsFn: func() []tsm1.FileStat {
return data
},
}, tsdb.DefaultCompactFullWriteColdDuration,
}, tsm1.DefaultCompactFullWriteColdDuration,
)
expFiles := []tsm1.FileStat{}
@ -1859,7 +1858,7 @@ func TestDefaultPlanner_PlanLevel2_MinFiles(t *testing.T) {
PathsFn: func() []tsm1.FileStat {
return data
},
}, tsdb.DefaultCompactFullWriteColdDuration,
}, tsm1.DefaultCompactFullWriteColdDuration,
)
expFiles := []tsm1.FileStat{}
@ -1903,7 +1902,7 @@ func TestDefaultPlanner_PlanLevel_Tombstone(t *testing.T) {
PathsFn: func() []tsm1.FileStat {
return data
},
}, tsdb.DefaultCompactFullWriteColdDuration,
}, tsm1.DefaultCompactFullWriteColdDuration,
)
expFiles := []tsm1.FileStat{data[0], data[1]}
@ -1960,7 +1959,7 @@ func TestDefaultPlanner_PlanLevel_Multiple(t *testing.T) {
PathsFn: func() []tsm1.FileStat {
return data
},
}, tsdb.DefaultCompactFullWriteColdDuration,
}, tsm1.DefaultCompactFullWriteColdDuration,
)
expFiles1 := []tsm1.FileStat{data[0], data[1], data[2], data[3], data[4], data[5], data[6], data[7]}
@ -2050,7 +2049,7 @@ func TestDefaultPlanner_PlanLevel_InUse(t *testing.T) {
PathsFn: func() []tsm1.FileStat {
return data
},
}, tsdb.DefaultCompactFullWriteColdDuration,
}, tsm1.DefaultCompactFullWriteColdDuration,
)
expFiles1 := data[0:8]
@ -2112,7 +2111,7 @@ func TestDefaultPlanner_PlanOptimize_NoLevel4(t *testing.T) {
PathsFn: func() []tsm1.FileStat {
return data
},
}, tsdb.DefaultCompactFullWriteColdDuration,
}, tsm1.DefaultCompactFullWriteColdDuration,
)
expFiles := []tsm1.FileStat{}
@ -2159,7 +2158,7 @@ func TestDefaultPlanner_PlanOptimize_Level4(t *testing.T) {
PathsFn: func() []tsm1.FileStat {
return data
},
}, tsdb.DefaultCompactFullWriteColdDuration,
}, tsm1.DefaultCompactFullWriteColdDuration,
)
expFiles1 := []tsm1.FileStat{data[0], data[1], data[2], data[3], data[4], data[5]}
@ -2228,7 +2227,7 @@ func TestDefaultPlanner_PlanOptimize_Multiple(t *testing.T) {
PathsFn: func() []tsm1.FileStat {
return data
},
}, tsdb.DefaultCompactFullWriteColdDuration,
}, tsm1.DefaultCompactFullWriteColdDuration,
)
expFiles1 := []tsm1.FileStat{data[0], data[1], data[2], data[3]}
@ -2281,7 +2280,7 @@ func TestDefaultPlanner_PlanOptimize_Optimized(t *testing.T) {
PathsFn: func() []tsm1.FileStat {
return data
},
}, tsdb.DefaultCompactFullWriteColdDuration,
}, tsm1.DefaultCompactFullWriteColdDuration,
)
expFiles := []tsm1.FileStat{}
@ -2313,7 +2312,7 @@ func TestDefaultPlanner_PlanOptimize_Tombstones(t *testing.T) {
PathsFn: func() []tsm1.FileStat {
return data
},
}, tsdb.DefaultCompactFullWriteColdDuration,
}, tsm1.DefaultCompactFullWriteColdDuration,
)
expFiles := []tsm1.FileStat{data[0], data[1], data[2]}
@ -2400,7 +2399,7 @@ func TestDefaultPlanner_Plan_SkipMaxSizeFiles(t *testing.T) {
PathsFn: func() []tsm1.FileStat {
return data
},
}, tsdb.DefaultCompactFullWriteColdDuration,
}, tsm1.DefaultCompactFullWriteColdDuration,
)
tsm := cp.Plan(time.Now())
@ -2659,7 +2658,7 @@ func TestDefaultPlanner_Plan_CompactsMiddleSteps(t *testing.T) {
PathsFn: func() []tsm1.FileStat {
return data
},
}, tsdb.DefaultCompactFullWriteColdDuration,
}, tsm1.DefaultCompactFullWriteColdDuration,
)
expFiles := []tsm1.FileStat{data[0], data[1], data[2], data[3]}
@ -2702,7 +2701,7 @@ func TestDefaultPlanner_Plan_LargeGeneration(t *testing.T) {
},
}
},
}, tsdb.DefaultCompactFullWriteColdDuration,
}, tsm1.DefaultCompactFullWriteColdDuration,
)
tsm := cp.Plan(time.Now())
@ -2770,7 +2769,7 @@ func TestDefaultPlanner_Plan_ForceFull(t *testing.T) {
},
}
},
}, tsdb.DefaultCompactFullWriteColdDuration,
}, tsm1.DefaultCompactFullWriteColdDuration,
)
tsm := cp.PlanLevel(1)

127
tsdb/tsm1/config.go Normal file
View File

@ -0,0 +1,127 @@
package tsm1
import (
"runtime"
"time"
"github.com/influxdata/platform/toml"
)
var DefaultMaxConcurrentOpens = runtime.GOMAXPROCS(0)
const (
DefaultMADVWillNeed = false
)
// Config contains all of the configuration necessary to run a tsm1 engine.
type Config struct {
// MacConcurrentOpens controls the concurrency of opening tsm files during
// engine opening.
MaxConcurrentOpens int `toml:"max-concurrent-opens"`
// MADVWillNeed controls whether we hint to the kernel that we intend to page
// in mmap'd sections of TSM files. This setting defaults to off, as it has
// been found to be problematic in some cases. It may help users who have
// slow disks.
MADVWillNeed bool `toml:"use-madv-willneed"`
Compaction CompactionConfig `toml:"compaction"`
Cache CacheConfig `toml:"cache"`
}
// NewConfig constructs a Config with the default values.
func NewConfig() Config {
return Config{
MaxConcurrentOpens: DefaultMaxConcurrentOpens,
MADVWillNeed: DefaultMADVWillNeed,
Cache: CacheConfig{
MaxMemorySize: toml.Size(DefaultCacheMaxMemorySize),
SnapshotMemorySize: toml.Size(DefaultCacheSnapshotMemorySize),
SnapshotWriteColdDuration: toml.Duration(DefaultCacheSnapshotWriteColdDuration),
},
Compaction: CompactionConfig{
FullWriteColdDuration: toml.Duration(DefaultCompactFullWriteColdDuration),
Throughput: toml.Size(DefaultCompactThroughput),
ThroughputBurst: toml.Size(DefaultCompactThroughputBurst),
MaxConcurrent: DefaultCompactMaxConcurrent,
},
}
}
const (
DefaultCompactFullWriteColdDuration = time.Duration(4 * time.Hour)
DefaultCompactThroughput = 48 * 1024 * 1024
DefaultCompactThroughputBurst = 48 * 1024 * 1024
DefaultCompactMaxConcurrent = 0
)
// CompactionConfing holds all of the configuration for compactions. Eventually we want
// to move this out of tsm1 so that it can be scheduled more intelligently.
type CompactionConfig struct {
// FullWriteColdDuration is the duration at which the engine will compact all TSM
// files in a shard if it hasn't received a write or delete
FullWriteColdDuration toml.Duration `toml:"full-write-cold-duration"`
// Throughput is the rate limit in bytes per second that we will allow TSM compactions
// to write to disk. Not that short bursts are allowed to happen at a possibly larger
// value, set by CompactThroughputBurst. A value of 0 here will disable compaction rate
// limiting
Throughput toml.Size `toml:"throughput"`
// ThroughputBurst is the rate limit in bytes per second that we will allow TSM compactions
// to write to disk. If this is not set, the burst value will be set to equal the normal
// throughput
ThroughputBurst toml.Size `toml:"throughput-burst"`
// MaxConcurrent is the maximum number of concurrent full and level compactions that can
// run at one time. A value of 0 results in 50% of runtime.GOMAXPROCS(0) used at runtime.
MaxConcurrent int `toml:"max-concurrent"`
}
const (
DefaultCacheMaxMemorySize = 1024 * 1024 * 1024 // 1GB
DefaultCacheSnapshotMemorySize = 25 * 1024 * 1024 // 25MB
DefaultCacheSnapshotWriteColdDuration = time.Duration(10 * time.Minute)
)
// CacheConfig holds all of the configuration for the in memory cache of values that
// are waiting to be snapshot.
type CacheConfig struct {
// MaxMemorySize is the maximum size a shard's cache can reach before it starts
// rejecting writes.
MaxMemorySize toml.Size `toml:"max-memory-size"`
// SnapshotMemorySize is the size at which the engine will snapshot the cache and
// write it to a TSM file, freeing up memory
SnapshotMemorySize toml.Size `toml:"snapshot-memory-size"`
// SnapshotWriteColdDuration is the length of time at which the engine will snapshot
// the cache and write it to a new TSM file if the shard hasn't received writes or
// deletes
SnapshotWriteColdDuration toml.Duration `toml:"snapshot-write-cold-duration"`
}
const (
DefaultWALEnabled = true
DefaultWALFsyncDelay = time.Duration(0)
)
// WALConfig holds all of the configuration about the WAL.
type WALConfig struct {
// Enabled controls if the WAL is enabled.
Enabled bool `toml:"enabled"`
// WALFsyncDelay is the amount of time that a write will wait before fsyncing. A
// duration greater than 0 can be used to batch up multiple fsync calls. This is
// useful for slower disks or when WAL write contention is seen. A value of 0 fsyncs
// every write to the WAL.
FsyncDelay toml.Duration `toml:"fsync-delay"`
}
func NewWALConfig() WALConfig {
return WALConfig{
Enabled: DefaultWALEnabled,
FsyncDelay: toml.Duration(DefaultWALFsyncDelay),
}
}

View File

@ -45,7 +45,7 @@ func init() {
vals = vals[:0]
// Check one out to force the allocation now and hold onto it
for i := 0; i < runtime.NumCPU(); i++ {
v := p.Get(tsdb.DefaultMaxPointsPerBlock)
v := p.Get(MaxPointsPerBlock)
vals = append(vals, v)
}
// Add them all back

View File

@ -1,53 +0,0 @@
// Generated by tmpl
// https://github.com/benbjohnson/tmpl
//
// DO NOT EDIT!
// Source: engine.gen.go.tmpl
package tsm1
import (
"context"
"github.com/influxdata/influxdb/query"
)
// buildFloatCursor creates a cursor for a float field.
func (e *Engine) buildFloatCursor(ctx context.Context, measurement, seriesKey, field string, opt query.IteratorOptions) floatCursor {
key := SeriesFieldKeyBytes(seriesKey, field)
cacheValues := e.Cache.Values(key)
keyCursor := e.KeyCursor(ctx, key, opt.SeekTime(), opt.Ascending)
return newFloatCursor(opt.SeekTime(), opt.Ascending, cacheValues, keyCursor)
}
// buildIntegerCursor creates a cursor for a integer field.
func (e *Engine) buildIntegerCursor(ctx context.Context, measurement, seriesKey, field string, opt query.IteratorOptions) integerCursor {
key := SeriesFieldKeyBytes(seriesKey, field)
cacheValues := e.Cache.Values(key)
keyCursor := e.KeyCursor(ctx, key, opt.SeekTime(), opt.Ascending)
return newIntegerCursor(opt.SeekTime(), opt.Ascending, cacheValues, keyCursor)
}
// buildUnsignedCursor creates a cursor for a unsigned field.
func (e *Engine) buildUnsignedCursor(ctx context.Context, measurement, seriesKey, field string, opt query.IteratorOptions) unsignedCursor {
key := SeriesFieldKeyBytes(seriesKey, field)
cacheValues := e.Cache.Values(key)
keyCursor := e.KeyCursor(ctx, key, opt.SeekTime(), opt.Ascending)
return newUnsignedCursor(opt.SeekTime(), opt.Ascending, cacheValues, keyCursor)
}
// buildStringCursor creates a cursor for a string field.
func (e *Engine) buildStringCursor(ctx context.Context, measurement, seriesKey, field string, opt query.IteratorOptions) stringCursor {
key := SeriesFieldKeyBytes(seriesKey, field)
cacheValues := e.Cache.Values(key)
keyCursor := e.KeyCursor(ctx, key, opt.SeekTime(), opt.Ascending)
return newStringCursor(opt.SeekTime(), opt.Ascending, cacheValues, keyCursor)
}
// buildBooleanCursor creates a cursor for a boolean field.
func (e *Engine) buildBooleanCursor(ctx context.Context, measurement, seriesKey, field string, opt query.IteratorOptions) booleanCursor {
key := SeriesFieldKeyBytes(seriesKey, field)
cacheValues := e.Cache.Values(key)
keyCursor := e.KeyCursor(ctx, key, opt.SeekTime(), opt.Ascending)
return newBooleanCursor(opt.SeekTime(), opt.Ascending, cacheValues, keyCursor)
}

View File

@ -1,19 +0,0 @@
package tsm1
import (
"context"
"github.com/influxdata/influxdb/query"
)
{{range .}}
// build{{.Name}}Cursor creates a cursor for a {{.name}} field.
func (e *Engine) build{{.Name}}Cursor(ctx context.Context, measurement, seriesKey, field string, opt query.IteratorOptions) {{.name}}Cursor {
key := SeriesFieldKeyBytes(seriesKey, field)
cacheValues := e.Cache.Values(key)
keyCursor := e.KeyCursor(ctx, key, opt.SeekTime(), opt.Ascending)
return new{{.Name}}Cursor(opt.SeekTime(), opt.Ascending, cacheValues, keyCursor)
}
{{end}}

View File

@ -28,17 +28,14 @@ import (
"go.uber.org/zap"
)
//go:generate env GO111MODULE=on go run github.com/benbjohnson/tmpl -data=@iterator.gen.go.tmpldata iterator.gen.go.tmpl engine.gen.go.tmpl array_cursor.gen.go.tmpl array_cursor_iterator.gen.go.tmpl
//go:generate env GO111MODULE=on go run github.com/benbjohnson/tmpl -data=@iterator.gen.go.tmpldata iterator.gen.go.tmpl array_cursor.gen.go.tmpl array_cursor_iterator.gen.go.tmpl
//go:generate env GO111MODULE=on go run github.com/influxdata/platform/tools/tmpl -i -data=file_store.gen.go.tmpldata file_store.gen.go.tmpl=file_store.gen.go
//go:generate env GO111MODULE=on go run github.com/influxdata/platform/tools/tmpl -i -d isArray=y -data=file_store.gen.go.tmpldata file_store.gen.go.tmpl=file_store_array.gen.go
//go:generate env GO111MODULE=on go run github.com/benbjohnson/tmpl -data=@encoding.gen.go.tmpldata encoding.gen.go.tmpl
//go:generate env GO111MODULE=on go run github.com/benbjohnson/tmpl -data=@compact.gen.go.tmpldata compact.gen.go.tmpl
//go:generate env GO111MODULE=on go run github.com/benbjohnson/tmpl -data=@reader.gen.go.tmpldata reader.gen.go.tmpl
var (
// Ensure Engine implements the interface.
_ tsdb.Engine = &Engine{}
// Static objects to prevent small allocs.
var ( // Static objects to prevent small allocs.
keyFieldSeparatorBytes = []byte(keyFieldSeparator)
emptyBytes = []byte{}
)
@ -68,6 +65,9 @@ const (
// deleteFlushThreshold is the size in bytes of a batch of series keys to delete.
deleteFlushThreshold = 50 * 1024 * 1024
// MaxPointsPerBlock is the maximum number of points in an encoded block in a TSM file
MaxPointsPerBlock = 1000
)
// Statistics gathered by the engine.
@ -108,6 +108,32 @@ const (
statTSMFullCompactionQueue = "tsmFullCompactionQueue"
)
// An EngineOption is a functional option for changing the configuration of
// an Engine.
type EngineOption func(i *Engine)
// WithWAL sets the WAL for the Engine
var WithWAL = func(wal Log) EngineOption {
return func(e *Engine) {
e.WAL = wal
}
}
// WithTraceLogging sets if trace logging is enabled for the engine.
var WithTraceLogging = func(logging bool) EngineOption {
return func(e *Engine) {
e.FileStore.enableTraceLogging(logging)
}
}
// WithCompactionPlanner sets the compaction planner for the engine.
var WithCompactionPlanner = func(planner CompactionPlanner) EngineOption {
return func(e *Engine) {
planner.SetFileStore(e.FileStore)
e.CompactionPlan = planner
}
}
// Engine represents a storage engine with compressed blocks.
type Engine struct {
mu sync.RWMutex
@ -129,7 +155,6 @@ type Engine struct {
snapDone chan struct{} // channel to signal snapshot compactions to stop
snapWG *sync.WaitGroup // waitgroup for running snapshot compactions
id uint64
path string
sfile *tsdb.SeriesFile
logger *zap.Logger // Logger to be used for important messages
@ -168,66 +193,50 @@ type Engine struct {
}
// NewEngine returns a new instance of Engine.
func NewEngine(id uint64, idx *tsi1.Index, path string, walPath string, sfile *tsdb.SeriesFile, opt tsdb.EngineOptions) tsdb.Engine {
func NewEngine(path string, idx *tsi1.Index, config Config, options ...EngineOption) *Engine {
fs := NewFileStore(path)
fs.openLimiter = opt.OpenLimiter
if opt.FileStoreObserver != nil {
fs.WithObserver(opt.FileStoreObserver)
}
fs.tsmMMAPWillNeed = opt.Config.TSMWillNeed
fs.openLimiter = limiter.NewFixed(config.MaxConcurrentOpens)
fs.tsmMMAPWillNeed = config.MADVWillNeed
cache := NewCache(uint64(opt.Config.CacheMaxMemorySize))
cache := NewCache(uint64(config.Cache.MaxMemorySize))
c := NewCompactor()
c.Dir = path
c.FileStore = fs
c.RateLimit = opt.CompactionThroughputLimiter
var planner CompactionPlanner = NewDefaultPlanner(fs, time.Duration(opt.Config.CompactFullWriteColdDuration))
if opt.CompactionPlannerCreator != nil {
planner = opt.CompactionPlannerCreator(opt.Config).(CompactionPlanner)
planner.SetFileStore(fs)
}
c.RateLimit = limiter.NewRate(
int(config.Compaction.Throughput),
int(config.Compaction.ThroughputBurst))
logger := zap.NewNop()
stats := &EngineStatistics{}
e := &Engine{
id: id,
path: path,
index: idx,
sfile: sfile,
logger: logger,
traceLogger: logger,
traceLogging: opt.Config.TraceLoggingEnabled,
path: path,
index: idx,
sfile: idx.SeriesFile(),
logger: logger,
traceLogger: logger,
WAL: NopWAL{},
Cache: cache,
FileStore: fs,
Compactor: c,
CompactionPlan: planner,
FileStore: fs,
Compactor: c,
CompactionPlan: NewDefaultPlanner(fs,
time.Duration(config.Compaction.FullWriteColdDuration)),
CacheFlushMemorySizeThreshold: uint64(opt.Config.CacheSnapshotMemorySize),
CacheFlushWriteColdDuration: time.Duration(opt.Config.CacheSnapshotWriteColdDuration),
CacheFlushMemorySizeThreshold: uint64(config.Cache.SnapshotMemorySize),
CacheFlushWriteColdDuration: time.Duration(config.Cache.SnapshotWriteColdDuration),
enableCompactionsOnOpen: true,
formatFileName: DefaultFormatFileName,
stats: stats,
compactionLimiter: opt.CompactionLimiter,
scheduler: newScheduler(stats, opt.CompactionLimiter.Capacity()),
compactionLimiter: limiter.NewFixed(config.Compaction.MaxConcurrent),
scheduler: newScheduler(stats, config.Compaction.MaxConcurrent),
}
if opt.WALEnabled {
wal := NewWAL(walPath)
wal.syncDelay = time.Duration(opt.Config.WALFsyncDelay)
e.WAL = wal
for _, option := range options {
option(e)
}
if e.traceLogging {
fs.enableTraceLogging(true)
if wal, ok := e.WAL.(*WAL); ok {
wal.enableTraceLogging(true)
}
}
return e
}
@ -241,6 +250,15 @@ func (e *Engine) WithParseFileNameFunc(parseFileNameFunc ParseFileNameFunc) {
e.Compactor.WithParseFileNameFunc(parseFileNameFunc)
}
func (e *Engine) WithFileStoreObserver(obs FileStoreObserver) {
e.FileStore.WithObserver(obs)
}
func (e *Engine) WithCompactionPlanner(planner CompactionPlanner) {
planner.SetFileStore(e.FileStore)
e.CompactionPlan = planner
}
// SetEnabled sets whether the engine is enabled.
func (e *Engine) SetEnabled(enabled bool) {
e.enableCompactionsOnOpen = enabled

View File

@ -6,7 +6,6 @@ import (
"io/ioutil"
"math"
"os"
"path"
"path/filepath"
"runtime"
"strings"
@ -717,21 +716,15 @@ func TestEngine_SnapshotsDisabled(t *testing.T) {
// Generate temporary file.
dir, _ := ioutil.TempDir("", "tsm")
walPath := filepath.Join(dir, "wal")
os.MkdirAll(walPath, 0777)
defer os.RemoveAll(dir)
// Create a tsm1 engine.
db := path.Base(dir)
opt := tsdb.NewEngineOptions()
idx := MustOpenIndex(1, db, filepath.Join(dir, "index"), tsdb.NewSeriesIDSet(), sfile.SeriesFile, opt)
idx := MustOpenIndex(filepath.Join(dir, "index"), tsdb.NewSeriesIDSet(), sfile.SeriesFile)
defer idx.Close()
e := tsm1.NewEngine(1, idx, dir, walPath, sfile.SeriesFile, opt).(*tsm1.Engine)
// mock the planner so compactions don't run during the test
e.CompactionPlan = &mockPlanner{}
config := tsm1.NewConfig()
e := tsm1.NewEngine(filepath.Join(dir, "data"), idx, config,
tsm1.WithCompactionPlanner(newMockPlanner()))
e.SetEnabled(false)
if err := e.Open(); err != nil {
@ -935,31 +928,19 @@ func NewEngine() (*Engine, error) {
panic(err)
}
db := "db0"
dbPath := filepath.Join(root, "data", db)
if err := os.MkdirAll(dbPath, os.ModePerm); err != nil {
return nil, err
}
// Setup series file.
sfile := tsdb.NewSeriesFile(filepath.Join(dbPath, tsdb.DefaultSeriesFileDirectory))
sfile := tsdb.NewSeriesFile(filepath.Join(root, "_series"))
sfile.Logger = logger.New(os.Stdout)
if err = sfile.Open(); err != nil {
return nil, err
}
opt := tsdb.NewEngineOptions()
idxPath := filepath.Join(root, "index")
idx := MustOpenIndex(idxPath, tsdb.NewSeriesIDSet(), sfile)
// Initialise series id sets. Need to do this as it's normally done at the
// store level.
seriesIDs := tsdb.NewSeriesIDSet()
opt.SeriesIDSets = seriesIDSets([]*tsdb.SeriesIDSet{seriesIDs})
idxPath := filepath.Join(dbPath, "index")
idx := MustOpenIndex(1, db, idxPath, seriesIDs, sfile, opt)
tsm1Engine := tsm1.NewEngine(1, idx, filepath.Join(root, "data"), filepath.Join(root, "wal"), sfile, opt).(*tsm1.Engine)
config := tsm1.NewConfig()
tsm1Engine := tsm1.NewEngine(filepath.Join(root, "data"), idx, config,
tsm1.WithCompactionPlanner(newMockPlanner()))
return &Engine{
Engine: tsm1Engine,
@ -1018,18 +999,13 @@ func (e *Engine) Reopen() error {
return err
}
db := path.Base(e.root)
opt := tsdb.NewEngineOptions()
// Re-initialise the series id set
seriesIDSet := tsdb.NewSeriesIDSet()
opt.SeriesIDSets = seriesIDSets([]*tsdb.SeriesIDSet{seriesIDSet})
// Re-open index.
e.index = MustOpenIndex(1, db, e.indexPath, seriesIDSet, e.sfile, opt)
e.index = MustOpenIndex(e.indexPath, tsdb.NewSeriesIDSet(), e.sfile)
// Re-initialize engine.
e.Engine = tsm1.NewEngine(1, e.index, filepath.Join(e.root, "data"), filepath.Join(e.root, "wal"), e.sfile, opt).(*tsm1.Engine)
config := tsm1.NewConfig()
e.Engine = tsm1.NewEngine(filepath.Join(e.root, "data"), e.index, config,
tsm1.WithCompactionPlanner(newMockPlanner()))
// Reopen engine
if err := e.Engine.Open(); err != nil {
@ -1096,8 +1072,8 @@ func (e *Engine) MustWriteSnapshot() {
}
}
func MustOpenIndex(id uint64, database, path string, seriesIDSet *tsdb.SeriesIDSet, sfile *tsdb.SeriesFile, options tsdb.EngineOptions) *tsi1.Index {
idx := tsi1.NewIndex(sfile, database, tsi1.NewConfig(), tsi1.WithPath(path))
func MustOpenIndex(path string, seriesIDSet *tsdb.SeriesIDSet, sfile *tsdb.SeriesFile) *tsi1.Index {
idx := tsi1.NewIndex(sfile, tsi1.NewConfig(), tsi1.WithPath(path))
if err := idx.Open(); err != nil {
panic(err)
}
@ -1149,6 +1125,10 @@ func MustParsePointString(buf string) models.Point { return MustParsePointsStrin
type mockPlanner struct{}
func newMockPlanner() tsm1.CompactionPlanner {
return &mockPlanner{}
}
func (m *mockPlanner) Plan(lastWrite time.Time) []tsm1.CompactionGroup { return nil }
func (m *mockPlanner) PlanLevel(level int) []tsm1.CompactionGroup { return nil }
func (m *mockPlanner) PlanOptimize() []tsm1.CompactionGroup { return nil }
@ -1183,12 +1163,3 @@ func (itr *seriesIterator) Next() (tsdb.SeriesElem, error) {
itr.keys = itr.keys[1:]
return s, nil
}
type seriesIDSets []*tsdb.SeriesIDSet
func (a seriesIDSets) ForEach(f func(ids *tsdb.SeriesIDSet)) error {
for _, v := range a {
f(v)
}
return nil
}

View File

@ -150,6 +150,16 @@ type TSMFile interface {
MeasurementStats() (MeasurementStats, error)
}
// FileStoreObserver is passed notifications before the file store adds or deletes files. In this way, it can
// be sure to observe every file that is added or removed even in the presence of process death.
type FileStoreObserver interface {
// FileFinishing is called before a file is renamed to it's final name.
FileFinishing(path string) error
// FileUnlinking is called before a file is unlinked.
FileUnlinking(path string) error
}
// Statistics gathered by the FileStore.
const (
statFileStoreBytes = "diskBytes"
@ -195,7 +205,7 @@ type FileStore struct {
parseFileName ParseFileNameFunc
obs tsdb.FileStoreObserver
obs FileStoreObserver
}
// FileStat holds information about a TSM file on disk.
@ -245,7 +255,10 @@ func NewFileStore(dir string) *FileStore {
}
// WithObserver sets the observer for the file store.
func (f *FileStore) WithObserver(obs tsdb.FileStoreObserver) {
func (f *FileStore) WithObserver(obs FileStoreObserver) {
if obs == nil {
obs = noFileStoreObserver{}
}
f.obs = obs
}

View File

@ -15,7 +15,6 @@ import (
"github.com/influxdata/platform/pkg/bytesutil"
"github.com/influxdata/platform/pkg/file"
"github.com/influxdata/platform/tsdb"
)
// ErrFileInUse is returned when attempting to remove or close a TSM file that is still being used.
@ -252,7 +251,10 @@ func NewTSMReader(f *os.File, options ...tsmReaderOption) (*TSMReader, error) {
}
// WithObserver sets the observer for the TSM reader.
func (t *TSMReader) WithObserver(obs tsdb.FileStoreObserver) {
func (t *TSMReader) WithObserver(obs FileStoreObserver) {
if obs == nil {
obs = noFileStoreObserver{}
}
t.tombstoner.WithObserver(obs)
}

View File

@ -15,7 +15,6 @@ import (
"sync"
"github.com/influxdata/platform/pkg/file"
"github.com/influxdata/platform/tsdb"
)
const (
@ -55,7 +54,7 @@ type Tombstoner struct {
lastAppliedOffset int64
// Optional observer for when tombstone files are written.
obs tsdb.FileStoreObserver
obs FileStoreObserver
}
// NewTombstoner constructs a Tombstoner for the given path. FilterFn can be nil.
@ -78,7 +77,10 @@ type Tombstone struct {
}
// WithObserver sets a FileStoreObserver for when the tombstone file is written.
func (t *Tombstoner) WithObserver(obs tsdb.FileStoreObserver) {
func (t *Tombstoner) WithObserver(obs FileStoreObserver) {
if obs == nil {
obs = noFileStoreObserver{}
}
t.obs = obs
}

View File

@ -150,14 +150,19 @@ func NewWAL(path string) *WAL {
}
}
// enableTraceLogging must be called before the WAL is opened.
func (l *WAL) enableTraceLogging(enabled bool) {
// EnableTraceLogging must be called before the WAL is opened.
func (l *WAL) EnableTraceLogging(enabled bool) {
l.traceLogging = enabled
if enabled {
l.traceLogger = l.logger
}
}
// WithFsyncDelay sets the fsync delay and should be called before the WAL is opened.
func (l *WAL) WithFsyncDelay(delay time.Duration) {
l.syncDelay = delay
}
// WithLogger sets the WAL's logger.
func (l *WAL) WithLogger(log *zap.Logger) {
l.logger = log.With(zap.String("service", "wal"))