diff --git a/docs/generated/redact_safe.md b/docs/generated/redact_safe.md index 6d7a953bd888..a892cc49bb66 100644 --- a/docs/generated/redact_safe.md +++ b/docs/generated/redact_safe.md @@ -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` @@ -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` diff --git a/pkg/kv/kvserver/concurrency/lock/locking.go b/pkg/kv/kvserver/concurrency/lock/locking.go index 98549396ee2e..75d665d8f0e7 100644 --- a/pkg/kv/kvserver/concurrency/lock/locking.go +++ b/pkg/kv/kvserver/concurrency/lock/locking.go @@ -24,3 +24,6 @@ func init() { } } } + +// SafeValue implements redact.SafeValue. +func (WaitPolicy) SafeValue() {} diff --git a/pkg/kv/kvserver/replica_test.go b/pkg/kv/kvserver/replica_test.go index 275f9efd35ce..050f22c23426 100644 --- a/pkg/kv/kvserver/replica_test.go +++ b/pkg/kv/kvserver/replica_test.go @@ -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" @@ -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"} @@ -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 diff --git a/pkg/kv/kvserver/replica_write.go b/pkg/kv/kvserver/replica_write.go index 8ab95775b724..32f91a3e4e85 100644 --- a/pkg/kv/kvserver/replica_write.go +++ b/pkg/kv/kvserver/replica_write.go @@ -12,7 +12,6 @@ package kvserver import ( "context" - "fmt" "time" "github.com/cockroachdb/cockroach/pkg/base" @@ -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" ) @@ -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. @@ -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 { @@ -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: @@ -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, ) } diff --git a/pkg/roachpb/batch.go b/pkg/roachpb/batch.go index 641b36b44ab0..07fc49b46348 100644 --- a/pkg/roachpb/batch.go +++ b/pkg/roachpb/batch.go @@ -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 @@ -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 diff --git a/pkg/roachpb/method.go b/pkg/roachpb/method.go index 692ceceb9fa7..baaeb32cf080 100644 --- a/pkg/roachpb/method.go +++ b/pkg/roachpb/method.go @@ -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 diff --git a/pkg/roachpb/string_test.go b/pkg/roachpb/string_test.go index b8ddac040bdf..6f7305104279 100644 --- a/pkg/roachpb/string_test.go +++ b/pkg/roachpb/string_test.go @@ -11,7 +11,6 @@ package roachpb_test import ( - "fmt" "testing" // Hook up the pretty printer. @@ -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) { @@ -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 @@ -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) } } diff --git a/pkg/storage/enginepb/mvcc.go b/pkg/storage/enginepb/mvcc.go index db4268914db0..8f8d4f7a1180 100644 --- a/pkg/storage/enginepb/mvcc.go +++ b/pkg/storage/enginepb/mvcc.go @@ -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 @@ -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