diff --git a/pkg/ccl/streamingccl/streamproducer/event_stream.go b/pkg/ccl/streamingccl/streamproducer/event_stream.go index 0c82dac74454..25f34677cc79 100644 --- a/pkg/ccl/streamingccl/streamproducer/event_stream.go +++ b/pkg/ccl/streamingccl/streamproducer/event_stream.go @@ -484,10 +484,11 @@ func (s *eventStream) streamLoop(ctx context.Context, frontier *span.Frontier) e } } +const defaultBatchSize = 1 << 20 + func setConfigDefaults(cfg *streampb.StreamPartitionSpec_ExecutionConfig) { const defaultInitialScanParallelism = 16 const defaultMinCheckpointFrequency = 10 * time.Second - const defaultBatchSize = 1 << 20 if cfg.InitialScanParallelism <= 0 { cfg.InitialScanParallelism = defaultInitialScanParallelism @@ -502,16 +503,6 @@ func setConfigDefaults(cfg *streampb.StreamPartitionSpec_ExecutionConfig) { } } -func validateSpecs(evalCtx *eval.Context, spec streampb.StreamPartitionSpec) error { - if !evalCtx.SessionData().AvoidBuffering { - return errors.New("partition streaming requires 'SET avoid_buffering = true' option") - } - if len(spec.Spans) == 0 { - return errors.AssertionFailedf("expected at least one span, got none") - } - return nil -} - func streamPartition( evalCtx *eval.Context, streamID streampb.StreamID, opaqueSpec []byte, ) (eval.ValueGenerator, error) { @@ -519,10 +510,12 @@ func streamPartition( if err := protoutil.Unmarshal(opaqueSpec, &spec); err != nil { return nil, errors.Wrapf(err, "invalid partition spec for stream %d", streamID) } - if err := validateSpecs(evalCtx, spec); err != nil { - return nil, err + if !evalCtx.SessionData().AvoidBuffering { + return nil, errors.New("partition streaming requires 'SET avoid_buffering = true' option") + } + if len(spec.Spans) == 0 { + return nil, errors.AssertionFailedf("expected at least one span, got none") } - setConfigDefaults(&spec.Config) execCfg := evalCtx.Planner.ExecutorConfig().(*sql.ExecutorConfig) diff --git a/pkg/ccl/streamingccl/streamproducer/span_config_event_stream.go b/pkg/ccl/streamingccl/streamproducer/span_config_event_stream.go index b3c02133eeb7..35b5a736bf36 100644 --- a/pkg/ccl/streamingccl/streamproducer/span_config_event_stream.go +++ b/pkg/ccl/streamingccl/streamproducer/span_config_event_stream.go @@ -33,7 +33,7 @@ import ( type spanConfigEventStream struct { execCfg *sql.ExecutorConfig - spec streampb.StreamPartitionSpec + spec streampb.SpanConfigEventStreamSpec mon *mon.BytesMonitor acc mon.BoundAccount @@ -82,15 +82,15 @@ func (s *spanConfigEventStream) Start(ctx context.Context, txn *kv.Txn) error { s.doneChan = make(chan struct{}) // Reserve batch kvsSize bytes from monitor. - if err := s.acc.Grow(ctx, s.spec.Config.BatchByteSize); err != nil { - return errors.Wrapf(err, "failed to allocated %d bytes from monitor", s.spec.Config.BatchByteSize) + if err := s.acc.Grow(ctx, defaultBatchSize); err != nil { + return errors.Wrapf(err, "failed to allocated %d bytes from monitor", defaultBatchSize) } s.rfc = rangefeedcache.NewWatcher( "spanconfig-subscriber", s.execCfg.Clock, s.execCfg.RangeFeedFactory, - int(s.spec.Config.BatchByteSize), - s.spec.Spans, + defaultBatchSize, + roachpb.Spans{s.spec.Span}, true, // withPrevValue spanconfigkvsubscriber.NewSpanConfigDecoder().TranslateEvent, s.handleUpdate, @@ -212,13 +212,10 @@ func (s *spanConfigEventStream) streamLoop(ctx context.Context) error { // TODO(msbutler): We may not need a pacer, given how little traffic will come from this // stream. That being said, we'd still want to buffer updates to ensure we // don't clog up the rangefeed. Consider using async flushing. - pacer := makeCheckpointPacer(s.spec.Config.MinCheckpointFrequency) + pacer := makeCheckpointPacer(s.spec.MinCheckpointFrequency) bufferedEvents := make([]streampb.StreamedSpanConfigEntry, 0) batcher := makeStreamEventBatcher() - frontier, err := makeSpanConfigFrontier(s.spec.Spans) - if err != nil { - return err - } + frontier := makeSpanConfigFrontier(s.spec.Span) for { select { @@ -247,7 +244,7 @@ func (s *spanConfigEventStream) streamLoop(ctx context.Context) error { if err != nil { return err } - if !tenantID.Equal(s.spec.Config.SpanConfigForTenant) { + if !tenantID.Equal(s.spec.TenantID) { continue } @@ -278,19 +275,17 @@ func (s *spanConfigEventStream) streamLoop(ctx context.Context) error { } } -func makeSpanConfigFrontier(spans roachpb.Spans) (*spanConfigFrontier, error) { - if len(spans) != 1 { - return nil, errors.AssertionFailedf("unexpected input span length %d", len(spans)) - } +func makeSpanConfigFrontier(span roachpb.Span) *spanConfigFrontier { + checkpoint := streampb.StreamEvent_StreamCheckpoint{ ResolvedSpans: []jobspb.ResolvedSpan{{ - Span: spans[0], + Span: span, }, }, } return &spanConfigFrontier{ checkpoint: checkpoint, - }, nil + } } type spanConfigFrontier struct { @@ -301,13 +296,13 @@ func (spf *spanConfigFrontier) update(frontier hlc.Timestamp) { spf.checkpoint.ResolvedSpans[0].Timestamp = frontier } -func streamSpanConfigPartition( - evalCtx *eval.Context, spec streampb.StreamPartitionSpec, +func streamSpanConfigs( + evalCtx *eval.Context, spec streampb.SpanConfigEventStreamSpec, ) (eval.ValueGenerator, error) { - if err := validateSpecs(evalCtx, spec); err != nil { - return nil, err + + if !evalCtx.SessionData().AvoidBuffering { + return nil, errors.New("partition streaming requires 'SET avoid_buffering = true' option") } - setConfigDefaults(&spec.Config) return &spanConfigEventStream{ spec: spec, diff --git a/pkg/ccl/streamingccl/streamproducer/stream_lifetime.go b/pkg/ccl/streamingccl/streamproducer/stream_lifetime.go index 82ee7890ae46..64ca9e54bf62 100644 --- a/pkg/ccl/streamingccl/streamproducer/stream_lifetime.go +++ b/pkg/ccl/streamingccl/streamproducer/stream_lifetime.go @@ -354,7 +354,6 @@ func setupSpanConfigsStream( }); err != nil { return nil, err } - spanConfigKey := evalCtx.Codec.TablePrefix(uint32(spanConfigID)) // TODO(msbutler): crop this span to the keyspan within the span config @@ -362,11 +361,10 @@ func setupSpanConfigsStream( // to stream span configs, which will make testing easier. span := roachpb.Span{Key: spanConfigKey, EndKey: spanConfigKey.PrefixEnd()} - spec := streampb.StreamPartitionSpec{ - Spans: roachpb.Spans{span}, - Config: streampb.StreamPartitionSpec_ExecutionConfig{ - MinCheckpointFrequency: streamingccl.StreamReplicationMinCheckpointFrequency.Get(&evalCtx.Settings.SV), - SpanConfigForTenant: tenantID, - }} - return streamSpanConfigPartition(evalCtx, spec) + spec := streampb.SpanConfigEventStreamSpec{ + Span: span, + TenantID: tenantID, + MinCheckpointFrequency: streamingccl.StreamReplicationMinCheckpointFrequency.Get(&evalCtx.Settings.SV), + } + return streamSpanConfigs(evalCtx, spec) } diff --git a/pkg/cmd/teamcity-trigger/main.go b/pkg/cmd/teamcity-trigger/main.go index 840e6907da60..f5bf16f67edd 100644 --- a/pkg/cmd/teamcity-trigger/main.go +++ b/pkg/cmd/teamcity-trigger/main.go @@ -194,7 +194,6 @@ func runTC(queueBuild func(string, map[string]string)) { opts["env.EXTRA_BAZEL_FLAGS"] = fmt.Sprintf("%s --@io_bazel_rules_go//go/config:race --test_env=GORACE=halt_on_error=1", extraBazelFlags) opts["env.STRESSFLAGS"] = fmt.Sprintf("-maxruns %d -maxtime %s -maxfails %d -p %d", maxRuns, maxTime, maxFails, noParallelism) - opts["env.TAGS"] = "race" queueBuildThenWait(buildID, opts) delete(opts, "env.TAGS") } diff --git a/pkg/kv/kvserver/client_lease_test.go b/pkg/kv/kvserver/client_lease_test.go index dfdcde67809d..b782f7c04201 100644 --- a/pkg/kv/kvserver/client_lease_test.go +++ b/pkg/kv/kvserver/client_lease_test.go @@ -16,7 +16,6 @@ import ( "math" "runtime" "strconv" - "strings" "sync" "sync/atomic" "testing" @@ -909,15 +908,24 @@ func gossipLiveness(t *testing.T, tc *testcluster.TestCluster) { } // This test replicates the behavior observed in -// https://github.com/cockroachdb/cockroach/issues/62485. We verify that -// when a dc with the leaseholder is lost, a node in a dc that does not have the -// lease preference can steal the lease, upreplicate the range and then give up the -// lease in a single cycle of the replicate_queue. +// https://github.com/cockroachdb/cockroach/issues/62485. We verify that when a +// dc with the leaseholder is lost, a node in a dc that does not have the lease +// preference, can steal the lease, upreplicate the range and then give up the +// lease in a short period of time. Previously, the replicate queue would +// reprocess, instead of requeue replicas. This behavior changed in #85219, to +// prevent queue priority inversion. Subsequently, this test only asserts that +// the lease preferences are satisfied quickly, rather than in a single +// replicate queue process() call. func TestLeasePreferencesDuringOutage(t *testing.T) { defer leaktest.AfterTest(t)() - skip.WithIssue(t, 88769, "flaky test") defer log.Scope(t).Close(t) + // This is a hefty test, so we skip it under short. + skip.UnderShort(t) + // The test has 5 nodes. Its possible in stress-race for nodes to be starved + // out heartbeating their liveness. + skip.UnderStressRace(t) + stickyRegistry := server.NewStickyVFSRegistry() ctx := context.Background() manualClock := hlc.NewHybridManualClock() @@ -947,15 +955,34 @@ func TestLeasePreferencesDuringOutage(t *testing.T) { locality("us", "mi"), locality("us", "mi"), } - // Disable expiration based lease transfers. It is possible that a (pseudo) - // dead node acquires the lease and we are forced to wait out the expiration - // timer, if this were not set. + + // This test disables the replicate queue. We wish to enable the replicate + // queue only for range we are testing, after marking some servers as dead. + // We also wait until the expected live stores are considered live from n1, + // if we didn't do this it would be possible for the range to process and be + // seen as unavailable due to manual clock jumps. + var testRangeID int64 + var clockJumpMu syncutil.Mutex + atomic.StoreInt64(&testRangeID, -1) + disabledQueueBypassFn := func(rangeID roachpb.RangeID) bool { + if rangeID == roachpb.RangeID(atomic.LoadInt64(&testRangeID)) { + clockJumpMu.Lock() + defer clockJumpMu.Unlock() + return true + } + return false + } settings := cluster.MakeTestingClusterSettings() sv := &settings.SV - kvserver.TransferExpirationLeasesFirstEnabled.Override(ctx, sv, false) - kvserver.ExpirationLeasesOnly.Override(ctx, sv, false) + // The remaining live stores (n1,n4,n5) may become suspect due to manual + // clock jumps. Disable the suspect timer to prevent them becoming suspect + // when we bump the clocks. + liveness.TimeAfterNodeSuspect.Override(ctx, sv, 0) + timeUntilNodeDead := liveness.TimeUntilNodeDead.Get(sv) + for i := 0; i < numNodes; i++ { serverArgs[i] = base.TestServerArgs{ + Settings: settings, Locality: localities[i], Knobs: base.TestingKnobs{ Server: &server.TestingKnobs{ @@ -967,6 +994,7 @@ func TestLeasePreferencesDuringOutage(t *testing.T) { // The Raft leadership may not end up on the eu node, but it needs to // be able to acquire the lease anyway. AllowLeaseRequestProposalsWhenNotLeader: true, + BaseQueueDisabledBypassFilter: disabledQueueBypassFn, }, }, StoreSpecs: []base.StoreSpec{ @@ -992,105 +1020,106 @@ func TestLeasePreferencesDuringOutage(t *testing.T) { require.NoError(t, tc.WaitForVoters(key, tc.Targets(1, 3)...)) tc.TransferRangeLeaseOrFatal(t, *repl.Desc(), tc.Target(1)) - // Shutdown the sf datacenter, which is going to kill the node with the lease. - tc.StopServer(1) - tc.StopServer(2) - - wait := func(duration time.Duration) { - manualClock.Increment(duration.Nanoseconds()) - // Gossip and heartbeat all the live stores, we do this manually otherwise the - // allocator on server 0 may see everyone as temporarily dead due to the - // clock move above. - for _, i := range []int{0, 3, 4} { - require.NoError(t, tc.Servers[i].HeartbeatNodeLiveness()) - require.NoError(t, tc.GetFirstStoreFromServer(t, i).GossipStore(ctx, true)) + func() { + // Lock the clockJumpMu, in order to prevent processing the test range before + // the intended stores are considered live from n1. If we didn't do this, it + // is possible for n1 to process the test range and find it unavailable + // (unactionable). + clockJumpMu.Lock() + defer clockJumpMu.Unlock() + + // Enable queue processing of the test range, right before we stop the sf + // datacenter. We expect the test range to be enqueued into the replicate + // queue shortly after. + rangeID := repl.GetRangeID() + atomic.StoreInt64(&testRangeID, int64(rangeID)) + + // Shutdown the sf datacenter, which is going to kill the node with the lease. + tc.StopServer(1) + tc.StopServer(2) + + wait := func(duration time.Duration) { + manualClock.Increment(duration.Nanoseconds()) + // Gossip and heartbeat all the live stores, we do this manually otherwise the + // allocator on server 0 may see everyone as temporarily dead due to the + // clock move above. + for _, i := range []int{0, 3, 4} { + require.NoError(t, tc.Servers[i].HeartbeatNodeLiveness()) + require.NoError(t, tc.GetFirstStoreFromServer(t, i).GossipStore(ctx, true)) + } } - } - // We need to wait until 2 and 3 are considered to be dead. - timeUntilNodeDead := liveness.TimeUntilNodeDead.Get(&tc.GetFirstStoreFromServer(t, 0).GetStoreConfig().Settings.SV) - wait(timeUntilNodeDead) - - checkDead := func(store *kvserver.Store, storeIdx int) error { - if dead, timetoDie, err := store.GetStoreConfig().StorePool.IsDead( - tc.GetFirstStoreFromServer(t, storeIdx).StoreID()); err != nil || !dead { - // Sometimes a gossip update arrives right after server shutdown and - // after we manually moved the time, so move it again. - if err == nil { - wait(timetoDie) + wait(timeUntilNodeDead) + + checkDead := func(store *kvserver.Store, storeIdx int) error { + if dead, timetoDie, err := store.GetStoreConfig().StorePool.IsDead( + tc.GetFirstStoreFromServer(t, storeIdx).StoreID()); err != nil || !dead { + // Sometimes a gossip update arrives right after server shutdown and + // after we manually moved the time, so move it again. + if err == nil { + wait(timetoDie) + } + // NB: errors.Wrapf(nil, ...) returns nil. + // nolint:errwrap + return errors.Errorf("expected server %d to be dead, instead err=%v, dead=%v", storeIdx, err, dead) } - // NB: errors.Wrapf(nil, ...) returns nil. - // nolint:errwrap - return errors.Errorf("expected server %d to be dead, instead err=%v, dead=%v", storeIdx, err, dead) + return nil } - return nil - } + + testutils.SucceedsSoon(t, func() error { + store := tc.GetFirstStoreFromServer(t, 0) + sl, available, _ := store.GetStoreConfig().StorePool.TestingGetStoreList() + if available != 3 { + return errors.Errorf( + "expected all 3 remaining stores to be live, but only got %d, stores=%v", + available, sl) + } + if err := checkDead(store, 1); err != nil { + return err + } + if err := checkDead(store, 2); err != nil { + return err + } + return nil + }) + }() + + // Send a request to force lease acquisition on _some_ remaining live node. + // Note, we expect this to be n1 (server 0). + ba := &kvpb.BatchRequest{} + ba.Add(getArgs(key)) + _, pErr := tc.Servers[0].DistSenderI().(kv.Sender).Send(ctx, ba) + require.Nil(t, pErr) testutils.SucceedsSoon(t, func() error { - store := tc.GetFirstStoreFromServer(t, 0) - sl, _, _ := store.GetStoreConfig().StorePool.TestingGetStoreList() - if len(sl.TestingStores()) != 3 { - return errors.Errorf("expected all 3 remaining stores to be live, but only got %v", - sl.TestingStores()) - } - if err := checkDead(store, 1); err != nil { - return err + // Validate that we upreplicated outside of SF. NB: This will occur prior + // to the lease preference being satisfied. + require.Equal(t, 3, len(repl.Desc().Replicas().Voters().VoterDescriptors())) + for _, replDesc := range repl.Desc().Replicas().Voters().VoterDescriptors() { + serv, err := tc.FindMemberServer(replDesc.StoreID) + require.NoError(t, err) + servLocality := serv.Locality() + dc, ok := servLocality.Find("dc") + require.True(t, ok) + if dc == "sf" { + return errors.Errorf( + "expected no replicas in dc=sf, but found replica in "+ + "dc=%s node_id=%v desc=%v", + dc, replDesc.NodeID, repl.Desc()) + } } - if err := checkDead(store, 2); err != nil { - return err + // Validate that the lease also transferred to a preferred locality. n4 + // (us) and n5 (us) are the only valid stores to be leaseholders during the + // outage. n1 is the original leaseholder, expect it to not be the + // leaseholder now. + if !repl.OwnsValidLease(ctx, tc.Servers[0].Clock().NowAsClockTimestamp()) { + return nil } - return nil - }) - _, _, enqueueError := tc.GetFirstStoreFromServer(t, 0). - Enqueue(ctx, "replicate", repl, true /* skipShouldQueue */, false /* async */) - require.NoError(t, enqueueError, "failed to enqueue replica for replication") - - var newLeaseHolder roachpb.ReplicationTarget - testutils.SucceedsSoon(t, func() error { - var err error - newLeaseHolder, err = tc.FindRangeLeaseHolder(*repl.Desc(), nil) - return err + return errors.Errorf( + "expected no leaseholder in region=us, but found %v", + repl.CurrentLeaseStatus(ctx), + ) }) - - srv, err := tc.FindMemberServer(newLeaseHolder.StoreID) - require.NoError(t, err) - loc := srv.Locality() - region, ok := loc.Find("region") - require.True(t, ok) - require.Equal(t, "us", region) - require.Equal(t, 3, len(repl.Desc().Replicas().Voters().VoterDescriptors())) - // Validate that we upreplicated outside of SF. - for _, replDesc := range repl.Desc().Replicas().Voters().VoterDescriptors() { - serv, err := tc.FindMemberServer(replDesc.StoreID) - require.NoError(t, err) - memberLoc := serv.Locality() - dc, ok := memberLoc.Find("dc") - require.True(t, ok) - require.NotEqual(t, "sf", dc) - } - history := repl.GetLeaseHistory() - // Make sure we see the eu node as a lease holder in the second to last - // leaseholder change. - // Since we can have expiration and epoch based leases at the tail of the - // history, we need to ignore them together if they originate from the same - // leaseholder. - nextNodeID := history[len(history)-1].Replica.NodeID - lastMove := len(history) - 2 - for ; lastMove >= 0; lastMove-- { - if history[lastMove].Replica.NodeID != nextNodeID { - break - } - } - lastMove++ - var leasesMsg []string - for _, h := range history { - leasesMsg = append(leasesMsg, h.String()) - } - leaseHistory := strings.Join(leasesMsg, ", ") - require.Greater(t, lastMove, 0, - "must have at least one leaseholder change in history (lease history: %s)", leaseHistory) - require.Equal(t, tc.Target(0).NodeID, history[lastMove-1].Replica.NodeID, - "node id prior to last lease move (lease history: %s)", leaseHistory) } // This test verifies that when a node starts flapping its liveness, all leases diff --git a/pkg/kv/kvserver/queue.go b/pkg/kv/kvserver/queue.go index c3ba4a1fe672..5c7234024ae9 100644 --- a/pkg/kv/kvserver/queue.go +++ b/pkg/kv/kvserver/queue.go @@ -655,13 +655,24 @@ func (bq *baseQueue) maybeAdd(ctx context.Context, repl replicaInQueue, now hlc. } bq.mu.Lock() - stopped := bq.mu.stopped || bq.mu.disabled + stopped := bq.mu.stopped + disabled := bq.mu.disabled bq.mu.Unlock() if stopped { return } + if disabled { + // The disabled queue bypass is used in tests which enable manual + // replication, however still require specific range(s) to be processed + // through the queue. + bypassDisabled := bq.store.TestingKnobs().BaseQueueDisabledBypassFilter + if bypassDisabled == nil || !bypassDisabled(repl.GetRangeID()) { + return + } + } + if !repl.IsInitialized() { return } @@ -729,10 +740,16 @@ func (bq *baseQueue) addInternal( } if bq.mu.disabled { - if log.V(3) { - log.Infof(ctx, "queue disabled") + // The disabled queue bypass is used in tests which enable manual + // replication, however still require specific range(s) to be processed + // through the queue. + bypassDisabled := bq.store.TestingKnobs().BaseQueueDisabledBypassFilter + if bypassDisabled == nil || !bypassDisabled(desc.RangeID) { + if log.V(3) { + log.Infof(ctx, "queue disabled") + } + return false, errQueueDisabled } - return false, errQueueDisabled } // If the replica is currently in purgatory, don't re-add it. diff --git a/pkg/kv/kvserver/testing_knobs.go b/pkg/kv/kvserver/testing_knobs.go index bcc9c98210ab..e735de1f61a0 100644 --- a/pkg/kv/kvserver/testing_knobs.go +++ b/pkg/kv/kvserver/testing_knobs.go @@ -493,6 +493,10 @@ type StoreTestingKnobs struct { // required. BaseQueueInterceptor func(ctx context.Context, bq *baseQueue) + // BaseQueueDisabledBypassFilter checks whether the replica for the given + // rangeID should ignore the queue being disabled, and be processed anyway. + BaseQueueDisabledBypassFilter func(rangeID roachpb.RangeID) bool + // InjectReproposalError injects an error in tryReproposeWithNewLeaseIndex. // If nil is returned, reproposal will be attempted. InjectReproposalError func(p *ProposalData) error diff --git a/pkg/repstream/streampb/stream.proto b/pkg/repstream/streampb/stream.proto index 7e37476a46d5..f966abf65c8b 100644 --- a/pkg/repstream/streampb/stream.proto +++ b/pkg/repstream/streampb/stream.proto @@ -66,15 +66,25 @@ message StreamPartitionSpec { google.protobuf.Duration min_checkpoint_frequency = 2 [(gogoproto.nullable) = false, (gogoproto.stdduration) = true]; - // Controls batch size in bytes. + // Controls the batch size, in bytes, sent over pgwire to the consumer. int64 batch_byte_size = 3; - - roachpb.TenantID span_config_for_tenant = 4 [(gogoproto.nullable) = false]; } ExecutionConfig config = 3 [(gogoproto.nullable) = false]; } +// SpanConfigEventStreamSpec is the span config event stream specification. +message SpanConfigEventStreamSpec { + roachpb.Span span = 1 [(gogoproto.nullable) = false]; + + roachpb.TenantID tenant_id = 2 [(gogoproto.nullable) = false, (gogoproto.customname) = "TenantID"]; + + // Controls how often checkpoint records are published. + google.protobuf.Duration min_checkpoint_frequency = 3 + [(gogoproto.nullable) = false, (gogoproto.stdduration) = true]; + +} + message ReplicationStreamSpec { message Partition { // ID of the node this partition resides diff --git a/pkg/ui/workspaces/cluster-ui/src/api/sqlApi.ts b/pkg/ui/workspaces/cluster-ui/src/api/sqlApi.ts index fa9933605bf9..31c11e99c77c 100644 --- a/pkg/ui/workspaces/cluster-ui/src/api/sqlApi.ts +++ b/pkg/ui/workspaces/cluster-ui/src/api/sqlApi.ts @@ -147,7 +147,7 @@ export function isUpgradeError(message: string): boolean { } /** - * errorMessage cleans the error message returned by the sqlApi, + * sqlApiErrorMessage cleans the error message returned by the sqlApi, * removing information not useful for the user. * e.g. the error message * "$executing stmt 1: run-query-via-api: only users with either MODIFYCLUSTERSETTING @@ -170,7 +170,6 @@ export function sqlApiErrorMessage(message: string): string { const idx = message.indexOf(":") + 1; return idx < message.length ? message.substring(idx) : message; } - return message; } diff --git a/pkg/ui/workspaces/cluster-ui/src/api/tableDetailsApi.ts b/pkg/ui/workspaces/cluster-ui/src/api/tableDetailsApi.ts index f4f20d7f1ec2..460519368800 100644 --- a/pkg/ui/workspaces/cluster-ui/src/api/tableDetailsApi.ts +++ b/pkg/ui/workspaces/cluster-ui/src/api/tableDetailsApi.ts @@ -9,6 +9,7 @@ // licenses/APL.txt. import { + combineQueryErrors, executeInternalSql, formatApiResult, LARGE_RESULT_SIZE, @@ -111,7 +112,7 @@ const getTableId: TableDetailsQuery = { }; // Table create statement. -type TableCreateStatementRow = { create_statement: string }; +export type TableCreateStatementRow = { create_statement: string }; const getTableCreateStatement: TableDetailsQuery = { createStmt: (dbName, tableName) => { @@ -129,22 +130,22 @@ const getTableCreateStatement: TableDetailsQuery = { txn_result: SqlTxnResult, resp: TableDetailsResponse, ) => { + if (txn_result.error) { + resp.createStmtResp.error = txn_result.error; + } if (!txnResultIsEmpty(txn_result)) { resp.createStmtResp.create_statement = txn_result.rows[0].create_statement; - } else { + } else if (!txn_result.error) { txn_result.error = new Error( "getTableCreateStatement: unexpected empty results", ); } - if (txn_result.error) { - resp.createStmtResp.error = txn_result.error; - } }, }; // Table grants. -type TableGrantsResponse = { +export type TableGrantsResponse = { grants: TableGrantsRow[]; }; @@ -181,7 +182,7 @@ const getTableGrants: TableDetailsQuery = { }; // Table schema details. -type TableSchemaDetailsRow = { +export type TableSchemaDetailsRow = { columns: string[]; indexes: string[]; }; @@ -332,7 +333,7 @@ const getTableZoneConfig: TableDetailsQuery = { }; // Table heuristics details. -type TableHeuristicDetailsRow = { +export type TableHeuristicDetailsRow = { stats_last_created_at: moment.Moment; }; @@ -371,7 +372,7 @@ type TableDetailsStats = { }; // Table span stats. -type TableSpanStatsRow = { +export type TableSpanStatsRow = { approximate_disk_bytes: number; live_bytes: number; total_bytes: number; @@ -418,7 +419,7 @@ const getTableSpanStats: TableDetailsQuery = { }, }; -type TableReplicaData = SqlApiQueryResponse<{ +export type TableReplicaData = SqlApiQueryResponse<{ nodeIDs: number[]; nodeCount: number; replicaCount: number; @@ -471,7 +472,7 @@ const getTableReplicas: TableDetailsQuery = { }; // Table index usage stats. -type TableIndexUsageStats = { +export type TableIndexUsageStats = { has_index_recommendations: boolean; }; @@ -569,6 +570,7 @@ export function createTableDetailsReq( max_result_size: LARGE_RESULT_SIZE, timeout: LONG_TIMEOUT, database: dbName, + separate_txns: true, }; } @@ -605,19 +607,24 @@ async function fetchTableDetails( csIndexUnusedDuration, ); const resp = await executeInternalSql(req); + const errs: Error[] = []; resp.execution.txn_results.forEach(txn_result => { - if (txn_result.rows) { - const query: TableDetailsQuery = - tableDetailQueries[txn_result.statement - 1]; - query.addToTableDetail(txn_result, detailsResponse); + if (txn_result.error) { + errs.push(txn_result.error); } + const query: TableDetailsQuery = + tableDetailQueries[txn_result.statement - 1]; + query.addToTableDetail(txn_result, detailsResponse); }); if (resp.error) { detailsResponse.error = resp.error; } + + detailsResponse.error = combineQueryErrors(errs, detailsResponse.error); return formatApiResult( detailsResponse, detailsResponse.error, - "retrieving table details information", + `retrieving table details information for table '${tableName}'`, + false, ); } diff --git a/pkg/ui/workspaces/cluster-ui/src/databaseDetailsPage/databaseDetailsConnected.ts b/pkg/ui/workspaces/cluster-ui/src/databaseDetailsPage/databaseDetailsConnected.ts index d557dabbd96d..8cdeb925e326 100644 --- a/pkg/ui/workspaces/cluster-ui/src/databaseDetailsPage/databaseDetailsConnected.ts +++ b/pkg/ui/workspaces/cluster-ui/src/databaseDetailsPage/databaseDetailsConnected.ts @@ -37,7 +37,7 @@ import { selectDatabaseDetailsTablesSortSetting, selectDatabaseDetailsViewModeSetting, } from "../store/databaseDetails/databaseDetails.selectors"; -import { combineLoadingErrors, deriveTableDetailsMemoized } from "../databases"; +import { deriveTableDetailsMemoized } from "../databases"; import { selectDropUnusedIndexDuration, selectIndexRecommendationsEnabled, @@ -56,11 +56,8 @@ const mapStateToProps = ( return { loading: !!databaseDetails[database]?.inFlight, loaded: !!databaseDetails[database]?.valid, - lastError: combineLoadingErrors( - databaseDetails[database]?.lastError, - databaseDetails[database]?.data?.maxSizeReached, - null, - ), + requestError: databaseDetails[database]?.lastError, + queryError: databaseDetails[database]?.data?.results?.error, name: database, showNodeRegionsColumn: Object.keys(nodeRegions).length > 1 && !isTenant, viewMode: selectDatabaseDetailsViewModeSetting(state), diff --git a/pkg/ui/workspaces/cluster-ui/src/databaseDetailsPage/databaseDetailsPage.stories.tsx b/pkg/ui/workspaces/cluster-ui/src/databaseDetailsPage/databaseDetailsPage.stories.tsx index 10b683a6ceab..a2516260dbbf 100644 --- a/pkg/ui/workspaces/cluster-ui/src/databaseDetailsPage/databaseDetailsPage.stories.tsx +++ b/pkg/ui/workspaces/cluster-ui/src/databaseDetailsPage/databaseDetailsPage.stories.tsx @@ -34,7 +34,8 @@ const history = H.createHashHistory(); const withLoadingIndicator: DatabaseDetailsPageProps = { loading: true, loaded: false, - lastError: undefined, + requestError: undefined, + queryError: undefined, showIndexRecommendations: false, csIndexUnusedDuration: indexUnusedDuration, name: randomName(), @@ -68,7 +69,8 @@ const withLoadingIndicator: DatabaseDetailsPageProps = { const withoutData: DatabaseDetailsPageProps = { loading: false, loaded: true, - lastError: null, + requestError: null, + queryError: undefined, showIndexRecommendations: false, csIndexUnusedDuration: indexUnusedDuration, name: randomName(), @@ -108,7 +110,8 @@ function createTable(): DatabaseDetailsPageDataTable { return { loading: false, loaded: true, - lastError: null, + requestError: null, + queryError: undefined, name: randomName(), details: { columnCount: _.random(5, 42), diff --git a/pkg/ui/workspaces/cluster-ui/src/databaseDetailsPage/databaseDetailsPage.tsx b/pkg/ui/workspaces/cluster-ui/src/databaseDetailsPage/databaseDetailsPage.tsx index a063c3668493..aad2ce5ce0d2 100644 --- a/pkg/ui/workspaces/cluster-ui/src/databaseDetailsPage/databaseDetailsPage.tsx +++ b/pkg/ui/workspaces/cluster-ui/src/databaseDetailsPage/databaseDetailsPage.tsx @@ -24,14 +24,12 @@ import { SortedTable, SortSetting, } from "src/sortedtable"; -import * as format from "src/util/format"; import { DATE_FORMAT, EncodeDatabaseTableUri } from "src/util/format"; import { mvccGarbage, syncHistory, unique } from "../util"; import styles from "./databaseDetailsPage.module.scss"; import sortableTableStyles from "src/sortedtable/sortedtable.module.scss"; import { baseHeadingClasses } from "src/transactionsPage/transactionsPageClasses"; -import { Moment } from "moment-timezone"; import { Anchor } from "../anchor"; import LoadingError from "../sqlActivity/errorComponent"; import { Loading } from "../loading"; @@ -47,10 +45,22 @@ import { TableStatistics } from "src/tableStatistics"; import { Timestamp, Timezone } from "../timestamp"; import { DbDetailsBreadcrumbs, - IndexRecWithIconCell, + DiskSizeCell, + IndexesCell, MVCCInfoCell, TableNameCell, -} from "./helperComponents"; +} from "./tableCells"; +import { + isMaxSizeError, + SqlApiQueryResponse, + SqlExecutionErrorMessage, + TableHeuristicDetailsRow, + TableIndexUsageStats, + TableSchemaDetailsRow, + TableSpanStatsRow, +} from "../api"; +import { checkInfoAvailable } from "../databases"; +import { InlineAlert } from "@cockroachlabs/ui-components"; const cx = classNames.bind(styles); const sortableTableCx = classNames.bind(sortableTableStyles); @@ -97,7 +107,10 @@ const sortableTableCx = classNames.bind(sortableTableStyles); export interface DatabaseDetailsPageData { loading: boolean; loaded: boolean; - lastError: Error; + // Request error when getting table names. + requestError?: Error; + // Query error when getting table names. + queryError?: SqlExecutionErrorMessage; name: string; tables: DatabaseDetailsPageDataTable[]; sortSettingTables: SortSetting; @@ -116,23 +129,24 @@ export interface DatabaseDetailsPageDataTable { name: string; loading: boolean; loaded: boolean; - lastError: Error; + // Request error when getting table details. + requestError?: Error; + // Query error when getting table details. + queryError?: SqlExecutionErrorMessage; details: DatabaseDetailsPageDataTableDetails; } -export interface DatabaseDetailsPageDataTableDetails { - columnCount: number; - indexCount: number; - userCount: number; +interface GrantsData { roles: string[]; - grants: string[]; - statsLastUpdated?: Moment; - hasIndexRecommendations: boolean; - totalBytes: number; - liveBytes: number; - livePercentage: number; - replicationSizeInBytes: number; - rangeCount: number; + privileges: string[]; +} + +export interface DatabaseDetailsPageDataTableDetails { + schemaDetails?: SqlApiQueryResponse; + grants: SqlApiQueryResponse; + statsLastUpdated?: SqlApiQueryResponse; + indexStatRecs?: SqlApiQueryResponse; + spanStats?: SqlApiQueryResponse; // Array of node IDs used to unambiguously filter by node and region. nodes?: number[]; // String of nodes grouped by region in alphabetical order, e.g. @@ -171,7 +185,6 @@ interface DatabaseDetailsPageState { pagination: ISortedTablePagination; filters?: Filters; activeFilters?: number; - lastDetailsError: Error; } const tablePageSize = 20; @@ -209,7 +222,6 @@ export class DatabaseDetailsPage extends React.Component< current: 1, pageSize: 20, }, - lastDetailsError: null, }; const { history } = this.props; @@ -252,7 +264,7 @@ export class DatabaseDetailsPage extends React.Component< } componentDidMount(): void { - if (!this.props.loaded && !this.props.loading && !this.props.lastError) { + if (!this.props.loaded && !this.props.loading && !this.props.requestError) { this.props.refreshDatabaseDetails( this.props.name, this.props.csIndexUnusedDuration, @@ -302,7 +314,7 @@ export class DatabaseDetailsPage extends React.Component< i++ ) { const table = filteredTables[i]; - if (!table.loaded && !table.loading && table.lastError == undefined) { + if (!table.loaded && !table.loading && table.requestError == undefined) { return true; } } @@ -311,7 +323,6 @@ export class DatabaseDetailsPage extends React.Component< } private refresh(): void { - let lastDetailsError: Error; // Load everything by default let filteredTables = this.props.tables; @@ -337,22 +348,7 @@ export class DatabaseDetailsPage extends React.Component< } filteredTables.forEach(table => { - if (table.lastError !== undefined) { - lastDetailsError = table.lastError; - } - if ( - lastDetailsError && - this.state.lastDetailsError?.name != lastDetailsError?.name - ) { - this.setState({ lastDetailsError: lastDetailsError }); - } - - if ( - !table.loaded && - !table.loading && - (table.lastError === undefined || - table.lastError?.name === "GetDatabaseInfoError") - ) { + if (!table.loaded && !table.loading && table.requestError === undefined) { this.props.refreshTableDetails( this.props.name, table.name, @@ -528,16 +524,6 @@ export class DatabaseDetailsPage extends React.Component< } } - checkInfoAvailable = ( - error: Error, - cell: React.ReactNode, - ): React.ReactNode => { - if (error) { - return "(unavailable)"; - } - return cell; - }; - private columnsForTablesViewMode(): ColumnDescriptor[] { return ( [ @@ -561,12 +547,8 @@ export class DatabaseDetailsPage extends React.Component< Replication Size ), - cell: table => - this.checkInfoAvailable( - table.lastError, - format.Bytes(table.details.replicationSizeInBytes), - ), - sort: table => table.details.replicationSizeInBytes, + cell: table => , + sort: table => table.details.spanStats?.approximate_disk_bytes, className: cx("database-table__col-size"), name: "replicationSize", }, @@ -580,8 +562,12 @@ export class DatabaseDetailsPage extends React.Component< ), cell: table => - this.checkInfoAvailable(table.lastError, table.details.rangeCount), - sort: table => table.details.rangeCount, + checkInfoAvailable( + table.requestError, + table.details.spanStats?.error, + table.details.spanStats?.range_count, + ), + sort: table => table.details.spanStats?.range_count, className: cx("database-table__col-range-count"), name: "rangeCount", }, @@ -595,8 +581,12 @@ export class DatabaseDetailsPage extends React.Component< ), cell: table => - this.checkInfoAvailable(table.lastError, table.details.columnCount), - sort: table => table.details.columnCount, + checkInfoAvailable( + table.requestError, + table.details.schemaDetails?.error, + table.details.schemaDetails?.columns?.length, + ), + sort: table => table.details.schemaDetails?.columns?.length, className: cx("database-table__col-column-count"), name: "columnCount", }, @@ -609,19 +599,13 @@ export class DatabaseDetailsPage extends React.Component< Indexes ), - cell: table => { - return table.details.hasIndexRecommendations && - this.props.showIndexRecommendations - ? this.checkInfoAvailable( - table.lastError, - , - ) - : this.checkInfoAvailable( - table.lastError, - table.details.indexCount, - ); - }, - sort: table => table.details.indexCount, + cell: table => ( + + ), + sort: table => table.details.schemaDetails?.indexes?.length, className: cx("database-table__col-index-count"), name: "indexCount", }, @@ -635,8 +619,9 @@ export class DatabaseDetailsPage extends React.Component< ), cell: table => - this.checkInfoAvailable( - table.lastError, + checkInfoAvailable( + table.requestError, + null, table.details.nodesByRegionString || "None", ), sort: table => table.details.nodesByRegionString, @@ -664,11 +649,14 @@ export class DatabaseDetailsPage extends React.Component< ), cell: table => - this.checkInfoAvailable( - table.lastError, - , + checkInfoAvailable( + table.requestError, + table.details.spanStats?.error, + table.details.spanStats ? ( + + ) : null, ), - sort: table => table.details.livePercentage, + sort: table => table.details.spanStats?.live_percentage, className: cx("database-table__col-column-count"), name: "livePercentage", }, @@ -683,7 +671,7 @@ export class DatabaseDetailsPage extends React.Component< ), cell: table => ( @@ -727,8 +715,12 @@ export class DatabaseDetailsPage extends React.Component< ), cell: table => - this.checkInfoAvailable(table.lastError, table.details.userCount), - sort: table => table.details.userCount, + checkInfoAvailable( + table.requestError, + table.details.grants?.error, + table.details.grants?.roles.length, + ), + sort: table => table.details.grants?.roles.length, className: cx("database-table__col-user-count"), name: "userCount", }, @@ -739,11 +731,12 @@ export class DatabaseDetailsPage extends React.Component< ), cell: table => - this.checkInfoAvailable( - table.lastError, - table.details.roles.join(", "), + checkInfoAvailable( + table.requestError, + table.details.grants?.error, + table.details.grants?.roles.join(", "), ), - sort: table => table.details.roles.join(", "), + sort: table => table.details.grants?.roles.join(", "), className: cx("database-table__col-roles"), name: "roles", }, @@ -754,11 +747,12 @@ export class DatabaseDetailsPage extends React.Component< ), cell: table => - this.checkInfoAvailable( - table.lastError, - table.details.grants.join(", "), + checkInfoAvailable( + table.requestError, + table.details.grants?.error, + table.details.grants?.privileges.join(", "), ), - sort: table => table.details.grants.join(", "), + sort: table => table.details.grants?.privileges?.join(", "), className: cx("database-table__col-grants"), name: "grants", }, @@ -864,54 +858,46 @@ export class DatabaseDetailsPage extends React.Component< ( - - - This database has no tables. - - } - /> - )} + error={this.props.requestError} renderError={() => LoadingError({ statsType: "databases", - error: this.props.lastError, + error: this.props.requestError, }) } - /> - {!this.props.loading && ( - + {isMaxSizeError(this.props.queryError?.message) && ( + + Not all tables are displayed because the maximum number of + tables was reached in the console.  + + } + /> + )} + <>} - renderError={() => - LoadingError({ - statsType: "part of the information", - error: this.state.lastDetailsError, - }) + disableSortSizeLimit={disableTableSortSize} + renderNoResult={ +
+ + This database has no tables. +
} /> - )} +
- { + return ( + <> + {checkInfoAvailable( + table.requestError, + table.details?.spanStats?.error, + table.details?.spanStats?.approximate_disk_bytes + ? format.Bytes(table.details?.spanStats?.approximate_disk_bytes) + : null, + )} + + ); +}; + export const TableNameCell = ({ table, dbDetails, @@ -58,19 +77,55 @@ export const TableNameCell = ({ linkURL += `?tab=grants`; } } + let icon = ; + if (table.requestError || table.queryError) { + icon = ( + + + + ); + } return ( - + {icon} {table.name} ); }; -export const IndexRecWithIconCell = ({ +export const IndexesCell = ({ table, + showIndexRecommendations, }: { table: DatabaseDetailsPageDataTable; + showIndexRecommendations: boolean; }): JSX.Element => { + const elem = ( + <> + {checkInfoAvailable( + table.requestError, + table.details?.schemaDetails?.error, + table.details?.schemaDetails?.indexes?.length, + )} + + ); + // If index recommendations are not enabled or we don't have any index recommendations, + // just return the number of indexes. + if ( + !table.details.indexStatRecs?.has_index_recommendations || + !showIndexRecommendations + ) { + return elem; + } + // Display an icon indicating we have index recommendations next to the number of indexes. return (
- {table.details.indexCount} + {elem}
); }; @@ -92,12 +147,16 @@ export const MVCCInfoCell = ({ return ( <>

- {format.Percentage(details.livePercentage, 1, 1)} + {format.Percentage(details?.spanStats?.live_percentage, 1, 1)}

- {format.Bytes(details.liveBytes)}{" "} + + {format.Bytes(details?.spanStats?.live_bytes)} + {" "} live data /{" "} - {format.Bytes(details.totalBytes)} + + {format.Bytes(details?.spanStats?.total_bytes)} + {" total"}

diff --git a/pkg/ui/workspaces/cluster-ui/src/databases/combiners.ts b/pkg/ui/workspaces/cluster-ui/src/databases/combiners.ts index 743eeefdbd7f..3379ce9ba580 100644 --- a/pkg/ui/workspaces/cluster-ui/src/databases/combiners.ts +++ b/pkg/ui/workspaces/cluster-ui/src/databases/combiners.ts @@ -139,24 +139,19 @@ const deriveDatabaseTableDetails = ( name: table, loading: !!details?.inFlight, loaded: !!details?.valid, - lastError: details?.lastError, + requestError: details?.lastError, + queryError: details?.data?.results?.error, details: { - columnCount: results?.schemaDetails.columns?.length || 0, - indexCount: results?.schemaDetails.indexes.length || 0, - userCount: normalizedRoles.length, - roles: normalizedRoles, - grants: normalizedPrivileges, - statsLastUpdated: - results?.heuristicsDetails.stats_last_created_at || null, - hasIndexRecommendations: - results?.stats.indexStats.has_index_recommendations || false, - totalBytes: results?.stats?.spanStats.total_bytes || 0, - liveBytes: results?.stats?.spanStats.live_bytes || 0, - livePercentage: results?.stats?.spanStats.live_percentage || 0, - replicationSizeInBytes: - results?.stats?.spanStats.approximate_disk_bytes || 0, + schemaDetails: results?.schemaDetails, + grants: { + roles: normalizedRoles, + privileges: normalizedPrivileges, + error: results?.grantsResp?.error, + }, + statsLastUpdated: results?.heuristicsDetails, + indexStatRecs: results?.stats.indexStats, + spanStats: results?.stats.spanStats, nodes: nodes, - rangeCount: results?.stats?.spanStats.range_count || 0, nodesByRegionString: getNodesByRegionString(nodes, nodeRegions, isTenant), }, }; diff --git a/pkg/ui/workspaces/db-console/src/views/databases/databaseDetailsPage/redux.spec.ts b/pkg/ui/workspaces/db-console/src/views/databases/databaseDetailsPage/redux.spec.ts index f181f4c64e75..0631d305b6d7 100644 --- a/pkg/ui/workspaces/db-console/src/views/databases/databaseDetailsPage/redux.spec.ts +++ b/pkg/ui/workspaces/db-console/src/views/databases/databaseDetailsPage/redux.spec.ts @@ -92,19 +92,22 @@ class TestDriver { // Expect Moment type field to be equal. expect( // Moments are the same - moment(statsLastUpdated).isSame(expectedStatsLastUpdated) || + moment(statsLastUpdated.stats_last_created_at).isSame( + expectedStatsLastUpdated.stats_last_created_at, + ) || // Moments are null. - (statsLastUpdated === expectedStatsLastUpdated && - statsLastUpdated === null), + (statsLastUpdated.stats_last_created_at === + expectedStatsLastUpdated.stats_last_created_at && + statsLastUpdated.stats_last_created_at === null), ).toBe(true); } assertTableRoles(name: string, expected: string[]) { - expect(this.findTable(name).details.roles).toEqual(expected); + expect(this.findTable(name).details.grants.roles).toEqual(expected); } assertTableGrants(name: string, expected: string[]) { - expect(this.findTable(name).details.grants).toEqual(expected); + expect(this.findTable(name).details.grants.privileges).toEqual(expected); } async refreshDatabaseDetails() { @@ -145,7 +148,8 @@ describe("Database Details Page", function () { driver.assertProperties({ loading: false, loaded: false, - lastError: undefined, + requestError: undefined, + queryError: undefined, name: "things", search: null, filters: defaultFilters, @@ -186,7 +190,8 @@ describe("Database Details Page", function () { driver.assertProperties({ loading: false, loaded: true, - lastError: undefined, + requestError: null, + queryError: undefined, name: "things", search: null, filters: defaultFilters, @@ -203,21 +208,19 @@ describe("Database Details Page", function () { name: `"public"."foo"`, loading: false, loaded: false, - lastError: undefined, + requestError: undefined, + queryError: undefined, details: { - columnCount: 0, - indexCount: 0, - userCount: 0, - roles: [], - grants: [], - statsLastUpdated: null, - hasIndexRecommendations: false, - livePercentage: 0, - liveBytes: 0, - totalBytes: 0, + schemaDetails: undefined, + grants: { + error: undefined, + roles: [], + privileges: [], + }, + statsLastUpdated: undefined, + indexStatRecs: undefined, + spanStats: undefined, nodes: [], - replicationSizeInBytes: 0, - rangeCount: 0, nodesByRegionString: "", }, }, @@ -225,21 +228,19 @@ describe("Database Details Page", function () { name: `"public"."bar"`, loading: false, loaded: false, - lastError: undefined, + requestError: undefined, + queryError: undefined, details: { - columnCount: 0, - indexCount: 0, - userCount: 0, - roles: [], - grants: [], - statsLastUpdated: null, - hasIndexRecommendations: false, - livePercentage: 0, - totalBytes: 0, - liveBytes: 0, + schemaDetails: undefined, + grants: { + error: undefined, + roles: [], + privileges: [], + }, + statsLastUpdated: undefined, + indexStatRecs: undefined, + spanStats: undefined, nodes: [], - replicationSizeInBytes: 0, - rangeCount: 0, nodesByRegionString: "", }, }, @@ -382,20 +383,29 @@ describe("Database Details Page", function () { name: `"public"."foo"`, loading: false, loaded: true, - lastError: null, + requestError: null, + queryError: undefined, details: { - columnCount: 3, - indexCount: 2, - userCount: 2, - roles: ["admin", "public"], - grants: ["CREATE", "SELECT"], - statsLastUpdated: mockStatsLastCreatedTimestamp, - hasIndexRecommendations: true, - liveBytes: 200, - totalBytes: 400, - livePercentage: 0.5, - replicationSizeInBytes: 100, - rangeCount: 400, + schemaDetails: { + columns: ["a", "b", "c"], + indexes: ["d", "e"], + }, + grants: { + error: undefined, + roles: ["admin", "public"], + privileges: ["CREATE", "SELECT"], + }, + statsLastUpdated: { + stats_last_created_at: mockStatsLastCreatedTimestamp, + }, + indexStatRecs: { has_index_recommendations: true }, + spanStats: { + approximate_disk_bytes: 100, + live_bytes: 200, + total_bytes: 400, + range_count: 400, + live_percentage: 0.5, + }, nodes: [1, 2, 3], nodesByRegionString: "", }, @@ -405,20 +415,27 @@ describe("Database Details Page", function () { name: `"public"."bar"`, loading: false, loaded: true, - lastError: null, + requestError: null, + queryError: undefined, details: { - columnCount: 2, - indexCount: 4, - userCount: 3, - roles: ["root", "app", "data"], - grants: ["ALL", "SELECT", "INSERT"], - statsLastUpdated: null, - hasIndexRecommendations: false, - liveBytes: 100, - totalBytes: 100, - livePercentage: 1, - replicationSizeInBytes: 10, - rangeCount: 50, + schemaDetails: { + columns: ["a", "b"], + indexes: ["c", "d", "e", "f"], + }, + grants: { + error: undefined, + roles: ["root", "app", "data"], + privileges: ["ALL", "SELECT", "INSERT"], + }, + statsLastUpdated: { stats_last_created_at: null }, + indexStatRecs: { has_index_recommendations: false }, + spanStats: { + live_percentage: 1, + live_bytes: 100, + total_bytes: 100, + range_count: 50, + approximate_disk_bytes: 10, + }, nodes: [1, 2, 3, 4, 5], nodesByRegionString: "", }, diff --git a/pkg/ui/workspaces/db-console/src/views/databases/databaseDetailsPage/redux.ts b/pkg/ui/workspaces/db-console/src/views/databases/databaseDetailsPage/redux.ts index 89bba471a1de..47cd3488ac09 100644 --- a/pkg/ui/workspaces/db-console/src/views/databases/databaseDetailsPage/redux.ts +++ b/pkg/ui/workspaces/db-console/src/views/databases/databaseDetailsPage/redux.ts @@ -15,7 +15,6 @@ import { defaultFilters, Filters, ViewMode, - combineLoadingErrors, deriveTableDetailsMemoized, } from "@cockroachlabs/cluster-ui"; @@ -81,11 +80,8 @@ export const mapStateToProps = ( return { loading: !!databaseDetails[database]?.inFlight, loaded: !!databaseDetails[database]?.valid, - lastError: combineLoadingErrors( - databaseDetails[database]?.lastError, - databaseDetails[database]?.data?.maxSizeReached, - null, - ), + requestError: databaseDetails[database]?.lastError, + queryError: databaseDetails[database]?.data?.results?.error, name: database, showNodeRegionsColumn: selectIsMoreThanOneNode(state), viewMode: viewModeLocalSetting.selector(state),