Skip to content

Commit

Permalink
Merge pull request #65125 from erikgrinaker/backport21.1-64095
Browse files Browse the repository at this point in the history
release-21.1: storage: fix usage of in-memory size attribute
  • Loading branch information
erikgrinaker authored May 13, 2021
2 parents 88a464f + e11c255 commit a5f8630
Show file tree
Hide file tree
Showing 15 changed files with 95 additions and 36 deletions.
5 changes: 4 additions & 1 deletion pkg/base/test_server_args.go
Original file line number Diff line number Diff line change
Expand Up @@ -161,10 +161,13 @@ type TestClusterArgs struct {
}

var (
// DefaultTestStoreSpec is just a single in memory store of 100 MiB
// DefaultTestStoreSpec is just a single in memory store of 512 MiB
// with no special attributes.
DefaultTestStoreSpec = StoreSpec{
InMemory: true,
Size: SizeSpec{
InBytes: 512 << 20,
},
}
)

Expand Down
3 changes: 2 additions & 1 deletion pkg/kv/kvserver/client_metrics_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -232,12 +232,13 @@ func TestStoreMetrics(t *testing.T) {
base.TestClusterArgs{
ReplicationMode: base.ReplicationManual,
ServerArgs: base.TestServerArgs{
CacheSize: 1 << 20, /* 1 MiB */
StoreSpecs: []base.StoreSpec{
{
InMemory: true,
// Specify a size to trigger the BlockCache in Pebble.
Size: base.SizeSpec{
InBytes: 1 << 20,
InBytes: 512 << 20, /* 512 MiB */
},
},
},
Expand Down
7 changes: 6 additions & 1 deletion pkg/kv/kvserver/store_snapshot.go
Original file line number Diff line number Diff line change
Expand Up @@ -930,7 +930,12 @@ func SendEmptySnapshot(
) error {
// Create an engine to use as a buffer for the empty snapshot.
eng := storage.NewInMem(
context.Background(), roachpb.Attributes{}, 1<<20, nil /* settings */)
context.Background(),
roachpb.Attributes{},
1<<20, /* cacheSize 1MiB */
512<<20, /* storeSize 512 MiB */
nil, /* settings */
)
defer eng.Close()

var ms enginepb.MVCCStats
Expand Down
4 changes: 2 additions & 2 deletions pkg/server/config.go
Original file line number Diff line number Diff line change
Expand Up @@ -509,13 +509,13 @@ func (cfg *Config) CreateEngines(ctx context.Context) (Engines, error) {
"engine no registry available. Please use " +
"Knobs.Server.StickyEngineRegistry to provide one.")
}
e, err := knobs.StickyEngineRegistry.GetOrCreateStickyInMemEngine(ctx, spec)
e, err := knobs.StickyEngineRegistry.GetOrCreateStickyInMemEngine(ctx, cfg, spec)
if err != nil {
return Engines{}, err
}
engines = append(engines, e)
} else {
engines = append(engines, storage.NewInMem(ctx, spec.Attributes, sizeInBytes, cfg.Settings))
engines = append(engines, storage.NewInMem(ctx, spec.Attributes, cfg.CacheSize, sizeInBytes, cfg.Settings))
}
} else {
if spec.Size.Percent > 0 {
Expand Down
4 changes: 2 additions & 2 deletions pkg/server/settings_cache_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -43,8 +43,8 @@ func TestCachedSettingsStoreAndLoad(t *testing.T) {

ctx := context.Background()
attrs := roachpb.Attributes{}
cacheSize := int64(1 << 20)
engine := storage.NewInMemForTesting(ctx, attrs, cacheSize)
storeSize := int64(512 << 20) /* 512 MiB */
engine := storage.NewInMemForTesting(ctx, attrs, storeSize)
defer engine.Close()

require.NoError(t, storeCachedSettingsKVs(ctx, engine, testSettings))
Expand Down
8 changes: 3 additions & 5 deletions pkg/server/sticky_engine.go
Original file line number Diff line number Diff line change
Expand Up @@ -46,9 +46,7 @@ type StickyInMemEnginesRegistry interface {
// Note that if you re-create an existing sticky engine the new attributes
// and cache size will be ignored.
// One must Close() on the sticky engine before another can be fetched.
GetOrCreateStickyInMemEngine(
ctx context.Context, spec base.StoreSpec,
) (storage.Engine, error)
GetOrCreateStickyInMemEngine(ctx context.Context, cfg *Config, spec base.StoreSpec) (storage.Engine, error)
// CloseAllStickyInMemEngines closes all sticky in memory engines that were
// created by this registry.
CloseAllStickyInMemEngines()
Expand Down Expand Up @@ -86,7 +84,7 @@ func NewStickyInMemEnginesRegistry() StickyInMemEnginesRegistry {

// GetOrCreateStickyInMemEngine implements the StickyInMemEnginesRegistry interface.
func (registry *stickyInMemEnginesRegistryImpl) GetOrCreateStickyInMemEngine(
ctx context.Context, spec base.StoreSpec,
ctx context.Context, cfg *Config, spec base.StoreSpec,
) (storage.Engine, error) {
registry.mu.Lock()
defer registry.mu.Unlock()
Expand All @@ -110,7 +108,7 @@ func (registry *stickyInMemEnginesRegistryImpl) GetOrCreateStickyInMemEngine(
// create a random one since that is what we like to do in tests (for
// better test coverage).
Engine: storage.NewInMem(
ctx, spec.Attributes, spec.Size.InBytes, storage.MakeRandomSettingsForSeparatedIntents()),
ctx, spec.Attributes, cfg.CacheSize, spec.Size.InBytes, storage.MakeRandomSettingsForSeparatedIntents()),
}
registry.entries[spec.StickyInMemoryEngineID] = engine
return engine, nil
Expand Down
20 changes: 13 additions & 7 deletions pkg/server/sticky_engine_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@ import (

"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/storage"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/log"
Expand All @@ -28,30 +29,35 @@ func TestStickyEngines(t *testing.T) {

ctx := context.Background()
attrs := roachpb.Attributes{}
cacheSize := int64(1 << 20)
cacheSize := int64(1 << 20) /* 1 MiB */
storeSize := int64(512 << 20) /* 512 MiB */

registry := NewStickyInMemEnginesRegistry()

cfg1 := MakeConfig(ctx, cluster.MakeTestingClusterSettings())
cfg1.CacheSize = cacheSize
spec1 := base.StoreSpec{
StickyInMemoryEngineID: "engine1",
Attributes: attrs,
Size: base.SizeSpec{InBytes: cacheSize},
Size: base.SizeSpec{InBytes: storeSize},
}
engine1, err := registry.GetOrCreateStickyInMemEngine(ctx, spec1)
engine1, err := registry.GetOrCreateStickyInMemEngine(ctx, &cfg1, spec1)
require.NoError(t, err)
require.False(t, engine1.Closed())

cfg2 := MakeConfig(ctx, cluster.MakeTestingClusterSettings())
cfg2.CacheSize = cacheSize
spec2 := base.StoreSpec{
StickyInMemoryEngineID: "engine2",
Attributes: attrs,
Size: base.SizeSpec{InBytes: cacheSize},
Size: base.SizeSpec{InBytes: storeSize},
}
engine2, err := registry.GetOrCreateStickyInMemEngine(ctx, spec2)
engine2, err := registry.GetOrCreateStickyInMemEngine(ctx, &cfg2, spec2)
require.NoError(t, err)
require.False(t, engine2.Closed())

// Regetting the engine whilst it is not closed will fail.
_, err = registry.GetOrCreateStickyInMemEngine(ctx, spec1)
_, err = registry.GetOrCreateStickyInMemEngine(ctx, &cfg1, spec1)
require.EqualError(t, err, "sticky engine engine1 has not been closed")

// Close the engine, which allows it to be refetched.
Expand All @@ -60,7 +66,7 @@ func TestStickyEngines(t *testing.T) {
require.False(t, engine1.(*stickyInMemEngine).Engine.Closed())

// Refetching the engine should give back the same engine.
engine1Refetched, err := registry.GetOrCreateStickyInMemEngine(ctx, spec1)
engine1Refetched, err := registry.GetOrCreateStickyInMemEngine(ctx, &cfg1, spec1)
require.NoError(t, err)
require.Equal(t, engine1, engine1Refetched)
require.False(t, engine1.Closed())
Expand Down
8 changes: 7 additions & 1 deletion pkg/storage/batch_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1100,7 +1100,13 @@ func TestDecodeKey(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)

e := newPebbleInMem(context.Background(), roachpb.Attributes{}, 1<<20, nil /* settings */)
e := newPebbleInMem(
context.Background(),
roachpb.Attributes{},
1<<20, /* cacheSize */
512<<20, /* storeSize */
nil, /* settings */
)
defer e.Close()

tests := []MVCCKey{
Expand Down
17 changes: 15 additions & 2 deletions pkg/storage/engine_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1503,7 +1503,14 @@ func TestSupportsPrev(t *testing.T) {
})
}
t.Run("pebble", func(t *testing.T) {
eng := newPebbleInMem(context.Background(), roachpb.Attributes{}, 1<<20, nil /* settings */)

eng := newPebbleInMem(
context.Background(),
roachpb.Attributes{},
1<<20, /* cacheSize */
512<<20, /* storeSize */
nil, /* settings */
)
defer eng.Close()
runTest(t, eng, engineTest{
engineIterSupportsPrev: true,
Expand Down Expand Up @@ -1627,7 +1634,13 @@ func TestScanSeparatedIntents(t *testing.T) {
for name, enableSeparatedIntents := range map[string]bool{"interleaved": false, "separated": true} {
t.Run(name, func(t *testing.T) {
settings := makeSettingsForSeparatedIntents(false, enableSeparatedIntents)
eng := newPebbleInMem(ctx, roachpb.Attributes{}, 1<<20, settings)
eng := newPebbleInMem(
ctx,
roachpb.Attributes{},
1<<20, /* cacheSize */
512<<20, /* storeSize */
settings,
)
defer eng.Close()

for _, key := range keys {
Expand Down
11 changes: 7 additions & 4 deletions pkg/storage/in_mem.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,9 +25,12 @@ import (
//
// FIXME(tschottdorf): make the signature similar to NewPebble (require a cfg).
func NewInMem(
ctx context.Context, attrs roachpb.Attributes, cacheSize int64, settings *cluster.Settings,
ctx context.Context,
attrs roachpb.Attributes,
cacheSize, storeSize int64,
settings *cluster.Settings,
) Engine {
return newPebbleInMem(ctx, attrs, cacheSize, settings)
return newPebbleInMem(ctx, attrs, cacheSize, storeSize, settings)
}

// The ForTesting functions randomize the settings for separated intents. This
Expand All @@ -43,9 +46,9 @@ func NewInMem(

// NewInMemForTesting allocates and returns a new, opened in-memory engine. The caller
// must call the engine's Close method when the engine is no longer needed.
func NewInMemForTesting(ctx context.Context, attrs roachpb.Attributes, cacheSize int64) Engine {
func NewInMemForTesting(ctx context.Context, attrs roachpb.Attributes, storeSize int64) Engine {
settings := MakeRandomSettingsForSeparatedIntents()
return newPebbleInMem(ctx, attrs, cacheSize, settings)
return newPebbleInMem(ctx, attrs, 0 /* cacheSize */, storeSize, settings)
}

// NewDefaultInMemForTesting allocates and returns a new, opened in-memory engine with
Expand Down
8 changes: 7 additions & 1 deletion pkg/storage/multi_iterator_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,13 @@ func TestMultiIterator(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)

pebble := newPebbleInMem(context.Background(), roachpb.Attributes{}, 1<<20, nil /* settings */)
pebble := newPebbleInMem(
context.Background(),
roachpb.Attributes{},
1<<20, /* cacheSize */
512<<20, /* storeSize */
nil, /* settings */
)
defer pebble.Close()

// Each `input` is turned into an iterator and these are passed to a new
Expand Down
8 changes: 7 additions & 1 deletion pkg/storage/mvcc_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -85,7 +85,13 @@ func createTestPebbleEngine() Engine {
}

func createTestPebbleEngineWithSettings(settings *cluster.Settings) Engine {
return newPebbleInMem(context.Background(), roachpb.Attributes{}, 1<<20, settings)
return newPebbleInMem(
context.Background(),
roachpb.Attributes{},
1<<20, /* cacheSize */
512<<20, /* storeSize */
settings,
)
}

// TODO(sumeer): the following is legacy from when we had multiple engine
Expand Down
11 changes: 6 additions & 5 deletions pkg/storage/pebble.go
Original file line number Diff line number Diff line change
Expand Up @@ -576,7 +576,10 @@ func NewPebble(ctx context.Context, cfg PebbleConfig) (*Pebble, error) {
}

func newPebbleInMem(
ctx context.Context, attrs roachpb.Attributes, cacheSize int64, settings *cluster.Settings,
ctx context.Context,
attrs roachpb.Attributes,
cacheSize, storeSize int64,
settings *cluster.Settings,
) *Pebble {
opts := DefaultPebbleOptions()
opts.Cache = pebble.NewCache(cacheSize)
Expand All @@ -587,10 +590,8 @@ func newPebbleInMem(
ctx,
PebbleConfig{
StorageConfig: base.StorageConfig{
Attrs: attrs,
// TODO(bdarnell): The hard-coded 512 MiB is wrong; see
// https://github.com/cockroachdb/cockroach/issues/16750
MaxSize: 512 << 20, /* 512 MiB */
Attrs: attrs,
MaxSize: storeSize,
Settings: settings,
},
Opts: opts,
Expand Down
8 changes: 7 additions & 1 deletion pkg/storage/pebble_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -424,7 +424,13 @@ func TestPebbleDiskSlowEmit(t *testing.T) {

settings := cluster.MakeTestingClusterSettings()
MaxSyncDurationFatalOnExceeded.Override(&settings.SV, false)
p := newPebbleInMem(context.Background(), roachpb.Attributes{}, 1<<20, settings)
p := newPebbleInMem(
context.Background(),
roachpb.Attributes{},
1<<20, /* cacheSize */
512<<20, /* storeSize */
settings,
)
defer p.Close()

require.Equal(t, uint64(0), p.diskSlowCount)
Expand Down
9 changes: 7 additions & 2 deletions pkg/testutils/localtestcluster/local_test_cluster.go
Original file line number Diff line number Diff line change
Expand Up @@ -133,8 +133,13 @@ func (ltc *LocalTestCluster) Start(t testing.TB, baseCtx *base.Config, initFacto
clusterID := &cfg.RPCContext.ClusterID
server := rpc.NewServer(cfg.RPCContext) // never started
ltc.Gossip = gossip.New(ambient, clusterID, nc, cfg.RPCContext, server, ltc.stopper, metric.NewRegistry(), roachpb.Locality{}, zonepb.DefaultZoneConfigRef())
ltc.Eng = storage.NewInMem(ambient.AnnotateCtx(context.Background()), roachpb.Attributes{},
50<<20, storage.MakeRandomSettingsForSeparatedIntents())
ltc.Eng = storage.NewInMem(
ambient.AnnotateCtx(context.Background()),
roachpb.Attributes{},
0, /* cacheSize */
50<<20, /* storeSize */
storage.MakeRandomSettingsForSeparatedIntents(),
)
ltc.stopper.AddCloser(ltc.Eng)

ltc.Stores = kvserver.NewStores(ambient, ltc.Clock)
Expand Down

0 comments on commit a5f8630

Please sign in to comment.