Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
110102: sql: do not generate index recs when they are not shown r=mgartner a=mgartner

While diagnosing a bug with index recommendations, I noticed that index
recommendations were being generated for `EXPLAIN` modes that never show
the recommendations: `OPT`, `DDL`, and `VEC`. This commit eliminates
that unnecessary computation.

Epic: None

Release note: None


110201: roachpb: add lock durability info to Get/Scan/ReverseScan requests r=nvanbenschoten a=arulajmani

This patch adds lock durability information to Get, Scan, and ReverseScan requests. This field is only ever meaningful in conjunction with a locking strength that's not lock.None.

By default, all locking requests ask for best-effort locks. This preserves the mixed version story between 23.1 <-> 23.2 nodes. However, transactions that need them for correctness (read: read committed), will now have the option to ask for guranteed durability locks. These will then correspond to replicated locks. Note that the field here is named in terms of durability guarantees, and not the specific implementation detail we'll use to provide this -- this is intentional. It allows us to offer a different kind of durable locks in the future, for example schemes that selectively replicate locks because they have buy-in from the both the kvserver and kvclient, in conjunction with some scheme to verify locks at commit time.

Informs #109672

Release note: None

110238: skip: add Unimplemented skip reason r=knz a=stevendanna

Some test cases are skipped because we intend to work on a feature in the future and have written the test to show the desired end-state once written.

Having a special method for this case allows us to semantically distinguish these kinds of skips from skips that are the results of bugs or unknown issues.

Epic: none

Release note: None

110389: concurrency: correctly defer to higher lock strength during dup access r=nvanbenschoten a=arulajmani

Burns down a TODO which only did so for lock.None lock strength. This was left over from a time when the lock table only expected lock.None and lock.Exclusive lock strengths.

When a key is being accessed with more than one lock strengths in a batch, we want to defer conflict resolution to the higher lock strength, as the lower lock strength access has sufficient protection from it. This means that if a lock table scan is resumed somewhere in the middle, and we've already checked for conflicts at the key before, we can simply ignore it and move on. Note that it doesn't matter if there is a conflict at the key -- we won't proceed to evaluation before coming back to this key, while holding latches and restarting our scan from the begining, so it'll be caught then.

Epic: none

Release note: None

110391: ui: use MAX downsampler for sql connection rate r=sean- a=dhartunian

Epic: None

Release note (ui change): The "SQL Connection Rate" metric on the SQL Dashboard is downsampled using the MAX function instead of SUM. This improves situations where zooming out would cause the connection rate to increase for downsampled data.

110429: ui: fix generated `DROP INDEX` statement on table details page r=zachlite a=zachlite

This commit does two things:
1. Fixes the generated `DROP INDEX` statement by considering already quoted fully-qualified names.

2. Adds unit tests for the `QuoteIdentifier` utility to clarify its intended usage and limitations.

Epic: none
Release note (bug fix): Fixed a UI issue where the DROP_UNUSED index recommendations produced by the
table details page produced an invalid `DROP INDEX` statement.

Co-authored-by: Marcus Gartner <marcus@cockroachlabs.com>
Co-authored-by: Arul Ajmani <arulajmani@gmail.com>
Co-authored-by: Steven Danna <danna@cockroachlabs.com>
Co-authored-by: David Hartunian <davidh@cockroachlabs.com>
Co-authored-by: Zach Lite <zach@cockroachlabs.com>
  • Loading branch information
6 people committed Sep 12, 2023
7 parents b99bcd9 + 275ed3c + 65d846e + b5f8685 + 8a1c2b8 + a508484 + b33f9f3 commit 7463887
Show file tree
Hide file tree
Showing 32 changed files with 337 additions and 191 deletions.
4 changes: 2 additions & 2 deletions pkg/ccl/kvccl/kvfollowerreadsccl/followerreads_test.go
Expand Up @@ -208,7 +208,7 @@ func TestCanSendToFollower(t *testing.T) {
},
{
name: "stale locking read",
ba: batch(txn(stale), &kvpb.GetRequest{KeyLocking: lock.Exclusive}),
ba: batch(txn(stale), &kvpb.GetRequest{KeyLockingStrength: lock.Exclusive}),
exp: false,
},
{
Expand Down Expand Up @@ -345,7 +345,7 @@ func TestCanSendToFollower(t *testing.T) {
},
{
name: "stale locking read, global reads policy",
ba: batch(txn(stale), &kvpb.GetRequest{KeyLocking: lock.Exclusive}),
ba: batch(txn(stale), &kvpb.GetRequest{KeyLockingStrength: lock.Exclusive}),
ctPolicy: roachpb.LEAD_FOR_GLOBAL_READS,
exp: false,
},
Expand Down
4 changes: 2 additions & 2 deletions pkg/kv/kvclient/kvcoord/txn_interceptor_heartbeater_test.go
Expand Up @@ -176,7 +176,7 @@ func TestTxnHeartbeaterLoopStartedOnFirstLock(t *testing.T) {
if write {
ba.Add(&kvpb.PutRequest{RequestHeader: keyAHeader})
} else {
ba.Add(&kvpb.ScanRequest{RequestHeader: keyAHeader, KeyLocking: lock.Exclusive})
ba.Add(&kvpb.ScanRequest{RequestHeader: keyAHeader, KeyLockingStrength: lock.Exclusive})
}

br, pErr = th.SendLocked(ctx, ba)
Expand Down Expand Up @@ -344,7 +344,7 @@ func TestTxnHeartbeaterLoopStartsBeforeExpiry(t *testing.T) {
ba.Header = kvpb.Header{Txn: txn.Clone()}
keyA := roachpb.Key("a")
keyAHeader := kvpb.RequestHeader{Key: keyA}
ba.Add(&kvpb.GetRequest{RequestHeader: keyAHeader, KeyLocking: lock.Exclusive})
ba.Add(&kvpb.GetRequest{RequestHeader: keyAHeader, KeyLockingStrength: lock.Exclusive})

br, pErr := th.SendLocked(ctx, ba)
require.Nil(t, pErr)
Expand Down
12 changes: 6 additions & 6 deletions pkg/kv/kvclient/kvcoord/txn_interceptor_pipeliner_test.go
Expand Up @@ -122,8 +122,8 @@ func TestTxnPipeliner1PCTransaction(t *testing.T) {
ba := &kvpb.BatchRequest{}
ba.Header = kvpb.Header{Txn: &txn}
scanArgs := kvpb.ScanRequest{
RequestHeader: kvpb.RequestHeader{Key: keyA, EndKey: keyB},
KeyLocking: lock.Exclusive,
RequestHeader: kvpb.RequestHeader{Key: keyA, EndKey: keyB},
KeyLockingStrength: lock.Exclusive,
}
ba.Add(&scanArgs)
putArgs := kvpb.PutRequest{RequestHeader: kvpb.RequestHeader{Key: keyA}}
Expand Down Expand Up @@ -1353,7 +1353,7 @@ func TestTxnPipelinerRecordsLocksOnFailure(t *testing.T) {
ba.Header = kvpb.Header{Txn: &txn}
ba.Add(&kvpb.PutRequest{RequestHeader: kvpb.RequestHeader{Key: keyA}})
ba.Add(&kvpb.DeleteRangeRequest{RequestHeader: kvpb.RequestHeader{Key: keyB, EndKey: keyB.Next()}})
ba.Add(&kvpb.ScanRequest{RequestHeader: kvpb.RequestHeader{Key: keyC, EndKey: keyC.Next()}, KeyLocking: lock.Exclusive})
ba.Add(&kvpb.ScanRequest{RequestHeader: kvpb.RequestHeader{Key: keyC, EndKey: keyC.Next()}, KeyLockingStrength: lock.Exclusive})

mockPErr := kvpb.NewErrorf("boom")
mockSender.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) {
Expand Down Expand Up @@ -1382,7 +1382,7 @@ func TestTxnPipelinerRecordsLocksOnFailure(t *testing.T) {
ba.Requests = nil
ba.Add(&kvpb.PutRequest{RequestHeader: kvpb.RequestHeader{Key: keyD}})
ba.Add(&kvpb.DeleteRangeRequest{RequestHeader: kvpb.RequestHeader{Key: keyE, EndKey: keyE.Next()}})
ba.Add(&kvpb.ScanRequest{RequestHeader: kvpb.RequestHeader{Key: keyF, EndKey: keyF.Next()}, KeyLocking: lock.Exclusive})
ba.Add(&kvpb.ScanRequest{RequestHeader: kvpb.RequestHeader{Key: keyF, EndKey: keyF.Next()}, KeyLockingStrength: lock.Exclusive})

mockSender.MockSend(func(ba *kvpb.BatchRequest) (*kvpb.BatchResponse, *kvpb.Error) {
require.Len(t, ba.Requests, 3)
Expand Down Expand Up @@ -1450,7 +1450,7 @@ func TestTxnPipelinerIgnoresLocksOnUnambiguousFailure(t *testing.T) {
ba.Header = kvpb.Header{Txn: &txn}
ba.Add(&kvpb.ConditionalPutRequest{RequestHeader: kvpb.RequestHeader{Key: keyA}})
ba.Add(&kvpb.DeleteRangeRequest{RequestHeader: kvpb.RequestHeader{Key: keyB, EndKey: keyB.Next()}})
ba.Add(&kvpb.ScanRequest{RequestHeader: kvpb.RequestHeader{Key: keyC, EndKey: keyC.Next()}, KeyLocking: lock.Exclusive})
ba.Add(&kvpb.ScanRequest{RequestHeader: kvpb.RequestHeader{Key: keyC, EndKey: keyC.Next()}, KeyLockingStrength: lock.Exclusive})

condFailedErr := kvpb.NewError(&kvpb.ConditionFailedError{})
condFailedErr.SetErrorIndex(0)
Expand Down Expand Up @@ -1480,7 +1480,7 @@ func TestTxnPipelinerIgnoresLocksOnUnambiguousFailure(t *testing.T) {
ba.Requests = nil
ba.Add(&kvpb.ConditionalPutRequest{RequestHeader: kvpb.RequestHeader{Key: keyD}})
ba.Add(&kvpb.DeleteRangeRequest{RequestHeader: kvpb.RequestHeader{Key: keyE, EndKey: keyE.Next()}})
ba.Add(&kvpb.ScanRequest{RequestHeader: kvpb.RequestHeader{Key: keyF, EndKey: keyF.Next()}, KeyLocking: lock.Exclusive})
ba.Add(&kvpb.ScanRequest{RequestHeader: kvpb.RequestHeader{Key: keyF, EndKey: keyF.Next()}, KeyLockingStrength: lock.Exclusive})

lockConflictErr := kvpb.NewError(&kvpb.LockConflictError{})
lockConflictErr.SetErrorIndex(2)
Expand Down
45 changes: 33 additions & 12 deletions pkg/kv/kvclient/kvstreamer/size.go
Expand Up @@ -11,6 +11,7 @@
package kvstreamer

import (
"fmt"
"unsafe"

"github.com/cockroachdb/cockroach/pkg/kv/kvpb"
Expand All @@ -24,8 +25,10 @@ const (
int32Size = int64(unsafe.Sizeof(int32(0)))
requestUnionSliceOverhead = int64(unsafe.Sizeof([]kvpb.RequestUnion{}))
requestUnionOverhead = int64(unsafe.Sizeof(kvpb.RequestUnion{}))
requestOverhead = int64(unsafe.Sizeof(kvpb.RequestUnion_Get{}) +
getRequestOverhead = int64(unsafe.Sizeof(kvpb.RequestUnion_Get{}) +
unsafe.Sizeof(kvpb.GetRequest{}))
scanRequestOverhead = int64(unsafe.Sizeof(kvpb.RequestUnion_Scan{}) +
unsafe.Sizeof(kvpb.ScanRequest{}))
responseUnionOverhead = int64(unsafe.Sizeof(kvpb.ResponseUnion_Get{}))
getResponseOverhead = int64(unsafe.Sizeof(kvpb.GetResponse{}))
scanResponseOverhead = int64(unsafe.Sizeof(kvpb.ScanResponse{}))
Expand All @@ -34,10 +37,13 @@ const (
var zeroInt32Slice []int32

func init() {
scanRequestOverhead := int64(unsafe.Sizeof(kvpb.RequestUnion_Scan{}) +
unsafe.Sizeof(kvpb.ScanRequest{}))
if requestOverhead != scanRequestOverhead {
panic("GetRequest and ScanRequest have different overheads")
reverseScanRequestOverhead := int64(unsafe.Sizeof(kvpb.RequestUnion_ReverseScan{}) +
unsafe.Sizeof(kvpb.ReverseScanRequest{}))
if reverseScanRequestOverhead != scanRequestOverhead {
panic(fmt.Sprintf(
"ReverseScanRequest and ScanRequest have different overheads %d and scan req %d",
reverseScanRequestOverhead, scanRequestOverhead,
))
}
scanResponseUnionOverhead := int64(unsafe.Sizeof(kvpb.ResponseUnion_Scan{}))
if responseUnionOverhead != scanResponseUnionOverhead {
Expand All @@ -53,17 +59,32 @@ func init() {
// - they account for things differently from how the memory usage is accounted
// for by the KV layer for the purposes of tracking TargetBytes limit.

// requestSize calculates the footprint of a request including the overhead. key
// and endKey are the keys from the span of the request header (we choose to
// avoid taking in a roachpb.Span in order to reduce allocations).
func requestSize(key, endKey roachpb.Key) int64 {
return requestOverhead + int64(cap(key)) + int64(cap(endKey))
// scanRequestSize calculates the footprint of a {,Reverse}Scan request,
// including the overhead. key and endKey are the keys from the span of the
// request header (we choose to avoid taking in a roachpb.Span in order to
// reduce allocations).
func scanRequestSize(key, endKey roachpb.Key) int64 {
return scanRequestOverhead + int64(cap(key)) + int64(cap(endKey))
}

// getRequestSize calculates the footprint of a Get request for a given key,
// including its overhead.
func getRequestSize(key roachpb.Key) int64 {
return getRequestOverhead + int64(cap(key))
}

func requestsMemUsage(reqs []kvpb.RequestUnion) (memUsage int64) {
for _, r := range reqs {
h := r.GetInner().Header()
memUsage += requestSize(h.Key, h.EndKey)
req := r.GetInner()
h := req.Header()
switch req.Method() {
case kvpb.Get:
memUsage += getRequestSize(h.Key)
case kvpb.Scan, kvpb.ReverseScan:
memUsage += scanRequestSize(h.Key, h.EndKey)
default:
panic(fmt.Sprintf("unexpected request type %s", r.GetInner()))
}
}
return memUsage
}
Expand Down
8 changes: 4 additions & 4 deletions pkg/kv/kvclient/kvstreamer/streamer.go
Expand Up @@ -1466,7 +1466,7 @@ func calculateFootprint(
}
if get.ResumeSpan != nil {
// This Get wasn't completed.
fp.resumeReqsMemUsage += requestSize(get.ResumeSpan.Key, get.ResumeSpan.EndKey)
fp.resumeReqsMemUsage += getRequestSize(get.ResumeSpan.Key)
fp.numIncompleteGets++
} else {
// This Get was completed.
Expand Down Expand Up @@ -1501,7 +1501,7 @@ func calculateFootprint(
}
if scan.ResumeSpan != nil {
// This Scan wasn't completed.
fp.resumeReqsMemUsage += requestSize(scan.ResumeSpan.Key, scan.ResumeSpan.EndKey)
fp.resumeReqsMemUsage += scanRequestSize(scan.ResumeSpan.Key, scan.ResumeSpan.EndKey)
fp.numIncompleteScans++
}
}
Expand Down Expand Up @@ -1744,7 +1744,7 @@ func buildResumeSingleRangeBatch(
newGet := gets[0]
gets = gets[1:]
newGet.req.SetSpan(*get.ResumeSpan)
newGet.req.KeyLocking = s.keyLocking
newGet.req.KeyLockingStrength = s.keyLocking
newGet.union.Get = &newGet.req
resumeReq.reqs[resumeReqIdx].Value = &newGet.union
resumeReq.positions = append(resumeReq.positions, position)
Expand Down Expand Up @@ -1772,7 +1772,7 @@ func buildResumeSingleRangeBatch(
scans = scans[1:]
newScan.req.SetSpan(*scan.ResumeSpan)
newScan.req.ScanFormat = kvpb.BATCH_RESPONSE
newScan.req.KeyLocking = s.keyLocking
newScan.req.KeyLockingStrength = s.keyLocking
newScan.union.Scan = &newScan.req
resumeReq.reqs[resumeReqIdx].Value = &newScan.union
resumeReq.positions = append(resumeReq.positions, position)
Expand Down
32 changes: 16 additions & 16 deletions pkg/kv/kvpb/api.go
Expand Up @@ -198,28 +198,28 @@ type Request interface {
// strength of a read-only request.
type LockingReadRequest interface {
Request
KeyLockingStrength() lock.Strength
KeyLocking() (lock.Strength, lock.Durability)
}

var _ LockingReadRequest = (*GetRequest)(nil)

// KeyLockingStrength implements the LockingReadRequest interface.
func (gr *GetRequest) KeyLockingStrength() lock.Strength {
return gr.KeyLocking
// KeyLocking implements the LockingReadRequest interface.
func (gr *GetRequest) KeyLocking() (lock.Strength, lock.Durability) {
return gr.KeyLockingStrength, gr.KeyLockingDurability
}

var _ LockingReadRequest = (*ScanRequest)(nil)

// KeyLockingStrength implements the LockingReadRequest interface.
func (sr *ScanRequest) KeyLockingStrength() lock.Strength {
return sr.KeyLocking
// KeyLocking implements the LockingReadRequest interface.
func (sr *ScanRequest) KeyLocking() (lock.Strength, lock.Durability) {
return sr.KeyLockingStrength, sr.KeyLockingDurability
}

var _ LockingReadRequest = (*ReverseScanRequest)(nil)

// KeyLockingStrength implements the LockingReadRequest interface.
func (rsr *ReverseScanRequest) KeyLockingStrength() lock.Strength {
return rsr.KeyLocking
// KeyLocking implements the LockingReadRequest interface.
func (rsr *ReverseScanRequest) KeyLocking() (lock.Strength, lock.Durability) {
return rsr.KeyLockingStrength, rsr.KeyLockingDurability
}

// SizedWriteRequest is an interface used to expose the number of bytes a
Expand Down Expand Up @@ -1197,7 +1197,7 @@ func NewGet(key roachpb.Key, forUpdate bool) Request {
RequestHeader: RequestHeader{
Key: key,
},
KeyLocking: scanLockStrength(forUpdate),
KeyLockingStrength: scanLockStrength(forUpdate),
}
}

Expand Down Expand Up @@ -1323,7 +1323,7 @@ func NewScan(key, endKey roachpb.Key, forUpdate bool) Request {
Key: key,
EndKey: endKey,
},
KeyLocking: scanLockStrength(forUpdate),
KeyLockingStrength: scanLockStrength(forUpdate),
}
}

Expand All @@ -1336,7 +1336,7 @@ func NewReverseScan(key, endKey roachpb.Key, forUpdate bool) Request {
Key: key,
EndKey: endKey,
},
KeyLocking: scanLockStrength(forUpdate),
KeyLockingStrength: scanLockStrength(forUpdate),
}
}

Expand All @@ -1355,7 +1355,7 @@ func flagForLockStrength(l lock.Strength) flag {
}

func (gr *GetRequest) flags() flag {
maybeLocking := flagForLockStrength(gr.KeyLocking)
maybeLocking := flagForLockStrength(gr.KeyLockingStrength)
return isRead | isTxn | maybeLocking | updatesTSCache | needsRefresh | canSkipLocked
}

Expand Down Expand Up @@ -1445,12 +1445,12 @@ func (*RevertRangeRequest) flags() flag {
}

func (sr *ScanRequest) flags() flag {
maybeLocking := flagForLockStrength(sr.KeyLocking)
maybeLocking := flagForLockStrength(sr.KeyLockingStrength)
return isRead | isRange | isTxn | maybeLocking | updatesTSCache | needsRefresh | canSkipLocked
}

func (rsr *ReverseScanRequest) flags() flag {
maybeLocking := flagForLockStrength(rsr.KeyLocking)
maybeLocking := flagForLockStrength(rsr.KeyLockingStrength)
return isRead | isRange | isReverse | isTxn | maybeLocking | updatesTSCache | needsRefresh | canSkipLocked
}

Expand Down

0 comments on commit 7463887

Please sign in to comment.