Skip to content

Commit

Permalink
kvserver: add safe formatters for lease and friends
Browse files Browse the repository at this point in the history
This patch makes some types survive log redacting. In particular, it
ensures that the assertions added in previous patches don't get too
redacted.

Release note: None
  • Loading branch information
andreimatei committed Apr 21, 2021
1 parent cf7875b commit 3f38eb2
Show file tree
Hide file tree
Showing 7 changed files with 45 additions and 23 deletions.
2 changes: 2 additions & 0 deletions docs/generated/redact_safe.md
Original file line number Diff line number Diff line change
Expand Up @@ -4,8 +4,10 @@ File | Type
--|--
pkg/cli/exit/exit.go | `Code`
pkg/jobs/jobspb/wrap.go | `Type`
pkg/kv/kvserver/closedts/ctpb/entry.go | `LAI`
pkg/kv/kvserver/concurrency/lock/locking.go | `WaitPolicy`
pkg/kv/kvserver/raft.go | `SnapshotRequest_Type`
pkg/roachpb/data.go | `LeaseSequence`
pkg/roachpb/data.go | `ReplicaChangeType`
pkg/roachpb/metadata.go | `NodeID`
pkg/roachpb/metadata.go | `StoreID`
Expand Down
3 changes: 3 additions & 0 deletions pkg/kv/kvserver/closedts/ctpb/entry.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,9 @@ type Epoch int64
// mix-ups in positional arguments.
type LAI int64

// SafeValue implements the redact.SafeValue interface.
func (LAI) SafeValue() {}

// String formats Entry for human consumption as well as testing (by avoiding
// randomness in the output caused by map iteraton order).
func (e Entry) String() string {
Expand Down
1 change: 1 addition & 0 deletions pkg/kv/kvserver/kvserverbase/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -16,5 +16,6 @@ go_library(
"//pkg/roachpb",
"//pkg/settings",
"//pkg/util/hlc",
"@com_github_cockroachdb_redact//:redact",
],
)
14 changes: 13 additions & 1 deletion pkg/kv/kvserver/kvserverbase/base.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/redact"
)

// MergeQueueEnabled is a setting that controls whether the merge queue is
Expand All @@ -39,9 +40,20 @@ var MergeQueueEnabled = settings.RegisterBoolSetting(
// larger than the heartbeat interval used by the coordinator.
const TxnCleanupThreshold = time.Hour

// CmdIDKey is a Raft command id.
// CmdIDKey is a Raft command id. This will be logged unredacted - keep it random.
type CmdIDKey string

// SafeFormat implements redact.SafeFormatter.
func (s CmdIDKey) SafeFormat(sp redact.SafePrinter, verb rune) {
sp.Printf("%q", redact.SafeString(s))
}

func (s CmdIDKey) String() string {
return redact.StringWithoutMarkers(s)
}

var _ redact.SafeFormatter = CmdIDKey("")

// FilterArgs groups the arguments to a ReplicaCommandFilter.
type FilterArgs struct {
Ctx context.Context
Expand Down
20 changes: 10 additions & 10 deletions pkg/kv/kvserver/replica_application_state_machine.go
Original file line number Diff line number Diff line change
Expand Up @@ -1044,17 +1044,17 @@ func (b *replicaAppBatch) assertNoWriteBelowClosedTimestamp(cmd *replicatedCmd)
wts := cmd.raftCmd.ReplicatedEvalResult.WriteTimestamp
if !wts.IsEmpty() && wts.LessEq(b.state.RaftClosedTimestamp) {
wts := wts // Make a shadow variable that escapes to the heap.
var req string
var req redact.StringBuilder
if cmd.proposal != nil {
req = cmd.proposal.Request.String()
req.Print(cmd.proposal.Request)
} else {
req = "request unknown; not leaseholder"
req.SafeString("request unknown; not leaseholder")
}
return wrapWithNonDeterministicFailure(errors.AssertionFailedf(
"command writing below closed timestamp; cmd: %x, write ts: %s, "+
"batch state closed: %s, command closed: %s, request: %s, lease: %s",
cmd.idKey, wts,
b.state.RaftClosedTimestamp.String(), cmd.raftCmd.ClosedTimestamp,
b.state.RaftClosedTimestamp, cmd.raftCmd.ClosedTimestamp,
req, b.state.Lease),
"command writing below closed timestamp")
}
Expand All @@ -1067,14 +1067,14 @@ func (b *replicaAppBatch) assertNoCmdClosedTimestampRegression(cmd *replicatedCm
if !raftClosedTimestampAssertionsEnabled {
return nil
}
existingClosed := b.state.RaftClosedTimestamp
existingClosed := &b.state.RaftClosedTimestamp
newClosed := cmd.raftCmd.ClosedTimestamp
if newClosed != nil && !newClosed.IsEmpty() && newClosed.Less(existingClosed) {
var req string
if newClosed != nil && !newClosed.IsEmpty() && newClosed.Less(*existingClosed) {
var req redact.StringBuilder
if cmd.IsLocal() && cmd.proposal.Request.IsIntentWrite() {
req = cmd.proposal.Request.String()
req.Print(cmd.proposal.Request)
} else {
req = "<unknown; not leaseholder>"
req.SafeString("<unknown; not leaseholder>")
}
var prevReq redact.StringBuilder
if req := b.closedTimestampSetter.leaseReq; req != nil {
Expand All @@ -1086,7 +1086,7 @@ func (b *replicaAppBatch) assertNoCmdClosedTimestampRegression(cmd *replicatedCm
return errors.AssertionFailedf(
"raft closed timestamp regression in cmd: %x; batch state: %s, command: %s, lease: %s, req: %s, applying at LAI: %d.\n"+
"Closed timestamp was set by req: %s under lease: %s; applied at LAI: %d. Batch idx: %d.",
cmd.idKey, existingClosed.String(), newClosed.String(), b.state.Lease, req, cmd.leaseIndex,
cmd.idKey, existingClosed, newClosed, b.state.Lease, req, cmd.leaseIndex,
prevReq, b.closedTimestampSetter.lease, b.closedTimestampSetter.leaseIdx, b.entries)
}
return nil
Expand Down
26 changes: 15 additions & 11 deletions pkg/roachpb/data.go
Original file line number Diff line number Diff line change
Expand Up @@ -1852,25 +1852,29 @@ func (crt ChangeReplicasTrigger) Removed() []ReplicaDescriptor {
// LeaseSequence is a custom type for a lease sequence number.
type LeaseSequence int64

// String implements the fmt.Stringer interface.
func (s LeaseSequence) String() string {
return strconv.FormatInt(int64(s), 10)
}
// SafeValue implements the redact.SafeValue interface.
func (s LeaseSequence) SafeValue() {}

var _ fmt.Stringer = &Lease{}

func (l Lease) String() string {
return redact.StringWithoutMarkers(l)
}

// SafeFormat implements the redact.SafeFormatter interface.
func (l Lease) SafeFormat(w redact.SafePrinter, _ rune) {
if l.Empty() {
return "<empty>"
}
var proposedSuffix string
if l.ProposedTS != nil {
proposedSuffix = fmt.Sprintf(" pro=%s", l.ProposedTS)
w.SafeString("<empty>")
return
}
if l.Type() == LeaseExpiration {
return fmt.Sprintf("repl=%s seq=%s start=%s exp=%s%s", l.Replica, l.Sequence, l.Start, l.Expiration, proposedSuffix)
w.Printf("repl=%s seq=%d start=%s exp=%s", l.Replica, l.Sequence, l.Start, l.Expiration)
} else {
w.Printf("repl=%s seq=%d start=%s epo=%d", l.Replica, l.Sequence, l.Start, l.Epoch)
}
if l.ProposedTS != nil {
w.Printf(" pro=%s", l.ProposedTS)
}
return fmt.Sprintf("repl=%s seq=%s start=%s epo=%d%s", l.Replica, l.Sequence, l.Start, l.Epoch, proposedSuffix)
}

// Empty returns true for the Lease zero-value.
Expand Down
2 changes: 1 addition & 1 deletion pkg/roachpb/errors.go
Original file line number Diff line number Diff line change
Expand Up @@ -503,7 +503,7 @@ func (e *LeaseRejectedError) Error() string {
}

func (e *LeaseRejectedError) message(_ *Error) string {
return fmt.Sprintf("cannot replace lease %s with %s: %s", e.Existing, e.Requested, e.Message)
return fmt.Sprintf("cannot replace lease %s with %s: %s", e.Existing, e.Requested.String(), e.Message)
}

var _ ErrorDetailInterface = &LeaseRejectedError{}
Expand Down

0 comments on commit 3f38eb2

Please sign in to comment.