Skip to content

Commit

Permalink
server,storage: capture filesystem properties in store descriptors
Browse files Browse the repository at this point in the history
We have found numerous times during troubleshooting that it is useful
to inspect the filesystem type and mount options and, when hosted
without containers, the block device used to support the filesystem.

This patch achieves this by retrieving the filesystem details and
attaching them to store descriptors.

Release note: None
  • Loading branch information
knz committed Nov 10, 2021
1 parent 2de17e7 commit e1f30c6
Show file tree
Hide file tree
Showing 9 changed files with 883 additions and 109 deletions.
16 changes: 11 additions & 5 deletions pkg/kv/kvserver/store.go
Original file line number Diff line number Diff line change
Expand Up @@ -79,7 +79,7 @@ import (
"github.com/cockroachdb/errors"
"github.com/cockroachdb/logtags"
"github.com/cockroachdb/redact"
"go.etcd.io/etcd/raft/v3"
raft "go.etcd.io/etcd/raft/v3"
"golang.org/x/time/rate"
)

Expand Down Expand Up @@ -2510,6 +2510,11 @@ func (s *Store) Attrs() roachpb.Attributes {
return s.engine.Attrs()
}

// Properties returns the properties of the underlying store.
func (s *Store) Properties() roachpb.StoreProperties {
return s.engine.Properties()
}

// Capacity returns the capacity of the underlying storage engine. Note that
// this does not include reservations.
// Note that Capacity() has the side effect of updating some of the store's
Expand Down Expand Up @@ -2615,10 +2620,11 @@ func (s *Store) Descriptor(ctx context.Context, useCached bool) (*roachpb.StoreD

// Initialize the store descriptor.
return &roachpb.StoreDescriptor{
StoreID: s.Ident.StoreID,
Attrs: s.Attrs(),
Node: *s.nodeDesc,
Capacity: capacity,
StoreID: s.Ident.StoreID,
Attrs: s.Attrs(),
Node: *s.nodeDesc,
Capacity: capacity,
Properties: s.Properties(),
}, nil
}

Expand Down
14 changes: 14 additions & 0 deletions pkg/roachpb/metadata.go
Original file line number Diff line number Diff line change
Expand Up @@ -526,6 +526,20 @@ func (p Percentiles) SafeFormat(w redact.SafePrinter, _ rune) {
p.P10, p.P25, p.P50, p.P75, p.P90, p.PMax)
}

func (sc FileStoreProperties) String() string {
return redact.StringWithoutMarkers(sc)
}

// SafeFormat implements the redact.SafeFormatter interface.
func (sc FileStoreProperties) SafeFormat(w redact.SafePrinter, _ rune) {
w.Printf("{path=%s, fs=%s, blkdev=%s, mnt=%s opts=%s}",
sc.Path,
redact.SafeString(sc.FsType),
sc.BlockDevice,
sc.MountPoint,
sc.MountOptions)
}

// String returns a string representation of the StoreCapacity.
func (sc StoreCapacity) String() string {
return redact.StringWithoutMarkers(sc)
Expand Down
817 changes: 714 additions & 103 deletions pkg/roachpb/metadata.pb.go

Large diffs are not rendered by default.

29 changes: 29 additions & 0 deletions pkg/roachpb/metadata.proto
Original file line number Diff line number Diff line change
Expand Up @@ -334,6 +334,34 @@ message StoreCapacity {
optional Percentiles writes_per_replica = 7 [(gogoproto.nullable) = false];
}

// StoreProperties contains configuration and OS-level details for a storage device.
message StoreProperties {
// encrypted indicates whether the store is encrypted.
optional bool encrypted = 1 [(gogoproto.nullable) = false];
// read_only indicates whether the store is attached read_only.
optional bool read_only = 2 [(gogoproto.nullable) = false];

// disk_properties reports details about the underlying filesystem,
// when the store is supported by a file store. Unset otherwise.
optional FileStoreProperties file_store_properties = 3;
}

// FileStoreProperties contains configuration and OS-level details for a file store.
message FileStoreProperties {
option (gogoproto.goproto_stringer) = false;

// path reports the configured filesystem path for the store.
optional string path = 1 [(gogoproto.nullable) = false];
// fs_type reports the external filesystem type (ufs, ext4, etc), if known.
optional string fs_type = 2 [(gogoproto.nullable) = false];
// block_device reports which block devices supports the filesystem, if known.
optional string block_device = 3 [(gogoproto.nullable) = false];
// mount_point reports the mount point of the filesystem, if known.
optional string mount_point = 4 [(gogoproto.nullable) = false];
// mount_options reports the mount options, if known.
optional string mount_options = 5 [(gogoproto.nullable) = false];
}

// NodeDescriptor holds details on node physical/network topology.
message NodeDescriptor {
option (gogoproto.equal) = true;
Expand Down Expand Up @@ -369,6 +397,7 @@ message StoreDescriptor {
optional Attributes attrs = 2 [(gogoproto.nullable) = false];
optional NodeDescriptor node = 3 [(gogoproto.nullable) = false];
optional StoreCapacity capacity = 4 [(gogoproto.nullable) = false];
optional StoreProperties properties = 5 [(gogoproto.nullable) = false];
}

// StoreDeadReplicas holds a storeID and a list of dead replicas on that store.
Expand Down
2 changes: 2 additions & 0 deletions pkg/server/config.go
Original file line number Diff line number Diff line change
Expand Up @@ -539,6 +539,7 @@ func (cfg *Config) CreateEngines(ctx context.Context) (Engines, error) {
if err != nil {
return Engines{}, err
}
details = append(details, redact.Sprintf("store %d: %+v", i, e.Properties()))
engines = append(engines, e)
} else {
e, err := storage.Open(ctx,
Expand Down Expand Up @@ -604,6 +605,7 @@ func (cfg *Config) CreateEngines(ctx context.Context) (Engines, error) {
if err != nil {
return Engines{}, err
}
details = append(details, redact.Sprintf("store %d: %+v", i, eng.Properties()))
engines = append(engines, eng)
}
}
Expand Down
2 changes: 2 additions & 0 deletions pkg/storage/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,7 @@ go_library(
"slice_go1.9.go",
"sst_iterator.go",
"sst_writer.go",
"store_properties.go",
"temp_engine.go",
"testing_knobs.go",
":gen-resourcelimitreached-stringer", # keep
Expand Down Expand Up @@ -79,6 +80,7 @@ go_library(
"@com_github_cockroachdb_pebble//vfs/atomicfs",
"@com_github_cockroachdb_redact//:redact",
"@com_github_dustin_go_humanize//:go-humanize",
"@com_github_elastic_gosigar//:gosigar",
"@com_github_gogo_protobuf//proto",
],
)
Expand Down
2 changes: 2 additions & 0 deletions pkg/storage/engine.go
Original file line number Diff line number Diff line change
Expand Up @@ -743,6 +743,8 @@ type Engine interface {
Attrs() roachpb.Attributes
// Capacity returns capacity details for the engine's available storage.
Capacity() (roachpb.StoreCapacity, error)
// Properties returns the low-level properties for the engine's underlying storage.
Properties() roachpb.StoreProperties
// Compact forces compaction over the entire database.
Compact() error
// Flush causes the engine to write all in-memory data to disk
Expand Down
11 changes: 10 additions & 1 deletion pkg/storage/pebble.go
Original file line number Diff line number Diff line change
Expand Up @@ -49,7 +49,7 @@ import (
"github.com/cockroachdb/pebble/bloom"
"github.com/cockroachdb/pebble/vfs"
"github.com/cockroachdb/redact"
"github.com/dustin/go-humanize"
humanize "github.com/dustin/go-humanize"
)

const maxSyncDurationFatalOnExceededDefault = true
Expand Down Expand Up @@ -470,6 +470,7 @@ type Pebble struct {
ballastSize int64
maxSize int64
attrs roachpb.Attributes
properties roachpb.StoreProperties
// settings must be non-nil if this Pebble instance will be used to write
// intents.
settings *cluster.Settings
Expand Down Expand Up @@ -661,6 +662,8 @@ func NewPebble(ctx context.Context, cfg PebbleConfig) (*Pebble, error) {
}
}

storeProps := computeStoreProperties(ctx, cfg.Dir, cfg.Opts.ReadOnly, env != nil /* encryptionEnabled */)

p := &Pebble{
readOnly: cfg.Opts.ReadOnly,
path: cfg.Dir,
Expand All @@ -669,6 +672,7 @@ func NewPebble(ctx context.Context, cfg PebbleConfig) (*Pebble, error) {
ballastSize: cfg.BallastSize,
maxSize: cfg.MaxSize,
attrs: cfg.Attrs,
properties: storeProps,
settings: cfg.Settings,
encryption: env,
fileRegistry: fileRegistry,
Expand Down Expand Up @@ -1062,6 +1066,11 @@ func (p *Pebble) Attrs() roachpb.Attributes {
return p.attrs
}

// Properties implements the Engine interface.
func (p *Pebble) Properties() roachpb.StoreProperties {
return p.properties
}

// Capacity implements the Engine interface.
func (p *Pebble) Capacity() (roachpb.StoreCapacity, error) {
dir := p.path
Expand Down
99 changes: 99 additions & 0 deletions pkg/storage/store_properties.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,99 @@
// Copyright 2021 The Cockroach Authors.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.

package storage

import (
"context"
"path/filepath"

"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/elastic/gosigar"
)

func computeStoreProperties(
ctx context.Context, dir string, readonly bool, encryptionEnabled bool,
) roachpb.StoreProperties {
props := roachpb.StoreProperties{
ReadOnly: readonly,
Encrypted: encryptionEnabled,
}

// In-memory store?
if dir == "" {
return props
}

fsprops := getFileSystemProperties(ctx, dir)
props.FileStoreProperties = &fsprops
return props
}

func getFileSystemProperties(ctx context.Context, dir string) roachpb.FileStoreProperties {
fsprops := roachpb.FileStoreProperties{
Path: dir,
}

// Find which filesystem supports the store.

absPath, err := filepath.Abs(dir)
if err != nil {
log.Warningf(ctx, "cannot compute absolute file path for %q: %v", dir, err)
return fsprops
}

// Alas, only BSD reliably populates "fs" in os.StatFs(),
// so we must find the filesystem manually.
//
// Note that scanning the list of mounts is also
// what linux' df(1) command does.
//
var fslist gosigar.FileSystemList
if err := fslist.Get(); err != nil {
log.Warningf(ctx, "cannot retrieve filesystem list: %v", err)
return fsprops
}

var fsInfo *gosigar.FileSystem
// We're reading the list of mounts in reverse order: we're assuming
// that mounts are LIFO and can only be stacked, so the best match
// will necessarily be the first filesystem that's a prefix of the
// target directory, when looking from the end of the file.
//
// TODO(ssd): Steven points out that gosigar reads from /etc/mtab on
// linux, which is sometimes managed by the user command 'mount' and
// can sometimes miss entries when `mount -n` is used. It might be
// better to change gosigar to use /proc/mounts instead.
//
// FWIW, we are OK with this for now, since the systems where crdb
// is typically being deployed are well-behaved in that regard:
// Kubernetes mirrors /proc/mount in /etc/mtab.
for i := len(fslist.List) - 1; i >= 0; i-- {
// filepath.Rel can reliably tell us if a path is relative to
// another: if it is not, an error is returned.
_, err := filepath.Rel(fslist.List[i].DirName, absPath)
if err == nil {
fsInfo = &fslist.List[i]
break
}
}
if fsInfo == nil {
// This is surprising!? We're expecting at least a match on the
// root filesystem. Oh well.
return fsprops
}

fsprops.FsType = fsInfo.SysTypeName
fsprops.BlockDevice = fsInfo.DevName
fsprops.MountPoint = fsInfo.DirName
fsprops.MountOptions = fsInfo.Options
return fsprops
}

0 comments on commit e1f30c6

Please sign in to comment.