Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
54687: roachtest: log teardown to different file r=andreimatei a=tbg

We perform various operations on teardown which typically produce a few
pages of log output (dead nodes, consistency, getting logs, ...).
These tend to make the real test failure harder to find, so this
commit switches out the loggers before we perform teardown; teardown
will log to `teardown.log` instead. The main log (`test.log`) now
ends as follows:

```
[...]
09:41:06 allocator.go:315: 0 mis-replicated ranges
09:41:07 test.go:337: test failure: 	allocator.go:321,test_runner.go:755: boom!
09:41:07 test_runner.go:769: tearing down after failure; see teardown.log
```

Release note: None


54742: kvserver: properly redact the unavailable range message r=knz a=tbg

Saw this while investigating #54444.

Release note: None

55034: cli/flags: remove `doctor` from the list of timeout-supporting commands r=tbg a=knz

Fixes #54931.

This was unintentionally added - doctor is not meant to support
configurable timeouts (just yet).

Release note: None

Co-authored-by: Tobias Grieger <tobias.b.grieger@gmail.com>
Co-authored-by: Raphael 'kena' Poss <knz@thaumogen.net>
  • Loading branch information
3 people committed Sep 30, 2020
4 parents df0bae0 + 5537a56 + 0c717f9 + ba6b0cc commit 485c196
Show file tree
Hide file tree
Showing 10 changed files with 88 additions and 46 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
1 change: 0 additions & 1 deletion pkg/cli/flags.go
Original file line number Diff line number Diff line change
Expand Up @@ -581,7 +581,6 @@ func init() {
statusNodeCmd,
lsNodesCmd,
debugZipCmd,
doctorClusterCmd,
// If you add something here, make sure the actual implementation
// of the command uses `cmdTimeoutContext(.)` or it will ignore
// the timeout.
Expand Down
12 changes: 12 additions & 0 deletions pkg/cmd/roachtest/test_runner.go
Original file line number Diff line number Diff line change
Expand Up @@ -755,9 +755,21 @@ func (r *testRunner) runTest(
t.spec.Run(runCtx, t, c)
}()

teardownL, err := c.l.ChildLogger("teardown", quietStderr, quietStdout)
if err != nil {
return false, err
}
select {
case <-done:
s := "success"
if t.Failed() {
s = "failure"
}
c.l.Printf("tearing down after %s; see teardown.log", s)
l, c.l, t.l = teardownL, teardownL, teardownL
case <-time.After(timeout):
c.l.Printf("tearing down after timeout; see teardown.log")
l, c.l, t.l = teardownL, teardownL, teardownL
// Timeouts are often opaque. Improve our changes by dumping the stack
// so that at least we can piece together what the test is trying to
// do at this very moment.
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 485c196

Please sign in to comment.