diff --git a/docs/generated/settings/settings-for-tenants.txt b/docs/generated/settings/settings-for-tenants.txt index fb0f6ec544d8..72e07afa86b5 100644 --- a/docs/generated/settings/settings-for-tenants.txt +++ b/docs/generated/settings/settings-for-tenants.txt @@ -269,6 +269,7 @@ sql.stats.non_default_columns.min_retention_period duration 24h0m0s minimum rete sql.stats.persisted_rows.max integer 1000000 maximum number of rows of statement and transaction statistics that will be persisted in the system tables sql.stats.post_events.enabled boolean false if set, an event is logged for every CREATE STATISTICS job sql.stats.response.max integer 20000 the maximum number of statements and transaction stats returned in a CombinedStatements request +sql.stats.response.show_internal.enabled boolean false controls if statistics for internal executions should be returned by the CombinedStatements endpoint. This endpoint is used to display statistics on the Statement and Transaction fingerprint pages under SQL Activity sql.stats.system_tables.enabled boolean true when true, enables use of statistics on system tables by the query optimizer sql.stats.system_tables_autostats.enabled boolean true when true, enables automatic collection of statistics on system tables sql.telemetry.query_sampling.enabled boolean false when set to true, executed queries will emit an event on the telemetry logging channel diff --git a/docs/generated/settings/settings.html b/docs/generated/settings/settings.html index 199ca927ddce..f53ad01bdb55 100644 --- a/docs/generated/settings/settings.html +++ b/docs/generated/settings/settings.html @@ -203,6 +203,7 @@ sql.stats.persisted_rows.maxinteger1000000maximum number of rows of statement and transaction statistics that will be persisted in the system tables sql.stats.post_events.enabledbooleanfalseif set, an event is logged for every CREATE STATISTICS job sql.stats.response.maxinteger20000the maximum number of statements and transaction stats returned in a CombinedStatements request +sql.stats.response.show_internal.enabledbooleanfalsecontrols if statistics for internal executions should be returned by the CombinedStatements endpoint. This endpoint is used to display statistics on the Statement and Transaction fingerprint pages under SQL Activity sql.stats.system_tables.enabledbooleantruewhen true, enables use of statistics on system tables by the query optimizer sql.stats.system_tables_autostats.enabledbooleantruewhen true, enables automatic collection of statistics on system tables sql.telemetry.query_sampling.enabledbooleanfalsewhen set to true, executed queries will emit an event on the telemetry logging channel diff --git a/pkg/ccl/backupccl/backupinfo/BUILD.bazel b/pkg/ccl/backupccl/backupinfo/BUILD.bazel index a7ca79d41cfc..5d359950deec 100644 --- a/pkg/ccl/backupccl/backupinfo/BUILD.bazel +++ b/pkg/ccl/backupccl/backupinfo/BUILD.bazel @@ -33,6 +33,7 @@ go_library( "//pkg/sql/sem/tree", "//pkg/sql/stats", "//pkg/storage", + "//pkg/util", "//pkg/util/ctxgroup", "//pkg/util/encoding", "//pkg/util/hlc", diff --git a/pkg/ccl/backupccl/backupinfo/manifest_handling.go b/pkg/ccl/backupccl/backupinfo/manifest_handling.go index e45022ed3c60..000578e202b6 100644 --- a/pkg/ccl/backupccl/backupinfo/manifest_handling.go +++ b/pkg/ccl/backupccl/backupinfo/manifest_handling.go @@ -39,6 +39,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/stats" + "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/ctxgroup" "github.com/cockroachdb/cockroach/pkg/util/encoding" "github.com/cockroachdb/cockroach/pkg/util/hlc" @@ -82,13 +83,11 @@ const ( // WriteMetadataSST controls if we write the experimental new format BACKUP // metadata file. -// kv.bulkio.rite_metadata_sst.enabled set to false by default due to -// https://github.com/cockroachdb/cockroach/issues/85564. var WriteMetadataSST = settings.RegisterBoolSetting( settings.TenantWritable, "kv.bulkio.write_metadata_sst.enabled", "write experimental new format BACKUP metadata file", - true, + util.ConstantWithMetamorphicTestBool("write-metadata-sst", false), ) // IsGZipped detects whether the given bytes represent GZipped data. This check diff --git a/pkg/keys/constants.go b/pkg/keys/constants.go index 4390db95ef57..caee9bcdec3b 100644 --- a/pkg/keys/constants.go +++ b/pkg/keys/constants.go @@ -133,6 +133,10 @@ var ( // LocalRangeLastReplicaGCTimestampSuffix is the suffix for a range's last // replica GC timestamp (for GC of old replicas). LocalRangeLastReplicaGCTimestampSuffix = []byte("rlrt") + // LocalRangeMVCCRangeKeyGCLockSuffix is the suffix for a lock obtained + // by range tombstone operations to ensure they don't overlap with + // GC requests while allowing point traffic to go through unobstructed. + LocalRangeMVCCRangeKeyGCLockSuffix = []byte("rltu") // localRangeLastVerificationTimestampSuffix is DEPRECATED and remains to // prevent reuse. localRangeLastVerificationTimestampSuffix = []byte("rlvt") diff --git a/pkg/keys/keys.go b/pkg/keys/keys.go index 1d39b9c0e700..57d4a132d12a 100644 --- a/pkg/keys/keys.go +++ b/pkg/keys/keys.go @@ -279,6 +279,12 @@ func RangeGCThresholdKey(rangeID roachpb.RangeID) roachpb.Key { return MakeRangeIDPrefixBuf(rangeID).RangeGCThresholdKey() } +// MVCCRangeKeyGCKey returns a range local key protecting range +// tombstone mvcc stats calculations during range tombstone GC. +func MVCCRangeKeyGCKey(rangeID roachpb.RangeID) roachpb.Key { + return MakeRangeIDPrefixBuf(rangeID).MVCCRangeKeyGCKey() +} + // RangeVersionKey returns a system-local for the range version. func RangeVersionKey(rangeID roachpb.RangeID) roachpb.Key { return MakeRangeIDPrefixBuf(rangeID).RangeVersionKey() @@ -1055,3 +1061,9 @@ func (b RangeIDPrefixBuf) RaftReplicaIDKey() roachpb.Key { func (b RangeIDPrefixBuf) RangeLastReplicaGCTimestampKey() roachpb.Key { return append(b.unreplicatedPrefix(), LocalRangeLastReplicaGCTimestampSuffix...) } + +// MVCCRangeKeyGCKey returns a range local key protecting range +// tombstone mvcc stats calculations during range tombstone GC. +func (b RangeIDPrefixBuf) MVCCRangeKeyGCKey() roachpb.Key { + return append(b.unreplicatedPrefix(), LocalRangeMVCCRangeKeyGCLockSuffix...) +} diff --git a/pkg/kv/kvserver/batcheval/cmd_add_sstable.go b/pkg/kv/kvserver/batcheval/cmd_add_sstable.go index 4a794e33b21a..4f9f1a246cf1 100644 --- a/pkg/kv/kvserver/batcheval/cmd_add_sstable.go +++ b/pkg/kv/kvserver/batcheval/cmd_add_sstable.go @@ -57,8 +57,17 @@ func declareKeysAddSSTable( // NB: The range end key is not available, so this will pessimistically // latch up to args.EndKey.Next(). If EndKey falls on the range end key, the // span will be tightened during evaluation. + // Even if we obtain latches beyond the end range here, it won't cause + // contention with the subsequent range because latches are enforced per + // range. l, r := rangeTombstonePeekBounds(args.Key, args.EndKey, rs.GetStartKey().AsRawKey(), nil) latchSpans.AddMVCC(spanset.SpanReadOnly, roachpb.Span{Key: l, EndKey: r}, header.Timestamp) + + // Obtain a read only lock on range key GC key to serialize with + // range tombstone GC requests. + latchSpans.AddNonMVCC(spanset.SpanReadOnly, roachpb.Span{ + Key: keys.MVCCRangeKeyGCKey(rs.GetRangeID()), + }) } } diff --git a/pkg/kv/kvserver/batcheval/cmd_clear_range.go b/pkg/kv/kvserver/batcheval/cmd_clear_range.go index 88ca15ed92a8..11b06f026c94 100644 --- a/pkg/kv/kvserver/batcheval/cmd_clear_range.go +++ b/pkg/kv/kvserver/batcheval/cmd_clear_range.go @@ -56,9 +56,18 @@ func declareKeysClearRange( // NB: The range end key is not available, so this will pessimistically latch // up to args.EndKey.Next(). If EndKey falls on the range end key, the span // will be tightened during evaluation. + // Even if we obtain latches beyond the end range here, it won't cause + // contention with the subsequent range because latches are enforced per + // range. args := req.(*roachpb.ClearRangeRequest) l, r := rangeTombstonePeekBounds(args.Key, args.EndKey, rs.GetStartKey().AsRawKey(), nil) latchSpans.AddMVCC(spanset.SpanReadOnly, roachpb.Span{Key: l, EndKey: r}, header.Timestamp) + + // Obtain a read only lock on range key GC key to serialize with + // range tombstone GC requests. + latchSpans.AddNonMVCC(spanset.SpanReadOnly, roachpb.Span{ + Key: keys.MVCCRangeKeyGCKey(rs.GetRangeID()), + }) } // ClearRange wipes all MVCC versions of keys covered by the specified diff --git a/pkg/kv/kvserver/batcheval/cmd_delete_range.go b/pkg/kv/kvserver/batcheval/cmd_delete_range.go index 54acdf8a0744..8369732305d6 100644 --- a/pkg/kv/kvserver/batcheval/cmd_delete_range.go +++ b/pkg/kv/kvserver/batcheval/cmd_delete_range.go @@ -49,6 +49,9 @@ func declareKeysDeleteRange( // NB: The range end key is not available, so this will pessimistically // latch up to args.EndKey.Next(). If EndKey falls on the range end key, the // span will be tightened during evaluation. + // Even if we obtain latches beyond the end range here, it won't cause + // contention with the subsequent range because latches are enforced per + // range. l, r := rangeTombstonePeekBounds(args.Key, args.EndKey, rs.GetStartKey().AsRawKey(), nil) latchSpans.AddMVCC(spanset.SpanReadOnly, roachpb.Span{Key: l, EndKey: r}, header.Timestamp) @@ -56,6 +59,12 @@ func declareKeysDeleteRange( latchSpans.AddNonMVCC(spanset.SpanReadOnly, roachpb.Span{ Key: keys.RangeDescriptorKey(rs.GetStartKey()), }) + + // Obtain a read only lock on range key GC key to serialize with + // range tombstone GC requests. + latchSpans.AddNonMVCC(spanset.SpanReadOnly, roachpb.Span{ + Key: keys.MVCCRangeKeyGCKey(rs.GetRangeID()), + }) } } diff --git a/pkg/kv/kvserver/batcheval/cmd_end_transaction.go b/pkg/kv/kvserver/batcheval/cmd_end_transaction.go index 3d68657c5f24..e664a1b827b1 100644 --- a/pkg/kv/kvserver/batcheval/cmd_end_transaction.go +++ b/pkg/kv/kvserver/batcheval/cmd_end_transaction.go @@ -151,6 +151,12 @@ func declareKeysEndTxn( Key: abortspan.MinKey(rs.GetRangeID()), EndKey: abortspan.MaxKey(rs.GetRangeID()), }) + + // Protect range tombstones from collection by GC to avoid interference + // with MVCCStats calculation. + latchSpans.AddNonMVCC(spanset.SpanReadOnly, roachpb.Span{ + Key: keys.MVCCRangeKeyGCKey(rs.GetRangeID()), + }) } if mt := et.InternalCommitTrigger.MergeTrigger; mt != nil { // Merges copy over the RHS abort span to the LHS, and compute @@ -183,6 +189,11 @@ func declareKeysEndTxn( Key: leftPeekBound, EndKey: rightPeekBound, }) + // Protect range tombstones from collection by GC to avoid interference + // with MVCCStats calculation. + latchSpans.AddNonMVCC(spanset.SpanReadOnly, roachpb.Span{ + Key: keys.MVCCRangeKeyGCKey(mt.LeftDesc.RangeID), + }) } } } diff --git a/pkg/kv/kvserver/batcheval/cmd_gc.go b/pkg/kv/kvserver/batcheval/cmd_gc.go index b148481d6d3b..4bb69417e3b2 100644 --- a/pkg/kv/kvserver/batcheval/cmd_gc.go +++ b/pkg/kv/kvserver/batcheval/cmd_gc.go @@ -37,22 +37,25 @@ func declareKeysGC( _ time.Duration, ) { gcr := req.(*roachpb.GCRequest) - // When GC-ing MVCC range key tombstones or individual range keys, we need to - // serialize with all writes that overlap the MVCC range tombstone, as well as - // the immediate left/right neighboring keys. This is because a range key - // write has non-local effects, i.e. it can fragment or merge other range keys - // at other timestamps and at its boundaries, and this has a non-commutative - // effect on MVCC stats -- if someone writes a new range key while we're GCing - // one below, the stats would come out wrong. - // Note that we only need to serialize with writers (including other GC - // processes) and not with readers (that are guaranteed to be above the GC - // threshold). To achieve this, we declare read-write access at - // hlc.MaxTimestamp which will not block any readers. - for _, span := range mergeAdjacentSpans(makeLookupBoundariesForGCRanges( - rs.GetStartKey().AsRawKey(), nil, gcr.RangeKeys, - )) { - latchSpans.AddMVCC(spanset.SpanReadWrite, span, hlc.MaxTimestamp) + if gcr.RangeKeys != nil { + // When GC-ing MVCC range key tombstones, we need to serialize with + // range key writes that overlap the MVCC range tombstone, as well as + // the immediate left/right neighboring keys. This is because a range + // key write has non-local effects, i.e. it can fragment or merge other + // range keys at other timestamps and at its boundaries, and this has + // a non-commutative effect on MVCC stats -- if someone writes a new + // range key while we're GCing one below, the stats would come out wrong. + // + // To achieve this, we use a virtual latch key that will prevent any + // range key write operations until GC request is done. All other range + // tombstone write operations obtain a read latch on this key and can + // run concurrently. + latchSpans.AddNonMVCC(spanset.SpanReadWrite, roachpb.Span{ + Key: keys.MVCCRangeKeyGCKey(rs.GetRangeID()), + }) } + // For ClearRangeKey request we still obtain a wide write lock as we don't + // expect any operations running on the range. if rk := gcr.ClearRangeKey; rk != nil { latchSpans.AddMVCC(spanset.SpanReadWrite, roachpb.Span{Key: rk.StartKey, EndKey: rk.EndKey}, hlc.MaxTimestamp) diff --git a/pkg/kv/kvserver/batcheval/cmd_revert_range.go b/pkg/kv/kvserver/batcheval/cmd_revert_range.go index bbf862dc5cd0..0f8f52ccb7a6 100644 --- a/pkg/kv/kvserver/batcheval/cmd_revert_range.go +++ b/pkg/kv/kvserver/batcheval/cmd_revert_range.go @@ -54,8 +54,17 @@ func declareKeysRevertRange( // NB: The range end key is not available, so this will pessimistically // latch up to args.EndKey.Next(). If EndKey falls on the range end key, the // span will be tightened during evaluation. + // Even if we obtain latches beyond the end range here, it won't cause + // contention with the subsequent range because latches are enforced per + // range. l, r := rangeTombstonePeekBounds(args.Key, args.EndKey, rs.GetStartKey().AsRawKey(), nil) latchSpans.AddMVCC(spanset.SpanReadOnly, roachpb.Span{Key: l, EndKey: r}, header.Timestamp) + + // Obtain a read only lock on range key GC key to serialize with + // range tombstone GC requests. + latchSpans.AddNonMVCC(spanset.SpanReadOnly, roachpb.Span{ + Key: keys.MVCCRangeKeyGCKey(rs.GetRangeID()), + }) } // isEmptyKeyTimeRange checks if the span has no writes in (since,until]. diff --git a/pkg/kv/kvserver/gc/gc_iterator.go b/pkg/kv/kvserver/gc/gc_iterator.go index f4d97b8fba4c..a4c3fd5bd7af 100644 --- a/pkg/kv/kvserver/gc/gc_iterator.go +++ b/pkg/kv/kvserver/gc/gc_iterator.go @@ -198,7 +198,7 @@ func (it *gcIterator) currentRangeTS() hlc.Timestamp { } it.cachedRangeTombstoneKey = append(it.cachedRangeTombstoneKey[:0], rangeTombstoneStartKey...) - if v, ok := it.it.RangeKeys().FirstBelow(it.threshold); ok { + if v, ok := it.it.RangeKeys().FirstAtOrBelow(it.threshold); ok { it.cachedRangeTombstoneTS = v.Timestamp } else { it.cachedRangeTombstoneTS = hlc.Timestamp{} diff --git a/pkg/kv/kvserver/raft_transport.go b/pkg/kv/kvserver/raft_transport.go index 566a834715e0..b3bd100ea3bd 100644 --- a/pkg/kv/kvserver/raft_transport.go +++ b/pkg/kv/kvserver/raft_transport.go @@ -505,6 +505,9 @@ func (t *RaftTransport) SendAsync( case ch <- req: return true default: + if logRaftSendQueueFullEvery.ShouldLog() { + log.Warningf(t.AnnotateCtx(context.Background()), "raft send queue to n%d is full", toNodeID) + } releaseRaftMessageRequest(req) return false } diff --git a/pkg/kv/kvserver/store_raft.go b/pkg/kv/kvserver/store_raft.go index 92164cd63f8e..eefd61f34f95 100644 --- a/pkg/kv/kvserver/store_raft.go +++ b/pkg/kv/kvserver/store_raft.go @@ -32,6 +32,11 @@ import ( "go.etcd.io/etcd/raft/v3/raftpb" ) +var ( + logRaftRecvQueueFullEvery = log.Every(1 * time.Second) + logRaftSendQueueFullEvery = log.Every(1 * time.Second) +) + type raftRequestInfo struct { req *kvserverpb.RaftMessageRequest size int64 // size of req in bytes @@ -305,6 +310,9 @@ func (s *Store) HandleRaftUncoalescedRequest( // that dropping the request is safe. Raft will retry. s.metrics.RaftRcvdDropped.Inc(1) s.metrics.RaftRcvdDroppedBytes.Inc(size) + if logRaftRecvQueueFullEvery.ShouldLog() { + log.Warningf(ctx, "raft receive queue for r%d is full", req.RangeID) + } return false } return enqueue diff --git a/pkg/server/cluster_settings.go b/pkg/server/cluster_settings.go index da7643356946..b15b119d18aa 100644 --- a/pkg/server/cluster_settings.go +++ b/pkg/server/cluster_settings.go @@ -21,3 +21,13 @@ var SQLStatsResponseMax = settings.RegisterIntSetting( 20000, settings.NonNegativeInt, ).WithPublic() + +// SQLStatsShowInternal controls if statistics for internal executions should be returned by the +// CombinedStatements endpoint. +var SQLStatsShowInternal = settings.RegisterBoolSetting( + settings.TenantWritable, + "sql.stats.response.show_internal.enabled", + "controls if statistics for internal executions should be returned by the CombinedStatements endpoint. This "+ + "endpoint is used to display statistics on the Statement and Transaction fingerprint pages under SQL Activity", + false, +).WithPublic() diff --git a/pkg/server/combined_statement_stats.go b/pkg/server/combined_statement_stats.go index 299f13422459..b50b7e3244d8 100644 --- a/pkg/server/combined_statement_stats.go +++ b/pkg/server/combined_statement_stats.go @@ -70,7 +70,9 @@ func getCombinedStatementStats( startTime := getTimeFromSeconds(req.Start) endTime := getTimeFromSeconds(req.End) limit := SQLStatsResponseMax.Get(&settings.SV) - whereClause, orderAndLimit, args := getCombinedStatementsQueryClausesAndArgs(startTime, endTime, limit, testingKnobs) + showInternal := SQLStatsShowInternal.Get(&settings.SV) + whereClause, orderAndLimit, args := getCombinedStatementsQueryClausesAndArgs( + startTime, endTime, limit, testingKnobs, showInternal) statements, err := collectCombinedStatements(ctx, ie, whereClause, args, orderAndLimit) if err != nil { return nil, serverError(ctx, err) @@ -98,13 +100,17 @@ func getCombinedStatementStats( // The whereClause will be in the format `WHERE A = $1 AND B = $2` and // args will return the list of arguments in order that will replace the actual values. func getCombinedStatementsQueryClausesAndArgs( - start, end *time.Time, limit int64, testingKnobs *sqlstats.TestingKnobs, + start, end *time.Time, limit int64, testingKnobs *sqlstats.TestingKnobs, showInternal bool, ) (whereClause string, orderAndLimitClause string, args []interface{}) { var buffer strings.Builder buffer.WriteString(testingKnobs.GetAOSTClause()) - // Filter out internal statements by app name. - buffer.WriteString(fmt.Sprintf(" WHERE app_name NOT LIKE '%s%%'", catconstants.InternalAppNamePrefix)) + if showInternal { + buffer.WriteString(" WHERE true") + } else { + // Filter out internal statements by app name. + buffer.WriteString(fmt.Sprintf(" WHERE app_name NOT LIKE '%s%%'", catconstants.InternalAppNamePrefix)) + } if start != nil { buffer.WriteString(" AND aggregated_ts >= $1") @@ -357,7 +363,8 @@ func getStatementDetails( testingKnobs *sqlstats.TestingKnobs, ) (*serverpb.StatementDetailsResponse, error) { limit := SQLStatsResponseMax.Get(&settings.SV) - whereClause, args, err := getStatementDetailsQueryClausesAndArgs(req, testingKnobs) + showInternal := SQLStatsShowInternal.Get(&settings.SV) + whereClause, args, err := getStatementDetailsQueryClausesAndArgs(req, testingKnobs, showInternal) if err != nil { return nil, serverError(ctx, err) } @@ -407,7 +414,7 @@ func getStatementDetails( // The whereClause will be in the format `WHERE A = $1 AND B = $2` and // args will return the list of arguments in order that will replace the actual values. func getStatementDetailsQueryClausesAndArgs( - req *serverpb.StatementDetailsRequest, testingKnobs *sqlstats.TestingKnobs, + req *serverpb.StatementDetailsRequest, testingKnobs *sqlstats.TestingKnobs, showInternal bool, ) (whereClause string, args []interface{}, err error) { var buffer strings.Builder buffer.WriteString(testingKnobs.GetAOSTClause()) @@ -420,18 +427,24 @@ func getStatementDetailsQueryClausesAndArgs( args = append(args, sqlstatsutil.EncodeUint64ToBytes(fingerprintID)) buffer.WriteString(fmt.Sprintf(" WHERE fingerprint_id = $%d", len(args))) - // Filter out internal statements by app name. - buffer.WriteString(fmt.Sprintf(" AND app_name NOT LIKE '%s%%'", catconstants.InternalAppNamePrefix)) + if !showInternal { + // Filter out internal statements by app name. + buffer.WriteString(fmt.Sprintf(" AND app_name NOT LIKE '%s%%'", catconstants.InternalAppNamePrefix)) + } // Statements are grouped ignoring the app name in the Statements/Transactions page, so when // calling for the Statement Details endpoint, this value can be empty or a list of app names. if len(req.AppNames) > 0 { if !(len(req.AppNames) == 1 && req.AppNames[0] == "") { + hasInternal := false buffer.WriteString(" AND (") for i, app := range req.AppNames { if app == "(unset)" { app = "" } + if strings.Contains(app, catconstants.InternalAppNamePrefix) { + hasInternal = true + } if i != 0 { args = append(args, app) buffer.WriteString(fmt.Sprintf(" OR app_name = $%d", len(args))) @@ -440,6 +453,9 @@ func getStatementDetailsQueryClausesAndArgs( buffer.WriteString(fmt.Sprintf(" app_name = $%d", len(args))) } } + if hasInternal { + buffer.WriteString(fmt.Sprintf(" OR app_name LIKE '%s%%'", catconstants.InternalAppNamePrefix)) + } buffer.WriteString(" )") } } diff --git a/pkg/storage/mvcc.go b/pkg/storage/mvcc.go index dae16a090194..58c6baa27c3b 100644 --- a/pkg/storage/mvcc.go +++ b/pkg/storage/mvcc.go @@ -1184,7 +1184,7 @@ func mvccGetMetadata( // metadata), or the point version's timestamp if it was a tombstone. if hasRange { rangeKeys := iter.RangeKeys() - if v, ok := rangeKeys.FirstAbove(unsafeKey.Timestamp); ok { + if v, ok := rangeKeys.FirstAtOrAbove(unsafeKey.Timestamp); ok { meta.Deleted = true meta.Timestamp = rangeKeys.Versions[0].Timestamp.ToLegacyTimestamp() keyLastSeen := v.Timestamp @@ -2681,7 +2681,7 @@ func MVCCClearTimeRange( // is cheap, so we don't need any caching here. if !restoredMeta.Deleted { if rangeKeys := iter.RangeKeysIgnoringTime(); !rangeKeys.IsEmpty() { - if v, ok := rangeKeys.FirstAbove(k.Timestamp); ok { + if v, ok := rangeKeys.FirstAtOrAbove(k.Timestamp); ok { if v.Timestamp.LessEq(clearedMeta.Timestamp.ToTimestamp()) { restoredMeta.Deleted = true restoredMeta.KeyBytes = 0 @@ -2719,7 +2719,7 @@ func MVCCClearTimeRange( // revealed the key, since it may have been covered by the point key that // we cleared or a different range tombstone below the one we cleared. if !v.IsTombstone() { - if v, ok := clearRangeKeys.FirstAbove(k.Timestamp); ok { + if v, ok := clearRangeKeys.FirstAtOrAbove(k.Timestamp); ok { if !clearedMetaKey.Key.Equal(k.Key) || !clearedMeta.Timestamp.ToTimestamp().LessEq(v.Timestamp) { rangeKeys := iter.RangeKeysIgnoringTime() @@ -4503,7 +4503,7 @@ func mvccResolveWriteIntent( // synthesize a point tombstone at the lowest range tombstone covering it. // This is where the point key ceases to exist, contributing to GCBytesAge. if len(unsafeNextValueRaw) > 0 && hasRange { - if v, found := iter.RangeKeys().FirstAbove(unsafeNextKey.Timestamp); found { + if v, found := iter.RangeKeys().FirstAtOrAbove(unsafeNextKey.Timestamp); found { unsafeNextKey.Timestamp = v.Timestamp unsafeNextValueRaw = []byte{} } @@ -5009,7 +5009,7 @@ func MVCCGarbageCollect( // For non deletions, we need to find if we had a range tombstone // between this and next value (prevNanos) to use its timestamp for // computing GCBytesAge. - if kv, ok := rangeTombstones.FirstAbove(unsafeIterKey.Timestamp); ok { + if kv, ok := rangeTombstones.FirstAtOrAbove(unsafeIterKey.Timestamp); ok { if kv.Timestamp.WallTime < fromNS { fromNS = kv.Timestamp.WallTime } @@ -5562,7 +5562,7 @@ func computeStatsForIterWithVisitors( var nextRangeTombstone hlc.Timestamp if isValue { if !rangeTombstones.IsEmpty() && unsafeKey.Timestamp.LessEq(rangeTombstones.Newest()) { - if v, ok := rangeTombstones.FirstAbove(unsafeKey.Timestamp); ok { + if v, ok := rangeTombstones.FirstAtOrAbove(unsafeKey.Timestamp); ok { nextRangeTombstone = v.Timestamp } } diff --git a/pkg/storage/mvcc_key.go b/pkg/storage/mvcc_key.go index 1107233cb211..cd907b32bcc4 100644 --- a/pkg/storage/mvcc_key.go +++ b/pkg/storage/mvcc_key.go @@ -590,16 +590,16 @@ func (s *MVCCRangeKeyStack) Excise(from, to hlc.Timestamp) bool { return s.Versions.Excise(from, to) } -// FirstAbove does a binary search for the first range key version at or above -// the given timestamp. Returns false if no matching range key was found. -func (s MVCCRangeKeyStack) FirstAbove(ts hlc.Timestamp) (MVCCRangeKeyVersion, bool) { - return s.Versions.FirstAbove(ts) +// FirstAtOrAbove does a binary search for the first range key version at or +// above the given timestamp. Returns false if no matching range key was found. +func (s MVCCRangeKeyStack) FirstAtOrAbove(ts hlc.Timestamp) (MVCCRangeKeyVersion, bool) { + return s.Versions.FirstAtOrAbove(ts) } -// FirstBelow does a binary search for the first range key version at or below -// the given timestamp. Returns false if no matching range key was found. -func (s MVCCRangeKeyStack) FirstBelow(ts hlc.Timestamp) (MVCCRangeKeyVersion, bool) { - return s.Versions.FirstBelow(ts) +// FirstAtOrBelow does a binary search for the first range key version at or +// below the given timestamp. Returns false if no matching range key was found. +func (s MVCCRangeKeyStack) FirstAtOrBelow(ts hlc.Timestamp) (MVCCRangeKeyVersion, bool) { + return s.Versions.FirstAtOrBelow(ts) } // HasBetween checks whether an MVCC range key exists between the two given @@ -727,9 +727,9 @@ func (v *MVCCRangeKeyVersions) Excise(from, to hlc.Timestamp) bool { return true } -// FirstAbove does a binary search for the first range key version at or above -// the given timestamp. Returns false if no matching range key was found. -func (v MVCCRangeKeyVersions) FirstAbove(ts hlc.Timestamp) (MVCCRangeKeyVersion, bool) { +// FirstAtOrAbove does a binary search for the first range key version at or +// above the given timestamp. Returns false if no matching range key was found. +func (v MVCCRangeKeyVersions) FirstAtOrAbove(ts hlc.Timestamp) (MVCCRangeKeyVersion, bool) { // This is kind of odd due to sort.Search() semantics: we do a binary search // for the first range key that's below the timestamp, then return the // previous range key if any. @@ -743,9 +743,9 @@ func (v MVCCRangeKeyVersions) FirstAbove(ts hlc.Timestamp) (MVCCRangeKeyVersion, return MVCCRangeKeyVersion{}, false } -// FirstBelow does a binary search for the first range key version at or below -// the given timestamp. Returns false if no matching range key was found. -func (v MVCCRangeKeyVersions) FirstBelow(ts hlc.Timestamp) (MVCCRangeKeyVersion, bool) { +// FirstAtOrBelow does a binary search for the first range key version at or +// below the given timestamp. Returns false if no matching range key was found. +func (v MVCCRangeKeyVersions) FirstAtOrBelow(ts hlc.Timestamp) (MVCCRangeKeyVersion, bool) { if length := len(v); length > 0 { if i := sort.Search(length, func(i int) bool { return v[i].Timestamp.LessEq(ts) @@ -759,7 +759,7 @@ func (v MVCCRangeKeyVersions) FirstBelow(ts hlc.Timestamp) (MVCCRangeKeyVersion, // HasBetween checks whether an MVCC range key exists between the two given // timestamps (both inclusive, in order). func (v MVCCRangeKeyVersions) HasBetween(lower, upper hlc.Timestamp) bool { - if version, ok := v.FirstAbove(lower); ok { + if version, ok := v.FirstAtOrAbove(lower); ok { // Consider equal timestamps to be "between". This shouldn't really happen, // since MVCC enforces point and range keys can't have the same timestamp. return version.Timestamp.LessEq(upper) diff --git a/pkg/storage/mvcc_key_test.go b/pkg/storage/mvcc_key_test.go index dd8ce9e9c3a0..f9357a8a219b 100644 --- a/pkg/storage/mvcc_key_test.go +++ b/pkg/storage/mvcc_key_test.go @@ -733,7 +733,7 @@ func TestMVCCRangeKeyStackExcise(t *testing.T) { } } -func TestMVCCRangeKeyStackFirstAbove(t *testing.T) { +func TestMVCCRangeKeyStackFirstAtOrAbove(t *testing.T) { defer leaktest.AfterTest(t)() rangeKeys := rangeKeyStack("a", "f", map[int]MVCCValue{6: {}, 4: {}, 3: {}, 1: {}}) @@ -753,7 +753,7 @@ func TestMVCCRangeKeyStackFirstAbove(t *testing.T) { } for _, tc := range testcases { t.Run(fmt.Sprintf("%d", tc.ts), func(t *testing.T) { - v, ok := rangeKeys.FirstAbove(wallTS(tc.ts)) + v, ok := rangeKeys.FirstAtOrAbove(wallTS(tc.ts)) if tc.expect == 0 { require.False(t, ok) require.Empty(t, v) @@ -765,7 +765,7 @@ func TestMVCCRangeKeyStackFirstAbove(t *testing.T) { } } -func TestMVCCRangeKeyStackFirstBelow(t *testing.T) { +func TestMVCCRangeKeyStackFirstAtOrBelow(t *testing.T) { defer leaktest.AfterTest(t)() rangeKeys := rangeKeyStack("a", "f", map[int]MVCCValue{6: {}, 4: {}, 3: {}, 1: {}}) @@ -785,7 +785,7 @@ func TestMVCCRangeKeyStackFirstBelow(t *testing.T) { } for _, tc := range testcases { t.Run(fmt.Sprintf("%d", tc.ts), func(t *testing.T) { - v, ok := rangeKeys.FirstBelow(wallTS(tc.ts)) + v, ok := rangeKeys.FirstAtOrBelow(wallTS(tc.ts)) if tc.expect == 0 { require.False(t, ok) require.Empty(t, v) diff --git a/pkg/storage/read_as_of_iterator.go b/pkg/storage/read_as_of_iterator.go index 80c497638310..ed60b2c20f1e 100644 --- a/pkg/storage/read_as_of_iterator.go +++ b/pkg/storage/read_as_of_iterator.go @@ -146,7 +146,7 @@ func (f *ReadAsOfIterator) advance(seeked bool) { hasPoint, hasRange := f.iter.HasPointAndRange() f.newestRangeTombstone = hlc.Timestamp{} if hasRange { - if v, ok := f.iter.RangeKeys().FirstBelow(f.asOf); ok { + if v, ok := f.iter.RangeKeys().FirstAtOrBelow(f.asOf); ok { f.newestRangeTombstone = v.Timestamp } } diff --git a/pkg/storage/sst.go b/pkg/storage/sst.go index 2cbbbeaff51a..52e2ef99a5de 100644 --- a/pkg/storage/sst.go +++ b/pkg/storage/sst.go @@ -416,7 +416,7 @@ func CheckSSTConflicts( "ingested range key collides with an existing one: %s", sstTopTombstone) } if !extValueDeleted { - sstRangeKeyVersion, ok := sstRangeKeys.FirstAbove(extKey.Timestamp) + sstRangeKeyVersion, ok := sstRangeKeys.FirstAtOrAbove(extKey.Timestamp) if !ok { return enginepb.MVCCStats{}, errors.AssertionFailedf("expected range tombstone above timestamp %v", extKey.Timestamp) } diff --git a/pkg/ui/workspaces/cluster-ui/src/statementsPage/statementsPage.selectors.ts b/pkg/ui/workspaces/cluster-ui/src/statementsPage/statementsPage.selectors.ts index 510f6e9db8df..51039b1d601b 100644 --- a/pkg/ui/workspaces/cluster-ui/src/statementsPage/statementsPage.selectors.ts +++ b/pkg/ui/workspaces/cluster-ui/src/statementsPage/statementsPage.selectors.ts @@ -59,27 +59,28 @@ export const selectApps = createSelector(sqlStatsSelector, sqlStatsState => { } let sawBlank = false; + let sawInternal = false; const apps: { [app: string]: boolean } = {}; sqlStatsState.data.statements.forEach( (statement: ICollectedStatementStatistics) => { - const isNotInternalApp = + if ( sqlStatsState.data.internal_app_name_prefix && - !statement.key.key_data.app.startsWith( + statement.key.key_data.app.startsWith( sqlStatsState.data.internal_app_name_prefix, - ); - if ( - sqlStatsState.data.internal_app_name_prefix == undefined || - isNotInternalApp + ) ) { - if (statement.key.key_data.app) { - apps[statement.key.key_data.app] = true; - } else { - sawBlank = true; - } + sawInternal = true; + } else if (statement.key.key_data.app) { + apps[statement.key.key_data.app] = true; + } else { + sawBlank = true; } }, ); - return [].concat(sawBlank ? [unset] : []).concat(Object.keys(apps).sort()); + return [] + .concat(sawInternal ? [sqlStatsState.data.internal_app_name_prefix] : []) + .concat(sawBlank ? [unset] : []) + .concat(Object.keys(apps).sort()); }); // selectDatabases returns the array of all databases with statement statistics present diff --git a/pkg/ui/workspaces/cluster-ui/src/transactionsPage/utils.ts b/pkg/ui/workspaces/cluster-ui/src/transactionsPage/utils.ts index fc82280c33a7..db98af8cca22 100644 --- a/pkg/ui/workspaces/cluster-ui/src/transactionsPage/utils.ts +++ b/pkg/ui/workspaces/cluster-ui/src/transactionsPage/utils.ts @@ -42,9 +42,13 @@ export const getTrxAppFilterOptions = ( prefix: string, ): string[] => { const uniqueAppNames = new Set( - transactions - .filter(t => !t.stats_data.app.startsWith(prefix)) - .map(t => (t.stats_data.app ? t.stats_data.app : unset)), + transactions.map(t => + t.stats_data.app + ? t.stats_data.app.startsWith(prefix) + ? prefix + : t.stats_data.app + : unset, + ), ); return Array.from(uniqueAppNames).sort(); diff --git a/pkg/ui/workspaces/db-console/src/views/statements/statementsPage.tsx b/pkg/ui/workspaces/db-console/src/views/statements/statementsPage.tsx index 1e96cc8ecdb4..1ff027ffdbce 100644 --- a/pkg/ui/workspaces/db-console/src/views/statements/statementsPage.tsx +++ b/pkg/ui/workspaces/db-console/src/views/statements/statementsPage.tsx @@ -181,27 +181,26 @@ export const selectApps = createSelector( } let sawBlank = false; + let sawInternal = false; const apps: { [app: string]: boolean } = {}; state.data.statements.forEach( (statement: ICollectedStatementStatistics) => { - const isNotInternalApp = + if ( state.data.internal_app_name_prefix && - !statement.key.key_data.app.startsWith( + statement.key.key_data.app.startsWith( state.data.internal_app_name_prefix, - ); - if ( - state.data.internal_app_name_prefix == undefined || - isNotInternalApp + ) ) { - if (statement.key.key_data.app) { - apps[statement.key.key_data.app] = true; - } else { - sawBlank = true; - } + sawInternal = true; + } else if (statement.key.key_data.app) { + apps[statement.key.key_data.app] = true; + } else { + sawBlank = true; } }, ); return [] + .concat(sawInternal ? [state.data.internal_app_name_prefix] : []) .concat(sawBlank ? [unset] : []) .concat(Object.keys(apps)) .sort();