Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
111190: sql: add SHOW EXPERIMENTAL_FINGERPRINTS FROM VIRTUAL CLUSTER r=knz,msbutler a=stevendanna

This adds a user-facing interface for tenant fingerprinting that does not require crdb_internal functions.

We've moved the underlying implementation to a function that lives on the planner so that it can be shared from both call sites.

Additionally,

- we now issue the ExportRequests in parallel, and
- we allow users with MANAGETENANT to issue the fingerprint command.

Epic: none

Release note: None

111432: kv: do not panic when non-locking read requests ask for replicated locks r=nvanbenschoten a=arulajmani

Over in cockroachdb#111429, we saw a non-locking read request which specified
a key locking durability of lock.Replicated. This is not allowed --
however, it doesn't warrant a panic on the server. We fix this and
improve the error message.

Epic: none

Release note: None

Co-authored-by: Steven Danna <[email protected]>
Co-authored-by: Arul Ajmani <[email protected]>
  • Loading branch information
3 people committed Sep 29, 2023
3 parents d0c8f31 + 77f258d + 001fbe8 commit 79a213a
Show file tree
Hide file tree
Showing 55 changed files with 639 additions and 264 deletions.
7 changes: 5 additions & 2 deletions pkg/kv/kvserver/batcheval/cmd_add_sstable.go
Original file line number Diff line number Diff line change
Expand Up @@ -49,9 +49,11 @@ func declareKeysAddSSTable(
latchSpans *spanset.SpanSet,
lockSpans *lockspanset.LockSpanSet,
maxOffset time.Duration,
) {
) error {
args := req.(*kvpb.AddSSTableRequest)
DefaultDeclareIsolatedKeys(rs, header, req, latchSpans, lockSpans, maxOffset)
if err := DefaultDeclareIsolatedKeys(rs, header, req, latchSpans, lockSpans, maxOffset); err != nil {
return err
}
// We look up the range descriptor key to return its span.
latchSpans.AddNonMVCC(spanset.SpanReadOnly, roachpb.Span{Key: keys.RangeDescriptorKey(rs.GetStartKey())})

Expand All @@ -72,6 +74,7 @@ func declareKeysAddSSTable(
Key: keys.MVCCRangeKeyGCKey(rs.GetRangeID()),
})
}
return nil
}

// AddSSTableRewriteConcurrency sets the concurrency of a single SST rewrite.
Expand Down
3 changes: 2 additions & 1 deletion pkg/kv/kvserver/batcheval/cmd_barrier.go
Original file line number Diff line number Diff line change
Expand Up @@ -32,7 +32,7 @@ func declareKeysBarrier(
latchSpans *spanset.SpanSet,
_ *lockspanset.LockSpanSet,
_ time.Duration,
) {
) error {
// Barrier is special-cased in the concurrency manager to *not* actually
// grab these latches. Instead, any conflicting latches with these are waited
// on, but new latches aren't inserted.
Expand All @@ -44,6 +44,7 @@ func declareKeysBarrier(
// follower. We don't currently need any guarantees regarding concurrent
// reads, so this is acceptable.
latchSpans.AddNonMVCC(spanset.SpanReadWrite, req.Header().Span())
return nil
}

// Barrier evaluation is a no-op, as all the latch waiting happens in
Expand Down
8 changes: 6 additions & 2 deletions pkg/kv/kvserver/batcheval/cmd_clear_range.go
Original file line number Diff line number Diff line change
Expand Up @@ -46,8 +46,11 @@ func declareKeysClearRange(
latchSpans *spanset.SpanSet,
lockSpans *lockspanset.LockSpanSet,
maxOffset time.Duration,
) {
DefaultDeclareIsolatedKeys(rs, header, req, latchSpans, lockSpans, maxOffset)
) error {
err := DefaultDeclareIsolatedKeys(rs, header, req, latchSpans, lockSpans, maxOffset)
if err != nil {
return err
}
// We look up the range descriptor key to check whether the span
// is equal to the entire range for fast stats updating.
latchSpans.AddNonMVCC(spanset.SpanReadOnly, roachpb.Span{Key: keys.RangeDescriptorKey(rs.GetStartKey())})
Expand All @@ -71,6 +74,7 @@ func declareKeysClearRange(
latchSpans.AddNonMVCC(spanset.SpanReadOnly, roachpb.Span{
Key: keys.MVCCRangeKeyGCKey(rs.GetRangeID()),
})
return nil
}

// ClearRange wipes all MVCC versions of keys covered by the specified
Expand Down
4 changes: 3 additions & 1 deletion pkg/kv/kvserver/batcheval/cmd_clear_range_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -189,7 +189,9 @@ func TestCmdClearRange(t *testing.T) {
// should not cross the range bounds.
var latchSpans spanset.SpanSet
var lockSpans lockspanset.LockSpanSet
declareKeysClearRange(&desc, &cArgs.Header, cArgs.Args, &latchSpans, &lockSpans, 0)
require.NoError(t,
declareKeysClearRange(&desc, &cArgs.Header, cArgs.Args, &latchSpans, &lockSpans, 0),
)
batch := &wrappedBatch{Batch: spanset.NewBatchAt(eng.NewBatch(), &latchSpans, cArgs.Header.Timestamp)}
defer batch.Close()

Expand Down
3 changes: 2 additions & 1 deletion pkg/kv/kvserver/batcheval/cmd_compute_checksum.go
Original file line number Diff line number Diff line change
Expand Up @@ -36,7 +36,7 @@ func declareKeysComputeChecksum(
latchSpans *spanset.SpanSet,
_ *lockspanset.LockSpanSet,
_ time.Duration,
) {
) error {
// The correctness of range merges depends on the lease applied index of a
// range not being bumped while the RHS is subsumed. ComputeChecksum bumps a
// range's LAI and thus needs to be serialized with Subsume requests, in order
Expand All @@ -47,6 +47,7 @@ func declareKeysComputeChecksum(
// declare access over at least one key. We choose to declare read-only access
// over the range descriptor key.
latchSpans.AddNonMVCC(spanset.SpanReadOnly, roachpb.Span{Key: keys.RangeDescriptorKey(rs.GetStartKey())})
return nil
}

// ReplicaChecksumVersion versions the checksum computation. Requests silently no-op
Expand Down
6 changes: 3 additions & 3 deletions pkg/kv/kvserver/batcheval/cmd_conditional_put.go
Original file line number Diff line number Diff line change
Expand Up @@ -33,12 +33,12 @@ func declareKeysConditionalPut(
latchSpans *spanset.SpanSet,
lockSpans *lockspanset.LockSpanSet,
maxOffset time.Duration,
) {
) error {
args := req.(*kvpb.ConditionalPutRequest)
if args.Inline {
DefaultDeclareKeys(rs, header, req, latchSpans, lockSpans, maxOffset)
return DefaultDeclareKeys(rs, header, req, latchSpans, lockSpans, maxOffset)
} else {
DefaultDeclareIsolatedKeys(rs, header, req, latchSpans, lockSpans, maxOffset)
return DefaultDeclareIsolatedKeys(rs, header, req, latchSpans, lockSpans, maxOffset)
}
}

Expand Down
12 changes: 9 additions & 3 deletions pkg/kv/kvserver/batcheval/cmd_delete_range.go
Original file line number Diff line number Diff line change
Expand Up @@ -38,12 +38,17 @@ func declareKeysDeleteRange(
latchSpans *spanset.SpanSet,
lockSpans *lockspanset.LockSpanSet,
maxOffset time.Duration,
) {
) error {
args := req.(*kvpb.DeleteRangeRequest)
if args.Inline {
DefaultDeclareKeys(rs, header, req, latchSpans, lockSpans, maxOffset)
if err := DefaultDeclareKeys(rs, header, req, latchSpans, lockSpans, maxOffset); err != nil {
return err
}
} else {
DefaultDeclareIsolatedKeys(rs, header, req, latchSpans, lockSpans, maxOffset)
err := DefaultDeclareIsolatedKeys(rs, header, req, latchSpans, lockSpans, maxOffset)
if err != nil {
return err
}
}

// When writing range tombstones, we must look for adjacent range tombstones
Expand Down Expand Up @@ -77,6 +82,7 @@ func declareKeysDeleteRange(
})
}
}
return nil
}

const maxDeleteRangeBatchBytes = 32 << 20
Expand Down
4 changes: 3 additions & 1 deletion pkg/kv/kvserver/batcheval/cmd_delete_range_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -254,7 +254,9 @@ func TestDeleteRangeTombstone(t *testing.T) {
// bounds.
var latchSpans spanset.SpanSet
var lockSpans lockspanset.LockSpanSet
declareKeysDeleteRange(evalCtx.Desc, &h, req, &latchSpans, &lockSpans, 0)
require.NoError(t,
declareKeysDeleteRange(evalCtx.Desc, &h, req, &latchSpans, &lockSpans, 0),
)
batch := spanset.NewBatchAt(engine.NewBatch(), &latchSpans, h.Timestamp)
defer batch.Close()

Expand Down
10 changes: 7 additions & 3 deletions pkg/kv/kvserver/batcheval/cmd_end_transaction.go
Original file line number Diff line number Diff line change
Expand Up @@ -48,13 +48,14 @@ func init() {
// declareKeys{End,Heartbeat}Transaction.
func declareKeysWriteTransaction(
_ ImmutableRangeState, header *kvpb.Header, req kvpb.Request, latchSpans *spanset.SpanSet,
) {
) error {
if header.Txn != nil {
header.Txn.AssertInitialized(context.TODO())
latchSpans.AddNonMVCC(spanset.SpanReadWrite, roachpb.Span{
Key: keys.TransactionKey(req.Header().Key, header.Txn.ID),
})
}
return nil
}

func declareKeysEndTxn(
Expand All @@ -64,9 +65,11 @@ func declareKeysEndTxn(
latchSpans *spanset.SpanSet,
_ *lockspanset.LockSpanSet,
_ time.Duration,
) {
) error {
et := req.(*kvpb.EndTxnRequest)
declareKeysWriteTransaction(rs, header, req, latchSpans)
if err := declareKeysWriteTransaction(rs, header, req, latchSpans); err != nil {
return err
}
var minTxnTS hlc.Timestamp
if header.Txn != nil {
header.Txn.AssertInitialized(context.TODO())
Expand Down Expand Up @@ -208,6 +211,7 @@ func declareKeysEndTxn(
}
}
}
return nil
}

// EndTxn either commits or aborts (rolls back) an extant transaction according
Expand Down
8 changes: 6 additions & 2 deletions pkg/kv/kvserver/batcheval/cmd_export.go
Original file line number Diff line number Diff line change
Expand Up @@ -76,14 +76,18 @@ func declareKeysExport(
latchSpans *spanset.SpanSet,
lockSpans *lockspanset.LockSpanSet,
maxOffset time.Duration,
) {
DefaultDeclareIsolatedKeys(rs, header, req, latchSpans, lockSpans, maxOffset)
) error {
err := DefaultDeclareIsolatedKeys(rs, header, req, latchSpans, lockSpans, maxOffset)
if err != nil {
return err
}
latchSpans.AddNonMVCC(spanset.SpanReadOnly, roachpb.Span{Key: keys.RangeGCThresholdKey(header.RangeID)})
// Export requests will usually not hold latches during their evaluation.
//
// See call to `AssertAllowed()` in GetGCThreshold() to understand why we need
// to disable these assertions for export requests.
latchSpans.DisableUndeclaredAccessAssertions()
return nil
}

// evalExport dumps the requested keys into files of non-overlapping key ranges
Expand Down
3 changes: 2 additions & 1 deletion pkg/kv/kvserver/batcheval/cmd_gc.go
Original file line number Diff line number Diff line change
Expand Up @@ -38,7 +38,7 @@ func declareKeysGC(
latchSpans *spanset.SpanSet,
_ *lockspanset.LockSpanSet,
_ time.Duration,
) {
) error {
gcr := req.(*kvpb.GCRequest)
if gcr.RangeKeys != nil {
// When GC-ing MVCC range key tombstones, we need to serialize with
Expand Down Expand Up @@ -112,6 +112,7 @@ func declareKeysGC(
// Needed for updating optional GC hint.
latchSpans.AddNonMVCC(spanset.SpanReadWrite, roachpb.Span{Key: keys.RangeGCHintKey(rs.GetRangeID())})
latchSpans.DisableUndeclaredAccessAssertions()
return nil
}

// Create latches and merge adjacent.
Expand Down
4 changes: 2 additions & 2 deletions pkg/kv/kvserver/batcheval/cmd_heartbeat_txn.go
Original file line number Diff line number Diff line change
Expand Up @@ -36,8 +36,8 @@ func declareKeysHeartbeatTransaction(
latchSpans *spanset.SpanSet,
_ *lockspanset.LockSpanSet,
_ time.Duration,
) {
declareKeysWriteTransaction(rs, header, req, latchSpans)
) error {
return declareKeysWriteTransaction(rs, header, req, latchSpans)
}

// HeartbeatTxn updates the transaction status and heartbeat
Expand Down
3 changes: 2 additions & 1 deletion pkg/kv/kvserver/batcheval/cmd_lease_info.go
Original file line number Diff line number Diff line change
Expand Up @@ -34,8 +34,9 @@ func declareKeysLeaseInfo(
latchSpans *spanset.SpanSet,
_ *lockspanset.LockSpanSet,
_ time.Duration,
) {
) error {
latchSpans.AddNonMVCC(spanset.SpanReadOnly, roachpb.Span{Key: keys.RangeLeaseKey(rs.GetRangeID())})
return nil
}

// LeaseInfo returns information about the lease holder for the range.
Expand Down
3 changes: 2 additions & 1 deletion pkg/kv/kvserver/batcheval/cmd_lease_request.go
Original file line number Diff line number Diff line change
Expand Up @@ -35,12 +35,13 @@ func declareKeysRequestLease(
latchSpans *spanset.SpanSet,
_ *lockspanset.LockSpanSet,
_ time.Duration,
) {
) error {
// NOTE: RequestLease is run on replicas that do not hold the lease, so
// acquiring latches would not help synchronize with other requests. As
// such, the request does not declare latches. See also
// concurrency.shouldIgnoreLatches().
latchSpans.DisableUndeclaredAccessAssertions()
return nil
}

// RequestLease sets the range lease for this range. The command fails
Expand Down
3 changes: 2 additions & 1 deletion pkg/kv/kvserver/batcheval/cmd_lease_transfer.go
Original file line number Diff line number Diff line change
Expand Up @@ -35,7 +35,7 @@ func declareKeysTransferLease(
latchSpans *spanset.SpanSet,
_ *lockspanset.LockSpanSet,
_ time.Duration,
) {
) error {
// TransferLease must not run concurrently with any other request so it uses
// latches to synchronize with all other reads and writes on the outgoing
// leaseholder. Additionally, it observes the state of the timestamp cache
Expand All @@ -56,6 +56,7 @@ func declareKeysTransferLease(
// reads. We'd need to be careful here, so we should only pull on this if we
// decide that doing so is important.
declareAllKeys(latchSpans)
return nil
}

// TransferLease sets the lease holder for the range.
Expand Down
3 changes: 2 additions & 1 deletion pkg/kv/kvserver/batcheval/cmd_migrate.go
Original file line number Diff line number Diff line change
Expand Up @@ -36,7 +36,7 @@ func declareKeysMigrate(
latchSpans *spanset.SpanSet,
_ *lockspanset.LockSpanSet,
_ time.Duration,
) {
) error {
// TODO(irfansharif): This will eventually grow to capture the super set of
// all keys accessed by all migrations defined here. That could get
// cumbersome. We could spruce up the migration type and allow authors to
Expand All @@ -45,6 +45,7 @@ func declareKeysMigrate(

latchSpans.AddNonMVCC(spanset.SpanReadWrite, roachpb.Span{Key: keys.RangeVersionKey(rs.GetRangeID())})
latchSpans.AddNonMVCC(spanset.SpanReadOnly, roachpb.Span{Key: keys.RangeDescriptorKey(rs.GetStartKey())})
return nil
}

// migrationRegistry is a global registry of all KV-level migrations. See
Expand Down
3 changes: 2 additions & 1 deletion pkg/kv/kvserver/batcheval/cmd_probe.go
Original file line number Diff line number Diff line change
Expand Up @@ -29,13 +29,14 @@ func declareKeysProbe(
_ *spanset.SpanSet,
_ *lockspanset.LockSpanSet,
_ time.Duration,
) {
) error {
// Declare no keys. This means that we're not even serializing with splits
// (i.e. a probe could be directed at a key that will become the right-hand
// side of the split, and the split races ahead of the probe though the probe
// will still execute on the left-hand side). This is acceptable; we want the
// probe to bypass as much of the above-raft machinery as possible so that it
// gives us a signal on the replication layer alone.
return nil
}

func init() {
Expand Down
3 changes: 2 additions & 1 deletion pkg/kv/kvserver/batcheval/cmd_push_txn.go
Original file line number Diff line number Diff line change
Expand Up @@ -42,10 +42,11 @@ func declareKeysPushTransaction(
latchSpans *spanset.SpanSet,
_ *lockspanset.LockSpanSet,
_ time.Duration,
) {
) error {
pr := req.(*kvpb.PushTxnRequest)
latchSpans.AddNonMVCC(spanset.SpanReadWrite, roachpb.Span{Key: keys.TransactionKey(pr.PusheeTxn.Key, pr.PusheeTxn.ID)})
latchSpans.AddNonMVCC(spanset.SpanReadWrite, roachpb.Span{Key: keys.AbortSpanKey(rs.GetRangeID(), pr.PusheeTxn.ID)})
return nil
}

// PushTxn resolves conflicts between concurrent txns (or between
Expand Down
6 changes: 3 additions & 3 deletions pkg/kv/kvserver/batcheval/cmd_put.go
Original file line number Diff line number Diff line change
Expand Up @@ -33,12 +33,12 @@ func declareKeysPut(
latchSpans *spanset.SpanSet,
lockSpans *lockspanset.LockSpanSet,
maxOffset time.Duration,
) {
) error {
args := req.(*kvpb.PutRequest)
if args.Inline {
DefaultDeclareKeys(rs, header, req, latchSpans, lockSpans, maxOffset)
return DefaultDeclareKeys(rs, header, req, latchSpans, lockSpans, maxOffset)
} else {
DefaultDeclareIsolatedKeys(rs, header, req, latchSpans, lockSpans, maxOffset)
return DefaultDeclareIsolatedKeys(rs, header, req, latchSpans, lockSpans, maxOffset)
}
}

Expand Down
3 changes: 2 additions & 1 deletion pkg/kv/kvserver/batcheval/cmd_query_intent.go
Original file line number Diff line number Diff line change
Expand Up @@ -34,11 +34,12 @@ func declareKeysQueryIntent(
latchSpans *spanset.SpanSet,
_ *lockspanset.LockSpanSet,
_ time.Duration,
) {
) error {
// QueryIntent requests read the specified keys at the maximum timestamp in
// order to read any intent present, if one exists, regardless of the
// timestamp it was written at.
latchSpans.AddNonMVCC(spanset.SpanReadOnly, req.Header().Span())
return nil
}

// QueryIntent checks if an intent exists for the specified transaction at the
Expand Down
3 changes: 2 additions & 1 deletion pkg/kv/kvserver/batcheval/cmd_query_locks.go
Original file line number Diff line number Diff line change
Expand Up @@ -35,9 +35,10 @@ func declareKeysQueryLocks(
latchSpans *spanset.SpanSet,
_ *lockspanset.LockSpanSet,
_ time.Duration,
) {
) error {
// Latch on the range descriptor during evaluation of query locks.
latchSpans.AddNonMVCC(spanset.SpanReadOnly, roachpb.Span{Key: keys.RangeDescriptorKey(rs.GetStartKey())})
return nil
}

// QueryLocks uses the concurrency manager to query the state of locks
Expand Down
3 changes: 2 additions & 1 deletion pkg/kv/kvserver/batcheval/cmd_query_txn.go
Original file line number Diff line number Diff line change
Expand Up @@ -36,9 +36,10 @@ func declareKeysQueryTransaction(
latchSpans *spanset.SpanSet,
_ *lockspanset.LockSpanSet,
_ time.Duration,
) {
) error {
qr := req.(*kvpb.QueryTxnRequest)
latchSpans.AddNonMVCC(spanset.SpanReadOnly, roachpb.Span{Key: keys.TransactionKey(qr.Txn.Key, qr.Txn.ID)})
return nil
}

// QueryTxn fetches the current state of a transaction.
Expand Down
8 changes: 6 additions & 2 deletions pkg/kv/kvserver/batcheval/cmd_range_stats.go
Original file line number Diff line number Diff line change
Expand Up @@ -34,11 +34,15 @@ func declareKeysRangeStats(
latchSpans *spanset.SpanSet,
lockSpans *lockspanset.LockSpanSet,
maxOffset time.Duration,
) {
DefaultDeclareKeys(rs, header, req, latchSpans, lockSpans, maxOffset)
) error {
err := DefaultDeclareKeys(rs, header, req, latchSpans, lockSpans, maxOffset)
if err != nil {
return err
}
// The request will return the descriptor and lease.
latchSpans.AddNonMVCC(spanset.SpanReadOnly, roachpb.Span{Key: keys.RangeDescriptorKey(rs.GetStartKey())})
latchSpans.AddNonMVCC(spanset.SpanReadOnly, roachpb.Span{Key: keys.RangeLeaseKey(rs.GetRangeID())})
return nil
}

// RangeStats returns the MVCC statistics for a range.
Expand Down
Loading

0 comments on commit 79a213a

Please sign in to comment.