diff --git a/pkg/settings/values.go b/pkg/settings/values.go index f53a40626bd9..c089de591abe 100644 --- a/pkg/settings/values.go +++ b/pkg/settings/values.go @@ -16,7 +16,7 @@ import ( // MaxSettings is the maximum number of settings that the system supports. // Exported for tests. -const MaxSettings = 1023 +const MaxSettings = 1151 // Values is a container that stores values for all registered settings. // Each setting is assigned a unique slot (up to MaxSettings). diff --git a/pkg/storage/open.go b/pkg/storage/open.go index fc1398c24a3e..851258c237a7 100644 --- a/pkg/storage/open.go +++ b/pkg/storage/open.go @@ -560,7 +560,7 @@ func Open( var cfg engineConfig cfg.env = env cfg.settings = settings - cfg.opts = DefaultPebbleOptions() + cfg.opts = DefaultPebbleOptionsForOpen(&cfg.settings.SV) cfg.opts.FS = env cfg.opts.ReadOnly = env.IsReadOnly() for _, opt := range opts { diff --git a/pkg/storage/pebble.go b/pkg/storage/pebble.go index 0cbdfbcda64c..29894803b43c 100644 --- a/pkg/storage/pebble.go +++ b/pkg/storage/pebble.go @@ -538,6 +538,24 @@ var ( 1 /* min */, 80 /* max */)), settings.IntInRange(1, 100), ) + valueSeparationLatencyTolerantMinimumSize = settings.RegisterIntSetting( + settings.SystemVisible, + "storage.value_separation.latency_tolerant_minimum_size", + "the minimum size of a value that will be separated into a blob file given the value is "+ + "latency tolerant (in the range local keyspace)", + int64(metamorphic.ConstantWithTestRange("storage.value_separation.latency_tolerant_minimum_size", + 32 /* 32 bytes (default) */, 25 /* 25 bytes (minimum) */, 512 /* 512 bytes (maximum) */)), + settings.IntWithMinimum(1), + ) + valueSeparationMVCCGarbageMinimumSize = settings.RegisterIntSetting( + settings.SystemVisible, + "storage.value_separation.mvcc_history_minimum_size", + "the minimum size of a value that will be separated into a blob file given the value is "+ + "likely not the latest version of a key", + int64(metamorphic.ConstantWithTestRange("storage.value_separation.mvcc_history_minimum_size", + 32 /* 32 bytes (default) */, 25 /* 25 bytes (minimum) */, 512 /* 512 bytes (maximum) */)), + settings.IntWithMinimum(1), + ) ) // This setting controls deletion pacing. This helps prevent disk slowness @@ -614,8 +632,7 @@ const DefaultMemtableSize = 64 << 20 // 64 MB const mvccWallTimeIntervalCollector = "MVCCTimeInterval" -// DefaultPebbleOptions returns the default pebble options. -func DefaultPebbleOptions() *pebble.Options { +func defaultPebbleOptions(sv *settings.Values) *pebble.Options { opts := &pebble.Options{ Comparer: &EngineComparer, FS: vfs.Default, @@ -642,7 +659,7 @@ func DefaultPebbleOptions() *pebble.Options { opts.FlushDelayRangeKey = 10 * time.Second opts.Experimental.ShortAttributeExtractor = shortAttributeExtractorForValues - opts.Experimental.SpanPolicyFunc = spanPolicyFunc + opts.Experimental.SpanPolicyFunc = spanPolicyFuncFactory(sv) opts.Experimental.UserKeyCategories = userKeyCategories // Every 5 minutes, log iterators that have been open for more than 1 minute. @@ -683,6 +700,20 @@ func DefaultPebbleOptions() *pebble.Options { return opts } +// DefaultPebbleOptions returns the default pebble options for general use +// (e.g., SST writers, external iterators, tests). This does not use cluster +// settings and should not be used when opening a production Pebble engine. +func DefaultPebbleOptions() *pebble.Options { + return defaultPebbleOptions(nil /* sv */) +} + +// DefaultPebbleOptionsForOpen returns the default pebble options for opening +// a production Pebble engine. It uses cluster settings to configure value +// storage policies. +func DefaultPebbleOptionsForOpen(sv *settings.Values) *pebble.Options { + return defaultPebbleOptions(sv) +} + var ( spanPolicyLocalRangeIDEndKey = EncodeMVCCKey(MVCCKey{Key: keys.LocalRangeIDPrefix.AsRawKey().PrefixEnd()}) spanPolicyLockTableStartKey = EncodeMVCCKey(MVCCKey{Key: keys.LocalRangeLockTablePrefix}) @@ -690,45 +721,53 @@ var ( spanPolicyLocalEndKey = EncodeMVCCKey(MVCCKey{Key: keys.LocalPrefix.PrefixEnd()}) ) -// spanPolicyFunc is a pebble.SpanPolicyFunc that applies special policies for +// spanPolicyFuncFactory returns a pebble.SpanPolicyFunc that applies special policies for // the CockroachDB keyspace. -func spanPolicyFunc(startKey []byte) (policy pebble.SpanPolicy, endKey []byte, _ error) { - // There's no special policy for non-local keys. - if !bytes.HasPrefix(startKey, keys.LocalPrefix) { - return pebble.SpanPolicy{}, nil, nil - } - // Prefer fast compression for all local keys, since they shouldn't take up - // a significant part of the space. - policy.PreferFastCompression = true - - // The first section of the local keyspace is the Range-ID keyspace. It - // extends from the beginning of the keyspace to the Range Local keys. The - // Range-ID keyspace includes the raft log, which is rarely read and - // receives ~half the writes. - if cockroachkvs.Compare(startKey, spanPolicyLocalRangeIDEndKey) < 0 { - if !bytes.HasPrefix(startKey, keys.LocalRangeIDPrefix) { - return pebble.SpanPolicy{}, nil, errors.AssertionFailedf("startKey %s is not a Range-ID key", startKey) +func spanPolicyFuncFactory(sv *settings.Values) func([]byte) (pebble.SpanPolicy, []byte, error) { + return func(startKey []byte) (policy pebble.SpanPolicy, endKey []byte, _ error) { + // There's no special policy for non-local keys. + if !bytes.HasPrefix(startKey, keys.LocalPrefix) { + return pebble.SpanPolicy{}, nil, nil + } + // Prefer fast compression for all local keys, since they shouldn't take up + // a significant part of the space. + policy.PreferFastCompression = true + + // The first section of the local keyspace is the Range-ID keyspace. It + // extends from the beginning of the keyspace to the Range Local keys. The + // Range-ID keyspace includes the raft log, which is rarely read and + // receives ~half the writes. + if cockroachkvs.Compare(startKey, spanPolicyLocalRangeIDEndKey) < 0 { + if !bytes.HasPrefix(startKey, keys.LocalRangeIDPrefix) { + return pebble.SpanPolicy{}, nil, errors.AssertionFailedf("startKey %s is not a Range-ID key", startKey) + } + if sv != nil { + policy.ValueStoragePolicy = pebble.ValueStoragePolicyAdjustment{ + OverrideBlobSeparationMinimumSize: int(valueSeparationLatencyTolerantMinimumSize.Get(sv)), + } + } else { + policy.ValueStoragePolicy = pebble.ValueStorageLatencyTolerant + } + return policy, spanPolicyLocalRangeIDEndKey, nil } - policy.ValueStoragePolicy = pebble.ValueStorageLatencyTolerant - return policy, spanPolicyLocalRangeIDEndKey, nil - } - // We also disable value separation for lock keys. - if cockroachkvs.Compare(startKey, spanPolicyLockTableEndKey) >= 0 { - // Not a lock key, so use default value separation within sstable (by - // suffix) and into blob files. - // NB: there won't actually be a suffix in these local keys. - return policy, spanPolicyLocalEndKey, nil - } - if cockroachkvs.Compare(startKey, spanPolicyLockTableStartKey) < 0 { - // Not a lock key, so use default value separation within sstable (by - // suffix) and into blob files. - // NB: there won't actually be a suffix in these local keys. - return policy, spanPolicyLockTableStartKey, nil + // We also disable value separation for lock keys. + if cockroachkvs.Compare(startKey, spanPolicyLockTableEndKey) >= 0 { + // Not a lock key, so use default value separation within sstable (by + // suffix) and into blob files. + // NB: there won't actually be a suffix in these local keys. + return policy, spanPolicyLocalEndKey, nil + } + if cockroachkvs.Compare(startKey, spanPolicyLockTableStartKey) < 0 { + // Not a lock key, so use default value separation within sstable (by + // suffix) and into blob files. + // NB: there won't actually be a suffix in these local keys. + return policy, spanPolicyLockTableStartKey, nil + } + // Lock key. Disable value separation. + policy.ValueStoragePolicy = pebble.ValueStorageLowReadLatency + return policy, spanPolicyLockTableEndKey, nil } - // Lock key. Disable value separation. - policy.ValueStoragePolicy = pebble.ValueStorageLowReadLatency - return policy, spanPolicyLockTableEndKey, nil } func shortAttributeExtractorForValues( @@ -1060,7 +1099,7 @@ func newPebble(ctx context.Context, cfg engineConfig) (p *Pebble, err error) { return pebble.ValueSeparationPolicy{ Enabled: true, MinimumSize: int(valueSeparationMinimumSize.Get(&cfg.settings.SV)), - MinimumMVCCGarbageSize: 32, + MinimumMVCCGarbageSize: int(valueSeparationMVCCGarbageMinimumSize.Get(&cfg.settings.SV)), MaxBlobReferenceDepth: int(valueSeparationMaxReferenceDepth.Get(&cfg.settings.SV)), RewriteMinimumAge: valueSeparationRewriteMinimumAge.Get(&cfg.settings.SV), GarbageRatioLowPriority: lowPri, diff --git a/pkg/storage/pebble_test.go b/pkg/storage/pebble_test.go index dbb061e3173b..5a2272b14396 100644 --- a/pkg/storage/pebble_test.go +++ b/pkg/storage/pebble_test.go @@ -1903,7 +1903,7 @@ func TestPebbleSpanPolicyFunc(t *testing.T) { for _, tc := range cases { t.Run(fmt.Sprintf("%x", tc.startKey), func(t *testing.T) { ek := EngineKey{Key: tc.startKey}.Encode() - policy, endKey, err := spanPolicyFunc(ek) + policy, endKey, err := spanPolicyFuncFactory(nil /* sv */)(ek) require.NoError(t, err) require.Equal(t, tc.wantPolicy, policy) require.Equal(t, tc.wantEndKey, endKey)