Skip to content

Commit

Permalink
kvserver: properly redact the unavailable range message
Browse files Browse the repository at this point in the history
Saw this while investigating cockroachdb#54444.

Release note: None
  • Loading branch information
tbg committed Sep 30, 2020
1 parent a99227d commit 0c717f9
Show file tree
Hide file tree
Showing 8 changed files with 76 additions and 45 deletions.
2 changes: 2 additions & 0 deletions docs/generated/redact_safe.md
Original file line number Diff line number Diff line change
Expand Up @@ -3,6 +3,7 @@ The following types are considered always safe for reporting:
File | Type
--|--
pkg/jobs/jobspb/wrap.go | `Type`
pkg/kv/kvserver/concurrency/lock/locking.go | `WaitPolicy`
pkg/kv/kvserver/raft.go | `SnapshotRequest_Type`
pkg/roachpb/data.go | `ReplicaChangeType`
pkg/roachpb/metadata.go | `NodeID`
Expand All @@ -11,6 +12,7 @@ pkg/roachpb/metadata.go | `RangeID`
pkg/roachpb/metadata.go | `ReplicaID`
pkg/roachpb/metadata.go | `RangeGeneration`
pkg/roachpb/metadata.go | `ReplicaType`
pkg/roachpb/method.go | `Method`
pkg/sql/catalog/descpb/structured.go | `ID`
pkg/sql/catalog/descpb/structured.go | `FamilyID`
pkg/sql/catalog/descpb/structured.go | `IndexID`
Expand Down
3 changes: 3 additions & 0 deletions pkg/kv/kvserver/concurrency/lock/locking.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,3 +24,6 @@ func init() {
}
}
}

// SafeValue implements redact.SafeValue.
func (WaitPolicy) SafeValue() {}
13 changes: 8 additions & 5 deletions pkg/kv/kvserver/replica_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -66,6 +66,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/uuid"
"github.com/cockroachdb/errors"
"github.com/cockroachdb/logtags"
"github.com/cockroachdb/redact"
"github.com/gogo/protobuf/proto"
"github.com/kr/pretty"
opentracing "github.com/opentracing/opentracing-go"
Expand Down Expand Up @@ -12972,12 +12973,13 @@ func TestRangeUnavailableMessage(t *testing.T) {
1: IsLiveMapEntry{IsLive: true},
}
rs := raft.Status{}
act := rangeUnavailableMessage(desc, lm, &rs, &ba, dur)
const exp = `have been waiting 60.00s for proposing command RequestLease [/Min,/Min).
var s redact.StringBuilder
rangeUnavailableMessage(&s, desc, lm, &rs, &ba, dur)
const exp = `have been waiting 60.00s for proposing command RequestLease [‹/Min›,‹/Min›).
This range is likely unavailable.
Please submit this message to Cockroach Labs support along with the following information:
Descriptor: r10:{-} [(n1,s10):1, (n2,s20):2, next=3, gen=0]
Descriptor: r10:‹{a-z}› [(n1,s10):1, (n2,s20):2, next=3, gen=0]
Live: (n1,s10):1
Non-live: (n2,s20):2
Raft Status: {"id":"0","term":0,"vote":"0","commit":0,"lead":"0","raftState":"StateFollower","applied":0,"progress":{},"leadtransferee":"0"}
Expand All @@ -12989,8 +12991,9 @@ support contract. Otherwise, please open an issue at:
https://github.com/cockroachdb/cockroach/issues/new/choose
`

require.Equal(t, exp, act)
act := s.RedactableString()
t.Log(act)
require.EqualValues(t, exp, act)
}

// Test that, depending on the request's ClientRangeInfo, descriptor and lease
Expand Down
26 changes: 15 additions & 11 deletions pkg/kv/kvserver/replica_write.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,7 +12,6 @@ package kvserver

import (
"context"
"fmt"
"time"

"github.com/cockroachdb/cockroach/pkg/base"
Expand All @@ -31,6 +30,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/errors"
"github.com/cockroachdb/redact"
"go.etcd.io/etcd/raft"
)

Expand Down Expand Up @@ -219,9 +219,10 @@ func (r *Replica) executeWriteBatch(
slowTimer.Read = true
r.store.metrics.SlowRaftRequests.Inc(1)

log.Errorf(ctx, "range unavailable: %v",
rangeUnavailableMessage(r.Desc(), r.store.cfg.NodeLiveness.GetIsLiveMap(),
r.RaftStatus(), ba, timeutil.Since(startPropTime)))
var s redact.StringBuilder
rangeUnavailableMessage(&s, r.Desc(), r.store.cfg.NodeLiveness.GetIsLiveMap(),
r.RaftStatus(), ba, timeutil.Since(startPropTime))
log.Errorf(ctx, "range unavailable: %v", s)
case <-ctxDone:
// If our context was canceled, return an AmbiguousResultError,
// which indicates to the caller that the command may have executed.
Expand All @@ -241,16 +242,13 @@ func (r *Replica) executeWriteBatch(
}

func rangeUnavailableMessage(
s *redact.StringBuilder,
desc *roachpb.RangeDescriptor,
lm IsLiveMap,
rs *raft.Status,
ba *roachpb.BatchRequest,
dur time.Duration,
) string {
cpy := *desc
desc = &cpy
desc.StartKey, desc.EndKey = nil, nil // scrub PII

) {
var liveReplicas, otherReplicas []roachpb.ReplicaDescriptor
for _, rDesc := range desc.Replicas().All() {
if lm[rDesc.NodeID].IsLive {
Expand All @@ -259,7 +257,13 @@ func rangeUnavailableMessage(
otherReplicas = append(otherReplicas, rDesc)
}
}
return fmt.Sprintf(`have been waiting %.2fs for proposing command %s.

// Ensure that these are going to redact nicely.
var _ redact.SafeFormatter = ba
var _ redact.SafeFormatter = desc
var _ redact.SafeFormatter = roachpb.ReplicaDescriptors{}

s.Printf(`have been waiting %.2fs for proposing command %s.
This range is likely unavailable.
Please submit this message to Cockroach Labs support along with the following information:
Expand All @@ -280,7 +284,7 @@ support contract. Otherwise, please open an issue at:
desc,
roachpb.MakeReplicaDescriptors(liveReplicas),
roachpb.MakeReplicaDescriptors(otherReplicas),
rs,
redact.Safe(rs), // raft status contains no PII
desc.RangeID,
)
}
Expand Down
51 changes: 29 additions & 22 deletions pkg/roachpb/batch.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/storage/enginepb"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/errors"
"github.com/cockroachdb/redact"
)

//go:generate go run -tags gen-batch gen_batch.go
Expand Down Expand Up @@ -571,47 +572,53 @@ func (ba BatchRequest) Split(canSplitET bool) [][]RequestUnion {
return parts
}

// String gives a brief summary of the contained requests and keys in the batch.
// TODO(tschottdorf): the key range is useful information, but requires `keys`.
// See #2198.
func (ba BatchRequest) String() string {
var str []string
if ba.Txn != nil {
str = append(str, fmt.Sprintf("[txn: %s]", ba.Txn.Short()))
}
if ba.WaitPolicy != lock.WaitPolicy_Block {
str = append(str, fmt.Sprintf("[wait-policy: %s]", ba.WaitPolicy))
}
if ba.CanForwardReadTimestamp {
str = append(str, "[can-forward-ts]")
}
// SafeFormat implements redact.SafeFormatter.
// It gives a brief summary of the contained requests and keys in the batch.
func (ba BatchRequest) SafeFormat(s redact.SafePrinter, _ rune) {
for count, arg := range ba.Requests {
// Limit the strings to provide just a summary. Without this limit
// a log message with a BatchRequest can be very long.
if count >= 20 && count < len(ba.Requests)-5 {
if count == 20 {
str = append(str, fmt.Sprintf("... %d skipped ...", len(ba.Requests)-25))
s.Printf(",... %d skipped ...", len(ba.Requests)-25)
}
continue
}
if count > 0 {
s.Print(redact.SafeString(", "))
}

req := arg.GetInner()
if et, ok := req.(*EndTxnRequest); ok {
h := req.Header()
str = append(str, fmt.Sprintf("%s(commit:%t) [%s] ",
req.Method(), et.Commit, h.Key))
s.Printf("%s(commit:%t) [%s]",
req.Method(), et.Commit, h.Key)
} else {
h := req.Header()
var s string
if req.Method() == PushTxn {
pushReq := req.(*PushTxnRequest)
s = fmt.Sprintf("PushTxn(%s->%s)", pushReq.PusherTxn.Short(), pushReq.PusheeTxn.Short())
s.Printf("PushTxn(%s->%s)", pushReq.PusherTxn.Short(), pushReq.PusheeTxn.Short())
} else {
s = req.Method().String()
s.Print(req.Method())
}
str = append(str, fmt.Sprintf("%s [%s,%s)", s, h.Key, h.EndKey))
s.Printf(" [%s,%s)", h.Key, h.EndKey)
}
}
return strings.Join(str, ", ")
{
if ba.Txn != nil {
s.Printf(", [txn: %s]", ba.Txn.Short())
}
}
if ba.WaitPolicy != lock.WaitPolicy_Block {
s.Printf(", [wait-policy: %s]", ba.WaitPolicy)
}
if ba.CanForwardReadTimestamp {
s.Printf(", [can-forward-ts]")
}
}

func (ba BatchRequest) String() string {
return redact.StringWithoutMarkers(ba)
}

// ValidateForEvaluation performs sanity checks on the batch when it's received
Expand Down
3 changes: 3 additions & 0 deletions pkg/roachpb/method.go
Original file line number Diff line number Diff line change
Expand Up @@ -13,6 +13,9 @@ package roachpb
// Method is the enumerated type for methods.
type Method int

// SafeValue implements redact.SafeValue.
func (Method) SafeValue() {}

//go:generate stringer -type=Method
const (
// Get fetches the value for a key from the KV map, respecting a
Expand Down
18 changes: 13 additions & 5 deletions pkg/roachpb/string_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -11,7 +11,6 @@
package roachpb_test

import (
"fmt"
"testing"

// Hook up the pretty printer.
Expand All @@ -21,6 +20,8 @@ import (
"github.com/cockroachdb/cockroach/pkg/storage/enginepb"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/uuid"
"github.com/cockroachdb/redact"
"github.com/stretchr/testify/require"
)

func TestTransactionString(t *testing.T) {
Expand Down Expand Up @@ -64,6 +65,7 @@ func TestBatchRequestString(t *testing.T) {
hlc.Timestamp{}, // now
0, // maxOffsetNs
)
txn.ID = uuid.NamespaceDNS
ba.Txn = &txn
ba.WaitPolicy = lock.WaitPolicy_Error
ba.CanForwardReadTimestamp = true
Expand All @@ -76,9 +78,15 @@ func TestBatchRequestString(t *testing.T) {
ru.MustSetInner(&roachpb.EndTxnRequest{})
ba.Requests = append(ba.Requests, ru)

e := fmt.Sprintf(`[txn: %s], [wait-policy: Error], [can-forward-ts], Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), ... 76 skipped ..., Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), EndTxn(commit:false) [/Min] `,
ba.Txn.Short())
if e != ba.String() {
t.Fatalf("e = %s\nv = %s", e, ba.String())
{
exp := `Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min),... 76 skipped ..., Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), Get [/Min,/Min), EndTxn(commit:false) [/Min], [txn: 6ba7b810], [wait-policy: Error], [can-forward-ts]`
act := ba.String()
require.Equal(t, exp, act)
}

{
exp := `Get [‹/Min›,‹/Min›), Get [‹/Min›,‹/Min›), Get [‹/Min›,‹/Min›), Get [‹/Min›,‹/Min›), Get [‹/Min›,‹/Min›), Get [‹/Min›,‹/Min›), Get [‹/Min›,‹/Min›), Get [‹/Min›,‹/Min›), Get [‹/Min›,‹/Min›), Get [‹/Min›,‹/Min›), Get [‹/Min›,‹/Min›), Get [‹/Min›,‹/Min›), Get [‹/Min›,‹/Min›), Get [‹/Min›,‹/Min›), Get [‹/Min›,‹/Min›), Get [‹/Min›,‹/Min›), Get [‹/Min›,‹/Min›), Get [‹/Min›,‹/Min›), Get [‹/Min›,‹/Min›), Get [‹/Min›,‹/Min›),... 76 skipped ..., Get [‹/Min›,‹/Min›), Get [‹/Min›,‹/Min›), Get [‹/Min›,‹/Min›), Get [‹/Min›,‹/Min›), EndTxn(commit:false) [‹/Min›], [txn: 6ba7b810], [wait-policy: Error], [can-forward-ts]`
act := redact.Sprint(ba)
require.EqualValues(t, exp, act)
}
}
5 changes: 3 additions & 2 deletions pkg/storage/enginepb/mvcc.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@ import (
"strings"

"github.com/cockroachdb/errors"
"github.com/cockroachdb/redact"
)

// TxnEpoch is a zero-indexed epoch for a transaction. When a transaction
Expand Down Expand Up @@ -82,8 +83,8 @@ func TxnSeqIsIgnored(seq TxnSeq, ignored []IgnoredSeqNumRange) bool {
}

// Short returns a prefix of the transaction's ID.
func (t TxnMeta) Short() string {
return t.ID.Short()
func (t TxnMeta) Short() redact.SafeString {
return redact.SafeString(t.ID.Short())
}

// Total returns the range size as the sum of the key and value
Expand Down

0 comments on commit 0c717f9

Please sign in to comment.