Merge pull request #1321 from influxdata/jmw-config-cleanup
Storage engine config cleanuppull/10616/head
commit
6b57c7ded0
|
@ -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 {
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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
|
||||
}
|
|
@ -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)
|
||||
}
|
||||
|
|
|
@ -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")
|
||||
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
|
|
112
tsdb/config.go
112
tsdb/config.go
|
@ -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
|
||||
}
|
||||
|
|
|
@ -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"
|
||||
)
|
128
tsdb/engine.go
128
tsdb/engine.go
|
@ -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
|
||||
}
|
|
@ -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
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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 }
|
||||
|
||||
|
|
|
@ -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()
|
||||
}
|
||||
|
|
|
@ -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)
|
||||
}
|
||||
|
|
|
@ -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
|
||||
}
|
||||
|
||||
|
|
|
@ -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}}
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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),
|
||||
}
|
||||
}
|
|
@ -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
|
||||
|
|
|
@ -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)
|
||||
}
|
|
@ -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}}
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
}
|
||||
|
|
|
@ -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
|
||||
}
|
||||
|
||||
|
|
|
@ -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)
|
||||
}
|
||||
|
||||
|
|
|
@ -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
|
||||
}
|
||||
|
||||
|
|
|
@ -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"))
|
||||
|
|
Loading…
Reference in New Issue