From f314232625bbc0863cae0a50e229b40213830c37 Mon Sep 17 00:00:00 2001 From: Radu Berinde Date: Tue, 7 Feb 2023 20:15:46 -0800 Subject: [PATCH 01/16] storage: don't modify the given cfg.Opts This change improves the `NewPebble` code to not modify the given `cfg.Opts`. Such behavior is surprising and can trip up tests that reuse the same config. Also, `ResolveEncryptedEnvOptions` and `wrapFilesystemMiddleware` no longer modify the Options directly; and `CheckNoRegistryFile` is now a standalone function. Release note: None Epic: none --- pkg/cli/debug.go | 15 ++- pkg/storage/pebble.go | 117 ++++++++++++----------- pkg/storage/pebble_file_registry.go | 4 +- pkg/storage/pebble_file_registry_test.go | 12 +-- 4 files changed, 71 insertions(+), 77 deletions(-) diff --git a/pkg/cli/debug.go b/pkg/cli/debug.go index 13f2e47345c4..ba1fb809eebd 100644 --- a/pkg/cli/debug.go +++ b/pkg/cli/debug.go @@ -1532,17 +1532,14 @@ func pebbleCryptoInitializer() error { } } - cfg := storage.PebbleConfig{ - StorageConfig: storageConfig, - Opts: storage.DefaultPebbleOptions(), - } - - // This has the side effect of storing the encrypted FS into cfg.Opts.FS. - _, _, err := storage.ResolveEncryptedEnvOptions(&cfg) + _, encryptedEnv, err := storage.ResolveEncryptedEnvOptions(&storageConfig, vfs.Default, false /* readOnly */) if err != nil { return err } - - pebbleToolFS.set(cfg.Opts.FS) + if encryptedEnv != nil { + pebbleToolFS.set(encryptedEnv.FS) + } else { + pebbleToolFS.set(vfs.Default) + } return nil } diff --git a/pkg/storage/pebble.go b/pkg/storage/pebble.go index 1240cf7a545f..f4724eb8e16a 100644 --- a/pkg/storage/pebble.go +++ b/pkg/storage/pebble.go @@ -619,10 +619,9 @@ func shortAttributeExtractorForValues( } // wrapFilesystemMiddleware wraps the Option's vfs.FS with disk-health checking -// and ENOSPC detection. It mutates the provided options to set the FS and -// returns a Closer that should be invoked when the filesystem will no longer be -// used. -func wrapFilesystemMiddleware(opts *pebble.Options) io.Closer { +// and ENOSPC detection. Returns the new FS and a Closer that should be invoked +// when the filesystem will no longer be used. +func wrapFilesystemMiddleware(opts *pebble.Options) (vfs.FS, io.Closer) { // Set disk-health check interval to min(5s, maxSyncDurationDefault). This // is mostly to ease testing; the default of 5s is too infrequent to test // conveniently. See the disk-stalled roachtest for an example of how this @@ -634,8 +633,7 @@ func wrapFilesystemMiddleware(opts *pebble.Options) io.Closer { // Instantiate a file system with disk health checking enabled. This FS // wraps the filesystem with a layer that times all write-oriented // operations. - var closer io.Closer - opts.FS, closer = vfs.WithDiskHealthChecks(opts.FS, diskHealthCheckInterval, + fs, closer := vfs.WithDiskHealthChecks(opts.FS, diskHealthCheckInterval, func(name string, opType vfs.OpType, duration time.Duration) { opts.EventListener.DiskSlow(pebble.DiskSlowInfo{ Path: name, @@ -644,10 +642,10 @@ func wrapFilesystemMiddleware(opts *pebble.Options) io.Closer { }) }) // If we encounter ENOSPC, exit with an informative exit code. - opts.FS = vfs.OnDiskFull(opts.FS, func() { + fs = vfs.OnDiskFull(fs, func() { exit.WithCode(exit.DiskFull()) }) - return closer + return fs, closer } type pebbleLogger struct { @@ -812,21 +810,23 @@ func (p *Pebble) SetStoreID(ctx context.Context, storeID int32) { p.storeIDPebbleLog.Set(ctx, storeID) } -// ResolveEncryptedEnvOptions fills in cfg.Opts.FS with an encrypted vfs if this -// store has encryption-at-rest enabled. Also returns the associated file -// registry and EncryptionStatsHandler. -func ResolveEncryptedEnvOptions(cfg *PebbleConfig) (*PebbleFileRegistry, *EncryptionEnv, error) { - fileRegistry := &PebbleFileRegistry{FS: cfg.Opts.FS, DBDir: cfg.Dir, ReadOnly: cfg.Opts.ReadOnly} +// ResolveEncryptedEnvOptions creates the EncryptionEnv and associated file +// registry if this store has encryption-at-rest enabled; otherwise returns a +// nil EncryptionEnv. +func ResolveEncryptedEnvOptions( + cfg *base.StorageConfig, fs vfs.FS, readOnly bool, +) (*PebbleFileRegistry, *EncryptionEnv, error) { + var fileRegistry *PebbleFileRegistry if cfg.UseFileRegistry { + fileRegistry = &PebbleFileRegistry{FS: fs, DBDir: cfg.Dir, ReadOnly: readOnly} if err := fileRegistry.Load(); err != nil { return nil, nil, err } } else { - if err := fileRegistry.CheckNoRegistryFile(); err != nil { + if err := CheckNoRegistryFile(fs, cfg.Dir); err != nil { return nil, nil, fmt.Errorf("encryption was used on this store before, but no encryption flags " + "specified. You need a CCL build and must fully specify the --enterprise-encryption flag") } - fileRegistry = nil } var env *EncryptionEnv @@ -840,37 +840,40 @@ func ResolveEncryptedEnvOptions(cfg *PebbleConfig) (*PebbleFileRegistry, *Encryp } var err error env, err = NewEncryptedEnvFunc( - cfg.Opts.FS, + fs, fileRegistry, cfg.Dir, - cfg.Opts.ReadOnly, + readOnly, cfg.EncryptionOptions, ) if err != nil { return nil, nil, err } - // TODO(jackson): Should this just return an EncryptionEnv, - // rather than mutating cfg.Opts? - cfg.Opts.FS = env.FS } return fileRegistry, env, nil } // NewPebble creates a new Pebble instance, at the specified path. func NewPebble(ctx context.Context, cfg PebbleConfig) (p *Pebble, err error) { - // pebble.Open also calls EnsureDefaults, but only after doing a clone. Call - // EnsureDefaults beforehand so we have a matching cfg here for when we save - // cfg.FS and cfg.ReadOnly later on. - if cfg.Opts == nil { - cfg.Opts = DefaultPebbleOptions() - } if cfg.Settings == nil { return nil, errors.AssertionFailedf("NewPebble requires cfg.Settings to be set") } - // Initialize the FS, wrapping it with disk health-checking and - // ENOSPC-detection. - filesystemCloser := wrapFilesystemMiddleware(cfg.Opts) + var opts *pebble.Options + if cfg.Opts == nil { + opts = DefaultPebbleOptions() + } else { + // Clone the given options so that we are free to modify them. + opts = cfg.Opts.Clone() + } + + // pebble.Open also calls EnsureDefaults, but only after doing a clone. Call + // EnsureDefaults here to make sure we have a working FS. + opts.EnsureDefaults() + + // Wrap the FS with disk health-checking and ENOSPC-detection. + var filesystemCloser io.Closer + opts.FS, filesystemCloser = wrapFilesystemMiddleware(opts) defer func() { if err != nil { filesystemCloser.Close() @@ -886,42 +889,40 @@ func NewPebble(ctx context.Context, cfg PebbleConfig) (p *Pebble, err error) { logCtx = logtags.AddTag(logCtx, "s", storeIDContainer) logCtx = logtags.AddTag(logCtx, "pebble", nil) - cfg.Opts.EnsureDefaults() - cfg.Opts.ErrorIfNotExists = cfg.MustExist - cfg.Opts.WALMinSyncInterval = func() time.Duration { + opts.ErrorIfNotExists = cfg.MustExist + opts.WALMinSyncInterval = func() time.Duration { return minWALSyncInterval.Get(&cfg.Settings.SV) } - cfg.Opts.Experimental.EnableValueBlocks = func() bool { + opts.Experimental.EnableValueBlocks = func() bool { version := cfg.Settings.Version.ActiveVersionOrEmpty(logCtx) return !version.Less(clusterversion.ByKey( clusterversion.V23_1EnablePebbleFormatSSTableValueBlocks)) && valueBlocksEnabled.Get(&cfg.Settings.SV) } - auxDir := cfg.Opts.FS.PathJoin(cfg.Dir, base.AuxiliaryDir) - if err := cfg.Opts.FS.MkdirAll(auxDir, 0755); err != nil { + auxDir := opts.FS.PathJoin(cfg.Dir, base.AuxiliaryDir) + if err := opts.FS.MkdirAll(auxDir, 0755); err != nil { return nil, err } - ballastPath := base.EmergencyBallastFile(cfg.Opts.FS.PathJoin, cfg.Dir) + ballastPath := base.EmergencyBallastFile(opts.FS.PathJoin, cfg.Dir) // For some purposes, we want to always use an unencrypted - // filesystem. The call below to ResolveEncryptedEnvOptions will - // replace cfg.Opts.FS with a VFS wrapped with encryption-at-rest if - // necessary. Before we do that, save a handle on the unencrypted - // FS for those that need it. Some call sites need the unencrypted - // FS for the purpose of atomic renames. - unencryptedFS := cfg.Opts.FS - fileRegistry, env, err := ResolveEncryptedEnvOptions(&cfg) + // filesystem. + unencryptedFS := opts.FS + fileRegistry, encryptionEnv, err := ResolveEncryptedEnvOptions(&cfg.StorageConfig, opts.FS, opts.ReadOnly) if err != nil { return nil, err } + if encryptionEnv != nil { + opts.FS = encryptionEnv.FS + } // If no logger was passed, the previous call to `EnsureDefaults` on - // `cfg.Opts` will set the logger to pebble's `DefaultLogger`. In + // `opts` will set the logger to pebble's `DefaultLogger`. In // crdb, we want pebble-related logs to go to the storage channel, // so we update the logger here accordingly. - if cfg.Opts.Logger == nil || cfg.Opts.Logger == pebble.DefaultLogger { - cfg.Opts.Logger = pebbleLogger{ + if opts.Logger == nil || opts.Logger == pebble.DefaultLogger { + opts.Logger = pebbleLogger{ ctx: logCtx, depth: 1, } @@ -930,7 +931,7 @@ func NewPebble(ctx context.Context, cfg PebbleConfig) (p *Pebble, err error) { // Establish the emergency ballast if we can. If there's not sufficient // disk space, the ballast will be reestablished from Capacity when the // store's capacity is queried periodically. - if !cfg.Opts.ReadOnly { + if !opts.ReadOnly { du, err := unencryptedFS.GetDiskUsage(cfg.Dir) // If the FS is an in-memory FS, GetDiskUsage returns // vfs.ErrUnsupported and we skip ballast creation. @@ -942,16 +943,16 @@ func NewPebble(ctx context.Context, cfg PebbleConfig) (p *Pebble, err error) { return nil, errors.Wrap(err, "resizing ballast") } if resized { - cfg.Opts.Logger.Infof("resized ballast %s to size %s", + opts.Logger.Infof("resized ballast %s to size %s", ballastPath, humanizeutil.IBytes(cfg.BallastSize)) } } } - storeProps := computeStoreProperties(ctx, cfg.Dir, cfg.Opts.ReadOnly, env != nil /* encryptionEnabled */) + storeProps := computeStoreProperties(ctx, cfg.Dir, opts.ReadOnly, encryptionEnv != nil /* encryptionEnabled */) p = &Pebble{ - readOnly: cfg.Opts.ReadOnly, + readOnly: opts.ReadOnly, path: cfg.Dir, auxDir: auxDir, ballastPath: ballastPath, @@ -960,11 +961,11 @@ func NewPebble(ctx context.Context, cfg PebbleConfig) (p *Pebble, err error) { attrs: cfg.Attrs, properties: storeProps, settings: cfg.Settings, - encryption: env, + encryption: encryptionEnv, fileRegistry: fileRegistry, - fs: cfg.Opts.FS, + fs: opts.FS, unencryptedFS: unencryptedFS, - logger: cfg.Opts.Logger, + logger: opts.Logger, logCtx: logCtx, storeIDPebbleLog: storeIDContainer, closer: filesystemCloser, @@ -978,8 +979,8 @@ func NewPebble(ctx context.Context, cfg PebbleConfig) (p *Pebble, err error) { // the compactions concurrency which has already been set and allow us // to update the compactionConcurrency on the fly by changing the // Pebble.atomic.compactionConcurrency variable. - p.atomic.compactionConcurrency = uint64(cfg.Opts.MaxConcurrentCompactions()) - cfg.Opts.MaxConcurrentCompactions = func() int { + p.atomic.compactionConcurrency = uint64(opts.MaxConcurrentCompactions()) + opts.MaxConcurrentCompactions = func() int { return int(atomic.LoadUint64(&p.atomic.compactionConcurrency)) } @@ -1015,7 +1016,7 @@ func NewPebble(ctx context.Context, cfg PebbleConfig) (p *Pebble, err error) { ) p.eventListener = &el - cfg.Opts.EventListener = &el + opts.EventListener = &el p.wrappedIntentWriter = wrapIntentWriter(p) // Read the current store cluster version. @@ -1051,10 +1052,10 @@ func NewPebble(ctx context.Context, cfg PebbleConfig) (p *Pebble, err error) { } if WorkloadCollectorEnabled { - p.replayer.Attach(cfg.Opts) + p.replayer.Attach(opts) } - db, err := pebble.Open(cfg.StorageConfig.Dir, cfg.Opts) + db, err := pebble.Open(cfg.StorageConfig.Dir, opts) if err != nil { return nil, err } diff --git a/pkg/storage/pebble_file_registry.go b/pkg/storage/pebble_file_registry.go index 89f26a7ba936..c368586dabe7 100644 --- a/pkg/storage/pebble_file_registry.go +++ b/pkg/storage/pebble_file_registry.go @@ -90,8 +90,8 @@ const ( // CheckNoRegistryFile checks that no registry file currently exists. // CheckNoRegistryFile should be called if the file registry will not be used. -func (r *PebbleFileRegistry) CheckNoRegistryFile() error { - filename, err := atomicfs.ReadMarker(r.FS, r.DBDir, registryMarkerName) +func CheckNoRegistryFile(fs vfs.FS, dbDir string) error { + filename, err := atomicfs.ReadMarker(fs, dbDir, registryMarkerName) if oserror.IsNotExist(err) { // ReadMarker may return oserror.IsNotExist if the data // directory does not exist. diff --git a/pkg/storage/pebble_file_registry_test.go b/pkg/storage/pebble_file_registry_test.go index a0f6bc8a783e..c86c32f4e15f 100644 --- a/pkg/storage/pebble_file_registry_test.go +++ b/pkg/storage/pebble_file_registry_test.go @@ -190,12 +190,11 @@ func TestFileRegistryCheckNoFile(t *testing.T) { mem := vfs.NewMem() fileEntry := &enginepb.FileEntry{EnvType: enginepb.EnvType_Data, EncryptionSettings: []byte("foo")} + require.NoError(t, CheckNoRegistryFile(mem, "" /* dbDir */)) registry := &PebbleFileRegistry{FS: mem} - require.NoError(t, registry.CheckNoRegistryFile()) require.NoError(t, registry.Load()) require.NoError(t, registry.SetFileEntry("/foo", fileEntry)) - registry = &PebbleFileRegistry{FS: mem} - require.Error(t, registry.CheckNoRegistryFile()) + require.Error(t, CheckNoRegistryFile(mem, "" /* dbDir */)) } func TestFileRegistryElideUnencrypted(t *testing.T) { @@ -293,8 +292,8 @@ func TestFileRegistryRecordsReadAndWrite(t *testing.T) { } // Create a file registry and add entries for a few files. + require.NoError(t, CheckNoRegistryFile(mem, "" /* dbDir */)) registry1 := &PebbleFileRegistry{FS: mem} - require.NoError(t, registry1.CheckNoRegistryFile()) require.NoError(t, registry1.Load()) for filename, entry := range files { require.NoError(t, registry1.SetFileEntry(filename, entry)) @@ -332,10 +331,7 @@ func TestFileRegistry(t *testing.T) { switch d.Cmd { case "check-no-registry-file": require.Nil(t, registry) - registry = &PebbleFileRegistry{FS: fs} - err := registry.CheckNoRegistryFile() - registry = nil - if err == nil { + if err := CheckNoRegistryFile(fs, "" /* dbDir */); err == nil { fmt.Fprintf(&buf, "OK\n") } else { fmt.Fprintf(&buf, "Error: %s\n", err) From daf2dadad2f1d241665290139d80b2c94909c00b Mon Sep 17 00:00:00 2001 From: Tobias Grieger Date: Wed, 8 Feb 2023 13:53:24 +0100 Subject: [PATCH 02/16] kvserver: de-flake TestReplicaProbeRequest Chanced upon this failure mode in unrelated PR #96781. Epic: none Release note: None --- pkg/kv/kvserver/replica_probe_test.go | 25 ++++++++++++++++--------- 1 file changed, 16 insertions(+), 9 deletions(-) diff --git a/pkg/kv/kvserver/replica_probe_test.go b/pkg/kv/kvserver/replica_probe_test.go index 2247cc23fba7..9cf1d784b2fa 100644 --- a/pkg/kv/kvserver/replica_probe_test.go +++ b/pkg/kv/kvserver/replica_probe_test.go @@ -120,17 +120,24 @@ func TestReplicaProbeRequest(t *testing.T) { // stack, with both routing policies. for _, srv := range tc.Servers { db := srv.DB() - { + for _, policy := range []roachpb.RoutingPolicy{ + roachpb.RoutingPolicy_LEASEHOLDER, + roachpb.RoutingPolicy_NEAREST, + } { var b kv.Batch b.AddRawRequest(probeReq) - b.Header.RoutingPolicy = roachpb.RoutingPolicy_LEASEHOLDER - require.NoError(t, db.Run(ctx, &b)) - } - { - var b kv.Batch - b.AddRawRequest(probeReq) - b.Header.RoutingPolicy = roachpb.RoutingPolicy_NEAREST - require.NoError(t, db.Run(ctx, &b)) + b.Header.RoutingPolicy = policy + err := db.Run(ctx, &b) + if errors.HasType(err, (*roachpb.AmbiguousResultError)(nil)) { + // Rare but it can happen that we're proposing on a replica + // that is just about to get a snapshot. In that case we'll + // get: + // + // result is ambiguous: unable to determine whether command was applied via snapshot + t.Logf("ignoring: %s", err) + err = nil + } + require.NoError(t, err) } } // Check expected number of probes seen on each Replica in the apply loop. From 00038b732937f3649842f8e9e1c5e1da173fac12 Mon Sep 17 00:00:00 2001 From: Tobias Grieger Date: Tue, 24 Jan 2023 14:48:18 +0100 Subject: [PATCH 03/16] kvserver: remove assertion in preDestroyRaftMuLocked There are four callers to this method, and in all of them it is obvious that the replica is marked as destroyed. The invariant this code was checking is important, but also this code primarily stages updates into a batch. It is not actually necessary to have the replica marked as destroyed at this point yet, though in practice we do do it. What you really need is to prevent further modification of the data, but that still allows reads to be served until we actually commit the batch. Removing the dependency on `*Replica` will allow a series of simplifying refactors, at the end of which the end of the life of a Replica should be a little less nested, and a little more clear. There'll be a better place to assert this invariant as the work unfolds. Epic: CRDB-220 Release note: None --- pkg/kv/kvserver/replica_destroy.go | 12 +----------- 1 file changed, 1 insertion(+), 11 deletions(-) diff --git a/pkg/kv/kvserver/replica_destroy.go b/pkg/kv/kvserver/replica_destroy.go index bfda6f078962..8e502bfe319d 100644 --- a/pkg/kv/kvserver/replica_destroy.go +++ b/pkg/kv/kvserver/replica_destroy.go @@ -78,17 +78,7 @@ func (r *Replica) preDestroyRaftMuLocked( nextReplicaID roachpb.ReplicaID, opts clearRangeDataOptions, ) error { - r.mu.RLock() - desc := r.descRLocked() - removed := r.mu.destroyStatus.Removed() - r.mu.RUnlock() - - // The replica must be marked as destroyed before its data is removed. If - // not, we risk new commands being accepted and observing the missing data. - if !removed { - log.Fatalf(ctx, "replica not marked as destroyed before call to preDestroyRaftMuLocked: %v", r) - } - err := clearRangeData(desc.RangeID, reader, writer, opts) + err := clearRangeData(r.RangeID, reader, writer, opts) if err != nil { return err } From 8a89e2d31a776cb855e5197bb2461210cc77ebf1 Mon Sep 17 00:00:00 2001 From: Tobias Grieger Date: Tue, 24 Jan 2023 14:51:35 +0100 Subject: [PATCH 04/16] kvserver: inline writeTombstoneKey Prep for more refactoring. Epic: CRDB-220 Release note: None --- pkg/kv/kvserver/replica_destroy.go | 11 +---------- 1 file changed, 1 insertion(+), 10 deletions(-) diff --git a/pkg/kv/kvserver/replica_destroy.go b/pkg/kv/kvserver/replica_destroy.go index 8e502bfe319d..a97f10985936 100644 --- a/pkg/kv/kvserver/replica_destroy.go +++ b/pkg/kv/kvserver/replica_destroy.go @@ -229,16 +229,7 @@ func (r *Replica) setTombstoneKey( nextReplicaID = externalNextReplicaID } r.mu.Unlock() - return writeTombstoneKey(ctx, writer, r.RangeID, nextReplicaID) -} - -func writeTombstoneKey( - ctx context.Context, - writer storage.Writer, - rangeID roachpb.RangeID, - nextReplicaID roachpb.ReplicaID, -) error { - tombstoneKey := keys.RangeTombstoneKey(rangeID) + tombstoneKey := keys.RangeTombstoneKey(r.RangeID) tombstone := &roachpb.RangeTombstone{ NextReplicaID: nextReplicaID, } From 5984059d78792887acd00bf780c64ef6bb726060 Mon Sep 17 00:00:00 2001 From: Tobias Grieger Date: Tue, 24 Jan 2023 15:03:31 +0100 Subject: [PATCH 05/16] kvserver: improve assertions in setTombstoneKey It now checks in two ways that we're not regressing: 1. tombstone actually has to cover the replica's replicaID 2. tombstone itself must not regress. I had verified that all call sites at least claim to not regress and also conceptually I'm not aware of any that would want/need to rely on being forwarded by the old code. But there are lots of callers, so better to fail loudly in that case. If you see this commit, the `kvserver` tests have passed. Epic: CRDB-220 Release note: None --- pkg/kv/kvserver/replica_destroy.go | 53 ++++++++++++++++++++++++------ 1 file changed, 43 insertions(+), 10 deletions(-) diff --git a/pkg/kv/kvserver/replica_destroy.go b/pkg/kv/kvserver/replica_destroy.go index a97f10985936..2cd8d8f31e26 100644 --- a/pkg/kv/kvserver/replica_destroy.go +++ b/pkg/kv/kvserver/replica_destroy.go @@ -23,6 +23,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/timeutil" + "github.com/cockroachdb/errors" ) // DestroyReason indicates if a replica is alive, destroyed, corrupted or pending destruction. @@ -88,7 +89,7 @@ func (r *Replica) preDestroyRaftMuLocked( // NB: Legacy tombstones (which are in the replicated key space) are wiped // in clearRangeData, but that's OK since we're writing a new one in the same // batch (and in particular, sequenced *after* the wipe). - return r.setTombstoneKey(ctx, writer, nextReplicaID) + return r.setTombstoneKey(ctx, reader, writer, nextReplicaID) } func (r *Replica) postDestroyRaftMuLocked(ctx context.Context, ms enginepb.MVCCStats) error { @@ -221,19 +222,51 @@ func (r *Replica) disconnectReplicationRaftMuLocked(ctx context.Context) { // ID that it hasn't yet received a RangeDescriptor for if it receives raft // requests for that replica ID (as seen in #14231). func (r *Replica) setTombstoneKey( - ctx context.Context, writer storage.Writer, externalNextReplicaID roachpb.ReplicaID, + ctx context.Context, + reader storage.Reader, + writer storage.Writer, + externalNextReplicaID roachpb.ReplicaID, ) error { - r.mu.Lock() - nextReplicaID := r.mu.state.Desc.NextReplicaID - if externalNextReplicaID > nextReplicaID { - nextReplicaID = externalNextReplicaID + { + // TODO(this PR): this assertion is temporary. We'll remove it along with + // the dependency on *Replica. But having it in this commit and passing + // tests is a good sanity check. + + r.mu.Lock() + replReplicaID := r.replicaID + r.mu.Unlock() + + if replReplicaID >= externalNextReplicaID { + err := errors.AssertionFailedf( + "attempt to set tombstone at replicaID %d but this leaves replicaID %d alive", + externalNextReplicaID, replReplicaID, + ) + log.Fatalf(ctx, "%s", err) + return err // unreachable + } } - r.mu.Unlock() + tombstoneKey := keys.RangeTombstoneKey(r.RangeID) - tombstone := &roachpb.RangeTombstone{ - NextReplicaID: nextReplicaID, + + // Assert that the provided tombstone moves the existing one strictly forward. + // Failure to do so indicates that something is going wrong in the replica + // lifecycle. + { + var tombstone roachpb.RangeTombstone + if _, err := storage.MVCCGetProto( + ctx, reader, tombstoneKey, hlc.Timestamp{}, &tombstone, storage.MVCCGetOptions{}, + ); err != nil { + return err + } + if tombstone.NextReplicaID >= externalNextReplicaID { + return errors.AssertionFailedf( + "cannot rewind tombstone from %d to %d", tombstone.NextReplicaID, externalNextReplicaID, + ) + } } + + tombstone := roachpb.RangeTombstone{NextReplicaID: externalNextReplicaID} // "Blind" because ms == nil and timestamp.IsEmpty(). return storage.MVCCBlindPutProto(ctx, writer, nil, tombstoneKey, - hlc.Timestamp{}, hlc.ClockTimestamp{}, tombstone, nil) + hlc.Timestamp{}, hlc.ClockTimestamp{}, &tombstone, nil) } From 845ec927be94d06ec84311506990f0f757b3c545 Mon Sep 17 00:00:00 2001 From: Tobias Grieger Date: Mon, 6 Feb 2023 17:01:05 +0100 Subject: [PATCH 06/16] logstore: drop `found` param from LoadRaftReplicaID It now has to be there, so turn this into an assertion failure. See #95513. Epic: CRDB-220 Release note: None --- pkg/kv/kvserver/client_store_test.go | 6 ++---- pkg/kv/kvserver/kvstorage/replica_state.go | 8 +++----- pkg/kv/kvserver/logstore/stateloader.go | 13 ++++++++++--- pkg/kv/kvserver/loqrecovery/recovery_env_test.go | 2 +- pkg/kv/kvserver/replica.go | 5 +---- pkg/kv/kvserver/store_test.go | 5 ++--- 6 files changed, 19 insertions(+), 20 deletions(-) diff --git a/pkg/kv/kvserver/client_store_test.go b/pkg/kv/kvserver/client_store_test.go index 6e997152004d..b7fe2fbaa2a2 100644 --- a/pkg/kv/kvserver/client_store_test.go +++ b/pkg/kv/kvserver/client_store_test.go @@ -91,8 +91,7 @@ func TestStoreRaftReplicaID(t *testing.T) { require.NoError(t, err) repl, err := store.GetReplica(desc.RangeID) require.NoError(t, err) - replicaID, found, err := stateloader.Make(desc.RangeID).LoadRaftReplicaID(ctx, store.Engine()) - require.True(t, found) + replicaID, err := stateloader.Make(desc.RangeID).LoadRaftReplicaID(ctx, store.Engine()) require.NoError(t, err) require.Equal(t, repl.ReplicaID(), replicaID.ReplicaID) @@ -102,9 +101,8 @@ func TestStoreRaftReplicaID(t *testing.T) { require.NoError(t, err) rhsRepl, err := store.GetReplica(rhsDesc.RangeID) require.NoError(t, err) - rhsReplicaID, found, err := + rhsReplicaID, err := stateloader.Make(rhsDesc.RangeID).LoadRaftReplicaID(ctx, store.Engine()) - require.True(t, found) require.NoError(t, err) require.Equal(t, rhsRepl.ReplicaID(), rhsReplicaID.ReplicaID) } diff --git a/pkg/kv/kvserver/kvstorage/replica_state.go b/pkg/kv/kvserver/kvstorage/replica_state.go index cf45486046c4..975f64006445 100644 --- a/pkg/kv/kvserver/kvstorage/replica_state.go +++ b/pkg/kv/kvserver/kvstorage/replica_state.go @@ -46,13 +46,11 @@ func LoadReplicaState( replicaID roachpb.ReplicaID, ) (LoadedReplicaState, error) { sl := stateloader.Make(desc.RangeID) - id, found, err := sl.LoadRaftReplicaID(ctx, eng) + id, err := sl.LoadRaftReplicaID(ctx, eng) if err != nil { return LoadedReplicaState{}, err - } else if !found { - return LoadedReplicaState{}, errors.AssertionFailedf( - "r%d: RaftReplicaID not found", desc.RangeID) - } else if loaded := id.ReplicaID; loaded != replicaID { + } + if loaded := id.ReplicaID; loaded != replicaID { return LoadedReplicaState{}, errors.AssertionFailedf( "r%d: loaded RaftReplicaID %d does not match %d", desc.RangeID, loaded, replicaID) } diff --git a/pkg/kv/kvserver/logstore/stateloader.go b/pkg/kv/kvserver/logstore/stateloader.go index 3d05daaebae7..e36ca0639208 100644 --- a/pkg/kv/kvserver/logstore/stateloader.go +++ b/pkg/kv/kvserver/logstore/stateloader.go @@ -203,8 +203,15 @@ func (sl StateLoader) SetRaftReplicaID( // LoadRaftReplicaID loads the RaftReplicaID. func (sl StateLoader) LoadRaftReplicaID( ctx context.Context, reader storage.Reader, -) (replicaID roachpb.RaftReplicaID, found bool, err error) { - found, err = storage.MVCCGetProto(ctx, reader, sl.RaftReplicaIDKey(), +) (*roachpb.RaftReplicaID, error) { + var replicaID roachpb.RaftReplicaID + found, err := storage.MVCCGetProto(ctx, reader, sl.RaftReplicaIDKey(), hlc.Timestamp{}, &replicaID, storage.MVCCGetOptions{}) - return + if err != nil { + return nil, err + } + if !found { + return nil, errors.AssertionFailedf("no replicaID persisted") + } + return &replicaID, nil } diff --git a/pkg/kv/kvserver/loqrecovery/recovery_env_test.go b/pkg/kv/kvserver/loqrecovery/recovery_env_test.go index 8bcbe4cc9882..a3d7122266b1 100644 --- a/pkg/kv/kvserver/loqrecovery/recovery_env_test.go +++ b/pkg/kv/kvserver/loqrecovery/recovery_env_test.go @@ -747,7 +747,7 @@ func (e *quorumRecoveryEnv) handleDumpStore(t *testing.T, d datadriven.TestData) descriptorViews = append(descriptorViews, descriptorView(desc)) sl := stateloader.Make(desc.RangeID) - raftReplicaID, _, err := sl.LoadRaftReplicaID(ctx, store.engine) + raftReplicaID, err := sl.LoadRaftReplicaID(ctx, store.engine) if err != nil { t.Fatalf("failed to load Raft replica ID: %v", err) } diff --git a/pkg/kv/kvserver/replica.go b/pkg/kv/kvserver/replica.go index 270b137fd799..d6f8e77d6b02 100644 --- a/pkg/kv/kvserver/replica.go +++ b/pkg/kv/kvserver/replica.go @@ -1400,13 +1400,10 @@ func (r *Replica) assertStateRaftMuLockedReplicaMuRLocked( log.Fatalf(ctx, "replica's replicaID %d diverges from descriptor %+v", r.replicaID, r.mu.state.Desc) } } - diskReplID, found, err := r.mu.stateLoader.LoadRaftReplicaID(ctx, reader) + diskReplID, err := r.mu.stateLoader.LoadRaftReplicaID(ctx, reader) if err != nil { log.Fatalf(ctx, "%s", err) } - if !found { - log.Fatalf(ctx, "no replicaID persisted") - } if diskReplID.ReplicaID != r.replicaID { log.Fatalf(ctx, "disk replicaID %d does not match in-mem %d", diskReplID, r.replicaID) } diff --git a/pkg/kv/kvserver/store_test.go b/pkg/kv/kvserver/store_test.go index e85e6f805a85..ef8bb4371ca3 100644 --- a/pkg/kv/kvserver/store_test.go +++ b/pkg/kv/kvserver/store_test.go @@ -3784,10 +3784,9 @@ func TestStoreGetOrCreateReplicaWritesRaftReplicaID(t *testing.T) { }) require.NoError(t, err) require.True(t, created) - replicaID, found, err := repl.mu.stateLoader.LoadRaftReplicaID(ctx, tc.store.Engine()) + replicaID, err := repl.mu.stateLoader.LoadRaftReplicaID(ctx, tc.store.Engine()) require.NoError(t, err) - require.True(t, found) - require.Equal(t, roachpb.RaftReplicaID{ReplicaID: 7}, replicaID) + require.Equal(t, &roachpb.RaftReplicaID{ReplicaID: 7}, replicaID) } func BenchmarkStoreGetReplica(b *testing.B) { From 6c89f223c61773d7f62e46bb7ea46de1ea430d71 Mon Sep 17 00:00:00 2001 From: Tobias Grieger Date: Tue, 24 Jan 2023 15:19:17 +0100 Subject: [PATCH 07/16] kvserver: avoid *Replica in setTombstoneKey Make this all about engine interactions. This removes the first assertion added in the last commit (tests had passed with it) and replaces it with a corresponding assertion on the persisted ReplicaID. Epic: CRDB-220 Release note: None --- pkg/kv/kvserver/replica_destroy.go | 35 ++++++++++-------------------- 1 file changed, 12 insertions(+), 23 deletions(-) diff --git a/pkg/kv/kvserver/replica_destroy.go b/pkg/kv/kvserver/replica_destroy.go index 2cd8d8f31e26..98414702a8d3 100644 --- a/pkg/kv/kvserver/replica_destroy.go +++ b/pkg/kv/kvserver/replica_destroy.go @@ -17,6 +17,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/apply" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/logstore" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" @@ -79,17 +80,23 @@ func (r *Replica) preDestroyRaftMuLocked( nextReplicaID roachpb.ReplicaID, opts clearRangeDataOptions, ) error { - err := clearRangeData(r.RangeID, reader, writer, opts) + diskReplicaID, err := logstore.NewStateLoader(r.RangeID).LoadRaftReplicaID(ctx, reader) if err != nil { return err } + if diskReplicaID.ReplicaID >= nextReplicaID { + return errors.AssertionFailedf("replica r%d/%d must not survive its own tombstone", r.RangeID, diskReplicaID) + } + if err := clearRangeData(r.RangeID, reader, writer, opts); err != nil { + return err + } // Save a tombstone to ensure that replica IDs never get reused. // // NB: Legacy tombstones (which are in the replicated key space) are wiped // in clearRangeData, but that's OK since we're writing a new one in the same // batch (and in particular, sequenced *after* the wipe). - return r.setTombstoneKey(ctx, reader, writer, nextReplicaID) + return setTombstoneKey(ctx, r.RangeID, reader, writer, nextReplicaID) } func (r *Replica) postDestroyRaftMuLocked(ctx context.Context, ms enginepb.MVCCStats) error { @@ -221,32 +228,14 @@ func (r *Replica) disconnectReplicationRaftMuLocked(ctx context.Context) { // We have to be careful to set the right key, since a replica can be using an // ID that it hasn't yet received a RangeDescriptor for if it receives raft // requests for that replica ID (as seen in #14231). -func (r *Replica) setTombstoneKey( +func setTombstoneKey( ctx context.Context, + rangeID roachpb.RangeID, reader storage.Reader, writer storage.Writer, externalNextReplicaID roachpb.ReplicaID, ) error { - { - // TODO(this PR): this assertion is temporary. We'll remove it along with - // the dependency on *Replica. But having it in this commit and passing - // tests is a good sanity check. - - r.mu.Lock() - replReplicaID := r.replicaID - r.mu.Unlock() - - if replReplicaID >= externalNextReplicaID { - err := errors.AssertionFailedf( - "attempt to set tombstone at replicaID %d but this leaves replicaID %d alive", - externalNextReplicaID, replReplicaID, - ) - log.Fatalf(ctx, "%s", err) - return err // unreachable - } - } - - tombstoneKey := keys.RangeTombstoneKey(r.RangeID) + tombstoneKey := keys.RangeTombstoneKey(rangeID) // Assert that the provided tombstone moves the existing one strictly forward. // Failure to do so indicates that something is going wrong in the replica From f929c7ad31082061402b87e4f1808c36563ad1ff Mon Sep 17 00:00:00 2001 From: Tobias Grieger Date: Mon, 6 Feb 2023 17:11:59 +0100 Subject: [PATCH 08/16] kvserver: inline and remove setTombstoneKey Epic: CRDB-220 Release note: None --- pkg/kv/kvserver/replica_destroy.go | 77 ++++++++++++------------------ 1 file changed, 31 insertions(+), 46 deletions(-) diff --git a/pkg/kv/kvserver/replica_destroy.go b/pkg/kv/kvserver/replica_destroy.go index 98414702a8d3..29b5aeb2e9fc 100644 --- a/pkg/kv/kvserver/replica_destroy.go +++ b/pkg/kv/kvserver/replica_destroy.go @@ -80,23 +80,47 @@ func (r *Replica) preDestroyRaftMuLocked( nextReplicaID roachpb.ReplicaID, opts clearRangeDataOptions, ) error { - diskReplicaID, err := logstore.NewStateLoader(r.RangeID).LoadRaftReplicaID(ctx, reader) + rangeID := r.RangeID + diskReplicaID, err := logstore.NewStateLoader(rangeID).LoadRaftReplicaID(ctx, reader) if err != nil { return err } if diskReplicaID.ReplicaID >= nextReplicaID { - return errors.AssertionFailedf("replica r%d/%d must not survive its own tombstone", r.RangeID, diskReplicaID) + return errors.AssertionFailedf("replica r%d/%d must not survive its own tombstone", rangeID, diskReplicaID) } - if err := clearRangeData(r.RangeID, reader, writer, opts); err != nil { + if err := clearRangeData(rangeID, reader, writer, opts); err != nil { return err } // Save a tombstone to ensure that replica IDs never get reused. // - // NB: Legacy tombstones (which are in the replicated key space) are wiped - // in clearRangeData, but that's OK since we're writing a new one in the same - // batch (and in particular, sequenced *after* the wipe). - return setTombstoneKey(ctx, r.RangeID, reader, writer, nextReplicaID) + // TODO(tbg): put this on `stateloader.StateLoader` and consolidate the + // other read of the range tombstone key (in uninited replica creation + // as well). + + tombstoneKey := keys.RangeTombstoneKey(rangeID) + + // Assert that the provided tombstone moves the existing one strictly forward. + // Failure to do so indicates that something is going wrong in the replica + // lifecycle. + { + var tombstone roachpb.RangeTombstone + if _, err := storage.MVCCGetProto( + ctx, reader, tombstoneKey, hlc.Timestamp{}, &tombstone, storage.MVCCGetOptions{}, + ); err != nil { + return err + } + if tombstone.NextReplicaID >= nextReplicaID { + return errors.AssertionFailedf( + "cannot rewind tombstone from %d to %d", tombstone.NextReplicaID, nextReplicaID, + ) + } + } + + tombstone := roachpb.RangeTombstone{NextReplicaID: nextReplicaID} + // "Blind" because ms == nil and timestamp.IsEmpty(). + return storage.MVCCBlindPutProto(ctx, writer, nil, tombstoneKey, + hlc.Timestamp{}, hlc.ClockTimestamp{}, &tombstone, nil) } func (r *Replica) postDestroyRaftMuLocked(ctx context.Context, ms enginepb.MVCCStats) error { @@ -220,42 +244,3 @@ func (r *Replica) disconnectReplicationRaftMuLocked(ctx context.Context) { } r.mu.internalRaftGroup = nil } - -// setTombstoneKey writes a tombstone to disk to ensure that replica IDs never -// get reused. It determines what the minimum next replica ID can be using -// the provided nextReplicaID and the Replica's own ID. -// -// We have to be careful to set the right key, since a replica can be using an -// ID that it hasn't yet received a RangeDescriptor for if it receives raft -// requests for that replica ID (as seen in #14231). -func setTombstoneKey( - ctx context.Context, - rangeID roachpb.RangeID, - reader storage.Reader, - writer storage.Writer, - externalNextReplicaID roachpb.ReplicaID, -) error { - tombstoneKey := keys.RangeTombstoneKey(rangeID) - - // Assert that the provided tombstone moves the existing one strictly forward. - // Failure to do so indicates that something is going wrong in the replica - // lifecycle. - { - var tombstone roachpb.RangeTombstone - if _, err := storage.MVCCGetProto( - ctx, reader, tombstoneKey, hlc.Timestamp{}, &tombstone, storage.MVCCGetOptions{}, - ); err != nil { - return err - } - if tombstone.NextReplicaID >= externalNextReplicaID { - return errors.AssertionFailedf( - "cannot rewind tombstone from %d to %d", tombstone.NextReplicaID, externalNextReplicaID, - ) - } - } - - tombstone := roachpb.RangeTombstone{NextReplicaID: externalNextReplicaID} - // "Blind" because ms == nil and timestamp.IsEmpty(). - return storage.MVCCBlindPutProto(ctx, writer, nil, tombstoneKey, - hlc.Timestamp{}, hlc.ClockTimestamp{}, &tombstone, nil) -} From 31151533d9d0002ee4ee7d0f5a23bf674c19164e Mon Sep 17 00:00:00 2001 From: Tobias Grieger Date: Mon, 6 Feb 2023 17:15:01 +0100 Subject: [PATCH 09/16] kvserver: untie preDestroyRaftMuLocked from Replica Epic: CRDB-220 Release note: None --- pkg/kv/kvserver/replica_app_batch.go | 26 +++++++++----------------- pkg/kv/kvserver/replica_destroy.go | 12 +++--------- pkg/kv/kvserver/replica_raftstorage.go | 8 +------- 3 files changed, 13 insertions(+), 33 deletions(-) diff --git a/pkg/kv/kvserver/replica_app_batch.go b/pkg/kv/kvserver/replica_app_batch.go index 8a42cda8f2ea..24e23486891d 100644 --- a/pkg/kv/kvserver/replica_app_batch.go +++ b/pkg/kv/kvserver/replica_app_batch.go @@ -334,12 +334,10 @@ func (b *replicaAppBatch) runPostAddTriggersReplicaOnly( // required for correctness, since the merge protocol should guarantee that // no new replicas of the RHS can ever be created, but it doesn't hurt to // be careful. - if err := rhsRepl.preDestroyRaftMuLocked( - ctx, b.batch, b.batch, mergedTombstoneReplicaID, clearRangeDataOptions{ - ClearReplicatedByRangeID: true, - ClearUnreplicatedByRangeID: true, - }, - ); err != nil { + if err := preDestroyRaftMuLocked(ctx, rhsRepl.RangeID, b.batch, b.batch, mergedTombstoneReplicaID, clearRangeDataOptions{ + ClearReplicatedByRangeID: true, + ClearUnreplicatedByRangeID: true, + }); err != nil { return errors.Wrapf(err, "unable to destroy replica before merge") } @@ -476,17 +474,11 @@ func (b *replicaAppBatch) runPostAddTriggersReplicaOnly( // We've set the replica's in-mem status to reflect the pending destruction // above, and preDestroyRaftMuLocked will also add a range tombstone to the // batch, so that when we commit it, the removal is finalized. - if err := b.r.preDestroyRaftMuLocked( - ctx, - b.batch, - b.batch, - change.NextReplicaID(), - clearRangeDataOptions{ - ClearReplicatedBySpan: span, - ClearReplicatedByRangeID: true, - ClearUnreplicatedByRangeID: true, - }, - ); err != nil { + if err := preDestroyRaftMuLocked(ctx, b.r.RangeID, b.batch, b.batch, change.NextReplicaID(), clearRangeDataOptions{ + ClearReplicatedBySpan: span, + ClearReplicatedByRangeID: true, + ClearUnreplicatedByRangeID: true, + }); err != nil { return errors.Wrapf(err, "unable to destroy replica before removal") } } diff --git a/pkg/kv/kvserver/replica_destroy.go b/pkg/kv/kvserver/replica_destroy.go index 29b5aeb2e9fc..2e7fcf34881d 100644 --- a/pkg/kv/kvserver/replica_destroy.go +++ b/pkg/kv/kvserver/replica_destroy.go @@ -73,14 +73,14 @@ func (s destroyStatus) Removed() bool { // don't know the current replica ID. const mergedTombstoneReplicaID roachpb.ReplicaID = math.MaxInt32 -func (r *Replica) preDestroyRaftMuLocked( +func preDestroyRaftMuLocked( ctx context.Context, + rangeID roachpb.RangeID, reader storage.Reader, writer storage.Writer, nextReplicaID roachpb.ReplicaID, opts clearRangeDataOptions, ) error { - rangeID := r.RangeID diskReplicaID, err := logstore.NewStateLoader(rangeID).LoadRaftReplicaID(ctx, reader) if err != nil { return err @@ -179,13 +179,7 @@ func (r *Replica) destroyRaftMuLocked(ctx context.Context, nextReplicaID roachpb ClearReplicatedByRangeID: inited, ClearUnreplicatedByRangeID: true, } - if err := r.preDestroyRaftMuLocked( - ctx, - r.Engine(), - batch, - nextReplicaID, - opts, - ); err != nil { + if err := preDestroyRaftMuLocked(ctx, r.RangeID, r.Engine(), batch, nextReplicaID, opts); err != nil { return err } preTime := timeutil.Now() diff --git a/pkg/kv/kvserver/replica_raftstorage.go b/pkg/kv/kvserver/replica_raftstorage.go index 91ac402bb486..84098e86f71a 100644 --- a/pkg/kv/kvserver/replica_raftstorage.go +++ b/pkg/kv/kvserver/replica_raftstorage.go @@ -819,13 +819,7 @@ func clearSubsumedReplicaDiskData( ClearUnreplicatedByRangeID: true, MustUseClearRange: true, } - if err := sr.preDestroyRaftMuLocked( - ctx, - reader, - &subsumedReplSST, - subsumedNextReplicaID, - opts, - ); err != nil { + if err := preDestroyRaftMuLocked(ctx, sr.RangeID, reader, &subsumedReplSST, subsumedNextReplicaID, opts); err != nil { subsumedReplSST.Close() return err } From c299f8defdd2cdcf14694f24357636cfcee89a82 Mon Sep 17 00:00:00 2001 From: Tobias Grieger Date: Mon, 6 Feb 2023 17:18:24 +0100 Subject: [PATCH 10/16] kvserver,kvstorage: move clearRangeDataOptions This is in preparation for moving the storage interactions of replica removal, too. Epic: CRDB-220 Release note: None --- pkg/kv/kvserver/kvstorage/BUILD.bazel | 1 + pkg/kv/kvserver/kvstorage/destroy.go | 34 ++++++++++++++++++++++++++ pkg/kv/kvserver/replica_app_batch.go | 5 ++-- pkg/kv/kvserver/replica_destroy.go | 5 ++-- pkg/kv/kvserver/replica_raftstorage.go | 28 +++++---------------- pkg/kv/kvserver/store_split.go | 2 +- 6 files changed, 48 insertions(+), 27 deletions(-) create mode 100644 pkg/kv/kvserver/kvstorage/destroy.go diff --git a/pkg/kv/kvserver/kvstorage/BUILD.bazel b/pkg/kv/kvserver/kvstorage/BUILD.bazel index 5887d87140f9..dd127102c6df 100644 --- a/pkg/kv/kvserver/kvstorage/BUILD.bazel +++ b/pkg/kv/kvserver/kvstorage/BUILD.bazel @@ -5,6 +5,7 @@ go_library( name = "kvstorage", srcs = [ "cluster_version.go", + "destroy.go", "doc.go", "init.go", "replica_state.go", diff --git a/pkg/kv/kvserver/kvstorage/destroy.go b/pkg/kv/kvserver/kvstorage/destroy.go new file mode 100644 index 000000000000..86e5a32172a8 --- /dev/null +++ b/pkg/kv/kvserver/kvstorage/destroy.go @@ -0,0 +1,34 @@ +// Copyright 2023 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 kvstorage + +import "github.com/cockroachdb/cockroach/pkg/roachpb" + +// ClearRangeDataOptions specify which parts of a Replica are to be destroyed. +type ClearRangeDataOptions struct { + // ClearReplicatedByRangeID indicates that replicated RangeID-based keys + // (abort span, etc) should be removed. + ClearReplicatedByRangeID bool + // ClearUnreplicatedByRangeID indicates that unreplicated RangeID-based keys + // (logstore state incl. HardState, etc) should be removed. + ClearUnreplicatedByRangeID bool + // ClearReplicatedBySpan causes the state machine data (i.e. the replicated state + // for the given RSpan) that is key-addressable (i.e. range descriptor, user keys, + // locks) to be removed. No data is removed if this is the zero span. + ClearReplicatedBySpan roachpb.RSpan + + // If MustUseClearRange is true, a Pebble range tombstone will always be used + // to clear the key spans (unless empty). This is typically used when we need + // to write additional keys to an SST after this clear, e.g. a replica + // tombstone, since keys must be written in order. When this is false, a + // heuristic will be used instead. + MustUseClearRange bool +} diff --git a/pkg/kv/kvserver/replica_app_batch.go b/pkg/kv/kvserver/replica_app_batch.go index 24e23486891d..ac1446574f32 100644 --- a/pkg/kv/kvserver/replica_app_batch.go +++ b/pkg/kv/kvserver/replica_app_batch.go @@ -18,6 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvadmission" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvstorage" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" @@ -334,7 +335,7 @@ func (b *replicaAppBatch) runPostAddTriggersReplicaOnly( // required for correctness, since the merge protocol should guarantee that // no new replicas of the RHS can ever be created, but it doesn't hurt to // be careful. - if err := preDestroyRaftMuLocked(ctx, rhsRepl.RangeID, b.batch, b.batch, mergedTombstoneReplicaID, clearRangeDataOptions{ + if err := preDestroyRaftMuLocked(ctx, rhsRepl.RangeID, b.batch, b.batch, mergedTombstoneReplicaID, kvstorage.ClearRangeDataOptions{ ClearReplicatedByRangeID: true, ClearUnreplicatedByRangeID: true, }); err != nil { @@ -474,7 +475,7 @@ func (b *replicaAppBatch) runPostAddTriggersReplicaOnly( // We've set the replica's in-mem status to reflect the pending destruction // above, and preDestroyRaftMuLocked will also add a range tombstone to the // batch, so that when we commit it, the removal is finalized. - if err := preDestroyRaftMuLocked(ctx, b.r.RangeID, b.batch, b.batch, change.NextReplicaID(), clearRangeDataOptions{ + if err := preDestroyRaftMuLocked(ctx, b.r.RangeID, b.batch, b.batch, change.NextReplicaID(), kvstorage.ClearRangeDataOptions{ ClearReplicatedBySpan: span, ClearReplicatedByRangeID: true, ClearUnreplicatedByRangeID: true, diff --git a/pkg/kv/kvserver/replica_destroy.go b/pkg/kv/kvserver/replica_destroy.go index 2e7fcf34881d..e908e198b717 100644 --- a/pkg/kv/kvserver/replica_destroy.go +++ b/pkg/kv/kvserver/replica_destroy.go @@ -17,6 +17,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/apply" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvstorage" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/logstore" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" @@ -79,7 +80,7 @@ func preDestroyRaftMuLocked( reader storage.Reader, writer storage.Writer, nextReplicaID roachpb.ReplicaID, - opts clearRangeDataOptions, + opts kvstorage.ClearRangeDataOptions, ) error { diskReplicaID, err := logstore.NewStateLoader(rangeID).LoadRaftReplicaID(ctx, reader) if err != nil { @@ -169,7 +170,7 @@ func (r *Replica) destroyRaftMuLocked(ctx context.Context, nextReplicaID roachpb desc := r.Desc() inited := desc.IsInitialized() - opts := clearRangeDataOptions{ + opts := kvstorage.ClearRangeDataOptions{ ClearReplicatedBySpan: desc.RSpan(), // zero if !inited // TODO(tbg): if it's uninitialized, we might as well clear // the replicated state because there isn't any. This seems diff --git a/pkg/kv/kvserver/replica_raftstorage.go b/pkg/kv/kvserver/replica_raftstorage.go index 84098e86f71a..d6ca5d450016 100644 --- a/pkg/kv/kvserver/replica_raftstorage.go +++ b/pkg/kv/kvserver/replica_raftstorage.go @@ -16,6 +16,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvstorage" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/logstore" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/rditer" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/readsummary" @@ -396,26 +397,6 @@ func (r *Replica) updateRangeInfo(ctx context.Context, desc *roachpb.RangeDescri return nil } -type clearRangeDataOptions struct { - // ClearReplicatedByRangeID indicates that replicated RangeID-based keys - // (abort span, etc) should be removed. - ClearReplicatedByRangeID bool - // ClearUnreplicatedByRangeID indicates that unreplicated RangeID-based keys - // (logstore state incl. HardState, etc) should be removed. - ClearUnreplicatedByRangeID bool - // ClearReplicatedBySpan causes the state machine data (i.e. the replicated state - // for the given RSpan) that is key-addressable (i.e. range descriptor, user keys, - // locks) to be removed. No data is removed if this is the zero span. - ClearReplicatedBySpan roachpb.RSpan - - // If MustUseClearRange is true, a Pebble range tombstone will always be used - // to clear the key spans (unless empty). This is typically used when we need - // to write additional keys to an SST after this clear, e.g. a replica - // tombstone, since keys must be written in order. When this is false, a - // heuristic will be used instead. - MustUseClearRange bool -} - // clearRangeData clears the data associated with a range descriptor selected // by the provided clearRangeDataOptions. // @@ -423,7 +404,10 @@ type clearRangeDataOptions struct { // "CRDB Range" and "storage.ClearRange" context in the setting of this method could // be confusing. func clearRangeData( - rangeID roachpb.RangeID, reader storage.Reader, writer storage.Writer, opts clearRangeDataOptions, + rangeID roachpb.RangeID, + reader storage.Reader, + writer storage.Writer, + opts kvstorage.ClearRangeDataOptions, ) error { keySpans := rditer.Select(rangeID, rditer.SelectOpts{ ReplicatedBySpan: opts.ClearReplicatedBySpan, @@ -814,7 +798,7 @@ func clearSubsumedReplicaDiskData( // NOTE: We set mustClearRange to true because we are setting // RangeTombstoneKey. Since Clears and Puts need to be done in increasing // order of keys, it is not safe to use ClearRangeIter. - opts := clearRangeDataOptions{ + opts := kvstorage.ClearRangeDataOptions{ ClearReplicatedByRangeID: true, ClearUnreplicatedByRangeID: true, MustUseClearRange: true, diff --git a/pkg/kv/kvserver/store_split.go b/pkg/kv/kvserver/store_split.go index 7f71f16f277a..0edebed7f028 100644 --- a/pkg/kv/kvserver/store_split.go +++ b/pkg/kv/kvserver/store_split.go @@ -95,7 +95,7 @@ func splitPreApply( log.Fatalf(ctx, "failed to load hard state for removed rhs: %v", err) } } - if err := clearRangeData(split.RightDesc.RangeID, readWriter, readWriter, clearRangeDataOptions{ + if err := clearRangeData(split.RightDesc.RangeID, readWriter, readWriter, kvstorage.ClearRangeDataOptions{ // We know there isn't anything in these two replicated spans below in the // right-hand side (before the current batch), so setting these options // will in effect only clear the writes to the RHS replicated state we have From 45dad7c96fae62e43640cd9b2cc34cf129cbbfb1 Mon Sep 17 00:00:00 2001 From: Tobias Grieger Date: Mon, 6 Feb 2023 17:21:12 +0100 Subject: [PATCH 11/16] kvserver,kvstorage: move clearRangeData Epic: CRDB-220 Release note: None --- pkg/kv/kvserver/kvstorage/BUILD.bazel | 1 + pkg/kv/kvserver/kvstorage/destroy.go | 56 ++++++++++++++++++++++++- pkg/kv/kvserver/replica_destroy.go | 2 +- pkg/kv/kvserver/replica_raftstorage.go | 57 +------------------------- pkg/kv/kvserver/store_split.go | 2 +- 5 files changed, 60 insertions(+), 58 deletions(-) diff --git a/pkg/kv/kvserver/kvstorage/BUILD.bazel b/pkg/kv/kvserver/kvstorage/BUILD.bazel index dd127102c6df..cdde4176d5a9 100644 --- a/pkg/kv/kvserver/kvstorage/BUILD.bazel +++ b/pkg/kv/kvserver/kvstorage/BUILD.bazel @@ -17,6 +17,7 @@ go_library( "//pkg/keys", "//pkg/kv/kvserver/kvserverpb", "//pkg/kv/kvserver/logstore", + "//pkg/kv/kvserver/rditer", "//pkg/kv/kvserver/stateloader", "//pkg/roachpb", "//pkg/storage", diff --git a/pkg/kv/kvserver/kvstorage/destroy.go b/pkg/kv/kvserver/kvstorage/destroy.go index 86e5a32172a8..919b0de557dd 100644 --- a/pkg/kv/kvserver/kvstorage/destroy.go +++ b/pkg/kv/kvserver/kvstorage/destroy.go @@ -10,7 +10,31 @@ package kvstorage -import "github.com/cockroachdb/cockroach/pkg/roachpb" +import ( + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/rditer" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/storage" +) + +const ( + // ClearRangeThresholdPointKeys is the threshold (as number of point keys) + // beyond which we'll clear range data using a Pebble range tombstone rather + // than individual Pebble point tombstones. + // + // It is expensive for there to be many Pebble range tombstones in the same + // sstable because all of the tombstones in an sstable are loaded whenever the + // sstable is accessed. So we avoid using range deletion unless there is some + // minimum number of keys. The value here was pulled out of thin air. It might + // be better to make this dependent on the size of the data being deleted. Or + // perhaps we should fix Pebble to handle large numbers of range tombstones in + // an sstable better. + ClearRangeThresholdPointKeys = 64 + + // ClearRangeThresholdRangeKeys is the threshold (as number of range keys) + // beyond which we'll clear range data using a single RANGEKEYDEL across the + // span rather than clearing individual range keys. + ClearRangeThresholdRangeKeys = 8 +) // ClearRangeDataOptions specify which parts of a Replica are to be destroyed. type ClearRangeDataOptions struct { @@ -32,3 +56,33 @@ type ClearRangeDataOptions struct { // heuristic will be used instead. MustUseClearRange bool } + +// ClearRangeData clears the data associated with a range descriptor selected +// by the provided options. +// +// TODO(tbg): could rename this to XReplica. The use of "Range" in both the +// "CRDB Range" and "storage.ClearRange" context in the setting of this method could +// be confusing. +func ClearRangeData( + rangeID roachpb.RangeID, reader storage.Reader, writer storage.Writer, opts ClearRangeDataOptions, +) error { + keySpans := rditer.Select(rangeID, rditer.SelectOpts{ + ReplicatedBySpan: opts.ClearReplicatedBySpan, + ReplicatedByRangeID: opts.ClearReplicatedByRangeID, + UnreplicatedByRangeID: opts.ClearUnreplicatedByRangeID, + }) + + pointKeyThreshold, rangeKeyThreshold := ClearRangeThresholdPointKeys, ClearRangeThresholdRangeKeys + if opts.MustUseClearRange { + pointKeyThreshold, rangeKeyThreshold = 1, 1 + } + + for _, keySpan := range keySpans { + if err := storage.ClearRangeWithHeuristic( + reader, writer, keySpan.Key, keySpan.EndKey, pointKeyThreshold, rangeKeyThreshold, + ); err != nil { + return err + } + } + return nil +} diff --git a/pkg/kv/kvserver/replica_destroy.go b/pkg/kv/kvserver/replica_destroy.go index e908e198b717..b941e343707d 100644 --- a/pkg/kv/kvserver/replica_destroy.go +++ b/pkg/kv/kvserver/replica_destroy.go @@ -89,7 +89,7 @@ func preDestroyRaftMuLocked( if diskReplicaID.ReplicaID >= nextReplicaID { return errors.AssertionFailedf("replica r%d/%d must not survive its own tombstone", rangeID, diskReplicaID) } - if err := clearRangeData(rangeID, reader, writer, opts); err != nil { + if err := kvstorage.ClearRangeData(rangeID, reader, writer, opts); err != nil { return err } diff --git a/pkg/kv/kvserver/replica_raftstorage.go b/pkg/kv/kvserver/replica_raftstorage.go index d6ca5d450016..c4b107fa3be9 100644 --- a/pkg/kv/kvserver/replica_raftstorage.go +++ b/pkg/kv/kvserver/replica_raftstorage.go @@ -41,26 +41,6 @@ type replicaRaftStorage Replica var _ raft.Storage = (*replicaRaftStorage)(nil) -const ( - // clearRangeThresholdPointKeys is the threshold (as number of point keys) - // beyond which we'll clear range data using a Pebble range tombstone rather - // than individual Pebble point tombstones. - // - // It is expensive for there to be many Pebble range tombstones in the same - // sstable because all of the tombstones in an sstable are loaded whenever the - // sstable is accessed. So we avoid using range deletion unless there is some - // minimum number of keys. The value here was pulled out of thin air. It might - // be better to make this dependent on the size of the data being deleted. Or - // perhaps we should fix Pebble to handle large numbers of range tombstones in - // an sstable better. - clearRangeThresholdPointKeys = 64 - - // clearRangeThresholdRangeKeys is the threshold (as number of range keys) - // beyond which we'll clear range data using a single RANGEKEYDEL across the - // span rather than clearing individual range keys. - clearRangeThresholdRangeKeys = 8 -) - // All calls to raft.RawNode require that both Replica.raftMu and // Replica.mu are held. All of the functions exposed via the // raft.Storage interface will in turn be called from RawNode, so none @@ -397,39 +377,6 @@ func (r *Replica) updateRangeInfo(ctx context.Context, desc *roachpb.RangeDescri return nil } -// clearRangeData clears the data associated with a range descriptor selected -// by the provided clearRangeDataOptions. -// -// TODO(tbg): could rename this to clearReplicaData. The use of "Range" in both the -// "CRDB Range" and "storage.ClearRange" context in the setting of this method could -// be confusing. -func clearRangeData( - rangeID roachpb.RangeID, - reader storage.Reader, - writer storage.Writer, - opts kvstorage.ClearRangeDataOptions, -) error { - keySpans := rditer.Select(rangeID, rditer.SelectOpts{ - ReplicatedBySpan: opts.ClearReplicatedBySpan, - ReplicatedByRangeID: opts.ClearReplicatedByRangeID, - UnreplicatedByRangeID: opts.ClearUnreplicatedByRangeID, - }) - - pointKeyThreshold, rangeKeyThreshold := clearRangeThresholdPointKeys, clearRangeThresholdRangeKeys - if opts.MustUseClearRange { - pointKeyThreshold, rangeKeyThreshold = 1, 1 - } - - for _, keySpan := range keySpans { - if err := storage.ClearRangeWithHeuristic( - reader, writer, keySpan.Key, keySpan.EndKey, pointKeyThreshold, rangeKeyThreshold, - ); err != nil { - return err - } - } - return nil -} - // applySnapshot updates the replica and its store based on the given // (non-empty) snapshot and associated HardState. All snapshots must pass // through Raft for correctness, i.e. the parameters to this method must be @@ -856,8 +803,8 @@ func clearSubsumedReplicaDiskData( &subsumedReplSST, keySpans[i].EndKey, totalKeySpans[i].EndKey, - clearRangeThresholdPointKeys, - clearRangeThresholdRangeKeys, + kvstorage.ClearRangeThresholdPointKeys, + kvstorage.ClearRangeThresholdRangeKeys, ); err != nil { subsumedReplSST.Close() return err diff --git a/pkg/kv/kvserver/store_split.go b/pkg/kv/kvserver/store_split.go index 0edebed7f028..9b7dfa2521a1 100644 --- a/pkg/kv/kvserver/store_split.go +++ b/pkg/kv/kvserver/store_split.go @@ -95,7 +95,7 @@ func splitPreApply( log.Fatalf(ctx, "failed to load hard state for removed rhs: %v", err) } } - if err := clearRangeData(split.RightDesc.RangeID, readWriter, readWriter, kvstorage.ClearRangeDataOptions{ + if err := kvstorage.ClearRangeData(split.RightDesc.RangeID, readWriter, readWriter, kvstorage.ClearRangeDataOptions{ // We know there isn't anything in these two replicated spans below in the // right-hand side (before the current batch), so setting these options // will in effect only clear the writes to the RHS replicated state we have From a5cf9d54e3e3c066b64ece936ed3103699570b26 Mon Sep 17 00:00:00 2001 From: Tobias Grieger Date: Mon, 6 Feb 2023 17:24:33 +0100 Subject: [PATCH 12/16] kvserver,kvstorage: move-rename `preDestroyRaftMuLocked` It's now a first-class citizen of `kvstorage`, and we can now write better unit tests for it. Epic: CRDB-220 Release note: None --- pkg/kv/kvserver/kvstorage/destroy.go | 58 ++++++++++++++++++++++++++ pkg/kv/kvserver/replica_app_batch.go | 4 +- pkg/kv/kvserver/replica_destroy.go | 57 +------------------------ pkg/kv/kvserver/replica_raftstorage.go | 2 +- 4 files changed, 62 insertions(+), 59 deletions(-) diff --git a/pkg/kv/kvserver/kvstorage/destroy.go b/pkg/kv/kvserver/kvstorage/destroy.go index 919b0de557dd..75d43bec2b45 100644 --- a/pkg/kv/kvserver/kvstorage/destroy.go +++ b/pkg/kv/kvserver/kvstorage/destroy.go @@ -11,9 +11,15 @@ package kvstorage import ( + "context" + + "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/logstore" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/rditer" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage" + "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/errors" ) const ( @@ -86,3 +92,55 @@ func ClearRangeData( } return nil } + +// DestroyReplica destroys all or a part of the Replica's state, installing a +// RangeTombstone in its place. +func DestroyReplica( + ctx context.Context, + rangeID roachpb.RangeID, + reader storage.Reader, + writer storage.Writer, + nextReplicaID roachpb.ReplicaID, + opts ClearRangeDataOptions, +) error { + diskReplicaID, err := logstore.NewStateLoader(rangeID).LoadRaftReplicaID(ctx, reader) + if err != nil { + return err + } + if diskReplicaID.ReplicaID >= nextReplicaID { + return errors.AssertionFailedf("replica r%d/%d must not survive its own tombstone", rangeID, diskReplicaID) + } + if err := ClearRangeData(rangeID, reader, writer, opts); err != nil { + return err + } + + // Save a tombstone to ensure that replica IDs never get reused. + // + // TODO(tbg): put this on `stateloader.StateLoader` and consolidate the + // other read of the range tombstone key (in uninited replica creation + // as well). + + tombstoneKey := keys.RangeTombstoneKey(rangeID) + + // Assert that the provided tombstone moves the existing one strictly forward. + // Failure to do so indicates that something is going wrong in the replica + // lifecycle. + { + var tombstone roachpb.RangeTombstone + if _, err := storage.MVCCGetProto( + ctx, reader, tombstoneKey, hlc.Timestamp{}, &tombstone, storage.MVCCGetOptions{}, + ); err != nil { + return err + } + if tombstone.NextReplicaID >= nextReplicaID { + return errors.AssertionFailedf( + "cannot rewind tombstone from %d to %d", tombstone.NextReplicaID, nextReplicaID, + ) + } + } + + tombstone := roachpb.RangeTombstone{NextReplicaID: nextReplicaID} + // "Blind" because ms == nil and timestamp.IsEmpty(). + return storage.MVCCBlindPutProto(ctx, writer, nil, tombstoneKey, + hlc.Timestamp{}, hlc.ClockTimestamp{}, &tombstone, nil) +} diff --git a/pkg/kv/kvserver/replica_app_batch.go b/pkg/kv/kvserver/replica_app_batch.go index ac1446574f32..3fb6aa45e86f 100644 --- a/pkg/kv/kvserver/replica_app_batch.go +++ b/pkg/kv/kvserver/replica_app_batch.go @@ -335,7 +335,7 @@ func (b *replicaAppBatch) runPostAddTriggersReplicaOnly( // required for correctness, since the merge protocol should guarantee that // no new replicas of the RHS can ever be created, but it doesn't hurt to // be careful. - if err := preDestroyRaftMuLocked(ctx, rhsRepl.RangeID, b.batch, b.batch, mergedTombstoneReplicaID, kvstorage.ClearRangeDataOptions{ + if err := kvstorage.DestroyReplica(ctx, rhsRepl.RangeID, b.batch, b.batch, mergedTombstoneReplicaID, kvstorage.ClearRangeDataOptions{ ClearReplicatedByRangeID: true, ClearUnreplicatedByRangeID: true, }); err != nil { @@ -475,7 +475,7 @@ func (b *replicaAppBatch) runPostAddTriggersReplicaOnly( // We've set the replica's in-mem status to reflect the pending destruction // above, and preDestroyRaftMuLocked will also add a range tombstone to the // batch, so that when we commit it, the removal is finalized. - if err := preDestroyRaftMuLocked(ctx, b.r.RangeID, b.batch, b.batch, change.NextReplicaID(), kvstorage.ClearRangeDataOptions{ + if err := kvstorage.DestroyReplica(ctx, b.r.RangeID, b.batch, b.batch, change.NextReplicaID(), kvstorage.ClearRangeDataOptions{ ClearReplicatedBySpan: span, ClearReplicatedByRangeID: true, ClearUnreplicatedByRangeID: true, diff --git a/pkg/kv/kvserver/replica_destroy.go b/pkg/kv/kvserver/replica_destroy.go index b941e343707d..a0e3a4fab1d8 100644 --- a/pkg/kv/kvserver/replica_destroy.go +++ b/pkg/kv/kvserver/replica_destroy.go @@ -15,17 +15,12 @@ import ( "fmt" "math" - "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/apply" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvstorage" - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/logstore" "github.com/cockroachdb/cockroach/pkg/roachpb" - "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" - "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/timeutil" - "github.com/cockroachdb/errors" ) // DestroyReason indicates if a replica is alive, destroyed, corrupted or pending destruction. @@ -74,56 +69,6 @@ func (s destroyStatus) Removed() bool { // don't know the current replica ID. const mergedTombstoneReplicaID roachpb.ReplicaID = math.MaxInt32 -func preDestroyRaftMuLocked( - ctx context.Context, - rangeID roachpb.RangeID, - reader storage.Reader, - writer storage.Writer, - nextReplicaID roachpb.ReplicaID, - opts kvstorage.ClearRangeDataOptions, -) error { - diskReplicaID, err := logstore.NewStateLoader(rangeID).LoadRaftReplicaID(ctx, reader) - if err != nil { - return err - } - if diskReplicaID.ReplicaID >= nextReplicaID { - return errors.AssertionFailedf("replica r%d/%d must not survive its own tombstone", rangeID, diskReplicaID) - } - if err := kvstorage.ClearRangeData(rangeID, reader, writer, opts); err != nil { - return err - } - - // Save a tombstone to ensure that replica IDs never get reused. - // - // TODO(tbg): put this on `stateloader.StateLoader` and consolidate the - // other read of the range tombstone key (in uninited replica creation - // as well). - - tombstoneKey := keys.RangeTombstoneKey(rangeID) - - // Assert that the provided tombstone moves the existing one strictly forward. - // Failure to do so indicates that something is going wrong in the replica - // lifecycle. - { - var tombstone roachpb.RangeTombstone - if _, err := storage.MVCCGetProto( - ctx, reader, tombstoneKey, hlc.Timestamp{}, &tombstone, storage.MVCCGetOptions{}, - ); err != nil { - return err - } - if tombstone.NextReplicaID >= nextReplicaID { - return errors.AssertionFailedf( - "cannot rewind tombstone from %d to %d", tombstone.NextReplicaID, nextReplicaID, - ) - } - } - - tombstone := roachpb.RangeTombstone{NextReplicaID: nextReplicaID} - // "Blind" because ms == nil and timestamp.IsEmpty(). - return storage.MVCCBlindPutProto(ctx, writer, nil, tombstoneKey, - hlc.Timestamp{}, hlc.ClockTimestamp{}, &tombstone, nil) -} - func (r *Replica) postDestroyRaftMuLocked(ctx context.Context, ms enginepb.MVCCStats) error { // NB: we need the nil check below because it's possible that we're GC'ing a // Replica without a replicaID, in which case it does not have a sideloaded @@ -180,7 +125,7 @@ func (r *Replica) destroyRaftMuLocked(ctx context.Context, nextReplicaID roachpb ClearReplicatedByRangeID: inited, ClearUnreplicatedByRangeID: true, } - if err := preDestroyRaftMuLocked(ctx, r.RangeID, r.Engine(), batch, nextReplicaID, opts); err != nil { + if err := kvstorage.DestroyReplica(ctx, r.RangeID, r.Engine(), batch, nextReplicaID, opts); err != nil { return err } preTime := timeutil.Now() diff --git a/pkg/kv/kvserver/replica_raftstorage.go b/pkg/kv/kvserver/replica_raftstorage.go index c4b107fa3be9..5a28563740e8 100644 --- a/pkg/kv/kvserver/replica_raftstorage.go +++ b/pkg/kv/kvserver/replica_raftstorage.go @@ -750,7 +750,7 @@ func clearSubsumedReplicaDiskData( ClearUnreplicatedByRangeID: true, MustUseClearRange: true, } - if err := preDestroyRaftMuLocked(ctx, sr.RangeID, reader, &subsumedReplSST, subsumedNextReplicaID, opts); err != nil { + if err := kvstorage.DestroyReplica(ctx, sr.RangeID, reader, &subsumedReplSST, subsumedNextReplicaID, opts); err != nil { subsumedReplSST.Close() return err } From 6b7de866837293722f5537ebb742744b5630cda2 Mon Sep 17 00:00:00 2001 From: Tobias Grieger Date: Mon, 6 Feb 2023 17:27:34 +0100 Subject: [PATCH 13/16] kvstorage: improve comment on DestroyReplica Epic: CRDB-220 Release note: None --- pkg/kv/kvserver/kvstorage/destroy.go | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/pkg/kv/kvserver/kvstorage/destroy.go b/pkg/kv/kvserver/kvstorage/destroy.go index 75d43bec2b45..d6506eaf099e 100644 --- a/pkg/kv/kvserver/kvstorage/destroy.go +++ b/pkg/kv/kvserver/kvstorage/destroy.go @@ -94,7 +94,13 @@ func ClearRangeData( } // DestroyReplica destroys all or a part of the Replica's state, installing a -// RangeTombstone in its place. +// RangeTombstone in its place. Due to merges, splits, etc, there is a need +// to control which part of the state this method actually gets to remove, +// which is done via the provided options[^1]; the caller is always responsible +// for managing the remaining disk state accordingly. +// +// [^1] e.g., on a merge, the user data moves to the subsuming replica and must +// not be cleared. func DestroyReplica( ctx context.Context, rangeID roachpb.RangeID, From 690da3e2ac3b1b7268a0cfb1dcbe5464e948e9d1 Mon Sep 17 00:00:00 2001 From: Jayant Shrivastava Date: Mon, 6 Feb 2023 11:16:16 -0500 Subject: [PATCH 14/16] jobs: add metrics for paused jobs This change adds new metrics to count paused jobs for every job type. For example, the metric for paused changefeed jobs is `jobs.changefeed.currently_paused`. These metrics are counted at an interval defined by the cluster setting `jobs.metrics.interval.poll`. This is implemented by a job which periodically queries `system.jobs` to count the number of paused jobs. This job is of the newly added type `jobspb.TypePollJobsStats`. When a node starts it's job registry, it will create an adoptable stats polling job if it does not exist already using a transaction. This change adds a test which pauses and resumes changefeeds while asserting the value of the `jobs.changefeed.currently_paused` metric. It also adds a logictest to ensure one instance of the stats polling job is created in a cluster. Resolves: https://github.com/cockroachdb/cockroach/issues/85467 Release note (general change): This change adds new metrics to count paused jobs for every job type. For example, the metric for paused changefeed jobs is `jobs.changefeed.currently_paused`. These metrics are updated at an interval defined by the cluster setting `jobs.metrics.interval.poll`, which is defauled to 10 seconds. Epic: None --- .../settings/settings-for-tenants.txt | 2 +- docs/generated/settings/settings.html | 2 +- pkg/BUILD.bazel | 2 + pkg/cli/testdata/doctor/test_examine_cluster | 2 +- pkg/clusterversion/cockroach_versions.go | 8 ++ pkg/jobs/config.go | 15 +++ pkg/jobs/jobs_test.go | 104 +++++++++++++++++- pkg/jobs/jobspb/BUILD.bazel | 12 +- pkg/jobs/jobspb/jobs.proto | 13 +++ pkg/jobs/jobspb/wrap.go | 25 ++++- pkg/jobs/jobspb/wrap_test.go | 30 +++++ pkg/jobs/metrics.go | 13 +++ pkg/jobs/registry.go | 80 ++++++++++++++ pkg/jobs/registry_test.go | 22 ++-- pkg/jobs/testing_knobs.go | 10 +- pkg/jobs/update.go | 4 +- pkg/sql/BUILD.bazel | 1 + pkg/sql/job_statistics.go | 50 +++++++++ pkg/sql/logictest/testdata/logic_test/jobs | 6 + pkg/ts/catalog/chart_catalog.go | 37 +++++++ pkg/upgrade/upgradebase/testing_knobs.go | 5 + pkg/upgrade/upgrademanager/manager.go | 1 + pkg/upgrade/upgrades/BUILD.bazel | 2 + .../create_jobs_metrics_polling_job.go | 60 ++++++++++ .../create_jobs_metrics_polling_job_test.go | 69 ++++++++++++ pkg/upgrade/upgrades/upgrades.go | 5 + 26 files changed, 557 insertions(+), 23 deletions(-) create mode 100644 pkg/jobs/jobspb/wrap_test.go create mode 100644 pkg/sql/job_statistics.go create mode 100644 pkg/upgrade/upgrades/create_jobs_metrics_polling_job.go create mode 100644 pkg/upgrade/upgrades/create_jobs_metrics_polling_job_test.go diff --git a/docs/generated/settings/settings-for-tenants.txt b/docs/generated/settings/settings-for-tenants.txt index 4248fcea0444..9ef86cc407e2 100644 --- a/docs/generated/settings/settings-for-tenants.txt +++ b/docs/generated/settings/settings-for-tenants.txt @@ -295,4 +295,4 @@ trace.jaeger.agent string the address of a Jaeger agent to receive traces using trace.opentelemetry.collector string address of an OpenTelemetry trace collector to receive traces using the otel gRPC protocol, as :. If no port is specified, 4317 will be used. trace.span_registry.enabled boolean true if set, ongoing traces can be seen at https:///#/debug/tracez trace.zipkin.collector string the address of a Zipkin instance to receive traces, as :. If no port is specified, 9411 will be used. -version version 1000022.2-36 set the active cluster version in the format '.' +version version 1000022.2-38 set the active cluster version in the format '.' diff --git a/docs/generated/settings/settings.html b/docs/generated/settings/settings.html index 11081ead30f9..3d4de9bdf728 100644 --- a/docs/generated/settings/settings.html +++ b/docs/generated/settings/settings.html @@ -236,6 +236,6 @@
trace.opentelemetry.collector
stringaddress of an OpenTelemetry trace collector to receive traces using the otel gRPC protocol, as <host>:<port>. If no port is specified, 4317 will be used.
trace.span_registry.enabled
booleantrueif set, ongoing traces can be seen at https://<ui>/#/debug/tracez
trace.zipkin.collector
stringthe address of a Zipkin instance to receive traces, as <host>:<port>. If no port is specified, 9411 will be used. -
version
version1000022.2-36set the active cluster version in the format '<major>.<minor>' +
version
version1000022.2-38set the active cluster version in the format '<major>.<minor>' diff --git a/pkg/BUILD.bazel b/pkg/BUILD.bazel index 015538c056b9..880a6fc999dd 100644 --- a/pkg/BUILD.bazel +++ b/pkg/BUILD.bazel @@ -169,6 +169,7 @@ ALL_TESTS = [ "//pkg/internal/team:team_test", "//pkg/jobs/joberror:joberror_test", "//pkg/jobs/jobsauth:jobsauth_test", + "//pkg/jobs/jobspb:jobspb_test", "//pkg/jobs/jobsprotectedts:jobsprotectedts_test", "//pkg/jobs:jobs_test", "//pkg/keys:keys_test", @@ -1124,6 +1125,7 @@ GO_TARGETS = [ "//pkg/jobs/jobsauth:jobsauth", "//pkg/jobs/jobsauth:jobsauth_test", "//pkg/jobs/jobspb:jobspb", + "//pkg/jobs/jobspb:jobspb_test", "//pkg/jobs/jobsprotectedts:jobsprotectedts", "//pkg/jobs/jobsprotectedts:jobsprotectedts_test", "//pkg/jobs/jobstest:jobstest", diff --git a/pkg/cli/testdata/doctor/test_examine_cluster b/pkg/cli/testdata/doctor/test_examine_cluster index acf9b22e551f..6da9dc1183d4 100644 --- a/pkg/cli/testdata/doctor/test_examine_cluster +++ b/pkg/cli/testdata/doctor/test_examine_cluster @@ -3,5 +3,5 @@ debug doctor examine cluster debug doctor examine cluster Examining 53 descriptors and 52 namespace entries... ParentID 100, ParentSchemaID 101: relation "foo" (105): expected matching namespace entry, found none -Examining 12 jobs... +Examining 14 jobs... ERROR: validation failed diff --git a/pkg/clusterversion/cockroach_versions.go b/pkg/clusterversion/cockroach_versions.go index f439b365b15a..5d0f659e463e 100644 --- a/pkg/clusterversion/cockroach_versions.go +++ b/pkg/clusterversion/cockroach_versions.go @@ -410,6 +410,10 @@ const ( V23_1_DeleteDroppedFunctionDescriptors + // V23_1_CreateJobsMetricsPollingJob creates the permanent job + // responsible for polling the jobs table for metrics. + V23_1_CreateJobsMetricsPollingJob + // ************************************************* // Step (1): Add new versions here. // Do not add new versions to a patch release. @@ -707,6 +711,10 @@ var rawVersionsSingleton = keyedVersions{ Key: V23_1_DeleteDroppedFunctionDescriptors, Version: roachpb.Version{Major: 22, Minor: 2, Internal: 36}, }, + { + Key: V23_1_CreateJobsMetricsPollingJob, + Version: roachpb.Version{Major: 22, Minor: 2, Internal: 38}, + }, // ************************************************* // Step (2): Add new versions here. diff --git a/pkg/jobs/config.go b/pkg/jobs/config.go index 3bfbc7c439d2..d178f0abcbec 100644 --- a/pkg/jobs/config.go +++ b/pkg/jobs/config.go @@ -32,6 +32,7 @@ const ( executionErrorsMaxEntriesKey = "jobs.execution_errors.max_entries" executionErrorsMaxEntrySizeKey = "jobs.execution_errors.max_entry_size" debugPausePointsSettingKey = "jobs.debug.pausepoints" + metricsPollingIntervalKey = "jobs.metrics.interval.poll" ) const ( @@ -70,6 +71,10 @@ const ( // error. If this size is exceeded, the error will be formatted as a string // and then truncated to fit the size. defaultExecutionErrorsMaxEntrySize = 64 << 10 // 64 KiB + + // defaultPollForMetricsInterval is the default interval to poll the jobs + // table for metrics. + defaultPollForMetricsInterval = 10 * time.Second ) var ( @@ -100,6 +105,16 @@ var ( settings.PositiveDuration, ) + // PollJobsMetricsInterval is the interval at which a tenant in the cluster + // will poll the jobs table for metrics + PollJobsMetricsInterval = settings.RegisterDurationSetting( + settings.TenantWritable, + metricsPollingIntervalKey, + "the interval at which a node in the cluster will poll the jobs table for metrics", + defaultPollForMetricsInterval, + settings.PositiveDuration, + ) + gcIntervalSetting = settings.RegisterDurationSetting( settings.TenantWritable, gcIntervalSettingKey, diff --git a/pkg/jobs/jobs_test.go b/pkg/jobs/jobs_test.go index 940d5f254e60..85dbb8ed2392 100644 --- a/pkg/jobs/jobs_test.go +++ b/pkg/jobs/jobs_test.go @@ -48,6 +48,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/tests" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" + "github.com/cockroachdb/cockroach/pkg/testutils/skip" "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" "github.com/cockroachdb/cockroach/pkg/upgrade/upgradebase" "github.com/cockroachdb/cockroach/pkg/util/ctxgroup" @@ -229,7 +230,8 @@ func (rts *registryTestSuite) setUp(t *testing.T) { ManagerDisableJobCreation: true, } args.Knobs.UpgradeManager = &upgradebase.TestingKnobs{ - DontUseJobs: true, + DontUseJobs: true, + SkipJobMetricsPollingJobBootstrap: true, } args.Knobs.KeyVisualizer = &keyvisualizer.TestingKnobs{SkipJobBootstrap: true} @@ -3456,3 +3458,103 @@ func TestPausepoints(t *testing.T) { }) } } + +func TestPausedMetrics(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + skip.UnderShort(t) + + ctx := context.Background() + s, sqlDB, _ := serverutils.StartServer(t, base.TestServerArgs{ + Knobs: base.TestingKnobs{ + JobsTestingKnobs: jobs.NewTestingKnobsWithShortIntervals(), + }, + }) + defer s.Stopper().Stop(ctx) + + jobs.PollJobsMetricsInterval.Override(ctx, &s.ClusterSettings().SV, 10*time.Millisecond) + runner := sqlutils.MakeSQLRunner(sqlDB) + reg := s.JobRegistry().(*jobs.Registry) + + waitForPausedCount := func(typ jobspb.Type, numPaused int64) { + testutils.SucceedsSoon(t, func() error { + currentlyPaused := reg.MetricsStruct().JobMetrics[typ].CurrentlyPaused.Value() + if reg.MetricsStruct().JobMetrics[typ].CurrentlyPaused.Value() != numPaused { + return fmt.Errorf( + "expected (%+v) paused jobs of type (%+v), found (%+v)", + numPaused, + typ, + currentlyPaused, + ) + } + return nil + }) + } + + typeToRecord := map[jobspb.Type]jobs.Record{ + jobspb.TypeChangefeed: { + Details: jobspb.ChangefeedDetails{}, + Progress: jobspb.ChangefeedProgress{}, + Username: username.TestUserName(), + }, + jobspb.TypeImport: { + Details: jobspb.ImportDetails{}, + Progress: jobspb.ImportProgress{}, + Username: username.TestUserName(), + }, + jobspb.TypeSchemaChange: { + Details: jobspb.SchemaChangeDetails{}, + Progress: jobspb.SchemaChangeProgress{}, + Username: username.TestUserName(), + }, + } + for typ := range typeToRecord { + jobs.RegisterConstructor(typ, func(job *jobs.Job, _ *cluster.Settings) jobs.Resumer { + return jobs.FakeResumer{ + OnResume: func(ctx context.Context) error { + <-ctx.Done() + return ctx.Err() + }, + } + }, jobs.UsesTenantCostControl) + } + + makeJob := func(ctx context.Context, + typ jobspb.Type, + ) *jobs.StartableJob { + j, err := jobs.TestingCreateAndStartJob(ctx, reg, s.InternalDB().(isql.DB), typeToRecord[typ]) + if err != nil { + t.Fatal(err) + } + return j + } + + cfJob := makeJob(context.Background(), jobspb.TypeChangefeed) + cfJob2 := makeJob(context.Background(), jobspb.TypeChangefeed) + importJob := makeJob(context.Background(), jobspb.TypeImport) + scJob := makeJob(context.Background(), jobspb.TypeSchemaChange) + + // Pause all job types. + runner.Exec(t, "PAUSE JOB $1", cfJob.ID()) + waitForPausedCount(jobspb.TypeChangefeed, 1) + runner.Exec(t, "PAUSE JOB $1", cfJob2.ID()) + waitForPausedCount(jobspb.TypeChangefeed, 2) + runner.Exec(t, "PAUSE JOB $1", importJob.ID()) + waitForPausedCount(jobspb.TypeImport, 1) + runner.Exec(t, "PAUSE JOB $1", scJob.ID()) + waitForPausedCount(jobspb.TypeSchemaChange, 1) + + // Resume / cancel jobs. + runner.Exec(t, "RESUME JOB $1", cfJob.ID()) + waitForPausedCount(jobspb.TypeChangefeed, 1) + runner.Exec(t, "CANCEL JOB $1", cfJob2.ID()) + waitForPausedCount(jobspb.TypeChangefeed, 0) + runner.Exec(t, "RESUME JOB $1", importJob.ID()) + waitForPausedCount(jobspb.TypeImport, 0) + runner.Exec(t, "CANCEL JOB $1", scJob.ID()) + waitForPausedCount(jobspb.TypeSchemaChange, 0) + + runner.Exec(t, "CANCEL JOB $1", cfJob.ID()) + runner.Exec(t, "CANCEL JOB $1", importJob.ID()) +} diff --git a/pkg/jobs/jobspb/BUILD.bazel b/pkg/jobs/jobspb/BUILD.bazel index 4c27551604a4..52677f1acc61 100644 --- a/pkg/jobs/jobspb/BUILD.bazel +++ b/pkg/jobs/jobspb/BUILD.bazel @@ -1,7 +1,7 @@ load("//build/bazelutil/unused_checker:unused.bzl", "get_x_data") load("@rules_proto//proto:defs.bzl", "proto_library") load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library") -load("@io_bazel_rules_go//go:def.bzl", "go_library") +load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") go_library( name = "jobspb", @@ -64,4 +64,14 @@ go_proto_library( ], ) +go_test( + name = "jobspb_test", + srcs = ["wrap_test.go"], + args = ["-test.timeout=295s"], + deps = [ + ":jobspb", + "@com_github_stretchr_testify//assert", + ], +) + get_x_data(name = "get_x_data") diff --git a/pkg/jobs/jobspb/jobs.proto b/pkg/jobs/jobspb/jobs.proto index 2b2503812488..7982a35d4eb1 100644 --- a/pkg/jobs/jobspb/jobs.proto +++ b/pkg/jobs/jobspb/jobs.proto @@ -1149,6 +1149,12 @@ message SchemaTelemetryDetails { message SchemaTelemetryProgress { } +message PollJobsStatsDetails { +} + +message PollJobsStatsProgress { +} + message Payload { string description = 1; // If empty, the description is assumed to be the statement. @@ -1200,7 +1206,12 @@ message Payload { // and publish it to the telemetry event log. These jobs are typically // created by a built-in schedule named "sql-schema-telemetry". SchemaTelemetryDetails schema_telemetry = 37; + KeyVisualizerDetails keyVisualizerDetails = 38; + + // PollJobsStats jobs poll the jobs table for statistics metrics as the number of + // paused jobs. + PollJobsStatsDetails poll_jobs_stats = 39; } reserved 26; // PauseReason is used to describe the reason that the job is currently paused @@ -1263,6 +1274,7 @@ message Progress { RowLevelTTLProgress row_level_ttl = 25 [(gogoproto.customname)="RowLevelTTL"]; SchemaTelemetryProgress schema_telemetry = 26; KeyVisualizerProgress keyVisualizerProgress = 27; + PollJobsStatsProgress pollJobsStats = 28; } uint64 trace_id = 21 [(gogoproto.nullable) = false, (gogoproto.customname) = "TraceID", (gogoproto.customtype) = "github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb.TraceID"]; @@ -1293,6 +1305,7 @@ enum Type { ROW_LEVEL_TTL = 16 [(gogoproto.enumvalue_customname) = "TypeRowLevelTTL"]; AUTO_SCHEMA_TELEMETRY = 17 [(gogoproto.enumvalue_customname) = "TypeAutoSchemaTelemetry"]; KEY_VISUALIZER = 18 [(gogoproto.enumvalue_customname) = "TypeKeyVisualizer"]; + POLL_JOBS_STATS = 19 [(gogoproto.enumvalue_customname) = "TypePollJobsStats"]; } message Job { diff --git a/pkg/jobs/jobspb/wrap.go b/pkg/jobs/jobspb/wrap.go index 8bfb07805a95..94a101b146f5 100644 --- a/pkg/jobs/jobspb/wrap.go +++ b/pkg/jobs/jobspb/wrap.go @@ -145,6 +145,7 @@ var AutomaticJobTypes = [...]Type{ TypeAutoSpanConfigReconciliation, TypeAutoSQLStatsCompaction, TypeAutoSchemaTelemetry, + TypePollJobsStats, } // DetailsType returns the type for a payload detail. @@ -188,6 +189,8 @@ func DetailsType(d isPayload_Details) (Type, error) { return TypeAutoSchemaTelemetry, nil case *Payload_KeyVisualizerDetails: return TypeKeyVisualizer, nil + case *Payload_PollJobsStats: + return TypePollJobsStats, nil default: return TypeUnspecified, errors.Newf("Payload.Type called on a payload with an unknown details type: %T", d) } @@ -227,6 +230,7 @@ var JobDetailsForEveryJobType = map[Type]Details{ TypeRowLevelTTL: RowLevelTTLDetails{}, TypeAutoSchemaTelemetry: SchemaTelemetryDetails{}, TypeKeyVisualizer: KeyVisualizerDetails{}, + TypePollJobsStats: PollJobsStatsDetails{}, } // WrapProgressDetails wraps a ProgressDetails object in the protobuf wrapper @@ -272,6 +276,8 @@ func WrapProgressDetails(details ProgressDetails) interface { return &Progress_SchemaTelemetry{SchemaTelemetry: &d} case KeyVisualizerProgress: return &Progress_KeyVisualizerProgress{KeyVisualizerProgress: &d} + case PollJobsStatsProgress: + return &Progress_PollJobsStats{PollJobsStats: &d} default: panic(errors.AssertionFailedf("WrapProgressDetails: unknown progress type %T", d)) } @@ -315,6 +321,8 @@ func (p *Payload) UnwrapDetails() Details { return *d.SchemaTelemetry case *Payload_KeyVisualizerDetails: return *d.KeyVisualizerDetails + case *Payload_PollJobsStats: + return *d.PollJobsStats default: return nil } @@ -358,6 +366,8 @@ func (p *Progress) UnwrapDetails() ProgressDetails { return *d.SchemaTelemetry case *Progress_KeyVisualizerProgress: return *d.KeyVisualizerProgress + case *Progress_PollJobsStats: + return *d.PollJobsStats default: return nil } @@ -371,6 +381,17 @@ func (t Type) String() string { return strings.Replace(Type_name[int32(t)], "_", " ", -1) } +// TypeFromString is used to get the type corresponding to the string s +// where s := Type.String(). +func TypeFromString(s string) (Type, error) { + s = strings.Replace(s, " ", "_", -1) + t, ok := Type_value[s] + if !ok { + return TypeUnspecified, errors.New("invalid type string") + } + return Type(t), nil +} + // WrapPayloadDetails wraps a Details object in the protobuf wrapper struct // necessary to make it usable as the Details field of a Payload. // @@ -414,6 +435,8 @@ func WrapPayloadDetails(details Details) interface { return &Payload_SchemaTelemetry{SchemaTelemetry: &d} case KeyVisualizerDetails: return &Payload_KeyVisualizerDetails{KeyVisualizerDetails: &d} + case PollJobsStatsDetails: + return &Payload_PollJobsStats{PollJobsStats: &d} default: panic(errors.AssertionFailedf("jobs.WrapPayloadDetails: unknown details type %T", d)) } @@ -449,7 +472,7 @@ const ( func (Type) SafeValue() {} // NumJobTypes is the number of jobs types. -const NumJobTypes = 19 +const NumJobTypes = 20 // ChangefeedDetailsMarshaler allows for dependency injection of // cloud.SanitizeExternalStorageURI to avoid the dependency from this diff --git a/pkg/jobs/jobspb/wrap_test.go b/pkg/jobs/jobspb/wrap_test.go new file mode 100644 index 000000000000..4a6c45d1dd61 --- /dev/null +++ b/pkg/jobs/jobspb/wrap_test.go @@ -0,0 +1,30 @@ +// Copyright 2023 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 jobspb_test + +import ( + "testing" + + "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" + "github.com/stretchr/testify/assert" +) + +func TestTypeString(t *testing.T) { + for i := 0; i < jobspb.NumJobTypes; i++ { + typ := jobspb.Type(i) + typStr := typ.String() + convertedType, err := jobspb.TypeFromString(typStr) + if err != nil { + t.Fatal(err) + } + assert.Equal(t, convertedType, typ) + } +} diff --git a/pkg/jobs/metrics.go b/pkg/jobs/metrics.go index 4ff9399b6e87..4c480f833ac5 100644 --- a/pkg/jobs/metrics.go +++ b/pkg/jobs/metrics.go @@ -47,6 +47,7 @@ type Metrics struct { type JobTypeMetrics struct { CurrentlyRunning *metric.Gauge CurrentlyIdle *metric.Gauge + CurrentlyPaused *metric.Gauge ResumeCompleted *metric.Counter ResumeRetryError *metric.Counter ResumeFailed *metric.Counter @@ -82,6 +83,17 @@ func makeMetaCurrentlyIdle(typeStr string) metric.Metadata { } } +func makeMetaCurrentlyPaused(typeStr string) metric.Metadata { + return metric.Metadata{ + Name: fmt.Sprintf("jobs.%s.currently_paused", typeStr), + Help: fmt.Sprintf("Number of %s jobs currently considered Paused", + typeStr), + Measurement: "jobs", + Unit: metric.Unit_COUNT, + MetricType: io_prometheus_client.MetricType_GAUGE, + } +} + func makeMetaResumeCompeted(typeStr string) metric.Metadata { return metric.Metadata{ Name: fmt.Sprintf("jobs.%s.resume_completed", typeStr), @@ -214,6 +226,7 @@ func (m *Metrics) init(histogramWindowInterval time.Duration) { m.JobMetrics[jt] = &JobTypeMetrics{ CurrentlyRunning: metric.NewGauge(makeMetaCurrentlyRunning(typeStr)), CurrentlyIdle: metric.NewGauge(makeMetaCurrentlyIdle(typeStr)), + CurrentlyPaused: metric.NewGauge(makeMetaCurrentlyPaused(typeStr)), ResumeCompleted: metric.NewCounter(makeMetaResumeCompeted(typeStr)), ResumeRetryError: metric.NewCounter(makeMetaResumeRetryError(typeStr)), ResumeFailed: metric.NewCounter(makeMetaResumeFailed(typeStr)), diff --git a/pkg/jobs/registry.go b/pkg/jobs/registry.go index f5401fe813d3..91ed09d9788c 100644 --- a/pkg/jobs/registry.go +++ b/pkg/jobs/registry.go @@ -289,6 +289,9 @@ const ( // KeyVisualizerJobID A static job ID is used to easily check if the // Key Visualizer job already exists. KeyVisualizerJobID = jobspb.JobID(100) + + // JobMetricsPollerJobID A static job ID is used for the job metrics polling job. + JobMetricsPollerJobID = jobspb.JobID(101) ) // MakeJobID generates a new job ID. @@ -1134,6 +1137,83 @@ func (r *Registry) Start(ctx context.Context, stopper *stop.Stopper) error { }) } +// PollMetricsTask polls the jobs table for certain metrics at an interval. +func (r *Registry) PollMetricsTask(ctx context.Context) error { + var err error + t := timeutil.NewTimer() + defer t.Stop() + updateMetrics := func(ctx context.Context, s sqlliveness.Session) { + for { + t.Reset(PollJobsMetricsInterval.Get(&r.settings.SV)) + select { + case <-ctx.Done(): + err = ctx.Err() + return + case <-t.C: + t.Read = true + if err = r.updatePausedMetrics(ctx, s); err != nil { + log.Errorf(ctx, "failed to update paused metrics: %v", err) + return + } + } + } + } + r.withSession(ctx, updateMetrics) + return err +} + +const pausedJobsCountQuery = string(` + SELECT job_type, count(*) + FROM system.jobs + WHERE status = '` + StatusPaused + `' + GROUP BY job_type`) + +func (r *Registry) updatePausedMetrics(ctx context.Context, s sqlliveness.Session) error { + var metricUpdates map[jobspb.Type]int + err := r.db.Txn(ctx, func(ctx context.Context, txn isql.Txn) error { + // In case of transaction retries, reset this map here. + metricUpdates = make(map[jobspb.Type]int) + + // Run the claim transaction at low priority to ensure that it does not + // contend with foreground reads. + if err := txn.KV().SetUserPriority(roachpb.MinUserPriority); err != nil { + return err + } + rows, err := txn.QueryBufferedEx( + ctx, "poll-jobs-metrics-job", txn.KV(), sessiondata.InternalExecutorOverride{User: username.NodeUserName()}, + pausedJobsCountQuery, + ) + if err != nil { + return errors.Wrap(err, "could not query jobs table") + } + + for _, row := range rows { + typeString := *row[0].(*tree.DString) + count := *row[1].(*tree.DInt) + typ, err := jobspb.TypeFromString(string(typeString)) + if err != nil { + return err + } + metricUpdates[typ] = int(count) + } + + return nil + }) + if err == nil { + for _, v := range jobspb.Type_value { + if r.metrics.JobMetrics[v] != nil { + if _, ok := metricUpdates[jobspb.Type(v)]; ok { + r.metrics.JobMetrics[v].CurrentlyPaused.Update(int64(metricUpdates[jobspb.Type(v)])) + } else { + r.metrics.JobMetrics[v].CurrentlyPaused.Update(0) + } + } + } + } + + return err +} + func (r *Registry) maybeCancelJobs(ctx context.Context, s sqlliveness.Session) { r.mu.Lock() defer r.mu.Unlock() diff --git a/pkg/jobs/registry_test.go b/pkg/jobs/registry_test.go index d21525aeb070..52a606f84bb3 100644 --- a/pkg/jobs/registry_test.go +++ b/pkg/jobs/registry_test.go @@ -123,11 +123,13 @@ func TestRegistryGC(t *testing.T) { }, UpgradeManager: &upgradebase.TestingKnobs{ // This test wants to look at job records. - DontUseJobs: true, + DontUseJobs: true, + SkipJobMetricsPollingJobBootstrap: true, }, KeyVisualizer: &keyvisualizer.TestingKnobs{ SkipJobBootstrap: true, }, + JobsTestingKnobs: NewTestingKnobsWithShortIntervals(), }, }) defer s.Stopper().Stop(ctx) @@ -271,11 +273,13 @@ func TestRegistryGCPagination(t *testing.T) { }, UpgradeManager: &upgradebase.TestingKnobs{ // This test wants to count job records. - DontUseJobs: true, + DontUseJobs: true, + SkipJobMetricsPollingJobBootstrap: true, }, KeyVisualizer: &keyvisualizer.TestingKnobs{ SkipJobBootstrap: true, }, + JobsTestingKnobs: NewTestingKnobsWithShortIntervals(), }, }) db := sqlutils.MakeSQLRunner(sqlDB) @@ -477,18 +481,13 @@ func TestRetriesWithExponentialBackoff(t *testing.T) { bti.clock = timeutil.NewManualTime(timeutil.Now()) timeSource := hlc.NewClock(bti.clock, base.DefaultMaxClockOffset) // Set up the test cluster. - knobs := &TestingKnobs{ - TimeSource: timeSource, - } + // Set a small adopt and cancel intervals to reduce test time. + knobs := NewTestingKnobsWithIntervals(unitTime, unitTime, initialDelay, maxDelay) + knobs.TimeSource = timeSource if bti.afterJobStateMachineKnob != nil { knobs.AfterJobStateMachine = bti.afterJobStateMachineKnob } cs := cluster.MakeTestingClusterSettings() - // Set a small adopt and cancel intervals to reduce test time. - adoptIntervalSetting.Override(ctx, &cs.SV, unitTime) - cancelIntervalSetting.Override(ctx, &cs.SV, unitTime) - retryInitialDelaySetting.Override(ctx, &cs.SV, initialDelay) - retryMaxDelaySetting.Override(ctx, &cs.SV, maxDelay) args := base.TestServerArgs{ Settings: cs, Knobs: base.TestingKnobs{ @@ -501,7 +500,8 @@ func TestRetriesWithExponentialBackoff(t *testing.T) { ManagerDisableJobCreation: true, }, UpgradeManager: &upgradebase.TestingKnobs{ - DontUseJobs: true, + DontUseJobs: true, + SkipJobMetricsPollingJobBootstrap: true, }, KeyVisualizer: &keyvisualizer.TestingKnobs{ SkipJobBootstrap: true, diff --git a/pkg/jobs/testing_knobs.go b/pkg/jobs/testing_knobs.go index 1c85dbc55afa..81382bf5d4fe 100644 --- a/pkg/jobs/testing_knobs.go +++ b/pkg/jobs/testing_knobs.go @@ -116,15 +116,17 @@ type TestingIntervalOverrides struct { WaitForJobsMaxDelay *time.Duration } +const defaultShortInterval = 10 * time.Millisecond + // NewTestingKnobsWithShortIntervals return a TestingKnobs structure with -// overrides for short adopt and cancel intervals. +// overrides for short adopt, cancel, and retry intervals. func NewTestingKnobsWithShortIntervals() *TestingKnobs { - defaultShortInterval := 10 * time.Millisecond + interval := defaultShortInterval if util.RaceEnabled { - defaultShortInterval *= 5 + interval *= 5 } return NewTestingKnobsWithIntervals( - defaultShortInterval, defaultShortInterval, defaultShortInterval, defaultShortInterval, + interval, interval, interval, interval, ) } diff --git a/pkg/jobs/update.go b/pkg/jobs/update.go index 6a9cf39ad3b3..da44c4066c31 100644 --- a/pkg/jobs/update.go +++ b/pkg/jobs/update.go @@ -331,9 +331,9 @@ func UpdateHighwaterProgressed(highWater hlc.Timestamp, md JobMetadata, ju *JobU // if md.Status != StatusRunning { // return errors.New("job no longer running") // } -// md.UpdateStatus(StatusPaused) +// ju.UpdateStatus(StatusPaused) // // -// md.UpdatePayload(md.Payload) +// ju.UpdatePayload(md.Payload) // } // // Note that there are various convenience wrappers (like FractionProgressed) diff --git a/pkg/sql/BUILD.bazel b/pkg/sql/BUILD.bazel index 521f196eedbf..8c00bf7abb55 100644 --- a/pkg/sql/BUILD.bazel +++ b/pkg/sql/BUILD.bazel @@ -136,6 +136,7 @@ go_library( "inverted_join.go", "job_exec_context.go", "job_exec_context_test_util.go", + "job_statistics.go", "jobs_collection.go", "join.go", "join_predicate.go", diff --git a/pkg/sql/job_statistics.go b/pkg/sql/job_statistics.go new file mode 100644 index 000000000000..436e92c4a9bd --- /dev/null +++ b/pkg/sql/job_statistics.go @@ -0,0 +1,50 @@ +// Copyright 2023 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 sql + +import ( + "context" + + "github.com/cockroachdb/cockroach/pkg/jobs" + "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" +) + +type metricsPoller struct { + job *jobs.Job +} + +var _ jobs.Resumer = &metricsPoller{} + +// OnFailOrCancel is a part of the Resumer interface. +func (mp *metricsPoller) OnFailOrCancel( + ctx context.Context, execCtx interface{}, jobErr error, +) error { + return nil +} + +// Resume is part of the Resumer interface. +func (mp *metricsPoller) Resume(ctx context.Context, execCtx interface{}) error { + // The metrics polling job is a forever running background job. It's always + // safe to wind the SQL pod down whenever it's running, something we + // indicate through the job's idle status. + mp.job.MarkIdle(true) + + exec := execCtx.(JobExecContext) + return exec.ExecCfg().JobRegistry.PollMetricsTask(ctx) +} + +func init() { + createResumerFn := func(job *jobs.Job, settings *cluster.Settings) jobs.Resumer { + return &metricsPoller{job: job} + } + jobs.RegisterConstructor(jobspb.TypePollJobsStats, createResumerFn, jobs.UsesTenantCostControl) +} diff --git a/pkg/sql/logictest/testdata/logic_test/jobs b/pkg/sql/logictest/testdata/logic_test/jobs index d580755f8358..643ef09c5464 100644 --- a/pkg/sql/logictest/testdata/logic_test/jobs +++ b/pkg/sql/logictest/testdata/logic_test/jobs @@ -194,3 +194,9 @@ usage_count > 0 ORDER BY feature_name DESC ---- job.schema_change.successful + +# Ensure one POLL JOBS STATS job is running +query I +SELECT count(*) FROM [SHOW AUTOMATIC JOBS] WHERE job_type = 'POLL JOBS STATS' AND status = 'running' +---- +1 diff --git a/pkg/ts/catalog/chart_catalog.go b/pkg/ts/catalog/chart_catalog.go index c4083bcfc229..1639de5d793c 100644 --- a/pkg/ts/catalog/chart_catalog.go +++ b/pkg/ts/catalog/chart_catalog.go @@ -3391,6 +3391,7 @@ var charts = []sectionDescription{ "jobs.auto_sql_stats_compaction.currently_running", "jobs.stream_replication.currently_running", "jobs.key_visualizer.currently_running", + "jobs.poll_jobs_stats.currently_running", }, }, { @@ -3412,6 +3413,31 @@ var charts = []sectionDescription{ "jobs.stream_replication.currently_idle", "jobs.typedesc_schema_change.currently_idle", "jobs.key_visualizer.currently_idle", + "jobs.poll_jobs_stats.currently_idle", + }, + }, + { + Title: "Currently Paused", + Metrics: []string{ + "jobs.auto_create_stats.currently_paused", + "jobs.auto_span_config_reconciliation.currently_paused", + "jobs.auto_sql_stats_compaction.currently_paused", + "jobs.backup.currently_paused", + "jobs.changefeed.currently_paused", + "jobs.create_stats.currently_paused", + "jobs.import.currently_paused", + "jobs.migration.currently_paused", + "jobs.new_schema_change.currently_paused", + "jobs.restore.currently_paused", + "jobs.schema_change.currently_paused", + "jobs.schema_change_gc.currently_paused", + "jobs.stream_ingestion.currently_paused", + "jobs.stream_replication.currently_paused", + "jobs.typedesc_schema_change.currently_paused", + "jobs.auto_schema_telemetry.currently_paused", + "jobs.row_level_ttl.currently_paused", + "jobs.poll_jobs_stats.currently_paused", + "jobs.key_visualizer.currently_paused", }, }, { @@ -3599,6 +3625,17 @@ var charts = []sectionDescription{ "jobs.key_visualizer.resume_retry_error", }, }, + { + Title: "Jobs Stats Polling Job", + Metrics: []string{ + "jobs.poll_jobs_stats.fail_or_cancel_completed", + "jobs.poll_jobs_stats.fail_or_cancel_failed", + "jobs.poll_jobs_stats.fail_or_cancel_retry_error", + "jobs.poll_jobs_stats.resume_completed", + "jobs.poll_jobs_stats.resume_failed", + "jobs.poll_jobs_stats.resume_retry_error", + }, + }, }, }, { diff --git a/pkg/upgrade/upgradebase/testing_knobs.go b/pkg/upgrade/upgradebase/testing_knobs.go index f266aedfb6c6..6019fa51a38c 100644 --- a/pkg/upgrade/upgradebase/testing_knobs.go +++ b/pkg/upgrade/upgradebase/testing_knobs.go @@ -33,6 +33,11 @@ type TestingKnobs struct { // production. DontUseJobs bool + // SkipJobMetricsPollingJobBootstrap, if set, disables the + // clusterversion.V23_1_CreateJobsMetricsPollingJob upgrade, which prevents a + // job from being created. + SkipJobMetricsPollingJobBootstrap bool + // AfterRunPermanentUpgrades is called after each call to // RunPermanentUpgrades. AfterRunPermanentUpgrades func() diff --git a/pkg/upgrade/upgrademanager/manager.go b/pkg/upgrade/upgrademanager/manager.go index 844debdb6f83..2a6d601b3ce1 100644 --- a/pkg/upgrade/upgrademanager/manager.go +++ b/pkg/upgrade/upgrademanager/manager.go @@ -556,6 +556,7 @@ func (m *Manager) runMigration( LeaseManager: m.lm, InternalExecutor: m.ie, JobRegistry: m.jr, + TestingKnobs: &m.knobs, }); err != nil { return err } diff --git a/pkg/upgrade/upgrades/BUILD.bazel b/pkg/upgrade/upgrades/BUILD.bazel index 5fd374a39dcf..ce79cad06b8f 100644 --- a/pkg/upgrade/upgrades/BUILD.bazel +++ b/pkg/upgrade/upgrades/BUILD.bazel @@ -10,6 +10,7 @@ go_library( "alter_statement_statistics_index_recommendations.go", "alter_table_statistics_partial_predicate_and_id.go", "create_index_usage_statement_statistics.go", + "create_jobs_metrics_polling_job.go", "delete_descriptors_of_dropped_functions.go", "desc_id_sequence_for_system_tenant.go", "descriptor_utils.go", @@ -90,6 +91,7 @@ go_test( "alter_table_statistics_partial_predicate_and_id_test.go", "builtins_test.go", "create_index_usage_statement_statistics_test.go", + "create_jobs_metrics_polling_job_test.go", "delete_descriptors_of_dropped_functions_test.go", "desc_id_sequence_for_system_tenant_test.go", "descriptor_utils_test.go", diff --git a/pkg/upgrade/upgrades/create_jobs_metrics_polling_job.go b/pkg/upgrade/upgrades/create_jobs_metrics_polling_job.go new file mode 100644 index 000000000000..332b795fd602 --- /dev/null +++ b/pkg/upgrade/upgrades/create_jobs_metrics_polling_job.go @@ -0,0 +1,60 @@ +// Copyright 2022 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 upgrades + +import ( + "context" + + "github.com/cockroachdb/cockroach/pkg/clusterversion" + "github.com/cockroachdb/cockroach/pkg/jobs" + "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" + "github.com/cockroachdb/cockroach/pkg/security/username" + "github.com/cockroachdb/cockroach/pkg/sql/isql" + "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" + "github.com/cockroachdb/cockroach/pkg/upgrade" +) + +func createJobsMetricsPollingJob( + ctx context.Context, _ clusterversion.ClusterVersion, d upgrade.TenantDeps, +) error { + if d.TestingKnobs != nil && d.TestingKnobs.SkipJobMetricsPollingJobBootstrap { + return nil + } + return d.DB.Txn(ctx, func(ctx context.Context, txn isql.Txn) error { + row, err := d.DB.Executor().QueryRowEx( + ctx, + "check for existing job metrics polling job", + nil, + sessiondata.InternalExecutorOverride{User: username.RootUserName()}, + "SELECT * FROM system.jobs WHERE id = $1", + jobs.JobMetricsPollerJobID, + ) + if err != nil { + return err + } + + if row == nil { + jr := jobs.Record{ + JobID: jobs.JobMetricsPollerJobID, + Description: jobspb.TypePollJobsStats.String(), + Details: jobspb.PollJobsStatsDetails{}, + Progress: jobspb.PollJobsStatsProgress{}, + CreatedBy: &jobs.CreatedByInfo{Name: username.RootUser, ID: username.RootUserID}, + Username: username.RootUserName(), + NonCancelable: true, + } + if _, err := d.JobRegistry.CreateAdoptableJobWithTxn(ctx, jr, jobs.JobMetricsPollerJobID, txn); err != nil { + return err + } + } + return nil + }) +} diff --git a/pkg/upgrade/upgrades/create_jobs_metrics_polling_job_test.go b/pkg/upgrade/upgrades/create_jobs_metrics_polling_job_test.go new file mode 100644 index 000000000000..d729c454301c --- /dev/null +++ b/pkg/upgrade/upgrades/create_jobs_metrics_polling_job_test.go @@ -0,0 +1,69 @@ +// Copyright 2022 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 upgrades_test + +import ( + "context" + "testing" + + "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/clusterversion" + "github.com/cockroachdb/cockroach/pkg/server" + "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" + "github.com/cockroachdb/cockroach/pkg/upgrade/upgrades" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" +) + +func TestCreateJobsMetricsPollingJob(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + clusterArgs := base.TestClusterArgs{ + ServerArgs: base.TestServerArgs{ + Knobs: base.TestingKnobs{ + Server: &server.TestingKnobs{ + DisableAutomaticVersionUpgrade: make(chan struct{}), + BinaryVersionOverride: clusterversion.ByKey( + clusterversion.V23_1_CreateJobsMetricsPollingJob - 1), + }, + }, + }, + } + + var ( + ctx = context.Background() + tc = testcluster.StartTestCluster(t, 1, clusterArgs) + sqlDB = tc.ServerConn(0) + ) + defer tc.Stopper().Stop(ctx) + + var count int + row := sqlDB.QueryRow("SELECT count(*) FROM crdb_internal.jobs WHERE job_type = 'POLL STATS JOB'") + err := row.Scan(&count) + require.NoError(t, err) + assert.Equal(t, count, 0) + + upgrades.Upgrade( + t, + sqlDB, + clusterversion.V23_1_CreateJobsMetricsPollingJob, + nil, /* done */ + false, /* expectError */ + ) + + row = sqlDB.QueryRow("SELECT count(*) FROM crdb_internal.jobs WHERE job_type = 'POLL JOBS STATS'") + err = row.Scan(&count) + require.NoError(t, err) + assert.Equal(t, count, 1) +} diff --git a/pkg/upgrade/upgrades/upgrades.go b/pkg/upgrade/upgrades/upgrades.go index af59aaa64f77..65a1b66c876a 100644 --- a/pkg/upgrade/upgrades/upgrades.go +++ b/pkg/upgrade/upgrades/upgrades.go @@ -263,6 +263,11 @@ var upgrades = []upgradebase.Upgrade{ upgrade.NoPrecondition, deleteDescriptorsOfDroppedFunctions, ), + upgrade.NewPermanentTenantUpgrade("create jobs metrics polling job", + toCV(clusterversion.V23_1_CreateJobsMetricsPollingJob), + createJobsMetricsPollingJob, + "create jobs metrics polling job", + ), } func init() { From 00fd3ef1fadde1d01f28a44d2bbeb65e3ba64f06 Mon Sep 17 00:00:00 2001 From: Evan Wall Date: Mon, 6 Feb 2023 17:31:56 -0500 Subject: [PATCH 15/16] sql: wrap stacktraceless errors with errors.Wrap Fixes #95794 This replaces the previous attempt to add logging here #95797. The context itself cannot be augmented to add a stack trace to errors because it interferes with grpc timeout logic - gRPC compares errors directly without checking causes https://github.com/grpc/grpc-go/blob/v1.46.0/rpc_util.go#L833. Although the method signature allows it, `Context.Err()` should not be overriden to customize the error: ``` // If Done is not yet closed, Err returns nil. // If Done is closed, Err returns a non-nil error explaining why: // Canceled if the context was canceled // or DeadlineExceeded if the context's deadline passed. // After Err returns a non-nil error, successive calls to Err return the same error. Err() error ``` Additionally, a child context of the augmented context may end up being used which will circumvent the stack trace capture. This change instead wraps `errors.Wrap` in a few places that might end up helping debug the original problem: 1) Where we call `Context.Err()` directly. 2) Where gRPC returns an error after possibly calling `Context.Err()` internally or returns an error that does not have a stack trace. Release note: None --- pkg/ccl/kvccl/kvtenantccl/connector.go | 17 +++++----- pkg/kv/kvclient/kvcoord/transport.go | 5 ++- pkg/kv/kvnemesis/applier_test.go | 3 +- pkg/kv/txn.go | 2 +- pkg/rpc/context.go | 6 ++-- pkg/rpc/nodedialer/nodedialer.go | 11 ++++--- pkg/server/init.go | 2 +- pkg/sql/internal_result_channel.go | 26 +++++++++------ pkg/sql/user.go | 8 ++--- pkg/util/limit/limiter.go | 2 +- pkg/util/tracing/grpcinterceptor/BUILD.bazel | 1 + .../grpcinterceptor/grpc_interceptor.go | 32 +++++++++---------- 12 files changed, 63 insertions(+), 52 deletions(-) diff --git a/pkg/ccl/kvccl/kvtenantccl/connector.go b/pkg/ccl/kvccl/kvtenantccl/connector.go index 773926dcc4f7..e0ae0d25f253 100644 --- a/pkg/ccl/kvccl/kvtenantccl/connector.go +++ b/pkg/ccl/kvccl/kvtenantccl/connector.go @@ -461,7 +461,7 @@ func (c *Connector) RangeLookup( } return resp.Descriptors, resp.PrefetchedDescriptors, nil } - return nil, nil, ctx.Err() + return nil, nil, errors.Wrap(ctx.Err(), "range lookup") } // NodesUI implements the serverpb.TenantStatusServer interface @@ -544,7 +544,7 @@ func (c *Connector) NewIterator( rangeDescriptors = append(rangeDescriptors, e.RangeDescriptors...) } } - return nil, ctx.Err() + return nil, errors.Wrap(ctx.Err(), "new iterator") } // TokenBucket implements the kvtenant.TokenBucketProvider interface. @@ -575,7 +575,7 @@ func (c *Connector) TokenBucket( } return resp, nil } - return nil, ctx.Err() + return nil, errors.Wrap(ctx.Err(), "token bucket") } // GetSpanConfigRecords implements the spanconfig.KVAccessor interface. @@ -587,7 +587,7 @@ func (c *Connector) GetSpanConfigRecords( Targets: spanconfig.TargetsToProtos(targets), }) if err != nil { - return err + return errors.Wrap(err, "get span configs error") } records, err = spanconfig.EntriesToRecords(resp.SpanConfigEntries) @@ -617,7 +617,7 @@ func (c *Connector) UpdateSpanConfigRecords( MaxCommitTimestamp: maxCommitTS, }) if err != nil { - return err + return errors.Wrap(err, "update span configs error") } if resp.Error.IsSet() { // Logical error; propagate as such. @@ -655,13 +655,12 @@ func (c *Connector) GetAllSystemSpanConfigsThatApply( ) ([]roachpb.SpanConfig, error) { var spanConfigs []roachpb.SpanConfig if err := c.withClient(ctx, func(ctx context.Context, c *client) error { - var err error resp, err := c.GetAllSystemSpanConfigsThatApply( ctx, &roachpb.GetAllSystemSpanConfigsThatApplyRequest{ TenantID: id, }) if err != nil { - return err + return errors.Wrap(err, "get all system span configs that apply error") } spanConfigs = resp.SpanConfigs @@ -713,7 +712,7 @@ func (c *Connector) withClient( } return f(ctx, c) } - return ctx.Err() + return errors.Wrap(ctx.Err(), "with client") } // getClient returns the singleton InternalClient if one is currently active. If @@ -778,7 +777,7 @@ func (c *Connector) dialAddrs(ctx context.Context) (*client, error) { }, nil } } - return nil, ctx.Err() + return nil, errors.Wrap(ctx.Err(), "dial addrs") } func (c *Connector) dialAddr(ctx context.Context, addr string) (conn *grpc.ClientConn, err error) { diff --git a/pkg/kv/kvclient/kvcoord/transport.go b/pkg/kv/kvclient/kvcoord/transport.go index cefb03e72aa7..641f0ef26dc7 100644 --- a/pkg/kv/kvclient/kvcoord/transport.go +++ b/pkg/kv/kvclient/kvcoord/transport.go @@ -235,7 +235,10 @@ func (gt *grpcTransport) sendBatch( } span.ImportRemoteRecording(reply.CollectedSpans) } - return reply, err + if err != nil { + return nil, errors.Wrapf(err, "ba: %s RPC error", ba.String()) + } + return reply, nil } // NextInternalClient returns the next InternalClient to use for performing diff --git a/pkg/kv/kvnemesis/applier_test.go b/pkg/kv/kvnemesis/applier_test.go index 0125e9d455df..3193d284d3db 100644 --- a/pkg/kv/kvnemesis/applier_test.go +++ b/pkg/kv/kvnemesis/applier_test.go @@ -91,7 +91,6 @@ func TestApplier(t *testing.T) { "batch", step(batch(put(k1, 21), delRange(k2, k3, 22))), }, { - "rscan", step(reverseScan(k1, k3)), }, { @@ -200,7 +199,7 @@ func TestApplier(t *testing.T) { // Trim out context canceled location, which can be non-deterministic. // The wrapped string around the context canceled error depends on where // the context cancellation was noticed. - actual = regexp.MustCompile(` aborted .*: context canceled`).ReplaceAllString(actual, ` context canceled`) + actual = regexp.MustCompile(` (aborted .*|txn exec): context canceled`).ReplaceAllString(actual, ` context canceled`) } else { // Trim out the txn to avoid nondeterminism. actual = regexp.MustCompile(` txnpb:\(.*\)`).ReplaceAllLiteralString(actual, ` txnpb:`) diff --git a/pkg/kv/txn.go b/pkg/kv/txn.go index 5e0819bafce0..537494c45d32 100644 --- a/pkg/kv/txn.go +++ b/pkg/kv/txn.go @@ -922,7 +922,7 @@ func (txn *Txn) exec(ctx context.Context, fn func(context.Context, *Txn) error) // error condition this loop isn't capable of handling. for { if err := ctx.Err(); err != nil { - return err + return errors.Wrap(err, "txn exec") } err = fn(ctx, txn) diff --git a/pkg/rpc/context.go b/pkg/rpc/context.go index 2426de524182..0d81a39e8299 100644 --- a/pkg/rpc/context.go +++ b/pkg/rpc/context.go @@ -325,7 +325,7 @@ func (c *Connection) Connect(ctx context.Context) (*grpc.ClientConn, error) { select { case <-c.initialHeartbeatDone: case <-ctx.Done(): - return nil, ctx.Err() + return nil, errors.Wrap(ctx.Err(), "connect") } if err, _ := c.err.Load().(error); err != nil { @@ -1328,7 +1328,7 @@ func (s *pipe) send(ctx context.Context, m interface{}) error { case s.respC <- m: return nil case <-ctx.Done(): - return ctx.Err() + return errors.Wrap(ctx.Err(), "send") } } @@ -1352,7 +1352,7 @@ func (s *pipe) recv(ctx context.Context) (interface{}, error) { return nil, err } case <-ctx.Done(): - return nil, ctx.Err() + return nil, errors.Wrap(ctx.Err(), "recv") } } diff --git a/pkg/rpc/nodedialer/nodedialer.go b/pkg/rpc/nodedialer/nodedialer.go index 3fc01ac84e7e..d8a409d77662 100644 --- a/pkg/rpc/nodedialer/nodedialer.go +++ b/pkg/rpc/nodedialer/nodedialer.go @@ -105,7 +105,7 @@ func (n *Dialer) Dial( } // Don't trip the breaker if we're already canceled. if ctxErr := ctx.Err(); ctxErr != nil { - return nil, ctxErr + return nil, errors.Wrap(ctxErr, "dial") } breaker := n.getBreaker(nodeID, class) addr, err := n.resolver(nodeID) @@ -160,14 +160,14 @@ func (n *Dialer) DialInternalClient( addr, err := n.resolver(nodeID) if err != nil { - return nil, err + return nil, errors.Wrap(err, "resolver error") } log.VEventf(ctx, 2, "sending request to %s", addr) conn, err := n.dial(ctx, nodeID, addr, n.getBreaker(nodeID, class), true /* checkBreaker */, class) if err != nil { return nil, err } - return TracingInternalClient{InternalClient: roachpb.NewInternalClient(conn)}, err + return TracingInternalClient{InternalClient: roachpb.NewInternalClient(conn)}, nil } // dial performs the dialing of the remote connection. If breaker is nil, @@ -180,9 +180,10 @@ func (n *Dialer) dial( checkBreaker bool, class rpc.ConnectionClass, ) (_ *grpc.ClientConn, err error) { + const ctxWrapMsg = "dial" // Don't trip the breaker if we're already canceled. if ctxErr := ctx.Err(); ctxErr != nil { - return nil, ctxErr + return nil, errors.Wrap(ctxErr, ctxWrapMsg) } if checkBreaker && !breaker.Ready() { err = errors.Wrapf(circuit.ErrBreakerOpen, "unable to dial n%d", nodeID) @@ -198,7 +199,7 @@ func (n *Dialer) dial( if err != nil { // If we were canceled during the dial, don't trip the breaker. if ctxErr := ctx.Err(); ctxErr != nil { - return nil, ctxErr + return nil, errors.Wrap(ctxErr, ctxWrapMsg) } err = errors.Wrapf(err, "failed to connect to n%d at %v", nodeID, addr) if breaker != nil { diff --git a/pkg/server/init.go b/pkg/server/init.go index 5665f235bbb5..785afd7e9e21 100644 --- a/pkg/server/init.go +++ b/pkg/server/init.go @@ -471,7 +471,7 @@ func (s *initServer) attemptJoinTo( if err != nil { status, ok := grpcstatus.FromError(errors.UnwrapAll(err)) if !ok { - return nil, err + return nil, errors.Wrap(err, "failed to join cluster") } // TODO(irfansharif): Here we're logging the error and also returning diff --git a/pkg/sql/internal_result_channel.go b/pkg/sql/internal_result_channel.go index d8e4a282dddc..d85cb8bf370e 100644 --- a/pkg/sql/internal_result_channel.go +++ b/pkg/sql/internal_result_channel.go @@ -111,14 +111,16 @@ func newSyncIEResultChannel() *ieResultChannel { func (i *ieResultChannel) firstResult( ctx context.Context, ) (_ ieIteratorResult, done bool, err error) { + // errors.Wrap returns nil if ctx.Err() is nil. + const wrapMsg = "failed to read query result" select { case <-ctx.Done(): - return ieIteratorResult{}, true, ctx.Err() + return ieIteratorResult{}, true, errors.Wrap(ctx.Err(), wrapMsg) case <-i.doneCh: - return ieIteratorResult{}, true, ctx.Err() + return ieIteratorResult{}, true, errors.Wrap(ctx.Err(), wrapMsg) case res, ok := <-i.dataCh: if !ok { - return ieIteratorResult{}, true, ctx.Err() + return ieIteratorResult{}, true, errors.Wrap(ctx.Err(), wrapMsg) } return res, false, nil } @@ -128,11 +130,13 @@ func (i *ieResultChannel) maybeUnblockWriter(ctx context.Context) (done bool, er if i.async() { return false, nil } + // errors.Wrap returns nil if ctx.Err() is nil. + const wrapMsg = "maybe unblock writer" select { case <-ctx.Done(): - return true, ctx.Err() + return true, errors.Wrap(ctx.Err(), wrapMsg) case <-i.doneCh: - return true, ctx.Err() + return true, errors.Wrap(ctx.Err(), wrapMsg) case i.waitCh <- struct{}{}: return false, nil } @@ -181,13 +185,15 @@ func (i *ieResultChannel) close() error { var errIEResultChannelClosed = errors.New("ieResultReader closed") func (i *ieResultChannel) addResult(ctx context.Context, result ieIteratorResult) error { + // errors.Wrap returns nil if ctx.Err() is nil. + const wrapMsg = "add result" select { case <-ctx.Done(): - return ctx.Err() + return errors.Wrap(ctx.Err(), wrapMsg) case <-i.doneCh: // Prefer the context error if there is one. if ctxErr := ctx.Err(); ctxErr != nil { - return ctxErr + return errors.Wrap(ctx.Err(), wrapMsg) } return errIEResultChannelClosed case i.dataCh <- result: @@ -196,16 +202,18 @@ func (i *ieResultChannel) addResult(ctx context.Context, result ieIteratorResult } func (i *ieResultChannel) maybeBlock(ctx context.Context) error { + // errors.Wrap returns nil if ctx.Err() is nil. + const wrapMsg = "maybe block" if i.async() { return nil } select { case <-ctx.Done(): - return ctx.Err() + return errors.Wrap(ctx.Err(), wrapMsg) case <-i.doneCh: // Prefer the context error if there is one. if ctxErr := ctx.Err(); ctxErr != nil { - return ctxErr + return errors.Wrap(ctxErr, wrapMsg) } return errIEResultChannelClosed case <-i.waitCh: diff --git a/pkg/sql/user.go b/pkg/sql/user.go index 1e95a701650a..ca36a4b14d32 100644 --- a/pkg/sql/user.go +++ b/pkg/sql/user.go @@ -84,7 +84,7 @@ func GetUserSessionInitInfo( pwRetrieveFn func(ctx context.Context) (expired bool, hashedPassword password.PasswordHash, err error), err error, ) { - runFn := getUserInfoRunFn(execCfg, user, "get-user-timeout") + runFn := getUserInfoRunFn(execCfg, user, "get-user-session") if user.IsRootUser() { // As explained above, for root we report that the user exists @@ -225,7 +225,7 @@ func retrieveSessionInitInfoWithCache( retrieveAuthInfo, ) if retErr != nil { - return retErr + return errors.Wrap(retErr, "get auth info error") } // Avoid looking up default settings for root and non-existent users. if userName.IsRootUser() || !aInfo.UserExists { @@ -239,7 +239,7 @@ func retrieveSessionInitInfoWithCache( databaseName, retrieveDefaultSettings, ) - return retErr + return errors.Wrap(retErr, "get default settings error") }(); err != nil { // Failed to retrieve the user account. Report in logs for later investigation. log.Warningf(ctx, "user lookup for %q failed: %v", userName, err) @@ -706,7 +706,7 @@ func updateUserPasswordHash( userName username.SQLUsername, prevHash, newHash []byte, ) error { - runFn := getUserInfoRunFn(execCfg, userName, "set-hash-timeout") + runFn := getUserInfoRunFn(execCfg, userName, "set-user-password-hash") return runFn(ctx, func(ctx context.Context) error { return DescsTxn(ctx, execCfg, func(ctx context.Context, txn isql.Txn, d *descs.Collection) error { diff --git a/pkg/util/limit/limiter.go b/pkg/util/limit/limiter.go index b36a313e2b5f..264d94afed28 100644 --- a/pkg/util/limit/limiter.go +++ b/pkg/util/limit/limiter.go @@ -46,7 +46,7 @@ func MakeConcurrentRequestLimiter(spanName string, limit int) ConcurrentRequestL // is forced to block. func (l *ConcurrentRequestLimiter) Begin(ctx context.Context) (Reservation, error) { if err := ctx.Err(); err != nil { - return nil, err + return nil, errors.Wrap(err, "limiter begin") } res, err := l.sem.TryAcquire(ctx, 1) diff --git a/pkg/util/tracing/grpcinterceptor/BUILD.bazel b/pkg/util/tracing/grpcinterceptor/BUILD.bazel index 7e2567e11a15..7f04768b771f 100644 --- a/pkg/util/tracing/grpcinterceptor/BUILD.bazel +++ b/pkg/util/tracing/grpcinterceptor/BUILD.bazel @@ -9,6 +9,7 @@ go_library( deps = [ "//pkg/util/grpcutil", "//pkg/util/tracing", + "@com_github_cockroachdb_errors//:errors", "@io_opentelemetry_go_otel//attribute", "@io_opentelemetry_go_otel//codes", "@org_golang_google_grpc//:go_default_library", diff --git a/pkg/util/tracing/grpcinterceptor/grpc_interceptor.go b/pkg/util/tracing/grpcinterceptor/grpc_interceptor.go index 94fc1b9085b4..922060544072 100644 --- a/pkg/util/tracing/grpcinterceptor/grpc_interceptor.go +++ b/pkg/util/tracing/grpcinterceptor/grpc_interceptor.go @@ -17,6 +17,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/grpcutil" "github.com/cockroachdb/cockroach/pkg/util/tracing" + "github.com/cockroachdb/errors" "go.opentelemetry.io/otel/attribute" "go.opentelemetry.io/otel/codes" "google.golang.org/grpc" @@ -90,7 +91,7 @@ func ServerInterceptor(tracer *tracing.Tracer) grpc.UnaryServerInterceptor { req interface{}, info *grpc.UnaryServerInfo, handler grpc.UnaryHandler, - ) (resp interface{}, err error) { + ) (interface{}, error) { if methodExcludedFromTracing(info.FullMethod) { return handler(ctx, req) } @@ -111,7 +112,7 @@ func ServerInterceptor(tracer *tracing.Tracer) grpc.UnaryServerInterceptor { ) defer serverSpan.Finish() - resp, err = handler(ctx, req) + resp, err := handler(ctx, req) if err != nil { setGRPCErrorTag(serverSpan, err) serverSpan.Recordf("error: %s", err) @@ -243,15 +244,15 @@ func ClientInterceptor( if !methodExcludedFromTracing(method) { ctx = injectSpanMeta(ctx, tracer, clientSpan) } - var err error if invoker != nil { - err = invoker(ctx, method, req, resp, cc, opts...) + err := invoker(ctx, method, req, resp, cc, opts...) + if err != nil { + setGRPCErrorTag(clientSpan, err) + clientSpan.Recordf("error: %s", err) + return err + } } - if err != nil { - setGRPCErrorTag(clientSpan, err) - clientSpan.Recordf("error: %s", err) - } - return err + return nil } } @@ -377,7 +378,7 @@ func (cs *tracingClientStream) Header() (metadata.MD, error) { if err != nil { cs.finishFunc(err) } - return md, err + return md, errors.Wrap(err, "header error") } func (cs *tracingClientStream) SendMsg(m interface{}) error { @@ -385,22 +386,21 @@ func (cs *tracingClientStream) SendMsg(m interface{}) error { if err != nil { cs.finishFunc(err) } - return err + return errors.Wrap(err, "send msg error") } func (cs *tracingClientStream) RecvMsg(m interface{}) error { err := cs.ClientStream.RecvMsg(m) if err == io.EOF { cs.finishFunc(nil) + // Do not wrap EOF. return err } else if err != nil { cs.finishFunc(err) - return err - } - if !cs.desc.ServerStreams { + } else if !cs.desc.ServerStreams { cs.finishFunc(nil) } - return err + return errors.Wrap(err, "recv msg error") } func (cs *tracingClientStream) CloseSend() error { @@ -408,5 +408,5 @@ func (cs *tracingClientStream) CloseSend() error { if err != nil { cs.finishFunc(err) } - return err + return errors.Wrap(err, "close send error") } From 866d58a62d792f963ff963235406b53b4ccf597f Mon Sep 17 00:00:00 2001 From: Tobias Grieger Date: Tue, 3 Jan 2023 13:49:48 +0100 Subject: [PATCH 16/16] kvserver: improve reproposal assertions and documentation Reproposals are a deep rabbit hole and an area in which past changes were all related to subtle bugs. Write it all up and in particular make some simplifications that ought to be possible if my understanding is correct: - have proposals always enter `(*Replica).propose` without a MaxLeaseIndex or prior encoded command set, i.e. `propose` behaves the same for reproposals as for first proposals. - assert that after a failed call to tryReproposeWithNewLeaseIndex, the command is not in the proposals map, i.e. check absence of a leak. - replace code that should be impossible to reach (and had me confused for a good amount of time) with an assertion. - add long comment on `r.mu.proposals`. This commit also moves `tryReproposeWithNewLeaseIndex` off `(*Replica)`, which is possible due to recent changes[^1]. In doing so, I realized there was a (small) data race (now fixed): when returning a `NotLeaseholderError` from that method, we weren't acquiring `r.mu`. It may have looked as though we were holding it already since we're accessing `r.mu.propBuf`, however that field has special semantics - it wraps `r.mu` and acquires it when needed. [^1]: The "below raft" test mentioned in the previous comment was changed in https://github.com/cockroachdb/cockroach/pull/93785 and no longer causes a false positive. Epic: CRDB-220 Release note: None --- pkg/kv/kvserver/replica.go | 156 +++++++++++++++++- .../kvserver/replica_application_decoder.go | 3 +- pkg/kv/kvserver/replica_application_result.go | 125 ++++++++++---- .../replica_application_state_machine.go | 34 ++-- pkg/kv/kvserver/replica_proposal.go | 14 ++ pkg/kv/kvserver/replica_raft.go | 47 ++++-- 6 files changed, 307 insertions(+), 72 deletions(-) diff --git a/pkg/kv/kvserver/replica.go b/pkg/kv/kvserver/replica.go index 270b137fd799..9ac412c457c8 100644 --- a/pkg/kv/kvserver/replica.go +++ b/pkg/kv/kvserver/replica.go @@ -498,9 +498,13 @@ type Replica struct { // Instead, the buffer internally holds a reference to mu and will use // it appropriately. proposalBuf propBuf - // proposals stores the Raft in-flight commands which originated at - // this Replica, i.e. all commands for which propose has been called, - // but which have not yet applied. + + // proposals stores the Raft in-flight commands which originated at this + // Replica, i.e. all commands for which propose has been called, but which + // have not yet applied. A proposal is "pending" until it is "finalized", + // meaning that `finishApplication` has been invoked on the proposal (which + // informs the client that the proposal has now been applied, optionally + // with an error, which may be an AmbiguousResultError). // // The *ProposalData in the map are "owned" by it. Elements from the // map must only be referenced while the Replica.mu is held, except @@ -509,12 +513,146 @@ type Replica struct { // underneath raft. See comments on ProposalData fields for synchronization // requirements. // - // Due to Raft reproposals, multiple in-flight Raft entries can have - // the same CmdIDKey, all corresponding to the same KV request. However, - // not all Raft entries with a given command ID will correspond directly - // to the *RaftCommand contained in its associated *ProposalData. This - // is because the *RaftCommand can be mutated during reproposals by - // Replica.tryReproposeWithNewLeaseIndex. + // Due to Raft reproposals, multiple in-flight Raft entries can have the + // same CmdIDKey. There are two kinds of reproposals: + // + // (1) the exact same entry is handed to raft (possibly despite already being + // present in the log), usually after a timeout[^1]. + // + // (2) an existing proposal is updated with a new MaxLeaseIndex and handed to + // raft, i.e. we're intentionally creating a duplicate. This exists because + // for pipelined proposals, the client's goroutine returns without waiting + // for the proposal to apply.[^2][^3] When (2) is carried out, the existing + // copies of the proposal in the log will be "Superseded", see below. Note + // that (2) will only be invoked for proposals that aren't currently in the + // proposals map any more because they're in the middle of being applied; + // as part of (2), they are re-added to the map. + // + // To understand reproposals, we need a broad overview of entry application, + // which is batched (i.e. may process multiple log entries to be applied in + // a batched fashion). In entry application, the following steps are taken: + // + // 1. retrieve all local proposals: iterate through the entries in order, + // and look them up in the proposals map. For each "local" entry (i.e. + // tracked in the map), remove it from the map (unless the proposal + // is not superseded, see below) and attach the value to the entry. + // 2. for each entry: + // - stage written and in-memory effects of the entry (some may apply as no-ops + // if they fail below-raft checks such as the MaxLeaseIndex check) + // - Assuming the MaxLeaseIndex is violated and additional constraints are + // satisfied, carry out (2) from above. On success, we know now that there + // will be a reproposal in the log that can successfully apply. We unbind + // the local proposal (so we don't signal it) and apply the current entry + // as a no-op. + // 3. carry out additional side effects of the entire batch (stats updates etc). + // + // A prerequisite for (2) is that there currently aren't any copies of the proposal + // in the log that may ultimately apply, or we risk doubly applying commands - a + // correctness bug. After (2), any copies of the entry present in the log will have + // a MaxLeaseIndex strictly less than that of the in-memory command, and will be + // Superseded() by it. + // + // We can always safely create an identical copy (i.e. (1)) because of the + // replay protection conferred by the MaxLeaseIndex - all but the first + // proposal (that reach consensus) will be rejected (i.e. apply as a no-op). + // + // Naively, one might hope that by invoking (2) upon applying an entry for + // a command that is rejected due to a MaxLeaseIndex one could achieve the + // invariant that there is only ever one unapplied copy of the entry in the + // log, and then the in-memory proposal could reflect the MaxLeaseIndex + // assigned to this unapplied copy at all times. + // + // Unfortunately, for various reasons, this invariant does not hold: + // - entry application isn't durable, so upon a restart, we might roll + // back to a log position that yet has to catch up over multiple previous + // incarnations of (2), i.e. we will see the same entry multiple times at + // various MaxLeaseIndex values. + // (This technically not a problem, since we're losing the in-memory proposal + // during the restart anyway, but should be kept in mind anyway). + // - Raft proposal forwarding due to (1)-type reproposals could "in + // principle" lead to an old copy of the entry appearing again in the + // unapplied log, at least if we make the reasonable assumption that + // forwarded proposals may arrive at the leader with arbitrary delays. + // + // As a result, we can't "just" invoke (2) when seeing a rejected command, + // we additionally have to verify that there isn't a more recent reproposal + // underway that could apply successfully and supersedes the one we're + // currently looking at. + // So we carry out (2) only if the MaxLeaseIndex of the in-mem proposal matches + // that of the current entry, and update the in-mem MaxLeaseIndex with the result + // of (2) if it did. + // + // An example follows. Consider the following situation (where N is some base + // index not relevant to the example) in which we have one inflight proposal which + // has been triplicated in the log (due to [^1]): + // + // proposals[id] = p{Cmd{MaxLeaseIndex: 100, ...}} + // + // ... (unrelated entries) + // raftlog[N] = Cmd{MaxLeaseIndex: 100, ...} + // ... (unrelated entries) + // raftlog[N+12] = (same as N) + // ... (unrelated entries) + // raftlog[N+15] = (same as N) + // + // where we assume that the `MaxLeaseIndex` 100 is invalid, i.e. when we see + // the first copy of the command being applied, we've already applied some + // command with equal or higher `MaxLeaseIndex`. In a world without + // mechanism (2), `N` would be rejected, and would finalize the proposal + // (i.e. signal the client with an error and remove the entry from + // `proposals`). Later, `N+12` and `N+15` would similarly be rejected (but + // they wouldn't even be regarded as local proposals any more due to not + // being present in `proposals`). + // + // However, (2) exists and it will engage during application of `N`: realizing + // that the current copies of the entry are all going to be rejected, it will + // modify the proposal by assigning a new `MaxLeaseIndex` to it, and handing + // it to `(*Replica).propose` again (which hands it to the proposal buffer, + // which will at some point flush it, leading to re-insertion into the raft + // log and the `proposals` map). The result will be this picture: + // + // proposals[id] = p{Cmd{MaxLeaseIndex: 192, ...}} <-- modified + // + // ... (unrelated entries) + // raftlog[N] = Cmd{MaxLeaseIndex: 100, ...} <-- applied (as no-op) + // ... (unrelated entries) + // raftlog[N+12] = (same as N) (superseded) + // ... (unrelated entries) + // raftlog[N+15] = (same as N) (superseded) + // ... (unrelated entries) + // raftlog[N+18] = Cmd{MaxLeaseIndex: 192, ...} <-- modified + // + // `N+18` might (in fact, is likely to) apply successfully. As a result, when + // we consider `N+12` or `N+15` for application, we must *not* carry out (2) + // again, or we break replay protection. In other words, the `MaxLeaseIndex` + // of the command being applied must be compared with the `MaxLeaseIndex` of + // the command in the proposals map; only if they match do we know that this + // is the most recent (in MaxLeaseIndex order) copy of the command, and only + // then can (2) engage. In addition, an entry that doesn't pass this equality + // check must not signal the proposer and/or unlink from the proposals map (as a + // newer reproposal which might succeed is likely in the log)[^4]. + // + // Another way of framing the above is that `proposals[id].Cmd.MaxLeaseIndex` + // actually tracks the maximum `MaxLeaseIndex` of all copies that may be present in + // the log. + // + // If (2) results in an error (for example, since the proposal now fails to + // respect the closed timestamp), that error will finalize the proposal and + // is returned to the client. + // + // [^1]: https://github.com/cockroachdb/cockroach/blob/59ce13b6052a99a0318e3dfe017908ff5630db30/pkg/kv/kvserver/replica_raft.go#L1224 + // [^2]: https://github.com/cockroachdb/cockroach/blob/59ce13b6052a99a0318e3dfe017908ff5630db30/pkg/kv/kvserver/replica_application_result.go#L148 + // [^3]: it's debatable how useful this below-raft reproposal mechanism is. + // It was introduced in https://github.com/cockroachdb/cockroach/pull/35261, + // and perhaps could be phased out again if we also did + // https://github.com/cockroachdb/cockroach/issues/21849. Historical + // evidence points to https://github.com/cockroachdb/cockroach/issues/28876 + // as the motivation for introducing this mechanism, i.e. it was about + // reducing failure rates early in the life of a cluster when raft + // leaderships were being determined. Perhaps we could "simply" disable + // async writes unless leadership was stable instead, by blocking on the + // proposal anyway. + // [^4]: https://github.com/cockroachdb/cockroach/blob/ab6a8650621ae798377f12bbfc1eee2fbec95480/pkg/kv/kvserver/replica_application_decoder.go#L100-L114 proposals map[kvserverbase.CmdIDKey]*ProposalData // Indicates that the replica is in the process of applying log entries. // Updated to true in handleRaftReady before entries are removed from diff --git a/pkg/kv/kvserver/replica_application_decoder.go b/pkg/kv/kvserver/replica_application_decoder.go index c4aeafc59a35..53fe8401e12e 100644 --- a/pkg/kv/kvserver/replica_application_decoder.go +++ b/pkg/kv/kvserver/replica_application_decoder.go @@ -111,7 +111,8 @@ func (d *replicaDecoder) retrieveLocalProposals(ctx context.Context) (anyLocal b // criterion. While such proposals can be reproposed, only the first // instance that gets applied matters and so removing the command is // always what we want to happen. - cmd.Cmd.MaxLeaseIndex == cmd.proposal.command.MaxLeaseIndex + !cmd.proposal.Supersedes(cmd.Cmd.MaxLeaseIndex) + if shouldRemove { // Delete the proposal from the proposals map. There may be reproposals // of the proposal in the pipeline, but those will all have the same max diff --git a/pkg/kv/kvserver/replica_application_result.go b/pkg/kv/kvserver/replica_application_result.go index 06164f0a9b86..593dcc73536f 100644 --- a/pkg/kv/kvserver/replica_application_result.go +++ b/pkg/kv/kvserver/replica_application_result.go @@ -107,10 +107,34 @@ func (r *Replica) prepareLocalResult(ctx context.Context, cmd *replicatedCmd) { // new one. This is important for pipelined writes, since they don't // have a client watching to retry, so a failure to eventually apply // the proposal would be a user-visible error. - pErr = r.tryReproposeWithNewLeaseIndex(ctx, cmd) + pErr = tryReproposeWithNewLeaseIndex(ctx, cmd, (*replicaReproposer)(r)) + if pErr != nil { + // An error from tryReproposeWithNewLeaseIndex implies that the current + // entry is not superseded (i.e. we don't have a reproposal at a higher + // MaxLeaseIndex in the log). + // + // This implies that any additional copies of the command (which may be present + // in the log ahead of the current entry) will also fail. + // + // It is thus safe to signal the error back to the client, which is also + // the only sensible choice at this point. + // + // We also know that the proposal is not in the proposals map, since the + // command is local and wasn't superseded, which is the condition in + // retrieveLocalProposals for removing from the map. So we're not leaking + // a map entry here, which we assert against below (and which has coverage, + // at time of writing, through TestReplicaReproposalWithNewLeaseIndexError). log.Warningf(ctx, "failed to repropose with new lease index: %s", pErr) cmd.response.Err = pErr + + r.mu.RLock() + _, inMap := r.mu.proposals[cmd.ID] + r.mu.RUnlock() + + if inMap { + log.Fatalf(ctx, "failed reproposal unexpectedly in proposals map: %+v", cmd) + } } else { // Unbind the entry's local proposal because we just succeeded // in reproposing it and we don't want to acknowledge the client @@ -135,36 +159,77 @@ func (r *Replica) prepareLocalResult(ctx context.Context, cmd *replicatedCmd) { } } +// reproposer is used by tryReproposeWithNewLeaseIndex. +type reproposer interface { + trackEvaluatingRequest(context.Context, hlc.Timestamp) (hlc.Timestamp, TrackedRequestToken) + propose(context.Context, *ProposalData, TrackedRequestToken) *roachpb.Error + newNotLeaseHolderError(string) *roachpb.NotLeaseHolderError +} + +type replicaReproposer Replica + +var _ reproposer = (*replicaReproposer)(nil) + +func (r *replicaReproposer) trackEvaluatingRequest( + ctx context.Context, wts hlc.Timestamp, +) (hlc.Timestamp, TrackedRequestToken) { + // NB: must not hold r.mu here, the propBuf acquires it itself. + return r.mu.proposalBuf.TrackEvaluatingRequest(ctx, wts) +} + +func (r *replicaReproposer) propose( + ctx context.Context, p *ProposalData, tok TrackedRequestToken, +) *roachpb.Error { + return (*Replica)(r).propose(ctx, p, tok) +} + +func (r *replicaReproposer) newNotLeaseHolderError(msg string) *roachpb.NotLeaseHolderError { + r.mu.RLock() + defer r.mu.RUnlock() + return roachpb.NewNotLeaseHolderError( + *r.mu.state.Lease, + r.store.StoreID(), + r.mu.state.Desc, + msg, + ) +} + // tryReproposeWithNewLeaseIndex is used by prepareLocalResult to repropose // commands that have gotten an illegal lease index error, and that we know // could not have applied while their lease index was valid (that is, we // observed all applied entries between proposal and the lease index becoming // invalid, as opposed to skipping some of them by applying a snapshot). // -// It is not intended for use elsewhere and is only a top-level function so that -// it can avoid the below_raft_protos check. Returns a nil error if the command -// has already been successfully applied or has been reproposed here or by a -// different entry for the same proposal that hit an illegal lease index error. -func (r *Replica) tryReproposeWithNewLeaseIndex( - ctx context.Context, cmd *replicatedCmd, +// Returns a nil error if the command has already been successfully applied or +// has been reproposed here or by a different entry for the same proposal that +// hit an illegal lease index error. +// +// If this returns a nil error once, it will return a nil error for future calls +// as well, assuming that trackEvaluatingRequest returns monotonically increasing +// timestamps across subsequent calls. +func tryReproposeWithNewLeaseIndex( + ctx context.Context, cmd *replicatedCmd, r reproposer, ) *roachpb.Error { // Note that we don't need to validate anything about the proposal's // lease here - if we got this far, we know that everything but the // index is valid at this point in the log. p := cmd.proposal - if p.applied || cmd.Cmd.MaxLeaseIndex != p.command.MaxLeaseIndex { - // If the command associated with this rejected raft entry already - // applied then we don't want to repropose it. Doing so could lead - // to duplicate application of the same proposal. + if p.applied || p.Supersedes(cmd.Cmd.MaxLeaseIndex) { + // If the command associated with this rejected raft entry already applied + // then we don't want to repropose it. Doing so could lead to duplicate + // application of the same proposal. (We can see hit this case if an application + // batch contains multiple copies of the same proposal, in which case they are + // all marked as local, the first one will apply (and set p.applied) and the + // remaining copies will hit this branch). // - // Similarly, if the command associated with this rejected raft - // entry has a different (larger) MaxLeaseIndex than the one we - // decoded from the entry itself, the command must have already - // been reproposed (this can happen if there are multiple copies - // of the command in the logs; see TestReplicaRefreshMultiple). - // We must not create multiple copies with multiple lease indexes, - // so don't repropose it again. This ensures that at any time, - // there is only up to a single lease index that has a chance of + // Similarly, if the proposal associated with this rejected raft entry is + // superseded by a different (larger) MaxLeaseIndex than the one we decoded + // from the entry itself, the command must have already passed through + // tryReproposeWithNewLeaseIndex previously (this can happen if there are + // multiple copies of the command in the logs; see + // TestReplicaRefreshMultiple). We must not create multiple copies with + // multiple lease indexes, so don't repropose it again. This ensures that at + // any time, there is only up to a single lease index that has a chance of // succeeding in the Raft log for a given command. return nil } @@ -173,27 +238,31 @@ func (r *Replica) tryReproposeWithNewLeaseIndex( // it gets reproposed. // TODO(andrei): Only track if the request consults the ts cache. Some // requests (e.g. EndTxn) don't care about closed timestamps. - minTS, tok := r.mu.proposalBuf.TrackEvaluatingRequest(ctx, p.Request.WriteTimestamp()) + minTS, tok := r.trackEvaluatingRequest(ctx, p.Request.WriteTimestamp()) defer tok.DoneIfNotMoved(ctx) // NB: p.Request.Timestamp reflects the action of ba.SetActiveTimestamp. if p.Request.AppliesTimestampCache() && p.Request.WriteTimestamp().LessEq(minTS) { // The tracker wants us to forward the request timestamp, but we can't // do that without re-evaluating, so give up. The error returned here - // will go to back to DistSender, so send something it can digest. - err := roachpb.NewNotLeaseHolderError( - *r.mu.state.Lease, - r.store.StoreID(), - r.mu.state.Desc, - "reproposal failed due to closed timestamp", - ) - return roachpb.NewError(err) + // will go back to DistSender, so send something it can digest. + return roachpb.NewError(r.newNotLeaseHolderError("reproposal failed due to closed timestamp")) } // Some tests check for this log message in the trace. log.VEventf(ctx, 2, "retry: proposalIllegalLeaseIndex") + // Reset the command for reproposal. + prevMaxLeaseIndex := p.command.MaxLeaseIndex + prevEncodedCommand := p.encodedCommand + p.command.MaxLeaseIndex = 0 + p.encodedCommand = nil pErr := r.propose(ctx, p, tok.Move(ctx)) if pErr != nil { + // On error, reset the fields we zeroed out to their old value. + // This ensures that the proposal doesn't count as Superseded + // now. + p.command.MaxLeaseIndex = prevMaxLeaseIndex + p.encodedCommand = prevEncodedCommand return pErr } log.VEventf(ctx, 2, "reproposed command %x", cmd.ID) diff --git a/pkg/kv/kvserver/replica_application_state_machine.go b/pkg/kv/kvserver/replica_application_state_machine.go index 68f8ca59068f..80c02d68e843 100644 --- a/pkg/kv/kvserver/replica_application_state_machine.go +++ b/pkg/kv/kvserver/replica_application_state_machine.go @@ -220,12 +220,24 @@ func (sm *replicaStateMachine) ApplySideEffects( sm.r.handleReadWriteLocalEvalResult(ctx, *cmd.localResult) } - rejected := cmd.Rejected() - higherReproposalsExist := cmd.Cmd.MaxLeaseIndex != cmd.proposal.command.MaxLeaseIndex - if !rejected && higherReproposalsExist { - log.Fatalf(ctx, "finishing proposal with outstanding reproposal at a higher max lease index") + if higherReproposalsExist := cmd.proposal.Supersedes(cmd.Cmd.MaxLeaseIndex); higherReproposalsExist { + // If the command wasn't rejected, we just applied it and no higher + // reproposal must exist (since that one may also apply). + // + // If the command was rejected with ProposalRejectionPermanent, no higher + // reproposal should exist (after all, whoever made that reproposal should + // also have seen a permanent rejection). + // + // If it was rejected with ProposalRejectionIllegalLeaseIndex, then the + // subsequent call to tryReproposeWithNewLeaseIndex[^1] must have returned an + // error (or the proposal would not be IsLocal() now). But that call + // cannot return an error for a proposal that is already superseded + // initially. + // + // [^1]: see (*replicaDecoder).retrieveLocalProposals() + log.Fatalf(ctx, "finishing proposal with outstanding reproposal at a higher max lease index: %+v", cmd) } - if !rejected && cmd.proposal.applied { + if !cmd.Rejected() && cmd.proposal.applied { // If the command already applied then we shouldn't be "finishing" its // application again because it should only be able to apply successfully // once. We expect that when any reproposal for the same command attempts @@ -233,18 +245,6 @@ func (sm *replicaStateMachine) ApplySideEffects( // index check in checkForcedErr. log.Fatalf(ctx, "command already applied: %+v; unexpected successful result", cmd) } - // If any reproposals at a higher MaxLeaseIndex exist we know that they will - // never successfully apply, remove them from the map to avoid future - // reproposals. If there is no command referencing this proposal at a higher - // MaxLeaseIndex then it will already have been removed (see - // shouldRemove in replicaDecoder.retrieveLocalProposals()). It is possible - // that a later command in this batch referred to this proposal but it must - // have failed because it carried the same MaxLeaseIndex. - if higherReproposalsExist { - sm.r.mu.Lock() - delete(sm.r.mu.proposals, cmd.ID) - sm.r.mu.Unlock() - } cmd.proposal.applied = true } return cmd, nil diff --git a/pkg/kv/kvserver/replica_proposal.go b/pkg/kv/kvserver/replica_proposal.go index dfe25104af8d..150cab23a666 100644 --- a/pkg/kv/kvserver/replica_proposal.go +++ b/pkg/kv/kvserver/replica_proposal.go @@ -126,6 +126,20 @@ type ProposalData struct { tok TrackedRequestToken } +// Supersedes takes the MaxLeaseIndex of a RaftCommand obtained from a log +// entry. It returns true if the ProposalData tracks a different MaxIndex, +// implying that the log entry has been reproposed under an updated +// MaxLeaseIndex. This implies that the current log entry will have been +// rejected and should not be reproposed. +// +// Note that some commands such as lease requests (but not transfers) don't use +// MaxLeaseIndex. For these, false will be returned. +// +// See (*Replica).mu.proposals for a detailed explanation of reproposals. +func (proposal *ProposalData) Supersedes(entryMaxLeaseIndex uint64) bool { + return proposal.command.MaxLeaseIndex != entryMaxLeaseIndex +} + // finishApplication is called when a command application has finished. The // method will be called downstream of Raft if the command required consensus, // but can be called upstream of Raft if the command did not and was never diff --git a/pkg/kv/kvserver/replica_raft.go b/pkg/kv/kvserver/replica_raft.go index cce57af3de26..06185131b49c 100644 --- a/pkg/kv/kvserver/replica_raft.go +++ b/pkg/kv/kvserver/replica_raft.go @@ -316,32 +316,45 @@ func (r *Replica) evalAndPropose( // propose encodes a command, starts tracking it, and proposes it to Raft. // -// The method hands ownership of the command over to the Raft machinery. After -// the method returns, all access to the command must be performed while holding -// Replica.mu and Replica.raftMu. +// On success, the method hands ownership of the command over to the Raft +// machinery. After the method returns with a nil error, all access to the +// command must be performed while holding Replica.mu and Replica.raftMu. +// If the method returns with an error, the error is permanent for the +// proposal, that is, the caller must notify the client that the proposal +// failed and the client can retry, making a new proposal in the process. // -// propose takes ownership of the supplied token; the caller should tok.Move() -// it into this method. It will be used to untrack the request once it comes out -// of the proposal buffer. +// propose takes ownership of the supplied token, even on error; the caller +// should tok.Move() it into this method. It will be used to untrack the request +// once it comes out of the proposal buffer. If the method returns with an error, +// the token is released, since, as explained above, an error is permanent. +// +// The ProposalData must not be reproposed or reused should an error be returned +// from this method. Its MaxLeaseIndex and encodedCommand fields must be empty. +// Reproposals are a rich source of complexity. See the comment on `r.mu.proposals` +// for details. func (r *Replica) propose( ctx context.Context, p *ProposalData, tok TrackedRequestToken, ) (pErr *roachpb.Error) { defer tok.DoneIfNotMoved(ctx) - // If an error occurs reset the command's MaxLeaseIndex to its initial value. - // Failure to propose will propagate to the client. An invariant of this - // package is that proposals which are finished carry a raft command with a - // MaxLeaseIndex equal to the proposal command's max lease index. - defer func(prev uint64) { + defer func() { + // An error for this method if pErr != nil { - p.command.MaxLeaseIndex = prev + p.encodedCommand = nil } - }(p.command.MaxLeaseIndex) + }() - // Make sure the maximum lease index is unset. This field will be set in - // propBuf.Insert and its encoded bytes will be appended to the encoding - // buffer as a MaxLeaseFooter. - p.command.MaxLeaseIndex = 0 + if p.command.MaxLeaseIndex > 0 { + // MaxLeaseIndex should not be populated now. It is set only when the proposal buffer + // flushes this proposal into the local raft instance. + return roachpb.NewError(errors.AssertionFailedf("MaxLeaseIndex set: %+v", p)) + } + if p.encodedCommand != nil { + // This indicates someone took an existing proposal and handed it to this method + // again. The caller needs to properly reset the proposal if they're going to do + // that. + return roachpb.NewError(errors.AssertionFailedf("encodedCommand set: %+v", p)) + } // Determine the encoding style for the Raft command. prefix := true