Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
102478: kvserver: only return range info when `ClientRangeInfo` set r=erikgrinaker a=erikgrinaker

Previously, passing an empty `ClientRangeInfo` would always generate and return range info, because the zero values are always considered stale. An exception was made for lease requests, since these bypass DistSender and never use `ClientRangeInfo`.

This patch instead only returns range info when a non-empty `ClientRangeInfo` is passed, with a version gate for 23.1 compatibility.

Epic: none
Release note: None

103127: kvserver: include more non-redactable info in logs r=aliher1911 a=knz

This patch ensures that the trace copy in replication errors includes non-redactable bits. It also ensures that the purgatory errors are properly included.

Release note: None
Epic: None

Co-authored-by: Erik Grinaker <grinaker@cockroachlabs.com>
Co-authored-by: Raphael 'kena' Poss <knz@thaumogen.net>
  • Loading branch information
3 people committed May 12, 2023
3 parents 87c9598 + 15e23d3 + 7096f53 commit 2e2e5c1
Show file tree
Hide file tree
Showing 9 changed files with 121 additions and 40 deletions.
10 changes: 7 additions & 3 deletions pkg/kv/kvclient/kvcoord/dist_sender.go
Original file line number Diff line number Diff line change
Expand Up @@ -2149,13 +2149,13 @@ func (ds *DistSender) sendToReplicas(
ba.Replica = curReplica
ba.RangeID = desc.RangeID
// Communicate to the server the information our cache has about the
// range. If it's stale, the serve will return an update.
// range. If it's stale, the server will return an update.
ba.ClientRangeInfo = roachpb.ClientRangeInfo{
// Note that DescriptorGeneration will be 0 if the cached descriptor
// is "speculative" (see DescSpeculative()). Even if the speculation
// is correct, we want the serve to return an update, at which point
// the cached entry will no longer be "speculative".
DescriptorGeneration: routing.Desc().Generation,
DescriptorGeneration: desc.Generation,
// The LeaseSequence will be 0 if the cache doesn't have lease info,
// or has a speculative lease. Like above, this asks the server to
// return an update.
Expand All @@ -2167,7 +2167,11 @@ func (ds *DistSender) sendToReplicas(
defaultSendClosedTimestampPolicy,
),

ExplicitlyRequested: ba.ClientRangeInfo.ExplicitlyRequested,
// Range info is only returned when ClientRangeInfo is non-empty.
// Explicitly request an update for speculative/missing leases and
// descriptors, or when the client has requested it.
ExplicitlyRequested: ba.ClientRangeInfo.ExplicitlyRequested ||
(desc.Generation == 0 && routing.LeaseSeq() == 0),
}
br, err = transport.SendNext(ctx, ba)
ds.maybeIncrementErrCounters(br, err)
Expand Down
3 changes: 3 additions & 0 deletions pkg/kv/kvpb/api.proto
Original file line number Diff line number Diff line change
Expand Up @@ -2675,6 +2675,9 @@ message Header {
// by the client's DistSender, however it will preserve the value of the field
// `ExplicitlyRequested` so that requests passed to DistSender can request
// `RangeInfos` if desired.
//
// If empty, range info is never returned (from 23.2 onwards). Use
// ExplicitlyRequested to force an update for an otherwise-empty field.
ClientRangeInfo client_range_info = 17 [(gogoproto.nullable) = false];
// bounded_staleness is set when a read-only batch is performing a bounded
// staleness read and wants its timestamp to be chosen dynamically, based
Expand Down
1 change: 1 addition & 0 deletions pkg/kv/kvserver/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -263,6 +263,7 @@ go_test(
"closed_timestamp_test.go",
"consistency_queue_test.go",
"debug_print_test.go",
"errors_test.go",
"gossip_test.go",
"helpers_test.go",
"intent_resolver_integration_test.go",
Expand Down
29 changes: 29 additions & 0 deletions pkg/kv/kvserver/errors_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,29 @@
// Copyright 2023 The Cockroach Authors.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.

package kvserver

import (
"testing"

"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/errors"
"github.com/cockroachdb/redact"
"github.com/stretchr/testify/require"
)

func TestErrorFormatting(t *testing.T) {
defer leaktest.AfterTest(t)()

var e error = decommissionPurgatoryError{errors.New("hello")}
require.Equal(t, "hello", redact.Sprint(e).Redact().StripMarkers())
e = rangeMergePurgatoryError{errors.New("hello")}
require.Equal(t, "hello", redact.Sprint(e).Redact().StripMarkers())
}
7 changes: 7 additions & 0 deletions pkg/kv/kvserver/merge_queue.go
Original file line number Diff line number Diff line change
Expand Up @@ -181,6 +181,13 @@ func (mq *mergeQueue) shouldQueue(
// indicate that the error should send the range to purgatory.
type rangeMergePurgatoryError struct{ error }

var _ errors.SafeFormatter = decommissionPurgatoryError{}

func (e rangeMergePurgatoryError) SafeFormatError(p errors.Printer) (next error) {
p.Print(e.error)
return nil
}

func (rangeMergePurgatoryError) PurgatoryErrorMarker() {}

var _ PurgatoryError = rangeMergePurgatoryError{}
Expand Down
19 changes: 15 additions & 4 deletions pkg/kv/kvserver/replica_send.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,7 @@ import (
"reflect"
"runtime/pprof"

"github.com/cockroachdb/cockroach/pkg/clusterversion"
"github.com/cockroachdb/cockroach/pkg/kv/kvpb"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency"
Expand Down Expand Up @@ -307,12 +308,22 @@ func (r *Replica) maybeCommitWaitBeforeCommitTrigger(
func (r *Replica) maybeAddRangeInfoToResponse(
ctx context.Context, ba *kvpb.BatchRequest, br *kvpb.BatchResponse,
) {
// Ignore lease requests. These are submitted directly to the replica,
// bypassing the DistSender. They don't need range info returned, but their
// ClientRangeInfo is always empty, so they'll otherwise always get it.
if ba.IsSingleRequestLeaseRequest() {
// Only return range info if ClientRangeInfo is non-empty. In particular, we
// don't want to populate this for lease requests, since these bypass
// DistSender and never use ClientRangeInfo.
//
// From 23.2, all DistSenders ensure ExplicitlyRequested is set when otherwise
// empty. Fall back to check for lease requests, to avoid 23.1 regressions.
if r.ClusterSettings().Version.IsActive(ctx, clusterversion.V23_2) {
if ba.ClientRangeInfo == (roachpb.ClientRangeInfo{}) {
return
}
} else if ba.IsSingleRequestLeaseRequest() {
// TODO(erikgrinaker): Remove this branch when 23.1 support is dropped.
_ = clusterversion.V23_1
return
}

// Compare the client's info with the replica's info to detect if the client
// has stale knowledge. Note that the client can have more recent knowledge
// than the replica in case this is a follower.
Expand Down
78 changes: 48 additions & 30 deletions pkg/kv/kvserver/replica_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -253,23 +253,25 @@ func (tc *testContext) SendWrapped(args kvpb.Request) (kvpb.Response, *kvpb.Erro
return tc.SendWrappedWith(kvpb.Header{}, args)
}

// addBogusReplicaToRangeDesc modifies the range descriptor to include a second
// addBogusReplicaToRangeDesc modifies the range descriptor to include an additional
// replica. This is useful for tests that want to pretend they're transferring
// the range lease away, as the lease can only be obtained by Replicas which are
// part of the range descriptor.
// This is a workaround, but it's sufficient for the purposes of several tests.
func (tc *testContext) addBogusReplicaToRangeDesc(
ctx context.Context,
) (roachpb.ReplicaDescriptor, error) {
secondReplica := roachpb.ReplicaDescriptor{
NodeID: 2,
StoreID: 2,
ReplicaID: 2,
}
oldDesc := *tc.repl.Desc()
newID := oldDesc.NextReplicaID
newReplica := roachpb.ReplicaDescriptor{
NodeID: roachpb.NodeID(newID),
StoreID: roachpb.StoreID(newID),
ReplicaID: newID,
}
newDesc := oldDesc
newDesc.InternalReplicas = append(newDesc.InternalReplicas, secondReplica)
newDesc.NextReplicaID = 3
newDesc.InternalReplicas = append(newDesc.InternalReplicas, newReplica)
newDesc.NextReplicaID++
newDesc.IncrementGeneration()

dbDescKV, err := tc.store.DB().Get(ctx, keys.RangeDescriptorKey(oldDesc.StartKey))
if err != nil {
Expand Down Expand Up @@ -303,7 +305,7 @@ func (tc *testContext) addBogusReplicaToRangeDesc(
tc.repl.assertStateRaftMuLockedReplicaMuRLocked(ctx, tc.engine)
tc.repl.mu.RUnlock()
tc.repl.raftMu.Unlock()
return secondReplica, nil
return newReplica, nil
}

func newTransaction(
Expand Down Expand Up @@ -13867,24 +13869,54 @@ func TestRangeInfoReturned(t *testing.T) {
var tc testContext
tc.Start(ctx, t, stopper)

// Add a couple of bogus configuration changes to bump the generation to 2,
// and request a new lease to bump the lease sequence to 2.
_, err := tc.addBogusReplicaToRangeDesc(ctx)
require.NoError(t, err)
_, err = tc.addBogusReplicaToRangeDesc(ctx)
require.NoError(t, err)

{
lease, _ := tc.repl.GetLease()
tc.repl.RevokeLease(ctx, lease.Sequence)

tc.repl.mu.Lock()
st := tc.repl.leaseStatusAtRLocked(ctx, tc.Clock().NowAsClockTimestamp())
ll := tc.repl.requestLeaseLocked(ctx, st)
tc.repl.mu.Unlock()
select {
case pErr := <-ll.C():
require.NoError(t, pErr.GoError())
case <-time.After(5 * time.Second):
t.Fatal("timeout")
}
}

ri := tc.repl.GetRangeInfo(ctx)
require.False(t, ri.Lease.Empty())
require.Equal(t, roachpb.LAG_BY_CLUSTER_SETTING, ri.ClosedTimestampPolicy)
require.EqualValues(t, 2, ri.Desc.Generation)
require.EqualValues(t, 2, ri.Lease.Sequence)
staleDescGen := ri.Desc.Generation - 1
staleLeaseSeq := ri.Lease.Sequence - 1
wrongCTPolicy := roachpb.LEAD_FOR_GLOBAL_READS

requestLease := ri.Lease
requestLease.Sequence = 0

for _, test := range []struct {
cri roachpb.ClientRangeInfo
req kvpb.Request
exp *roachpb.RangeInfo
}{
{
// Empty client info. This case shouldn't happen.
// Empty client info doesn't return any info. This case shouldn't happen
// for requests via DistSender, but can happen e.g. with lease requests
// that are submitted directly to the replica.
cri: roachpb.ClientRangeInfo{},
exp: nil,
},
{
// ExplicitlyRequested returns lease info.
cri: roachpb.ClientRangeInfo{
ExplicitlyRequested: true,
},
exp: &ri,
},
{
Expand Down Expand Up @@ -13949,26 +13981,12 @@ func TestRangeInfoReturned(t *testing.T) {
},
exp: &ri,
},
{
// RequestLeaseRequest without ClientRangeInfo. These bypass
// DistSender and don't need range info returned.
cri: roachpb.ClientRangeInfo{},
req: &kvpb.RequestLeaseRequest{
Lease: requestLease,
PrevLease: ri.Lease,
},
exp: nil,
},
} {
t.Run("", func(t *testing.T) {
ba := &kvpb.BatchRequest{}
ba.Header.ClientRangeInfo = test.cri
req := test.req
if req == nil {
args := getArgs(roachpb.Key("a"))
req = &args
}
ba.Add(req)
req := getArgs(roachpb.Key("a"))
ba.Add(&req)
br, pErr := tc.Sender().Send(ctx, ba)
require.NoError(t, pErr.GoError())
if test.exp == nil {
Expand Down
12 changes: 10 additions & 2 deletions pkg/kv/kvserver/replicate_queue.go
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/tracing"
"github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb"
"github.com/cockroachdb/errors"
"github.com/cockroachdb/redact"
)

// The replicate queue processes replicas that required replication changes.
Expand Down Expand Up @@ -688,6 +689,13 @@ func (rq *replicateQueue) process(
// that the error should send the range to purgatory.
type decommissionPurgatoryError struct{ error }

var _ errors.SafeFormatter = decommissionPurgatoryError{}

func (e decommissionPurgatoryError) SafeFormatError(p errors.Printer) (next error) {
p.Print(e.error)
return nil
}

func (decommissionPurgatoryError) PurgatoryErrorMarker() {}

var _ PurgatoryError = decommissionPurgatoryError{}
Expand Down Expand Up @@ -740,7 +748,7 @@ func (rq *replicateQueue) processOneChangeWithTracing(
loggingThreshold := rq.logTracesThresholdFunc(rq.store.cfg.Settings, repl)
exceededDuration := loggingThreshold > time.Duration(0) && processDuration > loggingThreshold

var traceOutput string
var traceOutput redact.RedactableString
traceLoggingNeeded := (err != nil || exceededDuration) && log.ExpensiveLogEnabled(ctx, 1)
if traceLoggingNeeded {
// If we have tracing spans from execChangeReplicasTxn, filter it from
Expand All @@ -749,7 +757,7 @@ func (rq *replicateQueue) processOneChangeWithTracing(
rec = filterTracingSpans(sp.GetConfiguredRecording(),
replicaChangeTxnGetDescOpName, replicaChangeTxnUpdateDescOpName,
)
traceOutput = fmt.Sprintf("\ntrace:\n%s", rec)
traceOutput = redact.Sprintf("\ntrace:\n%s", rec)
}

if err != nil {
Expand Down
2 changes: 1 addition & 1 deletion pkg/roachpb/data.proto
Original file line number Diff line number Diff line change
Expand Up @@ -776,7 +776,7 @@ message ClientRangeInfo {
int64 lease_sequence = 2 [(gogoproto.casttype) = "LeaseSequence"];
RangeClosedTimestampPolicy closed_timestamp_policy = 3;
// ExplicitlyRequested causes range info to be returned even if other fields
// are up-to-date.
// are up-to-date or empty.
bool explicitly_requested = 4;
}

Expand Down

0 comments on commit 2e2e5c1

Please sign in to comment.