Skip to content

Commit

Permalink
storage/engine: add StorageConfig.MaxSize
Browse files Browse the repository at this point in the history
Move `RocksDBConfig.MaxSizeBytes` to `StorageConfig.MaxSize` so it can
be shared with Pebble.

Embeded `StorageConfig` in both `RocksDBConfig` and `PebbleConfig` so
simplify access to its fields.

Release note: None
  • Loading branch information
petermattis committed Oct 29, 2019
1 parent c3d602a commit 25b8acf
Show file tree
Hide file tree
Showing 5 changed files with 42 additions and 44 deletions.
3 changes: 3 additions & 0 deletions pkg/base/config.go
Original file line number Diff line number Diff line change
Expand Up @@ -659,6 +659,9 @@ type StorageConfig struct {
// Makes no sense for in-memory instances.
// TODO(hueypark): Implement this for pebble.
MustExist bool
// MaxSize is used for calculating free space and making rebalancing
// decisions. Zero indicates that there is no maximum size.
MaxSize int64
// Settings instance for cluster-wide knobs.
Settings *cluster.Settings
// UseFileRegistry is true if the file registry is needed (eg: encryption-at-rest).
Expand Down
2 changes: 1 addition & 1 deletion pkg/server/config.go
Original file line number Diff line number Diff line change
Expand Up @@ -498,6 +498,7 @@ func (cfg *Config) CreateEngines(ctx context.Context) (Engines, error) {
storageConfig := base.StorageConfig{
Attrs: spec.Attributes,
Dir: spec.Path,
MaxSize: sizeInBytes,
Settings: cfg.Settings,
UseFileRegistry: spec.UseFileRegistry,
ExtraOptions: spec.ExtraOptions,
Expand All @@ -515,7 +516,6 @@ func (cfg *Config) CreateEngines(ctx context.Context) (Engines, error) {
} else {
rocksDBConfig := engine.RocksDBConfig{
StorageConfig: storageConfig,
MaxSizeBytes: sizeInBytes,
MaxOpenFiles: openFileLimitPerStore,
WarnLargeBatchThreshold: 500 * time.Millisecond,
RocksDBOptions: spec.RocksDBOptions,
Expand Down
14 changes: 7 additions & 7 deletions pkg/storage/engine/pebble.go
Original file line number Diff line number Diff line change
Expand Up @@ -208,7 +208,7 @@ func DefaultPebbleOptions() *pebble.Options {
// a new Pebble instance.
type PebbleConfig struct {
// StorageConfig contains storage configs for all storage engines.
StorageConfig base.StorageConfig
base.StorageConfig
// Pebble specific options.
Opts *pebble.Options
}
Expand All @@ -219,6 +219,7 @@ type Pebble struct {

closed bool
path string
maxSize int64
attrs roachpb.Attributes
settings *cluster.Settings

Expand All @@ -242,9 +243,10 @@ func NewPebble(cfg PebbleConfig) (*Pebble, error) {

return &Pebble{
db: db,
path: cfg.StorageConfig.Dir,
attrs: cfg.StorageConfig.Attrs,
settings: cfg.StorageConfig.Settings,
path: cfg.Dir,
maxSize: cfg.MaxSize,
attrs: cfg.Attrs,
settings: cfg.Settings,
fs: cfg.Opts.FS,
}, nil
}
Expand Down Expand Up @@ -403,9 +405,7 @@ func (p *Pebble) Attrs() roachpb.Attributes {

// Capacity implements the Engine interface.
func (p *Pebble) Capacity() (roachpb.StoreCapacity, error) {
// Pebble doesn't have a capacity limiting parameter, so pass 0 for
// maxSizeBytes to denote no limit.
return computeCapacity(p.path, 0)
return computeCapacity(p.path, p.maxSize)
}

// Flush implements the Engine interface.
Expand Down
57 changes: 26 additions & 31 deletions pkg/storage/engine/rocksdb.go
Original file line number Diff line number Diff line change
Expand Up @@ -460,12 +460,9 @@ func (c RocksDBCache) Release() {
// up a new RocksDB instance.
type RocksDBConfig struct {
// StorageConfig contains storage configs for all storage engines.
StorageConfig base.StorageConfig
base.StorageConfig
// ReadOnly will open the database in read only mode if set to true.
ReadOnly bool
// MaxSizeBytes is used for calculating free space and making rebalancing
// decisions. Zero indicates that there is no maximum size.
MaxSizeBytes int64
// MaxOpenFiles controls the maximum number of file descriptors RocksDB
// creates. If MaxOpenFiles is zero, this is set to DefaultMaxOpenFiles.
MaxOpenFiles uint64
Expand Down Expand Up @@ -522,7 +519,7 @@ func SetRocksDBOpenHook(fn unsafe.Pointer) {
// The caller must call the engine's Close method when the engine is no longer
// needed.
func NewRocksDB(cfg RocksDBConfig, cache RocksDBCache) (*RocksDB, error) {
if cfg.StorageConfig.Dir == "" {
if cfg.Dir == "" {
return nil, errors.New("dir must be non-empty")
}

Expand All @@ -531,7 +528,7 @@ func NewRocksDB(cfg RocksDBConfig, cache RocksDBCache) (*RocksDB, error) {
cache: cache.ref(),
}

if err := r.setAuxiliaryDir(filepath.Join(cfg.StorageConfig.Dir, "auxiliary")); err != nil {
if err := r.setAuxiliaryDir(filepath.Join(cfg.Dir, "auxiliary")); err != nil {
return nil, err
}

Expand All @@ -549,23 +546,21 @@ func newRocksDBInMem(attrs roachpb.Attributes, cacheSize int64) InMem {

// TODO(bdarnell): The hard-coded 512 MiB is wrong; see
// https://github.com/cockroachdb/cockroach/issues/16750
rdb, err := newMemRocksDB(attrs, cache, 512<<20 /* MaxSizeBytes: 512 MiB */)
rdb, err := newMemRocksDB(attrs, cache, 512<<20 /* MaxSize: 512 MiB */)
if err != nil {
panic(err)
}
db := InMem{RocksDB: rdb}
return db
}

func newMemRocksDB(
attrs roachpb.Attributes, cache RocksDBCache, MaxSizeBytes int64,
) (*RocksDB, error) {
func newMemRocksDB(attrs roachpb.Attributes, cache RocksDBCache, maxSize int64) (*RocksDB, error) {
r := &RocksDB{
cfg: RocksDBConfig{
StorageConfig: base.StorageConfig{
Attrs: attrs,
Attrs: attrs,
MaxSize: maxSize,
},
MaxSizeBytes: MaxSizeBytes,
},
// dir: empty dir == "mem" RocksDB instance.
cache: cache.ref(),
Expand All @@ -588,8 +583,8 @@ func newMemRocksDB(

// String formatter.
func (r *RocksDB) String() string {
dir := r.cfg.StorageConfig.Dir
if r.cfg.StorageConfig.Dir == "" {
dir := r.cfg.Dir
if r.cfg.Dir == "" {
dir = "<in-mem>"
}
attrs := r.Attrs().String()
Expand All @@ -601,12 +596,12 @@ func (r *RocksDB) String() string {

func (r *RocksDB) open() error {
var existingVersion, newVersion storageVersion
if len(r.cfg.StorageConfig.Dir) != 0 {
log.Infof(context.TODO(), "opening rocksdb instance at %q", r.cfg.StorageConfig.Dir)
if len(r.cfg.Dir) != 0 {
log.Infof(context.TODO(), "opening rocksdb instance at %q", r.cfg.Dir)

// Check the version number.
var err error
if existingVersion, err = getVersion(r.cfg.StorageConfig.Dir); err != nil {
if existingVersion, err = getVersion(r.cfg.Dir); err != nil {
return err
}
if existingVersion < versionMinimum || existingVersion > versionCurrent {
Expand All @@ -626,7 +621,7 @@ func (r *RocksDB) open() error {
}

// Using the file registry forces the latest version. We can't downgrade!
if r.cfg.StorageConfig.UseFileRegistry {
if r.cfg.UseFileRegistry {
newVersion = versionCurrent
}
} else {
Expand All @@ -643,24 +638,24 @@ func (r *RocksDB) open() error {
maxOpenFiles = r.cfg.MaxOpenFiles
}

status := C.DBOpen(&r.rdb, goToCSlice([]byte(r.cfg.StorageConfig.Dir)),
status := C.DBOpen(&r.rdb, goToCSlice([]byte(r.cfg.Dir)),
C.DBOptions{
cache: r.cache.cache,
num_cpu: C.int(rocksdbConcurrency),
max_open_files: C.int(maxOpenFiles),
use_file_registry: C.bool(newVersion == versionCurrent),
must_exist: C.bool(r.cfg.StorageConfig.MustExist),
must_exist: C.bool(r.cfg.MustExist),
read_only: C.bool(r.cfg.ReadOnly),
rocksdb_options: goToCSlice([]byte(r.cfg.RocksDBOptions)),
extra_options: goToCSlice(r.cfg.StorageConfig.ExtraOptions),
extra_options: goToCSlice(r.cfg.ExtraOptions),
})
if err := statusToError(status); err != nil {
return errors.Wrap(err, "could not open rocksdb instance")
}

// Update or add the version file if needed and if on-disk.
if len(r.cfg.StorageConfig.Dir) != 0 && existingVersion < newVersion {
if err := writeVersionFile(r.cfg.StorageConfig.Dir, newVersion); err != nil {
if len(r.cfg.Dir) != 0 && existingVersion < newVersion {
if err := writeVersionFile(r.cfg.Dir, newVersion); err != nil {
return err
}
}
Expand Down Expand Up @@ -692,8 +687,8 @@ func (r *RocksDB) syncLoop() {
}

var min time.Duration
if r.cfg.StorageConfig.Settings != nil {
min = minWALSyncInterval.Get(&r.cfg.StorageConfig.Settings.SV)
if r.cfg.Settings != nil {
min = minWALSyncInterval.Get(&r.cfg.Settings.SV)
}
if delta := timeutil.Since(lastSync); delta < min {
s.Unlock()
Expand All @@ -713,7 +708,7 @@ func (r *RocksDB) syncLoop() {
// the WAL, and RocksDB's recovery terminates upon encountering any
// corruption. So, we must not call `DBSyncWAL` again after it has
// failed once.
if r.cfg.StorageConfig.Dir != "" && err == nil {
if r.cfg.Dir != "" && err == nil {
err = statusToError(C.DBSyncWAL(r.rdb))
lastSync = timeutil.Now()
}
Expand All @@ -733,7 +728,7 @@ func (r *RocksDB) Close() {
log.Errorf(context.TODO(), "closing unopened rocksdb instance")
return
}
if len(r.cfg.StorageConfig.Dir) == 0 {
if len(r.cfg.Dir) == 0 {
if log.V(1) {
log.Infof(context.TODO(), "closing in-memory rocksdb instance")
}
Expand All @@ -742,7 +737,7 @@ func (r *RocksDB) Close() {
log.Warning(context.TODO(), err)
}
} else {
log.Infof(context.TODO(), "closing rocksdb instance at %q", r.cfg.StorageConfig.Dir)
log.Infof(context.TODO(), "closing rocksdb instance at %q", r.cfg.Dir)
}
if r.rdb != nil {
if err := statusToError(C.DBClose(r.rdb)); err != nil {
Expand Down Expand Up @@ -782,7 +777,7 @@ func (r *RocksDB) Closed() bool {
// and potentially other labels to identify important attributes of
// the engine.
func (r *RocksDB) Attrs() roachpb.Attributes {
return r.cfg.StorageConfig.Attrs
return r.cfg.Attrs
}

// Put sets the given key to the value provided.
Expand Down Expand Up @@ -876,7 +871,7 @@ func (r *RocksDB) Iterate(start, end roachpb.Key, f func(MVCCKeyValue) (bool, er

// Capacity queries the underlying file system for disk capacity information.
func (r *RocksDB) Capacity() (roachpb.StoreCapacity, error) {
return computeCapacity(r.cfg.StorageConfig.Dir, r.cfg.MaxSizeBytes)
return computeCapacity(r.cfg.Dir, r.cfg.MaxSize)
}

// Compact forces compaction over the entire database.
Expand Down Expand Up @@ -3055,7 +3050,7 @@ func (r *RocksDB) setAuxiliaryDir(d string) error {

// PreIngestDelay implements the Engine interface.
func (r *RocksDB) PreIngestDelay(ctx context.Context) {
preIngestDelay(ctx, r, r.cfg.StorageConfig.Settings)
preIngestDelay(ctx, r, r.cfg.Settings)
}

// IngestExternalFiles atomically links a slice of files into the RocksDB
Expand Down
10 changes: 5 additions & 5 deletions pkg/storage/engine/temp_engine.go
Original file line number Diff line number Diff line change
Expand Up @@ -75,14 +75,14 @@ func NewRocksDBTempEngine(

cfg := RocksDBConfig{
StorageConfig: base.StorageConfig{
Attrs: roachpb.Attributes{},
Dir: tempStorage.Path,
Attrs: roachpb.Attributes{},
Dir: tempStorage.Path,
// MaxSize doesn't matter for temp storage - it's not enforced in any
// way.
MaxSize: 0,
UseFileRegistry: storeSpec.UseFileRegistry,
ExtraOptions: storeSpec.ExtraOptions,
},
// MaxSizeBytes doesn't matter for temp storage - it's not
// enforced in any way.
MaxSizeBytes: 0,
MaxOpenFiles: 128, // TODO(arjun): Revisit this.
}
rocksDBCache := NewRocksDBCache(0)
Expand Down

0 comments on commit 25b8acf

Please sign in to comment.