diff --git a/docs/generated/settings/settings-for-tenants.txt b/docs/generated/settings/settings-for-tenants.txt index e1e74805b049..451fb743b83f 100644 --- a/docs/generated/settings/settings-for-tenants.txt +++ b/docs/generated/settings/settings-for-tenants.txt @@ -49,7 +49,6 @@ server.auth_log.sql_sessions.enabled boolean false if set, log SQL session login server.authentication_cache.enabled boolean true enables a cache used during authentication to avoid lookups to system tables when retrieving per-user authentication-related information server.clock.forward_jump_check_enabled boolean false if enabled, forward clock jumps > max_offset/2 will cause a panic server.clock.persist_upper_bound_interval duration 0s the interval between persisting the wall time upper bound of the clock. The clock does not generate a wall time greater than the persisted timestamp and will panic if it sees a wall time greater than this value. When cockroach starts, it waits for the wall time to catch-up till this persisted timestamp. This guarantees monotonic wall time across server restarts. Not setting this or setting a value of 0 disables this feature. -server.consistency_check.max_rate byte size 8.0 MiB the rate limit (bytes/sec) to use for consistency checks; used in conjunction with server.consistency_check.interval to control the frequency of consistency checks. Note that setting this too high can negatively impact performance. server.eventlog.enabled boolean true if set, logged notable events are also stored in the table system.eventlog server.eventlog.ttl duration 2160h0m0s if nonzero, entries in system.eventlog older than this duration are deleted every 10m0s. Should not be lowered below 24 hours. server.host_based_authentication.configuration string host-based authentication configuration to use during connection authentication diff --git a/pkg/kv/kvserver/consistency_queue.go b/pkg/kv/kvserver/consistency_queue.go index 5f09ba9b3f3f..ef463b61af24 100644 --- a/pkg/kv/kvserver/consistency_queue.go +++ b/pkg/kv/kvserver/consistency_queue.go @@ -24,7 +24,7 @@ import ( ) var consistencyCheckInterval = settings.RegisterDurationSetting( - settings.TenantWritable, + settings.SystemOnly, "server.consistency_check.interval", "the time between range consistency checks; set to 0 to disable consistency checking."+ " Note that intervals that are too short can negatively impact performance.", @@ -33,14 +33,14 @@ var consistencyCheckInterval = settings.RegisterDurationSetting( ) var consistencyCheckRate = settings.RegisterByteSizeSetting( - settings.TenantWritable, + settings.SystemOnly, "server.consistency_check.max_rate", "the rate limit (bytes/sec) to use for consistency checks; used in "+ "conjunction with server.consistency_check.interval to control the "+ "frequency of consistency checks. Note that setting this too high can "+ "negatively impact performance.", 8<<20, // 8MB - validatePositive, + settings.PositiveInt, ).WithPublic() // consistencyCheckRateBurstFactor we use this to set the burst parameter on the diff --git a/pkg/kv/kvserver/store_snapshot.go b/pkg/kv/kvserver/store_snapshot.go index 53df7e6fe1ee..d0d483893a67 100644 --- a/pkg/kv/kvserver/store_snapshot.go +++ b/pkg/kv/kvserver/store_snapshot.go @@ -407,10 +407,27 @@ func (s *Store) reserveSnapshot( // RESTORE or manual SPLIT AT, since it prevents these empty snapshots from // getting stuck behind large snapshots managed by the replicate queue. if header.RangeSize != 0 { + queueCtx := ctx + if deadline, ok := queueCtx.Deadline(); ok { + // Enforce a more strict timeout for acquiring the snapshot reservation to + // ensure that if the reservation is acquired, the snapshot has sufficient + // time to complete. See the comment on snapshotReservationQueueTimeoutFraction + // and TestReserveSnapshotQueueTimeout. + timeoutFrac := snapshotReservationQueueTimeoutFraction.Get(&s.ClusterSettings().SV) + timeout := time.Duration(timeoutFrac * float64(timeutil.Until(deadline))) + var cancel func() + queueCtx, cancel = context.WithTimeout(queueCtx, timeout) // nolint:context + defer cancel() + } select { case s.snapshotApplySem <- struct{}{}: - case <-ctx.Done(): - return nil, ctx.Err() + case <-queueCtx.Done(): + if err := ctx.Err(); err != nil { + return nil, errors.Wrap(err, "acquiring snapshot reservation") + } + return nil, errors.Wrapf(queueCtx.Err(), + "giving up during snapshot reservation due to %q", + snapshotReservationQueueTimeoutFraction.Key()) case <-s.stopper.ShouldQuiesce(): return nil, errors.Errorf("stopped") } @@ -685,14 +702,6 @@ type SnapshotStorePool interface { throttle(reason throttleReason, why string, toStoreID roachpb.StoreID) } -// validatePositive is a function to validate that a settings value is positive. -func validatePositive(v int64) error { - if v <= 0 { - return errors.Errorf("%d is not positive", v) - } - return nil -} - // rebalanceSnapshotRate is the rate at which snapshots can be sent in the // context of up-replication or rebalancing (i.e. any snapshot that was not // requested by raft itself, to which `kv.snapshot_recovery.max_rate` applies). @@ -701,10 +710,10 @@ var rebalanceSnapshotRate = settings.RegisterByteSizeSetting( "kv.snapshot_rebalance.max_rate", "the rate limit (bytes/sec) to use for rebalance and upreplication snapshots", 32<<20, // 32mb/s - validatePositive, + settings.PositiveInt, ).WithPublic() -// recoverySnapshotRate is the rate at which Raft-initiated spanshots can be +// recoverySnapshotRate is the rate at which Raft-initiated snapshot can be // sent. Ideally, one would never see a Raft-initiated snapshot; we'd like all // replicas to start out as learners or via splits, and to never be cut off from // the log. However, it has proved unfeasible to completely get rid of them. @@ -718,7 +727,7 @@ var recoverySnapshotRate = settings.RegisterByteSizeSetting( "kv.snapshot_recovery.max_rate", "the rate limit (bytes/sec) to use for recovery snapshots", 32<<20, // 32mb/s - validatePositive, + settings.PositiveInt, ).WithPublic() // snapshotSenderBatchSize is the size that key-value batches are allowed to @@ -726,22 +735,126 @@ var recoverySnapshotRate = settings.RegisterByteSizeSetting( // places an upper-bound on the memory footprint of the sender of a Range // snapshot. It is also the granularity of rate limiting. var snapshotSenderBatchSize = settings.RegisterByteSizeSetting( - settings.TenantWritable, + settings.SystemOnly, "kv.snapshot_sender.batch_size", "size of key-value batches sent over the network during snapshots", 256<<10, // 256 KB - validatePositive, + settings.PositiveInt, +) + +// snapshotReservationQueueTimeoutFraction is the maximum fraction of a Range +// snapshot's total timeout that it is allowed to spend queued on the receiver +// waiting for a reservation. +// +// Enforcement of this snapshotApplySem-scoped timeout is intended to prevent +// starvation of snapshots in cases where a queue of snapshots waiting for +// reservations builds and no single snapshot acquires the semaphore with +// sufficient time to complete, but each holds the semaphore long enough to +// ensure that later snapshots in the queue encounter this same situation. This +// is a case of FIFO queuing + timeouts leading to starvation. By rejecting +// snapshot attempts earlier, we ensure that those that do acquire the semaphore +// have sufficient time to complete. +// +// Consider the following motivating example: +// +// With a 60s timeout set by the snapshotQueue/replicateQueue for each snapshot, +// 45s needed to actually stream the data, and a willingness to wait for as long +// as it takes to get the reservation (i.e. this fraction = 1.0) there can be +// starvation. Each snapshot spends so much time waiting for the reservation +// that it will itself fail during sending, while the next snapshot wastes +// enough time waiting for us that it will itself fail, ad infinitum: +// +// t | snap1 snap2 snap3 snap4 snap5 ... +// ----+------------------------------------ +// 0 | send +// 15 | queue queue +// 30 | queue +// 45 | ok send +// 60 | queue +// 75 | fail fail send +// 90 | fail send +// 105 | +// 120 | fail +// 135 | +// +// If we limit the amount of time we are willing to wait for a reservation to +// something that is small enough to, on success, give us enough time to +// actually stream the data, no starvation can occur. For example, with a 60s +// timeout, 45s needed to stream the data, we can wait at most 15s for a +// reservation and still avoid starvation: +// +// t | snap1 snap2 snap3 snap4 snap5 ... +// ----+------------------------------------ +// 0 | send +// 15 | queue queue +// 30 | fail fail send +// 45 | +// 60 | ok queue +// 75 | ok send +// 90 | +// 105 | +// 120 | ok +// 135 | +// +// In practice, the snapshot reservation logic (reserveSnapshot) doesn't know +// how long sending the snapshot will actually take. But it knows the timeout it +// has been given by the snapshotQueue/replicateQueue, which serves as an upper +// bound, under the assumption that snapshots can make progress in the absence +// of starvation. +// +// Without the reservation timeout fraction, if the product of the number of +// concurrent snapshots and the average streaming time exceeded this timeout, +// the starvation scenario could occur, since the average queuing time would +// exceed the timeout. With the reservation limit, progress will be made as long +// as the average streaming time is less than the guaranteed processing time for +// any snapshot that succeeds in acquiring a reservation: +// +// guaranteed_processing_time = (1 - reservation_queue_timeout_fraction) x timeout +// +// The timeout for the snapshot and replicate queues bottoms out at 60s (by +// default, see kv.queue.process.guaranteed_time_budget). Given a default +// reservation queue timeout fraction of 0.4, this translates to a guaranteed +// processing time of 36s for any snapshot attempt that manages to acquire a +// reservation. This means that a 512MiB snapshot will succeed if sent at a rate +// of 14MiB/s or above. +// +// Lower configured snapshot rate limits quickly lead to a much higher timeout +// since we apply a liberal multiplier (permittedRangeScanSlowdown). Concretely, +// we move past the 1-minute timeout once the rate limit is set to anything less +// than 10*range_size/guaranteed_budget(in MiB/s), which comes out to ~85MiB/s +// for a 512MiB range and the default 1m budget. In other words, the queue uses +// sumptuous timeouts, and so we'll also be excessively lenient with how long +// we're willing to wait for a reservation (but not to the point of allowing the +// starvation scenario). As long as the nodes between the cluster can transfer +// at around ~14MiB/s, even a misconfiguration of the rate limit won't cause +// issues and where it does, the setting can be set to 1.0, effectively +// reverting to the old behavior. +var snapshotReservationQueueTimeoutFraction = settings.RegisterFloatSetting( + settings.SystemOnly, + "kv.snapshot_receiver.reservation_queue_timeout_fraction", + "the fraction of a snapshot's total timeout that it is allowed to spend "+ + "queued on the receiver waiting for a reservation", + 0.4, + func(v float64) error { + const min, max = 0.25, 1.0 + if v < min { + return errors.Errorf("cannot set to a value less than %f: %f", min, v) + } else if v > max { + return errors.Errorf("cannot set to a value greater than %f: %f", max, v) + } + return nil + }, ) // snapshotSSTWriteSyncRate is the size of chunks to write before fsync-ing. // The default of 2 MiB was chosen to be in line with the behavior in bulk-io. // See sstWriteSyncRate. var snapshotSSTWriteSyncRate = settings.RegisterByteSizeSetting( - settings.TenantWritable, + settings.SystemOnly, "kv.snapshot_sst.sync_size", "threshold after which snapshot SST writes must fsync", bulkIOWriteBurst, - validatePositive, + settings.PositiveInt, ) func snapshotRateLimit( diff --git a/pkg/kv/kvserver/store_test.go b/pkg/kv/kvserver/store_test.go index 59a30ee3124a..9a2b7eecd6dd 100644 --- a/pkg/kv/kvserver/store_test.go +++ b/pkg/kv/kvserver/store_test.go @@ -46,6 +46,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" "github.com/cockroachdb/cockroach/pkg/testutils" + "github.com/cockroachdb/cockroach/pkg/testutils/skip" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -61,6 +62,7 @@ import ( "github.com/stretchr/testify/require" "go.etcd.io/etcd/raft/v3" "go.etcd.io/etcd/raft/v3/raftpb" + "golang.org/x/sync/errgroup" "golang.org/x/time/rate" ) @@ -2957,6 +2959,94 @@ func TestReserveSnapshotFullnessLimit(t *testing.T) { } } +// TestSnapshotReservationQueueTimeoutAvoidsStarvation verifies that the +// snapshot reservation queue applies a tighter queueing timeout than overall +// operation timeout to incoming snapshot requests, which enables it to avoid +// starvation even with its FIFO queueing policy under high concurrency. +func TestReserveSnapshotQueueTimeoutAvoidsStarvation(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + skip.UnderStress(t) + skip.UnderRace(t) + skip.UnderShort(t) + + // Run each snapshot with a 100-millisecond timeout, a 50-millisecond queue + // timeout, and a 15-millisecond process time. + const timeout = 100 * time.Millisecond + const maxQueueTimeout = 50 * time.Millisecond + const timeoutFrac = float64(maxQueueTimeout) / float64(timeout) + const processTime = 15 * time.Millisecond + // Run 8 workers that are each trying to perform snapshots for 3 seconds. + const workers = 8 + const duration = 3 * time.Second + // We expect that roughly duration / processTime snapshots "succeed". To avoid + // flakiness, we assert half of this. + const expSuccesses = int(duration / processTime) + const assertSuccesses = expSuccesses / 2 + // Sanity check config. If workers*processTime < timeout, then the queue time + // will never be large enough to create starvation. If timeout-maxQueueTimeout + // < processTime then most snapshots won't have enough time to complete. + require.Greater(t, workers*processTime, timeout) + require.Greater(t, timeout-maxQueueTimeout, processTime) + + ctx := context.Background() + stopper := stop.NewStopper() + defer stopper.Stop(ctx) + tsc := TestStoreConfig(nil) + // Set the concurrency to 1 explicitly, in case the default ever changes. + tsc.concurrentSnapshotApplyLimit = 1 + tc := testContext{} + tc.StartWithStoreConfig(t, stopper, tsc) + s := tc.store + snapshotReservationQueueTimeoutFraction.Override(ctx, &s.ClusterSettings().SV, timeoutFrac) + + var done int64 + var successes int64 + var g errgroup.Group + for i := 0; i < workers; i++ { + g.Go(func() error { + for atomic.LoadInt64(&done) == 0 { + if err := func() error { + snapCtx, cancel := context.WithTimeout(ctx, timeout) + defer cancel() + cleanup, err := s.reserveSnapshot(snapCtx, &SnapshotRequest_Header{RangeSize: 1}) + if err != nil { + if errors.Is(err, context.DeadlineExceeded) { + return nil + } + return err + } + defer cleanup() + if atomic.LoadInt64(&done) != 0 { + // If the test has ended, don't process. + return nil + } + // Process... + time.Sleep(processTime) + // Check for sufficient processing time. If we hit a timeout, don't + // count the process attempt as a success. We could make this more + // reactive and terminate the sleep as soon as the ctx is canceled, + // but let's assume the worst case. + if err := snapCtx.Err(); err != nil { + t.Logf("hit %v while processing", err) + } else { + atomic.AddInt64(&successes, 1) + } + return nil + }(); err != nil { + return err + } + } + return nil + }) + } + + time.Sleep(duration) + atomic.StoreInt64(&done, 1) + require.NoError(t, g.Wait()) + require.GreaterOrEqual(t, int(atomic.LoadInt64(&successes)), assertSuccesses) +} + func TestSnapshotRateLimit(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) diff --git a/pkg/rpc/context_test.go b/pkg/rpc/context_test.go index abce77ab3e77..ef6d052987b3 100644 --- a/pkg/rpc/context_test.go +++ b/pkg/rpc/context_test.go @@ -1331,6 +1331,89 @@ func grpcRunKeepaliveTestCase(testCtx context.Context, c grpcKeepaliveTestCase) return nil } +// TestGRPCDeadlinePropagation is a smoketest for gRPC deadline propagation. +// When RPC clients issue requests with deadlines/timeouts attached to their +// context, they are guaranteed that not only will remote RPCs respect this +// deadline/timeout if it is reached, but that the remote RPC will be aware of +// the timeout throughout its lifetime. In other words, deadlines/timeouts are +// communicated upfront by the client, not only after they have been reached. +// +// gRPC implements this through its "grpc-timeout" header field, which is +// attached to the header (first) frame of unary and streaming calls. +// +// For more, see https://grpc.io/docs/what-is-grpc/core-concepts/#deadlines, +// which says: +// > gRPC allows clients to specify how long they are willing to wait for an RPC +// > to complete before the RPC is terminated with a DEADLINE_EXCEEDED error. On +// > the server side, the server can query to see if a particular RPC has timed +// > out, or how much time is left to complete the RPC. +func TestGRPCDeadlinePropagation(t *testing.T) { + defer leaktest.AfterTest(t)() + + ctx := context.Background() + stopper := stop.NewStopper() + defer stopper.Stop(ctx) + clusterID := uuid.MakeV4() + clock := hlc.NewClock(timeutil.Unix(0, 20).UnixNano, time.Nanosecond) + + // Construct the server context. + const serverNodeID = 1 + serverCtx := newTestContext(clusterID, clock, stopper) + serverCtx.NodeID.Set(ctx, serverNodeID) + + // Register an UnknownServiceHandler that expects a BatchRequest and sends + // a BatchResponse. Record the context deadline of the request in the handler. + var serverDeadline time.Time + s := newTestServer(t, serverCtx, grpc.UnknownServiceHandler( + func(srv interface{}, stream grpc.ServerStream) error { + serverDeadline, _ = stream.Context().Deadline() + var ba roachpb.BatchRequest + if err := stream.RecvMsg(&ba); err != nil { + return err + } + return stream.SendMsg(&roachpb.BatchResponse{}) + }, + )) + RegisterHeartbeatServer(s, serverCtx.NewHeartbeatService()) + + // Begin listening. + ln, err := netutil.ListenAndServeGRPC(serverCtx.Stopper, s, util.TestAddr) + require.Nil(t, err) + remoteAddr := ln.Addr().String() + + // Construct the client context. + clientCtx := newTestContext(clusterID, clock, stopper) + defConn, err := clientCtx.GRPCDialNode(remoteAddr, serverNodeID, DefaultClass).Connect(ctx) + require.Nil(t, err) + + // Issue an RPC with a deadline far in the future. + clientDeadline := timeutil.Now().Add(4 * time.Hour) + ctxWithDeadline, cancel := context.WithDeadline(ctx, clientDeadline) + defer cancel() + + desc := grpc.StreamDesc{ + StreamName: "foo", + ClientStreams: true, + } + const method = "/cockroach.rpc.Testing/Foo" + cs, err := defConn.NewStream(ctxWithDeadline, &desc, method) + require.Nil(t, err) + require.Nil(t, cs.SendMsg(&roachpb.BatchRequest{})) + var br roachpb.BatchResponse + require.Nil(t, cs.RecvMsg(&br)) + require.Nil(t, cs.CloseSend()) + + // The server should have heard about the deadline, and it should be nearly + // identical to the client-side deadline. The values aren't exactly the same + // because the deadline (a fixed point in time) passes through a timeout (a + // duration of time) over the wire. However, we can assert that the client + // deadline is always earlier than the server deadline, but by no more than a + // small margin (relative to the 4-hour timeout). + require.NotZero(t, serverDeadline) + require.True(t, clientDeadline.Before(serverDeadline)) + require.True(t, serverDeadline.Before(clientDeadline.Add(1*time.Minute))) +} + func TestClusterIDMismatch(t *testing.T) { defer leaktest.AfterTest(t)() @@ -1597,8 +1680,8 @@ func TestGRPCDialClass(t *testing.T) { remoteAddr := ln.Addr().String() clientCtx := newTestContext(serverCtx.ClusterID.Get(), clock, stopper) - def1 := clientCtx.GRPCDialNode(remoteAddr, 1, DefaultClass) - sys1 := clientCtx.GRPCDialNode(remoteAddr, 1, SystemClass) + def1 := clientCtx.GRPCDialNode(remoteAddr, serverNodeID, DefaultClass) + sys1 := clientCtx.GRPCDialNode(remoteAddr, serverNodeID, SystemClass) require.False(t, sys1 == def1, "expected connections dialed with different classes to the same target to differ") defConn1, err := def1.Connect(context.Background()) @@ -1607,10 +1690,10 @@ func TestGRPCDialClass(t *testing.T) { require.Nil(t, err, "expected successful connection") require.False(t, sysConn1 == defConn1, "expected connections dialed with "+ "different classes to the sametarget to have separate underlying gRPC connections") - def2 := clientCtx.GRPCDialNode(remoteAddr, 1, DefaultClass) + def2 := clientCtx.GRPCDialNode(remoteAddr, serverNodeID, DefaultClass) require.True(t, def1 == def2, "expected connections dialed with the same "+ "class to the same target to be the same") - sys2 := clientCtx.GRPCDialNode(remoteAddr, 1, SystemClass) + sys2 := clientCtx.GRPCDialNode(remoteAddr, serverNodeID, SystemClass) require.True(t, sys1 == sys2, "expected connections dialed with the same "+ "class to the same target to be the same") for _, c := range []*Connection{def2, sys2} { @@ -1714,9 +1797,9 @@ func TestTestingKnobs(t *testing.T) { ln, err := netutil.ListenAndServeGRPC(serverCtx.Stopper, s, util.TestAddr) require.Nil(t, err) remoteAddr := ln.Addr().String() - sysConn, err := clientCtx.GRPCDialNode(remoteAddr, 1, SystemClass).Connect(context.Background()) + sysConn, err := clientCtx.GRPCDialNode(remoteAddr, serverNodeID, SystemClass).Connect(context.Background()) require.Nil(t, err) - defConn, err := clientCtx.GRPCDialNode(remoteAddr, 1, DefaultClass).Connect(context.Background()) + defConn, err := clientCtx.GRPCDialNode(remoteAddr, serverNodeID, DefaultClass).Connect(context.Background()) require.Nil(t, err) const unaryMethod = "/cockroach.rpc.Testing/Foo" const streamMethod = "/cockroach.rpc.Testing/Bar" diff --git a/pkg/settings/common.go b/pkg/settings/common.go index 64c853d2d57e..40c835cca61c 100644 --- a/pkg/settings/common.go +++ b/pkg/settings/common.go @@ -17,6 +17,7 @@ import ( // common implements basic functionality used by all setting types. type common struct { + key string description string class Class visibility Visibility @@ -27,7 +28,7 @@ type common struct { } // init must be called to initialize the fields that don't have defaults. -func (i *common) init(class Class, slotIdx int, description string) { +func (i *common) init(class Class, slotIdx int, key string, description string) { i.class = class if slotIdx < 1 { panic(fmt.Sprintf("Invalid slot index %d", slotIdx)) @@ -36,6 +37,7 @@ func (i *common) init(class Class, slotIdx int, description string) { panic("too many settings; increase MaxSettings") } i.slotIdx = slotIdx + i.key = key i.description = description } @@ -47,6 +49,10 @@ func (i *common) getSlotIdx() int { return i.slotIdx } +func (i common) Key() string { + return i.key +} + func (i common) Description() string { return i.description } @@ -103,7 +109,7 @@ func (i *common) SetOnChange(sv *Values, fn func(ctx context.Context)) { type internalSetting interface { NonMaskedSetting - init(class Class, slotIdx int, desc string) + init(class Class, slotIdx int, key string, desc string) isRetired() bool setToDefault(ctx context.Context, sv *Values) getSlotIdx() int diff --git a/pkg/settings/masked.go b/pkg/settings/masked.go index 9c68f7de5faf..5191926cfba0 100644 --- a/pkg/settings/masked.go +++ b/pkg/settings/masked.go @@ -38,6 +38,11 @@ func (s *MaskedSetting) Visibility() Visibility { return s.setting.Visibility() } +// Key returns the key string for the underlying setting. +func (s *MaskedSetting) Key() string { + return s.setting.Key() +} + // Description returns the description string for the underlying setting. func (s *MaskedSetting) Description() string { return s.setting.Description() diff --git a/pkg/settings/registry.go b/pkg/settings/registry.go index 74b695c9b5a9..9bef2a05f42b 100644 --- a/pkg/settings/registry.go +++ b/pkg/settings/registry.go @@ -144,7 +144,7 @@ func register(class Class, key, desc string, s internalSetting) { } registry[key] = s slotIdx := len(registry) - s.init(class, slotIdx, desc) + s.init(class, slotIdx, key, desc) } // NumRegisteredSettings returns the number of registered settings. diff --git a/pkg/settings/setting.go b/pkg/settings/setting.go index 7f8c29751320..a9b74673e9a9 100644 --- a/pkg/settings/setting.go +++ b/pkg/settings/setting.go @@ -30,6 +30,9 @@ type Setting interface { // CLUSTER SETTING `. String(sv *Values) string + // Key returns the name of the specific cluster setting. + Key() string + // Description contains a helpful text explaining what the specific cluster // setting is for. Description() string diff --git a/pkg/settings/settings_test.go b/pkg/settings/settings_test.go index 9bf27b1cd4f1..c7c9100cd6c2 100644 --- a/pkg/settings/settings_test.go +++ b/pkg/settings/settings_test.go @@ -216,6 +216,15 @@ func TestValidation(t *testing.T) { }) } +func TestIntrospection(t *testing.T) { + require.Equal(t, "b", boolTA.Typ()) + require.Equal(t, "bool.t", boolTA.Key()) + require.Equal(t, "desc", boolTA.Description()) + require.Equal(t, settings.Reserved, boolTA.Visibility()) + require.Equal(t, settings.SystemOnly, boolTA.Class()) + require.Equal(t, true, boolTA.Default()) +} + func TestCache(t *testing.T) { ctx := context.Background() sv := &settings.Values{}