Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
74341: sql/catalog: restore fast-path in FullIndexColumnIDs r=ajwerner a=nvanbenschoten

This commit restores a [fast-path](c9e116e#diff-19625608f4a6e23e6fe0818f3a621e716615765cb338d18fe34b43f0a535f06dL140)
in `FullIndexColumnIDs` which was lost in c9e116e. The fast-path avoided
the allocation of a `ColumnID` slice and a `IndexDescriptor_Direction`
slice in `FullIndexColumnIDs` when given a unique index. In such cases,
these slices are already stored on the `IndexDescriptor`.

```
name                   old time/op    new time/op    delta
KV/Scan/SQL/rows=1-10    94.9µs ±10%    94.9µs ± 8%    ~     (p=0.739 n=10+10)

name                   old alloc/op   new alloc/op   delta
KV/Scan/SQL/rows=1-10    20.1kB ± 0%    20.1kB ± 1%    ~     (p=0.424 n=10+10)

name                   old allocs/op  new allocs/op  delta
KV/Scan/SQL/rows=1-10       245 ± 0%       241 ± 0%  -1.63%  (p=0.000 n=10+8)
```
----

This is part of a collection of assorted micro-optimizations:
- #74336
- #74337
- #74338
- #74339
- #74340
- #74341
- #74342
- #74343
- #74344
- #74345
- #74346
- #74347
- #74348

Combined, these changes have the following effect on end-to-end SQL query performance:
```
name                      old time/op    new time/op    delta
KV/Scan/SQL/rows=1-10       94.4µs ±10%    92.3µs ±11%   -2.20%  (p=0.000 n=93+93)
KV/Scan/SQL/rows=10-10       102µs ±10%      99µs ±10%   -2.16%  (p=0.000 n=94+94)
KV/Update/SQL/rows=10-10     378µs ±15%     370µs ±11%   -2.04%  (p=0.003 n=95+91)
KV/Insert/SQL/rows=1-10      133µs ±14%     132µs ±12%     ~     (p=0.738 n=95+93)
KV/Insert/SQL/rows=10-10     197µs ±14%     196µs ±13%     ~     (p=0.902 n=95+94)
KV/Update/SQL/rows=1-10      186µs ±14%     185µs ±14%     ~     (p=0.351 n=94+93)
KV/Delete/SQL/rows=1-10      132µs ±13%     132µs ±14%     ~     (p=0.473 n=94+94)
KV/Delete/SQL/rows=10-10     254µs ±16%     250µs ±16%     ~     (p=0.086 n=100+99)

name                      old alloc/op   new alloc/op   delta
KV/Scan/SQL/rows=1-10       20.1kB ± 0%    19.1kB ± 1%   -4.91%  (p=0.000 n=96+96)
KV/Scan/SQL/rows=10-10      21.7kB ± 0%    20.7kB ± 1%   -4.61%  (p=0.000 n=96+97)
KV/Delete/SQL/rows=10-10    64.0kB ± 3%    63.7kB ± 3%   -0.55%  (p=0.000 n=100+100)
KV/Update/SQL/rows=1-10     45.8kB ± 1%    45.5kB ± 1%   -0.55%  (p=0.000 n=97+98)
KV/Update/SQL/rows=10-10     105kB ± 1%     105kB ± 1%   -0.10%  (p=0.008 n=97+98)
KV/Delete/SQL/rows=1-10     40.8kB ± 0%    40.7kB ± 0%   -0.08%  (p=0.001 n=95+96)
KV/Insert/SQL/rows=1-10     37.4kB ± 1%    37.4kB ± 0%     ~     (p=0.698 n=97+96)
KV/Insert/SQL/rows=10-10    76.4kB ± 1%    76.4kB ± 0%     ~     (p=0.822 n=99+98)

name                      old allocs/op  new allocs/op  delta
KV/Scan/SQL/rows=1-10          245 ± 0%       217 ± 0%  -11.43%  (p=0.000 n=95+92)
KV/Scan/SQL/rows=10-10         280 ± 0%       252 ± 0%  -10.11%  (p=0.000 n=75+97)
KV/Delete/SQL/rows=10-10       478 ± 0%       459 ± 0%   -4.04%  (p=0.000 n=94+97)
KV/Delete/SQL/rows=1-10        297 ± 1%       287 ± 1%   -3.34%  (p=0.000 n=97+97)
KV/Update/SQL/rows=1-10        459 ± 0%       444 ± 0%   -3.27%  (p=0.000 n=97+97)
KV/Insert/SQL/rows=1-10        291 ± 0%       286 ± 0%   -1.72%  (p=0.000 n=82+86)
KV/Update/SQL/rows=10-10       763 ± 1%       750 ± 1%   -1.68%  (p=0.000 n=96+98)
KV/Insert/SQL/rows=10-10       489 ± 0%       484 ± 0%   -1.03%  (p=0.000 n=98+98)
```


74355: kv: protect Replica's lastToReplica and lastFromReplica fields with raftMu r=nvanbenschoten a=nvanbenschoten

This commit moves the Replica's lastToReplica and lastFromReplica from
under the `Replica.mu` mutex to the `Replica.raftMu` mutex. These are
strictly Raft-specific pieces of state, so we don't need fine-grained
locking around them. As a reward, we don't need to grab the `Replica.mu`
exclusively (or at all) when setting the fields in
`Store.withReplicaForRequest`.

The locking in `setLastReplicaDescriptors` showed up in a mutex profile
under a write-heavy workload. It was responsible for **3.44%** of mutex
wait time. Grabbing the mutex was probably also slowing down request
processing, as the exclusive lock acquisition had to wait for read locks
to be dropped.

<img width="1584" alt="Screen Shot 2021-12-30 at 9 45 08 PM" src="https://user-images.githubusercontent.com/5438456/147800455-8da74dfd-5fd3-4831-818c-7e3c65763435.png">

74592: coldata: operate on Nulls value, not reference r=yuzefovich a=nvanbenschoten

This commit changes `col.Vec.SetNulls` to accept a `Nulls` struct by value instead of by pointer. This lets us avoid a heap allocation on each call to `Nulls.Or`.

We saw this in the "after" heap profiles in #74590, which looked like:

<img width="1749" alt="Screen Shot 2022-01-07 at 7 17 32 PM" src="https://user-images.githubusercontent.com/5438456/148624263-777a6d93-4df7-40da-84a3-18d5e47ab633.png">

```
      File: cockroach
Type: alloc_objects
Time: Jan 8, 2022 at 12:17am (UTC)
Showing nodes accounting for 5943494, 100% of 5943494 total
----------------------------------------------------------+-------------
      flat  flat%   sum%        cum   cum%   calls calls% + context 	 	 
----------------------------------------------------------+-------------
                                            843873 48.47% |   github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecproj.projPlusDecimalDecimalOp.Next.func1 /go/src/github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecproj/proj_non_const_ops.eg.go:3938
                                            823389 47.29% |   github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecproj.projPlusInt64Int64Op.Next.func1 /go/src/github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecproj/proj_non_const_ops.eg.go:5732
                                             73736  4.24% |   github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecproj.projPlusInt64DecimalOp.Next.func1 /go/src/github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecproj/proj_non_const_ops.eg.go:5870
   1740998 29.29% 29.29%    1740998 29.29%                | github.com/cockroachdb/cockroach/pkg/col/coldata.(*Nulls).Or /go/src/github.com/cockroachdb/cockroach/pkg/col/coldata/nulls.go:350
----------------------------------------------------------+-------------
                                            819219 49.50% |   github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecproj.projPlusInt64Int64Op.Next.func1 /go/src/github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecproj/proj_non_const_ops.eg.go:5732
                                            704530 42.57% |   github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecproj.projPlusDecimalDecimalOp.Next.func1 /go/src/github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecproj/proj_non_const_ops.eg.go:3938
                                            131076  7.92% |   github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecproj.projPlusInt64DecimalOp.Next.func1 /go/src/github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecproj/proj_non_const_ops.eg.go:5870
   1654825 27.84% 57.14%    1654825 27.84%                | github.com/cockroachdb/cockroach/pkg/col/coldata.(*Nulls).Or /go/src/github.com/cockroachdb/cockroach/pkg/col/coldata/nulls.go:348
----------------------------------------------------------+-------------
```

This PR eliminates one of these two heap allocations.

Co-authored-by: Nathan VanBenschoten <[email protected]>
  • Loading branch information
craig[bot] and nvanbenschoten committed Jan 10, 2022
4 parents 4937c04 + bb1bce2 + 410ef29 + 8c56129 commit 62392dc
Show file tree
Hide file tree
Showing 18 changed files with 1,338 additions and 1,341 deletions.
4 changes: 2 additions & 2 deletions pkg/col/coldata/nulls.go
Original file line number Diff line number Diff line change
Expand Up @@ -340,12 +340,12 @@ func (n *Nulls) SetNullBitmap(bm []byte, size int) {

// Or returns a new Nulls vector where NullAt(i) iff n1.NullAt(i) or
// n2.NullAt(i).
func (n *Nulls) Or(n2 *Nulls) *Nulls {
func (n Nulls) Or(n2 Nulls) Nulls {
// For simplicity, enforce that len(n.nulls) <= len(n2.nulls).
if len(n.nulls) > len(n2.nulls) {
n, n2 = n2, n
}
res := &Nulls{
res := Nulls{
maybeHasNulls: n.maybeHasNulls || n2.maybeHasNulls,
nulls: make([]byte, len(n2.nulls)),
}
Expand Down
8 changes: 4 additions & 4 deletions pkg/col/coldata/nulls_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -161,18 +161,18 @@ func TestNullsSet(t *testing.T) {
}
for _, withSel := range []bool{false, true} {
t.Run(fmt.Sprintf("WithSel=%t", withSel), func(t *testing.T) {
var srcNulls *Nulls
var srcNulls Nulls
if withSel {
args.Sel = make([]int, BatchSize())
// Make a selection vector with every even index. (This turns nulls10 into
// nulls5.)
for i := range args.Sel {
args.Sel[i] = i * 2
}
srcNulls = &nulls10
srcNulls = nulls10
} else {
args.Sel = nil
srcNulls = &nulls5
srcNulls = nulls5
}
for _, destStartIdx := range pos {
for _, srcStartIdx := range pos {
Expand Down Expand Up @@ -241,7 +241,7 @@ func TestNullsOr(t *testing.T) {
n1Choice, n2Choice := rng.Intn(len(nullsToChooseFrom)), rng.Intn(len(nullsToChooseFrom))
n1 := nullsToChooseFrom[n1Choice].Slice(0, length1)
n2 := nullsToChooseFrom[n2Choice].Slice(0, length2)
or := n1.Or(&n2)
or := n1.Or(n2)
require.Equal(t, or.maybeHasNulls, n1.maybeHasNulls || n2.maybeHasNulls)
maxLength := length1
if length2 > length1 {
Expand Down
6 changes: 3 additions & 3 deletions pkg/col/coldata/vec.go
Original file line number Diff line number Diff line change
Expand Up @@ -128,7 +128,7 @@ type Vec interface {
Nulls() *Nulls

// SetNulls sets the nulls vector for this column.
SetNulls(*Nulls)
SetNulls(Nulls)

// Length returns the length of the slice that is underlying this Vec.
Length() int
Expand Down Expand Up @@ -294,8 +294,8 @@ func (m *memColumn) Nulls() *Nulls {
return &m.nulls
}

func (m *memColumn) SetNulls(n *Nulls) {
m.nulls = *n
func (m *memColumn) SetNulls(n Nulls) {
m.nulls = n
}

func (m *memColumn) Length() int {
Expand Down
94 changes: 46 additions & 48 deletions pkg/kv/kvserver/replica.go
Original file line number Diff line number Diff line change
Expand Up @@ -279,6 +279,47 @@ type Replica struct {
stateMachine replicaStateMachine
// decoder is used to decode committed raft entries.
decoder replicaDecoder

// The last seen replica descriptors from incoming Raft messages. These are
// stored so that the replica still knows the replica descriptors for itself
// and for its message recipients in the circumstances when its RangeDescriptor
// is out of date.
//
// Normally, a replica knows about the other replica descriptors for a
// range via the RangeDescriptor stored in Replica.mu.state.Desc. But that
// descriptor is only updated during a Split or ChangeReplicas operation.
// There are periods during a Replica's lifetime when that information is
// out of date:
//
// 1. When a replica is being newly created as the result of an incoming
// Raft message for it. This is the common case for ChangeReplicas and an
// uncommon case for Splits. The leader will be sending the replica
// messages and the replica needs to be able to respond before it can
// receive an updated range descriptor (via a snapshot,
// changeReplicasTrigger, or splitTrigger).
//
// 2. If the node containing a replica is partitioned or down while the
// replicas for the range are updated. When the node comes back up, other
// replicas may begin communicating with it and it needs to be able to
// respond. Unlike 1 where there is no range descriptor, in this situation
// the replica has a range descriptor but it is out of date. Note that a
// replica being removed from a node and then quickly re-added before the
// replica has been GC'd will also use the last seen descriptors. In
// effect, this is another path for which the replica's local range
// descriptor is out of date.
//
// The last seen replica descriptors are updated on receipt of every raft
// message via Replica.setLastReplicaDescriptors (see
// Store.HandleRaftRequest). These last seen descriptors are used when
// the replica's RangeDescriptor contains missing or out of date descriptors
// for a replica (see Replica.sendRaftMessageRaftMuLocked).
//
// Removing a replica from Store.mu.replicas is not a problem because
// when a replica is completely removed, it won't be recreated until
// there is another event that will repopulate the replicas map in the
// range descriptor. When it is temporarily dropped and recreated, the
// newly recreated replica will have a complete range descriptor.
lastToReplica, lastFromReplica roachpb.ReplicaDescriptor
}

// Contains the lease history when enabled.
Expand Down Expand Up @@ -499,47 +540,6 @@ type Replica struct {
// live node will not lose leaseholdership.
lastUpdateTimes lastUpdateTimesMap

// The last seen replica descriptors from incoming Raft messages. These are
// stored so that the replica still knows the replica descriptors for itself
// and for its message recipients in the circumstances when its RangeDescriptor
// is out of date.
//
// Normally, a replica knows about the other replica descriptors for a
// range via the RangeDescriptor stored in Replica.mu.state.Desc. But that
// descriptor is only updated during a Split or ChangeReplicas operation.
// There are periods during a Replica's lifetime when that information is
// out of date:
//
// 1. When a replica is being newly created as the result of an incoming
// Raft message for it. This is the common case for ChangeReplicas and an
// uncommon case for Splits. The leader will be sending the replica
// messages and the replica needs to be able to respond before it can
// receive an updated range descriptor (via a snapshot,
// changeReplicasTrigger, or splitTrigger).
//
// 2. If the node containing a replica is partitioned or down while the
// replicas for the range are updated. When the node comes back up, other
// replicas may begin communicating with it and it needs to be able to
// respond. Unlike 1 where there is no range descriptor, in this situation
// the replica has a range descriptor but it is out of date. Note that a
// replica being removed from a node and then quickly re-added before the
// replica has been GC'd will also use the last seen descriptors. In
// effect, this is another path for which the replica's local range
// descriptor is out of date.
//
// The last seen replica descriptors are updated on receipt of every raft
// message via Replica.setLastReplicaDescriptors (see
// Store.HandleRaftRequest). These last seen descriptors are used when
// the replica's RangeDescriptor contains missing or out of date descriptors
// for a replica (see Replica.sendRaftMessage).
//
// Removing a replica from Store.mu.replicas is not a problem because
// when a replica is completely removed, it won't be recreated until
// there is another event that will repopulate the replicas map in the
// range descriptor. When it is temporarily dropped and recreated, the
// newly recreated replica will have a complete range descriptor.
lastToReplica, lastFromReplica roachpb.ReplicaDescriptor

// Computed checksum at a snapshot UUID.
checksums map[uuid.UUID]ReplicaChecksum

Expand Down Expand Up @@ -1063,13 +1063,11 @@ func (r *Replica) mergeInProgressRLocked() bool {
}

// setLastReplicaDescriptors sets the most recently seen replica
// descriptors to those contained in the *RaftMessageRequest, acquiring r.mu
// to do so.
func (r *Replica) setLastReplicaDescriptors(req *RaftMessageRequest) {
r.mu.Lock()
r.mu.lastFromReplica = req.FromReplica
r.mu.lastToReplica = req.ToReplica
r.mu.Unlock()
// descriptors to those contained in the *RaftMessageRequest.
func (r *Replica) setLastReplicaDescriptorsRaftMuLocked(req *RaftMessageRequest) {
r.raftMu.AssertHeld()
r.raftMu.lastFromReplica = req.FromReplica
r.raftMu.lastToReplica = req.ToReplica
}

// GetMVCCStats returns a copy of the MVCC stats object for this range.
Expand Down
20 changes: 10 additions & 10 deletions pkg/kv/kvserver/replica_raft.go
Original file line number Diff line number Diff line change
Expand Up @@ -752,7 +752,7 @@ func (r *Replica) handleRaftReadyRaftMuLocked(

msgApps, otherMsgs := splitMsgApps(rd.Messages)
r.traceMessageSends(msgApps, "sending msgApp")
r.sendRaftMessages(ctx, msgApps)
r.sendRaftMessagesRaftMuLocked(ctx, msgApps)

// Use a more efficient write-only batch because we don't need to do any
// reads from the batch. Any reads are performed on the underlying DB.
Expand Down Expand Up @@ -862,7 +862,7 @@ func (r *Replica) handleRaftReadyRaftMuLocked(
// Update raft log entry cache. We clear any older, uncommitted log entries
// and cache the latest ones.
r.store.raftEntryCache.Add(r.RangeID, rd.Entries, true /* truncate */)
r.sendRaftMessages(ctx, otherMsgs)
r.sendRaftMessagesRaftMuLocked(ctx, otherMsgs)
r.traceEntries(rd.CommittedEntries, "committed, before applying any entries")

applicationStart := timeutil.Now()
Expand Down Expand Up @@ -1010,7 +1010,7 @@ func (r *Replica) tick(ctx context.Context, livenessMap liveness.IsLiveMap) (boo
}

now := r.store.Clock().NowAsClockTimestamp()
if r.maybeQuiesceLocked(ctx, now, livenessMap) {
if r.maybeQuiesceRaftMuLockedReplicaMuLocked(ctx, now, livenessMap) {
return false, nil
}

Expand Down Expand Up @@ -1207,7 +1207,7 @@ func (r *Replica) maybeCoalesceHeartbeat(
return true
}

func (r *Replica) sendRaftMessages(ctx context.Context, messages []raftpb.Message) {
func (r *Replica) sendRaftMessagesRaftMuLocked(ctx context.Context, messages []raftpb.Message) {
var lastAppResp raftpb.Message
for _, message := range messages {
drop := false
Expand Down Expand Up @@ -1275,19 +1275,19 @@ func (r *Replica) sendRaftMessages(ctx context.Context, messages []raftpb.Messag
}

if !drop {
r.sendRaftMessage(ctx, message)
r.sendRaftMessageRaftMuLocked(ctx, message)
}
}
if lastAppResp.Index > 0 {
r.sendRaftMessage(ctx, lastAppResp)
r.sendRaftMessageRaftMuLocked(ctx, lastAppResp)
}
}

// sendRaftMessage sends a Raft message.
func (r *Replica) sendRaftMessage(ctx context.Context, msg raftpb.Message) {
// sendRaftMessageRaftMuLocked sends a Raft message.
func (r *Replica) sendRaftMessageRaftMuLocked(ctx context.Context, msg raftpb.Message) {
r.mu.RLock()
fromReplica, fromErr := r.getReplicaDescriptorByIDRLocked(roachpb.ReplicaID(msg.From), r.mu.lastToReplica)
toReplica, toErr := r.getReplicaDescriptorByIDRLocked(roachpb.ReplicaID(msg.To), r.mu.lastFromReplica)
fromReplica, fromErr := r.getReplicaDescriptorByIDRLocked(roachpb.ReplicaID(msg.From), r.raftMu.lastToReplica)
toReplica, toErr := r.getReplicaDescriptorByIDRLocked(roachpb.ReplicaID(msg.To), r.raftMu.lastFromReplica)
var startKey roachpb.RKey
if msg.Type == raftpb.MsgApp && r.mu.internalRaftGroup != nil {
// When the follower is potentially an uninitialized replica waiting for
Expand Down
16 changes: 8 additions & 8 deletions pkg/kv/kvserver/replica_raft_quiesce.go
Original file line number Diff line number Diff line change
Expand Up @@ -121,9 +121,9 @@ func (r *Replica) canUnquiesceRLocked() bool {
r.mu.internalRaftGroup != nil
}

// maybeQuiesceLocked checks to see if the replica is quiescable and initiates
// quiescence if it is. Returns true if the replica has been quiesced and false
// otherwise.
// maybeQuiesceRaftMuLockedReplicaMuLocked checks to see if the replica is
// quiescable and initiates quiescence if it is. Returns true if the replica has
// been quiesced and false otherwise.
//
// A quiesced range is not ticked and thus doesn't create MsgHeartbeat requests
// or cause elections. The Raft leader for a range checks various
Expand Down Expand Up @@ -178,14 +178,14 @@ func (r *Replica) canUnquiesceRLocked() bool {
// would quiesce. The fallout from this situation are undesirable raft
// elections which will cause throughput hiccups to the range, but not
// correctness issues.
func (r *Replica) maybeQuiesceLocked(
func (r *Replica) maybeQuiesceRaftMuLockedReplicaMuLocked(
ctx context.Context, now hlc.ClockTimestamp, livenessMap liveness.IsLiveMap,
) bool {
status, lagging, ok := shouldReplicaQuiesce(ctx, r, now, livenessMap)
if !ok {
return false
}
return r.quiesceAndNotifyLocked(ctx, status, lagging)
return r.quiesceAndNotifyRaftMuLockedReplicaMuLocked(ctx, status, lagging)
}

type quiescer interface {
Expand Down Expand Up @@ -398,10 +398,10 @@ func shouldReplicaQuiesce(
return status, lagging, true
}

func (r *Replica) quiesceAndNotifyLocked(
func (r *Replica) quiesceAndNotifyRaftMuLockedReplicaMuLocked(
ctx context.Context, status *raft.Status, lagging laggingReplicaSet,
) bool {
fromReplica, fromErr := r.getReplicaDescriptorByIDRLocked(r.mu.replicaID, r.mu.lastToReplica)
fromReplica, fromErr := r.getReplicaDescriptorByIDRLocked(r.mu.replicaID, r.raftMu.lastToReplica)
if fromErr != nil {
if log.V(4) {
log.Infof(ctx, "not quiescing: cannot find from replica (%d)", r.mu.replicaID)
Expand All @@ -416,7 +416,7 @@ func (r *Replica) quiesceAndNotifyLocked(
continue
}
toReplica, toErr := r.getReplicaDescriptorByIDRLocked(
roachpb.ReplicaID(id), r.mu.lastFromReplica)
roachpb.ReplicaID(id), r.raftMu.lastFromReplica)
if toErr != nil {
if log.V(4) {
log.Infof(ctx, "failed to quiesce: cannot find to replica (%d)", id)
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/store.go
Original file line number Diff line number Diff line change
Expand Up @@ -516,7 +516,7 @@ Store.HandleRaftRequest (which is part of the RaftMessageHandler interface),
ultimately resulting in a call to Replica.handleRaftReadyRaftMuLocked, which
houses the integration with the etcd/raft library (raft.RawNode). This may
generate Raft messages to be sent to other Stores; these are handed to
Replica.sendRaftMessages which ultimately hands them to the Store's
Replica.sendRaftMessagesRaftMuLocked which ultimately hands them to the Store's
RaftTransport.SendAsync method. Raft uses message passing (not
request-response), and outgoing messages will use a gRPC stream that differs
from that used for incoming messages (which makes asymmetric partitions more
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/store_raft.go
Original file line number Diff line number Diff line change
Expand Up @@ -213,7 +213,7 @@ func (s *Store) withReplicaForRequest(
return roachpb.NewError(err)
}
defer r.raftMu.Unlock()
r.setLastReplicaDescriptors(req)
r.setLastReplicaDescriptorsRaftMuLocked(req)
return f(ctx, r)
}

Expand Down
28 changes: 14 additions & 14 deletions pkg/sql/catalog/table_elements.go
Original file line number Diff line number Diff line change
Expand Up @@ -603,28 +603,28 @@ func FindDeleteOnlyNonPrimaryIndex(desc TableDescriptor, test func(idx Index) bo
return findIndex(desc.DeleteOnlyNonPrimaryIndexes(), test)
}

// FullIndexColumnIDs returns the index column IDs including any extra (implicit or
// stored (old STORING encoding)) column IDs for non-unique indexes. It also
// returns the direction with which each column was encoded.
// FullIndexColumnIDs returns the index column IDs including any extra (implicit
// or stored (old STORING encoding)) column IDs for non-unique indexes. It also
// returns the direction with which each column was encoded. The returned slices
// must not be modified.
func FullIndexColumnIDs(idx Index) ([]descpb.ColumnID, []descpb.IndexDescriptor_Direction) {
n := idx.NumKeyColumns()
if !idx.IsUnique() {
n += idx.NumKeySuffixColumns()
if idx.IsUnique() {
idxDesc := idx.IndexDesc()
return idxDesc.KeyColumnIDs, idxDesc.KeyColumnDirections
}
// Non-unique indexes have some of the primary-key columns appended to
// their key.
n := idx.NumKeyColumns() + idx.NumKeySuffixColumns()
ids := make([]descpb.ColumnID, 0, n)
dirs := make([]descpb.IndexDescriptor_Direction, 0, n)
for i := 0; i < idx.NumKeyColumns(); i++ {
ids = append(ids, idx.GetKeyColumnID(i))
dirs = append(dirs, idx.GetKeyColumnDirection(i))
}
// Non-unique indexes have some of the primary-key columns appended to
// their key.
if !idx.IsUnique() {
for i := 0; i < idx.NumKeySuffixColumns(); i++ {
// Extra columns are encoded in ascending order.
ids = append(ids, idx.GetKeySuffixColumnID(i))
dirs = append(dirs, descpb.IndexDescriptor_ASC)
}
for i := 0; i < idx.NumKeySuffixColumns(); i++ {
// Extra columns are encoded in ascending order.
ids = append(ids, idx.GetKeySuffixColumnID(i))
dirs = append(dirs, descpb.IndexDescriptor_ASC)
}
return ids, dirs
}
Expand Down
Loading

0 comments on commit 62392dc

Please sign in to comment.